From 90b1a6b7200aa5693c6dc3c2abbfa479ebff395f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:12:44 -0700 Subject: [PATCH 001/224] Add new AnyOf environment. This allows an SDK (including, importantly, an expansion service) to provide several alternatives environments suitable for running a pipeline. --- .../model/pipeline/v1/beam_runner_api.proto | 6 ++ .../core/construction/Environments.java | 47 +++++++++++++++ .../core/construction/EnvironmentsTest.java | 41 +++++++++++++ .../apache_beam/transforms/environments.py | 58 +++++++++++++++++++ 4 files changed, 152 insertions(+) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index db958f183c45..87af7c19dd79 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -1570,6 +1570,8 @@ message StandardEnvironments { EXTERNAL = 2 [(beam_urn) = "beam:env:external:v1"]; // An external non managed process to run user code. DEFAULT = 3 [(beam_urn) = "beam:env:default:v1"]; // Used as a stub when context is missing a runner-provided default environment. + + ANYOF = 4 [(beam_urn) = "beam:env:anyof:v1"]; // A selection of equivalent environments a runner may use. } } @@ -1590,6 +1592,10 @@ message ExternalPayload { map params = 2; // Arbitrary extra parameters to pass } +message AnyOfEnvironmentPayload { + repeated Environment environments = 1; +} + // These URNs are used to indicate capabilities of environments that cannot // simply be expressed as a component (such as a Coder or PTransform) that this // environment understands. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index f531b5be344d..6e05c006d283 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -28,8 +28,11 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.AnyOfEnvironmentPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.ArtifactInformation; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload; @@ -291,6 +294,50 @@ public static Environment createProcessEnvironment( .build(); } + public static Environment createAnyOfEnvironment(Environment... environments) { + AnyOfEnvironmentPayload.Builder payload = AnyOfEnvironmentPayload.newBuilder(); + for (Environment environment : environments) { + payload.addEnvironments(environment); + } + return Environment.newBuilder() + .setUrn(BeamUrns.getUrn(StandardEnvironments.Environments.ANYOF)) + .setPayload(payload.build().toByteString()) + .build(); + } + + public static List expandAnyOfEnvironments(Environment environment) { + return Stream.of(environment) + .flatMap( + env -> { + if (BeamUrns.getUrn(StandardEnvironments.Environments.ANYOF) + .equals(environment.getUrn())) { + try { + return AnyOfEnvironmentPayload.parseFrom(environment.getPayload()) + .getEnvironmentsList().stream() + .flatMap(subenv -> expandAnyOfEnvironments(subenv).stream()); + } catch (InvalidProtocolBufferException exn) { + throw new RuntimeException(exn); + } + } else { + return Stream.of(env); + } + }) + .collect(Collectors.toList()); + } + + public static Environment resolveAnyOfEnvironment( + Environment environment, String... preferredEnvironmentTypes) { + List allEnvironments = expandAnyOfEnvironments(environment); + for (String urn : preferredEnvironmentTypes) { + for (Environment env : allEnvironments) { + if (urn.equals(env.getUrn())) { + return env; + } + } + } + return allEnvironments.iterator().next(); + } + public static Optional getEnvironment(String ptransformId, Components components) { PTransform ptransform = components.getTransformsOrThrow(ptransformId); String envId = ptransform.getEnvironmentId(); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java index b71a654f1031..453f6ab6db88 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java @@ -19,15 +19,18 @@ import static org.apache.beam.runners.core.construction.Environments.JAVA_SDK_HARNESS_CONTAINER_URL; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; import static org.junit.Assert.assertEquals; import java.io.File; import java.io.IOException; import java.io.Serializable; +import java.util.HashMap; import java.util.List; import java.util.Optional; import org.apache.beam.model.pipeline.v1.Endpoints; @@ -353,4 +356,42 @@ public void testGetArtifactsBadNamedFileLogsWarn() throws Exception { assertThat(artifacts, hasSize(1)); expectedLogs.verifyWarn("name 'file_name' was not found"); } + + @Test + public void testExpandAnyOfEnvironmentsOnOrdinaryEnvironment() { + Environment env = Environments.createDockerEnvironment("java"); + assertThat(Environments.expandAnyOfEnvironments(env), contains(env)); + } + + @Test + public void testExpandAnyOfEnvironmentsOnNestedEnvironment() { + Environment envA = Environments.createDockerEnvironment("A"); + Environment envB = Environments.createDockerEnvironment("B"); + Environment envC = Environments.createDockerEnvironment("C"); + Environment env = + Environments.createAnyOfEnvironment(envA, Environments.createAnyOfEnvironment(envB, envC)); + assertThat(Environments.expandAnyOfEnvironments(env), contains(envA, envB, envC)); + } + + @Test + public void testResolveAnyOfEnvironment() { + Environment dockerEnv = Environments.createDockerEnvironment("A"); + Environment processEnv = + Environments.createProcessEnvironment("os", "arch", "cmd", new HashMap<>()); + Environment env = + Environments.createAnyOfEnvironment( + dockerEnv, Environments.createAnyOfEnvironment(processEnv)); + assertThat( + Environments.resolveAnyOfEnvironment( + env, BeamUrns.getUrn(StandardEnvironments.Environments.DOCKER)), + equalTo(dockerEnv)); + assertThat( + Environments.resolveAnyOfEnvironment( + env, BeamUrns.getUrn(StandardEnvironments.Environments.PROCESS)), + equalTo(processEnv)); + assertThat( + Environments.resolveAnyOfEnvironment( + env, BeamUrns.getUrn(StandardEnvironments.Environments.EXTERNAL)), + notNullValue()); + } } diff --git a/sdks/python/apache_beam/transforms/environments.py b/sdks/python/apache_beam/transforms/environments.py index 109fcb825347..b2fbe87a729a 100644 --- a/sdks/python/apache_beam/transforms/environments.py +++ b/sdks/python/apache_beam/transforms/environments.py @@ -61,6 +61,7 @@ __all__ = [ 'Environment', + 'AnyOfEnvironment', 'DefaultEnvironment', 'DockerEnvironment', 'ProcessEnvironment', @@ -584,6 +585,24 @@ def from_options(cls, options): resource_hints=resource_hints_from_options(options)) +def expand_anyof_environments(env_proto): + if env_proto.urn == common_urns.environments.ANYOF.urn: + for alt in beam_runner_api_pb2.AnyOfEnvironmentPayload.FromString( + env_proto.payload).environments: + yield from expand_anyof_environments(alt) + else: + yield env_proto + + +def resolve_anyof_environment(env_proto, *preferred_types): + all_environments = list(expand_anyof_environments(env_proto)) + for preferred_type in preferred_types: + for env in all_environments: + if env.urn == preferred_type: + return env + return all_environments[0] + + @Environment.register_urn(python_urns.EMBEDDED_PYTHON, None) class EmbeddedPythonEnvironment(Environment): def to_runner_api_parameter(self, context): @@ -796,6 +815,45 @@ def from_command_string(cls, command_string): command_string, capabilities=python_sdk_capabilities(), artifacts=()) +@Environment.register_urn( + common_urns.environments.ANYOF.urn, + beam_runner_api_pb2.AnyOfEnvironmentPayload) +class AnyOfEnvironment(Environment): + def __init__(self, environments): + self._environments = environments + + def to_runner_api_parameter(self, context): + # type: (PipelineContext) -> Tuple[str, beam_runner_api_pb2.AnyOfEnvironmentPayload] + return ( + common_urns.environments.ANYOF.urn, + beam_runner_api_pb2.AnyOfEnvironmentPayload( + environments=[ + env.to_runner_api(context) for env in self._environments + ])) + + @staticmethod + def from_runner_api_parameter(payload, # type: beam_runner_api_pb2.AnyOfEnvironmentPayload + capabilities, # type: Iterable[str] + artifacts, # type: Iterable[beam_runner_api_pb2.ArtifactInformation] + resource_hints, # type: Mapping[str, bytes] + context # type: PipelineContext + ): + # type: (...) -> AnyOfEnvironment + return AnyOfEnvironment([ + Environment.from_runner_api(env, context) + for env in payload.environments + ]) + + @staticmethod + def create_proto( + environments: Iterable[beam_runner_api_pb2.Environment] + ) -> beam_runner_api_pb2.Environment: + return beam_runner_api_pb2.Environment( + urn=common_urns.environments.ANYOF.urn, + payload=beam_runner_api_pb2.AnyOfEnvironmentPayload( + environments=environments).SerializeToString()) + + class PyPIArtifactRegistry(object): _registered_artifacts = set() # type: Set[Tuple[str, str]] From e5a9ab6233ec76b0aab17474c684b1fad7fe5082 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:23:42 -0700 Subject: [PATCH 002/224] Add AnyOf environment handling to the various runners. --- .../beam/runners/dataflow/DataflowRunner.java | 16 ++++++++++++++ .../control/DefaultJobBundleFactory.java | 6 +++++ .../runners/dataflow/dataflow_runner.py | 8 ++++++- .../portability/fn_api_runner/fn_runner.py | 22 ++++++++++++------- .../fn_api_runner/worker_handlers.py | 12 ++++++++++ 5 files changed, 55 insertions(+), 9 deletions(-) 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 d2b10f91c064..41d149b6bf58 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 @@ -877,6 +877,21 @@ public PTransformReplacement, PCollection> getRepla } } + private RunnerApi.Pipeline resolveAnyOfEnvironments(RunnerApi.Pipeline pipeline) { + RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); + RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); + componentsBuilder.clearEnvironments(); + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { + componentsBuilder.putEnvironments( + entry.getKey(), + Environments.resolveAnyOfEnvironment( + entry.getValue(), + BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER))); + } + return pipelineBuilder.build(); + } + protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline pipeline, DataflowPipelineOptions options) { String sdkHarnessContainerImageOverrides = options.getSdkHarnessContainerImageOverrides(); @@ -1173,6 +1188,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { PipelineTranslation.toProto(pipeline, portableComponents, false); // Note that `stageArtifacts` has to be called before `resolveArtifact` because // `resolveArtifact` updates local paths to staged paths in pipeline proto. + portablePipelineProto = resolveAnyOfEnvironments(portablePipelineProto); List packages = stageArtifacts(portablePipelineProto); portablePipelineProto = resolveArtifacts(portablePipelineProto); portablePipelineProto = applySdkEnvironmentOverrides(portablePipelineProto, options); diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java index 824c2c78cc50..019028f5b936 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java @@ -232,6 +232,12 @@ private ImmutableList createEnvironmentCaches( new CacheLoader() { @Override public WrappedSdkHarnessClient load(Environment environment) throws Exception { + // TODO(robertwb): Docker is the the safest fallback (if we are distributed) + // but it would be good to have the ability to make a more intellegent choice + // (e.g. in-process or loopback workers, especially if running locally). + environment = + Environments.resolveAnyOfEnvironment( + environment, BeamUrns.getUrn(StandardEnvironments.Environments.DOCKER)); EnvironmentFactory.Provider environmentFactoryProvider = environmentFactoryProviderMap.get(environment.getUrn()); ServerFactory serverFactory = environmentFactoryProvider.getServerFactory(); diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 7ad6ab04be68..dc315119e480 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -47,6 +47,7 @@ from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineRunner from apache_beam.runners.runner import PipelineState +from apache_beam.transforms import environments from apache_beam.typehints import typehints from apache_beam.utils import processes from apache_beam.utils.interactive_utils import is_in_notebook @@ -380,7 +381,6 @@ def run_pipeline(self, pipeline, options, pipeline_proto=None): self.proto_pipeline = pipeline_proto else: - from apache_beam.transforms import environments if options.view_as(SetupOptions).prebuild_sdk_container_engine: # if prebuild_sdk_container_engine is specified we will build a new sdk # container image with dependencies pre-installed and use that image, @@ -414,6 +414,12 @@ def run_pipeline(self, pipeline, options, pipeline_proto=None): self.proto_pipeline, self.proto_context = pipeline.to_runner_api( return_context=True, default_environment=self._default_environment) + # Dataflow can only handle Docker environments. + for env_id, env in self.proto_pipeline.components.environments.items(): + self.proto_pipeline.components.environments[env_id].CopyFrom( + environments.resolve_anyof_environment( + env, common_urns.environments.DOCKER.urn)) + # Optimize the pipeline if it not streaming and the pre_optimize # experiment is set. if not options.view_as(StandardOptions).streaming: diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py index 098e16933b73..9abf1d9ab8b6 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py @@ -230,11 +230,16 @@ def embed_default_docker_image(self, pipeline_proto): docker_env = environments.DockerEnvironment.from_container_image( environments.DockerEnvironment.default_docker_image()).to_runner_api( None) # type: ignore[arg-type] - for env_id, env in pipeline_proto.components.environments.items(): - if env == docker_env: - docker_env_id = env_id - break - else: + + def is_python_docker_env(env): + return any( + e == docker_env for e in environments.expand_anyof_environments(env)) + + python_docker_environments = set( + env_id + for (env_id, env) in pipeline_proto.components.environments.items() + if is_python_docker_env(env)) + if not python_docker_environments: # No matching docker environments. return pipeline_proto @@ -244,12 +249,13 @@ def embed_default_docker_image(self, pipeline_proto): break else: # No existing embedded environment. - pipeline_proto.components.environments[docker_env_id].CopyFrom( - embedded_env) + for docker_env_id in python_docker_environments: + pipeline_proto.components.environments[docker_env_id].CopyFrom( + embedded_env) return pipeline_proto for transform in pipeline_proto.components.transforms.values(): - if transform.environment_id == docker_env_id: + if transform.environment_id in python_docker_environments: transform.environment_id = embedded_env_id return pipeline_proto diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index b11c8349909c..de55235368e3 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -879,6 +879,18 @@ def get_worker_handlers( environment_id = next(iter(self._environments.keys())) environment = self._environments[environment_id] + if environment.urn == common_urns.environments.ANYOF.urn: + payload = beam_runner_api_pb2.AnyOfEnvironmentPayload.FromString( + environment.payload) + env_rankings = { + python_urns.EMBEDDED_PYTHON: 10, + common_urns.environments.EXTERNAL.urn: 5, + common_urns.environments.DOCKER.urn: 1, + } + environment = sorted( + payload.environments, + key=lambda env: env_rankings.get(env.urn, -1))[-1] + # assume all environments except EMBEDDED_PYTHON use gRPC. if environment.urn == python_urns.EMBEDDED_PYTHON: # special case for EmbeddedWorkerHandler: there's no need for a gRPC From b136f1d3dd6da8b7bea48e8587e0d2bb7f360483 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:25:57 -0700 Subject: [PATCH 003/224] Add AnyOf environment handling to the artifact staging services. --- .../jobsubmission/InMemoryJobService.java | 58 ++++++++++++------ .../portability/fn_api_runner/fn_runner.py | 19 +++--- .../runners/portability/local_job_service.py | 59 +++++++++++++++---- .../python/apache_beam/transforms/external.py | 20 +++++-- 4 files changed, 115 insertions(+), 41 deletions(-) diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java index 17efbf9a06ec..41e6135b9207 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java @@ -49,6 +49,7 @@ import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.core.construction.graph.PipelineValidator; import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; import org.apache.beam.sdk.fn.server.FnService; @@ -62,7 +63,6 @@ import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.StatusRuntimeException; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; 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.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -198,11 +198,7 @@ public void prepare( stagingService .getService() - .registerJob( - stagingSessionToken, - Maps.transformValues( - request.getPipeline().getComponents().getEnvironmentsMap(), - RunnerApi.Environment::getDependenciesList)); + .registerJob(stagingSessionToken, extractDependencies(request.getPipeline())); // send response PrepareJobResponse response = @@ -287,26 +283,50 @@ public void run(RunJobRequest request, StreamObserver responseOb } } + private Map> extractDependencies( + RunnerApi.Pipeline pipeline) { + Map> dependencies = new HashMap<>(); + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { + List subEnvs = Environments.expandAnyOfEnvironments(entry.getValue()); + for (int i = 0; i < subEnvs.size(); i++) { + dependencies.put(i + ":" + entry.getKey(), subEnvs.get(i).getDependenciesList()); + } + } + return dependencies; + } + private RunnerApi.Pipeline resolveDependencies(RunnerApi.Pipeline pipeline, String stagingToken) { Map> resolvedDependencies = stagingService.getService().getStagedArtifacts(stagingToken); Map newEnvironments = new HashMap<>(); for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { - if (entry.getValue().getDependenciesCount() > 0 && resolvedDependencies == null) { - throw new RuntimeException( - "Artifact dependencies provided but not staged for " + entry.getKey()); + List subEnvs = Environments.expandAnyOfEnvironments(entry.getValue()); + List newSubEnvs = new ArrayList<>(); + for (int i = 0; i < subEnvs.size(); i++) { + RunnerApi.Environment subEnv = subEnvs.get(i); + if (subEnv.getDependenciesCount() > 0 && resolvedDependencies == null) { + throw new RuntimeException( + "Artifact dependencies provided but not staged for " + entry.getKey()); + } + newSubEnvs.add( + subEnv.getDependenciesCount() == 0 + ? subEnv + : subEnv + .toBuilder() + .clearDependencies() + .addAllDependencies(resolvedDependencies.get(i + ":" + entry.getKey())) + .build()); + } + if (newSubEnvs.size() == 1) { + newEnvironments.put(entry.getKey(), newSubEnvs.get(0)); + } else { + newEnvironments.put( + entry.getKey(), + Environments.createAnyOfEnvironment( + newSubEnvs.toArray(new RunnerApi.Environment[newSubEnvs.size()]))); } - newEnvironments.put( - entry.getKey(), - entry.getValue().getDependenciesCount() == 0 - ? entry.getValue() - : entry - .getValue() - .toBuilder() - .clearDependencies() - .addAllDependencies(resolvedDependencies.get(entry.getKey())) - .build()); } RunnerApi.Pipeline.Builder builder = pipeline.toBuilder(); builder.getComponentsBuilder().clearEnvironments().putAllEnvironments(newEnvironments); diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py index 9abf1d9ab8b6..a736288dc62d 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py @@ -224,6 +224,10 @@ def run_via_runner_api(self, pipeline_proto, options): return self.run_stages(stage_context, stages) def embed_default_docker_image(self, pipeline_proto): + """Updates the pipeline proto to execute transforms that would normally + be executed in the default docker image for this SDK to execute inline + via the "embedded" environment. + """ # Context is unused for these types. embedded_env = environments.EmbeddedPythonEnvironment.default( ).to_runner_api(None) # type: ignore[arg-type] @@ -231,14 +235,14 @@ def embed_default_docker_image(self, pipeline_proto): environments.DockerEnvironment.default_docker_image()).to_runner_api( None) # type: ignore[arg-type] - def is_python_docker_env(env): + def is_this_python_docker_env(env): return any( e == docker_env for e in environments.expand_anyof_environments(env)) python_docker_environments = set( env_id for (env_id, env) in pipeline_proto.components.environments.items() - if is_python_docker_env(env)) + if is_this_python_docker_env(env)) if not python_docker_environments: # No matching docker environments. return pipeline_proto @@ -248,11 +252,12 @@ def is_python_docker_env(env): embedded_env_id = env_id break else: - # No existing embedded environment. - for docker_env_id in python_docker_environments: - pipeline_proto.components.environments[docker_env_id].CopyFrom( - embedded_env) - return pipeline_proto + # No existing embedded environment. Create one. + embedded_env_id = "python_embedded_env" + while embedded_env_id in pipeline_proto.components.environments: + embedded_env_id += '_' + pipeline_proto.components.environments[embedded_env_id].CopyFrom( + embedded_env) for transform in pipeline_proto.components.transforms.values(): if transform.environment_id in python_docker_environments: diff --git a/sdks/python/apache_beam/runners/portability/local_job_service.py b/sdks/python/apache_beam/runners/portability/local_job_service.py index 91ddb3fced15..6966e66d2c64 100644 --- a/sdks/python/apache_beam/runners/portability/local_job_service.py +++ b/sdks/python/apache_beam/runners/portability/local_job_service.py @@ -28,7 +28,9 @@ import time import traceback from typing import TYPE_CHECKING +from typing import Any from typing import List +from typing import Mapping from typing import Optional import grpc @@ -43,6 +45,7 @@ from apache_beam.portability.api import beam_job_api_pb2 from apache_beam.portability.api import beam_job_api_pb2_grpc from apache_beam.portability.api import beam_provision_api_pb2 +from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.portability.api import endpoints_pb2 from apache_beam.runners.job import utils as job_utils from apache_beam.runners.portability import abstract_job_service @@ -51,11 +54,11 @@ from apache_beam.runners.portability.fn_api_runner import fn_runner from apache_beam.runners.portability.fn_api_runner import worker_handlers from apache_beam.runners.worker.log_handler import LOGENTRY_TO_LOG_LEVEL_MAP +from apache_beam.transforms import environments from apache_beam.utils import thread_pool_executor if TYPE_CHECKING: from google.protobuf import struct_pb2 # pylint: disable=ungrouped-imports - from apache_beam.portability.api import beam_runner_api_pb2 _LOGGER = logging.getLogger(__name__) @@ -96,10 +99,8 @@ def create_beam_job(self, # type: (...) -> BeamJob self._artifact_service.register_job( staging_token=preparation_id, - dependency_sets={ - id: env.dependencies - for (id, env) in pipeline.components.environments.items() - }) + dependency_sets=_extract_dependency_sets( + pipeline.components.environments)) provision_info = fn_runner.ExtendedProvisionInfo( beam_provision_api_pb2.ProvisionInfo(pipeline_options=options), self._staging_dir, @@ -321,12 +322,9 @@ def _invoke_runner(self): def _update_dependencies(self): try: - for env_id, deps in self._artifact_service.resolved_deps( - self._job_id, timeout=0).items(): - # Slice assignment not supported for repeated fields. - env = self._pipeline_proto.components.environments[env_id] - del env.dependencies[:] - env.dependencies.extend(deps) + _update_dependency_sets( + self._pipeline_proto.components.environments, + self._artifact_service.resolved_deps(self._job_id, timeout=0)) self._provision_info.provision_info.ClearField('retrieval_token') except concurrent.futures.TimeoutError: # TODO(https://github.com/apache/beam/issues/20267): Require this once @@ -457,3 +455,42 @@ def emit(self, record): # Inform all message consumers. self._log_queues.put(msg) + + +def _extract_dependency_sets( + envs: Mapping[str, beam_runner_api_pb2.Environment] +) -> Mapping[Any, List[beam_runner_api_pb2.ArtifactInformation]]: + """Expands the set of environments into a mapping of (opaque) keys to + dependency sets. This is not 1:1 in the case of AnyOf environments. + + The values can then be resolved and the mapping passed back to + _update_dependency_sets to update the dependencies in the original protos. + """ + def dependencies_iter(): + for env_id, env in envs.items(): + for ix, sub_env in enumerate(environments.expand_anyof_environments(env)): + yield (env_id, ix), sub_env.dependencies + + return dict(dependencies_iter()) + + +def _update_dependency_sets( + envs: Mapping[str, beam_runner_api_pb2.Environment], + resolved_deps: Mapping[Any, List[beam_runner_api_pb2.ArtifactInformation]]): + """Takes the mapping of beam Environments (originally passed to + `_extract_dependency_sets`) and a set of (key-wise) updated dependencies, + and updates the original environment protos to contain the updated + dependencies. + """ + for env_id, env in envs.items(): + new_envs = [] + for ix, sub_env in enumerate(environments.expand_anyof_environments(env)): + # Slice assignment not supported for repeated fields. + del sub_env.dependencies[:] + sub_env.dependencies.extend(resolved_deps[env_id, ix]) + new_envs.append(sub_env) + if len(new_envs) == 1: + envs[env_id].CopyFrom(new_envs[0]) + else: + envs[env_id].CopyFrom( + environments.AnyOfEnvironment.create_proto(new_envs)) diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 0d0b6f1e7be2..52e27ecc2e8b 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -43,6 +43,7 @@ from apache_beam.portability.api import external_transforms_pb2 from apache_beam.runners import pipeline_context from apache_beam.runners.portability import artifact_service +from apache_beam.transforms import environments from apache_beam.transforms import ptransform from apache_beam.typehints import WithTypeHints from apache_beam.typehints import native_type_compatibility @@ -731,8 +732,9 @@ def expand(self, pvalueish): if response.error: raise RuntimeError(response.error) self._expanded_components = response.components - if any(env.dependencies - for env in self._expanded_components.environments.values()): + if any(e.dependencies + for env in self._expanded_components.environments.values() + for e in environments.expand_anyof_environments(env)): self._expanded_components = self._resolve_artifacts( self._expanded_components, service.artifact_service(), @@ -785,12 +787,22 @@ def service(expansion_service): yield stub def _resolve_artifacts(self, components, service, dest): - for env in components.environments.values(): - if env.dependencies: + def _resolve_artifacts_for(env): + if env.urn == common_urns.environments.ANYOF.urn: + env.CopyFrom( + environments.AnyOfEnvironment.create_proto([ + _resolve_artifacts_for(e) + for e in environments.expand_anyof_environments(env) + ])) + elif env.dependencies: resolved = list( artifact_service.resolve_artifacts(env.dependencies, service, dest)) del env.dependencies[:] env.dependencies.extend(resolved) + return env + + for env in components.environments.values(): + _resolve_artifacts_for(env) return components def _output_to_pvalueish(self, output_dict): From c6c70860400844b2597e4fd568e5f0f8cad7b0d0 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 30 Nov 2023 18:22:43 -0500 Subject: [PATCH 004/224] Enable triggering postcommit via specific file (#29549) * Enable triggering postcommit via specific file * address comments * Add file path trigger approach and its context to Comment Triggering Support section --- .github/build.gradle | 35 +- .github/workflows/README.md | 357 ++++++++---------- .github/workflows/beam_Java_JMH.yml | 2 +- .github/workflows/beam_PostCommit_Go.yml | 2 +- .../beam_PostCommit_Go_Dataflow_ARM.yml | 2 +- .../workflows/beam_PostCommit_Go_VR_Flink.yml | 2 +- .../workflows/beam_PostCommit_Go_VR_Samza.yml | 2 +- .../workflows/beam_PostCommit_Go_VR_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .../beam_PostCommit_Java_Avro_Versions.yml | 2 +- ...m_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .../beam_PostCommit_Java_DataflowV1.yml | 2 +- .../beam_PostCommit_Java_DataflowV2.yml | 2 +- ...beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- ...m_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .../beam_PostCommit_Java_Examples_Direct.yml | 2 +- .../beam_PostCommit_Java_Examples_Flink.yml | 2 +- .../beam_PostCommit_Java_Examples_Spark.yml | 2 +- .../beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- ...m_PostCommit_Java_IO_Performance_Tests.yml | 2 +- .../beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java21.yml | 2 +- ...beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- ...beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 +- ...am_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 +- ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Direct.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Flink.yml | 2 +- .../beam_PostCommit_Java_Nexmark_Spark.yml | 2 +- ...am_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .../beam_PostCommit_Java_PVR_Samza.yml | 2 +- ...m_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .../beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 +- .../beam_PostCommit_Java_Sickbay.yml | 2 +- .../beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 +- .../beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 +- .../beam_PostCommit_Java_Tpcds_Flink.yml | 2 +- .../beam_PostCommit_Java_Tpcds_Spark.yml | 2 +- ...stCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ..._ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- ...ava_ValidatesRunner_Dataflow_Streaming.yml | 2 +- ...ommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ..._ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- ...PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...va_ValidatesRunner_Direct_JavaVersions.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- ...mmit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...lidatesRunner_SparkStructuredStreaming.yml | 2 +- ...mmit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- ...stCommit_Java_ValidatesRunner_Twister2.yml | 2 +- ...am_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .../beam_PostCommit_PortableJar_Flink.yml | 2 +- .../beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .../workflows/beam_PostCommit_Python_Arm.yml | 2 +- ...am_PostCommit_Python_Examples_Dataflow.yml | 2 +- ...beam_PostCommit_Python_Examples_Direct.yml | 2 +- .../beam_PostCommit_Python_Examples_Flink.yml | 2 +- .../beam_PostCommit_Python_Examples_Spark.yml | 2 +- .../beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- .../beam_PostCommit_Python_Nexmark_Direct.yml | 2 +- ...mit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...on_ValidatesContainer_Dataflow_With_RC.yml | 2 +- ...Commit_Python_ValidatesRunner_Dataflow.yml | 2 +- ...ostCommit_Python_ValidatesRunner_Flink.yml | 2 +- ...ostCommit_Python_ValidatesRunner_Samza.yml | 2 +- ...ostCommit_Python_ValidatesRunner_Spark.yml | 2 +- ...m_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- ...eam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- ...am_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .../beam_PostCommit_Sickbay_Python.yml | 2 +- ...eam_PostCommit_TransformService_Direct.yml | 2 +- .../beam_PostCommit_Website_Test.yml | 2 +- .../workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .../workflows/beam_PostCommit_XVR_Flink.yml | 2 +- ...am_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- ...ostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- ...Commit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- ...ostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 +- .../workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .../workflows/beam_PostCommit_XVR_Spark3.yml | 2 +- .../beam_PreCommit_CommunityMetrics.yml | 2 +- .github/workflows/beam_PreCommit_GHA.yml | 2 +- .github/workflows/beam_PreCommit_Go.yml | 2 +- .../workflows/beam_PreCommit_GoPortable.yml | 2 +- .github/workflows/beam_PreCommit_GoPrism.yml | 2 +- .../workflows/beam_PreCommit_ItFramework.yml | 1 + .github/workflows/beam_PreCommit_Java.yml | 1 + ...it_Java_Amazon-Web-Services2_IO_Direct.yml | 1 + ...mit_Java_Amazon-Web-Services_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Amqp_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Azure_IO_Direct.yml | 1 + ...eam_PreCommit_Java_Cassandra_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Cdap_IO_Direct.yml | 1 + ...am_PreCommit_Java_Clickhouse_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Csv_IO_Direct.yml | 1 + ...beam_PreCommit_Java_Debezium_IO_Direct.yml | 1 + ...PreCommit_Java_ElasticSearch_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Examples_Dataflow.yml | 1 + ...reCommit_Java_Examples_Dataflow_Java21.yml | 1 + ...t_Java_File-schema-transform_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Flink_Versions.yml | 1 + .../beam_PreCommit_Java_GCP_IO_Direct.yml | 1 + ...am_PreCommit_Java_Google-ads_IO_Direct.yml | 1 + .../beam_PreCommit_Java_HBase_IO_Direct.yml | 1 + ...beam_PreCommit_Java_HCatalog_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Hadoop_IO_Direct.yml | 1 + .../beam_PreCommit_Java_IOs_Direct.yml | 1 + ...beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 1 + .../beam_PreCommit_Java_JDBC_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Jms_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kinesis_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Kudu_IO_Direct.yml | 1 + .../beam_PreCommit_Java_MongoDb_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Mqtt_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Neo4j_IO_Direct.yml | 1 + .../beam_PreCommit_Java_PVR_Flink_Batch.yml | 1 + .../beam_PreCommit_Java_PVR_Flink_Docker.yml | 1 + .../beam_PreCommit_Java_Parquet_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 1 + ...beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Redis_IO_Direct.yml | 1 + ...eCommit_Java_RequestResponse_IO_Direct.yml | 1 + ...m_PreCommit_Java_SingleStore_IO_Direct.yml | 1 + ...eam_PreCommit_Java_Snowflake_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Solr_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Spark3_Versions.yml | 1 + .../beam_PreCommit_Java_Splunk_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Thrift_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Tika_IO_Direct.yml | 1 + .../beam_PreCommit_Kotlin_Examples.yml | 1 + .../beam_PreCommit_Portable_Python.yml | 1 + .github/workflows/beam_PreCommit_Python.yml | 2 +- .../workflows/beam_PreCommit_PythonDocker.yml | 2 +- .../workflows/beam_PreCommit_PythonDocs.yml | 2 +- .../beam_PreCommit_PythonFormatter.yml | 2 +- .../workflows/beam_PreCommit_PythonLint.yml | 2 +- .../beam_PreCommit_Python_Coverage.yml | 2 +- .../beam_PreCommit_Python_Dataframes.yml | 2 +- .../beam_PreCommit_Python_Examples.yml | 2 +- .../beam_PreCommit_Python_Integration.yml | 2 +- .../beam_PreCommit_Python_PVR_Flink.yml | 1 + .../beam_PreCommit_Python_Runners.yml | 2 +- .../beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .github/workflows/beam_PreCommit_Spotless.yml | 1 + .../workflows/beam_PreCommit_Typescript.yml | 2 +- .github/workflows/beam_PreCommit_Website.yml | 2 +- .../beam_PreCommit_Website_Stage_GCS.yml | 2 +- .../workflows/beam_PreCommit_Whitespace.yml | 2 +- ...blish.yml => beam_Publish_BeamMetrics.yml} | 4 +- ...e_Publish.yml => beam_Publish_Website.yml} | 6 +- 167 files changed, 356 insertions(+), 323 deletions(-) rename .github/workflows/{beam_PostCommit_BeamMetrics_Publish.yml => beam_Publish_BeamMetrics.yml} (97%) rename .github/workflows/{beam_PostCommit_Website_Publish.yml => beam_Publish_Website.yml} (95%) diff --git a/.github/build.gradle b/.github/build.gradle index acfe7f3686a9..de2670545625 100644 --- a/.github/build.gradle +++ b/.github/build.gradle @@ -28,6 +28,7 @@ buildscript { /** check that yml are valid */ task check { doLast { + List errors = [] fileTree("${project.projectDir}/workflows").matching { include "*.yml" include "*.yaml" @@ -36,23 +37,41 @@ task check { // attempt load yml to make sure its valid def workflow = new org.yaml.snakeyaml.Yaml().load(it.newInputStream()) - // additional guards for running all tests functionality - // TODO(yathu) expand this also to post commits prior teardown Jenkins postcommits - if ( fname.startsWith("beam_PreCommit")) { + // additional guards to ensure tests configured in same way + if ( fname.startsWith("beam_PreCommit") || fname.startsWith("beam_PostCommit") ) { List paths try { paths = workflow.getAt(true).pull_request_target.paths as List } catch (Exception e) { - throw new GradleException("Fail to get the trigger path for ${fname}. " + - "Make sure precommit has a pull_request_target trigger.", e) + errors.add("Fail to get the trigger path for ${fname}. " + + "Make sure it has a pull_request_target trigger.") + return } - // precommit should triggered by this specific file + + // precommit and postcommit should triggered by this specific file + // this is to ensure not missing test during release branch verification if (paths != null && !paths.contains('release/trigger_all_tests.json')) { - throw new GradleException("Error validating ${fname}: " + - "Please add 'release/trigger_all_tests.json' to the trigger path for release verification run properly") + errors.add("Error validating ${fname}: " + + "Please add 'release/trigger_all_tests.json' to the trigger path") + return + } + + // postcommit should triggered by a specific file so that there is a way to exercise post for open PR + // TODO(https://github.com/apache/beam/issues/28909) + // remove file match trigger once a better trigger (e.g. comment trigger) is implemented + if (fname.startsWith("beam_PostCommit")) { + String triggerFile = '.github/trigger_files/' + fname.take(fname.lastIndexOf('.')) + '.json' + if (paths != null && !paths.contains(triggerFile)) { + errors.add("Error validating ${fname}: " + + "Please add ${triggerFile} to the trigger path") + return + } } } } + if (!errors.isEmpty()) { + throw new GradleException("Check failed: " + errors.join('\n')) + } } } diff --git a/.github/workflows/README.md b/.github/workflows/README.md index c4f437dedd29..d0ed952232ed 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -138,6 +138,8 @@ In order to make it easier for non-committers to interact with workflows, workfl 2) Each job should have the rerun action immediately after its checkout step. You can add a step that uses the `setup-action` action in your workflow, which encapsulates the checkout and rerun logic in one place. This should be gated on the comment trigger (example: https://github.com/apache/beam/blob/0ee2dc73ec6f555a5bf1a643dffd37f4927be67e/.github/workflows/beam_PreCommit_Go.yml#L65-L70) 3) Each job should have a descriptive name that includes the comment trigger (example: https://github.com/apache/beam/blob/ba8fc935222aeb070668fbafd588bc58e7a21289/.github/workflows/beam_PreCommit_CommunityMetrics.yml#L48) +**Note:** this approach is found not scalable ([#28909](https://github.com/apache/beam/issues/28909)) and currently only enabled for PreCommit workflows. For PostCommit jobs, it is currently replaced by a temporary approach of `pull_request_target` trigger with specific path `.github/trigger_files/`. + # Testing new workflows or workflow updates ## Testing New Workflows @@ -192,6 +194,8 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex ### PreCommit Jobs +PreCommit Jobs run in a schedule and also get triggered in a PR if relevant sources has changed. To manually trigger certain PreCommit job, comment with the Trigger Phrase (listed below) in the PR. + | Workflow name | Matrix | Trigger Phrase | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| | [ PreCommit Community Metrics ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml) | N/A |`Run CommunityMetrics PreCommit`| [![.github/workflows/beam_PreCommit_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_CommunityMetrics.yml?query=event%3Aschedule) | @@ -269,207 +273,168 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex ### PostCommit Jobs -| Workflow name | Matrix | Trigger Phrase | Cron Status | -|:-------------:|:------:|:--------------:|:-----------:| -| [ PostCommit BeamMetrics Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml) | N/A |`Run Beam Metrics Deployment`| [![.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_BeamMetrics_Publish.yml?query=event%3Aschedule) -| [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`Run Go PostCommit`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml?query=event%3Aschedule) | -| [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`Run Go PostCommit Dataflow ARM`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml?query=event%3Aschedule) | -| [ PostCommit Go VR Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml) | N/A |`Run Go Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml?query=event%3Aschedule) | -| [ PostCommit Go VR Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml) | N/A |`Run Go Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml?query=event%3Aschedule) | -| [ PostCommit Go VR Spark](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml) | N/A |`Run Go Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Go_VR_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Java Avro Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | N/A |`Run Java Avro Versions PostCommit`| [![.github/workflows/beam_PostCommit_Java_Avro_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml?query=event%3Aschedule) | -| [ PostCommit Java Dataflow V1 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | N/A |`Run PostCommit_Java_Dataflow`| [![.github/workflows/beam_PostCommit_Java_DataflowV1.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml?query=event%3Aschedule) | -| [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`Run PostCommit_Java_DataflowV2`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17','21'] |`Run Java_Examples_Dataflow_ARM PostCommit (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`Run Java examples on Dataflow`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17','21'] |`Run Java examples on Dataflow Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`Run Java Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17','21'] |`Run Java (matrix_element) Examples on Dataflow Runner V2`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`Run Java Examples_Direct`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`Run Java Examples_Flink`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml?query=event%3Aschedule) | -| [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`Run Java Examples_Spark`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`Run PostCommit_Java_Hadoop_Versions`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml?query=event%3Aschedule) | -| [ PostCommit 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?query=event%3Aschedule) -| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`Run Jpms Dataflow Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`Run Jpms Dataflow Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`Run Jpms Direct Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`Run Jpms Direct Java 17 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Direct Java21 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml) | N/A |`Run Jpms Direct Java21 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`Run Jpms Flink Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`Run Jpms Spark Java 11 PostCommit`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`Run Dataflow Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | N/A |`Run Dataflow Runner V2 Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | ['11','17'] |`Run Dataflow Runner V2 Java (matrix) Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`Run Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`Run Flink Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml?query=event%3Aschedule) | -| [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`Run Spark Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`Run Java Flink PortableValidatesRunner Streaming`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml?query=event%3Aschedule) | -| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`Run Java Samza PortableValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml?query=event%3Aschedule) | -| [ PostCommit Java SingleStoreIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml) | N/A |`Run Java SingleStoreIO_IT`| [![.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml?query=event%3Aschedule) | -| [ 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`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](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?query=event%3Aschedule) | -| [ 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`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | N/A |`Run Dataflow ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | ['11','17'] |`Run Direct ValidatesRunner Java (matrix_element)`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | N/A |`Run Direct ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | N/A |`Run Flink ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | N/A |`Run Flink ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml) | N/A |`Run Samza ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml) | N/A |`Run Spark ValidatesRunner Java 11`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml) | N/A |`Run Spark ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner SparkStructuredStreaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml) | N/A |`Run Spark StructuredStreaming ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner Twister2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml) | N/A |`Run Twister2 ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml?query=event%3Aschedule) | -| [ PostCommit Java ValidatesRunner ULR ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml) | N/A |`Run ULR Loopback ValidatesRunner`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml?query=event%3Aschedule) | -| [ PostCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml) | N/A |`Run Java PostCommit`| [![.github/workflows/beam_PostCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml?query=event%3Aschedule) | -| [ PostCommit Javadoc ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml) | N/A |`Run Javadoc PostCommit`| [![.github/workflows/beam_PostCommit_Javadoc.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ PostCommit Python Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml) | ['3.8','3.11'] |`Run Python Examples_Spark (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Python MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml) | N/A |`Run Python MongoDBIO_IT`| [![.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml?query=event%3Aschedule) | -| [ PostCommit Python Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | N/A |`Run Python Direct Runner Nexmark Tests`| [![.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesContainer Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesContainer Dataflow With RC ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python RC Dataflow ValidatesContainer (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`Run Python Dataflow ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`Run Python Flink ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`Run Python Samza ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml?query=event%3Aschedule) | -| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`Run Python Spark ValidatesRunner (matrix_element)`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml?query=event%3Aschedule) | -| [ PostCommit Python Xlang Gcp Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | N/A |`Run Python_Xlang_Gcp_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Python Xlang Gcp Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | N/A |`Run Python_Xlang_Gcp_Direct PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml?query=event%3Aschedule) | -| [ PostCommit Python Xlang IO Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | N/A |`Run Python_Xlang_IO_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit Sickbay Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python (matrix_element) PostCommit Sickbay`| [![.github/workflows/beam_PostCommit_Sickbay_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml?query=event%3Aschedule) | -| [ PostCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | N/A |`Run SQL PostCommit`| [![.github/workflows/beam_PostCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml?query=event%3Aschedule) | -| [ PostCommit TransformService Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | N/A |`Run TransformService_Direct PostCommit`| [![.github/workflows/beam_PostCommit_TransformService_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml?query=event%3Aschedule) -| [ PostCommit Website Publish ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml) | N/A | N/A | [![.github/workflows/beam_PostCommit_Website_Publish.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Publish.yml?query=event%3Aschedule) | -| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`Run Full Website Test`| [![.github/workflows/beam_PostCommit_Website_Test.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml?query=event%3Aschedule) | -| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`Run XVR_GoUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`Run XVR_Direct PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml?query=event%3Aschedule) | -| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`Run XVR_Flink PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml?query=event%3Aschedule) | -| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`Run XVR_JavaUsingPython_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`Run XVR_PythonUsingJava_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`Run XVR_PythonUsingJavaSQL_Dataflow PostCommit`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml?query=event%3Aschedule) | -| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`Run XVR_Samza PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml?query=event%3Aschedule) | -| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`Run XVR_Spark3 PostCommit`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml?query=event%3Aschedule) | -| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`Run Python ValidatesContainer Dataflow ARM (matrix_element)`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml?query=event%3Aschedule) | - -### PerformanceTests and Benchmark Jobs +PostCommit Jobs run in a schedule against master branch and generally do not get triggered in a PR. To manually trigger certain PostCommit job, attach a file named "Trigger file" (listed below) under `.github/trigger_files/` folder. -| Workflow name | Matrix | Trigger Phrase | Cron Status | +| Workflow name | Matrix | Trigger file | Cron Status | |:-------------:|:------:|:--------------:|:-----------:| -| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A |`Run TFT Criteo Benchmarks`| [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml?query=event%3Aschedule) -| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A |`Run Inference Benchmarks`| [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml?query=event%3Aschedule) -| [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml?query=event%3Aschedule) -| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A |`Run Java AvroIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml?query=event%3Aschedule) -| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A |`Run Java AvroIO Performance Test`| [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml?query=event%3Aschedule) -| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Avro`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml?query=event%3Aschedule) -| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A |`Run BigQueryIO Batch Performance Test Java Json`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml?query=event%3Aschedule) -| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A |`Run BigQueryIO Streaming Performance Test Java`| [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml?query=event%3Aschedule) -| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A |`Run BigQueryIO Read Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml?query=event%3Aschedule) -| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A |`Run BigQueryIO Write Performance Test Python`| [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml?query=event%3Aschedule) -| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A |`Run Java CdapIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml?query=event%3Aschedule) -| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A |`Run Java CompressedTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A |`Run Java CompressedTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A |`Run Java HadoopFormatIO Performance Test`| [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml?query=event%3Aschedule) -| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A |`Run Java JdbcIO Performance Test`| [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml?query=event%3Aschedule) -| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A |`Run Java KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml?query=event%3Aschedule) -| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A |`Run Java ManyFilesTextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A |`Run Java ManyFilesTextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A |`Run Java MongoDBIO Performance Test`| [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml?query=event%3Aschedule) -| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A |`Run Java ParquetIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A |`Run Java ParquetIO Performance Test`| [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A |`Run PubsubIO Performance Test Python`| [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml?query=event%3Aschedule) -| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A |`Run Java SingleStoreIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml?query=event%3Aschedule) -| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A |`Run SpannerIO Read 2GB Performance Test Python`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml?query=event%3Aschedule) -| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A |`Run SpannerIO Write 2GB Performance Test Python Batch`| [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml?query=event%3Aschedule) -| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A |`Run Java SparkReceiverIO Performance Test`| [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml?query=event%3Aschedule) -| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A |`Run SQLBigQueryIO Batch Performance Test Java`| [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml?query=event%3Aschedule) -| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A |`Run Java TextIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A |`Run Python TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml?query=event%3Aschedule) -| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A |`Run Java TextIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A |`Run Java TFRecordIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A |`Run Java TFRecordIO Performance Test`| [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] |`Run Python (matrix_element) WordCountIT Performance Test`| [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml?query=event%3Aschedule) -| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A |`Run Java XmlIO Performance Test HDFS`| [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml?query=event%3Aschedule) -| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A |`Run Java XmlIO Performance Test`| [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml?query=event%3Aschedule) -| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A |`Run Python xlang KafkaIO Performance Test`| [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml?query=event%3Aschedule) +| [ PostCommit Go ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml) | N/A |`beam_PostCommit_Go.json`| [![.github/workflows/beam_PostCommit_Go.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go.yml?query=event%3Aschedule) | +| [ PostCommit Go Dataflow ARM](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml) | N/A |`beam_PostCommit_Go_Dataflow_ARM.json`| [![.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_Dataflow_ARM.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Flink](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml) | N/A |`beam_PostCommit_Go_VR_Flink.json`| [![.github/workflows/beam_PostCommit_Go_VR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Samza](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml) | N/A |`beam_PostCommit_Go_VR_Samza.json`| [![.github/workflows/beam_PostCommit_Go_VR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Go VR Spark](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml) | N/A |`beam_PostCommit_Go_VR_Spark.json`| [![.github/workflows/beam_PostCommit_Go_VR_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Go_VR_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java Avro Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml) | N/A |`beam_PostCommit_Java_Avro_Versions.json`| [![.github/workflows/beam_PostCommit_Java_Avro_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Avro_Versions.yml?query=event%3Aschedule) | +| [ PostCommit Java Dataflow V1 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml) | N/A |`beam_PostCommit_Java_DataflowV1.json`| [![.github/workflows/beam_PostCommit_Java_DataflowV1.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV1.yml?query=event%3Aschedule) | +| [ PostCommit Java Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml) | N/A |`beam_PostCommit_Java_DataflowV2.json`| [![.github/workflows/beam_PostCommit_Java_DataflowV2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_DataflowV2.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml) | ['8','11','17','21'] |`beam_PostCommit_Java_Examples_Dataflow_ARM.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml) | N/A |`beam_PostCommit_Java_Examples_Dataflow.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml) | ['11','17','21'] |`beam_PostCommit_Java_Examples_Dataflow_Java.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml) | N/A |`beam_PostCommit_Java_Examples_Dataflow_V2.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml) | ['11','17','21'] |`beam_PostCommit_Java_Examples_Dataflow_V2_Java.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml) | N/A |`beam_PostCommit_Java_Examples_Direct.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`beam_PostCommit_Java_Examples_Flink.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`beam_PostCommit_Java_Examples_Spark.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`beam_PostCommit_Java_Hadoop_Versions.json`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml?query=event%3Aschedule) | +| [ PostCommit Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`beam_PostCommit_Java_InfluxDbIO_IT.json`| [![.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?query=event%3Aschedule) +| [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`beam_PostCommit_Java_Jpms_Dataflow_Java11.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`beam_PostCommit_Java_Jpms_Dataflow_Java17.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml) | N/A |`beam_PostCommit_Java_Jpms_Direct_Java11.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml) | N/A |`beam_PostCommit_Java_Jpms_Direct_Java17.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Direct Java21 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml) | N/A |`beam_PostCommit_Java_Jpms_Direct_Java21.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml) | N/A |`beam_PostCommit_Java_Jpms_Flink_Java11.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Jpms Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml) | N/A |`beam_PostCommit_Java_Jpms_Spark_Java11.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml) | N/A |`beam_PostCommit_Java_Nexmark_Dataflow.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml) | N/A |`beam_PostCommit_Java_Nexmark_Dataflow_V2.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Dataflow V2 Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml) | ['11','17'] |`beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`beam_PostCommit_Java_Nexmark_Direct.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`beam_PostCommit_Java_Nexmark_Flink.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`beam_PostCommit_Java_Nexmark_Spark.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`beam_PostCommit_Java_PVR_Flink_Streaming.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`beam_PostCommit_Java_PVR_Samza.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Java SingleStoreIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml) | N/A |`beam_PostCommit_Java_SingleStoreIO_IT.json`| [![.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml?query=event%3Aschedule) | +| [ PostCommit Java PVR Spark3 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml) | N/A |`beam_PostCommit_Java_PVR_Spark3_Streaming.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml](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?query=event%3Aschedule) | +| [ PostCommit Java PVR Spark Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml) | N/A |`beam_PostCommit_Java_PVR_Spark_Batch.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml](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?query=event%3Aschedule) | +| [ PostCommit Java Sickbay ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Sickbay.yml) | N/A |`beam_PostCommit_Java_Sickbay.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml) | N/A |`beam_PostCommit_Java_Tpcds_Dataflow.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Flink.yml) | N/A |`beam_PostCommit_Java_Tpcds_Flink.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java Tpcds Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Tpcds_Spark.yml) | N/A |`beam_PostCommit_Java_Tpcds_Spark.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml) | ['11','17'] |`beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow V2 Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow V2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Dataflow.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Direct JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml) | ['11','17'] |`beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Direct.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Flink Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Flink.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Samza.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Spark Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Spark.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner SparkStructuredStreaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner Twister2 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_Twister2.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml?query=event%3Aschedule) | +| [ PostCommit Java ValidatesRunner ULR ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml) | N/A |`beam_PostCommit_Java_ValidatesRunner_ULR.json`| [![.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml?query=event%3Aschedule) | +| [ PostCommit Java ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml) | N/A |`beam_PostCommit_Java.json`| [![.github/workflows/beam_PostCommit_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java.yml?query=event%3Aschedule) | +| [ PostCommit Javadoc ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml) | N/A |`beam_PostCommit_Javadoc.json`| [![.github/workflows/beam_PostCommit_Javadoc.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Javadoc.yml?query=event%3Aschedule) | +| [ PostCommit PortableJar Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Flink.yml) | N/A |`beam_PostCommit_PortableJar_Flink.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit PortableJar Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_PortableJar_Spark.yml) | N/A |`beam_PostCommit_PortableJar_Spark.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python.yml) | ['3.8','3.9','3.10','3.11'] |`beam_PostCommit_Python.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Python Arm](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Arm.yml) | ['3.8','3.9','3.10','3.11'] |`beam_PostCommit_Python_Arm.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Python Examples Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Dataflow.yml) | N/A |`beam_PostCommit_Python_Examples_Dataflow.json`| [![.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?query=event%3Aschedule) | +| [ 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'] |`beam_PostCommit_Python_Examples_Direct.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Python Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Flink.yml) | ['3.8','3.11'] |`beam_PostCommit_Python_Examples_Flink.json`| [![.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?query=event%3Aschedule) | +| [ PostCommit Python Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml) | ['3.8','3.11'] |`beam_PostCommit_Python_Examples_Spark.json`| [![.github/workflows/beam_PostCommit_Python_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Examples_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Python MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml) | N/A |`beam_PostCommit_Python_MongoDBIO_IT.json`| [![.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml?query=event%3Aschedule) | +| [ PostCommit Python Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml) | N/A |`beam_PostCommit_Python_Nexmark_Direct.json`| [![.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Nexmark_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesContainer Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml) | ['3.8','3.9','3.10','3.11'] |`beam_PostCommit_Python_ValidatesContainer_Dataflow.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesContainer Dataflow With RC ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml) | ['3.8','3.9','3.10','3.11'] |`beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml) | ['3.8','3.11'] |`beam_PostCommit_Python_ValidatesRunner_Dataflow.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml) | ['3.8','3.11'] |`beam_PostCommit_Python_ValidatesRunner_Flink.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml) | ['3.8','3.11'] |`beam_PostCommit_Python_ValidatesRunner_Samza.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml?query=event%3Aschedule) | +| [ PostCommit Python ValidatesRunner Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml) | ['3.8','3.9','3.11'] |`beam_PostCommit_Python_ValidatesRunner_Spark.json`| [![.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang Gcp Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml) | N/A |`beam_PostCommit_Python_Xlang_Gcp_Dataflow.json`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang Gcp Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml) | N/A |`beam_PostCommit_Python_Xlang_Gcp_Direct.json`| [![.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml?query=event%3Aschedule) | +| [ PostCommit Python Xlang IO Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml) | N/A |`beam_PostCommit_Python_Xlang_IO_Dataflow.json`| [![.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit Sickbay Python ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml) | ['3.8','3.9','3.10','3.11'] |`beam_PostCommit_Sickbay_Python.json`| [![.github/workflows/beam_PostCommit_Sickbay_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Sickbay_Python.yml?query=event%3Aschedule) | +| [ PostCommit SQL ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml) | N/A |`beam_PostCommit_SQL.json`| [![.github/workflows/beam_PostCommit_SQL.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_SQL.yml?query=event%3Aschedule) | +| [ PostCommit TransformService Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml) | N/A |`beam_PostCommit_TransformService_Direct.json`| [![.github/workflows/beam_PostCommit_TransformService_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_TransformService_Direct.yml?query=event%3Aschedule) +| [ PostCommit Website Test](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml) | N/A |`beam_PostCommit_Website_Test.json`| [![.github/workflows/beam_PostCommit_Website_Test.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Website_Test.yml?query=event%3Aschedule) | +| [ PostCommit XVR GoUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml) | N/A |`beam_PostCommit_XVR_GoUsingJava_Dataflow.json`| [![.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml) | N/A |`beam_PostCommit_XVR_Direct.json`| [![.github/workflows/beam_PostCommit_XVR_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Direct.yml?query=event%3Aschedule) | +| [ PostCommit XVR Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml) | N/A |`beam_PostCommit_XVR_Flink.json`| [![.github/workflows/beam_PostCommit_XVR_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Flink.yml?query=event%3Aschedule) | +| [ PostCommit XVR JavaUsingPython Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml) | N/A |`beam_PostCommit_XVR_JavaUsingPython_Dataflow.json`| [![.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR PythonUsingJava Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml) | N/A |`beam_PostCommit_XVR_PythonUsingJava_Dataflow.json`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml?query=event%3Aschedule) | +| [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`beam_PostCommit_XVR_Samza.json`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml?query=event%3Aschedule) | +| [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`beam_PostCommit_XVR_Spark3.json`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml?query=event%3Aschedule) | +| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`beam_Python_ValidatesContainer_Dataflow_ARM.json`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml?query=event%3Aschedule) | ### LoadTests Jobs -| Workflow name | Matrix | Trigger Phrase | Cron Status | -|:-------------:|:------:|:--------------:|:-----------:| -| [ LoadTests Go CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A |`Run LoadTests Go CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) | N/A |`Run Load Tests Go CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml?query=event%3Aschedule) -| [ LoadTests Go Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Go Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Go Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go GBK Dataflow Batch ](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?query=event%3Aschedule) -| [ LoadTests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Go GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go ParDo Dataflow Batch ](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.yml](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?query=event%3Aschedule) -| [ LoadTests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Go ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Go SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A |`Run Load Tests Go SideInput Flink Batch`| [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java CoGBK Dataflow Streaming ](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?query=event%3Aschedule) -| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) -| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) CoGBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) -| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java CoGBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A |`Run Load Tests Java Combine Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java Combine SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Java GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A |`Run Load Tests Java 11 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A |`Run Load Tests Java 17 GBK Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml?query=event%3Aschedule) -| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A |`Run Java Load Tests GBK Smoke`| [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml?query=event%3Aschedule) -| [ LoadTests Java GBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java GBK SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Java ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A |`Run Load Tests Java ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) -| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] |`Run Load Tests Java (matrix_element) ParDo Dataflow V2 Streaming`| [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) -| [ LoadTests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A |`Run Load Tests Java ParDo SparkStructuredStreaming Batch`| [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) -| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml?query=event%3Aschedule) -| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python CoGBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A |`Run Load Tests Python CoGBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python Combine Dataflow Batch ](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?query=event%3Aschedule) -| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python Combine Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A |`Run Load Tests Python Combine Flink Batch`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A |`Run Load Tests Python Combine Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python FnApiRunner Microbenchmark ](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?query=event%3Aschedule) -| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A |`Run Load Tests Python GBK Flink Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A |`Run Load Tests Python GBK reiterate Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A |`Run Load Tests Python ParDo Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A |`Run Python Load Tests ParDo Dataflow Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A |`Run Load Tests Python ParDo Flink Batch`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A |`Run Load Tests Python ParDo Flink Streaming`| [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml?query=event%3Aschedule) -| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A |`Run Load Tests Python SideInput Dataflow Batch`| [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) -| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A |`Run Python Load Tests Smoke`| [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml?query=event%3Aschedule) +| Workflow name | Matrix | Cron Status | +|:-------------:|:------:|:-----------:| +| [ LoadTests Go CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml?query=event%3Aschedule) +| [ LoadTests Go Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml) | N/A | [![.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?query=event%3Aschedule) +| [ LoadTests Go GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml](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?query=event%3Aschedule) +| [ LoadTests Go ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Go SideInput Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml) | N/A | [![.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?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | [![.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java CoGBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java Combine SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Batch Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Batch Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Streaming Java11 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Dataflow V2 Streaming Java17 ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml?query=event%3Aschedule) +| [ LoadTests Java GBK Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_Smoke.yml?query=event%3Aschedule) +| [ LoadTests Java GBK SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow V2 Batch JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml) | ['11','17'] | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Batch_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo Dataflow V2 Streaming JavaVersions ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml) | ['11','17'] | [![.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_Dataflow_V2_Streaming_JavaVersions.yml?query=event%3Aschedule) +| [ LoadTests Java ParDo SparkStructuredStreaming Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml?query=event%3Aschedule) +| [ LoadTests Java Combine Smoke ](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml) | N/A | [![.github/workflows/beam_Java_LoadTests_Combine_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_LoadTests_Combine_Smoke.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python CoGBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml) | N/A | [![.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?query=event%3Aschedule) +| [ LoadTests Python Combine Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Combine Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python FnApiRunner Microbenchmark ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml) | N/A | [![.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?query=event%3Aschedule) +| [ LoadTests Python GBK Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python GBK Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK reiterate Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python GBK reiterate Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_GBK_reiterate_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Dataflow Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Dataflow_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python ParDo Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml?query=event%3Aschedule) +| [ LoadTests Python SideInput Dataflow Batch ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml?query=event%3Aschedule) +| [ LoadTests Python Smoke ](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml) | N/A | [![.github/workflows/beam_LoadTests_Python_Smoke.yml](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_LoadTests_Python_Smoke.yml?query=event%3Aschedule) ### Other Jobs -| Workflow name | Matrix | Trigger Phrase | Cron Status | -|:-------------:|:------:|:--------------:|:-----------:| -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ 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?query=event%3Aschedule) | -| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml?query=event%3Aschedule) -| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A |`Run Community Metrics Prober`| [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml?query=event%3Aschedule) -| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml?query=event%3Aschedule) | -| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A |`Publish Docker Snapshots`| [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml?query=event%3Aschedule) | -| [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml?query=event%3Aschedule) | -| [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml?query=event%3Aschedule) | +| Workflow name | Matrix | Cron Status | +|:-------------:|:------:|:-----------:| +| [ Publish BeamMetrics ](https://github.com/apache/beam/actions/workflows/beam_Publish_BeamMetrics.yml) | N/A | [![.github/workflows/beam_Publish_BeamMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_BeamMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_BeamMetrics.yml?query=event%3Aschedule) +| [ Publish Website ](https://github.com/apache/beam/actions/workflows/beam_Publish_Website.yml) | N/A | [![.github/workflows/beam_Publish_Website.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Website.yml?query=event%3Aschedule) | +| [ Cancel Stale Dataflow Jobs ](https://github.com/apache/beam/actions/workflows/beam_CancelStaleDataflowJobs.yml) | N/A | [![.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?query=event%3Aschedule) | +| [ Clean Up GCP Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpGCPResources.yml) | N/A | [![.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?query=event%3Aschedule) | +| [ Clean Up Prebuilt SDK Images ](https://github.com/apache/beam/actions/workflows/beam_CleanUpPrebuiltSDKImages.yml) | N/A | [![.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?query=event%3Aschedule) | +| [ Cleanup Dataproc Resources ](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml) | N/A | [![.github/workflows/beam_CleanUpDataprocResources.yml](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CleanUpDataprocResources.yml?query=event%3Aschedule) +| [ Community Metrics Prober ](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml) | N/A | [![.github/workflows/beam_Prober_CommunityMetrics.yml](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Prober_CommunityMetrics.yml?query=event%3Aschedule) +| [ Publish Beam SDK Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml) | N/A | [![.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Beam_SDK_Snapshots.yml?query=event%3Aschedule) | +| [ Publish Docker Snapshots ](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml) | N/A | [![.github/workflows/beam_Publish_Docker_Snapshots.yml](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Publish_Docker_Snapshots.yml?query=event%3Aschedule) | +| [ Rotate IO-Datastores Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml) | N/A | [![.github/workflows/beam_IODatastoresCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml?query=event%3Aschedule) | +| [ Rotate Metrics Cluster Credentials ](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml) | N/A | [![.github/workflows/beam_MetricsCredentialsRotation.yml](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml?query=event%3Aschedule) | diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml index ccd3162873ad..a90d1344856a 100644 --- a/.github/workflows/beam_Java_JMH.yml +++ b/.github/workflows/beam_Java_JMH.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 0 * * 0' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_Java_JMH.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index f9da27ff2ce9..821ec55e2d1b 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Go.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml index 532c48e7b072..615b7f1f94b2 100644 --- a/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Go_Dataflow_ARM.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Go_Dataflow_ARM.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index ce1e82d22e2a..c85f0c00035a 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Go_VR_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 5d1ab6a8d13e..3ac3742500fc 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Go_VR_Samza.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index ed9f50583133..d85038befe38 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Go_VR_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index d9fa4d52f965..c19651301c0f 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 36bdec265dd5..9217df5b9ec1 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Avro_Versions.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index f21f301cc265..065884eea252 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_BigQueryEarlyRollout.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 68c901c2e527..c3883e862bc6 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_DataflowV1.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 90a97296df77..4ee629853326 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_DataflowV2.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 488031a9244a..6f508f257a7e 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 26ea487e0dcf..4c0d2295c60b 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -23,7 +23,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_ARM.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 747c9f0983fc..dfc909a4c6d5 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_Java.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index ae50ec506dff..51ef625ae7de 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs 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 1ed3cf5eb981..621b35d57935 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2_Java.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index fa7e76942808..e3022c96ea1a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Direct.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index 8e7a99f12ac8..12e76f334710 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Flink.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index f927fa8b3b46..5ff6a1b52485 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Examples_Spark.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index c0c85fb0cd31..c6a31be712a8 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 3/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 4bbc23b13de4..671b93e0dcff 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_IO_Performance_Tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index c1f75be3851a..0bfbdedadcca 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_InfluxDbIO_IT.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 2449054a1073..988566dff67d 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Dataflow_Java11.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 611c8a9d31de..22416c9a7aa6 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Dataflow_Java17.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 6fdff9c451eb..bac658ceea24 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Direct_Java11.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 3f62861eb025..73727c04e212 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Direct_Java17.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index e1926a430069..c2c1b44550fe 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 */6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Direct_Java21.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index fbc0d16e4994..81bc4b6a6100 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 2c2e2acc6f19..7405093a6ac0 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Jpms_Spark_Java11.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index f4e96961061e..b501aed67f2a 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Dataflow.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 875f54ea7632..0a4cdb8cd174 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Dataflow_V2.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs 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 ba3f8bb61005..20a061a0bfee 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 23d766c89823..1d98192b8255 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Direct.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index dd87fec145a4..4feedafc341c 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Flink.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index 05229fb4c89b..e1421397f7d6 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Nexmark_Spark.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index cffe074f58eb..f1b5e6cea23e 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_PVR_Flink_Streaming.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 729e95fe5219..f437efdad407 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_PVR_Samza.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index d3508e968fe0..3c6ea80f48d1 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_PVR_Spark3_Streaming.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 8cc977ddea82..71fa65b31608 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_PVR_Spark_Batch.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index e6de4b2538b5..5d7c2d393911 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Sickbay.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 2fff2365d817..ddbbfa3e5d42 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_SingleStoreIO_IT.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event permissions: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index cc04e5e3b493..12b74ae4e443 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Tpcds_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index 55f7b65c9b5d..5174baebcfb1 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Tpcds_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 8bf21480f621..370c9c406d46 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_Tpcds_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 338b148ff3fb..bc4fe988e878 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 86b6b32a4abb..66c41ea1f363 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index abe21ac3f783..8601739c0234 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 63625b48ea2f..d02cf0e57944 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 6/8 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs 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 b372c4c2acda..ef292ecdc0cd 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 6/8 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 16e21c9faa26..7cd0138e9b53 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index fafc821a2b20..5d81527b0fed 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 8171760f5528..25606da982aa 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index 6bc1b0ffa03c..cafd4da44ece 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 8f8993f3108e..29a7c8caaf82 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index ce67510b1e50..e0ac939e65cc 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 7883218d262a..a37aed2eba61 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 099f9ab1e724..c9f66ac46ee8 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index de3d4914052e..6e32c42041ee 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Twister2.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index eb2139c562e9..31f0b5d9be90 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 4/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 8e2a170dba63..607ba6436647 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Javadoc.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 2f066979681e..6142f607367d 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_PortableJar_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index 8b5bc031f7fa..a4aba3cdbd33 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_PortableJar_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 0b22466f517b..1dd4a4161eb4 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 1a88c468a67c..cf760512d30d 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -23,7 +23,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Arm.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index ae07cb0273e0..fdcd406df1aa 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Examples_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index b4b620e5dd1f..d30c73c9c1dc 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Examples_Direct.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 1f334f6a9963..adfdb586ea55 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Examples_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 6a33c63f24a0..b81323cac09c 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '0 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Examples_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 1c8daf4cffa3..7e594f39b191 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_MongoDBIO_IT.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index f63461369be0..2ce1db22e102 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Nexmark_Direct.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 196dd1eaa84e..ecdfda1971e4 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesContainer_Dataflow.json'] workflow_dispatch: issue_comment: types: [created] 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 ac2179dd2521..590ecd5ffad9 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 79bc303f1117..a9e3e33b8e50 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index b403f76b9f9e..e45ed572b079 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 4229304278c0..e861a0ae12ae 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Samza.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 06db87f8fb76..c89e1632049b 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Spark.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index dd899a538e9a..295a9eff9b14 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '15 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 33eb748a2f84..7ab8c7a56c14 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 4d71e507fe32..3df1d24aa455 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index eae7d4374d5b..077ec9d9ed0c 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_SQL.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index b4d054f07a3d..b9b761284b29 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -21,7 +21,7 @@ on: schedule: - cron: '0 8 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Sickbay_Python.json'] workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index e40112f0c5b1..2d6312cf2487 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_TransformService_Direct.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index ba372d223281..95e49faf13a5 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Website_Test.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index ee90f9176f6d..51a5acbc3603 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_Direct.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 00c756a714f3..6cf0e4077505 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '30 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_Flink.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index aab8a0e0a84f..f674412f195e 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 113c51626078..a5edf5a9131f 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_JavaUsingPython_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 6759930d5de8..cd10e823d63a 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index dda068049a17..d76955772ac4 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_PythonUsingJava_Dataflow.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index b05b588e0cf8..f9b9bd387688 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_Samza.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 0742196a6960..6b90316e110d 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -19,7 +19,7 @@ on: schedule: - cron: '45 5/6 * * *' pull_request_target: - paths: ['release/trigger_all_tests.json'] + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_XVR_Spark3.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index 570dd5dc3354..778383a7f7db 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -22,7 +22,7 @@ on: paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', 'release/trigger_all_tests.json'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_CommunityMetrics.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GHA.yml b/.github/workflows/beam_PreCommit_GHA.yml index 94811bc0f507..33edb12a14d5 100644 --- a/.github/workflows/beam_PreCommit_GHA.yml +++ b/.github/workflows/beam_PreCommit_GHA.yml @@ -22,7 +22,7 @@ on: paths: ['.github/**/*.yml'] pull_request_target: branches: ['master', 'release-*' ] - paths: ['.github/**/*.yml', 'release/trigger_all_tests.json'] + paths: ['.github/**/*.yml', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_GHA.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Go.yml b/.github/workflows/beam_PreCommit_Go.yml index 7df1296ca124..2e268e21684e 100644 --- a/.github/workflows/beam_PreCommit_Go.yml +++ b/.github/workflows/beam_PreCommit_Go.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', '.github/workflows/beam_PreCommit_Go.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json'] + paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Go.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GoPortable.yml b/.github/workflows/beam_PreCommit_GoPortable.yml index 609340e8ded3..8d62e73d5b25 100644 --- a/.github/workflows/beam_PreCommit_GoPortable.yml +++ b/.github/workflows/beam_PreCommit_GoPortable.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', '.github/workflows/beam_PreCommit_GoPortable.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json'] + paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_GoPortable.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_GoPrism.yml b/.github/workflows/beam_PreCommit_GoPrism.yml index 52c5553cd623..4af064f45c42 100644 --- a/.github/workflows/beam_PreCommit_GoPrism.yml +++ b/.github/workflows/beam_PreCommit_GoPrism.yml @@ -22,7 +22,7 @@ on: paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', '.github/workflows/beam_PreCommit_GoPrism.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json'] + paths: ['model/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/**', 'release/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_GoPrism.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index cf4cd2980176..92e99cf40113 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -29,6 +29,7 @@ on: paths: - 'it/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_ItFramework.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index fe694b4eab70..b4aa233d522d 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -78,6 +78,7 @@ on: - 'examples/kotlin/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java.json' - '!sdks/java/extensions/avro/**' - '!sdks/java/extensions/sql/**' - '!sdks/java/io/amazon-web-services/**' 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 16d6562f827f..77c5ddb42089 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 @@ -38,6 +38,7 @@ on: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.json' - "build.gradle" - "buildSrc/**" - "gradle/**" 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 f7c1d66e2f1e..eb9e10c5b6ba 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 @@ -38,6 +38,7 @@ on: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 8df097d8428f..9d6a2c9cb690 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/amqp/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Amqp_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index d32361708491..865015bef141 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -38,6 +38,7 @@ on: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Azure_IO_Direct.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 04e1b10c23bd..02a2be854e1a 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/cassandra/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Cassandra_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index f23aad3ea455..d54e2387983c 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -31,6 +31,7 @@ on: - "sdks/java/io/hadoop-common/**" - "sdks/java/io/hadoop-format/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Cdap_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index f7c5ea1b8615..5915fd585bde 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/clickhouse/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Clickhouse_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 496e0c15ea66..d44957bb9edc 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/csv/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Csv_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index 97264ac146a5..5eabeffd52b7 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/debezium/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Debezium_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index f51e50aa8200..becf5ac0477d 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/elasticsearch/**" - "sdks/java/io/elasticsearch-tests/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_ElasticSearch_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index b96e459470cf..5639a2002d7f 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -39,6 +39,7 @@ on: - 'examples/kotlin/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Examples_Dataflow.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index f482e22ec4a2..cb1bcdb5e491 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -37,6 +37,7 @@ on: - 'examples/kotlin/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Examples_Dataflow_Java21.json' issue_comment: types: [created] schedule: 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 8ce77fef6416..893f0197b5bb 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 @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/file-schema-transform/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_File-schema-transform_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index d5c244b2dbc9..97c99d57d350 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -31,6 +31,7 @@ on: - 'runners/flink/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Flink_Versions.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 73dd9d63d872..3f1219de012b 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -37,6 +37,7 @@ on: - "runners/core-construction-java/**" - "runners/core-java/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_GCP_IO_Direct.json' - "sdks/java/core/src/main/**" - "sdks/java/extensions/arrow/**" - "sdks/java/extensions/google-cloud-platform-core/**" diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml index 7e0ca73e7efd..9322565ca1d4 100644 --- a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/google-ads/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Google-ads_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 805561f3475d..5ebac3e1fc01 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/hbase/**" - "sdks/java/io/hadoop-common/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_HBase_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index bb9e5ee31f27..499576966272 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/hcatalog/**" - "sdks/java/io/hadoop-common/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_HCatalog_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index ea09a8df70d6..caa3b75099d2 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -53,6 +53,7 @@ on: - "sdks/java/io/hadoop-common/**" - "sdks/java/io/hadoop-format/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Hadoop_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 4a221a911009..8dea3a3c0b81 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_IOs_Direct.json' issue_comment: types: [created] workflow_dispatch: diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 52a03ceea204..8e06a57ceba6 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/influxdb/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_InfluxDb_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index f328dd513c88..64afee7bff3f 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/jdbc/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_JDBC_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 9abd63bb79c0..3e101fab78a5 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/jms/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Jms_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index c73c384c7881..1207618f3424 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -35,6 +35,7 @@ on: - "sdks/java/io/synthetic/**" - "sdks/java/io/expansion-service/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Kafka_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 2aa4d4c10fd8..234c31914a94 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -45,6 +45,7 @@ on: - "gradle.bat" - "settings.gradle.kts" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Kinesis_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 25f90072db5c..0df56a76d509 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/kudu/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Kudu_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 34fa18a23c71..0799ac469353 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/mongodb/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_MongoDb_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 0b26f28e8f69..0ee6810f8188 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/mqtt/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Mqtt_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index ed6d6baf456b..9253d588a6dd 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/neo4j/**" - "sdks/java/testing/test-utils/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Neo4j_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 0ff7785650de..a0dea4b64712 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -33,6 +33,7 @@ on: - 'runners/java-fn-execution/**' - 'sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_PVR_Flink_Batch.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 841c6dbdcbab..b63d29953d85 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -37,6 +37,7 @@ on: - 'runners/flink/**' - 'runners/java-fn-execution/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_PVR_Flink_Docker.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 998a6bbbac04..8621d0c8f42d 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/parquet/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Parquet_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index 00becbfd7ee6..042f436bfbaa 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -38,6 +38,7 @@ on: - "sdks/java/io/common/**" - "sdks/java/core/src/main/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Pulsar_IO_Direct.json' - "build.gradle" - "buildSrc/**" - "gradle/**" diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 7adbfe74bab5..bfa89b57871d 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/rabbitmq/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_RabbitMq_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 255df740a57f..23546c77ac24 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/redis/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Redis_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index cfe721c1e948..d2035836ccf0 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/rrio/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_RequestResponse_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 733921da7ad2..087340ce9530 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -29,6 +29,7 @@ on: - "sdks/java/io/singlestore/**" - "sdks/java/testing/test-utils/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_SingleStore_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index c84f0026b726..8706aa96335d 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -31,6 +31,7 @@ on: - "sdks/java/extensions/google-cloud-platform-core/**" - "sdks/java/testing/test-utils/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Snowflake_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index f3142b41dd4d..dbcce6c30b83 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/solr/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Solr_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 9665e9770bc0..ad4a09dd165b 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -29,6 +29,7 @@ on: paths: - 'runners/spark/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Spark3_Versions.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 7032cc62a62a..821e42bc26fb 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/splunk/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Splunk_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index da26e226e105..49e2d651e50e 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/thrift/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Thrift_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 20eff37521e2..5e3f81323502 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -27,6 +27,7 @@ on: paths: - "sdks/java/io/tika/**" - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Tika_IO_Direct.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml index 13d533442575..2a804bb9de09 100644 --- a/.github/workflows/beam_PreCommit_Kotlin_Examples.yml +++ b/.github/workflows/beam_PreCommit_Kotlin_Examples.yml @@ -39,6 +39,7 @@ on: - 'examples/kotlin/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Kotlin_Examples.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Portable_Python.yml b/.github/workflows/beam_PreCommit_Portable_Python.yml index 169bdb74a649..2fa4905ca736 100644 --- a/.github/workflows/beam_PreCommit_Portable_Python.yml +++ b/.github/workflows/beam_PreCommit_Portable_Python.yml @@ -43,6 +43,7 @@ on: - 'sdks/python/**' - 'release/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Portable_Python.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 670f41875ffb..61160bb7b68c 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -17,7 +17,7 @@ name: PreCommit Python on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 26ecf97c9b52..754fbac95d0d 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -17,7 +17,7 @@ name: PreCommit Python Docker on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_PythonDocker.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonDocs.yml b/.github/workflows/beam_PreCommit_PythonDocs.yml index 2ada89147321..708aa669efda 100644 --- a/.github/workflows/beam_PreCommit_PythonDocs.yml +++ b/.github/workflows/beam_PreCommit_PythonDocs.yml @@ -18,7 +18,7 @@ name: PreCommit Python Docs on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["sdks/python/**", 'release/trigger_all_tests.json'] + paths: ["sdks/python/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_PythonDocs.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonFormatter.yml b/.github/workflows/beam_PreCommit_PythonFormatter.yml index 17d9cd855138..8eaf99ba83a3 100644 --- a/.github/workflows/beam_PreCommit_PythonFormatter.yml +++ b/.github/workflows/beam_PreCommit_PythonFormatter.yml @@ -17,7 +17,7 @@ name: PreCommit Python Formatter on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "sdks/python/apache_beam/**", 'release/trigger_all_tests.json'] + paths: [ "sdks/python/apache_beam/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_PythonFormatter.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_PythonLint.yml b/.github/workflows/beam_PreCommit_PythonLint.yml index 583c484e8e93..2cc469170d57 100644 --- a/.github/workflows/beam_PreCommit_PythonLint.yml +++ b/.github/workflows/beam_PreCommit_PythonLint.yml @@ -17,7 +17,7 @@ name: PreCommit Python Lint on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: ["sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_PythonLint.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index f0444013df59..301e1d26d4b3 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -17,7 +17,7 @@ name: PreCommit Python Coverage on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Coverage.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 822c10862ac7..c827a120b0c4 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -17,7 +17,7 @@ name: PreCommit Python Dataframes on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Dataframes.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 36a0bde3ebdc..685d81d6027e 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -17,7 +17,7 @@ name: PreCommit Python Examples on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Examples.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index d0af4932168e..0eef876a37d7 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -17,7 +17,7 @@ name: PreCommit Python Integration on: pull_request_target: branches: [ "master", "release-*" ] - paths: ["model/**", "sdks/python/**", "release/**", 'release/trigger_all_tests.json'] + paths: ["model/**", "sdks/python/**", "release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Integration.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 7e083bf84313..2fd8add01da3 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -30,6 +30,7 @@ on: - 'runners/java-fn-execution/**' - 'runners/reference/**' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Python_PVR_Flink.json' issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index f43c4eb47b6e..61b916cb7a8d 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -17,7 +17,7 @@ name: PreCommit Python Runners on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Runners.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 6d06596ed4c8..27b23e0d50c2 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -17,7 +17,7 @@ name: PreCommit Python Transforms on: pull_request_target: branches: [ "master", "release-*" ] - paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json'] + paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_Transforms.json'] issue_comment: types: [created] push: diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index ac72f6b752da..0a3e95c675c5 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_SQL.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 9dadb4dcc2ff..73e78332356c 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL_Java11.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_SQL_Java11.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 8a1e88f7c654..fa1c0ea7089e 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -22,7 +22,7 @@ on: paths: ['sdks/java/extensions/sql/**','.github/workflows/beam_PreCommit_SQL_Java17.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json'] + paths: ['sdks/java/extensions/sql/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_SQL_Java17.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index 6aa590e7556e..ab0b28ee0b2b 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -36,6 +36,7 @@ on: - 'examples/kotlin/**' - '.test-infra/jenkins/' - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Spotless.json' issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Typescript.yml b/.github/workflows/beam_PreCommit_Typescript.yml index f3184dc0f90d..678b8d53efe1 100644 --- a/.github/workflows/beam_PreCommit_Typescript.yml +++ b/.github/workflows/beam_PreCommit_Typescript.yml @@ -24,7 +24,7 @@ on: paths: ['sdks/python/apache_beam/runners/interactive/extensions/**', '.github/workflows/beam_PreCommit_Typescript.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['sdks/python/apache_beam/runners/interactive/extensions/**', 'release/trigger_all_tests.json'] + paths: ['sdks/python/apache_beam/runners/interactive/extensions/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Typescript.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Website.yml b/.github/workflows/beam_PreCommit_Website.yml index 848b4d44deec..0626596768f8 100644 --- a/.github/workflows/beam_PreCommit_Website.yml +++ b/.github/workflows/beam_PreCommit_Website.yml @@ -22,7 +22,7 @@ on: paths: ['website/**','.github/workflows/beam_PreCommit_Website.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['website/**', 'release/trigger_all_tests.json'] + paths: ['website/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Website.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml index c7b51f137290..4696d3c6b9c8 100644 --- a/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml +++ b/.github/workflows/beam_PreCommit_Website_Stage_GCS.yml @@ -22,7 +22,7 @@ on: paths: ['website/**','.github/workflows/beam_PreCommit_Website_Stage_GCS.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['website/**', 'release/trigger_all_tests.json'] + paths: ['website/**', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Website_Stage_GCS.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index da58d309f354..e5a13e3cb3cf 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -22,7 +22,7 @@ on: paths: ['**.md', '**.build.gradle', 'build.gradle.kts', '.github/workflows/beam_PreCommit_Whitespace.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['**.md', '**.build.gradle', 'build.gradle.kts', 'release/trigger_all_tests.json'] + paths: ['**.md', '**.build.gradle', 'build.gradle.kts', 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Whitespace.json'] issue_comment: types: [created] schedule: diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_Publish_BeamMetrics.yml similarity index 97% rename from .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml rename to .github/workflows/beam_Publish_BeamMetrics.yml index b655eab8088c..53f62b3bf9d4 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_Publish_BeamMetrics.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: PostCommit BeamMetrics Publish +name: Publish BeamMetrics on: schedule: @@ -48,7 +48,7 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} jobs: - beam_PostCommit_BeamMetrics_Publish: + beam_Publish_BeamMetrics: if: | github.event_name == 'push' || github.event_name == 'workflow_dispatch' || diff --git a/.github/workflows/beam_PostCommit_Website_Publish.yml b/.github/workflows/beam_Publish_Website.yml similarity index 95% rename from .github/workflows/beam_PostCommit_Website_Publish.yml rename to .github/workflows/beam_Publish_Website.yml index ed1729021e98..726071641b7b 100644 --- a/.github/workflows/beam_PostCommit_Website_Publish.yml +++ b/.github/workflows/beam_Publish_Website.yml @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: PostCommit Website Publish +name: Publish Website on: schedule: @@ -47,13 +47,13 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} jobs: - beam_PostCommit_Website_Publish: + beam_Publish_Website: if: | github.event_name == 'workflow_dispatch' || (github.event_name == 'schedule' && github.repository == 'apache/beam') runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 30 - name: beam_PostCommit_Website_Publish + name: beam_Publish_Website steps: - uses: actions/checkout@v4 - name: Setup environment From 6550cec0e9cfcc35991d5d0170360eb92384a572 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:28:06 -0700 Subject: [PATCH 005/224] Add the ability to cache subprocess services for the duration of a pipeline. This can greatly reduce startup time when many cross-langauge transforms are used, but more importantly by keeping these processes alive we open up the potential for using them as workers as well. These can be cached across longer durations as well, but this is the default. --- sdks/python/apache_beam/pipeline.py | 10 +- .../apache_beam/utils/subprocess_server.py | 192 +++++++++++++----- .../utils/subprocess_server_test.py | 79 +++++++ 3 files changed, 227 insertions(+), 54 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index ed0736250d1f..53044982a066 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -48,6 +48,7 @@ # mypy: disallow-untyped-defs import abc +import contextlib import logging import os import re @@ -590,9 +591,12 @@ def run(self, test_runner_api='AUTO'): def __enter__(self): # type: () -> Pipeline - self._extra_context = subprocess_server.JavaJarServer.beam_services( - self._options.view_as(CrossLanguageOptions).beam_services) - self._extra_context.__enter__() + self._extra_context = contextlib.ExitStack() + self._extra_context.enter_context( + subprocess_server.JavaJarServer.beam_services( + self._options.view_as(CrossLanguageOptions).beam_services)) + self._extra_context.enter_context( + subprocess_server.SubprocessServer.cache_subprocesses()) return self def __exit__( diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py index 6ed1568b57a5..7d9cf50d8532 100644 --- a/sdks/python/apache_beam/utils/subprocess_server.py +++ b/sdks/python/apache_beam/utils/subprocess_server.py @@ -18,6 +18,7 @@ # pytype: skip-file import contextlib +import dataclasses import glob import hashlib import logging @@ -27,10 +28,11 @@ import signal import socket import subprocess -import tempfile import threading import time import zipfile +from typing import Any +from typing import Set from urllib.error import URLError from urllib.request import urlopen @@ -42,6 +44,75 @@ _LOGGER = logging.getLogger(__name__) +@dataclasses.dataclass +class _SharedCacheEntry: + obj: Any + owners: Set[str] + + +class _SharedCache: + """A cache that keeps objects alive (and repeatedly returns the same instance) + until the last user indicates that they're done. + + The typical usage is as follows:: + + try: + token = cache.register() + # All objects retrieved from the cache from this point on will be memoized + # and kept alive (including across other threads and callers) at least + # until the purge is called below (and possibly longer, if other calls + # to register were made). + obj = cache.get(...) + another_obj = cache.get(...) + ... + finally: + cache.purge(token) + """ + def __init__(self, constructor, destructor): + self._constructor = constructor + self._destructor = destructor + self._live_owners = set() + self._cache = {} + self._lock = threading.Lock() + self._counter = 0 + + def _next_id(self): + with self._lock: + self._counter += 1 + return self._counter + + def register(self): + owner = self._next_id() + self._live_owners.add(owner) + return owner + + def purge(self, owner): + if owner not in self._live_owners: + raise ValueError(f"{owner} not in {self._live_owners}") + self._live_owners.remove(owner) + to_delete = [] + with self._lock: + for key, entry in list(self._cache.items()): + if owner in entry.owners: + entry.owners.remove(owner) + if not entry.owners: + to_delete.append(entry.obj) + del self._cache[key] + # Actually call the destructors outside of the lock. + for value in to_delete: + self._destructor(value) + + def get(self, *key): + if not self._live_owners: + raise RuntimeError("At least one owner must be registered.") + with self._lock: + if key not in self._cache: + self._cache[key] = _SharedCacheEntry(self._constructor(*key), set()) + for owner in self._live_owners: + self._cache[key].owners.add(owner) + return self._cache[key].obj + + class SubprocessServer(object): """An abstract base class for running GRPC Servers as an external process. @@ -63,13 +134,26 @@ def __init__(self, stub_class, cmd, port=None): string "{{PORT}}" will be substituted in the command line arguments with the chosen port. """ - self._process_lock = threading.RLock() - self._process = None + self._owner_id = None self._stub_class = stub_class self._cmd = [str(arg) for arg in cmd] self._port = port self._grpc_channel = None + @classmethod + @contextlib.contextmanager + def cache_subprocesses(cls): + """A context that ensures any subprocess created or used in its duration + stay alive for at least the duration of this context. + + These subprocesses may be shared with other contexts as well. + """ + try: + unique_id = cls._cache.register() + yield + finally: + cls._cache.purge(unique_id) + def __enter__(self): return self.start() @@ -78,7 +162,7 @@ def __exit__(self, *unused_args): def start(self): try: - endpoint = self.start_process() + process, endpoint = self.start_process() wait_secs = .1 channel_options = [("grpc.max_receive_message_length", -1), ("grpc.max_send_message_length", -1)] @@ -86,10 +170,10 @@ def start(self): endpoint, options=channel_options) channel_ready = grpc.channel_ready_future(self._grpc_channel) while True: - if self._process is not None and self._process.poll() is not None: - _LOGGER.error("Starting job service with %s", self._process.args) + if process is not None and process.poll() is not None: + _LOGGER.error("Started job service with %s", process.args) raise RuntimeError( - 'Service failed to start up with error %s' % self._process.poll()) + 'Service failed to start up with error %s' % process.poll()) try: channel_ready.result(timeout=wait_secs) break @@ -106,60 +190,66 @@ def start(self): raise def start_process(self): - with self._process_lock: - if self._process: - self.stop() - if self._port: - port = self._port - cmd = self._cmd - else: - port, = pick_port(None) - cmd = [arg.replace('{{PORT}}', str(port)) for arg in self._cmd] - endpoint = 'localhost:%s' % port - _LOGGER.info("Starting service with %s", str(cmd).replace("',", "'")) - self._process = subprocess.Popen( - cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) - - # Emit the output of this command as info level logging. - def log_stdout(): - line = self._process.stdout.readline() - while line: - # The log obtained from stdout is bytes, decode it into string. - # Remove newline via rstrip() to not print an empty line. - _LOGGER.info(line.decode(errors='backslashreplace').rstrip()) - line = self._process.stdout.readline() - - t = threading.Thread(target=log_stdout) - t.daemon = True - t.start() - return endpoint + if self._owner_id is not None: + self._cache.purge(self._owner_id) + self._owner_id = self._cache.register() + return self._cache.get(tuple(self._cmd), self._port) + + def _really_start_process(cmd, port): + if not port: + port, = pick_port(None) + cmd = [arg.replace('{{PORT}}', str(port)) for arg in cmd] # pylint: disable=not-an-iterable + endpoint = 'localhost:%s' % port + _LOGGER.info("Starting service with %s", str(cmd).replace("',", "'")) + process = subprocess.Popen( + cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) + + # Emit the output of this command as info level logging. + def log_stdout(): + line = process.stdout.readline() + while line: + # The log obtained from stdout is bytes, decode it into string. + # Remove newline via rstrip() to not print an empty line. + _LOGGER.info(line.decode(errors='backslashreplace').rstrip()) + line = process.stdout.readline() + + t = threading.Thread(target=log_stdout) + t.daemon = True + t.start() + return process, endpoint def stop(self): self.stop_process() def stop_process(self): - with self._process_lock: - if not self._process: - return - for _ in range(5): - if self._process.poll() is not None: - break - logging.debug("Sending SIGINT to job_server") - self._process.send_signal(signal.SIGINT) - time.sleep(1) - if self._process.poll() is None: - self._process.kill() - self._process = None + if self._owner_id is not None: + self._cache.purge(self._owner_id) + self._owner_id = None if self._grpc_channel: try: self._grpc_channel.close() except: # pylint: disable=bare-except _LOGGER.error( - "Could not close the gRPC channel started for the " + + "Could not close the gRPC channel started for the " "expansion service") - - def local_temp_dir(self, **kwargs): - return tempfile.mkdtemp(dir=self._local_temp_root, **kwargs) + finally: + self._grpc_channel = None + + def _really_stop_process(process_and_endpoint): + process, _ = process_and_endpoint # pylint: disable=unpacking-non-sequence + if not process: + return + for _ in range(5): + if process.poll() is not None: + break + logging.debug("Sending SIGINT to process") + process.send_signal(signal.SIGINT) + time.sleep(1) + if process.poll() is None: + process.kill() + + _cache = _SharedCache( + constructor=_really_start_process, destructor=_really_stop_process) class JavaJarServer(SubprocessServer): @@ -184,7 +274,7 @@ def __init__(self, stub_class, path_to_jar, java_arguments, classpath=None): def start_process(self): if self._existing_service: - return self._existing_service + return None, self._existing_service else: if not shutil.which('java'): raise RuntimeError( diff --git a/sdks/python/apache_beam/utils/subprocess_server_test.py b/sdks/python/apache_beam/utils/subprocess_server_test.py index e0d0892c8e68..c0c8e5694b86 100644 --- a/sdks/python/apache_beam/utils/subprocess_server_test.py +++ b/sdks/python/apache_beam/utils/subprocess_server_test.py @@ -21,6 +21,7 @@ import glob import os +import random import re import shutil import socketserver @@ -166,5 +167,83 @@ def test_classpath_jar(self): os.chdir(oldwd) +class CacheTest(unittest.TestCase): + @staticmethod + def with_prefix(prefix): + return '%s-%s' % (prefix, random.random()) + + def test_memoization(self): + cache = subprocess_server._SharedCache(self.with_prefix, lambda x: None) + try: + token = cache.register() + a = cache.get('a') + self.assertEqual(a[0], 'a') + self.assertEqual(cache.get('a'), a) + b = cache.get('b') + self.assertEqual(b[0], 'b') + self.assertEqual(cache.get('b'), b) + finally: + cache.purge(token) + + def test_purged(self): + cache = subprocess_server._SharedCache(self.with_prefix, lambda x: None) + try: + token = cache.register() + a = cache.get('a') + self.assertEqual(cache.get('a'), a) + finally: + cache.purge(token) + + try: + token = cache.register() + new_a = cache.get('a') + self.assertNotEqual(new_a, a) + finally: + cache.purge(token) + + def test_multiple_owners(self): + cache = subprocess_server._SharedCache(self.with_prefix, lambda x: None) + try: + owner1 = cache.register() + a = cache.get('a') + try: + self.assertEqual(cache.get('a'), a) + owner2 = cache.register() + b = cache.get('b') + self.assertEqual(cache.get('b'), b) + finally: + cache.purge(owner2) + self.assertEqual(cache.get('a'), a) + self.assertEqual(cache.get('b'), b) + finally: + cache.purge(owner1) + + try: + owner3 = cache.register() + self.assertNotEqual(cache.get('a'), a) + self.assertNotEqual(cache.get('b'), b) + finally: + cache.purge(owner3) + + def test_interleaved_owners(self): + cache = subprocess_server._SharedCache(self.with_prefix, lambda x: None) + owner1 = cache.register() + a = cache.get('a') + self.assertEqual(cache.get('a'), a) + + owner2 = cache.register() + b = cache.get('b') + self.assertEqual(cache.get('b'), b) + + cache.purge(owner1) + self.assertNotEqual(cache.get('a'), a) + self.assertEqual(cache.get('b'), b) + + cache.purge(owner2) + owner3 = cache.register() + self.assertNotEqual(cache.get('b'), b) + cache.purge(owner3) + + if __name__ == '__main__': unittest.main() From e8cebdf4fe7e920b6494156aa3127037f23b58fe Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:33:13 -0700 Subject: [PATCH 006/224] Add an option to start loopback workers in expansion services. --- sdks/java/expansion-service/build.gradle | 3 ++ .../expansion/service/ExpansionService.java | 37 +++++++++++++++---- .../service/ExpansionServiceOptions.java | 5 +++ .../runners/portability/expansion_service.py | 14 +++++-- .../portability/expansion_service_main.py | 16 +++++++- 5 files changed, 62 insertions(+), 13 deletions(-) diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index 7bc77e3aea80..99c515cd4e63 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -36,11 +36,14 @@ test { dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") + implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":runners:core-construction-java") implementation project(path: ":runners:java-fn-execution") implementation project(path: ":sdks:java:fn-execution") + implementation project(path: ":sdks:java:harness") + permitUnusedDeclared project(path: ":model:fn-execution") permitUnusedDeclared project(path: ":sdks:java:fn-execution") implementation library.java.jackson_annotations implementation library.java.jackson_databind 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 c6509a5bbb4a..8e57ad706fed 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 @@ -36,6 +36,7 @@ import java.util.ServiceLoader; import java.util.Set; import java.util.stream.Collectors; +import org.apache.beam.fn.harness.ExternalWorkerService; import org.apache.beam.model.expansion.v1.ExpansionApi; import org.apache.beam.model.expansion.v1.ExpansionApi.DiscoverSchemaTransformRequest; import org.apache.beam.model.expansion.v1.ExpansionApi.DiscoverSchemaTransformResponse; @@ -514,6 +515,7 @@ default List getDependencies(RunnerApi.FunctionSpec spec, PipelineOption private @MonotonicNonNull Map registeredTransforms; private final PipelineOptions pipelineOptions; + private final @Nullable String loopbackAddress; public ExpansionService() { this(new String[] {}); @@ -524,7 +526,12 @@ public ExpansionService(String[] args) { } public ExpansionService(PipelineOptions opts) { + this(opts, null); + } + + public ExpansionService(PipelineOptions opts, @Nullable String loopbackAddress) { this.pipelineOptions = opts; + this.loopbackAddress = loopbackAddress; } private Map getRegisteredTransforms() { @@ -628,9 +635,19 @@ private Map loadRegisteredTransforms() { rehydratedComponents .getSdkComponents(request.getRequirementsList()) .withNewIdPrefix(request.getNamespace()); - sdkComponents.registerEnvironment( + RunnerApi.Environment defaultEnvironment = Environments.createOrGetDefaultEnvironment( - pipeline.getOptions().as(PortablePipelineOptions.class))); + pipeline.getOptions().as(PortablePipelineOptions.class)); + if (pipelineOptions.as(ExpansionServiceOptions.class).getAlsoStartLoopbackWorker()) { + PortablePipelineOptions externalOptions = + PipelineOptionsFactory.create().as(PortablePipelineOptions.class); + externalOptions.setDefaultEnvironmentType(Environments.ENVIRONMENT_EXTERNAL); + externalOptions.setDefaultEnvironmentConfig(loopbackAddress); + defaultEnvironment = + Environments.createAnyOfEnvironment( + defaultEnvironment, Environments.createOrGetDefaultEnvironment(externalOptions)); + } + sdkComponents.registerEnvironment(defaultEnvironment); Map outputMap = outputs.entrySet().stream() .collect( @@ -759,9 +776,12 @@ public static void main(String[] args) throws Exception { // Register the options class used by the expansion service. PipelineOptionsFactory.register(ExpansionServiceOptions.class); + @SuppressWarnings({"nullness"}) + PipelineOptions options = + PipelineOptionsFactory.fromArgs(Arrays.copyOfRange(args, 1, args.length)).create(); @SuppressWarnings("nullness") - ExpansionService service = new ExpansionService(Arrays.copyOfRange(args, 1, args.length)); + ExpansionService service = new ExpansionService(options, "localhost:" + port); StringBuilder registeredTransformsLog = new StringBuilder(); boolean registeredTransformsFound = false; @@ -794,11 +814,12 @@ public static void main(String[] args) throws Exception { System.out.println("\nDid not find any registered transforms or SchemaTransforms.\n"); } - Server server = - ServerBuilder.forPort(port) - .addService(service) - .addService(new ArtifactRetrievalService()) - .build(); + ServerBuilder serverBuilder = + ServerBuilder.forPort(port).addService(service).addService(new ArtifactRetrievalService()); + if (options.as(ExpansionServiceOptions.class).getAlsoStartLoopbackWorker()) { + serverBuilder.addService(new ExternalWorkerService(options)); + } + Server server = serverBuilder.build(); server.start(); server.awaitTermination(); } diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java index b3af511cd94f..6b569c7fce38 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java @@ -42,6 +42,11 @@ public interface ExpansionServiceOptions extends PipelineOptions { void setJavaClassLookupAllowlistFile(String file); + @Description("Whether to also start a loopback worker as part of this service.") + boolean getAlsoStartLoopbackWorker(); + + void setAlsoStartLoopbackWorker(boolean value); + @Description("Expansion service configuration file.") String getExpansionServiceConfigFile(); diff --git a/sdks/python/apache_beam/runners/portability/expansion_service.py b/sdks/python/apache_beam/runners/portability/expansion_service.py index 8be9d98508ed..50c793a0e8bd 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service.py @@ -22,6 +22,7 @@ import traceback from apache_beam import pipeline as beam_pipeline +from apache_beam.portability import common_urns from apache_beam.portability import python_urns from apache_beam.portability.api import beam_expansion_api_pb2 from apache_beam.portability.api import beam_expansion_api_pb2_grpc @@ -33,11 +34,18 @@ class ExpansionServiceServicer( beam_expansion_api_pb2_grpc.ExpansionServiceServicer): - def __init__(self, options=None): + def __init__(self, options=None, loopback_address=None): self._options = options or beam_pipeline.PipelineOptions( environment_type=python_urns.EMBEDDED_PYTHON, sdk_location='container') - self._default_environment = ( - environments.Environment.from_options(self._options)) + default_environment = (environments.Environment.from_options(self._options)) + if loopback_address: + loopback_environment = environments.Environment.from_options( + beam_pipeline.PipelineOptions( + environment_type=common_urns.environments.EXTERNAL.urn, + environment_config=loopback_address)) + default_environment = environments.AnyOfEnvironment( + [default_environment, loopback_environment]) + self._default_environment = default_environment def Expand(self, request, context=None): try: diff --git a/sdks/python/apache_beam/runners/portability/expansion_service_main.py b/sdks/python/apache_beam/runners/portability/expansion_service_main.py index 30cbdccb596e..307f6bd54182 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service_main.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service_main.py @@ -28,8 +28,10 @@ from apache_beam.options.pipeline_options import SetupOptions from apache_beam.portability.api import beam_artifact_api_pb2_grpc from apache_beam.portability.api import beam_expansion_api_pb2_grpc +from apache_beam.portability.api import beam_fn_api_pb2_grpc from apache_beam.runners.portability import artifact_service from apache_beam.runners.portability import expansion_service +from apache_beam.runners.worker import worker_pool_main from apache_beam.transforms import fully_qualified_named_transform from apache_beam.utils import thread_pool_executor @@ -41,6 +43,7 @@ def main(argv): parser.add_argument( '-p', '--port', type=int, help='port on which to serve the job api') parser.add_argument('--fully_qualified_name_glob', default=None) + parser.add_argument('--serve_loopback_worker', action='store_true') known_args, pipeline_args = parser.parse_known_args(argv) pipeline_options = PipelineOptions( pipeline_args + ["--experiments=beam_fn_api", "--sdk_location=container"]) @@ -52,14 +55,23 @@ def main(argv): with fully_qualified_named_transform.FullyQualifiedNamedTransform.with_filter( known_args.fully_qualified_name_glob): + address = '[::]:{}'.format(known_args.port) server = grpc.server(thread_pool_executor.shared_unbounded_instance()) + if known_args.serve_loopback_worker: + beam_fn_api_pb2_grpc.add_BeamFnExternalWorkerPoolServicer_to_server( + worker_pool_main.BeamFnExternalWorkerPoolServicer(), server) + loopback_address = address + else: + loopback_address = None beam_expansion_api_pb2_grpc.add_ExpansionServiceServicer_to_server( - expansion_service.ExpansionServiceServicer(pipeline_options), server) + expansion_service.ExpansionServiceServicer( + pipeline_options, loopback_address=loopback_address), + server) beam_artifact_api_pb2_grpc.add_ArtifactRetrievalServiceServicer_to_server( artifact_service.ArtifactRetrievalService( artifact_service.BeamFilesystemHandler(None).file_reader), server) - server.add_insecure_port('[::]:{}'.format(known_args.port)) + server.add_insecure_port(address) server.start() _LOGGER.info('Listening for expansion requests at %d', known_args.port) From e88fb5df88774ce570240413ffd09c51ea4bd97a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 2 Nov 2023 17:30:57 -0700 Subject: [PATCH 007/224] Use Java expansion services as workers by default in Python. Due to the AnyOf environment, remote runners can choose more expensive but remote-friendly options such as docker. --- CHANGES.md | 3 +++ sdks/python/apache_beam/transforms/external.py | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 9318e85d477b..4cf4cd1fd497 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -71,6 +71,9 @@ * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * The Python SDK now type checks `collections.abc.Collections` types properly. Some type hints that were erroneously allowed by the SDK may now fail. ([#29272](https://github.com/apache/beam/pull/29272)) +* Running multi-language pipelines locally no longer requires Docker. + Instead, the same (generally auto-started) subprocess used to perform the + expansion can also be used as the cross-language worker. ## Breaking Changes diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 52e27ecc2e8b..fc4ae3caa6df 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -980,7 +980,12 @@ def _default_args(self): to_stage = ','.join([self._path_to_jar] + sum(( JavaJarExpansionService._expand_jars(jar) for jar in self._classpath or []), [])) - return ['{{PORT}}', f'--filesToStage={to_stage}'] + args = ['{{PORT}}', f'--filesToStage={to_stage}'] + # TODO(robertwb): See if it's possible to scope this per pipeline. + # Checks to see if the cache is being used for this server. + if subprocess_server.SubprocessServer._cache._live_owners: + args.append('--alsoStartLoopbackWorker') + return args def __enter__(self): if self._service_count == 0: From d85ce3f38c531f1f880571b884c75cf958cbba60 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 3 Nov 2023 09:52:43 -0700 Subject: [PATCH 008/224] More complete documentation of environment proto definitions. --- .../model/pipeline/v1/beam_runner_api.proto | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index 87af7c19dd79..48f057bbc1ea 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -1563,15 +1563,26 @@ message Environment { message StandardEnvironments { enum Environments { - DOCKER = 0 [(beam_urn) = "beam:env:docker:v1"]; // A managed docker container to run user code. - - PROCESS = 1 [(beam_urn) = "beam:env:process:v1"]; // A managed native process to run user code. - - EXTERNAL = 2 [(beam_urn) = "beam:env:external:v1"]; // An external non managed process to run user code. - - DEFAULT = 3 [(beam_urn) = "beam:env:default:v1"]; // Used as a stub when context is missing a runner-provided default environment. - - ANYOF = 4 [(beam_urn) = "beam:env:anyof:v1"]; // A selection of equivalent environments a runner may use. + // A managed docker container to run user code. + // Payload should be DockerPayload. + DOCKER = 0 [(beam_urn) = "beam:env:docker:v1"]; + + // A managed native process to run user code. + // Payload should be ProcessPayload. + PROCESS = 1 [(beam_urn) = "beam:env:process:v1"]; + + // An external non managed process to run user code. + // Payload should be ExternalPayload. + EXTERNAL = 2 [(beam_urn) = "beam:env:external:v1"]; + + // Used as a stub when context is missing a runner-provided default environment. + DEFAULT = 3 [(beam_urn) = "beam:env:default:v1"]; + + // A selection of equivalent fully-specified environments a runner may use. + // Note that this environment itself does not declare any dependencies or capabilities, + // as those may differ among the several alternatives. + // Payload should be AnyOfEnvironmentPayload. + ANYOF = 4 [(beam_urn) = "beam:env:anyof:v1"]; } } @@ -1588,11 +1599,12 @@ message ProcessPayload { } message ExternalPayload { - ApiServiceDescriptor endpoint = 1; + ApiServiceDescriptor endpoint = 1; // Serving BeamFnExternalWorkerPool API. map params = 2; // Arbitrary extra parameters to pass } message AnyOfEnvironmentPayload { + // Each is fully contained (with their own dependencies, capabilities, etc.) repeated Environment environments = 1; } From 35a0b68c06b5e446d17c7c7081d2a7f13c85372c Mon Sep 17 00:00:00 2001 From: liferoad Date: Fri, 1 Dec 2023 09:08:21 -0500 Subject: [PATCH 009/224] Fix small doc issues (#29578) --- CHANGES.md | 2 +- sdks/python/apache_beam/io/gcp/bigquery.py | 18 ++--- .../www/site/content/en/blog/beam-2.52.0.md | 69 ++++++++++++++++++- 3 files changed, 78 insertions(+), 11 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 9318e85d477b..34a653d75ced 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -107,7 +107,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * Add `UseDataStreamForBatch` pipeline option to the Flink runner. When it is set to true, Flink runner will run batch jobs using the DataStream API. By default the option is set to false, so the batch jobs are still executed using the DataSet API. -* `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). +* `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621)). * state amd side input cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. (Python) ([#28770](https://github.com/apache/beam/issues/28770)). * Beam YAML stable release. Beam pipelines can now be written using YAML and leverage the Beam YAML framework which includes a preliminary set of IO's and turnkey transforms. More information can be found in the YAML root folder and in the [README](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md). diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 184138af7525..ac06425e95a9 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -72,7 +72,8 @@ or a table. Pipeline construction will fail with a validation error if neither or both are specified. -When reading via `ReadFromBigQuery`, bytes are returned decoded as bytes. +When reading via `ReadFromBigQuery` using `EXPORT`, +bytes are returned decoded as bytes. This is due to the fact that ReadFromBigQuery uses Avro exports by default. When reading from BigQuery using `apache_beam.io.BigQuerySource`, bytes are returned as base64-encoded bytes. To get base64-encoded bytes using @@ -2597,6 +2598,8 @@ def expand(self, input): class ReadFromBigQuery(PTransform): + # pylint: disable=line-too-long,W1401 + """Read data from BigQuery. This PTransform uses a BigQuery export job to take a snapshot of the table @@ -2653,8 +2656,7 @@ class ReadFromBigQuery(PTransform): :data:`None`, then the temp_location parameter is used. bigquery_job_labels (dict): A dictionary with string labels to be passed to BigQuery export and query jobs created by this transform. See: - https://cloud.google.com/bigquery/docs/reference/rest/v2/\ - Job#JobConfiguration + https://cloud.google.com/bigquery/docs/reference/rest/v2/Job#JobConfiguration use_json_exports (bool): By default, this transform works by exporting BigQuery data into Avro files, and reading those files. With this parameter, the transform will instead export to JSON files. JSON files @@ -2666,11 +2668,10 @@ class ReadFromBigQuery(PTransform): types (datetime.date, datetime.datetime, datetime.datetime, and datetime.datetime respectively). Avro exports are recommended. To learn more about BigQuery types, and Time-related type - representations, see: https://cloud.google.com/bigquery/docs/reference/\ - standard-sql/data-types + representations, + see: https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types To learn more about type conversions between BigQuery and Avro, see: - https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-avro\ - #avro_conversions + https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-avro\#avro_conversions temp_dataset (``apache_beam.io.gcp.internal.clients.bigquery.\ DatasetReference``): Temporary dataset reference to use when reading from BigQuery using a @@ -2690,8 +2691,7 @@ class ReadFromBigQuery(PTransform): (`PYTHON_DICT`). There is experimental support for producing a PCollection with a schema and yielding Beam Rows via the option `BEAM_ROW`. For more information on schemas, see - https://beam.apache.org/documentation/programming-guide/\ - #what-is-a-schema) + https://beam.apache.org/documentation/programming-guide/#what-is-a-schema) """ class Method(object): EXPORT = 'EXPORT' # This is currently the default. diff --git a/website/www/site/content/en/blog/beam-2.52.0.md b/website/www/site/content/en/blog/beam-2.52.0.md index 5654f16ceb37..2e604c8fabf8 100644 --- a/website/www/site/content/en/blog/beam-2.52.0.md +++ b/website/www/site/content/en/blog/beam-2.52.0.md @@ -41,7 +41,7 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * Add `UseDataStreamForBatch` pipeline option to the Flink runner. When it is set to true, Flink runner will run batch jobs using the DataStream API. By default the option is set to false, so the batch jobs are still executed using the DataSet API. -* `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621). +* `upload_graph` as one of the Experiments options for DataflowRunner is no longer required when the graph is larger than 10MB for Java SDK ([PR#28621](https://github.com/apache/beam/pull/28621)). * state amd side input cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. (Python) ([#28770](https://github.com/apache/beam/issues/28770)). * Beam YAML stable release. Beam pipelines can now be written using YAML and leverage the Beam YAML framework which includes a preliminary set of IO's and turnkey transforms. More information can be found in the YAML root folder and in the [README](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md). @@ -69,69 +69,136 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a According to git shortlog, the following people contributed to the 2.52.0 release. Thank you to all contributors! Ahmed Abualsaud + Ahmet Altay + Aleksandr Dudko + Alexey Romanenko + Anand Inguva + Andrei Gurau + Andrey Devyatkin + BjornPrime + Bruno Volpato + Bulat + Chamikara Jayalath + Damon + Danny McCormick + Devansh Modi + Dominik Dębowczyk + Ferran Fernández Garrido + Hai Joey Tran + Israel Herraiz + Jack McCluskey + Jan Lukavský + JayajP + Jeff Kinard + Jeffrey Kinard + Jiangjie Qin + Jing + Joar Wandborg + Johanna Öjeling + Julien Tournay + Kanishk Karanawat + Kenneth Knowles + Kerry Donny-Clark + Luís Bianchin + Minbo Bae + Pranav Bhandari + Rebecca Szper + Reuven Lax + Ritesh Ghorse + Robert Bradshaw + Robert Burke + RyuSA + Shunping Huang + Steven van Rossum + Svetak Sundhar + Tony Tang + Vitaly Terentyev + Vivek Sumanth + Vlado Djerek + Yi Hu + aku019 + brucearctor + caneff + damccorm + ddebowczyk92 + dependabot[bot] + dpcollins-google + edman124 + gabry.wu + illoise + johnjcasey + jonathan-lemos + kennknowles + liferoad + magicgoody + martin trieu + nancyxu123 + pablo rodriguez defino + tvalentyn + From 7c99c33035038a91d270f1f89f13af6ea7b62e6b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 1 Dec 2023 11:00:28 -0500 Subject: [PATCH 010/224] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#29575) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.16.6 to 1.16.8. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.16.6...v1.16.8) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 22 +++++++++++----------- sdks/go.sum | 44 ++++++++++++++++++++++---------------------- 2 files changed, 33 insertions(+), 33 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d64ee5033383..3223814d86b6 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.51.0 cloud.google.com/go/storage v1.35.1 - github.com/aws/aws-sdk-go-v2 v1.23.2 + github.com/aws/aws-sdk-go-v2 v1.23.4 github.com/aws/aws-sdk-go-v2/config v1.25.8 - github.com/aws/aws-sdk-go-v2/credentials v1.16.6 + github.com/aws/aws-sdk-go-v2/credentials v1.16.8 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 - github.com/aws/smithy-go v1.17.0 + github.com/aws/smithy-go v1.18.1 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 @@ -106,18 +106,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.6 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.5 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.5 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.17.5 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.20.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.25.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.1 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 45f8971db12e..b9914ab1b931 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,39 +81,39 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.23.2 h1:UoTll1Y5b88x8h53OlsJGgOHwpggdMr7UVnLjMb3XYg= -github.com/aws/aws-sdk-go-v2 v1.23.2/go.mod h1:i1XDttT4rnf6vxc9AuskLc6s7XBee8rlLilKlc03uAA= +github.com/aws/aws-sdk-go-v2 v1.23.4 h1:2P20ZjH0ouSAu/6yZep8oCmTReathLuEu6dwoqEgjts= +github.com/aws/aws-sdk-go-v2 v1.23.4/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= github.com/aws/aws-sdk-go-v2/config v1.25.8 h1:CHr7PIzyfevjNiqL9rU6xoqHZKCO2ldY6LmvRDfpRuI= github.com/aws/aws-sdk-go-v2/config v1.25.8/go.mod h1:zefIy117FDPOVU0xSOFG8mx9kJunuVopzI639tjYXc0= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.6 h1:TimIpn1p4v44i0sJMKsnpby1P9sP1ByKLsdm7bvOmwM= -github.com/aws/aws-sdk-go-v2/credentials v1.16.6/go.mod h1:+CLPlYf9FQLeXD8etOYiZxpLQqc3GL4EikxjkFFp1KA= +github.com/aws/aws-sdk-go-v2/credentials v1.16.8 h1:phw9nRLy/77bPk6Mfu2SHCOnHwfVB7WWrOa5rZIY2Fc= +github.com/aws/aws-sdk-go-v2/credentials v1.16.8/go.mod h1:MrS4SOin6adbO6wgWhdifyPiq+TX7fPPwyA/ZLC1F5M= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.6 h1:pPs23/JLSOlwnmSRNkdbt3upmBeF6QL/3MHEb6KzTyo= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.6/go.mod h1:jsoDHV44SxWv00wlbx0yA5M7n5rmE5rGk+OGA0suXSw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8 h1:tQZLSPC2Zj2CqZHonLmWEvCsbpMX5tQvaYJWHadcPek= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8/go.mod h1:5+YpvTHDFffykWr5qAGjqwoh8oVYZOddL3sSrEN7lws= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.5 h1:16Z1XuMUv63fcyW5bIUno6AFcX4drsrE0gof+xue6g4= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.5/go.mod h1:pRvFacV2qbRKy34ZFptHZW4wpauJA445bqFbvA6ikSo= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.5 h1:RxpMuBgzP3Dj1n5CZY6droLFcsn5gc7QsrIcaGQoeCs= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.5/go.mod h1:dO8Js7ym4Jzg/wcjTgCRVln/jFn3nI82XNhsG2lWbDI= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7 h1:eMqD7ku6WGdmcWWXPYun9m6yk6feSULLhJlAtN6rYG4= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7/go.mod h1:0oBIfcDV6LScxEW0VgOqxT3e4aqKRp+SYhB9wAd5E3Q= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7 h1:+XYhWhgWs5F3Zx8oa49CXzNvfXrItaDjZB/M172fcHQ= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7/go.mod h1:L6tcSRyCGxcKfDWUrmv2jv8G1cLDU7d0FUpEFpG9bVE= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.1 h1:rpkF4n0CyFcrJUG/rNNohoTmhtWlFTRI4BsZOh9PvLs= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.1/go.mod h1:l9ymW25HOqymeU2m1gbUQ3rUIsTwKs8gYHXkqDQUhiI= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeEaev/ZF01cQyNZgmYE9oYYePIMJs2mWSKG514= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.5 h1:F+XafeiK7Uf4YwTZfe/JLt+3cB6je9sI7l0TY4f2CkY= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.5/go.mod h1:NlZuvlkyu6l/F3+qIBsGGtYLL2Z71tCf5NFoNAaG1NY= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7 h1:dU+ZyhvqMB/T/TxjGagHMCdyUiqaThRIaMu3YvKiSQI= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7/go.mod h1:SGORuNqoXyWfTvTp/gBGJfv8jRvW/+nha0XhnIXVI+o= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -121,16 +121,16 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.17.5 h1:kuK22ZsITfzaZEkxEl5H/lhy2k3G4clBtcQBI93RbIc= -github.com/aws/aws-sdk-go-v2/service/sso v1.17.5/go.mod h1:/tLqstwPfJLHYGBB5/c8P1ITI82pcGs7cJQuXku2pOg= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.20.3 h1:l5d5nrTFMhiUWNoLnV7QNI4m42/3WVSXqSyqVy+elGk= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.20.3/go.mod h1:30gKZp2pHQJq3yTmVy+hJKDFynSoYzVqYaxe4yPi+xI= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.1 h1:V40g2daNO3l1J94JYwqfkyvQMYXi5I25fs3fNQW8iDs= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.1/go.mod h1:0ZWQJP/mBOUxkCvZKybZNz1XmdUKSBxoF0dzgfxtvDs= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1 h1:uQrj7SpUNC3r55vc1CDh3qV9wJC66lz546xM9dhSo5s= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1/go.mod h1:oyaTk5xEAOuPXX1kCD7HmIeuLqdj3Bk5yGkqGXtGi14= 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.25.6 h1:39dJNBt35p8dFSnQdoy+QbDaPenTxFqqDQFOb1GDYpE= -github.com/aws/aws-sdk-go-v2/service/sts v1.25.6/go.mod h1:6DKEi+8OnUrqEEh6OCam16AYQHWAOyNgRiUGnHoh7Cg= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.1 h1:K33V7L0XDdb23FMOZySr8bon1jou5SHn1fiv7NJ1SUg= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.1/go.mod h1:YtXUl/sfnS06VksYhr855hTQf2HphfT1Xv/EwuzbPjg= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.17.0 h1:wWJD7LX6PBV6etBUwO0zElG0nWN9rUhp0WdYeHSHAaI= -github.com/aws/smithy-go v1.17.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= +github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= +github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= From 5f6afcef55f8a117c0518e3aebacd5c94ca05b52 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 1 Dec 2023 11:01:08 -0500 Subject: [PATCH 011/224] Bump cryptography from 41.0.4 to 41.0.6 in /sdks/python/container/py38 (#29554) Bumps [cryptography](https://github.com/pyca/cryptography) from 41.0.4 to 41.0.6. - [Changelog](https://github.com/pyca/cryptography/blob/main/CHANGELOG.rst) - [Commits](https://github.com/pyca/cryptography/compare/41.0.4...41.0.6) --- updated-dependencies: - dependency-name: cryptography 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 5dffff5f80d9..83580f65f2d4 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -33,7 +33,7 @@ charset-normalizer==3.3.0 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.4 +cryptography==41.0.6 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 From 26135b074732338c7c1926e21bd394bf5e0ae5c0 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Fri, 1 Dec 2023 10:05:42 -0800 Subject: [PATCH 012/224] Support for upgrading BigQuery transforms without upgrading the pipeline (#29550) * Support for upgrading BigQuery transforms without upgrading the pipeline * Fix spotless * Addressing reviewer comments * Cleanup --- .../core/construction/TransformUpgrader.java | 52 +- .../expansion/service/ExpansionService.java | 18 + .../gcp/bigquery/BigQueryIOTranslation.java | 782 ++++++++++++++++++ .../bigquery/BigQueryIOTranslationTest.java | 303 +++++++ .../io/kafka/upgrade/KafkaIOTranslation.java | 26 +- 5 files changed, 1155 insertions(+), 26 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index 93fe4ddbfaf6..e6dce752d06a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -17,7 +17,11 @@ */ package org.apache.beam.runners.core.construction; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.net.ServerSocket; import java.util.ArrayList; import java.util.Collection; @@ -104,6 +108,15 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( }) .collect(Collectors.toList()); + if (!urnsToOverride.isEmpty() && transformsToOverride.isEmpty()) { + throw new IllegalArgumentException( + "A list of URNs for overriding transforms was provided but the pipeline did not contain " + + "any matching transforms. Either make sure to include at least one matching " + + "transform in the pipeline or avoid setting the 'transformsToOverride' " + + "PipelineOption. Provided list of URNs: " + + urnsToOverride); + } + String serviceAddress; TransformServiceLauncher service = null; @@ -232,11 +245,11 @@ RunnerApi.Pipeline updateTransformViaTransformService( expandedTransform.getOutputsMap().values().iterator().next()); } else { for (Map.Entry entry : transformToUpgrade.getOutputsMap().entrySet()) { - if (expandedTransform.getOutputsMap().keySet().contains(entry.getKey())) { + if (!expandedTransform.getOutputsMap().keySet().contains(entry.getKey())) { throw new IllegalArgumentException( - "Original transform did not have an output with tag " + "Original transform had an output with tag " + entry.getKey() - + " but upgraded transform did."); + + " but upgraded transform did not."); } String newOutput = expandedTransform.getOutputsMap().get(entry.getKey()); if (newOutput == null) { @@ -367,4 +380,37 @@ public void close() throws Exception { return null; } + + /** + * A utility method that converts an arbitrary serializable object into a byte array. + * + * @param object an instance of type {@code Serializable} + * @return {@code object} converted into a byte array. + */ + public static byte[] toByteArray(Object object) { + try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bos)) { + out.writeObject(object); + return bos.toByteArray(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * A utility method that converts a byte array obtained by invoking {@link #toByteArray(Object)} + * back to a Java object. + * + * @param bytes a {@code byte} array generated by invoking the {@link #toByteArray(Object)} + * method. + * @return re-generated object. + */ + public static Object fromByteArray(byte[] bytes) { + try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes); + ObjectInputStream in = new ObjectInputStream(bis)) { + return in.readObject(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } 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 8e57ad706fed..ba60fb99d4d1 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 @@ -30,6 +30,7 @@ import java.lang.reflect.Method; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -82,6 +83,7 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; 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; @@ -457,6 +459,22 @@ default Map> extractOutputs(OutputT output) { i++; } return indexToPCollection.build(); + } else if (output instanceof POutput) { + // This is needed to support custom output types. + Map, PValue> values = output.expand(); + Map> returnMap = new HashMap<>(); + for (Map.Entry, PValue> entry : values.entrySet()) { + if (!(entry.getValue() instanceof PCollection)) { + throw new UnsupportedOperationException( + "Unable to parse the output type " + + output.getClass() + + " due to key " + + entry.getKey() + + " not mapping to a PCollection"); + } + returnMap.put(entry.getKey().getId(), (PCollection) entry.getValue()); + } + return returnMap; } else { throw new UnsupportedOperationException("Unknown output type: " + output.getClass()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java new file mode 100644 index 000000000000..a0c3bdbece51 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -0,0 +1,782 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static org.apache.beam.runners.core.construction.TransformUpgrader.fromByteArray; +import static org.apache.beam.runners.core.construction.TransformUpgrader.toByteArray; + +import com.google.api.services.bigquery.model.Clustering; +import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.service.AutoService; +import com.google.cloud.bigquery.storage.v1.AppendRowsRequest.MissingValueInterpretation; +import com.google.cloud.bigquery.storage.v1.DataFormat; +import java.io.IOException; +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.runners.core.construction.SdkComponents; +import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.FromBeamRowFunction; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.QueryPriority; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.ToBeamRowFunction; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.SchemaUpdateOption; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.RowWriterFactory.AvroRowWriterFactory; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.logicaltypes.NanosDuration; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({"rawtypes", "nullness"}) +public class BigQueryIOTranslation { + + static class BigQueryIOReadTranslator implements TransformPayloadTranslator> { + + static Schema schema = + Schema.builder() + .addNullableStringField("json_table_ref") + .addNullableStringField("query") + .addNullableBooleanField("validate") + .addNullableBooleanField("flatten_results") + .addNullableBooleanField("use_legacy_sql") + .addNullableBooleanField("with_template_compatibility") + .addNullableByteArrayField("bigquery_services") + .addNullableByteArrayField("parse_fn") + .addNullableByteArrayField("datum_reader_factory") + .addNullableByteArrayField("query_priority") + .addNullableStringField("query_location") + .addNullableStringField("query_temp_dataset") + .addNullableByteArrayField("method") + .addNullableByteArrayField("format") + .addNullableArrayField("selected_fields", FieldType.STRING) + .addNullableStringField("row_restriction") + .addNullableByteArrayField("coder") + .addNullableStringField("kms_key") + .addNullableByteArrayField("type_descriptor") + .addNullableByteArrayField("to_beam_row_fn") + .addNullableStringField("from_beam_row_fn") + .addNullableBooleanField("use_avro_logical_types") + .addNullableBooleanField("projection_pushdown_applied") + .build(); + + public static final String BIGQUERY_READ_TRANSFORM_URN = + "beam:transform:org.apache.beam:bigquery_read:v1"; + + @Override + public String getUrn() { + return BIGQUERY_READ_TRANSFORM_URN; + } + + @Override + public RunnerApi.@Nullable FunctionSpec translate( + AppliedPTransform> application, SdkComponents components) + throws IOException { + // Setting an empty payload since BigQuery transform payload is not actually used by runners + // currently. + // This can be implemented if runners started actually using the BigQuery transform payload. + return FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build(); + } + + @Override + public Row toConfigRow(TypedRead transform) { + Map fieldValues = new HashMap<>(); + + if (transform.getJsonTableRef() != null) { + fieldValues.put("json_table_ref", transform.getJsonTableRef().get()); + } + if (transform.getQuery() != null) { + fieldValues.put("query", transform.getQuery().get()); + } + fieldValues.put("validate", transform.getValidate()); + fieldValues.put("flatten_results", transform.getFlattenResults()); + fieldValues.put("use_legacy_sql", transform.getUseLegacySql()); + fieldValues.put("with_template_compatibility", transform.getWithTemplateCompatibility()); + + if (transform.getBigQueryServices() != null) { + fieldValues.put("bigquery_services", toByteArray(transform.getBigQueryServices())); + } + if (transform.getParseFn() != null) { + fieldValues.put("parse_fn", toByteArray(transform.getParseFn())); + } + if (transform.getDatumReaderFactory() != null) { + fieldValues.put("datum_reader_factory", toByteArray(transform.getDatumReaderFactory())); + } + if (transform.getQueryPriority() != null) { + fieldValues.put("query_priority", toByteArray(transform.getQueryPriority())); + } + if (transform.getQueryLocation() != null) { + fieldValues.put("query_location", transform.getQueryLocation()); + } + if (transform.getQueryTempDataset() != null) { + fieldValues.put("query_temp_dataset", transform.getQueryTempDataset()); + } + if (transform.getMethod() != null) { + fieldValues.put("method", toByteArray(transform.getMethod())); + } + if (transform.getFormat() != null) { + fieldValues.put("format", toByteArray(transform.getFormat())); + } + if (transform.getSelectedFields() != null && !transform.getSelectedFields().get().isEmpty()) { + fieldValues.put("selected_fields", transform.getSelectedFields().get()); + } + if (transform.getRowRestriction() != null) { + fieldValues.put("row_restriction", transform.getRowRestriction().get()); + } + if (transform.getCoder() != null) { + fieldValues.put("coder", toByteArray(transform.getCoder())); + } + if (transform.getKmsKey() != null) { + fieldValues.put("kms_key", transform.getKmsKey()); + } + if (transform.getTypeDescriptor() != null) { + fieldValues.put("type_descriptor", toByteArray(transform.getTypeDescriptor())); + } + if (transform.getToBeamRowFn() != null) { + fieldValues.put("to_beam_row_fn", toByteArray(transform.getToBeamRowFn())); + } + if (transform.getFromBeamRowFn() != null) { + fieldValues.put("from_beam_row_fn", toByteArray(transform.getFromBeamRowFn())); + } + if (transform.getUseAvroLogicalTypes() != null) { + fieldValues.put("use_avro_logical_types", transform.getUseAvroLogicalTypes()); + } + fieldValues.put("projection_pushdown_applied", transform.getProjectionPushdownApplied()); + + return Row.withSchema(schema).withFieldValues(fieldValues).build(); + } + + @Override + public TypedRead fromConfigRow(Row configRow) { + BigQueryIO.TypedRead.Builder builder = new AutoValue_BigQueryIO_TypedRead.Builder<>(); + + String jsonTableRef = configRow.getString("json_table_ref"); + if (jsonTableRef != null) { + builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); + } + String query = configRow.getString("query"); + if (query != null) { + builder = builder.setQuery(StaticValueProvider.of(query)); + } + Boolean validate = configRow.getBoolean("validate"); + if (validate != null) { + builder = builder.setValidate(validate); + } + Boolean flattenResults = configRow.getBoolean("flatten_results"); + if (flattenResults != null) { + builder = builder.setFlattenResults(flattenResults); + } + Boolean useLegacySQL = configRow.getBoolean("use_legacy_sql"); + if (useLegacySQL != null) { + builder = builder.setUseLegacySql(useLegacySQL); + } + Boolean withTemplateCompatibility = configRow.getBoolean("with_template_compatibility"); + if (withTemplateCompatibility != null) { + builder = builder.setWithTemplateCompatibility(withTemplateCompatibility); + } + byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); + if (bigqueryServicesBytes != null) { + builder = + builder.setBigQueryServices((BigQueryServices) fromByteArray(bigqueryServicesBytes)); + } + byte[] parseFnBytes = configRow.getBytes("parse_fn"); + if (parseFnBytes != null) { + builder = builder.setParseFn((SerializableFunction) fromByteArray(parseFnBytes)); + } + byte[] datumReaderFactoryBytes = configRow.getBytes("datum_reader_factory"); + if (datumReaderFactoryBytes != null) { + builder = + builder.setDatumReaderFactory( + (SerializableFunction) fromByteArray(datumReaderFactoryBytes)); + } + byte[] queryPriorityBytes = configRow.getBytes("query_priority"); + if (queryPriorityBytes != null) { + builder = builder.setQueryPriority((QueryPriority) fromByteArray(queryPriorityBytes)); + } + String queryLocation = configRow.getString("query_location"); + if (queryLocation != null) { + builder = builder.setQueryLocation(queryLocation); + } + String queryTempDataset = configRow.getString("query_temp_dataset"); + if (queryTempDataset != null) { + builder = builder.setQueryTempDataset(queryTempDataset); + } + byte[] methodBytes = configRow.getBytes("method"); + if (methodBytes != null) { + builder = builder.setMethod((TypedRead.Method) fromByteArray(methodBytes)); + } + byte[] formatBytes = configRow.getBytes("format"); + if (methodBytes != null) { + builder = builder.setFormat((DataFormat) fromByteArray(formatBytes)); + } + Collection selectedFields = configRow.getArray("selected_fields"); + if (selectedFields != null && !selectedFields.isEmpty()) { + builder.setSelectedFields(StaticValueProvider.of(ImmutableList.of(selectedFields))); + } + String rowRestriction = configRow.getString("row_restriction"); + if (rowRestriction != null) { + builder = builder.setRowRestriction(StaticValueProvider.of(rowRestriction)); + } + byte[] coderBytes = configRow.getBytes("coder"); + if (coderBytes != null) { + builder = builder.setCoder((Coder) fromByteArray(coderBytes)); + } + String kmsKey = configRow.getString("kms_key"); + if (kmsKey != null) { + builder = builder.setKmsKey(kmsKey); + } + byte[] typeDescriptorBytes = configRow.getBytes("type_descriptor"); + if (typeDescriptorBytes != null) { + builder = builder.setTypeDescriptor((TypeDescriptor) fromByteArray(typeDescriptorBytes)); + } + byte[] toBeamRowFnBytes = configRow.getBytes("to_beam_row_fn"); + if (toBeamRowFnBytes != null) { + builder = builder.setToBeamRowFn((ToBeamRowFunction) fromByteArray(toBeamRowFnBytes)); + } + byte[] fromBeamRowFnBytes = configRow.getBytes("from_beam_row_fn"); + if (fromBeamRowFnBytes != null) { + builder = builder.setFromBeamRowFn((FromBeamRowFunction) fromByteArray(fromBeamRowFnBytes)); + } + Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); + if (useAvroLogicalTypes != null) { + builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); + } + Boolean projectionPushdownApplied = configRow.getBoolean("projection_pushdown_applied"); + if (projectionPushdownApplied != null) { + builder = builder.setProjectionPushdownApplied(projectionPushdownApplied); + } + + return builder.build(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar { + + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(AutoValue_BigQueryIO_TypedRead.class, new BigQueryIOReadTranslator()) + .build(); + } + } + + static class BigQueryIOWriteTranslator implements TransformPayloadTranslator> { + + static Schema schema = + Schema.builder() + .addNullableStringField("json_table_ref") + .addNullableByteArrayField("table_function") + .addNullableByteArrayField("format_function") + .addNullableByteArrayField("format_record_on_failure_function") + .addNullableByteArrayField("avro_row_writer_factory") + .addNullableByteArrayField("avro_schema_factory") + .addNullableBooleanField("use_avro_logical_types") + .addNullableByteArrayField("dynamic_destinations") + .addNullableStringField("json_schema") + .addNullableStringField("json_time_partitioning") + .addNullableByteArrayField("clustering") + .addNullableByteArrayField("create_disposition") + .addNullableByteArrayField("write_disposition") + .addNullableArrayField("schema_update_options", FieldType.BYTES) + .addNullableStringField("table_description") + .addNullableBooleanField("validate") + .addNullableByteArrayField("bigquery_services") + .addNullableInt32Field("max_files_per_bundle") + .addNullableInt64Field("max_file_size") + .addNullableInt32Field("num_file_shards") + .addNullableInt32Field("num_storage_write_api_streams") + .addNullableBooleanField("propagate_successful_storage_api_writes") + .addNullableInt32Field("max_files_per_partition") + .addNullableInt64Field("max_bytes_per_partition") + .addNullableLogicalTypeField("triggerring_frequency", new NanosDuration()) + .addNullableByteArrayField("method") + .addNullableStringField("load_job_project_id") + .addNullableByteArrayField("failed_insert_retry_policy") + .addNullableStringField("custom_gcs_temp_location") + .addNullableBooleanField("extended_error_info") + .addNullableBooleanField("skip_invalid_rows") + .addNullableBooleanField("ignore_unknown_values") + .addNullableBooleanField("ignore_insert_ids") + .addNullableInt32Field("max_retry_jobs") + .addNullableStringField("kms_key") + .addNullableArrayField("primary_key", FieldType.STRING) + .addNullableByteArrayField("default_missing_value_interpretation") + .addNullableBooleanField("optimize_writes") + .addNullableBooleanField("use_beam_schema") + .addNullableBooleanField("auto_sharding") + .addNullableBooleanField("propagate_successful") + .addNullableBooleanField("auto_schema_update") + .addNullableByteArrayField("write_protos_class") + .addNullableBooleanField("direct_write_protos") + .addNullableByteArrayField("deterministic_record_id_fn") + .addNullableStringField("write_temp_dataset") + .addNullableByteArrayField("row_mutation_information_fn") + .build(); + + public static final String BIGQUERY_WRITE_TRANSFORM_URN = + "beam:transform:org.apache.beam:bigquery_write:v1"; + + @Override + public String getUrn() { + return BIGQUERY_WRITE_TRANSFORM_URN; + } + + @Override + public @Nullable FunctionSpec translate( + AppliedPTransform> application, SdkComponents components) + throws IOException { + // Setting an empty payload since BigQuery transform payload is not actually used by runners + // currently. + // This can be implemented if runners started actually using the Kafka BigQuery payload. + return FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build(); + } + + @Override + public Row toConfigRow(Write transform) { + + Map fieldValues = new HashMap<>(); + + if (transform.getJsonTableRef() != null) { + fieldValues.put("json_table_ref", transform.getJsonTableRef().get()); + } + if (transform.getTableFunction() != null) { + fieldValues.put("table_function", toByteArray(transform.getTableFunction())); + } + if (transform.getFormatFunction() != null) { + fieldValues.put("format_function", toByteArray(transform.getFormatFunction())); + } + if (transform.getFormatRecordOnFailureFunction() != null) { + fieldValues.put( + "format_record_on_failure_function", + toByteArray(transform.getFormatRecordOnFailureFunction())); + } + if (transform.getAvroRowWriterFactory() != null) { + fieldValues.put( + "avro_row_writer_factory", toByteArray(transform.getAvroRowWriterFactory())); + } + fieldValues.put("use_avro_logical_types", transform.getUseAvroLogicalTypes()); + if (transform.getDynamicDestinations() != null) { + fieldValues.put("dynamic_destinations", toByteArray(transform.getDynamicDestinations())); + } + if (transform.getSchemaFromView() != null) { + // Property 'getSchemaFromView' cannot be used in a portable way across pipelines since it + // is bound to PCollections generated for the current pipeline instance. + throw new IllegalArgumentException( + "BigQueryIO.Write transforms cannot be converted to a " + + "portable row based config due to 'withSchemaFromView' property being set. Please " + + "retry without setting this property when configuring your transform"); + } + if (transform.getJsonSchema() != null) { + fieldValues.put("json_schema", transform.getJsonSchema().get()); + } + if (transform.getJsonTimePartitioning() != null) { + fieldValues.put( + "json_time_partitioning", toByteArray(transform.getJsonTimePartitioning().get())); + } + if (transform.getClustering() != null) { + fieldValues.put("clustering", toByteArray(transform.getClustering())); + } + if (transform.getCreateDisposition() != null) { + fieldValues.put("create_disposition", toByteArray(transform.getCreateDisposition())); + } + if (transform.getWriteDisposition() != null) { + fieldValues.put("write_disposition", toByteArray(transform.getWriteDisposition())); + } + if (transform.getSchemaUpdateOptions() != null + && !transform.getSchemaUpdateOptions().isEmpty()) { + List schemUpdateOptionsData = + transform.getSchemaUpdateOptions().stream() + .map(option -> toByteArray(option)) + .collect(Collectors.toList()); + fieldValues.put("schema_update_options", schemUpdateOptionsData); + } + if (transform.getTableDescription() != null) { + fieldValues.put("table_description", transform.getTableDescription()); + } + fieldValues.put("validate", transform.getValidate()); + if (transform.getBigQueryServices() != null) { + fieldValues.put("bigquery_services", toByteArray(transform.getBigQueryServices())); + } + if (transform.getMaxFilesPerBundle() != null) { + fieldValues.put("max_files_per_bundle", transform.getMaxFilesPerBundle()); + } + if (transform.getMaxFileSize() != null) { + fieldValues.put("max_file_size", transform.getMaxFileSize()); + } + fieldValues.put("num_file_shards", transform.getNumFileShards()); + fieldValues.put("num_storage_write_api_streams", transform.getNumStorageWriteApiStreams()); + fieldValues.put( + "propagate_successful_storage_api_writes", + transform.getPropagateSuccessfulStorageApiWrites()); + fieldValues.put("max_files_per_partition", transform.getMaxFilesPerPartition()); + fieldValues.put("max_bytes_per_partition", transform.getMaxBytesPerPartition()); + if (transform.getTriggeringFrequency() != null) { + fieldValues.put( + "triggerring_frequency", + Duration.ofMillis(transform.getTriggeringFrequency().getMillis())); + } + if (transform.getMethod() != null) { + fieldValues.put("method", toByteArray(transform.getMethod())); + } + if (transform.getLoadJobProjectId() != null) { + fieldValues.put("load_job_project_id", transform.getLoadJobProjectId()); + } + if (transform.getFailedInsertRetryPolicy() != null) { + fieldValues.put( + "failed_insert_retry_policy", toByteArray(transform.getFailedInsertRetryPolicy())); + } + if (transform.getCustomGcsTempLocation() != null) { + fieldValues.put("custom_gcs_temp_location", transform.getCustomGcsTempLocation().get()); + } + fieldValues.put("extended_error_info", transform.getExtendedErrorInfo()); + if (transform.getSkipInvalidRows() != null) { + fieldValues.put("skip_invalid_rows", transform.getSkipInvalidRows()); + } + if (transform.getIgnoreUnknownValues() != null) { + fieldValues.put("ignore_unknown_values", transform.getIgnoreUnknownValues()); + } + if (transform.getIgnoreInsertIds() != null) { + fieldValues.put("ignore_insert_ids", transform.getIgnoreInsertIds()); + } + fieldValues.put("max_retry_jobs", transform.getMaxRetryJobs()); + if (transform.getPropagateSuccessful() != null) { + fieldValues.put("propagate_successful", transform.getPropagateSuccessful()); + } + if (transform.getKmsKey() != null) { + fieldValues.put("kms_key", transform.getKmsKey()); + } + if (transform.getPrimaryKey() != null) { + fieldValues.put("primary_key", transform.getPrimaryKey()); + } + if (transform.getDefaultMissingValueInterpretation() != null) { + fieldValues.put( + "default_missing_value_interpretation", + toByteArray(transform.getDefaultMissingValueInterpretation())); + } + if (transform.getOptimizeWrites() != null) { + fieldValues.put("optimize_writes", transform.getOptimizeWrites()); + } + if (transform.getUseBeamSchema() != null) { + fieldValues.put("use_beam_schema", transform.getUseBeamSchema()); + } + if (transform.getAutoSharding() != null) { + fieldValues.put("auto_sharding", transform.getAutoSharding()); + } + if (transform.getAutoSchemaUpdate() != null) { + fieldValues.put("auto_schema_update", transform.getAutoSchemaUpdate()); + } + if (transform.getWriteProtosClass() != null) { + fieldValues.put("write_protos_class", toByteArray(transform.getWriteProtosClass())); + } + if (transform.getDirectWriteProtos() != null) { + fieldValues.put("direct_write_protos", transform.getDirectWriteProtos()); + } + if (transform.getDeterministicRecordIdFn() != null) { + fieldValues.put( + "deterministic_record_id_fn", toByteArray(transform.getDeterministicRecordIdFn())); + } + if (transform.getWriteTempDataset() != null) { + fieldValues.put("write_temp_dataset", toByteArray(transform.getDeterministicRecordIdFn())); + } + if (transform.getRowMutationInformationFn() != null) { + fieldValues.put( + "row_mutation_information_fn", toByteArray(transform.getRowMutationInformationFn())); + } + + return Row.withSchema(schema).withFieldValues(fieldValues).build(); + } + + @Override + public Write fromConfigRow(Row configRow) { + BigQueryIO.Write.Builder builder = new AutoValue_BigQueryIO_Write.Builder<>(); + + String jsonTableRef = configRow.getString("json_table_ref"); + if (jsonTableRef != null) { + builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); + } + byte[] tableFunctionBytes = configRow.getBytes("table_function"); + if (tableFunctionBytes != null) { + builder = + builder.setTableFunction( + (SerializableFunction) + fromByteArray(tableFunctionBytes)); + } + byte[] formatFunctionBytes = configRow.getBytes("format_function"); + if (formatFunctionBytes != null) { + builder = + builder.setFormatFunction( + (SerializableFunction) fromByteArray(formatFunctionBytes)); + } + byte[] formatRecordOnFailureFunctionBytes = + configRow.getBytes("format_record_on_failure_function"); + if (tableFunctionBytes != null) { + builder = + builder.setFormatRecordOnFailureFunction( + (SerializableFunction) + fromByteArray(formatRecordOnFailureFunctionBytes)); + } + byte[] avroRowWriterFactoryBytes = configRow.getBytes("avro_row_writer_factory"); + if (avroRowWriterFactoryBytes != null) { + builder = + builder.setAvroRowWriterFactory( + (AvroRowWriterFactory) fromByteArray(avroRowWriterFactoryBytes)); + } + byte[] avroSchemaFactoryBytes = configRow.getBytes("avro_schema_factory"); + if (tableFunctionBytes != null) { + builder = + builder.setAvroSchemaFactory( + (SerializableFunction) fromByteArray(avroSchemaFactoryBytes)); + } + Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); + if (useAvroLogicalTypes != null) { + builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); + } + byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); + if (dynamicDestinationsBytes != null) { + builder = + builder.setDynamicDestinations( + (DynamicDestinations) fromByteArray(dynamicDestinationsBytes)); + } + String jsonSchema = configRow.getString("json_schema"); + if (jsonSchema != null) { + builder = builder.setJsonSchema(StaticValueProvider.of(jsonSchema)); + } + String jsonTimePartitioning = configRow.getString("json_time_partitioning"); + if (jsonTimePartitioning != null) { + builder = builder.setJsonTimePartitioning(StaticValueProvider.of(jsonTimePartitioning)); + } + byte[] clusteringBytes = configRow.getBytes("clustering"); + if (clusteringBytes != null) { + builder = builder.setClustering((Clustering) fromByteArray(clusteringBytes)); + } + byte[] createDispositionBytes = configRow.getBytes("create_disposition"); + if (createDispositionBytes != null) { + builder = + builder.setCreateDisposition((CreateDisposition) fromByteArray(createDispositionBytes)); + } + byte[] writeDispositionBytes = configRow.getBytes("write_disposition"); + if (writeDispositionBytes != null) { + builder = + builder.setWriteDisposition((WriteDisposition) fromByteArray(writeDispositionBytes)); + } + Collection schemaUpdateOptionsData = configRow.getArray("schema_update_options"); + if (schemaUpdateOptionsData != null) { + Set schemaUpdateOptions = + schemaUpdateOptionsData.stream() + .map(data -> (SchemaUpdateOption) fromByteArray(data)) + .collect(Collectors.toSet()); + builder = builder.setSchemaUpdateOptions(schemaUpdateOptions); + } else { + // This property is not nullable. + builder = builder.setSchemaUpdateOptions(Collections.emptySet()); + } + String tableDescription = configRow.getString("table_description"); + if (tableDescription != null) { + builder = builder.setTableDescription(tableDescription); + } + Boolean validate = configRow.getBoolean("validate"); + if (validate != null) { + builder = builder.setValidate(validate); + } + byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); + if (bigqueryServicesBytes != null) { + builder = + builder.setBigQueryServices((BigQueryServices) fromByteArray(bigqueryServicesBytes)); + } + Integer maxFilesPerBundle = configRow.getInt32("max_files_per_bundle"); + if (maxFilesPerBundle != null) { + builder = builder.setMaxFilesPerBundle(maxFilesPerBundle); + } + Long maxFileSize = configRow.getInt64("max_file_size"); + if (maxFileSize != null) { + builder = builder.setMaxFileSize(maxFileSize); + } + Integer numFileShards = configRow.getInt32("num_file_shards"); + if (numFileShards != null) { + builder = builder.setNumFileShards(numFileShards); + } + Integer numStorageWriteApiStreams = configRow.getInt32("num_storage_write_api_streams"); + if (numStorageWriteApiStreams != null) { + builder = builder.setNumStorageWriteApiStreams(numStorageWriteApiStreams); + } + Boolean propagateSuccessfulStorageApiWrites = + configRow.getBoolean("propagate_successful_storage_api_writes"); + if (propagateSuccessfulStorageApiWrites != null) { + builder = + builder.setPropagateSuccessfulStorageApiWrites(propagateSuccessfulStorageApiWrites); + } + Integer maxFilesPerPartition = configRow.getInt32("max_files_per_partition"); + if (maxFilesPerPartition != null) { + builder = builder.setMaxFilesPerPartition(maxFilesPerPartition); + } + Long maxBytesPerPartition = configRow.getInt64("max_bytes_per_partition"); + if (maxBytesPerPartition != null) { + builder = builder.setMaxBytesPerPartition(maxBytesPerPartition); + } + Duration triggerringFrequency = configRow.getValue("triggerring_frequency"); + if (triggerringFrequency != null) { + builder = + builder.setTriggeringFrequency( + org.joda.time.Duration.millis(triggerringFrequency.toMillis())); + } + byte[] methodBytes = configRow.getBytes("method"); + if (methodBytes != null) { + builder = builder.setMethod((Write.Method) fromByteArray(methodBytes)); + } + String loadJobProjectId = configRow.getString("load_job_project_id"); + if (loadJobProjectId != null) { + builder = builder.setLoadJobProjectId(StaticValueProvider.of(loadJobProjectId)); + } + byte[] failedInsertRetryPolicyBytes = configRow.getBytes("failed_insert_retry_policy"); + if (failedInsertRetryPolicyBytes != null) { + builder = + builder.setFailedInsertRetryPolicy( + (InsertRetryPolicy) fromByteArray(failedInsertRetryPolicyBytes)); + } + String customGcsTempLocations = configRow.getString("custom_gcs_temp_location"); + if (customGcsTempLocations != null) { + builder = builder.setCustomGcsTempLocation(StaticValueProvider.of(customGcsTempLocations)); + } + Boolean extendedErrorInfo = configRow.getBoolean("extended_error_info"); + if (extendedErrorInfo != null) { + builder = builder.setExtendedErrorInfo(extendedErrorInfo); + } + Boolean skipInvalidRows = configRow.getBoolean("skip_invalid_rows"); + if (skipInvalidRows != null) { + builder = builder.setSkipInvalidRows(skipInvalidRows); + } + Boolean ignoreUnknownValues = configRow.getBoolean("ignore_unknown_values"); + if (ignoreUnknownValues != null) { + builder = builder.setIgnoreUnknownValues(ignoreUnknownValues); + } + Boolean ignoreInsertIds = configRow.getBoolean("ignore_insert_ids"); + if (ignoreInsertIds != null) { + builder = builder.setIgnoreInsertIds(ignoreInsertIds); + } + Integer maxRetryJobs = configRow.getInt32("max_retry_jobs"); + if (maxRetryJobs != null) { + builder = builder.setMaxRetryJobs(maxRetryJobs); + } + String kmsKey = configRow.getString("kms_key"); + if (kmsKey != null) { + builder = builder.setKmsKey(kmsKey); + } + Collection primaryKey = configRow.getArray("primary_key"); + if (primaryKey != null && !primaryKey.isEmpty()) { + builder = builder.setPrimaryKey(ImmutableList.of(primaryKey)); + } + byte[] defaultMissingValueInterpretationsBytes = + configRow.getBytes("default_missing_value_interpretation"); + if (defaultMissingValueInterpretationsBytes != null) { + builder = + builder.setDefaultMissingValueInterpretation( + (MissingValueInterpretation) + fromByteArray(defaultMissingValueInterpretationsBytes)); + } + Boolean optimizeWrites = configRow.getBoolean("optimize_writes"); + if (optimizeWrites != null) { + builder = builder.setOptimizeWrites(optimizeWrites); + } + Boolean useBeamSchema = configRow.getBoolean("use_beam_schema"); + if (useBeamSchema != null) { + builder = builder.setUseBeamSchema(useBeamSchema); + } + Boolean autoSharding = configRow.getBoolean("auto_sharding"); + if (autoSharding != null) { + builder = builder.setAutoSharding(autoSharding); + } + Boolean propagateSuccessful = configRow.getBoolean("propagate_successful"); + if (propagateSuccessful != null) { + builder = builder.setPropagateSuccessful(propagateSuccessful); + } + Boolean autoSchemaUpdate = configRow.getBoolean("auto_schema_update"); + if (autoSchemaUpdate != null) { + builder = builder.setAutoSchemaUpdate(autoSchemaUpdate); + } + byte[] writeProtosClasses = configRow.getBytes("write_protos_class"); + if (writeProtosClasses != null) { + builder = + builder.setWriteProtosClass( + (Class) fromByteArray(defaultMissingValueInterpretationsBytes)); + } + Boolean directWriteProtos = configRow.getBoolean("direct_write_protos"); + if (directWriteProtos != null) { + builder = builder.setDirectWriteProtos(directWriteProtos); + } + byte[] deterministicRecordIdFnBytes = configRow.getBytes("deterministic_record_id_fn"); + if (deterministicRecordIdFnBytes != null) { + builder = + builder.setDeterministicRecordIdFn( + (SerializableFunction) fromByteArray(deterministicRecordIdFnBytes)); + } + String writeTempDataset = configRow.getString("write_temp_dataset"); + if (writeTempDataset != null) { + builder = builder.setWriteTempDataset(writeTempDataset); + } + byte[] rowMutationInformationFnBytes = configRow.getBytes("row_mutation_information_fn"); + if (rowMutationInformationFnBytes != null) { + builder = + builder.setRowMutationInformationFn( + (SerializableFunction) fromByteArray(rowMutationInformationFnBytes)); + } + + return builder.build(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar { + + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(AutoValue_BigQueryIO_Write.class, new BigQueryIOWriteTranslator()) + .build(); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java new file mode 100644 index 000000000000..9de3b02c2531 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.junit.Test; + +public class BigQueryIOTranslationTest { + + // A mapping from Read transform builder methods to the corresponding schema fields in + // KafkaIOTranslation. + static final Map READ_TRANSFORM_SCHEMA_MAPPING = new HashMap<>(); + + static { + READ_TRANSFORM_SCHEMA_MAPPING.put("getJsonTableRef", "json_table_ref"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getQuery", "query"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getValidate", "validate"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getFlattenResults", "flatten_results"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getUseLegacySql", "use_legacy_sql"); + READ_TRANSFORM_SCHEMA_MAPPING.put( + "getWithTemplateCompatibility", "with_template_compatibility"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getBigQueryServices", "bigquery_services"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getParseFn", "parse_fn"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getDatumReaderFactory", "datum_reader_factory"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getQueryPriority", "query_priority"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getQueryLocation", "query_location"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getQueryTempDataset", "query_temp_dataset"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getMethod", "method"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getFormat", "format"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getSelectedFields", "selected_fields"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getRowRestriction", "row_restriction"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getCoder", "coder"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getKmsKey", "kms_key"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getTypeDescriptor", "type_descriptor"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getToBeamRowFn", "to_beam_row_fn"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getFromBeamRowFn", "from_beam_row_fn"); + READ_TRANSFORM_SCHEMA_MAPPING.put("getUseAvroLogicalTypes", "use_avro_logical_types"); + READ_TRANSFORM_SCHEMA_MAPPING.put( + "getProjectionPushdownApplied", "projection_pushdown_applied"); + } + + static final Map WRITE_TRANSFORM_SCHEMA_MAPPING = new HashMap<>(); + + static { + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getJsonTableRef", "json_table_ref"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getTableFunction", "table_function"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getFormatFunction", "format_function"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put( + "getFormatRecordOnFailureFunction", "format_record_on_failure_function"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getAvroRowWriterFactory", "avro_row_writer_factory"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getAvroSchemaFactory", "avro_schema_factory"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getUseAvroLogicalTypes", "use_avro_logical_types"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getDynamicDestinations", "dynamic_destinations"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getJsonSchema", "json_schema"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getJsonTimePartitioning", "json_time_partitioning"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getClustering", "clustering"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getCreateDisposition", "create_disposition"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getWriteDisposition", "write_disposition"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getSchemaUpdateOptions", "schema_update_options"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getTableDescription", "table_description"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getValidate", "validate"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getBigQueryServices", "bigquery_services"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxFilesPerBundle", "max_files_per_bundle"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxFileSize", "max_file_size"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getNumFileShards", "num_file_shards"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put( + "getNumStorageWriteApiStreams", "num_storage_write_api_streams"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put( + "getPropagateSuccessfulStorageApiWrites", "propagate_successful_storage_api_writes"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxFilesPerPartition", "max_files_per_partition"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxBytesPerPartition", "max_bytes_per_partition"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getTriggeringFrequency", "triggerring_frequency"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMethod", "method"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getLoadJobProjectId", "load_job_project_id"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getFailedInsertRetryPolicy", "failed_insert_retry_policy"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getCustomGcsTempLocation", "custom_gcs_temp_location"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getExtendedErrorInfo", "extended_error_info"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getSkipInvalidRows", "skip_invalid_rows"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getIgnoreUnknownValues", "ignore_unknown_values"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getIgnoreInsertIds", "ignore_insert_ids"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxRetryJobs", "max_retry_jobs"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getKmsKey", "kms_key"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getPrimaryKey", "primary_key"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put( + "getDefaultMissingValueInterpretation", "default_missing_value_interpretation"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getOptimizeWrites", "optimize_writes"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getUseBeamSchema", "use_beam_schema"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getAutoSharding", "auto_sharding"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getPropagateSuccessful", "propagate_successful"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getAutoSchemaUpdate", "auto_schema_update"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getWriteProtosClass", "write_protos_class"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getDirectWriteProtos", "direct_write_protos"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getDeterministicRecordIdFn", "deterministic_record_id_fn"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getWriteTempDataset", "write_temp_dataset"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put( + "getRowMutationInformationFn", "row_mutation_information_fn"); + } + + @Test + public void testReCreateReadTransformFromRowTable() { + // setting a subset of fields here. + BigQueryIO.TypedRead readTransform = + BigQueryIO.readTableRows() + .from("dummyproject:dummydataset.dummytable") + .withMethod(TypedRead.Method.DIRECT_READ) + .withKmsKey("dummykmskey") + .withTemplateCompatibility(); + + BigQueryIOTranslation.BigQueryIOReadTranslator translator = + new BigQueryIOTranslation.BigQueryIOReadTranslator(); + Row row = translator.toConfigRow(readTransform); + + BigQueryIO.TypedRead readTransformFromRow = + (BigQueryIO.TypedRead) translator.fromConfigRow(row); + assertNotNull(readTransformFromRow.getTable()); + assertEquals("dummyproject", readTransformFromRow.getTable().getProjectId()); + assertEquals("dummydataset", readTransformFromRow.getTable().getDatasetId()); + assertEquals("dummytable", readTransformFromRow.getTable().getTableId()); + assertEquals(TypedRead.Method.DIRECT_READ, readTransformFromRow.getMethod()); + assertEquals("dummykmskey", readTransformFromRow.getKmsKey()); + assertTrue(readTransformFromRow.getWithTemplateCompatibility()); + } + + static class DummyParseFn implements SerializableFunction { + @Override + public Object apply(SchemaAndRecord input) { + return null; + } + } + + @Test + public void testReCreateReadTransformFromRowQuery() { + // setting a subset of fields here. + BigQueryIO.TypedRead readTransform = + BigQueryIO.read(new DummyParseFn()) + .fromQuery("dummyquery") + .useAvroLogicalTypes() + .usingStandardSql(); + + BigQueryIOTranslation.BigQueryIOReadTranslator translator = + new BigQueryIOTranslation.BigQueryIOReadTranslator(); + Row row = translator.toConfigRow(readTransform); + + BigQueryIO.TypedRead readTransformFromRow = translator.fromConfigRow(row); + assertEquals("dummyquery", readTransformFromRow.getQuery().get()); + assertNotNull(readTransformFromRow.getParseFn()); + assertTrue(readTransformFromRow.getParseFn() instanceof DummyParseFn); + assertTrue(readTransformFromRow.getUseAvroLogicalTypes()); + assertFalse(readTransformFromRow.getUseLegacySql()); + } + + @Test + public void testReadTransformRowIncludesAllFields() { + // These fields do not represent properties of the transform. + List fieldsToIgnore = + ImmutableList.of("getFinalSchema", "getTableProvider", "getTable"); + + List getMethodNames = + Arrays.stream(BigQueryIO.TypedRead.class.getDeclaredMethods()) + .map( + method -> { + return method.getName(); + }) + .filter(methodName -> methodName.startsWith("get")) + .filter(methodName -> !fieldsToIgnore.contains(methodName)) + .collect(Collectors.toList()); + + // Just to make sure that this does not pass trivially. + assertTrue(getMethodNames.size() > 0); + + for (String getMethodName : getMethodNames) { + assertTrue( + "Method " + + getMethodName + + " will not be tracked when upgrading the 'BigQueryIO.TypedRead' transform. Please update" + + "'BigQueryIOTranslation.BigQueryIOReadTranslator' to track the new method " + + "and update this test.", + READ_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); + } + + // Confirming that all fields mentioned in `READ_TRANSFORM_SCHEMA_MAPPING` are + // actually available in the schema. + READ_TRANSFORM_SCHEMA_MAPPING.values().stream() + .forEach( + fieldName -> { + assertTrue( + "Field name " + + fieldName + + " was not found in the transform schema defined in " + + "BigQueryIOTranslation.BigQueryIOReadTranslator.", + BigQueryIOTranslation.BigQueryIOReadTranslator.schema + .getFieldNames() + .contains(fieldName)); + }); + } + + @Test + public void testReCreateWriteTransformFromRowTable() { + // setting a subset of fields here. + BigQueryIO.Write writeTransform = + BigQueryIO.write() + .to("dummyproject:dummydataset.dummytable") + .withAutoSharding() + .withTriggeringFrequency(org.joda.time.Duration.millis(10000)) + .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE) + .withCreateDisposition(CreateDisposition.CREATE_NEVER) + .withKmsKey("dummykmskey"); + + BigQueryIOTranslation.BigQueryIOWriteTranslator translator = + new BigQueryIOTranslation.BigQueryIOWriteTranslator(); + Row row = translator.toConfigRow(writeTransform); + + BigQueryIO.Write writeTransformFromRow = (BigQueryIO.Write) translator.fromConfigRow(row); + assertNotNull(writeTransformFromRow.getTable()); + assertEquals("dummyproject", writeTransformFromRow.getTable().get().getProjectId()); + assertEquals("dummydataset", writeTransformFromRow.getTable().get().getDatasetId()); + assertEquals("dummytable", writeTransformFromRow.getTable().get().getTableId()); + assertEquals(WriteDisposition.WRITE_TRUNCATE, writeTransformFromRow.getWriteDisposition()); + assertEquals(CreateDisposition.CREATE_NEVER, writeTransformFromRow.getCreateDisposition()); + assertEquals("dummykmskey", writeTransformFromRow.getKmsKey()); + } + + @Test + public void testWriteTransformRowIncludesAllFields() { + // These fields do not represent properties of the transform. + List fieldsToIgnore = + ImmutableList.of( + "getSchemaFromView", + "getStorageApiNumStreams", + "getStorageApiTriggeringFrequency", + "getTableWithDefaultProject", + "getTable"); + + List getMethodNames = + Arrays.stream(BigQueryIO.Write.class.getDeclaredMethods()) + .map( + method -> { + return method.getName(); + }) + .filter(methodName -> methodName.startsWith("get")) + .filter(methodName -> !fieldsToIgnore.contains(methodName)) + .collect(Collectors.toList()); + + // Just to make sure that this does not pass trivially. + assertTrue(getMethodNames.size() > 0); + + for (String getMethodName : getMethodNames) { + assertTrue( + "Method " + + getMethodName + + " will not be tracked when upgrading the 'BigQueryIO.Write' transform. Please update" + + "'BigQueryIOTranslation.BigQueryIOWriteTranslator' to track the new method " + + "and update this test.", + WRITE_TRANSFORM_SCHEMA_MAPPING.keySet().contains(getMethodName)); + } + + // Confirming that all fields mentioned in `WRITE_TRANSFORM_SCHEMA_MAPPING` are + // actually available in the schema. + WRITE_TRANSFORM_SCHEMA_MAPPING.values().stream() + .forEach( + fieldName -> { + assertTrue( + "Field name " + + fieldName + + " was not found in the transform schema defined in " + + "BigQueryIOTranslation.BigQueryIOWriteTranslator.", + BigQueryIOTranslation.BigQueryIOWriteTranslator.schema + .getFieldNames() + .contains(fieldName)); + }); + } +} diff --git a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java index 1757da6a6311..16580cd219b6 100644 --- a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java +++ b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.io.kafka.upgrade; +import static org.apache.beam.runners.core.construction.TransformUpgrader.fromByteArray; +import static org.apache.beam.runners.core.construction.TransformUpgrader.toByteArray; + import com.google.auto.service.AutoService; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -73,25 +72,6 @@ public class KafkaIOTranslation { public static final String KAFKA_WRITE_TRANSFORM_URN_V2 = "beam:transform:org.apache.beam:kafka_write:v2"; - private static byte[] toByteArray(Object object) { - try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bos)) { - out.writeObject(object); - return bos.toByteArray(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - private static Object fromByteArray(byte[] bytes) { - try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bis)) { - return in.readObject(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - static class KafkaIOReadWithMetadataTranslator implements TransformPayloadTranslator> { static Schema topicPartitionSchema = From 04fe4b53433f67e1a9e2d0a83d715100795a1bc3 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 1 Dec 2023 10:13:49 -0800 Subject: [PATCH 013/224] Consider underlying providers in both directions when doing affinity test. --- sdks/python/apache_beam/yaml/yaml_provider.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 01e39b770c9b..25cc366ec2c7 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -109,9 +109,9 @@ def affinity(self, other: "Provider"): # E.g. we could look at the the expected environments themselves. # Possibly, we could provide multiple expansions and have the runner itself # choose the actual implementation based on fusion (and other) criteria. - return ( - self.underlying_provider()._affinity(other) + - other.underlying_provider()._affinity(self)) + a = self.underlying_provider() + b = other.underlying_provider() + return a._affinity(b) + b._affinity(a) def _affinity(self, other: "Provider"): if self is other or self == other: From ebc4a5585e0935467ac3cb68b870605c8227224f Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Fri, 1 Dec 2023 10:40:42 -0800 Subject: [PATCH 014/224] Documents the transform upgrade feature (#29581) * temp commit * Documents the transform upgrade feature * Fix sub-topic numbers --- .../en/documentation/programming-guide.md | 64 +++++++++++++------ .../section-menu/en/documentation.html | 1 + 2 files changed, 45 insertions(+), 20 deletions(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 92163c12ab8b..547b7341fd71 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -8105,40 +8105,64 @@ class RetrieveTimingDoFn(beam.DoFn): ## 15 Transform service {#transform-service} The Apache Beam SDK versions 2.49.0 and later include a [Docker Compose](https://docs.docker.com/compose/) -service named _Transform service_. Use the Transform service to perform expansions of supported transforms -on Beam portable pipelines by using Docker. +service named _Transform service_. The following diagram illustrates the basic architecture of the Transform service. ![Diagram of the Transform service architecture](/images/transform_service.png) -To use the Transform service, Docker and Docker Compose must be available on the machine that starts the service. +To use the Transform service, Docker must be available on the machine that starts the service. -The Transform service has the following primary use cases: +The Transform service has several primary use cases. -* Perform expansion of cross-language transforms without installing other language runtimes. +### 15.1 Using the transform service to upgrade transforms {#transform-service-usage-upgrade} - The Transform service allows multi-language pipelines to use and expand cross-language transforms implemented - in other SDKs without requiring you to install runtimes for the implementation languages of those SDKs. - For example, with the Transform service, a Beam Python pipeline can use the Google Cloud Java I/O transforms and Java Kafka I/O transforms - without a local Java runtime installation. +Transform service can be used to upgrade (or downgrade) the Beam SDK versions of supported individual transforms used by Beam pipelines without changing the Beam version of the pipelines. +This feature is currently only available for Beam Java SDK 2.53.0 and later. Currently following transforms are available for upgrading. -* Upgrade transforms without upgrading the Apache Beam SDK version. +* BigQuery read transform (URN: _beam:transform:org.apache.beam:bigquery_read:v1_) +* BigQuery write transform (URN: _beam:transform:org.apache.beam:bigquery_write:v1_) +* Kafka read transform (URN: _beam:transform:org.apache.beam:kafka_read_with_metadata:v2_) +* Kafka write transform (URN: _beam:transform:org.apache.beam:kafka_write:v2_) - Use the Transform service to upgrade the Beam SDK versions of individual transforms used by Beam pipelines without upgrading the Beam version of the pipeline. - This feature is currently in development. For more details, see - [GitHub issue #27943: Upgrade transforms without upgrading the pipeline using the Transform Service](https://github.com/apache/beam/issues/27943). +To use this feature, you can simply execute a Java pipeline with additional pipeline options that specifies the URNs of the transforms you would like to upgrade and the Beam version you would like to upgrade the transforms to. Alll transforms in the pipeline with matching URNs will be upgraded. -### 15.1 Use the Transform service {#transform-service-usage} +For example, to upgrade the BigQuery read transform for a piepline run using Beam `2.53.0` to a future Beam version `2.xy.z`, you can specify following additional pipelines options. -In some cases, Apache Beam SDKs automatically start the Transform service, such as in the following scenarios: +{{< highlight java >}} +--transformsToOverride=beam:transform:org.apache.beam:bigquery_read:v1 --transformServiceBeamVersion=2.xy.z +{{< /highlight >}} + +{{< highlight py >}} +This feature is currently not available for Python SDK. +{{< /highlight >}} + +{{< highlight go >}} +This feature is currently not available for Go SDK. +{{< /highlight >}} + +Note that the framework will automatically download the relavent Docker containers and startup the transform service for you. + +Please see [here](https://cwiki.apache.org/confluence/display/BEAM/Transform+Service#TransformService-Upgradetransformswithoutupgradingthepipeline) for a full example that uses this feature to upgrade BigQuery read and write transforms. + +### 15.2 Using the Transform service for multi-language pipelines {#transform-service-usage-multi-language} + +Transform service implements the Beam expansion API. This allows Beam multi-language pipelines to use the transform service when expanding transforms available withn the transform service. +The main advantage here is that multi-language pipelines will be able to operate without installing support for additional language runtimes. For example, Beam Python pipelines that use Java transforms such as +`KafkaIO` can operate without installing Java locally during job submission as long as Docker is available in the system. + +In some cases, Apache Beam SDKs automatically start the Transform service. * The Java [`PythonExternalTransform` API](https://github.com/apache/beam/blob/master/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java) automatically starts the Transform service when a Python runtime isn't available locally, but Docker is. * The Apache Beam Python multi-language wrappers might automatically start the Transform service when you're using Java transforms, a Java language runtime isn't available locally, and Docker is available locally. -To manually start a Transform service instance by using utilities provided with the Apache Beam SDKs, use the following commands. +Beam users also have the option to [manually start](/documentation/programming-guide/#transform-service-usage-muanual) a stransform service and use that as the expansion service used by multi-language pipelines. + +### 15.3 Manually starting the transform service {#transform-service-usage-muanual} + +A Beam Transform service instance can be manually started by by using utilities provided with Apache Beam SDKs. {{< highlight java >}} java -jar beam-sdks-java-transform-service-launcher-.jar --port --beam_version --project_name --command up @@ -8166,11 +8190,11 @@ python -m apache_beam.utils.transform_service_launcher --port --beam_vers This feature is currently in development. {{< /highlight >}} -### 15.2 Portable transforms included in the Transform service {#transform-service-included-transforms} +### 15.4 Portable transforms included in the Transform service {#transform-service-included-transforms} -The Transform service includes portable transforms implemented in the Apache Beam Java and Python SDKs. +Beam Transform service includes a number of transforms implemented in the Apache Beam Java and Python SDKs. -The following transforms are included in the Trasnform service: +Currently, following transforms are included in the Transform service: * Java transforms: Google Cloud I/O connectors, the Kafka I/O connector, and the JDBC I/O connector @@ -8179,4 +8203,4 @@ The following transforms are included in the Trasnform service: [DataFrame](/documentation/dsls/dataframes/overview/) transforms. For a more comprehensive list of available transforms, see the -[Transform service](https://cwiki.apache.org/confluence/display/BEAM/Transform+Service) developer guide. +[Transform service](https://cwiki.apache.org/confluence/display/BEAM/Transform+Service#TransformService-TransformsincludedintheTransformservice) developer guide. diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 2ab5bec69e98..dcf0b857bf97 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -185,6 +185,7 @@
  • Batched DoFns
  • +
  • Transform service
  • From 39e1615f945651833ec60fbab090e81bab96e0c2 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Fri, 1 Dec 2023 15:55:54 -0500 Subject: [PATCH 015/224] fix typos (#29588) --- .../en/documentation/programming-guide.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 547b7341fd71..05a3232aa186 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -8118,16 +8118,16 @@ The Transform service has several primary use cases. ### 15.1 Using the transform service to upgrade transforms {#transform-service-usage-upgrade} Transform service can be used to upgrade (or downgrade) the Beam SDK versions of supported individual transforms used by Beam pipelines without changing the Beam version of the pipelines. -This feature is currently only available for Beam Java SDK 2.53.0 and later. Currently following transforms are available for upgrading. +This feature is currently only available for Beam Java SDK 2.53.0 and later. Currently, the following transforms are available for upgrading: * BigQuery read transform (URN: _beam:transform:org.apache.beam:bigquery_read:v1_) * BigQuery write transform (URN: _beam:transform:org.apache.beam:bigquery_write:v1_) * Kafka read transform (URN: _beam:transform:org.apache.beam:kafka_read_with_metadata:v2_) * Kafka write transform (URN: _beam:transform:org.apache.beam:kafka_write:v2_) -To use this feature, you can simply execute a Java pipeline with additional pipeline options that specifies the URNs of the transforms you would like to upgrade and the Beam version you would like to upgrade the transforms to. Alll transforms in the pipeline with matching URNs will be upgraded. +To use this feature, you can simply execute a Java pipeline with additional pipeline options that specify the URNs of the transforms you would like to upgrade and the Beam version you would like to upgrade the transforms to. All transforms in the pipeline with matching URNs will be upgraded. -For example, to upgrade the BigQuery read transform for a piepline run using Beam `2.53.0` to a future Beam version `2.xy.z`, you can specify following additional pipelines options. +For example, to upgrade the BigQuery read transform for a pipeline run using Beam `2.53.0` to a future Beam version `2.xy.z`, you can specify the following additional pipelines options. {{< highlight java >}} --transformsToOverride=beam:transform:org.apache.beam:bigquery_read:v1 --transformServiceBeamVersion=2.xy.z @@ -8141,28 +8141,28 @@ This feature is currently not available for Python SDK. This feature is currently not available for Go SDK. {{< /highlight >}} -Note that the framework will automatically download the relavent Docker containers and startup the transform service for you. +Note that the framework will automatically download the relevant Docker containers and startup the transform service for you. Please see [here](https://cwiki.apache.org/confluence/display/BEAM/Transform+Service#TransformService-Upgradetransformswithoutupgradingthepipeline) for a full example that uses this feature to upgrade BigQuery read and write transforms. ### 15.2 Using the Transform service for multi-language pipelines {#transform-service-usage-multi-language} -Transform service implements the Beam expansion API. This allows Beam multi-language pipelines to use the transform service when expanding transforms available withn the transform service. +Transform service implements the Beam expansion API. This allows Beam multi-language pipelines to use the transform service when expanding transforms available within the transform service. The main advantage here is that multi-language pipelines will be able to operate without installing support for additional language runtimes. For example, Beam Python pipelines that use Java transforms such as `KafkaIO` can operate without installing Java locally during job submission as long as Docker is available in the system. -In some cases, Apache Beam SDKs automatically start the Transform service. +In some cases, Apache Beam SDKs can automatically start the Transform service. * The Java [`PythonExternalTransform` API](https://github.com/apache/beam/blob/master/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java) automatically starts the Transform service when a Python runtime isn't available locally, but Docker is. * The Apache Beam Python multi-language wrappers might automatically start the Transform service when you're using Java transforms, a Java language runtime isn't available locally, and Docker is available locally. -Beam users also have the option to [manually start](/documentation/programming-guide/#transform-service-usage-muanual) a stransform service and use that as the expansion service used by multi-language pipelines. +Beam users also have the option to [manually start](/documentation/programming-guide/#transform-service-usage-muanual) a transform service and use that as the expansion service used by multi-language pipelines. ### 15.3 Manually starting the transform service {#transform-service-usage-muanual} -A Beam Transform service instance can be manually started by by using utilities provided with Apache Beam SDKs. +A Beam Transform service instance can be manually started by using utilities provided with Apache Beam SDKs. {{< highlight java >}} java -jar beam-sdks-java-transform-service-launcher-.jar --port --beam_version --project_name --command up @@ -8194,7 +8194,7 @@ This feature is currently in development. Beam Transform service includes a number of transforms implemented in the Apache Beam Java and Python SDKs. -Currently, following transforms are included in the Transform service: +Currently, the following transforms are included in the Transform service: * Java transforms: Google Cloud I/O connectors, the Kafka I/O connector, and the JDBC I/O connector From 6593a0f15c1a3cc9813059100425d5716eba0193 Mon Sep 17 00:00:00 2001 From: Pablo Date: Sat, 2 Dec 2023 08:13:54 -0800 Subject: [PATCH 016/224] Adding failure_mode parameter to Spanner Python transforms (#29529) * Adding failure_mode parameter to Spanner Python transforms * fix formatting * Improve pydoc * fixup docs --- .../spanner/SpannerTransformRegistrar.java | 10 +++++++++ sdks/python/apache_beam/io/gcp/spanner.py | 21 +++++++++++++++++++ .../io/gcp/tests/xlang_spannerio_it_test.py | 1 + 3 files changed, 32 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java index 38cd97da860a..809d7a275512 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java @@ -288,6 +288,7 @@ public static class Configuration extends CrossLanguageConfiguration { private @Nullable Integer groupingFactor; private @Nullable Duration commitDeadline; private @Nullable Duration maxCumulativeBackoff; + private @Nullable String failureMode; public void setTable(String table) { this.table = table; @@ -322,6 +323,10 @@ public void setMaxCumulativeBackoff(@Nullable Long maxCumulativeBackoff) { this.maxCumulativeBackoff = Duration.standardSeconds(maxCumulativeBackoff); } } + + public void setFailureMode(@Nullable String failureMode) { + this.failureMode = failureMode; + } } @Override @@ -361,6 +366,11 @@ public PTransform, PDone> buildExternal( writeTransform = writeTransform.withMaxCumulativeBackoff(configuration.maxCumulativeBackoff); } + if (configuration.failureMode != null) { + writeTransform = + writeTransform.withFailureMode( + SpannerIO.FailureMode.valueOf(configuration.failureMode)); + } return SpannerIO.WriteRows.of(writeTransform, operation, configuration.table); } } diff --git a/sdks/python/apache_beam/io/gcp/spanner.py b/sdks/python/apache_beam/io/gcp/spanner.py index c16daa4448b1..51c7fc65c171 100644 --- a/sdks/python/apache_beam/io/gcp/spanner.py +++ b/sdks/python/apache_beam/io/gcp/spanner.py @@ -124,6 +124,11 @@ class TimestampBoundMode(Enum): STRONG = auto() +class FailureMode(Enum): + FAIL_FAST = auto() + REPORT_FAILURES = auto() + + class ReadFromSpannerSchema(NamedTuple): instance_id: str database_id: str @@ -282,6 +287,7 @@ class WriteToSpannerSchema(NamedTuple): emulator_host: Optional[str] commit_deadline: Optional[int] max_cumulative_backoff: Optional[int] + failure_mode: Optional[str] _CLASS_DOC = \ @@ -346,6 +352,11 @@ class {row_type}(NamedTuple): (15min). If the mutations still have not been written after this time, they are treated as a failure, and handled according to the setting of failure_mode. Pass seconds as value. + :param failure_mode: Specifies the behavior for mutations that fail to be + written to Spanner. Default is FAIL_FAST. When FAIL_FAST is set, + an exception will be thrown for any failed mutation. When REPORT_FAILURES + is set, processing will continue instead of throwing an exception. Note + that REPORT_FAILURES can cause data loss if used incorrectly. :param expansion_service: The address (host:port) of the ExpansionService. """ @@ -392,6 +403,7 @@ def __init__( emulator_host=None, commit_deadline=None, max_cumulative_backoff=None, + failure_mode=None, expansion_service=None, ): max_cumulative_backoff = int( @@ -413,6 +425,7 @@ def __init__( emulator_host=emulator_host, commit_deadline=commit_deadline, max_cumulative_backoff=max_cumulative_backoff, + failure_mode=_get_enum_name(failure_mode), ), ), expansion_service=expansion_service or default_io_expansion_service(), @@ -445,6 +458,7 @@ def __init__( commit_deadline=None, max_cumulative_backoff=None, expansion_service=None, + failure_mode=None, ): max_cumulative_backoff = int( max_cumulative_backoff) if max_cumulative_backoff else None @@ -465,6 +479,7 @@ def __init__( emulator_host=emulator_host, commit_deadline=commit_deadline, max_cumulative_backoff=max_cumulative_backoff, + failure_mode=_get_enum_name(failure_mode), ), ), expansion_service=expansion_service or default_io_expansion_service(), @@ -497,6 +512,7 @@ def __init__( commit_deadline=None, max_cumulative_backoff=None, expansion_service=None, + failure_mode=None, ): max_cumulative_backoff = int( max_cumulative_backoff) if max_cumulative_backoff else None @@ -517,6 +533,7 @@ def __init__( emulator_host=emulator_host, commit_deadline=commit_deadline, max_cumulative_backoff=max_cumulative_backoff, + failure_mode=_get_enum_name(failure_mode), ), ), expansion_service=expansion_service or default_io_expansion_service(), @@ -548,6 +565,7 @@ def __init__( emulator_host=None, commit_deadline=None, max_cumulative_backoff=None, + failure_mode=None, expansion_service=None, ): max_cumulative_backoff = int( @@ -569,6 +587,7 @@ def __init__( emulator_host=emulator_host, commit_deadline=commit_deadline, max_cumulative_backoff=max_cumulative_backoff, + failure_mode=_get_enum_name(failure_mode), ), ), expansion_service=expansion_service or default_io_expansion_service(), @@ -600,6 +619,7 @@ def __init__( emulator_host=None, commit_deadline=None, max_cumulative_backoff=None, + failure_mode=None, expansion_service=None, ): max_cumulative_backoff = int( @@ -621,6 +641,7 @@ def __init__( emulator_host=emulator_host, commit_deadline=commit_deadline, max_cumulative_backoff=max_cumulative_backoff, + failure_mode=_get_enum_name(failure_mode), ), ), expansion_service=expansion_service or default_io_expansion_service(), diff --git a/sdks/python/apache_beam/io/gcp/tests/xlang_spannerio_it_test.py b/sdks/python/apache_beam/io/gcp/tests/xlang_spannerio_it_test.py index 5d701052965b..43a74f170531 100644 --- a/sdks/python/apache_beam/io/gcp/tests/xlang_spannerio_it_test.py +++ b/sdks/python/apache_beam/io/gcp/tests/xlang_spannerio_it_test.py @@ -234,6 +234,7 @@ def run_write_pipeline( database_id=self.database_id, project_id=self.project_id, table=self.table, + failure_mode=beam.io.gcp.spanner.FailureMode.REPORT_FAILURES, emulator_host=self.spanner_helper.get_emulator_host(), )) From f7ae6b41c3754eb6c50352b98fb8f296ec62e58e Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sun, 3 Dec 2023 04:19:11 +0000 Subject: [PATCH 017/224] Bump Pyarrow version to include 14.0.0 (#29536) * generate requirements file * update setup.py pyarrow * Update pyarrow except for windows * Skip tests for version 13 or greater. * Add reason for unittest skipIf * Fix pyarrow lower bound * Update sdks/python/setup.py Co-authored-by: tvalentyn --------- Co-authored-by: tvalentyn --- sdks/python/apache_beam/io/parquetio_test.py | 3 + .../py310/base_image_requirements.txt | 80 +++++++++---------- .../py311/base_image_requirements.txt | 76 +++++++++--------- .../py38/base_image_requirements.txt | 78 +++++++++--------- .../py39/base_image_requirements.txt | 80 +++++++++---------- sdks/python/setup.py | 14 +++- 6 files changed, 173 insertions(+), 158 deletions(-) diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index 1cd5f1208cc2..06d9a5271404 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -334,6 +334,9 @@ def test_write_batched_display_data(self): ] hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items)) + @unittest.skipIf( + ARROW_MAJOR_VERSION >= 13, + 'pyarrow 13.x and above does not throw ArrowInvalid error') def test_sink_transform_int96(self): with tempfile.NamedTemporaryFile() as dst: path = dst.name diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index a9f94104374e..a80e6611d9cf 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -25,38 +25,38 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 build==1.0.3 -cachetools==5.3.1 -certifi==2023.7.22 +cachetools==5.3.2 +certifi==2023.11.17 cffi==1.16.0 -charset-normalizer==3.3.0 +charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.4 +cryptography==41.0.6 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 -exceptiongroup==1.1.3 +exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.8.4 +fastavro==1.9.0 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.12.0 -google-api-python-client==2.104.0 +google-api-core==2.14.0 +google-api-python-client==2.108.0 google-apitools==0.5.31 -google-auth==2.23.3 +google-auth==2.23.4 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.35.0 -google-cloud-bigquery==3.12.0 +google-cloud-aiplatform==1.36.4 +google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 -google-cloud-dlp==3.12.3 +google-cloud-dlp==3.13.0 google-cloud-language==2.11.1 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.18.4 @@ -64,26 +64,26 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.12.0 +google-cloud-storage==2.13.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 googleapis-common-protos==1.61.0 -greenlet==3.0.0 -grpc-google-iam-v1==0.12.6 -grpcio==1.59.0 -grpcio-status==1.59.0 -guppy3==3.1.4 +greenlet==3.0.1 +grpc-google-iam-v1==0.12.7 +grpcio==1.59.3 +grpcio-status==1.59.3 +guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.88.1 -idna==3.4 +hypothesis==6.91.0 +idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 -jsonschema==4.19.1 -jsonschema-specifications==2023.7.1 +jsonschema==4.20.0 +jsonschema-specifications==2023.11.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -91,47 +91,47 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.9 +orjson==3.9.10 overrides==6.5.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.25.0 +protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==11.0.0 -pyarrow-hotfix==0.4 -pyasn1==0.5.0 +pyarrow==14.0.1 +pyarrow-hotfix==0.6 +pyasn1==0.5.1 pyasn1-modules==0.3.0 pycparser==2.21 pydot==1.4.2 -PyHamcrest==2.0.4 +PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.5.0 +pymongo==4.6.0 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 -pytest==7.4.2 +pytest==7.4.3 pytest-timeout==2.2.0 -pytest-xdist==3.3.1 +pytest-xdist==3.5.0 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.30.2 +referencing==0.31.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.10.6 +rpds-py==0.13.1 rsa==4.9 -scikit-learn==1.3.1 -scipy==1.11.3 +scikit-learn==1.3.2 +scipy==1.11.4 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==1.4.49 +SQLAlchemy==1.4.50 sqlparse==0.4.4 tenacity==8.2.3 testcontainers==3.7.1 @@ -139,9 +139,9 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.1 +tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.0.7 +urllib3==2.1.0 websocket-client==1.6.4 -wrapt==1.15.0 -zstandard==0.21.0 +wrapt==1.16.0 +zstandard==0.22.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 865b856683a4..04d9fc28c273 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -25,14 +25,14 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 build==1.0.3 -cachetools==5.3.1 -certifi==2023.7.22 +cachetools==5.3.2 +certifi==2023.11.17 cffi==1.16.0 -charset-normalizer==3.3.0 +charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.4 +cryptography==41.0.6 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 @@ -40,47 +40,47 @@ dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 execnet==2.0.2 -fastavro==1.8.4 +fastavro==1.9.0 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.12.0 +google-api-core==2.14.0 google-apitools==0.5.31 -google-auth==2.23.3 +google-auth==2.23.4 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.35.0 -google-cloud-bigquery==3.12.0 +google-cloud-aiplatform==1.36.4 +google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 -google-cloud-dlp==3.12.3 +google-cloud-dlp==3.13.0 google-cloud-language==2.11.1 google-cloud-pubsub==2.18.4 google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.12.0 +google-cloud-storage==2.13.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 googleapis-common-protos==1.61.0 -greenlet==3.0.0 -grpc-google-iam-v1==0.12.6 -grpcio==1.59.0 -grpcio-status==1.59.0 -guppy3==3.1.4 +greenlet==3.0.1 +grpc-google-iam-v1==0.12.7 +grpcio==1.59.3 +grpcio-status==1.59.3 +guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.88.1 -idna==3.4 +hypothesis==6.91.0 +idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 -jsonschema==4.19.1 -jsonschema-specifications==2023.7.1 +jsonschema==4.20.0 +jsonschema-specifications==2023.11.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -88,54 +88,54 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.9 +orjson==3.9.10 overrides==6.5.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.25.0 +protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==11.0.0 -pyarrow-hotfix==0.4 -pyasn1==0.5.0 +pyarrow==14.0.1 +pyarrow-hotfix==0.6 +pyasn1==0.5.1 pyasn1-modules==0.3.0 pycparser==2.21 pydot==1.4.2 -PyHamcrest==2.0.4 +PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.5.0 +pymongo==4.6.0 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 -pytest==7.4.2 +pytest==7.4.3 pytest-timeout==2.2.0 -pytest-xdist==3.3.1 +pytest-xdist==3.5.0 python-dateutil==2.8.2 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.30.2 +referencing==0.31.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.10.6 +rpds-py==0.13.1 rsa==4.9 -scikit-learn==1.3.1 -scipy==1.11.3 +scikit-learn==1.3.2 +scipy==1.11.4 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==1.4.49 +SQLAlchemy==1.4.50 sqlparse==0.4.4 tenacity==8.2.3 testcontainers==3.7.1 threadpoolctl==3.2.0 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.1 -urllib3==2.0.7 +tzlocal==5.2 +urllib3==2.1.0 websocket-client==1.6.4 -wrapt==1.15.0 -zstandard==0.21.0 +wrapt==1.16.0 +zstandard==0.22.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 83580f65f2d4..f22bbfe7d719 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -26,10 +26,10 @@ backports.zoneinfo==0.2.1 beautifulsoup4==4.12.2 bs4==0.0.1 build==1.0.3 -cachetools==5.3.1 -certifi==2023.7.22 +cachetools==5.3.2 +certifi==2023.11.17 cffi==1.16.0 -charset-normalizer==3.3.0 +charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 @@ -40,24 +40,24 @@ dill==0.3.1.1 dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 -exceptiongroup==1.1.3 +exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.8.4 +fastavro==1.9.0 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.12.0 -google-api-python-client==2.104.0 +google-api-core==2.14.0 +google-api-python-client==2.108.0 google-apitools==0.5.31 -google-auth==2.23.3 +google-auth==2.23.4 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.35.0 -google-cloud-bigquery==3.12.0 +google-cloud-aiplatform==1.36.4 +google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 -google-cloud-dlp==3.12.3 +google-cloud-dlp==3.13.0 google-cloud-language==2.11.1 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.18.4 @@ -65,28 +65,28 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.12.0 +google-cloud-storage==2.13.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 googleapis-common-protos==1.61.0 -greenlet==3.0.0 -grpc-google-iam-v1==0.12.6 -grpcio==1.59.0 -grpcio-status==1.59.0 -guppy3==3.1.4 +greenlet==3.0.1 +grpc-google-iam-v1==0.12.7 +grpcio==1.59.3 +grpcio-status==1.59.3 +guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.88.1 -idna==3.4 +hypothesis==6.91.0 +idna==3.6 importlib-metadata==6.8.0 -importlib-resources==6.1.0 +importlib-resources==6.1.1 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 -jsonschema==4.19.1 -jsonschema-specifications==2023.7.1 +jsonschema==4.20.0 +jsonschema-specifications==2023.11.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -94,7 +94,7 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.9 +orjson==3.9.10 overrides==6.5.0 packaging==23.2 pandas==1.5.3 @@ -102,40 +102,40 @@ parameterized==0.9.0 pkgutil_resolve_name==1.3.10 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.25.0 +protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==11.0.0 -pyarrow-hotfix==0.4 -pyasn1==0.5.0 +pyarrow==14.0.1 +pyarrow-hotfix==0.6 +pyasn1==0.5.1 pyasn1-modules==0.3.0 pycparser==2.21 pydot==1.4.2 -PyHamcrest==2.0.4 +PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.5.0 +pymongo==4.6.0 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 -pytest==7.4.2 +pytest==7.4.3 pytest-timeout==2.2.0 -pytest-xdist==3.3.1 +pytest-xdist==3.5.0 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.30.2 +referencing==0.31.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.10.6 +rpds-py==0.13.1 rsa==4.9 -scikit-learn==1.3.1 +scikit-learn==1.3.2 scipy==1.10.1 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==1.4.49 +SQLAlchemy==1.4.50 sqlparse==0.4.4 tenacity==8.2.3 testcontainers==3.7.1 @@ -143,10 +143,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.1 +tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.0.7 +urllib3==2.1.0 websocket-client==1.6.4 -wrapt==1.15.0 +wrapt==1.16.0 zipp==3.17.0 -zstandard==0.21.0 +zstandard==0.22.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 1b8ad7a2e748..f42b6f8c2b45 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -25,38 +25,38 @@ attrs==23.1.0 beautifulsoup4==4.12.2 bs4==0.0.1 build==1.0.3 -cachetools==5.3.1 -certifi==2023.7.22 +cachetools==5.3.2 +certifi==2023.11.17 cffi==1.16.0 -charset-normalizer==3.3.0 +charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.4 +cryptography==41.0.6 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 -exceptiongroup==1.1.3 +exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.8.4 +fastavro==1.9.0 fasteners==0.19 freezegun==1.2.2 future==0.18.3 -google-api-core==2.12.0 -google-api-python-client==2.104.0 +google-api-core==2.14.0 +google-api-python-client==2.108.0 google-apitools==0.5.31 -google-auth==2.23.3 +google-auth==2.23.4 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.35.0 -google-cloud-bigquery==3.12.0 +google-cloud-aiplatform==1.36.4 +google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.22.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 -google-cloud-dlp==3.12.3 +google-cloud-dlp==3.13.0 google-cloud-language==2.11.1 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.18.4 @@ -64,27 +64,27 @@ google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 google-cloud-resource-manager==1.10.4 google-cloud-spanner==3.40.1 -google-cloud-storage==2.12.0 +google-cloud-storage==2.13.0 google-cloud-videointelligence==2.11.4 google-cloud-vision==3.4.5 google-crc32c==1.5.0 google-resumable-media==2.6.0 googleapis-common-protos==1.61.0 -greenlet==3.0.0 -grpc-google-iam-v1==0.12.6 -grpcio==1.59.0 -grpcio-status==1.59.0 -guppy3==3.1.4 +greenlet==3.0.1 +grpc-google-iam-v1==0.12.7 +grpcio==1.59.3 +grpcio-status==1.59.3 +guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.88.1 -idna==3.4 +hypothesis==6.91.0 +idna==3.6 importlib-metadata==6.8.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 -jsonschema==4.19.1 -jsonschema-specifications==2023.7.1 +jsonschema==4.20.0 +jsonschema-specifications==2023.11.1 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -92,47 +92,47 @@ nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 -orjson==3.9.9 +orjson==3.9.10 overrides==6.5.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 proto-plus==1.22.3 -protobuf==4.25.0 +protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==11.0.0 -pyarrow-hotfix==0.4 -pyasn1==0.5.0 +pyarrow==14.0.1 +pyarrow-hotfix==0.6 +pyasn1==0.5.1 pyasn1-modules==0.3.0 pycparser==2.21 pydot==1.4.2 -PyHamcrest==2.0.4 +PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.5.0 +pymongo==4.6.0 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 -pytest==7.4.2 +pytest==7.4.3 pytest-timeout==2.2.0 -pytest-xdist==3.3.1 +pytest-xdist==3.5.0 python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.30.2 +referencing==0.31.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.10.6 +rpds-py==0.13.1 rsa==4.9 -scikit-learn==1.3.1 -scipy==1.11.3 +scikit-learn==1.3.2 +scipy==1.11.4 shapely==2.0.2 six==1.16.0 sortedcontainers==2.4.0 soupsieve==2.5 -SQLAlchemy==1.4.49 +SQLAlchemy==1.4.50 sqlparse==0.4.4 tenacity==8.2.3 testcontainers==3.7.1 @@ -140,10 +140,10 @@ threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 typing_extensions==4.8.0 -tzlocal==5.1 +tzlocal==5.2 uritemplate==4.1.1 -urllib3==2.0.7 +urllib3==2.1.0 websocket-client==1.6.4 -wrapt==1.15.0 +wrapt==1.16.0 zipp==3.17.0 -zstandard==0.21.0 +zstandard==0.22.0 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 1785cd75df80..cdd3ecb1994c 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -141,9 +141,21 @@ def cythonize(*args, **kwargs): # [BEAM-8181] pyarrow cannot be installed on 32-bit Windows platforms. if sys.platform == 'win32' and sys.maxsize <= 2**32: pyarrow_dependency = [''] +elif sys.platform == 'win32' or sys.platform == 'cygwin': + # https://github.com/apache/beam/issues/28410 - pyarrow>=13 seeing issues + # on windows with error + # C:\arrow\cpp\src\arrow\filesystem\s3fs.cc:2904: arrow::fs::FinalizeS3 was + # not called even though S3 was initialized. This could lead to a + # segmentation fault at exit. Keep pyarrow<13 until this is resolved. + pyarrow_dependency = [ + 'pyarrow>=3.0.0,<12.0.0', + # NOTE: We can remove this once Beam increases the pyarrow lower bound + # to a version that fixes CVE. + 'pyarrow-hotfix<1' + ] else: pyarrow_dependency = [ - 'pyarrow>=3.0.0,<12.0.0', + 'pyarrow>=3.0.0,<15.0.0', # NOTE(https://github.com/apache/beam/issues/29392): We can remove this # once Beam increases the pyarrow lower bound to a version that fixes CVE. 'pyarrow-hotfix<1' From a7b1f036445efc1c1f608b4c67e0f41dc524232e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Dec 2023 11:51:00 -0500 Subject: [PATCH 018/224] Bump cloud.google.com/go/spanner from 1.51.0 to 1.53.0 in /sdks (#29448) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.51.0 to 1.53.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.51.0...spanner/v1.53.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 3223814d86b6..c8d6591b1606 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.51.0 + cloud.google.com/go/spanner v1.53.0 cloud.google.com/go/storage v1.35.1 github.com/aws/aws-sdk-go-v2 v1.23.4 github.com/aws/aws-sdk-go-v2/config v1.25.8 diff --git a/sdks/go.sum b/sdks/go.sum index b9914ab1b931..182d277b55c4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.51.0 h1:l3exhhsVMKsx1E7Xd1QajYSvHmI1KZoWPW5tRxIIdvQ= -cloud.google.com/go/spanner v1.51.0/go.mod h1:c5KNo5LQ1X5tJwma9rSQZsXNBDNvj4/n8BVc3LNahq0= +cloud.google.com/go/spanner v1.53.0 h1:/NzWQJ1MEhdRcffiutRKbW/AIGVKhcTeivWTDjEyCCo= +cloud.google.com/go/spanner v1.53.0/go.mod h1:liG4iCeLqm5L3fFLU5whFITqP0e0orsAW1uUSrd4rws= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= From c86971158a52d985b37f2736f2c06d2d66772316 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Dec 2023 10:55:30 -0800 Subject: [PATCH 019/224] Bump google.golang.org/api from 0.151.0 to 0.152.0 in /sdks (#29596) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.151.0 to 0.152.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.151.0...v0.152.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 20 ++++++++++---------- sdks/go.sum | 44 ++++++++++++++++++++++---------------------- 2 files changed, 32 insertions(+), 32 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index c8d6591b1606..33c575deed5a 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -55,12 +55,12 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.18.0 - golang.org/x/oauth2 v0.13.0 + golang.org/x/oauth2 v0.14.0 golang.org/x/sync v0.5.0 golang.org/x/sys v0.14.0 golang.org/x/text v0.14.0 - google.golang.org/api v0.151.0 - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b + google.golang.org/api v0.152.0 + google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 @@ -89,15 +89,15 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect - golang.org/x/time v0.4.0 // indirect + golang.org/x/time v0.5.0 // indirect ) require ( - cloud.google.com/go v0.110.8 // indirect - cloud.google.com/go/compute v1.23.1 // indirect + cloud.google.com/go v0.110.10 // indirect + cloud.google.com/go/compute v1.23.3 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.3 // indirect - cloud.google.com/go/longrunning v0.5.2 // indirect + cloud.google.com/go/iam v1.1.5 // indirect + cloud.google.com/go/longrunning v0.5.4 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/andybalholm/brotli v1.0.4 // indirect @@ -178,6 +178,6 @@ require ( golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 182d277b55c4..94ced030dec8 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -8,8 +8,8 @@ cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.110.8 h1:tyNdfIxjzaWctIiLYOTalaLKZ17SI44SKFW26QbOhME= -cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5fk= +cloud.google.com/go v0.110.10 h1:LXy9GEO+timppncPIAZoOj3l58LIU9k+kn48AN7IO3Y= +cloud.google.com/go v0.110.10/go.mod h1:v1OoFqYxiBkUrruItNM3eT4lLByNjxmJSV/xDKJNnic= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -17,20 +17,20 @@ cloud.google.com/go/bigquery v1.57.1 h1:FiULdbbzUxWD0Y4ZGPSVCDLvqRSyCIO6zKV7E2nf cloud.google.com/go/bigquery v1.57.1/go.mod h1:iYzC0tGVWt1jqSzBHqCr3lrRn0u13E8e+AqowBsDgug= 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.1 h1:V97tBoDaZHb6leicZ1G6DLK2BAaZLJ/7+9BB/En3hR0= -cloud.google.com/go/compute v1.23.1/go.mod h1:CqB3xpmPKKt3OJpW2ndFIXnA9A4xAy/F3Xp1ixncW78= +cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= +cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/datacatalog v1.18.1 h1:xJp9mZrc2HPaoxIz3sP9pCmf/impifweQ/yGG9VBfio= +cloud.google.com/go/datacatalog v1.18.3 h1:zmdxP6nOjN5Qb1rtu9h4kbEVwerQ6Oshf+t747QJUew= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.15.0 h1:0P9WcsQeTWjuD1H14JIY7XQscIPQ4Laje8ti96IC5vg= cloud.google.com/go/datastore v1.15.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= -cloud.google.com/go/iam v1.1.3 h1:18tKG7DzydKWUnLjonWcJO6wjSCAtzh4GcRKlH/Hrzc= -cloud.google.com/go/iam v1.1.3/go.mod h1:3khUlaBXfPKKe7huYgEpDn6FtgRyMEqbkvBxrQyY5SE= -cloud.google.com/go/kms v1.15.3 h1:RYsbxTRmk91ydKCzekI2YjryO4c5Y2M80Zwcs9/D/cI= -cloud.google.com/go/longrunning v0.5.2 h1:u+oFqfEwwU7F9dIELigxbe0XVnBAo9wqMuQLA50CZ5k= -cloud.google.com/go/longrunning v0.5.2/go.mod h1:nqo6DQbNV2pXhGDbDMoN2bWz68MjZUzqv2YttZiveCs= +cloud.google.com/go/iam v1.1.5 h1:1jTsCu4bcsNsE4iiqNT5SHwrDRCfRmIaaaVFhRveTJI= +cloud.google.com/go/iam v1.1.5/go.mod h1:rB6P/Ic3mykPbFio+vo7403drjlgvoWfYpJhMXEbzv8= +cloud.google.com/go/kms v1.15.5 h1:pj1sRfut2eRbD9pFRjNnPNg/CzJPuQAzUujMIM1vVeM= +cloud.google.com/go/longrunning v0.5.4 h1:w8xEcbZodnA2BbW6sVirkkoC+1gP8wS57EUUgGS0GVg= +cloud.google.com/go/longrunning v0.5.4/go.mod h1:zqNVncI0BOP8ST6XQD1+VcvuShMmq7+xFSzOL++V0dI= cloud.google.com/go/profiler v0.4.0 h1:ZeRDZbsOBDyRG0OiK0Op1/XWZ3xeLwJc9zjkzczUxyY= cloud.google.com/go/profiler v0.4.0/go.mod h1:RvPlm4dilIr3oJtAOeFQU9Lrt5RoySHSDj4pTd6TWeU= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= @@ -562,8 +562,8 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= -golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= +golang.org/x/oauth2 v0.14.0 h1:P0Vrf/2538nmC0H+pEQ3MNFRRnVR7RlqyVw+bvm26z0= +golang.org/x/oauth2 v0.14.0/go.mod h1:lAtNWgaWfL4cm7j2OV8TxGi9Qb7ECORx8DktCY74OwM= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -634,8 +634,8 @@ golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.4.0 h1:Z81tqI5ddIoXDPvVQ7/7CC9TnLM7ubaFG2qXYd5BbYY= -golang.org/x/time v0.4.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= +golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= @@ -687,8 +687,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.151.0 h1:FhfXLO/NFdJIzQtCqjpysWwqKk8AzGWBUhMIx67cVDU= -google.golang.org/api v0.151.0/go.mod h1:ccy+MJ6nrYFgE3WgRx/AMXOxOmU8Q4hSa+jjibzhxcg= +google.golang.org/api v0.152.0 h1:t0r1vPnfMc260S2Ci+en7kfCZaLOPs5KI0sVV/6jZrY= +google.golang.org/api v0.152.0/go.mod h1:3qNJX5eOmhiWYc67jRA/3GsDw97UFb5ivv7Y2PrriAY= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -714,12 +714,12 @@ google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4 google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 h1:AB/lmRny7e2pLhFEYIbl5qkDAUt2h0ZRO4wGPhZf+ik= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405/go.mod h1:67X1fPuzjcrkymZzZV1vvkFeTn2Rvc6lYF9MYFGCcwE= +google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 h1:wpZ8pe2x1Q3f2KyT5f8oP/fa9rHAKgFPr/HZdNuS+PQ= +google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:J7XzRzVy1+IPwWHZUzoD0IccYZIrXILAQpc+Qy9CMhY= +google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 h1:JpwMPBpFN3uKhdaekDpiNlImDdkUAyiJ6ez/uxGaUSo= +google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:0xJLfVdJqpAPl8tDg1ujOCGzx6LFLttXT5NhllGOXY4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f h1:ultW7fxlIvee4HYrtnaRPon9HpEgFk5zYpmfMgtKB5I= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f/go.mod h1:L9KNLi232K1/xB6f7AlSX692koaRnKaWSR0stBki0Yc= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 70ef5072bf5d0a93d1e195ea5d4383d32b95ba38 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Mon, 4 Dec 2023 15:36:43 -0500 Subject: [PATCH 020/224] Update Java Healthcare Dependency (#29594) * Create HealthcareUtils file with shared resources * revert * Utilize new Package --- .../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 359aeea55a2f..41aed5174c66 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -731,7 +731,7 @@ class BeamModulePlugin implements Plugin { // Keep version consistent with the version in google_cloud_resourcemanager, managed by google_cloud_platform_libraries_bom google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20230806-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", - google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20231003-$google_clients_version", + google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20231101-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20230907-$google_clients_version", From 367e4ecb5e2244662b55546d7207a16441eb33c1 Mon Sep 17 00:00:00 2001 From: Clay Johnson Date: Mon, 4 Dec 2023 15:26:25 -0600 Subject: [PATCH 021/224] Replace usage of custom `gradle-command-action` with `gradle-build-action` (#29480) * Replace usage of custom `gradle-command-action` with `gradle-build-action` * Use `gradle-command-self-hosted-action` in Java Tests --- .github/actions/gradle-command-action | 1 - .../action.yml | 6 +-- .github/workflows/java_tests.yml | 40 +++++++------------ .gitmodules | 3 -- 4 files changed, 18 insertions(+), 32 deletions(-) delete mode 160000 .github/actions/gradle-command-action diff --git a/.github/actions/gradle-command-action b/.github/actions/gradle-command-action deleted file mode 160000 index 90ccf054e6b9..000000000000 --- a/.github/actions/gradle-command-action +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 90ccf054e6b9905f30f98c938bce4c6acd323b6b diff --git a/.github/actions/gradle-command-self-hosted-action/action.yml b/.github/actions/gradle-command-self-hosted-action/action.yml index 906b35169d9d..e2fd768220a3 100644 --- a/.github/actions/gradle-command-self-hosted-action/action.yml +++ b/.github/actions/gradle-command-self-hosted-action/action.yml @@ -35,9 +35,9 @@ runs: - name: Run Gradle Command shell: bash run: | - # Removing settings.xml is a workaround to avoid a decryption issue - # of Beam's gradle-command-action plugin and github's provided - # maven settings.xml file + # This step is a workaround to avoid a decryption issue of Beam's + # net.linguica.gradle.maven.settings plugin and github's provided maven + # settings.xml file if [ -f ~/.m2/settings.xml ]; then rm ~/.m2/settings.xml fi diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index 1132ba1c196b..e5864c4ad934 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -82,16 +82,12 @@ jobs: with: java-version: 8 go-version: 1.21 - - name: Remove default github maven configuration - # This step is a workaround to avoid a decryption issue of Beam's - # net.linguica.gradle.maven.settings plugin and github's provided maven - # settings.xml file - run: rm ~/.m2/settings.xml # :sdks:java:core:test - name: Run :sdks:java:core:test - uses: ./.github/actions/gradle-command-action + uses: ./.github/actions/gradle-command-self-hosted-action with: - arguments: -p sdks/java/core/ test + gradle-command: test + arguments: -p sdks/java/core/ - name: Upload test logs for :sdks:java:core:test uses: actions/upload-artifact@v3 if: always() @@ -100,9 +96,10 @@ jobs: path: sdks/java/core/build/reports/tests/test # :sdks:java:harness:test - name: Run :sdks:java:harness:test - uses: ./.github/actions/gradle-command-action + uses: ./.github/actions/gradle-command-self-hosted-action with: - arguments: -p sdks/java/harness/ test + gradle-command: test + arguments: -p sdks/java/harness/ if: always() - name: Upload test logs for :sdks:java:harness:test uses: actions/upload-artifact@v3 @@ -112,9 +109,10 @@ jobs: path: sdks/java/harness/build/reports/tests/test # :runners:core-java:test - name: Run :runners:core-java:test - uses: ./.github/actions/gradle-command-action + uses: ./.github/actions/gradle-command-self-hosted-action with: - arguments: -p runners/core-java/ test + gradle-command: test + arguments: -p runners/core-java/ if: always() - name: Upload test logs for :runners:core-java:test uses: actions/upload-artifact@v3 @@ -141,15 +139,11 @@ jobs: with: java-version: 8 go-version: 1.21 - - name: Remove default github maven configuration - # This step is a workaround to avoid a decryption issue of Beam's - # net.linguica.gradle.maven.settings plugin and github's provided maven - # settings.xml file - run: rm ~/.m2/settings.xml - name: Run WordCount Unix - uses: ./.github/actions/gradle-command-action + uses: ./.github/actions/gradle-command-self-hosted-action with: - arguments: -p examples/ integrationTest --tests org.apache.beam.examples.WordCountIT + gradle-command: integrationTest + arguments: -p examples/ --tests org.apache.beam.examples.WordCountIT -DintegrationTestRunner=direct -DintegrationTestPipelineOptions=["--runner=DirectRunner","--tempRoot=./tmp"] - name: Upload test logs @@ -191,15 +185,11 @@ jobs: service_account_key: ${{ secrets.GCP_SA_KEY }} project_id: ${{ secrets.GCP_PROJECT_ID }} export_default_credentials: true - - name: Remove default github maven configuration - # This step is a workaround to avoid a decryption issue of Beam's - # gradle-command-action plugin and github's provided maven - # settings.xml file - run: rm ~/.m2/settings.xml - name: Run WordCount - uses: ./.github/actions/gradle-command-action + uses: ./.github/actions/gradle-command-self-hosted-action with: - arguments: -p examples/ integrationTest --tests org.apache.beam.examples.WordCountIT + gradle-command: integrationTest + arguments: -p examples/ --tests org.apache.beam.examples.WordCountIT -DintegrationTestPipelineOptions=["--runner=DataflowRunner","--project=${{ secrets.GCP_PROJECT_ID }}","--tempRoot=gs://${{ secrets.GCP_TESTING_BUCKET }}/tmp/"] -DintegrationTestRunner=dataflow - name: Upload test logs diff --git a/.gitmodules b/.gitmodules index fa6e30a8850a..3a6406c405f3 100644 --- a/.gitmodules +++ b/.gitmodules @@ -7,6 +7,3 @@ [submodule ".github/actions/github-push-action"] path = .github/actions/github-push-action url = https://github.com/ad-m/github-push-action -[submodule ".github/actions/gradle-command-action"] - path = .github/actions/gradle-command-action - url = https://github.com/eskatos/gradle-command-action From c9c89fe7d5f88b004635bd941725f6213c7d1cdc Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 4 Dec 2023 21:57:49 +0000 Subject: [PATCH 022/224] Use UUIDs instead of object hashes to avoid collisions (#29542) * Add uuid fix test fix test Fix test * Change class name resembling its functionality * Add PID to the unique string * Change the unique id to be bytes * remove decode * Replace hash computation with a combined uuid. Resulting key should have the same length. * Mark internal classes as such. * misc fixup. * raise RuntimeError when more than 1 element in observed while CoGroupByKey * Add MLTransform dropping elements to known issues * Remove internal use comments since it is now evident from naming. * Remove references to hash * Remove references to hash * Remove references to hash * Remove references to hash * Edit for clarity * Clarify helper code. * yapf --------- Co-authored-by: Valentyn Tymofieiev Co-authored-by: tvalentyn --- CHANGES.md | 9 ++ .../elementwise/mltransform_test.py | 1 + .../apache_beam/ml/transforms/handlers.py | 119 +++++++++--------- .../ml/transforms/handlers_test.py | 46 +++++++ 4 files changed, 116 insertions(+), 59 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index f9ee5d289117..847532e85562 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -86,6 +86,8 @@ ## Bugfixes * Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Fixed MLTransform when duplicated elements are dropped in the output PCollection.([#29600](https://github.com/apache/beam/issues/29600)) + ## Security Fixes * Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). @@ -134,6 +136,10 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed [CVE-2023-39325](https://www.cve.org/CVERecord?id=CVE-2023-39325) (Java/Python/Go) ([#29118](https://github.com/apache/beam/issues/29118)). * Mitigated [CVE-2023-47248](https://nvd.nist.gov/vuln/detail/CVE-2023-47248) (Python) [#29392](https://github.com/apache/beam/issues/29392). +## Known issues + +* MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). + # [2.51.0] - 2023-10-03 ## New Features / Improvements @@ -168,6 +174,8 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246). * Python pipelines using BigQuery Storage Read API might need to pin `fastavro` dependency to 1.8.3 or earlier on some runners that don't use Beam Docker containers: [#28811](https://github.com/apache/beam/issues/28811) +* MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). + # [2.50.0] - 2023-08-30 @@ -229,6 +237,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Beam Python containers rely on a version of Debian/aom that has several security vulnerabilities: [CVE-2021-30474](https://nvd.nist.gov/vuln/detail/CVE-2021-30474), [CVE-2021-30475](https://nvd.nist.gov/vuln/detail/CVE-2021-30475), [CVE-2021-30473](https://nvd.nist.gov/vuln/detail/CVE-2021-30473), [CVE-2020-36133](https://nvd.nist.gov/vuln/detail/CVE-2020-36133), [CVE-2020-36131](https://nvd.nist.gov/vuln/detail/CVE-2020-36131), [CVE-2020-36130](https://nvd.nist.gov/vuln/detail/CVE-2020-36130), and [CVE-2020-36135](https://nvd.nist.gov/vuln/detail/CVE-2020-36135) * Python SDK's cross-language Bigtable sink mishandles records that don't have an explicit timestamp set: [#28632](https://github.com/apache/beam/issues/28632). To avoid this issue, set explicit timestamps for all records before writing to Bigtable. * Python SDK worker start-up logs, particularly PIP dependency installations, that are not logged at warning or higher are suppressed. This suppression is reverted in 2.51.0. +* MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). # [2.49.0] - 2023-07-17 diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py index 0db10718295b..261b480b1083 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/mltransform_test.py @@ -57,6 +57,7 @@ def check_mltransform_compute_and_apply_vocabulary_with_scalar(): Row(x=array([4])) Row(x=array([1])) Row(x=array([0])) +Row(x=array([0])) Row(x=array([2])) Row(x=array([3])) [END mltransform_compute_and_apply_vocabulary_with_scalar] '''.splitlines( diff --git a/sdks/python/apache_beam/ml/transforms/handlers.py b/sdks/python/apache_beam/ml/transforms/handlers.py index 8695d5146efa..e7d4f52ded85 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers.py +++ b/sdks/python/apache_beam/ml/transforms/handlers.py @@ -17,9 +17,9 @@ # pytype: skip-file import collections -import hashlib import os import typing +import uuid from typing import Dict from typing import List from typing import Optional @@ -49,6 +49,8 @@ 'TFTProcessHandler', ] +_ID_COLUMN = 'tmp_uuid' # Name for a temporary column. + RAW_DATA_METADATA_DIR = 'raw_data_metadata' SCHEMA_FILE = 'schema.pbtxt' # tensorflow transform doesn't support the types other than tf.int64, @@ -80,12 +82,12 @@ tft_process_handler_output_type = typing.Union[beam.Row, Dict[str, np.ndarray]] -class ConvertScalarValuesToListValues(beam.DoFn): +class _ConvertScalarValuesToListValues(beam.DoFn): def process( self, element, ): - hash_key, element = element + id, element = element new_dict = {} for key, value in element.items(): if isinstance(value, @@ -93,10 +95,10 @@ def process( new_dict[key] = [value] else: new_dict[key] = value - yield (hash_key, new_dict) + yield (id, new_dict) -class ConvertNamedTupleToDict( +class _ConvertNamedTupleToDict( beam.PTransform[beam.PCollection[typing.Union[beam.Row, typing.NamedTuple]], beam.PCollection[Dict[str, common_types.InstanceDictType]]]): @@ -121,76 +123,75 @@ def expand( return pcoll | beam.Map(lambda x: x._asdict()) -class ComputeAndAttachHashKey(beam.DoFn): +class _ComputeAndAttachUniqueID(beam.DoFn): """ - Computes and attaches a hash key to the element. - Only for internal use. No backwards compatibility guarantees. + Computes and attaches a unique id to each element in the PCollection. """ def process(self, element): - hash_object = hashlib.sha256() - for _, value in element.items(): - # handle the case where value is a list or numpy array - if isinstance(value, (list, np.ndarray)): - hash_object.update(str(list(value)).encode()) - else: # assume value is a primitive that can be turned into str - hash_object.update(str(value).encode()) - yield (hash_object.hexdigest(), element) + # UUID1 includes machine-specific bits and has a counter. As long as not too + # many are generated at the same time, they should be unique. + # UUID4 generation should be unique in practice as long as underlying random + # number generation is not compromised. + # A combintation of both should avoid the anecdotal pitfalls where + # replacing one with the other has helped some users. + # UUID collision will result in data loss, but we can detect that and fail. + + # TODO(https://github.com/apache/beam/issues/29593): Evaluate MLTransform + # implementation without CoGBK. + unique_key = uuid.uuid1().bytes + uuid.uuid4().bytes + yield (unique_key, element) -class GetMissingColumnsPColl(beam.DoFn): +class _GetMissingColumns(beam.DoFn): """ Returns data containing only the columns that are not present in the schema. This is needed since TFT only outputs columns that are transformed by any of the data processing transforms. - - Only for internal use. No backwards compatibility guarantees. """ def __init__(self, existing_columns): self.existing_columns = existing_columns def process(self, element): - new_dict = {} - hash_key, element = element - for key, value in element.items(): - if key not in self.existing_columns: - new_dict[key] = value - yield (hash_key, new_dict) + id, row_dict = element + new_dict = { + k: v + for k, v in row_dict.items() if k not in self.existing_columns + } + yield (id, new_dict) -class MakeHashKeyAsColumn(beam.DoFn): +class _MakeIdAsColumn(beam.DoFn): """ - Extracts the hash key from the element and adds it as a column. - - Only for internal use. No backwards compatibility guarantees. + Extracts the id from the element and adds it as a column instead. """ def process(self, element): - hash_key, element = element - element['hash_key'] = hash_key + id, element = element + element[_ID_COLUMN] = id yield element -class ExtractHashAndKeyPColl(beam.DoFn): +class _ExtractIdAndKeyPColl(beam.DoFn): """ - Extracts the hash key and return hashkey and element as a tuple. - - Only for internal use. No backwards compatibility guarantees. + Extracts the id and return id and element as a tuple. """ def process(self, element): - hashkey = element['hash_key'][0] - del element['hash_key'] - yield (hashkey.decode('utf-8'), element) + id = element[_ID_COLUMN][0] + del element[_ID_COLUMN] + yield (id, element) -class MergeDicts(beam.DoFn): +class _MergeDicts(beam.DoFn): """ - Merges the dictionaries in the PCollection. - - Only for internal use. No backwards compatibility guarantees. + Merges processed and unprocessed columns from CoGBK result into a single row. """ def process(self, element): - _, element = element + unused_row_id, row_dicts_tuple = element new_dict = {} - for d in element: + for d in row_dicts_tuple: + # After CoGBK, dicts with processed and unprocessed portions of each row + # are wrapped in 1-element lists, since all rows have a unique id. + # Assertion could fail due to UUID collision. + assert len(d) == 1, f"Expected 1 element, got: {len(d)}." new_dict.update(d[0]) yield new_dict @@ -323,7 +324,7 @@ def _get_raw_data_feature_spec_per_column( def get_raw_data_metadata( self, input_types: Dict[str, type]) -> dataset_metadata.DatasetMetadata: raw_data_feature_spec = self.get_raw_data_feature_spec(input_types) - raw_data_feature_spec['hash_key'] = tf.io.VarLenFeature(dtype=tf.string) + raw_data_feature_spec[_ID_COLUMN] = tf.io.VarLenFeature(dtype=tf.string) return self.convert_raw_data_feature_spec_to_dataset_metadata( raw_data_feature_spec) @@ -417,14 +418,14 @@ def process_data( # convert Row or NamedTuple to Dict raw_data = ( raw_data - | ConvertNamedTupleToDict().with_output_types( + | _ConvertNamedTupleToDict().with_output_types( Dict[str, typing.Union[tuple(column_type_mapping.values())]])) # type: ignore # AnalyzeAndTransformDataset raise type hint since this is # schema'd PCollection and the current output type would be a # custom type(NamedTuple) or a beam.Row type. else: column_type_mapping = self._map_column_names_to_types_from_transforms() - # Add hash key so TFT can output hash_key as output but as a no-op. + # Add id so TFT can output id as output but as a no-op. raw_data_metadata = self.get_raw_data_metadata( input_types=column_type_mapping) # Write untransformed metadata to a file so that it can be re-used @@ -445,21 +446,21 @@ def process_data( raw_data_metadata = metadata_io.read_metadata( os.path.join(self.artifact_location, RAW_DATA_METADATA_DIR)) - keyed_raw_data = (raw_data | beam.ParDo(ComputeAndAttachHashKey())) + keyed_raw_data = (raw_data | beam.ParDo(_ComputeAndAttachUniqueID())) feature_set = [feature.name for feature in raw_data_metadata.schema.feature] - columns_not_in_schema_with_hash = ( + keyed_columns_not_in_schema = ( keyed_raw_data - | beam.ParDo(GetMissingColumnsPColl(feature_set))) + | beam.ParDo(_GetMissingColumns(feature_set))) # To maintain consistency by outputting numpy array all the time, # whether a scalar value or list or np array is passed as input, # we will convert scalar values to list values and TFT will ouput # numpy array all the time. keyed_raw_data = keyed_raw_data | beam.ParDo( - ConvertScalarValuesToListValues()) + _ConvertScalarValuesToListValues()) - raw_data_list = (keyed_raw_data | beam.ParDo(MakeHashKeyAsColumn())) + raw_data_list = (keyed_raw_data | beam.ParDo(_MakeIdAsColumn())) with tft_beam.Context(temp_dir=self.artifact_location): data = (raw_data_list, raw_data_metadata) @@ -467,7 +468,7 @@ def process_data( transform_fn = ( data | "AnalyzeDataset" >> tft_beam.AnalyzeDataset(self.process_data_fn)) - # TODO: Remove the 'hash_key' column from the transformed + # TODO: Remove the 'id' column from the transformed # dataset schema generated by TFT. self.write_transform_artifacts(transform_fn, self.artifact_location) else: @@ -490,7 +491,7 @@ def process_data( # So we will use a RowTypeConstraint to create a schema'd PCollection. # this is needed since new columns are included in the # transformed_dataset. - del self.transformed_schema['hash_key'] + del self.transformed_schema[_ID_COLUMN] row_type = RowTypeConstraint.from_fields( list(self.transformed_schema.items())) @@ -498,17 +499,17 @@ def process_data( # is not transformed by any of the transforms, then the output will # not have that column. So we will join the missing columns from the # raw_data to the transformed_dataset. - transformed_dataset = ( - transformed_dataset | beam.ParDo(ExtractHashAndKeyPColl())) + keyed_transformed_dataset = ( + transformed_dataset | beam.ParDo(_ExtractIdAndKeyPColl())) # The grouping is needed here since tensorflow transform only outputs # columns that are transformed by any of the transforms. So we will # join the missing columns from the raw_data to the transformed_dataset - # using the hash key. + # using the id. transformed_dataset = ( - (transformed_dataset, columns_not_in_schema_with_hash) + (keyed_transformed_dataset, keyed_columns_not_in_schema) | beam.CoGroupByKey() - | beam.ParDo(MergeDicts())) + | beam.ParDo(_MergeDicts())) # The schema only contains the columns that are transformed. transformed_dataset = ( diff --git a/sdks/python/apache_beam/ml/transforms/handlers_test.py b/sdks/python/apache_beam/ml/transforms/handlers_test.py index 327c8c76c0e9..d67d8ec3e705 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers_test.py +++ b/sdks/python/apache_beam/ml/transforms/handlers_test.py @@ -569,6 +569,52 @@ def test_consume_mode_with_extra_columns_in_the_input(self): equal_to(expected_test_data_z, equals_fn=np.array_equal), label='unused column: z') + def test_handler_with_same_input_elements(self): + with beam.Pipeline() as p: + data = [ + { + 'x': 'I' + }, + { + 'x': 'love' + }, + { + 'x': 'Beam' + }, + { + 'x': 'Beam' + }, + { + 'x': 'is' + }, + { + 'x': 'awesome' + }, + ] + raw_data = (p | beam.Create(data)) + process_handler = handlers.TFTProcessHandler( + transforms=[tft.ComputeAndApplyVocabulary(columns=['x'])], + artifact_location=self.artifact_location, + ) + transformed_data = process_handler.process_data(raw_data) + + expected_data = [ + beam.Row(x=np.array([4])), + beam.Row(x=np.array([1])), + beam.Row(x=np.array([0])), + beam.Row(x=np.array([0])), + beam.Row(x=np.array([2])), + beam.Row(x=np.array([3])), + ] + + expected_data_x = [row.x for row in expected_data] + actual_data_x = transformed_data | beam.Map(lambda x: x.x) + + assert_that( + actual_data_x, + equal_to(expected_data_x, equals_fn=np.array_equal), + label='transformed data') + if __name__ == '__main__': unittest.main() From 6973255b3d078a5181c8e3d43d2adc8ade6cb246 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 10:19:29 -0500 Subject: [PATCH 023/224] Bump cloud.google.com/go/spanner from 1.53.0 to 1.53.1 in /sdks (#29610) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.53.0 to 1.53.1. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.53.0...spanner/v1.53.1) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 33c575deed5a..34d633febc49 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.53.0 + cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.35.1 github.com/aws/aws-sdk-go-v2 v1.23.4 github.com/aws/aws-sdk-go-v2/config v1.25.8 diff --git a/sdks/go.sum b/sdks/go.sum index 94ced030dec8..22955e00e751 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.53.0 h1:/NzWQJ1MEhdRcffiutRKbW/AIGVKhcTeivWTDjEyCCo= -cloud.google.com/go/spanner v1.53.0/go.mod h1:liG4iCeLqm5L3fFLU5whFITqP0e0orsAW1uUSrd4rws= +cloud.google.com/go/spanner v1.53.1 h1:xNmE0SXMSxNBuk7lRZ5G/S+A49X91zkSTt7Jn5Ptlvw= +cloud.google.com/go/spanner v1.53.1/go.mod h1:liG4iCeLqm5L3fFLU5whFITqP0e0orsAW1uUSrd4rws= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= From c53f0af09f983203c9ccf9c09327010436ae0b8c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 10:20:07 -0500 Subject: [PATCH 024/224] Bump golang.org/x/net from 0.18.0 to 0.19.0 in /sdks (#29574) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.18.0 to 0.19.0. - [Commits](https://github.com/golang/net/compare/v0.18.0...v0.19.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 34d633febc49..a914cd6793de 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -54,10 +54,10 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 - golang.org/x/net v0.18.0 + golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.14.0 golang.org/x/sync v0.5.0 - golang.org/x/sys v0.14.0 + golang.org/x/sys v0.15.0 golang.org/x/text v0.14.0 google.golang.org/api v0.152.0 google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 @@ -173,7 +173,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.15.0 // indirect + golang.org/x/crypto v0.16.0 // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 22955e00e751..e868d783d4a0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -495,8 +495,8 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.15.0 h1:frVn1TEaCEaZcn3Tmd7Y2b5KKPaZ+I32Q2OA3kYp5TA= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= +golang.org/x/crypto v0.16.0 h1:mMMrFzRSCF0GvB7Ne27XVtVAaXLrPmgPC7/v0tkwHaY= +golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -555,8 +555,8 @@ golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.18.0 h1:mIYleuAkSbHh0tCv7RvjL3F6ZVbLjq4+R7zbOn3Kokg= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= +golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= +golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -616,8 +616,8 @@ golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 17f3f0532c2e3c5e26316b56a6ae275306ee9729 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 5 Dec 2023 07:23:41 -0800 Subject: [PATCH 025/224] [#29568] Use gcr clone of images by default on Dataflow. (#29608) * [#29568] Override default image on dataflow. * Validate in test. --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go/pkg/beam/runners/dataflow/dataflow.go | 13 ++++++++++++- sdks/go/pkg/beam/runners/dataflow/dataflow_test.go | 10 ++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go index 7b43ba78f054..ca701979497a 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go @@ -35,6 +35,7 @@ import ( "cloud.google.com/go/storage" "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/pipelinex" @@ -442,7 +443,17 @@ func getContainerImage(ctx context.Context) string { if *image != "" { return *image } - return jobopts.GetEnvironmentConfig(ctx) + envConfig := jobopts.GetEnvironmentConfig(ctx) + if envConfig == core.DefaultDockerImage { + // It's possible the user set the image exactly manually, but unlikely. + // Prefer using the gcr.io image by default. + // Note: This doesn't change the dev experience, which requires a user + // to have a dev image. + // However, RC versions should automatically be picked up, since + // they are never tagged the RC number, just the main version. + return "gcr.io/cloud-dataflow/v1beta3/beam_go_sdk:" + core.SdkVersion + } + return envConfig } panic(fmt.Sprintf("Unsupported environment %v", urn)) } diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go index 663695f00c8e..2e0f28f263d6 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go @@ -22,6 +22,7 @@ import ( "sort" "testing" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core" "github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts" "github.com/apache/beam/sdks/v2/go/pkg/beam/options/jobopts" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/dataflow/dataflowlib" @@ -313,6 +314,15 @@ func TestGetJobOptions_DockerNoImage(t *testing.T) { } } +func TestGetJobOptions_DockerGCROverride(t *testing.T) { + resetGlobals() + *jobopts.EnvironmentType = "docker" + + if got, want := getContainerImage(context.Background()), "gcr.io/cloud-dataflow/v1beta3/beam_go_sdk:"+core.SdkVersion; got != want { + t.Fatalf("getContainerImage() = %q, want %q", got, want) + } +} + func TestGetJobOptions_TransformMapping(t *testing.T) { resetGlobals() *stagingLocation = "gs://testStagingLocation" From 4b4cc5d8088b4e81150b938d2f6537322d86de93 Mon Sep 17 00:00:00 2001 From: Clay Johnson Date: Tue, 5 Dec 2023 11:49:15 -0600 Subject: [PATCH 026/224] Esacpe strings in integrationTestPipelineOptions in Java Tests --- .github/workflows/java_tests.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index e5864c4ad934..30808563aaed 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -145,7 +145,7 @@ jobs: gradle-command: integrationTest arguments: -p examples/ --tests org.apache.beam.examples.WordCountIT -DintegrationTestRunner=direct - -DintegrationTestPipelineOptions=["--runner=DirectRunner","--tempRoot=./tmp"] + -DintegrationTestPipelineOptions=[\"--runner=DirectRunner\",\"--tempRoot=./tmp\"] - name: Upload test logs uses: actions/upload-artifact@v3 if: always() @@ -190,7 +190,7 @@ jobs: with: gradle-command: integrationTest arguments: -p examples/ --tests org.apache.beam.examples.WordCountIT - -DintegrationTestPipelineOptions=["--runner=DataflowRunner","--project=${{ secrets.GCP_PROJECT_ID }}","--tempRoot=gs://${{ secrets.GCP_TESTING_BUCKET }}/tmp/"] + -DintegrationTestPipelineOptions=[\"--runner=DataflowRunner\",\"--project=${{ secrets.GCP_PROJECT_ID }}\",\"--tempRoot=gs://${{ secrets.GCP_TESTING_BUCKET }}/tmp/\"] -DintegrationTestRunner=dataflow - name: Upload test logs uses: actions/upload-artifact@v3 From 59930cd2584a8f9d63124be8650aba76a9678464 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 10:22:34 -0800 Subject: [PATCH 027/224] Bump github.com/aws/aws-sdk-go-v2 from 1.23.4 to 1.23.5 in /sdks (#29611) Bumps [github.com/aws/aws-sdk-go-v2](https://github.com/aws/aws-sdk-go-v2) from 1.23.4 to 1.23.5. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.23.4...v1.23.5) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index a914cd6793de..b55c0bb474b8 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,7 +30,7 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.35.1 - github.com/aws/aws-sdk-go-v2 v1.23.4 + github.com/aws/aws-sdk-go-v2 v1.23.5 github.com/aws/aws-sdk-go-v2/config v1.25.8 github.com/aws/aws-sdk-go-v2/credentials v1.16.8 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 diff --git a/sdks/go.sum b/sdks/go.sum index e868d783d4a0..ed4921bfacc5 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,8 +81,8 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.23.4 h1:2P20ZjH0ouSAu/6yZep8oCmTReathLuEu6dwoqEgjts= -github.com/aws/aws-sdk-go-v2 v1.23.4/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= +github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= +github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= From 169dce40670fe22e3e257bb552084984f766ea70 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Tue, 5 Dec 2023 14:09:07 -0500 Subject: [PATCH 028/224] Initial RC Testing Guide Documentation (#29595) * Create HealthcareUtils file with shared resources * revert * Initial RC Testing Guide * Updates to testing guide. * update email * update email * Update contributor-docs/release-guide.md Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- contributor-docs/rc-testing-guide.md | 60 ++++++++++++++++++++++++++++ contributor-docs/release-guide.md | 8 ++-- 2 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 contributor-docs/rc-testing-guide.md diff --git a/contributor-docs/rc-testing-guide.md b/contributor-docs/rc-testing-guide.md new file mode 100644 index 000000000000..0ac932711206 --- /dev/null +++ b/contributor-docs/rc-testing-guide.md @@ -0,0 +1,60 @@ + + +# Release Candidate (RC) Testing Guide + +This guide is meant for anybody who is interested in testing Beam Release Candidates against downstream projects. Note +that one need not have any status on the Apache Beam project (eg. PMC Member, Committer) to vote; all are welcome. +Please subscribe to the [dev list](https://lists.apache.org/list.html?dev@beam.apache.org), and vote on the RC Vote email thread. + + +## RC Testing Objectives + +The RC testing process aims to: + + - Test new release candidates against existing code bases utilizing Apache Beam, to ensure there are no unexpected behaviors downstream. + - Incorporate a breadth of perspectives (including validation on multiple SDKs and multiple runners), before releasing a new version. + - Allow Beam Contributors to dogfood their changes and verify that they work as intended. + + +## Beam Release process overview +- For a comprehensive overview on the Beam release process, please take a look at our [release guide](https://github.com/apache/beam/blob/master/contributor-docs/release-guide.md). +- Note that release candidate votes will be open for 72 hours after the voting email is sent. + + +## Ideas for Python SDK Validators + +_Note: Do the following in a dev-like environment._ +- If you are a Python SDK user that utilizes notebooks (eg. Jupyter Notebooks, or Colab Notebooks), change `pip install` +to point to the new RC (e.g. `pip install apache_beam[gcp]==2.52.0rc1`). Re-execute the workflow to ensure everything +works as intended. +- If your workflow utilizes [Dataflow Templates](https://github.com/GoogleCloudPlatform/DataflowTemplates), or another way of launching your job, modify your `requirements.txt` file, `setup.py` file, or `DockerFile` to point to the new Beam RC. +- _Tip_: Run your pipeline both against Direct Runner, and another runner of your choice by modifying your job's `PipelineOptions`. + + +## Ideas for Java SDK Validators +_Note: Do the following in a dev-like environment._ +- If you are a Java SDK user that utilizes Maven or Gradle in your workflow, in your `pom.xml` file, modify the `beam.version` to reflect the newest RC, and modify the `` tag under ``, to point to the repository given in the vote email. [Example Changes](https://github.com/GoogleCloudPlatform/DataflowTemplates/pull/1090/files). + + +## Ideas for Go SDK Validators +_Note: Do the following in a dev-like environment._ +- If you utilize the Go SDK, use `go-get` to use the desired RC, for example, `go get -d github.com/apache/beam/sdks/v2@v2.xx.0-RC1` +- Utilize the `--environment_config`, to point to the new release, for example, `--environment_config=apache/beam_go_sdk:2.xx.0rc1` + + +## After validation + +- Reply to [dev list](https://lists.apache.org/list.html?dev@beam.apache.org) vote thread with your [vote](https://www.apache.org/foundation/voting.html) and an explanation of the use case you tested. +- [Optional]: If your use case can be well represented by a test, consider contributing a test to Beam! However, note that the of the value of validation is manual testing outside of the Beam CI workflow, so it is still recommended (and highly encouraged!) to validate these test cases in future releases. \ No newline at end of file diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index faa8ad5927cf..e00ecd694336 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -817,7 +817,7 @@ template; please adjust as you see fit. Reviewers are encouraged to test their own use cases with the release candidate, and vote +1 if no issues are found. Only PMC member votes will count towards the final vote, but votes from all community members is encouraged and helpful for finding regressions; you can either test your own - use cases or use cases from the validation sheet [10]. + use cases [13] or use cases from the validation sheet [10]. The complete staging area is available for your review, which includes: * GitHub Release notes [1], @@ -833,7 +833,7 @@ template; please adjust as you see fit. The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes. - For guidelines on how to try the release in your projects, check out our blog post at https://beam.apache.org/blog/validate-beam-release/. + For guidelines on how to try the release in your projects, check out our RC testing guide [13]. Thanks, Release Manager @@ -850,6 +850,7 @@ template; please adjust as you see fit. [10] https://docs.google.com/spreadsheets/d/1qk-N5vjXvbcEk68GjbkSZTR8AGqyNUM-oLFo_ZXBpJw/edit#gid=... [11] https://hub.docker.com/search?q=apache%2Fbeam&type=image [12] https://github.com/apache/beam/pull/... + [13] https://github.com/apache/beam/blob/master/contributor-docs/rc-testing-guide.md If there are any issues found in the release candidate, reply on the vote thread to cancel the vote. There’s no need to wait 72 hours. Go back to @@ -860,7 +861,8 @@ pull request, just correct it on the spot and the vote can continue as-is. ### Run validation tests The community is responsible for performing validation, but as release manager -you are expected to contribute as well. +you are expected to contribute as well. Please see the [RC Testing Guide](https://github.com/apache/beam/blob/master/contributor-docs/rc-testing-guide.md) +for ideas on helping validate testing on downstream projects. Before accepting an RC, as a community we try to exercise most (if not all) of the tests listed in this From eedbfe1fce4ce4bb1a7fad41e53b4c91e5ff6dc0 Mon Sep 17 00:00:00 2001 From: Jacob Tomlinson Date: Tue, 5 Dec 2023 19:30:02 +0000 Subject: [PATCH 029/224] Dask runner: Use bag sqrt scaling (#29366) * Ensure Dask Runner scales beyond 199 workers * Add copyright notice * Fix linting * Revert copyright change --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index c4aac7f2111f..d4d58879b7fe 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -24,6 +24,7 @@ """ import abc import dataclasses +import math import typing as t import apache_beam @@ -59,7 +60,10 @@ def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.transform) items = original_transform.values - return db.from_sequence(items) + return db.from_sequence( + items, + partition_size=max( + 1, math.ceil(math.sqrt(len(items)) / math.sqrt(100)))) class ParDo(DaskBagOp): From 68bd67fb084e7e3064a466277fb1362fab67cb3d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 14:38:02 -0500 Subject: [PATCH 030/224] Bump go.mongodb.org/mongo-driver from 1.12.1 to 1.13.0 in /sdks (#29343) Bumps [go.mongodb.org/mongo-driver](https://github.com/mongodb/mongo-go-driver) from 1.12.1 to 1.13.0. - [Release notes](https://github.com/mongodb/mongo-go-driver/releases) - [Commits](https://github.com/mongodb/mongo-go-driver/compare/v1.12.1...v1.13.0) --- updated-dependencies: - dependency-name: go.mongodb.org/mongo-driver 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 b55c0bb474b8..33e3e7507cb5 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/tetratelabs/wazero v1.5.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c - go.mongodb.org/mongo-driver v1.12.1 + go.mongodb.org/mongo-driver v1.13.0 golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.14.0 golang.org/x/sync v0.5.0 diff --git a/sdks/go.sum b/sdks/go.sum index ed4921bfacc5..0f61b39482a4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -477,8 +477,8 @@ github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.mongodb.org/mongo-driver v1.12.1 h1:nLkghSU8fQNaK7oUmDhQFsnrtcoNy7Z6LVFKsEecqgE= -go.mongodb.org/mongo-driver v1.12.1/go.mod h1:/rGBTebI3XYboVmgz+Wv3Bcbl3aD0QF9zl6kDDw18rQ= +go.mongodb.org/mongo-driver v1.13.0 h1:67DgFFjYOCMWdtTEmKFpV3ffWlFnh+CYZ8ZS/tXWUfY= +go.mongodb.org/mongo-driver v1.13.0/go.mod h1:/rGBTebI3XYboVmgz+Wv3Bcbl3aD0QF9zl6kDDw18rQ= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= From 70e41f4ebef55667943616f322708be950e91ddf Mon Sep 17 00:00:00 2001 From: Niel Markwick Date: Tue, 5 Dec 2023 20:59:40 +0100 Subject: [PATCH 031/224] Add docs and warning about using SpannerIO.Read in streaming pipelines (#29601) * Add docs and warning about using SpannerIO.Read in streaming. Add more documentation around SpannerIO.Read and .ReadAll explaining PartitionedRead API, batching, and how it should not be used for unbounded reads in Streaming pipelines. Add a warning if SpannerIO.ReadAll is applied to an unbounded input. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 87 ++++++++++++++++--- 1 file changed, 76 insertions(+), 11 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 786fa91f5582..b6ec8097a5fe 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -142,12 +142,15 @@ * *

    Reading from Cloud Spanner

    * - *

    To read from Cloud Spanner, apply {@link Read} transformation. It will return a {@link - * PCollection} of {@link Struct Structs}, where each element represents an individual row returned - * from the read operation. Both Query and Read APIs are supported. See more information about Bulk reading of a single query or table + * + *

    To perform a single read from Cloud Spanner, construct a {@link Read} transform using {@link + * SpannerIO#read() SpannerIO.read()}. It will return a {@link PCollection} of {@link Struct + * Structs}, where each element represents an individual row returned from the read operation. Both + * Query and Read APIs are supported. See more information about reading from Cloud Spanner * - *

    To execute a query, specify a {@link Read#withQuery(Statement)} or {@link + *

    To execute a Query, specify a {@link Read#withQuery(Statement)} or {@link * Read#withQuery(String)} during the construction of the transform. * *

    {@code
    @@ -158,8 +161,17 @@
      *         .withQuery("SELECT id, name, email FROM users"));
      * }
    * - *

    To use the Read API, specify a {@link Read#withTable(String) table name} and a {@link - * Read#withColumns(List) list of columns}. + *

    Reads by default use the PartitionQuery API + * which enforces some limitations on the type of queries that can be used so that the data can be + * read in parallel. If the query is not supported by the PartitionQuery API, then you can specify a + * non-partitioned read by setting {@link Read#withBatching(boolean) withBatching(false)}. If the + * amount of data being read by a non-partitioned read is very large, it may be useful to add a + * {@link Reshuffle#viaRandomKey()} transform on the output so that the downstream transforms can + * run in parallel. + * + *

    To read an entire Table, use {@link Read#withTable(String)} and optionally + * specify a {@link Read#withColumns(List) list of columns}. * *

    {@code
      * PCollection rows = p.apply(
    @@ -170,13 +182,26 @@
      *        .withColumns("id", "name", "email"));
      * }
    * - *

    To optimally read using index, specify the index name using {@link Read#withIndex}. + *

    To read using an Index, specify the index name using {@link + * Read#withIndex(String)}. + * + *

    {@code
    + * PCollection rows = p.apply(
    + *    SpannerIO.read()
    + *        .withInstanceId(instanceId)
    + *        .withDatabaseId(dbId)
    + *        .withTable("users")
    + *        .withIndex("users_by_name")
    + *        .withColumns("id", "name", "email"));
    + * }
    + * + *

    Read consistency

    * *

    The transform is guaranteed to be executed on a consistent snapshot of data, utilizing the * power of read only transactions. Staleness of data can be controlled using {@link * Read#withTimestampBound} or {@link Read#withTimestamp(Timestamp)} methods. Read more about transactions in - * Cloud Spanner. + * href="https://cloud.google.com/spanner/docs/transactions#read-only_transactions">Read more + * about transactions in Cloud Spanner. * *

    It is possible to read several {@link PCollection PCollections} within a single transaction. * Apply {@link SpannerIO#createTransaction()} transform, that lazily creates a transaction. The @@ -204,6 +229,29 @@ * .withTransaction(tx)); * } * + *

    Bulk reading of multiple queries or tables

    + * + * You can perform multiple consistent reads on a set of tables or using a set of queries by + * constructing a {@link ReadAll} transform using {@link SpannerIO#readAll() SpannerIO.readAll()}. + * This transform takes a {@link PCollection} of {@link ReadOperation} elements, and performs the + * partitioned read on each of them using the same Read Only Transaction for consistent results. + * + *

    Note that this transform should not be used in Streaming pipelines. This is + * because the same Read Only Transaction, which is created once when the pipeline is first + * executed, will be used for all reads. The data being read will therefore become stale, and if no + * reads are made for more than 1 hour, the transaction will automatically timeout and be closed by + * the Spanner server, meaning that any subsequent reads will fail. + * + *

    {@code
    + * // Build a collection of ReadOperations.
    + * PCollection reads = ...
    + *
    + * PCollection rows = reads.apply(
    + *     SpannerIO.readAll()
    + *         .withInstanceId(instanceId)
    + *         .withDatabaseId(dbId)
    + * }
    + * *

    Writing to Cloud Spanner

    * *

    The Cloud Spanner {@link Write} transform writes to Cloud Spanner by executing a collection of @@ -362,6 +410,12 @@ *

    {@link Write} can be used as a streaming sink, however as with batch mode note that the write * order of individual {@link Mutation}/{@link MutationGroup} objects is not guaranteed. * + *

    {@link Read} and {@link ReadAll} can be used in Streaming pipelines to read a set of Facts on + * pipeline startup. + * + *

    {@link ReadAll} should not be used on an unbounded {@code PCollection}, for the + * reasons stated above. + * *

    Updates to the I/O connector code

    * * For any significant significant updates to this I/O connector, please consider involving @@ -564,8 +618,10 @@ public ReadAll withTimestampBound(TimestampBound timestampBound) { } /** - * By default Batch API is used to read data from Cloud Spanner. It is useful to disable - * batching when the underlying query is not root-partitionable. + * By default the PartitionQuery + * API is used to read data from Cloud Spanner. It is useful to disable batching when the + * underlying query is not root-partitionable. */ public ReadAll withBatching(boolean batching) { return toBuilder().setBatching(batching).build(); @@ -585,6 +641,15 @@ public ReadAll withHighPriority() { @Override public PCollection expand(PCollection input) { + + if (PCollection.IsBounded.UNBOUNDED == input.isBounded()) { + // Warn that SpannerIO.ReadAll should not be used on unbounded inputs. + LOG.warn( + "SpannerIO.ReadAll({}) is being applied to an unbounded input. " + + "This is not supported and can lead to runtime failures.", + this.getName()); + } + PTransform, PCollection> readTransform; if (getBatching()) { readTransform = From 7d997348b5525718ba3b3eb68d134bcc9dbb5140 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= Date: Tue, 5 Dec 2023 21:09:04 +0100 Subject: [PATCH 032/224] [YAML] - PubSubLite Normalization (#29525) * [YAML] - PubSubLite Normalization * [YAML] - PubSubLite Normalization - spotlessApply * [YAML] - PubSubLite Normalization - fix comment --- ...PubsubLiteReadSchemaTransformProvider.java | 311 +++++++++++++-- ...ubsubLiteWriteSchemaTransformProvider.java | 245 ++++++++++-- .../internal/PubsubLiteDlqTest.java | 373 +++++++++++++++++- .../internal/PubsubLiteWriteDlqTest.java | 176 ++++++++- sdks/python/apache_beam/yaml/standard_io.yaml | 30 ++ 5 files changed, 1048 insertions(+), 87 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java index f6acd081ab23..98c5f8a6b388 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java @@ -23,14 +23,22 @@ import com.google.cloud.pubsublite.ProjectId; import com.google.cloud.pubsublite.SubscriptionName; import com.google.cloud.pubsublite.SubscriptionPath; +import com.google.cloud.pubsublite.proto.AttributeValues; +import com.google.cloud.pubsublite.proto.PubSubMessage; import com.google.cloud.pubsublite.proto.SequencedMessage; +import com.google.protobuf.ByteString; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.Consumer; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.pubsublite.internal.Uuid; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -40,12 +48,13 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; @@ -68,14 +77,12 @@ public class PubsubLiteReadSchemaTransformProvider private static final Logger LOG = LoggerFactory.getLogger(PubsubLiteReadSchemaTransformProvider.class); - public static final String VALID_FORMATS_STR = "AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; public static final Set VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); public static final TupleTag OUTPUT_TAG = new TupleTag() {}; public static final TupleTag ERROR_TAG = new TupleTag() {}; - public static final Schema ERROR_SCHEMA = - Schema.builder().addStringField("error").addNullableByteArrayField("row").build(); @Override protected @UnknownKeyFor @NonNull @Initialized Class @@ -84,30 +91,95 @@ public class PubsubLiteReadSchemaTransformProvider } public static class ErrorFn extends DoFn { - private SerializableFunction valueMapper; - private Counter errorCounter; + private final SerializableFunction valueMapper; + private final Counter errorCounter; private Long errorsInBundle = 0L; + private final boolean handleErrors; - public ErrorFn(String name, SerializableFunction valueMapper) { + private final List attributes; + + private final String attributeMap; + + private final Schema errorSchema; + + private final Schema attributeSchema; + + public ErrorFn( + String name, + SerializableFunction valueMapper, + Schema errorSchema, + boolean handleErrors) { + this.errorCounter = Metrics.counter(PubsubLiteReadSchemaTransformProvider.class, name); + this.valueMapper = valueMapper; + this.errorSchema = errorSchema; + this.handleErrors = handleErrors; + this.attributes = new ArrayList<>(); + this.attributeMap = ""; + this.attributeSchema = Schema.builder().build(); + } + + public ErrorFn( + String name, + SerializableFunction valueMapper, + Schema errorSchema, + List attributes, + String attributeMap, + Schema attributeSchema, + boolean handleErrors) { this.errorCounter = Metrics.counter(PubsubLiteReadSchemaTransformProvider.class, name); this.valueMapper = valueMapper; + this.errorSchema = errorSchema; + this.handleErrors = handleErrors; + this.attributes = attributes; + this.attributeMap = attributeMap; + this.attributeSchema = attributeSchema; } @ProcessElement public void process(@DoFn.Element SequencedMessage seqMessage, MultiOutputReceiver receiver) { + Row mappedRow = null; try { - receiver - .get(OUTPUT_TAG) - .output(valueMapper.apply(seqMessage.getMessage().getData().toByteArray())); + if (attributes.isEmpty() + && attributeSchema.getFields().isEmpty() + && attributeMap.isEmpty()) { + mappedRow = valueMapper.apply(seqMessage.getMessage().getData().toByteArray()); + } else { + PubSubMessage message = seqMessage.getMessage(); + Row row = valueMapper.apply(message.getData().toByteArray()); + Row.Builder rowBuilder = Row.withSchema(attributeSchema).addValues(row.getValues()); + Map stringAttributeMap = new HashMap<>(); + message + .getAttributesMap() + .forEach( + (attributeName, attributeValues) -> { + if (attributes.contains(attributeName)) { + processAttribute(attributeValues, rowBuilder::addValue); + } + + if (!attributeMap.isEmpty()) { + processAttribute( + attributeValues, value -> stringAttributeMap.put(attributeName, value)); + } + }); + if (!attributeMap.isEmpty() && !stringAttributeMap.isEmpty()) { + rowBuilder.addValue(stringAttributeMap); + } + mappedRow = rowBuilder.build(); + } } catch (Exception e) { + if (!handleErrors) { + throw new RuntimeException(e); + } errorsInBundle += 1; LOG.warn("Error while parsing the element", e); receiver .get(ERROR_TAG) .output( - Row.withSchema(ERROR_SCHEMA) - .addValues(e.toString(), seqMessage.getMessage().getData().toByteArray()) - .build()); + ErrorHandling.errorRecord( + errorSchema, seqMessage.getMessage().getData().toByteArray(), e)); + } + if (mappedRow != null) { + receiver.get(OUTPUT_TAG).output(mappedRow); } } @@ -127,15 +199,35 @@ public void finish(FinishBundleContext c) { "Format %s not supported. Only supported formats are %s", configuration.getFormat(), VALID_FORMATS_STR)); } - final Schema beamSchema = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()) - : AvroUtils.toBeamSchema( - new org.apache.avro.Schema.Parser().parse(configuration.getSchema())); - final SerializableFunction valueMapper = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.getJsonBytesToRowFunction(beamSchema) - : AvroUtils.getAvroBytesToRowFunction(beamSchema); + boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); + String format = configuration.getFormat(); + String inputSchema = configuration.getSchema(); + List attributes = configuration.getAttributes(); + SerializableFunction valueMapper; + Schema beamSchema; + + if (format != null && format.equals("RAW")) { + if (inputSchema != null) { + throw new IllegalArgumentException( + "To read from PubSubLite in RAW format, you can't provide a schema."); + } + beamSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); + valueMapper = getRawBytesToRowFunction(beamSchema); + + } else { + if (inputSchema == null) { + throw new IllegalArgumentException( + "To read from PubSubLite in JSON or AVRO format, you must provide a schema."); + } + beamSchema = + Objects.equals(configuration.getFormat(), "JSON") + ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) + : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); + valueMapper = + Objects.equals(configuration.getFormat(), "JSON") + ? JsonUtils.getJsonBytesToRowFunction(beamSchema) + : AvroUtils.getAvroBytesToRowFunction(beamSchema); + } return new SchemaTransform() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { @@ -147,7 +239,16 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { throw new IllegalArgumentException( "Unable to infer the project to read from Pubsub Lite. Please provide a project."); } - PCollectionTuple outputTuple = + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributeList = new ArrayList<>(); + if (attributes != null) { + attributeList = attributes; + } + String attributeMapValue = configuration.getAttributeMap(); + String attributeMap = attributeMapValue == null ? "" : attributeMapValue; + Schema resultingBeamSchema = + buildSchemaWithAttributes(beamSchema, attributeList, attributeMap); + PCollection readPubsubLite = input .getPipeline() .apply( @@ -161,16 +262,124 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .setName( SubscriptionName.of(configuration.getSubscriptionName())) .build()) - .build())) - .apply( - ParDo.of(new ErrorFn("PubsubLite-read-error-counter", valueMapper)) - .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + .build())); + String attributeId = configuration.getAttributeId(); + PCollectionTuple outputTuple; + PCollection transformSequencedMessage; + if (attributeId != null && !attributeId.isEmpty()) { + UuidDeduplicationOptions.Builder uuidExtractor = + UuidDeduplicationOptions.newBuilder() + .setUuidExtractor(getUuidFromMessage(attributeId)); + transformSequencedMessage = + readPubsubLite.apply(PubsubLiteIO.deduplicate(uuidExtractor.build())); + } else { + transformSequencedMessage = readPubsubLite; + } + + outputTuple = + transformSequencedMessage.apply( + ParDo.of( + new ErrorFn( + "PubsubLite-read-error-counter", + valueMapper, + errorSchema, + attributeList, + attributeMap, + resultingBeamSchema, + handleErrors)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); return PCollectionRowTuple.of( "output", - outputTuple.get(OUTPUT_TAG).setRowSchema(beamSchema), + outputTuple.get(OUTPUT_TAG).setRowSchema(resultingBeamSchema), "errors", - outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + outputTuple.get(ERROR_TAG).setRowSchema(errorSchema)); + } + }; + } + + /** + * Builds a new {@link Schema} by adding additional optional attributes and map field to the + * provided schema. + * + * @param schema The base schema to which additional attributes and map field will be added. + * @param attributes A list of optional attribute names to be added as STRING fields to the + * schema. + * @param attributesMap The name of the optional map field to be added to the schema. If empty, no + * map field will be added. + * @return A new {@link Schema} with the specified attributes and an optional map field. + * @throws IllegalArgumentException if the schema is null or if any attribute name in the + * attributes list is null or empty. + */ + public static Schema buildSchemaWithAttributes( + Schema schema, List attributes, String attributesMap) { + Schema.Builder schemaBuilder = Schema.builder(); + // Copy fields from the original schema + schema.getFields().forEach(field -> schemaBuilder.addField(field.getName(), field.getType())); + + // Add optional additional attributes as STRING fields + attributes.forEach( + attribute -> { + if (attribute == null || attribute.isEmpty()) { + throw new IllegalArgumentException( + "Attribute names in the attributes list must not be null or empty."); + } + schemaBuilder.addField(attribute, Schema.FieldType.STRING); + }); + + // Add an optional map field if attributesMap is not empty + if (!attributesMap.isEmpty()) { + schemaBuilder + .addMapField(attributesMap, Schema.FieldType.STRING, Schema.FieldType.STRING) + .build(); + } + return schemaBuilder.build(); + } + + /** + * Processes the attribute values, invoking the specified consumer with the processed value. If + * the attribute values are null or contain multiple values, an exception is thrown. + * + * @param attributeValues The attribute values to be processed. If null, the method does nothing. + * @param valueConsumer The consumer to accept the processed value. + * @throws RuntimeException if attributeValues is not null and contains multiple values. + */ + private static void processAttribute( + @Nullable AttributeValues attributeValues, Consumer valueConsumer) { + if (attributeValues != null) { + List valueList = attributeValues.getValuesList(); + if (valueList.size() != 1) { + throw new RuntimeException( + "Received an unparseable message with multiple values for an attribute."); + } + valueConsumer.accept(valueList.get(0).toStringUtf8()); + } + } + + public static SerializableFunction getRawBytesToRowFunction(Schema rawSchema) { + return new SimpleFunction() { + @Override + public Row apply(byte[] input) { + return Row.withSchema(rawSchema).addValue(input).build(); + } + }; + } + + public static SerializableFunction getUuidFromMessage( + String attributeId) { + return new SimpleFunction() { + @Override + public Uuid apply(SequencedMessage input) { + AttributeValues attribute = input.getMessage().getAttributesMap().get(attributeId); + if (attribute != null) { + if (attribute.getValuesCount() != 1) { + throw new RuntimeException( + "Received an unparseable message with multiple values for an attribute."); + } + return Uuid.of(attribute.getValues(0)); + } else { + throw new RuntimeException("Uuid attribute missing."); + } } }; } @@ -205,7 +414,7 @@ public abstract static class PubsubLiteReadSchemaTransformConfiguration { + "For AVRO data, this is a schema defined with AVRO schema syntax " + "(https://avro.apache.org/docs/1.10.2/spec.html#schemas). " + "For JSON data, this is a schema defined with JSON-schema syntax (https://json-schema.org/).") - public abstract String getSchema(); + public abstract @Nullable String getSchema(); @SchemaFieldDescription( "The GCP project where the Pubsub Lite reservation resides. This can be a " @@ -220,6 +429,36 @@ public abstract static class PubsubLiteReadSchemaTransformConfiguration { @SchemaFieldDescription("The region or zone where the Pubsub Lite reservation resides.") public abstract String getLocation(); + @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.") + public abstract @Nullable ErrorHandling getErrorHandling(); + + @SchemaFieldDescription( + "List of attribute keys whose values will be flattened into the " + + "output message as additional fields. For example, if the format is `RAW` " + + "and attributes is `[\"a\", \"b\"]` then this read will produce elements of " + + "the form `Row(payload=..., a=..., b=...)`") + public abstract @Nullable List getAttributes(); + + @SchemaFieldDescription( + "Name of a field in which to store the full set of attributes " + + "associated with this message. For example, if the format is `RAW` and " + + "`attribute_map` is set to `\"attrs\"` then this read will produce elements " + + "of the form `Row(payload=..., attrs=...)` where `attrs` is a Map type " + + "of string to string. " + + "If both `attributes` and `attribute_map` are set, the overlapping " + + "attribute values will be present in both the flattened structure and the " + + "attribute map.") + public abstract @Nullable String getAttributeMap(); + + @SchemaFieldDescription( + "The attribute on incoming Pubsub Lite messages to use as a unique " + + "record identifier. When specified, the value of this attribute (which " + + "can be any string that uniquely identifies the record) will be used for " + + "deduplication of messages. If not provided, we cannot guarantee " + + "that no duplicate data will be delivered on the Pub/Sub stream. In this " + + "case, deduplication of the stream will be strictly best effort.") + public abstract @Nullable String getAttributeId(); + public static Builder builder() { return new AutoValue_PubsubLiteReadSchemaTransformProvider_PubsubLiteReadSchemaTransformConfiguration .Builder(); @@ -237,6 +476,16 @@ public abstract static class Builder { public abstract Builder setLocation(String location); + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + + public abstract Builder setAttributes(List attributes); + + @SuppressWarnings("unused") + public abstract Builder setAttributeMap(String attributeMap); + + @SuppressWarnings("unused") + public abstract Builder setAttributeId(String attributeId); + public abstract PubsubLiteReadSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java index 1b4d2bd77fb5..c669f84f330f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java @@ -23,14 +23,18 @@ import com.google.cloud.pubsublite.ProjectId; import com.google.cloud.pubsublite.TopicName; import com.google.cloud.pubsublite.TopicPath; +import com.google.cloud.pubsublite.proto.AttributeValues; import com.google.cloud.pubsublite.proto.PubSubMessage; import com.google.protobuf.ByteString; -import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; +import org.apache.beam.sdk.io.gcp.pubsublite.internal.Uuid; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -40,11 +44,15 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; @@ -53,6 +61,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,13 +71,11 @@ public class PubsubLiteWriteSchemaTransformProvider extends TypedSchemaTransformProvider< PubsubLiteWriteSchemaTransformProvider.PubsubLiteWriteSchemaTransformConfiguration> { - public static final String SUPPORTED_FORMATS_STR = "JSON,AVRO"; + public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO"; public static final Set SUPPORTED_FORMATS = Sets.newHashSet(SUPPORTED_FORMATS_STR.split(",")); public static final TupleTag OUTPUT_TAG = new TupleTag() {}; public static final TupleTag ERROR_TAG = new TupleTag() {}; - public static final Schema ERROR_SCHEMA = - Schema.builder().addStringField("error").addNullableByteArrayField("row").build(); private static final Logger LOG = LoggerFactory.getLogger(PubsubLiteWriteSchemaTransformProvider.class); @@ -79,33 +86,80 @@ public class PubsubLiteWriteSchemaTransformProvider } public static class ErrorCounterFn extends DoFn { - private SerializableFunction toBytesFn; - private Counter errorCounter; + private final SerializableFunction toBytesFn; + private final Counter errorCounter; private long errorsInBundle = 0L; - public ErrorCounterFn(String name, SerializableFunction toBytesFn) { + private final Schema errorSchema; + + private final boolean handleErrors; + + private final List attributes; + + private final Schema schemaWithoutAttributes; + + public ErrorCounterFn( + String name, + SerializableFunction toBytesFn, + Schema errorSchema, + boolean handleErrors) { this.toBytesFn = toBytesFn; errorCounter = Metrics.counter(PubsubLiteWriteSchemaTransformProvider.class, name); + this.errorSchema = errorSchema; + this.handleErrors = handleErrors; + this.attributes = new ArrayList<>(); + this.schemaWithoutAttributes = Schema.builder().build(); + } + + public ErrorCounterFn( + String name, + SerializableFunction toBytesFn, + Schema errorSchema, + boolean handleErrors, + List attributes, + Schema schemaWithoutAttributes) { + this.toBytesFn = toBytesFn; + errorCounter = Metrics.counter(PubsubLiteWriteSchemaTransformProvider.class, name); + this.errorSchema = errorSchema; + this.handleErrors = handleErrors; + this.attributes = attributes; + this.schemaWithoutAttributes = schemaWithoutAttributes; } @ProcessElement public void process(@DoFn.Element Row row, MultiOutputReceiver receiver) { try { - PubSubMessage message = - PubSubMessage.newBuilder() - .setData(ByteString.copyFrom(Objects.requireNonNull(toBytesFn.apply(row)))) - .build(); + PubSubMessage message; + if (attributes.isEmpty()) { + message = + PubSubMessage.newBuilder() + .setData(ByteString.copyFrom(Objects.requireNonNull(toBytesFn.apply(row)))) + .build(); + } else { + Row.Builder builder = Row.withSchema(schemaWithoutAttributes); + schemaWithoutAttributes + .getFields() + .forEach(field -> builder.addValue(row.getValue(field.getName()))); + + Row resultingRow = builder.build(); + Map attributeValuesHashMap = + getStringAttributeValuesMap(row, attributes); + message = + PubSubMessage.newBuilder() + .setData( + ByteString.copyFrom(Objects.requireNonNull(toBytesFn.apply(resultingRow)))) + .putAllAttributes(attributeValuesHashMap) + .build(); + } receiver.get(OUTPUT_TAG).output(message); } catch (Exception e) { + if (!handleErrors) { + throw new RuntimeException(e); + } errorsInBundle += 1; - LOG.warn("Error while parsing the element", e); - receiver - .get(ERROR_TAG) - .output( - Row.withSchema(ERROR_SCHEMA) - .addValues(e.toString(), row.toString().getBytes(StandardCharsets.UTF_8)) - .build()); + LOG.warn("Error while processing the element", e); + receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e)); } } @@ -132,23 +186,59 @@ public void finish() { return new SchemaTransform() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - Schema inputSchema = input.get("input").getSchema(); - final SerializableFunction toBytesFn = - configuration.getFormat().equals("JSON") - ? JsonUtils.getRowToJsonBytesFunction(inputSchema) - : AvroUtils.getRowToAvroBytesFunction(inputSchema); + List attributesConfigValue = configuration.getAttributes(); + String attributeId = configuration.getAttributeId(); + List attributes = + attributesConfigValue != null ? attributesConfigValue : new ArrayList<>(); + Schema inputSchema; + if (!attributes.isEmpty()) { + inputSchema = getSchemaWithoutAttributes(input.get("input").getSchema(), attributes); + } else { + inputSchema = input.get("input").getSchema(); + } + ErrorHandling errorHandling = configuration.getErrorHandling(); + boolean handleErrors = ErrorHandling.hasOutput(errorHandling); + Schema errorSchema = ErrorHandling.errorSchema(inputSchema); + + final SerializableFunction toBytesFn; + if (configuration.getFormat().equals("RAW")) { + int numFields = inputSchema.getFields().size(); + if (numFields != 1) { + throw new IllegalArgumentException("Expecting exactly one field, found " + numFields); + } + if (!inputSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) { + throw new IllegalArgumentException( + "The input schema must have exactly one field of type byte."); + } + toBytesFn = getRowToRawBytesFunction(inputSchema.getField(0).getName()); + } else if (configuration.getFormat().equals("JSON")) { + toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema); + } else { + toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); + } PCollectionTuple outputTuple = input .get("input") .apply( "Map Rows to PubSubMessages", - ParDo.of(new ErrorCounterFn("PubSubLite-write-error-counter", toBytesFn)) + ParDo.of( + new ErrorCounterFn( + "PubSubLite-write-error-counter", + toBytesFn, + errorSchema, + handleErrors, + attributes, + inputSchema)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); outputTuple .get(OUTPUT_TAG) - .apply("Add UUIDs", PubsubLiteIO.addUuids()) + .apply( + "Add UUIDs", + (attributeId != null && !attributeId.isEmpty()) + ? new SetUuidFromPubSubMessage(attributeId) + : PubsubLiteIO.addUuids()) .apply( "Write to PS Lite", PubsubLiteIO.write( @@ -161,8 +251,53 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .build()) .build())); - return PCollectionRowTuple.of( - "errors", outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA)); + PCollection errorOutput = + outputTuple.get(ERROR_TAG).setRowSchema(ErrorHandling.errorSchema(errorSchema)); + + String outputString = errorHandling != null ? errorHandling.getOutput() : "errors"; + return PCollectionRowTuple.of(handleErrors ? outputString : "errors", errorOutput); + } + }; + } + + public static Schema getSchemaWithoutAttributes(Schema inputSchema, List attributes) { + Schema.Builder schemaBuilder = Schema.builder(); + + inputSchema + .getFields() + .forEach( + field -> { + if (!attributes.contains(field.getName())) { + schemaBuilder.addField(field.getName(), field.getType()); + } + }); + return schemaBuilder.build(); + } + + private static Map getStringAttributeValuesMap( + Row row, List attributes) { + Map attributeValuesHashMap = new HashMap<>(); + attributes.forEach( + attribute -> { + String value = row.getValue(attribute); + if (value != null) { + attributeValuesHashMap.put( + attribute, + AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8(value)).build()); + } + }); + return attributeValuesHashMap; + } + + public static SerializableFunction getRowToRawBytesFunction(String rowFieldName) { + return new SimpleFunction() { + @Override + public byte[] apply(Row input) { + byte[] rawBytes = input.getBytes(rowFieldName); + if (rawBytes == null) { + throw new NullPointerException(); + } + return rawBytes; } }; } @@ -205,6 +340,24 @@ public abstract static class PubsubLiteWriteSchemaTransformConfiguration { + SUPPORTED_FORMATS_STR) public abstract String getFormat(); + @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.") + public abstract @Nullable ErrorHandling getErrorHandling(); + + @SchemaFieldDescription( + "List of attribute keys whose values will be pulled out as " + + "Pubsub Lite message attributes. For example, if the format is `JSON` " + + "and attributes is `[\"a\", \"b\"]` then elements of the form " + + "`Row(any_field=..., a=..., b=...)` will result in Pubsub Lite messages whose " + + "payload has the contents of any_field and whose attribute will be " + + "populated with the values of `a` and `b`.") + public abstract @Nullable List getAttributes(); + + @SchemaFieldDescription( + "If set, will set an attribute for each Pubsub Lite message " + + "with the given name and a unique value. This attribute can then be used " + + "in a ReadFromPubSubLite PTransform to deduplicate messages.") + public abstract @Nullable String getAttributeId(); + public static Builder builder() { return new AutoValue_PubsubLiteWriteSchemaTransformProvider_PubsubLiteWriteSchemaTransformConfiguration .Builder(); @@ -220,7 +373,45 @@ public abstract static class Builder { public abstract Builder setFormat(String format); + public abstract Builder setErrorHandling(ErrorHandling errorHandling); + + public abstract Builder setAttributes(List attributes); + + @SuppressWarnings("unused") + public abstract Builder setAttributeId(String attributeId); + public abstract PubsubLiteWriteSchemaTransformConfiguration build(); } } + + public static class SetUuidFromPubSubMessage + extends PTransform, PCollection> { + private final String attributeId; + + public SetUuidFromPubSubMessage(String attributeId) { + this.attributeId = attributeId; + } + + @Override + public PCollection expand(PCollection input) { + return input.apply("SetUuidFromPubSubMessage", ParDo.of(new SetUuidFn(attributeId))); + } + + public static class SetUuidFn extends DoFn { + private final String attributeId; + + public SetUuidFn(String attributeId) { + this.attributeId = attributeId; + } + + @ProcessElement + public void processElement( + @Element PubSubMessage input, OutputReceiver outputReceiver) { + PubSubMessage.Builder builder = input.toBuilder(); + builder.putAttributes( + attributeId, AttributeValues.newBuilder().addValues(Uuid.random().value()).build()); + outputReceiver.output(builder.build()); + } + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java index 04b98ed6008a..e23536d800e5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java @@ -17,15 +17,26 @@ */ package org.apache.beam.sdk.io.gcp.pubsublite.internal; +import static org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider.getRawBytesToRowFunction; +import static org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider.getUuidFromMessage; + +import com.google.cloud.pubsublite.proto.AttributeValues; import com.google.cloud.pubsublite.proto.PubSubMessage; import com.google.cloud.pubsublite.proto.SequencedMessage; import com.google.protobuf.ByteString; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider.ErrorFn; +import org.apache.beam.sdk.io.gcp.pubsublite.UuidDeduplicationOptions; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -46,18 +57,131 @@ @RunWith(JUnit4.class) public class PubsubLiteDlqTest { - private static final TupleTag OUTPUTTAG = PubsubLiteReadSchemaTransformProvider.OUTPUT_TAG; - private static final TupleTag ERRORTAG = PubsubLiteReadSchemaTransformProvider.ERROR_TAG; + private static final TupleTag OUTPUT_TAG = PubsubLiteReadSchemaTransformProvider.OUTPUT_TAG; + private static final TupleTag ERROR_TAG = PubsubLiteReadSchemaTransformProvider.ERROR_TAG; - private static final Schema BEAMSCHEMA = + private static final Schema BEAM_RAW_SCHEMA = + Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); + private static final Schema BEAM_SCHEMA = Schema.of(Schema.Field.of("name", Schema.FieldType.STRING)); - private static final Schema ERRORSCHEMA = PubsubLiteReadSchemaTransformProvider.ERROR_SCHEMA; + + private static final Schema BEAM_SCHEMA_ATTRIBUTES = + Schema.of( + Schema.Field.of("name", Schema.FieldType.STRING), + Schema.Field.of("key1", Schema.FieldType.STRING), + Schema.Field.of("key2", Schema.FieldType.STRING)); + + private static final Schema BEAM_SCHEMA_ATTRIBUTES_AND_MAP = + Schema.of( + Schema.Field.of("name", Schema.FieldType.STRING), + Schema.Field.of("key1", Schema.FieldType.STRING), + Schema.Field.of("key2", Schema.FieldType.STRING), + Schema.Field.of( + "attrs", Schema.FieldType.map(Schema.FieldType.STRING, Schema.FieldType.STRING))); + + private static final Schema BEAM_SCHEMA_ATTRIBUTES_MAP = + Schema.of( + Schema.Field.of("name", Schema.FieldType.STRING), + Schema.Field.of( + "attrs", Schema.FieldType.map(Schema.FieldType.STRING, Schema.FieldType.STRING))); + + private static final Map STATIC_MAP; + + static { + Map tempMap = new HashMap<>(); + tempMap.put("key1", "first_key"); + tempMap.put("key2", "second_key"); + STATIC_MAP = Collections.unmodifiableMap(tempMap); + } + + private static final List RAW_ROWS; + + static { + try { + RAW_ROWS = + Arrays.asList( + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "a".getBytes("UTF-8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "b".getBytes("UTF-8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "c".getBytes("UTF-8")) + .build()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + private static final List ROWS_WITH_ATTRIBUTES = + Arrays.asList( + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "a") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "b") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "c") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build()); + private static final List ROWS_WITH_ATTRIBUTES_MAP = + Arrays.asList( + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_MAP) + .withFieldValue("name", "a") + .withFieldValue("attrs", STATIC_MAP) + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_MAP) + .withFieldValue("name", "b") + .withFieldValue("attrs", STATIC_MAP) + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_MAP) + .withFieldValue("name", "c") + .withFieldValue("attrs", STATIC_MAP) + .build()); + private static final List ROWS_WITH_ATTRIBUTES_AND_MAP = + Arrays.asList( + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_AND_MAP) + .withFieldValue("name", "a") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .withFieldValue("attrs", STATIC_MAP) + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_AND_MAP) + .withFieldValue("name", "b") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .withFieldValue("attrs", STATIC_MAP) + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES_AND_MAP) + .withFieldValue("name", "c") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .withFieldValue("attrs", STATIC_MAP) + .build()); private static final List ROWS = Arrays.asList( - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "a").build(), - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "b").build(), - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "c").build()); + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "a").build(), + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "b").build(), + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "c").build()); + + private static final Map ATTRIBUTE_VALUES_MAP = new HashMap<>(); + + static { + ATTRIBUTE_VALUES_MAP.put( + "key1", + AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("first_key")).build()); + ATTRIBUTE_VALUES_MAP.put( + "key2", + AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("second_key")).build()); + } private static final List MESSAGES = Arrays.asList( @@ -65,18 +189,45 @@ public class PubsubLiteDlqTest { .setMessage( PubSubMessage.newBuilder() .setData(ByteString.copyFromUtf8("{\"name\":\"a\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) .build()) .build(), SequencedMessage.newBuilder() .setMessage( PubSubMessage.newBuilder() .setData(ByteString.copyFromUtf8("{\"name\":\"b\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) .build()) .build(), SequencedMessage.newBuilder() .setMessage( PubSubMessage.newBuilder() .setData(ByteString.copyFromUtf8("{\"name\":\"c\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build()) + .build()); + + private static final List RAW_MESSAGES = + Arrays.asList( + SequencedMessage.newBuilder() + .setMessage( + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("a")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build()) + .build(), + SequencedMessage.newBuilder() + .setMessage( + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("b")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build()) + .build(), + SequencedMessage.newBuilder() + .setMessage( + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("c")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) .build()) .build()); @@ -102,40 +253,222 @@ public class PubsubLiteDlqTest { .build()); final SerializableFunction valueMapper = - JsonUtils.getJsonBytesToRowFunction(BEAMSCHEMA); + JsonUtils.getJsonBytesToRowFunction(BEAM_SCHEMA); @Rule public transient TestPipeline p = TestPipeline.create(); @Test - public void testPubsubLiteErrorFnSuccess() throws Exception { + public void testPubsubLiteErrorFnSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollection input = p.apply(Create.of(MESSAGES)); PCollectionTuple output = input.apply( - ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper)) - .withOutputTags(OUTPUTTAG, TupleTagList.of(ERRORTAG))); + ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(OUTPUTTAG).setRowSchema(BEAMSCHEMA); - output.get(ERRORTAG).setRowSchema(ERRORSCHEMA); + output.get(OUTPUT_TAG).setRowSchema(BEAM_SCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); - PAssert.that(output.get(OUTPUTTAG)).containsInAnyOrder(ROWS); + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(ROWS); p.run().waitUntilFinish(); } @Test - public void testPubsubLiteErrorFnFailure() throws Exception { + public void testPubsubLiteErrorFnFailure() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollection input = p.apply(Create.of(MESSAGESWITHERROR)); PCollectionTuple output = input.apply( - ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper)) - .withOutputTags(OUTPUTTAG, TupleTagList.of(ERRORTAG))); + ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(BEAM_SCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PCollection count = output.get(ERROR_TAG).apply("error_count", Count.globally()); + + PAssert.that(count).containsInAnyOrder(Collections.singletonList(3L)); + + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnRawSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + + List attributes = new ArrayList<>(); + String attributesMap = ""; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_RAW_SCHEMA, attributes, attributesMap); + SerializableFunction rawValueMapper = getRawBytesToRowFunction(BEAM_RAW_SCHEMA); + PCollection input = p.apply(Create.of(RAW_MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of(new ErrorFn("Read-Error-Counter", rawValueMapper, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(RAW_ROWS); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnWithAttributesSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributes = new ArrayList<>(); + attributes.add("key1"); + attributes.add("key2"); + String attributeMap = ""; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_SCHEMA, attributes, attributeMap); + + PCollection input = p.apply(Create.of(MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorFn( + "Read-Error-Counter", + valueMapper, + errorSchema, + attributes, + attributeMap, + beamAttributeSchema, + Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(OUTPUTTAG).setRowSchema(BEAMSCHEMA); - output.get(ERRORTAG).setRowSchema(ERRORSCHEMA); + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); - PCollection count = output.get(ERRORTAG).apply("error_count", Count.globally()); + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(ROWS_WITH_ATTRIBUTES); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnWithAttributeMapSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + // empty list of attributes + List attributes = new ArrayList<>(); + String attributeMap = "attrs"; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_SCHEMA, attributes, attributeMap); + + PCollection input = p.apply(Create.of(MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorFn( + "Read-Error-Counter", + valueMapper, + errorSchema, + attributes, + attributeMap, + beamAttributeSchema, + Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(ROWS_WITH_ATTRIBUTES_MAP); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnWithAttributesAndAttributeMapSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributes = new ArrayList<>(); + attributes.add("key1"); + attributes.add("key2"); + String attributeMap = "attrs"; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_SCHEMA, attributes, attributeMap); + + PCollection input = p.apply(Create.of(MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorFn( + "Read-Error-Counter", + valueMapper, + errorSchema, + attributes, + attributeMap, + beamAttributeSchema, + Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(ROWS_WITH_ATTRIBUTES_AND_MAP); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnWithAttributesFailure() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributes = new ArrayList<>(); + attributes.add("randomKey1"); + attributes.add("randomKey2"); + String attributeMap = ""; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_SCHEMA, attributes, attributeMap); + + PCollection input = p.apply(Create.of(MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorFn( + "Read-Error-Counter", + valueMapper, + errorSchema, + attributes, + attributeMap, + beamAttributeSchema, + Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PCollection count = output.get(ERROR_TAG).apply("error_count", Count.globally()); PAssert.that(count).containsInAnyOrder(Collections.singletonList(3L)); p.run().waitUntilFinish(); } + + @Test + public void testPubsubLiteErrorFnWithDedupingSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + + PCollection input = p.apply(Create.of(MESSAGES)); + UuidDeduplicationOptions.Builder uuidExtractor = + UuidDeduplicationOptions.newBuilder().setUuidExtractor(getUuidFromMessage("key1")); + PCollectionTuple output = + input + .apply(PubsubLiteIO.deduplicate(uuidExtractor.build())) + .apply( + ParDo.of(new ErrorFn("Read-Error-Counter", valueMapper, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(BEAM_SCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PCollection count = output.get(OUTPUT_TAG).apply("error_count", Count.globally()); + + // We are deduping so we should only have 1 value + PAssert.that(count).containsInAnyOrder(Collections.singletonList(1L)); + + p.run().waitUntilFinish(); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java index d42eb249b27e..eac0e4264c1a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java @@ -17,16 +17,24 @@ */ package org.apache.beam.sdk.io.gcp.pubsublite.internal; +import com.google.cloud.pubsublite.proto.AttributeValues; import com.google.cloud.pubsublite.proto.PubSubMessage; import com.google.protobuf.ByteString; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteWriteSchemaTransformProvider; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteWriteSchemaTransformProvider.ErrorCounterFn; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; import org.apache.beam.sdk.schemas.utils.JsonUtils; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -47,38 +55,188 @@ public class PubsubLiteWriteDlqTest { PubsubLiteWriteSchemaTransformProvider.OUTPUT_TAG; private static final TupleTag ERROR_TAG = PubsubLiteWriteSchemaTransformProvider.ERROR_TAG; - private static final Schema BEAMSCHEMA = + private static final Schema BEAM_SCHEMA = Schema.of(Schema.Field.of("name", Schema.FieldType.STRING)); - private static final Schema ERRORSCHEMA = PubsubLiteWriteSchemaTransformProvider.ERROR_SCHEMA; + + private static final Schema BEAM_RAW_SCHEMA = + Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); + + private static final Schema BEAM_SCHEMA_ATTRIBUTES = + Schema.of( + Schema.Field.of("name", Schema.FieldType.STRING), + Schema.Field.of("key1", Schema.FieldType.STRING), + Schema.Field.of("key2", Schema.FieldType.STRING)); + + private static final List RAW_ROWS; + + static { + try { + RAW_ROWS = + Arrays.asList( + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "a".getBytes("UTF8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "b".getBytes("UTF8")) + .build(), + Row.withSchema(BEAM_RAW_SCHEMA) + .withFieldValue("payload", "c".getBytes("UTF8")) + .build()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } private static final List ROWS = Arrays.asList( - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "a").build(), - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "b").build(), - Row.withSchema(BEAMSCHEMA).withFieldValue("name", "c").build()); + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "a").build(), + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "b").build(), + Row.withSchema(BEAM_SCHEMA).withFieldValue("name", "c").build()); + + private static final List ROWSATTRIBUTES = + Arrays.asList( + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "a") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "b") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build(), + Row.withSchema(BEAM_SCHEMA_ATTRIBUTES) + .withFieldValue("name", "c") + .withFieldValue("key1", "first_key") + .withFieldValue("key2", "second_key") + .build()); + + private static final Map ATTRIBUTE_VALUES_MAP = new HashMap<>(); + + static { + ATTRIBUTE_VALUES_MAP.put( + "key1", + AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("first_key")).build()); + ATTRIBUTE_VALUES_MAP.put( + "key2", + AttributeValues.newBuilder().addValues(ByteString.copyFromUtf8("second_key")).build()); + } + + private static final List MESSAGES_RAW = + Arrays.asList( + PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("a")).build(), + PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("b")).build(), + PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("c")).build()); private static final List MESSAGES = Arrays.asList( PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("{\"name\":\"a\"}")).build(), PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("{\"name\":\"b\"}")).build(), PubSubMessage.newBuilder().setData(ByteString.copyFromUtf8("{\"name\":\"c\"}")).build()); + private static final List MESSAGES_WITH_ATTRIBUTES = + Arrays.asList( + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("{\"name\":\"a\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build(), + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("{\"name\":\"b\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build(), + PubSubMessage.newBuilder() + .setData(ByteString.copyFromUtf8("{\"name\":\"c\"}")) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build()); final SerializableFunction valueMapper = - JsonUtils.getRowToJsonBytesFunction(BEAMSCHEMA); + JsonUtils.getRowToJsonBytesFunction(BEAM_SCHEMA); + + final SerializableFunction valueMapperRaw = + PubsubLiteWriteSchemaTransformProvider.getRowToRawBytesFunction("payload"); @Rule public transient TestPipeline p = TestPipeline.create(); @Test - public void testPubsubLiteErrorFnSuccess() throws Exception { + public void testPubsubLiteErrorFnSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); PCollection input = p.apply(Create.of(ROWS)); PCollectionTuple output = input.apply( - ParDo.of(new ErrorCounterFn("ErrorCounter", valueMapper)) + ParDo.of(new ErrorCounterFn("ErrorCounter", valueMapper, errorSchema, Boolean.TRUE)) .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA); + output.get(ERROR_TAG).setRowSchema(errorSchema); PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(MESSAGES); p.run().waitUntilFinish(); } + + @Test + public void testPubsubLiteErrorFnSuccessRawEvents() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + PCollection input = p.apply(Create.of(RAW_ROWS)); + PCollectionTuple output = + input.apply( + ParDo.of(new ErrorCounterFn("ErrorCounter", valueMapperRaw, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(MESSAGES_RAW); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnSuccessWithAttributes() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributes = new ArrayList<>(); + attributes.add("key1"); + attributes.add("key2"); + Schema schema = + PubsubLiteWriteSchemaTransformProvider.getSchemaWithoutAttributes( + BEAM_SCHEMA_ATTRIBUTES, attributes); + PCollection input = p.apply(Create.of(ROWSATTRIBUTES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorCounterFn( + "ErrorCounter", valueMapper, errorSchema, Boolean.TRUE, attributes, schema)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(MESSAGES_WITH_ATTRIBUTES); + p.run().waitUntilFinish(); + } + + @Test + public void testPubsubLiteErrorFnSuccessWithAttributesAndDedupingSuccess() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + List attributes = new ArrayList<>(); + attributes.add("key1"); + attributes.add("key2"); + Schema schema = + PubsubLiteWriteSchemaTransformProvider.getSchemaWithoutAttributes( + BEAM_SCHEMA_ATTRIBUTES, attributes); + PCollection input = p.apply(Create.of(ROWSATTRIBUTES)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorCounterFn( + "ErrorCounter", valueMapper, errorSchema, Boolean.TRUE, attributes, schema)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PCollection count = + output + .get(OUTPUT_TAG) + .apply( + ParDo.of( + new PubsubLiteWriteSchemaTransformProvider.SetUuidFromPubSubMessage.SetUuidFn( + "unique_key"))) + .apply("error_count", Count.globally()); + PAssert.that(count).containsInAnyOrder(Collections.singletonList(3L)); + p.run().waitUntilFinish(); + } } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index b19c1e5b063e..9c430ae2d2d3 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -80,6 +80,36 @@ config: gradle_target: 'sdks:java:io:expansion-service:shadowJar' +- type: renaming + transforms: + 'ReadFromPubSubLite': 'ReadFromPubSubLite' + 'WriteToPubSubLite': 'WriteToPubSubLite' + config: + mappings: + 'ReadFromPubSubLite': + 'project': 'project' + 'schema': 'schema' + 'format': 'format' + 'subscription_name': 'subscriptionName' + 'location': 'location' + 'attributes': 'attributes' + 'attribute_map': 'attributeMap' + 'attribute_id': 'attributeId' + 'WriteToPubSubLite': + 'project': 'project' + 'format': 'format' + 'topic_name': 'topicName' + 'location': 'location' + 'attributes': 'attributes' + 'attribute_id': 'attributeId' + underlying_provider: + type: beamJar + transforms: + 'ReadFromPubSubLite': 'beam:schematransform:org.apache.beam:pubsublite_read:v1' + 'WriteToPubSubLite': 'beam:schematransform:org.apache.beam:pubsublite_write:v1' + config: + gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar' + - type: python transforms: 'ReadFromBigQuery': 'apache_beam.yaml.yaml_io.read_from_bigquery' From 22ae2e35b98b57fc2f31dacd9a188dd7bcaf0cc7 Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Tue, 5 Dec 2023 15:15:58 -0500 Subject: [PATCH 033/224] Feature/dead letter queue core (#29164) * 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 * Add error handling base implementation & test DLQ enabled class * Add test cases * apply spotless * Fix Checkstyles * Fix Checkstyles * make DLH serializable * rename dead letter to bad record * make DLH serializable * Change bad record router name, and use multioutputreceiver instead of process context * Refactor BadRecord to be nested * clean up checkstyle * Update error handler test * Add metrics for counting error records, and for measuring feature usage * apply spotless * fix checkstyle * make metric reporting static * spotless * Rework annotations to be an explicit label on a PTransform, instead of using java annotations * fix checkstyle * Address comments * Address comments * Fix test cases, spotless * remove flatting without error collections * fix nullness * spotless + encoding issues * spotless * throw error when error handler isn't used * add concrete bad record error handler class * spotless, fix test category * fix checkstyle * clean up comments * fix test case * remove "failing transform" field on bad record, add note to CHANGES.md * fix failing test cases * fix failing test cases * apply spotless * apply final comments * apply final comments * apply final comments --- CHANGES.md | 1 + .../construction/PTransformTranslation.java | 7 + .../java/org/apache/beam/sdk/Pipeline.java | 24 ++ .../beam/sdk/transforms/PTransform.java | 14 ++ .../transforms/errorhandling/BadRecord.java | 218 +++++++++++++++++ .../errorhandling/BadRecordRouter.java | 85 +++++++ .../errorhandling/ErrorHandler.java | 227 ++++++++++++++++++ .../errorhandling/package-info.java | 23 ++ .../errorhandling/BRHEnabledPTransform.java | 90 +++++++ .../errorhandling/BadRecordRouterTest.java | 180 ++++++++++++++ .../errorhandling/ErrorHandlerTest.java | 136 +++++++++++ 11 files changed, 1005 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/package-info.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BRHEnabledPTransform.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouterTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlerTest.java diff --git a/CHANGES.md b/CHANGES.md index 847532e85562..4b977bf3790d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -74,6 +74,7 @@ * Running multi-language pipelines locally no longer requires Docker. Instead, the same (generally auto-started) subprocess used to perform the expansion can also be used as the cross-language worker. +* Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)) ## Breaking Changes 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 8f415e718e95..6829e0d6b232 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 @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.ServiceLoader; import java.util.Set; import org.apache.beam.model.pipeline.v1.RunnerApi; @@ -502,6 +503,12 @@ public RunnerApi.PTransform translate( SchemaTranslation.schemaToProto(configRow.getSchema(), true).toByteArray())); } + for (Entry annotation : + appliedPTransform.getTransform().getAnnotations().entrySet()) { + transformBuilder.putAnnotations( + annotation.getKey(), ByteString.copyFrom(annotation.getValue())); + } + return transformBuilder.build(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index bd0215e1326e..50158f109cb0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -22,6 +22,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.transform; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -43,6 +44,9 @@ import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.PBegin; @@ -318,6 +322,7 @@ public PipelineResult run(PipelineOptions options) { LOG.debug("Running {} via {}", this, runner); try { validate(options); + validateErrorHandlers(); return runner.run(this); } catch (UserCodeException e) { // This serves to replace the stack with one that ends here and @@ -343,6 +348,13 @@ public SchemaRegistry getSchemaRegistry() { return schemaRegistry; } + public BadRecordErrorHandler registerBadRecordErrorHandler( + PTransform, OutputT> sinkTransform) { + BadRecordErrorHandler errorHandler = new BadRecordErrorHandler<>(sinkTransform, this); + errorHandlers.add(errorHandler); + return errorHandler; + } + ///////////////////////////////////////////////////////////////////////////// // Below here are operations that aren't normally called by users. @@ -511,6 +523,8 @@ public static OutputT applyTran private final Multimap> instancePerName = ArrayListMultimap.create(); private final PipelineOptions defaultOptions; + private final List> errorHandlers = new ArrayList<>(); + private Pipeline(TransformHierarchy transforms, PipelineOptions options) { this.transforms = transforms; this.defaultOptions = options; @@ -715,4 +729,14 @@ public boolean apply(@Nonnull final Map.Entry> input) { return input != null && input.getValue().size() == 1; } } + + private void validateErrorHandlers() { + for (ErrorHandler errorHandler : errorHandlers) { + if (!errorHandler.isClosed()) { + throw new IllegalStateException( + "One or more ErrorHandlers aren't closed, and this pipeline" + + "cannot be run. See the ErrorHandler documentation for expected usage"); + } + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index 7b3dfb188033..c0c3638b28dd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -21,6 +21,7 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -216,6 +217,17 @@ public ResourceHints getResourceHints() { return resourceHints; } + /** Returns annotations map to provide additional hints to the runner. */ + public Map getAnnotations() { + return annotations; + } + + public PTransform addAnnotation( + @NonNull String annotationType, byte @NonNull [] annotation) { + annotations.put(annotationType, annotation); + return this; + } + ///////////////////////////////////////////////////////////////////////////// // See the note about about PTransform's fake Serializability, to @@ -229,6 +241,8 @@ public ResourceHints getResourceHints() { protected transient @NonNull ResourceHints resourceHints = ResourceHints.create(); + protected transient @NonNull Map annotations = new HashMap<>(); + protected PTransform() { this.name = null; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java new file mode 100644 index 000000000000..fd49078350c4 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.auto.value.AutoValue; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.io.Serializable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@AutoValue +@DefaultSchema(AutoValueSchema.class) +public abstract class BadRecord implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(BadRecord.class); + + /** Information about the record that failed. */ + public abstract Record getRecord(); + + /** Information about why the record failed. */ + public abstract Failure getFailure(); + + public static Builder builder() { + return new AutoValue_BadRecord.Builder(); + } + + public static Coder getCoder(Pipeline pipeline) { + try { + SchemaRegistry schemaRegistry = pipeline.getSchemaRegistry(); + return SchemaCoder.of( + schemaRegistry.getSchema(BadRecord.class), + TypeDescriptor.of(BadRecord.class), + schemaRegistry.getToRowFunction(BadRecord.class), + schemaRegistry.getFromRowFunction(BadRecord.class)); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public static BadRecord fromExceptionInformation( + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description) + throws IOException { + Preconditions.checkArgumentNotNull(record); + + // Build up record information + BadRecord.Record.Builder recordBuilder = Record.builder(); + recordBuilder.addHumanReadableJson(record).addCoderAndEncodedRecord(coder, record); + + // Build up failure information + BadRecord.Failure.Builder failureBuilder = Failure.builder().setDescription(description); + + // It's possible for us to want to handle an error scenario where no actual exception object + // exists + if (exception != null) { + failureBuilder.setException(exception.toString()).addExceptionStackTrace(exception); + } + + return BadRecord.builder() + .setRecord(recordBuilder.build()) + .setFailure(failureBuilder.build()) + .build(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setRecord(Record record); + + public abstract Builder setFailure(Failure error); + + public abstract BadRecord build(); + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Record implements Serializable { + + /** The failing record, encoded as JSON. Will be null if serialization as JSON fails. */ + public abstract @Nullable String getHumanReadableJsonRecord(); + + /** + * Nullable to account for failing to encode, or if there is no coder for the record at the time + * of failure. + */ + @SuppressWarnings("mutable") + public abstract byte @Nullable [] getEncodedRecord(); + + /** The coder for the record, or null if there is no coder. */ + public abstract @Nullable String getCoder(); + + public static Builder builder() { + return new AutoValue_BadRecord_Record.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setHumanReadableJsonRecord(@Nullable String jsonRecord); + + public Builder addHumanReadableJson(Object record) { + ObjectWriter objectWriter = new ObjectMapper().writer().withDefaultPrettyPrinter(); + try { + this.setHumanReadableJsonRecord(objectWriter.writeValueAsString(record)); + } catch (Exception e) { + LOG.error( + "Unable to serialize record as JSON. Human readable record attempted via .toString", + e); + try { + this.setHumanReadableJsonRecord(record.toString()); + } catch (Exception e2) { + LOG.error( + "Unable to serialize record via .toString. Human readable record will be null", e2); + } + } + return this; + } + + @SuppressWarnings("mutable") + public abstract Builder setEncodedRecord(byte @Nullable [] encodedRecord); + + public abstract Builder setCoder(@Nullable String coder); + + public Builder addCoderAndEncodedRecord(@Nullable Coder coder, T record) { + // We will sometimes not have a coder for a failing record, for example if it has already + // been + // modified within the dofn. + if (coder != null) { + this.setCoder(coder.toString()); + try { + this.setEncodedRecord(CoderUtils.encodeToByteArray(coder, record)); + } catch (IOException e) { + LOG.error( + "Unable to encode failing record using provided coder." + + " BadRecord will be published without encoded bytes", + e); + } + } + return this; + } + + public abstract Record build(); + } + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Failure implements Serializable { + + /** The exception itself, e.g. IOException. Null if there is a failure without an exception. */ + public abstract @Nullable String getException(); + + /** The full stacktrace. Null if there is a failure without an exception. */ + public abstract @Nullable String getExceptionStacktrace(); + + /** The description of what was being attempted when the failure occurred. */ + public abstract String getDescription(); + + public static Builder builder() { + return new AutoValue_BadRecord_Failure.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setException(@Nullable String exception); + + public abstract Builder setExceptionStacktrace(@Nullable String stacktrace); + + public Builder addExceptionStackTrace(Exception exception) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + PrintStream printStream = new PrintStream(stream, false, Charsets.UTF_8.name()); + exception.printStackTrace(printStream); + printStream.close(); + + this.setExceptionStacktrace(new String(stream.toByteArray(), Charsets.UTF_8)); + return this; + } + + public abstract Builder setDescription(String description); + + public abstract Failure build(); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java new file mode 100644 index 000000000000..7670baf5c6ff --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import java.io.Serializable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.TupleTag; +import org.checkerframework.checker.nullness.qual.Nullable; + +public interface BadRecordRouter extends Serializable { + + BadRecordRouter THROWING_ROUTER = new ThrowingBadRecordRouter(); + + BadRecordRouter RECORDING_ROUTER = new RecordingBadRecordRouter(); + + TupleTag BAD_RECORD_TAG = new TupleTag<>(); + + void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description) + throws Exception; + + class ThrowingBadRecordRouter implements BadRecordRouter { + + @Override + public void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description) + throws Exception { + if (exception != null) { + throw exception; + } else { + Preconditions.checkArgumentNotNull(record); + String encodedRecord = + BadRecord.Record.builder() + .addHumanReadableJson(record) + .build() + .getHumanReadableJsonRecord(); + if (encodedRecord == null) { + encodedRecord = "Unable to serialize bad record"; + } + throw new RuntimeException("Encountered Bad Record: " + encodedRecord); + } + } + } + + class RecordingBadRecordRouter implements BadRecordRouter { + + @Override + public void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description) + throws Exception { + outputReceiver + .get(BAD_RECORD_TAG) + .output(BadRecord.fromExceptionInformation(record, coder, exception, description)); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java new file mode 100644 index 000000000000..9e0298d885eb --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.POutput; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An Error Handler is a utility object used for plumbing error PCollections to a configured sink + * Error Handlers must be closed before a pipeline is run to properly pipe error collections to the + * sink, and the pipeline will be rejected if any handlers aren't closed. + * + * @param The type of the error object. This will usually be a {@link BadRecord}, but can + * be any type + * @param The return type of the sink PTransform. + *

    Usage of Error Handlers: + *

    Simple usage with one DLQ + *

    {@code
    + * PCollection records = ...;
    + * try (ErrorHandler errorHandler = pipeline.registerErrorHandler(SomeSink.write())) {
    + *  PCollection results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler));
    + * }
    + * results.apply(SomeOtherTransform);
    + * }
    + * Usage with multiple DLQ stages + *
    {@code
    + * PCollection records = ...;
    + * try (ErrorHandler errorHandler = pipeline.registerErrorHandler(SomeSink.write())) {
    + *  PCollection results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler))
    + *                        .apply(OtherTransform.builder().withDeadLetterQueue(errorHandler));
    + * }
    + * results.apply(SomeOtherTransform);
    + * }
    + */ +public interface ErrorHandler extends AutoCloseable { + + void addErrorCollection(PCollection errorCollection); + + boolean isClosed(); + + @Nullable + OutputT getOutput(); + + class PTransformErrorHandler + implements ErrorHandler { + + private static final Logger LOG = LoggerFactory.getLogger(PTransformErrorHandler.class); + private final PTransform, OutputT> sinkTransform; + + private final Pipeline pipeline; + + private final Coder coder; + + private final List> errorCollections = new ArrayList<>(); + + private @Nullable OutputT sinkOutput = null; + + private boolean closed = false; + + /** + * Constructs a new ErrorHandler, but should not be called directly. Instead, call + * pipeline.registerErrorHandler to ensure safe pipeline construction + */ + @Internal + public PTransformErrorHandler( + PTransform, OutputT> sinkTransform, + Pipeline pipeline, + Coder coder) { + this.sinkTransform = sinkTransform; + this.pipeline = pipeline; + this.coder = coder; + } + + @Override + public void addErrorCollection(PCollection errorCollection) { + errorCollections.add(errorCollection); + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public @Nullable OutputT getOutput() { + if (!this.isClosed()) { + throw new IllegalStateException( + "ErrorHandler must be finalized before the output can be returned"); + } + return sinkOutput; + } + + @Override + public void close() { + if (closed) { + throw new IllegalStateException( + "Error handler is already closed, and may not be closed twice"); + } + closed = true; + PCollection flattened; + if (errorCollections.isEmpty()) { + LOG.info("Empty list of error pcollections passed to ErrorHandler."); + flattened = pipeline.apply(Create.empty(coder)); + } else { + flattened = PCollectionList.of(errorCollections).apply(Flatten.pCollections()); + } + LOG.debug( + "{} error collections are being sent to {}", + errorCollections.size(), + sinkTransform.getName()); + String sinkTransformName = sinkTransform.getName(); + sinkOutput = + flattened + .apply( + "Record Error Metrics to " + sinkTransformName, + new WriteErrorMetrics(sinkTransformName)) + .apply( + "Write to error Sink", + sinkTransform.addAnnotation( + "FeatureMetric", "ErrorHandler".getBytes(StandardCharsets.UTF_8))); + } + + public static class WriteErrorMetrics + extends PTransform, PCollection> { + + private final Counter errorCounter; + + public WriteErrorMetrics(String sinkTransformName) { + errorCounter = Metrics.counter("ErrorMetrics", sinkTransformName + "-input"); + } + + @Override + public PCollection expand(PCollection input) { + return input.apply(ParDo.of(new CountErrors(errorCounter))); + } + + public static class CountErrors extends DoFn { + + private final Counter errorCounter; + + public CountErrors(Counter errorCounter) { + this.errorCounter = errorCounter; + } + + @ProcessElement + public void processElement(@Element ErrorT error, OutputReceiver receiver) { + errorCounter.inc(); + receiver.output(error); + } + } + } + } + + class BadRecordErrorHandler + extends PTransformErrorHandler { + + /** Constructs a new ErrorHandler for handling BadRecords. */ + @Internal + public BadRecordErrorHandler( + PTransform, OutputT> sinkTransform, Pipeline pipeline) { + super(sinkTransform, pipeline, BadRecord.getCoder(pipeline)); + } + } + + /** + * A default, placeholder error handler that exists to allow usage of .addErrorCollection() + * without effects. This enables more simple codepaths without checking for whether the user + * configured an error handler or not. + */ + @Internal + class DefaultErrorHandler + implements ErrorHandler { + + @Override + public void addErrorCollection(PCollection errorCollection) {} + + @Override + public boolean isClosed() { + throw new IllegalArgumentException( + "No Op handler should not be closed. This implies this IO is misconfigured."); + } + + @Override + public @Nullable OutputT getOutput() { + throw new IllegalArgumentException( + "No Op handler has no output. This implies this IO is misconfigured."); + } + + @Override + public void close() { + throw new IllegalArgumentException( + "No Op handler should not be closed. This implies this IO is misconfigured."); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/package-info.java new file mode 100644 index 000000000000..c41e10143c12 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/package-info.java @@ -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. + */ +/** Provides utilities for handling errors in Pipelines. */ +@DefaultAnnotation(NonNull.class) +package org.apache.beam.sdk.transforms.errorhandling; + +import edu.umd.cs.findbugs.annotations.DefaultAnnotation; +import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BRHEnabledPTransform.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BRHEnabledPTransform.java new file mode 100644 index 000000000000..b279ad938dba --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BRHEnabledPTransform.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** + * Dummy PTransform that is configurable with a Bad Record Handler. TODO(johncasey) look to factor + * some of this out for easy use in other IOs + */ +public class BRHEnabledPTransform extends PTransform, PCollection> { + + private ErrorHandler errorHandler = new DefaultErrorHandler<>(); + + private BadRecordRouter badRecordRouter = BadRecordRouter.THROWING_ROUTER; + + private static final TupleTag RECORDS = new TupleTag<>(); + + public BRHEnabledPTransform() {} + + public BRHEnabledPTransform withBadRecordHandler(ErrorHandler errorHandler) { + this.errorHandler = errorHandler; + this.badRecordRouter = BadRecordRouter.RECORDING_ROUTER; + return this; + } + + @Override + public PCollection expand(PCollection input) { + // TODO this pattern is a clunky. Look to improve this once we have ParDo level error handling. + PCollectionTuple pCollectionTuple = + input.apply( + "NoOpDoFn", + ParDo.of(new OddIsBad(badRecordRouter)) + .withOutputTags(RECORDS, TupleTagList.of(BadRecordRouter.BAD_RECORD_TAG))); + + errorHandler.addErrorCollection( + pCollectionTuple + .get(BadRecordRouter.BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(input.getPipeline()))); + + return pCollectionTuple.get(RECORDS).setCoder(BigEndianIntegerCoder.of()); + } + + public static class OddIsBad extends DoFn { + + private final BadRecordRouter badRecordRouter; + + public OddIsBad(BadRecordRouter badRecordRouter) { + this.badRecordRouter = badRecordRouter; + } + + @ProcessElement + public void processElement(@Element Integer element, MultiOutputReceiver receiver) + throws Exception { + if (element % 2 == 0) { + receiver.get(RECORDS).output(element); + } else { + badRecordRouter.route( + receiver, + element, + BigEndianIntegerCoder.of(), + new RuntimeException("Integer was odd"), + "Integer was odd"); + } + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouterTest.java new file mode 100644 index 000000000000..fb18a6077db1 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouterTest.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import java.util.function.BiFunction; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Record; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +@RunWith(JUnit4.class) +public class BadRecordRouterTest { + + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Mock private MultiOutputReceiver outputReceiver; + + @Mock private OutputReceiver badRecordOutputReceiver; + + private static final BiFunction< + BadRecord.Builder, BadRecord.Failure.Builder, ArgumentMatcher> + ignoreStacktraceMatcher = + (expectedBuilder, failure) -> + (ArgumentMatcher) + argument -> { + // This complex matcher means we don't need to maintain an expected stacktrace + String stackTrace = argument.getFailure().getExceptionStacktrace(); + failure.setExceptionStacktrace(stackTrace); + BadRecord expected = expectedBuilder.setFailure(failure.build()).build(); + return expected.equals(argument); + }; + + @Test + public void testThrowingHandlerWithException() throws Exception { + BadRecordRouter handler = BadRecordRouter.THROWING_ROUTER; + + thrown.expect(IOException.class); + + handler.route(outputReceiver, new Object(), null, new IOException(), "desc"); + } + + @Test + public void testThrowingHandlerWithNoException() throws Exception { + BadRecordRouter handler = BadRecordRouter.THROWING_ROUTER; + + thrown.expect(RuntimeException.class); + + handler.route(outputReceiver, new Object(), null, null, "desc"); + } + + @Test + public void testRecordingHandler() throws Exception { + when(outputReceiver.get(BAD_RECORD_TAG)).thenReturn(badRecordOutputReceiver); + + BadRecordRouter handler = BadRecordRouter.RECORDING_ROUTER; + + handler.route(outputReceiver, 5, BigEndianIntegerCoder.of(), new RuntimeException(), "desc"); + + BadRecord.Builder expectedBuilder = + BadRecord.builder() + .setRecord( + Record.builder() + .setHumanReadableJsonRecord("5") + .setEncodedRecord(new byte[] {0, 0, 0, 5}) + .setCoder("BigEndianIntegerCoder") + .build()); + + BadRecord.Failure.Builder failure = + BadRecord.Failure.builder() + .setException("java.lang.RuntimeException") + .setDescription("desc"); + + verify(badRecordOutputReceiver) + .output(ArgumentMatchers.argThat(ignoreStacktraceMatcher.apply(expectedBuilder, failure))); + } + + @Test + public void testNoCoder() throws Exception { + when(outputReceiver.get(BAD_RECORD_TAG)).thenReturn(badRecordOutputReceiver); + + BadRecordRouter handler = BadRecordRouter.RECORDING_ROUTER; + + handler.route(outputReceiver, 5, null, new RuntimeException(), "desc"); + + BadRecord.Builder expectedBuilder = + BadRecord.builder().setRecord(Record.builder().setHumanReadableJsonRecord("5").build()); + + BadRecord.Failure.Builder failure = + BadRecord.Failure.builder() + .setException("java.lang.RuntimeException") + .setDescription("desc"); + + verify(badRecordOutputReceiver) + .output(ArgumentMatchers.argThat(ignoreStacktraceMatcher.apply(expectedBuilder, failure))); + } + + @Test + public void testFailingCoder() throws Exception { + when(outputReceiver.get(BAD_RECORD_TAG)).thenReturn(badRecordOutputReceiver); + + BadRecordRouter handler = BadRecordRouter.RECORDING_ROUTER; + + Coder failingCoder = + new Coder() { + @Override + public void encode(Integer value, OutputStream outStream) + throws CoderException, IOException { + throw new CoderException("Failing Coder"); + } + + @Override + public Integer decode(InputStream inStream) throws CoderException, IOException { + return null; + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} + }; + + handler.route(outputReceiver, 5, failingCoder, new RuntimeException(), "desc"); + + BadRecord.Builder expectedBuilder = + BadRecord.builder() + .setRecord( + Record.builder() + .setHumanReadableJsonRecord("5") + .setCoder(failingCoder.toString()) + .build()); + + BadRecord.Failure.Builder failure = + BadRecord.Failure.builder() + .setException("java.lang.RuntimeException") + .setDescription("desc"); + + verify(badRecordOutputReceiver) + .output(ArgumentMatchers.argThat(ignoreStacktraceMatcher.apply(expectedBuilder, failure))); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlerTest.java new file mode 100644 index 000000000000..b0a5733cb1bd --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlerTest.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import java.util.Objects; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Record; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ErrorHandlerTest { + @Rule public final TestPipeline pipeline = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(NeedsRunner.class) + public void testNoUsageErrorHandlerUsage() throws Exception { + try (BadRecordErrorHandler> eh = + pipeline.registerBadRecordErrorHandler(new DummySinkTransform<>())) {} + + pipeline.run(); + } + + @Test + public void testUnclosedErrorHandlerUsage() { + + pipeline.registerBadRecordErrorHandler(new DummySinkTransform<>()); + + // Expected to be thrown because the error handler isn't closed + thrown.expect(IllegalStateException.class); + + pipeline.run(); + } + + @Test + public void testBRHEnabledPTransform() { + PCollection record = pipeline.apply(Create.of(1, 2, 3, 4)); + record.apply(new BRHEnabledPTransform()); + + // unhandled runtime exception thrown by the BRHEnabledPTransform + thrown.expect(RuntimeException.class); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testErrorHandlerWithBRHTransform() throws Exception { + PCollection record = pipeline.apply(Create.of(1, 2, 3, 4)); + DummySinkTransform transform = new DummySinkTransform<>(); + ErrorHandler> eh = + pipeline.registerBadRecordErrorHandler(transform); + record.apply(new BRHEnabledPTransform().withBadRecordHandler(eh)); + eh.close(); + PCollection badRecords = eh.getOutput(); + + // We use a more complex satisfies statement to ensure we don't need to preserve stacktraces + // in test cases + PAssert.that(badRecords) + .satisfies( + (records) -> { + int count = 0; + for (BadRecord badRecord : records) { + count++; + + Record r = null; + + if (Objects.equals(badRecord.getRecord().getHumanReadableJsonRecord(), "1")) { + r = + Record.builder() + .setHumanReadableJsonRecord("1") + .setEncodedRecord(new byte[] {0, 0, 0, 1}) + .setCoder("BigEndianIntegerCoder") + .build(); + } else { + r = + Record.builder() + .setHumanReadableJsonRecord("3") + .setEncodedRecord(new byte[] {0, 0, 0, 3}) + .setCoder("BigEndianIntegerCoder") + .build(); + } + + BadRecord.Builder expectedBuilder = BadRecord.builder().setRecord(r); + + BadRecord.Failure.Builder failure = + BadRecord.Failure.builder() + .setException("java.lang.RuntimeException: Integer was odd") + .setDescription("Integer was odd"); + + failure.setExceptionStacktrace(badRecord.getFailure().getExceptionStacktrace()); + expectedBuilder.setFailure(failure.build()); + Assert.assertEquals("Expect failure to match", expectedBuilder.build(), badRecord); + } + Assert.assertEquals("Expect 2 errors", 2, count); + return null; + }); + + pipeline.run().waitUntilFinish(); + } + + public static class DummySinkTransform extends PTransform, PCollection> { + + @Override + public PCollection expand(PCollection input) { + return input; + } + } +} From 40685cdd0ee5311c6c9594be31e6b33d5ae547b9 Mon Sep 17 00:00:00 2001 From: Damon Date: Tue, 5 Dec 2023 12:45:25 -0800 Subject: [PATCH 034/224] [RRIO] [Call] Implement the Repeater (#29490) * [RRIO] [Call] Implement the Repeater * Define repeatable exception types * Tests validate expected errors * Refactor per PR comments --- .../beam/io/requestresponse/Repeater.java | 140 +++++++ .../UserCodeRemoteSystemException.java | 41 +++ .../beam/io/requestresponse/RepeaterTest.java | 345 ++++++++++++++++++ 3 files changed, 526 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Repeater.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeRemoteSystemException.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RepeaterTest.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Repeater.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Repeater.java new file mode 100644 index 000000000000..e298511494a1 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Repeater.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.Optional; +import java.util.Set; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; + +/** + * Repeats a method invocation when it encounters an error, pausing invocations using {@link + * Sleeper} for a {@link BackOff#nextBackOffMillis}. + */ +@AutoValue +abstract class Repeater { + + /** {@link Set} of {@link UserCodeExecutionException}s that warrant repeating. */ + static final Set> REPEATABLE_ERROR_TYPES = + ImmutableSet.of( + UserCodeRemoteSystemException.class, + UserCodeTimeoutException.class, + UserCodeQuotaException.class); + + static Builder builder() { + return new AutoValue_Repeater.Builder<>(); + } + + /** + * The {@link ThrowableFunction} to invoke repeatedly until it succeeds, throws a {@link + * UserCodeExecutionException} that is not {@link #REPEATABLE_ERROR_TYPES}, or {@link + * BackOff#STOP}. + */ + abstract ThrowableFunction getThrowableFunction(); + + /** + * The {@link Sleeper} that pauses execution of the {@link #getThrowableFunction} when it throws a + * {@link #REPEATABLE_ERROR_TYPES} {@link UserCodeExecutionException}. Uses {@link + * Sleeper#DEFAULT} by default. + */ + abstract Sleeper getSleeper(); + + /** + * The {@link BackOff} that reports to {@link #getSleeper} how long to pause execution. It reports + * a {@link BackOff#STOP} to stop repeating invocation attempts. Uses {@link + * FluentBackoff#DEFAULT#getBackOff} by default. + */ + abstract BackOff getBackOff(); + + /** + * Applies the {@link InputT} to the {@link ThrowableFunction}, returning the {@link OutputT} if + * successful. If the function throws an exception that {@link #REPEATABLE_ERROR_TYPES} contains, + * repeats the invocation after {@link Sleeper#sleep} for the amount of time reported by {@link + * BackOff#nextBackOffMillis}. Throws the latest encountered {@link UserCodeExecutionException} + * when {@link BackOff} reports a {@link BackOff#STOP}. + */ + OutputT apply(InputT input) throws UserCodeExecutionException { + Optional latestError = Optional.empty(); + long waitFor = 0L; + while (waitFor != BackOff.STOP) { + try { + getSleeper().sleep(waitFor); + return getThrowableFunction().apply(input); + } catch (UserCodeExecutionException e) { + if (!REPEATABLE_ERROR_TYPES.contains(e.getClass())) { + throw e; + } + latestError = Optional.of(e); + } catch (InterruptedException ignored) { + } + try { + waitFor = getBackOff().nextBackOffMillis(); + } catch (IOException e) { + throw new UserCodeExecutionException(e); + } + } + throw latestError.orElse( + new UserCodeExecutionException("failed to process for input: " + input)); + } + + /** + * A {@link FunctionalInterface} for executing a {@link UserCodeExecutionException} throwable + * function. + */ + @FunctionalInterface + interface ThrowableFunction { + /** Returns the result of invoking this function on the given input. */ + OutputT apply(InputT input) throws UserCodeExecutionException; + } + + @AutoValue.Builder + abstract static class Builder { + + /** See {@link #getThrowableFunction}. */ + abstract Builder setThrowableFunction( + ThrowableFunction value); + + /** See {@link #getSleeper}. */ + abstract Builder setSleeper(Sleeper value); + + abstract Optional getSleeper(); + + /** See {@link #getBackOff}. */ + abstract Builder setBackOff(BackOff value); + + abstract Optional getBackOff(); + + abstract Repeater autoBuild(); + + final Repeater build() { + if (!getSleeper().isPresent()) { + setSleeper(Sleeper.DEFAULT); + } + + if (!getBackOff().isPresent()) { + setBackOff(FluentBackoff.DEFAULT.backoff()); + } + + return autoBuild(); + } + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeRemoteSystemException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeRemoteSystemException.java new file mode 100644 index 000000000000..dac16344bec7 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/UserCodeRemoteSystemException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +/** + * A {@link UserCodeExecutionException} that signals an error with a remote system. Examples of such + * errors include an HTTP 5XX error or gRPC INTERNAL (13) error. + */ +public class UserCodeRemoteSystemException extends UserCodeExecutionException { + public UserCodeRemoteSystemException(String message) { + super(message); + } + + public UserCodeRemoteSystemException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeRemoteSystemException(Throwable cause) { + super(cause); + } + + public UserCodeRemoteSystemException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RepeaterTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RepeaterTest.java new file mode 100644 index 000000000000..14bb25ce4ad4 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RepeaterTest.java @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.io.requestresponse; + +import static org.apache.beam.io.requestresponse.Repeater.REPEATABLE_ERROR_TYPES; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertThrows; + +import java.lang.reflect.InvocationTargetException; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.UncheckedExecutionException; +import org.junit.Rule; +import org.junit.Test; + +public class RepeaterTest { + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + private static final TupleTag OUTPUT_TAG = new TupleTag() {}; + private static final TupleTag FAILURE_TAG = new TupleTag() {}; + + private static final int LIMIT = 3; + private static final FluentBackoff FLUENT_BACKOFF = FluentBackoff.DEFAULT.withMaxRetries(LIMIT); + + @Test + public void givenCallerQuotaErrorsExceedsLimit_emitsIntoFailurePCollection() { + PCollectionTuple pct = + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(LIMIT + 1, UserCodeQuotaException.class), + new SetupTeardownImpl(0))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + PAssert.that(pct.get(OUTPUT_TAG)).empty(); + PAssert.that(pct.get(FAILURE_TAG)).containsInAnyOrder(UserCodeQuotaException.class.getName()); + + pipeline.run(); + } + + @Test + public void givenSetupQuotaErrorsExceedsLimit_throws() { + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(0), + new SetupTeardownImpl(LIMIT + 1, UserCodeQuotaException.class))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + UncheckedExecutionException thrown = + assertThrows(UncheckedExecutionException.class, pipeline::run); + assertThat(thrown.getCause(), allOf(notNullValue(), instanceOf(UserCodeException.class))); + assertThat( + thrown.getCause().getCause(), + allOf(notNullValue(), instanceOf(UserCodeQuotaException.class))); + } + + @Test + public void givenCallerTimeoutErrorsExceedsLimit_emitsIntoFailurePCollection() { + PCollectionTuple pct = + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(LIMIT + 1, UserCodeTimeoutException.class), + new SetupTeardownImpl(0))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + PAssert.that(pct.get(OUTPUT_TAG)).empty(); + PAssert.that(pct.get(FAILURE_TAG)).containsInAnyOrder(UserCodeTimeoutException.class.getName()); + + pipeline.run(); + } + + @Test + public void givenSetupTimeoutErrorsExceedsLimit_throws() { + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(0), + new SetupTeardownImpl(LIMIT + 1, UserCodeTimeoutException.class))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + UncheckedExecutionException thrown = + assertThrows(UncheckedExecutionException.class, pipeline::run); + assertThat(thrown.getCause(), allOf(notNullValue(), instanceOf(UserCodeException.class))); + assertThat( + thrown.getCause().getCause(), + allOf(notNullValue(), instanceOf(UserCodeTimeoutException.class))); + } + + @Test + public void givenCallerRemoteSystemExceptionExceedsLimit_emitsIntoFailurePCollection() { + PCollectionTuple pct = + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(LIMIT + 1, UserCodeRemoteSystemException.class), + new SetupTeardownImpl(0))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + PAssert.that(pct.get(OUTPUT_TAG)).empty(); + PAssert.that(pct.get(FAILURE_TAG)) + .containsInAnyOrder(UserCodeRemoteSystemException.class.getName()); + + pipeline.run(); + } + + @Test + public void givenSetupRemoteSystemErrorsExceedsLimit_throws() { + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(0), + new SetupTeardownImpl(LIMIT + 1, UserCodeRemoteSystemException.class))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + UncheckedExecutionException thrown = + assertThrows(UncheckedExecutionException.class, pipeline::run); + assertThat(thrown.getCause(), allOf(notNullValue(), instanceOf(UserCodeException.class))); + assertThat( + thrown.getCause().getCause(), + allOf(notNullValue(), instanceOf(UserCodeRemoteSystemException.class))); + } + + @Test + public void givenCallerNonRepeatableError_emitsIntoFailurePCollection() { + PCollectionTuple pct = + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(1, UserCodeExecutionException.class), + new SetupTeardownImpl(0))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + PAssert.that(pct.get(OUTPUT_TAG)).empty(); + PAssert.that(pct.get(FAILURE_TAG)) + .containsInAnyOrder(UserCodeExecutionException.class.getName()); + + pipeline.run(); + } + + @Test + public void givenSetupNonRepeatableError_throws() { + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(0), + new SetupTeardownImpl(1, UserCodeExecutionException.class))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + UncheckedExecutionException thrown = + assertThrows(UncheckedExecutionException.class, pipeline::run); + assertThat(thrown.getCause(), allOf(notNullValue(), instanceOf(UserCodeException.class))); + assertThat( + thrown.getCause().getCause(), + allOf(notNullValue(), instanceOf(UserCodeExecutionException.class))); + } + + @Test + public void givenRepeatableErrorBelowLimit_emitsIntoOutputPCollection() { + PCollectionTuple pct = + pipeline + .apply(Create.of(1)) + .apply( + ParDo.of( + new DoFnWithRepeaters( + new CallerImpl(LIMIT - 1, UserCodeQuotaException.class), + new SetupTeardownImpl(0))) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(FAILURE_TAG))); + + PAssert.that(pct.get(OUTPUT_TAG)).containsInAnyOrder(2); + PAssert.that(pct.get(FAILURE_TAG)).empty(); + + pipeline.run(); + } + + private static class DoFnWithRepeaters extends DoFn { + private final CallerImpl caller; + private final SetupTeardownImpl setupTeardown; + + private DoFnWithRepeaters(CallerImpl caller, SetupTeardownImpl setupTeardown) { + this.caller = caller; + this.setupTeardown = setupTeardown; + } + + @Setup + public void setup() throws UserCodeExecutionException { + Repeater repeater = + Repeater.builder() + .setSleeper(new NoOpSleeper()) + .setBackOff(FLUENT_BACKOFF.backoff()) + .setThrowableFunction( + input -> { + setupTeardown.setup(); + return null; + }) + .build(); + repeater.apply(null); + } + + @ProcessElement + public void process(@Element Integer element, MultiOutputReceiver receiver) { + Repeater repeater = + Repeater.builder() + .setSleeper(new NoOpSleeper()) + .setBackOff(FLUENT_BACKOFF.backoff()) + .setThrowableFunction(caller::call) + .build(); + try { + Integer output = repeater.apply(element); + receiver.get(OUTPUT_TAG).output(output); + } catch (UserCodeExecutionException e) { + receiver.get(FAILURE_TAG).output(getRepeatableErrorTypeName(e.getClass())); + } + } + } + + private static class CallerImpl implements Caller { + + private int wantNumErrors; + private final Class wantThrowWith; + private final String exceptionName; + + private CallerImpl(int wantNumErrors) { + this(wantNumErrors, UserCodeExecutionException.class); + } + + private CallerImpl( + int wantNumErrors, Class wantThrowWith) { + this.wantNumErrors = wantNumErrors; + this.wantThrowWith = wantThrowWith; + this.exceptionName = getRepeatableErrorTypeName(wantThrowWith); + } + + @Override + public Integer call(Integer request) throws UserCodeExecutionException { + if (wantNumErrors > 0) { + wantNumErrors--; + try { + throw wantThrowWith.getConstructor(String.class).newInstance(exceptionName); + } catch (InstantiationException + | NoSuchMethodException + | InvocationTargetException + | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + return request * 2; + } + } + + private static class SetupTeardownImpl implements SetupTeardown { + private int wantNumErrors; + private final Class wantThrowWith; + private final String exceptionName; + + private SetupTeardownImpl(int wantNumErrors) { + this(wantNumErrors, UserCodeExecutionException.class); + } + + private SetupTeardownImpl( + int wantNumErrors, Class wantThrowWith) { + this.wantNumErrors = wantNumErrors; + this.wantThrowWith = wantThrowWith; + this.exceptionName = getRepeatableErrorTypeName(wantThrowWith); + } + + @Override + public void setup() throws UserCodeExecutionException { + if (wantNumErrors > 0) { + wantNumErrors--; + try { + throw wantThrowWith.getConstructor(String.class).newInstance(exceptionName); + } catch (InstantiationException + | NoSuchMethodException + | InvocationTargetException + | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static String getRepeatableErrorTypeName(Class e) { + for (Class ex : REPEATABLE_ERROR_TYPES) { + if (ex.equals(e)) { + return ex.getName(); + } + } + return e.getName(); + } + + // We don't want to wait for sleep to delay test invocations. + private static class NoOpSleeper implements Sleeper { + @Override + public void sleep(long millis) throws InterruptedException {} + } +} From 50904522a5c5b2bb0e3a7675f0479e52e8e21aa7 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Tue, 5 Dec 2023 15:39:50 -0500 Subject: [PATCH 035/224] [yaml] Unmark as experimental Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/yaml/README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index 247b42b6839a..e1528a8e4804 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -44,8 +44,9 @@ or consumption (e.g. a lineage analysis tool) and expect it to be more easily manipulated and semantically meaningful than the Beam protos themselves (which concern themselves more with execution). -It should be noted that everything here is still EXPERIMENTAL and subject -to change. Feedback is welcome at dev@apache.beam.org. +It should be noted that everything here is still under development, but any +features already included are considered stable. Feedback is welcome at +dev@apache.beam.org. ## Example pipelines From e16d401987d9656bf791afc0d4679b09b8f3fe04 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 5 Dec 2023 13:15:33 -0800 Subject: [PATCH 036/224] [YAML] Clean up some confusing error messages. (#29481) * Provide sane defaults rather than NullPtrExceptions for optional BigQuery parameters. * Better error when cross-language is used with (incompatible) local streaming Python runner. * Add format and schema to readme PubSub examples. --- ...torageWriteApiSchemaTransformProvider.java | 11 +++++--- .../runners/direct/direct_runner.py | 11 ++++++++ sdks/python/apache_beam/yaml/README.md | 28 ++++++++++++++++++- sdks/python/apache_beam/yaml/readme_test.py | 4 +-- 4 files changed, 47 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java index 39e6fd7c809d..98cc246ce0dd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java @@ -338,10 +338,13 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { if (inputRows.isBounded() == IsBounded.UNBOUNDED) { Long triggeringFrequency = configuration.getTriggeringFrequencySeconds(); Boolean autoSharding = configuration.getAutoSharding(); - Integer numStreams = configuration.getNumStreams(); + int numStreams = configuration.getNumStreams() == null ? 0 : configuration.getNumStreams(); + boolean useAtLeastOnceSemantics = + configuration.getUseAtLeastOnceSemantics() == null + ? false + : configuration.getUseAtLeastOnceSemantics(); // Triggering frequency is only applicable for exactly-once - if (configuration.getUseAtLeastOnceSemantics() == null - || !configuration.getUseAtLeastOnceSemantics()) { + if (!useAtLeastOnceSemantics) { write = write.withTriggeringFrequency( (triggeringFrequency == null || triggeringFrequency <= 0) @@ -349,7 +352,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { : Duration.standardSeconds(triggeringFrequency)); } // set num streams if specified, otherwise default to autoSharding - if (numStreams != null && numStreams > 0) { + if (numStreams > 0) { write = write.withNumStorageWriteApiStreams(numStreams); } else if (autoSharding == null || autoSharding) { write = write.withAutoSharding(); diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index db53e4122bbc..a470ba80d8ee 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -507,6 +507,17 @@ def run_pipeline(self, pipeline, options): from apache_beam.runners.direct.transform_evaluator import \ TransformEvaluatorRegistry from apache_beam.testing.test_stream import TestStream + from apache_beam.transforms.external import ExternalTransform + + class VerifyNoCrossLanguageTransforms(PipelineVisitor): + """Visitor determining whether a Pipeline uses a TestStream.""" + def visit_transform(self, applied_ptransform): + if isinstance(applied_ptransform.transform, ExternalTransform): + raise RuntimeError( + "Streaming Python direct runner " + "does not support cross-language pipelines.") + + pipeline.visit(VerifyNoCrossLanguageTransforms()) # If the TestStream I/O is used, use a mock test clock. class TestStreamUsageVisitor(PipelineVisitor): diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index e1528a8e4804..0b5c118ba8af 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -274,6 +274,13 @@ pipeline: - type: ReadFromPubSub config: topic: myPubSubTopic + format: json + schema: + type: object + properties: + col1: {type: string} + col2: {type: integer} + col3: {type: number} - type: WindowInto windowing: type: fixed @@ -282,6 +289,7 @@ pipeline: - type: WriteToPubSub config: topic: anotherPubSubTopic + format: json ``` Rather than using an explicit `WindowInto` operation, one may instead tag a @@ -295,6 +303,8 @@ pipeline: - type: ReadFromPubSub config: topic: myPubSubTopic + format: ... + schema: ... - type: SomeAggregation windowing: type: sliding @@ -303,6 +313,7 @@ pipeline: - type: WriteToPubSub config: topic: anotherPubSubTopic + format: json ``` Note that the `Sql` operation itself is often a from of aggregation, and @@ -316,6 +327,8 @@ pipeline: - type: ReadFromPubSub config: topic: myPubSubTopic + format: ... + schema: ... - type: Sql config: query: "select col1, count(*) as c from PCOLLECTION" @@ -325,6 +338,7 @@ pipeline: - type: WriteToPubSub config: topic: anotherPubSubTopic + format: json ``` The specified windowing is applied to all inputs, in this case resulting in @@ -337,11 +351,15 @@ pipeline: name: ReadLeft config: topic: leftTopic + format: ... + schema: ... - type: ReadFromPubSub name: ReadRight config: topic: rightTopic + format: ... + schema: ... - type: Sql config: @@ -364,7 +382,9 @@ pipeline: transforms: - type: ReadFromPubSub config: - topic: myPubSubTopic + topic: myPubSubTopic + format: ... + schema: ... windowing: type: fixed size: 60 @@ -374,6 +394,7 @@ pipeline: - type: WriteToPubSub config: topic: anotherPubSubTopic + format: json ``` One can also specify windowing at the top level of a pipeline (or composite), @@ -388,12 +409,15 @@ pipeline: - type: ReadFromPubSub config: topic: myPubSubTopic + format: ... + schema: ... - type: Sql config: query: "select col1, count(*) as c from PCOLLECTION" - type: WriteToPubSub config: topic: anotherPubSubTopic + format: json windowing: type: fixed size: 60 @@ -410,6 +434,8 @@ pipeline: type: ReadFromPubSub config: topic: myPubSubTopic + format: ... + schema: ... windowing: type: fixed size: 10 diff --git a/sdks/python/apache_beam/yaml/readme_test.py b/sdks/python/apache_beam/yaml/readme_test.py index 7f2d193bf35f..8ec9b22cc92f 100644 --- a/sdks/python/apache_beam/yaml/readme_test.py +++ b/sdks/python/apache_beam/yaml/readme_test.py @@ -99,7 +99,7 @@ def guess_name_and_type(expr): class FakeReadFromPubSub(beam.PTransform): - def __init__(self, topic): + def __init__(self, topic, format, schema): pass def expand(self, p): @@ -112,7 +112,7 @@ def expand(self, p): class FakeWriteToPubSub(beam.PTransform): - def __init__(self, topic): + def __init__(self, topic, format): pass def expand(self, pcoll): From dff9b107b3bc55e49fb0272e6c8d215ed4e54c0a Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 6 Dec 2023 10:19:27 -0500 Subject: [PATCH 037/224] fix whitespace check (#29634) --- sdks/python/apache_beam/yaml/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index 0b5c118ba8af..395cd5321967 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -45,7 +45,7 @@ easily manipulated and semantically meaningful than the Beam protos themselves (which concern themselves more with execution). It should be noted that everything here is still under development, but any -features already included are considered stable. Feedback is welcome at +features already included are considered stable. Feedback is welcome at dev@apache.beam.org. ## Example pipelines From ee84aaf4430a8c8b74dee363548f8f7dd0c2d039 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 6 Dec 2023 10:19:56 -0500 Subject: [PATCH 038/224] Update dataflow container tag (#29635) --- 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 fa54c6ce126e..0dfe0182eebc 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-20231116' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231205' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From 9fa7298d1ad3b0d8f44125a8193f928834c40590 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 6 Dec 2023 11:33:41 -0500 Subject: [PATCH 039/224] Shutdown most of Jenkins PostCommit (#29626) * Shutdown most of Jenkins PostCommit * Remove mass comment related steps in release guide and actions --- .github/workflows/README.md | 42 ++++++ .github/workflows/cut_release_branch.yml | 78 +---------- .../job_PostCommit_CommunityMetrics.groovy | 45 ------ ..._CrossLanguageValidatesRunner_Flink.groovy | 52 ------- ...atesRunner_JavaUsingPython_Dataflow.groovy | 49 ------- ...sRunner_PythonUsingJavaSQL_Dataflow.groovy | 49 ------- ..._CrossLanguageValidatesRunner_Samza.groovy | 52 ------- ...CrossLanguageValidatesRunner_Spark3.groovy | 51 ------- .test-infra/jenkins/job_PostCommit_Go.groovy | 43 ------ ...PostCommit_Go_ValidatesRunner_Flink.groovy | 38 ----- ...PostCommit_Go_ValidatesRunner_Samza.groovy | 38 ----- ...PostCommit_Go_ValidatesRunner_Spark.groovy | 38 ----- .../jenkins/job_PostCommit_Java.groovy | 49 ------- .../job_PostCommit_Java_Avro_Versions.groovy | 47 ------- .../job_PostCommit_Java_Dataflow.groovy | 48 ------- ...ommit_Java_Dataflow_Examples_Java11.groovy | 46 ------- ...ommit_Java_Dataflow_Examples_Java17.groovy | 46 ------- ...ostCommit_Java_Examples_Dataflow_V2.groovy | 46 ------- ...it_Java_Examples_Dataflow_V2_Java11.groovy | 51 ------- ...it_Java_Examples_Dataflow_V2_Java17.groovy | 51 ------- ...job_PostCommit_Java_Examples_Direct.groovy | 43 ------ .../job_PostCommit_Java_Examples_Flink.groovy | 43 ------ .../job_PostCommit_Java_Examples_Spark.groovy | 43 ------ ...job_PostCommit_Java_Hadoop_Versions.groovy | 48 ------- ...ostCommit_Java_Jpms_Dataflow_Java11.groovy | 49 ------- ...ostCommit_Java_Jpms_Dataflow_Java17.groovy | 51 ------- ..._PostCommit_Java_Jpms_Direct_Java11.groovy | 49 ------- ..._PostCommit_Java_Jpms_Direct_Java17.groovy | 51 ------- ...b_PostCommit_Java_Jpms_Flink_Java11.groovy | 49 ------- ...b_PostCommit_Java_Jpms_Spark_Java11.groovy | 49 ------- ...ob_PostCommit_Java_Nexmark_Dataflow.groovy | 60 -------- ...PostCommit_Java_Nexmark_Dataflow_V2.groovy | 67 --------- ...mit_Java_Nexmark_Dataflow_V2_Java11.groovy | 67 --------- ...mit_Java_Nexmark_Dataflow_V2_Java17.groovy | 67 --------- .../job_PostCommit_Java_Nexmark_Direct.groovy | 54 -------- .../job_PostCommit_Java_Nexmark_Flink.groovy | 56 -------- .../job_PostCommit_Java_Nexmark_Spark.groovy | 63 --------- ...ableValidatesRunner_Flink_Streaming.groovy | 50 ------- ..._Java_PortableValidatesRunner_Samza.groovy | 43 ------ ...bleValidatesRunner_Spark3_Streaming.groovy | 43 ------ ...PortableValidatesRunner_Spark_Batch.groovy | 44 ------ .../job_PostCommit_Java_Sickbay.groovy | 44 ------ ...ob_PostCommit_Java_SingleStoreIO_IT.groovy | 87 ------------ .../job_PostCommit_Java_Tpcds_Dataflow.groovy | 99 ------------- .../job_PostCommit_Java_Tpcds_Flink.groovy | 94 ------------- .../job_PostCommit_Java_Tpcds_Spark.groovy | 130 ------------------ ...tCommit_Java_ValidatesRunner_Direct.groovy | 43 ------ ..._Java_ValidatesRunner_Direct_Java11.groovy | 57 -------- ..._Java_ValidatesRunner_Direct_Java17.groovy | 55 -------- ...stCommit_Java_ValidatesRunner_Samza.groovy | 44 ------ ...stCommit_Java_ValidatesRunner_Spark.groovy | 44 ------ ...atesRunner_SparkStructuredStreaming.groovy | 43 ------ ...t_Java_ValidatesRunner_Spark_Java11.groovy | 60 -------- ...ommit_Java_ValidatesRunner_Twister2.groovy | 44 ------ ...t_Java_ValidatesRunner_ULR_LOOPBACK.groovy | 43 ------ .../jenkins/job_PostCommit_Javadoc.groovy | 50 ------- .../job_PostCommit_PortableJar_Flink.groovy | 38 ----- .../job_PostCommit_PortableJar_Spark.groovy | 38 ----- ...ython_Chicago_Taxi_Example_Dataflow.groovy | 65 --------- ...t_Python_Chicago_Taxi_Example_Flink.groovy | 80 ----------- ...t_Python_CrossLanguage_Gcp_Dataflow.groovy | 54 -------- ...mit_Python_CrossLanguage_Gcp_Direct.groovy | 49 ------- ...PostCommit_Python_Examples_Dataflow.groovy | 44 ------ ...ob_PostCommit_Python_Examples_Spark.groovy | 45 ------ .../job_PostCommit_Python_MongoDBIO_IT.groovy | 38 ----- ...stCommit_Python_MongoDBIO_Load_Test.groovy | 58 -------- ...ob_PostCommit_Python_Nexmark_Direct.groovy | 54 -------- .../job_PostCommit_Python_Sickbay.groovy | 49 ------- ..._Python_ValidatesContainer_Dataflow.groovy | 67 --------- ...Commit_Python_ValidatesRunner_Samza.groovy | 43 ------ .test-infra/jenkins/job_PostCommit_SQL.groovy | 45 ------ ..._PostCommit_TransformService_Direct.groovy | 53 ------- .../job_PostCommit_Website_Test.groovy | 36 ----- contributor-docs/release-guide.md | 8 +- .../src/main/scripts/verify_release_build.sh | 9 +- 75 files changed, 52 insertions(+), 3806 deletions(-) delete mode 100644 .test-infra/jenkins/job_PostCommit_CommunityMetrics.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_JavaUsingPython_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJavaSQL_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Samza.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark3.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Go.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Samza.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Hadoop_Versions.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Flink_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Jpms_Spark_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Nexmark_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Flink_Streaming.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Samza.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark3_Streaming.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Sickbay.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Tpcds_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Tpcds_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_Tpcds_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Samza.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Twister2.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_ULR_LOOPBACK.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Javadoc.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_PortableJar_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Flink.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Examples_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Examples_Spark.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_MongoDBIO_IT.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_MongoDBIO_Load_Test.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Nexmark_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_Sickbay.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Samza.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_SQL.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Website_Test.groovy diff --git a/.github/workflows/README.md b/.github/workflows/README.md index d0ed952232ed..b39dfc9d5924 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -294,6 +294,7 @@ PostCommit Jobs run in a schedule against master branch and generally do not get | [ PostCommit Java Examples Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml) | N/A |`beam_PostCommit_Java_Examples_Flink.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Flink.yml?query=event%3Aschedule) | | [ PostCommit Java Examples Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml) | N/A |`beam_PostCommit_Java_Examples_Spark.json`| [![.github/workflows/beam_PostCommit_Java_Examples_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Examples_Spark.yml?query=event%3Aschedule) | | [ PostCommit Java Hadoop Versions ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml) | N/A |`beam_PostCommit_Java_Hadoop_Versions.json`| [![.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Hadoop_Versions.yml?query=event%3Aschedule) | +| [ PostCommit Java IO Performance Tests ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml) | N/A |`beam_PostCommit_Java_IO_Performance_Tests.json`| [![.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml?query=event%3Aschedule) | [ PostCommit Java InfluxDbIO Integration Test ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml) | N/A |`beam_PostCommit_Java_InfluxDbIO_IT.json`| [![.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?query=event%3Aschedule) | [ PostCommit Java Jpms Dataflow Java11 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml) | N/A |`beam_PostCommit_Java_Jpms_Dataflow_Java11.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml?query=event%3Aschedule) | | [ PostCommit Java Jpms Dataflow Java17 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml) | N/A |`beam_PostCommit_Java_Jpms_Dataflow_Java17.json`| [![.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml?query=event%3Aschedule) | @@ -367,6 +368,47 @@ PostCommit Jobs run in a schedule against master branch and generally do not get | [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`beam_PostCommit_XVR_Spark3.json`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml?query=event%3Aschedule) | | [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`beam_Python_ValidatesContainer_Dataflow_ARM.json`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml?query=event%3Aschedule) | +### PerformanceTests and Benchmark Jobs + +| Workflow name | Matrix | Cron Status | +|:-------------:|:------:|:-----------:| +| [ CloudML Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml) | N/A | [![.github/workflows/beam_CloudML_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_CloudML_Benchmarks_Dataflow.yml?query=event%3Aschedule) +| [ Inference Python Benchmarks Dataflow ](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml) | N/A | [![.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml?query=event%3Aschedule) +| [ Java JMH ](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml) | N/A | [![.github/workflows/beam_Java_JMH.yml](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Java_JMH.yml?query=event%3Aschedule) +| [ Performance Tests AvroIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml?query=event%3Aschedule) +| [ Performance Tests AvroIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_AvroIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_AvroIOIT.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Batch Java Avro ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml) | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Batch Java Json ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml) | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Streaming Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml) | N/A | [![.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Read Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml) | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml?query=event%3Aschedule) +| [ Performance Tests BigQueryIO Write Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml) | N/A | [![.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml?query=event%3Aschedule) +| [ PerformanceTests Cdap ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml) | N/A | [![.github/workflows/beam_PerformanceTests_Cdap.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Cdap.yml?query=event%3Aschedule) +| [ PerformanceTests Compressed TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests Compressed TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests HadoopFormat ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml) | N/A | [![.github/workflows/beam_PerformanceTests_HadoopFormat.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_HadoopFormat.yml?query=event%3Aschedule) +| [ PerformanceTests JDBC ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml) | N/A | [![.github/workflows/beam_PerformanceTests_JDBC.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_JDBC.yml?query=event%3Aschedule) +| [ PerformanceTests Kafka IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml) | N/A | [![.github/workflows/beam_PerformanceTests_Kafka_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_Kafka_IO.yml?query=event%3Aschedule) +| [ PerformanceTests ManyFiles TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests ManyFiles TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests MongoDBIO IT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_MongoDBIO_IT.yml?query=event%3Aschedule) +| [ PerformanceTests ParquetIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests ParquetIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_ParquetIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_ParquetIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests PubsubIOIT Python Streaming ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml) | N/A | [![.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml?query=event%3Aschedule) +| [ PerformanceTests SingleStoreIO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml) | N/A | [![.github/workflows/beam_PerformanceTests_SingleStoreIO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SingleStoreIO.yml?query=event%3Aschedule) +| [ PerformanceTests SpannerIO Read 2GB Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml) | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml?query=event%3Aschedule) +| [ PerformanceTests SpannerIO Write 2GB Python Batch ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml) | N/A | [![.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml?query=event%3Aschedule) +| [ PerformanceTests SparkReceiver IO ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml) | N/A | [![.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SparkReceiver_IO.yml?query=event%3Aschedule) +| [ PerformanceTests SQLBigQueryIO Batch Java ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml) | N/A | [![.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml) | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT_Python.yml?query=event%3Aschedule) +| [ PerformanceTests TextIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_TextIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TextIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests TFRecordIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests TFRecordIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_TFRecordIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests WordCountIT PythonVersions ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml) | ['3.8'] | [![.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml?query=event%3Aschedule) +| [ PerformanceTests XmlIOIT HDFS ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml) | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml?query=event%3Aschedule) +| [ PerformanceTests XmlIOIT ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml) | N/A | [![.github/workflows/beam_PerformanceTests_XmlIOIT.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_XmlIOIT.yml?query=event%3Aschedule) +| [ PerformanceTests xlang KafkaIO Python ](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml) | N/A | [![.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml?query=event%3Aschedule) + ### LoadTests Jobs | Workflow name | Matrix | Cron Status | diff --git a/.github/workflows/cut_release_branch.yml b/.github/workflows/cut_release_branch.yml index 4201d6018c60..cff45e2f229e 100644 --- a/.github/workflows/cut_release_branch.yml +++ b/.github/workflows/cut_release_branch.yml @@ -21,10 +21,9 @@ # To learn more about GitHub Actions in Apache Beam check the CI.md # Workflow used after https://github.com/apache/beam/commit/4183e747becebd18becee5fff547af365910fc9c -# If help is needed debugging issues, you can view the cut_release_branch.sh and start_snapshot_build.sh scripts at that commit +# If help is needed debugging issues, you can view the cut_release_branch.sh script at that commit # for guidance on how to do this manually. -# (https://github.com/apache/beam/blob/4183e747becebd18becee5fff547af365910fc9c/release/src/main/scripts/cut_release_branch.sh and -# https://github.com/apache/beam/blob/4183e747becebd18becee5fff547af365910fc9c/release/src/main/scripts/start_snapshot_build.sh). +# (https://github.com/apache/beam/blob/4183e747becebd18becee5fff547af365910fc9c/release/src/main/scripts/cut_release_branch.sh). name: Cut Release Branch on: workflow_dispatch: @@ -37,12 +36,6 @@ on: description: Next release version required: true default: '2.XX.0' - JENKINS_USERNAME: - description: Username of the current Jenkins user. Used to update mass_comment.py with all Jenkins jobs. - required: true - JENKINS_TOKEN: - description: API Token for the current Jenkins user. Can be generated at https://ci-beam.apache.org/user//configure - required: true CREATE_RELEASE_BRANCH: description: Whether to cut the release branch. You shouldnt skip this unless it has already been completed successfully (yes/no) required: true @@ -102,11 +95,6 @@ jobs: SCRIPT_DIR: ./release/src/main/scripts RELEASE: ${{ github.event.inputs.RELEASE_VERSION }} steps: - - name: Mask Jenkins token - run: | - echo "::add-mask::$JENKINS_TOKEN" - env: - JENKINS_TOKEN: ${{ github.event.inputs.JENKINS_TOKEN }} - name: Validate Next Version run: | if [[ $NEXT_RELEASE =~ ([0-9]+\.[0-9]+\.[0-9]+) ]]; then @@ -123,34 +111,6 @@ jobs: git config user.email actions@"$RUNNER_NAME".local - name: Install xmllint run: sudo apt-get install -y libxml2-utils - - name: Update mass_comment.py with new Jenkins jobs - run: | - rm release/src/main/scripts/jenkins_jobs.txt - for obj in $(curl https://ci-beam.apache.org/api/json | jq '.jobs[]' -c); do - echo $obj - url=$(echo $obj | jq '.url' -r || echo "") - name=$(echo $obj | jq '.name' -r || echo "") - if [ "$url" != "" ] && [ "$name" != "" ]; then - curl --user $JENKINS_USERNAME:$JENKINS_TOKEN $url/config.xml > /tmp/config.xml - - if [ "$(xmllint --xpath 'string(//disabled)' /tmp/config.xml)" = "true" ]; then - echo "SKIPPING $url - disabled" - else - phrase=$(xmllint --xpath 'string(//triggers/org.jenkinsci.plugins.ghprb.GhprbTrigger/triggerPhrase)' /tmp/config.xml) - if [ "$phrase" != "" ]; then - echo "Appending ${phrase},${name}" - echo "${phrase},${name}" >> /tmp/result - fi - fi - - rm /tmp/config.xml - fi - done - - cat /tmp/result | sort | uniq | grep -i -E 'precommit|postcommit|validates|vr|example|test' | grep -v -i -E 'load|perf|website' >> release/src/main/scripts/jenkins_jobs.txt - env: - JENKINS_USERNAME: ${{ github.event.inputs.JENKINS_USERNAME }} - JENKINS_TOKEN: ${{ github.event.inputs.JENKINS_TOKEN }} - name: Update .asf.yaml to protect new release branch from force push run: | sed -i -e "s/master: {}/master: {}\n release-${RELEASE}: {}/g" .asf.yaml @@ -177,36 +137,6 @@ jobs: BRANCH_NAME: snapshot_build-${{ github.event.inputs.RELEASE_VERSION }} RELEASE_BRANCH: release-${{ github.event.inputs.RELEASE_VERSION }} steps: - - name: Install Hub - run: | - cd ~ - wget https://github.com/github/hub/releases/download/v2.14.2/hub-linux-amd64-2.14.2.tgz - tar zvxvf hub-linux-amd64-2.14.2.tgz - sudo ./hub-linux-amd64-2.14.2/install - echo "eval "$(hub alias -s)"" >> ~/.bashrc - - uses: actions/checkout@v4 - - name: Set git config - run: | - git config user.name $GITHUB_ACTOR - git config user.email actions@"$RUNNER_NAME".local - - name: Create Snapshot Branch - run: | - git remote add ${REMOTE_NAME} ${REMOTE_URL} - git checkout -b ${BRANCH_NAME} - touch empty_file.json - git add -A - git commit -m "Add empty file in order to create PR" - git push -f ${REMOTE_NAME} - - name: Create Pull Request - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + - name: Trigger snapshot build run: | - hub pull-request -b apache:${RELEASE_BRANCH} -F- <<<"[DO NOT MERGE]Start snapshot build for release process - - - Run Gradle Publish" - - echo "NOTE: If there is no jenkins job started, please comment generated PR with: Run Gradle Publish" - echo "Things remained you need to do manually after build successful:" - echo "1. Close this generated PR in github website." - echo "2. Delete your remote branch ${BRANCH_NAME} form your beam repo in github website." + echo "Working in progress. Please trigger 'Release Nightly Snapshot' GHA workflow manually." diff --git a/.test-infra/jenkins/job_PostCommit_CommunityMetrics.groovy b/.test-infra/jenkins/job_PostCommit_CommunityMetrics.groovy deleted file mode 100644 index b6b958a830a4..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CommunityMetrics.groovy +++ /dev/null @@ -1,45 +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 commonJobProperties - -job('beam_Prober_CommunityMetrics') { - description('Health check probes for the Community Metrics infrastructure') - commonJobProperties.setTopLevelMainJobProperties(delegate) - - commonJobProperties.enablePhraseTriggeringFromPullRequest(delegate, - 'Community Metrics Prober', - 'Run Community Metrics Prober') - - commonJobProperties.setAutoJob(delegate) - - commonJobProperties.enablePhraseTriggeringFromPullRequest( - delegate, - 'Community Metrics Prober', - 'Run Community Metrics Prober') - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':communityMetricsProber') - switches('--rerun-tasks') - commonJobProperties.setGradleSwitches(delegate) - } - } -} diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Flink.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Flink.groovy deleted file mode 100644 index 0d2a8612bdfe..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Flink.groovy +++ /dev/null @@ -1,52 +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 commonJobProperties -import CommonTestProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_Flink', - 'Run XVR_Flink PostCommit', 'Flink CrossLanguageValidatesRunner Tests', this) { - description('Runs the CrossLanguageValidatesRunner suite on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE FLINK USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:job-server:validatesCrossLanguageRunner") - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - // only run non-python task (e.g. GoUsingJava) once - switches("-PskipNonPythonTask=${pythonVersion != CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS[0]}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_JavaUsingPython_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_JavaUsingPython_Dataflow.groovy deleted file mode 100644 index 764bf697cbee..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_JavaUsingPython_Dataflow.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Dataflow runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_JavaUsingPython_Dataflow', - 'Run XVR_JavaUsingPython_Dataflow PostCommit', 'Dataflow CrossLanguageValidatesRunner JavaUsingPython Tests', this) { - description('Runs the CrossLanguageValidatesRunner JavaUsingPython suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE DATAFLOW JAVA WITH PYTHON EXTERNAL TRANSFORMS USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerJavaUsingPython') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJavaSQL_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJavaSQL_Dataflow.groovy deleted file mode 100644 index 8c5d7f5be8fe..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_PythonUsingJavaSQL_Dataflow.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_DATAFLOW_USING_SQL_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Dataflow runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow', - 'Run XVR_PythonUsingJavaSQL_Dataflow PostCommit', 'Dataflow CrossLanguageValidatesRunner PythonUsingJavaSQL Tests', this) { - description('Runs the CrossLanguageValidatesRunner PythonUsingJavaSQL suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_DATAFLOW_USING_SQL_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE DATAFLOW PYTHON WITH JAVA SQL TRANSFORMS USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerPythonUsingSql') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Samza.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Samza.groovy deleted file mode 100644 index 1319fdec1fcc..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Samza.groovy +++ /dev/null @@ -1,52 +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 commonJobProperties -import CommonTestProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Samza runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_Samza', - 'Run XVR_Samza PostCommit', 'Samza CrossLanguageValidatesRunner Tests', this) { - description('Runs the CrossLanguageValidatesRunner suite on the Samza runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE SAMZA USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:samza:job-server:validatesCrossLanguageRunner") - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - // only run non-python task (e.g. GoUsingJava) once - switches("-PskipNonPythonTask=${pythonVersion != CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS[0]}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark3.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark3.groovy deleted file mode 100644 index eec4eac88278..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark3.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_Spark3', - 'Run XVR_Spark3 PostCommit', 'Spark3 CrossLanguageValidatesRunner Tests', this) { - description('Runs the CrossLanguageValidatesRunner suite on the Spark3 runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE SPARK3 USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:job-server:validatesCrossLanguageRunner') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - // only run non-python task (e.g. GoUsingJava) once - switches("-PskipNonPythonTask=${pythonVersion != CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS[0]}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Go.groovy b/.test-infra/jenkins/job_PostCommit_Go.groovy deleted file mode 100644 index 3d1c6a3b53ff..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Go.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This is the Go postcommit which runs a gradle build, and the current set -// of postcommit tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Go', 'Run Go PostCommit', - 'Go PostCommit (\"Run Go PostCommit\")', this) { - description('Runs Go PostCommit tests against master.') - previousNames(/beam_PostCommit_Go_GradleBuild/) - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties( - delegate, - 'master', - 300) // increased to 5 hours. - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':goPostCommit') - commonJobProperties.setGradleSwitches(delegate) - switches('--no-parallel') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Flink.groovy deleted file mode 100644 index e0cd744dc434..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Flink.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Go integration tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Go_VR_Flink', - 'Run Go Flink ValidatesRunner', 'Go Flink ValidatesRunner Tests', this) { - description('Runs Go integration tests on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 100, true, 'beam', true, 40) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:go:test:flinkValidatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Samza.groovy b/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Samza.groovy deleted file mode 100644 index 891f25a7e12b..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Samza.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Go integration tests against the Samza runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Go_VR_Samza', - 'Run Go Samza ValidatesRunner', 'Go Samza ValidatesRunner Tests', this) { - description('Runs Go integration tests on the Samza runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:go:test:samzaValidatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Spark.groovy deleted file mode 100644 index 383d4234749c..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Go_ValidatesRunner_Spark.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Go integration tests against the Spark runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Go_VR_Spark', - 'Run Go Spark ValidatesRunner', 'Go Spark ValidatesRunner Tests', this) { - description('Runs Go integration tests on the Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:go:test:sparkValidatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java.groovy b/.test-infra/jenkins/job_PostCommit_Java.groovy deleted file mode 100644 index cb3e0930dc89..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java', 'Run Java PostCommit', - 'Java SDK Post Commit Tests', this) { - - description('Runs PostCommit tests on the Java SDK.') - previousNames(/beam_PostCommit_Java_GradleBuild/) - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':javaPostCommit') - commonJobProperties.setGradleSwitches(delegate) - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy b/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy deleted file mode 100644 index ea07cb263928..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java tests that depends on Avro against different Avro API versions -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Avro_Versions', 'Run PostCommit_Java_Avro_Versions', - 'Java Avro Versions Post Commit Tests', this) { - - description('Java Avro Versions Post Commit Tests') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":javaAvroVersionsTest") - commonJobProperties.setGradleSwitches(delegate) - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_Dataflow.groovy deleted file mode 100644 index 2ab27da0f1b0..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Dataflow.groovy +++ /dev/null @@ -1,48 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_DataflowV1', 'Run PostCommit_Java_Dataflow', - 'Dataflow Java Post Commit Tests', this) { - - description('Dataflow Java Post Commit Tests') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:google-cloud-dataflow-java:postCommit") - commonJobProperties.setGradleSwitches(delegate) - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java11.groovy deleted file mode 100644 index 7e1f383ca3c5..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java11.groovy +++ /dev/null @@ -1,46 +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 PostcommitJobBuilder -import CommonJobProperties as commonJobProperties - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_Java11', - 'Run Java examples on Dataflow Java 11', 'Google Cloud Dataflow Runner Examples Java 11', this) { - - description('Runs the Java Examples suite on the Java 11 enabled Dataflow runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:examples:java11PostCommit') - - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java17.groovy deleted file mode 100644 index 14df1a6724e9..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Dataflow_Examples_Java17.groovy +++ /dev/null @@ -1,46 +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 PostcommitJobBuilder -import CommonJobProperties as commonJobProperties - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_Java17', - 'Run Java examples on Dataflow Java 17', 'Google Cloud Dataflow Runner Examples Java 17', this) { - - description('Runs the Java Examples suite on the Java 17 enabled Dataflow runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:examples:java17PostCommit') - - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2.groovy deleted file mode 100644 index 5bfda20db3cf..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2.groovy +++ /dev/null @@ -1,46 +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 PostcommitJobBuilder -import CommonJobProperties as commonJobProperties - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2', - 'Run Java Examples on Dataflow Runner V2', 'Google Cloud Dataflow Runner V2 Examples', this) { - - description('Runs the Java Examples suite on Dataflow runner V2.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest') - - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy deleted file mode 100644 index 6229f7c48a72..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy +++ /dev/null @@ -1,51 +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 PostcommitJobBuilder -import CommonJobProperties as commonJobProperties - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_java11', - 'Run Java 11 Examples on Dataflow Runner V2', 'Google Cloud Dataflow Runner V2 Examples Java 11', this) { - - description('Runs the Java Examples suite on Dataflow runner V2 using the Java 11 SDK harness.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest') - - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - switches '-PdisableSpotlessCheck=true' - switches '-PdisableCheckStyle=true' - switches '-PtestJavaVersion=11' - switches '-PskipCheckerFramework' - switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy deleted file mode 100644 index 7e52a7e09789..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy +++ /dev/null @@ -1,51 +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 PostcommitJobBuilder -import CommonJobProperties as commonJobProperties - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_java17', - 'Run Java 17 Examples on Dataflow Runner V2', 'Google Cloud Dataflow Runner V2 Examples Java 17', this) { - - description('Runs the Java Examples suite on Dataflow runner V2 using the Java 17 SDK harness.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest') - - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - switches '-PdisableSpotlessCheck=true' - switches '-PdisableCheckStyle=true' - switches '-PtestJavaVersion=17' - switches '-PskipCheckerFramework' - switches "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Direct.groovy deleted file mode 100644 index dbea565102df..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Direct.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Java examples tests with DirectRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Direct', - 'Run Java Examples_Direct', 'Java Direct Runner Examples', this) { - - description('Run Java Examples on Direct Runner') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct:examplesIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Flink.groovy deleted file mode 100644 index bf759bac04ec..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Flink.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Java examples tests with FlinkRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Flink', - 'Run Java Examples_Flink', 'Java Flink Runner Examples', this) { - - description('Run Java Examples on Flink Runner') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:examplesIntegrationTest") - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy deleted file mode 100644 index f4af7dea6c0a..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Java examples tests with SparkRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Spark', - 'Run Java Examples_Spark', 'Java Spark Runner Examples', this) { - - description('Run Java Examples on Spark Runner') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:examplesIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Hadoop_Versions.groovy b/.test-infra/jenkins/job_PostCommit_Java_Hadoop_Versions.groovy deleted file mode 100644 index dfca212d64fb..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Hadoop_Versions.groovy +++ /dev/null @@ -1,48 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Hadoop_Versions', 'Run PostCommit_Java_Hadoop_Versions', - 'Java Hadoop Versions Post Commit Tests', this) { - - description('Java Hadoop Versions Post Commit Tests') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":javaHadoopVersionsTest") - commonJobProperties.setGradleSwitches(delegate) - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java11.groovy deleted file mode 100644 index e65bc273431a..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java11.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java11', 'Run Jpms Dataflow Java 11 PostCommit', - 'JPMS Java 11 Dataflow Post Commit Tests', this) { - - description('Runs JPMS tests on Dataflow using the Java 11 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_11_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy deleted file mode 100644 index f518985ca7a8..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java17', 'Run Jpms Dataflow Java 17 PostCommit', - 'JPMS Java 17 Dataflow Post Commit Tests', this) { - - description('Runs JPMS tests on Dataflow using the Java 17 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-PskipCheckerFramework") - switches("-PtestJavaVersion=17") - switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java11.groovy deleted file mode 100644 index 9faf78d5694a..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java11.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java11', 'Run Jpms Direct Java 11 PostCommit', - 'JPMS Java 11 direct runner Post Commit Tests', this) { - - description('Runs JPMS tests on the direct runner using the Java 11 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_11_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy deleted file mode 100644 index 04c31389ecbe..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java17', 'Run Jpms Direct Java 17 PostCommit', - 'JPMS Java 17 direct runner Post Commit Tests', this) { - - description('Runs JPMS tests on the direct runner using the Java 17 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-PskipCheckerFramework") - switches("-PtestJavaVersion=17") - switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Flink_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Flink_Java11.groovy deleted file mode 100644 index fcd9a0d4ea2f..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Flink_Java11.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Flink_Java11', 'Run Jpms Flink Java 11 PostCommit', - 'JPMS Java 11 Flink Post Commit Tests', this) { - - description('Runs JPMS tests on Flink using the Java 11 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:flinkRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_11_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Spark_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Spark_Java11.groovy deleted file mode 100644 index 35a9ef961f95..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Spark_Java11.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Spark_Java11', 'Run Jpms Spark Java 11 PostCommit', - 'JPMS Java 11 Spark Post Commit Tests', this) { - - description('Runs JPMS tests on Spark using the Java 11 SDK.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:jpms-tests:sparkRunnerIntegrationTest') - commonJobProperties.setGradleSwitches(delegate) - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_11_HOME}") - // Specify maven home on Jenkins, needed by Maven archetype integration tests. - switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy deleted file mode 100644 index 868bc69a8468..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy +++ /dev/null @@ -1,60 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of ValidatesRunner tests against the Dataflow runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Dataflow', - 'Dataflow Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Dataflow', - 'Run Dataflow Runner Nexmark Tests', 'Dataflow Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'region' : 'us-central1', - 'suite' : 'STRESS', - 'numWorkers' : 4, - 'maxNumWorkers' : 4, - 'autoscalingAlgorithm' : 'NONE', - 'nexmarkParallel' : 16, - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - Nexmark.standardJob(delegate, Runner.DATAFLOW, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2.groovy deleted file mode 100644 index a8ea21876f91..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2.groovy +++ /dev/null @@ -1,67 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of Nexmark tests against the Dataflow runner V2. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Dataflow_V2', - 'Dataflow Runner V2 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_DataflowV2', - 'Run Dataflow Runner V2 Nexmark Tests', 'Dataflow Runner V2 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2 against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'influxTags' : '{\\\"runnerVersion\\\":\\\"V2\\\",\\\"javaVersion\\\":\\\"8\\\"}', - 'exportSummaryToBigQuery' : false, - 'region' : 'us-central1', - 'suite' : 'STRESS', - 'numWorkers' : 4, - 'maxNumWorkers' : 4, - 'autoscalingAlgorithm' : 'NONE', - 'nexmarkParallel' : 16, - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - - def final JOB_SPECIFIC_SWITCHES = [ - '-Pnexmark.runner.version="V2"' - ] - - Nexmark.nonQueryLanguageJobs(delegate, Runner.DATAFLOW, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext, JOB_SPECIFIC_SWITCHES, Nexmark.DEFAULT_JAVA_RUNTIME_VERSION) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java11.groovy deleted file mode 100644 index 16866841cc72..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java11.groovy +++ /dev/null @@ -1,67 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of Nexmark tests against the Dataflow runner V2. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Dataflow_V2_Java11', - 'Dataflow Runner V2 Java 11 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2 on Java 11.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_DataflowV2_Java11', - 'Run Dataflow Runner V2 Java 11 Nexmark Tests', 'Dataflow Runner V2 Java 11 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2 on Java 11 against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'influxTags' : '{\\\"runnerVersion\\\":\\\"V2\\\",\\\"javaVersion\\\":\\\"11\\\"}', - 'exportSummaryToBigQuery' : false, - 'region' : 'us-central1', - 'suite' : 'STRESS', - 'numWorkers' : 4, - 'maxNumWorkers' : 4, - 'autoscalingAlgorithm' : 'NONE', - 'nexmarkParallel' : 16, - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - - def final JOB_SPECIFIC_SWITCHES = [ - '-Pnexmark.runner.version="V2"' - ] - - Nexmark.nonQueryLanguageJobs(delegate, Runner.DATAFLOW, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext, JOB_SPECIFIC_SWITCHES, Nexmark.JAVA_11_RUNTIME_VERSION) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java17.groovy deleted file mode 100644 index 40bfb477b568..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow_V2_Java17.groovy +++ /dev/null @@ -1,67 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of Nexmark tests against the Dataflow runner V2. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Dataflow_V2_Java17', - 'Dataflow Runner V2 Java 17 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2 on Java 17.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_DataflowV2_Java17', - 'Run Dataflow Runner V2 Java 17 Nexmark Tests', 'Dataflow Runner V2 Java 17 Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Dataflow runner V2 on Java 17 against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'influxTags' : '{\\\"runnerVersion\\\":\\\"V2\\\",\\\"javaVersion\\\":\\\"17\\\"}', - 'exportSummaryToBigQuery' : false, - 'region' : 'us-central1', - 'suite' : 'STRESS', - 'numWorkers' : 4, - 'maxNumWorkers' : 4, - 'autoscalingAlgorithm' : 'NONE', - 'nexmarkParallel' : 16, - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - - def final JOB_SPECIFIC_SWITCHES = [ - '-Pnexmark.runner.version="V2"' - ] - - Nexmark.nonQueryLanguageJobs(delegate, Runner.DATAFLOW, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext, JOB_SPECIFIC_SWITCHES, Nexmark.JAVA_17_RUNTIME_VERSION) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Direct.groovy deleted file mode 100644 index c441c1e73e39..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Direct.groovy +++ /dev/null @@ -1,54 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Direct', - 'Direct Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Direct runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Direct', - 'Run Direct Runner Nexmark Tests', 'Direct Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Direct runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'suite' : 'SMOKE', - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - Nexmark.standardJob(delegate, Runner.DIRECT, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Flink.groovy deleted file mode 100644 index cdbb2ab8afec..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Flink.groovy +++ /dev/null @@ -1,56 +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 commonJobProperties -import CommonTestProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Flink', - 'Flink Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Flink runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf', true, 40) - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Flink', - 'Run Flink Runner Nexmark Tests', 'Flink Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Flink runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam', true, 40) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'suite' : 'SMOKE', - 'streamTimeout' : 60, - ] - - Nexmark.standardJob(delegate, Runner.FLINK, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Spark.groovy deleted file mode 100644 index 7df873f63067..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Spark.groovy +++ /dev/null @@ -1,63 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -// This job runs the suite of ValidatesRunner tests against the Spark runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Spark', - 'Spark Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Spark runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_Spark', - 'Run Spark Runner Nexmark Tests', 'Spark Runner Nexmark Tests', this) { - - description('Runs the Nexmark suite on the Spark runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'suite' : 'SMOKE', - 'streamTimeout': 60 - ] - // Spark doesn't run streaming jobs, therefore run only batch variants. - Nexmark.batchOnlyJob(delegate, Runner.SPARK, SDK.JAVA, JOB_SPECIFIC_OPTIONS, triggeringContext) - - def final SPARK_STRUCTURED_STREAMING_JOB_SPECIFIC_OPTIONS = [ - 'suite' : 'SMOKE', - 'streamTimeout': 60, - // Skip query 3 (SparkStructuredStreamingRunner does not support State/Timers yet) - 'skipQueries' : 3, - ] - Nexmark.batchOnlyJob(delegate, Runner.SPARK_STRUCTURED_STREAMING, SDK.JAVA, SPARK_STRUCTURED_STREAMING_JOB_SPECIFIC_OPTIONS, triggeringContext) -} diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Flink_Streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Flink_Streaming.groovy deleted file mode 100644 index fb3e24867f8c..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Flink_Streaming.groovy +++ /dev/null @@ -1,50 +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 commonJobProperties -import CommonTestProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Flink_Streaming', - 'Run Java Flink PortableValidatesRunner Streaming', 'Java Flink PortableValidatesRunner Streaming Tests', this) { - description('Runs the Java PortableValidatesRunner suite on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:job-server:validatesPortableRunnerStreaming") - commonJobProperties.setGradleSwitches(delegate) - } - // TODO(https://github.com/apache/beam/issues/20668): Enable this test suite once we have support. - //gradle { - // rootBuildScriptDir(commonJobProperties.checkoutDir) - // tasks(":runners:flink:${CommonTestProperties.getFlinkVersion()}:job-server:validatesPortableRunnerStreamingCheckpoint") - // commonJobProperties.setGradleSwitches(delegate) - //} - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Samza.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Samza.groovy deleted file mode 100644 index 881d4e1ca85f..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Samza.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Java ValidatesRunner tests against the Samza runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Samza', - 'Run Java Samza PortableValidatesRunner', 'Java Samza PortableValidatesRunner Tests', this) { - description('Runs the Java PortableValidatesRunner suite on the Samza runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:samza:job-server:validatesPortableRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark3_Streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark3_Streaming.groovy deleted file mode 100644 index 2164ab554b3b..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark3_Streaming.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Java ValidatesRunner tests against the Spark runner in streaming mode. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Spark3_Streaming', - 'Run Java Spark v3 PortableValidatesRunner Streaming', 'Java Spark v3 PortableValidatesRunner Streaming Tests', this) { - description('Runs the Java PortableValidatesRunner suite on the Spark v3 runner in streaming mode.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:job-server:validatesPortableRunnerStreaming') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy deleted file mode 100644 index ba08cc2e4404..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Java ValidatesRunner tests against the Spark runner in batch mode. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Spark_Batch', - 'Run Java Spark PortableValidatesRunner Batch', 'Java Spark PortableValidatesRunner Batch Tests', this) { - description('Runs the Java PortableValidatesRunner suite on the Spark runner in batch mode.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:job-server:validatesPortableRunnerBatch') - tasks(':runners:spark:3:job-server:validatesPortableRunnerDocker') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Sickbay.groovy b/.test-infra/jenkins/job_PostCommit_Java_Sickbay.groovy deleted file mode 100644 index 40e6ac61fa83..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Sickbay.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Java sickbay tests. -this.buildSchedule = 'H H * * *' -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Sickbay', - 'Run Java Sickbay', 'Java Sickbay Tests', this) { - - description('Run Java Sickbay Tests') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Execute shell command to run sickbay tests. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':javaPostCommitSickbay') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy deleted file mode 100644 index 901b76364333..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy +++ /dev/null @@ -1,87 +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 PostcommitJobBuilder -import Kubernetes - -String jobName = "beam_PostCommit_Java_SingleStoreIO_IT" - -void waitForPodWithLabel(job, Kubernetes k8s, String label) { - job.steps { - shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for-pod-with-label.sh ${label} 600") - } -} - -void waitFor(job, Kubernetes k8s, String resource) { - job.steps { - shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for.sh ${resource} 600") - } -} - - -// This job runs the integration test of java SingleStoreIO class. -PostcommitJobBuilder.postCommitJob(jobName, - 'Run Java SingleStoreIO_IT', 'Java SingleStoreIO Integration Test',this) { - description('Runs the Java SingleStoreIO Integration Test.') - - // Set common parameters. - common.setTopLevelMainJobProperties(delegate) - - // Deploy SingleStoreDB cluster - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-rbac.yaml")) - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster-crd.yaml")) - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-operator.yaml")) - waitForPodWithLabel(delegate, k8s, "sdb-operator") - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster.yaml")) - waitFor(delegate, k8s, "memsqlclusters.memsql.com") - - String singlestoreHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("svc-sdb-cluster-ddl", singlestoreHostName) - - // Define test options - Map pipelineOptions = [ - tempRoot : 'gs://temp-storage-for-perf-tests', - project : 'apache-beam-testing', - runner : 'DataflowRunner', - singleStoreServerName : "\$${singlestoreHostName}", - singleStoreUsername : "admin", - singleStorePassword : "secretpass", - singleStorePort: "3306", - numberOfRecords: "1000", - ] - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("--info") - switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") - switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIODefaultMapperIT") - tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOSchemaTransformIT") - tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOConnectionAttributesIT") - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Dataflow.groovy deleted file mode 100644 index f2152861be90..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Dataflow.groovy +++ /dev/null @@ -1,99 +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 commonJobProperties -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static TpcdsDatabaseProperties.tpcdsBigQueryArgs -import static TpcdsDatabaseProperties.tpcdsInfluxDBArgs -import static TpcdsDatabaseProperties.tpcdsQueriesArg - - -// This job runs the suite of Tpcds tests against the Dataflow runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Dataflow', - 'Dataflow Runner Tpcds Tests', this) { - description('Runs the Tpcds suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING DATAFLOW RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:google-cloud-dataflow-java"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--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', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Dataflow', - 'Run Dataflow Runner Tpcds Tests', 'Dataflow Runner Tpcds Tests', this) { - - description('Runs the Tpcds suite on the Dataflow runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING DATAFLOW RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:google-cloud-dataflow-java"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--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', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Flink.groovy deleted file mode 100644 index b6bff8cfffd3..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Flink.groovy +++ /dev/null @@ -1,94 +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 commonJobProperties -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static TpcdsDatabaseProperties.tpcdsBigQueryArgs -import static TpcdsDatabaseProperties.tpcdsInfluxDBArgs -import static TpcdsDatabaseProperties.tpcdsQueriesArg - -// This job runs the Tpcds benchmark suite against the Flink runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Flink', - 'Flink Runner Tpcds Tests', this) { - description('Runs the Tpcds suite on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING FLINK RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:flink:1.13"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=FlinkRunner', - '--parallelism=4', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/flink/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Flink', - 'Run Flink Runner Tpcds Tests', 'Flink Runner Tpcds Tests', this) { - - description('Runs the Tpcds suite on the Flink runner against a Pull Request, on demand.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING FLINK RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:flink:1.13"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=FlinkRunner', - '--parallelism=4', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/flink/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Spark.groovy deleted file mode 100644 index 0bd28b232a9e..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_Tpcds_Spark.groovy +++ /dev/null @@ -1,130 +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 commonJobProperties -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder -import InfluxDBCredentialsHelper - -import static TpcdsDatabaseProperties.tpcdsBigQueryArgs -import static TpcdsDatabaseProperties.tpcdsInfluxDBArgs -import static TpcdsDatabaseProperties.tpcdsQueriesArg - -// This job runs the Tpcds benchmark suite against the Spark runner. -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Spark', - 'Spark Runner Tpcds Tests', this) { - description('Runs the Tpcds suite on the Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING SPARK 3 RDD RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:spark:3"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=SparkRunner', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/spark3-rdd/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - shell('echo "*** RUN TPC-DS USING SPARK 3 DATASET RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:spark:3"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=SparkStructuredStreamingRunner', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/spark3-dataset/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Spark', - 'Run Spark Runner Tpcds Tests', 'Spark Runner Tpcds Tests', this) { - - description('Runs the Tpcds suite on the Spark runner against a Pull Request, on demand.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - InfluxDBCredentialsHelper.useCredentials(delegate) - - // Gradle goals for this job. - steps { - shell('echo "*** RUN TPC-DS USING SPARK 3 RDD RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:spark:3"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=SparkRunner', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/spark3-rdd/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - shell('echo "*** RUN TPC-DS USING SPARK 3 DATASET RUNNER ***"') - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:testing:tpcds:run') - commonJobProperties.setGradleSwitches(delegate) - switches('-Ptpcds.runner=":runners:spark:3"' + - ' -Ptpcds.args="' + - [ - commonJobProperties.mapToArgString(tpcdsBigQueryArgs), - commonJobProperties.mapToArgString(tpcdsInfluxDBArgs), - '--runner=SparkStructuredStreamingRunner', - '--dataSize=1GB', - '--sourceType=PARQUET', - '--dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned', - '--resultsDirectory=gs://beam-tpcds/results/spark3-dataset/', - '--tpcParallel=1', - '--queries=' + tpcdsQueriesArg - ].join(' ')) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct.groovy deleted file mode 100644 index 13173390be42..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Direct -// runner compiled with Java 8. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Direct', - 'Run Direct ValidatesRunner', 'Direct Runner ValidatesRunner Tests for Java', this) { - - description('Builds the Direct Runner and runs ValidatesRunner test suite in Java.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct-java:validatesRunner') - } - } - - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java11.groovy deleted file mode 100644 index ee9a99d4d834..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java11.groovy +++ /dev/null @@ -1,57 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests using Java 11 against the Direct -// runner compiled with Java 8. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Direct_Java11', - 'Run Direct ValidatesRunner Java 11', 'Direct Runner ValidatesRunner Tests for Java 11', this) { - - description('Builds the Direct Runner with Java 8 and runs ValidatesRunner test suite in Java 11.') - - def JAVA_11_HOME = '/usr/lib/jvm/java-11-openjdk-amd64' - def JAVA_8_HOME = '/usr/lib/jvm/java-8-openjdk-amd64' - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct-java:shadowJar') - tasks(':runners:direct-java:shadowTestJar') - switches("-Dorg.gradle.java.home=${JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct-java:validatesRunner') - switches("-Dorg.gradle.java.home=${JAVA_11_HOME}") - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - } - } - - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java17.groovy deleted file mode 100644 index 50681914cd5c..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Direct_Java17.groovy +++ /dev/null @@ -1,55 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests using Java 17 against the Direct -// runner compiled with Java 8. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Direct_Java17', - 'Run Direct ValidatesRunner Java 17', 'Direct Runner ValidatesRunner Tests for Java 17', this) { - - description('Builds the Direct Runner with Java 8 and runs ValidatesRunner test suite in Java 17.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct-java:shadowJar') - tasks(':runners:direct-java:shadowTestJar') - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:direct-java:validatesRunner') - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_17_HOME}") - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - } - } - - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Samza.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Samza.groovy deleted file mode 100644 index 44231c2b07ec..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Samza.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Samza runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Samza', - 'Run Samza ValidatesRunner', 'Apache Samza Runner ValidatesRunner Tests', this) { - description('Runs the ValidatesRunner suite on the Samza runner.') - previousNames(/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/) - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:samza:validatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy deleted file mode 100644 index 0cb6bd2d3710..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Spark runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Spark', - 'Run Spark ValidatesRunner', 'Apache Spark Runner ValidatesRunner Tests', this) { - description('Runs the ValidatesRunner suite on the Spark runner.') - previousNames(/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/) - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:validatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy deleted file mode 100644 index 31d5d5cf07b4..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Spark runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming', - 'Run Spark StructuredStreaming ValidatesRunner', 'Apache Spark SparkStructuredStreaming Runner ValidatesRunner Tests', this) { - description('Runs the ValidatesRunner suite on the Structured Streaming Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:validatesStructuredStreamingRunnerBatch') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark_Java11.groovy deleted file mode 100644 index d0da52927b8a..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark_Java11.groovy +++ /dev/null @@ -1,60 +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 commonJobProperties -import PostcommitJobBuilder - - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Spark_Java11', - 'Run Spark ValidatesRunner Java 11', 'Apache Spark Runner ValidatesRunner Tests On Java 11', this) { - - description('Runs the ValidatesRunner suite on the Spark runner with Java 11.') - - def JAVA_11_HOME = '/usr/lib/jvm/java-11-openjdk-amd64' - def JAVA_8_HOME = '/usr/lib/jvm/java-8-openjdk-amd64' - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 270) - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:jar') - tasks(':runners:spark:3:testJar') - switches("-Dorg.gradle.java.home=${JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:3:validatesRunner') - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - switches('-x jar') - switches('-x testJar') - switches('-x classes') - switches('-x testClasses') - switches("-Dorg.gradle.java.home=${JAVA_11_HOME}") - - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Twister2.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Twister2.groovy deleted file mode 100644 index fa33ded11187..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Twister2.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of ValidatesRunner tests against the Twister2 runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Twister2', - 'Run Twister2 ValidatesRunner', 'Twister2 Runner ValidatesRunner Tests', this) { - description('Runs the ValidatesRunner suite on the Twister2 runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - previousNames(/beam_PostCommit_Java_ValidatesRunner_Twister2_Gradle/) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:twister2:validatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_ULR_LOOPBACK.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_ULR_LOOPBACK.groovy deleted file mode 100644 index 855e8ac5400e..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_ULR_LOOPBACK.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Direct -// runner compiled with Java 8. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_ULR', - 'Run ULR Loopback ValidatesRunner', 'Universal Local Runner ValidatesRunner Tests for Java, LOOPBACK mode', this) { - - description('Builds the Universal Local Runner and runs the Java ValidatesRunner test suite in LOOPBACK mode (no Docker).') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:portability:java:ulrLoopbackValidatesRunner') - } - } - - } diff --git a/.test-infra/jenkins/job_PostCommit_Javadoc.groovy b/.test-infra/jenkins/job_PostCommit_Javadoc.groovy deleted file mode 100644 index ae2ab53bd3fe..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Javadoc.groovy +++ /dev/null @@ -1,50 +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 commonJobProperties -import PostcommitJobBuilder - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Javadoc', 'Run Javadoc PostCommit', - 'Javadoc generation', this) { - - description('PostCommit job that generates aggregated Javadoc') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - publishers { - archiveJavadoc { - javadocDir("src/sdks/java/javadoc/build/docs/javadoc") - keepAll(false) - } - recordIssues { - tools { - java() - javaDoc() - } - enabledForFailure(true) - } - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:java:javadoc:aggregateJavadoc') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy b/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy deleted file mode 100644 index 0c6f51f8be54..000000000000 --- a/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// Tests creation and execution of portable pipeline Jars on the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_PortableJar_Flink', - 'Run PortableJar_Flink PostCommit', 'Flink Portable Jar Tests', this) { - description('Tests creation and execution of portable pipeline Jars on the Flink runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:portable:py38:testPipelineJarFlinkRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_PortableJar_Spark.groovy b/.test-infra/jenkins/job_PostCommit_PortableJar_Spark.groovy deleted file mode 100644 index 1f1963a9b2e4..000000000000 --- a/.test-infra/jenkins/job_PostCommit_PortableJar_Spark.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// Tests creation and execution of portable pipeline Jars on the Spark runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_PortableJar_Spark', - 'Run PortableJar_Spark PostCommit', 'Spark Portable Jar Tests', this) { - description('Tests creation and execution of portable pipeline Jars on the Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:portable:py38:testPipelineJarSparkRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Dataflow.groovy deleted file mode 100644 index bd04d3e0c3a5..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Dataflow.groovy +++ /dev/null @@ -1,65 +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 commonJobProperties -import PhraseTriggeringPostCommitBuilder -import CronJobBuilder -import LoadTestsBuilder - -def chicagoTaxiJob = { scope -> - scope.description('Runs the Chicago Taxi Example on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(scope) - - def pipelineOptions = [ - num_workers : 5, - autoscaling_algorithm: 'NONE', - ] - - // Gradle goals for this job. - scope.steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - commonJobProperties.setGradleSwitches(delegate) - tasks(':sdks:python:test-suites:dataflow:chicagoTaxiExample') - switches('-PgcsRoot=gs://temp-storage-for-perf-tests/chicago-taxi') - switches("-PpipelineOptions=\"${LoadTestsBuilder.parseOptions(pipelineOptions)}\"") - } - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PostCommit_Python_Chicago_Taxi_Dataflow', - 'Run Chicago Taxi on Dataflow', - 'Chicago Taxi Example on Dataflow ("Run Chicago Taxi on Dataflow")', - this - ) { - chicagoTaxiJob(delegate) - } - -// TODO(https://github.com/apache/beam/issues/19973): Chicago Taxi Example doesn't work in Python 3. -// Uncomment below once it is fixed. -// -// CronJobBuilder.cronJob( -// 'beam_PostCommit_Python_Chicago_Taxi_Dataflow', -// 'H H * * *', -// this -// ) { -// chicagoTaxiJob(delegate) -// } diff --git a/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Flink.groovy b/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Flink.groovy deleted file mode 100644 index 6cf852a16893..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Chicago_Taxi_Example_Flink.groovy +++ /dev/null @@ -1,80 +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 commonJobProperties -import CronJobBuilder -import Flink -import LoadTestsBuilder -import PhraseTriggeringPostCommitBuilder - -import static LoadTestsBuilder.DOCKER_BEAM_JOBSERVER -import static LoadTestsBuilder.DOCKER_CONTAINER_REGISTRY -import static PythonTestProperties.CHICAGO_TAXI_EXAMPLE_FLINK_PYTHON_VERSION - -def chicagoTaxiJob = { scope -> - scope.description('Runs the Chicago Taxi Example on the Flink runner.') - commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 120) - - def numberOfWorkers = 5 - def beamSdkDockerImage = "beam_python${CHICAGO_TAXI_EXAMPLE_FLINK_PYTHON_VERSION}_sdk:latest" - - Flink flink = new Flink(scope, 'beam_PostCommit_Python_Chicago_Taxi_Flink') - flink.setUp( - [ - "${DOCKER_CONTAINER_REGISTRY}/${beamSdkDockerImage}" - ], - numberOfWorkers, - "${DOCKER_BEAM_JOBSERVER}/beam_flink${CommonTestProperties.getFlinkVersion()}_job_server:latest") - - def pipelineOptions = [ - parallelism : numberOfWorkers, - job_endpoint : 'localhost:8099', - environment_options : "docker_container_image=${DOCKER_CONTAINER_REGISTRY}/${beamSdkDockerImage}", - environment_type : 'DOCKER', - execution_mode_for_batch: 'BATCH_FORCED', - ] - - scope.steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:portable:py${CHICAGO_TAXI_EXAMPLE_FLINK_PYTHON_VERSION.replace('.', '')}:chicagoTaxiExample") - switches('-PgcsRoot=gs://temp-storage-for-perf-tests/chicago-taxi') - switches("-PpipelineOptions=\"${LoadTestsBuilder.parseOptions(pipelineOptions)}\"") - } - } -} - -PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PostCommit_Python_Chicago_Taxi_Flink', - 'Run Chicago Taxi on Flink', - 'Chicago Taxi Example on Flink ("Run Chicago Taxi on Flink")', - this - ) { - chicagoTaxiJob(delegate) - } - -// TODO(https://github.com/apache/beam/issues/19973): Chicago Taxi Example doesn't work in Python 3. -// Uncomment below once it is fixed. -// -// CronJobBuilder.cronJob( -// 'beam_PostCommit_Python_Chicago_Taxi_Flink', -// 'H H * * *', -// this -// ) { -// chicagoTaxiJob(delegate) -// } diff --git a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Dataflow.groovy deleted file mode 100644 index 1280fcb4e233..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Dataflow.groovy +++ /dev/null @@ -1,54 +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 commonJobProperties -import PostcommitJobBuilder - - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - - -// This job runs end-to-end cross language GCP IO tests with DataflowRunner. -// Collects tests with the @pytest.mark.uses_gcp_java_expansion_service decorator -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Xlang_Gcp_Dataflow', - 'Run Python_Xlang_Gcp_Dataflow PostCommit', 'Python_Xlang_Gcp_Dataflow (\"Run Python_Xlang_Gcp_Dataflow PostCommit\")', this) { - description('Runs end-to-end cross language GCP IO tests on the Dataflow runner. \"Run Python_Xlang_Gcp_Dataflow PostCommit\"') - - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - - // Gradle goals for this job. - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:dataflow:gcpCrossLanguagePostCommit") - commonJobProperties.setGradleSwitches(delegate) - switches("-PuseWheelDistribution") - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Direct.groovy deleted file mode 100644 index e4bf771be1ae..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_CrossLanguage_Gcp_Direct.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - - -// This job runs end-to-end cross language GCP IO tests with DirectRunner. -// Collects tests with the @pytest.mark.uses_gcp_java_expansion_service decorator -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Xlang_Gcp_Direct', - 'Run Python_Xlang_Gcp_Direct PostCommit', 'Python_Xlang_Gcp_Direct (\"Run Python_Xlang_Gcp_Direct PostCommit\")', this) { - description('Runs end-to-end cross language GCP IO tests on the Direct runner. \"Run Python_Xlang_Gcp_Direct PostCommit\"') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:direct:gcpCrossLanguagePostCommit") - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_Examples_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_Examples_Dataflow.groovy deleted file mode 100644 index 8f91f8102edc..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Examples_Dataflow.groovy +++ /dev/null @@ -1,44 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Python examples tests with DataflowRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Examples_Dataflow', - 'Run Python Examples_Dataflow', 'Python Dataflow Runner Examples', this) { - - description('Runs the Python Examples with DataflowRunner') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:dataflow:examplesPostCommit") - commonJobProperties.setGradleSwitches(delegate) - switches("-PuseWheelDistribution") - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_Examples_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Python_Examples_Spark.groovy deleted file mode 100644 index 6277106c8cbc..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Examples_Spark.groovy +++ /dev/null @@ -1,45 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Python examples tests with SparkRunner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Examples_Spark', - 'Run Python Examples_Spark', 'Python Spark Runner Examples', this) { - - description('Runs the Python Examples with Spark Runner') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to run examples. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:portable:sparkExamplesPostCommit") - commonJobProperties.setGradleSwitches(delegate) - } - } - } - - diff --git a/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_IT.groovy deleted file mode 100644 index 437d6ab9613c..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_IT.groovy +++ /dev/null @@ -1,38 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the integration test of python mongodbio class. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_MongoDBIO_IT', - 'Run Python MongoDBIO_IT', 'Python MongoDBIO Integration Test',this) { - description('Runs the Python MongoDBIO Integration Test.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:direct:mongodbioIT') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_Load_Test.groovy b/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_Load_Test.groovy deleted file mode 100644 index 9a66b51e5423..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_MongoDBIO_Load_Test.groovy +++ /dev/null @@ -1,58 +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 - -String jobName = "beam_python_mongoio_load_test" - -job(jobName) { - common.setTopLevelMainJobProperties(delegate) - common.setAutoJob(delegate, 'H H/12 * * *') - common.enablePhraseTriggeringFromPullRequest( - delegate, - 'Python MongoDBIO Load Test', - 'Run Python MongoDBIO Load Test') - - String namespace = common.getKubernetesNamespace(jobName) - String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) - Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) - - k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/mongodb/load-balancer/mongo.yml")) - String mongoHostName = "LOAD_BALANCER_IP" - k8s.loadBalancerIP("mongo-load-balancer-service", mongoHostName) - - Map pipelineOptions = [ - temp_location: 'gs://temp-storage-for-perf-tests/loadtests', - project : 'apache-beam-testing', - region : 'us-central1', - mongo_uri : "mongodb://root:uuinkkS@\$${mongoHostName}:27017", - num_documents: '1000000', - batch_size : '10000', - runner : 'DataflowRunner', - num_workers : '5' - ] - - steps { - gradle { - rootBuildScriptDir(common.checkoutDir) - common.setGradleSwitches(delegate) - switches("-Popts=\'${common.mapToArgString(pipelineOptions)}\'") - tasks(":sdks:python:test-suites:dataflow:mongodbioIT") - } - } -} diff --git a/.test-infra/jenkins/job_PostCommit_Python_Nexmark_Direct.groovy b/.test-infra/jenkins/job_PostCommit_Python_Nexmark_Direct.groovy deleted file mode 100644 index 6927dbe49cdb..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Nexmark_Direct.groovy +++ /dev/null @@ -1,54 +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 commonJobProperties -import CommonTestProperties.Runner -import CommonTestProperties.SDK -import CommonTestProperties.TriggeringContext -import NexmarkBuilder as Nexmark -import NoPhraseTriggeringPostCommitBuilder -import PhraseTriggeringPostCommitBuilder - -NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Python_Nexmark_Direct', - 'Python Direct Runner Nexmark Tests', this) { - - description('Runs the Python Nexmark suite on the Direct runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240, true, 'beam-perf') - - commonJob(delegate, TriggeringContext.POST_COMMIT) - } - -PhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Python_Nexmark_Direct', - 'Run Python Direct Runner Nexmark Tests', 'Python Direct Runner Nexmark Tests', this) { - - description('Runs the Python Nexmark suite on the Direct runner against a Pull Request, on demand.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - commonJob(delegate, TriggeringContext.PR) - } - -private void commonJob(delegate, TriggeringContext triggeringContext) { - def final JOB_SPECIFIC_OPTIONS = [ - 'suite' : 'SMOKE', - 'enforceEncodability' : true, - 'enforceImmutability' : true - ] - Nexmark.standardPythonJob(delegate, Runner.DIRECT, SDK.PYTHON, JOB_SPECIFIC_OPTIONS, triggeringContext) -} diff --git a/.test-infra/jenkins/job_PostCommit_Python_Sickbay.groovy b/.test-infra/jenkins/job_PostCommit_Python_Sickbay.groovy deleted file mode 100644 index f9504ee811f3..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_Sickbay.groovy +++ /dev/null @@ -1,49 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.ALL_SUPPORTED_VERSIONS - -// This job defines the Python postcommit sickbay tests. -this.buildSchedule = 'H H * * *' -ALL_SUPPORTED_VERSIONS.each { pythonVersion -> - def versionSuffix = pythonVersion.replace('.', '') - PostcommitJobBuilder.postCommitJob("beam_PostCommit_Sickbay_Python${versionSuffix}", - "Run Python ${pythonVersion} PostCommit Sickbay", - "Python${versionSuffix}_PC_Sickbay(\"Run Python ${pythonVersion} PostCommit Sickbay\")", this) { - description("Runs Python postcommit sickbay tests using Python ${pythonVersion}.") - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 180) - - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to test Python SDK. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(":sdks:python:test-suites:dataflow:py${versionSuffix}:postCommitSickbay") - commonJobProperties.setGradleSwitches(delegate) - } - } - } -} \ No newline at end of file diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy deleted file mode 100644 index 254096448668..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy +++ /dev/null @@ -1,67 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.VALIDATES_CONTAINER_DATAFLOW_PYTHON_VERSIONS - -// This job runs the suite of Python ValidatesContainer tests against the -// Dataflow runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Py_ValCont', - 'Run Python Dataflow ValidatesContainer', 'Google Cloud Dataflow Runner Python ValidatesContainer Tests', this) { - description('Runs Python ValidatesContainer suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to test Python SDK. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:dataflow:validatesContainerTests') - commonJobProperties.setGradleSwitches(delegate) - } - } - } - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Py_ValCont_with_RC', - 'Run Python RC Dataflow ValidatesContainer', 'Google Cloud Dataflow Runner Python ValidatesContainer Tests with RC Dependencies', this) { - description('Runs Python ValidatesContainer suite on the Dataflow runner by installing Release Candidates.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute shell command to test Python SDK. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:dataflow:validatesContainerTests') - switches('-PtestRCDependencies=true') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Samza.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Samza.groovy deleted file mode 100644 index c49e3677ab8a..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Samza.groovy +++ /dev/null @@ -1,43 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Python ValidatesRunner tests against the Samza runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_VR_Samza', - 'Run Python Samza ValidatesRunner', 'Python Samza ValidatesRunner Tests', this) { - description('Runs the Python ValidatesRunner suite on the Samza runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins. - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:portable:samzaValidatesRunner') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_SQL.groovy b/.test-infra/jenkins/job_PostCommit_SQL.groovy deleted file mode 100644 index 9207cdbae17d..000000000000 --- a/.test-infra/jenkins/job_PostCommit_SQL.groovy +++ /dev/null @@ -1,45 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the Java postcommit tests, including the suite of integration -// tests. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_SQL', 'Run SQL PostCommit', - 'SQL Post Commit Tests', this) { - - description('Runs PostCommit tests for Beam SQL.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sqlPostCommit') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy deleted file mode 100644 index 03d29069a52c..000000000000 --- a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy +++ /dev/null @@ -1,53 +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 commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs multi-language pipelines using the Docker Compose based TransformService against the Direct runner. -// Collects tests with the @pytest.mark.uses_transform_service decorator -PostcommitJobBuilder.postCommitJob('beam_PostCommit_TransformService_Direct', - 'Run TransformService_Direct PostCommit', 'Direct TransformService Tests', this) { - description('Runs the TransformService suite on the Direct runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/pytest*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN TRANSFORM SERVICE Python SDK TESTS USING THE DIRECT RUNNER AND THE PYTHON VERSION ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava') - commonJobProperties.setGradleSwitches(delegate) - switches '-PtestJavaVersion=11' - switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" - switches("-PuseWheelDistribution") - switches("-PpythonVersion=${pythonVersion}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Website_Test.groovy b/.test-infra/jenkins/job_PostCommit_Website_Test.groovy deleted file mode 100644 index 6c1c9cc48f5b..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Website_Test.groovy +++ /dev/null @@ -1,36 +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 commonJobProperties -import PostcommitJobBuilder - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Website_Test', 'Run Full Website Test', - 'Full Website Test', this) { - - description('Test to validate the Beam website, including external links.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 30, true, 'git-websites') - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':website:testWebsite -PdisableExternal=false') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index e00ecd694336..f9a7ab1e85cb 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -288,18 +288,14 @@ diagram: This should be accomplished by the [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) -workflow. This workflow will also update -[mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) -to contain all of the active Jenkins jobs. +workflow. The following must be manually done or confirmed: - [ ] The `master` branch has the SNAPSHOT/dev version incremented. - [ ] The release branch has the SNAPSHOT/dev version to be released. - [ ] The Dataflow container image should be modified to the version to be released. -- [ ] Due to a bug/limitation in the workflow, you must navigate to the pull - request found in the logs and comment `Run Gradle Publish`. -- [ ] After publish, close the PR. +- [ ] Due to current limitation in the workflow, you must navigate to https://github.com/apache/beam/actions/workflows/beam_Release_NightlySnapshot.yml and click "Run workflow" and select the branch just created (release-2.xx.0) to build a snapshot. - [ ] Manually update `CHANGES.md` on `master` by adding a new section for the next release ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). diff --git a/release/src/main/scripts/verify_release_build.sh b/release/src/main/scripts/verify_release_build.sh index 51008d40831f..d304456ecb4a 100755 --- a/release/src/main/scripts/verify_release_build.sh +++ b/release/src/main/scripts/verify_release_build.sh @@ -27,9 +27,8 @@ # Instructions: https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line # 2. Please set RELEASE_BUILD_CONFIGS in script.config before running this # script. -# 3. Please manually comment trigger phrases to the created PR to start -# Gradle release build and all PostCommit jobs, or run mass_comment.py -# to do so. Phrases are listed in COMMENTS_TO_ADD in mass_comment.py. +# 3. Please manually start "Release Nightly Snapshot" GHA workflow on the +# release branch to start Gradle release build. . script.config @@ -144,9 +143,9 @@ if [[ ! -z `which hub` ]]; then hub pull-request -b apache:${RELEASE_BRANCH} -h ${GITHUB_USERNAME}:${WORKING_BRANCH} -F- <<<"[DO NOT MERGE] Run all PostCommit and PreCommit Tests against Release Branch - You can run many tests automatically using release/src/main/scripts/mass_comment.py." + Tests run on GHA should be queued shortly." echo "" - echo "[NOTE]: Please make sure all test targets (GHA and Jenkins) have been invoked." + echo "[NOTE]: Please make sure all test targets (GHA) have been invoked." echo "Please check the test results. If there is any failure, follow the policy in release guide." fi From 50e0751d11c8a3fc8607e0d2109b2e911be72925 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 6 Dec 2023 11:50:18 -0500 Subject: [PATCH 040/224] Bump github.com/nats-io/nats-server/v2 from 2.10.5 to 2.10.6 in /sdks (#29629) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.5 to 2.10.6. - [Release notes](https://github.com/nats-io/nats-server/releases) - [Changelog](https://github.com/nats-io/nats-server/blob/main/.goreleaser.yml) - [Commits](https://github.com/nats-io/nats-server/compare/v2.10.5...v2.10.6) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 33e3e7507cb5..40899ff53e7a 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,7 +45,7 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 - github.com/nats-io/nats-server/v2 v2.10.5 + github.com/nats-io/nats-server/v2 v2.10.6 github.com/nats-io/nats.go v1.31.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.0 @@ -146,7 +146,7 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect - github.com/klauspost/compress v1.17.2 // indirect + github.com/klauspost/compress v1.17.3 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect github.com/magiconair/properties v1.8.7 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 0f61b39482a4..6fb7c9bdc093 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -321,8 +321,8 @@ github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4= -github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.17.3 h1:qkRjuerhUU1EmXLYGkSH6EZL+vPSxIrYjLNAK4slzwA= +github.com/klauspost/compress v1.17.3/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -366,8 +366,8 @@ github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7P github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= github.com/nats-io/jwt/v2 v2.5.3 h1:/9SWvzc6hTfamcgXJ3uYRpgj+QuY2aLNqRiqrKcrpEo= github.com/nats-io/jwt/v2 v2.5.3/go.mod h1:iysuPemFcc7p4IoYots3IuELSI4EDe9Y0bQMe+I3Bf4= -github.com/nats-io/nats-server/v2 v2.10.5 h1:hhWt6m9ja/mNnm6ixc85jCthDaiUFPaeJI79K/MD980= -github.com/nats-io/nats-server/v2 v2.10.5/go.mod h1:xUMTU4kS//SDkJCSvFwN9SyJ9nUuLhSkzB/Qz0dvjjg= +github.com/nats-io/nats-server/v2 v2.10.6 h1:40U3ngyAKyC1tNT4Kw7PjuvivY74NTYD3qyIHxZUHKQ= +github.com/nats-io/nats-server/v2 v2.10.6/go.mod h1:IrTXS8o4Roa3G2kW8L5mEtSdmSrFjKhYb/m2g0gQ/vc= github.com/nats-io/nats.go v1.31.0 h1:/WFBHEc/dOKBF6qf1TZhrdEfTmOZ5JzdJ+Y3m6Y/p7E= github.com/nats-io/nats.go v1.31.0/go.mod h1:di3Bm5MLsoB4Bx61CBTsxuarI36WbhAwOm8QrW39+i8= github.com/nats-io/nkeys v0.4.6 h1:IzVe95ru2CT6ta874rt9saQRkWfe2nFj1NtvYSLqMzY= From 9090b5b38430dc01f5b29c995f99ea4def2e0c6d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 6 Dec 2023 08:51:22 -0800 Subject: [PATCH 041/224] Better introspection of PTransforms defined with @ptransform_fn. (#29589) --- sdks/python/apache_beam/transforms/ptransform.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index fcff86d4c50c..0296f7ea21ea 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -37,6 +37,8 @@ class and wrapper class that allows lambda functions to be used as # pytype: skip-file import copy +import functools +import inspect import itertools import json import logging @@ -1076,6 +1078,11 @@ def callable_ptransform_factory(*args, **kwargs): 'type hints for %s: %s', res.default_label(), res.get_type_hints()) return res + # The signature of this PTransform constructor is that of fn minus the first + # argument (which is where the pvalue is passed during expand). + callable_ptransform_factory.__signature__ = inspect.signature( # type: ignore + functools.partial(fn, None)) + return callable_ptransform_factory From b566480cec6d413efa0d62711b3f3f9ca7a0f6fd Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 6 Dec 2023 12:02:41 -0500 Subject: [PATCH 042/224] fix: have a timeout on the wait when closing BigtableIO#Writer (#29548) * fix: have a timeout on the wait when closing BigtableIO#Writer * refactor * update code * address comments * address comments and fix hang * put service.close in final block --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 81 ++++++++++++++----- .../sdk/io/gcp/bigtable/BigtableService.java | 9 +-- .../io/gcp/bigtable/BigtableServiceImpl.java | 63 +++++++++------ .../io/gcp/bigtable/BigtableWriteOptions.java | 5 ++ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 11 +-- .../gcp/bigtable/BigtableServiceImplTest.java | 15 +++- 6 files changed, 119 insertions(+), 65 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 ad978e95016a..82c2d3142482 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 @@ -60,6 +60,7 @@ import org.apache.beam.sdk.io.range.ByteKey; import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.io.range.ByteKeyRangeTracker; +import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; @@ -180,6 +181,27 @@ * .withBatchElements(100)); // every batch will have 100 elements * } * + *

    Configure timeout for writes: + * + *

    {@code
    + * // Let each attempt run for 1 second, retry if the attempt failed.
    + * // Give up after the request is retried for 60 seconds.
    + * Duration attemptTimeout = Duration.millis(1000);
    + * Duration operationTimeout = Duration.millis(60 * 1000);
    + * data.apply("write",
    + *     BigtableIO.write()
    + *         .withProjectId("project")
    + *         .withInstanceId("instance")
    + *         .withTableId("table")
    + *         .withAttemptTimeout(attemptTimeout)
    + *         .withOperationTimeout(operationTimeout));
    + * }
    + * + *

    You can also limit the wait time in the finish bundle step by setting the + * bigtable_writer_wait_timeout_ms experimental flag when you run the pipeline. For example, + * --experiments=bigtable_writer_wait_timeout_ms=60000 will limit the wait time in finish bundle to + * be 10 minutes. + * *

    Optionally, BigtableIO.write() may be configured to emit {@link BigtableWriteResult} elements * after each group of inputs is written to Bigtable. These can be used to then trigger user code * after writes have completed. See {@link org.apache.beam.sdk.transforms.Wait} for details on the @@ -1118,6 +1140,8 @@ public static class WriteWithResults extends PTransform< PCollection>>, PCollection> { + private static final String BIGTABLE_WRITER_WAIT_TIMEOUT_MS = "bigtable_writer_wait_timeout_ms"; + private final BigtableConfig bigtableConfig; private final BigtableWriteOptions bigtableWriteOptions; @@ -1138,8 +1162,23 @@ public PCollection expand( bigtableConfig.validate(); bigtableWriteOptions.validate(); + // Get experimental flag and set on BigtableWriteOptions + PipelineOptions pipelineOptions = input.getPipeline().getOptions(); + String closeWaitTimeoutStr = + ExperimentalOptions.getExperimentValue(pipelineOptions, BIGTABLE_WRITER_WAIT_TIMEOUT_MS); + Duration closeWaitTimeout = null; + if (closeWaitTimeoutStr != null) { + long closeWaitTimeoutMs = Long.parseLong(closeWaitTimeoutStr); + checkState(closeWaitTimeoutMs > 0, "Close wait timeout must be positive"); + closeWaitTimeout = Duration.millis(closeWaitTimeoutMs); + } + return input.apply( - ParDo.of(new BigtableWriterFn(factory, bigtableConfig, bigtableWriteOptions))); + ParDo.of( + new BigtableWriterFn( + factory, + bigtableConfig, + bigtableWriteOptions.toBuilder().setCloseWaitTimeout(closeWaitTimeout).build()))); } @Override @@ -1195,6 +1234,7 @@ private static class BigtableWriterFn this.writeOptions = writeOptions; this.failures = new ConcurrentLinkedQueue<>(); this.id = factory.newId(); + LOG.debug("Created Bigtable Write Fn with writeOptions {} ", writeOptions); } @StartBundle @@ -1205,7 +1245,7 @@ public void startBundle(StartBundleContext c) throws IOException { if (bigtableWriter == null) { serviceEntry = factory.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); - bigtableWriter = serviceEntry.getService().openForWriting(writeOptions.getTableId().get()); + bigtableWriter = serviceEntry.getService().openForWriting(writeOptions); } } @@ -1227,27 +1267,26 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { - bigtableWriter.flush(); - checkForFailures(); - LOG.debug("Wrote {} records", recordsWritten); + try { + if (bigtableWriter != null) { + bigtableWriter.close(); + bigtableWriter = null; + } - for (Map.Entry entry : seenWindows.entrySet()) { - c.output( - BigtableWriteResult.create(entry.getValue()), - entry.getKey().maxTimestamp(), - entry.getKey()); - } - } + checkForFailures(); + LOG.debug("Wrote {} records", recordsWritten); - @Teardown - public void tearDown() throws Exception { - if (bigtableWriter != null) { - bigtableWriter.close(); - bigtableWriter = null; - } - if (serviceEntry != null) { - serviceEntry.close(); - serviceEntry = null; + for (Map.Entry entry : seenWindows.entrySet()) { + c.output( + BigtableWriteResult.create(entry.getValue()), + entry.getKey().maxTimestamp(), + entry.getKey()); + } + } finally { + if (serviceEntry != null) { + serviceEntry.close(); + serviceEntry = null; + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index c0f88331bd6e..b529d6530ef2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -45,13 +45,6 @@ interface Writer { CompletionStage writeRecord(KV> record) throws IOException; - /** - * Flushes the writer. - * - * @throws IOException if any writes did not succeed - */ - void flush() throws IOException; - /** * Closes the writer. * @@ -88,7 +81,7 @@ interface Reader { Reader createReader(BigtableSource source) throws IOException; /** Returns a {@link Writer} that will write to the specified table. */ - Writer openForWriting(String tableId) throws IOException; + Writer openForWriting(BigtableWriteOptions writeOptions) throws IOException; /** * Returns a set of row keys sampled from the underlying table. These contain information about diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 6e98b7e7866a..344229b383fd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -19,7 +19,9 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors.directExecutor; +import com.google.api.core.ApiFuture; import com.google.api.gax.batching.Batcher; +import com.google.api.gax.batching.BatchingException; import com.google.api.gax.grpc.GrpcCallContext; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.ResponseObserver; @@ -61,6 +63,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.ServiceCallMetric; @@ -106,8 +109,8 @@ class BigtableServiceImpl implements BigtableService { RetrySettings retry = settings.getStubSettings().readRowsSettings().getRetrySettings(); this.readAttemptTimeout = Duration.millis(retry.getInitialRpcTimeout().toMillis()); this.readOperationTimeout = Duration.millis(retry.getTotalTimeout().toMillis()); - LOG.info("Started Bigtable service with settings {}", settings); this.client = BigtableDataClient.create(settings); + LOG.info("Started Bigtable service with settings {}", settings); } private final BigtableDataClient client; @@ -119,8 +122,13 @@ class BigtableServiceImpl implements BigtableService { private final Duration readOperationTimeout; @Override - public BigtableWriterImpl openForWriting(String tableId) { - return new BigtableWriterImpl(client, projectId, instanceId, tableId); + public BigtableWriterImpl openForWriting(BigtableWriteOptions writeOptions) { + return new BigtableWriterImpl( + client, + projectId, + instanceId, + writeOptions.getTableId().get(), + writeOptions.getCloseWaitTimeout()); } @VisibleForTesting @@ -470,50 +478,55 @@ static class BigtableWriterImpl implements Writer { private String projectId; private String instanceId; private String tableId; + private Duration closeWaitTimeout; private Distribution bulkSize = Metrics.distribution("BigTable-" + tableId, "batchSize"); private Distribution latency = Metrics.distribution("BigTable-" + tableId, "batchLatencyMs"); BigtableWriterImpl( - BigtableDataClient client, String projectId, String instanceId, String tableId) { + BigtableDataClient client, + String projectId, + String instanceId, + String tableId, + Duration closeWaitTimeout) { this.projectId = projectId; this.instanceId = instanceId; this.tableId = tableId; + this.closeWaitTimeout = closeWaitTimeout; this.bulkMutation = client.newBulkMutationBatcher(tableId); } - @Override - public void flush() throws IOException { - if (bulkMutation != null) { - try { - stopwatch.start(); - bulkMutation.flush(); - bulkSize.update(outstandingMutations); - outstandingMutations = 0; - stopwatch.stop(); - latency.update(stopwatch.elapsed(TimeUnit.MILLISECONDS)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - // We fail since flush() operation was interrupted. - throw new IOException(e); - } - } - } - @Override public void close() throws IOException { if (bulkMutation != null) { try { stopwatch.start(); - bulkMutation.flush(); - bulkMutation.close(); + // closeAsync will send any remaining elements in the batch. + // If the experimental close wait timeout flag is set, + // set a timeout waiting for the future. + ApiFuture future = bulkMutation.closeAsync(); + if (Duration.ZERO.isShorterThan(closeWaitTimeout)) { + future.get(closeWaitTimeout.getMillis(), TimeUnit.MILLISECONDS); + } else { + future.get(); + } bulkSize.update(outstandingMutations); outstandingMutations = 0; stopwatch.stop(); latency.update(stopwatch.elapsed(TimeUnit.MILLISECONDS)); + } catch (BatchingException e) { + // Ignore batching failures because element failures are tracked as is in + // BigtableIOWriteFn. + // TODO: Bigtable client already tracks BatchingExceptions, use BatchingExceptions + // instead of tracking them separately in BigtableIOWriteFn. + } catch (TimeoutException e) { + // We fail because future.get() timed out + throw new IOException("BulkMutation took too long to close", e); + } catch (ExecutionException e) { + throw new IOException("Failed to close batch", e.getCause()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - // We fail since flush() operation was interrupted. + // We fail since close() operation was interrupted. throw new IOException(e); } bulkMutation = null; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java index 05e0e915f12a..a63cc575809b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -60,6 +60,9 @@ abstract class BigtableWriteOptions implements Serializable { /** Returns true if batch write flow control is enabled. Otherwise return false. */ abstract @Nullable Boolean getFlowControl(); + /** Returns the time to wait when closing the writer. */ + abstract @Nullable Duration getCloseWaitTimeout(); + abstract Builder toBuilder(); static Builder builder() { @@ -87,6 +90,8 @@ abstract static class Builder { abstract Builder setFlowControl(boolean enableFlowControl); + abstract Builder setCloseWaitTimeout(Duration timeout); + abstract BigtableWriteOptions build(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 714dc9f8619d..6cda518eb124 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1695,8 +1695,8 @@ public FakeBigtableReader createReader(BigtableSource source) { } @Override - public FakeBigtableWriter openForWriting(String tableId) { - return new FakeBigtableWriter(tableId); + public FakeBigtableWriter openForWriting(BigtableWriteOptions writeOptions) { + return new FakeBigtableWriter(writeOptions.getTableId().get()); } @Override @@ -1748,8 +1748,8 @@ public FakeBigtableReader createReader(BigtableSource source) { } @Override - public FailureBigtableWriter openForWriting(String tableId) { - return new FailureBigtableWriter(tableId, this, failureOptions); + public FailureBigtableWriter openForWriting(BigtableWriteOptions writeOptions) { + return new FailureBigtableWriter(writeOptions.getTableId().get(), this, failureOptions); } @Override @@ -1929,9 +1929,6 @@ public CompletionStage writeRecord(KV fakeFuture = SettableApiFuture.create(); + when(mockBatcher.closeAsync()).thenReturn(fakeFuture); ArgumentCaptor captor = ArgumentCaptor.forClass(RowMutationEntry.class); ApiFuture fakeResponse = SettableApiFuture.create(); - when(mockBatcher.add(any(RowMutationEntry.class))).thenReturn(fakeResponse); + when(mockBatcher.add(any(RowMutationEntry.class))) + .thenAnswer( + invocation -> { + fakeFuture.set(null); + return fakeResponse; + }); BigtableService.Writer underTest = new BigtableServiceImpl.BigtableWriterImpl( mockBigtableDataClient, bigtableDataSettings.getProjectId(), bigtableDataSettings.getInstanceId(), - TABLE_ID); + TABLE_ID, + Duration.millis(60000)); ByteString key = ByteString.copyFromUtf8("key"); Mutation mutation = @@ -780,7 +788,6 @@ public void testWrite() throws IOException, InterruptedException { assertEquals( "Family", captor.getValue().toProto().getMutations(0).getSetCell().getFamilyName()); underTest.close(); - verify(mockBatcher, times(1)).flush(); } private void verifyMetricWasSet(String method, String status, long count) { From 643a07610d7c369256eac7c88b5aa356d57cdf9f Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 6 Dec 2023 12:03:55 -0500 Subject: [PATCH 043/224] Add buildx install to postcommit_XVR_Go (#29623) * Add buildx install to postcommit_XVR_Go * Mirror go postcommit --- .../workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index f674412f195e..4325df19d632 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -74,6 +74,11 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: 3.8 + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v1 + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker us.gcr.io - name: run XVR GoUsingJava Dataflow script env: USER: github-actions @@ -93,4 +98,4 @@ jobs: 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 + files: '**/build/test-results/**/*.xml' From 3d8bb4a007f7b84b29482e9c6e7253d830343c48 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 6 Dec 2023 12:09:13 -0500 Subject: [PATCH 044/224] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#29631) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.16.8 to 1.16.9. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.16.9/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.16.8...v1.16.9) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 16 ++++++++-------- sdks/go.sum | 32 ++++++++++++++++---------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 40899ff53e7a..6035fa8884d0 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -32,7 +32,7 @@ require ( cloud.google.com/go/storage v1.35.1 github.com/aws/aws-sdk-go-v2 v1.23.5 github.com/aws/aws-sdk-go-v2/config v1.25.8 - github.com/aws/aws-sdk-go-v2/credentials v1.16.8 + github.com/aws/aws-sdk-go-v2/credentials v1.16.9 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 github.com/aws/smithy-go v1.18.1 @@ -106,18 +106,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.18.1 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.2 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.2 // 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 6fb7c9bdc093..2c875905d607 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -89,18 +89,18 @@ github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/y github.com/aws/aws-sdk-go-v2/config v1.25.8 h1:CHr7PIzyfevjNiqL9rU6xoqHZKCO2ldY6LmvRDfpRuI= github.com/aws/aws-sdk-go-v2/config v1.25.8/go.mod h1:zefIy117FDPOVU0xSOFG8mx9kJunuVopzI639tjYXc0= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.8 h1:phw9nRLy/77bPk6Mfu2SHCOnHwfVB7WWrOa5rZIY2Fc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.8/go.mod h1:MrS4SOin6adbO6wgWhdifyPiq+TX7fPPwyA/ZLC1F5M= +github.com/aws/aws-sdk-go-v2/credentials v1.16.9 h1:LQo3MUIOzod9JdUK+wxmSdgzLVYUbII3jXn3S/HJZU0= +github.com/aws/aws-sdk-go-v2/credentials v1.16.9/go.mod h1:R7mDuIJoCjH6TxGUc/cylE7Lp/o0bhKVoxdBThsjqCM= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8 h1:tQZLSPC2Zj2CqZHonLmWEvCsbpMX5tQvaYJWHadcPek= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.8/go.mod h1:5+YpvTHDFffykWr5qAGjqwoh8oVYZOddL3sSrEN7lws= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7 h1:eMqD7ku6WGdmcWWXPYun9m6yk6feSULLhJlAtN6rYG4= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.7/go.mod h1:0oBIfcDV6LScxEW0VgOqxT3e4aqKRp+SYhB9wAd5E3Q= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7 h1:+XYhWhgWs5F3Zx8oa49CXzNvfXrItaDjZB/M172fcHQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.7/go.mod h1:L6tcSRyCGxcKfDWUrmv2jv8G1cLDU7d0FUpEFpG9bVE= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= @@ -112,8 +112,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1: github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7 h1:dU+ZyhvqMB/T/TxjGagHMCdyUiqaThRIaMu3YvKiSQI= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.7/go.mod h1:SGORuNqoXyWfTvTp/gBGJfv8jRvW/+nha0XhnIXVI+o= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -121,13 +121,13 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.1 h1:V40g2daNO3l1J94JYwqfkyvQMYXi5I25fs3fNQW8iDs= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.1/go.mod h1:0ZWQJP/mBOUxkCvZKybZNz1XmdUKSBxoF0dzgfxtvDs= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1 h1:uQrj7SpUNC3r55vc1CDh3qV9wJC66lz546xM9dhSo5s= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.1/go.mod h1:oyaTk5xEAOuPXX1kCD7HmIeuLqdj3Bk5yGkqGXtGi14= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.2 h1:xJPydhNm0Hiqct5TVKEuHG7weC0+sOs4MUnd7A5n5F4= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.2/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2 h1:8dU9zqA77C5egbU6yd4hFLaiIdPv3rU+6cp7sz5FjCU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.1 h1:K33V7L0XDdb23FMOZySr8bon1jou5SHn1fiv7NJ1SUg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.1/go.mod h1:YtXUl/sfnS06VksYhr855hTQf2HphfT1Xv/EwuzbPjg= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.2 h1:fFrLsy08wEbAisqW3KDl/cPHrF43GmV79zXB9EwJiZw= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.2/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From cdbd77898a82eb8707d4954fc787789bdd6b587a Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Wed, 6 Dec 2023 14:22:58 -0500 Subject: [PATCH 045/224] [yaml] fix yaml label appending Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/yaml/yaml_transform.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index ff5547db034c..8a5ccb3bb8b0 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -962,7 +962,9 @@ def expand(self, pcolls): python_provider = yaml_provider.InlineProvider({}) options = pipeline.options.view_as(GoogleCloudOptions) - options.labels = ["yaml=true"] + if not options.labels: + options.labels = [] + options.labels += ["yaml=true"] result = expand_transform( self._spec, From aef21959bf6f41b4fb646ef06da97c8b8adbcb8d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 6 Dec 2023 15:39:58 -0500 Subject: [PATCH 046/224] Split BigQuery DatasetService and WriteStreamService (#29604) * Split BigQuery DatasetService and WriteStreamService * Avoid unnecessary STORAGE_WRITE_API client creation and teardown when DatasetService is not used for storage write api * Fix override in test fixture * Add missing datasetService.close() in DoFn teardown for completeness * Though currently DatasetServiceImpl.close() is a noop --- .../it/gcp/bigquery/BigQueryStreamingLT.java | 8 +-- .../sdk/io/gcp/bigquery/AppendClientInfo.java | 4 +- .../sdk/io/gcp/bigquery/BigQueryServices.java | 8 ++- .../io/gcp/bigquery/BigQueryServicesImpl.java | 62 ++++++++++++------- .../StorageApiFinalizeWritesDoFn.java | 30 ++++----- .../StorageApiFlushAndFinalizeDoFn.java | 22 ++++--- .../StorageApiWriteUnshardedRecords.java | 33 +++++++--- .../StorageApiWritesShardedRecords.java | 29 +++++++-- .../io/gcp/testing/FakeBigQueryServices.java | 7 ++- .../io/gcp/testing/FakeDatasetService.java | 3 +- .../bigquery/BigQueryServicesImplTest.java | 62 ++++++++----------- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 4 +- 12 files changed, 165 insertions(+), 107 deletions(-) diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java index 4589f79f1aaa..214d2745f03e 100644 --- a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java @@ -428,14 +428,14 @@ public CrashingBigQueryServices(Integer crashIntervalSeconds) { } @Override - public DatasetService getDatasetService(BigQueryOptions options) { - return new CrashingDatasetService(options); + public WriteStreamService getWriteStreamService(BigQueryOptions options) { + return new CrashingWriteStreamService(options); } - private class CrashingDatasetService extends BigQueryServicesImpl.DatasetServiceImpl { + private class CrashingWriteStreamService extends BigQueryServicesImpl.WriteStreamServiceImpl { private Instant lastCrash; - public CrashingDatasetService(BigQueryOptions bqOptions) { + public CrashingWriteStreamService(BigQueryOptions bqOptions) { super(bqOptions); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java index ee21e1185f37..3094af5855e9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java @@ -105,7 +105,7 @@ public AppendClientInfo withNoAppendClient() { } public AppendClientInfo withAppendClient( - BigQueryServices.DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService, Supplier getStreamName, boolean useConnectionPool, AppendRowsRequest.MissingValueInterpretation missingValueInterpretation) @@ -117,7 +117,7 @@ public AppendClientInfo withAppendClient( return toBuilder() .setStreamName(streamName) .setStreamAppendClient( - datasetService.getStreamAppendClient( + writeStreamService.getStreamAppendClient( streamName, getDescriptor(), useConnectionPool, missingValueInterpretation)) .build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index c9c96eb35f3f..a8e1adf643ab 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -63,6 +63,9 @@ public interface BigQueryServices extends Serializable { /** Returns a real, mock, or fake {@link DatasetService}. */ DatasetService getDatasetService(BigQueryOptions bqOptions); + /** Returns a real, mock, or fake {@link WriteStreamService}. */ + WriteStreamService getWriteStreamService(BigQueryOptions bqOptions); + /** Returns a real, mock, or fake {@link StorageClient}. */ StorageClient getStorageClient(BigQueryOptions bqOptions) throws IOException; @@ -112,7 +115,7 @@ JobStatistics dryRunQuery( } /** An interface to get, create and delete Cloud BigQuery datasets and tables. */ - public interface DatasetService extends AutoCloseable { + interface DatasetService extends AutoCloseable { // maps the values at // https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/get#TableMetadataView @@ -201,7 +204,10 @@ long insertAll( /** Patch BigQuery {@link Table} description. */ Table patchTableDescription(TableReference tableReference, @Nullable String tableDescription) throws IOException, InterruptedException; + } + /** An interface to get, create and flush Cloud BigQuery STORAGE API write streams. */ + interface WriteStreamService extends AutoCloseable { /** Create a Write Stream for use with the Storage Write API. */ WriteStream createWriteStream(String tableUrn, WriteStream.Type type) throws IOException, InterruptedException; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index fb288fc1d512..bb3b99f6fcd5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -197,6 +197,11 @@ public DatasetService getDatasetService(BigQueryOptions options) { return new DatasetServiceImpl(options); } + @Override + public WriteStreamService getWriteStreamService(BigQueryOptions options) { + return new WriteStreamServiceImpl(options); + } + @Override public StorageClient getStorageClient(BigQueryOptions options) throws IOException { return new StorageClientImpl(options); @@ -563,64 +568,43 @@ public static class DatasetServiceImpl implements DatasetService { private final ApiErrorExtractor errorExtractor; private final Bigquery client; - private final @Nullable BigQueryWriteClient newWriteClient; private final PipelineOptions options; private final long maxRowsPerBatch; private final long maxRowBatchSize; - private final long storageWriteMaxInflightRequests; - private final long storageWriteMaxInflightBytes; // aggregate the total time spent in exponential backoff private final Counter throttlingMsecs = Metrics.counter(DatasetServiceImpl.class, "throttling-msecs"); private @Nullable BoundedExecutorService executor; - private final BigQueryIOMetadata bqIOMetadata; @VisibleForTesting - DatasetServiceImpl( - Bigquery client, @Nullable BigQueryWriteClient newWriteClient, PipelineOptions options) { + DatasetServiceImpl(Bigquery client, PipelineOptions options) { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); this.errorExtractor = new ApiErrorExtractor(); this.client = client; - this.newWriteClient = newWriteClient; this.options = options; this.maxRowsPerBatch = bqOptions.getMaxStreamingRowsToBatch(); this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); - this.storageWriteMaxInflightRequests = bqOptions.getStorageWriteMaxInflightRequests(); - this.storageWriteMaxInflightBytes = bqOptions.getStorageWriteMaxInflightBytes(); - this.bqIOMetadata = BigQueryIOMetadata.create(); this.executor = null; } @VisibleForTesting - DatasetServiceImpl( - Bigquery client, - BigQueryWriteClient newWriteClient, - PipelineOptions options, - long maxRowsPerBatch) { + DatasetServiceImpl(Bigquery client, PipelineOptions options, long maxRowsPerBatch) { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); this.errorExtractor = new ApiErrorExtractor(); this.client = client; - this.newWriteClient = newWriteClient; this.options = options; this.maxRowsPerBatch = maxRowsPerBatch; this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); - this.storageWriteMaxInflightRequests = bqOptions.getStorageWriteMaxInflightRequests(); - this.storageWriteMaxInflightBytes = bqOptions.getStorageWriteMaxInflightBytes(); - this.bqIOMetadata = BigQueryIOMetadata.create(); this.executor = null; } public DatasetServiceImpl(BigQueryOptions bqOptions) { this.errorExtractor = new ApiErrorExtractor(); this.client = newBigQueryClient(bqOptions).build(); - this.newWriteClient = newBigQueryWriteClient(bqOptions); this.options = bqOptions; this.maxRowsPerBatch = bqOptions.getMaxStreamingRowsToBatch(); this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); - this.storageWriteMaxInflightRequests = bqOptions.getStorageWriteMaxInflightRequests(); - this.storageWriteMaxInflightBytes = bqOptions.getStorageWriteMaxInflightBytes(); - this.bqIOMetadata = BigQueryIOMetadata.create(); this.executor = null; } @@ -1344,6 +1328,38 @@ public Table patchTableDescription( ALWAYS_RETRY); } + @Override + public void close() throws Exception { + // Nothing to close + } + } + + @VisibleForTesting + public static class WriteStreamServiceImpl implements WriteStreamService { + private final BigQueryWriteClient newWriteClient; + private final long storageWriteMaxInflightRequests; + private final long storageWriteMaxInflightBytes; + private final BigQueryIOMetadata bqIOMetadata; + private final PipelineOptions options; + + @VisibleForTesting + WriteStreamServiceImpl(BigQueryWriteClient newWriteClient, PipelineOptions options) { + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); + this.newWriteClient = newWriteClient; + this.options = options; + this.storageWriteMaxInflightRequests = bqOptions.getStorageWriteMaxInflightRequests(); + this.storageWriteMaxInflightBytes = bqOptions.getStorageWriteMaxInflightBytes(); + this.bqIOMetadata = BigQueryIOMetadata.create(); + } + + public WriteStreamServiceImpl(BigQueryOptions bqOptions) { + this.newWriteClient = newBigQueryWriteClient(bqOptions); + this.options = bqOptions; + this.storageWriteMaxInflightRequests = bqOptions.getStorageWriteMaxInflightRequests(); + this.storageWriteMaxInflightBytes = bqOptions.getStorageWriteMaxInflightBytes(); + this.bqIOMetadata = BigQueryIOMetadata.create(); + } + @Override public WriteStream createWriteStream(String tableUrn, WriteStream.Type type) throws IOException { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java index 853a63fca8cc..cbe018ba739c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java @@ -25,7 +25,7 @@ import java.util.Collection; import java.util.Map; import java.util.Set; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.WriteStreamService; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; import org.apache.beam.sdk.metrics.Counter; @@ -64,27 +64,29 @@ class StorageApiFinalizeWritesDoFn extends DoFn, Void> { private Map> commitStreams; private final BigQueryServices bqServices; - private transient @Nullable DatasetService datasetService; + private transient @Nullable WriteStreamService writeStreamService; public StorageApiFinalizeWritesDoFn(BigQueryServices bqServices) { this.bqServices = bqServices; this.commitStreams = Maps.newHashMap(); - this.datasetService = null; + this.writeStreamService = null; } - private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { - if (datasetService == null) { - datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + private WriteStreamService getWriteStreamService(PipelineOptions pipelineOptions) + throws IOException { + if (writeStreamService == null) { + writeStreamService = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); } - return datasetService; + return writeStreamService; } @Teardown public void onTeardown() { try { - if (datasetService != null) { - datasetService.close(); - datasetService = null; + if (writeStreamService != null) { + writeStreamService.close(); + writeStreamService = null; } } catch (Exception e) { throw new RuntimeException(e); @@ -101,7 +103,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV throws Exception { String tableId = element.getKey(); String streamId = element.getValue(); - DatasetService datasetService = getDatasetService(pipelineOptions); + WriteStreamService writeStreamService = getWriteStreamService(pipelineOptions); RetryManager> retryManager = new RetryManager<>( @@ -113,7 +115,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV retryManager.addOperation( c -> { finalizeOperationsSent.inc(); - return datasetService.finalizeWriteStream(streamId); + return writeStreamService.finalizeWriteStream(streamId); }, contexts -> { RetryManager.Operation.Context firstContext = @@ -145,7 +147,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV @FinishBundle public void finishBundle(PipelineOptions pipelineOptions) throws Exception { - DatasetService datasetService = getDatasetService(pipelineOptions); + WriteStreamService writeStreamService = getWriteStreamService(pipelineOptions); for (Map.Entry> entry : commitStreams.entrySet()) { final String tableId = entry.getKey(); final Collection streamNames = entry.getValue(); @@ -161,7 +163,7 @@ public void finishBundle(PipelineOptions pipelineOptions) throws Exception { Iterable streamsToCommit = Iterables.filter(streamNames, s -> !alreadyCommittedStreams.contains(s)); batchCommitOperationsSent.inc(); - return datasetService.commitWriteStreams(tableId, streamsToCommit); + return writeStreamService.commitWriteStreams(tableId, streamsToCommit); }, contexts -> { RetryManager.Operation.Context firstContext = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java index 333a0c0b36bf..dec86c3360b0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java @@ -26,7 +26,7 @@ import java.time.Instant; import java.util.Objects; import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.WriteStreamService; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiFlushAndFinalizeDoFn.Operation; @@ -50,7 +50,7 @@ public class StorageApiFlushAndFinalizeDoFn extends DoFn, private static final Logger LOG = LoggerFactory.getLogger(StorageApiFlushAndFinalizeDoFn.class); private final BigQueryServices bqServices; - private transient @Nullable DatasetService datasetService = null; + private transient @Nullable WriteStreamService writeStreamService = null; private final Counter flushOperationsSent = Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsSent"); private final Counter flushOperationsSucceeded = @@ -112,19 +112,21 @@ public StorageApiFlushAndFinalizeDoFn(BigQueryServices bqServices) { this.bqServices = bqServices; } - private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { - if (datasetService == null) { - datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + private WriteStreamService getWriteStreamService(PipelineOptions pipelineOptions) + throws IOException { + if (writeStreamService == null) { + writeStreamService = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); } - return datasetService; + return writeStreamService; } @Teardown public void onTeardown() { try { - if (datasetService != null) { - datasetService.close(); - datasetService = null; + if (writeStreamService != null) { + writeStreamService.close(); + writeStreamService = null; } } catch (Exception e) { throw new RuntimeException(e); @@ -136,7 +138,7 @@ public void process(PipelineOptions pipelineOptions, @Element KV= 0) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 13c180a534f1..8afcbd36d732 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -56,6 +56,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.WriteStreamService; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; import org.apache.beam.sdk.metrics.Counter; @@ -271,7 +272,7 @@ class DestinationState { private long currentOffset = 0; private List pendingMessages; private List pendingTimestamps; - private transient @Nullable DatasetService maybeDatasetService; + private transient @Nullable WriteStreamService maybeWriteStreamService; private final Counter recordsAppended = Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); private final Counter appendFailures = @@ -298,7 +299,7 @@ public DestinationState( String tableUrn, String shortTableUrn, MessageConverter messageConverter, - DatasetService datasetService, + WriteStreamService writeStreamService, boolean useDefaultStream, int streamAppendClientCount, boolean usingMultiplexing, @@ -310,7 +311,7 @@ public DestinationState( this.shortTableUrn = shortTableUrn; this.pendingMessages = Lists.newArrayList(); this.pendingTimestamps = Lists.newArrayList(); - this.maybeDatasetService = datasetService; + this.maybeWriteStreamService = writeStreamService; this.useDefaultStream = useDefaultStream; this.initialTableSchema = messageConverter.getTableSchema(); this.initialDescriptor = messageConverter.getDescriptor(includeCdcColumns); @@ -356,7 +357,7 @@ String getOrCreateStreamName() throws Exception { () -> { if (!useDefaultStream) { this.streamName = - Preconditions.checkStateNotNull(maybeDatasetService) + Preconditions.checkStateNotNull(maybeWriteStreamService) .createWriteStream(tableUrn, Type.PENDING) .getName(); this.currentOffset = 0; @@ -397,7 +398,7 @@ AppendClientInfo generateClient(@Nullable TableSchema updatedSchema) throws Exce appendClientInfo .get() .withAppendClient( - Preconditions.checkStateNotNull(maybeDatasetService), + Preconditions.checkStateNotNull(maybeWriteStreamService), () -> streamName, usingMultiplexing, defaultMissingValueInterpretation)); @@ -438,7 +439,8 @@ SchemaAndDescriptor getCurrentTableSchema(String stream, @Nullable TableSchema u if (autoUpdateSchema) { @Nullable WriteStream writeStream = - Preconditions.checkStateNotNull(maybeDatasetService).getWriteStream(streamName); + Preconditions.checkStateNotNull(maybeWriteStreamService) + .getWriteStream(streamName); if (writeStream != null && writeStream.hasTableSchema()) { TableSchema updatedFromStream = writeStream.getTableSchema(); currentSchema.set(updatedFromStream); @@ -870,6 +872,7 @@ void postFlush() { private @Nullable Map destinations = Maps.newHashMap(); private final TwoLevelMessageConverterCache messageConverters; private transient @Nullable DatasetService maybeDatasetService; + private transient @Nullable WriteStreamService maybeWriteStreamService; private int numPendingRecords = 0; private int numPendingRecordBytes = 0; private final int flushThresholdBytes; @@ -977,6 +980,14 @@ private DatasetService initializeDatasetService(PipelineOptions pipelineOptions) return maybeDatasetService; } + private WriteStreamService initializeWriteStreamService(PipelineOptions pipelineOptions) { + if (maybeWriteStreamService == null) { + maybeWriteStreamService = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); + } + return maybeWriteStreamService; + } + @StartBundle public void startBundle() throws IOException { destinations = Maps.newHashMap(); @@ -989,6 +1000,7 @@ DestinationState createDestinationState( DestinationT destination, boolean useCdc, DatasetService datasetService, + WriteStreamService writeStreamService, BigQueryOptions bigQueryOptions) { TableDestination tableDestination1 = dynamicDestinations.getTable(destination); checkArgument( @@ -1019,7 +1031,7 @@ DestinationState createDestinationState( tableDestination1.getTableUrn(bigQueryOptions), tableDestination1.getShortTableUrn(), messageConverter, - datasetService, + writeStreamService, useDefaultStream, streamAppendClientCount, bigQueryOptions.getUseStorageApiConnectionPool(), @@ -1040,6 +1052,8 @@ public void process( MultiOutputReceiver o) throws Exception { DatasetService initializedDatasetService = initializeDatasetService(pipelineOptions); + WriteStreamService initializedWriteStreamService = + initializeWriteStreamService(pipelineOptions); dynamicDestinations.setSideInputAccessorFromProcessContext(c); DestinationState state = Preconditions.checkStateNotNull(destinations) @@ -1051,6 +1065,7 @@ public void process( destination, usesCdc, initializedDatasetService, + initializedWriteStreamService, pipelineOptions.as(BigQueryOptions.class))); OutputReceiver failedRowsReceiver = o.get(failedRowsTag); @@ -1116,6 +1131,10 @@ public void outputWithTimestamp(TableRow output, org.joda.time.Instant timestamp public void teardown() { destinations = null; try { + if (maybeWriteStreamService != null) { + maybeWriteStreamService.close(); + maybeWriteStreamService = null; + } if (maybeDatasetService != null) { maybeDatasetService.close(); maybeDatasetService = null; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index d3042984638f..8cf8ad0ee025 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -57,6 +57,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.WriteStreamService; import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiFlushAndFinalizeDoFn.Operation; import org.apache.beam.sdk.metrics.Counter; @@ -319,6 +320,8 @@ class WriteRecordsDoFn private transient @Nullable DatasetService datasetServiceInternal = null; + private transient @Nullable WriteStreamService writeStreamServiceInternal = null; + // Stores the current stream for this key. @StateId("streamName") private final StateSpec> streamNameSpec = StateSpecs.value(); @@ -358,7 +361,7 @@ String getOrCreateStream( ValueState streamName, ValueState streamOffset, Timer streamIdleTimer, - DatasetService datasetService, + WriteStreamService writeStreamService, Callable tryCreateTable) { try { final @Nullable String streamValue = streamName.read(); @@ -367,7 +370,7 @@ String getOrCreateStream( // In a buffered stream, data is only visible up to the offset to which it was flushed. CreateTableHelpers.createTableWrapper( () -> { - stream.set(datasetService.createWriteStream(tableId, Type.BUFFERED).getName()); + stream.set(writeStreamService.createWriteStream(tableId, Type.BUFFERED).getName()); return null; }, tryCreateTable); @@ -395,9 +398,22 @@ private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws return datasetServiceInternal; } + private WriteStreamService getWriteStreamService(PipelineOptions pipelineOptions) + throws IOException { + if (writeStreamServiceInternal == null) { + writeStreamServiceInternal = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); + } + return writeStreamServiceInternal; + } + @Teardown public void onTeardown() { try { + if (writeStreamServiceInternal != null) { + writeStreamServiceInternal.close(); + writeStreamServiceInternal = null; + } if (datasetServiceInternal != null) { datasetServiceInternal.close(); datasetServiceInternal = null; @@ -442,6 +458,7 @@ public void process( final String tableId = tableDestination.getTableUrn(bigQueryOptions); final String shortTableId = tableDestination.getShortTableUrn(); final DatasetService datasetService = getDatasetService(pipelineOptions); + final WriteStreamService writeStreamService = getWriteStreamService(pipelineOptions); Coder destinationCoder = dynamicDestinations.getDestinationCoder(); Callable tryCreateTable = @@ -462,7 +479,7 @@ public void process( Supplier getOrCreateStream = () -> getOrCreateStream( - tableId, streamName, streamOffset, idleTimer, datasetService, tryCreateTable); + tableId, streamName, streamOffset, idleTimer, writeStreamService, tryCreateTable); Callable getAppendClientInfo = () -> { @Nullable TableSchema tableSchema; @@ -500,7 +517,7 @@ public void process( client.close(); })) .withAppendClient( - datasetService, + writeStreamService, getOrCreateStream, false, defaultMissingValueInterpretation); @@ -569,7 +586,7 @@ public void process( appendClientInfo .get() .withAppendClient( - datasetService, + writeStreamService, getOrCreateStream, false, defaultMissingValueInterpretation)); @@ -618,7 +635,7 @@ public void process( appendClientInfo .get() .withAppendClient( - datasetService, + writeStreamService, getOrCreateStream, false, defaultMissingValueInterpretation)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java index 5e6e3ac7ed07..c258ce4ab7fb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java @@ -43,7 +43,7 @@ }) public class FakeBigQueryServices implements BigQueryServices { private JobService jobService; - private DatasetService datasetService; + private FakeDatasetService datasetService; private StorageClient storageClient; public FakeBigQueryServices withJobService(JobService jobService) { @@ -71,6 +71,11 @@ public DatasetService getDatasetService(BigQueryOptions bqOptions) { return datasetService; } + @Override + public WriteStreamService getWriteStreamService(BigQueryOptions bqOptions) { + return datasetService; + } + @Override public StorageClient getStorageClient(BigQueryOptions bqOptions) { return storageClient; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index f26c38d1e3c8..6a50127acd8f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -65,6 +65,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.WriteStreamService; import org.apache.beam.sdk.io.gcp.bigquery.ErrorContainer; import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy.Context; @@ -84,7 +85,7 @@ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -public class FakeDatasetService implements DatasetService, Serializable { +public class FakeDatasetService implements DatasetService, WriteStreamService, Serializable { // Table information must be static, as each ParDo will get a separate instance of // FakeDatasetServices, and they must all modify the same storage. static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table< diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index 474aea020f69..a8e1ad52237c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -473,8 +473,7 @@ public void testGetTableSucceeds() throws Exception { }); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); Table table = datasetService.getTable( @@ -509,7 +508,7 @@ public void testGetTableNullProjectSucceeds() throws Exception { options.setBigQueryProject("projectId"); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, null, options); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, options); Table table = datasetService.getTable( @@ -528,8 +527,7 @@ public void testGetTableNotFound() throws IOException, InterruptedException { }); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); TableReference tableRef = new TableReference() @@ -562,8 +560,7 @@ public void testGetTableThrows() throws Exception { thrown.expectMessage(String.format("Unable to get table: %s", tableRef.getTableId())); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); datasetService.getTable( tableRef, Collections.emptyList(), null, BackOff.STOP_BACKOFF, Sleeper.DEFAULT); } @@ -593,8 +590,7 @@ public void testIsTableEmptySucceeds() throws Exception { }); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); assertFalse(datasetService.isTableEmpty(tableRef, BackOff.ZERO_BACKOFF, Sleeper.DEFAULT)); @@ -610,8 +606,7 @@ public void testIsTableEmptyNoRetryForNotFound() throws IOException, Interrupted }); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); TableReference tableRef = new TableReference() @@ -644,8 +639,7 @@ public void testIsTableEmptyThrows() throws Exception { .setTableId("tableId"); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); thrown.expect(IOException.class); thrown.expectMessage(String.format("Unable to list table data: %s", tableRef.getTableId())); @@ -715,7 +709,7 @@ public void testInsertRateLimitRetry() throws Exception { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -760,7 +754,7 @@ public void testInsertQuotaExceededRetry() throws Exception { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -817,7 +811,7 @@ public void testInsertStoppedRetry() throws Exception { thrown.expectMessage("quotaExceeded"); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -876,7 +870,7 @@ public void testInsertRetrySelectRows() throws Exception { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -933,7 +927,7 @@ public void testInsertWithinRowCountLimits() throws Exception { PipelineOptionsFactory.fromArgs("--maxStreamingRowsToBatch=1").create(); options.as(GcsOptions.class).setExecutorService(Executors.newSingleThreadExecutor()); - DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, null, options); + DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, options); dataService.insertAll( ref, rows, @@ -983,7 +977,7 @@ public void testInsertWithinRequestByteSizeLimitsErrorsOut() throws Exception { DatasetServiceImpl dataService = new DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.fromArgs("--maxStreamingBatchSize=15").create()); + bigquery, PipelineOptionsFactory.fromArgs("--maxStreamingBatchSize=15").create()); List> failedInserts = Lists.newArrayList(); List> successfulRows = Lists.newArrayList(); RuntimeException e = @@ -1031,7 +1025,7 @@ public void testInsertRetryTransientsAboveRequestByteSizeLimits() throws Excepti DatasetServiceImpl dataService = new DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.fromArgs("--maxStreamingBatchSize=15").create()); + bigquery, PipelineOptionsFactory.fromArgs("--maxStreamingBatchSize=15").create()); List> failedInserts = Lists.newArrayList(); List> successfulRows = Lists.newArrayList(); dataService.insertAll( @@ -1086,7 +1080,7 @@ public void testInsertWithinRequestByteSizeLimits() throws Exception { PipelineOptionsFactory.fromArgs("--maxStreamingBatchSize=15").create(); options.as(GcsOptions.class).setExecutorService(Executors.newSingleThreadExecutor()); - DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, null, options); + DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, options); dataService.insertAll( ref, rows, @@ -1150,7 +1144,7 @@ public void testInsertFailsGracefully() throws Exception { row0FailureResponseFunction); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); // Expect it to fail. try { @@ -1209,7 +1203,7 @@ public void testFailInsertOtherRetry() throws Exception { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); thrown.expect(RuntimeException.class); thrown.expectMessage("actually forbidden"); try { @@ -1258,7 +1252,7 @@ public void testInsertTimeoutLog() throws Exception { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); RuntimeException e = assertThrows( RuntimeException.class, @@ -1340,7 +1334,7 @@ public void testInsertRetryPolicy() throws InterruptedException, IOException { }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( @@ -1387,7 +1381,7 @@ public void testSkipInvalidRowsIgnoreUnknownIgnoreInsertIdsValuesStreaming() setupMockResponses(allRowsSucceededResponseFunction, allRowsSucceededResponseFunction); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); // First, test with all flags disabled dataService.insertAll( @@ -1501,8 +1495,7 @@ public void testCreateTableSucceeds() throws IOException { }); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(BackOff.ZERO_BACKOFF, 0), Sleeper.DEFAULT); @@ -1535,8 +1528,7 @@ public void testCreateTableDoesNotRetry() throws IOException { thrown.expectMessage("actually forbidden"); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); try { services.tryCreateTable( testTable, new RetryBoundedBackOff(BackOff.ZERO_BACKOFF, 3), Sleeper.DEFAULT); @@ -1572,8 +1564,7 @@ public void testCreateTableSucceedsAlreadyExists() throws IOException { }); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(BackOff.ZERO_BACKOFF, 0), Sleeper.DEFAULT); @@ -1604,8 +1595,7 @@ public void testCreateTableRetry() throws IOException { }); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl( - bigquery, null, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(BackOff.ZERO_BACKOFF, 3), Sleeper.DEFAULT); @@ -1651,7 +1641,7 @@ public void testSimpleErrorRetrieval() throws InterruptedException, IOException }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( @@ -1711,7 +1701,7 @@ public void testExtendedErrorRetrieval() throws InterruptedException, IOExceptio }); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java index cef93913401e..c71946a54676 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java @@ -170,7 +170,7 @@ public void testTableGet() throws InterruptedException, IOException { onTableList(dataList); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(mockClient, null, options); + new BigQueryServicesImpl.DatasetServiceImpl(mockClient, options); services.getTable( new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table")); @@ -190,7 +190,7 @@ public void testInsertAll() throws Exception { onInsertAll(errorsIndices); TableReference ref = BigQueryHelpers.parseTableSpec("project:dataset.table"); - DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, null, options, 5); + DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, options, 5); List> rows = new ArrayList<>(); List ids = new ArrayList<>(); From 54c18943bfba24b0f65c23815daabf745f33b9cc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 6 Dec 2023 14:47:43 -0800 Subject: [PATCH 047/224] Bump google.golang.org/api from 0.152.0 to 0.153.0 in /sdks (#29628) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.152.0 to 0.153.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.152.0...v0.153.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6035fa8884d0..fb53e17b4a36 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,7 +59,7 @@ require ( golang.org/x/sync v0.5.0 golang.org/x/sys v0.15.0 golang.org/x/text v0.14.0 - google.golang.org/api v0.152.0 + google.golang.org/api v0.153.0 google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 diff --git a/sdks/go.sum b/sdks/go.sum index 2c875905d607..2cf0742086d0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -687,8 +687,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.152.0 h1:t0r1vPnfMc260S2Ci+en7kfCZaLOPs5KI0sVV/6jZrY= -google.golang.org/api v0.152.0/go.mod h1:3qNJX5eOmhiWYc67jRA/3GsDw97UFb5ivv7Y2PrriAY= +google.golang.org/api v0.153.0 h1:N1AwGhielyKFaUqH07/ZSIQR3uNPcV7NVw0vj+j4iR4= +google.golang.org/api v0.153.0/go.mod h1:3qNJX5eOmhiWYc67jRA/3GsDw97UFb5ivv7Y2PrriAY= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= From dab5e86b912e2b975ffcb297803160e6fbee2d5f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 6 Dec 2023 15:57:08 -0800 Subject: [PATCH 048/224] Fix for signature exceptions in ptransform_fn. For some cases (e.g. builting functions) signature retrieval fails. In this case, don't attempt to patch the (non-available) signature. --- sdks/python/apache_beam/transforms/ptransform.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index 0296f7ea21ea..04373fe92bf6 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -1080,8 +1080,14 @@ def callable_ptransform_factory(*args, **kwargs): # The signature of this PTransform constructor is that of fn minus the first # argument (which is where the pvalue is passed during expand). - callable_ptransform_factory.__signature__ = inspect.signature( # type: ignore - functools.partial(fn, None)) + try: + inspect.signature(fn) + except Exception: + # If we can't get a signature, don't attempt to patch it up. + pass + else: + callable_ptransform_factory.__signature__ = inspect.signature( # type: ignore + functools.partial(fn, None)) return callable_ptransform_factory From 70926f5da784e6f0ca1eebc64352e942a6e556c7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 7 Dec 2023 09:40:04 -0500 Subject: [PATCH 049/224] Bump go.mongodb.org/mongo-driver from 1.13.0 to 1.13.1 in /sdks (#29657) Bumps [go.mongodb.org/mongo-driver](https://github.com/mongodb/mongo-go-driver) from 1.13.0 to 1.13.1. - [Release notes](https://github.com/mongodb/mongo-go-driver/releases) - [Commits](https://github.com/mongodb/mongo-go-driver/compare/v1.13.0...v1.13.1) --- updated-dependencies: - dependency-name: go.mongodb.org/mongo-driver dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index fb53e17b4a36..2060c3c0a500 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/tetratelabs/wazero v1.5.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c - go.mongodb.org/mongo-driver v1.13.0 + go.mongodb.org/mongo-driver v1.13.1 golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.14.0 golang.org/x/sync v0.5.0 diff --git a/sdks/go.sum b/sdks/go.sum index 2cf0742086d0..3c99d6587274 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -477,8 +477,8 @@ github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.mongodb.org/mongo-driver v1.13.0 h1:67DgFFjYOCMWdtTEmKFpV3ffWlFnh+CYZ8ZS/tXWUfY= -go.mongodb.org/mongo-driver v1.13.0/go.mod h1:/rGBTebI3XYboVmgz+Wv3Bcbl3aD0QF9zl6kDDw18rQ= +go.mongodb.org/mongo-driver v1.13.1 h1:YIc7HTYsKndGK4RFzJ3covLz1byri52x0IoMB0Pt/vk= +go.mongodb.org/mongo-driver v1.13.1/go.mod h1:wcDf1JBCXy2mOW0bWHwO/IOYqdca1MPCwDtFu/Z9+eo= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= From ee5e782c5252efa8259bd4315b9edefef109e02a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 7 Dec 2023 09:54:26 -0500 Subject: [PATCH 050/224] Bump actions/setup-go from 4 to 5 (#29653) Bumps [actions/setup-go](https://github.com/actions/setup-go) from 4 to 5. - [Release notes](https://github.com/actions/setup-go/releases) - [Commits](https://github.com/actions/setup-go/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/setup-go dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/go_tests.yml | 2 +- .github/workflows/local_env_tests.yml | 4 ++-- .github/workflows/python_tests.yml | 2 +- .github/workflows/tour_of_beam_backend.yml | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 02947eff5ca0..2d180afc79e3 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -43,7 +43,7 @@ jobs: uses: actions/checkout@v4 with: fetch-depth: 2 - - uses: actions/setup-go@v4 + - uses: actions/setup-go@v5 with: go-version: '1.21' cache-dependency-path: | diff --git a/.github/workflows/local_env_tests.yml b/.github/workflows/local_env_tests.yml index 7dbff4feeb45..49d37298f1c0 100644 --- a/.github/workflows/local_env_tests.yml +++ b/.github/workflows/local_env_tests.yml @@ -46,7 +46,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - - uses: actions/setup-go@v4 + - uses: actions/setup-go@v5 with: go-version: '1.21' - uses: actions/setup-python@v4 @@ -64,7 +64,7 @@ jobs: runs-on: macos-latest steps: - uses: actions/checkout@v4 - - uses: actions/setup-go@v4 + - uses: actions/setup-go@v5 with: go-version: '1.21' - uses: actions/setup-python@v4 diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 0309329e84e1..483d4776aec3 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -168,7 +168,7 @@ jobs: with: python-version: ${{ matrix.python }} - name: Install go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: '1.21' - name: Download source from artifacts diff --git a/.github/workflows/tour_of_beam_backend.yml b/.github/workflows/tour_of_beam_backend.yml index 3585bd0e670f..e3a016a4b5a7 100644 --- a/.github/workflows/tour_of_beam_backend.yml +++ b/.github/workflows/tour_of_beam_backend.yml @@ -42,7 +42,7 @@ jobs: working-directory: ./learning/tour-of-beam/backend steps: - uses: actions/checkout@v4 - - uses: actions/setup-go@v4 + - uses: actions/setup-go@v5 with: # pin to the biggest Go version supported by Cloud Functions runtime go-version: '1.16' From 5d11c20cd4e5a0df132f4cd2cff0d880a0d75323 Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Thu, 7 Dec 2023 10:11:16 -0500 Subject: [PATCH 051/224] Add Error Handling to Kafka IO (#29546) * 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 * Add error handling base implementation & test DLQ enabled class * Add test cases * apply spotless * Fix Checkstyles * Fix Checkstyles * make DLH serializable * rename dead letter to bad record * make DLH serializable * Change bad record router name, and use multioutputreceiver instead of process context * Refactor BadRecord to be nested * clean up checkstyle * Update error handler test * Add metrics for counting error records, and for measuring feature usage * apply spotless * fix checkstyle * make metric reporting static * spotless * Rework annotations to be an explicit label on a PTransform, instead of using java annotations * fix checkstyle * Address comments * Address comments * Fix test cases, spotless * remove flatting without error collections * fix nullness * spotless + encoding issues * spotless * throw error when error handler isn't used * add concrete bad record error handler class * spotless, fix test category * fix checkstyle * clean up comments * fix test case * initial wiring of error handler into KafkaIO Read * remove "failing transform" field on bad record, add note to CHANGES.md * fix failing test cases * fix failing test cases * apply spotless * Add tests * Add tests * fix test case * add documentation * wire error handler into kafka write * fix failing test case * Add tests for writing to kafka with exception handling * fix sdf testing * fix sdf testing * spotless * deflake tests * add error handling to kafka streaming example update error handler to be serializable to support using it as a member of an auto-value based PTransform * apply final comments * apply final comments * apply final comments * add line to CHANGES.md * fix spotless * fix checkstyle * make sink transform static for serialization * spotless * fix typo * fix typo * fix spotbugs --- CHANGES.md | 1 + .../gradle/kafka/KafkaTestUtilities.groovy | 3 +- .../apache/beam/examples/KafkaStreaming.java | 67 +++++-- .../errorhandling/ErrorHandler.java | 32 +++- sdks/java/io/kafka/kafka-01103/build.gradle | 1 + sdks/java/io/kafka/kafka-100/build.gradle | 3 +- sdks/java/io/kafka/kafka-111/build.gradle | 1 + sdks/java/io/kafka/kafka-201/build.gradle | 1 + sdks/java/io/kafka/kafka-211/build.gradle | 1 + sdks/java/io/kafka/kafka-222/build.gradle | 1 + sdks/java/io/kafka/kafka-231/build.gradle | 1 + sdks/java/io/kafka/kafka-241/build.gradle | 1 + sdks/java/io/kafka/kafka-251/build.gradle | 1 + .../io/kafka/kafka-integration-test.gradle | 2 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 127 +++++++++++-- ...afkaIOReadImplementationCompatibility.java | 1 + .../apache/beam/sdk/io/kafka/KafkaWriter.java | 46 +++-- .../beam/sdk/io/kafka/ReadFromKafkaDoFn.java | 110 ++++++++---- .../sdk/io/kafka/KafkaIOExternalTest.java | 8 +- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 121 ++++++++++--- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 114 ++++++++++-- .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 168 +++++++++++++++--- 22 files changed, 659 insertions(+), 152 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 4b977bf3790d..7686b7a92d96 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -66,6 +66,7 @@ * TextIO now supports skipping multiple header lines (Java) ([#17990](https://github.com/apache/beam/issues/17990)). * Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) * Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). +* Added support for handling bad records to KafkaIO (Java) ([#29546](https://github.com/apache/beam/pull/29546)) ## New Features / Improvements diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy index cd2875fdb512..bb08e79edd3c 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/kafka/KafkaTestUtilities.groovy @@ -40,7 +40,7 @@ class KafkaTestUtilities { '"keySizeBytes": "10",' + '"valueSizeBytes": "90"' + '}', - "--readTimeout=120", + "--readTimeout=60", "--kafkaTopic=beam", "--withTestcontainers=true", "--kafkaContainerVersion=5.5.2", @@ -56,6 +56,7 @@ class KafkaTestUtilities { excludeTestsMatching "*SDFResumesCorrectly" //Kafka SDF does not work for kafka versions <2.0.1 excludeTestsMatching "*StopReadingFunction" //Kafka SDF does not work for kafka versions <2.0.1 excludeTestsMatching "*WatermarkUpdateWithSparseMessages" //Kafka SDF does not work for kafka versions <2.0.1 + excludeTestsMatching "*KafkaIOSDFReadWithErrorHandler" } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java index 34a4b646555d..602c34d4219d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java +++ b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java @@ -49,8 +49,11 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -60,6 +63,8 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.StringDeserializer; @@ -97,7 +102,7 @@ public interface KafkaStreamingOptions extends PipelineOptions { * to use your own Kafka server. */ @Description("Kafka server host") - @Default.String("kafka_server:9092") + @Default.String("localhost:9092") String getKafkaHost(); void setKafkaHost(String value); @@ -208,15 +213,22 @@ public void run() { // Start reading form Kafka with the latest offset consumerConfig.put("auto.offset.reset", "latest"); - PCollection> pCollection = - pipeline.apply( - KafkaIO.read() - .withBootstrapServers(options.getKafkaHost()) - .withTopic(TOPIC_NAME) - .withKeyDeserializer(StringDeserializer.class) - .withValueDeserializer(IntegerDeserializer.class) - .withConsumerConfigUpdates(consumerConfig) - .withoutMetadata()); + // Register an error handler for any deserialization errors. + // Errors are simulated with an intentionally failing deserializer + PCollection> pCollection; + try (BadRecordErrorHandler> errorHandler = + pipeline.registerBadRecordErrorHandler(new LogErrors())) { + pCollection = + pipeline.apply( + KafkaIO.read() + .withBootstrapServers(options.getKafkaHost()) + .withTopic(TOPIC_NAME) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializer(IntermittentlyFailingIntegerDeserializer.class) + .withConsumerConfigUpdates(consumerConfig) + .withBadRecordErrorHandler(errorHandler) + .withoutMetadata()); + } pCollection // Apply a window and a trigger ourput repeatedly. @@ -317,4 +329,39 @@ public void processElement(ProcessContext c, IntervalWindow w) throws Exception c.output(c.element()); } } + + // Simple PTransform to log Error information + static class LogErrors extends PTransform, PCollection> { + + @Override + public PCollection expand(PCollection input) { + return input.apply("Log Errors", ParDo.of(new LogErrorFn())); + } + + static class LogErrorFn extends DoFn { + @ProcessElement + public void processElement(@Element BadRecord record, OutputReceiver receiver) { + System.out.println(record); + receiver.output(record); + } + } + } + + // Intentionally failing deserializer to simulate bad data from Kafka + public static class IntermittentlyFailingIntegerDeserializer implements Deserializer { + + public static final IntegerDeserializer INTEGER_DESERIALIZER = new IntegerDeserializer(); + public int deserializeCount = 0; + + public IntermittentlyFailingIntegerDeserializer() {} + + @Override + public Integer deserialize(String topic, byte[] data) { + deserializeCount++; + if (deserializeCount % 10 == 0) { + throw new SerializationException("Expected Serialization Exception"); + } + return INTEGER_DESERIALIZER.deserialize(topic, data); + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java index 9e0298d885eb..e02965b72022 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.errorhandling; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -49,22 +52,24 @@ *

    Simple usage with one DLQ *

    {@code
      * PCollection records = ...;
    - * try (ErrorHandler errorHandler = pipeline.registerErrorHandler(SomeSink.write())) {
    - *  PCollection results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler));
    + * try (BadRecordErrorHandler errorHandler = pipeline.registerBadRecordErrorHandler(SomeSink.write())) {
    + *  PCollection results = records.apply(SomeIO.write().withErrorHandler(errorHandler));
      * }
      * results.apply(SomeOtherTransform);
      * }
    * Usage with multiple DLQ stages *
    {@code
      * PCollection records = ...;
    - * try (ErrorHandler errorHandler = pipeline.registerErrorHandler(SomeSink.write())) {
    - *  PCollection results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler))
    - *                        .apply(OtherTransform.builder().withDeadLetterQueue(errorHandler));
    + * try (BadRecordErrorHandler errorHandler = pipeline.registerBadRecordErrorHandler(SomeSink.write())) {
    + *  PCollection results = records.apply(SomeIO.write().withErrorHandler(errorHandler))
    + *                        .apply(OtherTransform.builder().withErrorHandler(errorHandler));
      * }
      * results.apply(SomeOtherTransform);
      * }
    + * This is marked as serializable despite never being needed on the runner, to enable it to be a + * parameter of an Autovalue configured PTransform. */ -public interface ErrorHandler extends AutoCloseable { +public interface ErrorHandler extends AutoCloseable, Serializable { void addErrorCollection(PCollection errorCollection); @@ -79,13 +84,16 @@ class PTransformErrorHandler private static final Logger LOG = LoggerFactory.getLogger(PTransformErrorHandler.class); private final PTransform, OutputT> sinkTransform; - private final Pipeline pipeline; + // transient as Pipelines are not serializable + private final transient Pipeline pipeline; private final Coder coder; - private final List> errorCollections = new ArrayList<>(); + // transient as PCollections are not serializable + private transient List> errorCollections = new ArrayList<>(); - private @Nullable OutputT sinkOutput = null; + // transient as PCollections are not serializable + private transient @Nullable OutputT sinkOutput = null; private boolean closed = false; @@ -103,6 +111,12 @@ public PTransformErrorHandler( this.coder = coder; } + private void readObject(ObjectInputStream aInputStream) + throws ClassNotFoundException, IOException { + aInputStream.defaultReadObject(); + errorCollections = new ArrayList<>(); + } + @Override public void addErrorCollection(PCollection errorCollection) { errorCollections.add(errorCollection); diff --git a/sdks/java/io/kafka/kafka-01103/build.gradle b/sdks/java/io/kafka/kafka-01103/build.gradle index a0fa372397a2..3a74bf04ef22 100644 --- a/sdks/java/io/kafka/kafka-01103/build.gradle +++ b/sdks/java/io/kafka/kafka-01103/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="0.11.0.3" undelimited="01103" + sdfCompatible=false } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-100/build.gradle b/sdks/java/io/kafka/kafka-100/build.gradle index 15ce8c0deeff..bd5fa67b1cfc 100644 --- a/sdks/java/io/kafka/kafka-100/build.gradle +++ b/sdks/java/io/kafka/kafka-100/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="1.0.0" undelimited="100" + sdfCompatible=false } -apply from: "../kafka-integration-test.gradle" \ No newline at end of file +apply from: "../kafka-integration-test.gradle" diff --git a/sdks/java/io/kafka/kafka-111/build.gradle b/sdks/java/io/kafka/kafka-111/build.gradle index fee4c382ed41..c2b0c8f82827 100644 --- a/sdks/java/io/kafka/kafka-111/build.gradle +++ b/sdks/java/io/kafka/kafka-111/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="1.1.1" undelimited="111" + sdfCompatible=false } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-201/build.gradle b/sdks/java/io/kafka/kafka-201/build.gradle index d395d0aa6269..a26ca4ac19cf 100644 --- a/sdks/java/io/kafka/kafka-201/build.gradle +++ b/sdks/java/io/kafka/kafka-201/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.0.1" undelimited="201" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-211/build.gradle b/sdks/java/io/kafka/kafka-211/build.gradle index 4de07193b5a2..433d6c93f361 100644 --- a/sdks/java/io/kafka/kafka-211/build.gradle +++ b/sdks/java/io/kafka/kafka-211/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.1.1" undelimited="211" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-222/build.gradle b/sdks/java/io/kafka/kafka-222/build.gradle index 57de58e81895..0f037e742968 100644 --- a/sdks/java/io/kafka/kafka-222/build.gradle +++ b/sdks/java/io/kafka/kafka-222/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.2.2" undelimited="222" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-231/build.gradle b/sdks/java/io/kafka/kafka-231/build.gradle index 3682791c5b67..712158dcd3ae 100644 --- a/sdks/java/io/kafka/kafka-231/build.gradle +++ b/sdks/java/io/kafka/kafka-231/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.3.1" undelimited="231" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-241/build.gradle b/sdks/java/io/kafka/kafka-241/build.gradle index 358c95aeb2fe..c0ac7df674b5 100644 --- a/sdks/java/io/kafka/kafka-241/build.gradle +++ b/sdks/java/io/kafka/kafka-241/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.4.1" undelimited="241" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-251/build.gradle b/sdks/java/io/kafka/kafka-251/build.gradle index f291ecccc36b..4de9f97a738a 100644 --- a/sdks/java/io/kafka/kafka-251/build.gradle +++ b/sdks/java/io/kafka/kafka-251/build.gradle @@ -18,6 +18,7 @@ project.ext { delimited="2.5.1" undelimited="251" + sdfCompatible=true } apply from: "../kafka-integration-test.gradle" \ No newline at end of file diff --git a/sdks/java/io/kafka/kafka-integration-test.gradle b/sdks/java/io/kafka/kafka-integration-test.gradle index 778f8a3c456c..1aeb0c97f93b 100644 --- a/sdks/java/io/kafka/kafka-integration-test.gradle +++ b/sdks/java/io/kafka/kafka-integration-test.gradle @@ -39,4 +39,4 @@ dependencies { configurations.create("kafkaVersion$undelimited") -tasks.register("kafkaVersion${undelimited}BatchIT",KafkaTestUtilities.KafkaBatchIT, project.ext.delimited, project.ext.undelimited, false, configurations, project) \ No newline at end of file +tasks.register("kafkaVersion${undelimited}BatchIT",KafkaTestUtilities.KafkaBatchIT, project.ext.delimited, project.ext.undelimited, project.ext.sdfCompatible, configurations, project) \ No newline at end of file diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 7e4fc55c6ce9..8fd0c34cfa90 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -81,6 +81,11 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.Manual; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; @@ -89,9 +94,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; @@ -167,6 +174,10 @@ * // signal. * .withCheckStopReadingFn(new SerializedFunction() {}) * + * //If you would like to send messages that fail to be parsed from Kafka to an alternate sink, + * //use the error handler pattern as defined in {@link ErrorHandler} + * .withBadRecordErrorHandler(errorHandler) + * * // finally, if you don't need Kafka metadata, you can drop it.g * .withoutMetadata() // PCollection> * ) @@ -469,6 +480,11 @@ * // or you can also set a custom timestamp with a function. * .withPublishTimestampFunction((elem, elemTs) -> ...) * + * // Optionally, records that fail to serialize can be sent to an error handler + * // See {@link ErrorHandler} for details of for details of configuring a bad record error + * // handler + * .withBadRecordErrorHandler(errorHandler) + * * // Optionally enable exactly-once sink (on supported runners). See JavaDoc for withEOS(). * .withEOS(20, "eos-sink-group-id"); * ); @@ -592,13 +608,7 @@ public static ReadSourceDescriptors readSourceDescriptors() { */ public static Write write() { return new AutoValue_KafkaIO_Write.Builder() - .setWriteRecordsTransform( - new AutoValue_KafkaIO_WriteRecords.Builder() - .setProducerConfig(WriteRecords.DEFAULT_PRODUCER_PROPERTIES) - .setEOS(false) - .setNumShards(0) - .setConsumerFactoryFn(KafkaIOUtils.KAFKA_CONSUMER_FACTORY_FN) - .build()) + .setWriteRecordsTransform(writeRecords()) .build(); } @@ -613,6 +623,8 @@ public static WriteRecords writeRecords() { .setEOS(false) .setNumShards(0) .setConsumerFactoryFn(KafkaIOUtils.KAFKA_CONSUMER_FACTORY_FN) + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) + .setBadRecordErrorHandler(new DefaultErrorHandler<>()) .build(); } @@ -691,6 +703,9 @@ public abstract static class Read @Pure public abstract @Nullable CheckStopReadingFn getCheckStopReadingFn(); + @Pure + public abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -739,6 +754,9 @@ abstract Builder setValueDeserializerProvider( abstract Builder setCheckStopReadingFn(@Nullable CheckStopReadingFn checkStopReadingFn); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + Builder setCheckStopReadingFn( @Nullable SerializableFunction checkStopReadingFn) { return setCheckStopReadingFn(CheckStopReadingFnWrapper.of(checkStopReadingFn)); @@ -1312,6 +1330,10 @@ public Read withCheckStopReadingFn( .build(); } + public Read withBadRecordErrorHandler(ErrorHandler badRecordErrorHandler) { + return toBuilder().setBadRecordErrorHandler(badRecordErrorHandler).build(); + } + /** Returns a {@link PTransform} for PCollection of {@link KV}, dropping Kafka metatdata. */ public PTransform>> withoutMetadata() { return new TypedWithoutMetadata<>(this); @@ -1529,6 +1551,11 @@ static class ReadFromKafkaViaUnbounded extends AbstractReadFromKafka @Override public PCollection> expand(PBegin input) { + if (kafkaRead.getBadRecordErrorHandler() != null) { + LOG.warn( + "The Legacy implementation of Kafka Read does not support writing malformed" + + "messages to an error handler. Use the SDF implementation instead."); + } // Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. Unbounded> unbounded = org.apache.beam.sdk.io.Read.from( @@ -1576,6 +1603,10 @@ public PCollection> expand(PBegin input) { if (kafkaRead.getStopReadTime() != null) { readTransform = readTransform.withBounded(); } + if (kafkaRead.getBadRecordErrorHandler() != null) { + readTransform = + readTransform.withBadRecordErrorHandler(kafkaRead.getBadRecordErrorHandler()); + } PCollection output; if (kafkaRead.isDynamicRead()) { Set topics = new HashSet<>(); @@ -1956,6 +1987,8 @@ public void populateDisplayData(DisplayData.Builder builder) { public abstract static class ReadSourceDescriptors extends PTransform, PCollection>> { + private final TupleTag>> records = new TupleTag<>(); + private static final Logger LOG = LoggerFactory.getLogger(ReadSourceDescriptors.class); @Pure @@ -1997,6 +2030,12 @@ public abstract static class ReadSourceDescriptors @Pure abstract @Nullable TimestampPolicyFactory getTimestampPolicyFactory(); + @Pure + abstract BadRecordRouter getBadRecordRouter(); + + @Pure + abstract ErrorHandler getBadRecordErrorHandler(); + abstract boolean isBounded(); abstract ReadSourceDescriptors.Builder toBuilder(); @@ -2041,6 +2080,12 @@ abstract ReadSourceDescriptors.Builder setCommitOffsetEnabled( abstract ReadSourceDescriptors.Builder setTimestampPolicyFactory( TimestampPolicyFactory policy); + abstract ReadSourceDescriptors.Builder setBadRecordRouter( + BadRecordRouter badRecordRouter); + + abstract ReadSourceDescriptors.Builder setBadRecordErrorHandler( + ErrorHandler badRecordErrorHandler); + abstract ReadSourceDescriptors.Builder setBounded(boolean bounded); abstract ReadSourceDescriptors build(); @@ -2052,6 +2097,8 @@ public static ReadSourceDescriptors read() { .setConsumerConfig(KafkaIOUtils.DEFAULT_CONSUMER_PROPERTIES) .setCommitOffsetEnabled(false) .setBounded(false) + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) + .setBadRecordErrorHandler(new ErrorHandler.DefaultErrorHandler<>()) .build() .withProcessingTime() .withMonotonicallyIncreasingWatermarkEstimator(); @@ -2305,6 +2352,14 @@ public ReadSourceDescriptors withConsumerConfigOverrides( return toBuilder().setConsumerConfig(consumerConfig).build(); } + public ReadSourceDescriptors withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return toBuilder() + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER) + .setBadRecordErrorHandler(errorHandler) + .build(); + } + ReadAllFromRow forExternalBuild() { return new ReadAllFromRow<>(this); } @@ -2395,9 +2450,18 @@ public PCollection> expand(PCollection Coder> recordCoder = KafkaRecordCoder.of(keyCoder, valueCoder); try { + PCollectionTuple pCollectionTuple = + input.apply( + ParDo.of(ReadFromKafkaDoFn.create(this, records)) + .withOutputTags(records, TupleTagList.of(BadRecordRouter.BAD_RECORD_TAG))); + getBadRecordErrorHandler() + .addErrorCollection( + pCollectionTuple + .get(BadRecordRouter.BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(input.getPipeline()))); PCollection>> outputWithDescriptor = - input - .apply(ParDo.of(ReadFromKafkaDoFn.create(this))) + pCollectionTuple + .get(records) .setCoder( KvCoder.of( input @@ -2538,6 +2602,12 @@ public abstract static class WriteRecords public abstract @Nullable SerializableFunction, ? extends Consumer> getConsumerFactoryFn(); + @Pure + public abstract BadRecordRouter getBadRecordRouter(); + + @Pure + public abstract ErrorHandler getBadRecordErrorHandler(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -2565,6 +2635,11 @@ abstract Builder setPublishTimestampFunction( abstract Builder setConsumerFactoryFn( SerializableFunction, ? extends Consumer> fn); + abstract Builder setBadRecordRouter(BadRecordRouter router); + + abstract Builder setBadRecordErrorHandler( + ErrorHandler badRecordErrorHandler); + abstract WriteRecords build(); } @@ -2711,6 +2786,14 @@ public WriteRecords withConsumerFactoryFn( return toBuilder().setConsumerFactoryFn(consumerFactoryFn).build(); } + public WriteRecords withBadRecordErrorHandler( + ErrorHandler badRecordErrorHandler) { + return toBuilder() + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER) + .setBadRecordErrorHandler(badRecordErrorHandler) + .build(); + } + @Override public PDone expand(PCollection> input) { checkArgument( @@ -2722,6 +2805,9 @@ public PDone expand(PCollection> input) { if (isEOS()) { checkArgument(getTopic() != null, "withTopic() is required when isEOS() is true"); + checkArgument( + getBadRecordErrorHandler() instanceof DefaultErrorHandler, + "BadRecordErrorHandling isn't supported with Kafka Exactly Once writing"); KafkaExactlyOnceSink.ensureEOSSupport(); // TODO: Verify that the group_id does not have existing state stored on Kafka unless @@ -2732,7 +2818,19 @@ public PDone expand(PCollection> input) { input.apply(new KafkaExactlyOnceSink<>(this)); } else { - input.apply(ParDo.of(new KafkaWriter<>(this))); + // Even though the errors are the only output from writing to Kafka, we maintain a + // PCollectionTuple + // with a void tag as the 'primary' output for easy forward compatibility + PCollectionTuple pCollectionTuple = + input.apply( + ParDo.of(new KafkaWriter<>(this)) + .withOutputTags( + new TupleTag(), TupleTagList.of(BadRecordRouter.BAD_RECORD_TAG))); + getBadRecordErrorHandler() + .addErrorCollection( + pCollectionTuple + .get(BadRecordRouter.BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(input.getPipeline()))); } return PDone.in(input.getPipeline()); } @@ -2995,6 +3093,15 @@ public Write withProducerConfigUpdates(Map configUpdates) getWriteRecordsTransform().withProducerConfigUpdates(configUpdates)); } + /** + * Configure a {@link BadRecordErrorHandler} for sending records to if they fail to serialize + * when being sent to Kafka. + */ + public Write withBadRecordErrorHandler(ErrorHandler badRecordErrorHandler) { + return withWriteRecordsTransform( + getWriteRecordsTransform().withBadRecordErrorHandler(badRecordErrorHandler)); + } + @Override public PDone expand(PCollection> input) { final String topic = Preconditions.checkStateNotNull(getTopic(), "withTopic() is required"); diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOReadImplementationCompatibility.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOReadImplementationCompatibility.java index b779de1d9cf1..a2cc9aaeb4d9 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOReadImplementationCompatibility.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOReadImplementationCompatibility.java @@ -111,6 +111,7 @@ Object getDefaultValue() { KEY_DESERIALIZER_PROVIDER, VALUE_DESERIALIZER_PROVIDER, CHECK_STOP_READING_FN(SDF), + BAD_RECORD_ERROR_HANDLER(SDF), ; @Nonnull private final ImmutableSet supportedImplementations; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java index c0c9772959f9..4f4663aa8cc8 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.SinkMetrics; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; import org.apache.beam.sdk.util.Preconditions; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -32,6 +33,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.errors.SerializationException; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +59,7 @@ public void setup() { // Suppression since errors are tracked in SendCallback(), and checked in finishBundle() @ProcessElement @SuppressWarnings("FutureReturnValueIgnored") - public void processElement(ProcessContext ctx) throws Exception { + public void processElement(ProcessContext ctx, MultiOutputReceiver receiver) throws Exception { Producer producer = Preconditions.checkStateNotNull(this.producer); checkForFailures(); @@ -75,19 +77,31 @@ public void processElement(ProcessContext ctx) throws Exception { topicName = spec.getTopic(); } - @SuppressWarnings({"nullness", "unused"}) // Kafka library not annotated - Future ignored = - producer.send( - new ProducerRecord<>( - topicName, - record.partition(), - timestampMillis, - record.key(), - record.value(), - record.headers()), - callback); - - elementsWritten.inc(); + try { + @SuppressWarnings({"nullness", "unused"}) // Kafka library not annotated + Future ignored = + producer.send( + new ProducerRecord<>( + topicName, + record.partition(), + timestampMillis, + record.key(), + record.value(), + record.headers()), + callback); + + elementsWritten.inc(); + } catch (SerializationException e) { + // This exception should only occur during the key and value deserialization when + // creating the Kafka Record. We can catch the exception here as producer.send serializes + // the record before starting the future. + badRecordRouter.route( + receiver, + record, + null, + e, + "Failure serializing Key or Value of Kakfa record writing from Kafka"); + } } @FinishBundle @@ -110,6 +124,8 @@ public void teardown() { private final WriteRecords spec; private final Map producerConfig; + private final BadRecordRouter badRecordRouter; + private transient @Nullable Producer producer = null; // first exception and number of failures since last invocation of checkForFailures(): private transient @Nullable Exception sendException = null; @@ -122,6 +138,8 @@ public void teardown() { this.producerConfig = new HashMap<>(spec.getProducerConfig()); + this.badRecordRouter = spec.getBadRecordRouter(); + if (spec.getKeySerializer() != null) { this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, spec.getKeySerializer()); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java index 1b6e3addce22..924833290f13 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; import org.apache.beam.sdk.transforms.splittabledofn.GrowableOffsetRangeTracker; import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; @@ -45,6 +46,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; 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.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; @@ -60,6 +62,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -144,29 +147,37 @@ abstract class ReadFromKafkaDoFn extends DoFn>> { - static ReadFromKafkaDoFn create(ReadSourceDescriptors transform) { + static ReadFromKafkaDoFn create( + ReadSourceDescriptors transform, + TupleTag>> recordTag) { if (transform.isBounded()) { - return new Bounded<>(transform); + return new Bounded<>(transform, recordTag); } else { - return new Unbounded<>(transform); + return new Unbounded<>(transform, recordTag); } } @UnboundedPerElement private static class Unbounded extends ReadFromKafkaDoFn { - Unbounded(ReadSourceDescriptors transform) { - super(transform); + Unbounded( + ReadSourceDescriptors transform, + TupleTag>> recordTag) { + super(transform, recordTag); } } @BoundedPerElement private static class Bounded extends ReadFromKafkaDoFn { - Bounded(ReadSourceDescriptors transform) { - super(transform); + Bounded( + ReadSourceDescriptors transform, + TupleTag>> recordTag) { + super(transform, recordTag); } } - private ReadFromKafkaDoFn(ReadSourceDescriptors transform) { + private ReadFromKafkaDoFn( + ReadSourceDescriptors transform, + TupleTag>> recordTag) { this.consumerConfig = transform.getConsumerConfig(); this.offsetConsumerConfig = transform.getOffsetConsumerConfig(); this.keyDeserializerProvider = @@ -178,6 +189,8 @@ private ReadFromKafkaDoFn(ReadSourceDescriptors transform) { this.createWatermarkEstimatorFn = transform.getCreateWatermarkEstimatorFn(); this.timestampPolicyFactory = transform.getTimestampPolicyFactory(); this.checkStopReadingFn = transform.getCheckStopReadingFn(); + this.badRecordRouter = transform.getBadRecordRouter(); + this.recordTag = recordTag; } private static final Logger LOG = LoggerFactory.getLogger(ReadFromKafkaDoFn.class); @@ -193,6 +206,10 @@ private ReadFromKafkaDoFn(ReadSourceDescriptors transform) { createWatermarkEstimatorFn; private final @Nullable TimestampPolicyFactory timestampPolicyFactory; + private final BadRecordRouter badRecordRouter; + + private final TupleTag>> recordTag; + // Valid between bundle start and bundle finish. private transient @Nullable Deserializer keyDeserializerInstance = null; private transient @Nullable Deserializer valueDeserializerInstance = null; @@ -361,7 +378,8 @@ public ProcessContinuation processElement( @Element KafkaSourceDescriptor kafkaSourceDescriptor, RestrictionTracker tracker, WatermarkEstimator watermarkEstimator, - OutputReceiver>> receiver) { + MultiOutputReceiver receiver) + throws Exception { final LoadingCache avgRecordSize = Preconditions.checkStateNotNull(this.avgRecordSize); final Deserializer keyDeserializerInstance = @@ -431,36 +449,52 @@ public ProcessContinuation processElement( if (!tracker.tryClaim(rawRecord.offset())) { return ProcessContinuation.stop(); } - KafkaRecord kafkaRecord = - new KafkaRecord<>( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - ConsumerSpEL.getRecordTimestamp(rawRecord), - ConsumerSpEL.getRecordTimestampType(rawRecord), - ConsumerSpEL.hasHeaders() ? rawRecord.headers() : null, - ConsumerSpEL.deserializeKey(keyDeserializerInstance, rawRecord), - ConsumerSpEL.deserializeValue(valueDeserializerInstance, rawRecord)); - int recordSize = - (rawRecord.key() == null ? 0 : rawRecord.key().length) - + (rawRecord.value() == null ? 0 : rawRecord.value().length); - avgRecordSize - .getUnchecked(kafkaSourceDescriptor.getTopicPartition()) - .update(recordSize, rawRecord.offset() - expectedOffset); - rawSizes.update(recordSize); - expectedOffset = rawRecord.offset() + 1; - Instant outputTimestamp; - // The outputTimestamp and watermark will be computed by timestampPolicy, where the - // WatermarkEstimator should be a manual one. - if (timestampPolicy != null) { - TimestampPolicyContext context = - updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); - outputTimestamp = timestampPolicy.getTimestampForRecord(context, kafkaRecord); - } else { - Preconditions.checkStateNotNull(this.extractOutputTimestampFn); - outputTimestamp = extractOutputTimestampFn.apply(kafkaRecord); + try { + KafkaRecord kafkaRecord = + new KafkaRecord<>( + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + ConsumerSpEL.getRecordTimestamp(rawRecord), + ConsumerSpEL.getRecordTimestampType(rawRecord), + ConsumerSpEL.hasHeaders() ? rawRecord.headers() : null, + ConsumerSpEL.deserializeKey(keyDeserializerInstance, rawRecord), + ConsumerSpEL.deserializeValue(valueDeserializerInstance, rawRecord)); + int recordSize = + (rawRecord.key() == null ? 0 : rawRecord.key().length) + + (rawRecord.value() == null ? 0 : rawRecord.value().length); + avgRecordSize + .getUnchecked(kafkaSourceDescriptor.getTopicPartition()) + .update(recordSize, rawRecord.offset() - expectedOffset); + rawSizes.update(recordSize); + expectedOffset = rawRecord.offset() + 1; + Instant outputTimestamp; + // The outputTimestamp and watermark will be computed by timestampPolicy, where the + // WatermarkEstimator should be a manual one. + if (timestampPolicy != null) { + TimestampPolicyContext context = + updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + outputTimestamp = timestampPolicy.getTimestampForRecord(context, kafkaRecord); + } else { + Preconditions.checkStateNotNull(this.extractOutputTimestampFn); + outputTimestamp = extractOutputTimestampFn.apply(kafkaRecord); + } + receiver + .get(recordTag) + .outputWithTimestamp(KV.of(kafkaSourceDescriptor, kafkaRecord), outputTimestamp); + } catch (SerializationException e) { + // This exception should only occur during the key and value deserialization when + // creating the Kafka Record + badRecordRouter.route( + receiver, + rawRecord, + null, + e, + "Failure deserializing Key or Value of Kakfa record reading from Kafka"); + if (timestampPolicy != null) { + updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + } } - receiver.outputWithTimestamp(KV.of(kafkaSourceDescriptor, kafkaRecord), outputTimestamp); } } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java index 2ccf7dcc3a93..38bf723a15a9 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java @@ -350,13 +350,7 @@ public void testConstructKafkaWrite() throws Exception { RunnerApi.PTransform writeComposite = result.getComponents().getTransformsOrThrow(transform.getSubtransforms(1)); RunnerApi.PTransform writeParDo = - result - .getComponents() - .getTransformsOrThrow( - result - .getComponents() - .getTransformsOrThrow(writeComposite.getSubtransforms(0)) - .getSubtransforms(0)); + result.getComponents().getTransformsOrThrow(writeComposite.getSubtransforms(0)); RunnerApi.ParDoPayload parDoPayload = RunnerApi.ParDoPayload.parseFrom(writeParDo.getSpec().getPayload()); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 2c8ace9c66c1..5b976687f2c1 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.UUID; @@ -43,6 +44,9 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.common.IOITHelper; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; +import org.apache.beam.sdk.io.kafka.KafkaIOTest.ErrorSinkTransform; +import org.apache.beam.sdk.io.kafka.KafkaIOTest.FailingLongSerializer; +import org.apache.beam.sdk.io.kafka.ReadFromKafkaDoFnTest.FailingDeserializer; import org.apache.beam.sdk.io.synthetic.SyntheticBoundedSource; import org.apache.beam.sdk.io.synthetic.SyntheticSourceOptions; import org.apache.beam.sdk.options.Default; @@ -72,6 +76,7 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; @@ -124,8 +129,6 @@ public class KafkaIOIT { private static final String RUN_TIME_METRIC_NAME = "run_time"; - private static final String READ_ELEMENT_METRIC_NAME = "kafka_read_element_count"; - private static final String NAMESPACE = KafkaIOIT.class.getName(); private static final String TEST_ID = UUID.randomUUID().toString(); @@ -352,6 +355,68 @@ public void processElement(@Element String element, OutputReceiver outpu } } + // This test verifies that bad data from Kafka is properly sent to the error handler + @Test + public void testKafkaIOSDFReadWithErrorHandler() throws IOException { + writePipeline + .apply(Create.of(KV.of("key", "val"))) + .apply( + "Write to Kafka", + KafkaIO.write() + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) + .withKeySerializer(StringSerializer.class) + .withValueSerializer(StringSerializer.class) + .withTopic(options.getKafkaTopic() + "-failingDeserialization")); + + PipelineResult writeResult = writePipeline.run(); + PipelineResult.State writeState = writeResult.waitUntilFinish(); + assertNotEquals(PipelineResult.State.FAILED, writeState); + + BadRecordErrorHandler> eh = + sdfReadPipeline.registerBadRecordErrorHandler(new ErrorSinkTransform()); + sdfReadPipeline.apply( + KafkaIO.read() + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) + .withTopic(options.getKafkaTopic() + "-failingDeserialization") + .withConsumerConfigUpdates(ImmutableMap.of("auto.offset.reset", "earliest")) + .withKeyDeserializer(FailingDeserializer.class) + .withValueDeserializer(FailingDeserializer.class) + .withBadRecordErrorHandler(eh)); + eh.close(); + + PAssert.thatSingleton(Objects.requireNonNull(eh.getOutput())).isEqualTo(1L); + + PipelineResult readResult = sdfReadPipeline.run(); + PipelineResult.State readState = + readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); + cancelIfTimeouted(readResult, readState); + assertNotEquals(PipelineResult.State.FAILED, readState); + } + + @Test + public void testKafkaIOWriteWithErrorHandler() throws IOException { + + BadRecordErrorHandler> eh = + writePipeline.registerBadRecordErrorHandler(new ErrorSinkTransform()); + writePipeline + .apply("Create single KV", Create.of(KV.of("key", 4L))) + .apply( + "Write to Kafka", + KafkaIO.write() + .withBootstrapServers(options.getKafkaBootstrapServerAddresses()) + .withKeySerializer(StringSerializer.class) + .withValueSerializer(FailingLongSerializer.class) + .withTopic(options.getKafkaTopic() + "-failingSerialization") + .withBadRecordErrorHandler(eh)); + eh.close(); + + PAssert.thatSingleton(Objects.requireNonNull(eh.getOutput())).isEqualTo(1L); + + PipelineResult writeResult = writePipeline.run(); + PipelineResult.State writeState = writeResult.waitUntilFinish(); + assertNotEquals(PipelineResult.State.FAILED, writeState); + } + // This test roundtrips a single KV to verify that externalWithMetadata // can handle null keys and values correctly. @Test @@ -484,9 +549,7 @@ public void testKafkaWithDynamicPartitions() throws IOException { public void testKafkaWithStopReadingFunction() { AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); - PipelineResult readResult = runWithStopReadingFn(checkStopReadingFn, "stop-reading"); - - assertEquals(-1, readElementMetric(readResult, NAMESPACE, READ_ELEMENT_METRIC_NAME)); + runWithStopReadingFn(checkStopReadingFn, "stop-reading", 0L); } private static class AlwaysStopCheckStopReadingFn implements CheckStopReadingFn { @@ -500,11 +563,7 @@ public Boolean apply(TopicPartition input) { public void testKafkaWithDelayedStopReadingFunction() { DelayedCheckStopReadingFn checkStopReadingFn = new DelayedCheckStopReadingFn(); - PipelineResult readResult = runWithStopReadingFn(checkStopReadingFn, "delayed-stop-reading"); - - assertEquals( - sourceOptions.numRecords, - readElementMetric(readResult, NAMESPACE, READ_ELEMENT_METRIC_NAME)); + runWithStopReadingFn(checkStopReadingFn, "delayed-stop-reading", sourceOptions.numRecords); } public static final Schema KAFKA_TOPIC_SCHEMA = @@ -644,7 +703,7 @@ private static class DelayedCheckStopReadingFn implements CheckStopReadingFn { @Override public Boolean apply(TopicPartition input) { - if (checkCount >= 5) { + if (checkCount >= 10) { return true; } checkCount++; @@ -652,7 +711,8 @@ public Boolean apply(TopicPartition input) { } } - private PipelineResult runWithStopReadingFn(CheckStopReadingFn function, String topicSuffix) { + private void runWithStopReadingFn( + CheckStopReadingFn function, String topicSuffix, Long expectedCount) { writePipeline .apply("Generate records", Read.from(new SyntheticBoundedSource(sourceOptions))) .apply("Measure write time", ParDo.of(new TimeMonitor<>(NAMESPACE, WRITE_TIME_METRIC_NAME))) @@ -661,21 +721,31 @@ private PipelineResult runWithStopReadingFn(CheckStopReadingFn function, String writeToKafka().withTopic(options.getKafkaTopic() + "-" + topicSuffix)); readPipeline.getOptions().as(Options.class).setStreaming(true); - readPipeline - .apply( - "Read from unbounded Kafka", - readFromKafka() - .withTopic(options.getKafkaTopic() + "-" + topicSuffix) - .withCheckStopReadingFn(function)) - .apply("Measure read time", ParDo.of(new TimeMonitor<>(NAMESPACE, READ_TIME_METRIC_NAME))); + PCollection count = + readPipeline + .apply( + "Read from unbounded Kafka", + readFromKafka() + .withTopic(options.getKafkaTopic() + "-" + topicSuffix) + .withCheckStopReadingFn(function)) + .apply( + "Measure read time", ParDo.of(new TimeMonitor<>(NAMESPACE, READ_TIME_METRIC_NAME))) + .apply("Window", Window.into(CalendarWindows.years(1))) + .apply( + "Counting element", + Combine.globally(Count.>combineFn()).withoutDefaults()); + + if (expectedCount == 0L) { + PAssert.that(count).empty(); + } else { + PAssert.thatSingleton(count).isEqualTo(expectedCount); + } PipelineResult writeResult = writePipeline.run(); writeResult.waitUntilFinish(); PipelineResult readResult = readPipeline.run(); readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); - - return readResult; } @Test @@ -686,7 +756,7 @@ public void testWatermarkUpdateWithSparseMessages() throws IOException, Interrup String topicName = "SparseDataTopicPartition-" + UUID.randomUUID(); Map records = new HashMap<>(); - for (int i = 0; i < 5; i++) { + for (int i = 1; i <= 5; i++) { records.put(i, String.valueOf(i)); } @@ -725,7 +795,7 @@ public void testWatermarkUpdateWithSparseMessages() throws IOException, Interrup PipelineResult readResult = sdfReadPipeline.run(); - Thread.sleep(options.getReadTimeout() * 1000); + Thread.sleep(options.getReadTimeout() * 1000 * 2); for (String value : records.values()) { kafkaIOITExpectedLogs.verifyError(value); @@ -753,11 +823,6 @@ public void processElement( } } - private long readElementMetric(PipelineResult result, String namespace, String name) { - MetricsReader metricsReader = new MetricsReader(result, namespace); - return metricsReader.getCounterMetric(name); - } - private Set readMetrics(PipelineResult writeResult, PipelineResult readResult) { BiFunction supplier = (reader, metricName) -> { diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index aeb5818e9134..b0df82bcdc19 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -51,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -87,6 +88,7 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Distinct; @@ -95,11 +97,15 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.Min; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; @@ -121,9 +127,12 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; +import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -136,7 +145,10 @@ import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.hamcrest.collection.IsIterableWithSize; import org.joda.time.Duration; @@ -1379,7 +1391,7 @@ public void testSink() throws Exception { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1404,13 +1416,81 @@ public void testSink() throws Exception { } } + public static class FailingLongSerializer implements Serializer { + // enables instantiation by registrys + public FailingLongSerializer() {} + + @Override + public byte[] serialize(String topic, Long data) { + throw new SerializationException("ExpectedSerializationException"); + } + + @Override + public void configure(Map configs, boolean isKey) { + // intentionally left blank for compatibility with older kafka versions + } + } + + @Test + public void testSinkWithSerializationErrors() throws Exception { + // Attempt to write 10 elements to Kafka, but they will all fail to serialize, and be sent to + // the DLQ + + int numElements = 10; + + try (MockProducerWrapper producerWrapper = + new MockProducerWrapper(new FailingLongSerializer())) { + + ProducerSendCompletionThread completionThread = + new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); + + String topic = "test"; + + BadRecordErrorHandler> eh = + p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + + p.apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()).withoutMetadata()) + .apply( + KafkaIO.write() + .withBootstrapServers("none") + .withTopic(topic) + .withKeySerializer(IntegerSerializer.class) + .withValueSerializer(FailingLongSerializer.class) + .withInputTimestamp() + .withProducerFactoryFn(new ProducerFactoryFn(producerWrapper.producerKey)) + .withBadRecordErrorHandler(eh)); + + eh.close(); + + PAssert.thatSingleton(Objects.requireNonNull(eh.getOutput())).isEqualTo(10L); + + p.run(); + + completionThread.shutdown(); + + verifyProducerRecords(producerWrapper.mockProducer, topic, 0, false, true); + } + } + + public static class ErrorSinkTransform + extends PTransform, PCollection> { + + @Override + public @UnknownKeyFor @NonNull @Initialized PCollection expand( + PCollection input) { + return input + .apply("Window", Window.into(CalendarWindows.years(1))) + .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); + } + } + @Test public void testValuesSink() throws Exception { // similar to testSink(), but use values()' interface. int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1442,7 +1522,7 @@ public void testRecordsSink() throws Exception { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1474,7 +1554,7 @@ public void testSinkToMultipleTopics() throws Exception { // Set different output topic names int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1519,7 +1599,7 @@ public void testKafkaWriteHeaders() throws Exception { // Set different output topic names int numElements = 1; SimpleEntry header = new SimpleEntry<>("header_key", "header_value"); - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1562,7 +1642,7 @@ public void testKafkaWriteHeaders() throws Exception { public void testSinkProducerRecordsWithCustomTS() throws Exception { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1601,7 +1681,7 @@ public void testSinkProducerRecordsWithCustomTS() throws Exception { public void testSinkProducerRecordsWithCustomPartition() throws Exception { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1725,7 +1805,7 @@ public void testExactlyOnceSink() { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -1803,7 +1883,7 @@ public void testSinkWithSendErrors() throws Throwable { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThreadWithErrors = new ProducerSendCompletionThread(producerWrapper.mockProducer, 10, 100).start(); @@ -1993,7 +2073,7 @@ public void testSourceWithPatternDisplayData() { @Test public void testSinkDisplayData() { - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { KafkaIO.Write write = KafkaIO.write() .withBootstrapServers("myServerA:9092,myServerB:9092") @@ -2017,7 +2097,7 @@ public void testSinkMetrics() throws Exception { int numElements = 1000; - try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) { + try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread(producerWrapper.mockProducer).start(); @@ -2109,14 +2189,22 @@ private static class MockProducerWrapper implements AutoCloseable { } } - MockProducerWrapper() { + MockProducerWrapper(Serializer valueSerializer) { producerKey = String.valueOf(ThreadLocalRandom.current().nextLong()); mockProducer = new MockProducer( + Cluster.empty() + .withPartitions( + ImmutableMap.of( + new TopicPartition("test", 0), + new PartitionInfo("test", 0, null, null, null), + new TopicPartition("test", 1), + new PartitionInfo("test", 1, null, null, null))), false, // disable synchronous completion of send. see ProducerSendCompletionThread // below. + new DefaultPartitioner(), new IntegerSerializer(), - new LongSerializer()) { + valueSerializer) { // override flush() so that it does not complete all the waiting sends, giving a chance // to diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index 554c6d2fcaf1..48b5b060a295 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.kafka; +import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -41,15 +42,20 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -64,7 +70,9 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; @@ -80,19 +88,22 @@ public class ReadFromKafkaDoFnTest { private final TopicPartition topicPartition = new TopicPartition("topic", 0); + private static final TupleTag>> RECORDS = + new TupleTag<>(); + @Rule public ExpectedException thrown = ExpectedException.none(); private final SimpleMockKafkaConsumer consumer = new SimpleMockKafkaConsumer(OffsetResetStrategy.NONE, topicPartition); private final ReadFromKafkaDoFn dofnInstance = - ReadFromKafkaDoFn.create(makeReadSourceDescriptor(consumer)); + ReadFromKafkaDoFn.create(makeReadSourceDescriptor(consumer), RECORDS); private final ExceptionMockKafkaConsumer exceptionConsumer = new ExceptionMockKafkaConsumer(OffsetResetStrategy.NONE, topicPartition); private final ReadFromKafkaDoFn exceptionDofnInstance = - ReadFromKafkaDoFn.create(makeReadSourceDescriptor(exceptionConsumer)); + ReadFromKafkaDoFn.create(makeReadSourceDescriptor(exceptionConsumer), RECORDS); private ReadSourceDescriptors makeReadSourceDescriptor( Consumer kafkaMockConsumer) { @@ -109,6 +120,31 @@ public Consumer apply(Map input) { .withBootstrapServers("bootstrap_server"); } + private ReadSourceDescriptors makeFailingReadSourceDescriptor( + Consumer kafkaMockConsumer) { + return ReadSourceDescriptors.read() + .withKeyDeserializer(FailingDeserializer.class) + .withValueDeserializer(FailingDeserializer.class) + .withConsumerFactoryFn( + new SerializableFunction, Consumer>() { + @Override + public Consumer apply(Map input) { + return kafkaMockConsumer; + } + }) + .withBootstrapServers("bootstrap_server"); + } + + public static class FailingDeserializer implements Deserializer { + + public FailingDeserializer() {} + + @Override + public String deserialize(String topic, byte[] data) { + throw new SerializationException("Intentional serialization exception"); + } + } + private static class ExceptionMockKafkaConsumer extends MockConsumer { private final TopicPartition topicPartition; @@ -254,23 +290,57 @@ public synchronized long position(TopicPartition partition) { } } - private static class MockOutputReceiver - implements OutputReceiver>> { + private static class MockMultiOutputReceiver implements MultiOutputReceiver { + + MockOutputReceiver>> mockOutputReceiver = + new MockOutputReceiver<>(); + + MockOutputReceiver badOutputReceiver = new MockOutputReceiver<>(); + + @Override + public @UnknownKeyFor @NonNull @Initialized OutputReceiver get( + @UnknownKeyFor @NonNull @Initialized TupleTag tag) { + if (RECORDS.equals(tag)) { + return (OutputReceiver) mockOutputReceiver; + } else if (BAD_RECORD_TAG.equals(tag)) { + return (OutputReceiver) badOutputReceiver; + } else { + throw new RuntimeException("Invalid Tag"); + } + } + + public List>> getGoodRecords() { + return mockOutputReceiver.getOutputs(); + } - private final List>> records = - new ArrayList<>(); + public List getBadRecords() { + return badOutputReceiver.getOutputs(); + } @Override - public void output(KV> output) {} + public @UnknownKeyFor @NonNull @Initialized + OutputReceiver<@UnknownKeyFor @NonNull @Initialized Row> getRowReceiver( + @UnknownKeyFor @NonNull @Initialized TupleTag tag) { + return null; + } + } + + private static class MockOutputReceiver implements OutputReceiver { + + private final List records = new ArrayList<>(); + + @Override + public void output(T output) { + records.add(output); + } @Override public void outputWithTimestamp( - KV> output, - @UnknownKeyFor @NonNull @Initialized Instant timestamp) { + T output, @UnknownKeyFor @NonNull @Initialized Instant timestamp) { records.add(output); } - public List>> getOutputs() { + public List getOutputs() { return this.records; } } @@ -386,7 +456,7 @@ public void testInitialRestrictionWithException() throws Exception { @Test public void testProcessElement() throws Exception { - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); consumer.setNumOfRecordsPerPoll(3L); long startOffset = 5L; OffsetRangeTracker tracker = @@ -396,7 +466,8 @@ public void testProcessElement() throws Exception { ProcessContinuation result = dofnInstance.processElement(descriptor, tracker, null, receiver); assertEquals(ProcessContinuation.stop(), result); assertEquals( - createExpectedRecords(descriptor, startOffset, 3, "key", "value"), receiver.getOutputs()); + createExpectedRecords(descriptor, startOffset, 3, "key", "value"), + receiver.getGoodRecords()); } @Test @@ -406,7 +477,7 @@ public void testRawSizeMetric() throws Exception { MetricsContainerImpl container = new MetricsContainerImpl("any"); MetricsEnvironment.setCurrentContainer(container); - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); consumer.setNumOfRecordsPerPoll(numElements); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0, numElements)); KafkaSourceDescriptor descriptor = @@ -427,7 +498,7 @@ public void testRawSizeMetric() throws Exception { @Test public void testProcessElementWithEmptyPoll() throws Exception { - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); consumer.setNumOfRecordsPerPoll(-1); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); ProcessContinuation result = @@ -437,12 +508,12 @@ public void testProcessElementWithEmptyPoll() throws Exception { null, receiver); assertEquals(ProcessContinuation.resume(), result); - assertTrue(receiver.getOutputs().isEmpty()); + assertTrue(receiver.getGoodRecords().isEmpty()); } @Test public void testProcessElementWhenTopicPartitionIsRemoved() throws Exception { - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); consumer.setRemoved(); consumer.setNumOfRecordsPerPoll(10); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); @@ -457,7 +528,7 @@ public void testProcessElementWhenTopicPartitionIsRemoved() throws Exception { @Test public void testProcessElementWhenTopicPartitionIsStopped() throws Exception { - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); ReadFromKafkaDoFn instance = ReadFromKafkaDoFn.create( makeReadSourceDescriptor(consumer) @@ -470,7 +541,8 @@ public Boolean apply(TopicPartition input) { return true; } }) - .build()); + .build(), + RECORDS); instance.setup(); consumer.setNumOfRecordsPerPoll(10); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); @@ -489,7 +561,7 @@ public void testProcessElementWithException() throws Exception { thrown.expect(KafkaException.class); thrown.expectMessage("SeekException"); - MockOutputReceiver receiver = new MockOutputReceiver(); + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); exceptionDofnInstance.processElement( @@ -499,6 +571,61 @@ public void testProcessElementWithException() throws Exception { receiver); } + @Test + public void testProcessElementWithDeserializationExceptionDefaultRecordHandler() + throws Exception { + thrown.expect(SerializationException.class); + thrown.expectMessage("Intentional serialization exception"); + + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); + OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); + + consumer.setNumOfRecordsPerPoll(1); + + ReadFromKafkaDoFn dofnInstance = + ReadFromKafkaDoFn.create(makeFailingReadSourceDescriptor(consumer), RECORDS); + + dofnInstance.setup(); + + dofnInstance.processElement( + KafkaSourceDescriptor.of(topicPartition, null, null, null, null, null), + tracker, + null, + receiver); + + Assert.assertEquals("OutputRecordSize", 0, receiver.getGoodRecords().size()); + Assert.assertEquals("OutputErrorSize", 0, receiver.getBadRecords().size()); + } + + @Test + public void testProcessElementWithDeserializationExceptionRecordingRecordHandler() + throws Exception { + MockMultiOutputReceiver receiver = new MockMultiOutputReceiver(); + OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, 1L)); + + consumer.setNumOfRecordsPerPoll(1); + + // Because we never actually execute the pipeline, no data will actually make it to the error + // handler. This will just configure the ReadSourceDesriptors to route the errors to the output + // PCollection instead of rethrowing. + ReadSourceDescriptors descriptors = + makeFailingReadSourceDescriptor(consumer) + .withBadRecordErrorHandler(new DefaultErrorHandler<>()); + + ReadFromKafkaDoFn dofnInstance = ReadFromKafkaDoFn.create(descriptors, RECORDS); + + dofnInstance.setup(); + + dofnInstance.processElement( + KafkaSourceDescriptor.of(topicPartition, null, null, null, null, null), + tracker, + null, + receiver); + + Assert.assertEquals("OutputRecordSize", 0, receiver.getGoodRecords().size()); + Assert.assertEquals("OutputErrorSize", 1, receiver.getBadRecords().size()); + } + private static final TypeDescriptor KAFKA_SOURCE_DESCRIPTOR_TYPE_DESCRIPTOR = new TypeDescriptor() {}; @@ -522,7 +649,8 @@ private BoundednessVisitor testBoundedness( .apply( ParDo.of( ReadFromKafkaDoFn.create( - readSourceDescriptorsDecorator.apply(makeReadSourceDescriptor(consumer))))) + readSourceDescriptorsDecorator.apply(makeReadSourceDescriptor(consumer)), + RECORDS))) .setCoder( KvCoder.of( SerializableCoder.of(KafkaSourceDescriptor.class), From e620b5aac97898dacf7a4587c7859e6e1ea956ca Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 7 Dec 2023 10:12:35 -0500 Subject: [PATCH 052/224] Bump golang.org/x/oauth2 from 0.14.0 to 0.15.0 in /sdks (#29630) Bumps [golang.org/x/oauth2](https://github.com/golang/oauth2) from 0.14.0 to 0.15.0. - [Commits](https://github.com/golang/oauth2/compare/v0.14.0...v0.15.0) --- updated-dependencies: - dependency-name: golang.org/x/oauth2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 2060c3c0a500..c46c7e28a58c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -55,7 +55,7 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.13.1 golang.org/x/net v0.19.0 - golang.org/x/oauth2 v0.14.0 + golang.org/x/oauth2 v0.15.0 golang.org/x/sync v0.5.0 golang.org/x/sys v0.15.0 golang.org/x/text v0.14.0 diff --git a/sdks/go.sum b/sdks/go.sum index 3c99d6587274..5df50d7fd020 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -562,8 +562,8 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.14.0 h1:P0Vrf/2538nmC0H+pEQ3MNFRRnVR7RlqyVw+bvm26z0= -golang.org/x/oauth2 v0.14.0/go.mod h1:lAtNWgaWfL4cm7j2OV8TxGi9Qb7ECORx8DktCY74OwM= +golang.org/x/oauth2 v0.15.0 h1:s8pnnxNVzjWyrvYdFUQq5llS1PX2zhPXmccZv99h7uQ= +golang.org/x/oauth2 v0.15.0/go.mod h1:q48ptWNTY5XWf+JNten23lcvHpLJ0ZSxF5ttTHKVCAM= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From d2d19ef5dfd8d2b0967d178312e11d5c00e12a7b Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Thu, 7 Dec 2023 10:18:05 -0500 Subject: [PATCH 053/224] Bugfix/extend java precommit timeout (#29664) * 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 * extend timeout * Update .github/workflows/beam_PreCommit_Java.yml Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- .github/workflows/beam_PreCommit_Java.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index b4aa233d522d..3254ac2dc97b 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -155,7 +155,6 @@ env: jobs: beam_PreCommit_Java: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - timeout-minutes: 180 runs-on: [self-hosted, ubuntu-20.04, main] strategy: fail-fast: false From 8d9fd02af5ebf5ee8b67b50ef0c9ac2d5050298c Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 7 Dec 2023 11:08:30 -0500 Subject: [PATCH 054/224] Bump Java VR interval and shutdown remaining Dataflow VR Jenkins PostCommit (#29647) --- ...stCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ..._ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- ...ava_ValidatesRunner_Dataflow_Streaming.yml | 2 +- ...ommit_Java_ValidatesRunner_Dataflow.groovy | 51 ---------------- ...ava_ValidatesRunner_Dataflow_Java11.groovy | 60 ------------------- ...ava_ValidatesRunner_Dataflow_Java17.groovy | 57 ------------------ ..._ValidatesRunner_Dataflow_Streaming.groovy | 50 ---------------- ...it_Java_ValidatesRunner_Dataflow_V2.groovy | 51 ---------------- ...lidatesRunner_Dataflow_V2_Streaming.groovy | 51 ---------------- ...mit_Python_ValidatesRunner_Dataflow.groovy | 45 -------------- 10 files changed, 3 insertions(+), 368 deletions(-) delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java11.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java17.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.groovy delete mode 100644 .test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Dataflow.groovy diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index bc4fe988e878..4a494bf55899 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow on: schedule: - - cron: '30 4/6 * * *' + - cron: '30 4/8 * * *' pull_request_target: paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json'] workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 66c41ea1f363..b04ac0857390 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow JavaVersions on: schedule: - - cron: '30 4/6 * * *' + - cron: '30 4/8 * * *' pull_request_target: paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.json'] workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 8601739c0234..7a53103809b5 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -19,7 +19,7 @@ name: PostCommit Java ValidatesRunner Dataflow Streaming on: schedule: - - cron: '30 4/6 * * *' + - cron: '30 4/8 * * *' pull_request_target: paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json'] workflow_dispatch: diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy deleted file mode 100644 index b0c0fa2c6e47..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Dataflow -// runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow', - 'Run Dataflow ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests (streaming/batch auto)', this) { - - description('Runs the ValidatesRunner suite on the Dataflow runner.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 480) - previousNames(/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunner') - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java11.groovy deleted file mode 100644 index bb71f605cd4f..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java11.groovy +++ /dev/null @@ -1,60 +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 commonJobProperties -import PostcommitJobBuilder - - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11', - 'Run Dataflow ValidatesRunner Java 11', 'Google Cloud Dataflow Runner ValidatesRunner Tests On Java 11', this) { - - description('Runs the ValidatesRunner suite on the Dataflow runner with Java 11 worker harness.') - - def JAVA_11_HOME = '/usr/lib/jvm/java-11-openjdk-amd64' - def JAVA_8_HOME = '/usr/lib/jvm/java-8-openjdk-amd64' - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 480) - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:testJar') - tasks(':runners:google-cloud-dataflow-java:worker:shadowJar') - switches("-Dorg.gradle.java.home=${JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunner') - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - switches('-x jar') - switches('-x testJar') - switches('-x classes') - switches('-x testClasses') - switches("-Dorg.gradle.java.home=${JAVA_11_HOME}") - - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java17.groovy deleted file mode 100644 index 7bfcc256619f..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Java17.groovy +++ /dev/null @@ -1,57 +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 commonJobProperties -import PostcommitJobBuilder - - -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow_Java17', - 'Run Dataflow ValidatesRunner Java 17', 'Google Cloud Dataflow Runner ValidatesRunner Tests On Java 17', this) { - - description('Runs the ValidatesRunner suite on the Dataflow runner with Java 17 worker harness.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 480) - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:testJar') - tasks(':runners:google-cloud-dataflow-java:worker:shadowJar') - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_8_HOME}") - } - - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunner') - switches('-x shadowJar') - switches('-x shadowTestJar') - switches('-x compileJava') - switches('-x compileTestJava') - switches('-x jar') - switches('-x testJar') - switches('-x classes') - switches('-x testClasses') - switches("-Dorg.gradle.java.home=${commonJobProperties.JAVA_17_HOME}") - - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.groovy deleted file mode 100644 index 5d5c060175b9..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.groovy +++ /dev/null @@ -1,50 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Dataflow -// runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming', - 'Run Dataflow Streaming ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests (streaming)', this) { - - description('Runs the ValidatesRunner suite on the Dataflow runner forcing streaming mode.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 720) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunnerStreaming') - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2.groovy deleted file mode 100644 index abb3c9f5b442..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Dataflow -// runner V2. -this.buildSchedule = 'H H/8 * * *' -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_VR_Dataflow_V2', - 'Run Java Dataflow V2 ValidatesRunner', 'Google Cloud Dataflow Runner V2 Java ValidatesRunner Tests', this) { - - description('Runs Java ValidatesRunner suite on the Dataflow runner V2.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 390) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunnerV2') - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.groovy deleted file mode 100644 index 3c4b5bf3a750..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.groovy +++ /dev/null @@ -1,51 +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 commonJobProperties -import PostcommitJobBuilder - - -// This job runs the suite of ValidatesRunner tests against the Dataflow -// runner V2. -this.buildSchedule = 'H H/8 * * *' -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_VR_Dataflow_V2_Streaming', - 'Run Java Dataflow V2 ValidatesRunner Streaming', 'Google Cloud Dataflow Runner V2 Java ValidatesRunner Tests (streaming)', this) { - - description('Runs Java ValidatesRunner suite on the Dataflow runner V2 forcing streaming mode.') - - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 510) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:google-cloud-dataflow-java:validatesRunnerV2Streaming') - // Increase parallel worker threads above processor limit since most time is - // spent waiting on Dataflow jobs. ValidatesRunner tests on Dataflow are slow - // because each one launches a Dataflow job with about 3 mins of overhead. - // 3 x num_cores strikes a good balance between maxing out parallelism without - // overloading the machines. - commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Dataflow.groovy deleted file mode 100644 index db052a0046ce..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesRunner_Dataflow.groovy +++ /dev/null @@ -1,45 +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 commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Python ValidatesRunner tests against the -// Dataflow runner V2. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Py_VR_Dataflow', 'Run Python Dataflow ValidatesRunner', - 'Google Cloud Dataflow Runner Python ValidatesRunner Tests', this) { - description('Runs Python ValidatesRunner suite on the Dataflow runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 200) - - publishers { - archiveJunit('**/pytest*.xml') - } - - // Execute gradle task to test Python SDK. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':sdks:python:test-suites:dataflow:validatesRunnerBatchTests') - tasks(':sdks:python:test-suites:dataflow:validatesRunnerStreamingTests') - switches('-PuseWheelDistribution') - commonJobProperties.setGradleSwitches(delegate) - } - } - } From 84d070bae9b59160d4d88aa2fab9d98b4ed2589f Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 7 Dec 2023 12:40:26 -0500 Subject: [PATCH 055/224] Add per-worker keys to stateful BatchElements, plumb through batching options to all model handlers (#29642) * Key streaming BatchElements bundles per-worker * Plumb through max_batch_duration_secs support * Linting * Tag shared handle --- .../ml/inference/huggingface_inference.py | 15 ++++++++++ .../ml/inference/onnx_inference.py | 18 ++++++++++++ .../ml/inference/pytorch_inference.py | 10 +++++++ .../ml/inference/sklearn_inference.py | 10 +++++++ .../ml/inference/tensorflow_inference.py | 10 +++++++ .../ml/inference/tensorrt_inference.py | 7 ++++- .../ml/inference/vertex_ai_inference.py | 22 +++++++++++++- .../ml/inference/xgboost_inference.py | 21 ++++++++++++++ sdks/python/apache_beam/transforms/util.py | 29 ++++++++++++++++++- 9 files changed, 139 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/ml/inference/huggingface_inference.py b/sdks/python/apache_beam/ml/inference/huggingface_inference.py index 878d7bfc9cf2..1bc92c462c91 100644 --- a/sdks/python/apache_beam/ml/inference/huggingface_inference.py +++ b/sdks/python/apache_beam/ml/inference/huggingface_inference.py @@ -224,6 +224,7 @@ def __init__( inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """ @@ -255,6 +256,8 @@ def __init__( Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -277,6 +280,8 @@ def __init__( self._batching_kwargs["min_batch_size"] = min_batch_size if max_batch_size is not None: self._batching_kwargs["max_batch_size"] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._large_model = large_model self._framework = framework @@ -405,6 +410,7 @@ def __init__( inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """ @@ -436,6 +442,8 @@ def __init__( Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -458,6 +466,8 @@ def __init__( self._batching_kwargs["min_batch_size"] = min_batch_size if max_batch_size is not None: self._batching_kwargs["max_batch_size"] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._large_model = large_model self._framework = "" @@ -579,6 +589,7 @@ def __init__( inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """ @@ -617,6 +628,8 @@ def __init__( Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -639,6 +652,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._large_model = large_model # Check if the device is specified twice. If true then the device parameter diff --git a/sdks/python/apache_beam/ml/inference/onnx_inference.py b/sdks/python/apache_beam/ml/inference/onnx_inference.py index 18e115a61887..f7b6c0115af8 100644 --- a/sdks/python/apache_beam/ml/inference/onnx_inference.py +++ b/sdks/python/apache_beam/ml/inference/onnx_inference.py @@ -19,6 +19,7 @@ from typing import Callable from typing import Dict from typing import Iterable +from typing import Mapping from typing import Optional from typing import Sequence @@ -63,6 +64,9 @@ def __init__( #pylint: disable=dangerous-default-value *, inference_fn: NumpyInferenceFn = default_numpy_inference_fn, large_model: bool = False, + min_batch_size: Optional[int] = None, + max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, **kwargs): """ Implementation of the ModelHandler interface for onnx using numpy arrays as input. @@ -80,6 +84,10 @@ def __init__( #pylint: disable=dangerous-default-value memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should set this to True if N*W > M. + min_batch_size: the minimum batch size to use when batching inputs. + max_batch_size: the maximum batch size to use when batching inputs. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. kwargs: 'env_vars' can be used to set environment variables before loading the model. """ @@ -90,6 +98,13 @@ def __init__( #pylint: disable=dangerous-default-value self._model_inference_fn = inference_fn self._env_vars = kwargs.get('env_vars', {}) self._large_model = large_model + self._batching_kwargs = {} + if min_batch_size is not None: + self._batching_kwargs["min_batch_size"] = min_batch_size + if max_batch_size is not None: + self._batching_kwargs["max_batch_size"] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs def load_model(self) -> ort.InferenceSession: """Loads and initializes an onnx inference session for processing.""" @@ -143,3 +158,6 @@ def get_metrics_namespace(self) -> str: def share_model_across_processes(self) -> bool: return self._large_model + + def batch_elements_kwargs(self) -> Mapping[str, Any]: + return self._batching_kwargs diff --git a/sdks/python/apache_beam/ml/inference/pytorch_inference.py b/sdks/python/apache_beam/ml/inference/pytorch_inference.py index 26e593fdd7de..480dc538195c 100644 --- a/sdks/python/apache_beam/ml/inference/pytorch_inference.py +++ b/sdks/python/apache_beam/ml/inference/pytorch_inference.py @@ -193,6 +193,7 @@ def __init__( torch_script_model_path: Optional[str] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, load_model_args: Optional[Dict[str, Any]] = None, **kwargs): @@ -227,6 +228,8 @@ def __init__( batch will be fed into the inference_fn as a Sequence of Tensors. max_batch_size: the maximum batch size to use when batching inputs. This batch will be fed into the inference_fn as a Sequence of Tensors. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -254,6 +257,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._torch_script_model_path = torch_script_model_path self._load_model_args = load_model_args if load_model_args else {} self._env_vars = kwargs.get('env_vars', {}) @@ -421,6 +426,7 @@ def __init__( torch_script_model_path: Optional[str] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, load_model_args: Optional[Dict[str, Any]] = None, **kwargs): @@ -460,6 +466,8 @@ def __init__( batch will be fed into the inference_fn as a Sequence of Keyed Tensors. max_batch_size: the maximum batch size to use when batching inputs. This batch will be fed into the inference_fn as a Sequence of Keyed Tensors. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -487,6 +495,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._torch_script_model_path = torch_script_model_path self._load_model_args = load_model_args if load_model_args else {} self._env_vars = kwargs.get('env_vars', {}) diff --git a/sdks/python/apache_beam/ml/inference/sklearn_inference.py b/sdks/python/apache_beam/ml/inference/sklearn_inference.py index c2bd2cee66e6..befeca7f33b2 100644 --- a/sdks/python/apache_beam/ml/inference/sklearn_inference.py +++ b/sdks/python/apache_beam/ml/inference/sklearn_inference.py @@ -90,6 +90,7 @@ def __init__( inference_fn: NumpyInferenceFn = _default_numpy_inference_fn, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """ Implementation of the ModelHandler interface for scikit-learn @@ -111,6 +112,8 @@ def __init__( max_batch_size: the maximum batch size to use when batching inputs. This batch will be fed into the inference_fn as a Sequence of Numpy ndarrays. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -126,6 +129,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._env_vars = kwargs.get('env_vars', {}) self._large_model = large_model @@ -212,6 +217,7 @@ def __init__( inference_fn: PandasInferenceFn = _default_pandas_inference_fn, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """Implementation of the ModelHandler interface for scikit-learn that @@ -236,6 +242,8 @@ def __init__( max_batch_size: the maximum batch size to use when batching inputs. This batch will be fed into the inference_fn as a Sequence of Pandas Dataframes. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -251,6 +259,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._env_vars = kwargs.get('env_vars', {}) self._large_model = large_model diff --git a/sdks/python/apache_beam/ml/inference/tensorflow_inference.py b/sdks/python/apache_beam/ml/inference/tensorflow_inference.py index 991ae971d9e6..0802868a1dd4 100644 --- a/sdks/python/apache_beam/ml/inference/tensorflow_inference.py +++ b/sdks/python/apache_beam/ml/inference/tensorflow_inference.py @@ -110,6 +110,7 @@ def __init__( inference_fn: TensorInferenceFn = default_numpy_inference_fn, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """Implementation of the ModelHandler interface for Tensorflow. @@ -154,6 +155,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._large_model = large_model def load_model(self) -> tf.Module: @@ -235,6 +238,7 @@ def __init__( inference_fn: TensorInferenceFn = default_tensor_inference_fn, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, large_model: bool = False, **kwargs): """Implementation of the ModelHandler interface for Tensorflow. @@ -258,6 +262,10 @@ def __init__( once the model is loaded. inference_fn: inference function to use during RunInference. Defaults to default_numpy_inference_fn. + min_batch_size: the minimum batch size to use when batching inputs. + max_batch_size: the maximum batch size to use when batching inputs. + max_batch_duration_secs: the maximum amount of time to buffer a batch + before emitting; used in streaming contexts. large_model: set to true if your model is large enough to run into memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should @@ -280,6 +288,8 @@ def __init__( self._batching_kwargs['min_batch_size'] = min_batch_size if max_batch_size is not None: self._batching_kwargs['max_batch_size'] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs self._large_model = large_model def load_model(self) -> tf.Module: diff --git a/sdks/python/apache_beam/ml/inference/tensorrt_inference.py b/sdks/python/apache_beam/ml/inference/tensorrt_inference.py index ff9bb78d5795..53b81c0c36c1 100644 --- a/sdks/python/apache_beam/ml/inference/tensorrt_inference.py +++ b/sdks/python/apache_beam/ml/inference/tensorrt_inference.py @@ -230,6 +230,7 @@ def __init__( *, inference_fn: TensorRTInferenceFn = _default_tensorRT_inference_fn, large_model: bool = False, + max_batch_duration_secs: Optional[int] = None, **kwargs): """Implementation of the ModelHandler interface for TensorRT. @@ -253,6 +254,8 @@ def __init__( memory pressure if you load multiple copies. Given a model that consumes N memory and a machine with W cores and M memory, you should set this to True if N*W > M. + max_batch_duration_secs: the maximum amount of time to buffer + a batch before emitting; used in streaming contexts. kwargs: Additional arguments like 'engine_path' and 'onnx_path' are currently supported. 'env_vars' can be used to set environment variables before loading the model. @@ -262,6 +265,7 @@ def __init__( """ self.min_batch_size = min_batch_size self.max_batch_size = max_batch_size + self.max_batch_duration_secs = max_batch_duration_secs self.inference_fn = inference_fn if 'engine_path' in kwargs: self.engine_path = kwargs.get('engine_path') @@ -274,7 +278,8 @@ def batch_elements_kwargs(self): """Sets min_batch_size and max_batch_size of a TensorRT engine.""" return { 'min_batch_size': self.min_batch_size, - 'max_batch_size': self.max_batch_size + 'max_batch_size': self.max_batch_size, + 'max_batch_duration_secs': self.max_batch_duration_secs } def load_model(self) -> TensorRTEngine: diff --git a/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py b/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py index 8c902421f603..95660441a848 100644 --- a/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py +++ b/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py @@ -20,6 +20,7 @@ from typing import Any from typing import Dict from typing import Iterable +from typing import Mapping from typing import Optional from typing import Sequence @@ -69,6 +70,10 @@ def __init__( experiment: Optional[str] = None, network: Optional[str] = None, private: bool = False, + *, + min_batch_size: Optional[int] = None, + max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, **kwargs): """Implementation of the ModelHandler interface for Vertex AI. **NOTE:** This API and its implementation are under development and @@ -97,9 +102,21 @@ def __init__( private: optional. if the deployed Vertex AI endpoint is private, set to true. Requires a network to be provided as well. + min_batch_size: optional. the minimum batch size to use when batching + inputs. + max_batch_size: optional. the maximum batch size to use when batching + inputs. + max_batch_duration_secs: optional. the maximum amount of time to buffer + a batch before emitting; used in streaming contexts. """ - + self._batching_kwargs = {} self._env_vars = kwargs.get('env_vars', {}) + if min_batch_size is not None: + self._batching_kwargs["min_batch_size"] = min_batch_size + if max_batch_size is not None: + self._batching_kwargs["max_batch_size"] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs if private and network is None: raise ValueError( @@ -231,3 +248,6 @@ def run_inference( def validate_inference_args(self, inference_args: Optional[Dict[str, Any]]): pass + + def batch_elements_kwargs(self) -> Mapping[str, Any]: + return self._batching_kwargs diff --git a/sdks/python/apache_beam/ml/inference/xgboost_inference.py b/sdks/python/apache_beam/ml/inference/xgboost_inference.py index 374980c56ece..ff6f098b4150 100644 --- a/sdks/python/apache_beam/ml/inference/xgboost_inference.py +++ b/sdks/python/apache_beam/ml/inference/xgboost_inference.py @@ -21,6 +21,7 @@ from typing import Callable from typing import Dict from typing import Iterable +from typing import Mapping from typing import Optional from typing import Sequence from typing import Union @@ -75,6 +76,10 @@ def __init__( Callable[..., xgboost.XGBModel]], model_state: str, inference_fn: XGBoostInferenceFn = default_xgboost_inference_fn, + *, + min_batch_size: Optional[int] = None, + max_batch_size: Optional[int] = None, + max_batch_duration_secs: Optional[int] = None, **kwargs): """Implementation of the ModelHandler interface for XGBoost. @@ -95,6 +100,12 @@ def __init__( configuration. inference_fn: the inference function to use during RunInference. default=default_xgboost_inference_fn + min_batch_size: optional. the minimum batch size to use when batching + inputs. + max_batch_size: optional. the maximum batch size to use when batching + inputs. + max_batch_duration_secs: optional. the maximum amount of time to buffer + a batch before emitting; used in streaming contexts. kwargs: 'env_vars' can be used to set environment variables before loading the model. @@ -115,6 +126,13 @@ def __init__( self._model_state = model_state self._inference_fn = inference_fn self._env_vars = kwargs.get('env_vars', {}) + self._batching_kwargs = {} + if min_batch_size is not None: + self._batching_kwargs["min_batch_size"] = min_batch_size + if max_batch_size is not None: + self._batching_kwargs["max_batch_size"] = max_batch_size + if max_batch_duration_secs is not None: + self._batching_kwargs["max_batch_duration_secs"] = max_batch_duration_secs def load_model(self) -> Union[xgboost.Booster, xgboost.XGBModel]: model = self._model_class() @@ -129,6 +147,9 @@ def load_model(self) -> Union[xgboost.Booster, xgboost.XGBModel]: def get_metrics_namespace(self) -> str: return 'BeamML_XGBoost' + def batch_elements_kwargs(self) -> Mapping[str, Any]: + return self._batching_kwargs + class XGBoostModelHandlerNumpy(XGBoostModelHandler[numpy.ndarray, PredictionResult, diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index cacfdb37d7b4..c554bef6c36d 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -70,6 +70,7 @@ from apache_beam.typehints import trivial_inference from apache_beam.typehints.decorators import get_signature from apache_beam.typehints.sharded_key_type import ShardedKeyType +from apache_beam.utils import shared from apache_beam.utils import windowed_value from apache_beam.utils.annotations import deprecated from apache_beam.utils.sharded_key import ShardedKey @@ -748,6 +749,32 @@ def flush_batch( return _StatefulBatchElementsDoFn() +class SharedKey(): + """A class that holds a per-process UUID used to key elements for streaming + BatchElements. + """ + def __init__(self): + self.key = uuid.uuid4().hex + + +def load_shared_key(): + return SharedKey() + + +class WithSharedKey(DoFn): + """A DoFn that keys elements with a per-process UUID. Used in streaming + BatchElements. + """ + def __init__(self): + self.shared_handle = shared.Shared() + + def setup(self): + self.key = self.shared_handle.acquire(load_shared_key, "WithSharedKey").key + + def process(self, element): + yield (self.key, element) + + @typehints.with_input_types(T) @typehints.with_output_types(List[T]) class BatchElements(PTransform): @@ -826,7 +853,7 @@ def expand(self, pcoll): raise NotImplementedError("Requires stateful processing (BEAM-2687)") elif self._max_batch_dur is not None: coder = coders.registry.get_coder(pcoll) - return pcoll | WithKeys(0) | ParDo( + return pcoll | ParDo(WithSharedKey()) | ParDo( _pardo_stateful_batch_elements( coder, self._batch_size_estimator, From 74308c39b8b9609ea24af1607fca2d49370da870 Mon Sep 17 00:00:00 2001 From: Lawrence Qiu Date: Wed, 6 Dec 2023 13:47:16 -0500 Subject: [PATCH 056/224] Update to Libraries-Bom 26.28.0 --- .../org/apache/beam/gradle/BeamModulePlugin.groovy | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 41aed5174c66..b5465442a1d8 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -598,16 +598,16 @@ class BeamModulePlugin implements Plugin { def dbcp2_version = "2.9.0" def errorprone_version = "2.10.0" // Try to keep gax_version consistent with gax-grpc version in google_cloud_platform_libraries_bom - def gax_version = "2.36.0" + def gax_version = "2.38.0" def google_ads_version = "26.0.0" def google_clients_version = "2.0.0" def google_cloud_bigdataoss_version = "2.2.16" // Try to keep google_cloud_spanner_version consistent with google_cloud_spanner_bom in google_cloud_platform_libraries_bom - def google_cloud_spanner_version = "6.52.1" + def google_cloud_spanner_version = "6.55.0" def google_code_gson_version = "2.10.1" def google_oauth_clients_version = "1.34.1" // Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom - def grpc_version = "1.58.0" + def grpc_version = "1.59.1" def guava_version = "32.1.2-jre" def hadoop_version = "2.10.2" def hamcrest_version = "2.1" @@ -626,7 +626,7 @@ class BeamModulePlugin implements Plugin { def postgres_version = "42.2.16" def powermock_version = "2.0.9" // Try to keep protobuf_version consistent with the protobuf version in google_cloud_platform_libraries_bom - def protobuf_version = "3.24.4" + def protobuf_version = "3.25.1" def qpid_jms_client_version = "0.61.0" def quickcheck_version = "1.0" def sbe_tool_version = "1.25.1" @@ -754,7 +754,7 @@ class BeamModulePlugin implements Plugin { // The release notes shows the versions set by the BOM: // https://github.com/googleapis/java-cloud-bom/releases/tag/v26.21.0 // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.26.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.28.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", From 6e50c1bbc881104cb03d0cdc6edf09eaba0e93ae Mon Sep 17 00:00:00 2001 From: Lawrence Qiu Date: Wed, 6 Dec 2023 13:56:48 -0500 Subject: [PATCH 057/224] Update libraries-bom java url version to 26.28.0 --- sdks/java/container/license_scripts/dep_urls_java.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index 7244bfcc6e2a..05849646aa8e 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -46,7 +46,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '26.26.0': + '26.28.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: From 33a122cd2acdb8b0ce82cff2da274b94594a21d2 Mon Sep 17 00:00:00 2001 From: Niel Markwick Date: Thu, 7 Dec 2023 19:41:08 +0100 Subject: [PATCH 058/224] Add additional log messages and javadoc around reads (#29640) --- .../apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java | 3 ++- .../apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java | 5 +++++ .../org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java | 5 +++++ .../java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 6 ++++++ 4 files changed, 18 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java index 5e5c215fdebc..1e25d1e72abb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java @@ -246,7 +246,8 @@ public void processElement(ProcessContext c) throws Exception { } } catch (SpannerException e) { serviceCallMetric.call(e.getErrorCode().getGrpcStatusCode().toString()); - LOG.error("Error while processing element", e); + LOG.error( + "Error while reading partition for operation: " + op.getReadOperation().toString(), e); throw (e); } serviceCallMetric.call("ok"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java index 45cc331657f1..2cc07b888983 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java @@ -34,6 +34,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** A naive version of Spanner read that doesn't use the Batch API. */ @VisibleForTesting @@ -44,6 +46,8 @@ abstract class NaiveSpannerRead extends PTransform, PCollection> { + private static final Logger LOG = LoggerFactory.getLogger(NaiveSpannerRead.class); + public static NaiveSpannerRead create( SpannerConfig spannerConfig, PCollectionView txView, @@ -109,6 +113,7 @@ public void processElement(ProcessContext c) throws Exception { } } catch (SpannerException e) { serviceCallMetric.call(e.getErrorCode().getGrpcStatusCode().toString()); + LOG.error("Error while reading operation: " + op.toString(), e); throw (e); } serviceCallMetric.call("ok"); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java index 959b0cb55025..4cbfe12b231f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java @@ -69,6 +69,11 @@ abstract static class Builder { abstract Builder setKeySet(KeySet keySet); + /** + * Note: {@link PartitionOptions} are currently ignored. See + * PartitionOptions in RPC documents + */ abstract Builder setPartitionOptions(PartitionOptions partitionOptions); abstract ReadOperation build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index b6ec8097a5fe..6db79ab69b47 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -852,6 +852,11 @@ public Read withIndex(String index) { return withReadOperation(getReadOperation().withIndex(index)); } + /** + * Note that {@link PartitionOptions} are currently ignored. See + * PartitionOptions in RPC documents + */ public Read withPartitionOptions(PartitionOptions partitionOptions) { return withReadOperation(getReadOperation().withPartitionOptions(partitionOptions)); } @@ -2165,6 +2170,7 @@ public void processElement(ProcessContext c) throws Exception { // fall through and retry individual mutationGroups. } else if (failureMode == FailureMode.FAIL_FAST) { mutationGroupsWriteFail.inc(mutations.size()); + LOG.error("Failed to write a batch of mutation groups", e); throw e; } else { throw new IllegalArgumentException("Unknown failure mode " + failureMode); From 51d71c0c43ea2e3076f18f511f72ca96a73489de Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Thu, 7 Dec 2023 10:53:09 -0800 Subject: [PATCH 059/224] Remove experimental note about streaming. (#29652) --- website/www/site/content/en/documentation/runners/direct.md | 4 ++++ website/www/site/content/en/documentation/sdks/python.md | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/runners/direct.md b/website/www/site/content/en/documentation/runners/direct.md index 26e23b4bd09f..85329d41d309 100644 --- a/website/www/site/content/en/documentation/runners/direct.md +++ b/website/www/site/content/en/documentation/runners/direct.md @@ -76,6 +76,10 @@ Local execution is limited by the memory available in your local environment. It ### Streaming execution +{{< paragraph class="language-py" >}} +Streaming support for Python DirectRunner is limited. For known issues, see: https://github.com/apache/beam/issues/24528. +{{< /paragraph >}} + If your pipeline uses an unbounded data source or sink, you must set the `streaming` option to `true`. ### Parallel execution diff --git a/website/www/site/content/en/documentation/sdks/python.md b/website/www/site/content/en/documentation/sdks/python.md index dc9f6a54d893..2902001066dd 100644 --- a/website/www/site/content/en/documentation/sdks/python.md +++ b/website/www/site/content/en/documentation/sdks/python.md @@ -28,7 +28,7 @@ See the [Python API reference](https://beam.apache.org/releases/pydoc/) for more ## Python streaming pipelines Python [streaming pipeline execution](/documentation/sdks/python-streaming) -is experimentally available (with some [limitations](/documentation/sdks/python-streaming/#unsupported-features)) +is available (with some [limitations](/documentation/sdks/python-streaming/#unsupported-features)) starting with Beam SDK version 2.5.0. ## Python type safety From 174e49dc6aaaa6ac1b9606ae953aceb463869b94 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Thu, 7 Dec 2023 10:55:33 -0800 Subject: [PATCH 060/224] Add cloud profiler dependency to 3.11 containers. (#29651) --- .../base_image_requirements_manual.txt | 3 +-- .../py310/base_image_requirements.txt | 24 ++++++++--------- .../py311/base_image_requirements.txt | 25 ++++++++++-------- .../py38/base_image_requirements.txt | 26 +++++++++---------- .../py39/base_image_requirements.txt | 26 +++++++++---------- 5 files changed, 53 insertions(+), 51 deletions(-) diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt index f2f3ea44b44c..dd18da1c63f5 100644 --- a/sdks/python/container/base_image_requirements_manual.txt +++ b/sdks/python/container/base_image_requirements_manual.txt @@ -32,8 +32,7 @@ cython<1 # some versions of libraries that launch Beam pipelines, like tensorflow-transform. # Leaving 'future' in our containers for now prevent breaking tft users. future -# TODO(https://github.com/GoogleCloudPlatform/cloud-profiler-python/issues/127): Unpin python version. -google-cloud-profiler;python_version<"3.11" +google-cloud-profiler guppy3 mmh3 # Optimizes execution of some Beam codepaths. TODO: Make it Beam's dependency. nltk # Commonly used for natural language processing. diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index a80e6611d9cf..cd6018bfc1fa 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -32,7 +32,7 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.6 +cryptography==41.0.7 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 @@ -41,18 +41,18 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.0 +fastavro==1.9.1 fasteners==0.19 -freezegun==1.2.2 +freezegun==1.3.1 future==0.18.3 google-api-core==2.14.0 -google-api-python-client==2.108.0 +google-api-python-client==2.109.0 google-apitools==0.5.31 -google-auth==2.23.4 +google-auth==2.25.1 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.36.4 +google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.22.0 +google-cloud-bigquery-storage==2.23.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 @@ -83,7 +83,7 @@ iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 -jsonschema-specifications==2023.11.1 +jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -108,7 +108,7 @@ pycparser==2.21 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.0 +pymongo==4.6.1 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 @@ -119,11 +119,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.0 +referencing==0.31.1 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.1 +rpds-py==0.13.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -142,6 +142,6 @@ typing_extensions==4.8.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.6.4 +websocket-client==1.7.0 wrapt==1.16.0 zstandard==0.22.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 04d9fc28c273..1fae235ee477 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -32,7 +32,7 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.6 +cryptography==41.0.7 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 @@ -40,22 +40,24 @@ dnspython==2.4.2 docker==6.1.3 docopt==0.6.2 execnet==2.0.2 -fastavro==1.9.0 +fastavro==1.9.1 fasteners==0.19 -freezegun==1.2.2 +freezegun==1.3.1 future==0.18.3 google-api-core==2.14.0 +google-api-python-client==2.109.0 google-apitools==0.5.31 -google-auth==2.23.4 +google-auth==2.25.1 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.36.4 +google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.22.0 +google-cloud-bigquery-storage==2.23.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 google-cloud-dlp==3.13.0 google-cloud-language==2.11.1 +google-cloud-profiler==4.1.0 google-cloud-pubsub==2.18.4 google-cloud-pubsublite==1.8.3 google-cloud-recommendations-ai==0.10.5 @@ -80,7 +82,7 @@ iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 -jsonschema-specifications==2023.11.1 +jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -105,7 +107,7 @@ pycparser==2.21 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.0 +pymongo==4.6.1 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 @@ -115,11 +117,11 @@ pytest-xdist==3.5.0 python-dateutil==2.8.2 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.0 +referencing==0.31.1 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.1 +rpds-py==0.13.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -135,7 +137,8 @@ threadpoolctl==3.2.0 tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.2 +uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.6.4 +websocket-client==1.7.0 wrapt==1.16.0 zstandard==0.22.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index f22bbfe7d719..ab4203ecbe37 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -33,7 +33,7 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.6 +cryptography==41.0.7 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 @@ -42,18 +42,18 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.0 +fastavro==1.9.1 fasteners==0.19 -freezegun==1.2.2 +freezegun==1.3.1 future==0.18.3 google-api-core==2.14.0 -google-api-python-client==2.108.0 +google-api-python-client==2.109.0 google-apitools==0.5.31 -google-auth==2.23.4 +google-auth==2.25.1 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.36.4 +google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.22.0 +google-cloud-bigquery-storage==2.23.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 @@ -80,13 +80,13 @@ hdfs==2.7.3 httplib2==0.22.0 hypothesis==6.91.0 idna==3.6 -importlib-metadata==6.8.0 +importlib-metadata==7.0.0 importlib-resources==6.1.1 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 -jsonschema-specifications==2023.11.1 +jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -112,7 +112,7 @@ pycparser==2.21 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.0 +pymongo==4.6.1 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 @@ -123,11 +123,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.0 +referencing==0.31.1 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.1 +rpds-py==0.13.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.10.1 @@ -146,7 +146,7 @@ typing_extensions==4.8.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.6.4 +websocket-client==1.7.0 wrapt==1.16.0 zipp==3.17.0 zstandard==0.22.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index f42b6f8c2b45..308ffa736207 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -32,7 +32,7 @@ charset-normalizer==3.3.2 click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 -cryptography==41.0.6 +cryptography==41.0.7 Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 @@ -41,18 +41,18 @@ docker==6.1.3 docopt==0.6.2 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.0 +fastavro==1.9.1 fasteners==0.19 -freezegun==1.2.2 +freezegun==1.3.1 future==0.18.3 google-api-core==2.14.0 -google-api-python-client==2.108.0 +google-api-python-client==2.109.0 google-apitools==0.5.31 -google-auth==2.23.4 +google-auth==2.25.1 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.36.4 +google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.22.0 +google-cloud-bigquery-storage==2.23.0 google-cloud-bigtable==2.21.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 @@ -79,12 +79,12 @@ hdfs==2.7.3 httplib2==0.22.0 hypothesis==6.91.0 idna==3.6 -importlib-metadata==6.8.0 +importlib-metadata==7.0.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 -jsonschema-specifications==2023.11.1 +jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -109,7 +109,7 @@ pycparser==2.21 pydot==1.4.2 PyHamcrest==2.1.0 pyjsparser==2.7.1 -pymongo==4.6.0 +pymongo==4.6.1 PyMySQL==1.1.0 pyparsing==3.1.1 pyproject_hooks==1.0.0 @@ -120,11 +120,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.0 +referencing==0.31.1 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.1 +rpds-py==0.13.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -143,7 +143,7 @@ typing_extensions==4.8.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.6.4 +websocket-client==1.7.0 wrapt==1.16.0 zipp==3.17.0 zstandard==0.22.0 From 90dd93f5241284da2e49c818af03e98b5132d30a Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 7 Dec 2023 14:12:41 -0500 Subject: [PATCH 061/224] Disable failing mypy check in yaml_io.py (#29666) * Disable failing pylint check in yaml_io.py * Correct suppression * Fine-grained disable --- sdks/python/apache_beam/yaml/yaml_io.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index bf4009719b80..b8abc47056c7 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -152,10 +152,11 @@ def _create_parser( elif format == 'avro': beam_schema = avroio.avro_schema_to_beam_schema(schema) covert_to_row = avroio.avro_dict_to_beam_row(schema, beam_schema) + # pylint: disable=line-too-long return ( beam_schema, lambda record: covert_to_row( - fastavro.schemaless_reader(io.BytesIO(record), schema))) + fastavro.schemaless_reader(io.BytesIO(record), schema))) # type: ignore[call-arg] else: raise ValueError(f'Unknown format: {format}') From 07ebdc6a6f5c66b393b25f200e67245a31b8409f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 11:15:00 -0800 Subject: [PATCH 062/224] Guard entire signature setting. --- sdks/python/apache_beam/transforms/ptransform.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index 04373fe92bf6..c99d90e184e2 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -1081,13 +1081,11 @@ def callable_ptransform_factory(*args, **kwargs): # The signature of this PTransform constructor is that of fn minus the first # argument (which is where the pvalue is passed during expand). try: - inspect.signature(fn) - except Exception: - # If we can't get a signature, don't attempt to patch it up. - pass - else: callable_ptransform_factory.__signature__ = inspect.signature( # type: ignore functools.partial(fn, None)) + except Exception: + # Sometimes we can't get the original signature. + pass return callable_ptransform_factory From b426dfaf5adf993e83234487aa2ecf0711b4350c Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 11:37:47 -0800 Subject: [PATCH 063/224] Add the ability to generate markdown documentation from a set of providers. (#29639) --- .../python/apache_beam/transforms/external.py | 35 ++-- sdks/python/apache_beam/typehints/schemas.py | 14 ++ .../apache_beam/yaml/generate_yaml_docs.py | 188 ++++++++++++++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 3 +- sdks/python/apache_beam/yaml/yaml_io.py | 2 +- sdks/python/apache_beam/yaml/yaml_mapping.py | 22 ++ sdks/python/apache_beam/yaml/yaml_provider.py | 97 +++++---- sdks/python/setup.py | 4 + 8 files changed, 305 insertions(+), 60 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/generate_yaml_docs.py mode change 100644 => 100755 sdks/python/apache_beam/yaml/yaml_provider.py diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index fc4ae3caa6df..93de059ecaaa 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -373,7 +373,7 @@ def _has_constructor(self): # Information regarding a SchemaTransform available in an external SDK. SchemaTransformsConfig = namedtuple( 'SchemaTransformsConfig', - ['identifier', 'configuration_schema', 'inputs', 'outputs']) + ['identifier', 'configuration_schema', 'inputs', 'outputs', 'description']) class SchemaAwareExternalTransform(ptransform.PTransform): @@ -444,22 +444,23 @@ def discover_iter(expansion_service, ignore_errors=True): discover_response = service.DiscoverSchemaTransform( beam_expansion_api_pb2.DiscoverSchemaTransformRequest()) - for identifier in discover_response.schema_transform_configs: - proto_config = discover_response.schema_transform_configs[identifier] - try: - schema = named_tuple_from_schema(proto_config.config_schema) - except Exception as exn: - if ignore_errors: - logging.info("Bad schema for %s: %s", identifier, str(exn)[:250]) - continue - else: - raise - - yield SchemaTransformsConfig( - identifier=identifier, - configuration_schema=schema, - inputs=proto_config.input_pcollection_names, - outputs=proto_config.output_pcollection_names) + for identifier in discover_response.schema_transform_configs: + proto_config = discover_response.schema_transform_configs[identifier] + try: + schema = named_tuple_from_schema(proto_config.config_schema) + except Exception as exn: + if ignore_errors: + logging.info("Bad schema for %s: %s", identifier, str(exn)[:250]) + continue + else: + raise + + yield SchemaTransformsConfig( + identifier=identifier, + configuration_schema=schema, + inputs=proto_config.input_pcollection_names, + outputs=proto_config.output_pcollection_names, + description=proto_config.description) @staticmethod def discover_config(expansion_service, name): diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index ea836430e8e2..6a6544aaa07c 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -245,6 +245,10 @@ def typing_to_runner_api(self, type_: type) -> schema_pb2.FieldType: if isinstance(type_, schema_pb2.Schema): return schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=type_)) + if hasattr(type_, '_beam_schema_proto') and type_._beam_schema_proto.obj: + return schema_pb2.FieldType( + row_type=schema_pb2.RowType(schema=type_._beam_schema_proto.obj)) + if isinstance(type_, row_type.RowTypeConstraint): if type_.schema_id is None: schema_id = SCHEMA_REGISTRY.generate_new_id() @@ -557,6 +561,7 @@ def named_tuple_from_schema(self, schema: schema_pb2.Schema) -> type: '__reduce__', _named_tuple_reduce_method(schema.SerializeToString())) setattr(user_type, row_type._BEAM_SCHEMA_ID, schema.id) + user_type._beam_schema_proto = _Ephemeral(schema) self.schema_registry.add(user_type, schema) coders.registry.register_coder(user_type, coders.RowCoder) @@ -627,6 +632,15 @@ def union_schema_type(element_types): return named_tuple_from_schema(named_fields_to_schema(union_fields_and_types)) +class _Ephemeral: + """Helper class for wrapping unpicklable objects.""" + def __init__(self, obj): + self.obj = obj + + def __reduce__(self): + return _Ephemeral, (None, ) + + # Registry of typings for a schema by UUID class LogicalTypeRegistry(object): def __init__(self): diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py new file mode 100644 index 000000000000..d3d6c0a3e316 --- /dev/null +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -0,0 +1,188 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 argparse +import re + +import yaml + +from apache_beam.yaml import yaml_provider +from apache_beam.portability.api import schema_pb2 +from apache_beam.typehints import native_type_compatibility +from apache_beam.typehints.schemas import typing_from_runner_api +from apache_beam.utils import subprocess_server + + +def _fake_value(name, beam_type): + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + if beam_type.atomic_type == schema_pb2.STRING: + return f'"{name}"' + elif beam_type.atomic_type == schema_pb2.BOOLEAN: + return "true|false" + else: + return name + elif type_info == "array_type": + return [_fake_value(name, beam_type.array_type.element_type), '...'] + elif type_info == "iterable_type": + return [_fake_value(name, beam_type.iterable_type.element_type), '...'] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type == schema_pb2.STRING: + return { + 'a': _fake_value(name + '_value_a', beam_type.map_type.value_type), + 'b': _fake_value(name + '_value_b', beam_type.map_type.value_type), + 'c': '...', + } + else: + return { + _fake_value(name + '_key', beam_type.map_type.key_type): _fake_value( + name + '_value', beam_type.map_type.value_type) + } + elif type_info == "row_type": + return _fake_row(beam_type.row_type.schema) + elif type_info == "logical_type": + return name + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def _fake_row(schema): + if schema is None: + return '...' + return {f.name: _fake_value(f.name, f.type) for f in schema.fields} + + +def pretty_example(provider, t): + spec = {'type': t} + try: + requires_inputs = provider.requires_inputs(t, {}) + except Exception: + requires_inputs = False + if requires_inputs: + spec['input'] = '...' + config_schema = provider.config_schema(t) + if config_schema is None or config_schema.fields: + spec['config'] = _fake_row(config_schema) + s = yaml.dump(spec, sort_keys=False) + return s.replace("'", "") + + +def config_docs(schema): + if schema is None: + return '' + elif not schema.fields: + return 'No configuration parameters.' + + def pretty_type(beam_type): + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return schema_pb2.AtomicType.Name(beam_type.atomic_type).lower() + elif type_info == "array_type": + return f'Array[{pretty_type(beam_type.array_type.element_type)}]' + elif type_info == "iterable_type": + return f'Iterable[{pretty_type(beam_type.iterable_type.element_type)}]' + elif type_info == "map_type": + return f'Map[{pretty_type(beam_type.map_type.key_type)}, {pretty_type(beam_type.map_type.value_type)}]' + elif type_info == "row_type": + return 'Row' + else: + return '?' + + def maybe_row_parameters(t): + if t.WhichOneof("type_info") == "row_type": + return indent('\n\nRow fields:\n\n' + config_docs(t.row_type.schema), 4) + else: + return '' + + def maybe_optional(t): + return " (Optional)" if t.nullable else "" + + def lines(): + for f in schema.fields: + yield ''.join([ + f'**{f.name}** `{pretty_type(f.type)}`', + maybe_optional(f.type), + ': ' + f.description if f.description else '', + maybe_row_parameters(f.type), + ]) + + return '\n\n'.join('*' + indent(line, 2) for line in lines()).strip() + + +def indent(lines, size): + return '\n'.join(' ' * size + line for line in lines.split('\n')) + + +def longest(func, xs): + return max([func(x) or '' for x in xs], key=len) + + +def io_grouping_key(transform_name): + """Place reads and writes next to each other, after all other transforms.""" + if transform_name.startswith('ReadFrom'): + return 1, transform_name[8:], 0 + elif transform_name.startswith('WriteTo'): + return 1, transform_name[7:], 1 + else: + return 0, transform_name + + +SKIP = [ + 'Combine', + 'Filter', + 'MapToFields', +] + + +def transform_docs(t, providers): + return '\n'.join([ + f'## {t}', + '', + longest(lambda p: p.description(t), providers), + '', + '### Configuration', + '', + longest(lambda p: config_docs(p.config_schema(t)), providers), + '', + '### Usage', + '', + indent(longest(lambda p: pretty_example(p, t), providers), 4), + ]) + + +def main(): + parser = argparse.ArgumentParser() + parser.add_argument('output_file') + parser.add_argument('--include', default='.*') + parser.add_argument( + '--exclude', default='(Combine)|(Filter)|(MapToFields)-.*') + options = parser.parse_args() + include = re.compile(options.include).match + exclude = re.compile(options.exclude).match + + with subprocess_server.SubprocessServer.cache_subprocesses(): + with open(options.output_file, 'w') as fout: + providers = yaml_provider.standard_providers() + for transform in sorted(providers.keys(), key=io_grouping_key): + if include(transform) and not exclude(transform): + print(transform) + fout.write(transform_docs(transform, providers[transform])) + fout.write('\n\n') + + +if __name__ == '__main__': + main() diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 9c430ae2d2d3..506ea6634aa3 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -68,8 +68,7 @@ 'format': 'format' 'topic': 'topic' 'bootstrap_servers': 'bootstrapServers' - 'producer_config_updates': 'ProducerConfigUpdates' - 'error_handling': 'errorHandling' + 'producer_config_updates': 'producerConfigUpdates' 'file_descriptor_path': 'fileDescriptorPath' 'message_name': 'messageName' underlying_provider: diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index b8abc47056c7..7db195ea84a2 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -304,7 +304,7 @@ def write_to_pubsub( attributes_map: Optional[str] = None, id_attribute: Optional[str] = None, timestamp_attribute: Optional[str] = None): - """Writes messages from Cloud Pub/Sub. + """Writes messages to Cloud Pub/Sub. Args: topic: Cloud Pub/Sub topic in the form "/topics//". diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 42af11ae2456..08c7a59819a7 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -16,6 +16,8 @@ # """This module defines the basic MapToFields operation.""" +import functools +import inspect import itertools from collections import abc from typing import Any @@ -23,6 +25,7 @@ from typing import Collection from typing import Dict from typing import Mapping +from typing import NamedTuple from typing import Optional from typing import Union @@ -267,6 +270,11 @@ def checking_func(row): return func +class ErrorHandlingConfig(NamedTuple): + output: str + # TODO: Other parameters are valid here too, but not common to Java. + + def exception_handling_args(error_handling_spec): if error_handling_spec: return { @@ -294,12 +302,26 @@ def expand(self, pcoll): def maybe_with_exception_handling_transform_fn(transform_fn): + @functools.wraps(transform_fn) def expand(pcoll, error_handling=None, **kwargs): wrapped_pcoll = beam.core._MaybePValueWithErrors( pcoll, exception_handling_args(error_handling)) return transform_fn(wrapped_pcoll, **kwargs).as_result(_map_errors_to_standard_format()) + original_signature = inspect.signature(transform_fn) + new_parameters = list(original_signature.parameters.values()) + error_handling_param = inspect.Parameter( + 'error_handling', + inspect.Parameter.KEYWORD_ONLY, + default=None, + annotation=ErrorHandlingConfig) + if new_parameters[-1].kind == inspect.Parameter.VAR_KEYWORD: + new_parameters.insert(-1, error_handling_param) + else: + new_parameters.append(error_handling_param) + expand.__signature__ = original_signature.replace(parameters=new_parameters) + return expand diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py old mode 100644 new mode 100755 index 25cc366ec2c7..5ba15ff17e8b --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -25,6 +25,7 @@ import json import logging import os +import re import subprocess import sys import urllib.parse @@ -35,6 +36,7 @@ from typing import Mapping from typing import Optional +import docstring_parser import yaml from yaml.loader import SafeLoader @@ -73,6 +75,9 @@ def provided_transforms(self) -> Iterable[str]: def config_schema(self, type): return None + def description(self, type): + return None + def requires_inputs(self, typ: str, args: Mapping[str, Any]) -> bool: """Returns whether this transform requires inputs. @@ -167,6 +172,10 @@ def config_schema(self, type): return named_tuple_to_schema( self.schema_transforms()[self._urns[type]].configuration_schema) + def description(self, type): + if self._urns[type] in self.schema_transforms(): + return self.schema_transforms()[self._urns[type]].description + def requires_inputs(self, typ, args): if self._urns[typ] in self.schema_transforms(): return bool(self.schema_transforms()[self._urns[typ]].inputs) @@ -412,7 +421,8 @@ def config_schema(self, typ): if isinstance(factory, type) and issubclass(factory, beam.PTransform): # https://bugs.python.org/issue40897 params = dict(inspect.signature(factory.__init__).parameters) - del params['self'] + if 'self' in params: + del params['self'] else: params = inspect.signature(factory).parameters @@ -423,15 +433,37 @@ def type_of(p): else: return t + docs = { + param.arg_name: param.description + for param in self.get_docs(typ).params + } + names_and_types = [ (name, typing_to_runner_api(type_of(p))) for name, p in params.items() ] return schema_pb2.Schema( fields=[ - schema_pb2.Field(name=name, type=type) for name, - type in names_and_types + schema_pb2.Field(name=name, type=type, description=docs.get(name)) + for (name, type) in names_and_types ]) + def description(self, typ): + def empty_if_none(s): + return s or '' + + docs = self.get_docs(typ) + return ( + empty_if_none(docs.short_description) + '\n\n' + + empty_if_none(docs.long_description)).strip() or None + + def get_docs(self, typ): + docstring = self._transform_factories[typ].__doc__ or '' + # These "extra" docstring parameters are not relevant for YAML and mess + # up the parsing. + docstring = re.sub('Pandas Parameters\s+-----.*', '', docstring, flags=re.S) + return docstring_parser.parse( + docstring, docstring_parser.DocstringStyle.GOOGLE) + def create_transform(self, type, args, yaml_create_transform): return self._transform_factories[type](**args) @@ -727,14 +759,32 @@ def config_schema(self, type): self._transforms[type]) if underlying_schema is None: return None - underlying_schema_types = {f.name: f.type for f in underlying_schema.fields} + defaults = self._defaults.get(type, {}) + underlying_schema_fields = {f.name: f for f in underlying_schema.fields} + missing = set(self._mappings[type].values()) - set( + underlying_schema_fields.keys()) + if missing: + raise ValueError( + f"Mapping destinations {missing} for {type} are not in the " + f"underlying config schema {list(underlying_schema_fields.keys())}") + + def with_name( + original: schema_pb2.Field, new_name: str) -> schema_pb2.Field: + result = schema_pb2.Field() + result.CopyFrom(original) + result.name = new_name + return result + return schema_pb2.Schema( fields=[ - schema_pb2.Field(name=src, type=underlying_schema_types[dest]) - for src, - dest in self._mappings[type].items() + with_name(underlying_schema_fields[dest], src) + for (src, dest) in self._mappings[type].items() + if dest not in defaults ]) + def description(self, typ): + return self._underlying_provider.description(typ) + def requires_inputs(self, typ, args): return self._underlying_provider.requires_inputs(typ, args) @@ -810,36 +860,3 @@ def standard_providers(): create_combine_providers(), io_providers(), parse_providers(standard_providers)) - - -def list_providers(): - def pretty_type(field_type): - if field_type.WhichOneof('type_info') == 'row_type': - return pretty_schema(field_type.row_type.schema) - else: - t = typing_from_runner_api(field_type) - optional_base = native_type_compatibility.extract_optional_type(t) - if optional_base: - t = optional_base - suffix = '?' - else: - suffix = '' - s = str(t) - if s.startswith('=1.5.2,<2.0', + 'docstring-parser>=0.15,<1.0', # Pinning docutils as a workaround for Sphinx issue: # https://github.com/sphinx-doc/sphinx/issues/9727 'docutils==0.17.1', 'pandas<2.0.0', ], 'test': [ + 'docstring-parser>=0.15,<1.0', 'freezegun>=0.3.12', 'joblib>=1.0.1', 'mock>=1.0.1,<6.0.0', @@ -356,6 +358,7 @@ def get_portability_package_data(): 'testcontainers[mysql]>=3.0.3,<4.0.0', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<=7.0.0', + 'pyyaml>=3.12,<7.0.0', ], 'gcp': [ 'cachetools>=3.1.0,<6', @@ -418,6 +421,7 @@ def get_portability_package_data(): 'distributed >= 2022.6', ], 'yaml': [ + 'docstring-parser>=0.15,<1.0', 'pyyaml>=3.12,<7.0.0', ] + dataframe_dependency }, From 5becfb8ed430fe9a317cd2ffded576fe2ab8e980 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 7 Dec 2023 14:42:31 -0500 Subject: [PATCH 064/224] Archive JUnit test result on timeout (#29667) --- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 +- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .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_IO_Performance_Tests.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_Direct_Java21.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_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 +- .../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_SQL.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 +- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Spark3.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_Java21.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_Google-ads_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 +- .../workflows/beam_PreCommit_Java_RequestResponse_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_SQL.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .github/workflows/beam_PreCommit_SQL_Java17.yml | 2 +- 102 files changed, 102 insertions(+), 102 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index ffb38e34a454..56ba4083e1dc 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -116,7 +116,7 @@ jobs: '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_4 }}' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 8727b2387403..9cbfadbbb6d5 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -92,7 +92,7 @@ jobs: -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Avro_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index a231d00f5ede..4bba1eaeba7e 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -92,7 +92,7 @@ jobs: -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Json_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 7259fb1838f2..5b368727ed52 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -92,7 +92,7 @@ jobs: -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Streaming_Java_test_arguments_1 }}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index c02d16688a69..3e28f7b4d59f 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -91,7 +91,7 @@ jobs: '-DintegrationTestPipelineOptions=[${{env.beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1}}]' \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index c19651301c0f..4417d3a21746 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :javaPostCommit - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 9217df5b9ec1..e20218b865ae 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :javaAvroVersionsTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 065884eea252..ce930ba7a438 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -87,7 +87,7 @@ jobs: gradle-command: :sdks:java:io:google-cloud-platform:bigQueryEarlyRolloutIntegrationTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index c3883e862bc6..78dcb37c8934 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -84,7 +84,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:postCommit - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 4ee629853326..22fbd5d6fd46 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:postCommitRunnerV2 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 6f508f257a7e..426291fac138 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -81,7 +81,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 4c0d2295c60b..a8e24f33adfb 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -114,7 +114,7 @@ jobs: -PskipCheckerFramework \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index dfc909a4c6d5..decca84683ec 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -87,7 +87,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 51ef625ae7de..9733b6d36434 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -81,7 +81,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" 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 621b35d57935..060fe4396d85 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -95,7 +95,7 @@ jobs: -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index e3022c96ea1a..cec2acab0f95 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:direct:examplesIntegrationTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index 12e76f334710..a9f5d99bc2c6 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -83,7 +83,7 @@ jobs: gradle-command: :runners:flink:1.15:examplesIntegrationTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 5ff6a1b52485..da58dc67c9c1 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:spark:3:examplesIntegrationTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index c6a31be712a8..f2489137fd64 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :javaHadoopVersionsTest - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 671b93e0dcff..a340f028fbfe 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -108,7 +108,7 @@ jobs: exportTable: io_performance_metrics_test - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 988566dff67d..cb39dd2f26f1 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -81,7 +81,7 @@ jobs: arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 22416c9a7aa6..f33ba9c39b74 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -86,7 +86,7 @@ jobs: -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index bac658ceea24..f7b0f22cbfa5 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -81,7 +81,7 @@ jobs: arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 73727c04e212..0ae09bcbefc4 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -86,7 +86,7 @@ jobs: -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index c2c1b44550fe..e6740017afa2 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -87,7 +87,7 @@ jobs: -Pjava21Home=$JAVA_HOME_21_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 81bc4b6a6100..b6a4e1c84174 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -81,7 +81,7 @@ jobs: arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 7405093a6ac0..6e093ead474c 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -81,7 +81,7 @@ jobs: arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index f1b5e6cea23e..76e2b702ba66 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -80,7 +80,7 @@ jobs: gradle-command: runners:flink:1.15:job-server:validatesPortableRunnerStreaming - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index f437efdad407..604a37779d17 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:samza:job-server:validatesPortableRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 3c6ea80f48d1..201016c5602e 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:spark:3:job-server:validatesPortableRunnerStreaming - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 71fa65b31608..d8a0672e03b5 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -83,7 +83,7 @@ jobs: :runners:spark:3:job-server:validatesPortableRunnerDocker \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 5d7c2d393911..b2b3440e23b3 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :javaPostCommitSickbay - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 4a494bf55899..e01ea98912d0 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -83,7 +83,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index b04ac0857390..a0009b76f699 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -101,7 +101,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 7a53103809b5..cc5f86b0b5a9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -83,7 +83,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index d02cf0e57944..8b9d456ed828 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -83,7 +83,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" 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 ef292ecdc0cd..6bca7477cf52 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -83,7 +83,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 7cd0138e9b53..90f47d5451f4 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -80,7 +80,7 @@ jobs: run: ./gradlew :runners:direct-java:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 5d81527b0fed..9f7897ffd84c 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -96,7 +96,7 @@ jobs: -Dorg.gradle.java.home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 25606da982aa..1f265ea5dc16 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -81,7 +81,7 @@ jobs: gradle-command: :runners:flink:1.15:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index cafd4da44ece..c7b9219f6ea9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -99,7 +99,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 29a7c8caaf82..c4583f2f72d4 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:samza:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index e0ac939e65cc..75a01cd8d5bb 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:spark:3:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index a37aed2eba61..844082ef9460 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:spark:3:validatesStructuredStreamingRunnerBatch - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index c9f66ac46ee8..d88ebbe28a85 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -99,7 +99,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 6e32c42041ee..2206e1767fc2 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:twister2:validatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 31f0b5d9be90..a8b3678c6321 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -79,7 +79,7 @@ jobs: run: ./gradlew :runners:portability:java:ulrLoopbackValidatesRunner - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 077ec9d9ed0c..2b7ce87eeab3 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sqlPostCommit - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 51a5acbc3603..4b6fe15ce199 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -99,7 +99,7 @@ jobs: -PskipNonPythonTask=false \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 6cf0e4077505..96ce79b25087 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -100,7 +100,7 @@ jobs: -PskipNonPythonTask=false \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 4325df19d632..9a0ce786cbb0 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -88,7 +88,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerGoUsingJava - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index a5edf5a9131f..f7f3c73373a3 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -87,7 +87,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index f9b9bd387688..75fbb8eaffa4 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -99,7 +99,7 @@ jobs: -PskipNonPythonTask=false \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 6b90316e110d..a685e57bb592 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -99,7 +99,7 @@ jobs: -PskipNonPythonTask=false \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index 92e99cf40113..0d8bb00160d7 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -91,7 +91,7 @@ jobs: run: ./gradlew -p it build - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 3254ac2dc97b..6f4669e0c784 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -186,7 +186,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" 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 77c5ddb42089..5e0bc8631fd9 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 @@ -119,7 +119,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" 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 eb9e10c5b6ba..97192a26f5fa 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 @@ -119,7 +119,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 9d6a2c9cb690..241cf89380a8 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 865015bef141..4cdcfed4dadc 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -112,7 +112,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index 02a2be854e1a..dd15d18f26d1 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index d54e2387983c..ffa8c9187a68 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -98,7 +98,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 5915fd585bde..209d30231805 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index d44957bb9edc..5cd015ac9832 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index 5eabeffd52b7..a2cf6c1b0645 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -103,7 +103,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index becf5ac0477d..f7a3ca8df639 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -107,7 +107,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 5639a2002d7f..ee372e84d749 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -114,7 +114,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index cb1bcdb5e491..c411594c5c35 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -122,7 +122,7 @@ jobs: max-workers: 12 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" 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 893f0197b5bb..cd0c92367902 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 @@ -95,7 +95,7 @@ jobs: -Dfile.encoding=UTF-8 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index 97c99d57d350..303a84c2df34 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -94,7 +94,7 @@ jobs: -PdisableSpotlessCheck=true -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 3f1219de012b..c7a5a84af224 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -116,7 +116,7 @@ jobs: -PenableJacocoReport \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml index 9322565ca1d4..2271c1cf8d37 100644 --- a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -92,7 +92,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 5ebac3e1fc01..beb7acc89f6a 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -96,7 +96,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 499576966272..c1e627025c0a 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -96,7 +96,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index caa3b75099d2..5b8202d96217 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -134,7 +134,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 8dea3a3c0b81..7763ab690449 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -95,7 +95,7 @@ jobs: -Dfile.encoding=UTF-8 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 8e06a57ceba6..1d66fd56539f 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 64afee7bff3f..60390f31223f 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -101,7 +101,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 3e101fab78a5..431b5e0c6566 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -101,7 +101,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 1207618f3424..028ba721a4f5 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -103,7 +103,7 @@ jobs: --no-parallel \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 234c31914a94..e0598d0beebe 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -126,7 +126,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 0df56a76d509..47c360a62efd 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 0799ac469353..4b9895f9cd93 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 0ee6810f8188..934a428fa96d 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 9253d588a6dd..48a7da5f82a4 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -103,7 +103,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index a0dea4b64712..966f51f1a4a3 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -99,7 +99,7 @@ jobs: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH }} - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index b63d29953d85..2997e20357f5 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -104,7 +104,7 @@ jobs: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 8621d0c8f42d..6f167bc89680 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index 042f436bfbaa..f10ae32d36ee 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -112,7 +112,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index bfa89b57871d..4be46123ebaa 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 23546c77ac24..6429cc9d7c9e 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index d2035836ccf0..d297059bc71f 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -92,7 +92,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 087340ce9530..5b026bbe9c54 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -96,7 +96,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 8706aa96335d..4668b3536683 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -105,7 +105,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index dbcce6c30b83..a656ce6b1263 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index ad4a09dd165b..695800453d00 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -103,7 +103,7 @@ jobs: arguments: -PdisableSpotlessCheck=true - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 821e42bc26fb..51408f545677 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 49e2d651e50e..4183ea0f01a5 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 5e3f81323502..7f402097c5a8 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index 0a3e95c675c5..c9b53244e876 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -92,7 +92,7 @@ jobs: -PenableJacocoReport \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 73e78332356c..fea4f759f74f 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -101,7 +101,7 @@ jobs: -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index fa1c0ea7089e..fbbda1ac7766 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -99,7 +99,7 @@ jobs: -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 - if: failure() + if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" From 318227345264e70ed2241538b8ab42a64af5a282 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 13:00:33 -0800 Subject: [PATCH 065/224] Add a pipeline update compatibility version option. (#29140) This can be used to migrate to best practices and good default with new versions of Beam while still allowing users of older SDKs to update their SDK version without breaking update compatibility. Also add the mechanisms to propagate this option for cross-language transforms. --- .../v1/beam_expansion_api.proto | 5 +++ .../runners/core/construction/External.java | 1 + .../beam/sdk/options/StreamingOptions.java | 11 +++++ .../expansion/service/ExpansionService.java | 15 ++++++- .../service/ExpansionServerTest.java | 15 +++++-- .../apache_beam/options/pipeline_options.py | 43 +++++++++++++++++++ sdks/python/apache_beam/pipeline_test.py | 8 +++- .../runners/portability/expansion_service.py | 15 ++++++- .../python/apache_beam/transforms/external.py | 3 +- 9 files changed, 107 insertions(+), 9 deletions(-) diff --git a/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto b/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto index e70d978c56ac..a4736f8b4938 100644 --- a/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto +++ b/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto @@ -29,6 +29,7 @@ option go_package = "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagem option java_package = "org.apache.beam.model.expansion.v1"; option java_outer_classname = "ExpansionApi"; +import "google/protobuf/struct.proto"; import "org/apache/beam/model/pipeline/v1/beam_runner_api.proto"; import "org/apache/beam/model/pipeline/v1/schema.proto"; @@ -57,6 +58,10 @@ message ExpansionRequest { // A set of requirements that must be used by the expansion service to // interpret the components provided with this request. repeated string requirements = 5; + + // (Optional) A set of Pipeline Options that should be used + // when expanding this transform. + google.protobuf.Struct pipeline_options = 6; } message ExpansionResponse { 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 93a1ade474a5..3951148d1a58 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 @@ -275,6 +275,7 @@ public OutputT expand(InputT input) { .setTransform(ptransformBuilder.build()) .setNamespace(getNamespace()) .build(); + requestBuilder.setPipelineOptions(PipelineOptionsTranslation.toProto(p.getOptions())); ExpansionApi.ExpansionResponse response = clientFactory.getExpansionServiceClient(endpoint).expand(request); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java index 3fc3c7d4c471..e389ab89cf9c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.options; +import org.checkerframework.checker.nullness.qual.Nullable; + /** Options used to configure streaming. */ public interface StreamingOptions extends ApplicationNameOptions, PipelineOptions { /** @@ -30,4 +32,13 @@ public interface StreamingOptions extends ApplicationNameOptions, PipelineOption boolean isStreaming(); void setStreaming(boolean value); + + @Description( + "If set, attempts to produce a pipeline compatible with this prior version of the Beam SDK." + + " This string should be interpreted and compared per https://semver.org/." + + " See, for example, https://cloud.google.com/dataflow/docs/guides/updating-a-pipeline.") + @Nullable + String getUpdateCompatibilityVersion(); + + void setUpdateCompatibilityVersion(@Nullable String updateCompatibilityVersion); } 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 ba60fb99d4d1..43690c603701 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 @@ -50,6 +50,7 @@ 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.PipelineOptionsTranslation; import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.runners.core.construction.SdkComponents; @@ -67,6 +68,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; @@ -584,7 +586,8 @@ private Map loadRegisteredTransforms() { request.getTransform().getSpec().getUrn()); LOG.debug("Full transform: {}", request.getTransform()); Set existingTransformIds = request.getComponents().getTransformsMap().keySet(); - Pipeline pipeline = createPipeline(); + Pipeline pipeline = + createPipeline(PipelineOptionsTranslation.fromProto(request.getPipelineOptions())); boolean isUseDeprecatedRead = ExperimentalOptions.hasExperiment(pipelineOptions, "use_deprecated_read") || ExperimentalOptions.hasExperiment( @@ -707,7 +710,7 @@ private Map loadRegisteredTransforms() { .build(); } - protected Pipeline createPipeline() { + protected Pipeline createPipeline(PipelineOptions requestOptions) { // TODO: [https://github.com/apache/beam/issues/21064]: implement proper validation PipelineOptions effectiveOpts = PipelineOptionsFactory.create(); PortablePipelineOptions portableOptions = effectiveOpts.as(PortablePipelineOptions.class); @@ -728,6 +731,14 @@ protected Pipeline createPipeline() { .as(ExpansionServiceOptions.class) .setExpansionServiceConfig( pipelineOptions.as(ExpansionServiceOptions.class).getExpansionServiceConfig()); + // TODO(https://github.com/apache/beam/issues/20090): Figure out the correct subset of options + // to propagate. + if (requestOptions.as(StreamingOptions.class).getUpdateCompatibilityVersion() != null) { + effectiveOpts + .as(StreamingOptions.class) + .setUpdateCompatibilityVersion( + requestOptions.as(StreamingOptions.class).getUpdateCompatibilityVersion()); + } return Pipeline.create(effectiveOpts); } diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServerTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServerTest.java index 03938a073c4f..5d0273964db4 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServerTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServerTest.java @@ -23,6 +23,7 @@ import static org.hamcrest.core.Is.is; import java.util.Arrays; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.junit.Test; @@ -58,7 +59,7 @@ public void testPassingPipelineArguments() { ExpansionService service = new ExpansionService(args); assertThat( service - .createPipeline() + .createPipeline(PipelineOptionsFactory.create()) .getOptions() .as(PortablePipelineOptions.class) .getDefaultEnvironmentType(), @@ -70,7 +71,11 @@ public void testNonEmptyFilesToStage() { String[] args = {"--filesToStage=nonExistent1.jar,nonExistent2.jar"}; ExpansionService service = new ExpansionService(args); assertThat( - service.createPipeline().getOptions().as(PortablePipelineOptions.class).getFilesToStage(), + service + .createPipeline(PipelineOptionsFactory.create()) + .getOptions() + .as(PortablePipelineOptions.class) + .getFilesToStage(), equalTo(Arrays.asList("nonExistent1.jar", "nonExistent2.jar"))); } @@ -79,7 +84,11 @@ public void testEmptyFilesToStageIsOK() { String[] args = {"--filesToStage="}; ExpansionService service = new ExpansionService(args); assertThat( - service.createPipeline().getOptions().as(PortablePipelineOptions.class).getFilesToStage(), + service + .createPipeline(PipelineOptionsFactory.create()) + .getOptions() + .as(PortablePipelineOptions.class) + .getFilesToStage(), equalTo(Arrays.asList(""))); } } diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 2dba78948444..b02aa7da7de1 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -36,6 +36,7 @@ from apache_beam.options.value_provider import StaticValueProvider from apache_beam.options.value_provider import ValueProvider from apache_beam.transforms.display import HasDisplayData +from apache_beam.utils import proto_utils __all__ = [ 'PipelineOptions', @@ -390,6 +391,36 @@ def get_all_options( return result + def to_runner_api(self): + def to_struct_value(o): + if isinstance(o, (bool, int, str)): + return o + elif isinstance(o, (tuple, list)): + return [to_struct_value(e) for e in o] + elif isinstance(o, dict): + return {str(k): to_struct_value(v) for k, v in o.items()} + else: + return str(o) # Best effort. + + return proto_utils.pack_Struct( + **{ + f'beam:option:{k}:v1': to_struct_value(v) + for (k, v) in self.get_all_options( + drop_default=True, retain_unknown_options=True).items() + if v is not None + }) + + @classmethod + def from_runner_api(cls, proto_options): + def from_urn(key): + assert key.startswith('beam:option:') + assert key.endswith(':v1') + return key[12:-3] + + return cls( + **{from_urn(key): value + for (key, value) in proto_options.items()}) + def display_data(self): return self.get_all_options(drop_default=True, retain_unknown_options=True) @@ -528,6 +559,18 @@ def _add_argparse_args(cls, parser): 'exception if a transform is created with a non-unique label.') +class StreamingOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--update_compatibility_version', + default=None, + help='Attempt to produce a pipeline compatible with the given prior ' + 'version of the Beam SDK. ' + 'See for example, https://cloud.google.com/dataflow/docs/guides/' + 'updating-a-pipeline') + + class CrossLanguageOptions(PipelineOptions): @classmethod def _add_argparse_args(cls, parser): diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index 113d1a99990c..38d8b1f03a11 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -872,7 +872,9 @@ def test_dir(self): 'slices', 'style', 'view_as', - 'display_data' + 'display_data', + 'from_runner_api', + 'to_runner_api', }, { attr @@ -884,7 +886,9 @@ def test_dir(self): 'get_all_options', 'style', 'view_as', - 'display_data' + 'display_data', + 'from_runner_api', + 'to_runner_api', }, { attr diff --git a/sdks/python/apache_beam/runners/portability/expansion_service.py b/sdks/python/apache_beam/runners/portability/expansion_service.py index 50c793a0e8bd..c7728098f30c 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service.py @@ -19,9 +19,11 @@ """ # pytype: skip-file +import copy import traceback from apache_beam import pipeline as beam_pipeline +from apache_beam.options import pipeline_options from apache_beam.portability import common_urns from apache_beam.portability import python_urns from apache_beam.portability.api import beam_expansion_api_pb2 @@ -49,7 +51,18 @@ def __init__(self, options=None, loopback_address=None): def Expand(self, request, context=None): try: - pipeline = beam_pipeline.Pipeline(options=self._options) + options = copy.deepcopy(self._options) + request_options = pipeline_options.PipelineOptions.from_runner_api( + request.pipeline_options) + # TODO(https://github.com/apache/beam/issues/20090): Figure out the + # correct subset of options to apply to expansion. + if request_options.view_as( + pipeline_options.StreamingOptions).update_compatibility_version: + options.view_as( + pipeline_options.StreamingOptions + ).update_compatibility_version = request_options.view_as( + pipeline_options.StreamingOptions).update_compatibility_version + pipeline = beam_pipeline.Pipeline(options=options) def with_pipeline(component, pcoll_id=None): component.pipeline = pipeline diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 93de059ecaaa..a69ecbaee220 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -723,7 +723,8 @@ def expand(self, pvalueish): components=components, namespace=self._external_namespace, transform=transform_proto, - output_coder_requests=output_coders) + output_coder_requests=output_coders, + pipeline_options=pipeline._options.to_runner_api()) expansion_service = _maybe_use_transform_service( self._expansion_service, pipeline.options) From f1a7f71266cfb53c94eb6ed982c1b1bb137b6ffb Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Thu, 7 Dec 2023 15:28:57 -0800 Subject: [PATCH 066/224] Bump the image used by Dataflow runner in dev SDKs. (#29672) --- 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 0dfe0182eebc..82e2306ced3c 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-20231205' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231207' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From ed3a58217f42f59cfe90592bf08e545ca9b5c3cf Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 16:39:42 -0800 Subject: [PATCH 067/224] Fix some lint errors in Python. (#29675) --- sdks/python/apache_beam/yaml/generate_yaml_docs.py | 8 ++++---- sdks/python/apache_beam/yaml/yaml_provider.py | 5 ++--- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py index d3d6c0a3e316..1ed629310c1f 100644 --- a/sdks/python/apache_beam/yaml/generate_yaml_docs.py +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -20,11 +20,9 @@ import yaml -from apache_beam.yaml import yaml_provider from apache_beam.portability.api import schema_pb2 -from apache_beam.typehints import native_type_compatibility -from apache_beam.typehints.schemas import typing_from_runner_api from apache_beam.utils import subprocess_server +from apache_beam.yaml import yaml_provider def _fake_value(name, beam_type): @@ -96,7 +94,9 @@ def pretty_type(beam_type): elif type_info == "iterable_type": return f'Iterable[{pretty_type(beam_type.iterable_type.element_type)}]' elif type_info == "map_type": - return f'Map[{pretty_type(beam_type.map_type.key_type)}, {pretty_type(beam_type.map_type.value_type)}]' + return ( + f'Map[{pretty_type(beam_type.map_type.key_type)}, ' + f'{pretty_type(beam_type.map_type.value_type)}]') elif type_info == "row_type": return 'Row' else: diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 5ba15ff17e8b..6a2d313183e5 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -48,11 +48,9 @@ from apache_beam.transforms import external from apache_beam.transforms import window from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform -from apache_beam.typehints import native_type_compatibility from apache_beam.typehints import schemas from apache_beam.typehints import trivial_inference from apache_beam.typehints.schemas import named_tuple_to_schema -from apache_beam.typehints.schemas import typing_from_runner_api from apache_beam.typehints.schemas import typing_to_runner_api from apache_beam.utils import python_callable from apache_beam.utils import subprocess_server @@ -460,7 +458,8 @@ def get_docs(self, typ): docstring = self._transform_factories[typ].__doc__ or '' # These "extra" docstring parameters are not relevant for YAML and mess # up the parsing. - docstring = re.sub('Pandas Parameters\s+-----.*', '', docstring, flags=re.S) + docstring = re.sub( + r'Pandas Parameters\s+-----.*', '', docstring, flags=re.S) return docstring_parser.parse( docstring, docstring_parser.DocstringStyle.GOOGLE) From 85bf38889599ffabdb6a42dd0ab345b6105fca8f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 17:31:04 -0800 Subject: [PATCH 068/224] More efficient map side inputs for small maps. (#29587) The ability to perform point lookups for multi-map side inputs is great for maps that are too large to fit into memory, but can be very inefficient in requiring an entire state request per key for small maps. This change adds an optional protocol to request an entire map as a stream of key-values in one (possibly paginated) API call, and uses this to bulk pre-fetch an initial set of values from the map before falling back to point lookups. --- .../model/fn_execution/v1/beam_fn_api.proto | 34 ++++++- .../control/RemoteExecutionTest.java | 13 ++- .../fn/harness/state/MultimapSideInput.java | 94 +++++++++++++++++-- .../harness/state/FakeBeamFnStateClient.java | 4 + .../harness/state/MultimapSideInputTest.java | 50 ++++++++-- .../portability/fn_api_runner/execution.py | 21 ++++- .../fn_api_runner/worker_handlers.py | 53 +++++++---- .../runners/worker/bundle_processor.py | 49 +++++++++- 8 files changed, 274 insertions(+), 44 deletions(-) diff --git a/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto b/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto index 777ce8636b7b..4b40c7fa4e4e 100644 --- a/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto +++ b/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto @@ -52,7 +52,7 @@ import "google/protobuf/duration.proto"; message FnApiTransforms { enum Runner { // DataSource is a Root Transform, and a source of data for downstream - // transforms in the same ProcessBundleDescriptor. + // transforms in the same ProcessBundleDescriptor. // It represents a stream of values coming in from an external source/over // a data channel, typically from the runner. It's not the PCollection itself // but a description of how to get the portion of the PCollection for a given @@ -82,7 +82,7 @@ message FnApiTransforms { // request will be sent with the matching instruction ID and transform ID. // Each PCollection that exits the ProcessBundleDescriptor subgraph will have // it's own DataSink, keyed by a transform ID determined by the runner. - // + // // The DataSink will take in a stream of elements for a given instruction ID // and encode them for transmission to the remote sink. The coder ID must be // for a windowed value coder. @@ -924,6 +924,35 @@ message StateKey { bytes window = 3; } + // Represents a request for the keys and values associated with a specified window in a PCollection. See + // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further + // details. + // + // This is expected to be more efficient than iterating over they keys and + // looking up the values one at a time. If a runner chooses not to implement + // this protocol, or a key has too many values to fit into a single response, + // the runner is free to fail the request and a fallback to point lookups + // will be performed by the SDK. + // + // Can only be used to perform StateGetRequests on side inputs of the URN + // beam:side_input:multimap:v1. + // + // For a PCollection>, the response data stream will be a + // concatenation of all KVs associated with the specified window, + // encoded with the the KV> coder. + // See + // https://s.apache.org/beam-fn-api-send-and-receive-data for further + // details. + message MultimapKeysValuesSideInput { + // (Required) The id of the PTransform containing a side input. + string transform_id = 1; + // (Required) The id of the side input. + string side_input_id = 2; + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + bytes window = 3; + } + // Represents a request for an unordered set of values associated with a // specified user key and window for a PTransform. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -999,6 +1028,7 @@ message StateKey { BagUserState bag_user_state = 3; IterableSideInput iterable_side_input = 4; MultimapKeysSideInput multimap_keys_side_input = 5; + MultimapKeysValuesSideInput multimap_keys_values_side_input = 8; MultimapKeysUserState multimap_keys_user_state = 6; MultimapUserState multimap_user_state = 7; } diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index ea23e28ddb66..6a16ca18fef9 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -915,9 +915,10 @@ public Coder valueCoder() { // Expect the following requests for the first bundle: // * one to read iterable side input // * one to read keys from multimap side input + // * one to attempt multimap side input bulk read // * one to read key1 iterable from multimap side input // * one to read key2 iterable from multimap side input - assertEquals(4, stateRequestHandler.receivedRequests.size()); + assertEquals(5, stateRequestHandler.receivedRequests.size()); assertEquals( stateRequestHandler.receivedRequests.get(0).getStateKey().getIterableSideInput(), BeamFnApi.StateKey.IterableSideInput.newBuilder() @@ -931,14 +932,20 @@ public Coder valueCoder() { .setTransformId(transformId) .build()); assertEquals( - stateRequestHandler.receivedRequests.get(2).getStateKey().getMultimapSideInput(), + stateRequestHandler.receivedRequests.get(2).getStateKey().getMultimapKeysValuesSideInput(), + BeamFnApi.StateKey.MultimapKeysValuesSideInput.newBuilder() + .setSideInputId(multimapView.getTagInternal().getId()) + .setTransformId(transformId) + .build()); + assertEquals( + stateRequestHandler.receivedRequests.get(3).getStateKey().getMultimapSideInput(), BeamFnApi.StateKey.MultimapSideInput.newBuilder() .setSideInputId(multimapView.getTagInternal().getId()) .setTransformId(transformId) .setKey(encode("key1")) .build()); assertEquals( - stateRequestHandler.receivedRequests.get(3).getStateKey().getMultimapSideInput(), + stateRequestHandler.receivedRequests.get(4).getStateKey().getMultimapSideInput(), BeamFnApi.StateKey.MultimapSideInput.newBuilder() .setSideInputId(multimapView.getTagInternal().getId()) .setTransformId(transformId) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java index 4bba03875774..619eea6cc70f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java @@ -20,13 +20,21 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.function.Function; import org.apache.beam.fn.harness.Cache; import org.apache.beam.fn.harness.Caches; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; /** @@ -38,11 +46,14 @@ }) public class MultimapSideInput implements MultimapView { + private static final int BULK_READ_SIZE = 100; + private final Cache cache; private final BeamFnStateClient beamFnStateClient; private final StateRequest keysRequest; private final Coder keyCoder; private final Coder valueCoder; + private volatile Function> bulkReadResult; public MultimapSideInput( Cache cache, @@ -71,17 +82,66 @@ public Iterable get() { @Override public Iterable get(K k) { - ByteStringOutputStream output = new ByteStringOutputStream(); - try { - keyCoder.encode(k, output); - } catch (IOException e) { - throw new IllegalStateException( - String.format( - "Failed to encode key %s for side input id %s.", - k, keysRequest.getStateKey().getMultimapKeysSideInput().getSideInputId()), - e); + ByteString encodedKey = encodeKey(k); + + if (bulkReadResult == null) { + synchronized (this) { + if (bulkReadResult == null) { + Map> bulkRead = new HashMap<>(); + StateKey bulkReadStateKey = + StateKey.newBuilder() + .setMultimapKeysValuesSideInput( + StateKey.MultimapKeysValuesSideInput.newBuilder() + .setTransformId( + keysRequest.getStateKey().getMultimapKeysSideInput().getTransformId()) + .setSideInputId( + keysRequest.getStateKey().getMultimapKeysSideInput().getSideInputId()) + .setWindow( + keysRequest.getStateKey().getMultimapKeysSideInput().getWindow())) + .build(); + + StateRequest bulkReadRequest = + keysRequest.toBuilder().setStateKey(bulkReadStateKey).build(); + try { + Iterator>> entries = + StateFetchingIterators.readAllAndDecodeStartingFrom( + Caches.subCache(cache, "ValuesForKey", encodedKey), + beamFnStateClient, + bulkReadRequest, + KvCoder.of(keyCoder, IterableCoder.of(valueCoder))) + .iterator(); + while (bulkRead.size() < BULK_READ_SIZE && entries.hasNext()) { + KV> entry = entries.next(); + bulkRead.put(encodeKey(entry.getKey()), entry.getValue()); + } + if (entries.hasNext()) { + bulkReadResult = bulkRead::get; + } else { + bulkReadResult = + key -> { + Iterable result = bulkRead.get(key); + if (result == null) { + // As we read the entire set of values, we don't have to do a lookup to know + // this key doesn't exist. + // Missing keys are treated as empty iterables in this multimap. + return Collections.emptyList(); + } else { + return result; + } + }; + } + } catch (Exception exn) { + bulkReadResult = bulkRead::get; + } + } + } + } + + Iterable bulkReadValues = bulkReadResult.apply(encodedKey); + if (bulkReadValues != null) { + return bulkReadValues; } - ByteString encodedKey = output.toByteString(); + StateKey stateKey = StateKey.newBuilder() .setMultimapSideInput( @@ -98,4 +158,18 @@ public Iterable get(K k) { return StateFetchingIterators.readAllAndDecodeStartingFrom( Caches.subCache(cache, "ValuesForKey", encodedKey), beamFnStateClient, request, valueCoder); } + + private ByteString encodeKey(K k) { + ByteStringOutputStream output = new ByteStringOutputStream(); + try { + keyCoder.encode(k, output); + } catch (IOException e) { + throw new IllegalStateException( + String.format( + "Failed to encode key %s for side input id %s.", + k, keysRequest.getStateKey().getMultimapKeysSideInput().getSideInputId()), + e); + } + return output.toByteString(); + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java index 4aaaa3d945d9..5447d6f0c6d6 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java @@ -136,6 +136,10 @@ public CompletableFuture handle(StateRequest.Builder requestBuild if (key.getTypeCase() == TypeCase.MULTIMAP_SIDE_INPUT || key.getTypeCase() == TypeCase.RUNNER) { assertEquals(RequestCase.GET, request.getRequestCase()); } + if (key.getTypeCase() == TypeCase.MULTIMAP_KEYS_VALUES_SIDE_INPUT && !data.containsKey(key)) { + // Allow testing this not being supported rather than blindly returning the empty list. + throw new UnsupportedOperationException("No multimap keys values states provided."); + } switch (request.getRequestCase()) { case GET: diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java index a4c20e3593ac..17ebf4234396 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java @@ -27,6 +27,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.KV; @@ -50,12 +52,38 @@ public class MultimapSideInputTest { private static final byte[] B = "B".getBytes(StandardCharsets.UTF_8); private static final byte[] UNKNOWN = "UNKNOWN".getBytes(StandardCharsets.UTF_8); + @Test + public void testGetWithBulkRead() throws Exception { + FakeBeamFnStateClient fakeBeamFnStateClient = + new FakeBeamFnStateClient( + ImmutableMap.of( + keysValuesStateKey(), + KV.of( + KvCoder.of(ByteArrayCoder.of(), IterableCoder.of(StringUtf8Coder.of())), + asList(KV.of(A, asList("A1", "A2", "A3")), KV.of(B, asList("B1", "B2")))))); + + MultimapSideInput multimapSideInput = + new MultimapSideInput<>( + Caches.noop(), + fakeBeamFnStateClient, + "instructionId", + keysStateKey(), + ByteArrayCoder.of(), + StringUtf8Coder.of()); + assertArrayEquals( + new String[] {"A1", "A2", "A3"}, Iterables.toArray(multimapSideInput.get(A), String.class)); + assertArrayEquals( + new String[] {"B1", "B2"}, Iterables.toArray(multimapSideInput.get(B), String.class)); + assertArrayEquals( + new String[] {}, Iterables.toArray(multimapSideInput.get(UNKNOWN), String.class)); + } + @Test public void testGet() throws Exception { FakeBeamFnStateClient fakeBeamFnStateClient = new FakeBeamFnStateClient( ImmutableMap.of( - stateKey(), KV.of(ByteArrayCoder.of(), asList(A, B)), + keysStateKey(), KV.of(ByteArrayCoder.of(), asList(A, B)), key(A), KV.of(StringUtf8Coder.of(), asList("A1", "A2", "A3")), key(B), KV.of(StringUtf8Coder.of(), asList("B1", "B2")))); @@ -64,7 +92,7 @@ public void testGet() throws Exception { Caches.noop(), fakeBeamFnStateClient, "instructionId", - stateKey(), + keysStateKey(), ByteArrayCoder.of(), StringUtf8Coder.of()); assertArrayEquals( @@ -82,7 +110,7 @@ public void testGetCached() throws Exception { FakeBeamFnStateClient fakeBeamFnStateClient = new FakeBeamFnStateClient( ImmutableMap.of( - stateKey(), KV.of(ByteArrayCoder.of(), asList(A, B)), + keysStateKey(), KV.of(ByteArrayCoder.of(), asList(A, B)), key(A), KV.of(StringUtf8Coder.of(), asList("A1", "A2", "A3")), key(B), KV.of(StringUtf8Coder.of(), asList("B1", "B2")))); @@ -94,7 +122,7 @@ public void testGetCached() throws Exception { cache, fakeBeamFnStateClient, "instructionId", - stateKey(), + keysStateKey(), ByteArrayCoder.of(), StringUtf8Coder.of()); assertArrayEquals( @@ -117,7 +145,7 @@ public void testGetCached() throws Exception { throw new IllegalStateException("Unexpected call for test."); }, "instructionId", - stateKey(), + keysStateKey(), ByteArrayCoder.of(), StringUtf8Coder.of()); assertArrayEquals( @@ -132,7 +160,7 @@ public void testGetCached() throws Exception { } } - private StateKey stateKey() throws IOException { + private StateKey keysStateKey() throws IOException { return StateKey.newBuilder() .setMultimapKeysSideInput( StateKey.MultimapKeysSideInput.newBuilder() @@ -142,6 +170,16 @@ private StateKey stateKey() throws IOException { .build(); } + private StateKey keysValuesStateKey() throws IOException { + return StateKey.newBuilder() + .setMultimapKeysValuesSideInput( + StateKey.MultimapKeysValuesSideInput.newBuilder() + .setTransformId("ptransformId") + .setSideInputId("sideInputId") + .setWindow(ByteString.copyFromUtf8("encodedWindow"))) + .build(); + } + private StateKey key(byte[] key) throws IOException { ByteStringOutputStream out = new ByteStringOutputStream(); ByteArrayCoder.of().encode(key, out); diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py index bc60b5dd86cd..fbcb58e16133 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py @@ -23,6 +23,7 @@ import copy import itertools import logging +import struct import typing import uuid import weakref @@ -351,7 +352,7 @@ def append(self, elements_data): self._values_by_window[key, window].append(value) def encoded_items(self): - # type: () -> Iterator[Tuple[bytes, bytes, bytes]] + # type: () -> Iterator[Tuple[bytes, bytes, bytes, int]] value_coder_impl = self._value_coder.get_impl() key_coder_impl = self._key_coder.get_impl() for (key, window), values in self._values_by_window.items(): @@ -360,7 +361,7 @@ def encoded_items(self): output_stream = create_OutputStream() for value in values: value_coder_impl.encode_to_stream(value, output_stream, True) - yield encoded_key, encoded_window, output_stream.get() + yield encoded_key, encoded_window, output_stream.get(), len(values) class GenericNonMergingWindowFn(window.NonMergingWindowFn): @@ -979,7 +980,7 @@ def commit_side_inputs_to_state( elements_by_window.append(element_data) if func_spec.urn == common_urns.side_inputs.ITERABLE.urn: - for _, window, elements_data in elements_by_window.encoded_items(): + for _, window, elements_data, _ in elements_by_window.encoded_items(): state_key = beam_fn_api_pb2.StateKey( iterable_side_input=beam_fn_api_pb2.StateKey.IterableSideInput( transform_id=consuming_transform_id, @@ -987,7 +988,8 @@ def commit_side_inputs_to_state( window=window)) self.state_servicer.append_raw(state_key, elements_data) elif func_spec.urn == common_urns.side_inputs.MULTIMAP.urn: - for key, window, elements_data in elements_by_window.encoded_items(): + for (key, window, elements_data, + elements_count) in elements_by_window.encoded_items(): state_key = beam_fn_api_pb2.StateKey( multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput( transform_id=consuming_transform_id, @@ -995,6 +997,17 @@ def commit_side_inputs_to_state( window=window, key=key)) self.state_servicer.append_raw(state_key, elements_data) + + kv_iter_state_key = beam_fn_api_pb2.StateKey( + multimap_keys_values_side_input=beam_fn_api_pb2.StateKey. + MultimapKeysValuesSideInput( + transform_id=consuming_transform_id, + side_input_id=tag, + window=window)) + self.state_servicer.append_raw( + kv_iter_state_key, + # KV> encoding. + key + struct.pack('>i', elements_count) + elements_data) else: raise ValueError("Unknown access pattern: '%s'" % func_spec.urn) diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index de55235368e3..b0b4b1957dd8 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -945,6 +945,15 @@ def get_worker(self, worker_id): class StateServicer(beam_fn_api_pb2_grpc.BeamFnStateServicer, sdk_worker.StateHandler): + _SUPPORTED_STATE_TYPES = frozenset([ + 'runner', + 'multimap_side_input', + 'multimap_keys_values_side_input', + 'iterable_side_input', + 'bag_user_state', + 'multimap_user_state' + ]) + class CopyOnWriteState(object): def __init__(self, underlying): # type: (DefaultDict[bytes, Buffer]) -> None @@ -1038,6 +1047,11 @@ def get_raw(self, continuation_token=None # type: Optional[bytes] ): # type: (...) -> Tuple[bytes, Optional[bytes]] + + if state_key.WhichOneof('type') not in self._SUPPORTED_STATE_TYPES: + raise NotImplementedError( + 'Unknown state type: ' + state_key.WhichOneof('type')) + with self._lock: full_state = self._state[self._to_key(state_key)] if self._use_continuation_tokens: @@ -1104,24 +1118,27 @@ def State(self, # Note that this eagerly mutates state, assuming any failures are fatal. # Thus it is safe to ignore instruction_id. for request in request_stream: - request_type = request.WhichOneof('request') - if request_type == 'get': - data, continuation_token = self._state.get_raw( - request.state_key, request.get.continuation_token) - yield beam_fn_api_pb2.StateResponse( - id=request.id, - get=beam_fn_api_pb2.StateGetResponse( - data=data, continuation_token=continuation_token)) - elif request_type == 'append': - self._state.append_raw(request.state_key, request.append.data) - yield beam_fn_api_pb2.StateResponse( - id=request.id, append=beam_fn_api_pb2.StateAppendResponse()) - elif request_type == 'clear': - self._state.clear(request.state_key) - yield beam_fn_api_pb2.StateResponse( - id=request.id, clear=beam_fn_api_pb2.StateClearResponse()) - else: - raise NotImplementedError('Unknown state request: %s' % request_type) + try: + request_type = request.WhichOneof('request') + if request_type == 'get': + data, continuation_token = self._state.get_raw( + request.state_key, request.get.continuation_token) + yield beam_fn_api_pb2.StateResponse( + id=request.id, + get=beam_fn_api_pb2.StateGetResponse( + data=data, continuation_token=continuation_token)) + elif request_type == 'append': + self._state.append_raw(request.state_key, request.append.data) + yield beam_fn_api_pb2.StateResponse( + id=request.id, append=beam_fn_api_pb2.StateAppendResponse()) + elif request_type == 'clear': + self._state.clear(request.state_key) + yield beam_fn_api_pb2.StateResponse( + id=request.id, clear=beam_fn_api_pb2.StateClearResponse()) + else: + raise NotImplementedError('Unknown state request: %s' % request_type) + except Exception as exn: + yield beam_fn_api_pb2.StateResponse(id=request.id, error=str(exn)) class SingletonStateHandlerFactory(sdk_worker.StateHandlerFactory): diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 7ff0ad258bc2..b35997c4250f 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -378,6 +378,11 @@ def __reduce__(self): class StateBackedSideInputMap(object): + + _BULK_READ_LIMIT = 100 + _BULK_READ_FULLY = "fully" + _BULK_READ_PARTIALLY = "partially" + def __init__(self, state_handler, # type: sdk_worker.CachingStateHandler transform_id, # type: str @@ -417,12 +422,53 @@ def __getitem__(self, window): side_input_id=self._tag, window=self._target_window_coder.encode(target_window), key=b'')) + kv_iter_state_key = beam_fn_api_pb2.StateKey( + multimap_keys_values_side_input=beam_fn_api_pb2.StateKey. + MultimapKeysValuesSideInput( + transform_id=self._transform_id, + side_input_id=self._tag, + window=self._target_window_coder.encode(target_window))) cache = {} - key_coder_impl = self._element_coder.key_coder().get_impl() + key_coder = self._element_coder.key_coder() + key_coder_impl = key_coder.get_impl() value_coder = self._element_coder.value_coder() class MultiMap(object): + _bulk_read = None + _lock = threading.Lock() + def __getitem__(self, key): + if self._bulk_read is None: + with self._lock: + if self._bulk_read is None: + try: + # Attempt to bulk read the key-values over the iterable + # protocol which, if supported, can be much more efficient + # than point lookups if it fits into memory. + for ix, (k, vs) in enumerate(_StateBackedIterable( + state_handler, + kv_iter_state_key, + coders.TupleCoder( + (key_coder, coders.IterableCoder(value_coder))))): + cache[k] = vs + if ix > StateBackedSideInputMap._BULK_READ_LIMIT: + self._bulk_read = ( + StateBackedSideInputMap._BULK_READ_PARTIALLY) + break + else: + # We reached the end of the iteration without breaking. + self._bulk_read = ( + StateBackedSideInputMap._BULK_READ_FULLY) + except Exception: + _LOGGER.error( + "Iterable access of map side inputs unsupported.", + exc_info=True) + self._bulk_read = ( + StateBackedSideInputMap._BULK_READ_PARTIALLY) + + if (self._bulk_read == StateBackedSideInputMap._BULK_READ_FULLY): + return cache.get(key, []) + if key not in cache: keyed_state_key = beam_fn_api_pb2.StateKey() keyed_state_key.CopyFrom(state_key) @@ -430,6 +476,7 @@ def __getitem__(self, key): key_coder_impl.encode_nested(key)) cache[key] = _StateBackedIterable( state_handler, keyed_state_key, value_coder) + return cache[key] def __reduce__(self): From 209b0959ce83358825dd2042d7fb276f7205194a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 8 Dec 2023 10:59:25 -0500 Subject: [PATCH 069/224] Bump github.com/aws/aws-sdk-go-v2/config from 1.25.8 to 1.26.0 in /sdks (#29678) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.25.8 to 1.26.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/config/v1.25.8...config/v1.26.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config 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 | 24 ++++++++++++------------ sdks/go.sum | 48 ++++++++++++++++++++++++------------------------ 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index c46c7e28a58c..6beb93088f0c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.35.1 - github.com/aws/aws-sdk-go-v2 v1.23.5 - github.com/aws/aws-sdk-go-v2/config v1.25.8 - github.com/aws/aws-sdk-go-v2/credentials v1.16.9 + github.com/aws/aws-sdk-go-v2 v1.24.0 + github.com/aws/aws-sdk-go-v2/config v1.26.0 + github.com/aws/aws-sdk-go-v2/credentials v1.16.11 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 - github.com/aws/smithy-go v1.18.1 + github.com/aws/smithy-go v1.19.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 @@ -106,18 +106,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.18.2 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.4 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.4 // 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 5df50d7fd020..e4bf59644599 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,39 +81,39 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= -github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= +github.com/aws/aws-sdk-go-v2 v1.24.0 h1:890+mqQ+hTpNuw0gGP6/4akolQkSToDJgHfQE7AwGuk= +github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.25.8 h1:CHr7PIzyfevjNiqL9rU6xoqHZKCO2ldY6LmvRDfpRuI= -github.com/aws/aws-sdk-go-v2/config v1.25.8/go.mod h1:zefIy117FDPOVU0xSOFG8mx9kJunuVopzI639tjYXc0= +github.com/aws/aws-sdk-go-v2/config v1.26.0 h1:uItWWbD/FmHPGSa6GJFyZJD/RPakVjS0fmoq1vccjNw= +github.com/aws/aws-sdk-go-v2/config v1.26.0/go.mod h1:8Rf77VTcX9MMkoMIsCnuwmef+Y1bs2Zhvw9IXHdD/Po= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.9 h1:LQo3MUIOzod9JdUK+wxmSdgzLVYUbII3jXn3S/HJZU0= -github.com/aws/aws-sdk-go-v2/credentials v1.16.9/go.mod h1:R7mDuIJoCjH6TxGUc/cylE7Lp/o0bhKVoxdBThsjqCM= +github.com/aws/aws-sdk-go-v2/credentials v1.16.11 h1:Gcut3tJSU7F/C5W/NnFimqnJqljF58rmaw7QlbigN3U= +github.com/aws/aws-sdk-go-v2/credentials v1.16.11/go.mod h1:CysUbSCfqvEbEQTd9Ubg2RrJy2EFM+AUHJOqqj0guTo= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 h1:v+HbZaCGmOwnTTVS86Fleq0vPzOd7tnJGbFhP0stNLs= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VDLBtCMkQYOw1QYfAEZCVLrfI0ezve8wd4= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9/go.mod h1:hqamLz7g1/4EJP+GH5NBhcUMLjW+gKLQabgyz6/7WAU= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeEaev/ZF01cQyNZgmYE9oYYePIMJs2mWSKG514= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4 h1:/b31bi3YVNlkzkBrm9LfpaKoaYZUxIAj4sHfOTmLfqw= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4/go.mod h1:2aGXHFmbInwgP9ZfpmdIfOELL79zhdNYNmReK8qDfdQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 h1:Nf2sHxjMJR8CSImIVCONRi4g0Su3J+TSTbS7G0pUeMU= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9/go.mod h1:idky4TER38YIjr2cADF1/ugFMKvZV7p//pVeV5LZbF0= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -121,16 +121,16 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.2 h1:xJPydhNm0Hiqct5TVKEuHG7weC0+sOs4MUnd7A5n5F4= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.2/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2 h1:8dU9zqA77C5egbU6yd4hFLaiIdPv3rU+6cp7sz5FjCU= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.2/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.4 h1:2UVO4N/polvKeP+yCA8TLEmidEKxmNTeVpsZnj/bbgA= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.4/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4 h1:3JXkQ1F5n73qTpSPas6AQ8/6HFksgnB24JlNPLt3SlM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.2 h1:fFrLsy08wEbAisqW3KDl/cPHrF43GmV79zXB9EwJiZw= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.2/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.4 h1:gaRFldXhoT36jVMfQ+AjAYwSfjO5LMgy1u0ObcKFhhc= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.4/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= -github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= +github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= +github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= From a7976f549080c7da41b833d66ed262958d9a5ccb Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Fri, 8 Dec 2023 10:59:49 -0500 Subject: [PATCH 070/224] [Python][RRIO] Call PTransform with setup teardown (#29585) * rrio call ptransform with setup teardown * add main * add SECS suffix * update names --- .../apache_beam/io/requestresponseio.py | 181 ++++++++++++++++-- .../io/requestresponseio_it_test.py | 18 +- .../apache_beam/io/requestresponseio_test.py | 88 +++++++++ 3 files changed, 270 insertions(+), 17 deletions(-) create mode 100644 sdks/python/apache_beam/io/requestresponseio_test.py diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponseio.py index fc742fa00cad..0ec586e64018 100644 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ b/sdks/python/apache_beam/io/requestresponseio.py @@ -17,11 +17,24 @@ """``PTransform`` for reading from and writing to Web APIs.""" import abc +import concurrent.futures +import contextlib +import logging +import sys +from typing import Generic +from typing import Optional from typing import TypeVar +import apache_beam as beam +from apache_beam.pvalue import PCollection + RequestT = TypeVar('RequestT') ResponseT = TypeVar('ResponseT') +DEFAULT_TIMEOUT_SECS = 30 # seconds + +_LOGGER = logging.getLogger(__name__) + class UserCodeExecutionException(Exception): """Base class for errors related to calling Web APIs.""" @@ -37,8 +50,10 @@ class UserCodeTimeoutException(UserCodeExecutionException): """Extends ``UserCodeExecutionException`` to signal a user code timeout.""" -class Caller(metaclass=abc.ABCMeta): - """Interfaces user custom code intended for API calls.""" +class Caller(contextlib.AbstractContextManager, abc.ABC): + """Interface for user custom code intended for API calls. + For setup and teardown of clients when applicable, implement the + ``__enter__`` and ``__exit__`` methods respectively.""" @abc.abstractmethod def __call__(self, request: RequestT, *args, **kwargs) -> ResponseT: """Calls a Web API with the ``RequestT`` and returns a @@ -48,18 +63,156 @@ def __call__(self, request: RequestT, *args, **kwargs) -> ResponseT: """ pass + def __enter__(self): + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + return None + + +class ShouldBackOff(abc.ABC): + """ + ShouldBackOff provides mechanism to apply adaptive throttling. + """ + pass + + +class Repeater(abc.ABC): + """Repeater provides mechanism to repeat requests for a + configurable condition.""" + pass -class SetupTeardown(metaclass=abc.ABCMeta): - """Interfaces user custom code to set up and teardown the API clients. - Called by ``RequestResponseIO`` within its DoFn's setup and teardown - methods. + +class CacheReader(abc.ABC): + """CacheReader provides mechanism to read from the cache.""" + pass + + +class CacheWriter(abc.ABC): + """CacheWriter provides mechanism to write to the cache.""" + pass + + +class PreCallThrottler(abc.ABC): + """PreCallThrottler provides a throttle mechanism before sending request.""" + pass + + +class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], + beam.PCollection[ResponseT]]): + """A :class:`RequestResponseIO` transform to read and write to APIs. + + Processes an input :class:`~apache_beam.pvalue.PCollection` of requests + by making a call to the API as defined in :class:`Caller`'s `__call__` + and returns a :class:`~apache_beam.pvalue.PCollection` of responses. + """ + def __init__( + self, + caller: [Caller], + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + should_backoff: Optional[ShouldBackOff] = None, + repeater: Optional[Repeater] = None, + cache_reader: Optional[CacheReader] = None, + cache_writer: Optional[CacheWriter] = None, + throttler: Optional[PreCallThrottler] = None, + ): """ - @abc.abstractmethod - def setup(self) -> None: - """Called during the DoFn's setup lifecycle method.""" - pass + Instantiates a RequestResponseIO transform. - @abc.abstractmethod - def teardown(self) -> None: - """Called during the DoFn's teardown lifecycle method.""" - pass + Args: + caller (~apache_beam.io.requestresponseio.Caller): an implementation of + `Caller` object that makes call to the API. + timeout (float): timeout value in seconds to wait for response from API. + should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): + (Optional) provides methods for backoff. + repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) + provides methods to repeat requests to API. + cache_reader (~apache_beam.io.requestresponseio.CacheReader): (Optional) + provides methods to read external cache. + cache_writer (~apache_beam.io.requestresponseio.CacheWriter): (Optional) + provides methods to write to external cache. + throttler (~apache_beam.io.requestresponseio.PreCallThrottler): + (Optional) provides methods to pre-throttle a request. + """ + self._caller = caller + self._timeout = timeout + self._should_backoff = should_backoff + self._repeater = repeater + self._cache_reader = cache_reader + self._cache_writer = cache_writer + self._throttler = throttler + + def expand(self, requests: PCollection[RequestT]) -> PCollection[ResponseT]: + # TODO(riteshghorse): add Cache and Throttle PTransforms. + return requests | _Call( + caller=self._caller, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater) + + +class _Call(beam.PTransform[beam.PCollection[RequestT], + beam.PCollection[ResponseT]]): + """(Internal-only) PTransform that invokes a remote function on each element + of the input PCollection. + + This PTransform uses a `Caller` object to invoke the actual API calls, + and uses ``__enter__`` and ``__exit__`` to manage setup and teardown of + clients when applicable. Additionally, a timeout value is specified to + regulate the duration of each call, defaults to 30 seconds. + + Args: + caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable + object that invokes API call. + timeout (float): timeout value in seconds to wait for response from API. + """ + def __init__( + self, + caller: Caller, + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + should_backoff: Optional[ShouldBackOff] = None, + repeater: Optional[Repeater] = None, + ): + """Initialize the _Call transform. + Args: + caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable + object that invokes API call. + timeout (float): timeout value in seconds to wait for response from API. + should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): + (Optional) provides methods for backoff. + repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) provides + methods to repeat requests to API. + """ + self._caller = caller + self._timeout = timeout + self._should_backoff = should_backoff + self._repeater = repeater + + def expand( + self, + requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: + return requests | beam.ParDo(_CallDoFn(self._caller, self._timeout)) + + +class _CallDoFn(beam.DoFn, Generic[RequestT, ResponseT]): + def setup(self): + self._caller.__enter__() + + def __init__(self, caller: Caller, timeout: float): + self._caller = caller + self._timeout = timeout + + def process(self, request, *args, **kwargs): + with concurrent.futures.ThreadPoolExecutor() as executor: + future = executor.submit(self._caller, request) + try: + yield future.result(timeout=self._timeout) + except concurrent.futures.TimeoutError: + raise UserCodeTimeoutException( + f'Timeout {self._timeout} exceeded ' + f'while completing request: {request}') + except RuntimeError: + raise UserCodeExecutionException('could not complete request') + + def teardown(self): + self._caller.__exit__(*sys.exc_info()) diff --git a/sdks/python/apache_beam/io/requestresponseio_it_test.py b/sdks/python/apache_beam/io/requestresponseio_it_test.py index f291ff96a4d7..aae6b4e6ef2c 100644 --- a/sdks/python/apache_beam/io/requestresponseio_it_test.py +++ b/sdks/python/apache_beam/io/requestresponseio_it_test.py @@ -23,10 +23,13 @@ import urllib3 +import apache_beam as beam from apache_beam.io.requestresponseio import Caller +from apache_beam.io.requestresponseio import RequestResponseIO from apache_beam.io.requestresponseio import UserCodeExecutionException from apache_beam.io.requestresponseio import UserCodeQuotaException from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.testing.test_pipeline import TestPipeline _HTTP_PATH = '/v1/echo' _PAYLOAD = base64.b64encode(bytes('payload', 'utf-8')) @@ -86,7 +89,6 @@ def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: ``UserCodeExecutionException``, ``UserCodeTimeoutException``, or a ``UserCodeQuotaException``. """ - try: resp = urllib3.request( "POST", @@ -104,8 +106,8 @@ def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: if resp.status == 429: # Too Many Requests raise UserCodeQuotaException(resp.reason) - - raise UserCodeExecutionException(resp.reason) + else: + raise UserCodeExecutionException(resp.status, resp.reason, request) except urllib3.exceptions.HTTPError as e: raise UserCodeExecutionException(e) @@ -167,6 +169,16 @@ def test_not_found_should_raise(self): self.assertRaisesRegex( UserCodeExecutionException, "Not Found", lambda: client(req)) + def test_request_response_io(self): + client, options = EchoHTTPCallerTestIT._get_client_and_options() + req = EchoRequest(id=options.never_exceed_quota_id, payload=_PAYLOAD) + with TestPipeline(is_integration_test=True) as test_pipeline: + output = ( + test_pipeline + | 'Create PCollection' >> beam.Create([req]) + | 'RRIO Transform' >> RequestResponseIO(client)) + self.assertIsNotNone(output) + if __name__ == '__main__': unittest.main(argv=sys.argv[:1]) diff --git a/sdks/python/apache_beam/io/requestresponseio_test.py b/sdks/python/apache_beam/io/requestresponseio_test.py new file mode 100644 index 000000000000..2828a3578871 --- /dev/null +++ b/sdks/python/apache_beam/io/requestresponseio_test.py @@ -0,0 +1,88 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import time +import unittest + +import apache_beam as beam +from apache_beam.io.requestresponseio import Caller +from apache_beam.io.requestresponseio import RequestResponseIO +from apache_beam.io.requestresponseio import UserCodeExecutionException +from apache_beam.io.requestresponseio import UserCodeTimeoutException +from apache_beam.testing.test_pipeline import TestPipeline + + +class AckCaller(Caller): + """AckCaller acknowledges the incoming request by returning a + request with ACK.""" + def __enter__(self): + pass + + def __call__(self, request: str): + return f"ACK: {request}" + + def __exit__(self, exc_type, exc_val, exc_tb): + return None + + +class CallerWithTimeout(AckCaller): + """CallerWithTimeout sleeps for 2 seconds before responding. + Used to test timeout in RequestResponseIO.""" + def __call__(self, request: str, *args, **kwargs): + time.sleep(2) + return f"ACK: {request}" + + +class CallerWithRuntimeError(AckCaller): + """CallerWithRuntimeError raises a `RuntimeError` for RequestResponseIO + to raise a UserCodeExecutionException.""" + def __call__(self, request: str, *args, **kwargs): + if not request: + raise RuntimeError("Exception expected, not an error.") + + +class TestCaller(unittest.TestCase): + def test_valid_call(self): + caller = AckCaller() + with TestPipeline() as test_pipeline: + output = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller)) + + self.assertIsNotNone(output) + + def test_call_timeout(self): + caller = CallerWithTimeout() + with self.assertRaises(UserCodeTimeoutException): + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["timeout_request"]) + | RequestResponseIO(caller=caller, timeout=1)) + + def test_call_runtime_error(self): + caller = CallerWithRuntimeError() + with self.assertRaises(UserCodeExecutionException): + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create([""]) + | RequestResponseIO(caller=caller)) + + +if __name__ == '__main__': + unittest.main() From ff03c129e07b9353a2005c76d91bdd12e4636565 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 8 Dec 2023 12:03:52 -0500 Subject: [PATCH 071/224] Shrink Java PreCommit timeout (#29671) * This test suite has a flakiness of stuck indefinitely. When this does not happen, it finishes within an hour. --- .github/workflows/beam_PreCommit_Java.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 6f4669e0c784..21898e5f758b 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -161,6 +161,7 @@ jobs: matrix: job_name: [beam_PreCommit_Java] job_phrase: [Run Java PreCommit] + timeout-minutes: 120 if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || From 47ccb8a5ee5b20b8d3ae07bdf33e1c82fd6ecd08 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 7 Dec 2023 14:26:09 -0800 Subject: [PATCH 072/224] [YAML] Better IO documentation. --- sdks/python/apache_beam/io/avroio.py | 16 ++++++++++--- sdks/python/apache_beam/io/parquetio.py | 7 +----- .../apache_beam/yaml/generate_yaml_docs.py | 2 +- sdks/python/apache_beam/yaml/standard_io.yaml | 8 ------- sdks/python/apache_beam/yaml/yaml_io.py | 24 +++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 3 ++- 6 files changed, 41 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 9225acf346e4..24df59ddc5cc 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -75,8 +75,15 @@ class ReadFromAvro(PTransform): - """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading avro - files.""" + """A `PTransform` for reading records from avro files. + + Each record of the resulting PCollection will contain + a single record read from a source. Records that are of simple types will be + mapped to beam Rows with a single `record` field containing the records + value. Records that are of Avro type ``RECORD`` will be mapped to Beam rows + that comply with the schema contained in the Avro file that contains those + records. + """ def __init__( self, file_pattern=None, @@ -355,7 +362,10 @@ def split_points_unclaimed(stop_position): class WriteToAvro(beam.transforms.PTransform): - """A ``PTransform`` for writing avro files.""" + """A ``PTransform`` for writing avro files. + + If the input has a schema, a corresponding avro schema will be automatically + generated and used to write the output records.""" def __init__( self, file_path_prefix, diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 4696e5ae7927..48c51428c17d 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -231,9 +231,7 @@ def display_data(self): class ReadFromParquet(PTransform): - """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading - Parquet files as a `PCollection` of dictionaries. This `PTransform` is - currently experimental. No backward-compatibility guarantees.""" + """A `PTransform` for reading Parquet files.""" def __init__( self, file_pattern=None, @@ -465,9 +463,6 @@ def split_points_unclaimed(stop_position): class WriteToParquet(PTransform): """A ``PTransform`` for writing parquet files. - - This ``PTransform`` is currently experimental. No backward-compatibility - guarantees. """ def __init__( self, diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py index d3d6c0a3e316..3c2022642f51 100644 --- a/sdks/python/apache_beam/yaml/generate_yaml_docs.py +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -116,7 +116,7 @@ def lines(): yield ''.join([ f'**{f.name}** `{pretty_type(f.type)}`', maybe_optional(f.type), - ': ' + f.description if f.description else '', + indent(': ' + f.description if f.description else '', 2), maybe_row_parameters(f.type), ]) diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 506ea6634aa3..b49d40d5a40b 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -239,13 +239,5 @@ transforms: 'ReadFromJdbc': 'beam:schematransform:org.apache.beam:jdbc_read:v1' 'WriteToJdbc': 'beam:schematransform:org.apache.beam:jdbc_write:v1' - 'ReadFromMySql': 'beam:schematransform:org.apache.beam:jdbc_read:v1' - 'WriteToMySql': 'beam:schematransform:org.apache.beam:jdbc_write:v1' - 'ReadFromPostgres': 'beam:schematransform:org.apache.beam:jdbc_read:v1' - 'WriteToPostgres': 'beam:schematransform:org.apache.beam:jdbc_write:v1' - 'ReadFromOracle': 'beam:schematransform:org.apache.beam:jdbc_read:v1' - 'WriteToOracle': 'beam:schematransform:org.apache.beam:jdbc_write:v1' - 'ReadFromSqlServer': 'beam:schematransform:org.apache.beam:jdbc_read:v1' - 'WriteToSqlServer': 'beam:schematransform:org.apache.beam:jdbc_write:v1' config: gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index 7db195ea84a2..1cd7231e35dd 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -52,11 +52,29 @@ def read_from_text(path: str): # TODO(yaml): Consider passing the filename and offset, possibly even # by default. + + """Reads lines from a text files. + + The resulting PCollection consists of rows with a single string filed named + "line." + + Args: + path (str): The file path to read from. The path can contain glob + characters such as ``*`` and ``?``. + """ return beam_io.ReadFromText(path) | beam.Map(lambda s: beam.Row(line=s)) @beam.ptransform_fn def write_to_text(pcoll, path: str): + """Writes a PCollection to a (set of) text files(s). + + The input must be a PCollection whose schema has exactly one field. + + Args: + path (str): The file path to write to. The files written will + begin with this prefix, followed by a shard identifier. + """ try: field_names = [ name for name, @@ -76,6 +94,11 @@ def write_to_text(pcoll, path: str): def read_from_bigquery( query=None, table=None, row_restriction=None, fields=None): + """Reads data from BigQuery. + + Exactly one of table or query must be set. + If query is set, neither row_restriction nor fields should be set. + """ if query is None: assert table is not None else: @@ -95,6 +118,7 @@ def write_to_bigquery( create_disposition=BigQueryDisposition.CREATE_IF_NEEDED, write_disposition=BigQueryDisposition.WRITE_APPEND, error_handling=None): + """Writes data to a BigQuery table.""" class WriteToBigQueryHandlingErrors(beam.PTransform): def default_label(self): return 'WriteToBigQuery' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 5ba15ff17e8b..24d6d5a80952 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -453,7 +453,8 @@ def empty_if_none(s): docs = self.get_docs(typ) return ( - empty_if_none(docs.short_description) + '\n\n' + + empty_if_none(docs.short_description) + + ('\n\n' if docs.blank_after_short_description else '\n') + empty_if_none(docs.long_description)).strip() or None def get_docs(self, typ): From 5c670da47c9baf3013729070f4ef45223a5662e2 Mon Sep 17 00:00:00 2001 From: Zechen Jiang Date: Fri, 8 Dec 2023 10:23:46 -0800 Subject: [PATCH 073/224] Add Data Sampling support for periodic sampling (#29590) --- .../pkg/beam/core/runtime/exec/datasampler.go | 153 ++++++++++++++++++ .../core/runtime/exec/datasampler_test.go | 148 +++++++++++++++++ .../pkg/beam/core/runtime/exec/datasource.go | 1 + .../pkg/beam/core/runtime/exec/pcollection.go | 32 +++- .../core/runtime/exec/pcollection_test.go | 4 +- .../pkg/beam/core/runtime/exec/to_string.go | 1 + .../pkg/beam/core/runtime/exec/translate.go | 18 ++- .../beam/core/runtime/exec/translate_test.go | 4 +- .../pkg/beam/core/runtime/graphx/translate.go | 2 + .../pkg/beam/core/runtime/harness/harness.go | 33 +++- .../beam/core/runtime/harness/harness_test.go | 2 +- .../beam/core/runtime/harness/init/init.go | 23 ++- sdks/go/pkg/beam/core/typex/fulltype.go | 11 +- .../beam/runners/dataflow/dataflowlib/job.go | 1 + 14 files changed, 403 insertions(+), 30 deletions(-) create mode 100644 sdks/go/pkg/beam/core/runtime/exec/datasampler.go create mode 100644 sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasampler.go b/sdks/go/pkg/beam/core/runtime/exec/datasampler.go new file mode 100644 index 000000000000..426213a12afd --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/datasampler.go @@ -0,0 +1,153 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 exec + +import ( + "context" + "sync" + "time" +) + +// DataSample contains property for sampled element +type dataSample struct { + PCollectionID string + Timestamp time.Time + Element []byte +} + +// DataSampler manages sampled elements based on PCollectionID +type DataSampler struct { + sampleChannel chan *dataSample + samplesMap sync.Map // Key: PCollectionID string, Value: *OutputSamples pointer + ctx context.Context +} + +// NewDataSampler inits a new Data Sampler object and returns pointer to it. +func NewDataSampler(ctx context.Context) *DataSampler { + return &DataSampler{ + sampleChannel: make(chan *dataSample, 1000), + ctx: ctx, + } +} + +// Process processes sampled element. +func (d *DataSampler) Process() { + for { + select { + case <-d.ctx.Done(): + return + case sample := <-d.sampleChannel: + d.addSample(sample) + } + } +} + +// GetSamples returns samples for given pCollectionID. +// If no pCollectionID is provided, return all available samples +func (d *DataSampler) GetSamples(pids []string) map[string][]*dataSample { + if len(pids) == 0 { + return d.getAllSamples() + } + return d.getSamplesForPCollections(pids) +} + +// SendSample is called by PCollection Node to send sampled element to Data Sampler async +func (d *DataSampler) SendSample(pCollectionID string, element []byte, timestamp time.Time) { + sample := dataSample{ + PCollectionID: pCollectionID, + Element: element, + Timestamp: timestamp, + } + d.sampleChannel <- &sample +} + +func (d *DataSampler) getAllSamples() map[string][]*dataSample { + var res = make(map[string][]*dataSample) + d.samplesMap.Range(func(key any, value any) bool { + pid := key.(string) + samples := d.getSamples(pid) + if len(samples) > 0 { + res[pid] = samples + } + return true + }) + return res +} + +func (d *DataSampler) getSamplesForPCollections(pids []string) map[string][]*dataSample { + var res = make(map[string][]*dataSample) + for _, pid := range pids { + samples := d.getSamples(pid) + if len(samples) > 0 { + res[pid] = samples + } + } + return res +} + +func (d *DataSampler) addSample(sample *dataSample) { + p, ok := d.samplesMap.Load(sample.PCollectionID) + if !ok { + p = &outputSamples{maxElements: 10, sampleIndex: 0} + d.samplesMap.Store(sample.PCollectionID, p) + } + outputSamples := p.(*outputSamples) + outputSamples.addSample(sample) +} + +func (d *DataSampler) getSamples(pCollectionID string) []*dataSample { + p, ok := d.samplesMap.Load(pCollectionID) + if !ok { + return nil + } + outputSamples := p.(*outputSamples) + return outputSamples.getSamples() +} + +type outputSamples struct { + elements []*dataSample + mu sync.Mutex + maxElements int + sampleIndex int +} + +func (o *outputSamples) addSample(element *dataSample) { + o.mu.Lock() + defer o.mu.Unlock() + + if len(o.elements) < o.maxElements { + o.elements = append(o.elements, element) + } else { + o.elements[o.sampleIndex] = element + o.sampleIndex = (o.sampleIndex + 1) % o.maxElements + } +} + +func (o *outputSamples) getSamples() []*dataSample { + o.mu.Lock() + defer o.mu.Unlock() + if len(o.elements) == 0 { + return nil + } + samples := o.elements + + // Reset index and samples + o.sampleIndex = 0 + // Release memory since samples are only returned once based on best efforts + o.elements = nil + + return samples +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go b/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go new file mode 100644 index 000000000000..d648fd89efaa --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 exec + +import ( + "context" + "reflect" + "sort" + "testing" + "time" +) + +// TestDataSampler verifies that the DataSampler works correctly. +func TestDataSampler(t *testing.T) { + timestamp := time.Now() + tests := []struct { + name string + samples []dataSample + pids []string + want map[string][]*dataSample + }{ + { + name: "GetAllSamples", + samples: []dataSample{ + {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, + {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, + }, + pids: []string{}, + want: map[string][]*dataSample{ + "pid1": {{PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}}, + "pid2": {{PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}}, + }, + }, + { + name: "GetSamplesForPCollections", + samples: []dataSample{ + {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, + {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, + }, + pids: []string{"pid1"}, + want: map[string][]*dataSample{ + "pid1": {{PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}}, + }, + }, + { + name: "GetSamplesForPCollectionsWithNoResult", + samples: []dataSample{ + {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, + {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, + }, + pids: []string{"pid3"}, + want: map[string][]*dataSample{}, + }, + { + name: "GetSamplesForPCollectionsTooManySamples", + samples: []dataSample{ + {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element2"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element3"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element4"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element5"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element6"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element7"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element8"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element9"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element10"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element11"), Timestamp: timestamp}, + }, + pids: []string{"pid1"}, + want: map[string][]*dataSample{ + "pid1": { + {PCollectionID: "pid1", Element: []byte("element2"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element3"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element4"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element5"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element6"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element7"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element8"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element9"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element10"), Timestamp: timestamp}, + {PCollectionID: "pid1", Element: []byte("element11"), Timestamp: timestamp}, + }}, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + dataSampler := NewDataSampler(ctx) + go dataSampler.Process() + for _, sample := range test.samples { + dataSampler.SendSample(sample.PCollectionID, sample.Element, sample.Timestamp) + } + var samplesCount = -1 + var samples map[string][]*dataSample + for i := 0; i < 5; i++ { + samples = dataSampler.GetSamples(test.pids) + if len(samples) == len(test.want) { + samplesCount = len(samples) + break + } + time.Sleep(time.Second) + } + cancel() + if samplesCount != len(test.want) { + t.Errorf("got an unexpected number of sampled elements: %v, want: %v", samplesCount, len(test.want)) + } + if !verifySampledElements(samples, test.want) { + t.Errorf("got an unexpected sampled elements: %v, want: %v", samples, test.want) + } + }) + } +} + +func verifySampledElements(samples, want map[string][]*dataSample) bool { + if len(samples) != len(want) { + return false + } + for pid, samples := range samples { + expected, ok := want[pid] + if !ok { + return false + } + sort.SliceStable(samples, func(i, j int) bool { + return string(samples[i].Element) < string(samples[j].Element) + }) + sort.SliceStable(expected, func(i, j int) bool { + return string(expected[i].Element) < string(expected[j].Element) + }) + if !reflect.DeepEqual(samples, expected) { + return false + } + } + return true +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 401cdbef7a37..674de44cf35b 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -241,6 +241,7 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { return err } // Collect the actual size of the element, and reset the bytecounter reader. + // TODO(zechenj18) 2023-12-07: currently we never sample anything from the DataSource, we need to validate CoGBKs and similar types with the sampling implementation n.PCol.addSize(int64(bcr.reset())) // Check if there's a continuation and return residuals diff --git a/sdks/go/pkg/beam/core/runtime/exec/pcollection.go b/sdks/go/pkg/beam/core/runtime/exec/pcollection.go index 3b2e3ab3bf2c..ed13c65a6f5b 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pcollection.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pcollection.go @@ -16,12 +16,14 @@ package exec import ( + "bytes" "context" "fmt" "math" "math/rand" "sync" "sync/atomic" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" ) @@ -32,19 +34,22 @@ import ( // In particular, must not be placed after a Multiplex, and must be placed // after a Flatten. type PCollection struct { - UID UnitID - PColID string - Out Node // Out is the consumer of this PCollection. - Coder *coder.Coder - Seed int64 + UID UnitID + PColID string + Out Node // Out is the consumer of this PCollection. + Coder *coder.Coder + WindowCoder *coder.WindowCoder + Seed int64 r *rand.Rand nextSampleIdx int64 // The index of the next value to sample. elementCoder ElementEncoder + windowCoder WindowEncoder elementCount int64 // must use atomic operations. sizeMu sync.Mutex sizeCount, sizeSum, sizeMin, sizeMax int64 + dataSampler *DataSampler } // ID returns the debug id for this unit. @@ -57,6 +62,7 @@ func (p *PCollection) Up(ctx context.Context) error { // dedicated rand source p.r = rand.New(rand.NewSource(p.Seed)) p.elementCoder = MakeElementEncoder(p.Coder) + p.windowCoder = MakeWindowEncoder(p.WindowCoder) return nil } @@ -93,9 +99,19 @@ func (p *PCollection) ProcessElement(ctx context.Context, elm *FullValue, values } else { p.nextSampleIdx = cur + p.r.Int63n(cur/10+2) + 1 } - var w byteCounter - p.elementCoder.Encode(elm, &w) - p.addSize(int64(w.count)) + + if p.dataSampler == nil { + var w byteCounter + p.elementCoder.Encode(elm, &w) + p.addSize(int64(w.count)) + } else { + var buf bytes.Buffer + EncodeWindowedValueHeader(p.windowCoder, elm.Windows, elm.Timestamp, elm.Pane, &buf) + winSize := buf.Len() + p.elementCoder.Encode(elm, &buf) + p.addSize(int64(buf.Len() - winSize)) + p.dataSampler.SendSample(p.PColID, buf.Bytes(), time.Now()) + } } return p.Out.ProcessElement(ctx, elm, values...) } diff --git a/sdks/go/pkg/beam/core/runtime/exec/pcollection_test.go b/sdks/go/pkg/beam/core/runtime/exec/pcollection_test.go index 1cb6adee97d6..1b702588c051 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pcollection_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pcollection_test.go @@ -30,7 +30,7 @@ import ( // randomness for the samples. func TestPCollection(t *testing.T) { a := &CaptureNode{UID: 1} - pcol := &PCollection{UID: 2, Out: a, Coder: coder.NewVarInt()} + pcol := &PCollection{UID: 2, Out: a, Coder: coder.NewVarInt(), WindowCoder: coder.NewGlobalWindow()} // The "large" 2nd value is to ensure the values are encoded properly, // and that Min & Max are behaving. inputs := []any{int64(1), int64(2000000000), int64(3)} @@ -99,7 +99,7 @@ func BenchmarkPCollection(b *testing.B) { Elm: int64(1), }}) } - pcol := &PCollection{UID: 2, Out: out, Coder: coder.NewVarInt()} + pcol := &PCollection{UID: 2, Out: out, Coder: coder.NewVarInt(), WindowCoder: coder.NewGlobalWindow()} n := &FixedRoot{UID: 3, Elements: process, Out: pcol} p, err := NewPlan("a", []Unit{n, pcol, out}) if err != nil { diff --git a/sdks/go/pkg/beam/core/runtime/exec/to_string.go b/sdks/go/pkg/beam/core/runtime/exec/to_string.go index 2196fd951806..df7050483b16 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/to_string.go +++ b/sdks/go/pkg/beam/core/runtime/exec/to_string.go @@ -45,6 +45,7 @@ func (m *ToString) ProcessElement(ctx context.Context, elm *FullValue, values .. Elm: elm.Elm, Elm2: fmt.Sprintf("%v", elm.Elm2), Timestamp: elm.Timestamp, + Pane: elm.Pane, } return m.Out.ProcessElement(ctx, &ret, values...) diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 6b3e3e457229..115fe187daa4 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -51,8 +51,8 @@ const ( ) // UnmarshalPlan converts a model bundle descriptor into an execution Plan. -func UnmarshalPlan(desc *fnpb.ProcessBundleDescriptor) (*Plan, error) { - b, err := newBuilder(desc) +func UnmarshalPlan(desc *fnpb.ProcessBundleDescriptor, dataSampler *DataSampler) (*Plan, error) { + b, err := newBuilder(desc, dataSampler) if err != nil { return nil, err } @@ -169,8 +169,9 @@ type builder struct { nodes map[string]*PCollection // PCollectionID -> Node (cache) links map[linkID]Node // linkID -> Node (cache) - units []Unit // result - idgen *GenID + units []Unit // result + idgen *GenID + dataSampler *DataSampler } // linkID represents an incoming data link to an Node. @@ -179,7 +180,7 @@ type linkID struct { input int // input index. If > 0, it's a side input. } -func newBuilder(desc *fnpb.ProcessBundleDescriptor) (*builder, error) { +func newBuilder(desc *fnpb.ProcessBundleDescriptor, dataSampler *DataSampler) (*builder, error) { // Preprocess graph structure to allow insertion of Multiplex, // Flatten and Discard. @@ -216,7 +217,8 @@ func newBuilder(desc *fnpb.ProcessBundleDescriptor) (*builder, error) { nodes: make(map[string]*PCollection), links: make(map[linkID]Node), - idgen: &GenID{}, + idgen: &GenID{}, + dataSampler: dataSampler, } return b, nil } @@ -411,11 +413,11 @@ func (b *builder) makePCollection(id string) (*PCollection, error) { } func (b *builder) newPCollectionNode(id string, out Node) (*PCollection, error) { - ec, _, err := b.makeCoderForPCollection(id) + ec, wc, err := b.makeCoderForPCollection(id) if err != nil { return nil, err } - u := &PCollection{UID: b.idgen.New(), Out: out, PColID: id, Coder: ec, Seed: rand.Int63()} + u := &PCollection{UID: b.idgen.New(), Out: out, PColID: id, Coder: ec, WindowCoder: wc, Seed: rand.Int63(), dataSampler: b.dataSampler} b.nodes[id] = u b.units = append(b.units, u) return u, nil diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate_test.go b/sdks/go/pkg/beam/core/runtime/exec/translate_test.go index c6a70fe07a02..a9917ec456fe 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate_test.go @@ -460,7 +460,7 @@ func TestUnmarshalPlan(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - plan, err := UnmarshalPlan(test.inputDesc) + plan, err := UnmarshalPlan(test.inputDesc, nil) if err != nil && test.outputError == nil { t.Errorf("there is an error where should not be. UnmarshalPlan(%v) = (%v, %v), want (%v, %v)", test.inputDesc, plan, err, test.outputPlan, test.outputError) } else if err != nil && !reflect.DeepEqual(err, test.outputError) { @@ -503,7 +503,7 @@ func TestNewBuilder(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - b, err := newBuilder(test.inputDesc) + b, err := newBuilder(test.inputDesc, nil) if err != nil && test.outputError == nil { t.Errorf("There is an error where should not be. newBuilder(%v) = (%v, %v), want (%v, %v)", test.inputDesc, b, err, test.outputBuilder, test.outputError) } else if err != nil && err != test.outputError { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index 9ef28eb7809b..3b7cfc5639cd 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -73,6 +73,7 @@ const ( URNMultiCore = "beam:protocol:multi_core_bundle_processing:v1" URNWorkerStatus = "beam:protocol:worker_status:v1" URNMonitoringInfoShortID = "beam:protocol:monitoring_info_short_ids:v1" + URNDataSampling = "beam:protocol:data_sampling:v1" URNRequiresSplittableDoFn = "beam:requirement:pardo:splittable_dofn:v1" URNRequiresBundleFinalization = "beam:requirement:pardo:finalization:v1" @@ -109,6 +110,7 @@ func goCapabilities() []string { URNMonitoringInfoShortID, URNBaseVersionGo, URNToString, + URNDataSampling, } return append(capabilities, knownStandardCoders()...) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index c5db9a85f367..6a66c81a0a60 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -30,6 +30,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/exec" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/harness/statecache" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/hooks" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" @@ -40,6 +41,7 @@ import ( "golang.org/x/sync/singleflight" "google.golang.org/grpc" "google.golang.org/protobuf/types/known/durationpb" + "google.golang.org/protobuf/types/known/timestamppb" ) // URNMonitoringInfoShortID is a URN indicating support for short monitoring info IDs. @@ -157,6 +159,11 @@ func MainWithOptions(ctx context.Context, loggingEndpoint, controlEndpoint strin runnerCapabilities: rcMap, } + if enabled, ok := rcMap[graphx.URNDataSampling]; ok && enabled { + ctrl.dataSampler = exec.NewDataSampler(ctx) + go ctrl.dataSampler.Process() + } + // if the runner supports worker status api then expose SDK harness status if opts.StatusEndpoint != "" { statusHandler, err := newWorkerStatusHandler(ctx, opts.StatusEndpoint, ctrl.cache, func(statusInfo *strings.Builder) { ctrl.metStoreToString(statusInfo) }) @@ -304,6 +311,7 @@ type control struct { // TODO(BEAM-11097): Cache is currently unused. cache *statecache.SideInputCache runnerCapabilities map[string]bool + dataSampler *exec.DataSampler } func (c *control) metStoreToString(statusInfo *strings.Builder) { @@ -345,7 +353,7 @@ func (c *control) getOrCreatePlan(bdID bundleDescriptorID) (*exec.Plan, error) { } desc = newDesc.(*fnpb.ProcessBundleDescriptor) } - newPlan, err := exec.UnmarshalPlan(desc) + newPlan, err := exec.UnmarshalPlan(desc, c.dataSampler) if err != nil { return nil, errors.WithContextf(err, "invalid bundle desc: %v\n%v\n", bdID, desc.String()) } @@ -654,7 +662,28 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe }, }, } - + case req.GetSampleData() != nil: + msg := req.GetSampleData() + var samples = make(map[string]*fnpb.SampleDataResponse_ElementList) + var elementsMap = c.dataSampler.GetSamples(msg.GetPcollectionIds()) + + for pid, elements := range elementsMap { + var elementList fnpb.SampleDataResponse_ElementList + for i := range elements { + var sampledElement = &fnpb.SampledElement{ + Element: elements[i].Element, + SampleTimestamp: timestamppb.New(elements[i].Timestamp), + } + elementList.Elements = append(elementList.Elements, sampledElement) + } + samples[pid] = &elementList + } + return &fnpb.InstructionResponse{ + InstructionId: string(instID), + Response: &fnpb.InstructionResponse_SampleData{ + SampleData: &fnpb.SampleDataResponse{ElementSamples: samples}, + }, + } default: return fail(ctx, instID, "Unexpected request: %v", req) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go index 84c5770c71a1..91dd3c591d5b 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go @@ -94,7 +94,7 @@ func invalidDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor { func TestControl_getOrCreatePlan(t *testing.T) { testBDID := bundleDescriptorID("test") - testPlan, err := exec.UnmarshalPlan(validDescriptor(t)) + testPlan, err := exec.UnmarshalPlan(validDescriptor(t), nil) if err != nil { t.Fatal("bad testPlan") } diff --git a/sdks/go/pkg/beam/core/runtime/harness/init/init.go b/sdks/go/pkg/beam/core/runtime/harness/init/init.go index 55f3d0beabdc..27d44f7029e5 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/init/init.go +++ b/sdks/go/pkg/beam/core/runtime/harness/init/init.go @@ -22,6 +22,7 @@ import ( "context" "encoding/json" "flag" + "slices" "strings" "time" @@ -31,6 +32,7 @@ import ( "runtime/debug" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/harness" // Import gcs filesystem so that it can be used to upload heap dumps. @@ -78,6 +80,13 @@ func hook() { return } + // Extract environment variables. These are optional runner supported capabilities. + // Expected env variables: + // RUNNER_CAPABILITIES : list of runner supported capability urn. + // STATUS_ENDPOINT : Endpoint to connect to status server used for worker status reporting. + statusEndpoint := os.Getenv("STATUS_ENDPOINT") + runnerCapabilities := strings.Split(os.Getenv("RUNNER_CAPABILITIES"), " ") + // Initialization logging // // We use direct output to stderr here, because it is expected that logging @@ -91,6 +100,14 @@ func hook() { os.Exit(1) } runtime.GlobalOptions.Import(opt.Options) + var experiments []string + if e, ok := opt.Options.Options["experiments"]; ok { + experiments = strings.Split(e, ",") + } + // TODO(zechenj18) 2023-12-07: Remove once the data sampling URN is properly sent in via the capabilities + if slices.Contains(experiments, "enable_data_sampling") { + runnerCapabilities = append(runnerCapabilities, graphx.URNDataSampling) + } } defer func() { @@ -120,12 +137,6 @@ func hook() { fmt.Println("Error Setting Rlimit ", err) } - // Extract environment variables. These are optional runner supported capabilities. - // Expected env variables: - // RUNNER_CAPABILITIES : list of runner supported capability urn. - // STATUS_ENDPOINT : Endpoint to connect to status server used for worker status reporting. - statusEndpoint := os.Getenv("STATUS_ENDPOINT") - runnerCapabilities := strings.Split(os.Getenv("RUNNER_CAPABILITIES"), " ") options := harness.Options{ StatusEndpoint: statusEndpoint, RunnerCapabilities: runnerCapabilities, diff --git a/sdks/go/pkg/beam/core/typex/fulltype.go b/sdks/go/pkg/beam/core/typex/fulltype.go index 41ef0ab09d22..ff5520c28617 100644 --- a/sdks/go/pkg/beam/core/typex/fulltype.go +++ b/sdks/go/pkg/beam/core/typex/fulltype.go @@ -124,7 +124,7 @@ func New(t reflect.Type, components ...FullType) FullType { if len(components) != 2 { panic(fmt.Sprintf("Invalid number of components for KV: %v, %v", t, components)) } - if isAnyNonKVComposite(components) { + if isAnyNonKVAndNonWindowedComposite(components) { panic(fmt.Sprintf("Invalid to nest composite composites inside KV: %v, %v", t, components)) } return &tree{class, t, components} @@ -169,6 +169,15 @@ func isAnyNonKVComposite(list []FullType) bool { return false } +func isAnyNonKVAndNonWindowedComposite(list []FullType) bool { + for _, t := range list { + if t.Class() == Composite && t.Type() != KVType && t.Type() != WindowedValueType { + return true + } + } + return false +} + // Convenience functions. // IsW returns true iff the type is a WindowedValue. diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job.go b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job.go index eb26071d10ec..ed706ec1a482 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job.go @@ -154,6 +154,7 @@ func Translate(ctx context.Context, p *pipepb.Pipeline, opts *JobOptions, worker return nil, err } + opts.Options.Options["experiments"] = strings.Join(opts.Experiments, ",") job := &df.Job{ ProjectId: opts.Project, Name: opts.Name, From e19ace02a28f4fa19ca3db7f0dc43e9c4f4ab3c1 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 8 Dec 2023 10:36:25 -0800 Subject: [PATCH 074/224] [#29605] Go SDK: Eagerly create timer writers to ensure is_last sent, minmize lock contention. (#29607) --- sdks/go/examples/timer_wordcap/wordcap.go | 1 - .../pkg/beam/core/runtime/harness/datamgr.go | 23 +++++++++++++++++++ .../beam/core/runtime/harness/datamgr_test.go | 12 ++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go index db64c10eb4d7..01f87edc59bd 100644 --- a/sdks/go/examples/timer_wordcap/wordcap.go +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -118,7 +118,6 @@ func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Prov // Clean up the state that has been evicted. s.ElementBag.Clear(sp) s.MinTime.Clear(sp) - s.OutputState.ClearTag(tp, tag) // Clean up the fired timer tag. (Temporary workaround for a runner bug.) } } } diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index ed57e3eca59b..0f2de99dd2ad 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -386,14 +386,31 @@ func (c *DataChannel) read(ctx context.Context) { return } + // Consolidating required timer writer creation to a optional single lock section. + type seenTimers struct { + InstID instructionID + PTransformID, FamilyID string + } + neededTimers := map[seenTimers]struct{}{} + // Each message may contain segments for multiple streams, so we // must treat each segment in isolation. We maintain a local cache // to reduce lock contention. iterateElements(c, cache, &seenLast, msg.GetTimers(), func(elm *fnpb.Elements_Timers) exec.Elements { + neededTimers[seenTimers{InstID: instructionID(elm.GetInstructionId()), PTransformID: elm.GetTransformId(), FamilyID: elm.GetTimerFamilyId()}] = struct{}{} return exec.Elements{Timers: elm.GetTimers(), PtransformID: elm.GetTransformId(), TimerFamilyID: elm.GetTimerFamilyId()} }) + // Creating a writer is necessary to ensure a "is_last" signal is returned for timers that aren't set. + if len(neededTimers) > 0 { + c.mu.Lock() + for key := range neededTimers { + c.makeTimerWriterLocked(ctx, clientID{ptransformID: key.PTransformID, instID: key.InstID}, key.FamilyID) + } + c.mu.Unlock() + } + iterateElements(c, cache, &seenLast, msg.GetData(), func(elm *fnpb.Elements_Data) exec.Elements { return exec.Elements{Data: elm.GetData(), PtransformID: elm.GetTransformId()} @@ -629,7 +646,13 @@ func (w *dataWriter) Write(p []byte) (n int, err error) { func (c *DataChannel) makeTimerWriter(ctx context.Context, id clientID, family string) *timerWriter { c.mu.Lock() defer c.mu.Unlock() + return c.makeTimerWriterLocked(ctx, id, family) +} +// makeTimerWriterLocked does the work of makeTimerWriter, but doesn't call the lock methods. +// +// c.mu must be locked when this is called. +func (c *DataChannel) makeTimerWriterLocked(ctx context.Context, id clientID, family string) *timerWriter { var m map[timerKey]*timerWriter var ok bool if m, ok = c.timerWriters[id.instID]; !ok { diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index c7f8ac5858c1..92c4d0a8f8cd 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -389,6 +389,18 @@ func TestElementChan(t *testing.T) { return elms }, wantSum: 0, wantCount: 0, + }, { + name: "SomeTimersAndADataThenReaderThenCleanup", + sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { + client.Send(&fnpb.Elements{ + Timers: []*fnpb.Elements_Timers{timerElm(1, false), timerElm(2, true)}, + Data: []*fnpb.Elements_Data{dataElm(3, true)}, + }) + elms := openChan(ctx, t, c, timerID) + c.removeInstruction(instID) + return elms + }, + wantSum: 6, wantCount: 3, }, } for _, test := range tests { From b7530a24fccedea74355c8d1fc4ce92c1b39365a Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Fri, 8 Dec 2023 12:22:25 -0800 Subject: [PATCH 075/224] Retry building a wheel up to 3 times. (#29676) --- sdks/python/build.gradle | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle index 7795e77e3963..ab6f75fc653b 100644 --- a/sdks/python/build.gradle +++ b/sdks/python/build.gradle @@ -99,11 +99,12 @@ platform_identifiers_map.each { platform, idsuffix -> environment CIBW_ENVIRONMENT: "SETUPTOOLS_USE_DISTUTILS=stdlib" // note: sync cibuildwheel version with GitHub Action // .github/workflow/build_wheel.yml:build_wheels "Install cibuildwheel" step + // note(https://github.com/pypa/cibuildwheel/issues/1692): cibuildwheel appears to timeout occasionally. executable 'sh' args '-c', ". ${envdir}/bin/activate && " + "pip install cibuildwheel==2.9.0 && " + "cibuildwheel --print-build-identifiers --platform ${platform} --archs ${archs} && " + - "cibuildwheel --output-dir ${buildDir} --platform ${platform} --archs ${archs}" + "for i in {1..3}; do cibuildwheel --output-dir ${buildDir} --platform ${platform} --archs ${archs} && break; done" } } } @@ -162,4 +163,4 @@ tasks.register("wordCount") { args '-c', ". ${envdir}/bin/activate && python -m apache_beam.examples.wordcount --runner DirectRunner --output /tmp/output.txt" } } -} \ No newline at end of file +} From d23ed6f7f039d2e21166400a18a0aae365d8ddb2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 8 Dec 2023 13:34:43 -0800 Subject: [PATCH 076/224] Bump cloud.google.com/go/bigtable from 1.20.0 to 1.21.0 in /sdks (#29677) Bumps [cloud.google.com/go/bigtable](https://github.com/googleapis/google-cloud-go) from 1.20.0 to 1.21.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.20.0...pubsub/v1.21.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 6beb93088f0c..78a92a9915e3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -24,7 +24,7 @@ go 1.20 require ( cloud.google.com/go/bigquery v1.57.1 - cloud.google.com/go/bigtable v1.20.0 + cloud.google.com/go/bigtable v1.21.0 cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 diff --git a/sdks/go.sum b/sdks/go.sum index e4bf59644599..9ac23df87813 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.57.1 h1:FiULdbbzUxWD0Y4ZGPSVCDLvqRSyCIO6zKV7E2nf5uA= cloud.google.com/go/bigquery v1.57.1/go.mod h1:iYzC0tGVWt1jqSzBHqCr3lrRn0u13E8e+AqowBsDgug= -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/bigtable v1.21.0 h1:BFN4jhkA9ULYYV2Ug7AeOtetVLnN2jKuIq5TcRc5C38= +cloud.google.com/go/bigtable v1.21.0/go.mod h1:V0sYNRtk0dgAKjyRr/MyBpHpSXqh+9P39euf820EZ74= cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= From 98a26906bfa62708bd796f4de781346e7e019e40 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 8 Dec 2023 14:35:10 -0800 Subject: [PATCH 077/224] Guard keys-values multi map input with a flag. (#29690) This is needed as some runners (e.g. Dataflow) do not gracefully return errors on unkown state read types. This flag will be set via runner capabilites in a future PR. --- .../control/RemoteExecutionTest.java | 13 +- .../fn/harness/state/MultimapSideInput.java | 120 +++++++++++------- .../harness/state/MultimapSideInputTest.java | 12 +- .../runners/worker/bundle_processor.py | 62 ++++----- 4 files changed, 115 insertions(+), 92 deletions(-) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index 6a16ca18fef9..ea23e28ddb66 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -915,10 +915,9 @@ public Coder valueCoder() { // Expect the following requests for the first bundle: // * one to read iterable side input // * one to read keys from multimap side input - // * one to attempt multimap side input bulk read // * one to read key1 iterable from multimap side input // * one to read key2 iterable from multimap side input - assertEquals(5, stateRequestHandler.receivedRequests.size()); + assertEquals(4, stateRequestHandler.receivedRequests.size()); assertEquals( stateRequestHandler.receivedRequests.get(0).getStateKey().getIterableSideInput(), BeamFnApi.StateKey.IterableSideInput.newBuilder() @@ -932,20 +931,14 @@ public Coder valueCoder() { .setTransformId(transformId) .build()); assertEquals( - stateRequestHandler.receivedRequests.get(2).getStateKey().getMultimapKeysValuesSideInput(), - BeamFnApi.StateKey.MultimapKeysValuesSideInput.newBuilder() - .setSideInputId(multimapView.getTagInternal().getId()) - .setTransformId(transformId) - .build()); - assertEquals( - stateRequestHandler.receivedRequests.get(3).getStateKey().getMultimapSideInput(), + stateRequestHandler.receivedRequests.get(2).getStateKey().getMultimapSideInput(), BeamFnApi.StateKey.MultimapSideInput.newBuilder() .setSideInputId(multimapView.getTagInternal().getId()) .setTransformId(transformId) .setKey(encode("key1")) .build()); assertEquals( - stateRequestHandler.receivedRequests.get(4).getStateKey().getMultimapSideInput(), + stateRequestHandler.receivedRequests.get(3).getStateKey().getMultimapSideInput(), BeamFnApi.StateKey.MultimapSideInput.newBuilder() .setSideInputId(multimapView.getTagInternal().getId()) .setTransformId(transformId) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java index 619eea6cc70f..ec7429fcdc0e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java @@ -54,6 +54,7 @@ public class MultimapSideInput implements MultimapView { private final Coder keyCoder; private final Coder valueCoder; private volatile Function> bulkReadResult; + private final boolean useBulkRead; public MultimapSideInput( Cache cache, @@ -62,6 +63,18 @@ public MultimapSideInput( StateKey stateKey, Coder keyCoder, Coder valueCoder) { + // TODO(robertwb): Plumb the value of useBulkRead from runner capabilities. + this(cache, beamFnStateClient, instructionId, stateKey, keyCoder, valueCoder, false); + } + + public MultimapSideInput( + Cache cache, + BeamFnStateClient beamFnStateClient, + String instructionId, + StateKey stateKey, + Coder keyCoder, + Coder valueCoder, + boolean useBulkRead) { checkArgument( stateKey.hasMultimapKeysSideInput(), "Expected MultimapKeysSideInput StateKey but received %s.", @@ -72,6 +85,7 @@ public MultimapSideInput( StateRequest.newBuilder().setInstructionId(instructionId).setStateKey(stateKey).build(); this.keyCoder = keyCoder; this.valueCoder = valueCoder; + this.useBulkRead = useBulkRead; } @Override @@ -84,62 +98,70 @@ public Iterable get() { public Iterable get(K k) { ByteString encodedKey = encodeKey(k); - if (bulkReadResult == null) { - synchronized (this) { - if (bulkReadResult == null) { - Map> bulkRead = new HashMap<>(); - StateKey bulkReadStateKey = - StateKey.newBuilder() - .setMultimapKeysValuesSideInput( - StateKey.MultimapKeysValuesSideInput.newBuilder() - .setTransformId( - keysRequest.getStateKey().getMultimapKeysSideInput().getTransformId()) - .setSideInputId( - keysRequest.getStateKey().getMultimapKeysSideInput().getSideInputId()) - .setWindow( - keysRequest.getStateKey().getMultimapKeysSideInput().getWindow())) - .build(); + if (useBulkRead) { + if (bulkReadResult == null) { + synchronized (this) { + if (bulkReadResult == null) { + Map> bulkRead = new HashMap<>(); + StateKey bulkReadStateKey = + StateKey.newBuilder() + .setMultimapKeysValuesSideInput( + StateKey.MultimapKeysValuesSideInput.newBuilder() + .setTransformId( + keysRequest + .getStateKey() + .getMultimapKeysSideInput() + .getTransformId()) + .setSideInputId( + keysRequest + .getStateKey() + .getMultimapKeysSideInput() + .getSideInputId()) + .setWindow( + keysRequest.getStateKey().getMultimapKeysSideInput().getWindow())) + .build(); - StateRequest bulkReadRequest = - keysRequest.toBuilder().setStateKey(bulkReadStateKey).build(); - try { - Iterator>> entries = - StateFetchingIterators.readAllAndDecodeStartingFrom( - Caches.subCache(cache, "ValuesForKey", encodedKey), - beamFnStateClient, - bulkReadRequest, - KvCoder.of(keyCoder, IterableCoder.of(valueCoder))) - .iterator(); - while (bulkRead.size() < BULK_READ_SIZE && entries.hasNext()) { - KV> entry = entries.next(); - bulkRead.put(encodeKey(entry.getKey()), entry.getValue()); - } - if (entries.hasNext()) { + StateRequest bulkReadRequest = + keysRequest.toBuilder().setStateKey(bulkReadStateKey).build(); + try { + Iterator>> entries = + StateFetchingIterators.readAllAndDecodeStartingFrom( + Caches.subCache(cache, "ValuesForKey", encodedKey), + beamFnStateClient, + bulkReadRequest, + KvCoder.of(keyCoder, IterableCoder.of(valueCoder))) + .iterator(); + while (bulkRead.size() < BULK_READ_SIZE && entries.hasNext()) { + KV> entry = entries.next(); + bulkRead.put(encodeKey(entry.getKey()), entry.getValue()); + } + if (entries.hasNext()) { + bulkReadResult = bulkRead::get; + } else { + bulkReadResult = + key -> { + Iterable result = bulkRead.get(key); + if (result == null) { + // As we read the entire set of values, we don't have to do a lookup to know + // this key doesn't exist. + // Missing keys are treated as empty iterables in this multimap. + return Collections.emptyList(); + } else { + return result; + } + }; + } + } catch (Exception exn) { bulkReadResult = bulkRead::get; - } else { - bulkReadResult = - key -> { - Iterable result = bulkRead.get(key); - if (result == null) { - // As we read the entire set of values, we don't have to do a lookup to know - // this key doesn't exist. - // Missing keys are treated as empty iterables in this multimap. - return Collections.emptyList(); - } else { - return result; - } - }; } - } catch (Exception exn) { - bulkReadResult = bulkRead::get; } } } - } - Iterable bulkReadValues = bulkReadResult.apply(encodedKey); - if (bulkReadValues != null) { - return bulkReadValues; + Iterable bulkReadValues = bulkReadResult.apply(encodedKey); + if (bulkReadValues != null) { + return bulkReadValues; + } } StateKey stateKey = diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java index 17ebf4234396..23a572894b40 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java @@ -69,7 +69,8 @@ public void testGetWithBulkRead() throws Exception { "instructionId", keysStateKey(), ByteArrayCoder.of(), - StringUtf8Coder.of()); + StringUtf8Coder.of(), + true); assertArrayEquals( new String[] {"A1", "A2", "A3"}, Iterables.toArray(multimapSideInput.get(A), String.class)); assertArrayEquals( @@ -94,7 +95,8 @@ public void testGet() throws Exception { "instructionId", keysStateKey(), ByteArrayCoder.of(), - StringUtf8Coder.of()); + StringUtf8Coder.of(), + true); assertArrayEquals( new String[] {"A1", "A2", "A3"}, Iterables.toArray(multimapSideInput.get(A), String.class)); assertArrayEquals( @@ -124,7 +126,8 @@ public void testGetCached() throws Exception { "instructionId", keysStateKey(), ByteArrayCoder.of(), - StringUtf8Coder.of()); + StringUtf8Coder.of(), + true); assertArrayEquals( new String[] {"A1", "A2", "A3"}, Iterables.toArray(multimapSideInput.get(A), String.class)); @@ -147,7 +150,8 @@ public void testGetCached() throws Exception { "instructionId", keysStateKey(), ByteArrayCoder.of(), - StringUtf8Coder.of()); + StringUtf8Coder.of(), + true); assertArrayEquals( new String[] {"A1", "A2", "A3"}, Iterables.toArray(multimapSideInput.get(A), String.class)); diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index b35997c4250f..02a2f6016f71 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -388,7 +388,8 @@ def __init__(self, transform_id, # type: str tag, # type: Optional[str] side_input_data, # type: pvalue.SideInputData - coder # type: WindowedValueCoder + coder, # type: WindowedValueCoder + use_bulk_read = False, # type: bool ): # type: (...) -> None self._state_handler = state_handler @@ -399,6 +400,7 @@ def __init__(self, self._target_window_coder = coder.window_coder # TODO(robertwb): Limit the cache size. self._cache = {} # type: Dict[BoundedWindow, Any] + self._use_bulk_read = use_bulk_read def __getitem__(self, window): target_window = self._side_input_data.window_mapping_fn(window) @@ -432,42 +434,44 @@ def __getitem__(self, window): key_coder = self._element_coder.key_coder() key_coder_impl = key_coder.get_impl() value_coder = self._element_coder.value_coder() + use_bulk_read = self._use_bulk_read class MultiMap(object): _bulk_read = None _lock = threading.Lock() def __getitem__(self, key): - if self._bulk_read is None: - with self._lock: - if self._bulk_read is None: - try: - # Attempt to bulk read the key-values over the iterable - # protocol which, if supported, can be much more efficient - # than point lookups if it fits into memory. - for ix, (k, vs) in enumerate(_StateBackedIterable( - state_handler, - kv_iter_state_key, - coders.TupleCoder( - (key_coder, coders.IterableCoder(value_coder))))): - cache[k] = vs - if ix > StateBackedSideInputMap._BULK_READ_LIMIT: + if use_bulk_read: + if self._bulk_read is None: + with self._lock: + if self._bulk_read is None: + try: + # Attempt to bulk read the key-values over the iterable + # protocol which, if supported, can be much more efficient + # than point lookups if it fits into memory. + for ix, (k, vs) in enumerate(_StateBackedIterable( + state_handler, + kv_iter_state_key, + coders.TupleCoder( + (key_coder, coders.IterableCoder(value_coder))))): + cache[k] = vs + if ix > StateBackedSideInputMap._BULK_READ_LIMIT: + self._bulk_read = ( + StateBackedSideInputMap._BULK_READ_PARTIALLY) + break + else: + # We reached the end of the iteration without breaking. self._bulk_read = ( - StateBackedSideInputMap._BULK_READ_PARTIALLY) - break - else: - # We reached the end of the iteration without breaking. + StateBackedSideInputMap._BULK_READ_FULLY) + except Exception: + _LOGGER.error( + "Iterable access of map side inputs unsupported.", + exc_info=True) self._bulk_read = ( - StateBackedSideInputMap._BULK_READ_FULLY) - except Exception: - _LOGGER.error( - "Iterable access of map side inputs unsupported.", - exc_info=True) - self._bulk_read = ( - StateBackedSideInputMap._BULK_READ_PARTIALLY) - - if (self._bulk_read == StateBackedSideInputMap._BULK_READ_FULLY): - return cache.get(key, []) + StateBackedSideInputMap._BULK_READ_PARTIALLY) + + if (self._bulk_read == StateBackedSideInputMap._BULK_READ_FULLY): + return cache.get(key, []) if key not in cache: keyed_state_key = beam_fn_api_pb2.StateKey() From 75bce947ceb1cab37c5a34eebc921cb246efd964 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 8 Dec 2023 12:38:20 -0800 Subject: [PATCH 078/224] Add a runner capability for multimap keys values side input reading. --- .../org/apache/beam/model/pipeline/v1/beam_runner_api.proto | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index 48f057bbc1ea..08f05fc51b69 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -1675,6 +1675,11 @@ message StandardRunnerProtocols { // https://s.apache.org/beam-fn-api-control-data-embedding CONTROL_RESPONSE_ELEMENTS_EMBEDDING = 6 [(beam_urn) = "beam:protocol:control_response_elements_embedding:v1"]; + + // Indicates that this runner can handle the multimap_keys_values_side_input + // style read of a multimap side input. + MULTIMAP_KEYS_VALUES_SIDE_INPUT = 7 + [(beam_urn) = "beam:protocol:multimap_keys_values_side_input:v1"]; } } From 4891a81a5449f941297b4ed12b9ed1cb5a9d2629 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 8 Dec 2023 17:20:22 -0800 Subject: [PATCH 079/224] [YAML] Add and cleanup documentation for several builtin transforms. (#29673) --- .../transforms/SchemaTransformProvider.java | 5 ++ .../expansion/service/ExpansionService.java | 1 + .../SqlTransformSchemaTransformProvider.java | 17 +++- .../python/apache_beam/transforms/external.py | 22 ++++- .../apache_beam/yaml/standard_providers.yaml | 14 ++- sdks/python/apache_beam/yaml/yaml_mapping.py | 22 +++++ sdks/python/apache_beam/yaml/yaml_provider.py | 90 ++++++++++++++++--- 7 files changed, 149 insertions(+), 22 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java index e542007c9a55..e73ec5d870c6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java @@ -36,6 +36,11 @@ public interface SchemaTransformProvider { /** Returns an id that uniquely represents this transform. */ String identifier(); + /** Returns a description of this transform to be used for documentation. */ + default String description() { + return ""; + } + /** * Returns the expected schema of the configuration object. Note this is distinct from the schema * of the transform itself. 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 43690c603701..7760cab64acc 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 @@ -767,6 +767,7 @@ DiscoverSchemaTransformResponse discover(DiscoverSchemaTransformRequest request) transformProvider.getAllProviders()) { SchemaTransformConfig.Builder schemaTransformConfigBuilder = SchemaTransformConfig.newBuilder(); + schemaTransformConfigBuilder.setDescription(provider.description()); schemaTransformConfigBuilder.setConfigSchema( SchemaTranslation.schemaToProto(provider.configurationSchema(), true)); schemaTransformConfigBuilder.addAllInputPcollectionNames(provider.inputCollectionNames()); diff --git a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java index 54415644152f..f032da0799d8 100644 --- a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java +++ b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java @@ -71,6 +71,21 @@ public String identifier() { return "schematransform:org.apache.beam:sql_transform:v1"; } + @Override + public String description() { + return "A transform that executes a SQL query on its input PCollections.\n\n" + + "If a single input is given, it may be referred to as `PCOLLECTION`, e.g. the query could be of the form" + + "\n\n" + + " SELECT a, sum(b) FROM PCOLLECTION" + + "\n\n" + + "If multiple inputs are given, the should be named as they are in the query, e.g." + + "\n\n" + + " SELECT a, b, c FROM pcoll_1 join pcoll_2 using (b)" + + "\n\n" + + "For more details about Beam SQL in general see " + + "[the Beam SQL documentation](https://beam.apache.org/documentation/dsls/sql/overview/)."; + } + @Override public Schema configurationSchema() { List providers = new ArrayList<>(); @@ -82,7 +97,7 @@ public Schema configurationSchema() { EnumerationType providerEnum = EnumerationType.create(providers); return Schema.of( - Schema.Field.of("query", Schema.FieldType.STRING), + Schema.Field.of("query", Schema.FieldType.STRING).withDescription("SQL query to execute"), Schema.Field.nullable( "ddl", Schema.FieldType.STRING), // TODO: Underlying builder seems more capable? Schema.Field.nullable("dialect", Schema.FieldType.logicalType(QUERY_ENUMERATION)), diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index a69ecbaee220..997cea347d33 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -41,6 +41,7 @@ from apache_beam.portability.api import beam_expansion_api_pb2_grpc from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.portability.api import external_transforms_pb2 +from apache_beam.portability.api import schema_pb2 from apache_beam.runners import pipeline_context from apache_beam.runners.portability import artifact_service from apache_beam.transforms import environments @@ -51,6 +52,7 @@ from apache_beam.typehints.schemas import named_fields_to_schema from apache_beam.typehints.schemas import named_tuple_from_schema from apache_beam.typehints.schemas import named_tuple_to_schema +from apache_beam.typehints.schemas import typing_from_runner_api from apache_beam.typehints.trivial_inference import instance_to_type from apache_beam.typehints.typehints import Union from apache_beam.typehints.typehints import UnionConstraint @@ -450,8 +452,24 @@ def discover_iter(expansion_service, ignore_errors=True): schema = named_tuple_from_schema(proto_config.config_schema) except Exception as exn: if ignore_errors: - logging.info("Bad schema for %s: %s", identifier, str(exn)[:250]) - continue + truncated_schema = schema_pb2.Schema() + truncated_schema.CopyFrom(proto_config.config_schema) + for field in truncated_schema.fields: + try: + typing_from_runner_api(field.type) + except Exception: + if field.type.nullable: + # Set it to an empty placeholder type. + field.type.CopyFrom( + schema_pb2.FieldType( + nullable=True, + row_type=schema_pb2.RowType( + schema=schema_pb2.Schema()))) + try: + schema = named_tuple_from_schema(truncated_schema) + except Exception as exn: + logging.info("Bad schema for %s: %s", identifier, str(exn)[:250]) + continue else: raise diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index c612d4412081..0f1bc14c47c4 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -18,23 +18,18 @@ # TODO(robertwb): Add more providers. # TODO(robertwb): Perhaps auto-generate this file? -- type: 'beamJar' - config: - gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' - version: BEAM_VERSION - transforms: - Sql: 'beam:external:java:sql:v1' - MapToFields-java: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" - MapToFields-generic: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" - - type: renaming transforms: + 'Sql': 'Sql' 'MapToFields-java': 'MapToFields-java' 'MapToFields-generic': 'MapToFields-java' 'Filter-java': 'Filter-java' 'Explode': 'Explode' config: mappings: + 'Sql': + query: 'query' + # Unfortunately dialect is a java logical type. 'MapToFields-generic': language: 'language' append: 'append' @@ -57,6 +52,7 @@ underlying_provider: type: beamJar transforms: + Sql: "schematransform:org.apache.beam:sql_transform:v1" MapToFields-java: "beam:schematransform:org.apache.beam:yaml:map_to_fields-java:v1" Filter-java: "beam:schematransform:org.apache.beam:yaml:filter-java:v1" Explode: "beam:schematransform:org.apache.beam:yaml:explode:v1" diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 08c7a59819a7..0ce706bbea58 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -326,6 +326,28 @@ def expand(pcoll, error_handling=None, **kwargs): class _Explode(beam.PTransform): + """Explodes (aka unnest/flatten) one or more fields producing multiple rows. + + Given one or more fields of iterable type, produces multiple rows, one for + each value of that field. For example, a row of the form `('a', [1, 2, 3])` + would expand to `('a', 1)`, `('a', 2')`, and `('a', 3)` when exploded on + the second field. + + This is akin to a `FlatMap` when paired with the MapToFields transform. + + Args: + fields: The list of fields to expand. + cross_product: If multiple fields are specified, indicates whether the + full cross-product of combinations should be produced, or if the + first element of the first field corresponds to the first element + of the second field, etc. For example, the row + `(['a', 'b'], [1, 2])` would expand to the four rows + `('a', 1)`, `('a', 2)`, `('b', 1)`, and `('b', 2)` when + `cross_product` is set to `true` but only the two rows + `('a', 1)` and `('b', 2)` when it is set to `false`. + Only meaningful (and required) if multiple rows are specified. + error_handling: Whether and how to handle errors during iteration. + """ def __init__( self, fields: Union[str, Collection[str]], diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 6a2d313183e5..f6078769c654 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -546,7 +546,7 @@ def dicts_to_rows(o): def create_builtin_provider(): - def create(elements: Iterable[Any], reshuffle: bool = True): + def create(elements: Iterable[Any], reshuffle: Optional[bool] = True): """Creates a collection containing a specified set of elements. YAML/JSON-style mappings will be interpreted as Beam rows. For example:: @@ -560,17 +560,48 @@ def create(elements: Iterable[Any], reshuffle: bool = True): Args: elements: The set of elements that should belong to the PCollection. YAML/JSON-style mappings will be interpreted as Beam rows. - reshuffle (optional): Whether to introduce a reshuffle if there is more - than one element in the collection. Defaults to True. + reshuffle (optional): Whether to introduce a reshuffle (to possibly + redistribute the work) if there is more than one element in the + collection. Defaults to True. """ - return beam.Create([element_to_rows(e) for e in elements], reshuffle) + return beam.Create([element_to_rows(e) for e in elements], + reshuffle=reshuffle is not False) # Or should this be posargs, args? # pylint: disable=dangerous-default-value def fully_qualified_named_transform( constructor: str, - args: Iterable[Any] = (), - kwargs: Mapping[str, Any] = {}): + args: Optional[Iterable[Any]] = (), + kwargs: Optional[Mapping[str, Any]] = {}): + """A Python PTransform identified by fully qualified name. + + This allows one to import, construct, and apply any Beam Python transform. + This can be useful for using transforms that have not yet been exposed + via a YAML interface. Note, however, that conversion may be required if this + transform does not accept or produce Beam Rows. + + For example, + + type: PyTransform + config: + constructor: apache_beam.pkg.mod.SomeClass + args: [1, 'foo'] + kwargs: + baz: 3 + + can be used to access the transform + `apache_beam.pkg.mod.SomeClass(1, 'foo', baz=3)`. + + Args: + constructor: Fully qualified name of a callable used to construct the + transform. Often this is a class such as + `apache_beam.pkg.mod.SomeClass` but it can also be a function or + any other callable that returns a PTransform. + args: A list of parameters to pass to the callable as positional + arguments. + kwargs: A list of parameters to pass to the callable as keyword + arguments. + """ with FullyQualifiedNamedTransform.with_filter('*'): return constructor >> FullyQualifiedNamedTransform( constructor, args, kwargs) @@ -579,6 +610,19 @@ def fully_qualified_named_transform( # exactly zero or one PCollection in yaml (as they would be interpreted as # PBegin and the PCollection itself respectively). class Flatten(beam.PTransform): + """Flattens multiple PCollections into a single PCollection. + + The elements of the resulting PCollection will be the (disjoint) union of + all the elements of all the inputs. + + Note that in YAML transforms can always take a list of inputs which will + be implicitly flattened. + """ + def __init__(self): + # Suppress the "label" argument from the superclass for better docs. + # pylint: disable=useless-parent-delegation + super().__init__() + def expand(self, pcolls): if isinstance(pcolls, beam.PCollection): pipeline_arg = {} @@ -592,6 +636,24 @@ def expand(self, pcolls): return pcolls | beam.Flatten(**pipeline_arg) class WindowInto(beam.PTransform): + # pylint: disable=line-too-long + + """A window transform assigning windows to each element of a PCollection. + + The assigned windows will affect all downstream aggregating operations, + which will aggregate by window as well as by key. + + See [the Beam documentation on windowing](https://beam.apache.org/documentation/programming-guide/#windowing) + for more details. + + Note that any Yaml transform can have a + [windowing parameter](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md#windowing), + which is applied to its inputs (if any) or outputs (if there are no inputs) + which means that explicit WindowInto operations are not typically needed. + + Args: + windowing: the type and parameters of the windowing to perform + """ def __init__(self, windowing): self._window_transform = self._parse_window_spec(windowing) @@ -617,13 +679,21 @@ def _parse_window_spec(spec): # TODO: Triggering, etc. return beam.WindowInto(window_fn) - def log_and_return(x): - logging.info(x) - return x + def LogForTesting(): + """Logs each element of its input PCollection. + + The output of this transform is a copy of its input for ease of use in + chain-style pipelines. + """ + def log_and_return(x): + logging.info(x) + return x + + return beam.Map(log_and_return) return InlineProvider({ 'Create': create, - 'LogForTesting': lambda: beam.Map(log_and_return), + 'LogForTesting': LogForTesting, 'PyTransform': fully_qualified_named_transform, 'Flatten': Flatten, 'WindowInto': WindowInto, From aafa3b81e39f1d8ab9af21c6bf498c23295c4099 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 9 Dec 2023 12:21:22 +0000 Subject: [PATCH 080/224] Mock packages instead of installing for pydocs (#29692) --- sdks/python/scripts/generate_pydoc.sh | 2 +- sdks/python/tox.ini | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index 06ad06320fcf..3c232e126ab7 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -133,7 +133,7 @@ autodoc_inherit_docstrings = False autodoc_member_order = 'bysource' autodoc_mock_imports = ["tensorrt", "cuda", "torch", "onnxruntime", "onnx", "tensorflow", "tensorflow_hub", - "tensorflow_transform", "tensorflow_metadata", "transformers"] + "tensorflow_transform", "tensorflow_metadata", "transformers", "xgboost", "datatable", "transformers"] # Allow a special section for documenting DataFrame API napoleon_custom_sections = ['Differences from pandas'] diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index e4cf09cacba4..28e282460e47 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -148,10 +148,6 @@ deps = sphinx_rtd_theme==0.4.3 docutils<0.18 Jinja2==3.0.3 # TODO(https://github.com/apache/beam/issues/21587): Sphinx version is too old. - torch - xgboost<=1.7.6 - datatable==1.0.0 - transformers commands = time {toxinidir}/scripts/generate_pydoc.sh From a96fa74f0ccb8d0c5490ba2872bfd52675a15558 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Mon, 11 Dec 2023 00:35:32 +0300 Subject: [PATCH 081/224] Pass Java SchemaTransform descriptions to Python SDK (#29606) * pipe thru schematransform descriptions --- .../schemas/transforms/SchemaTransformProvider.java | 6 +++++- .../transforms/TypedSchemaTransformProviderTest.java | 6 ++++++ ...BigQueryStorageWriteApiSchemaTransformProvider.java | 10 ++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java index e73ec5d870c6..c76d7a25e69b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java @@ -36,7 +36,11 @@ public interface SchemaTransformProvider { /** Returns an id that uniquely represents this transform. */ String identifier(); - /** Returns a description of this transform to be used for documentation. */ + /** + * Returns a description regarding the {@link SchemaTransform} represented by the {@link + * SchemaTransformProvider}. Please keep the language generic (i.e. not specific to any + * programming language). The description may be markdown formatted. + */ default String description() { return ""; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java index db7b1436a128..2b698f4f67bb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java @@ -61,6 +61,11 @@ public String identifier() { return "fake:v1"; } + @Override + public String description() { + return "Description of fake provider"; + } + @Override protected Class configurationClass() { return Configuration.class; @@ -115,6 +120,7 @@ public void testFrom() { Configuration outputConfig = ((FakeSchemaTransform) provider.from(inputConfig)).config; assertEquals("field1", outputConfig.getField1()); assertEquals(13, outputConfig.getField2().intValue()); + assertEquals("Description of fake provider", provider.description()); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java index 98cc246ce0dd..8c4edd2244b4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java @@ -98,6 +98,16 @@ public String identifier() { return String.format("beam:schematransform:org.apache.beam:bigquery_storage_write:v2"); } + @Override + public String description() { + return String.format( + "Writes data to BigQuery using the Storage Write API (https://cloud.google.com/bigquery/docs/write-api)." + + "\n\nThis expects a single PCollection of Beam Rows and outputs two dead-letter queues (DLQ) that " + + "contain failed rows. The first DLQ has tag [%s] and contains the failed rows. The second DLQ has " + + "tag [%s] and contains failed rows and along with their respective errors.", + FAILED_ROWS_TAG, FAILED_ROWS_WITH_ERRORS_TAG); + } + @Override public List inputCollectionNames() { return Collections.singletonList(INPUT_ROWS_TAG); From 75ba21ff5c34a3c71e3667879922ce642cd79183 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Mon, 11 Dec 2023 10:48:24 +0100 Subject: [PATCH 082/224] [runners-flink] #29558 emit watermark before source shutdown * emit watermark when there is no more work in bounded source reader * added position to FlinkBoundedSource checkpoint --- .../io/source/FlinkSourceReaderBase.java | 80 ++++------------ .../io/source/FlinkSourceSplitEnumerator.java | 1 - .../bounded/FlinkBoundedSourceReader.java | 92 +++++++++++++++---- .../unbounded/FlinkUnboundedSourceReader.java | 51 ++++++++++ .../io/source/FlinkSourceReaderTestBase.java | 8 +- .../bounded/FlinkBoundedSourceReaderTest.java | 18 +++- 6 files changed, 161 insertions(+), 89 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java index f0b93e0dde0f..ce4404f8ce9a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderBase.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io.source; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -42,7 +40,6 @@ import org.apache.beam.runners.flink.metrics.FlinkMetricContainerWithoutAccumulator; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.compat.FlinkSourceCompat; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; @@ -149,19 +146,11 @@ public List> snapshotState(long checkpointId) { // Add all the source splits being actively read. beamSourceReaders.forEach( (splitId, readerAndOutput) -> { - Source.Reader reader = readerAndOutput.reader; - if (reader instanceof BoundedSource.BoundedReader) { - // Sometimes users may decide to run a bounded source in streaming mode as "finite - // stream." - // For bounded source, the checkpoint granularity is the entire source split. - // So, in case of failure, all the data from this split will be consumed again. - splitsState.add(new FlinkSourceSplit<>(splitId, reader.getCurrentSource())); - } else if (reader instanceof UnboundedSource.UnboundedReader) { - // The checkpoint for unbounded sources is fine granular. - byte[] checkpointState = - getAndEncodeCheckpointMark((UnboundedSource.UnboundedReader) reader); - splitsState.add( - new FlinkSourceSplit<>(splitId, reader.getCurrentSource(), checkpointState)); + try { + splitsState.add(getReaderCheckpoint(splitId, readerAndOutput)); + } catch (IOException e) { + throw new IllegalStateException( + String.format("Failed to get checkpoint for split %d", splitId), e); } }); return splitsState; @@ -228,9 +217,17 @@ public void close() throws Exception { */ protected abstract CompletableFuture isAvailableForAliveReaders(); + /** Create {@link FlinkSourceSplit} for given {@code splitId}. */ + protected abstract FlinkSourceSplit getReaderCheckpoint( + int splitId, ReaderAndOutput readerAndOutput) throws IOException; + + /** Create {@link Source.Reader} for given {@link FlinkSourceSplit}. */ + protected abstract Source.Reader createReader(@Nonnull FlinkSourceSplit sourceSplit) + throws IOException; + // ----------------- protected helper methods for subclasses -------------------- - protected Optional createAndTrackNextReader() throws IOException { + protected final Optional createAndTrackNextReader() throws IOException { FlinkSourceSplit sourceSplit = sourceSplits.poll(); if (sourceSplit != null) { Source.Reader reader = createReader(sourceSplit); @@ -241,7 +238,7 @@ protected Optional createAndTrackNextReader() throws IOExceptio return Optional.empty(); } - protected void finishSplit(int splitIndex) throws IOException { + protected final void finishSplit(int splitIndex) throws IOException { ReaderAndOutput readerAndOutput = beamSourceReaders.remove(splitIndex); if (readerAndOutput != null) { LOG.info("Finished reading from split {}", readerAndOutput.splitId); @@ -252,7 +249,7 @@ protected void finishSplit(int splitIndex) throws IOException { } } - protected boolean checkIdleTimeoutAndMaybeStartCountdown() { + protected final boolean checkIdleTimeoutAndMaybeStartCountdown() { if (idleTimeoutMs <= 0) { idleTimeoutFuture.complete(null); } else if (!idleTimeoutCountingDown) { @@ -262,7 +259,7 @@ protected boolean checkIdleTimeoutAndMaybeStartCountdown() { return idleTimeoutFuture.isDone(); } - protected boolean noMoreSplits() { + protected final boolean noMoreSplits() { return noMoreSplits; } @@ -308,49 +305,6 @@ protected Map allReaders() { protected static void ignoreReturnValue(Object o) { // do nothing. } - // ------------------------------ private methods ------------------------------ - - @SuppressWarnings("unchecked") - private - byte[] getAndEncodeCheckpointMark(UnboundedSource.UnboundedReader reader) { - UnboundedSource source = - (UnboundedSource) reader.getCurrentSource(); - CheckpointMarkT checkpointMark = (CheckpointMarkT) reader.getCheckpointMark(); - Coder coder = source.getCheckpointMarkCoder(); - try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - coder.encode(checkpointMark, baos); - return baos.toByteArray(); - } catch (IOException ioe) { - throw new RuntimeException("Failed to encode checkpoint mark.", ioe); - } - } - - private Source.Reader createReader(@Nonnull FlinkSourceSplit sourceSplit) - throws IOException { - Source beamSource = sourceSplit.getBeamSplitSource(); - if (beamSource instanceof BoundedSource) { - return ((BoundedSource) beamSource).createReader(pipelineOptions); - } else if (beamSource instanceof UnboundedSource) { - return createUnboundedSourceReader(beamSource, sourceSplit.getSplitState()); - } else { - throw new IllegalStateException("Unknown source type " + beamSource.getClass()); - } - } - - private - Source.Reader createUnboundedSourceReader( - Source beamSource, @Nullable byte[] splitState) throws IOException { - UnboundedSource unboundedSource = - (UnboundedSource) beamSource; - Coder coder = unboundedSource.getCheckpointMarkCoder(); - if (splitState == null) { - return unboundedSource.createReader(pipelineOptions, null); - } else { - try (ByteArrayInputStream bais = new ByteArrayInputStream(splitState)) { - return unboundedSource.createReader(pipelineOptions, coder.decode(bais)); - } - } - } // -------------------- protected helper class --------------------- diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceSplitEnumerator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceSplitEnumerator.java index 292697479bcd..8ceab393533d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceSplitEnumerator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceSplitEnumerator.java @@ -121,7 +121,6 @@ public void addReader(int subtaskId) { List> splitsForSubtask = pendingSplits.remove(subtaskId); if (splitsForSubtask != null) { assignSplitsAndLog(splitsForSubtask, subtaskId); - pendingSplits.remove(subtaskId); } else { if (splitsInitialized) { LOG.info("There is no split for subtask {}. Signaling no more splits.", subtaskId); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java index b015b527aa45..a25964af809d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java @@ -18,18 +18,29 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceReaderBase; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.core.io.InputStatus; @@ -50,6 +61,8 @@ */ public class FlinkBoundedSourceReader extends FlinkSourceReaderBase> { private static final Logger LOG = LoggerFactory.getLogger(FlinkBoundedSourceReader.class); + private static final VarLongCoder LONG_CODER = VarLongCoder.of(); + private final Map consumedFromSplit = new HashMap<>(); private @Nullable Source.Reader currentReader; private int currentSplitId; @@ -62,6 +75,40 @@ public FlinkBoundedSourceReader( currentSplitId = -1; } + @Override + protected FlinkSourceSplit getReaderCheckpoint(int splitId, ReaderAndOutput readerAndOutput) + throws CoderException { + // Sometimes users may decide to run a bounded source in streaming mode as "finite + // stream." + // For bounded source, the checkpoint granularity is the entire source split. + // So, in case of failure, all the data from this split will be consumed again. + return new FlinkSourceSplit<>( + splitId, readerAndOutput.reader.getCurrentSource(), asBytes(consumedFromSplit(splitId))); + } + + @Override + protected Source.Reader createReader(@Nonnull FlinkSourceSplit sourceSplit) + throws IOException { + Source beamSource = sourceSplit.getBeamSplitSource(); + byte[] state = sourceSplit.getSplitState(); + if (state != null) { + consumedFromSplit.put(Integer.parseInt(sourceSplit.splitId()), fromBytes(state)); + } + return ((BoundedSource) beamSource).createReader(pipelineOptions); + } + + private byte[] asBytes(long l) throws CoderException { + return CoderUtils.encodeToByteArray(LONG_CODER, l); + } + + private long fromBytes(byte[] b) throws CoderException { + return CoderUtils.decodeFromByteArray(LONG_CODER, b); + } + + private long consumedFromSplit(int splitId) { + return consumedFromSplit.getOrDefault(splitId, 0L); + } + @VisibleForTesting protected FlinkBoundedSourceReader( String stepName, @@ -78,26 +125,28 @@ public InputStatus pollNext(ReaderOutput> output) throws Except checkExceptionAndMaybeThrow(); if (currentReader == null && !moveToNextNonEmptyReader()) { // Nothing to read for now. - if (noMoreSplits() && checkIdleTimeoutAndMaybeStartCountdown()) { - // All the source splits have been read and idle timeout has passed. - LOG.info( - "All splits have finished reading, and idle time {} ms has passed.", idleTimeoutMs); - return InputStatus.END_OF_INPUT; - } else { - // This reader either hasn't received NoMoreSplitsEvent yet or it is waiting for idle - // timeout. - return InputStatus.NOTHING_AVAILABLE; + if (noMoreSplits()) { + output.emitWatermark(Watermark.MAX_WATERMARK); + if (checkIdleTimeoutAndMaybeStartCountdown()) { + // All the source splits have been read and idle timeout has passed. + LOG.info( + "All splits have finished reading, and idle time {} ms has passed.", idleTimeoutMs); + return InputStatus.END_OF_INPUT; + } } + // This reader either hasn't received NoMoreSplitsEvent yet or it is waiting for idle + // timeout. + return InputStatus.NOTHING_AVAILABLE; } - Source.Reader tempCurrentReader = currentReader; - if (tempCurrentReader != null) { - T record = tempCurrentReader.getCurrent(); + if (currentReader != null) { + // make null checks happy + final @Nonnull Source.Reader splitReader = currentReader; + // store number of processed elements from this split + consumedFromSplit.compute(currentSplitId, (k, v) -> v == null ? 1 : v + 1); + T record = splitReader.getCurrent(); WindowedValue windowedValue = WindowedValue.of( - record, - tempCurrentReader.getCurrentTimestamp(), - GlobalWindow.INSTANCE, - PaneInfo.NO_FIRING); + record, splitReader.getCurrentTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); if (timestampExtractor == null) { output.collect(windowedValue); } else { @@ -107,11 +156,12 @@ public InputStatus pollNext(ReaderOutput> output) throws Except // If the advance() invocation throws exception here, the job will just fail over and read // everything again from // the beginning. So the failover granularity is the entire Flink job. - if (!invocationUtil.invokeAdvance(tempCurrentReader)) { + if (!invocationUtil.invokeAdvance(splitReader)) { finishSplit(currentSplitId); + consumedFromSplit.remove(currentSplitId); + LOG.debug("Finished reading from {}", currentSplitId); currentReader = null; currentSplitId = -1; - LOG.debug("Finished reading from {}", currentSplitId); } // Always return MORE_AVAILABLE here regardless of the availability of next record. If there // is no more @@ -138,6 +188,12 @@ private boolean moveToNextNonEmptyReader() throws IOException { if (invocationUtil.invokeStart(rao.reader)) { currentSplitId = Integer.parseInt(rao.splitId); currentReader = rao.reader; + long toSkipAfterStart = + MoreObjects.firstNonNull(consumedFromSplit.remove(currentSplitId), 0L); + @Nonnull Source.Reader reader = Preconditions.checkArgumentNotNull(currentReader); + while (toSkipAfterStart > 0 && reader.advance()) { + toSkipAfterStart--; + } return true; } else { finishSplit(Integer.parseInt(rao.splitId)); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 0a7acb669efd..7b02702e244c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.unbounded; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -25,9 +27,12 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceReaderBase; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -179,6 +184,22 @@ protected CompletableFuture isAvailableForAliveReaders() { } } + @Override + protected FlinkSourceSplit getReaderCheckpoint(int splitId, ReaderAndOutput readerAndOutput) { + // The checkpoint for unbounded sources is fine granular. + byte[] checkpointState = + getAndEncodeCheckpointMark((UnboundedSource.UnboundedReader) readerAndOutput.reader); + return new FlinkSourceSplit<>( + splitId, readerAndOutput.reader.getCurrentSource(), checkpointState); + } + + @Override + protected Source.Reader createReader(@Nonnull FlinkSourceSplit sourceSplit) + throws IOException { + Source beamSource = sourceSplit.getBeamSplitSource(); + return createUnboundedSourceReader(beamSource, sourceSplit.getSplitState()); + } + // -------------- private helper methods ---------------- private void emitRecord( @@ -274,4 +295,34 @@ private void createPendingBytesGauge(SourceReaderContext context) { return pendingBytes; }); } + + @SuppressWarnings("unchecked") + private + byte[] getAndEncodeCheckpointMark(UnboundedSource.UnboundedReader reader) { + UnboundedSource source = + (UnboundedSource) reader.getCurrentSource(); + CheckpointMarkT checkpointMark = (CheckpointMarkT) reader.getCheckpointMark(); + Coder coder = source.getCheckpointMarkCoder(); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + coder.encode(checkpointMark, baos); + return baos.toByteArray(); + } catch (IOException ioe) { + throw new RuntimeException("Failed to encode checkpoint mark.", ioe); + } + } + + private + Source.Reader createUnboundedSourceReader( + Source beamSource, @Nullable byte[] splitState) throws IOException { + UnboundedSource unboundedSource = + (UnboundedSource) beamSource; + Coder coder = unboundedSource.getCheckpointMarkCoder(); + if (splitState == null) { + return unboundedSource.createReader(pipelineOptions, null); + } else { + try (ByteArrayInputStream bais = new ByteArrayInputStream(splitState)) { + return unboundedSource.createReader(pipelineOptions, coder.decode(bais)); + } + } + } } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java index 462a1ba0153d..c635a5778b5c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSourceReaderTestBase.java @@ -363,11 +363,9 @@ public int numCollectedRecords() { } public boolean allRecordsConsumed() { - boolean allRecordsConsumed = true; - for (Source source : sources) { - allRecordsConsumed = allRecordsConsumed && ((TestSource) source).isConsumptionCompleted(); - } - return allRecordsConsumed; + return sources.stream() + .map(TestSource.class::cast) + .allMatch(TestSource::isConsumptionCompleted); } public boolean allTimestampReceived() { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java index 84cb2a72ddaf..022f1abde826 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.verify; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -34,6 +35,7 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; @@ -62,6 +64,20 @@ public void testPollWithIdleTimeout() throws Exception { } } + @Test + public void testPollEmitsMaxWatermark() throws Exception { + ManuallyTriggeredScheduledExecutorService executor = + new ManuallyTriggeredScheduledExecutorService(); + ReaderOutput>> mockReaderOutput = + Mockito.mock(ReaderOutput.class); + try (FlinkBoundedSourceReader> reader = + (FlinkBoundedSourceReader>) createReader(executor, Long.MAX_VALUE)) { + reader.notifyNoMoreSplits(); + assertEquals(InputStatus.NOTHING_AVAILABLE, reader.pollNext(mockReaderOutput)); + verify(mockReaderOutput).emitWatermark(Watermark.MAX_WATERMARK); + } + } + @Test public void testPollWithoutIdleTimeout() throws Exception { ReaderOutput>> mockReaderOutput = @@ -107,8 +123,6 @@ public void testSnapshotStateAndRestore() throws Exception { snapshot = reader.snapshotState(0L); } - // Create a new validating output because the first split will be consumed from very beginning. - validatingOutput = new RecordsValidatingOutput(splits); // Create another reader, add the snapshot splits back. try (SourceReader>, FlinkSourceSplit>> reader = createReader()) { From 1061fcf0969676652e43617d34c53bc5bc2a7c82 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Dec 2023 09:04:49 -0500 Subject: [PATCH 083/224] Bump github.com/nats-io/nats-server/v2 from 2.10.6 to 2.10.7 in /sdks (#29698) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.6 to 2.10.7. - [Release notes](https://github.com/nats-io/nats-server/releases) - [Changelog](https://github.com/nats-io/nats-server/blob/main/.goreleaser.yml) - [Commits](https://github.com/nats-io/nats-server/compare/v2.10.6...v2.10.7) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 78a92a9915e3..ce83a6aa1009 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,7 +45,7 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 - github.com/nats-io/nats-server/v2 v2.10.6 + github.com/nats-io/nats-server/v2 v2.10.7 github.com/nats-io/nats.go v1.31.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.0 @@ -146,7 +146,7 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect - github.com/klauspost/compress v1.17.3 // indirect + github.com/klauspost/compress v1.17.4 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect github.com/magiconair/properties v1.8.7 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 9ac23df87813..847cf80e03df 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -321,8 +321,8 @@ github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.3 h1:qkRjuerhUU1EmXLYGkSH6EZL+vPSxIrYjLNAK4slzwA= -github.com/klauspost/compress v1.17.3/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= +github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -366,8 +366,8 @@ github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7P github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= github.com/nats-io/jwt/v2 v2.5.3 h1:/9SWvzc6hTfamcgXJ3uYRpgj+QuY2aLNqRiqrKcrpEo= github.com/nats-io/jwt/v2 v2.5.3/go.mod h1:iysuPemFcc7p4IoYots3IuELSI4EDe9Y0bQMe+I3Bf4= -github.com/nats-io/nats-server/v2 v2.10.6 h1:40U3ngyAKyC1tNT4Kw7PjuvivY74NTYD3qyIHxZUHKQ= -github.com/nats-io/nats-server/v2 v2.10.6/go.mod h1:IrTXS8o4Roa3G2kW8L5mEtSdmSrFjKhYb/m2g0gQ/vc= +github.com/nats-io/nats-server/v2 v2.10.7 h1:f5VDy+GMu7JyuFA0Fef+6TfulfCs5nBTgq7MMkFJx5Y= +github.com/nats-io/nats-server/v2 v2.10.7/go.mod h1:V2JHOvPiPdtfDXTuEUsthUnCvSDeFrK4Xn9hRo6du7c= github.com/nats-io/nats.go v1.31.0 h1:/WFBHEc/dOKBF6qf1TZhrdEfTmOZ5JzdJ+Y3m6Y/p7E= github.com/nats-io/nats.go v1.31.0/go.mod h1:di3Bm5MLsoB4Bx61CBTsxuarI36WbhAwOm8QrW39+i8= github.com/nats-io/nkeys v0.4.6 h1:IzVe95ru2CT6ta874rt9saQRkWfe2nFj1NtvYSLqMzY= From 5c7ec723a44d085b89f28d2af6c387375eec498b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Dec 2023 09:26:10 -0500 Subject: [PATCH 084/224] Bump github.com/aws/aws-sdk-go-v2/config from 1.26.0 to 1.26.1 in /sdks (#29702) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.26.0 to 1.26.1. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/config/v1.26.0...config/v1.26.1) --- 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 | 12 ++++++------ sdks/go.sum | 24 ++++++++++++------------ 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index ce83a6aa1009..11bd34cf079b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,8 +31,8 @@ require ( cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.35.1 github.com/aws/aws-sdk-go-v2 v1.24.0 - github.com/aws/aws-sdk-go-v2/config v1.26.0 - github.com/aws/aws-sdk-go-v2/credentials v1.16.11 + github.com/aws/aws-sdk-go-v2/config v1.26.1 + github.com/aws/aws-sdk-go-v2/credentials v1.16.12 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 github.com/aws/smithy-go v1.19.0 @@ -109,15 +109,15 @@ require ( github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.18.4 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.4 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 // 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 847cf80e03df..df82d0afbce1 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,11 +86,11 @@ github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0o github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.26.0 h1:uItWWbD/FmHPGSa6GJFyZJD/RPakVjS0fmoq1vccjNw= -github.com/aws/aws-sdk-go-v2/config v1.26.0/go.mod h1:8Rf77VTcX9MMkoMIsCnuwmef+Y1bs2Zhvw9IXHdD/Po= +github.com/aws/aws-sdk-go-v2/config v1.26.1 h1:z6DqMxclFGL3Zfo+4Q0rLnAZ6yVkzCRxhRMsiRQnD1o= +github.com/aws/aws-sdk-go-v2/config v1.26.1/go.mod h1:ZB+CuKHRbb5v5F0oJtGdhFTelmrxd4iWO1lf0rQwSAg= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.11 h1:Gcut3tJSU7F/C5W/NnFimqnJqljF58rmaw7QlbigN3U= -github.com/aws/aws-sdk-go-v2/credentials v1.16.11/go.mod h1:CysUbSCfqvEbEQTd9Ubg2RrJy2EFM+AUHJOqqj0guTo= +github.com/aws/aws-sdk-go-v2/credentials v1.16.12 h1:v/WgB8NxprNvr5inKIiVVrXPuuTegM+K8nncFkr1usU= +github.com/aws/aws-sdk-go-v2/credentials v1.16.12/go.mod h1:X21k0FjEJe+/pauud82HYiQbEr9jRKY3kXEIQ4hXeTQ= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= @@ -102,8 +102,8 @@ github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VD github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9/go.mod h1:hqamLz7g1/4EJP+GH5NBhcUMLjW+gKLQabgyz6/7WAU= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= -github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= -github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= +github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2 h1:GrSw8s0Gs/5zZ0SX+gX4zQjRnRsMJDJ2sLur1gRBhEM= +github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= @@ -121,13 +121,13 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.4 h1:2UVO4N/polvKeP+yCA8TLEmidEKxmNTeVpsZnj/bbgA= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.4/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4 h1:3JXkQ1F5n73qTpSPas6AQ8/6HFksgnB24JlNPLt3SlM= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.4/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 h1:ldSFWz9tEHAwHNmjx2Cvy1MjP5/L9kNoR0skc6wyOOM= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.4 h1:gaRFldXhoT36jVMfQ+AjAYwSfjO5LMgy1u0ObcKFhhc= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.4/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 h1:5UYvv8JUvllZsRnfrcMQ+hJ9jNICmcgKPAO1CER25Wg= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.5/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From 218af9de868198c60ed210cc388a663a80f3423b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 11 Dec 2023 10:44:00 -0500 Subject: [PATCH 085/224] Add per test timeout in recently changed dataflow legacy worker tests (#29696) --- .../beam/runners/dataflow/worker/BatchDataflowWorkerTest.java | 2 ++ .../runners/dataflow/worker/DataflowWorkUnitClientTest.java | 3 +++ .../runners/dataflow/worker/StreamingDataflowWorkerTest.java | 1 - .../dataflow/worker/StreamingModeExecutionContextTest.java | 4 +++- .../dataflow/worker/StreamingSideInputDoFnRunnerTest.java | 4 +++- .../dataflow/worker/StreamingSideInputFetcherTest.java | 4 +++- .../dataflow/worker/StreamingStepMetricsContainerTest.java | 4 +++- .../worker/logging/DataflowWorkerLoggingInitializerTest.java | 2 ++ .../dataflow/worker/streaming/ActiveWorkStateTest.java | 4 +++- .../dataflow/worker/streaming/WeightBoundedQueueTest.java | 3 +++ .../worker/streaming/sideinput/SideInputStateFetcherTest.java | 3 +++ .../worker/windmill/client/WindmillStreamPoolTest.java | 3 +++ .../windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java | 2 ++ .../worker/windmill/client/grpc/GrpcWindmillServerTest.java | 3 ++- .../windmill/client/grpc/StreamingEngineClientTest.java | 2 ++ .../worker/windmill/client/grpc/WindmillStreamSenderTest.java | 2 ++ .../worker/windmill/state/WindmillStateCacheTest.java | 4 +++- .../worker/windmill/state/WindmillStateInternalsTest.java | 4 +++- .../worker/windmill/state/WindmillStateReaderTest.java | 3 +++ .../work/budget/EvenGetWorkBudgetDistributorTest.java | 2 ++ .../windmill/work/budget/GetWorkBudgetRefresherTest.java | 3 +++ .../worker/windmill/work/budget/GetWorkBudgetTest.java | 3 +++ 22 files changed, 56 insertions(+), 9 deletions(-) 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 b4f544129db6..e33412a19d94 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 @@ -50,6 +50,7 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -61,6 +62,7 @@ @RunWith(JUnit4.class) public class BatchDataflowWorkerTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public FastNanoClockAndSleeper clockAndSleeper = new FastNanoClockAndSleeper(); @Mock WorkUnitClient mockWorkUnitClient; @Mock DataflowWorkProgressUpdater mockProgressUpdater; 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 3c63f3cc19d2..5329fb0f601c 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 @@ -50,6 +50,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TestRule; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -60,6 +61,8 @@ /** Unit tests for {@link DataflowWorkUnitClient}. */ @RunWith(JUnit4.class) public class DataflowWorkUnitClientTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + 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"; 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 6826607513d9..2793cdd81820 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 @@ -195,7 +195,6 @@ // released (2.11.0) @SuppressWarnings("unused") public class StreamingDataflowWorkerTest { - private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorkerTest.class); private static final IntervalWindow DEFAULT_WINDOW = new IntervalWindow(new Instant(1234), Duration.millis(1000)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 9991520d593b..60ecaa3e37e0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -73,7 +73,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -83,7 +85,7 @@ /** Tests for {@link StreamingModeExecutionContext}. */ @RunWith(JUnit4.class) public class StreamingModeExecutionContextTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock private SideInputStateFetcher sideInputStateFetcher; @Mock private WindmillStateReader stateReader; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 3c121ab27f76..07c1080d8f23 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -69,7 +69,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -79,7 +81,7 @@ /** Unit tests for {@link StreamingSideInputDoFnRunner}. */ @RunWith(JUnit4.class) public class StreamingSideInputDoFnRunnerTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10)); static TupleTag mainOutputTag = new TupleTag<>(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index a7196613fbb1..d4fee95ead54 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -56,7 +56,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -65,7 +67,7 @@ /** Tests for {@link StreamingSideInputFetcher}. */ @RunWith(JUnit4.class) public class StreamingSideInputFetcherTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10)); static TupleTag mainOutputTag = new TupleTag<>(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java index 9e6d45a2351b..9be65c198ac7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java @@ -37,7 +37,9 @@ import org.apache.beam.sdk.metrics.NoOpCounter; import org.apache.beam.sdk.metrics.NoOpHistogram; import org.apache.beam.sdk.util.HistogramData; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -47,7 +49,7 @@ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class StreamingStepMetricsContainerTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private MetricsContainerRegistry registry = StreamingStepMetricsContainer.createRegistry(); private MetricsContainer c1 = registry.getContainer("s1"); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java index c1b134cafa1c..425b2140a962 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -55,6 +55,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.LoggerFactory; @@ -68,6 +69,7 @@ */ @RunWith(JUnit4.class) public class DataflowWorkerLoggingInitializerTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public TemporaryFolder logFolder = new TemporaryFolder(); @Rule public RestoreSystemProperties restoreProperties = new RestoreSystemProperties(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 12ae816de829..540166f226a6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -41,13 +41,15 @@ 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.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class ActiveWorkStateTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private final WindmillStateCache.ForComputation computationStateCache = mock(WindmillStateCache.ForComputation.class); private Map> readOnlyActiveWork; 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 index b2d98fb0e954..4f035c88774c 100644 --- 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 @@ -22,12 +22,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class WeightBoundedQueueTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final int MAX_WEIGHT = 10; @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java index daf814618791..1e188da2dd63 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/sideinput/SideInputStateFetcherTest.java @@ -51,7 +51,9 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -63,6 +65,7 @@ @SuppressWarnings("deprecation") @RunWith(JUnit4.class) public class SideInputStateFetcherTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String STATE_FAMILY = "state"; @Mock private MetricTrackingWindmillServerStub server; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java index 264540531bf8..a2f5e71d04c3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java @@ -30,12 +30,15 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class WindmillStreamPoolTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final int DEFAULT_NUM_STREAMS = 10; private static final int NEW_STREAM_HOLDS = 2; private final ConcurrentHashMap< diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java index e3b07bf7aa4d..253d6ff3a48f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java @@ -55,12 +55,14 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mockito; @RunWith(JUnit4.class) public class GrpcGetWorkerMetadataStreamTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String IPV6_ADDRESS_1 = "2001:db8:0000:bac5:0000:0000:fed0:81a2"; private static final String IPV6_ADDRESS_2 = "2001:db8:0000:bac5:0000:0000:fed0:82a3"; private static final List DIRECT_PATH_ENDPOINTS = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index d9f4b72716cb..5f8a452a0433 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -94,6 +94,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ErrorCollector; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.Logger; @@ -105,7 +106,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class GrpcWindmillServerTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServerTest.class); private static final int STREAM_CHUNK_SIZE = 2 << 20; private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index 8a2c643a5b76..46983a618e4a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -66,11 +66,13 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class StreamingEngineClientTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final WindmillServiceAddress DEFAULT_WINDMILL_SERVICE_ADDRESS = WindmillServiceAddress.create(HostAndPort.fromParts(WindmillChannelFactory.LOCALHOST, 443)); private static final ImmutableMap DEFAULT = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java index c8d2974f923d..5fd1814e511f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java @@ -43,11 +43,13 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class WindmillStreamSenderTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final GetWorkRequest GET_WORK_REQUEST = GetWorkRequest.newBuilder().setClientId(1L).setJobId("job").setProjectId("project").build(); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java index cc6633f1b704..35d01aaffb8a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java @@ -35,14 +35,16 @@ import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache}. */ @RunWith(JUnit4.class) public class WindmillStateCacheTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String COMPUTATION = "computation"; private static final long SHARDING_KEY = 123; private static final WindmillComputationKey COMPUTATION_KEY = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 8971c39ccaa1..d2590ceb8466 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -98,7 +98,9 @@ import org.joda.time.Instant; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -112,7 +114,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class WindmillStateInternalsTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); public static final Range FULL_ORDERED_LIST_RANGE = Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS); private static final StateNamespace NAMESPACE = new StateNamespaceForTest("ns"); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 430e31ee04ff..7ef74639bb55 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -54,7 +54,9 @@ import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -69,6 +71,7 @@ "FutureReturnValueIgnored", }) public class WindmillStateReaderTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final VarIntCoder INT_CODER = VarIntCoder.of(); private static final String COMPUTATION = "computation"; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index 14da55fe2389..54b605efbf3c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -40,11 +40,13 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class EvenGetWorkBudgetDistributorTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); private ManagedChannel inProcessChannel; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetRefresherTest.java index fd85410cc91d..101e111cb657 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetRefresherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetRefresherTest.java @@ -21,13 +21,16 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mockito; @RunWith(JUnit4.class) public class GetWorkBudgetRefresherTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final int WAIT_BUFFER = 10; private final Runnable redistributeBudget = Mockito.mock(Runnable.class); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java index 97789abaaa97..2d2806bef618 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetTest.java @@ -19,12 +19,15 @@ import static org.junit.Assert.assertEquals; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class GetWorkBudgetTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void testCreateWithNoBudget() { From ccd93a95a04edbd49dd48dcb5d2633f429bc48f2 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 8 Dec 2023 14:20:30 -0800 Subject: [PATCH 086/224] Plumb runner capabilities to Python workers. --- sdks/python/apache_beam/portability/common_urns.py | 2 ++ .../portability/fn_api_runner/worker_handlers.py | 9 ++++++++- .../apache_beam/runners/worker/bundle_processor.py | 10 +++++++++- .../runners/worker/bundle_processor_test.py | 6 +++--- .../apache_beam/runners/worker/log_handler_test.py | 2 +- sdks/python/apache_beam/runners/worker/sdk_worker.py | 6 ++++++ .../apache_beam/runners/worker/sdk_worker_main.py | 10 +++++----- .../apache_beam/runners/worker/sdk_worker_test.py | 8 ++++---- sdks/python/gen_protos.py | 1 - 9 files changed, 38 insertions(+), 16 deletions(-) diff --git a/sdks/python/apache_beam/portability/common_urns.py b/sdks/python/apache_beam/portability/common_urns.py index 3799af5d2e1b..e7b086c5a649 100644 --- a/sdks/python/apache_beam/portability/common_urns.py +++ b/sdks/python/apache_beam/portability/common_urns.py @@ -34,6 +34,7 @@ StandardPTransforms = beam_runner_api_pb2_urns.StandardPTransforms StandardRequirements = beam_runner_api_pb2_urns.StandardRequirements StandardResourceHints = beam_runner_api_pb2_urns.StandardResourceHints +StandardRunnerProtocols = beam_runner_api_pb2_urns.StandardRunnerProtocols StandardSideInputTypes = beam_runner_api_pb2_urns.StandardSideInputTypes StandardUserStateTypes = beam_runner_api_pb2_urns.StandardUserStateTypes ExpansionMethods = external_transforms_pb2_urns.ExpansionMethods @@ -73,6 +74,7 @@ monitoring_info_labels = MonitoringInfo.MonitoringInfoLabels protocols = StandardProtocols.Enum +runner_protocols = StandardRunnerProtocols.Enum requirements = StandardRequirements.Enum displayData = StandardDisplayData.DisplayData diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index b0b4b1957dd8..b0421a6e43af 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -87,6 +87,10 @@ # Time-based flush is enabled in the fn_api_runner by default. DATA_BUFFER_TIME_LIMIT_MS = 1000 +FNAPI_RUNNER_CAPABILITIES = frozenset([ + common_urns.runner_protocols.MULTIMAP_KEYS_VALUES_SIDE_INPUT.urn, +]) + _LOGGER = logging.getLogger(__name__) T = TypeVar('T') @@ -363,6 +367,7 @@ def __init__(self, self.data_conn = self.data_plane_handler state_cache = StateCache(STATE_CACHE_SIZE_MB * MB_TO_BYTES) self.bundle_processor_cache = sdk_worker.BundleProcessorCache( + FNAPI_RUNNER_CAPABILITIES, SingletonStateHandlerFactory( sdk_worker.GlobalCachingStateHandler(state_cache, state)), data_plane.InMemoryDataChannelFactory( @@ -433,6 +438,7 @@ def GetProvisionInfo(self, request, context=None): info.control_endpoint.CopyFrom(worker.control_api_service_descriptor()) else: info = self._base_info + info.runner_capabilities[:] = FNAPI_RUNNER_CAPABILITIES return beam_provision_api_pb2.GetProvisionInfoResponse(info=info) @@ -663,7 +669,8 @@ def start_worker(self): self.control_address, state_cache_size=self._state_cache_size, data_buffer_time_limit_ms=self._data_buffer_time_limit_ms, - worker_id=self.worker_id) + worker_id=self.worker_id, + runner_capabilities=FNAPI_RUNNER_CAPABILITIES) self.worker_thread = threading.Thread( name='run_worker', target=self.worker.run) self.worker_thread.daemon = True diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 02a2f6016f71..9f51c62b83a0 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -920,6 +920,7 @@ class BundleProcessor(object): """ A class for processing bundles of elements. """ def __init__(self, + runner_capabilities, # type: FrozenSet[str] process_bundle_descriptor, # type: beam_fn_api_pb2.ProcessBundleDescriptor state_handler, # type: sdk_worker.CachingStateHandler data_channel_factory, # type: data_plane.DataChannelFactory @@ -930,11 +931,14 @@ def __init__(self, """Initialize a bundle processor. Args: + runner_capabilities (``FrozenSet[str]``): The set of capabilities of the + runner with which we will be interacting process_bundle_descriptor (``beam_fn_api_pb2.ProcessBundleDescriptor``): a description of the stage that this ``BundleProcessor``is to execute. state_handler (CachingStateHandler). data_channel_factory (``data_plane.DataChannelFactory``). """ + self.runner_capabilities = runner_capabilities self.process_bundle_descriptor = process_bundle_descriptor self.state_handler = state_handler self.data_channel_factory = data_channel_factory @@ -976,12 +980,14 @@ def create_execution_tree( ): # type: (...) -> collections.OrderedDict[str, operations.DoOperation] transform_factory = BeamTransformFactory( + self.runner_capabilities, descriptor, self.data_channel_factory, self.counter_factory, self.state_sampler, self.state_handler, - self.data_sampler) + self.data_sampler, + ) self.timers_info = transform_factory.extract_timers_info() @@ -1267,6 +1273,7 @@ class ExecutionContext: class BeamTransformFactory(object): """Factory for turning transform_protos into executable operations.""" def __init__(self, + runner_capabilities, # type: FrozenSet[str] descriptor, # type: beam_fn_api_pb2.ProcessBundleDescriptor data_channel_factory, # type: data_plane.DataChannelFactory counter_factory, # type: counters.CounterFactory @@ -1274,6 +1281,7 @@ def __init__(self, state_handler, # type: sdk_worker.CachingStateHandler data_sampler, # type: Optional[data_sampler.DataSampler] ): + self.runner_capabilities = runner_capabilities self.descriptor = descriptor self.data_channel_factory = data_channel_factory self.counter_factory = counter_factory diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor_test.py b/sdks/python/apache_beam/runners/worker/bundle_processor_test.py index 292b8431063c..dafb4dbd4bf0 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor_test.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor_test.py @@ -267,7 +267,7 @@ def test_disabled_by_default(self): """ descriptor = beam_fn_api_pb2.ProcessBundleDescriptor() descriptor.pcollections['a'].unique_name = 'a' - _ = BundleProcessor(descriptor, None, None) + _ = BundleProcessor(set(), descriptor, None, None) self.assertEqual(len(descriptor.transforms), 0) def test_can_sample(self): @@ -301,7 +301,7 @@ def test_can_sample(self): # Create and process a fake bundle. The instruction id doesn't matter # here. processor = BundleProcessor( - descriptor, None, None, data_sampler=data_sampler) + set(), descriptor, None, None, data_sampler=data_sampler) processor.process_bundle('instruction_id') samples = data_sampler.wait_for_samples([PCOLLECTION_ID]) @@ -377,7 +377,7 @@ def test_can_sample_exceptions(self): # Create and process a fake bundle. The instruction id doesn't matter # here. processor = BundleProcessor( - descriptor, None, None, data_sampler=data_sampler) + set(), descriptor, None, None, data_sampler=data_sampler) with self.assertRaisesRegex(RuntimeError, 'expected exception'): processor.process_bundle('instruction_id') diff --git a/sdks/python/apache_beam/runners/worker/log_handler_test.py b/sdks/python/apache_beam/runners/worker/log_handler_test.py index 9eb9299cac39..2cf7dff9d57f 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler_test.py +++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py @@ -286,7 +286,7 @@ def test_extracts_transform_id_during_exceptions(self): # Create and process a fake bundle. The instruction id doesn't matter # here. - processor = BundleProcessor(descriptor, None, None) + processor = BundleProcessor(set(), descriptor, None, None) with self.assertRaisesRegex(RuntimeError, 'expected exception'): processor.process_bundle('instruction_id') diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index bfd6544d802b..b55f505a6adc 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -174,6 +174,7 @@ def __init__( # Unrecoverable SDK harness initialization error (if any) # that should be reported to the runner when proocessing the first bundle. deferred_exception=None, # type: Optional[Exception] + runner_capabilities=frozenset(), # type: FrozenSet[str] ): # type: (...) -> None self._alive = True @@ -202,6 +203,7 @@ def __init__( self._state_cache, credentials) self._profiler_factory = profiler_factory self.data_sampler = data_sampler + self.runner_capabilities = runner_capabilities def default_factory(id): # type: (str) -> beam_fn_api_pb2.ProcessBundleDescriptor @@ -212,6 +214,7 @@ def default_factory(id): self._fns = KeyedDefaultDict(default_factory) # BundleProcessor cache across all workers. self._bundle_processor_cache = BundleProcessorCache( + self.runner_capabilities, state_handler_factory=self._state_handler_factory, data_channel_factory=self._data_channel_factory, fns=self._fns, @@ -419,12 +422,14 @@ class BundleProcessorCache(object): def __init__( self, + runner_capabilities, # type: FrozenSet[str] state_handler_factory, # type: StateHandlerFactory data_channel_factory, # type: data_plane.DataChannelFactory fns, # type: MutableMapping[str, beam_fn_api_pb2.ProcessBundleDescriptor] data_sampler=None, # type: Optional[data_sampler.DataSampler] ): # type: (...) -> None + self.runner_capabilities = runner_capabilities self.fns = fns self.state_handler_factory = state_handler_factory self.data_channel_factory = data_channel_factory @@ -485,6 +490,7 @@ def get(self, instruction_id, bundle_descriptor_id): # Make sure we instantiate the processor while not holding the lock. processor = bundle_processor.BundleProcessor( + self.runner_capabilities, self.fns[bundle_descriptor_id], self.state_handler_factory.create_state_handler( self.fns[bundle_descriptor_id].state_api_service_descriptor), diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py index 1af0071edc14..cd49c69a80aa 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py @@ -103,10 +103,9 @@ def create_harness(environment, dry_run=False): pickle_library = sdk_pipeline_options.view_as(SetupOptions).pickle_library pickler.set_library(pickle_library) - if 'SEMI_PERSISTENT_DIRECTORY' in environment: - semi_persistent_directory = environment['SEMI_PERSISTENT_DIRECTORY'] - else: - semi_persistent_directory = None + semi_persistent_directory = environment.get('SEMI_PERSISTENT_DIRECTORY', None) + runner_capabilities = frozenset( + environment.get('RUNNER_CAPABILITIES', '').split()) _LOGGER.info('semi_persistent_directory: %s', semi_persistent_directory) _worker_id = environment.get('WORKER_ID', None) @@ -167,7 +166,8 @@ def create_harness(environment, dry_run=False): sdk_pipeline_options.view_as(ProfilingOptions)), enable_heap_dump=enable_heap_dump, data_sampler=data_sampler, - deferred_exception=deferred_exception) + deferred_exception=deferred_exception, + runner_capabilities=runner_capabilities) return fn_log_handler, sdk_harness, sdk_pipeline_options diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py index 8570c5a7722c..4e202910345c 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py @@ -126,7 +126,7 @@ def test_fn_registration(self): def test_inactive_bundle_processor_returns_empty_progress_response(self): bundle_processor = mock.MagicMock() - bundle_processor_cache = BundleProcessorCache(None, None, {}) + bundle_processor_cache = BundleProcessorCache(None, None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) split_request = beam_fn_api_pb2.InstructionRequest( @@ -153,7 +153,7 @@ def test_inactive_bundle_processor_returns_empty_progress_response(self): def test_failed_bundle_processor_returns_failed_progress_response(self): bundle_processor = mock.MagicMock() - bundle_processor_cache = BundleProcessorCache(None, None, {}) + bundle_processor_cache = BundleProcessorCache(None, None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) @@ -172,7 +172,7 @@ def test_failed_bundle_processor_returns_failed_progress_response(self): def test_inactive_bundle_processor_returns_empty_split_response(self): bundle_processor = mock.MagicMock() - bundle_processor_cache = BundleProcessorCache(None, None, {}) + bundle_processor_cache = BundleProcessorCache(None, None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) split_request = beam_fn_api_pb2.InstructionRequest( @@ -258,7 +258,7 @@ def test_harness_monitoring_infos_and_metadata(self): def test_failed_bundle_processor_returns_failed_split_response(self): bundle_processor = mock.MagicMock() - bundle_processor_cache = BundleProcessorCache(None, None, {}) + bundle_processor_cache = BundleProcessorCache(None, None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index 2b488af0afb5..a2cd1bd4cef3 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -527,7 +527,6 @@ def generate_proto_files(force=False): generate_init_files_lite(PYTHON_OUTPUT_PATH) for proto_package in proto_packages: generate_urn_files(proto_package, PYTHON_OUTPUT_PATH) - generate_init_files_full(PYTHON_OUTPUT_PATH) From 5c879cc5c047f44a7c939b09d2a49e331ed4a8f4 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 8 Dec 2023 14:33:05 -0800 Subject: [PATCH 087/224] Toggle use of bulk multimap state read on runner capabilities. --- .../org/apache/beam/fn/harness/FnApiDoFnRunner.java | 4 ++++ .../beam/fn/harness/state/FnApiStateAccessor.java | 12 +++++++++++- .../beam/fn/harness/state/MultimapSideInput.java | 11 ----------- .../apache_beam/runners/worker/bundle_processor.py | 7 +++++-- 4 files changed, 20 insertions(+), 14 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index ddf52125b2e4..19c13775684e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -168,6 +169,7 @@ static class Factory runner = new FnApiDoFnRunner<>( context.getPipelineOptions(), + context.getRunnerCapabilities(), context.getShortIdMap(), context.getBeamFnStateClient(), context.getPTransformId(), @@ -336,6 +338,7 @@ static class Factory runnerCapabilities, ShortIdMap shortIds, BeamFnStateClient beamFnStateClient, String pTransformId, @@ -740,6 +743,7 @@ private ByteString encodeProgress(double value) throws IOException { this.stateAccessor = new FnApiStateAccessor( pipelineOptions, + runnerCapabilities, pTransformId, processBundleInstructionId, cacheTokens, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java index 69d9a1ff6c8e..204c491dc102 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; import org.apache.beam.fn.harness.Cache; @@ -33,7 +34,9 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest.CacheToken; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; +import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.SideInputReader; +import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -74,6 +77,7 @@ }) public class FnApiStateAccessor implements SideInputReader, StateBinder { private final PipelineOptions pipelineOptions; + private final Set runnerCapabilites; private final Map stateKeyObjectCache; private final Map, SideInputSpec> sideInputSpecMap; private final BeamFnStateClient beamFnStateClient; @@ -91,6 +95,7 @@ public class FnApiStateAccessor implements SideInputReader, StateBinder { public FnApiStateAccessor( PipelineOptions pipelineOptions, + Set runnerCapabilites, String ptransformId, Supplier processBundleInstructionId, Supplier> cacheTokens, @@ -103,6 +108,7 @@ public FnApiStateAccessor( Supplier currentKeySupplier, Supplier currentWindowSupplier) { this.pipelineOptions = pipelineOptions; + this.runnerCapabilites = runnerCapabilites; this.stateKeyObjectCache = Maps.newHashMap(); this.sideInputSpecMap = sideInputSpecMap; this.beamFnStateClient = beamFnStateClient; @@ -238,7 +244,11 @@ public ResultT get() { processBundleInstructionId.get(), key, ((KvCoder) sideInputSpec.getCoder()).getKeyCoder(), - ((KvCoder) sideInputSpec.getCoder()).getValueCoder())); + ((KvCoder) sideInputSpec.getCoder()).getValueCoder(), + runnerCapabilites.contains( + BeamUrns.getUrn( + RunnerApi.StandardRunnerProtocols.Enum + .MULTIMAP_KEYS_VALUES_SIDE_INPUT)))); default: throw new IllegalStateException( String.format( diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java index ec7429fcdc0e..0c7726441021 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java @@ -56,17 +56,6 @@ public class MultimapSideInput implements MultimapView { private volatile Function> bulkReadResult; private final boolean useBulkRead; - public MultimapSideInput( - Cache cache, - BeamFnStateClient beamFnStateClient, - String instructionId, - StateKey stateKey, - Coder keyCoder, - Coder valueCoder) { - // TODO(robertwb): Plumb the value of useBulkRead from runner capabilities. - this(cache, beamFnStateClient, instructionId, stateKey, keyCoder, valueCoder, false); - } - public MultimapSideInput( Cache cache, BeamFnStateClient beamFnStateClient, diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 9f51c62b83a0..cf2b61d48b50 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -1707,8 +1707,11 @@ def _create_pardo_operation( transform_id, tag, si, - input_tags_to_coders[tag]) for tag, - si in tagged_side_inputs + input_tags_to_coders[tag], + use_bulk_read=( + common_urns.runner_protocols.MULTIMAP_KEYS_VALUES_SIDE_INPUT.urn + in factory.runner_capabilities)) + for (tag, si) in tagged_side_inputs ] else: side_input_maps = [] From aacf1ee5910ffc5c2401476891f0e6930221a8ce Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 11 Dec 2023 17:20:50 +0000 Subject: [PATCH 088/224] Support Embeddings in mltransform (#29564) * Make base.py framework agnostic and add helper transforms * Add tests for base.py * Add sentence-transformers * Add tensorflow hub * Add vertex_ai * Make TFTProcessHandler a PTransform * raise RuntimeError in ArtifactsFetcher when it is used for embeddings * Add JsonPickle to requirements * Add tox tests * Mock frameworks in pydocs Fix tox.ini Fix pydoc Fix indent in pydoc * Add Row type check * Remove requires_chaining * change name of PTransformProvider to MLTransformProvider * remove batch_len in utility fun * Change type annotation and redundant comments * Remove get_transforms method * remove requires_chaining from tft * add tests to sentence-transformers * Pass inference_args to RunInference * Add TODO GH issue * refactor variables in vertex_ai embeddings * remove try/catch and throw error if options is empty for GCS artifact location * Refactor NotImplementedError message * remove tensorflow hub from this PR * Add _validate_transform method * add more tests * fix test * Fix test * Add more tests in sentence-transformer * use np.max instead of max * round to 2 decimals * Remove gradle command action * Refactor throwing dataflow client exception * skip the test if gcp is not installed * remove toxTests for hub * remove toxTests for hub * Fix values in assert for sentence_transformer_test * rename sentence_transformers to huggingface * fix pydocs * Change the model name for tests since it is getting different results on different machines * Fix pydoc in vertexai * add suffix to artifact_location * Revert "add suffix to artifact_location" This reverts commit cfb18831abc458ed5ed987bcd729d2f06d9710c1. * add no_xdist * Try fixing pydoc for vertexai * change tox.ini to use pytest directly * raise FileExistError if Attribute file is already present * modify build.gradle to match tox task names * Add note to CHANGES.md * change gcs bucket to gs://temp-storage-for-perf-tests * Add TODO GH links * Update CHANGES.md Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- CHANGES.md | 1 + sdks/python/apache_beam/ml/transforms/base.py | 446 ++++++++++++++++-- .../apache_beam/ml/transforms/base_test.py | 339 ++++++++++++- .../ml/transforms/embeddings/__init__.py | 21 + .../ml/transforms/embeddings/huggingface.py | 131 +++++ .../transforms/embeddings/huggingface_test.py | 278 +++++++++++ .../ml/transforms/embeddings/vertex_ai.py | 154 ++++++ .../transforms/embeddings/vertex_ai_test.py | 249 ++++++++++ .../apache_beam/ml/transforms/handlers.py | 6 +- .../ml/transforms/handlers_test.py | 12 +- sdks/python/apache_beam/ml/transforms/tft.py | 15 + .../apache_beam/ml/transforms/tft_test.py | 7 +- .../python/apache_beam/ml/transforms/utils.py | 16 +- .../python/apache_beam/typehints/typehints.py | 8 +- .../py310/base_image_requirements.txt | 1 + .../py311/base_image_requirements.txt | 1 + .../py38/base_image_requirements.txt | 1 + .../py39/base_image_requirements.txt | 1 + sdks/python/scripts/generate_pydoc.sh | 4 +- sdks/python/setup.py | 1 + sdks/python/test-suites/tox/py38/build.gradle | 4 + sdks/python/tox.ini | 12 + 22 files changed, 1636 insertions(+), 72 deletions(-) create mode 100644 sdks/python/apache_beam/ml/transforms/embeddings/__init__.py create mode 100644 sdks/python/apache_beam/ml/transforms/embeddings/huggingface.py create mode 100644 sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py create mode 100644 sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py create mode 100644 sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py diff --git a/CHANGES.md b/CHANGES.md index 7686b7a92d96..60b5a820cf3b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -67,6 +67,7 @@ * Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) * Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). * Added support for handling bad records to KafkaIO (Java) ([#29546](https://github.com/apache/beam/pull/29546)) +* Add support for generating text embeddings in MLTransform for Vertex AI and Hugging Face Hub models.([#29564](https://github.com/apache/beam/pull/29564)) ## New Features / Improvements diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index b3a30bb5f125..d5f4d1b60e14 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -14,20 +14,42 @@ # See the License for the specific language governing permissions and # limitations under the License. -# pytype: skip-file - import abc +import collections +import logging +import os +import tempfile +import uuid +from typing import Any from typing import Dict from typing import Generic from typing import List +from typing import Mapping from typing import Optional from typing import Sequence from typing import TypeVar +from typing import Union + +import jsonpickle +import numpy as np import apache_beam as beam +from apache_beam.io.filesystems import FileSystems from apache_beam.metrics.metric import Metrics +from apache_beam.ml.inference.base import ModelHandler +from apache_beam.ml.inference.base import ModelT +from apache_beam.options.pipeline_options import PipelineOptions -__all__ = ['MLTransform', 'ProcessHandler', 'BaseOperation'] +_LOGGER = logging.getLogger(__name__) +_ATTRIBUTE_FILE_NAME = 'attributes.json' + +__all__ = [ + 'MLTransform', + 'ProcessHandler', + 'MLTransformProvider', + 'BaseOperation', + 'EmbeddingsManager' +] TransformedDatasetT = TypeVar('TransformedDatasetT') TransformedMetadataT = TypeVar('TransformedMetadataT') @@ -42,12 +64,62 @@ OperationOutputT = TypeVar('OperationOutputT') +def _convert_list_of_dicts_to_dict_of_lists( + list_of_dicts: Sequence[Dict[str, Any]]) -> Dict[str, List[Any]]: + keys_to_element_list = collections.defaultdict(list) + for d in list_of_dicts: + for key, value in d.items(): + keys_to_element_list[key].append(value) + return keys_to_element_list + + +def _convert_dict_of_lists_to_lists_of_dict( + dict_of_lists: Dict[str, List[Any]]) -> List[Dict[str, Any]]: + batch_length = len(next(iter(dict_of_lists.values()))) + result: List[Dict[str, Any]] = [{} for _ in range(batch_length)] + # all the values in the dict_of_lists should have same length + for key, values in dict_of_lists.items(): + assert len(values) == batch_length, ( + "This function expects all the values " + "in the dict_of_lists to have same length." + ) + for i in range(len(values)): + result[i][key] = values[i] + return result + + class ArtifactMode(object): PRODUCE = 'produce' CONSUME = 'consume' -class BaseOperation(Generic[OperationInputT, OperationOutputT], abc.ABC): +class MLTransformProvider: + """ + Data processing transforms that are intended to be used with MLTransform + should subclass MLTransformProvider and implement + get_ptransform_for_processing(). + + get_ptransform_for_processing() method should return a PTransform that can be + used to process the data. + + """ + @abc.abstractmethod + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + """ + Returns a PTransform that can be used to process the data. + """ + + def get_counter(self): + """ + Returns the counter name for the data processing transform. + """ + counter_name = self.__class__.__name__ + return Metrics.counter(MLTransform, f'BeamML_{counter_name}') + + +class BaseOperation(Generic[OperationInputT, OperationOutputT], + MLTransformProvider, + abc.ABC): def __init__(self, columns: List[str]) -> None: """ Base Opertation class data processing transformations. @@ -76,33 +148,53 @@ def __call__(self, data: OperationInputT, transformed_data = self.apply_transform(data, output_column_name) return transformed_data - def get_counter(self): - """ - Returns the counter name for the operation. - """ - counter_name = self.__class__.__name__ - return Metrics.counter(MLTransform, f'BeamML_{counter_name}') - -class ProcessHandler(Generic[ExampleT, MLTransformOutputT], abc.ABC): +class ProcessHandler(beam.PTransform[beam.PCollection[ExampleT], + beam.PCollection[MLTransformOutputT]], + abc.ABC): """ Only for internal use. No backwards compatibility guarantees. """ @abc.abstractmethod - def process_data( - self, pcoll: beam.PCollection[ExampleT] - ) -> beam.PCollection[MLTransformOutputT]: + def append_transform(self, transform: BaseOperation): """ - Logic to process the data. This will be the entrypoint in - beam.MLTransform to process incoming data. + Append transforms to the ProcessHandler. """ + +# TODO:https://github.com/apache/beam/issues/29356 +# Add support for inference_fn +class EmbeddingsManager(MLTransformProvider): + def __init__( + self, + columns: List[str], + *, + # common args for all ModelHandlers. + load_model_args: Optional[Dict[str, Any]] = None, + min_batch_size: Optional[int] = None, + max_batch_size: Optional[int] = None, + large_model: bool = False, + **kwargs): + self.load_model_args = load_model_args or {} + self.min_batch_size = min_batch_size + self.max_batch_size = max_batch_size + self.large_model = large_model + self.columns = columns + self.inference_args = kwargs.pop('inference_args', {}) + + if kwargs: + _LOGGER.warning("Ignoring the following arguments: %s", kwargs.keys()) + + # TODO:https://github.com/apache/beam/pull/29564 add set_model_handler method @abc.abstractmethod - def append_transform(self, transform: BaseOperation): + def get_model_handler(self) -> ModelHandler: """ - Append transforms to the ProcessHandler. + Return framework specific model handler. """ + def get_columns_to_apply(self): + return self.columns + class MLTransform(beam.PTransform[beam.PCollection[ExampleT], beam.PCollection[MLTransformOutputT]], @@ -112,7 +204,7 @@ def __init__( *, write_artifact_location: Optional[str] = None, read_artifact_location: Optional[str] = None, - transforms: Optional[Sequence[BaseOperation]] = None): + transforms: Optional[List[MLTransformProvider]] = None): """ MLTransform is a Beam PTransform that can be used to apply transformations to the data. MLTransform is used to wrap the @@ -157,9 +249,6 @@ def __init__( i-th transform is the output of the (i-1)-th transform. Multi-input transforms are not supported yet. """ - if transforms: - _ = [self._validate_transform(transform) for transform in transforms] - if read_artifact_location and write_artifact_location: raise ValueError( 'Only one of read_artifact_location or write_artifact_location can ' @@ -177,19 +266,10 @@ def __init__( artifact_location = write_artifact_location # type: ignore[assignment] artifact_mode = ArtifactMode.PRODUCE - # avoid circular import - # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.ml.transforms.handlers import TFTProcessHandler - # TODO: When new ProcessHandlers(eg: JaxProcessHandler) are introduced, - # create a mapping between transforms and ProcessHandler since - # ProcessHandler is not exposed to the user. - process_handler: ProcessHandler = TFTProcessHandler( - artifact_location=artifact_location, - artifact_mode=artifact_mode, - transforms=transforms) # type: ignore[arg-type] - - self._process_handler = process_handler - self.transforms = transforms + self._parent_artifact_location = artifact_location + + self._artifact_mode = artifact_mode + self.transforms = transforms or [] self._counter = Metrics.counter( MLTransform, f'BeamML_{self.__class__.__name__}') @@ -209,12 +289,34 @@ def expand( Returns: A PCollection of MLTransformOutputT type """ + _ = [self._validate_transform(transform) for transform in self.transforms] + if self._artifact_mode == ArtifactMode.PRODUCE: + ptransform_partitioner = _MLTransformToPTransformMapper( + transforms=self.transforms, + artifact_location=self._parent_artifact_location, + artifact_mode=self._artifact_mode, + pipeline_options=pcoll.pipeline.options) + ptransform_list = ptransform_partitioner.create_and_save_ptransform_list() + else: + ptransform_list = ( + _MLTransformToPTransformMapper.load_transforms_from_artifact_location( + self._parent_artifact_location)) + + # the saved transforms has artifact mode set to PRODUCE. + # set the artifact mode to CONSUME. + for i in range(len(ptransform_list)): + if hasattr(ptransform_list[i], 'artifact_mode'): + ptransform_list[i].artifact_mode = self._artifact_mode + + for ptransform in ptransform_list: + pcoll = pcoll | ptransform + _ = ( pcoll.pipeline | "MLTransformMetricsUsage" >> MLTransformMetricsUsage(self)) - return self._process_handler.process_data(pcoll) + return pcoll # type: ignore[return-value] - def with_transform(self, transform: BaseOperation): + def with_transform(self, transform: MLTransformProvider): """ Add a transform to the MLTransform pipeline. Args: @@ -223,11 +325,11 @@ def with_transform(self, transform: BaseOperation): A MLTransform instance. """ self._validate_transform(transform) - self._process_handler.append_transform(transform) + self.transforms.append(transform) return self def _validate_transform(self, transform): - if not isinstance(transform, BaseOperation): + if not isinstance(transform, MLTransformProvider): raise TypeError( 'transform must be a subclass of BaseOperation. ' 'Got: %s instead.' % type(transform)) @@ -243,9 +345,7 @@ def _increment_counters(): # increment for MLTransform. self._ml_transform._counter.inc() # increment if data processing transforms are passed. - transforms = ( - self._ml_transform.transforms or - self._ml_transform._process_handler.transforms) + transforms = self._ml_transform.transforms if transforms: for transform in transforms: transform.get_counter().inc() @@ -254,3 +354,263 @@ def _increment_counters(): pipeline | beam.Create([None]) | beam.Map(lambda _: _increment_counters())) + + +class _TransformAttributeManager: + """ + Base class used for saving and loading the attributes. + """ + @staticmethod + def save_attributes(artifact_location): + """ + Save the attributes to json file using stdlib json. + """ + raise NotImplementedError + + @staticmethod + def load_attributes(artifact_location): + """ + Load the attributes from json file. + """ + raise NotImplementedError + + +class _JsonPickleTransformAttributeManager(_TransformAttributeManager): + """ + Use Jsonpickle to save and load the attributes. Here the attributes refer + to the list of PTransforms that are used to process the data. + + jsonpickle is used to serialize the PTransforms and save it to a json file and + is compatible across python versions. + """ + @staticmethod + def _is_remote_path(path): + is_gcs = path.find('gs://') != -1 + # TODO:https://github.com/apache/beam/issues/29356 + # Add support for other remote paths. + if not is_gcs and path.find('://') != -1: + raise RuntimeError( + "Artifact locations are currently supported for only available for " + "local paths and GCS paths. Got: %s" % path) + return is_gcs + + @staticmethod + def save_attributes( + ptransform_list, + artifact_location, + **kwargs, + ): + # if an artifact location is present, instead of overwriting the + # existing file, raise an error since the same artifact location + # can be used by multiple beam jobs and this could result in undesired + # behavior. + if FileSystems.exists(FileSystems.join(artifact_location, + _ATTRIBUTE_FILE_NAME)): + raise FileExistsError( + "The artifact location %s already exists and contains %s. Please " + "specify a different location." % + (artifact_location, _ATTRIBUTE_FILE_NAME)) + + if _JsonPickleTransformAttributeManager._is_remote_path(artifact_location): + temp_dir = tempfile.mkdtemp() + temp_json_file = os.path.join(temp_dir, _ATTRIBUTE_FILE_NAME) + with open(temp_json_file, 'w+') as f: + f.write(jsonpickle.encode(ptransform_list)) + with open(temp_json_file, 'rb') as f: + from apache_beam.runners.dataflow.internal import apiclient + _LOGGER.info('Creating artifact location: %s', artifact_location) + # pipeline options required to for the client to configure project. + options = kwargs.get('options') + try: + apiclient.DataflowApplicationClient(options=options).stage_file( + gcs_or_local_path=artifact_location, + file_name=_ATTRIBUTE_FILE_NAME, + stream=f, + mime_type='application/json') + except Exception as exc: + if not options: + raise RuntimeError( + "Failed to create Dataflow client. " + "Pipeline options are required to save the attributes." + "in the artifact location %s" % artifact_location) from exc + raise + else: + if not FileSystems.exists(artifact_location): + FileSystems.mkdirs(artifact_location) + # FileSystems.open() fails if the file does not exist. + with open(os.path.join(artifact_location, _ATTRIBUTE_FILE_NAME), + 'w+') as f: + f.write(jsonpickle.encode(ptransform_list)) + + @staticmethod + def load_attributes(artifact_location): + with FileSystems.open(os.path.join(artifact_location, _ATTRIBUTE_FILE_NAME), + 'rb') as f: + return jsonpickle.decode(f.read()) + + +_transform_attribute_manager = _JsonPickleTransformAttributeManager + + +class _MLTransformToPTransformMapper: + """ + This class takes in a list of data processing transforms compatible to be + wrapped around MLTransform and returns a list of PTransforms that are used to + run the data processing transforms. + + The _MLTransformToPTransformMapper is responsible for loading and saving the + PTransforms or attributes of PTransforms to the artifact location to seal + the gap between the training and inference pipelines. + """ + def __init__( + self, + transforms: List[MLTransformProvider], + artifact_location: str, + artifact_mode: str = ArtifactMode.PRODUCE, + pipeline_options: Optional[PipelineOptions] = None, + ): + self.transforms = transforms + self._parent_artifact_location = artifact_location + self.artifact_mode = artifact_mode + self.pipeline_options = pipeline_options + + def create_and_save_ptransform_list(self): + ptransform_list = self.create_ptransform_list() + self.save_transforms_in_artifact_location(ptransform_list) + return ptransform_list + + def create_ptransform_list(self): + previous_ptransform_type = None + current_ptransform = None + ptransform_list = [] + for transform in self.transforms: + if not isinstance(transform, MLTransformProvider): + raise RuntimeError( + 'Transforms must be instances of MLTransformProvider and ' + 'implement get_ptransform_for_processing() method.') + # for each instance of PTransform, create a new artifact location + current_ptransform = transform.get_ptransform_for_processing( + artifact_location=os.path.join( + self._parent_artifact_location, uuid.uuid4().hex[:6]), + artifact_mode=self.artifact_mode) + append_transform = hasattr(current_ptransform, 'append_transform') + if (type(current_ptransform) != + previous_ptransform_type) or not append_transform: + ptransform_list.append(current_ptransform) + previous_ptransform_type = type(current_ptransform) + # If different PTransform is appended to the list and the PTransform + # supports append_transform, append the transform to the PTransform. + if append_transform: + ptransform_list[-1].append_transform(transform) + return ptransform_list + + def save_transforms_in_artifact_location(self, ptransform_list): + """ + Save the ptransform references to json file. + """ + _transform_attribute_manager.save_attributes( + ptransform_list=ptransform_list, + artifact_location=self._parent_artifact_location, + options=self.pipeline_options) + + @staticmethod + def load_transforms_from_artifact_location(artifact_location): + return _transform_attribute_manager.load_attributes(artifact_location) + + +class _TextEmbeddingHandler(ModelHandler): + """ + A ModelHandler intended to be work on list[dict[str, str]] inputs. + + The inputs to the model handler are expected to be a list of dicts. + + For example, if the original mode is used with RunInference to take a + PCollection[E] to a PCollection[P], this ModelHandler would take a + PCollection[Dict[str, E]] to a PCollection[Dict[str, P]]. + + _TextEmbeddingHandler will accept an EmbeddingsManager instance, which + contains the details of the model to be loaded and the inference_fn to be + used. The purpose of _TextEmbeddingHandler is to generate embeddings for + text inputs using the EmbeddingsManager instance. + + If the input is not a text column, a RuntimeError will be raised. + + This is an internal class and offers no backwards compatibility guarantees. + + Args: + embeddings_manager: An EmbeddingsManager instance. + """ + def __init__(self, embeddings_manager: EmbeddingsManager): + self.embedding_config = embeddings_manager + self._underlying = self.embedding_config.get_model_handler() + self.columns = self.embedding_config.get_columns_to_apply() + + def load_model(self): + model = self._underlying.load_model() + return model + + def _validate_column_data(self, batch): + if not isinstance(batch[0], (str, bytes)): + raise TypeError( + 'Embeddings can only be generated on Dict[str, str].' + f'Got Dict[str, {type(batch[0])}] instead.') + + def _validate_batch(self, batch: Sequence[Dict[str, List[str]]]): + if not batch or not isinstance(batch[0], dict): + raise TypeError( + 'Expected data to be dicts, got ' + f'{type(batch[0])} instead.') + + def _process_batch( + self, + dict_batch: Dict[str, List[Any]], + model: ModelT, + inference_args: Optional[Dict[str, Any]]) -> Dict[str, List[Any]]: + result: Dict[str, List[Any]] = collections.defaultdict(list) + for key, batch in dict_batch.items(): + if key in self.columns: + self._validate_column_data(batch) + prediction = self._underlying.run_inference( + batch, model, inference_args) + if isinstance(prediction, np.ndarray): + prediction = prediction.tolist() + result[key] = prediction # type: ignore[assignment] + else: + result[key] = prediction # type: ignore[assignment] + else: + result[key] = batch + return result + + def run_inference( + self, + batch: Sequence[Dict[str, List[str]]], + model: ModelT, + inference_args: Optional[Dict[str, Any]] = None, + ) -> List[Dict[str, Union[List[float], List[str]]]]: + """ + Runs inference on a batch of text inputs. The inputs are expected to be + a list of dicts. Each dict should have the same keys, and the shape + should be of the same size for a single key across the batch. + """ + self._validate_batch(batch) + dict_batch = _convert_list_of_dicts_to_dict_of_lists(list_of_dicts=batch) + transformed_batch = self._process_batch(dict_batch, model, inference_args) + return _convert_dict_of_lists_to_lists_of_dict( + dict_of_lists=transformed_batch, + ) + + def get_metrics_namespace(self) -> str: + return ( + self._underlying.get_metrics_namespace() or + 'BeamML_TextEmbeddingHandler') + + def batch_elements_kwargs(self) -> Mapping[str, Any]: + batch_sizes_map = {} + if self.embedding_config.max_batch_size: + batch_sizes_map['max_batch_size'] = self.embedding_config.max_batch_size + if self.embedding_config.min_batch_size: + batch_sizes_map['min_batch_size'] = self.embedding_config.min_batch_size + return (self._underlying.batch_elements_kwargs() or batch_sizes_map) + + def validate_inference_args(self, _): + pass diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index 2e447964541b..e07959436198 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -16,11 +16,16 @@ # # pytype: skip-file +import os import shutil import tempfile import typing import unittest +from typing import Any +from typing import Dict from typing import List +from typing import Optional +from typing import Sequence import numpy as np from parameterized import param @@ -28,28 +33,36 @@ import apache_beam as beam from apache_beam.metrics.metric import MetricsFilter +from apache_beam.ml.inference.base import ModelHandler +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.transforms import base from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: - from apache_beam.ml.transforms import base from apache_beam.ml.transforms import tft + from apache_beam.ml.transforms.handlers import TFTProcessHandler from apache_beam.ml.transforms.tft import TFTOperation except ImportError: tft = None # type: ignore -if tft is None: - raise unittest.SkipTest('tensorflow_transform is not installed') +try: + class _FakeOperation(TFTOperation): + def __init__(self, name, *args, **kwargs): + super().__init__(*args, **kwargs) + self.name = name -class _FakeOperation(TFTOperation): - def __init__(self, name, *args, **kwargs): - super().__init__(*args, **kwargs) - self.name = name + def apply_transform(self, inputs, output_column_name, **kwargs): + return {output_column_name: inputs} +except: # pylint: disable=bare-except + pass - def apply_transform(self, inputs, output_column_name, **kwargs): - return {output_column_name: inputs} +try: + from apache_beam.runners.dataflow.internal import apiclient +except ImportError: + apiclient = None # type: ignore class BaseMLTransformTest(unittest.TestCase): @@ -59,6 +72,7 @@ def setUp(self) -> None: def tearDown(self): shutil.rmtree(self.artifact_location) + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_appends_transforms_to_process_handler_correctly(self): fake_fn_1 = _FakeOperation(name='fake_fn_1', columns=['x']) transforms = [fake_fn_1] @@ -67,12 +81,11 @@ def test_ml_transform_appends_transforms_to_process_handler_correctly(self): ml_transform = ml_transform.with_transform( transform=_FakeOperation(name='fake_fn_2', columns=['x'])) - self.assertEqual(len(ml_transform._process_handler.transforms), 2) - self.assertEqual( - ml_transform._process_handler.transforms[0].name, 'fake_fn_1') - self.assertEqual( - ml_transform._process_handler.transforms[1].name, 'fake_fn_2') + self.assertEqual(len(ml_transform.transforms), 2) + self.assertEqual(ml_transform.transforms[0].name, 'fake_fn_1') + self.assertEqual(ml_transform.transforms[1].name, 'fake_fn_2') + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_on_dict(self): transforms = [tft.ScaleTo01(columns=['x'])] data = [{'x': 1}, {'x': 2}] @@ -91,6 +104,7 @@ def test_ml_transform_on_dict(self): assert_that( actual_output, equal_to(expected_output, equals_fn=np.array_equal)) + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_on_list_dict(self): transforms = [tft.ScaleTo01(columns=['x'])] data = [{'x': [1, 2, 3]}, {'x': [4, 5, 6]}] @@ -162,6 +176,7 @@ def test_ml_transform_on_list_dict(self): }, ), ]) + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_dict_output_pcoll_schema( self, input_data, input_types, expected_dtype): transforms = [tft.ScaleTo01(columns=['x'])] @@ -178,6 +193,7 @@ def test_ml_transform_dict_output_pcoll_schema( if name in expected_dtype: self.assertEqual(expected_dtype[name], typ) + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_fail_for_non_global_windows_in_produce_mode(self): transforms = [tft.ScaleTo01(columns=['x'])] with beam.Pipeline() as p: @@ -193,6 +209,7 @@ def test_ml_transform_fail_for_non_global_windows_in_produce_mode(self): write_artifact_location=self.artifact_location, )) + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transform_on_multiple_columns_single_transform(self): transforms = [tft.ScaleTo01(columns=['x', 'y'])] data = [{'x': [1, 2, 3], 'y': [1.0, 10.0, 20.0]}] @@ -217,6 +234,7 @@ def test_ml_transform_on_multiple_columns_single_transform(self): equal_to(expected_output_y, equals_fn=np.array_equal), label='y') + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_ml_transforms_on_multiple_columns_multiple_transforms(self): transforms = [ tft.ScaleTo01(columns=['x']), @@ -245,6 +263,7 @@ def test_ml_transforms_on_multiple_columns_multiple_transforms(self): equal_to(expected_output_y, equals_fn=np.array_equal), label='actual_output_y') + @unittest.skipIf(tft is None, 'tft module is not installed.') def test_mltransform_with_counter(self): transforms = [ tft.ComputeAndApplyVocabulary(columns=['y']), @@ -269,6 +288,298 @@ def test_mltransform_with_counter(self): self.assertEqual( result.metrics().query(mltransform_counter)['counters'][0].result, 1) + def test_non_ptransfrom_provider_class_to_mltransform(self): + class Add: + def __call__(self, x): + return x + 1 + + with self.assertRaisesRegex(TypeError, 'transform must be a subclass of'): + with beam.Pipeline() as p: + _ = ( + p + | beam.Create([{ + 'x': 1 + }]) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + Add())) + + +class FakeModel: + def __call__(self, example: List[str]) -> List[str]: + for i in range(len(example)): + example[i] = example[i][::-1] + return example + + +class FakeModelHandler(ModelHandler): + def run_inference( + self, + batch: Sequence[str], + model: Any, + inference_args: Optional[Dict[str, Any]] = None): + return model(batch) + + def load_model(self): + return FakeModel() + + +class FakeEmbeddingsManager(base.EmbeddingsManager): + def __init__(self, columns): + super().__init__(columns=columns) + + def get_model_handler(self) -> ModelHandler: + return FakeModelHandler() + + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + return (RunInference(model_handler=base._TextEmbeddingHandler(self))) + + +class TextEmbeddingHandlerTest(unittest.TestCase): + def setUp(self) -> None: + self.embedding_conig = FakeEmbeddingsManager(columns=['x']) + self.artifact_location = tempfile.mkdtemp() + + def tearDown(self) -> None: + shutil.rmtree(self.artifact_location) + + def test_handler_with_incompatible_datatype(self): + text_handler = base._TextEmbeddingHandler( + embeddings_manager=self.embedding_conig) + data = [ + ('x', 1), + ('x', 2), + ('x', 3), + ] + with self.assertRaises(TypeError): + text_handler.run_inference(data, None, None) + + def test_handler_with_dict_inputs(self): + data = [ + { + 'x': "Hello world" + }, + { + 'x': "Apache Beam" + }, + ] + expected_data = [{key: value[::-1] + for key, value in d.items()} for d in data] + with beam.Pipeline() as p: + result = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + assert_that( + result, + equal_to(expected_data), + ) + + def test_handler_with_batch_sizes(self): + self.embedding_conig.max_batch_size = 100 + self.embedding_conig.min_batch_size = 10 + data = [ + { + 'x': "Hello world" + }, + { + 'x': "Apache Beam" + }, + ] * 100 + expected_data = [{key: value[::-1] + for key, value in d.items()} for d in data] + with beam.Pipeline() as p: + result = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + assert_that( + result, + equal_to(expected_data), + ) + + def test_handler_on_multiple_columns(self): + self.embedding_conig.columns = ['x', 'y'] + data = [ + { + 'x': "Hello world", 'y': "Apache Beam", 'z': 'unchanged' + }, + { + 'x': "Apache Beam", 'y': "Hello world", 'z': 'unchanged' + }, + ] + self.embedding_conig.columns = ['x', 'y'] + expected_data = [{ + key: (value[::-1] if key in self.embedding_conig.columns else value) + for key, + value in d.items() + } for d in data] + with beam.Pipeline() as p: + result = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + assert_that( + result, + equal_to(expected_data), + ) + + def test_handler_with_list_data(self): + data = [{ + 'x': ['Hello world', 'Apache Beam'], + }, { + 'x': ['Apache Beam', 'Hello world'], + }] + with self.assertRaises(TypeError): + with beam.Pipeline() as p: + _ = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_conig)) + + +class TestUtilFunctions(unittest.TestCase): + def test_list_of_dicts_to_dict_of_lists_normal(self): + input_list = [{'a': 1, 'b': 2}, {'a': 3, 'b': 4}] + expected_output = {'a': [1, 3], 'b': [2, 4]} + self.assertEqual( + base._convert_list_of_dicts_to_dict_of_lists(input_list), + expected_output) + + def test_list_of_dicts_to_dict_of_lists_on_list_inputs(self): + input_list = [{'a': [1, 2, 10], 'b': 3}, {'a': [1], 'b': 5}] + expected_output = {'a': [[1, 2, 10], [1]], 'b': [3, 5]} + self.assertEqual( + base._convert_list_of_dicts_to_dict_of_lists(input_list), + expected_output) + + def test_dict_of_lists_to_lists_of_dict_normal(self): + input_dict = {'a': [1, 3], 'b': [2, 4]} + expected_output = [{'a': 1, 'b': 2}, {'a': 3, 'b': 4}] + self.assertEqual( + base._convert_dict_of_lists_to_lists_of_dict(input_dict), + expected_output) + + def test_dict_of_lists_to_lists_of_dict_unequal_length(self): + input_dict = {'a': [1, 3], 'b': [2]} + with self.assertRaises(AssertionError): + base._convert_dict_of_lists_to_lists_of_dict(input_dict) + + +class TestJsonPickleTransformAttributeManager(unittest.TestCase): + def setUp(self): + self.attribute_manager = base._transform_attribute_manager + self.artifact_location = tempfile.mkdtemp() + + def tearDown(self): + shutil.rmtree(self.artifact_location) + + @unittest.skipIf(tft is None, 'tft module is not installed.') + def test_save_tft_process_handler(self): + transforms = [ + tft.ScaleTo01(columns=['x']), + tft.ComputeAndApplyVocabulary(columns=['y']) + ] + process_handler = TFTProcessHandler( + transforms=transforms, + artifact_location=self.artifact_location, + ) + self.attribute_manager.save_attributes( + ptransform_list=[process_handler], + artifact_location=self.artifact_location, + ) + + files = os.listdir(self.artifact_location) + self.assertTrue(len(files) == 1) + self.assertTrue(files[0] == base._ATTRIBUTE_FILE_NAME) + + def test_save_run_inference(self): + self.attribute_manager.save_attributes( + ptransform_list=[RunInference(model_handler=FakeModelHandler())], + artifact_location=self.artifact_location, + ) + files = os.listdir(self.artifact_location) + self.assertTrue(len(files) == 1) + self.assertTrue(files[0] == base._ATTRIBUTE_FILE_NAME) + + def test_save_and_load_run_inference(self): + ptransform_list = [RunInference(model_handler=FakeModelHandler())] + self.attribute_manager.save_attributes( + ptransform_list=ptransform_list, + artifact_location=self.artifact_location, + ) + loaded_ptransform_list = self.attribute_manager.load_attributes( + artifact_location=self.artifact_location, + ) + + self.assertTrue(len(loaded_ptransform_list) == len(ptransform_list)) + self.assertListEqual( + list(loaded_ptransform_list[0].__dict__.keys()), + list(ptransform_list[0].__dict__.keys())) + + get_keys = lambda x: list(x.__dict__.keys()) + for i, transform in enumerate(ptransform_list): + self.assertListEqual( + get_keys(transform), get_keys(loaded_ptransform_list[i])) + if hasattr(transform, 'model_handler'): + model_handler = transform.model_handler + loaded_model_handler = loaded_ptransform_list[i].model_handler + self.assertListEqual( + get_keys(model_handler), get_keys(loaded_model_handler)) + + def test_mltransform_to_ptransform_wrapper(self): + transforms = [ + FakeEmbeddingsManager(columns=['x']), + FakeEmbeddingsManager(columns=['y', 'z']), + ] + ptransform_mapper = base._MLTransformToPTransformMapper( + transforms=transforms, + artifact_location=self.artifact_location, + artifact_mode=None) + + ptransform_list = ptransform_mapper.create_ptransform_list() + self.assertTrue(len(ptransform_list) == 2) + + self.assertEqual(type(ptransform_list[0]), RunInference) + expected_columns = [['x'], ['y', 'z']] + for i in range(len(ptransform_list)): + self.assertEqual(type(ptransform_list[i]), RunInference) + self.assertEqual( + type(ptransform_list[i]._model_handler), base._TextEmbeddingHandler) + self.assertEqual( + ptransform_list[i]._model_handler.columns, expected_columns[i]) + + @unittest.skipIf(apiclient is None, 'apache_beam[gcp] is not installed.') + def test_with_gcs_location_with_none_options(self): + path = 'gs://fake_path' + with self.assertRaises(RuntimeError): + self.attribute_manager.save_attributes( + ptransform_list=[], artifact_location=path, options=None) + with self.assertRaises(RuntimeError): + self.attribute_manager.save_attributes( + ptransform_list=[], artifact_location=path) + + def test_with_same_local_artifact_location(self): + artifact_location = self.artifact_location + attribute_manager = base._JsonPickleTransformAttributeManager() + + ptransform_list = [RunInference(model_handler=FakeModelHandler())] + + attribute_manager.save_attributes( + ptransform_list, artifact_location=artifact_location) + + with self.assertRaises(FileExistsError): + attribute_manager.save_attributes([lambda x: x], + artifact_location=artifact_location) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/__init__.py b/sdks/python/apache_beam/ml/transforms/embeddings/__init__.py new file mode 100644 index 000000000000..bda6256b79ef --- /dev/null +++ b/sdks/python/apache_beam/ml/transforms/embeddings/__init__.py @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# TODO: Add dead letter queue for RunInference transforms. + +""" +This module contains embedding configs that can be used to generate +embeddings using MLTransform. +""" diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface.py b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface.py new file mode 100644 index 000000000000..e979296b0b83 --- /dev/null +++ b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface.py @@ -0,0 +1,131 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +__all__ = ["SentenceTransformerEmbeddings"] + +from typing import Any +from typing import Callable +from typing import Dict +from typing import List +from typing import Mapping +from typing import Optional +from typing import Sequence + +import apache_beam as beam +from apache_beam.ml.inference.base import ModelHandler +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.transforms.base import EmbeddingsManager +from apache_beam.ml.transforms.base import _TextEmbeddingHandler +from sentence_transformers import SentenceTransformer + + +# TODO: https://github.com/apache/beam/issues/29621 +# Use HuggingFaceModelHandlerTensor once the import issue is fixed. +# Right now, the hugging face model handler import torch and tensorflow +# at the same time, which adds too much weigth to the container unnecessarily. +class _SentenceTransformerModelHandler(ModelHandler): + """ + Note: Intended for internal use and guarantees no backwards compatibility. + """ + def __init__( + self, + model_name: str, + model_class: Callable, + load_model_args: Optional[dict] = None, + min_batch_size: Optional[int] = None, + max_batch_size: Optional[int] = None, + max_seq_length: Optional[int] = None, + large_model: bool = False, + **kwargs): + self._max_seq_length = max_seq_length + self.model_name = model_name + self._model_class = model_class + self._load_model_args = load_model_args + self._min_batch_size = min_batch_size + self._max_batch_size = max_batch_size + self._large_model = large_model + self._kwargs = kwargs + + def run_inference( + self, + batch: Sequence[str], + model: SentenceTransformer, + inference_args: Optional[Dict[str, Any]] = None, + ): + inference_args = inference_args or {} + return model.encode(batch, **inference_args) + + def load_model(self): + model = self._model_class(self.model_name, **self._load_model_args) + if self._max_seq_length: + model.max_seq_length = self._max_seq_length + return model + + def share_model_across_processes(self) -> bool: + return self._large_model + + def batch_elements_kwargs(self) -> Mapping[str, Any]: + batch_sizes = {} + if self._min_batch_size: + batch_sizes["min_batch_size"] = self._min_batch_size + if self._max_batch_size: + batch_sizes["max_batch_size"] = self._max_batch_size + return batch_sizes + + +class SentenceTransformerEmbeddings(EmbeddingsManager): + def __init__( + self, + model_name: str, + columns: List[str], + max_seq_length: Optional[int] = None, + **kwargs): + """ + Embedding config for sentence-transformers. This config can be used with + MLTransform to embed text data. Models are loaded using the RunInference + PTransform with the help of ModelHandler. + + Args: + model_name: Name of the model to use. The model should be hosted on + HuggingFace Hub or compatible with sentence_transformers. + columns: List of columns to be embedded. + max_seq_length: Max sequence length to use for the model if applicable. + min_batch_size: The minimum batch size to be used for inference. + max_batch_size: The maximum batch size to be used for inference. + large_model: Whether to share the model across processes. + """ + super().__init__(columns, **kwargs) + self.model_name = model_name + self.max_seq_length = max_seq_length + + def get_model_handler(self): + return _SentenceTransformerModelHandler( + model_class=SentenceTransformer, + max_seq_length=self.max_seq_length, + model_name=self.model_name, + load_model_args=self.load_model_args, + min_batch_size=self.min_batch_size, + max_batch_size=self.max_batch_size, + large_model=self.large_model) + + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + # wrap the model handler in a _TextEmbeddingHandler since + # the SentenceTransformerEmbeddings works on text input data. + return ( + RunInference( + model_handler=_TextEmbeddingHandler(self), + inference_args=self.inference_args, + )) diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py new file mode 100644 index 000000000000..779a6daf8f3c --- /dev/null +++ b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py @@ -0,0 +1,278 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 os +import shutil +import tempfile +import unittest +import uuid + +import numpy as np +from parameterized import parameterized + +import apache_beam as beam +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.transforms import base +from apache_beam.ml.transforms.base import MLTransform +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + +# pylint: disable=ungrouped-imports +try: + from apache_beam.ml.transforms.embeddings.huggingface import SentenceTransformerEmbeddings + import torch +except ImportError: + SentenceTransformerEmbeddings = None # type: ignore + +# pylint: disable=ungrouped-imports +try: + import tensorflow_transform as tft + from apache_beam.ml.transforms.tft import ScaleTo01 +except ImportError: + tft = None + +test_query = "This is a test" +test_query_column = "feature_1" +DEFAULT_MODEL_NAME = "sentence-transformers/all-mpnet-base-v2" +_parameterized_inputs = [ + ([{ + test_query_column: 'That is a happy person' + }, { + test_query_column: 'That is a very happy person' + }], + 'thenlper/gte-base', [0.11, 0.11]), + ([{ + test_query_column: test_query, + }], DEFAULT_MODEL_NAME, [0.13]), + ( + [{ + test_query_column: 'query: how much protein should a female eat', + }, + { + test_query_column: ( + "passage: As a general guideline, the CDC's " + "average requirement of protein for women " + "ages 19 to 70 is 46 grams per day. But, " + "as you can see from this chart, you'll need " + "to increase that if you're expecting or training" + " for a marathon. Check out the chart below " + "to see how much protein " + "you should be eating each day.") + }], + 'intfloat/e5-base-v2', + # this model requires inputs to be specified as query: and passage: + [0.1, 0.1]), +] + + +@unittest.skipIf( + SentenceTransformerEmbeddings is None, + 'sentence-transformers is not installed.') +class SentenceTrasformerEmbeddingsTest(unittest.TestCase): + def setUp(self) -> None: + self.artifact_location = tempfile.mkdtemp(prefix='sentence_transformers_') + # this bucket has TTL and will be deleted periodically + self.gcs_artifact_location = os.path.join( + 'gs://temp-storage-for-perf-tests/sentence_transformers', + uuid.uuid4().hex) + + def tearDown(self) -> None: + shutil.rmtree(self.artifact_location) + + def test_sentence_transformer_embeddings(self): + model_name = DEFAULT_MODEL_NAME + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, columns=[test_query_column]) + with beam.Pipeline() as pipeline: + result_pcoll = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def assert_element(element): + assert len(element[test_query_column]) == 768 + + _ = (result_pcoll | beam.Map(assert_element)) + + @unittest.skipIf(tft is None, 'Tensorflow Transform is not installed.') + def test_embeddings_with_scale_to_0_1(self): + model_name = DEFAULT_MODEL_NAME + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, + columns=[test_query_column], + ) + with beam.Pipeline() as pipeline: + transformed_pcoll = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config).with_transform( + ScaleTo01(columns=[test_query_column]))) + + def assert_element(element): + assert max(element.feature_1) == 1 + + _ = (transformed_pcoll | beam.Map(assert_element)) + + @parameterized.expand(_parameterized_inputs) + def test_embeddings_with_read_artifact_location( + self, inputs, model_name, output): + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, columns=[test_query_column]) + + with beam.Pipeline() as p: + result_pcoll = ( + p + | "CreateData" >> beam.Create(inputs) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + max_ele_pcoll = ( + result_pcoll + | beam.Map(lambda x: round(max(x[test_query_column]), 2))) + + assert_that(max_ele_pcoll, equal_to(output)) + + with beam.Pipeline() as p: + result_pcoll = ( + p + | "CreateData" >> beam.Create(inputs) + | "MLTransform" >> + MLTransform(read_artifact_location=self.artifact_location)) + max_ele_pcoll = ( + result_pcoll + | beam.Map(lambda x: round(max(x[test_query_column]), 2))) + + assert_that(max_ele_pcoll, equal_to(output)) + + def test_sentence_transformer_with_int_data_types(self): + model_name = DEFAULT_MODEL_NAME + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, columns=[test_query_column]) + with self.assertRaises(TypeError): + with beam.Pipeline() as pipeline: + _ = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: 1 + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + @parameterized.expand(_parameterized_inputs) + def test_with_gcs_artifact_location(self, inputs, model_name, output): + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, columns=[test_query_column]) + + with beam.Pipeline() as p: + result_pcoll = ( + p + | "CreateData" >> beam.Create(inputs) + | "MLTransform" >> + MLTransform(write_artifact_location=self.gcs_artifact_location + ).with_transform(embedding_config)) + max_ele_pcoll = ( + result_pcoll + | beam.Map(lambda x: round(np.max(x[test_query_column]), 2))) + + assert_that(max_ele_pcoll, equal_to(output)) + + with beam.Pipeline() as p: + result_pcoll = ( + p + | "CreateData" >> beam.Create(inputs) + | "MLTransform" >> + MLTransform(read_artifact_location=self.gcs_artifact_location)) + max_ele_pcoll = ( + result_pcoll + | beam.Map(lambda x: round(np.max(x[test_query_column]), 2))) + + assert_that(max_ele_pcoll, equal_to(output)) + + def test_embeddings_with_inference_args(self): + model_name = DEFAULT_MODEL_NAME + + inference_args = {'convert_to_numpy': False} + embedding_config = SentenceTransformerEmbeddings( + model_name=model_name, + columns=[test_query_column], + inference_args=inference_args) + with beam.Pipeline() as pipeline: + result_pcoll = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def assert_element(element): + assert type(element) == torch.Tensor + + _ = ( + result_pcoll + | beam.Map(lambda x: x[test_query_column]) + | beam.Map(assert_element)) + + def test_mltransform_to_ptransform_with_sentence_transformer(self): + model_name = '' + transforms = [ + SentenceTransformerEmbeddings(columns=['x'], model_name=model_name), + SentenceTransformerEmbeddings( + columns=['y', 'z'], model_name=model_name) + ] + ptransform_mapper = base._MLTransformToPTransformMapper( + transforms=transforms, + artifact_location=self.artifact_location, + artifact_mode=None) + + ptransform_list = ptransform_mapper.create_and_save_ptransform_list() + self.assertTrue(len(ptransform_list) == 2) + + self.assertEqual(type(ptransform_list[0]), RunInference) + expected_columns = [['x'], ['y', 'z']] + for i in range(len(ptransform_list)): + self.assertEqual(type(ptransform_list[i]), RunInference) + self.assertEqual( + type(ptransform_list[i]._model_handler), base._TextEmbeddingHandler) + self.assertEqual( + ptransform_list[i]._model_handler.columns, expected_columns[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.model_name, model_name) + ptransform_list = ( + base._MLTransformToPTransformMapper. + load_transforms_from_artifact_location(self.artifact_location)) + for i in range(len(ptransform_list)): + self.assertEqual(type(ptransform_list[i]), RunInference) + self.assertEqual( + type(ptransform_list[i]._model_handler), base._TextEmbeddingHandler) + self.assertEqual( + ptransform_list[i]._model_handler.columns, expected_columns[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.model_name, model_name) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py new file mode 100644 index 000000000000..1f4c1577eb79 --- /dev/null +++ b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py @@ -0,0 +1,154 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +# Vertex AI Python SDK is required for this module. +# Follow https://cloud.google.com/vertex-ai/docs/python-sdk/use-vertex-ai-python-sdk # pylint: disable=line-too-long +# to install Vertex AI Python SDK. + +from typing import Any +from typing import Dict +from typing import Iterable +from typing import List +from typing import Optional +from typing import Sequence + +from google.auth.credentials import Credentials + +import apache_beam as beam +import vertexai +from apache_beam.ml.inference.base import ModelHandler +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.transforms.base import EmbeddingsManager +from apache_beam.ml.transforms.base import _TextEmbeddingHandler +from vertexai.language_models import TextEmbeddingInput +from vertexai.language_models import TextEmbeddingModel + +__all__ = ["VertexAITextEmbeddings"] + +DEFAULT_TASK_TYPE = "RETRIEVAL_DOCUMENT" +# TODO: https://github.com/apache/beam/issues/29356 +# Can this list be automatically pulled from Vertex SDK? +TASK_TYPE_INPUTS = [ + "RETRIEVAL_DOCUMENT", + "RETRIEVAL_QUERY", + "SEMANTIC_SIMILARITY", + "CLASSIFICATION", + "CLUSTERING" +] +_BATCH_SIZE = 5 # Vertex AI limits requests to 5 at a time. + + +class _VertexAITextEmbeddingHandler(ModelHandler): + """ + Note: Intended for internal use and guarantees no backwards compatibility. + """ + def __init__( + self, + model_name: str, + title: Optional[str] = None, + task_type: str = DEFAULT_TASK_TYPE, + project: Optional[str] = None, + location: Optional[str] = None, + credentials: Optional[Credentials] = None, + ): + vertexai.init(project=project, location=location, credentials=credentials) + self.model_name = model_name + if task_type not in TASK_TYPE_INPUTS: + raise ValueError( + f"task_type must be one of {TASK_TYPE_INPUTS}, got {task_type}") + self.task_type = task_type + self.title = title + + def run_inference( + self, + batch: Sequence[str], + model: Any, + inference_args: Optional[Dict[str, Any]] = None, + ) -> Iterable: + embeddings = [] + batch_size = _BATCH_SIZE + for i in range(0, len(batch), batch_size): + text_batch = batch[i:i + batch_size] + text_batch = [ + TextEmbeddingInput( + text=text, title=self.title, task_type=self.task_type) + for text in text_batch + ] + embeddings_batch = model.get_embeddings(text_batch) + embeddings.extend([el.values for el in embeddings_batch]) + return embeddings + + def load_model(self): + model = TextEmbeddingModel.from_pretrained(self.model_name) + return model + + +class VertexAITextEmbeddings(EmbeddingsManager): + def __init__( + self, + model_name: str, + columns: List[str], + title: Optional[str] = None, + task_type: str = DEFAULT_TASK_TYPE, + project: Optional[str] = None, + location: Optional[str] = None, + credentials: Optional[Credentials] = None, + **kwargs): + """ + Embedding Config for Vertex AI Text Embedding models following + https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings # pylint: disable=line-too-long + Text Embeddings are generated for a batch of text using the Vertex AI SDK. + Embeddings are returned in a list for each text in the batch. Look at + https://cloud.google.com/vertex-ai/docs/generative-ai/learn/model-versioning#stable-versions-available.md # pylint: disable=line-too-long + for more information on model versions and lifecycle. + + Args: + model_name: The name of the Vertex AI Text Embedding model. + columns: The columns containing the text to be embedded. + task_type: The downstream task for the embeddings. Valid values are + RETRIEVAL_QUERY, RETRIEVAL_DOCUMENT, SEMANTIC_SIMILARITY, + CLASSIFICATION, CLUSTERING. For more information on the task type, + look at https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings # pylint: disable=line-too-long + title: Identifier of the text content. + project: The default GCP project for API calls. + location: The default location for API calls. + credentials: Custom credentials for API calls. + Defaults to environment credentials. + """ + self.model_name = model_name + self.project = project + self.location = location + self.credentials = credentials + self.title = title + self.task_type = task_type + super().__init__(columns=columns, **kwargs) + + def get_model_handler(self) -> ModelHandler: + return _VertexAITextEmbeddingHandler( + model_name=self.model_name, + project=self.project, + location=self.location, + credentials=self.credentials, + title=self.title, + task_type=self.task_type, + ) + + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + return ( + RunInference( + model_handler=_TextEmbeddingHandler(self), + inference_args=self.inference_args)) diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py new file mode 100644 index 000000000000..04a730eaefb0 --- /dev/null +++ b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py @@ -0,0 +1,249 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 os +import shutil +import tempfile +import unittest +import uuid + +import apache_beam as beam +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.transforms import base +from apache_beam.ml.transforms.base import MLTransform + +try: + from apache_beam.ml.transforms.embeddings.vertex_ai import VertexAITextEmbeddings +except ImportError: + VertexAITextEmbeddings = None # type: ignore + +# pylint: disable=ungrouped-imports +try: + import tensorflow_transform as tft + from apache_beam.ml.transforms.tft import ScaleTo01 +except ImportError: + tft = None + +test_query = "This is a test" +test_query_column = "feature_1" +model_name: str = "textembedding-gecko@002" + + +@unittest.skipIf( + VertexAITextEmbeddings is None, 'Vertex AI Python SDK is not installed.') +class VertexAIEmbeddingsTest(unittest.TestCase): + def setUp(self) -> None: + self.artifact_location = tempfile.mkdtemp(prefix='_vertex_ai_test') + self.gcs_artifact_location = os.path.join( + 'gs://temp-storage-for-perf-tests/vertex_ai', uuid.uuid4().hex) + + def tearDown(self) -> None: + shutil.rmtree(self.artifact_location) + + def test_vertex_ai_text_embeddings(self): + embedding_config = VertexAITextEmbeddings( + model_name=model_name, columns=[test_query_column]) + with beam.Pipeline() as pipeline: + transformed_pcoll = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def assert_element(element): + assert len(element[test_query_column]) == 768 + + _ = (transformed_pcoll | beam.Map(assert_element)) + + @unittest.skipIf(tft is None, 'Tensorflow Transform is not installed.') + def test_embeddings_with_scale_to_0_1(self): + embedding_config = VertexAITextEmbeddings( + model_name=model_name, + columns=[test_query_column], + ) + with beam.Pipeline() as pipeline: + transformed_pcoll = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config).with_transform( + ScaleTo01(columns=[test_query_column]))) + + def assert_element(element): + assert max(element.feature_1) == 1 + + _ = (transformed_pcoll | beam.Map(assert_element)) + + def pipeline_with_configurable_artifact_location( + self, + pipeline, + embedding_config=None, + read_artifact_location=None, + write_artifact_location=None): + if write_artifact_location: + return ( + pipeline + | MLTransform(write_artifact_location=write_artifact_location). + with_transform(embedding_config)) + elif read_artifact_location: + return ( + pipeline + | MLTransform(read_artifact_location=read_artifact_location)) + else: + raise NotImplementedError + + def test_embeddings_with_read_artifact_location(self): + with beam.Pipeline() as p: + embedding_config = VertexAITextEmbeddings( + model_name=model_name, columns=[test_query_column]) + + with beam.Pipeline() as p: + data = ( + p + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }])) + _ = self.pipeline_with_configurable_artifact_location( + pipeline=data, + embedding_config=embedding_config, + write_artifact_location=self.artifact_location) + + with beam.Pipeline() as p: + data = ( + p + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }, { + test_query_column: test_query + }])) + result_pcoll = self.pipeline_with_configurable_artifact_location( + pipeline=data, read_artifact_location=self.artifact_location) + + def assert_element(element): + assert round(element, 2) == 0.15 + + _ = ( + result_pcoll + | beam.Map(lambda x: max(x[test_query_column])) + # 0.14797046780586243 + | beam.Map(assert_element)) + + def test_with_int_data_types(self): + embedding_config = VertexAITextEmbeddings( + model_name=model_name, columns=[test_query_column]) + with self.assertRaises(TypeError): + with beam.Pipeline() as pipeline: + _ = ( + pipeline + | "CreateData" >> beam.Create([{ + test_query_column: 1 + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def test_with_gcs_artifact_location(self): + with beam.Pipeline() as p: + embedding_config = VertexAITextEmbeddings( + model_name=model_name, columns=[test_query_column]) + + with beam.Pipeline() as p: + data = ( + p + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }])) + _ = self.pipeline_with_configurable_artifact_location( + pipeline=data, + embedding_config=embedding_config, + write_artifact_location=self.gcs_artifact_location) + + with beam.Pipeline() as p: + data = ( + p + | "CreateData" >> beam.Create([{ + test_query_column: test_query + }, { + test_query_column: test_query + }])) + result_pcoll = self.pipeline_with_configurable_artifact_location( + pipeline=data, read_artifact_location=self.gcs_artifact_location) + + def assert_element(element): + assert round(element, 2) == 0.15 + + _ = ( + result_pcoll + | beam.Map(lambda x: max(x[test_query_column])) + # 0.14797046780586243 + | beam.Map(assert_element)) + + def test_mltransform_to_ptransform_with_vertex(self): + model_name = 'textembedding-gecko@002' + transforms = [ + VertexAITextEmbeddings( + columns=['x'], + model_name=model_name, + task_type='RETRIEVAL_DOCUMENT'), + VertexAITextEmbeddings( + columns=['y', 'z'], model_name=model_name, task_type='CLUSTERING') + ] + ptransform_mapper = base._MLTransformToPTransformMapper( + transforms=transforms, + artifact_location=self.artifact_location, + artifact_mode=None) + + ptransform_list = ptransform_mapper.create_and_save_ptransform_list() + self.assertTrue(len(ptransform_list) == 2) + + self.assertEqual(type(ptransform_list[0]), RunInference) + expected_columns = [['x'], ['y', 'z']] + expected_task_type = ['RETRIEVAL_DOCUMENT', 'CLUSTERING'] + for i in range(len(ptransform_list)): + self.assertEqual(type(ptransform_list[i]), RunInference) + self.assertEqual( + type(ptransform_list[i]._model_handler), base._TextEmbeddingHandler) + self.assertEqual( + ptransform_list[i]._model_handler.columns, expected_columns[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.task_type, + expected_task_type[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.model_name, model_name) + ptransform_list = ( + base._MLTransformToPTransformMapper. + load_transforms_from_artifact_location(self.artifact_location)) + for i in range(len(ptransform_list)): + self.assertEqual(type(ptransform_list[i]), RunInference) + self.assertEqual( + type(ptransform_list[i]._model_handler), base._TextEmbeddingHandler) + self.assertEqual( + ptransform_list[i]._model_handler.columns, expected_columns[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.task_type, + expected_task_type[i]) + self.assertEqual( + ptransform_list[i]._model_handler._underlying.model_name, model_name) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/ml/transforms/handlers.py b/sdks/python/apache_beam/ml/transforms/handlers.py index e7d4f52ded85..db6ca849a625 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers.py +++ b/sdks/python/apache_beam/ml/transforms/handlers.py @@ -388,7 +388,7 @@ def _get_transformed_data_schema( transformed_types[name] = typing.Sequence[bytes] # type: ignore[assignment] return transformed_types - def process_data( + def expand( self, raw_data: beam.PCollection[tft_process_handler_input_type] ) -> beam.PCollection[tft_process_handler_output_type]: """ @@ -513,7 +513,7 @@ def process_data( # The schema only contains the columns that are transformed. transformed_dataset = ( - transformed_dataset | "ConvertToRowType" >> + transformed_dataset + | "ConvertToRowType" >> beam.Map(lambda x: beam.Row(**x)).with_output_types(row_type)) - return transformed_dataset diff --git a/sdks/python/apache_beam/ml/transforms/handlers_test.py b/sdks/python/apache_beam/ml/transforms/handlers_test.py index d67d8ec3e705..f13a916824c4 100644 --- a/sdks/python/apache_beam/ml/transforms/handlers_test.py +++ b/sdks/python/apache_beam/ml/transforms/handlers_test.py @@ -298,7 +298,7 @@ def test_tft_process_handler_verify_artifacts(self): transforms=[tft.ScaleTo01(columns=['x'])], artifact_location=self.artifact_location, ) - _ = process_handler.process_data(raw_data) + _ = raw_data | process_handler self.assertTrue( os.path.exists( @@ -315,7 +315,7 @@ def test_tft_process_handler_verify_artifacts(self): raw_data = (p | beam.Create([{'x': np.array([2, 5])}])) process_handler = handlers.TFTProcessHandler( artifact_location=self.artifact_location, artifact_mode='consume') - transformed_data = process_handler.process_data(raw_data) + transformed_data = raw_data | process_handler transformed_data |= beam.Map(lambda x: x.x) # the previous min is 1 and max is 6. So this should scale by (1, 6) @@ -494,7 +494,7 @@ def test_tft_process_handler_unused_column(self): transforms=[scale_to_0_1_fn], artifact_location=self.artifact_location, ) - transformed_pcoll = process_handler.process_data(raw_data) + transformed_pcoll = raw_data | process_handler transformed_pcoll_x = transformed_pcoll | beam.Map(lambda x: x.x) transformed_pcoll_y = transformed_pcoll | beam.Map(lambda x: x.y) assert_that( @@ -520,7 +520,7 @@ def test_consume_mode_with_extra_columns_in_the_input(self): transforms=[tft.ScaleTo01(columns=['x'])], artifact_location=self.artifact_location, ) - _ = process_handler.process_data(raw_data) + _ = raw_data | process_handler test_data = [{ 'x': np.array([2, 5]), 'y': np.array([1, 2]), 'z': 'fake_string' @@ -548,7 +548,7 @@ def test_consume_mode_with_extra_columns_in_the_input(self): raw_data = (p | beam.Create(test_data)) process_handler = handlers.TFTProcessHandler( artifact_location=self.artifact_location, artifact_mode='consume') - transformed_data = process_handler.process_data(raw_data) + transformed_data = raw_data | process_handler transformed_data_x = transformed_data | beam.Map(lambda x: x.x) transformed_data_y = transformed_data | beam.Map(lambda x: x.y) @@ -596,7 +596,7 @@ def test_handler_with_same_input_elements(self): transforms=[tft.ComputeAndApplyVocabulary(columns=['x'])], artifact_location=self.artifact_location, ) - transformed_data = process_handler.process_data(raw_data) + transformed_data = raw_data | process_handler expected_data = [ beam.Row(x=np.array([4])), diff --git a/sdks/python/apache_beam/ml/transforms/tft.py b/sdks/python/apache_beam/ml/transforms/tft.py index c7b8ff015324..8b571d9a685e 100644 --- a/sdks/python/apache_beam/ml/transforms/tft.py +++ b/sdks/python/apache_beam/ml/transforms/tft.py @@ -42,6 +42,7 @@ from typing import Tuple from typing import Union +import apache_beam as beam import tensorflow as tf import tensorflow_transform as tft from apache_beam.ml.transforms.base import BaseOperation @@ -95,6 +96,20 @@ def __init__(self, columns: List[str]) -> None: "Columns are not specified. Please specify the column for the " " op %s" % self.__class__.__name__) + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + from apache_beam.ml.transforms.handlers import TFTProcessHandler + params = {} + artifact_location = kwargs.get('artifact_location') + if not artifact_location: + raise RuntimeError( + "artifact_location is not specified. Please specify the " + "artifact_location for the op %s" % self.__class__.__name__) + + artifact_mode = kwargs.get('artifact_mode') + if artifact_mode: + params['artifact_mode'] = artifact_mode + return TFTProcessHandler(artifact_location=artifact_location, **params) + @tf.function def _split_string_with_delimiter(self, data, delimiter): """ diff --git a/sdks/python/apache_beam/ml/transforms/tft_test.py b/sdks/python/apache_beam/ml/transforms/tft_test.py index 38ded6a809af..9f15db45bd28 100644 --- a/sdks/python/apache_beam/ml/transforms/tft_test.py +++ b/sdks/python/apache_beam/ml/transforms/tft_test.py @@ -711,8 +711,13 @@ def test_count_per_key_on_list(self): ])) def validate_count_per_key(key_vocab_filename): + files = os.listdir(self.artifact_location) + files.remove(base._ATTRIBUTE_FILE_NAME) key_vocab_location = os.path.join( - self.artifact_location, 'transform_fn/assets', key_vocab_filename) + self.artifact_location, + files[0], + 'transform_fn/assets', + key_vocab_filename) with open(key_vocab_location, 'r') as f: key_vocab_list = [line.strip() for line in f] return key_vocab_list diff --git a/sdks/python/apache_beam/ml/transforms/utils.py b/sdks/python/apache_beam/ml/transforms/utils.py index 19bb02c5ae1b..fadf611b0e66 100644 --- a/sdks/python/apache_beam/ml/transforms/utils.py +++ b/sdks/python/apache_beam/ml/transforms/utils.py @@ -17,9 +17,11 @@ __all__ = ['ArtifactsFetcher'] +import os import typing import tensorflow_transform as tft +from apache_beam.ml.transforms import base class ArtifactsFetcher(): @@ -28,8 +30,18 @@ class ArtifactsFetcher(): to the TFTProcessHandlers in MLTransform. """ def __init__(self, artifact_location): - self.artifact_location = artifact_location - self.transform_output = tft.TFTransformOutput(self.artifact_location) + files = os.listdir(artifact_location) + files.remove(base._ATTRIBUTE_FILE_NAME) + # TODO: https://github.com/apache/beam/issues/29356 + # Integrate ArtifactFetcher into MLTransform. + if len(files) > 1: + raise NotImplementedError( + "MLTransform may have been utilized alongside transforms written " + "in TensorFlow Transform, in conjunction with those from different " + "frameworks. Currently, retrieving artifacts from this " + "multi-framework setup is not supported.") + self._artifact_location = os.path.join(artifact_location, files[0]) + self.transform_output = tft.TFTransformOutput(self._artifact_location) def get_vocab_list( self, diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 4fd4b97e82cd..70eb78b6ffc6 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -1020,13 +1020,13 @@ def __getitem__(self, type_param): class CollectionHint(CompositeTypeHint): """ A Collection type-hint. - + Collection[X] defines a type-hint for a collection of homogenous types. 'X' may be either a built-in Python type or another nested TypeConstraint. This represents a collections.abc.Collection type, which implements __contains__, __iter__, and __len__. This acts as a parent type for - sets but has fewer guarantees for mixins. + sets but has fewer guarantees for mixins. """ class CollectionTypeConstraint(SequenceTypeConstraint): def __init__(self, type_param): @@ -1302,6 +1302,8 @@ def is_consistent_with(sub, base): relation, but also handles the special Any type as well as type parameterization. """ + from apache_beam.pvalue import Row + from apache_beam.typehints.row_type import RowTypeConstraint if sub == base: # Common special case. return True @@ -1313,6 +1315,8 @@ def is_consistent_with(sub, base): return all(is_consistent_with(c, base) for c in sub.union_types) elif isinstance(base, TypeConstraint): return base._consistent_with_check_(sub) + elif isinstance(sub, RowTypeConstraint): + return base == Row elif isinstance(sub, TypeConstraint): # Nothing but object lives above any type constraints. return base == object diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index cd6018bfc1fa..6ec2cc0a7565 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -84,6 +84,7 @@ joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 jsonschema-specifications==2023.11.2 +jsonpickle==3.0.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 1fae235ee477..435eb9712917 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -81,6 +81,7 @@ idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonpickle==3.0.2 jsonschema==4.20.0 jsonschema-specifications==2023.11.2 mmh3==4.0.1 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index ab4203ecbe37..51fb324d7c44 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -85,6 +85,7 @@ importlib-resources==6.1.1 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonpickle==3.0.2 jsonschema==4.20.0 jsonschema-specifications==2023.11.2 mmh3==4.0.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 308ffa736207..ce723259aa7c 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -84,6 +84,7 @@ iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 jsonschema==4.20.0 +jsonpickle==3.0.2 jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index 3c232e126ab7..82740ae67c9f 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -133,7 +133,9 @@ autodoc_inherit_docstrings = False autodoc_member_order = 'bysource' autodoc_mock_imports = ["tensorrt", "cuda", "torch", "onnxruntime", "onnx", "tensorflow", "tensorflow_hub", - "tensorflow_transform", "tensorflow_metadata", "transformers", "xgboost", "datatable", "transformers"] + "tensorflow_transform", "tensorflow_metadata", "transformers", "xgboost", "datatable", "transformers", + "sentence_transformers", + ] # Allow a special section for documenting DataFrame API napoleon_custom_sections = ['Differences from pandas'] diff --git a/sdks/python/setup.py b/sdks/python/setup.py index e624f3176bb3..7e6d2217d757 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -298,6 +298,7 @@ def get_portability_package_data(): 'httplib2>=0.8,<0.23.0', 'js2py>=0.74,<1', 'jsonschema>=4.0.0,<5.0.0', + 'jsonpickle>=3.0.0,<4.0.0', # numpy can have breaking changes in minor versions. # Use a strict upper bound. 'numpy>=1.14.3,<1.25.0', # Update pyproject.toml as well. diff --git a/sdks/python/test-suites/tox/py38/build.gradle b/sdks/python/test-suites/tox/py38/build.gradle index b1ed5f88c7c9..1e03b5058083 100644 --- a/sdks/python/test-suites/tox/py38/build.gradle +++ b/sdks/python/test-suites/tox/py38/build.gradle @@ -141,6 +141,10 @@ toxTask "testPy38transformers-430", "py38-transformers-430", "${posargs}" test.dependsOn "testPy38transformers-430" preCommitPyCoverage.dependsOn "testPy38transformers-430" +toxTask "testPy38embeddingsMLTransform", "py38-embeddings", "${posargs}" +test.dependsOn "testPy38embeddingsMLTransform" +preCommitPyCoverage.dependsOn "testPy38embeddingsMLTransform" + toxTask "whitespacelint", "whitespacelint", "${posargs}" task archiveFilesToLint(type: Zip) { diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 28e282460e47..dbe90c084af2 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -419,3 +419,15 @@ commands = # Run all Vertex AI unit tests # Allow exit code 5 (no tests run) so that we can run this command safely on arbitrary subdirectories. /bin/sh -c 'pytest -o junit_suite_name={envname} --junitxml=pytest_{envname}.xml -n 6 -m uses_vertex_ai {posargs}; ret=$?; [ $ret = 5 ] && exit 0 || exit $ret' + + +[testenv:py{38,39,310,311}-embeddings] +deps = + sentence-transformers==2.2.2 +extras = test,gcp +commands = + # Log aiplatform and its dependencies version for debugging + /bin/sh -c "pip freeze | grep -E sentence-transformers" + /bin/sh -c "pip freeze | grep -E google-cloud-aiplatform" + # Allow exit code 5 (no tests run) so that we can run this command safely on arbitrary subdirectories. + /bin/sh -c 'pytest apache_beam/ml/transforms/embeddings -o junit_suite_name={envname} --junitxml=pytest_{envname}.xml -n 6 {posargs}; ret=$?; [ $ret = 5 ] && exit 0 || exit $ret' From b91a5174d149145ceab4477bcfcaff857beeac2f Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Mon, 11 Dec 2023 14:27:51 -0500 Subject: [PATCH 089/224] [Python] Log dependencies in runtime environment (#29705) * log runtime dependencies * log py version * Update boot.go * move print block * fix indent * fix indent * fix indent --- sdks/python/container/boot.go | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go index ded10a44204a..cbf05bbd04a7 100644 --- a/sdks/python/container/boot.go +++ b/sdks/python/container/boot.go @@ -409,6 +409,9 @@ func installSetupPackages(ctx context.Context, logger *tools.Logger, files []str if err := pipInstallPackage(ctx, logger, files, workDir, workflowFile, false, true, nil); err != nil { return fmt.Errorf("failed to install workflow: %v", err) } + if err := logRuntimeDependencies(ctx, logger); err != nil { + logger.Warnf(ctx, "couldn't fetch the runtime python dependencies: %v", err) + } return nil } @@ -453,3 +456,28 @@ func processArtifactsInSetupOnlyMode() { log.Fatalf("Failed to install required packages: %v", setupErr) } } + +// logRuntimeDependencies logs the python dependencies +// installed in the runtime environment. +func logRuntimeDependencies(ctx context.Context, logger *tools.Logger) error { + logger.Printf(ctx, "Logging runtime dependencies:") + pythonVersion, err := expansionx.GetPythonVersion() + if err != nil { + return err + } + logger.Printf(ctx, "Using Python version:") + args := []string{"--version"} + bufLogger := tools.NewBufferedLogger(logger) + if err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...); err != nil { + bufLogger.FlushAtError(ctx) + } else { + bufLogger.FlushAtDebug(ctx) + } + args = []string{"-m", "pip", "freeze"} + if err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...); err != nil { + bufLogger.FlushAtError(ctx) + } else { + bufLogger.FlushAtDebug(ctx) + } + return nil +} From f93423009fd2d73a76e44332b09f9fe38d75cd2b Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 11 Dec 2023 23:17:49 +0000 Subject: [PATCH 090/224] Save Job IDs of dataflow load tests (#29693) * Publish job (#29685) * Update schema if the default schema is different than the table schema * Add job id label to the BQ publisher for dataflow jobs * Append job id to each metric * Fix lint --- .../load_tests/load_test_metrics_utils.py | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 1ff46a3f7d19..d1da4667dcb8 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -33,6 +33,8 @@ import logging import time import uuid +from typing import Any +from typing import Dict from typing import List from typing import Mapping from typing import Optional @@ -45,6 +47,7 @@ from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricResults from apache_beam.metrics.metric import MetricsFilter +from apache_beam.runners.dataflow.dataflow_runner import DataflowPipelineResult from apache_beam.runners.runner import PipelineResult from apache_beam.transforms.window import TimestampedValue from apache_beam.utils.timestamp import Timestamp @@ -65,6 +68,7 @@ SUBMIT_TIMESTAMP_LABEL = 'timestamp' METRICS_TYPE_LABEL = 'metric' VALUE_LABEL = 'value' +JOB_ID_LABEL = 'job_id' SCHEMA = [{ 'name': ID_LABEL, 'field_type': 'STRING', 'mode': 'REQUIRED' @@ -80,6 +84,8 @@ 'mode': 'REQUIRED' }, { 'name': VALUE_LABEL, 'field_type': 'FLOAT', 'mode': 'REQUIRED' + }, { + 'name': JOB_ID_LABEL, 'field_type': 'STRING', 'mode': 'NULLABLE' }] _LOGGER = logging.getLogger(__name__) @@ -254,13 +260,27 @@ def publish_metrics( # Under each key there is list of objects of each metric type. It is # required to prepare metrics for publishing purposes. Expected is to have # a list of dictionaries matching the schema. + insert_dicts = self._prepare_all_metrics(metrics, metric_id) insert_dicts += self._prepare_extra_metrics(metric_id, extra_metrics) + + # Add job id for dataflow jobs for easier debugging. + job_id = None + if isinstance(result, DataflowPipelineResult): + job_id = result.job_id() + self._add_job_id_to_metrics(insert_dicts, job_id) + if len(insert_dicts) > 0: for publisher in self.publishers: publisher.publish(insert_dicts) + def _add_job_id_to_metrics(self, metrics: List[Dict[str, Any]], + job_id) -> List[Dict[str, Any]]: + for metric in metrics: + metric[JOB_ID_LABEL] = job_id + return metrics + def _prepare_extra_metrics( self, metric_id: str, extra_metrics: Optional[dict] = None): ts = time.time() @@ -479,6 +499,12 @@ def _get_or_create_table(self, bq_schemas, dataset): table = bigquery.Table(table_ref, schema=bq_schemas) self._bq_table = self._client.create_table(table) + def _update_schema(self): + table_schema = self._bq_table.schema + if self.schema and len(table_schema) != self.schema: + self._bq_table.schema = self._prepare_schema() + self._bq_table = self._client.update_table(self._bq_table, ["schema"]) + def _get_dataset(self, dataset_name): bq_dataset_ref = self._client.dataset(dataset_name) try: @@ -490,6 +516,8 @@ def _get_dataset(self, dataset_name): return bq_dataset def save(self, results): + # update schema if needed + self._update_schema() return self._client.insert_rows(self._bq_table, results) From eff6c203991909613f2871179c265bab095002a2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 12 Dec 2023 07:54:46 -0500 Subject: [PATCH 091/224] Bump actions/setup-java from 3 to 4 (#29573) Bumps [actions/setup-java](https://github.com/actions/setup-java) from 3 to 4. - [Release notes](https://github.com/actions/setup-java/releases) - [Commits](https://github.com/actions/setup-java/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/setup-java dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../beam_PostCommit_Java_Jpms_Direct_Java21.yml | 2 +- .github/workflows/build_release_candidate.yml | 10 +++++----- .github/workflows/code_completion_plugin_tests.yml | 2 +- .github/workflows/run_rc_validation.yml | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index e6740017afa2..2c918b354977 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -71,7 +71,7 @@ jobs: github_token: ${{ secrets.GITHUB_TOKEN }} github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - name: Set up Java - uses: actions/setup-java@v3.11.0 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: | diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 6f1499a63769..abf583b4dda6 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -59,7 +59,7 @@ jobs: ref: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" repository: apache/beam - name: Install Java 8 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '8' @@ -107,7 +107,7 @@ jobs: echo "Must provide an apache password to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" fi - name: Install Java 8 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '8' @@ -243,7 +243,7 @@ jobs: - name: Free Disk Space (Ubuntu) uses: jlumbroso/free-disk-space@v1.3.0 - name: Install Java 11 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '11' @@ -255,7 +255,7 @@ jobs: - run: echo "JAVA11_HOME=${JAVA_HOME}" >> "$GITHUB_OUTPUT" id: export-java11 - name: Install Java 8 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '8' @@ -306,7 +306,7 @@ jobs: with: node-version: '16' - name: Install Java 8 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '8' diff --git a/.github/workflows/code_completion_plugin_tests.yml b/.github/workflows/code_completion_plugin_tests.yml index 38ffd2fbd3f4..9dbb5f6452f2 100644 --- a/.github/workflows/code_completion_plugin_tests.yml +++ b/.github/workflows/code_completion_plugin_tests.yml @@ -73,7 +73,7 @@ jobs: # Setup Java environment for the next steps - name: Setup Java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: '11' diff --git a/.github/workflows/run_rc_validation.yml b/.github/workflows/run_rc_validation.yml index 35fb3ce05eae..130a2ce25eea 100644 --- a/.github/workflows/run_rc_validation.yml +++ b/.github/workflows/run_rc_validation.yml @@ -120,7 +120,7 @@ jobs: - name: Setup Java JDK - uses: actions/setup-java@v3.8.0 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 11 @@ -187,7 +187,7 @@ jobs: uses: azure/setup-kubectl@v3 - name: Setup Java JDK - uses: actions/setup-java@v3.8.0 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 11 From 4431479fd905027e5ab0e1f17fcbab62f41fd9ae Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Tue, 12 Dec 2023 12:37:57 +0100 Subject: [PATCH 092/224] Add InfluxDB credentials as env variables where they are missing --- .github/workflows/beam_Java_JMH.yml | 2 ++ .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 ++ .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 ++ .../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_Tpcds_Dataflow.yml | 4 +++- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 ++ .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 ++ 10 files changed, 21 insertions(+), 1 deletion(-) diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml index a90d1344856a..7764e7223472 100644 --- a/.github/workflows/beam_Java_JMH.yml +++ b/.github/workflows/beam_Java_JMH.yml @@ -49,6 +49,8 @@ env: GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} INFLUXDB_HOST: http://10.128.0.96:8086 INFLUXDB_DATABASE: beam_test_metrics + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_Java_JMH: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index b501aed67f2a..ab4c63e772ea 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -72,6 +72,8 @@ env: --enforceEncodability=true --enforceImmutability=true --runner=DataflowRunner + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Dataflow: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 0a4cdb8cd174..63321d8aa6c1 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -73,6 +73,8 @@ env: --enforceEncodability=true --enforceImmutability=true --runner=DataflowRunner + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Dataflow_V2: 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 20a061a0bfee..1a4c38c46914 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -72,6 +72,8 @@ env: --enforceEncodability=true --enforceImmutability=true --runner=DataflowRunner + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Dataflow_V2_Java: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 1d98192b8255..5e1975b9ae83 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -67,6 +67,8 @@ env: --enforceEncodability=true --enforceImmutability=true --runner=DirectRunner + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Direct: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index 4feedafc341c..405a6399d2e8 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -66,6 +66,8 @@ env: --suite=SMOKE --streamTimeout=60 --runner=FlinkRunner + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Flink: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index e1421397f7d6..29291f441ce0 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -66,6 +66,8 @@ env: --suite=SMOKE --streamTimeout=60 --streaming=false + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} jobs: beam_PostCommit_Java_Nexmark_Spark: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index 12b74ae4e443..9704b7f03d61 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -57,7 +57,9 @@ env: --sourceType=PARQUET --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned --resultsDirectory=gs://beam-tpcds/results/dataflow/ - --tpcParallel=1 + --tpcParallel=1 + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} tpcdsBigQueryArgs: | --bigQueryTable=tpcds --bigQueryDataset=tpcds diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index 5174baebcfb1..820a4c9792cc 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -55,6 +55,8 @@ env: --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned --resultsDirectory=gs://beam-tpcds/results/flink/ --tpcParallel=1 + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} tpcdsBigQueryArgs: | --bigQueryTable=tpcds --bigQueryDataset=tpcds diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 370c9c406d46..580cfc83132c 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -54,6 +54,8 @@ env: --dataDirectory=gs://beam-tpcds/datasets/parquet/nonpartitioned --resultsDirectory=gs://beam-tpcds/results/spark3-rdd/ --tpcParallel=1 + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} tpcdsBigQueryArgs: | --bigQueryTable=tpcds --bigQueryDataset=tpcds From f9479c6819072fda6ab226d438dd799b9402ed2a Mon Sep 17 00:00:00 2001 From: Julian Braha Date: Tue, 12 Dec 2023 08:38:37 -0500 Subject: [PATCH 093/224] Support BigQuery tables with spaces (#29649) --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 86f7a7f122b7..686c29c13e33 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -577,7 +577,7 @@ public class BigQueryIO { private static final String DATASET_REGEXP = "[-\\w.]{1,1024}"; /** Regular expression that matches Table IDs. */ - private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}"; + private static final String TABLE_REGEXP = "[-\\w$@ ]{1,1024}"; /** * Matches table specifications in the form {@code "[project_id]:[dataset_id].[table_id]"} or From d24c40a125c12a1fee8a55dd63b9d317992f1e9f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 12 Dec 2023 08:39:17 -0500 Subject: [PATCH 094/224] Bump com.gradle.common-custom-user-data-gradle-plugin (#29632) Bumps com.gradle.common-custom-user-data-gradle-plugin from 1.11.3 to 1.12.1. --- updated-dependencies: - dependency-name: com.gradle.common-custom-user-data-gradle-plugin 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> --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index 3bd606327703..11833e818920 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -25,7 +25,7 @@ pluginManagement { plugins { id("com.gradle.enterprise") version "3.15.1" - id("com.gradle.common-custom-user-data-gradle-plugin") version "1.11.3" + id("com.gradle.common-custom-user-data-gradle-plugin") version "1.12.1" } From 5e00feea6c6b355092a9746a0af0d0989a42f371 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Tue, 12 Dec 2023 07:15:34 -0800 Subject: [PATCH 095/224] Update links to point to About Beam ML (#29709) * Update Beam ML links * Update Beam ML links --- examples/notebooks/beam-ml/README.md | 6 +++--- .../beam-ml/automatic_model_refresh.ipynb | 10 ---------- examples/notebooks/beam-ml/mltransform_basic.ipynb | 2 +- examples/notebooks/beam-ml/per_key_models.ipynb | 2 +- .../notebooks/beam-ml/run_custom_inference.ipynb | 2 +- .../run_inference_pytorch_tensorflow_sklearn.ipynb | 4 ++-- .../beam-ml/speech_emotion_tensorflow.ipynb | 2 +- .../apache_beam/examples/inference/README.md | 2 +- sdks/python/apache_beam/ml/inference/__init__.py | 2 +- sdks/python/apache_beam/ml/inference/base.py | 14 +++++++------- .../site/content/en/documentation/ml/about-ml.md | 2 +- .../en/documentation/ml/inference-overview.md | 4 ++-- .../en/documentation/ml/multi-model-pipelines.md | 2 +- .../site/content/en/documentation/ml/overview.md | 4 ++-- .../site/content/en/documentation/sdks/python.md | 2 +- .../transforms/python/elementwise/runinference.md | 2 +- .../en/get-started/resources/learning-resources.md | 2 +- 17 files changed, 27 insertions(+), 37 deletions(-) diff --git a/examples/notebooks/beam-ml/README.md b/examples/notebooks/beam-ml/README.md index 0ae937e9e284..5a90f1c68b49 100644 --- a/examples/notebooks/beam-ml/README.md +++ b/examples/notebooks/beam-ml/README.md @@ -25,13 +25,13 @@ transform. This transform allows you to make predictions and inference on data with machine learning (ML) models. The model handler abstracts the user from the configuration needed for specific frameworks, such as Tensorflow, PyTorch, and others. For a full list of supported frameworks, -see the Apache Beam [Machine Learning](https://beam.apache.org/documentation/sdks/python-machine-learning) page. +see the [About Beam ML](https://beam.apache.org/documentation/ml/about-ml/) page. ## Using The Notebooks These notebooks illustrate ways to use Apache Beam's RunInference transforms, as well as different -use cases for [ModelHandler](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.ModelHandler) implementations. -Beam comes with [multiple ModelHandler implementations](https://beam.apache.org/documentation/sdks/python-machine-learning/#modify-a-pipeline-to-use-an-ml-model). +use cases for [`ModelHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.ModelHandler) implementations. +Beam comes with [multiple `ModelHandler` implementations](https://beam.apache.org/documentation/ml/about-ml/#modify-a-python-pipeline-to-use-an-ml-model). ### Loading the Notebooks diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index cf05979c5b33..b0e9bc2f53e4 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -15,16 +15,6 @@ } }, "cells": [ - { - "cell_type": "markdown", - "metadata": { - "id": "view-in-github", - "colab_type": "text" - }, - "source": [ - "\"Open" - ] - }, { "cell_type": "code", "source": [ diff --git a/examples/notebooks/beam-ml/mltransform_basic.ipynb b/examples/notebooks/beam-ml/mltransform_basic.ipynb index 43068624eb82..b0af96d08593 100644 --- a/examples/notebooks/beam-ml/mltransform_basic.ipynb +++ b/examples/notebooks/beam-ml/mltransform_basic.ipynb @@ -55,7 +55,7 @@ "id": "d3b81cf2-8603-42bd-995e-9e14631effd0" }, "source": [ - "This notebook demonstrates how to use `MLTransform` to preprocess data for machine learning workflows. Apache Beam provides a set of transforms for preprocessing data for training and inference. The `MLTransform` class wraps various transforms in one PTransform, simplifying your workflow. For a list of available preprocessing transforms see the [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/#transforms) page in Apache Beam documentation.\n", + "This notebook demonstrates how to use `MLTransform` to preprocess data for machine learning workflows. Apache Beam provides a set of transforms for preprocessing data for training and inference. The `MLTransform` class wraps various transforms in one `PTransform`, simplifying your workflow. For a list of available preprocessing transforms see the [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/#transforms) page in the Apache Beam documentation.\n", "\n", "This notebook uses data processing transforms defined in the [apache_beam/ml/transforms/tft](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html) module." ] diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index 53845c0b3e19..3e71c1d119a2 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -402,7 +402,7 @@ { "cell_type": "markdown", "source": [ - "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If your worker might run out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." + "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If your worker might run out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/ml/about-ml/#use-a-keyed-modelhandler-object)." ], "metadata": { "id": "IP65_5nNGIb8" diff --git a/examples/notebooks/beam-ml/run_custom_inference.ipynb b/examples/notebooks/beam-ml/run_custom_inference.ipynb index a66c5847de0e..2ca3b69bb72e 100644 --- a/examples/notebooks/beam-ml/run_custom_inference.ipynb +++ b/examples/notebooks/beam-ml/run_custom_inference.ipynb @@ -60,7 +60,7 @@ "NLP locates named entities in unstructured text and classifies the entities using pre-defined labels, such as person name, organization, date, and so on.\n", "\n", "This example illustrates how to use the popular `spaCy` package to load a machine learning (ML) model and perform inference in an Apache Beam pipeline using the RunInference `PTransform`.\n", - "For more information about the RunInference API, see [Machine Learning](https://beam.apache.org/documentation/sdks/python-machine-learning) in the Apache Beam documentation." + "For more information about the RunInference API, see [About Beam ML](https://beam.apache.org/documentation/ml/about-ml) in the Apache Beam documentation." ] }, { diff --git a/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb b/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb index a314f6cd7116..115b70b11e94 100644 --- a/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb +++ b/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb @@ -71,7 +71,7 @@ "You can use Apache Beam versions 2.40.0 and later with the [RunInference API](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.RunInference) for local and remote inference with batch and streaming pipelines.\n", "The RunInference API leverages Apache Beam concepts, such as the `BatchElements` transform and the `Shared` class, to support models in your pipelines that create transforms optimized for machine learning inference.\n", "\n", - "For more information about the RunInference API, see [Machine Learning](https://beam.apache.org/documentation/sdks/python-machine-learning) in the Apache Beam documentation." + "For more information about the RunInference API, see [About Beam ML](https://beam.apache.org/documentation/ml/about-ml) in the Apache Beam documentation." ] }, { @@ -221,7 +221,7 @@ " # Pad the token tensors to max length to make sure that all of the tensors\n", " # are of the same length and stack-able by the RunInference API. Normally, you would batch first\n", " # then tokenize the batch, padding each tensor the max length in the batch.\n", - " # See: https://beam.apache.org/documentation/sdks/python-machine-learning/#unable-to-batch-tensor-elements\n", + " # See: https://beam.apache.org/documentation/ml/about-ml/#unable-to-batch-tensor-elements\n", " tokens = self._tokenizer(text_input, return_tensors='pt', padding='max_length', max_length=512)\n", " # Squeeze because tokenization adds an extra dimension, which is empty,\n", " # in this case because we tokenize one element at a time.\n", diff --git a/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb b/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb index 098cb150bfd2..c2dfb06a6e67 100644 --- a/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb +++ b/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb @@ -132,7 +132,7 @@ "* **[IPython](https://ipython.readthedocs.io/en/stable/index.html)**: Creates visualizations for multimedia content. Here we have used it for playing audio files.\n", "* **[Sklearn](https://scikit-learn.org/stable/index.html)**: Offers comprehensive tools for Machine Learning. Here we have used it for preprocessing and splitting the data.\n", "* **[TensorFlow](https://www.tensorflow.org/api_docs)** and **[Keras](https://keras.io/api/)**: Enables building and training complex Machine Learning and Deep Learning model.\n", - "* **[TFModelHandlerNumpy](https://beam.apache.org/documentation/sdks/python-machine-learning/#tensorflow)**: Defines the configuration used to load/use the model that we train. We use TFModelHandlerNumpy because the model was trained with TensorFlow and takes numpy arrays as input.\n", + "* **[TFModelHandlerNumpy](https://beam.apache.org/documentation/ml/about-ml/#tensorflow)**: Defines the configuration used to load/use the model that we train. We use TFModelHandlerNumpy because the model was trained with TensorFlow and takes numpy arrays as input.\n", "* **[RunInference](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.html#apache_beam.ml.inference.RunInference)**: Loads the model and obtains predictions as part of the Apache Beam pipeline. For more information, see docs on prediction and inference.\n", "* **[Apache Beam](https://beam.apache.org/documentation/)**: Builds a pipeline for Image Processing." ] diff --git a/sdks/python/apache_beam/examples/inference/README.md b/sdks/python/apache_beam/examples/inference/README.md index cd92d9c127ee..3bb68440ed60 100644 --- a/sdks/python/apache_beam/examples/inference/README.md +++ b/sdks/python/apache_beam/examples/inference/README.md @@ -113,7 +113,7 @@ pip install skl2onnx ### Additional resources For more information, see the -[Machine Learning](/documentation/sdks/python-machine-learning) and the +[About Beam ML](/documentation/ml/about-ml) and the [RunInference transform](/documentation/transforms/python/elementwise/runinference) documentation. --- diff --git a/sdks/python/apache_beam/ml/inference/__init__.py b/sdks/python/apache_beam/ml/inference/__init__.py index 0433761ded8a..3ba45ad7def0 100644 --- a/sdks/python/apache_beam/ml/inference/__init__.py +++ b/sdks/python/apache_beam/ml/inference/__init__.py @@ -21,7 +21,7 @@ Note: on top of the frameworks captured in submodules below, Beam also has a supported TensorFlow model handler via the tfx-bsl library. See -https://beam.apache.org/documentation/sdks/python-machine-learning/#tensorflow +https://beam.apache.org/documentation/ml/about-ml/#tensorflow for more information on using TensorFlow in Beam. """ diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 90ba22fcd314..63038f9c9b7d 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -143,7 +143,7 @@ class KeyModelPathMapping(Generic[KeyT]): information see the KeyedModelHandler documentation https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler documentation and the website section on model updates - https://beam.apache.org/documentation/sdks/python-machine-learning/#automatic-model-refresh + https://beam.apache.org/documentation/ml/about-ml/#automatic-model-refresh """ keys: List[KeyT] update_path: str @@ -224,7 +224,7 @@ def update_model_path(self, model_path: Optional[str] = None): used when a ModelHandler represents a single model, not multiple models. This will be true in most cases. For more information see the website section on model updates - https://beam.apache.org/documentation/sdks/python-machine-learning/#automatic-model-refresh + https://beam.apache.org/documentation/ml/about-ml/#automatic-model-refresh """ pass @@ -239,7 +239,7 @@ def update_model_paths( the KeyedModelHandler documentation https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler documentation and the website section on model updates - https://beam.apache.org/documentation/sdks/python-machine-learning/#automatic-model-refresh + https://beam.apache.org/documentation/ml/about-ml/#automatic-model-refresh """ pass @@ -452,7 +452,7 @@ def __init__( KeyedModelHandlers support Automatic Model Refresh to update your model to a newer version without stopping your streaming pipeline. For an overview of this feature, see - https://beam.apache.org/documentation/sdks/python-machine-learning/#automatic-model-refresh + https://beam.apache.org/documentation/ml/about-ml/#automatic-model-refresh To use this feature with a KeyedModelHandler that has many models per key, @@ -465,7 +465,7 @@ def __init__( will update the model corresponding to keys 'k1' and 'k2' with path 'update/path/1' and the model corresponding to 'k3' with 'update/path/2'. In order to do a side input update: (1) all restrictions mentioned in - https://beam.apache.org/documentation/sdks/python-machine-learning/#automatic-model-refresh + https://beam.apache.org/documentation/ml/about-ml/#automatic-model-refresh must be met, (2) all update_paths must be non-empty, even if they are not being updated from their original values, and (3) the set of keys originally defined cannot change. This means that if originally you have @@ -486,7 +486,7 @@ def __init__( memory (OOM) exception. To avoid this issue, use the parameter `max_models_per_worker_hint` to limit the number of models that are loaded at the same time. For more information about memory management, see - `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long + `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long Args: @@ -498,7 +498,7 @@ def __init__( example, if your worker has 8 GB of memory provisioned and your workers take up 1 GB each, you should set this to 7 to allow all models to sit in memory with some buffer. For more information about memory management, - see `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long + see `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long """ self._metrics_collectors: Dict[str, _MetricsCollector] = {} self._default_metrics_collector: _MetricsCollector = None diff --git a/website/www/site/content/en/documentation/ml/about-ml.md b/website/www/site/content/en/documentation/ml/about-ml.md index 1753c0ed56c0..70ad546c40a1 100644 --- a/website/www/site/content/en/documentation/ml/about-ml.md +++ b/website/www/site/content/en/documentation/ml/about-ml.md @@ -61,7 +61,7 @@ To keep your model up to date and performing well as your data grows and evolves ## Use RunInference -The [RunInference API](/documentation/sdks/python-machine-learning/) is a `PTransform` optimized for machine learning inferences that lets you efficiently use ML models in your pipelines. The API includes the following features: +The RunInference API is a `PTransform` optimized for machine learning inferences that lets you efficiently use ML models in your pipelines. The API includes the following features: - To efficiently feed your model, dynamically batches inputs based on pipeline throughput using Apache Beam's `BatchElements` transform. - To balance memory and throughput usage, determines the optimal number of models to load using a central model manager. Shares these models across threads and processes as needed to maximize throughput. diff --git a/website/www/site/content/en/documentation/ml/inference-overview.md b/website/www/site/content/en/documentation/ml/inference-overview.md index a79c9a6b01e5..1821102c17fe 100644 --- a/website/www/site/content/en/documentation/ml/inference-overview.md +++ b/website/www/site/content/en/documentation/ml/inference-overview.md @@ -38,7 +38,7 @@ Beam provides different ways to implement inference as part of your pipeline. Yo -The RunInference API is available with the Beam Python SDK versions 2.40.0 and later. You can use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Starting with Apache Beam 2.40.0, PyTorch and Scikit-learn frameworks are supported. Tensorflow models are supported through `tfx-bsl`. For more deatils about using RunInference with Python, see [Machine Learning with Python](/documentation/sdks/python-machine-learning/). +The RunInference API is available with the Beam Python SDK versions 2.40.0 and later. You can use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Starting with Apache Beam 2.40.0, PyTorch and Scikit-learn frameworks are supported. Tensorflow models are supported through `tfx-bsl`. For more deatils about using RunInference with Python, see [About Beam ML](/documentation/ml/about-ml). The RunInference API is available with the Beam Java SDK versions 2.41.0 and later through Apache Beam's [Multi-language Pipelines framework](/documentation/programming-guide/#multi-language-pipelines). For information about the Java wrapper transform, see [RunInference.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java). To try it out, see the [Java Sklearn Mnist Classification example](https://github.com/apache/beam/tree/master/examples/multi-language). @@ -47,7 +47,7 @@ You can create multiple types of transforms using the RunInference API: the API {{< table >}} | Task | Example | | ------- | ---------------| -| I want to use the RunInference transform | [Modify a Python pipeline to use an ML model](/documentation/sdks/python-machine-learning/#modify-a-python-pipeline-to-use-an-ml-model) | +| I want to use the RunInference transform | [Modify a Python pipeline to use an ML model](/documentation/ml/about-ml/#modify-a-python-pipeline-to-use-an-ml-model) | | I want to use RunInference with PyTorch | [Use RunInference with PyTorch](/documentation/transforms/python/elementwise/runinference-pytorch/) | | I want to use RunInference with Sklearn | [Use RunInference with Sklearn](/documentation/transforms/python/elementwise/runinference-sklearn/) | | I want to use pre-trained models (PyTorch, Scikit-learn, or TensorFlow) | [Use pre-trained models](/documentation/ml/about-ml/#use-pre-trained-models) |: diff --git a/website/www/site/content/en/documentation/ml/multi-model-pipelines.md b/website/www/site/content/en/documentation/ml/multi-model-pipelines.md index c42c8b8ae661..dcb96e17a468 100644 --- a/website/www/site/content/en/documentation/ml/multi-model-pipelines.md +++ b/website/www/site/content/en/documentation/ml/multi-model-pipelines.md @@ -33,7 +33,7 @@ all of those steps together by encapsulating them in a single Apache Beam Direct resilient and scalable end-to-end machine learning systems. To deploy your machine learning model in an Apache Beam pipeline, use -the [`RunInferenceAPI`](/documentation/sdks/python-machine-learning/), which +the [`RunInferenceAPI`](/documentation/ml/about-ml), which facilitates the integration of your model as a `PTransform` step in your DAG. Composing multiple `RunInference` transforms within a single DAG makes it possible to build a pipeline that consists of multiple ML models. In this way, Apache Beam supports the development of complex ML systems. diff --git a/website/www/site/content/en/documentation/ml/overview.md b/website/www/site/content/en/documentation/ml/overview.md index 63aaf0f86e20..3a49d40548e8 100644 --- a/website/www/site/content/en/documentation/ml/overview.md +++ b/website/www/site/content/en/documentation/ml/overview.md @@ -54,7 +54,7 @@ Beam provides different ways to implement inference as part of your pipeline. Yo ### RunInference -The RunInfernce API is available with the Beam Python SDK versions 2.40.0 and later. You can use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Starting with Apache Beam 2.40.0, PyTorch and Scikit-learn frameworks are supported. Tensorflow models are supported through `tfx-bsl`. For more deatils about using RunInference with Python, see [Machine Learning with Python](/documentation/sdks/python-machine-learning/). +The RunInfernce API is available with the Beam Python SDK versions 2.40.0 and later. You can use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Starting with Apache Beam 2.40.0, PyTorch and Scikit-learn frameworks are supported. Tensorflow models are supported through `tfx-bsl`. For more deatils about using RunInference, see [About Beam ML](/documentation/ml/about-ml). The RunInference API is available with the Beam Java SDK versions 2.41.0 and later through Apache Beam's [Multi-language Pipelines framework](/documentation/programming-guide/#multi-language-pipelines). For information about the Java wrapper transform, see [RunInference.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java). To try it out, see the [Java Sklearn Mnist Classification example](https://github.com/apache/beam/tree/master/examples/multi-language). @@ -63,7 +63,7 @@ You can create multiple types of transforms using the RunInference API: the API {{< table >}} | Task | Example | | ------- | ---------------| -| I want to use the RunInference transform | [Modify a Python pipeline to use an ML model](/documentation/sdks/python-machine-learning/#modify-a-python-pipeline-to-use-an-ml-model) | +| I want to use the RunInference transform | [Modify a Python pipeline to use an ML model](/documentation/ml/about-ml/#modify-a-python-pipeline-to-use-an-ml-model) | | I want to use RunInference with PyTorch | [Use RunInference with PyTorch](/documentation/transforms/python/elementwise/runinference-pytorch/) | | I want to use RunInference with Sklearn | [Use RunInference with Sklearn](/documentation/transforms/python/elementwise/runinference-sklearn/) | | I want to use pre-trained models (PyTorch, Scikit-learn, or TensorFlow) | [Use pre-trained models](/documentation/ml/about-ml/#use-pre-trained-models) | diff --git a/website/www/site/content/en/documentation/sdks/python.md b/website/www/site/content/en/documentation/sdks/python.md index 2902001066dd..f51218327676 100644 --- a/website/www/site/content/en/documentation/sdks/python.md +++ b/website/www/site/content/en/documentation/sdks/python.md @@ -52,7 +52,7 @@ To integrate machine learning models into your pipelines for making inferences, [library from `tfx_bsl`](https://github.com/tensorflow/tfx-bsl/tree/master/tfx_bsl/beam). You can create multiple types of transforms using the RunInference API: the API takes multiple types of setup parameters from model handlers, and the parameter type determines the model implementation. For more information, -see [Machine Learning](/documentation/sdks/python-machine-learning). +see [About Beam ML](/documentation/ml/about-ml). [TensorFlow Extended (TFX)](https://www.tensorflow.org/tfx) is an end-to-end platform for deploying production ML pipelines. TFX is integrated with Beam. For more information, see [TFX user guide](https://www.tensorflow.org/tfx/guide). diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/runinference.md b/website/www/site/content/en/documentation/transforms/python/elementwise/runinference.md index 369b74714424..47944b9a232f 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/runinference.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/runinference.md @@ -31,7 +31,7 @@ limitations under the License. Uses models to do local and remote inference. A `RunInference` transform performs inference on a `PCollection` of examples using a machine learning (ML) model. The transform outputs a `PCollection` that contains the input examples and output predictions. Avaliable in Apache Beam 2.40.0 and later versions. -For more information, read about Beam RunInference APIs on [Machine Learning with Python](https://beam.apache.org/documentation/sdks/python-machine-learning) page and explore [RunInference API pipeline examples](https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/inference). +For more information about Beam RunInference APIs, see the [About Beam ML](https://beam.apache.org/documentation/ml/about-ml) page and the [RunInference API pipeline](https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/inference) examples. ## Examples diff --git a/website/www/site/content/en/get-started/resources/learning-resources.md b/website/www/site/content/en/get-started/resources/learning-resources.md index abd4566f4d3c..14bd90ee80b9 100644 --- a/website/www/site/content/en/get-started/resources/learning-resources.md +++ b/website/www/site/content/en/get-started/resources/learning-resources.md @@ -69,7 +69,7 @@ If you have additional material that you would like to see here, please let us k ### Machine Learning -* **[Machine Learning with Python using the RunInference API](/documentation/sdks/python-machine-learning/)** - Use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Follow the [RunInference API pipeline examples](https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/inference) to do image classification, image segmentation, language modeling, and MNIST digit classification. See examples of [RunInference transforms](/documentation/transforms/python/elementwise/runinference/). +* **[Machine Learning using the RunInference API](/documentation/ml/about-ml)** - Use Apache Beam with the RunInference API to use machine learning (ML) models to do local and remote inference with batch and streaming pipelines. Follow the [RunInference API pipeline examples](https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/inference) to do image classification, image segmentation, language modeling, and MNIST digit classification. See examples of [RunInference transforms](/documentation/transforms/python/elementwise/runinference/). * **[Machine Learning Preprocessing and Prediction](https://cloud.google.com/dataflow/examples/molecules-walkthrough)** - Predict the molecular energy from data stored in the [Spatial Data File](https://en.wikipedia.org/wiki/Spatial_Data_File) (SDF) format. Train a [TensorFlow](https://www.tensorflow.org/) model with [tf.Transform](https://github.com/tensorflow/transform) for preprocessing in Python. This also shows how to create batch and streaming prediction pipelines in Apache Beam. * **[Machine Learning Preprocessing](https://cloud.google.com/blog/products/ai-machine-learning/pre-processing-tensorflow-pipelines-tftransform-google-cloud)** - Find the optimal parameter settings for simulated physical machines like a bottle filler or cookie machine. The goal of each simulated machine is to have the same input/output of the actual machine, making it a "digital twin". This uses [tf.Transform](https://github.com/tensorflow/transform) for preprocessing. From 12a4f677c6949856e4de303c2254acb1abee42f2 Mon Sep 17 00:00:00 2001 From: lrakla Date: Tue, 12 Dec 2023 10:20:46 -0500 Subject: [PATCH 096/224] Fixes dictionary size change in direct runners (#29122) --- sdks/python/apache_beam/runners/worker/sdk_worker.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index b55f505a6adc..d553f0839e76 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -589,11 +589,12 @@ def _schedule_periodic_shutdown(self): # type: () -> None def shutdown_inactive_bundle_processors(): # type: () -> None - for descriptor_id, last_access_time in self.last_access_times.items(): - if (time.time() - last_access_time > - DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S): - BundleProcessorCache._shutdown_cached_bundle_processors( - self.cached_bundle_processors[descriptor_id]) + with self._lock: + for descriptor_id, last_access_time in self.last_access_times.items(): + if (time.time() - last_access_time > + DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S): + BundleProcessorCache._shutdown_cached_bundle_processors( + self.cached_bundle_processors[descriptor_id]) self.periodic_shutdown = PeriodicThread( DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S, From 649da779acd9f0a7e443be6edf2f963f3b362b91 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 12 Dec 2023 10:54:04 -0500 Subject: [PATCH 097/224] Update to most recent dataflow python container (#29717) --- 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 82e2306ced3c..b43f615189da 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-20231207' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231212' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From f4d4a0d4696b5c2bdbe0c18714bbbd57c3a3f391 Mon Sep 17 00:00:00 2001 From: JayajP Date: Tue, 12 Dec 2023 08:59:39 -0800 Subject: [PATCH 098/224] Add utility method to parse labels/base name from a mangled bigquery metric name (#29500) --- .../io/gcp/bigquery/BigQuerySinkMetrics.java | 61 ++++++++++++++++++- .../gcp/bigquery/BigQuerySinkMetricsTest.java | 44 +++++++++++++ 2 files changed, 104 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java index 24323fce6895..a89707c1919f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java @@ -17,8 +17,10 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import com.google.auto.value.AutoValue; import io.grpc.Status; import java.time.Instant; +import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; import javax.annotation.Nonnull; @@ -30,6 +32,8 @@ import org.apache.beam.sdk.metrics.Histogram; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.util.HistogramData; +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.collect.ImmutableMap; /** * Helper class to create perworker metrics for BigQuery Sink stages. @@ -41,7 +45,7 @@ public class BigQuerySinkMetrics { private static Boolean supportMetricsDeletion = false; - private static final String METRICS_NAMESPACE = "BigQuerySink"; + public static final String METRICS_NAMESPACE = "BigQuerySink"; // Status codes private static final String UNKNOWN = Status.Code.UNKNOWN.toString(); @@ -79,6 +83,23 @@ enum RowStatus { private static final char METRIC_KV_DELIMITER = ':'; private static final char METRIC_NAME_DELIMITER = '-'; + @AutoValue + public abstract static class ParsedMetricName { + public abstract String getBaseName(); + + public abstract ImmutableMap getMetricLabels(); + + public static ParsedMetricName create( + String baseName, ImmutableMap metricLabels) { + return new AutoValue_BigQuerySinkMetrics_ParsedMetricName(baseName, metricLabels); + } + + public static ParsedMetricName create(String baseName) { + ImmutableMap emptyMap = ImmutableMap.of(); + return new AutoValue_BigQuerySinkMetrics_ParsedMetricName(baseName, emptyMap); + } + } + /** * Returns a metric name that merges the baseName with metricLables formatted as. * @@ -94,6 +115,44 @@ private static String createLabeledMetricName( return nameBuilder.toString(); } + /** + * Parse a 'metric name' String that was created with 'createLabeledMetricName'. The input string + * should be formatted as. + * + *

    '{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' + * + * @param metricName + * @return Returns a ParsedMetricName object if the input string is properly formatted. If the + * input string is empty or malformed, returns null. + */ + public static @Nullable ParsedMetricName parseMetricName(String metricName) { + if (metricName.isEmpty()) { + return null; + } + + List metricNameSplit = Splitter.on(METRIC_NAME_DELIMITER).splitToList(metricName); + ImmutableMap.Builder metricLabelsBuilder = ImmutableMap.builder(); + + if (metricNameSplit.size() == 1) { + return ParsedMetricName.create(metricNameSplit.get(0)); + } + + if (metricNameSplit.size() != 2) { + return null; + } + + List labels = Splitter.on(LABEL_DELIMITER).splitToList(metricNameSplit.get(1)); + for (String label : labels) { + List kv = Splitter.on(METRIC_KV_DELIMITER).splitToList(label); + if (kv.size() != 2) { + continue; + } + metricLabelsBuilder.put(kv.get(0), kv.get(1)); + } + + return ParsedMetricName.create(metricNameSplit.get(0), metricLabelsBuilder.build()); + } + /** * @param method StorageWriteAPI method associated with this metric. * @param rpcStatus RPC return status. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java index c937c6b299b2..d754927bdb3e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java @@ -20,6 +20,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; @@ -36,6 +38,7 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.HistogramData; import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.hamcrest.collection.IsMapContaining; import org.junit.Test; @@ -301,4 +304,45 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { testContainer.perWorkerHistograms.get(KV.of(histogramName, bucketType)).values, containsInAnyOrder(Double.valueOf(15.0))); } + + @Test + public void testParseMetricName_noLabels() { + String baseMetricName = "baseMetricName"; + BigQuerySinkMetrics.ParsedMetricName metricName = + BigQuerySinkMetrics.ParsedMetricName.create(baseMetricName); + + assertThat(BigQuerySinkMetrics.parseMetricName(baseMetricName), equalTo(metricName)); + } + + @Test + public void testParseMetricName_successfulLabels() { + String metricName = "baseLabel-key1:val1;key2:val2;key3:val3;"; + ImmutableMap metricLabels = + ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3"); + BigQuerySinkMetrics.ParsedMetricName expectedName = + BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); + + assertThat(BigQuerySinkMetrics.parseMetricName(metricName), equalTo(expectedName)); + } + + @Test + public void testParseMetricName_malformedMetricName() { + String malformedMetricName = "baseLabel-key1:val1-key2:val2"; + assertThat(BigQuerySinkMetrics.parseMetricName(malformedMetricName), is(nullValue())); + } + + @Test + public void testParseMetricName_malformedMetricLabels() { + String metricName = "baseLabel-key1:val1:malformedField;key2:val2;"; + ImmutableMap metricLabels = ImmutableMap.of("key2", "val2"); + BigQuerySinkMetrics.ParsedMetricName expectedName = + BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); + + assertThat(BigQuerySinkMetrics.parseMetricName(metricName), equalTo(expectedName)); + } + + @Test + public void testParseMetricName_emptyString() { + assertThat(BigQuerySinkMetrics.parseMetricName(""), is(nullValue())); + } } From 0e85810d0d6801525dcbf81c87d5c8cbf85491de Mon Sep 17 00:00:00 2001 From: JayajP Date: Tue, 12 Dec 2023 11:06:48 -0800 Subject: [PATCH 099/224] Add HistogramData::getAndReset method (#29484) * Add HistogramData::getAndReset method * spotless apply --- .../apache/beam/sdk/util/HistogramData.java | 12 +++++++++ .../beam/sdk/util/HistogramDataTest.java | 26 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java index 2dd44ce27bbc..e1e01950d656 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java @@ -186,6 +186,18 @@ public synchronized void clear() { this.sumOfSquaredDeviations = 0; } + /** + * Copies all updates to a new histogram object and resets 'this' histogram. + * + * @return New histogram object that has the the same updates as 'this'. + */ + public synchronized HistogramData getAndReset() { + HistogramData other = new HistogramData(this.getBucketType()); + other.update(this); + this.clear(); + return other; + } + public synchronized void record(double value) { double rangeTo = bucketType.getRangeTo(); double rangeFrom = bucketType.getRangeFrom(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java index 8716bbc1411f..8a518ec799ae 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java @@ -354,4 +354,30 @@ public void testStatistics_sumOfSquaredDeviations() { assertThat(histogram.getSumOfSquaredDeviations(), equalTo(8250.0)); } + + @Test + public void testGetAndReset_resetSucceeds() { + HistogramData originalHistogram = HistogramData.linear(0, 10, 10); + originalHistogram.record(15.0, 25.0, 35.0, 45.0); + originalHistogram.getAndReset(); + + HistogramData emptyHistogramData = HistogramData.linear(0, 10, 10); + assertThat(originalHistogram, equalTo(emptyHistogramData)); + assertThat(originalHistogram.getMean(), equalTo(0.0)); + assertThat(originalHistogram.getSumOfSquaredDeviations(), equalTo(0.0)); + } + + @Test + public void testGetAndReset_getSucceeds() { + HistogramData originalHistogram = HistogramData.linear(0, 10, 10); + originalHistogram.record(15.0, 25.0, 35.0, 45.0, 55.0); + HistogramData copyHistogram = originalHistogram.getAndReset(); + + HistogramData duplicateHistogram = HistogramData.linear(0, 10, 10); + duplicateHistogram.record(15.0, 25.0, 35.0, 45.0, 55.0); + assertThat(copyHistogram, equalTo(duplicateHistogram)); + assertThat(copyHistogram.getBucketType(), equalTo(originalHistogram.getBucketType())); + assertThat(copyHistogram.getMean(), equalTo(35.0)); + assertThat(copyHistogram.getSumOfSquaredDeviations(), equalTo(1000.0)); + } } From 475a0c7869051331a03f2bcb65af45113fd984a3 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Tue, 12 Dec 2023 14:08:45 -0500 Subject: [PATCH 100/224] Fix syntax (#29719) * correct syntax * remove unused inference_args --- .../ml/inference/huggingface_inference.py | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/sdks/python/apache_beam/ml/inference/huggingface_inference.py b/sdks/python/apache_beam/ml/inference/huggingface_inference.py index 1bc92c462c91..25367d22eaa3 100644 --- a/sdks/python/apache_beam/ml/inference/huggingface_inference.py +++ b/sdks/python/apache_beam/ml/inference/huggingface_inference.py @@ -221,7 +221,6 @@ def __init__( *, inference_fn: Optional[Callable[..., Iterable[PredictionResult]]] = None, load_model_args: Optional[Dict[str, Any]] = None, - inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, max_batch_duration_secs: Optional[int] = None, @@ -250,10 +249,6 @@ def __init__( load_model_args (Dict[str, Any]): (Optional) Keyword arguments to provide load options while loading models from Hugging Face Hub. Defaults to None. - inference_args (Dict[str, Any]): (Optional) Non-batchable arguments - required as inputs to the model's inference function. Unlike Tensors - in `batch`, these parameters will not be dynamically batched. - Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. max_batch_duration_secs: the maximum amount of time to buffer a batch @@ -273,7 +268,6 @@ def __init__( self._device = device self._inference_fn = inference_fn self._model_config_args = load_model_args if load_model_args else {} - self._inference_args = inference_args if inference_args else {} self._batching_kwargs = {} self._env_vars = kwargs.get("env_vars", {}) if min_batch_size is not None: @@ -293,7 +287,7 @@ def load_model(self): model = self._model_class.from_pretrained( self._model_uri, **self._model_config_args) if self._framework == 'pt': - if self._device == "GPU" and is_gpu_available_torch: + if self._device == "GPU" and is_gpu_available_torch(): model.to(torch.device("cuda")) if callable(getattr(model, 'requires_grad_', None)): model.requires_grad_(False) @@ -407,7 +401,6 @@ def __init__( *, inference_fn: Optional[Callable[..., Iterable[PredictionResult]]] = None, load_model_args: Optional[Dict[str, Any]] = None, - inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, max_batch_duration_secs: Optional[int] = None, @@ -436,10 +429,6 @@ def __init__( load_model_args (Dict[str, Any]): (Optional) keyword arguments to provide load options while loading models from Hugging Face Hub. Defaults to None. - inference_args (Dict[str, Any]): (Optional) Non-batchable arguments - required as inputs to the model's inference function. Unlike Tensors - in `batch`, these parameters will not be dynamically batched. - Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. max_batch_duration_secs: the maximum amount of time to buffer a batch @@ -459,7 +448,6 @@ def __init__( self._device = device self._inference_fn = inference_fn self._model_config_args = load_model_args if load_model_args else {} - self._inference_args = inference_args if inference_args else {} self._batching_kwargs = {} self._env_vars = kwargs.get("env_vars", {}) if min_batch_size is not None: @@ -586,7 +574,6 @@ def __init__( device: Optional[str] = None, inference_fn: PipelineInferenceFn = _default_pipeline_inference_fn, load_pipeline_args: Optional[Dict[str, Any]] = None, - inference_args: Optional[Dict[str, Any]] = None, min_batch_size: Optional[int] = None, max_batch_size: Optional[int] = None, max_batch_duration_secs: Optional[int] = None, @@ -623,9 +610,6 @@ def __init__( Default is _default_pipeline_inference_fn. load_pipeline_args (Dict[str, Any]): keyword arguments to provide load options while loading pipelines from Hugging Face. Defaults to None. - inference_args (Dict[str, Any]): Non-batchable arguments - required as inputs to the model's inference function. - Defaults to None. min_batch_size: the minimum batch size to use when batching inputs. max_batch_size: the maximum batch size to use when batching inputs. max_batch_duration_secs: the maximum amount of time to buffer a batch @@ -644,7 +628,6 @@ def __init__( self._model = model self._inference_fn = inference_fn self._load_pipeline_args = load_pipeline_args if load_pipeline_args else {} - self._inference_args = inference_args if inference_args else {} self._batching_kwargs = {} self._framework = "torch" self._env_vars = kwargs.get('env_vars', {}) From efe7e6aad1f9b10c66486776c208c19e3fcaca5b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 12 Dec 2023 14:20:17 -0500 Subject: [PATCH 101/224] Deal with trailing slash in tempRoot (#29478) * Deal with trailing slash in tempRoot * Add FileSystems.matchNewDirectory and replace the raw string concatenate throughout the code base * Fix test assert * Use File.separator per suggestion --- .../runners/dataflow/TestDataflowRunner.java | 14 ++++++++++--- .../org/apache/beam/sdk/io/FileSystems.java | 17 ++++++++++++++++ .../apache/beam/sdk/io/FileSystemsTest.java | 20 +++++++++++++++++++ .../sdk/extensions/gcp/util/GcsUtilIT.java | 4 +++- .../io/gcp/bigquery/BigQueryClusteringIT.java | 4 +++- .../sdk/io/gcp/bigquery/BigQueryIOJsonIT.java | 4 +++- .../sdk/io/gcp/bigquery/BigQueryIOReadIT.java | 4 +++- .../BigQueryIOStorageReadTableRowIT.java | 4 +++- .../sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java | 4 +++- .../gcp/bigquery/BigQueryNestedRecordsIT.java | 4 +++- .../BigQuerySchemaUpdateOptionsIT.java | 7 +++++-- .../BigQueryTimePartitioningClusteringIT.java | 4 +++- .../io/gcp/bigquery/BigQueryToTableIT.java | 10 +++++++--- .../beam/sdk/io/gcp/storage/GcsKmsKeyIT.java | 3 ++- .../beam/sdk/io/gcp/storage/GcsMatchIT.java | 3 ++- 15 files changed, 88 insertions(+), 18 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java index efb725f6a67b..b00194dacb08 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java @@ -23,6 +23,7 @@ import com.google.api.services.dataflow.model.JobMetrics; import com.google.api.services.dataflow.model.MetricUpdate; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.File; import java.io.IOException; import java.math.BigDecimal; import java.util.List; @@ -33,12 +34,12 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; 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.Strings; import org.checkerframework.checker.nullness.qual.Nullable; @@ -74,8 +75,15 @@ public class TestDataflowRunner extends PipelineRunner { public static TestDataflowRunner fromOptions(PipelineOptions options) { TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class); String tempLocation = - Joiner.on("/") - .join(dataflowOptions.getTempRoot(), dataflowOptions.getJobName(), "output", "results"); + FileSystems.matchNewDirectory( + dataflowOptions.getTempRoot(), dataflowOptions.getJobName(), "output", "results") + .toString(); + // to keep exact same behavior prior to matchNewDirectory introduced + if (tempLocation.endsWith("/")) { + tempLocation = tempLocation.substring(0, tempLocation.length() - 1); + } else if (tempLocation.endsWith(File.separator)) { + tempLocation = tempLocation.substring(0, tempLocation.length() - File.separator.length()); + } dataflowOptions.setTempLocation(tempLocation); return new TestDataflowRunner( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java index eb78bd0e27d2..c58ce9ad8308 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java @@ -47,6 +47,7 @@ import org.apache.beam.sdk.io.fs.MatchResult.Status; import org.apache.beam.sdk.io.fs.MoveOptions; import org.apache.beam.sdk.io.fs.MoveOptions.StandardMoveOptions; +import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.common.ReflectHelpers; @@ -603,4 +604,20 @@ public static ResourceId matchNewResource(String singleResourceSpec, boolean isD return getFileSystemInternal(parseScheme(singleResourceSpec)) .matchNewResource(singleResourceSpec, isDirectory); } + + /** + * Returns a new {@link ResourceId} that represents the named directory resource. + * + * @param singleResourceSpec the root directory, for example "/abc" + * @param baseNames a list of named directory, for example ["d", "e", "f"] + * @return the ResourceId for the resolved directory. In same example as above, it corresponds to + * "/abc/d/e/f". + */ + public static ResourceId matchNewDirectory(String singleResourceSpec, String... baseNames) { + ResourceId currentDir = matchNewResource(singleResourceSpec, true); + for (String dir : baseNames) { + currentDir = currentDir.resolve(dir, StandardResolveOptions.RESOLVE_DIRECTORY); + } + return currentDir; + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java index 64a722495667..d3fcfb291fca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; @@ -316,6 +317,25 @@ public void testInvalidSchemaMatchNewResource() { assertEquals("file", FileSystems.matchNewResource("c:/tmp/f1", false)); } + @Test + public void testMatchNewDirectory() { + List>> testCases = + ImmutableList.>>builder() + .add(KV.of("/abc/d/", KV.of("/abc", new String[] {"d"}))) + .add(KV.of("/abc/d/", KV.of("/abc/", new String[] {"d"}))) + .add(KV.of("/abc/d/", KV.of("/abc", new String[] {"d/"}))) + .add(KV.of("/abc/d/e/f/", KV.of("/abc", new String[] {"d", "e", "f"}))) + .add(KV.of("/abc/", KV.of("/abc", new String[] {}))) + .build(); + for (KV> testCase : testCases) { + ResourceId expected = FileSystems.matchNewResource(testCase.getKey(), true); + ResourceId actual = + FileSystems.matchNewDirectory( + testCase.getValue().getKey(), testCase.getValue().getValue()); + assertEquals(expected, actual); + } + } + private static List toResourceIds(List paths, final boolean isDirectory) { return FluentIterable.from(paths) .transform(path -> (ResourceId) LocalResourceId.fromPath(path, isDirectory)) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java index 08b9f5456dd2..bc9853beca46 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java @@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.testing.UsesKms; @@ -53,7 +54,8 @@ public void testRewriteMultiPart() throws IOException { // Using a KMS key is necessary to trigger multi-part rewrites (bucket is created // with a bucket default key). assertNotNull(options.getTempRoot()); - options.setTempLocation(options.getTempRoot() + "/testRewriteMultiPart"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "testRewriteMultiPart").toString()); GcsOptions gcsOptions = options.as(GcsOptions.class); GcsUtil gcsUtil = gcsOptions.getGcsUtil(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java index 1b3c844e2a9f..3ee6931dd98d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java @@ -27,6 +27,7 @@ import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -68,7 +69,8 @@ public class BigQueryClusteringIT { public void setUp() { PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); options = TestPipeline.testingPipelineOptions().as(BigQueryClusteringITOptions.class); - options.setTempLocation(options.getTempRoot() + "/temp-it/"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "temp-it").toString()); bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java index 4b7e02e4bf47..9da050a059cc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOJsonIT.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.stream.Collectors; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; @@ -70,7 +71,8 @@ public class BigQueryIOJsonIT { static { TestPipelineOptions opt = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); - testOptions.setTempLocation(opt.getTempRoot() + "/java-tmp"); + testOptions.setTempLocation( + FileSystems.matchNewDirectory(opt.getTempRoot(), "java-tmp").toString()); } @Rule public final transient TestPipeline p = TestPipeline.fromOptions(testOptions); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java index 8a3bc0fac2b7..6291eac62b33 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -71,7 +72,8 @@ private void setupTestEnvironment(String recordSize) { PipelineOptionsFactory.register(BigQueryIOReadOptions.class); options = TestPipeline.testingPipelineOptions().as(BigQueryIOReadOptions.class); options.setNumRecords(numOfRecords.get(recordSize)); - options.setTempLocation(options.getTempRoot() + "/temp-it/"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "temp-it").toString()); project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); options.setInputTable(project + ":" + datasetId + "." + tablePrefix + recordSize); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java index 35e2676c70ef..7998bac65055 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TableRowParser; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; import org.apache.beam.sdk.options.Description; @@ -88,7 +89,8 @@ private void setUpTestEnvironment(String tableName) { options = TestPipeline.testingPipelineOptions().as(BigQueryIOStorageReadTableRowOptions.class); String project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); options.setInputTable(project + ":" + DATASET_ID + "." + TABLE_PREFIX + tableName); - options.setTempLocation(options.getTempRoot() + "/temp-it/"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "temp-it").toString()); } private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOptions) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java index 85a180932f65..edf93dba3ae4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java @@ -26,6 +26,7 @@ import java.security.SecureRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.testing.TestPipeline; @@ -87,7 +88,8 @@ private void testQueryAndWrite(Method method) throws Exception { String outputTableId = "testQueryAndWrite_" + method.name(); String outputTableSpec = project + ":" + BIG_QUERY_DATASET_ID + "." + outputTableId; - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); Pipeline p = Pipeline.create(options); // Reading triggers BQ query and extract jobs. Writing triggers either a load job or performs a // streaming insert (depending on method). diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryNestedRecordsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryNestedRecordsIT.java index b85dc62c5fe9..e8d3a0ea22c4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryNestedRecordsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryNestedRecordsIT.java @@ -19,6 +19,7 @@ import com.google.api.services.bigquery.model.TableRow; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -63,7 +64,8 @@ public void testNestedRecords() throws Exception { TestPipelineOptions testOptions = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); Options options = testOptions.as(Options.class); - options.setTempLocation(testOptions.getTempRoot() + "/temp-it/"); + options.setTempLocation( + FileSystems.matchNewDirectory(testOptions.getTempRoot(), "temp-it").toString()); runPipeline(options); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java index 833a0a0829c7..c5e954a46380 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.SchemaUpdateOption; @@ -150,7 +151,8 @@ private void runWriteTest( List> expectedResult) throws Exception { Options options = TestPipeline.testingPipelineOptions().as(Options.class); - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); Pipeline p = Pipeline.create(options); Create.Values input = Create.of(rowToInsert); @@ -264,7 +266,8 @@ public void runWriteTestTempTableAndDynamicDestination() throws Exception { } Options options = TestPipeline.testingPipelineOptions().as(Options.class); - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); Pipeline p = Pipeline.create(options); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java index da5f396e8d89..22e4feb3c050 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -90,7 +91,8 @@ public static void setupTestEnvironment() throws Exception { public void setUp() { PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); options = TestPipeline.testingPipelineOptions().as(BigQueryClusteringITOptions.class); - options.setTempLocation(options.getTempRoot() + "/temp-it/"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "temp-it").toString()); bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java index 1abe7752b2e0..31bb627e8451 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java @@ -39,6 +39,7 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.util.BackOffAdapter; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -118,7 +119,8 @@ private void runBigQueryToTablePipeline(BigQueryToTableOptions options) { private BigQueryToTableOptions setupLegacyQueryTest(String outputTable) { BigQueryToTableOptions options = TestPipeline.testingPipelineOptions().as(BigQueryToTableOptions.class); - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); options.setQuery("SELECT * FROM (SELECT \"apple\" as fruit), (SELECT \"orange\" as fruit),"); options.setOutput(outputTable); options.setOutputSchema(BigQueryToTableIT.LEGACY_QUERY_TABLE_SCHEMA); @@ -128,7 +130,8 @@ private BigQueryToTableOptions setupLegacyQueryTest(String outputTable) { private BigQueryToTableOptions setupNewTypesQueryTest(String outputTable) { BigQueryToTableOptions options = TestPipeline.testingPipelineOptions().as(BigQueryToTableOptions.class); - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); options.setQuery( String.format( "SELECT bytes, date, time FROM [%s:%s.%s]", @@ -140,7 +143,8 @@ private BigQueryToTableOptions setupNewTypesQueryTest(String outputTable) { private BigQueryToTableOptions setupStandardQueryTest(String outputTable) { BigQueryToTableOptions options = this.setupLegacyQueryTest(outputTable); - options.setTempLocation(options.getTempRoot() + "/bq_it_temp"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "bq_it_temp").toString()); options.setQuery( "SELECT * FROM (SELECT \"apple\" as fruit) UNION ALL (SELECT \"orange\" as fruit)"); options.setUsingStandardSql(true); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java index 94f632e7daa1..67cf729c417f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java @@ -70,7 +70,8 @@ public void testGcsWriteWithKmsKey() { TestPipelineOptions options = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); assertNotNull(options.getTempRoot()); - options.setTempLocation(options.getTempRoot() + "/testGcsWriteWithKmsKey"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "testGcsWriteWithKmsKey").toString()); GcsOptions gcsOptions = options.as(GcsOptions.class); ResourceId filenamePrefix = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsMatchIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsMatchIT.java index 336bcd768f7a..5f04c973adee 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsMatchIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsMatchIT.java @@ -144,7 +144,8 @@ public Void apply(Iterable input) { public static void setUp() throws Exception { options = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); assertNotNull(options.getTempRoot()); - options.setTempLocation(options.getTempRoot() + "/GcsMatchIT"); + options.setTempLocation( + FileSystems.matchNewDirectory(options.getTempRoot(), "GcsMatchIT").toString()); GcsOptions gcsOptions = options.as(GcsOptions.class); String dstFolderName = gcsOptions.getGcpTempLocation() From d59e1920d23546332368847597255051611e5bd5 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 12 Dec 2023 14:52:17 -0500 Subject: [PATCH 102/224] Bump dataflowLegacyContainerVersion (#29723) --- runners/google-cloud-dataflow-java/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 5d967d90f2b8..c7e5e1ba6236 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -51,7 +51,7 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20231026' +ext.dataflowLegacyContainerVersion = 'beam-master-20231212' ext.dataflowFnapiContainerVersion = 'beam-master-20231026' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' From 90e79ae373ab38cf4e48e9854c28aaffb0938458 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johanna=20=C3=96jeling?= <51084516+johannaojeling@users.noreply.github.com> Date: Tue, 12 Dec 2023 22:19:06 +0100 Subject: [PATCH 103/224] [Go SDK]: Implement natsio.Read transform for reading from NATS (#29410) --- CHANGES.md | 1 + sdks/go/pkg/beam/io/natsio/common.go | 4 + sdks/go/pkg/beam/io/natsio/end_estimator.go | 77 +++++ .../pkg/beam/io/natsio/end_estimator_test.go | 78 +++++ sdks/go/pkg/beam/io/natsio/example_test.go | 18 ++ sdks/go/pkg/beam/io/natsio/helper_test.go | 48 ++- sdks/go/pkg/beam/io/natsio/read.go | 289 ++++++++++++++++++ sdks/go/pkg/beam/io/natsio/read_option.go | 98 ++++++ sdks/go/pkg/beam/io/natsio/read_test.go | 212 +++++++++++++ sdks/go/pkg/beam/io/natsio/time_policy.go | 50 +++ .../go/pkg/beam/io/natsio/time_policy_test.go | 45 +++ .../pkg/beam/io/natsio/watermark_estimator.go | 33 ++ .../io/natsio/watermark_estimator_test.go | 67 ++++ sdks/go/pkg/beam/io/natsio/write_test.go | 4 +- 14 files changed, 1020 insertions(+), 4 deletions(-) create mode 100644 sdks/go/pkg/beam/io/natsio/end_estimator.go create mode 100644 sdks/go/pkg/beam/io/natsio/end_estimator_test.go create mode 100644 sdks/go/pkg/beam/io/natsio/read.go create mode 100644 sdks/go/pkg/beam/io/natsio/read_option.go create mode 100644 sdks/go/pkg/beam/io/natsio/read_test.go create mode 100644 sdks/go/pkg/beam/io/natsio/time_policy.go create mode 100644 sdks/go/pkg/beam/io/natsio/time_policy_test.go create mode 100644 sdks/go/pkg/beam/io/natsio/watermark_estimator.go create mode 100644 sdks/go/pkg/beam/io/natsio/watermark_estimator_test.go diff --git a/CHANGES.md b/CHANGES.md index 60b5a820cf3b..70ce1a70b7ee 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ * Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). * Added support for handling bad records to KafkaIO (Java) ([#29546](https://github.com/apache/beam/pull/29546)) * Add support for generating text embeddings in MLTransform for Vertex AI and Hugging Face Hub models.([#29564](https://github.com/apache/beam/pull/29564)) +* NATS IO connector added (Go) ([#29000](https://github.com/apache/beam/issues/29000)). ## New Features / Improvements diff --git a/sdks/go/pkg/beam/io/natsio/common.go b/sdks/go/pkg/beam/io/natsio/common.go index 53f595516987..72640894c76e 100644 --- a/sdks/go/pkg/beam/io/natsio/common.go +++ b/sdks/go/pkg/beam/io/natsio/common.go @@ -31,6 +31,10 @@ type natsFn struct { } func (fn *natsFn) Setup() error { + if fn.nc != nil && fn.js != nil { + return nil + } + var opts []nats.Option if fn.CredsFile != "" { opts = append(opts, nats.UserCredentials(fn.CredsFile)) diff --git a/sdks/go/pkg/beam/io/natsio/end_estimator.go b/sdks/go/pkg/beam/io/natsio/end_estimator.go new file mode 100644 index 000000000000..6b2f18e10ce3 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/end_estimator.go @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +import ( + "context" + "errors" + "fmt" + + "github.com/nats-io/nats.go/jetstream" +) + +type endEstimator struct { + js jetstream.JetStream + stream string + subject string +} + +func newEndEstimator(js jetstream.JetStream, stream string, subject string) *endEstimator { + return &endEstimator{ + js: js, + stream: stream, + subject: subject, + } +} + +func (e *endEstimator) Estimate() int64 { + ctx := context.Background() + end, err := e.getEndSeqNo(ctx) + if err != nil { + panic(err) + } + return end +} + +func (e *endEstimator) getEndSeqNo(ctx context.Context) (int64, error) { + str, err := e.js.Stream(ctx, e.stream) + if err != nil { + return -1, fmt.Errorf("error getting stream: %v", err) + } + + msg, err := str.GetLastMsgForSubject(ctx, e.subject) + if err != nil { + if isMessageNotFound(err) { + return 1, nil + } + + return -1, fmt.Errorf("error getting last message: %v", err) + } + + return int64(msg.Sequence) + 1, nil +} + +func isMessageNotFound(err error) bool { + var jsErr jetstream.JetStreamError + if errors.As(err, &jsErr) { + apiErr := jsErr.APIError() + if apiErr.ErrorCode == jetstream.JSErrCodeMessageNotFound { + return true + } + } + + return false +} diff --git a/sdks/go/pkg/beam/io/natsio/end_estimator_test.go b/sdks/go/pkg/beam/io/natsio/end_estimator_test.go new file mode 100644 index 000000000000..855547a0e297 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/end_estimator_test.go @@ -0,0 +1,78 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package natsio + +import ( + "context" + "fmt" + "testing" + + "github.com/nats-io/nats.go" +) + +func Test_endEstimator_Estimate(t *testing.T) { + tests := []struct { + name string + msgs []*nats.Msg + subject string + want int64 + }{ + { + name: "Estimate end for published messages", + msgs: []*nats.Msg{ + { + Subject: "subject.1", + Data: []byte("msg1"), + }, + { + Subject: "subject.1", + Data: []byte("msg2"), + }, + { + Subject: "subject.2", + Data: []byte("msg3"), + }, + }, + subject: "subject.1", + want: 3, + }, + { + name: "Estimate end for no published messages", + subject: "subject.1", + want: 1, + }, + } + for i, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + ctx := context.Background() + srv := newServer(t) + url := srv.ClientURL() + conn := newConn(t, url) + js := newJetStream(t, conn) + + stream := fmt.Sprintf("STREAM-%d", i) + subjectFilter := "subject.*" + + createStream(ctx, t, js, stream, []string{subjectFilter}) + publishMessages(ctx, t, js, tt.msgs) + + estimator := newEndEstimator(js, stream, tt.subject) + if got := estimator.Estimate(); got != tt.want { + t.Fatalf("Estimate() = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/sdks/go/pkg/beam/io/natsio/example_test.go b/sdks/go/pkg/beam/io/natsio/example_test.go index 0516b8efa921..984261a3686b 100644 --- a/sdks/go/pkg/beam/io/natsio/example_test.go +++ b/sdks/go/pkg/beam/io/natsio/example_test.go @@ -22,9 +22,27 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/io/natsio" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/v2/go/pkg/beam/x/debug" "github.com/nats-io/nats.go" ) +func ExampleRead() { + beam.Init() + + p, s := beam.NewPipelineWithRoot() + + uri := "nats://localhost:4222" + stream := "EVENTS" + subject := "events.*" + + col := natsio.Read(s, uri, stream, subject) + debug.Print(s, col) + + if err := beamx.Run(context.Background(), p); err != nil { + log.Fatalf("Failed to execute job: %v", err) + } +} + func ExampleWrite() { beam.Init() diff --git a/sdks/go/pkg/beam/io/natsio/helper_test.go b/sdks/go/pkg/beam/io/natsio/helper_test.go index cd47ed331de0..ac7eedac1d44 100644 --- a/sdks/go/pkg/beam/io/natsio/helper_test.go +++ b/sdks/go/pkg/beam/io/natsio/helper_test.go @@ -18,6 +18,7 @@ package natsio import ( "context" "testing" + "time" "github.com/nats-io/nats-server/v2/server" "github.com/nats-io/nats-server/v2/test" @@ -62,8 +63,8 @@ func newJetStream(t *testing.T, conn *nats.Conn) jetstream.JetStream { } func createStream( - t *testing.T, ctx context.Context, + t *testing.T, js jetstream.JetStream, stream string, subjects []string, @@ -89,8 +90,8 @@ func createStream( } func createConsumer( - t *testing.T, ctx context.Context, + t *testing.T, js jetstream.JetStream, stream string, subjects []string, @@ -128,3 +129,46 @@ func fetchMessages(t *testing.T, cons jetstream.Consumer, size int) []jetstream. return result } + +func publishMessages(ctx context.Context, t *testing.T, js jetstream.JetStream, msgs []*nats.Msg) { + t.Helper() + + for _, msg := range msgs { + if _, err := js.PublishMsg(ctx, msg); err != nil { + t.Fatalf("Failed to publish message: %v", err) + } + } +} + +func messagesWithPublishingTime( + t *testing.T, + pubMsgs []jetstream.Msg, + pubIndices []int, + want []any, +) []any { + t.Helper() + + wantWTime := make([]any, len(want)) + + for i := range want { + pubIdx := pubIndices[i] + pubMsg := pubMsgs[pubIdx] + + wantMsg := want[i].(ConsumerMessage) + wantMsg.PublishingTime = messageTimestamp(t, pubMsg) + wantWTime[i] = wantMsg + } + + return wantWTime +} + +func messageTimestamp(t *testing.T, msg jetstream.Msg) time.Time { + t.Helper() + + metadata, err := msg.Metadata() + if err != nil { + t.Fatalf("Failed to retrieve metadata: %v", err) + } + + return metadata.Timestamp +} diff --git a/sdks/go/pkg/beam/io/natsio/read.go b/sdks/go/pkg/beam/io/natsio/read.go new file mode 100644 index 000000000000..df5a53accbe1 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/read.go @@ -0,0 +1,289 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 natsio + +import ( + "context" + "fmt" + "math" + "reflect" + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" + "github.com/apache/beam/sdks/v2/go/pkg/beam/io/rtrackers/offsetrange" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" + "github.com/nats-io/nats.go" + "github.com/nats-io/nats.go/jetstream" +) + +func init() { + register.DoFn5x2[ + context.Context, *watermarkEstimator, *sdf.LockRTracker, []byte, + func(beam.EventTime, ConsumerMessage), sdf.ProcessContinuation, error, + ]( + &readFn{}, + ) + register.Emitter2[beam.EventTime, ConsumerMessage]() + beam.RegisterType(reflect.TypeOf((*ConsumerMessage)(nil)).Elem()) +} + +const ( + defaultFetchSize = 100 + defaultStartSeqNo = 1 + defaultEndSeqNo = math.MaxInt64 + fetchTimeout = 3 * time.Second + assumedLag = 1 * time.Second + resumeDelay = 5 * time.Second +) + +type ConsumerMessage struct { + Subject string + PublishingTime time.Time + ID string + Headers map[string][]string + Data []byte +} + +// Read reads messages from NATS JetStream and returns a PCollection. +// Read takes a variable number of ReadOptionFn to configure the read operation: +// - UserCredentials: path to the user credentials file. Defaults to empty. +// - ProcessingTimePolicy: whether to use the pipeline processing time of the messages as the event +// time. Defaults to true. +// - PublishingTimePolicy: whether to use the publishing time of the messages as the event time. +// Defaults to false. +// - FetchSize: the maximum number of messages to retrieve at a time. Defaults to 100. +// - StartSeqNo: the start sequence number of messages to read. Defaults to 1. +// - EndSeqNo: the end sequence number of messages to read (exclusive). Defaults to math.MaxInt64. +func Read( + s beam.Scope, + uri string, + stream string, + subject string, + opts ...ReadOptionFn, +) beam.PCollection { + s = s.Scope("natsio.Read") + + option := &readOption{ + TimePolicy: processingTimePolicy, + FetchSize: defaultFetchSize, + StartSeqNo: defaultStartSeqNo, + EndSeqNo: defaultEndSeqNo, + } + + for _, opt := range opts { + if err := opt(option); err != nil { + panic(fmt.Sprintf("natsio.Read: invalid option: %v", err)) + } + } + + imp := beam.Impulse(s) + return beam.ParDo(s, newReadFn(uri, stream, subject, option), imp) +} + +type readFn struct { + natsFn + Stream string + Subject string + TimePolicy timePolicy + FetchSize int + StartSeqNo int64 + EndSeqNo int64 + timestampFn timestampFn +} + +func newReadFn(uri string, stream string, subject string, option *readOption) *readFn { + return &readFn{ + natsFn: natsFn{ + URI: uri, + CredsFile: option.CredsFile, + }, + Stream: stream, + Subject: subject, + TimePolicy: option.TimePolicy, + FetchSize: option.FetchSize, + StartSeqNo: option.StartSeqNo, + EndSeqNo: option.EndSeqNo, + } +} + +func (fn *readFn) Setup() error { + if err := fn.natsFn.Setup(); err != nil { + return err + } + + fn.timestampFn = fn.TimePolicy.TimestampFn() + return nil +} + +func (fn *readFn) CreateInitialRestriction(_ []byte) offsetrange.Restriction { + return offsetrange.Restriction{ + Start: fn.StartSeqNo, + End: fn.EndSeqNo, + } +} + +func (fn *readFn) SplitRestriction( + _ []byte, + rest offsetrange.Restriction, +) []offsetrange.Restriction { + return []offsetrange.Restriction{rest} +} + +func (fn *readFn) RestrictionSize(_ []byte, rest offsetrange.Restriction) (float64, error) { + if err := fn.natsFn.Setup(); err != nil { + return -1, err + } + + rt, err := fn.createRTracker(rest) + if err != nil { + return -1, err + } + + _, remaining := rt.GetProgress() + return remaining, nil +} + +func (fn *readFn) CreateTracker(rest offsetrange.Restriction) (*sdf.LockRTracker, error) { + rt, err := fn.createRTracker(rest) + if err != nil { + return nil, err + } + + return sdf.NewLockRTracker(rt), nil +} + +func (fn *readFn) TruncateRestriction(rt *sdf.LockRTracker, _ []byte) offsetrange.Restriction { + start := rt.GetRestriction().(offsetrange.Restriction).Start + return offsetrange.Restriction{ + Start: start, + End: start, + } +} + +func (fn *readFn) InitialWatermarkEstimatorState( + et beam.EventTime, + _ offsetrange.Restriction, + _ []byte, +) int64 { + return et.Milliseconds() +} + +func (fn *readFn) CreateWatermarkEstimator(ms int64) *watermarkEstimator { + return &watermarkEstimator{state: ms} +} + +func (fn *readFn) WatermarkEstimatorState(we *watermarkEstimator) int64 { + return we.state +} + +func (fn *readFn) ProcessElement( + ctx context.Context, + we *watermarkEstimator, + rt *sdf.LockRTracker, + _ []byte, + emit func(beam.EventTime, ConsumerMessage), +) (sdf.ProcessContinuation, error) { + startSeqNo := rt.GetRestriction().(offsetrange.Restriction).Start + cons, err := fn.createConsumer(ctx, startSeqNo) + if err != nil { + return sdf.StopProcessing(), err + } + + for { + msgs, err := cons.Fetch(fn.FetchSize, jetstream.FetchMaxWait(fetchTimeout)) + if err != nil { + return nil, fmt.Errorf("error fetching messages: %v", err) + } + + count := 0 + for msg := range msgs.Messages() { + metadata, err := msg.Metadata() + if err != nil { + return sdf.StopProcessing(), fmt.Errorf("error retrieving metadata: %v", err) + } + + seqNo := int64(metadata.Sequence.Stream) + if !rt.TryClaim(seqNo) { + return sdf.StopProcessing(), nil + } + + et := fn.timestampFn(metadata.Timestamp) + consMsg := createConsumerMessage(msg, metadata.Timestamp) + emit(et, consMsg) + + count++ + } + + if err := msgs.Error(); err != nil { + return sdf.StopProcessing(), fmt.Errorf("error in message batch: %v", err) + } + + if count == 0 { + fn.updateWatermarkManually(we) + return sdf.ResumeProcessingIn(resumeDelay), nil + } + } +} + +func (fn *readFn) createRTracker(rest offsetrange.Restriction) (sdf.RTracker, error) { + if rest.End < math.MaxInt64 { + return offsetrange.NewTracker(rest), nil + } + + estimator := newEndEstimator(fn.js, fn.Stream, fn.Subject) + rt, err := offsetrange.NewGrowableTracker(rest, estimator) + if err != nil { + return nil, fmt.Errorf("error creating growable tracker: %v", err) + } + + return rt, nil +} + +func (fn *readFn) createConsumer( + ctx context.Context, + startSeqNo int64, +) (jetstream.Consumer, error) { + cfg := jetstream.OrderedConsumerConfig{ + FilterSubjects: []string{fn.Subject}, + DeliverPolicy: jetstream.DeliverByStartSequencePolicy, + OptStartSeq: uint64(startSeqNo), + MaxResetAttempts: 5, + } + + cons, err := fn.js.OrderedConsumer(ctx, fn.Stream, cfg) + if err != nil { + return nil, fmt.Errorf("error creating consumer: %v", err) + } + + return cons, nil +} + +func createConsumerMessage(msg jetstream.Msg, publishingTime time.Time) ConsumerMessage { + return ConsumerMessage{ + Subject: msg.Subject(), + PublishingTime: publishingTime, + ID: msg.Headers().Get(nats.MsgIdHdr), + Headers: msg.Headers(), + Data: msg.Data(), + } +} + +func (fn *readFn) updateWatermarkManually(we *watermarkEstimator) { + t := time.Now().Add(-1 * assumedLag) + et := fn.timestampFn(t) + we.ObserveTimestamp(et.ToTime()) +} diff --git a/sdks/go/pkg/beam/io/natsio/read_option.go b/sdks/go/pkg/beam/io/natsio/read_option.go new file mode 100644 index 000000000000..f9d715be1022 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/read_option.go @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 natsio + +import "errors" + +var ( + errInvalidFetchSize = errors.New("fetch size must be greater than 0") + errInvalidStartSeqNo = errors.New("start sequence number must be greater than 0") + errInvalidEndSeqNo = errors.New("end sequence number must be greater than 0") +) + +type readOption struct { + CredsFile string + TimePolicy timePolicy + FetchSize int + StartSeqNo int64 + EndSeqNo int64 +} + +// ReadOptionFn is a function that can be passed to Read to configure options for reading +// from NATS. +type ReadOptionFn func(option *readOption) error + +// ReadUserCredentials sets the user credentials when connecting to NATS. +func ReadUserCredentials(credsFile string) ReadOptionFn { + return func(o *readOption) error { + o.CredsFile = credsFile + return nil + } +} + +// ReadProcessingTimePolicy specifies that the pipeline processing time of the messages should be +// used to compute the watermark estimate. +func ReadProcessingTimePolicy() ReadOptionFn { + return func(o *readOption) error { + o.TimePolicy = processingTimePolicy + return nil + } +} + +// ReadPublishingTimePolicy specifies that the publishing time of the messages should be used to +// compute the watermark estimate. +func ReadPublishingTimePolicy() ReadOptionFn { + return func(o *readOption) error { + o.TimePolicy = publishingTimePolicy + return nil + } +} + +// ReadFetchSize sets the maximum number of messages to retrieve at a time. +func ReadFetchSize(size int) ReadOptionFn { + return func(o *readOption) error { + if size <= 0 { + return errInvalidFetchSize + } + + o.FetchSize = size + return nil + } +} + +// ReadStartSeqNo sets the start sequence number of messages to read. +func ReadStartSeqNo(seqNo int64) ReadOptionFn { + return func(o *readOption) error { + if seqNo <= 0 { + return errInvalidStartSeqNo + } + + o.StartSeqNo = seqNo + return nil + } +} + +// ReadEndSeqNo sets the end sequence number of messages to read (exclusive). +func ReadEndSeqNo(seqNo int64) ReadOptionFn { + return func(o *readOption) error { + if seqNo <= 0 { + return errInvalidEndSeqNo + } + + o.EndSeqNo = seqNo + return nil + } +} diff --git a/sdks/go/pkg/beam/io/natsio/read_test.go b/sdks/go/pkg/beam/io/natsio/read_test.go new file mode 100644 index 000000000000..faf0b0540c81 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/read_test.go @@ -0,0 +1,212 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 natsio + +import ( + "context" + "fmt" + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" + "github.com/nats-io/nats.go" +) + +func TestRead(t *testing.T) { + tests := []struct { + name string + input []*nats.Msg + subject string + opts []ReadOptionFn + pubIndices []int + want []any + }{ + { + name: "Read messages from bounded stream with single subject", + input: []*nats.Msg{ + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"123"}}, + Data: []byte("msg1"), + }, + { + Subject: "subject.2", + Header: nats.Header{nats.MsgIdHdr: []string{"124"}}, + Data: []byte("msg2"), + }, + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"125"}}, + Data: []byte("msg3"), + }, + }, + subject: "subject.1", + opts: []ReadOptionFn{ + ReadEndSeqNo(4), + }, + pubIndices: []int{0, 2}, + want: []any{ + ConsumerMessage{ + Subject: "subject.1", + ID: "123", + Headers: map[string][]string{nats.MsgIdHdr: {"123"}}, + Data: []byte("msg1"), + }, + ConsumerMessage{ + Subject: "subject.1", + ID: "125", + Headers: map[string][]string{nats.MsgIdHdr: {"125"}}, + Data: []byte("msg3"), + }, + }, + }, + { + name: "Read messages from bounded stream with wildcard subject", + input: []*nats.Msg{ + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"123"}}, + Data: []byte("msg1"), + }, + { + Subject: "subject.2", + Header: nats.Header{nats.MsgIdHdr: []string{"124"}}, + Data: []byte("msg2"), + }, + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"125"}}, + Data: []byte("msg3"), + }, + }, + subject: "subject.*", + opts: []ReadOptionFn{ + ReadEndSeqNo(4), + }, + pubIndices: []int{0, 1, 2}, + want: []any{ + ConsumerMessage{ + Subject: "subject.1", + ID: "123", + Headers: map[string][]string{nats.MsgIdHdr: {"123"}}, + Data: []byte("msg1"), + }, + ConsumerMessage{ + Subject: "subject.2", + ID: "124", + Headers: map[string][]string{nats.MsgIdHdr: {"124"}}, + Data: []byte("msg2"), + }, + ConsumerMessage{ + Subject: "subject.1", + ID: "125", + Headers: map[string][]string{nats.MsgIdHdr: {"125"}}, + Data: []byte("msg3"), + }, + }, + }, + { + name: "Read messages from bounded stream with custom fetch size", + input: []*nats.Msg{ + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"123"}}, + Data: []byte("msg1"), + }, + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"124"}}, + Data: []byte("msg2"), + }, + }, + subject: "subject.1", + opts: []ReadOptionFn{ + ReadFetchSize(1), + ReadEndSeqNo(3), + }, + pubIndices: []int{0, 1}, + want: []any{ + ConsumerMessage{ + Subject: "subject.1", + ID: "123", + Headers: map[string][]string{nats.MsgIdHdr: {"123"}}, + Data: []byte("msg1"), + }, + ConsumerMessage{ + Subject: "subject.1", + ID: "124", + Headers: map[string][]string{nats.MsgIdHdr: {"124"}}, + Data: []byte("msg2"), + }, + }, + }, + { + name: "Read messages from bounded stream with custom start seq no", + input: []*nats.Msg{ + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"123"}}, + Data: []byte("msg1"), + }, + { + Subject: "subject.1", + Header: nats.Header{nats.MsgIdHdr: []string{"124"}}, + Data: []byte("msg2"), + }, + }, + subject: "subject.1", + opts: []ReadOptionFn{ + ReadStartSeqNo(2), + ReadEndSeqNo(3), + }, + pubIndices: []int{1}, + want: []any{ + ConsumerMessage{ + Subject: "subject.1", + ID: "124", + Headers: map[string][]string{nats.MsgIdHdr: {"124"}}, + Data: []byte("msg2"), + }, + }, + }, + } + for i, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + ctx := context.Background() + srv := newServer(t) + url := srv.ClientURL() + conn := newConn(t, url) + js := newJetStream(t, conn) + + stream := fmt.Sprintf("STREAM-%d", i) + subjectFilter := "subject.*" + + createStream(ctx, t, js, stream, []string{subjectFilter}) + publishMessages(ctx, t, js, tt.input) + + cons := createConsumer(ctx, t, js, stream, []string{subjectFilter}) + pubMsgs := fetchMessages(t, cons, len(tt.input)) + wantWTime := messagesWithPublishingTime(t, pubMsgs, tt.pubIndices, tt.want) + + p, s := beam.NewPipelineWithRoot() + got := Read(s, url, stream, tt.subject, tt.opts...) + + passert.Equals(s, got, wantWTime...) + ptest.RunAndValidate(t, p) + }) + } +} diff --git a/sdks/go/pkg/beam/io/natsio/time_policy.go b/sdks/go/pkg/beam/io/natsio/time_policy.go new file mode 100644 index 000000000000..1c2dbf5165f7 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/time_policy.go @@ -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 natsio + +import ( + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" +) + +type timePolicy int + +const ( + processingTimePolicy timePolicy = iota + publishingTimePolicy +) + +type timestampFn func(time.Time) mtime.Time + +func processingTime(_ time.Time) mtime.Time { + return mtime.Now() +} + +func publishingTime(t time.Time) mtime.Time { + return mtime.FromTime(t) +} + +func (p timePolicy) TimestampFn() timestampFn { + switch p { + case processingTimePolicy: + return processingTime + case publishingTimePolicy: + return publishingTime + default: + panic("unsupported time policy") + } +} diff --git a/sdks/go/pkg/beam/io/natsio/time_policy_test.go b/sdks/go/pkg/beam/io/natsio/time_policy_test.go new file mode 100644 index 000000000000..2452334c8cdd --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/time_policy_test.go @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 natsio + +import ( + "testing" + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" +) + +func Test_timePolicy_TimestampFn(t *testing.T) { + t.Run("processingTime", func(t *testing.T) { + pubTime := time.Date(2020, 1, 2, 3, 4, 5, 6e6, time.UTC) + + t1 := mtime.Now() + got := processingTimePolicy.TimestampFn()(pubTime) + t2 := mtime.Now() + + if got < t1 || got > t2 { + t.Errorf("timestamp = %v, want between %v and %v", got, t1, t2) + } + }) + + t.Run("publishingTime", func(t *testing.T) { + pubTime := time.Date(2020, 1, 2, 3, 4, 5, 6e6, time.UTC) + + if got, want := publishingTimePolicy.TimestampFn()(pubTime), mtime.FromTime(pubTime); got != want { + t.Errorf("timestamp = %v, want %v", got, want) + } + }) +} diff --git a/sdks/go/pkg/beam/io/natsio/watermark_estimator.go b/sdks/go/pkg/beam/io/natsio/watermark_estimator.go new file mode 100644 index 000000000000..b23eb37ac855 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/watermark_estimator.go @@ -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. + +package natsio + +import "time" + +type watermarkEstimator struct { + state int64 +} + +func (e *watermarkEstimator) CurrentWatermark() time.Time { + return time.UnixMilli(e.state) +} + +func (e *watermarkEstimator) ObserveTimestamp(t time.Time) { + ms := t.UnixMilli() + if ms > e.state { + e.state = ms + } +} diff --git a/sdks/go/pkg/beam/io/natsio/watermark_estimator_test.go b/sdks/go/pkg/beam/io/natsio/watermark_estimator_test.go new file mode 100644 index 000000000000..91a9a840a6e0 --- /dev/null +++ b/sdks/go/pkg/beam/io/natsio/watermark_estimator_test.go @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 natsio + +import ( + "testing" + "time" +) + +func Test_watermarkEstimator_CurrentWatermark(t *testing.T) { + ms := int64(1577934245000) + we := &watermarkEstimator{ + state: ms, + } + if got, want := we.CurrentWatermark(), time.UnixMilli(ms); got != want { + t.Errorf("CurrentWatermark() = %v, want %v", got, want) + } +} + +func Test_watermarkEstimator_ObserveTimestamp(t *testing.T) { + t1 := time.Date(2020, 1, 2, 3, 4, 5, 6e6, time.UTC) + t2 := time.Date(2020, 1, 2, 3, 4, 5, 7e6, time.UTC) + + tests := []struct { + name string + state int64 + t time.Time + want int64 + }{ + { + name: "Update watermark when the time is greater than the current state", + state: t1.UnixMilli(), + t: t2, + want: t2.UnixMilli(), + }, + { + name: "Keep existing watermark when the time is not greater than the current state", + state: t2.UnixMilli(), + t: t1, + want: t2.UnixMilli(), + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + we := &watermarkEstimator{ + state: tt.state, + } + we.ObserveTimestamp(tt.t) + if got, want := we.state, tt.want; got != want { + t.Errorf("state = %v, want %v", got, want) + } + }) + } +} diff --git a/sdks/go/pkg/beam/io/natsio/write_test.go b/sdks/go/pkg/beam/io/natsio/write_test.go index 5e9387ece5f6..678874ce11e9 100644 --- a/sdks/go/pkg/beam/io/natsio/write_test.go +++ b/sdks/go/pkg/beam/io/natsio/write_test.go @@ -187,8 +187,8 @@ func TestWrite(t *testing.T) { js := newJetStream(t, conn) subjects := []string{subject} - createStream(t, ctx, js, stream, subjects) - cons := createConsumer(t, ctx, js, stream, subjects) + createStream(ctx, t, js, stream, subjects) + cons := createConsumer(ctx, t, js, stream, subjects) p, s := beam.NewPipelineWithRoot() From 276aa023903d9a42b7f9dce7dd32aaa946d87605 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 12 Dec 2023 23:56:40 +0000 Subject: [PATCH 104/224] Add suffix for vocab files in tft transforms (#29720) * Add suffix for vocab files * add test * default params to None similar to TFT * fix lint * Update tft.py --- sdks/python/apache_beam/ml/transforms/tft.py | 28 ++++--- .../apache_beam/ml/transforms/tft_test.py | 81 ++++++++++++++++++- 2 files changed, 98 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/ml/transforms/tft.py b/sdks/python/apache_beam/ml/transforms/tft.py index 8b571d9a685e..f1ff0e4dfe57 100644 --- a/sdks/python/apache_beam/ml/transforms/tft.py +++ b/sdks/python/apache_beam/ml/transforms/tft.py @@ -172,8 +172,8 @@ def __init__( num_oov_buckets: Any lookup of an out-of-vocabulary token will return a bucket ID based on its hash if `num_oov_buckets` is greater than zero. Otherwise it is assigned the `default_value`. - vocab_filename: The file name for the vocabulary file. If not provided, - the default name would be `compute_and_apply_vocab' + vocab_filename: The file name for the vocabulary file. The vocab file + will be suffixed with the column name. NOTE in order to make your pipelines resilient to implementation details please set `vocab_filename` when you are using the vocab_filename on a downstream component. @@ -183,8 +183,7 @@ def __init__( self._top_k = top_k self._frequency_threshold = frequency_threshold self._num_oov_buckets = num_oov_buckets - self._vocab_filename = vocab_filename if vocab_filename else ( - 'compute_and_apply_vocab') + self._vocab_filename = vocab_filename self._name = name self.split_string_by_delimiter = split_string_by_delimiter @@ -196,6 +195,9 @@ def apply_transform( data = self._split_string_with_delimiter( data, self.split_string_by_delimiter) + vocab_filename = self._vocab_filename + if vocab_filename: + vocab_filename = vocab_filename + f'_{output_column_name}' return { output_column_name: tft.compute_and_apply_vocabulary( x=data, @@ -203,7 +205,7 @@ def apply_transform( top_k=self._top_k, frequency_threshold=self._frequency_threshold, num_oov_buckets=self._num_oov_buckets, - vocab_filename=self._vocab_filename, + vocab_filename=vocab_filename, name=self._name) } @@ -535,7 +537,7 @@ def __init__( ngram_range: Tuple[int, int] = (1, 1), ngrams_separator: Optional[str] = None, compute_word_count: bool = False, - key_vocab_filename: str = 'key_vocab_mapping', + key_vocab_filename: Optional[str] = None, name: Optional[str] = None, ): """ @@ -558,7 +560,9 @@ def __init__( compute_word_count: A boolean that specifies whether to compute the unique word count over the entire dataset. Defaults to False. key_vocab_filename: The file name for the key vocabulary file when - compute_word_count is True. + compute_word_count is True. If empty, a file name + will be chosen based on the current scope. If provided, the vocab + file will be suffixed with the column name. name: A name for the operation (optional). Note that original order of the input may not be preserved. @@ -585,10 +589,14 @@ def apply_transform(self, data: tf.SparseTensor, output_col_name: str): data, self.split_string_by_delimiter) output = tft.bag_of_words( data, self.ngram_range, self.ngrams_separator, self.name) - # word counts are written to the key_vocab_filename - self.compute_word_count_fn(data, self.key_vocab_filename) + # word counts are written to the file only if compute_word_count is True + key_vocab_filename = self.key_vocab_filename + if key_vocab_filename: + key_vocab_filename = key_vocab_filename + f'_{output_col_name}' + self.compute_word_count_fn(data, key_vocab_filename) return {output_col_name: output} -def count_unqiue_words(data: tf.SparseTensor, output_vocab_name: str) -> None: +def count_unqiue_words( + data: tf.SparseTensor, output_vocab_name: Optional[str]) -> None: tft.count_per_key(data, key_vocabulary_filename=output_vocab_name) diff --git a/sdks/python/apache_beam/ml/transforms/tft_test.py b/sdks/python/apache_beam/ml/transforms/tft_test.py index 9f15db45bd28..558b4ede2ec6 100644 --- a/sdks/python/apache_beam/ml/transforms/tft_test.py +++ b/sdks/python/apache_beam/ml/transforms/tft_test.py @@ -357,6 +357,85 @@ def test_string_split_with_multiple_delimiters(self): ] assert_that(result, equal_to(expected_result, equals_fn=np.array_equal)) + def test_multiple_columns_with_default_vocab_name(self): + data = [{ + 'x': ['I', 'like', 'pie'], 'y': ['Apach', 'Beam', 'is', 'awesome'] + }, + { + 'x': ['yum', 'yum', 'pie'], + 'y': ['Beam', 'is', 'a', 'unified', 'model'] + }] + with beam.Pipeline() as p: + result = ( + p + | "Create" >> beam.Create(data) + | "MLTransform" >> base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + tft.ComputeAndApplyVocabulary(columns=['x', 'y']))) + + expected_data_x = [np.array([3, 2, 1]), np.array([0, 0, 1])] + + expected_data_y = [np.array([6, 1, 0, 4]), np.array([1, 0, 5, 2, 3])] + + actual_data_x = (result | beam.Map(lambda x: x.x)) + actual_data_y = (result | beam.Map(lambda x: x.y)) + + assert_that( + actual_data_x, + equal_to(expected_data_x, equals_fn=np.array_equal), + label='x') + assert_that( + actual_data_y, + equal_to(expected_data_y, equals_fn=np.array_equal), + label='y') + files = os.listdir(self.artifact_location) + files.remove(base._ATTRIBUTE_FILE_NAME) + assert len(files) == 1 + tft_vocab_assets = os.listdir( + os.path.join( + self.artifact_location, files[0], 'transform_fn', 'assets')) + assert len(tft_vocab_assets) == 2 + + def test_multiple_columns_with_vocab_name(self): + data = [{ + 'x': ['I', 'like', 'pie'], 'y': ['Apach', 'Beam', 'is', 'awesome'] + }, + { + 'x': ['yum', 'yum', 'pie'], + 'y': ['Beam', 'is', 'a', 'unified', 'model'] + }] + with beam.Pipeline() as p: + result = ( + p + | "Create" >> beam.Create(data) + | "MLTransform" >> base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + tft.ComputeAndApplyVocabulary( + columns=['x', 'y'], vocab_filename='my_vocab'))) + + expected_data_x = [np.array([3, 2, 1]), np.array([0, 0, 1])] + + expected_data_y = [np.array([6, 1, 0, 4]), np.array([1, 0, 5, 2, 3])] + + actual_data_x = (result | beam.Map(lambda x: x.x)) + actual_data_y = (result | beam.Map(lambda x: x.y)) + + assert_that( + actual_data_x, + equal_to(expected_data_x, equals_fn=np.array_equal), + label='x') + assert_that( + actual_data_y, + equal_to(expected_data_y, equals_fn=np.array_equal), + label='y') + files = os.listdir(self.artifact_location) + files.remove(base._ATTRIBUTE_FILE_NAME) + assert len(files) == 1 + tft_vocab_assets = os.listdir( + os.path.join( + self.artifact_location, files[0], 'transform_fn', 'assets')) + assert len(tft_vocab_assets) == 2 + class TFIDIFTest(unittest.TestCase): def setUp(self) -> None: @@ -717,7 +796,7 @@ def validate_count_per_key(key_vocab_filename): self.artifact_location, files[0], 'transform_fn/assets', - key_vocab_filename) + key_vocab_filename + '_x') with open(key_vocab_location, 'r') as f: key_vocab_list = [line.strip() for line in f] return key_vocab_list From 8be85d29f5341fd0a5e656a469b163774bb91195 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Tue, 12 Dec 2023 23:15:06 -0500 Subject: [PATCH 105/224] fix: optimize segment reader (#29694) * fix: optimize segemant reader * rename the method * Add a test for refill request --- .../io/gcp/bigtable/BigtableServiceImpl.java | 9 ++- .../gcp/bigtable/BigtableServiceImplTest.java | 63 +++++++++++++++++++ 2 files changed, 70 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 344229b383fd..0fb0b6a88efa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -339,11 +339,16 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { + if (future != null && future.isDone()) { + // Add rows from the future to the buffer and reset the future + // so we can do prefetching + consumeReadRowsFuture(); + } if (buffer.size() < refillSegmentWaterMark && future == null) { future = fetchNextSegment(); } if (buffer.isEmpty() && future != null) { - waitReadRowsFuture(); + consumeReadRowsFuture(); } currentRow = buffer.poll(); return currentRow != null; @@ -407,7 +412,7 @@ public void onComplete() { return future; } - private void waitReadRowsFuture() throws IOException { + private void consumeReadRowsFuture() throws IOException { try { UpstreamResults r = future.get(); buffer.addAll(r.rows); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 3e2e803bb334..9a07f625ca1c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -594,6 +594,69 @@ public void testReadFillBuffer() throws IOException { Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); } + /** This test verifies that a refill request is sent before we read all the rows in the buffer. */ + @Test + public void testRefillOnLowWatermark() throws IOException { + int segmentSize = 30; + RowSet.Builder ranges = RowSet.newBuilder(); + // generate 3 pages of rows + ranges.addRowRanges( + generateRowRange( + generateByteString(DEFAULT_PREFIX, 0), + generateByteString(DEFAULT_PREFIX, segmentSize * 3))); + + List> expectedResults = + ImmutableList.of( + generateSegmentResult(DEFAULT_PREFIX, 0, segmentSize), + generateSegmentResult(DEFAULT_PREFIX, segmentSize, segmentSize), + generateSegmentResult(DEFAULT_PREFIX, segmentSize * 2, segmentSize), + ImmutableList.of()); + + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); + BigtableService.Reader underTest = + new BigtableServiceImpl.BigtableSegmentReaderImpl( + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + TABLE_ID, + ranges.build(), + RowFilter.getDefaultInstance(), + segmentSize, + DEFAULT_BYTE_SEGMENT_SIZE, + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), + mockCallMetric); + + Assert.assertTrue(underTest.start()); + + int refillWatermark = Math.max(1, (int) (segmentSize * 0.1)); + + Assert.assertTrue(refillWatermark > 1); + + // Make sure refill happens on the second page. At this point, there + // should be 3 calls made on the callable. + for (int i = 0; i < segmentSize * 2 - refillWatermark + 1; i++) { + underTest.getCurrentRow(); + underTest.advance(); + } + + verify(callable, times(3)) + .call(queryCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + } + /** * This test checks that the buffer will stop filling up once the byte limit is reached. It will * cancel the controller after reached the limit. This test completes one fill and contains one From b2073c6c986304e65456efd2ba7af89774bfa61b Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Wed, 13 Dec 2023 04:50:29 -0800 Subject: [PATCH 106/224] Fix typo in notebook (#29728) --- examples/notebooks/beam-ml/run_custom_inference.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/beam-ml/run_custom_inference.ipynb b/examples/notebooks/beam-ml/run_custom_inference.ipynb index 2ca3b69bb72e..dde42399ff35 100644 --- a/examples/notebooks/beam-ml/run_custom_inference.ipynb +++ b/examples/notebooks/beam-ml/run_custom_inference.ipynb @@ -104,7 +104,7 @@ "\n", "To learn more about `spaCy`, create a `spaCy` language object in memory using `spaCy`'s trained models.\n", "You can install these models as Python packages.\n", - "For more inforamtion, see spaCy's [Models and Languages](https://spacy.io/usage/models) documentation." + "For more information, see spaCy's [Models and Languages](https://spacy.io/usage/models) documentation." ] }, { From c8deda7174300f64a2ea757b2b915adad56ab356 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 13 Dec 2023 08:35:35 -0500 Subject: [PATCH 107/224] Upgrade go version to resolve vulnerabilities (#29743) --- CHANGES.md | 2 +- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 70ce1a70b7ee..cf7ef0d4db06 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -81,7 +81,7 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) ## Deprecations 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 b5465442a1d8..82ab2593cb61 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2205,7 +2205,7 @@ class BeamModulePlugin implements Plugin { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.4" + project.ext.goVersion = "go1.21.5" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From 5d42cb654c08508125abd0a038acd97248298dfa Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Wed, 13 Dec 2023 09:21:58 -0500 Subject: [PATCH 108/224] revert kafka host change (#29725) --- .../src/main/java/org/apache/beam/examples/KafkaStreaming.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java index 602c34d4219d..5bad7911e646 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java +++ b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java @@ -102,7 +102,7 @@ public interface KafkaStreamingOptions extends PipelineOptions { * to use your own Kafka server. */ @Description("Kafka server host") - @Default.String("localhost:9092") + @Default.String("kafka_server:9092") String getKafkaHost(); void setKafkaHost(String value); From 86720287c2144ce4490a11fb1457d8869fb43075 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Wed, 13 Dec 2023 15:31:47 +0100 Subject: [PATCH 109/224] Deep-copy the process bundle descriptor when creating a new bundle processor. (#29742) --- sdks/python/apache_beam/runners/worker/sdk_worker.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index d553f0839e76..a64fb4c967a6 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -489,11 +489,18 @@ def get(self, instruction_id, bundle_descriptor_id): pass # Make sure we instantiate the processor while not holding the lock. + + # Reduce risks of concurrent modifications of the same protos + # captured in bundle descriptor when the same bundle descriptor is used + # in different instructions. + pbd = beam_fn_api_pb2.ProcessBundleDescriptor() + pbd.MergeFrom(self.fns[bundle_descriptor_id]) + processor = bundle_processor.BundleProcessor( self.runner_capabilities, - self.fns[bundle_descriptor_id], + pbd, self.state_handler_factory.create_state_handler( - self.fns[bundle_descriptor_id].state_api_service_descriptor), + pbd.state_api_service_descriptor), self.data_channel_factory, self.data_sampler) with self._lock: From db0bb446aa0607b3f8cf2a0cb2a133b2eddff826 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 31 Oct 2023 16:35:24 -0400 Subject: [PATCH 110/224] Add outputWindowedValue capability to Java SDK --- .../core/construction/SplittableParDo.java | 10 + .../SplittableParDoNaiveBounded.java | 38 +++ ...oundedSplittableProcessElementInvoker.java | 25 +- .../beam/runners/core/SimpleDoFnRunner.java | 79 ++++- .../apache/beam/sdk/transforms/Create.java | 191 ++++++++++++ .../org/apache/beam/sdk/transforms/DoFn.java | 58 ++++ .../sdk/transforms/DoFnOutputReceivers.java | 27 ++ .../beam/sdk/transforms/DoFnTester.java | 22 ++ .../transforms/windowing/GlobalWindow.java | 5 + .../beam/sdk/transforms/CreateTest.java | 81 +++++ .../transforms/reflect/DoFnInvokersTest.java | 22 ++ .../sql/zetasql/BeamZetaSqlCalcRel.java | 12 + .../beam/fn/harness/FnApiDoFnRunner.java | 295 +++++++++++++++++- .../StorageApiWriteUnshardedRecords.java | 21 ++ .../beam/sdk/io/gcp/spanner/SpannerIO.java | 12 + .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 11 + .../sdk/io/pulsar/ReadFromPulsarDoFnTest.java | 13 + ...adFromSparkReceiverWithOffsetDoFnTest.java | 13 + 18 files changed, 923 insertions(+), 12 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index 5ea2c4968dd9..9cf0606b68b9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -21,6 +21,7 @@ import com.google.auto.service.AutoService; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -669,6 +670,15 @@ public void output(RestrictionT part) { public void outputWithTimestamp(RestrictionT part, Instant timestamp) { throw new UnsupportedOperationException(); } + + @Override + public void outputWindowedValue( + RestrictionT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException(); + } }; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java index 061e1cb11b5c..ad6b51539742 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import java.util.Collection; import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.core.construction.SplittableParDo.ProcessKeyedElements; @@ -525,6 +526,15 @@ public void output(OutputT output) { public void outputWithTimestamp(OutputT output, Instant timestamp) { outerContext.outputWithTimestamp(output, timestamp); } + + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outerContext.outputWindowedValue(output, timestamp, windows, paneInfo); + } }; } @@ -543,6 +553,15 @@ public void output(T output) { public void outputWithTimestamp(T output, Instant timestamp) { outerContext.outputWithTimestamp(tag, output, timestamp); } + + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outerContext.outputWindowedValue(tag, output, timestamp, windows, paneInfo); + } }; } @@ -583,6 +602,15 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { outerContext.outputWithTimestamp(output, timestamp); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outerContext.outputWindowedValue(output, timestamp, windows, paneInfo); + } + @Override public void output(TupleTag tag, T output) { outerContext.output(tag, output); @@ -593,6 +621,16 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp outerContext.outputWithTimestamp(tag, output, timestamp); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outerContext.outputWindowedValue(tag, output, timestamp, windows, paneInfo); + } + @Override public InputT element() { return element; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 85a46eb7dc04..51cd8c690aee 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.util.Collection; import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.Future; @@ -42,6 +43,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; import org.apache.beam.sdk.transforms.splittabledofn.TimestampObservingWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -388,11 +390,20 @@ public void output(OutputT output) { @Override public void outputWithTimestamp(OutputT value, Instant timestamp) { + outputWindowedValue(value, timestamp, element.getWindows(), element.getPane()); + } + + @Override + public void outputWindowedValue( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { noteOutput(); if (watermarkEstimator instanceof TimestampObservingWatermarkEstimator) { ((TimestampObservingWatermarkEstimator) watermarkEstimator).observeTimestamp(timestamp); } - output.outputWindowedValue(value, timestamp, element.getWindows(), element.getPane()); + output.outputWindowedValue(value, timestamp, windows, paneInfo); } @Override @@ -402,11 +413,21 @@ public void output(TupleTag tag, T value) { @Override public void outputWithTimestamp(TupleTag tag, T value, Instant timestamp) { + outputWindowedValue(tag, value, timestamp, element.getWindows(), element.getPane()); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { noteOutput(); if (watermarkEstimator instanceof TimestampObservingWatermarkEstimator) { ((TimestampObservingWatermarkEstimator) watermarkEstimator).observeTimestamp(timestamp); } - output.outputWindowedValue(tag, value, timestamp, element.getWindows(), element.getPane()); + output.outputWindowedValue(tag, value, timestamp, windows, paneInfo); } private void noteOutput() { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 16986fdf8d52..b375d38c5a98 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -21,6 +21,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -413,22 +414,40 @@ public void output(OutputT output) { @Override public void outputWithTimestamp(OutputT output, Instant timestamp) { - checkTimestamp(elem.getTimestamp(), timestamp); outputWithTimestamp(mainOutputTag, output, timestamp); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); + } + @Override public void output(TupleTag tag, T output) { checkNotNull(tag, "Tag passed to output cannot be null"); - outputWindowedValue(tag, elem.withValue(output)); + SimpleDoFnRunner.this.outputWindowedValue(tag, elem.withValue(output)); } @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkNotNull(tag, "Tag passed to outputWithTimestamp cannot be null"); checkTimestamp(elem.getTimestamp(), timestamp); - outputWindowedValue( - tag, WindowedValue.of(output, timestamp, elem.getWindows(), elem.getPane())); + outputWindowedValue(tag, output, timestamp, elem.getWindows(), elem.getPane()); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + SimpleDoFnRunner.this.outputWindowedValue( + tag, WindowedValue.of(output, timestamp, windows, paneInfo)); } @Override @@ -838,16 +857,38 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { outputWithTimestamp(mainOutputTag, output, timestamp); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); + } + @Override public void output(TupleTag tag, T output) { checkTimestamp(timestamp(), timestamp); - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); + outputWithTimestamp(tag, output, timestamp); } @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkTimestamp(timestamp(), timestamp); - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); + outputWindowedValue( + tag, output, timestamp, Collections.singleton(window()), PaneInfo.NO_FIRING); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(timestamp(), timestamp); + SimpleDoFnRunner.this.outputWindowedValue( + tag, WindowedValue.of(output, timestamp, windows, paneInfo)); } @Override @@ -1045,16 +1086,38 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { outputWithTimestamp(mainOutputTag, output, timestamp); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); + } + @Override public void output(TupleTag tag, T output) { checkTimestamp(this.timestamp, timestamp); - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); + outputWithTimestamp(tag, output, timestamp); } @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkTimestamp(this.timestamp, timestamp); - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); + outputWindowedValue( + tag, output, timestamp, Collections.singleton(window()), PaneInfo.NO_FIRING); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(this.timestamp, timestamp); + SimpleDoFnRunner.this.outputWindowedValue( + tag, WindowedValue.of(output, timestamp, windows, paneInfo)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index 079379953cd9..dc81e23b10ee 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -55,7 +55,10 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -241,6 +244,37 @@ public static TimestampedValues timestamped( return timestamped(ImmutableList.>builder().add(elem).add(elems).build()); } + /** + * Returns a new {@link Create.WindowedValues} transform that produces a {@link PCollection} + * containing the elements of the provided {@code Iterable} with the specified windowing metadata. + * + *

    The argument should not be modified after this is called. + * + *

    By default, {@code Create.WindowedValues} can automatically determine the {@code Coder} to + * use if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. Otherwise, use {@link Create.WindowedValues#withCoder} to set the coder explicitly. + * + *

    Likewise, the window coder can be inferred if the window type is registered with the {@link + * CoderRegistry}. Otherwise, use {@link Create.WindowedValues#withWindowCoder(Coder)} to set the + * window coder explicitly. + */ + public static WindowedValues windowedValues(Iterable> elems) { + return new WindowedValues<>(elems, Optional.absent(), Optional.absent(), Optional.absent()); + } + + /** + * Returns a new {@link Create.WindowedValues} transform that produces a {@link PCollection} + * containing the specified elements with the specified windowing metadata. + * + *

    The arguments should not be modified after this is called. + */ + @SafeVarargs + public static WindowedValues windowedValues( + WindowedValue elem, @SuppressWarnings("unchecked") WindowedValue... elems) { + return windowedValues(ImmutableList.>builder().add(elem).add(elems).build()); + } + /** * Returns a new root transform that produces a {@link PCollection} containing the specified * elements with the specified timestamps. @@ -727,6 +761,163 @@ public void processElement(@Element TimestampedValue element, OutputReceiver< } } + /** + * A {@code PTransform} that creates a {@code PCollection} whose elements have associated + * windowing metadata. + */ + public static class WindowedValues extends PTransform> { + + /** + * Returns a {@link Create.WindowedValues} PTransform like this one that uses the given {@code + * Coder} to decode each of the objects into a value of type {@code T}. + * + *

    By default, {@code Create.TimestampedValues} can automatically determine the {@code Coder} + * to use if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. + * + *

    Note that for {@link Create.WindowedValues with no elements}, the {@link VoidCoder} is + * used. + */ + public WindowedValues withCoder(Coder coder) { + return new WindowedValues<>(windowedValues, Optional.of(coder), windowCoder, typeDescriptor); + } + + /** + * Returns a {@link Create.WindowedValues} PTransform like this one that uses the given {@code + * Coder} to decode each of the objects into a value of type {@code T}. + * + *

    By default, {@code Create.WindowedValues} can automatically determine the {@code Coder} to + * use if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. + * + *

    Note that for {@link Create.WindowedValues with no elements}, the {@link + * GlobalWindow.Coder} is used. + */ + public WindowedValues withWindowCoder(Coder windowCoder) { + return new WindowedValues<>( + windowedValues, elementCoder, Optional.of(windowCoder), typeDescriptor); + } + + /** + * Returns a {@link Create.WindowedValues} PTransform like this one that uses the given {@code + * Schema} to represent objects. + */ + public WindowedValues withSchema( + Schema schema, + TypeDescriptor typeDescriptor, + SerializableFunction toRowFunction, + SerializableFunction fromRowFunction) { + return withCoder(SchemaCoder.of(schema, typeDescriptor, toRowFunction, fromRowFunction)); + } + + /** + * Returns a {@link Create.WindowedValues} PTransform like this one that uses the given {@code + * TypeDescriptor} to determine the {@code Coder} to use to decode each of the objects into a + * value of type {@code T}. Note that a default coder must be registered for the class described + * in the {@code TypeDescriptor}. + * + *

    By default, {@code Create.TimestampedValues} can automatically determine the {@code Coder} + * to use if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. + * + *

    Note that for {@link Create.WindowedValues} with no elements, the {@link VoidCoder} is + * used. + */ + public WindowedValues withType(TypeDescriptor type) { + return new WindowedValues<>(windowedValues, elementCoder, windowCoder, Optional.of(type)); + } + + @Override + public PCollection expand(PBegin input) { + try { + Coder coder = null; + CoderRegistry coderRegistry = input.getPipeline().getCoderRegistry(); + SchemaRegistry schemaRegistry = input.getPipeline().getSchemaRegistry(); + if (elementCoder.isPresent()) { + coder = elementCoder.get(); + } else if (typeDescriptor.isPresent()) { + try { + coder = + SchemaCoder.of( + schemaRegistry.getSchema(typeDescriptor.get()), + typeDescriptor.get(), + schemaRegistry.getToRowFunction(typeDescriptor.get()), + schemaRegistry.getFromRowFunction(typeDescriptor.get())); + } catch (NoSuchSchemaException e) { + // No schema registered. + } + if (coder == null) { + coder = coderRegistry.getCoder(typeDescriptor.get()); + } + } else { + Iterable rawElements = Iterables.transform(windowedValues, WindowedValue::getValue); + coder = getDefaultCreateCoder(coderRegistry, schemaRegistry, rawElements); + } + + Coder windowCoder; + if (this.windowCoder.isPresent()) { + windowCoder = this.windowCoder.get(); + } else if (Iterables.isEmpty(windowedValues)) { + windowCoder = GlobalWindow.Coder.INSTANCE; + } else { + Iterable rawWindows = + Iterables.concat(Iterables.transform(windowedValues, WindowedValue::getWindows)); + windowCoder = getDefaultCreateCoder(coderRegistry, schemaRegistry, rawWindows); + } + + PCollection> intermediate = + Pipeline.applyTransform( + input, + Create.of(windowedValues) + .withCoder(WindowedValue.getFullCoder(coder, windowCoder))); + + PCollection output = intermediate.apply(ParDo.of(new ConvertWindowedValues<>())); + output.setCoder(coder); + return output; + } catch (CannotProvideCoderException e) { + throw new IllegalArgumentException( + "Unable to infer a coder and no Coder was specified. " + + "Please set a coder by invoking CreateTimestamped.withCoder() explicitly.", + e); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** The timestamped elements of the resulting PCollection. */ + private final transient Iterable> windowedValues; + + /** The coder used to encode the values to and from a binary representation. */ + private final transient Optional> elementCoder; + + private final Optional> windowCoder; + + /** The value type. */ + private final transient Optional> typeDescriptor; + + private WindowedValues( + Iterable> windowedValues, + Optional> elementCoder, + Optional> windowCoder, + Optional> typeDescriptor) { + this.windowedValues = windowedValues; + this.elementCoder = elementCoder; + this.windowCoder = windowCoder; + this.typeDescriptor = typeDescriptor; + } + + private static class ConvertWindowedValues extends DoFn, T> { + @ProcessElement + public void processElement(@Element WindowedValue element, OutputReceiver r) { + r.outputWindowedValue( + element.getValue(), element.getTimestamp(), element.getWindows(), element.getPane()); + } + } + } + private static Coder getDefaultCreateCoder( CoderRegistry coderRegistry, SchemaRegistry schemaRegistry, Iterable elems) throws CannotProvideCoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 2993d8cca97a..c22b726c99a2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -24,6 +24,7 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import java.util.Collection; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; @@ -185,6 +186,31 @@ public abstract class WindowedContext { */ public abstract void outputWithTimestamp(OutputT output, Instant timestamp); + /** + * Adds the given element to the main output {@code PCollection}, with the given windowing + * metadata. + * + *

    Once passed to {@code outputWindowedValue} the element should not be modified in any way. + * + *

    If invoked from {@link ProcessElement}), the timestamp must not be older than the input + * element's timestamp minus {@link DoFn#getAllowedTimestampSkew}. The output element will be in + * the same windows as the input element. + * + *

    If invoked from {@link StartBundle} or {@link FinishBundle}, this will attempt to use the + * {@link org.apache.beam.sdk.transforms.windowing.WindowFn} of the input {@code PCollection} to + * determine what windows the element should be in, throwing an exception if the {@code + * WindowFn} attempts to access any information about the input element except for the + * timestamp. + * + *

    Note: A splittable {@link DoFn} is not allowed to output from {@link StartBundle} + * or {@link FinishBundle} methods. + */ + public abstract void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo); + /** * Adds the given element to the output {@code PCollection} with the given tag. * @@ -231,6 +257,32 @@ public abstract class WindowedContext { * @see ParDo.SingleOutput#withOutputTags */ public abstract void outputWithTimestamp(TupleTag tag, T output, Instant timestamp); + + /** + * Adds the given element to the main output {@code PCollection}, with the given windowing + * metadata. + * + *

    Once passed to {@code outputWindowedValue} the element should not be modified in any way. + * + *

    If invoked from {@link ProcessElement}), the timestamp must not be older than the input + * element's timestamp minus {@link DoFn#getAllowedTimestampSkew}. The output element will be in + * the same windows as the input element. + * + *

    If invoked from {@link StartBundle} or {@link FinishBundle}, this will attempt to use the + * {@link org.apache.beam.sdk.transforms.windowing.WindowFn} of the input {@code PCollection} to + * determine what windows the element should be in, throwing an exception if the {@code + * WindowFn} attempts to access any information about the input element except for the + * timestamp. + * + *

    Note: A splittable {@link DoFn} is not allowed to output from {@link StartBundle} + * or {@link FinishBundle} methods. + */ + public abstract void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo); } /** Information accessible when running a {@link DoFn.ProcessElement} method. */ @@ -342,6 +394,12 @@ public interface OutputReceiver { void output(T output); void outputWithTimestamp(T output, Instant timestamp); + + void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo); } /** Receives tagged output for a multi-output function. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java index 4f3719f2cc0e..7adfd7768d34 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java @@ -20,12 +20,15 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.checkerframework.checker.nullness.qual.Nullable; @@ -58,6 +61,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { outputReceiver.outputWithTimestamp(schemaCoder.getFromRowFunction().apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outputReceiver.outputWindowedValue( + schemaCoder.getFromRowFunction().apply(output), timestamp, windows, paneInfo); + } } private static class WindowedContextOutputReceiver implements OutputReceiver { @@ -87,6 +100,20 @@ public void outputWithTimestamp(T output, Instant timestamp) { ((DoFn.WindowedContext) context).outputWithTimestamp(output, timestamp); } } + + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + if (outputTag != null) { + context.outputWindowedValue(outputTag, output, timestamp, windows, paneInfo); + } else { + ((DoFn.WindowedContext) context) + .outputWindowedValue(output, timestamp, windows, paneInfo); + } + } } private static class WindowedContextMultiOutputReceiver implements MultiOutputReceiver { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index f72de2af35ef..c0915c2dcd75 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -581,6 +582,15 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { outputWithTimestamp(mainOutputTag, output, timestamp); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); + } + @Override public void output(TupleTag tag, T output) { outputWithTimestamp(tag, output, element.getTimestamp()); @@ -591,6 +601,18 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp getMutableOutput(tag) .add(ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + for (BoundedWindow w : windows) { + getMutableOutput(tag).add(ValueInSingleWindow.of(output, timestamp, w, paneInfo)); + } + } } /** @deprecated Use {@link TestPipeline} with the {@code DirectRunner}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java index 28b92b390297..cadb2b33bb2a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java @@ -54,6 +54,11 @@ public int hashCode() { return GlobalWindow.class.hashCode(); } + @Override + public String toString() { + return "GlobalWindow"; + } + private GlobalWindow() {} /** {@link Coder} for encoding and decoding {@code GlobalWindow}s. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 689e5af7055b..85c8d0d04ede 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -35,6 +35,7 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import java.util.stream.Collectors; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -57,7 +58,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create.Values.CreateSource; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; @@ -240,6 +245,22 @@ public void processElement(ProcessContext c) { } } + private static class FormatMetadata extends DoFn { + @ProcessElement + public void processElement( + @Element String e, + @Timestamp Instant timestamp, + BoundedWindow w, + PaneInfo p, + OutputReceiver o) { + o.output(formatMetadata(e, timestamp, w, p)); + } + } + + private static String formatMetadata(String s, Instant timestamp, BoundedWindow w, PaneInfo p) { + return s + ":" + timestamp.getMillis() + ":" + w + ":" + p; + } + @Test @Category(NeedsRunner.class) public void testCreateTimestamped() { @@ -321,6 +342,66 @@ public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws assertThat(p.apply(values).getCoder(), equalTo(coder)); } + @Test + @Category(NeedsRunner.class) + public void testCreateWindowedValues() { + List> data = + Arrays.asList( + WindowedValue.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValue.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValue.of( + "c", new Instant(3L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + + // The easiest way to directly check the created PCollection with PAssert and without relying on + // other + // mechanisms than built-in DoFn processing is to dump it all to a string. + List formattedData = + data.stream() + .flatMap( + (WindowedValue windowedValue) -> + windowedValue.getWindows().stream() + .map( + (BoundedWindow w) -> + formatMetadata( + windowedValue.getValue(), + windowedValue.getTimestamp(), + w, + windowedValue.getPane()))) + .collect(Collectors.toList()); + + PCollection output = + p.apply(Create.windowedValues(data).withWindowCoder(GlobalWindow.Coder.INSTANCE)) + .apply(ParDo.of(new FormatMetadata())); + + PAssert.that(output).containsInAnyOrder(formattedData); + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testCreateWindowedValuesEmpty() { + PCollection output = + p.apply( + Create.windowedValues(new ArrayList>()) + .withCoder(StringUtf8Coder.of())); + + PAssert.that(output).empty(); + p.run(); + } + + @Test + public void testCreateWindowedValuesEmptyUnspecifiedCoder() { + p.enableAbandonedNodeEnforcement(false); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("determine a default Coder"); + thrown.expectMessage("Create.empty(Coder)"); + thrown.expectMessage("Create.empty(TypeDescriptor)"); + thrown.expectMessage("withCoder(Coder)"); + thrown.expectMessage("withType(TypeDescriptor)"); + p.apply(Create.windowedValues(new ArrayList<>())); + } + @Test @Category(ValidatesRunner.class) public void testCreateWithVoidType() throws Exception { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index a99d18a3d91f..bcd1f9ccf7b5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -41,6 +41,7 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; @@ -75,6 +76,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.UserCodeException; import org.joda.time.Instant; import org.junit.Before; @@ -556,6 +558,15 @@ public void output(SomeRestriction output) { public void outputWithTimestamp(SomeRestriction output, Instant timestamp) { fail("Unexpected output with timestamp"); } + + @Override + public void outputWindowedValue( + SomeRestriction output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + fail("Unexpected outputWindowedValue"); + } }; } }); @@ -800,6 +811,17 @@ public void outputWithTimestamp(String output, Instant instant) { invoked = true; assertEquals("foo", output); } + + @Override + public void outputWindowedValue( + String output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + assertFalse(invoked); + invoked = true; + assertEquals("foo", output); + } }; } }); diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java index d60ebe46b370..ad856c1c3a7d 100644 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java +++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java @@ -26,6 +26,7 @@ import com.google.zetasql.Value; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayDeque; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,6 +49,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; @@ -361,6 +363,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { c.output(tag, output, timestamp, w); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException( + "outputWindowedValue not supported in finish bundle here"); + } } private static RuntimeException extractException(Throwable e) { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 19c13775684e..69114703a528 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -2214,15 +2214,28 @@ public void output(TupleTag tag, T output) { @Override public void outputWithTimestamp(OutputT output, Instant timestamp) { - // TODO: Check that timestamp is valid once all runners can provide proper timestamps. + // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all + // runners can provide proper timestamps. outputTo( mainOutputConsumer, WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all + // runners can provide proper timestamps. + outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - // TODO: Check that timestamp is valid once all runners can provide proper timestamps. + // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all + // runners can provide proper timestamps. FnDataReceiver> consumer = (FnDataReceiver) localNameToConsumer.get(tag.getId()); if (consumer == null) { @@ -2232,6 +2245,23 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp consumer, WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all + // runners can provide proper timestamps. + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + if (consumer == null) { + throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); + } + outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @Override public State state(String stateId, boolean alwaysFetched) { StateDeclaration stateDeclaration = doFnSignature.stateDeclarations().get(stateId); @@ -2376,6 +2406,46 @@ public Instant timestamp(DoFn doFn) { currentElement.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(timestamp); + double size = + doFnInvoker.invokeGetSize( + new DelegatingArgumentProvider( + this, this.errorContextPrefix + "/GetSize") { + @Override + public Object restriction() { + return output; + } + + @Override + public Instant timestamp(DoFn doFn) { + return timestamp; + } + + @Override + public RestrictionTracker restrictionTracker() { + return doFnInvoker.invokeNewTracker(this); + } + }); + + outputTo( + mainOutputConsumer, + (WindowedValue) + WindowedValue.of( + KV.of( + KV.of( + currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), + size), + timestamp, + windows, + paneInfo)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { // Note that the OutputReceiver/RowOutputReceiver specifically will use the non-tag versions @@ -2384,6 +2454,19 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp String.format("Non-main output %s unsupported in %s", tag, errorContextPrefix)); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + // Note that the OutputReceiver/RowOutputReceiver specifically will use the non-tag versions + // of these methods when producing output. + throw new UnsupportedOperationException( + String.format("Non-main output %s unsupported in %s", tag, errorContextPrefix)); + } + @Override public State state(String stateId, boolean alwaysFetched) { throw new UnsupportedOperationException( @@ -2491,6 +2574,46 @@ public Instant timestamp(DoFn doFn) { currentElement.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(timestamp); + double size = + doFnInvoker.invokeGetSize( + new DelegatingArgumentProvider( + this, errorContextPrefix + "/GetSize") { + @Override + public Object restriction() { + return output; + } + + @Override + public Instant timestamp(DoFn doFn) { + return timestamp; + } + + @Override + public RestrictionTracker restrictionTracker() { + return doFnInvoker.invokeNewTracker(this); + } + }); + + outputTo( + mainOutputConsumer, + (WindowedValue) + WindowedValue.of( + KV.of( + KV.of( + currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), + size), + timestamp, + windows, + paneInfo)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { // Note that the OutputReceiver/RowOutputReceiver specifically will use the non-tag versions @@ -2498,6 +2621,19 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new UnsupportedOperationException( String.format("Non-main output %s unsupported in %s", tag, errorContextPrefix)); } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + // Note that the OutputReceiver/RowOutputReceiver specifically will use the non-tag versions + // of these methods when producing output. + throw new UnsupportedOperationException( + String.format("Non-main output %s unsupported in %s", tag, errorContextPrefix)); + } } /** Provides arguments for a {@link DoFnInvoker} for a non-window observing method. */ @@ -2534,6 +2670,16 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { output, timestamp, currentElement.getWindows(), currentElement.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(timestamp); + outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkTimestamp(timestamp); @@ -2547,6 +2693,22 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp WindowedValue.of( output, timestamp, currentElement.getWindows(), currentElement.getPane())); } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimestamp(timestamp); + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + if (consumer == null) { + throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); + } + outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } } /** Provides base arguments for a {@link DoFnInvoker} for a non-window observing method. */ @@ -2676,6 +2838,16 @@ public void outputWithTimestamp(Row output, Instant timestamp) { ProcessBundleContextBase.this.outputWithTimestamp( fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + ProcessBundleContextBase.this.outputWindowedValue( + fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; @Override @@ -2711,6 +2883,16 @@ public void output(T output) { public void outputWithTimestamp(T output, Instant timestamp) { ProcessBundleContextBase.this.outputWithTimestamp(tag, output, timestamp); } + + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + ProcessBundleContextBase.this.outputWindowedValue( + tag, output, timestamp, windows, paneInfo); + } }; } @@ -2746,6 +2928,16 @@ public void outputWithTimestamp(Row output, Instant timestamp) { ProcessBundleContextBase.this.outputWithTimestamp( tag, fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + ProcessBundleContextBase.this.outputWindowedValue( + tag, fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; } @@ -2860,6 +3052,16 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkOnWindowExpirationTimestamp(timestamp); + outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @Override public void output(TupleTag tag, T output) { FnDataReceiver> consumer = @@ -2885,6 +3087,19 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkOnWindowExpirationTimestamp(timestamp); + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @SuppressWarnings( "deprecation") // Allowed Skew is deprecated for users, but must be respected private void checkOnWindowExpirationTimestamp(Instant timestamp) { @@ -2956,6 +3171,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { context.outputWithTimestamp(fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue( + fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; @Override @@ -2988,6 +3213,15 @@ public void output(T output) { public void outputWithTimestamp(T output, Instant timestamp) { context.outputWithTimestamp(tag, output, timestamp); } + + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue(tag, output, timestamp, windows, paneInfo); + } }; } @@ -3019,6 +3253,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { context.outputWithTimestamp(tag, fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue( + tag, fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; } @@ -3104,6 +3348,16 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + checkTimerTimestamp(timestamp); + outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + } + @Override public void output(TupleTag tag, T output) { checkTimerTimestamp(currentTimer.getHoldTimestamp()); @@ -3130,6 +3384,14 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) {} + @Override public TimeDomain timeDomain() { return currentTimeDomain; @@ -3216,6 +3478,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { context.outputWithTimestamp(fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue( + fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; @Override @@ -3248,6 +3520,15 @@ public void output(T output) { public void outputWithTimestamp(T output, Instant timestamp) { context.outputWithTimestamp(tag, output, timestamp); } + + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue(tag, output, timestamp, windows, paneInfo); + } }; } @@ -3279,6 +3560,16 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { context.outputWithTimestamp(tag, fromRowFunction.apply(output), timestamp); } + + @Override + public void outputWindowedValue( + Row output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + context.outputWindowedValue( + tag, fromRowFunction.apply(output), timestamp, windows, paneInfo); + } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 8afcbd36d732..8f24ebc8ad9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -37,6 +37,7 @@ import java.io.IOException; import java.time.Instant; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -67,7 +68,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -1092,6 +1095,15 @@ public void outputWithTimestamp( BigQueryStorageApiInsertError output, org.joda.time.Instant timestamp) { context.output(failedRowsTag, output, timestamp, GlobalWindow.INSTANCE); } + + @Override + public void outputWindowedValue( + BigQueryStorageApiInsertError output, + org.joda.time.Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException("outputWindowedValue not supported"); + } }; @Nullable OutputReceiver successfulRowsReceiver = null; if (successfulRowsTag != null) { @@ -1106,6 +1118,15 @@ public void output(TableRow output) { public void outputWithTimestamp(TableRow output, org.joda.time.Instant timestamp) { context.output(successfulRowsTag, output, timestamp, GlobalWindow.INSTANCE); } + + @Override + public void outputWindowedValue( + TableRow output, + org.joda.time.Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException("outputWindowedValue not supported"); + } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 6db79ab69b47..3128de45fde3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -56,6 +56,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -102,9 +103,11 @@ import org.apache.beam.sdk.transforms.Wait; import org.apache.beam.sdk.transforms.WithTimestamps; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.FluentBackoff; @@ -2002,6 +2005,15 @@ public void output(Iterable output) { public void outputWithTimestamp(Iterable output, Instant timestamp) { c.output(output, timestamp, GlobalWindow.INSTANCE); } + + @Override + public void outputWindowedValue( + Iterable output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException("outputWindowedValue not supported"); + } } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index 48b5b060a295..845d974af0b4 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -50,6 +50,8 @@ import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -340,6 +342,15 @@ public void outputWithTimestamp( records.add(output); } + @Override + public void outputWindowedValue( + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException("Not expecting outputWindowedValue"); + } + public List getOutputs() { return this.records; } diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java index 273a1915d2bb..b72fe423efb1 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java @@ -21,11 +21,14 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.internal.DefaultImplementation; @@ -173,6 +176,16 @@ public void outputWithTimestamp( records.add(output); } + @Override + public void outputWindowedValue( + PulsarMessage output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException( + "unsupported outputWindowedValue in mock outputreceiver"); + } + public List getOutputs() { return records; } diff --git a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java index 33827164c6b7..bb0e6524241d 100644 --- a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java +++ b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java @@ -22,12 +22,15 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; @@ -64,6 +67,16 @@ public void outputWithTimestamp( records.add(output); } + @Override + public void outputWindowedValue( + String output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + throw new UnsupportedOperationException( + "Not expecting to receive call to outputWindowedValue"); + } + public List getOutputs() { return this.records; } From 06c3340e338ab380f7f5a18b11d7ed8cbe0f8bd0 Mon Sep 17 00:00:00 2001 From: Tom Stepp Date: Wed, 13 Dec 2023 07:46:43 -0800 Subject: [PATCH 111/224] Add logs to confirm JvmInitializer completed running and Spanner successfully connected. (#29727) * Add logs to confirm JvmInitializer completed running. * Add log to confirm Spanner successfully connected. --- .../src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java | 2 ++ .../org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java | 1 + 2 files changed, 3 insertions(+) diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java index c12adf8c02f1..f739a797af80 100644 --- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java +++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java @@ -35,6 +35,7 @@ public static void runOnStartup() { // We write to standard out since logging has yet to be initialized. System.out.format("Running JvmInitializer#onStartup for %s%n", initializer); initializer.onStartup(); + System.out.format("Completed JvmInitializer#onStartup for %s%n", initializer); } } @@ -52,6 +53,7 @@ public static void runBeforeProcessing(PipelineOptions options) { for (JvmInitializer initializer : ReflectHelpers.loadServicesOrdered(JvmInitializer.class)) { logger.info("Running JvmInitializer#beforeProcessing for {}", initializer); initializer.beforeProcessing(options); + logger.info("Completed JvmInitializer#beforeProcessing for {}", initializer); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java index bdfc933b5c74..471fe0329bf1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java @@ -98,6 +98,7 @@ public static SpannerAccessor getOrCreate(SpannerConfig spannerConfig) { // Connect to spanner for this SpannerConfig. LOG.info("Connecting to {}", spannerConfig); self = SpannerAccessor.createAndConnect(spannerConfig); + LOG.info("Successfully connected to {}", spannerConfig); spannerAccessors.put(spannerConfig, self); refcounts.putIfAbsent(spannerConfig, new AtomicInteger(0)); } From 8f90fdd62d3d9e4c04b80948e885fb76b7522c18 Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Wed, 13 Dec 2023 11:14:09 -0500 Subject: [PATCH 112/224] Upgrade Debian Bullseye to Bookworm (#29721) * Upgrade Debian Bullseye to Bookworm * Keep Playground images unchanged --- release/go-licenses/Dockerfile | 2 +- release/src/main/Dockerfile | 2 +- sdks/go/container/Dockerfile | 2 +- sdks/python/container/Dockerfile | 2 +- sdks/python/expansion-service-container/Dockerfile | 2 +- sdks/python/test-suites/direct/common.gradle | 2 +- .../content/en/documentation/runtime/environments.md | 10 +++++----- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/release/go-licenses/Dockerfile b/release/go-licenses/Dockerfile index 7008454ea6aa..1dfddc6e3745 100644 --- a/release/go-licenses/Dockerfile +++ b/release/go-licenses/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ############################################################################### -FROM golang:1.20-bullseye +FROM golang:1.20-bookworm WORKDIR /usr/src/app COPY go.mod ./ diff --git a/release/src/main/Dockerfile b/release/src/main/Dockerfile index 04d39765cfc2..62d389889ea5 100644 --- a/release/src/main/Dockerfile +++ b/release/src/main/Dockerfile @@ -23,7 +23,7 @@ # # then run `service docker start` and `docker login docker.io` to connect to the docker # daemon and authenticate with Docker Hub. -FROM debian:bullseye +FROM debian:bookworm LABEL Author "Apache Beam " SHELL ["/bin/bash", "-c"] diff --git a/sdks/go/container/Dockerfile b/sdks/go/container/Dockerfile index 30a9830cd195..7f1884cfb719 100644 --- a/sdks/go/container/Dockerfile +++ b/sdks/go/container/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ############################################################################### -FROM debian:bullseye +FROM debian:bookworm LABEL Author "Apache Beam " ARG TARGETOS diff --git a/sdks/python/container/Dockerfile b/sdks/python/container/Dockerfile index a49933ee6604..7bea6229668f 100644 --- a/sdks/python/container/Dockerfile +++ b/sdks/python/container/Dockerfile @@ -17,7 +17,7 @@ ############################################################################### ARG py_version -FROM python:"${py_version}"-bullseye as beam +FROM python:"${py_version}"-bookworm as beam LABEL Author "Apache Beam " ARG TARGETOS ARG TARGETARCH diff --git a/sdks/python/expansion-service-container/Dockerfile b/sdks/python/expansion-service-container/Dockerfile index c6a33cdae88e..d3cd4a4afad3 100644 --- a/sdks/python/expansion-service-container/Dockerfile +++ b/sdks/python/expansion-service-container/Dockerfile @@ -18,7 +18,7 @@ # We just need to support one Python version supported by Beam. # Picking the current default Beam Python version which is Python 3.8. -FROM python:3.8-bullseye as expansion-service +FROM python:3.8-bookworm as expansion-service LABEL Author "Apache Beam " ARG TARGETOS ARG TARGETARCH diff --git a/sdks/python/test-suites/direct/common.gradle b/sdks/python/test-suites/direct/common.gradle index da7aea95e1b0..771e0be19cfe 100644 --- a/sdks/python/test-suites/direct/common.gradle +++ b/sdks/python/test-suites/direct/common.gradle @@ -194,7 +194,7 @@ tasks.register("azureIntegrationTest") { doLast { exec { executable 'sh' - args '-c', "${rootDir}/sdks/python/apache_beam/io/azure/integration_test/azure_integration_test.sh python:${pythonContainerVersion}-bullseye" + args '-c', "${rootDir}/sdks/python/apache_beam/io/azure/integration_test/azure_integration_test.sh python:${pythonContainerVersion}-bookworm" } } } diff --git a/website/www/site/content/en/documentation/runtime/environments.md b/website/www/site/content/en/documentation/runtime/environments.md index 452fb6141e63..3d730700e1f3 100644 --- a/website/www/site/content/en/documentation/runtime/environments.md +++ b/website/www/site/content/en/documentation/runtime/environments.md @@ -180,13 +180,13 @@ Beam offers a way to provide your own custom container image. The easiest way to 1. Copy necessary artifacts from Apache Beam base image to your image. ``` # This can be any container image, - FROM python:3.7-bullseye + FROM python:3.8-bookworm # Install SDK. (needed for Python SDK) - RUN pip install --no-cache-dir apache-beam[gcp]==2.35.0 + RUN pip install --no-cache-dir apache-beam[gcp]==2.52.0 # Copy files from official SDK image, including script/dependencies. - COPY --from=apache/beam_python3.7_sdk:2.35.0 /opt/apache/beam /opt/apache/beam + COPY --from=apache/beam_python3.8_sdk:2.52.0 /opt/apache/beam /opt/apache/beam # Perform any additional customizations if desired @@ -194,7 +194,7 @@ Beam offers a way to provide your own custom container image. The easiest way to ENTRYPOINT ["/opt/apache/beam/boot"] ``` ->**NOTE**: This example assumes necessary dependencies (in this case, Python 3.7 and pip) have been installed on the existing base image. Installing the Apache Beam SDK into the image will ensure that the image has the necessary SDK dependencies and reduce the worker startup time. +>**NOTE**: This example assumes necessary dependencies (in this case, Python 3.8 and pip) have been installed on the existing base image. Installing the Apache Beam SDK into the image will ensure that the image has the necessary SDK dependencies and reduce the worker startup time. >The version specified in the `RUN` instruction must match the version used to launch the pipeline.
    >**Make sure that the Python or Java runtime version specified in the base image is the same as the version used to run the pipeline.** @@ -202,7 +202,7 @@ Beam offers a way to provide your own custom container image. The easiest way to 2. [Build](https://docs.docker.com/engine/reference/commandline/build/) and [push](https://docs.docker.com/engine/reference/commandline/push/) the image using Docker. ``` - export BASE_IMAGE="apache/beam_python3.7_sdk:2.25.0" + export BASE_IMAGE="apache/beam_python3.8_sdk:2.52.0" export IMAGE_NAME="myremoterepo/mybeamsdk" export TAG="latest" From 951b3b1a81b340f358ce0d1bfe7017416b1763bd Mon Sep 17 00:00:00 2001 From: clmccart Date: Wed, 13 Dec 2023 09:10:01 -0800 Subject: [PATCH 113/224] Per DoFn latency instrumentation (#29592) * adds active latency breakdown to windmill API proto --------- Co-authored-by: Claire McCarthy --- .../core/metrics/ExecutionStateSampler.java | 10 +- .../core/metrics/ExecutionStateTracker.java | 9 +- .../worker/ActiveMessageMetadata.java | 32 ++++ .../worker/DataflowExecutionContext.java | 60 ++++++- .../worker/DataflowExecutionStateSampler.java | 135 ++++++++++++++++ .../worker/StreamingDataflowWorker.java | 25 ++- .../worker/streaming/ActiveWorkState.java | 13 +- .../worker/streaming/ComputationState.java | 10 +- .../dataflow/worker/streaming/Work.java | 74 ++++++++- .../worker/DataflowExecutionContextTest.java | 76 +++++++++ .../DataflowExecutionStateSamplerTest.java | 148 ++++++++++++++++++ .../dataflow/worker/FakeWindmillServer.java | 10 +- .../worker/StreamingDataflowWorkerTest.java | 89 ++++++++++- .../worker/streaming/ActiveWorkStateTest.java | 4 +- .../windmill/src/main/proto/windmill.proto | 16 ++ 15 files changed, 667 insertions(+), 44 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ActiveMessageMetadata.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java index a2b745c74ede..9478c218b075 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateSampler.java @@ -44,10 +44,10 @@ public class ExecutionStateSampler { private static final ExecutionStateSampler INSTANCE = new ExecutionStateSampler(SYSTEM_MILLIS_PROVIDER); - private final MillisProvider clock; - @VisibleForTesting volatile long lastSampleTimeMillis; + protected final MillisProvider clock; + @VisibleForTesting protected volatile long lastSampleTimeMillis; - private ExecutionStateSampler(MillisProvider clock) { + protected ExecutionStateSampler(MillisProvider clock) { this.clock = clock; } @@ -147,12 +147,12 @@ public synchronized void stop() { } /** Add the tracker to the sampling set. */ - void addTracker(ExecutionStateTracker tracker) { + protected void addTracker(ExecutionStateTracker tracker) { this.activeTrackers.add(tracker); } /** Remove the tracker from the sampling set. */ - void removeTracker(ExecutionStateTracker tracker) { + protected void removeTracker(ExecutionStateTracker tracker) { activeTrackers.remove(tracker); // Attribute any remaining time since the last sampling while removing the tracker. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java index 0eae4a93245c..dc6fd2f8248c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java @@ -304,7 +304,14 @@ public long getNextLullReportMs() { return nextLullReportMs; } - void takeSample(long millisSinceLastSample) { + /** + * Called periodically by the {@link ExecutionStateSampler} to report time recorded by the + * tracker. + * + * @param millisSinceLastSample the time since the last sample was reported. As an approximation, + * all of that time should be associated with this tracker. + */ + public void takeSample(long millisSinceLastSample) { if (SAMPLING_UPDATER.compareAndSet(this, 0, 1)) { try { takeSampleOnce(millisSinceLastSample); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ActiveMessageMetadata.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ActiveMessageMetadata.java new file mode 100644 index 000000000000..bc6b930a432d --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ActiveMessageMetadata.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker; + +import com.google.auto.value.AutoValue; + +@AutoValue +public abstract class ActiveMessageMetadata { + + public abstract String userStepName(); + + public abstract long startTime(); + + static ActiveMessageMetadata create(String userStepName, Long startTime) { + return new AutoValue_ActiveMessageMetadata(userStepName, startTime); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java index d1de0833d87d..b861295d8b9d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java @@ -24,8 +24,11 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.IntSummaryStatistics; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Optional; import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.StepContext; @@ -46,6 +49,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Closer; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.DateTimeUtils.MillisProvider; import org.joda.time.Instant; /** Execution context for the Dataflow worker. */ @@ -245,6 +249,16 @@ public static class DataflowExecutionStateTracker extends ExecutionStateTracker private final ContextActivationObserverRegistry contextActivationObserverRegistry; private final String workItemId; + /** + * Metadata on the message whose processing is currently being managed by this tracker. If no + * message is actively being processed, activeMessageMetadata will be null. + */ + @Nullable private ActiveMessageMetadata activeMessageMetadata = null; + + private final MillisProvider clock = System::currentTimeMillis; + + private final Map processingTimesByStep = new HashMap<>(); + public DataflowExecutionStateTracker( ExecutionStateSampler sampler, DataflowOperationContext.DataflowExecutionState otherState, @@ -287,17 +301,32 @@ protected void takeSampleOnce(long millisSinceLastSample) { super.takeSampleOnce(millisSinceLastSample); } + /** + * Enter a new state on the tracker. If the new state is a Dataflow processing state, tracks the + * activeMessageMetadata with the start time of the new state. + */ @Override public Closeable enterState(ExecutionState newState) { Closeable baseCloseable = super.enterState(newState); final boolean isDataflowProcessElementState = newState.isProcessElementState && newState instanceof DataflowExecutionState; if (isDataflowProcessElementState) { - elementExecutionTracker.enter(((DataflowExecutionState) newState).getStepName()); + DataflowExecutionState newDFState = (DataflowExecutionState) newState; + if (newDFState.getStepName() != null && newDFState.getStepName().userName() != null) { + if (this.activeMessageMetadata != null) { + recordActiveMessageInProcessingTimesMap(); + } + this.activeMessageMetadata = + ActiveMessageMetadata.create(newDFState.getStepName().userName(), clock.getMillis()); + } + elementExecutionTracker.enter(newDFState.getStepName()); } return () -> { if (isDataflowProcessElementState) { + if (this.activeMessageMetadata != null) { + recordActiveMessageInProcessingTimesMap(); + } elementExecutionTracker.exit(); } baseCloseable.close(); @@ -307,5 +336,34 @@ public Closeable enterState(ExecutionState newState) { public String getWorkItemId() { return this.workItemId; } + + public Optional getActiveMessageMetadata() { + return Optional.ofNullable(activeMessageMetadata); + } + + public Map getProcessingTimesByStep() { + return Collections.unmodifiableMap(processingTimesByStep); + } + + /** + * Transitions the metadata for the currently active message to an entry in the completed + * processing times map. Sets the activeMessageMetadata to null after the entry has been + * recorded. + */ + private void recordActiveMessageInProcessingTimesMap() { + if (this.activeMessageMetadata == null) { + return; + } + this.processingTimesByStep.compute( + this.activeMessageMetadata.userStepName(), + (k, v) -> { + if (v == null) { + v = new IntSummaryStatistics(); + } + v.accept((int) (System.currentTimeMillis() - this.activeMessageMetadata.startTime())); + return v; + }); + this.activeMessageMetadata = null; + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java new file mode 100644 index 000000000000..c5c3b2d41805 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java @@ -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. + */ +package org.apache.beam.runners.dataflow.worker; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.HashMap; +import java.util.IntSummaryStatistics; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.beam.runners.core.metrics.ExecutionStateSampler; +import org.apache.beam.runners.core.metrics.ExecutionStateTracker; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionContext.DataflowExecutionStateTracker; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.joda.time.DateTimeUtils.MillisProvider; + +public final class DataflowExecutionStateSampler extends ExecutionStateSampler { + + private static final MillisProvider SYSTEM_MILLIS_PROVIDER = System::currentTimeMillis; + private static final DataflowExecutionStateSampler INSTANCE = + new DataflowExecutionStateSampler(SYSTEM_MILLIS_PROVIDER); + + private final ConcurrentHashMap activeTrackersByWorkId = + new ConcurrentHashMap<>(); + private final ConcurrentHashMap> + completedProcessingMetrics = new ConcurrentHashMap<>(); + + public static DataflowExecutionStateSampler instance() { + return INSTANCE; + } + + @VisibleForTesting + public static DataflowExecutionStateSampler newForTest(MillisProvider clock) { + return new DataflowExecutionStateSampler(checkNotNull(clock)); + } + + public DataflowExecutionStateSampler(MillisProvider clock) { + super(clock); + } + + @Override + public void addTracker(ExecutionStateTracker tracker) { + if (!(tracker instanceof DataflowExecutionStateTracker)) { + return; + } + DataflowExecutionStateTracker dfTracker = (DataflowExecutionStateTracker) tracker; + this.activeTrackersByWorkId.put(dfTracker.getWorkItemId(), dfTracker); + } + + private static Map mergeStepStatsMaps( + Map map1, Map map2) { + for (Entry steps : map2.entrySet()) { + map1.compute( + steps.getKey(), + (k, v) -> { + if (v == null) { + return steps.getValue(); + } + v.combine(steps.getValue()); + return v; + }); + } + return map1; + } + + @Override + public void removeTracker(ExecutionStateTracker tracker) { + if (!(tracker instanceof DataflowExecutionContext.DataflowExecutionStateTracker)) { + return; + } + DataflowExecutionStateTracker dfTracker = (DataflowExecutionStateTracker) tracker; + completedProcessingMetrics.put(dfTracker.getWorkItemId(), dfTracker.getProcessingTimesByStep()); + activeTrackersByWorkId.remove(dfTracker.getWorkItemId()); + + // Attribute any remaining time since the last sampling while removing the tracker. + // + // There is a race condition here; if sampling happens in the time between when we remove the + // tracker from activeTrackers and read the lastSampleTicks value, the sampling time will + // be lost for the tracker being removed. This is acceptable as sampling is already an + // approximation of actual execution time. + long millisSinceLastSample = clock.getMillis() - this.lastSampleTimeMillis; + if (millisSinceLastSample > 0) { + tracker.takeSample(millisSinceLastSample); + } + } + + @Override + public void doSampling(long millisSinceLastSample) { + for (DataflowExecutionStateTracker tracker : activeTrackersByWorkId.values()) { + tracker.takeSample(millisSinceLastSample); + } + } + + public Optional getActiveMessageMetadataForWorkId(String workId) { + if (activeTrackersByWorkId.containsKey(workId)) { + return Optional.ofNullable( + activeTrackersByWorkId.get(workId).getActiveMessageMetadata().orElse(null)); + } + return Optional.ofNullable(null); + } + + public Map getProcessingDistributionsForWorkId(String workId) { + if (!activeTrackersByWorkId.containsKey(workId)) { + if (completedProcessingMetrics.containsKey(workId)) { + return completedProcessingMetrics.get(workId); + } + return new HashMap<>(); + } + DataflowExecutionStateTracker tracker = activeTrackersByWorkId.get(workId); + return mergeStepStatsMaps( + completedProcessingMetrics.getOrDefault(workId, new HashMap<>()), + tracker.getProcessingTimesByStep()); + } + + public void resetForWorkId(String workId) { + completedProcessingMetrics.remove(workId); + } +} 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 92f7520676ad..0ddafa25f861 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 @@ -57,7 +57,6 @@ import java.util.function.Supplier; 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.MetricsLogger; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.internal.CustomSources; @@ -287,6 +286,8 @@ public class StreamingDataflowWorker { // Possibly overridden by streaming engine config. private int maxWorkItemCommitBytes = Integer.MAX_VALUE; + private final DataflowExecutionStateSampler sampler = DataflowExecutionStateSampler.instance(); + @VisibleForTesting StreamingDataflowWorker( List mapTasks, @@ -513,12 +514,8 @@ private static void sleep(int millis) { } /** 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); + private static void setUpWorkLoggingContext(String workId, String computationId) { + DataflowWorkerLoggingMDC.setWorkId(workId); DataflowWorkerLoggingMDC.setStageName(computationId); } @@ -579,7 +576,7 @@ public void start() { memoryMonitorThread.start(); dispatchThread.start(); commitThread.start(); - ExecutionStateSampler.instance().start(); + sampler.start(); // Periodically report workers counters and other updates. globalWorkerUpdatesTimer = executorSupplier.apply("GlobalWorkerUpdatesTimer"); @@ -951,7 +948,7 @@ private void process( final ByteString key = workItem.getKey(); work.setState(State.PROCESSING); - setUpWorkLoggingContext(workItem, computationId); + setUpWorkLoggingContext(work.getLatencyTrackingId(), computationId); LOG.debug("Starting processing for {}:\n{}", computationId, work); @@ -997,7 +994,7 @@ private void process( (InstructionOutputNode) Iterables.getOnlyElement(mapTaskNetwork.successors(readNode)); DataflowExecutionContext.DataflowExecutionStateTracker executionStateTracker = new DataflowExecutionContext.DataflowExecutionStateTracker( - ExecutionStateSampler.instance(), + sampler, stageInfo .executionStateRegistry() .getState( @@ -1007,7 +1004,7 @@ private void process( ScopedProfiler.INSTANCE.emptyScope()), stageInfo.deltaCounters(), options, - computationId); + work.getLatencyTrackingId()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( pendingDeltaCounters, @@ -1166,7 +1163,8 @@ public void close() { // Add the output to the commit queue. work.setState(State.COMMIT_QUEUED); - outputBuilder.addAllPerWorkItemLatencyAttributions(work.getLatencyAttributions()); + outputBuilder.addAllPerWorkItemLatencyAttributions( + work.getLatencyAttributions(false, work.getLatencyTrackingId(), sampler)); WorkItemCommitRequest commitRequest = outputBuilder.build(); int byteLimit = maxWorkItemCommitBytes; @@ -1296,6 +1294,7 @@ public void close() { stageInfo.timerProcessingMsecs().addValue(processingTimeMsecs); } + sampler.resetForWorkId(work.getLatencyTrackingId()); DataflowWorkerLoggingMDC.setWorkId(null); DataflowWorkerLoggingMDC.setStageName(null); } @@ -1878,7 +1877,7 @@ private void refreshActiveWork() { clock.get().minus(Duration.millis(options.getActiveWorkRefreshPeriodMillis())); for (Map.Entry entry : computationMap.entrySet()) { - active.put(entry.getKey(), entry.getValue().getKeysToRefresh(refreshDeadline)); + active.put(entry.getKey(), entry.getValue().getKeysToRefresh(refreshDeadline, sampler)); } metricTrackingWindmillServer.refreshActiveWork(active); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 9858666c40a2..16266de9d47c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -32,6 +32,7 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; @@ -210,14 +211,17 @@ private synchronized ImmutableMap getStuckCommitsAt( return stuckCommits.build(); } - synchronized ImmutableList getKeysToRefresh(Instant refreshDeadline) { + synchronized ImmutableList getKeysToRefresh( + Instant refreshDeadline, DataflowExecutionStateSampler sampler) { return activeWork.entrySet().stream() - .flatMap(entry -> toKeyedGetDataRequestStream(entry, refreshDeadline)) + .flatMap(entry -> toKeyedGetDataRequestStream(entry, refreshDeadline, sampler)) .collect(toImmutableList()); } private static Stream toKeyedGetDataRequestStream( - Entry> shardedKeyAndWorkQueue, Instant refreshDeadline) { + Entry> shardedKeyAndWorkQueue, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); Deque workQueue = shardedKeyAndWorkQueue.getValue(); @@ -229,7 +233,8 @@ private static Stream toKeyedGetDataRequestStream( .setKey(shardedKey.key()) .setShardingKey(shardedKey.shardingKey()) .setWorkToken(work.getWorkItem().getWorkToken()) - .addAllLatencyAttribution(work.getLatencyAttributions()) + .addAllLatencyAttribution( + work.getLatencyAttributions(true, work.getLatencyTrackingId(), sampler)) .build()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index 9d7a9131f584..4ac1d8bc9fac 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -19,14 +19,15 @@ 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.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.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.Instant; @@ -120,8 +121,9 @@ private void forceExecute(Work work) { } /** Adds any work started before the refreshDeadline to the GetDataRequest builder. */ - public List getKeysToRefresh(Instant refreshDeadline) { - return activeWorkState.getKeysToRefresh(refreshDeadline); + public ImmutableList getKeysToRefresh( + Instant refreshDeadline, DataflowExecutionStateSampler sampler) { + return activeWorkState.getKeysToRefresh(refreshDeadline, sampler); } public void printActiveWork(PrintWriter writer) { 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 index cc3f6d1b12b2..3a77a8322b4b 100644 --- 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 @@ -21,12 +21,22 @@ import java.util.ArrayList; import java.util.Collection; import java.util.EnumMap; +import java.util.IntSummaryStatistics; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.ActiveMessageMetadata; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.ActiveElementMetadata; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.Distribution; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -93,6 +103,14 @@ public Instant getStateStartTime() { return currentState.startTime(); } + public String getLatencyTrackingId() { + StringBuilder workIdBuilder = new StringBuilder(33); + workIdBuilder.append(Long.toHexString(workItem.getShardingKey())); + workIdBuilder.append('-'); + workIdBuilder.append(Long.toHexString(workItem.getWorkToken())); + return workIdBuilder.toString(); + } + private void recordGetWorkStreamLatencies( Collection getWorkStreamLatencies) { for (Windmill.LatencyAttribution latency : getWorkStreamLatencies) { @@ -101,7 +119,8 @@ private void recordGetWorkStreamLatencies( } } - public Collection getLatencyAttributions() { + public ImmutableList getLatencyAttributions( + boolean isHeartbeat, String workId, DataflowExecutionStateSampler sampler) { List list = new ArrayList<>(); for (Windmill.LatencyAttribution.State state : Windmill.LatencyAttribution.State.values()) { Duration duration = totalDurationPerState.getOrDefault(state, Duration.ZERO); @@ -111,13 +130,54 @@ public Collection getLatencyAttributions() { if (duration.equals(Duration.ZERO)) { continue; } - list.add( - Windmill.LatencyAttribution.newBuilder() - .setState(state) - .setTotalDurationMillis(duration.getMillis()) - .build()); + LatencyAttribution.Builder laBuilder = Windmill.LatencyAttribution.newBuilder(); + if (state == LatencyAttribution.State.ACTIVE) { + laBuilder = addActiveLatencyBreakdownToBuilder(isHeartbeat, laBuilder, workId, sampler); + } + Windmill.LatencyAttribution la = + laBuilder.setState(state).setTotalDurationMillis(duration.getMillis()).build(); + list.add(la); + } + return ImmutableList.copyOf(list); + } + + private static LatencyAttribution.Builder addActiveLatencyBreakdownToBuilder( + boolean isHeartbeat, + LatencyAttribution.Builder builder, + String workId, + DataflowExecutionStateSampler sampler) { + if (isHeartbeat) { + ActiveLatencyBreakdown.Builder stepBuilder = ActiveLatencyBreakdown.newBuilder(); + Optional activeMessage = + sampler.getActiveMessageMetadataForWorkId(workId); + if (!activeMessage.isPresent()) { + return builder; + } + stepBuilder.setUserStepName(activeMessage.get().userStepName()); + ActiveElementMetadata.Builder activeElementBuilder = ActiveElementMetadata.newBuilder(); + activeElementBuilder.setProcessingTimeMillis( + System.currentTimeMillis() - activeMessage.get().startTime()); + stepBuilder.setActiveMessageMetadata(activeElementBuilder); + builder.addActiveLatencyBreakdown(stepBuilder.build()); + return builder; + } + + Map processingDistributions = + sampler.getProcessingDistributionsForWorkId(workId); + for (Entry entry : processingDistributions.entrySet()) { + ActiveLatencyBreakdown.Builder stepBuilder = ActiveLatencyBreakdown.newBuilder(); + stepBuilder.setUserStepName(entry.getKey()); + Distribution.Builder distributionBuilder = + Distribution.newBuilder() + .setCount(entry.getValue().getCount()) + .setMin(entry.getValue().getMin()) + .setMax(entry.getValue().getMax()) + .setMean((long) entry.getValue().getAverage()) + .setSum(entry.getValue().getSum()); + stepBuilder.setProcessingTimesDistribution(distributionBuilder.build()); + builder.addActiveLatencyBreakdown(stepBuilder.build()); } - return list; + return builder; } boolean isStuckCommittingAt(Instant stuckCommitDeadline) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java index 7008dd4a4e97..3dc026ca7c83 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java @@ -17,14 +17,23 @@ */ package org.apache.beam.runners.dataflow.worker; +import static org.apache.beam.runners.core.metrics.ExecutionStateTracker.PROCESS_STATE_NAME; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.auto.service.AutoService; import java.io.Closeable; import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.IntSummaryStatistics; +import java.util.Map; +import org.apache.beam.runners.core.metrics.ExecutionStateSampler; import org.apache.beam.runners.core.metrics.ExecutionStateTracker; +import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionState; +import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -111,4 +120,71 @@ public void testContextActivationObserverActivation() throws Exception { // AutoRegistrationClassNotActive class is not registered as registrar for the same is disabled. assertFalse(AutoRegistrationClassNotActive.WAS_CALLED); } + + @Test + public void testDataflowExecutionStateTrackerRecordsActiveMessageMetadata() throws IOException { + DataflowExecutionContext.DataflowExecutionStateTracker tracker = + new DataflowExecutionContext.DataflowExecutionStateTracker( + ExecutionStateSampler.instance(), null, null, PipelineOptionsFactory.create(), ""); + StreamingModeExecutionState state = + new StreamingModeExecutionState( + NameContextsForTests.nameContextForTest(), + PROCESS_STATE_NAME, + null, + NoopProfileScope.NOOP, + null); + + Closeable closure = tracker.enterState(state); + + // After entering a process state, we should have an active message tracked. + ActiveMessageMetadata expectedMetadata = + ActiveMessageMetadata.create(NameContextsForTests.nameContextForTest().userName(), 1l); + assertTrue(tracker.getActiveMessageMetadata().isPresent()); + Assert.assertEquals( + expectedMetadata.userStepName(), tracker.getActiveMessageMetadata().get().userStepName()); + + closure.close(); + + // Once the state closes, the active message should get cleared. + assertFalse(tracker.getActiveMessageMetadata().isPresent()); + } + + @Test + public void testDataflowExecutionStateTrackerRecordsCompletedProcessingTimes() + throws IOException { + DataflowExecutionContext.DataflowExecutionStateTracker tracker = + new DataflowExecutionContext.DataflowExecutionStateTracker( + ExecutionStateSampler.instance(), null, null, PipelineOptionsFactory.create(), ""); + + // Enter a processing state + StreamingModeExecutionState state = + new StreamingModeExecutionState( + NameContextsForTests.nameContextForTest(), + PROCESS_STATE_NAME, + null, + NoopProfileScope.NOOP, + null); + tracker.enterState(state); + // Enter a new processing state + StreamingModeExecutionState newState = + new StreamingModeExecutionState( + NameContextsForTests.nameContextForTest(), + PROCESS_STATE_NAME, + null, + NoopProfileScope.NOOP, + null); + tracker.enterState(newState); + + // The first completed state should be recorded and the new state should be active. + Map gotProcessingTimes = tracker.getProcessingTimesByStep(); + Assert.assertEquals(1, gotProcessingTimes.size()); + Assert.assertEquals( + new HashSet<>(Arrays.asList(NameContextsForTests.nameContextForTest().userName())), + gotProcessingTimes.keySet()); + ActiveMessageMetadata expectedMetadata = + ActiveMessageMetadata.create(NameContextsForTests.nameContextForTest().userName(), 1l); + assertTrue(tracker.getActiveMessageMetadata().isPresent()); + Assert.assertEquals( + expectedMetadata.userStepName(), tracker.getActiveMessageMetadata().get().userStepName()); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java new file mode 100644 index 000000000000..1377148bc20a --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.HashMap; +import java.util.IntSummaryStatistics; +import java.util.Map; +import java.util.Optional; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionContext.DataflowExecutionStateTracker; +import org.apache.beam.runners.dataflow.worker.counters.NameContext; +import org.joda.time.DateTimeUtils.MillisProvider; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class DataflowExecutionStateSamplerTest { + + private MillisProvider clock; + private DataflowExecutionStateSampler sampler; + + @Before + public void setUp() { + clock = mock(MillisProvider.class); + sampler = DataflowExecutionStateSampler.newForTest(clock); + } + + private final TestOperationContext.TestDataflowExecutionState step1act1 = + new TestOperationContext.TestDataflowExecutionState( + createNameContext("test-stage1"), "activity1"); + + private NameContext createNameContext(String userName) { + return NameContext.create("", "", "", userName); + } + + @Test + public void testAddTrackerRemoveTrackerActiveMessageMetadataGetsUpdated() { + String workId = "work-item-id1"; + ActiveMessageMetadata testMetadata = + ActiveMessageMetadata.create(step1act1.getStepName().userName(), clock.getMillis()); + DataflowExecutionStateTracker trackerMock = createMockTracker(workId); + when(trackerMock.getActiveMessageMetadata()).thenReturn(Optional.of(testMetadata)); + + sampler.addTracker(trackerMock); + assertThat(sampler.getActiveMessageMetadataForWorkId(workId).get(), equalTo(testMetadata)); + + sampler.removeTracker(trackerMock); + Assert.assertFalse(sampler.getActiveMessageMetadataForWorkId(workId).isPresent()); + } + + @Test + public void testRemoveTrackerCompletedProcessingTimesGetsUpdated() { + String workId = "work-item-id1"; + Map testCompletedProcessingTimes = new HashMap<>(); + testCompletedProcessingTimes.put("some-step", new IntSummaryStatistics()); + DataflowExecutionStateTracker trackerMock = createMockTracker(workId); + when(trackerMock.getProcessingTimesByStep()).thenReturn(testCompletedProcessingTimes); + + sampler.addTracker(trackerMock); + sampler.removeTracker(trackerMock); + + assertThat( + sampler.getProcessingDistributionsForWorkId(workId), equalTo(testCompletedProcessingTimes)); + } + + @Test + public void testGetCompletedProcessingTimesAndActiveMessageFromActiveTracker() { + String workId = "work-item-id1"; + Map testCompletedProcessingTimes = new HashMap<>(); + IntSummaryStatistics testSummaryStats = new IntSummaryStatistics(); + testSummaryStats.accept(1); + testSummaryStats.accept(3); + testSummaryStats.accept(5); + testCompletedProcessingTimes.put("some-step", testSummaryStats); + ActiveMessageMetadata testMetadata = + ActiveMessageMetadata.create(step1act1.getStepName().userName(), clock.getMillis()); + DataflowExecutionStateTracker trackerMock = createMockTracker(workId); + when(trackerMock.getActiveMessageMetadata()).thenReturn(Optional.of(testMetadata)); + when(trackerMock.getProcessingTimesByStep()).thenReturn(testCompletedProcessingTimes); + + sampler.addTracker(trackerMock); + + assertThat(sampler.getActiveMessageMetadataForWorkId(workId).get(), equalTo(testMetadata)); + assertThat( + sampler.getProcessingDistributionsForWorkId(workId), equalTo(testCompletedProcessingTimes)); + } + + @Test + public void testResetForWorkIdClearsMaps() { + String workId1 = "work-item-id1"; + String workId2 = "work-item-id2"; + DataflowExecutionStateTracker tracker1Mock = createMockTracker(workId1); + DataflowExecutionStateTracker tracker2Mock = createMockTracker(workId2); + + sampler.addTracker(tracker1Mock); + sampler.addTracker(tracker2Mock); + + assertThat( + sampler.getActiveMessageMetadataForWorkId(workId1), + equalTo(tracker1Mock.getActiveMessageMetadata())); + assertThat( + sampler.getProcessingDistributionsForWorkId(workId1), + equalTo(tracker1Mock.getProcessingTimesByStep())); + assertThat( + sampler.getActiveMessageMetadataForWorkId(workId2), + equalTo(tracker2Mock.getActiveMessageMetadata())); + assertThat( + sampler.getProcessingDistributionsForWorkId(workId2), + equalTo(tracker2Mock.getProcessingTimesByStep())); + + sampler.removeTracker(tracker1Mock); + sampler.removeTracker(tracker2Mock); + sampler.resetForWorkId(workId2); + + assertThat( + sampler.getProcessingDistributionsForWorkId(workId1), + equalTo(tracker1Mock.getProcessingTimesByStep())); + Assert.assertTrue(sampler.getProcessingDistributionsForWorkId(workId2).isEmpty()); + } + + private DataflowExecutionStateTracker createMockTracker(String workItemId) { + DataflowExecutionStateTracker trackerMock = mock(DataflowExecutionStateTracker.class); + when(trackerMock.getWorkItemId()).thenReturn(workItemId); + return trackerMock; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 092f5e59a13c..a434b2001207 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -80,7 +80,7 @@ class FakeWindmillServer extends WindmillServerStub { private final ErrorCollector errorCollector; private final ConcurrentHashMap> droppedStreamingCommits; private int commitsRequested = 0; - private int numGetDataRequests = 0; + private List getDataRequests = new ArrayList<>(); private boolean isReady = true; private boolean dropStreamingCommits = false; @@ -144,7 +144,7 @@ private void validateGetDataRequest(Windmill.GetDataRequest request) { public Windmill.GetDataResponse getData(Windmill.GetDataRequest request) { LOG.info("getDataRequest: {}", request.toString()); validateGetDataRequest(request); - ++numGetDataRequests; + getDataRequests.add(request); GetDataResponse response = dataToOffer.getOrDefault(request); LOG.debug("getDataResponse: {}", response.toString()); return response; @@ -431,7 +431,11 @@ public Windmill.Exception getException() throws InterruptedException { } public int numGetDataRequests() { - return numGetDataRequests; + return getDataRequests.size(); + } + + public List getGetDataRequests() { + return getDataRequests; } public ArrayList getStatsReceived() { 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 2793cdd81820..31a9af9004a8 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 @@ -25,6 +25,7 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasProperty; import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -174,6 +175,7 @@ import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Assert; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -3273,7 +3275,8 @@ public void testLatencyAttributionProtobufsPopulated() { work.setState(Work.State.COMMITTING); clock.sleep(Duration.millis(60)); - Iterator it = work.getLatencyAttributions().iterator(); + Iterator it = + work.getLatencyAttributions(false, "", DataflowExecutionStateSampler.instance()).iterator(); assertTrue(it.hasNext()); LatencyAttribution lat = it.next(); assertSame(State.QUEUED, lat.getState()); @@ -3486,12 +3489,19 @@ public void testLatencyAttributionPopulatedInCommitRequest() throws Exception { worker.stop(); - assertEquals( - workItemCommitRequest.get((long) workToken).getPerWorkItemLatencyAttributions(0), + LatencyAttribution.Builder expectedActiveLA = LatencyAttribution.newBuilder() .setState(State.ACTIVE) - .setTotalDurationMillis(dofnWaitTimeMs) - .build()); + .setTotalDurationMillis(dofnWaitTimeMs); + assertThat( + workItemCommitRequest.get((long) workToken).getPerWorkItemLatencyAttributions(0), + hasProperty("state", Matchers.equalTo(State.ACTIVE))); + assertThat( + workItemCommitRequest.get((long) workToken).getPerWorkItemLatencyAttributions(0), + hasProperty("totalDurationMillis", Matchers.equalTo(1000L))); + assertThat( + workItemCommitRequest.get((long) workToken).getPerWorkItemLatencyAttributions(0), + hasProperty("activeLatencyBreakdown")); if (streamingEngine) { // Initial fake latency provided to FakeWindmillServer when invoke receiveWork in // GetWorkStream(). @@ -3504,6 +3514,75 @@ public void testLatencyAttributionPopulatedInCommitRequest() throws Exception { } } + @Test + public void testDoFnLatencyBreakdownsReportedOnCommit() throws Exception { + List instructions = + Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeDoFnInstruction(new SlowDoFn(), 0, StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); + + FakeWindmillServer server = new FakeWindmillServer(errorCollector); + StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); + options.setActiveWorkRefreshPeriodMillis(100); + StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */); + worker.start(); + + server.whenGetWorkCalled().thenReturn(makeInput(0, TimeUnit.MILLISECONDS.toMicros(0))); + + Map result = server.waitForAndGetCommits(1); + Windmill.WorkItemCommitRequest commit = result.get(0L); + + Windmill.LatencyAttribution.Builder laBuilder = + LatencyAttribution.newBuilder().setState(State.ACTIVE).setTotalDurationMillis(100); + for (LatencyAttribution la : commit.getPerWorkItemLatencyAttributionsList()) { + if (la.getState() == State.ACTIVE) { + assertThat(la.getActiveLatencyBreakdownCount(), equalTo(1)); + assertThat( + la.getActiveLatencyBreakdown(0).getUserStepName(), equalTo(DEFAULT_PARDO_USER_NAME)); + Assert.assertTrue(la.getActiveLatencyBreakdown(0).hasProcessingTimesDistribution()); + Assert.assertFalse(la.getActiveLatencyBreakdown(0).hasActiveMessageMetadata()); + } + } + + worker.stop(); + } + + @Test + public void testDoFnActiveMessageMetadataReportedOnHeartbeat() throws Exception { + if (!streamingEngine) { + return; + } + List instructions = + Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeDoFnInstruction(new SlowDoFn(), 0, StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); + + FakeWindmillServer server = new FakeWindmillServer(errorCollector); + StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); + options.setActiveWorkRefreshPeriodMillis(10); + StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */); + worker.start(); + + server.whenGetWorkCalled().thenReturn(makeInput(0, TimeUnit.MILLISECONDS.toMicros(0))); + + Map result = server.waitForAndGetCommits(1); + + assertThat(server.numGetDataRequests(), greaterThan(0)); + Windmill.GetDataRequest heartbeat = server.getGetDataRequests().get(2); + + for (LatencyAttribution la : + heartbeat.getRequests(0).getRequests(0).getLatencyAttributionList()) { + if (la.getState() == State.ACTIVE) { + assertTrue(la.getActiveLatencyBreakdownCount() > 0); + assertTrue(la.getActiveLatencyBreakdown(0).hasActiveMessageMetadata()); + } + } + + worker.stop(); + } + @Test public void testLimitOnOutputBundleSize() throws Exception { // This verifies that ReadOperation, StreamingModeExecutionContext, and windmill sinks diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 540166f226a6..ea57f687fd95 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Optional; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; 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; @@ -253,7 +254,8 @@ public void testGetKeysToRefresh() { activeWorkState.activateWorkForKey(shardedKey1, freshWork); activeWorkState.activateWorkForKey(shardedKey2, refreshableWork2); - ImmutableList requests = activeWorkState.getKeysToRefresh(refreshDeadline); + ImmutableList requests = + activeWorkState.getKeysToRefresh(refreshDeadline, DataflowExecutionStateSampler.instance()); ImmutableList expected = ImmutableList.of( diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto index 4b69533a151d..6aaeb57001e0 100644 --- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto +++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto @@ -59,6 +59,21 @@ message KeyedMessageBundle { } message LatencyAttribution { + message ActiveLatencyBreakdown { + message Distribution { + optional int64 count = 1; + optional int64 sum = 2; + optional int64 min = 3; + optional int64 max = 4; + optional int64 mean = 5; + } + message ActiveElementMetadata { + optional int64 processing_time_millis = 1; + } + optional string user_step_name = 1; + optional Distribution processing_times_distribution = 2; + optional ActiveElementMetadata active_message_metadata = 3; + } enum State { UNKNOWN = 0; QUEUED = 1; @@ -79,6 +94,7 @@ message LatencyAttribution { } optional State state = 1; optional int64 total_duration_millis = 2; + repeated ActiveLatencyBreakdown active_latency_breakdown = 3; } message PerStepNamespaceMetrics { From fa47640888cba41ab675b8a4bdb22b1d3a4b1fa1 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 13 Dec 2023 13:00:39 -0500 Subject: [PATCH 114/224] Remove comment body from action (#29750) --- .github/actions/setup-action/action.yml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/actions/setup-action/action.yml b/.github/actions/setup-action/action.yml index 743e89a931da..4c4bb2753194 100644 --- a/.github/actions/setup-action/action.yml +++ b/.github/actions/setup-action/action.yml @@ -36,8 +36,10 @@ runs: shell: bash if: github.event_name == 'issue_comment' && github.event.comment.body != inputs.comment_phrase run: | - echo "The comment ${{ github.event.comment.body }} does not match the phrase for this instance: ${{ inputs.comment_phrase }}. Exiting." + echo "The comment $PHRASE does not match the phrase for this instance: ${{ inputs.comment_phrase }}. Exiting." exit 1 + env: + PHRASE: "${{ github.event.comment.body }}" - name: Check out repository code if pull request commit shell: bash if: ${{ github.event_name == 'pull_request_target' }} @@ -69,4 +71,4 @@ runs: - name: expose gcloud path shell: bash run: | - echo KUBELET_GCLOUD_CONFIG_PATH=/var/lib/kubelet/pods/$POD_UID/volumes/kubernetes.io~empty-dir/gcloud >> $GITHUB_ENV \ No newline at end of file + echo KUBELET_GCLOUD_CONFIG_PATH=/var/lib/kubelet/pods/$POD_UID/volumes/kubernetes.io~empty-dir/gcloud >> $GITHUB_ENV From a04a4ad4cd2cd5ee5e22f131176e5e4867120d82 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Wed, 13 Dec 2023 13:36:16 -0500 Subject: [PATCH 115/224] [yaml] Normalize drop on MapToFields Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/yaml/yaml_mapping.py | 13 +++++++++++-- sdks/python/apache_beam/yaml/yaml_transform.py | 2 ++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 0ce706bbea58..13b4d900791a 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -48,6 +48,17 @@ from apache_beam.yaml.yaml_provider import dicts_to_rows +def normalize_mapping(spec): + """ + Normalizes various fields for mapping transforms. + """ + if spec['type'] == 'MapToFields': + config = spec.get('config') + if isinstance(config.get('drop'), str): + config['drop'] = [config['drop']] + return spec + + def _check_mapping_arguments( transform_name, expression=None, callable=None, name=None, path=None): # Argument checking @@ -453,8 +464,6 @@ def normalize_fields(pcoll, fields, drop=(), append=False, language='generic'): raise ValueError("Can only use expressions on a schema'd input.") from exn input_schema = {} - if isinstance(drop, str): - drop = [drop] if drop and not append: raise ValueError("Can only drop fields if append is true.") for name in drop: diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 8a5ccb3bb8b0..5d71583e1932 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -36,6 +36,7 @@ from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform from apache_beam.yaml import yaml_provider from apache_beam.yaml.yaml_combine import normalize_combine +from apache_beam.yaml.yaml_mapping import normalize_mapping __all__ = ["YamlTransform"] @@ -904,6 +905,7 @@ def preprocess_langauges(spec): for phase in [ ensure_transforms_have_types, + normalize_mapping, normalize_combine, preprocess_langauges, ensure_transforms_have_providers, From ad2d8dc6e306a82955ef65f48486d7aa775f9ea3 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 13 Dec 2023 15:00:03 -0500 Subject: [PATCH 116/224] Update Bigtable python client version (#29753) --- sdks/python/container/py310/base_image_requirements.txt | 2 +- sdks/python/container/py311/base_image_requirements.txt | 2 +- sdks/python/container/py38/base_image_requirements.txt | 2 +- sdks/python/container/py39/base_image_requirements.txt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 6ec2cc0a7565..a82ce0050da0 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -53,7 +53,7 @@ google-auth-httplib2==0.1.1 google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.23.0 -google-cloud-bigtable==2.21.0 +google-cloud-bigtable==2.22.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 google-cloud-dlp==3.13.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 435eb9712917..149368b15ed6 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -52,7 +52,7 @@ google-auth-httplib2==0.1.1 google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.23.0 -google-cloud-bigtable==2.21.0 +google-cloud-bigtable==2.22.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 google-cloud-dlp==3.13.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 51fb324d7c44..3bd996dbe2b9 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -54,7 +54,7 @@ google-auth-httplib2==0.1.1 google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.23.0 -google-cloud-bigtable==2.21.0 +google-cloud-bigtable==2.22.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 google-cloud-dlp==3.13.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index ce723259aa7c..a6bb3a31fe0a 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -53,7 +53,7 @@ google-auth-httplib2==0.1.1 google-cloud-aiplatform==1.37.0 google-cloud-bigquery==3.13.0 google-cloud-bigquery-storage==2.23.0 -google-cloud-bigtable==2.21.0 +google-cloud-bigtable==2.22.0 google-cloud-core==2.3.3 google-cloud-datastore==2.18.0 google-cloud-dlp==3.13.0 From e37fa77c3b60fc9eb9f1fe9cf3d8f57ce4a21b5e Mon Sep 17 00:00:00 2001 From: Jeff Kinard <35542536+Polber@users.noreply.github.com> Date: Wed, 13 Dec 2023 15:01:56 -0500 Subject: [PATCH 117/224] Pass getPort method through JDBC Resource Managers (#29732) Signed-off-by: Jeffrey Kinard --- .../apache/beam/it/jdbc/AbstractJDBCResourceManager.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java index 6d50dddb0ccd..73c55d001381 100644 --- a/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java +++ b/it/jdbc/src/main/java/org/apache/beam/it/jdbc/AbstractJDBCResourceManager.java @@ -99,11 +99,15 @@ public String getPassword() { return password; } + public int getPort() { + return this.getPort(getJDBCPort()); + } + @Override public synchronized String getUri() { return String.format( "jdbc:%s://%s:%d/%s", - getJDBCPrefix(), this.getHost(), this.getPort(getJDBCPort()), this.getDatabaseName()); + getJDBCPrefix(), this.getHost(), this.getPort(), this.getDatabaseName()); } public abstract String getJDBCPrefix(); From 8d41b62e828c7eb2e739529d518a588d40e23f48 Mon Sep 17 00:00:00 2001 From: jrmccluskey Date: Wed, 13 Dec 2023 20:03:28 +0000 Subject: [PATCH 118/224] Moving to 2.54.0-SNAPSHOT on master branch. --- .asf.yaml | 1 + gradle.properties | 4 ++-- sdks/go/pkg/beam/core/core.go | 2 +- sdks/python/apache_beam/version.py | 2 +- sdks/typescript/package.json | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index ef141f3bb3fa..d126c562838f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,6 +49,7 @@ github: protected_branches: master: {} + release-2.53.0: {} release-2.52.0: {} release-2.51.0: {} release-2.50.0: {} diff --git a/gradle.properties b/gradle.properties index ef84ea9a5cc9..8f8736a100f6 100644 --- a/gradle.properties +++ b/gradle.properties @@ -30,8 +30,8 @@ signing.gnupg.useLegacyGpg=true # buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy. # To build a custom Beam version make sure you change it in both places, see # https://github.com/apache/beam/issues/21302. -version=2.53.0-SNAPSHOT -sdk_version=2.53.0.dev +version=2.54.0-SNAPSHOT +sdk_version=2.54.0.dev javaVersion=1.8 diff --git a/sdks/go/pkg/beam/core/core.go b/sdks/go/pkg/beam/core/core.go index 07326d96528d..862b6eee94ba 100644 --- a/sdks/go/pkg/beam/core/core.go +++ b/sdks/go/pkg/beam/core/core.go @@ -27,7 +27,7 @@ const ( // SdkName is the human readable name of the SDK for UserAgents. SdkName = "Apache Beam SDK for Go" // SdkVersion is the current version of the SDK. - SdkVersion = "2.53.0.dev" + SdkVersion = "2.54.0.dev" // DefaultDockerImage represents the associated image for this release. DefaultDockerImage = "apache/beam_go_sdk:" + SdkVersion diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index fa890eab5005..f0df92e31e57 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -17,4 +17,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.53.0.dev' +__version__ = '2.54.0.dev' diff --git a/sdks/typescript/package.json b/sdks/typescript/package.json index 918846a79add..556ea1f6322a 100644 --- a/sdks/typescript/package.json +++ b/sdks/typescript/package.json @@ -1,6 +1,6 @@ { "name": "apache-beam", - "version": "2.53.0-SNAPSHOT", + "version": "2.54.0-SNAPSHOT", "devDependencies": { "@google-cloud/bigquery": "^5.12.0", "@types/mocha": "^9.0.0", From c49e9ec7dc7b028945a72ab1e72ff72ed7f28d4d Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Wed, 13 Dec 2023 15:54:42 -0500 Subject: [PATCH 119/224] test revert _beam_schema_proto (#29754) --- sdks/python/apache_beam/typehints/schemas.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index 6a6544aaa07c..b8176dccb8e2 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -561,7 +561,6 @@ def named_tuple_from_schema(self, schema: schema_pb2.Schema) -> type: '__reduce__', _named_tuple_reduce_method(schema.SerializeToString())) setattr(user_type, row_type._BEAM_SCHEMA_ID, schema.id) - user_type._beam_schema_proto = _Ephemeral(schema) self.schema_registry.add(user_type, schema) coders.registry.register_coder(user_type, coders.RowCoder) From 19858e990ad6ab6d74c6e3e6ac7e5a143759fbe0 Mon Sep 17 00:00:00 2001 From: Zechen Jiang Date: Wed, 13 Dec 2023 13:56:19 -0800 Subject: [PATCH 120/224] [#29760] Only respond to sampling request while data sampling is enabled (#29761) --- .../pkg/beam/core/runtime/harness/harness.go | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index 6a66c81a0a60..2888b39853e3 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -665,19 +665,21 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe case req.GetSampleData() != nil: msg := req.GetSampleData() var samples = make(map[string]*fnpb.SampleDataResponse_ElementList) - var elementsMap = c.dataSampler.GetSamples(msg.GetPcollectionIds()) - - for pid, elements := range elementsMap { - var elementList fnpb.SampleDataResponse_ElementList - for i := range elements { - var sampledElement = &fnpb.SampledElement{ - Element: elements[i].Element, - SampleTimestamp: timestamppb.New(elements[i].Timestamp), + if c.dataSampler != nil { + var elementsMap = c.dataSampler.GetSamples(msg.GetPcollectionIds()) + for pid, elements := range elementsMap { + var elementList fnpb.SampleDataResponse_ElementList + for i := range elements { + var sampledElement = &fnpb.SampledElement{ + Element: elements[i].Element, + SampleTimestamp: timestamppb.New(elements[i].Timestamp), + } + elementList.Elements = append(elementList.Elements, sampledElement) } - elementList.Elements = append(elementList.Elements, sampledElement) + samples[pid] = &elementList } - samples[pid] = &elementList } + return &fnpb.InstructionResponse{ InstructionId: string(instID), Response: &fnpb.InstructionResponse_SampleData{ From a9f5ab14d0689568c6822f93bec0c2ca7658cb57 Mon Sep 17 00:00:00 2001 From: Marc <53709151+marc7806@users.noreply.github.com> Date: Wed, 13 Dec 2023 23:31:01 +0100 Subject: [PATCH 121/224] Add support to use side inputs with Combine.PerKeyWithHotKeyFanout (#28867) --- .../apache/beam/sdk/transforms/Combine.java | 24 +++++++++++++++---- .../beam/sdk/transforms/CombineTest.java | 24 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index ffbfac460dcd..f1a964fa5a61 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1555,7 +1555,7 @@ public PerKey withSideInputs( */ public PerKeyWithHotKeyFanout withHotKeyFanout( SerializableFunction hotKeyFanout) { - return new PerKeyWithHotKeyFanout<>(fn, fnDisplayData, hotKeyFanout, fewKeys); + return new PerKeyWithHotKeyFanout<>(fn, fnDisplayData, hotKeyFanout, fewKeys, sideInputs); } /** @@ -1578,7 +1578,8 @@ public Integer apply(K unused) { return hotKeyFanout; } }, - fewKeys); + fewKeys, + sideInputs); } /** Returns the {@link GlobalCombineFn} used by this Combine operation. */ @@ -1624,18 +1625,20 @@ public static class PerKeyWithHotKeyFanout private final GlobalCombineFn fn; private final DisplayData.ItemSpec> fnDisplayData; private final SerializableFunction hotKeyFanout; - private final boolean fewKeys; + private final List> sideInputs; private PerKeyWithHotKeyFanout( GlobalCombineFn fn, DisplayData.ItemSpec> fnDisplayData, SerializableFunction hotKeyFanout, - boolean fewKeys) { + boolean fewKeys, + List> sideInputs) { this.fn = fn; this.fnDisplayData = fnDisplayData; this.hotKeyFanout = hotKeyFanout; this.fewKeys = fewKeys; + this.sideInputs = sideInputs; } @Override @@ -1928,6 +1931,10 @@ public void processElement( fewKeys ? Combine.fewKeys(hotPreCombine, fnDisplayData) : Combine.perKey(hotPreCombine, fnDisplayData); + if (!sideInputs.isEmpty()) { + hotPreCombineTransform = hotPreCombineTransform.withSideInputs(sideInputs); + } + PCollection>> precombinedHot = split .get(hot) @@ -1975,6 +1982,10 @@ public KV> apply(KV element) { fewKeys ? Combine.fewKeys(postCombine, fnDisplayData) : Combine.perKey(postCombine, fnDisplayData); + if (!sideInputs.isEmpty()) { + postCombineTransform = postCombineTransform.withSideInputs(sideInputs); + } + return PCollectionList.of(precombinedHot) .and(preprocessedCold) .apply(Flatten.pCollections()) @@ -1993,6 +2004,11 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData.item("fanoutFn", hotKeyFanout.getClass()).withLabel("Fanout Function")); } + /** Returns the side inputs used by this Combine operation. */ + public List> getSideInputs() { + return sideInputs; + } + /** * Used to store either an input or accumulator value, for flattening the hot and cold key * paths. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 024fedd177e5..f070378a64ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -1031,6 +1031,30 @@ public void testWithFanoutPreservesSideInputs() { assertEquals(Collections.singletonList(view), combine.getSideInputs()); } + + @Test + @Category({ValidatesRunner.class, UsesSideInputs.class}) + public void testHotKeyCombineWithSideInputs() { + PCollection> input = + createInput( + pipeline, + Arrays.asList( + KV.of("a", 1), KV.of("a", 1), KV.of("a", 4), KV.of("b", 1), KV.of("b", 13))); + PCollection sum = + input.apply(Values.create()).apply("Sum", Combine.globally(new SumInts())); + PCollectionView sumView = sum.apply(View.asSingleton()); + + PCollection> combinePerKeyWithSideInputsAndHotKey = + input.apply( + Combine.perKey(new TestCombineFnWithContext(sumView)) + .withSideInputs(sumView) + .withHotKeyFanout(1)); + + PAssert.that(combinePerKeyWithSideInputsAndHotKey) + .containsInAnyOrder(Arrays.asList(KV.of("a", "20:114"), KV.of("b", "20:113"))); + + pipeline.run(); + } } /** Tests validating windowing behaviors. */ From def026bfe11716c70b2d9d0ed7b21c078460db69 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 13 Dec 2023 15:33:55 -0800 Subject: [PATCH 122/224] Bump google.golang.org/api from 0.153.0 to 0.154.0 in /sdks (#29734) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.153.0 to 0.154.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.153.0...v0.154.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 17 ++++++++++++----- sdks/go.sum | 37 ++++++++++++++++++++++++++----------- 2 files changed, 38 insertions(+), 16 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 11bd34cf079b..b186a42930e3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,8 +59,8 @@ require ( golang.org/x/sync v0.5.0 golang.org/x/sys v0.15.0 golang.org/x/text v0.14.0 - google.golang.org/api v0.153.0 - google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 + google.golang.org/api v0.154.0 + google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 @@ -77,6 +77,8 @@ require ( dario.cat/mergo v1.0.0 // indirect github.com/Microsoft/hcsshim v0.11.1 // indirect github.com/containerd/log v0.1.0 // indirect + github.com/go-logr/logr v1.3.0 // indirect + github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect @@ -89,6 +91,11 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 // indirect + go.opentelemetry.io/otel v1.21.0 // indirect + go.opentelemetry.io/otel/metric v1.21.0 // indirect + go.opentelemetry.io/otel/trace v1.21.0 // indirect golang.org/x/time v0.5.0 // indirect ) @@ -130,7 +137,7 @@ require ( github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/go-control-plane v0.11.1 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.2 // indirect - github.com/felixge/httpsnoop v1.0.2 // indirect + github.com/felixge/httpsnoop v1.0.4 // indirect github.com/goccy/go-json v0.9.11 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect @@ -178,6 +185,6 @@ require ( golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index df82d0afbce1..763f27080752 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -21,7 +21,7 @@ cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiV cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/datacatalog v1.18.3 h1:zmdxP6nOjN5Qb1rtu9h4kbEVwerQ6Oshf+t747QJUew= +cloud.google.com/go/datacatalog v1.19.0 h1:rbYNmHwvAOOwnW2FPXYkaK3Mf1MmGqRzK0mMiIEyLdo= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.15.0 h1:0P9WcsQeTWjuD1H14JIY7XQscIPQ4Laje8ti96IC5vg= @@ -187,8 +187,8 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA= github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= -github.com/felixge/httpsnoop v1.0.2 h1:+nS9g82KMXccJ/wp0zyRW9ZBHFETmMGtkk+2CTTrW4o= -github.com/felixge/httpsnoop v1.0.2/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= +github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= +github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= @@ -199,6 +199,11 @@ github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gorp/gorp v2.2.0+incompatible h1:xAUh4QgEeqPPhK3vxZN+bzrim1z5Av6q837gtjUlshc= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= +github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= @@ -485,6 +490,16 @@ go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= +go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= +go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= +go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= +go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= +go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -687,8 +702,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.153.0 h1:N1AwGhielyKFaUqH07/ZSIQR3uNPcV7NVw0vj+j4iR4= -google.golang.org/api v0.153.0/go.mod h1:3qNJX5eOmhiWYc67jRA/3GsDw97UFb5ivv7Y2PrriAY= +google.golang.org/api v0.154.0 h1:X7QkVKZBskztmpPKWQXgjJRPA2dJYrL6r+sYPRLj050= +google.golang.org/api v0.154.0/go.mod h1:qhSMkM85hgqiokIYsrRyKxrjfBeIhgl4Z2JmeRkYylc= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -714,12 +729,12 @@ google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4 google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 h1:wpZ8pe2x1Q3f2KyT5f8oP/fa9rHAKgFPr/HZdNuS+PQ= -google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:J7XzRzVy1+IPwWHZUzoD0IccYZIrXILAQpc+Qy9CMhY= -google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 h1:JpwMPBpFN3uKhdaekDpiNlImDdkUAyiJ6ez/uxGaUSo= -google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:0xJLfVdJqpAPl8tDg1ujOCGzx6LFLttXT5NhllGOXY4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f h1:ultW7fxlIvee4HYrtnaRPon9HpEgFk5zYpmfMgtKB5I= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f/go.mod h1:L9KNLi232K1/xB6f7AlSX692koaRnKaWSR0stBki0Yc= +google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f h1:Vn+VyHU5guc9KjB5KrjI2q0wCOWEOIh0OEsleqakHJg= +google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f/go.mod h1:nWSwAFPb+qfNJXsoeO3Io7zf4tMSfN8EA8RlDA04GhY= +google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f h1:2yNACc1O40tTnrsbk9Cv6oxiW8pxI/pXj0wRtdlYmgY= +google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f/go.mod h1:Uy9bTZJqmfrw2rIBxgGLnamc78euZULUBrLZ9XTITKI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 h1:DC7wcm+i+P1rN3Ff07vL+OndGg5OhNddHyTA+ocPqYE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4/go.mod h1:eJVxU6o+4G1PSczBr85xmyvSNYAKvAYgkub40YGomFM= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 921e40a12f467c51161bf33a0144fb8a1d4ca334 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Thu, 14 Dec 2023 05:58:29 +0300 Subject: [PATCH 123/224] Dynamic SchemaTransform wrapper provider (#29561) * wrapper provider * add typehints * add GenerateSequence schematransform and expansion service for java core * address comments; add description property * add description test * add experimental note --- ...nerateSequenceSchemaTransformProvider.java | 201 +++++++++++++ .../beam/sdk/providers/package-info.java | 23 ++ ...teSequenceSchemaTransformProviderTest.java | 61 ++++ .../io/external/xlang_kafkaio_it_test.py | 4 +- .../python/apache_beam/transforms/external.py | 1 + .../external_schematransform_provider.py | 277 ++++++++++++++++++ .../external_schematransform_provider_test.py | 140 +++++++++ sdks/python/apache_beam/typehints/schemas.py | 3 + sdks/python/pytest.ini | 2 +- sdks/python/test-suites/dataflow/build.gradle | 4 +- sdks/python/test-suites/direct/build.gradle | 8 +- sdks/python/test-suites/gradle.properties | 4 +- sdks/python/test-suites/xlang/build.gradle | 9 +- 13 files changed, 723 insertions(+), 14 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/providers/package-info.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProviderTest.java create mode 100644 sdks/python/apache_beam/transforms/external_schematransform_provider.py create mode 100644 sdks/python/apache_beam/transforms/external_schematransform_provider_test.py diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java new file mode 100644 index 000000000000..f4cada661b03 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.providers; + +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 com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.providers.GenerateSequenceSchemaTransformProvider.GenerateSequenceConfiguration; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.joda.time.Duration; + +@AutoService(SchemaTransformProvider.class) +public class GenerateSequenceSchemaTransformProvider + extends TypedSchemaTransformProvider { + public static final String OUTPUT_ROWS_TAG = "output"; + public static final Schema OUTPUT_SCHEMA = Schema.builder().addInt64Field("value").build(); + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:generate_sequence:v1"; + } + + @Override + public List inputCollectionNames() { + return Collections.emptyList(); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_ROWS_TAG); + } + + @Override + public String description() { + return String.format( + "Outputs a PCollection of Beam Rows, each containing a single INT64 " + + "number called \"value\". The count is produced from the given \"start\"" + + "value and either up to the given \"end\" or until 2^63 - 1.\n" + + "To produce an unbounded PCollection, simply do not specify an \"end\" value. " + + "Unbounded sequences can specify a \"rate\" for output elements.\n" + + "In all cases, the sequence of numbers is generated in parallel, so there is no " + + "inherent ordering between the generated values"); + } + + @Override + public Class configurationClass() { + return GenerateSequenceConfiguration.class; + } + + @Override + public SchemaTransform from(GenerateSequenceConfiguration configuration) { + return new GenerateSequenceSchemaTransform(configuration); + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class GenerateSequenceConfiguration { + @AutoValue + public abstract static class Rate { + @SchemaFieldDescription("Number of elements component of the rate.") + public abstract Long getElements(); + + @SchemaFieldDescription("Number of seconds component of the rate.") + @Nullable + public abstract Long getSeconds(); + + public static Builder builder() { + return new AutoValue_GenerateSequenceSchemaTransformProvider_GenerateSequenceConfiguration_Rate + .Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setElements(Long elements); + + public abstract Builder setSeconds(Long seconds); + + public abstract Rate build(); + } + } + + public static Builder builder() { + return new AutoValue_GenerateSequenceSchemaTransformProvider_GenerateSequenceConfiguration + .Builder(); + } + + @SchemaFieldDescription("The minimum number to generate (inclusive).") + public abstract Long getStart(); + + @SchemaFieldDescription( + "The maximum number to generate (exclusive). Will be an unbounded sequence if left unspecified.") + @Nullable + public abstract Long getEnd(); + + @SchemaFieldDescription( + "Specifies the rate to generate a given number of elements per a given number of seconds. " + + "Applicable only to unbounded sequences.") + @Nullable + public abstract Rate getRate(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setStart(Long start); + + public abstract Builder setEnd(Long end); + + public abstract Builder setRate(Rate rate); + + public abstract GenerateSequenceConfiguration build(); + } + + public void validate() { + checkNotNull(this.getStart(), "Must specify a starting point \"start\"."); + Long start = this.getStart(); + Long end = this.getEnd(); + if (end != null) { + checkArgument(end == -1 || end >= start, "Invalid range [%s, %s)", start, end); + } + Rate rate = this.getRate(); + if (rate != null) { + checkArgument( + rate.getElements() > 0, + "Invalid rate specification. Expected positive elements component but received %s.", + rate.getElements()); + checkArgument( + Optional.ofNullable(rate.getSeconds()).orElse(1L) > 0, + "Invalid rate specification. Expected positive seconds component but received %s.", + rate.getSeconds()); + } + } + } + + protected static class GenerateSequenceSchemaTransform extends SchemaTransform { + private final GenerateSequenceConfiguration configuration; + + GenerateSequenceSchemaTransform(GenerateSequenceConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( + input.getAll().isEmpty(), "Expected no inputs but got: %s", input.getAll().keySet()); + + Long end = Optional.ofNullable(configuration.getEnd()).orElse(-1L); + GenerateSequenceConfiguration.Rate rate = configuration.getRate(); + + GenerateSequence sequence = GenerateSequence.from(configuration.getStart()).to(end); + if (rate != null) { + sequence = + sequence.withRate( + rate.getElements(), + Duration.standardSeconds(Optional.ofNullable(rate.getSeconds()).orElse(1L))); + } + + return PCollectionRowTuple.of( + OUTPUT_ROWS_TAG, + input + .getPipeline() + .apply(sequence) + .apply( + MapElements.into(TypeDescriptors.rows()) + .via(l -> Row.withSchema(OUTPUT_SCHEMA).withFieldValue("value", l).build())) + .setRowSchema(OUTPUT_SCHEMA)); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/package-info.java new file mode 100644 index 000000000000..6d90b7d018ad --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/package-info.java @@ -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. + */ + +/** + * Defines {@link org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider}s for transforms in + * the core module. + */ +package org.apache.beam.sdk.providers; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProviderTest.java new file mode 100644 index 000000000000..dcff3dedb843 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProviderTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.providers; + +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.providers.GenerateSequenceSchemaTransformProvider.GenerateSequenceConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class GenerateSequenceSchemaTransformProviderTest { + @Rule public transient TestPipeline p = TestPipeline.create(); + + @Test + @Category(NeedsRunner.class) + public void testGenerateSequence() { + GenerateSequenceConfiguration config = + GenerateSequenceConfiguration.builder().setStart(0L).setEnd(10L).build(); + SchemaTransform sequence = new GenerateSequenceSchemaTransformProvider().from(config); + + List expected = new ArrayList<>(10); + for (long i = 0L; i < 10L; i++) { + expected.add( + Row.withSchema(GenerateSequenceSchemaTransformProvider.OUTPUT_SCHEMA) + .withFieldValue("value", i) + .build()); + } + + PAssert.that( + PCollectionRowTuple.empty(p) + .apply(sequence) + .get(GenerateSequenceSchemaTransformProvider.OUTPUT_ROWS_TAG)) + .containsInAnyOrder(expected); + p.run(); + } +} diff --git a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py index a2f350e8cb7a..a7bf686d0642 100644 --- a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py +++ b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py @@ -143,7 +143,7 @@ def test_local_kafkaio_null_key(self): self.run_kafka_write(pipeline_creator) self.run_kafka_read(pipeline_creator, None) - @pytest.mark.uses_io_expansion_service + @pytest.mark.uses_io_java_expansion_service @unittest.skipUnless( os.environ.get('EXPANSION_PORT'), "EXPANSION_PORT environment var is not provided.") @@ -162,7 +162,7 @@ def test_hosted_kafkaio_populated_key(self): self.run_kafka_write(pipeline_creator) self.run_kafka_read(pipeline_creator, b'key') - @pytest.mark.uses_io_expansion_service + @pytest.mark.uses_io_java_expansion_service @unittest.skipUnless( os.environ.get('EXPANSION_PORT'), "EXPANSION_PORT environment var is not provided.") diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 997cea347d33..71dfc545204a 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -1067,6 +1067,7 @@ def __init__( append_args=None): path_to_jar = subprocess_server.JavaJarServer.path_to_beam_jar( gradle_target, gradle_appendix) + self.gradle_target = gradle_target super().__init__( path_to_jar, extra_args, classpath=classpath, append_args=append_args) diff --git a/sdks/python/apache_beam/transforms/external_schematransform_provider.py b/sdks/python/apache_beam/transforms/external_schematransform_provider.py new file mode 100644 index 000000000000..fd650087893a --- /dev/null +++ b/sdks/python/apache_beam/transforms/external_schematransform_provider.py @@ -0,0 +1,277 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 re +from collections import namedtuple +from typing import Dict +from typing import List +from typing import Tuple + +from apache_beam.transforms import PTransform +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import SchemaAwareExternalTransform +from apache_beam.transforms.external import SchemaTransformsConfig +from apache_beam.typehints.schemas import named_tuple_to_schema +from apache_beam.typehints.schemas import typing_from_runner_api + +__all__ = ['ExternalSchemaTransform', 'ExternalSchemaTransformProvider'] + + +def snake_case_to_upper_camel_case(string): + """Convert snake_case to UpperCamelCase""" + components = string.split('_') + output = ''.join(n.capitalize() for n in components) + return output + + +def snake_case_to_lower_camel_case(string): + """Convert snake_case to lowerCamelCase""" + if len(string) <= 1: + return string.lower() + upper = snake_case_to_upper_camel_case(string) + return upper[0].lower() + upper[1:] + + +def camel_case_to_snake_case(string): + """Convert camelCase to snake_case""" + arr = [] + word = [] + for i, n in enumerate(string): + # If seeing an upper letter after a lower letter, we just witnessed a word + # If seeing an upper letter and the next letter is lower, we may have just + # witnessed an all caps word + if n.isupper() and ((i > 0 and string[i - 1].islower()) or + (i + 1 < len(string) and string[i + 1].islower())): + arr.append(''.join(word)) + word = [n.lower()] + else: + word.append(n.lower()) + arr.append(''.join(word)) + return '_'.join(arr).strip('_') + + +# Information regarding a Wrapper parameter. +ParamInfo = namedtuple('ParamInfo', ['type', 'description', 'original_name']) + + +def get_config_with_descriptions( + schematransform: SchemaTransformsConfig) -> Dict[str, ParamInfo]: + # Prepare a configuration schema that includes types and descriptions + schema = named_tuple_to_schema(schematransform.configuration_schema) + descriptions = schematransform.configuration_schema._field_descriptions + fields_with_descriptions = {} + for field in schema.fields: + fields_with_descriptions[camel_case_to_snake_case(field.name)] = ParamInfo( + typing_from_runner_api(field.type), + descriptions[field.name], + field.name) + + return fields_with_descriptions + + +class ExternalSchemaTransform(PTransform): + """Template for a wrapper class of an external SchemaTransform + + This is a superclass for dynamically generated SchemaTransform wrappers and + is not meant to be manually instantiated. + + Experimental; no backwards compatibility guarantees.""" + + # These attributes need to be set when + # creating an ExternalSchemaTransform type + default_expansion_service = None + description: str = "" + identifier: str = "" + configuration_schema: Dict[str, ParamInfo] = {} + + def __init__(self, expansion_service=None, **kwargs): + self._kwargs = kwargs + self._expansion_service = \ + expansion_service or self.default_expansion_service + + def expand(self, input): + camel_case_kwargs = { + snake_case_to_lower_camel_case(k): v + for k, v in self._kwargs.items() + } + + external_schematransform = SchemaAwareExternalTransform( + identifier=self.identifier, + expansion_service=self._expansion_service, + rearrange_based_on_discovery=True, + **camel_case_kwargs) + + return input | external_schematransform + + +STANDARD_URN_PATTERN = r"^beam:schematransform:org.apache.beam:([\w-]+):(\w+)$" + + +def infer_name_from_identifier(identifier: str, pattern: str): + """Infer a class name from an identifier, adhering to the input pattern""" + match = re.match(pattern, identifier) + if not match: + return None + groups = match.groups() + + components = [snake_case_to_upper_camel_case(n) for n in groups] + # Special handling for standard SchemaTransform identifiers: + # We don't include the version number if it's the first version + if (pattern == STANDARD_URN_PATTERN and components[1].lower() == 'v1'): + return components[0] + else: + return ''.join(components) + + +class ExternalSchemaTransformProvider: + """Dynamically discovers Schema-aware external transforms from a given list + of expansion services and provides them as ready PTransforms. + + A :class:`ExternalSchemaTransform` subclass is generated for each external + transform, and is named based on what can be inferred from the URN + (see :param urn_pattern). + + These classes are generated when :class:`ExternalSchemaTransformProvider` is + initialized. We need to give it one or more expansion service addresses that + are already up and running: + >>> provider = ExternalSchemaTransformProvider(["localhost:12345", + ... "localhost:12121"]) + We can also give it the gradle target of a standard Beam expansion service: + >>> provider = ExternalSchemaTransform(BeamJarExpansionService( + ... "sdks:java:io:google-cloud-platform:expansion-service:shadowJar")) + Let's take a look at the output of :func:`get_available()` to know the + available transforms in the expansion service(s) we provided: + >>> provider.get_available() + [('JdbcWrite', 'beam:schematransform:org.apache.beam:jdbc_write:v1'), + ('BigtableRead', 'beam:schematransform:org.apache.beam:bigtable_read:v1'), + ...] + + Then retrieve a transform by :func:`get()`, :func:`get_urn()`, or by directly + accessing it as an attribute of :class:`ExternalSchemaTransformProvider`. + All of the following commands do the same thing: + >>> provider.get('BigqueryStorageRead') + >>> provider.get_urn( + ... 'beam:schematransform:org.apache.beam:bigquery_storage_read:v1') + >>> provider.BigqueryStorageRead + + To know more about the usage of a given transform, take a look at the + `description` attribute. This returns some documentation IF the underlying + SchemaTransform provides any. + >>> provider.BigqueryStorageRead.description + + Similarly, the `configuration_schema` attribute returns information about the + parameters, including their names, types, and any documentation that the + underlying SchemaTransform may provide: + >>> provider.BigqueryStorageRead.configuration_schema + {'query': ParamInfo(type=typing.Optional[str], description='The SQL query to + be executed to read from the BigQuery table.', original_name='query'), + 'row_restriction': ParamInfo(type=typing.Optional[str]...} + + The retrieved external transform can be used as a normal PTransform like so:: + + with Pipeline() as p: + _ = (p + | 'Read from BigQuery` >> provider.BigqueryStorageRead( + query=query, + row_restriction=restriction) + | 'Some processing' >> beam.Map(...)) + + Experimental; no backwards compatibility guarantees. + """ + def __init__(self, expansion_services, urn_pattern=STANDARD_URN_PATTERN): + f"""Initialize an ExternalSchemaTransformProvider + + :param expansion_services: + A list of expansion services to discover transforms from. + Supported forms: + * a string representing the expansion service address + * a :attr:`BeamJarExpansionService` pointing to a gradle target + :param urn_pattern: + The regular expression used to match valid transforms. In addition to + validating, the captured groups are used to infer a name for each class. + By default, the following pattern is used: [{STANDARD_URN_PATTERN}] + """ + self._urn_pattern = urn_pattern + self._transforms: Dict[str, type(ExternalSchemaTransform)] = {} + self._name_to_urn: Dict[str, str] = {} + + if isinstance(expansion_services, set): + expansion_services = list(expansion_services) + if not isinstance(expansion_services, list): + expansion_services = [expansion_services] + self.expansion_services = expansion_services + self._create_wrappers() + + def _create_wrappers(self): + # multiple services can overlap and include the same URNs. If this happens, + # we prioritize by the order of services in the list + identifiers = set() + for service in self.expansion_services: + target = service + if isinstance(service, BeamJarExpansionService): + target = service.gradle_target + try: + schematransform_configs = SchemaAwareExternalTransform.discover(service) + except Exception as e: + logging.exception( + "Encountered an error while discovering expansion service %s:\n%s", + target, + e) + continue + skipped_urns = [] + for config in schematransform_configs: + identifier = config.identifier + if identifier not in identifiers: + identifiers.add(identifier) + + name = infer_name_from_identifier(identifier, self._urn_pattern) + if name is None: + skipped_urns.append(identifier) + continue + + self._transforms[identifier] = type( + name, (ExternalSchemaTransform, ), + dict( + identifier=identifier, + default_expansion_service=service, + schematransform=config, + description=config.description, + configuration_schema=get_config_with_descriptions(config))) + self._name_to_urn[name] = identifier + + if skipped_urns: + logging.info( + "Skipped URN(s) in %s that don't follow the pattern [%s]: %s", + target, + self._urn_pattern, + skipped_urns) + + for transform in self._transforms.values(): + setattr(self, transform.__name__, transform) + + def get_available(self) -> List[Tuple[str, str]]: + """Get a list of available ExternalSchemaTransform names and identifiers""" + return list(self._name_to_urn.items()) + + def get(self, name) -> ExternalSchemaTransform: + """Get an ExternalSchemaTransform by its inferred class name""" + return self._transforms[self._name_to_urn[name]] + + def get_urn(self, identifier) -> ExternalSchemaTransform: + """Get an ExternalSchemaTransform by its SchemaTransform identifier""" + return self._transforms[identifier] diff --git a/sdks/python/apache_beam/transforms/external_schematransform_provider_test.py b/sdks/python/apache_beam/transforms/external_schematransform_provider_test.py new file mode 100644 index 000000000000..bf951e671c2a --- /dev/null +++ b/sdks/python/apache_beam/transforms/external_schematransform_provider_test.py @@ -0,0 +1,140 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import logging +import os +import unittest + +import pytest + +import apache_beam as beam +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external_schematransform_provider import STANDARD_URN_PATTERN +from apache_beam.transforms.external_schematransform_provider import ExternalSchemaTransformProvider +from apache_beam.transforms.external_schematransform_provider import camel_case_to_snake_case +from apache_beam.transforms.external_schematransform_provider import infer_name_from_identifier +from apache_beam.transforms.external_schematransform_provider import snake_case_to_lower_camel_case +from apache_beam.transforms.external_schematransform_provider import snake_case_to_upper_camel_case + + +class NameUtilsTest(unittest.TestCase): + def test_snake_case_to_upper_camel_case(self): + test_cases = [("", ""), ("test", "Test"), ("test_name", "TestName"), + ("test_double_underscore", "TestDoubleUnderscore"), + ("TEST_CAPITALIZED", "TestCapitalized"), + ("_prepended_underscore", "PrependedUnderscore"), + ("appended_underscore_", "AppendedUnderscore")] + for case in test_cases: + self.assertEqual(case[1], snake_case_to_upper_camel_case(case[0])) + + def test_snake_case_to_lower_camel_case(self): + test_cases = [("", ""), ("test", "test"), ("test_name", "testName"), + ("test_double_underscore", "testDoubleUnderscore"), + ("TEST_CAPITALIZED", "testCapitalized"), + ("_prepended_underscore", "prependedUnderscore"), + ("appended_underscore_", "appendedUnderscore")] + for case in test_cases: + self.assertEqual(case[1], snake_case_to_lower_camel_case(case[0])) + + def test_camel_case_to_snake_case(self): + test_cases = [("", ""), ("Test", "test"), ("TestName", "test_name"), + ("TestDoubleUnderscore", + "test_double_underscore"), ("MyToLoFo", "my_to_lo_fo"), + ("BEGINNINGAllCaps", + "beginning_all_caps"), ("AllCapsENDING", "all_caps_ending"), + ("AllCapsMIDDLEWord", "all_caps_middle_word"), + ("lowerCamelCase", "lower_camel_case")] + for case in test_cases: + self.assertEqual(case[1], camel_case_to_snake_case(case[0])) + + def test_infer_name_from_identifier(self): + standard_test_cases = [ + ("beam:schematransform:org.apache.beam:transform:v1", "Transform"), + ("beam:schematransform:org.apache.beam:my_transform:v1", + "MyTransform"), ( + "beam:schematransform:org.apache.beam:my_transform:v2", + "MyTransformV2"), + ("beam:schematransform:org.apache.beam:fe_fi_fo_fum:v2", "FeFiFoFumV2"), + ("beam:schematransform:bad_match:my_transform:v1", None) + ] + for case in standard_test_cases: + self.assertEqual( + case[1], infer_name_from_identifier(case[0], STANDARD_URN_PATTERN)) + + custom_pattern_cases = [ + # (, , ) + ( + r"^custom:transform:([\w-]+):(\w+)$", + "custom:transform:my_transform:v1", + "MyTransformV1"), + ( + r"^org.user:([\w-]+):([\w-]+):([\w-]+):external$", + "org.user:some:custom_transform:we_made:external", + "SomeCustomTransformWeMade"), + ( + r"^([\w-]+):user.transforms", + "my_eXTErnal:user.transforms", + "MyExternal"), + (r"^([\w-]+):user.transforms", "my_external:badinput.transforms", None), + ] + for case in custom_pattern_cases: + self.assertEqual(case[2], infer_name_from_identifier(case[1], case[0])) + + +@pytest.mark.uses_io_java_expansion_service +@unittest.skipUnless( + os.environ.get('EXPANSION_PORT'), + "EXPANSION_PORT environment var is not provided.") +class ExternalSchemaTransformProviderTest(unittest.TestCase): + def setUp(self): + self.test_pipeline = TestPipeline(is_integration_test=True) + + def test_generate_sequence_config_schema_and_description(self): + provider = ExternalSchemaTransformProvider( + BeamJarExpansionService(":sdks:java:io:expansion-service:shadowJar")) + + self.assertTrue(( + 'GenerateSequence', + 'beam:schematransform:org.apache.beam:generate_sequence:v1' + ) in provider.get_available()) + + GenerateSequence = provider.get('GenerateSequence') + config_schema = GenerateSequence.configuration_schema + for param in ['start', 'end', 'rate']: + self.assertTrue(param in config_schema) + + description_substring = ( + "Outputs a PCollection of Beam Rows, each " + "containing a single INT64") + self.assertTrue(description_substring in GenerateSequence.description) + + def test_run_generate_sequence(self): + provider = ExternalSchemaTransformProvider( + BeamJarExpansionService(":sdks:java:io:expansion-service:shadowJar")) + + with beam.Pipeline() as p: + numbers = p | provider.GenerateSequence( + start=0, end=10) | beam.Map(lambda row: row.value) + + assert_that(numbers, equal_to([i for i in range(10)])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index b8176dccb8e2..147a46f0bea5 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -540,6 +540,7 @@ def named_tuple_from_schema(self, schema: schema_pb2.Schema) -> type: type_name = 'BeamSchema_{}'.format(schema.id.replace('-', '_')) subfields = [] + descriptions = {} for field in schema.fields: try: field_py_type = self.typing_from_runner_api(field.type) @@ -550,6 +551,7 @@ def named_tuple_from_schema(self, schema: schema_pb2.Schema) -> type: "Failed to decode schema due to an issue with Field proto:\n\n" f"{text_format.MessageToString(field)}") from e + descriptions[field.name] = field.description subfields.append((field.name, field_py_type)) user_type = NamedTuple(type_name, subfields) @@ -560,6 +562,7 @@ def named_tuple_from_schema(self, schema: schema_pb2.Schema) -> type: user_type, '__reduce__', _named_tuple_reduce_method(schema.SerializeToString())) + setattr(user_type, "_field_descriptions", descriptions) setattr(user_type, row_type._BEAM_SCHEMA_ID, schema.id) self.schema_registry.add(user_type, schema) diff --git a/sdks/python/pytest.ini b/sdks/python/pytest.ini index 140476b29e50..4ffbb4524c06 100644 --- a/sdks/python/pytest.ini +++ b/sdks/python/pytest.ini @@ -33,7 +33,7 @@ markers = uses_gcp_java_expansion_service: collect Cross Language GCP Java transforms test runs uses_java_expansion_service: collect Cross Language Java transforms test runs uses_python_expansion_service: collect Cross Language Python transforms test runs - uses_io_expansion_service: collect Cross Language transform test runs (with Kafka bootstrap server) + uses_io_java_expansion_service: collect Cross Language IO Java transform test runs (with Kafka bootstrap server) uses_transform_service: collect Cross Language test runs that uses the Transform Service xlang_sql_expansion_service: collect for Cross Language with SQL expansion service test runs it_postcommit: collect for post-commit integration test runs diff --git a/sdks/python/test-suites/dataflow/build.gradle b/sdks/python/test-suites/dataflow/build.gradle index b55716a42df2..04a79683fd36 100644 --- a/sdks/python/test-suites/dataflow/build.gradle +++ b/sdks/python/test-suites/dataflow/build.gradle @@ -67,13 +67,13 @@ task examplesPostCommit { } task gcpCrossLanguagePostCommit { - getVersionsAsList('cross_language_validates_gcp_py_versions').each { + getVersionsAsList('cross_language_validates_py_versions').each { dependsOn.add(":sdks:python:test-suites:dataflow:py${getVersionSuffix(it)}:gcpCrossLanguagePythonUsingJava") } } task ioCrossLanguagePostCommit { - getVersionsAsList('cross_language_validates_gcp_py_versions').each { + getVersionsAsList('cross_language_validates_py_versions').each { dependsOn.add(":sdks:python:test-suites:dataflow:py${getVersionSuffix(it)}:ioCrossLanguagePythonUsingJava") } } diff --git a/sdks/python/test-suites/direct/build.gradle b/sdks/python/test-suites/direct/build.gradle index fc408411ec22..ea643c3303aa 100644 --- a/sdks/python/test-suites/direct/build.gradle +++ b/sdks/python/test-suites/direct/build.gradle @@ -32,7 +32,13 @@ tasks.register("examplesPostCommit") { } task gcpCrossLanguagePostCommit { - getVersionsAsList('cross_language_validates_gcp_py_versions').each { + getVersionsAsList('cross_language_validates_py_versions').each { dependsOn.add(":sdks:python:test-suites:direct:py${getVersionSuffix(it)}:gcpCrossLanguagePythonUsingJava") } } + +task ioCrossLanguagePostCommit { + getVersionsAsList('cross_language_validates_py_versions').each { + dependsOn.add(":sdks:python:test-suites:direct:py${getVersionSuffix(it)}:ioCrossLanguagePythonUsingJava") + } +} diff --git a/sdks/python/test-suites/gradle.properties b/sdks/python/test-suites/gradle.properties index 72fc651733db..c4e94bed4d84 100644 --- a/sdks/python/test-suites/gradle.properties +++ b/sdks/python/test-suites/gradle.properties @@ -47,5 +47,5 @@ samza_validates_runner_postcommit_py_versions=3.8,3.11 # spark runner test-suites spark_examples_postcommit_py_versions=3.8,3.11 -# cross language gcp io postcommit python test suites -cross_language_validates_gcp_py_versions=3.8,3.11 +# cross language postcommit python test suites +cross_language_validates_py_versions=3.8,3.11 diff --git a/sdks/python/test-suites/xlang/build.gradle b/sdks/python/test-suites/xlang/build.gradle index df3ebdd1582c..5a124ac20ce2 100644 --- a/sdks/python/test-suites/xlang/build.gradle +++ b/sdks/python/test-suites/xlang/build.gradle @@ -56,20 +56,17 @@ def gcpXlangCommon = new CrossLanguageTaskCommon().tap { startJobServer = setupTask cleanupJobServer = cleanupTask } -xlangTasks.add(gcpXlangCommon) - -def ioExpansionProject = project.project(':sdks:java:io:expansion-service') def ioXlangCommon = new CrossLanguageTaskCommon().tap { name = "ioCrossLanguage" - expansionProjectPath = ioExpansionProject.getPath() - collectMarker = "uses_io_expansion_service" + expansionProjectPath = project.project(':sdks:java:io:expansion-service').getPath() + collectMarker = "uses_io_java_expansion_service" startJobServer = setupTask cleanupJobServer = cleanupTask //See .test-infra/kafka/bitnami/README.md for setup instructions additionalEnvs = ["KAFKA_BOOTSTRAP_SERVER":project.findProperty('kafkaBootstrapServer')] } -xlangTasks.add(ioXlangCommon) +xlangTasks.addAll(gcpXlangCommon, ioXlangCommon) ext.xlangTasks = xlangTasks \ No newline at end of file From 4264c2c3e2586eb0a6175d71093509408194de06 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Wed, 13 Dec 2023 20:11:29 -0800 Subject: [PATCH 124/224] Fixes breakages of the upgrade feature (#29731) * Fixes breakages of the upgrade feature * Fix spotless * Addressing reviewer comments * Removing unused import * Reverting the PreCommit update --- .../core/construction/TransformUpgrader.java | 13 +- .../gcp/bigquery/BigQueryIOTranslation.java | 705 ++++++++++-------- sdks/java/io/kafka/upgrade/build.gradle | 2 - .../io/kafka/upgrade/KafkaIOTranslation.java | 414 +++++----- .../kafka/upgrade/KafkaIOTranslationTest.java | 11 + 5 files changed, 624 insertions(+), 521 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index e6dce752d06a..b142ab4af1c9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -20,6 +20,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InvalidClassException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.net.ServerSocket; @@ -49,12 +50,16 @@ 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; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A utility class that allows upgrading transforms of a given pipeline using the Beam Transform * Service. */ public class TransformUpgrader implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(TransformUpgrader.class); private static final String UPGRADE_NAMESPACE = "transform:upgrade:"; private ExpansionServiceClientFactory clientFactory; @@ -405,10 +410,16 @@ public static byte[] toByteArray(Object object) { * method. * @return re-generated object. */ - public static Object fromByteArray(byte[] bytes) { + public static Object fromByteArray(byte[] bytes) throws InvalidClassException { try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes); ObjectInputStream in = new ObjectInputStream(bis)) { return in.readObject(); + } catch (InvalidClassException e) { + LOG.info( + "An object cannot be re-generated from the provided byte array. Caller may use the " + + "default value for the parameter when upgrading. Underlying error: " + + e); + throw e; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java index a0c3bdbece51..1056328eb4c5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -26,6 +26,7 @@ import com.google.cloud.bigquery.storage.v1.AppendRowsRequest.MissingValueInterpretation; import com.google.cloud.bigquery.storage.v1.DataFormat; import java.io.IOException; +import java.io.InvalidClassException; import java.time.Duration; import java.util.Collection; import java.util.Collections; @@ -63,10 +64,14 @@ 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.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @SuppressWarnings({"rawtypes", "nullness"}) public class BigQueryIOTranslation { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryIOTranslation.class); + static class BigQueryIOReadTranslator implements TransformPayloadTranslator> { static Schema schema = @@ -184,105 +189,124 @@ public Row toConfigRow(TypedRead transform) { @Override public TypedRead fromConfigRow(Row configRow) { - BigQueryIO.TypedRead.Builder builder = new AutoValue_BigQueryIO_TypedRead.Builder<>(); + try { + BigQueryIO.TypedRead.Builder builder = new AutoValue_BigQueryIO_TypedRead.Builder<>(); - String jsonTableRef = configRow.getString("json_table_ref"); - if (jsonTableRef != null) { - builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); - } - String query = configRow.getString("query"); - if (query != null) { - builder = builder.setQuery(StaticValueProvider.of(query)); - } - Boolean validate = configRow.getBoolean("validate"); - if (validate != null) { - builder = builder.setValidate(validate); - } - Boolean flattenResults = configRow.getBoolean("flatten_results"); - if (flattenResults != null) { - builder = builder.setFlattenResults(flattenResults); - } - Boolean useLegacySQL = configRow.getBoolean("use_legacy_sql"); - if (useLegacySQL != null) { - builder = builder.setUseLegacySql(useLegacySQL); - } - Boolean withTemplateCompatibility = configRow.getBoolean("with_template_compatibility"); - if (withTemplateCompatibility != null) { - builder = builder.setWithTemplateCompatibility(withTemplateCompatibility); - } - byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); - if (bigqueryServicesBytes != null) { - builder = - builder.setBigQueryServices((BigQueryServices) fromByteArray(bigqueryServicesBytes)); - } - byte[] parseFnBytes = configRow.getBytes("parse_fn"); - if (parseFnBytes != null) { - builder = builder.setParseFn((SerializableFunction) fromByteArray(parseFnBytes)); - } - byte[] datumReaderFactoryBytes = configRow.getBytes("datum_reader_factory"); - if (datumReaderFactoryBytes != null) { - builder = - builder.setDatumReaderFactory( - (SerializableFunction) fromByteArray(datumReaderFactoryBytes)); - } - byte[] queryPriorityBytes = configRow.getBytes("query_priority"); - if (queryPriorityBytes != null) { - builder = builder.setQueryPriority((QueryPriority) fromByteArray(queryPriorityBytes)); - } - String queryLocation = configRow.getString("query_location"); - if (queryLocation != null) { - builder = builder.setQueryLocation(queryLocation); - } - String queryTempDataset = configRow.getString("query_temp_dataset"); - if (queryTempDataset != null) { - builder = builder.setQueryTempDataset(queryTempDataset); - } - byte[] methodBytes = configRow.getBytes("method"); - if (methodBytes != null) { - builder = builder.setMethod((TypedRead.Method) fromByteArray(methodBytes)); - } - byte[] formatBytes = configRow.getBytes("format"); - if (methodBytes != null) { - builder = builder.setFormat((DataFormat) fromByteArray(formatBytes)); - } - Collection selectedFields = configRow.getArray("selected_fields"); - if (selectedFields != null && !selectedFields.isEmpty()) { - builder.setSelectedFields(StaticValueProvider.of(ImmutableList.of(selectedFields))); - } - String rowRestriction = configRow.getString("row_restriction"); - if (rowRestriction != null) { - builder = builder.setRowRestriction(StaticValueProvider.of(rowRestriction)); - } - byte[] coderBytes = configRow.getBytes("coder"); - if (coderBytes != null) { - builder = builder.setCoder((Coder) fromByteArray(coderBytes)); - } - String kmsKey = configRow.getString("kms_key"); - if (kmsKey != null) { - builder = builder.setKmsKey(kmsKey); - } - byte[] typeDescriptorBytes = configRow.getBytes("type_descriptor"); - if (typeDescriptorBytes != null) { - builder = builder.setTypeDescriptor((TypeDescriptor) fromByteArray(typeDescriptorBytes)); - } - byte[] toBeamRowFnBytes = configRow.getBytes("to_beam_row_fn"); - if (toBeamRowFnBytes != null) { - builder = builder.setToBeamRowFn((ToBeamRowFunction) fromByteArray(toBeamRowFnBytes)); - } - byte[] fromBeamRowFnBytes = configRow.getBytes("from_beam_row_fn"); - if (fromBeamRowFnBytes != null) { - builder = builder.setFromBeamRowFn((FromBeamRowFunction) fromByteArray(fromBeamRowFnBytes)); - } - Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); - if (useAvroLogicalTypes != null) { - builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); - } - Boolean projectionPushdownApplied = configRow.getBoolean("projection_pushdown_applied"); - if (projectionPushdownApplied != null) { - builder = builder.setProjectionPushdownApplied(projectionPushdownApplied); - } + String jsonTableRef = configRow.getString("json_table_ref"); + if (jsonTableRef != null) { + builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); + } + String query = configRow.getString("query"); + if (query != null) { + builder = builder.setQuery(StaticValueProvider.of(query)); + } + Boolean validate = configRow.getBoolean("validate"); + if (validate != null) { + builder = builder.setValidate(validate); + } + Boolean flattenResults = configRow.getBoolean("flatten_results"); + if (flattenResults != null) { + builder = builder.setFlattenResults(flattenResults); + } + Boolean useLegacySQL = configRow.getBoolean("use_legacy_sql"); + if (useLegacySQL != null) { + builder = builder.setUseLegacySql(useLegacySQL); + } + Boolean withTemplateCompatibility = configRow.getBoolean("with_template_compatibility"); + if (withTemplateCompatibility != null) { + builder = builder.setWithTemplateCompatibility(withTemplateCompatibility); + } + byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); + if (bigqueryServicesBytes != null) { + try { + builder = + builder.setBigQueryServices( + (BigQueryServices) fromByteArray(bigqueryServicesBytes)); + } catch (InvalidClassException e) { + LOG.warn( + "Could not use the provided `BigQueryServices` implementation when upgrading." + + "Using the default."); + builder.setBigQueryServices(new BigQueryServicesImpl()); + } + } + byte[] parseFnBytes = configRow.getBytes("parse_fn"); + if (parseFnBytes != null) { + builder = builder.setParseFn((SerializableFunction) fromByteArray(parseFnBytes)); + } + byte[] datumReaderFactoryBytes = configRow.getBytes("datum_reader_factory"); + if (datumReaderFactoryBytes != null) { + builder = + builder.setDatumReaderFactory( + (SerializableFunction) fromByteArray(datumReaderFactoryBytes)); + } + byte[] queryPriorityBytes = configRow.getBytes("query_priority"); + if (queryPriorityBytes != null) { + builder = builder.setQueryPriority((QueryPriority) fromByteArray(queryPriorityBytes)); + } + String queryLocation = configRow.getString("query_location"); + if (queryLocation != null) { + builder = builder.setQueryLocation(queryLocation); + } + String queryTempDataset = configRow.getString("query_temp_dataset"); + if (queryTempDataset != null) { + builder = builder.setQueryTempDataset(queryTempDataset); + } + byte[] methodBytes = configRow.getBytes("method"); + if (methodBytes != null) { + builder = builder.setMethod((TypedRead.Method) fromByteArray(methodBytes)); + } + byte[] formatBytes = configRow.getBytes("format"); + if (methodBytes != null) { + builder = builder.setFormat((DataFormat) fromByteArray(formatBytes)); + } + Collection selectedFields = configRow.getArray("selected_fields"); + if (selectedFields != null && !selectedFields.isEmpty()) { + builder.setSelectedFields(StaticValueProvider.of(ImmutableList.of(selectedFields))); + } + String rowRestriction = configRow.getString("row_restriction"); + if (rowRestriction != null) { + builder = builder.setRowRestriction(StaticValueProvider.of(rowRestriction)); + } + byte[] coderBytes = configRow.getBytes("coder"); + if (coderBytes != null) { + try { + builder = builder.setCoder((Coder) fromByteArray(coderBytes)); + } catch (InvalidClassException e) { + LOG.warn( + "Could not use the provided `Coder` implementation when upgrading." + + "Using the default."); + } + } + String kmsKey = configRow.getString("kms_key"); + if (kmsKey != null) { + builder = builder.setKmsKey(kmsKey); + } + byte[] typeDescriptorBytes = configRow.getBytes("type_descriptor"); + if (typeDescriptorBytes != null) { + builder = builder.setTypeDescriptor((TypeDescriptor) fromByteArray(typeDescriptorBytes)); + } + byte[] toBeamRowFnBytes = configRow.getBytes("to_beam_row_fn"); + if (toBeamRowFnBytes != null) { + builder = builder.setToBeamRowFn((ToBeamRowFunction) fromByteArray(toBeamRowFnBytes)); + } + byte[] fromBeamRowFnBytes = configRow.getBytes("from_beam_row_fn"); + if (fromBeamRowFnBytes != null) { + builder = + builder.setFromBeamRowFn((FromBeamRowFunction) fromByteArray(fromBeamRowFnBytes)); + } + Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); + if (useAvroLogicalTypes != null) { + builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); + } + Boolean projectionPushdownApplied = configRow.getBoolean("projection_pushdown_applied"); + if (projectionPushdownApplied != null) { + builder = builder.setProjectionPushdownApplied(projectionPushdownApplied); + } - return builder.build(); + return builder.build(); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } } } @@ -529,239 +553,260 @@ public Row toConfigRow(Write transform) { @Override public Write fromConfigRow(Row configRow) { - BigQueryIO.Write.Builder builder = new AutoValue_BigQueryIO_Write.Builder<>(); - - String jsonTableRef = configRow.getString("json_table_ref"); - if (jsonTableRef != null) { - builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); - } - byte[] tableFunctionBytes = configRow.getBytes("table_function"); - if (tableFunctionBytes != null) { - builder = - builder.setTableFunction( - (SerializableFunction) - fromByteArray(tableFunctionBytes)); - } - byte[] formatFunctionBytes = configRow.getBytes("format_function"); - if (formatFunctionBytes != null) { - builder = - builder.setFormatFunction( - (SerializableFunction) fromByteArray(formatFunctionBytes)); - } - byte[] formatRecordOnFailureFunctionBytes = - configRow.getBytes("format_record_on_failure_function"); - if (tableFunctionBytes != null) { - builder = - builder.setFormatRecordOnFailureFunction( - (SerializableFunction) - fromByteArray(formatRecordOnFailureFunctionBytes)); - } - byte[] avroRowWriterFactoryBytes = configRow.getBytes("avro_row_writer_factory"); - if (avroRowWriterFactoryBytes != null) { - builder = - builder.setAvroRowWriterFactory( - (AvroRowWriterFactory) fromByteArray(avroRowWriterFactoryBytes)); - } - byte[] avroSchemaFactoryBytes = configRow.getBytes("avro_schema_factory"); - if (tableFunctionBytes != null) { - builder = - builder.setAvroSchemaFactory( - (SerializableFunction) fromByteArray(avroSchemaFactoryBytes)); - } - Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); - if (useAvroLogicalTypes != null) { - builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); - } - byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); - if (dynamicDestinationsBytes != null) { - builder = - builder.setDynamicDestinations( - (DynamicDestinations) fromByteArray(dynamicDestinationsBytes)); - } - String jsonSchema = configRow.getString("json_schema"); - if (jsonSchema != null) { - builder = builder.setJsonSchema(StaticValueProvider.of(jsonSchema)); - } - String jsonTimePartitioning = configRow.getString("json_time_partitioning"); - if (jsonTimePartitioning != null) { - builder = builder.setJsonTimePartitioning(StaticValueProvider.of(jsonTimePartitioning)); - } - byte[] clusteringBytes = configRow.getBytes("clustering"); - if (clusteringBytes != null) { - builder = builder.setClustering((Clustering) fromByteArray(clusteringBytes)); - } - byte[] createDispositionBytes = configRow.getBytes("create_disposition"); - if (createDispositionBytes != null) { - builder = - builder.setCreateDisposition((CreateDisposition) fromByteArray(createDispositionBytes)); - } - byte[] writeDispositionBytes = configRow.getBytes("write_disposition"); - if (writeDispositionBytes != null) { - builder = - builder.setWriteDisposition((WriteDisposition) fromByteArray(writeDispositionBytes)); - } - Collection schemaUpdateOptionsData = configRow.getArray("schema_update_options"); - if (schemaUpdateOptionsData != null) { - Set schemaUpdateOptions = - schemaUpdateOptionsData.stream() - .map(data -> (SchemaUpdateOption) fromByteArray(data)) - .collect(Collectors.toSet()); - builder = builder.setSchemaUpdateOptions(schemaUpdateOptions); - } else { - // This property is not nullable. - builder = builder.setSchemaUpdateOptions(Collections.emptySet()); - } - String tableDescription = configRow.getString("table_description"); - if (tableDescription != null) { - builder = builder.setTableDescription(tableDescription); - } - Boolean validate = configRow.getBoolean("validate"); - if (validate != null) { - builder = builder.setValidate(validate); - } - byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); - if (bigqueryServicesBytes != null) { - builder = - builder.setBigQueryServices((BigQueryServices) fromByteArray(bigqueryServicesBytes)); - } - Integer maxFilesPerBundle = configRow.getInt32("max_files_per_bundle"); - if (maxFilesPerBundle != null) { - builder = builder.setMaxFilesPerBundle(maxFilesPerBundle); - } - Long maxFileSize = configRow.getInt64("max_file_size"); - if (maxFileSize != null) { - builder = builder.setMaxFileSize(maxFileSize); - } - Integer numFileShards = configRow.getInt32("num_file_shards"); - if (numFileShards != null) { - builder = builder.setNumFileShards(numFileShards); - } - Integer numStorageWriteApiStreams = configRow.getInt32("num_storage_write_api_streams"); - if (numStorageWriteApiStreams != null) { - builder = builder.setNumStorageWriteApiStreams(numStorageWriteApiStreams); - } - Boolean propagateSuccessfulStorageApiWrites = - configRow.getBoolean("propagate_successful_storage_api_writes"); - if (propagateSuccessfulStorageApiWrites != null) { - builder = - builder.setPropagateSuccessfulStorageApiWrites(propagateSuccessfulStorageApiWrites); - } - Integer maxFilesPerPartition = configRow.getInt32("max_files_per_partition"); - if (maxFilesPerPartition != null) { - builder = builder.setMaxFilesPerPartition(maxFilesPerPartition); - } - Long maxBytesPerPartition = configRow.getInt64("max_bytes_per_partition"); - if (maxBytesPerPartition != null) { - builder = builder.setMaxBytesPerPartition(maxBytesPerPartition); - } - Duration triggerringFrequency = configRow.getValue("triggerring_frequency"); - if (triggerringFrequency != null) { - builder = - builder.setTriggeringFrequency( - org.joda.time.Duration.millis(triggerringFrequency.toMillis())); - } - byte[] methodBytes = configRow.getBytes("method"); - if (methodBytes != null) { - builder = builder.setMethod((Write.Method) fromByteArray(methodBytes)); - } - String loadJobProjectId = configRow.getString("load_job_project_id"); - if (loadJobProjectId != null) { - builder = builder.setLoadJobProjectId(StaticValueProvider.of(loadJobProjectId)); - } - byte[] failedInsertRetryPolicyBytes = configRow.getBytes("failed_insert_retry_policy"); - if (failedInsertRetryPolicyBytes != null) { - builder = - builder.setFailedInsertRetryPolicy( - (InsertRetryPolicy) fromByteArray(failedInsertRetryPolicyBytes)); - } - String customGcsTempLocations = configRow.getString("custom_gcs_temp_location"); - if (customGcsTempLocations != null) { - builder = builder.setCustomGcsTempLocation(StaticValueProvider.of(customGcsTempLocations)); - } - Boolean extendedErrorInfo = configRow.getBoolean("extended_error_info"); - if (extendedErrorInfo != null) { - builder = builder.setExtendedErrorInfo(extendedErrorInfo); - } - Boolean skipInvalidRows = configRow.getBoolean("skip_invalid_rows"); - if (skipInvalidRows != null) { - builder = builder.setSkipInvalidRows(skipInvalidRows); - } - Boolean ignoreUnknownValues = configRow.getBoolean("ignore_unknown_values"); - if (ignoreUnknownValues != null) { - builder = builder.setIgnoreUnknownValues(ignoreUnknownValues); - } - Boolean ignoreInsertIds = configRow.getBoolean("ignore_insert_ids"); - if (ignoreInsertIds != null) { - builder = builder.setIgnoreInsertIds(ignoreInsertIds); - } - Integer maxRetryJobs = configRow.getInt32("max_retry_jobs"); - if (maxRetryJobs != null) { - builder = builder.setMaxRetryJobs(maxRetryJobs); - } - String kmsKey = configRow.getString("kms_key"); - if (kmsKey != null) { - builder = builder.setKmsKey(kmsKey); - } - Collection primaryKey = configRow.getArray("primary_key"); - if (primaryKey != null && !primaryKey.isEmpty()) { - builder = builder.setPrimaryKey(ImmutableList.of(primaryKey)); - } - byte[] defaultMissingValueInterpretationsBytes = - configRow.getBytes("default_missing_value_interpretation"); - if (defaultMissingValueInterpretationsBytes != null) { - builder = - builder.setDefaultMissingValueInterpretation( - (MissingValueInterpretation) - fromByteArray(defaultMissingValueInterpretationsBytes)); - } - Boolean optimizeWrites = configRow.getBoolean("optimize_writes"); - if (optimizeWrites != null) { - builder = builder.setOptimizeWrites(optimizeWrites); - } - Boolean useBeamSchema = configRow.getBoolean("use_beam_schema"); - if (useBeamSchema != null) { - builder = builder.setUseBeamSchema(useBeamSchema); - } - Boolean autoSharding = configRow.getBoolean("auto_sharding"); - if (autoSharding != null) { - builder = builder.setAutoSharding(autoSharding); - } - Boolean propagateSuccessful = configRow.getBoolean("propagate_successful"); - if (propagateSuccessful != null) { - builder = builder.setPropagateSuccessful(propagateSuccessful); - } - Boolean autoSchemaUpdate = configRow.getBoolean("auto_schema_update"); - if (autoSchemaUpdate != null) { - builder = builder.setAutoSchemaUpdate(autoSchemaUpdate); - } - byte[] writeProtosClasses = configRow.getBytes("write_protos_class"); - if (writeProtosClasses != null) { - builder = - builder.setWriteProtosClass( - (Class) fromByteArray(defaultMissingValueInterpretationsBytes)); - } - Boolean directWriteProtos = configRow.getBoolean("direct_write_protos"); - if (directWriteProtos != null) { - builder = builder.setDirectWriteProtos(directWriteProtos); - } - byte[] deterministicRecordIdFnBytes = configRow.getBytes("deterministic_record_id_fn"); - if (deterministicRecordIdFnBytes != null) { - builder = - builder.setDeterministicRecordIdFn( - (SerializableFunction) fromByteArray(deterministicRecordIdFnBytes)); - } - String writeTempDataset = configRow.getString("write_temp_dataset"); - if (writeTempDataset != null) { - builder = builder.setWriteTempDataset(writeTempDataset); - } - byte[] rowMutationInformationFnBytes = configRow.getBytes("row_mutation_information_fn"); - if (rowMutationInformationFnBytes != null) { - builder = - builder.setRowMutationInformationFn( - (SerializableFunction) fromByteArray(rowMutationInformationFnBytes)); - } - - return builder.build(); + try { + BigQueryIO.Write.Builder builder = new AutoValue_BigQueryIO_Write.Builder<>(); + + String jsonTableRef = configRow.getString("json_table_ref"); + if (jsonTableRef != null) { + builder = builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef)); + } + byte[] tableFunctionBytes = configRow.getBytes("table_function"); + if (tableFunctionBytes != null) { + builder = + builder.setTableFunction( + (SerializableFunction) + fromByteArray(tableFunctionBytes)); + } + byte[] formatFunctionBytes = configRow.getBytes("format_function"); + if (formatFunctionBytes != null) { + builder = + builder.setFormatFunction( + (SerializableFunction) fromByteArray(formatFunctionBytes)); + } + byte[] formatRecordOnFailureFunctionBytes = + configRow.getBytes("format_record_on_failure_function"); + if (tableFunctionBytes != null) { + builder = + builder.setFormatRecordOnFailureFunction( + (SerializableFunction) + fromByteArray(formatRecordOnFailureFunctionBytes)); + } + byte[] avroRowWriterFactoryBytes = configRow.getBytes("avro_row_writer_factory"); + if (avroRowWriterFactoryBytes != null) { + builder = + builder.setAvroRowWriterFactory( + (AvroRowWriterFactory) fromByteArray(avroRowWriterFactoryBytes)); + } + byte[] avroSchemaFactoryBytes = configRow.getBytes("avro_schema_factory"); + if (tableFunctionBytes != null) { + builder = + builder.setAvroSchemaFactory( + (SerializableFunction) fromByteArray(avroSchemaFactoryBytes)); + } + Boolean useAvroLogicalTypes = configRow.getBoolean("use_avro_logical_types"); + if (useAvroLogicalTypes != null) { + builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes); + } + byte[] dynamicDestinationsBytes = configRow.getBytes("dynamic_destinations"); + if (dynamicDestinationsBytes != null) { + builder = + builder.setDynamicDestinations( + (DynamicDestinations) fromByteArray(dynamicDestinationsBytes)); + } + String jsonSchema = configRow.getString("json_schema"); + if (jsonSchema != null) { + builder = builder.setJsonSchema(StaticValueProvider.of(jsonSchema)); + } + String jsonTimePartitioning = configRow.getString("json_time_partitioning"); + if (jsonTimePartitioning != null) { + builder = builder.setJsonTimePartitioning(StaticValueProvider.of(jsonTimePartitioning)); + } + byte[] clusteringBytes = configRow.getBytes("clustering"); + if (clusteringBytes != null) { + builder = builder.setClustering((Clustering) fromByteArray(clusteringBytes)); + } + byte[] createDispositionBytes = configRow.getBytes("create_disposition"); + if (createDispositionBytes != null) { + builder = + builder.setCreateDisposition( + (CreateDisposition) fromByteArray(createDispositionBytes)); + } + byte[] writeDispositionBytes = configRow.getBytes("write_disposition"); + if (writeDispositionBytes != null) { + builder = + builder.setWriteDisposition((WriteDisposition) fromByteArray(writeDispositionBytes)); + } + Collection schemaUpdateOptionsData = configRow.getArray("schema_update_options"); + if (schemaUpdateOptionsData != null) { + Set schemaUpdateOptions = + schemaUpdateOptionsData.stream() + .map( + data -> { + try { + return (SchemaUpdateOption) fromByteArray(data); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + builder = builder.setSchemaUpdateOptions(schemaUpdateOptions); + } else { + // This property is not nullable. + builder = builder.setSchemaUpdateOptions(Collections.emptySet()); + } + String tableDescription = configRow.getString("table_description"); + if (tableDescription != null) { + builder = builder.setTableDescription(tableDescription); + } + Boolean validate = configRow.getBoolean("validate"); + if (validate != null) { + builder = builder.setValidate(validate); + } + byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services"); + if (bigqueryServicesBytes != null) { + try { + builder = + builder.setBigQueryServices( + (BigQueryServices) fromByteArray(bigqueryServicesBytes)); + } catch (InvalidClassException e) { + LOG.warn( + "Could not use the provided `BigQueryServices` implementation when upgrading." + + "Using the default."); + builder.setBigQueryServices(new BigQueryServicesImpl()); + } + } + Integer maxFilesPerBundle = configRow.getInt32("max_files_per_bundle"); + if (maxFilesPerBundle != null) { + builder = builder.setMaxFilesPerBundle(maxFilesPerBundle); + } + Long maxFileSize = configRow.getInt64("max_file_size"); + if (maxFileSize != null) { + builder = builder.setMaxFileSize(maxFileSize); + } + Integer numFileShards = configRow.getInt32("num_file_shards"); + if (numFileShards != null) { + builder = builder.setNumFileShards(numFileShards); + } + Integer numStorageWriteApiStreams = configRow.getInt32("num_storage_write_api_streams"); + if (numStorageWriteApiStreams != null) { + builder = builder.setNumStorageWriteApiStreams(numStorageWriteApiStreams); + } + Boolean propagateSuccessfulStorageApiWrites = + configRow.getBoolean("propagate_successful_storage_api_writes"); + if (propagateSuccessfulStorageApiWrites != null) { + builder = + builder.setPropagateSuccessfulStorageApiWrites(propagateSuccessfulStorageApiWrites); + } + Integer maxFilesPerPartition = configRow.getInt32("max_files_per_partition"); + if (maxFilesPerPartition != null) { + builder = builder.setMaxFilesPerPartition(maxFilesPerPartition); + } + Long maxBytesPerPartition = configRow.getInt64("max_bytes_per_partition"); + if (maxBytesPerPartition != null) { + builder = builder.setMaxBytesPerPartition(maxBytesPerPartition); + } + Duration triggerringFrequency = configRow.getValue("triggerring_frequency"); + if (triggerringFrequency != null) { + builder = + builder.setTriggeringFrequency( + org.joda.time.Duration.millis(triggerringFrequency.toMillis())); + } + byte[] methodBytes = configRow.getBytes("method"); + if (methodBytes != null) { + builder = builder.setMethod((Write.Method) fromByteArray(methodBytes)); + } + String loadJobProjectId = configRow.getString("load_job_project_id"); + if (loadJobProjectId != null) { + builder = builder.setLoadJobProjectId(StaticValueProvider.of(loadJobProjectId)); + } + byte[] failedInsertRetryPolicyBytes = configRow.getBytes("failed_insert_retry_policy"); + if (failedInsertRetryPolicyBytes != null) { + builder = + builder.setFailedInsertRetryPolicy( + (InsertRetryPolicy) fromByteArray(failedInsertRetryPolicyBytes)); + } + String customGcsTempLocations = configRow.getString("custom_gcs_temp_location"); + if (customGcsTempLocations != null) { + builder = + builder.setCustomGcsTempLocation(StaticValueProvider.of(customGcsTempLocations)); + } + Boolean extendedErrorInfo = configRow.getBoolean("extended_error_info"); + if (extendedErrorInfo != null) { + builder = builder.setExtendedErrorInfo(extendedErrorInfo); + } + Boolean skipInvalidRows = configRow.getBoolean("skip_invalid_rows"); + if (skipInvalidRows != null) { + builder = builder.setSkipInvalidRows(skipInvalidRows); + } + Boolean ignoreUnknownValues = configRow.getBoolean("ignore_unknown_values"); + if (ignoreUnknownValues != null) { + builder = builder.setIgnoreUnknownValues(ignoreUnknownValues); + } + Boolean ignoreInsertIds = configRow.getBoolean("ignore_insert_ids"); + if (ignoreInsertIds != null) { + builder = builder.setIgnoreInsertIds(ignoreInsertIds); + } + Integer maxRetryJobs = configRow.getInt32("max_retry_jobs"); + if (maxRetryJobs != null) { + builder = builder.setMaxRetryJobs(maxRetryJobs); + } + String kmsKey = configRow.getString("kms_key"); + if (kmsKey != null) { + builder = builder.setKmsKey(kmsKey); + } + Collection primaryKey = configRow.getArray("primary_key"); + if (primaryKey != null && !primaryKey.isEmpty()) { + builder = builder.setPrimaryKey(ImmutableList.of(primaryKey)); + } + byte[] defaultMissingValueInterpretationsBytes = + configRow.getBytes("default_missing_value_interpretation"); + if (defaultMissingValueInterpretationsBytes != null) { + builder = + builder.setDefaultMissingValueInterpretation( + (MissingValueInterpretation) + fromByteArray(defaultMissingValueInterpretationsBytes)); + } + Boolean optimizeWrites = configRow.getBoolean("optimize_writes"); + if (optimizeWrites != null) { + builder = builder.setOptimizeWrites(optimizeWrites); + } + Boolean useBeamSchema = configRow.getBoolean("use_beam_schema"); + if (useBeamSchema != null) { + builder = builder.setUseBeamSchema(useBeamSchema); + } + Boolean autoSharding = configRow.getBoolean("auto_sharding"); + if (autoSharding != null) { + builder = builder.setAutoSharding(autoSharding); + } + Boolean propagateSuccessful = configRow.getBoolean("propagate_successful"); + if (propagateSuccessful != null) { + builder = builder.setPropagateSuccessful(propagateSuccessful); + } + Boolean autoSchemaUpdate = configRow.getBoolean("auto_schema_update"); + if (autoSchemaUpdate != null) { + builder = builder.setAutoSchemaUpdate(autoSchemaUpdate); + } + byte[] writeProtosClasses = configRow.getBytes("write_protos_class"); + if (writeProtosClasses != null) { + builder = + builder.setWriteProtosClass( + (Class) fromByteArray(defaultMissingValueInterpretationsBytes)); + } + Boolean directWriteProtos = configRow.getBoolean("direct_write_protos"); + if (directWriteProtos != null) { + builder = builder.setDirectWriteProtos(directWriteProtos); + } + byte[] deterministicRecordIdFnBytes = configRow.getBytes("deterministic_record_id_fn"); + if (deterministicRecordIdFnBytes != null) { + builder = + builder.setDeterministicRecordIdFn( + (SerializableFunction) fromByteArray(deterministicRecordIdFnBytes)); + } + String writeTempDataset = configRow.getString("write_temp_dataset"); + if (writeTempDataset != null) { + builder = builder.setWriteTempDataset(writeTempDataset); + } + byte[] rowMutationInformationFnBytes = configRow.getBytes("row_mutation_information_fn"); + if (rowMutationInformationFnBytes != null) { + builder = + builder.setRowMutationInformationFn( + (SerializableFunction) fromByteArray(rowMutationInformationFnBytes)); + } + + return builder.build(); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } } } diff --git a/sdks/java/io/kafka/upgrade/build.gradle b/sdks/java/io/kafka/upgrade/build.gradle index 78776e6e8264..ca2823740dc6 100644 --- a/sdks/java/io/kafka/upgrade/build.gradle +++ b/sdks/java/io/kafka/upgrade/build.gradle @@ -16,8 +16,6 @@ * limitations under the License. */ -import java.util.stream.Collectors - plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.kafka.upgrade', diff --git a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java index 16580cd219b6..eedd2282b1f7 100644 --- a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java +++ b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java @@ -22,6 +22,7 @@ import com.google.auto.service.AutoService; import java.io.IOException; +import java.io.InvalidClassException; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -48,6 +49,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.NanosInstant; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; 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.ImmutableMap; @@ -203,162 +205,183 @@ public Row toConfigRow(Read transform) { if (transform.getCheckStopReadingFn() != null) { fieldValues.put("check_stop_reading_fn", toByteArray(transform.getCheckStopReadingFn())); } + if (transform.getBadRecordErrorHandler() != null) { + throw new RuntimeException( + "Upgrading KafkaIO read transforms that have `withBadRecordErrorHandler` property set" + + "is not supported yet."); + } return Row.withSchema(schema).withFieldValues(fieldValues).build(); } @Override public Read fromConfigRow(Row configRow) { - Read transform = KafkaIO.read(); - - Map consumerConfig = configRow.getMap("consumer_config"); - if (consumerConfig != null) { - Map updatedConsumerConfig = new HashMap<>(); - consumerConfig.forEach( - (key, dataBytes) -> { - // Adding all allowed properties. - if (!KafkaIOUtils.DISALLOWED_CONSUMER_PROPERTIES.containsKey(key)) { - if (consumerConfig.get(key) == null) { - throw new IllegalArgumentException( - "Encoded value of the consumer config property " + key + " was null"); + try { + Read transform = KafkaIO.read(); + + Map consumerConfig = configRow.getMap("consumer_config"); + if (consumerConfig != null) { + Map updatedConsumerConfig = new HashMap<>(); + consumerConfig.forEach( + (key, dataBytes) -> { + // Adding all allowed properties. + if (!KafkaIOUtils.DISALLOWED_CONSUMER_PROPERTIES.containsKey(key)) { + if (consumerConfig.get(key) == null) { + throw new IllegalArgumentException( + "Encoded value of the consumer config property " + key + " was null"); + } + try { + updatedConsumerConfig.put(key, fromByteArray(consumerConfig.get(key))); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } } - updatedConsumerConfig.put(key, fromByteArray(consumerConfig.get(key))); - } - }); - transform = transform.withConsumerConfigUpdates(updatedConsumerConfig); - } - Collection topics = configRow.getArray("topics"); - if (topics != null) { - transform = transform.withTopics(new ArrayList<>(topics)); - } - Collection topicPartitionRows = configRow.getArray("topic_partitions"); - if (topicPartitionRows != null) { - Collection topicPartitions = - topicPartitionRows.stream() - .map( - row -> { - String topic = row.getString("topic"); - if (topic == null) { - throw new IllegalArgumentException("Expected the topic to be not null"); - } - Integer partition = row.getInt32("partition"); - if (partition == null) { - throw new IllegalArgumentException("Expected the partition to be not null"); - } - return new TopicPartition(topic, partition); - }) - .collect(Collectors.toList()); - transform = transform.withTopicPartitions(Lists.newArrayList(topicPartitions)); - } - String topicPattern = configRow.getString("topic_pattern"); - if (topicPattern != null) { - transform = transform.withTopicPattern(topicPattern); - } + }); + transform = transform.withConsumerConfigUpdates(updatedConsumerConfig); + } + Collection topics = configRow.getArray("topics"); + if (topics != null) { + transform = transform.withTopics(new ArrayList<>(topics)); + } + Collection topicPartitionRows = configRow.getArray("topic_partitions"); + if (topicPartitionRows != null) { + Collection topicPartitions = + topicPartitionRows.stream() + .map( + row -> { + String topic = row.getString("topic"); + if (topic == null) { + throw new IllegalArgumentException("Expected the topic to be not null"); + } + Integer partition = row.getInt32("partition"); + if (partition == null) { + throw new IllegalArgumentException( + "Expected the partition to be not null"); + } + return new TopicPartition(topic, partition); + }) + .collect(Collectors.toList()); + transform = transform.withTopicPartitions(Lists.newArrayList(topicPartitions)); + } + String topicPattern = configRow.getString("topic_pattern"); + if (topicPattern != null) { + transform = transform.withTopicPattern(topicPattern); + } - byte[] keyDeserializerProvider = configRow.getBytes("key_deserializer_provider"); - if (keyDeserializerProvider != null) { + byte[] keyDeserializerProvider = configRow.getBytes("key_deserializer_provider"); + if (keyDeserializerProvider != null) { + + byte[] keyCoder = configRow.getBytes("key_coder"); + if (keyCoder != null) { + transform = + transform.withKeyDeserializerProviderAndCoder( + (DeserializerProvider) fromByteArray(keyDeserializerProvider), + (org.apache.beam.sdk.coders.Coder) fromByteArray(keyCoder)); + } else { + transform = + transform.withKeyDeserializer( + (DeserializerProvider) fromByteArray(keyDeserializerProvider)); + } + } - byte[] keyCoder = configRow.getBytes("key_coder"); - if (keyCoder != null) { - transform = - transform.withKeyDeserializerProviderAndCoder( - (DeserializerProvider) fromByteArray(keyDeserializerProvider), - (org.apache.beam.sdk.coders.Coder) fromByteArray(keyCoder)); - } else { - transform = - transform.withKeyDeserializer( - (DeserializerProvider) fromByteArray(keyDeserializerProvider)); + byte[] valueDeserializerProvider = configRow.getBytes("value_deserializer_provider"); + if (valueDeserializerProvider != null) { + byte[] valueCoder = configRow.getBytes("value_coder"); + if (valueCoder != null) { + transform = + transform.withValueDeserializerProviderAndCoder( + (DeserializerProvider) fromByteArray(valueDeserializerProvider), + (org.apache.beam.sdk.coders.Coder) fromByteArray(valueCoder)); + } else { + transform = + transform.withValueDeserializer( + (DeserializerProvider) fromByteArray(valueDeserializerProvider)); + } } - } - byte[] valueDeserializerProvider = configRow.getBytes("value_deserializer_provider"); - if (valueDeserializerProvider != null) { - byte[] valueCoder = configRow.getBytes("value_coder"); - if (valueCoder != null) { + byte[] consumerFactoryFn = configRow.getBytes("consumer_factory_fn"); + if (consumerFactoryFn != null) { + transform = + transform.withConsumerFactoryFn( + (SerializableFunction, Consumer>) + fromByteArray(consumerFactoryFn)); + } + byte[] watermarkFn = configRow.getBytes("watermark_fn"); + if (watermarkFn != null) { + transform = transform.withWatermarkFn2((SerializableFunction) fromByteArray(watermarkFn)); + } + Long maxNumRecords = configRow.getInt64("max_num_records"); + if (maxNumRecords != null) { + transform = transform.withMaxNumRecords(maxNumRecords); + } + Duration maxReadTime = configRow.getValue("max_read_time"); + if (maxReadTime != null) { transform = - transform.withValueDeserializerProviderAndCoder( - (DeserializerProvider) fromByteArray(valueDeserializerProvider), - (org.apache.beam.sdk.coders.Coder) fromByteArray(valueCoder)); - } else { + transform.withMaxReadTime(org.joda.time.Duration.millis(maxReadTime.toMillis())); + } + Instant startReadTime = configRow.getValue("start_read_time"); + if (startReadTime != null) { + transform = transform.withStartReadTime(startReadTime); + } + Instant stopReadTime = configRow.getValue("stop_read_time"); + if (stopReadTime != null) { + transform = transform.withStopReadTime(stopReadTime); + } + Boolean isCommitOffsetFinalizeEnabled = + configRow.getBoolean("is_commit_offset_finalize_enabled"); + if (isCommitOffsetFinalizeEnabled != null && isCommitOffsetFinalizeEnabled) { + transform = transform.commitOffsetsInFinalize(); + } + Boolean isDynamicRead = configRow.getBoolean("is_dynamic_read"); + if (isDynamicRead != null && isDynamicRead) { + Duration watchTopicPartitionDuration = + configRow.getValue("watch_topic_partition_duration"); + if (watchTopicPartitionDuration == null) { + throw new IllegalArgumentException( + "Expected watchTopicPartitionDuration to be available when isDynamicRead is set to true"); + } transform = - transform.withValueDeserializer( - (DeserializerProvider) fromByteArray(valueDeserializerProvider)); + transform.withDynamicRead( + org.joda.time.Duration.millis(watchTopicPartitionDuration.toMillis())); } - } - byte[] consumerFactoryFn = configRow.getBytes("consumer_factory_fn"); - if (consumerFactoryFn != null) { - transform = - transform.withConsumerFactoryFn( - (SerializableFunction, Consumer>) - fromByteArray(consumerFactoryFn)); - } - byte[] watermarkFn = configRow.getBytes("watermark_fn"); - if (watermarkFn != null) { - transform = transform.withWatermarkFn2((SerializableFunction) fromByteArray(watermarkFn)); - } - Long maxNumRecords = configRow.getInt64("max_num_records"); - if (maxNumRecords != null) { - transform = transform.withMaxNumRecords(maxNumRecords); - } - Duration maxReadTime = configRow.getValue("max_read_time"); - if (maxReadTime != null) { - transform = - transform.withMaxReadTime(org.joda.time.Duration.millis(maxReadTime.toMillis())); - } - Instant startReadTime = configRow.getValue("start_read_time"); - if (startReadTime != null) { - transform = transform.withStartReadTime(startReadTime); - } - Instant stopReadTime = configRow.getValue("stop_read_time"); - if (stopReadTime != null) { - transform = transform.withStopReadTime(stopReadTime); - } - Boolean isCommitOffsetFinalizeEnabled = - configRow.getBoolean("is_commit_offset_finalize_enabled"); - if (isCommitOffsetFinalizeEnabled != null && isCommitOffsetFinalizeEnabled) { - transform = transform.commitOffsetsInFinalize(); - } - Boolean isDynamicRead = configRow.getBoolean("is_dynamic_read"); - if (isDynamicRead != null && isDynamicRead) { - Duration watchTopicPartitionDuration = configRow.getValue("watch_topic_partition_duration"); - if (watchTopicPartitionDuration == null) { - throw new IllegalArgumentException( - "Expected watchTopicPartitionDuration to be available when isDynamicRead is set to true"); + byte[] timestampPolicyFactory = configRow.getBytes("timestamp_policy_factory"); + if (timestampPolicyFactory != null) { + transform = + transform.withTimestampPolicyFactory( + (TimestampPolicyFactory) fromByteArray(timestampPolicyFactory)); + } + Map offsetConsumerConfig = configRow.getMap("offset_consumer_config"); + if (offsetConsumerConfig != null) { + Map updatedOffsetConsumerConfig = new HashMap<>(); + offsetConsumerConfig.forEach( + (key, dataBytes) -> { + if (offsetConsumerConfig.get(key) == null) { + throw new IllegalArgumentException( + "Encoded value for the offset consumer config key " + key + " was null."); + } + try { + updatedOffsetConsumerConfig.put( + key, fromByteArray(offsetConsumerConfig.get(key))); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } + }); + transform = transform.withOffsetConsumerConfigOverrides(updatedOffsetConsumerConfig); } - transform = - transform.withDynamicRead( - org.joda.time.Duration.millis(watchTopicPartitionDuration.toMillis())); - } - byte[] timestampPolicyFactory = configRow.getBytes("timestamp_policy_factory"); - if (timestampPolicyFactory != null) { - transform = - transform.withTimestampPolicyFactory( - (TimestampPolicyFactory) fromByteArray(timestampPolicyFactory)); - } - Map offsetConsumerConfig = configRow.getMap("offset_consumer_config"); - if (offsetConsumerConfig != null) { - Map updatedOffsetConsumerConfig = new HashMap<>(); - offsetConsumerConfig.forEach( - (key, dataBytes) -> { - if (offsetConsumerConfig.get(key) == null) { - throw new IllegalArgumentException( - "Encoded value for the offset consumer config key " + key + " was null."); - } - updatedOffsetConsumerConfig.put(key, fromByteArray(offsetConsumerConfig.get(key))); - }); - transform = transform.withOffsetConsumerConfigOverrides(updatedOffsetConsumerConfig); - } + byte[] checkStopReadinfFn = configRow.getBytes("check_stop_reading_fn"); + if (checkStopReadinfFn != null) { + transform = + transform.withCheckStopReadingFn( + (SerializableFunction) + fromByteArray(checkStopReadinfFn)); + } - byte[] checkStopReadinfFn = configRow.getBytes("check_stop_reading_fn"); - if (checkStopReadinfFn != null) { - transform = - transform.withCheckStopReadingFn( - (SerializableFunction) fromByteArray(checkStopReadinfFn)); + return transform; + } catch (InvalidClassException e) { + throw new RuntimeException(e); } - - return transform; } } @@ -476,68 +499,83 @@ public Row toConfigRow(Write transform) { }); fieldValues.put("producer_config", producerConfigMap); } + if (writeRecordsTransform.getBadRecordErrorHandler() != null + && !(writeRecordsTransform.getBadRecordErrorHandler() + instanceof ErrorHandler.DefaultErrorHandler)) { + throw new RuntimeException( + "Upgrading KafkaIO write transforms that have `withBadRecordErrorHandler` property set" + + "is not supported yet."); + } return Row.withSchema(schema).withFieldValues(fieldValues).build(); } @Override public Write fromConfigRow(Row configRow) { - Write transform = KafkaIO.write(); + try { + Write transform = KafkaIO.write(); - String bootstrapServers = configRow.getString("bootstrap_servers"); - if (bootstrapServers != null) { - transform = transform.withBootstrapServers(bootstrapServers); - } - String topic = configRow.getValue("topic"); - if (topic != null) { - transform = transform.withTopic(topic); - } - byte[] keySerializerBytes = configRow.getBytes("key_serializer"); - if (keySerializerBytes != null) { - transform = transform.withKeySerializer((Class) fromByteArray(keySerializerBytes)); - } - byte[] valueSerializerBytes = configRow.getBytes("value_serializer"); - if (valueSerializerBytes != null) { - transform = transform.withValueSerializer((Class) fromByteArray(valueSerializerBytes)); - } - byte[] producerFactoryFnBytes = configRow.getBytes("producer_factory_fn"); - if (producerFactoryFnBytes != null) { - transform = - transform.withProducerFactoryFn( - (SerializableFunction) fromByteArray(producerFactoryFnBytes)); - } - Boolean isEOS = configRow.getBoolean("eos"); - if (isEOS != null && isEOS) { - Integer numShards = configRow.getInt32("num_shards"); - String sinkGroupId = configRow.getString("sink_group_id"); - if (numShards == null) { - throw new IllegalArgumentException( - "Expected numShards to be provided when EOS is set to true"); + String bootstrapServers = configRow.getString("bootstrap_servers"); + if (bootstrapServers != null) { + transform = transform.withBootstrapServers(bootstrapServers); } - if (sinkGroupId == null) { - throw new IllegalArgumentException( - "Expected sinkGroupId to be provided when EOS is set to true"); + String topic = configRow.getValue("topic"); + if (topic != null) { + transform = transform.withTopic(topic); + } + byte[] keySerializerBytes = configRow.getBytes("key_serializer"); + if (keySerializerBytes != null) { + transform = transform.withKeySerializer((Class) fromByteArray(keySerializerBytes)); + } + byte[] valueSerializerBytes = configRow.getBytes("value_serializer"); + if (valueSerializerBytes != null) { + transform = transform.withValueSerializer((Class) fromByteArray(valueSerializerBytes)); + } + byte[] producerFactoryFnBytes = configRow.getBytes("producer_factory_fn"); + if (producerFactoryFnBytes != null) { + transform = + transform.withProducerFactoryFn( + (SerializableFunction) fromByteArray(producerFactoryFnBytes)); + } + Boolean isEOS = configRow.getBoolean("eos"); + if (isEOS != null && isEOS) { + Integer numShards = configRow.getInt32("num_shards"); + String sinkGroupId = configRow.getString("sink_group_id"); + if (numShards == null) { + throw new IllegalArgumentException( + "Expected numShards to be provided when EOS is set to true"); + } + if (sinkGroupId == null) { + throw new IllegalArgumentException( + "Expected sinkGroupId to be provided when EOS is set to true"); + } + transform = transform.withEOS(numShards, sinkGroupId); + } + byte[] consumerFactoryFnBytes = configRow.getBytes("consumer_factory_fn"); + if (consumerFactoryFnBytes != null) { + transform = + transform.withConsumerFactoryFn( + (SerializableFunction) fromByteArray(consumerFactoryFnBytes)); } - transform = transform.withEOS(numShards, sinkGroupId); - } - byte[] consumerFactoryFnBytes = configRow.getBytes("consumer_factory_fn"); - if (consumerFactoryFnBytes != null) { - transform = - transform.withConsumerFactoryFn( - (SerializableFunction) fromByteArray(consumerFactoryFnBytes)); - } - Map producerConfig = configRow.getMap("producer_config"); - if (producerConfig != null && !producerConfig.isEmpty()) { - Map updatedProducerConfig = new HashMap<>(); - producerConfig.forEach( - (key, dataBytes) -> { - updatedProducerConfig.put(key, fromByteArray((byte[]) dataBytes)); - }); - transform = transform.withProducerConfigUpdates(updatedProducerConfig); - } + Map producerConfig = configRow.getMap("producer_config"); + if (producerConfig != null && !producerConfig.isEmpty()) { + Map updatedProducerConfig = new HashMap<>(); + producerConfig.forEach( + (key, dataBytes) -> { + try { + updatedProducerConfig.put(key, fromByteArray((byte[]) dataBytes)); + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } + }); + transform = transform.withProducerConfigUpdates(updatedProducerConfig); + } - return transform; + return transform; + } catch (InvalidClassException e) { + throw new RuntimeException(e); + } } } diff --git a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java index d99bee0ad209..be54d7830d5e 100644 --- a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java +++ b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOReadWithMetadataTranslator; import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOWriteTranslator; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; import org.junit.Test; @@ -123,6 +124,9 @@ public void testReCreateReadTransformFromRow() throws Exception { @Test public void testReadTransformRowIncludesAllFields() throws Exception { + // TODO: support 'withBadRecordErrorHandler' property. + List fieldsToIgnore = + ImmutableList.of("getBadRecordRouter", "getBadRecordErrorHandler"); List getMethodNames = Arrays.stream(Read.class.getDeclaredMethods()) .map( @@ -130,6 +134,7 @@ public void testReadTransformRowIncludesAllFields() throws Exception { return method.getName(); }) .filter(methodName -> methodName.startsWith("get")) + .filter(methodName -> !fieldsToIgnore.contains(methodName)) .collect(Collectors.toList()); // Just to make sure that this does not pass trivially. @@ -189,6 +194,11 @@ public void testReCreateWriteTransformFromRow() throws Exception { @Test public void testWriteTransformRowIncludesAllFields() throws Exception { + // For these fields, default value will suffice (so no need to serialize when upgrading). + // TODO: support 'withBadRecordErrorHandler' property. + List fieldsToIgnore = + ImmutableList.of("getBadRecordRouter", "getBadRecordErrorHandler"); + // Write transform delegates property handling to the WriteRecords class. So we inspect the // WriteRecords class here. List getMethodNames = @@ -198,6 +208,7 @@ public void testWriteTransformRowIncludesAllFields() throws Exception { return method.getName(); }) .filter(methodName -> methodName.startsWith("get")) + .filter(methodName -> !fieldsToIgnore.contains(methodName)) .collect(Collectors.toList()); // Just to make sure that this does not pass trivially. From 75746c6fe4a5489b6ec584569a7dc8eba14f537b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 09:24:41 -0500 Subject: [PATCH 125/224] Bump cloud.google.com/go/storage from 1.35.1 to 1.36.0 in /sdks (#29767) Bumps [cloud.google.com/go/storage](https://github.com/googleapis/google-cloud-go) from 1.35.1 to 1.36.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/storage/v1.35.1...spanner/v1.36.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/storage 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 b186a42930e3..075df0812cac 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -29,7 +29,7 @@ require ( cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.53.1 - cloud.google.com/go/storage v1.35.1 + cloud.google.com/go/storage v1.36.0 github.com/aws/aws-sdk-go-v2 v1.24.0 github.com/aws/aws-sdk-go-v2/config v1.26.1 github.com/aws/aws-sdk-go-v2/credentials v1.16.12 diff --git a/sdks/go.sum b/sdks/go.sum index 763f27080752..284879e23c6f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -43,8 +43,8 @@ cloud.google.com/go/spanner v1.53.1/go.mod h1:liG4iCeLqm5L3fFLU5whFITqP0e0orsAW1 cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.35.1 h1:B59ahL//eDfx2IIKFBeT5Atm9wnNmj3+8xG/W4WB//w= -cloud.google.com/go/storage v1.35.1/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= +cloud.google.com/go/storage v1.36.0 h1:P0mOkAcaJxhCTvAkMhxMfrTKiNcub4YmmPBtlhAyTr8= +cloud.google.com/go/storage v1.36.0/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= From 00462b13cc1eb36302013dea18b701e9e70b9058 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 10:09:51 -0500 Subject: [PATCH 126/224] Bump google.golang.org/grpc from 1.59.0 to 1.60.0 in /sdks (#29714) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.59.0 to 1.60.0. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.59.0...v1.60.0) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 9 +++++---- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 075df0812cac..87c61f675973 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( golang.org/x/text v0.14.0 google.golang.org/api v0.154.0 google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f - google.golang.org/grpc v1.59.0 + google.golang.org/grpc v1.60.0 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 @@ -184,7 +184,7 @@ require ( golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect - google.golang.org/appengine v1.6.7 // indirect + google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 284879e23c6f..fe76820fc059 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -241,6 +241,7 @@ github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvq github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -709,8 +710,8 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= -google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -744,8 +745,8 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= -google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= +google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 1ce707c38650a7f9c5120061adfeed37889cc634 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 14 Dec 2023 10:13:29 -0500 Subject: [PATCH 127/224] Add user-agent string for Beam in the GCS request header. (#29765) --- sdks/python/apache_beam/io/gcp/gcsio.py | 7 ++++++- .../apache_beam/runners/dataflow/internal/apiclient.py | 6 +++++- sdks/python/apache_beam/runners/interactive/utils.py | 8 +++++++- .../runners/portability/sdk_container_builder.py | 6 +++++- sdks/python/setup.py | 2 +- 5 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index 838c956b6cf6..087d32a9e054 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -40,6 +40,7 @@ from google.cloud.storage.fileio import BlobReader from google.cloud.storage.fileio import BlobWriter +from apache_beam import version as beam_version from apache_beam.internal.gcp import auth from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions @@ -111,7 +112,11 @@ def __init__(self, storage_client=None, pipeline_options=None): if credentials: storage_client = storage.Client( credentials=credentials.get_google_auth_credentials(), - project=pipeline_options.view_as(GoogleCloudOptions).project) + project=pipeline_options.view_as(GoogleCloudOptions).project, + extra_headers={ + "User-Agent": "apache-beam/%s (GPN:Beam)" % + beam_version.__version__ + }) else: storage_client = storage.Client.create_anonymous_client() self.client = storage_client diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index c313268cafc5..2ef50b3c2edb 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -514,7 +514,11 @@ def __init__(self, options, root_staging_location=None): # client in terms of which GCP project to use. self._storage_client = storage.Client( credentials=storage_credentials, - project=self.google_cloud_options.project) + project=self.google_cloud_options.project, + extra_headers={ + "User-Agent": "apache-beam/%s (GPN:Beam)" % + beam_version.__version__ + }) else: self._storage_client = storage.Client.create_anonymous_client() self._sdk_image_overrides = self._get_sdk_image_overrides(options) diff --git a/sdks/python/apache_beam/runners/interactive/utils.py b/sdks/python/apache_beam/runners/interactive/utils.py index 78fc26b95e69..524e532c2e3e 100644 --- a/sdks/python/apache_beam/runners/interactive/utils.py +++ b/sdks/python/apache_beam/runners/interactive/utils.py @@ -30,6 +30,7 @@ import pandas as pd import apache_beam as beam +from apache_beam import version as beam_version from apache_beam.dataframe.convert import to_pcollection from apache_beam.dataframe.frame_base import DeferredBase from apache_beam.internal.gcp import auth @@ -457,7 +458,12 @@ def assert_bucket_exists(bucket_name): # We set project to None, so it will not try to use project id from # the environment (ADC). storage_client = storage.Client( - credentials=credentials.get_google_auth_credentials(), project=None) + credentials=credentials.get_google_auth_credentials(), + project=None, + extra_headers={ + "User-Agent": "apache-beam/%s (GPN:Beam)" % + beam_version.__version__ + }) else: storage_client = storage.Client.create_anonymous_client() storage_client.get_bucket(bucket_name) 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 ea86eafcc34a..d5d1bca981d4 100644 --- a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py +++ b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py @@ -213,7 +213,11 @@ def __init__(self, options): if credentials: self._storage_client = storage.Client( credentials=credentials.get_google_auth_credentials(), - project=self._google_cloud_options.project) + project=self._google_cloud_options.project, + extra_headers={ + "User-Agent": "apache-beam/%s (GPN:Beam)" % + beam_version.__version__ + }) else: self._storage_client = storage.Client.create_anonymous_client() self._cloudbuild_client = cloudbuild.CloudbuildV1( diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 7e6d2217d757..d3fca2147239 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -373,7 +373,7 @@ 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', + 'google-cloud-storage>=2.14.0,<3', # GCP packages required by tests 'google-cloud-bigquery>=2.0.0,<4', 'google-cloud-bigquery-storage>=2.6.3,<3', From bc42a637d566b1dbc042632114afb539555ff353 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 14 Dec 2023 10:45:17 -0800 Subject: [PATCH 128/224] [#28543][prism] Implement State API (#29712) --- .../runners/prism/internal/engine/data.go | 176 ++++++++++++++ .../prism/internal/engine/elementmanager.go | 226 +++++++++++++++++- .../prism/internal/engine/engine_test.go | 159 ++++++++++++ .../beam/runners/prism/internal/execute.go | 22 ++ .../runners/prism/internal/execute_test.go | 8 +- .../runners/prism/internal/handlepardo.go | 9 +- .../runners/prism/internal/jobservices/job.go | 3 +- .../prism/internal/jobservices/management.go | 20 +- .../beam/runners/prism/internal/preprocess.go | 14 +- .../pkg/beam/runners/prism/internal/stage.go | 22 +- .../prism/internal/unimplemented_test.go | 48 ++-- .../beam/runners/prism/internal/urns/urns.go | 5 + .../runners/prism/internal/worker/bundle.go | 6 +- .../runners/prism/internal/worker/worker.go | 89 +++++-- sdks/go/test/integration/primitives/state.go | 53 ++-- 15 files changed, 769 insertions(+), 91 deletions(-) create mode 100644 sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/data.go b/sdks/go/pkg/beam/runners/prism/internal/engine/data.go index 6fc192ac83be..6679f484aa2d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/data.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/data.go @@ -15,10 +15,30 @@ package engine +import ( + "bytes" + "fmt" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/exec" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" + "golang.org/x/exp/slog" +) + +// StateData is a "union" between Bag state and MultiMap state to increase common code. +type StateData struct { + Bag [][]byte + Multimap map[string][][]byte +} + // TentativeData is where data for in progress bundles is put // until the bundle executes successfully. type TentativeData struct { Raw map[string][][]byte + + // state is a map from transformID + UserStateID, to window, to userKey, to datavalues. + state map[LinkID]map[typex.Window]map[string]StateData } // WriteData adds data to a given global collectionID. @@ -28,3 +48,159 @@ func (d *TentativeData) WriteData(colID string, data []byte) { } d.Raw[colID] = append(d.Raw[colID], data) } + +func (d *TentativeData) toWindow(wKey []byte) typex.Window { + if len(wKey) == 0 { + return window.GlobalWindow{} + } + // TODO: Custom Window handling. + w, err := exec.MakeWindowDecoder(coder.NewIntervalWindow()).DecodeSingle(bytes.NewBuffer(wKey)) + if err != nil { + panic(fmt.Sprintf("error decoding append bag user state window key %v: %v", wKey, err)) + } + return w +} + +// GetBagState retrieves available state from the tentative bundle data. +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) GetBagState(stateID LinkID, wKey, uKey []byte) [][]byte { + winMap := d.state[stateID] + w := d.toWindow(wKey) + data := winMap[w][string(uKey)] + slog.Debug("State() Bag.Get", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("Window", w), slog.Any("Data", data)) + return data.Bag +} + +func (d *TentativeData) appendState(stateID LinkID, wKey []byte) map[string]StateData { + if d.state == nil { + d.state = map[LinkID]map[typex.Window]map[string]StateData{} + } + winMap, ok := d.state[stateID] + if !ok { + winMap = map[typex.Window]map[string]StateData{} + d.state[stateID] = winMap + } + w := d.toWindow(wKey) + kmap, ok := winMap[w] + if !ok { + kmap = map[string]StateData{} + winMap[w] = kmap + } + return kmap +} + +// AppendBagState appends the incoming data to the existing tentative data bundle. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) AppendBagState(stateID LinkID, wKey, uKey, data []byte) { + kmap := d.appendState(stateID, wKey) + kmap[string(uKey)] = StateData{Bag: append(kmap[string(uKey)].Bag, data)} + slog.Debug("State() Bag.Append", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("Window", wKey), slog.Any("NewData", data)) +} + +func (d *TentativeData) clearState(stateID LinkID, wKey []byte) map[string]StateData { + if d.state == nil { + return nil + } + winMap, ok := d.state[stateID] + if !ok { + return nil + } + w := d.toWindow(wKey) + return winMap[w] +} + +// ClearBagState clears any tentative data for the state. Since state data is only initialized if any exists, +// Clear takes the approach to not create state that doesn't already exist. Existing state is zeroed +// to allow that to be committed post bundle commpletion. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) ClearBagState(stateID LinkID, wKey, uKey []byte) { + kmap := d.clearState(stateID, wKey) + if kmap == nil { + return + } + // Zero the current entry to clear. + // Delete makes it difficult to delete the persisted stage state for the key. + kmap[string(uKey)] = StateData{} + slog.Debug("State() Bag.Clear", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("WindowKey", wKey)) +} + +// GetMultimapState retrieves available state from the tentative bundle data. +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) GetMultimapState(stateID LinkID, wKey, uKey, mapKey []byte) [][]byte { + winMap := d.state[stateID] + w := d.toWindow(wKey) + data := winMap[w][string(uKey)].Multimap[string(mapKey)] + slog.Debug("State() Multimap.Get", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("Window", w), slog.Any("Data", data)) + return data +} + +// AppendMultimapState appends the incoming data to the existing tentative data bundle. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) AppendMultimapState(stateID LinkID, wKey, uKey, mapKey, data []byte) { + kmap := d.appendState(stateID, wKey) + stateData, ok := kmap[string(uKey)] + if !ok || stateData.Multimap == nil { // Incase of All Key Clear tombstones, we may have a nil map. + stateData = StateData{Multimap: map[string][][]byte{}} + kmap[string(uKey)] = stateData + } + stateData.Multimap[string(mapKey)] = append(stateData.Multimap[string(mapKey)], data) + // The Multimap field is aliased to the instance we stored in kmap, + // so we don't need to re-assign back to kmap after appending the data to mapKey. + slog.Debug("State() Multimap.Append", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("MapKey", mapKey), slog.Any("Window", wKey), slog.Any("NewData", data)) +} + +// ClearMultimapState clears any tentative data for the state. Since state data is only initialized if any exists, +// Clear takes the approach to not create state that doesn't already exist. Existing state is zeroed +// to allow that to be committed post bundle commpletion. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) ClearMultimapState(stateID LinkID, wKey, uKey, mapKey []byte) { + kmap := d.clearState(stateID, wKey) + if kmap == nil { + return + } + // Nil the current entry to clear. + // Delete makes it difficult to delete the persisted stage state for the key. + userMap, ok := kmap[string(uKey)] + if !ok || userMap.Multimap == nil { + return + } + userMap.Multimap[string(mapKey)] = nil + // The Multimap field is aliased to the instance we stored in kmap, + // so we don't need to re-assign back to kmap after clearing the data from mapKey. + slog.Debug("State() Multimap.Clear", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("Window", wKey)) +} + +// GetMultimapKeysState retrieves all available user map keys. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) GetMultimapKeysState(stateID LinkID, wKey, uKey []byte) [][]byte { + winMap := d.state[stateID] + w := d.toWindow(wKey) + userMap := winMap[w][string(uKey)] + var keys [][]byte + for k := range userMap.Multimap { + keys = append(keys, []byte(k)) + } + slog.Debug("State() MultimapKeys.Get", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("Window", w), slog.Any("Keys", keys)) + return keys +} + +// ClearMultimapKeysState clears tentative data for all user map keys. Since state data is only initialized if any exists, +// Clear takes the approach to not create state that doesn't already exist. Existing state is zeroed +// to allow that to be committed post bundle commpletion. +// +// The stateID has the Transform and Local fields populated, for the Transform and UserStateID respectively. +func (d *TentativeData) ClearMultimapKeysState(stateID LinkID, wKey, uKey []byte) { + kmap := d.clearState(stateID, wKey) + if kmap == nil { + return + } + // Zero the current entry to clear. + // Delete makes it difficult to delete the persisted stage state for the key. + kmap[string(uKey)] = StateData{} + slog.Debug("State() MultimapKeys.Clear", slog.Any("StateID", stateID), slog.Any("UserKey", uKey), slog.Any("WindowKey", wKey)) +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 656525c67047..6cb552354186 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -23,6 +23,7 @@ import ( "context" "fmt" "io" + "strings" "sync" "sync/atomic" @@ -39,6 +40,7 @@ type element struct { pane typex.PaneInfo elmBytes []byte + keyBytes []byte } type elements struct { @@ -51,6 +53,7 @@ type PColInfo struct { WDec exec.WindowDecoder WEnc exec.WindowEncoder EDec func(io.Reader) []byte + KeyDec func(io.Reader) []byte } // ToData recodes the elements with their approprate windowed value header. @@ -182,6 +185,12 @@ func (em *ElementManager) StageAggregates(ID string) { em.stages[ID].aggregate = true } +// StageStateful marks the given stage as stateful, which means elements are +// processed by key. +func (em *ElementManager) StageStateful(ID string) { + em.stages[ID].stateful = true +} + // Impulse marks and initializes the given stage as an impulse which // is a root transform that starts processing. func (em *ElementManager) Impulse(stageID string) { @@ -257,10 +266,13 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) ss := em.stages[stageID] watermark, ready := ss.bundleReady(em) if ready { - bundleID, ok := ss.startBundle(watermark, nextBundID) + bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) if !ok { continue } + if reschedule { + em.watermarkRefreshes.insert(stageID) + } rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} em.inprogressBundles.insert(rb.BundleID) @@ -278,7 +290,11 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) v := em.livePending.Load() slog.Debug("Bundles: nothing in progress and no refreshes", slog.Int64("pendingElementCount", v)) if v > 0 { - panic(fmt.Sprintf("nothing in progress and no refreshes with non zero pending elements: %v", v)) + var stageState []string + for id, ss := range em.stages { + stageState = append(stageState, fmt.Sprintln(id, ss.pending, ss.pendingByKeys, ss.inprogressKeys, ss.inprogressKeysByBundle)) + } + panic(fmt.Sprintf("nothing in progress and no refreshes with non zero pending elements: %v\n%v", v, strings.Join(stageState, ""))) } } else if len(em.inprogressBundles) == 0 { v := em.livePending.Load() @@ -304,6 +320,56 @@ func (em *ElementManager) InputForBundle(rb RunBundle, info PColInfo) [][]byte { return es.ToData(info) } +// StateForBundle retreives relevant state for the given bundle, WRT the data in the bundle. +// +// TODO(lostluck): Consider unifiying with InputForBundle, to reduce lock contention. +func (em *ElementManager) StateForBundle(rb RunBundle) TentativeData { + ss := em.stages[rb.StageID] + ss.mu.Lock() + defer ss.mu.Unlock() + var ret TentativeData + keys := ss.inprogressKeysByBundle[rb.BundleID] + // TODO(lostluck): Also track windows per bundle, to reduce copying. + if len(ss.state) > 0 { + ret.state = map[LinkID]map[typex.Window]map[string]StateData{} + } + for link, winMap := range ss.state { + for w, keyMap := range winMap { + for key := range keys { + data, ok := keyMap[key] + if !ok { + continue + } + linkMap, ok := ret.state[link] + if !ok { + linkMap = map[typex.Window]map[string]StateData{} + ret.state[link] = linkMap + } + wlinkMap, ok := linkMap[w] + if !ok { + wlinkMap = map[string]StateData{} + linkMap[w] = wlinkMap + } + var mm map[string][][]byte + if len(data.Multimap) > 0 { + mm = map[string][][]byte{} + for uk, v := range data.Multimap { + // Clone the "holding" slice, but refer to the existing data bytes. + mm[uk] = append([][]byte(nil), v...) + } + } + // Clone the "holding" slice, but refer to the existing data bytes. + wlinkMap[key] = StateData{ + Bag: append([][]byte(nil), data.Bag...), + Multimap: mm, + } + } + } + } + + return ret +} + // reElementResiduals extracts the windowed value header from residual bytes, and explodes them // back out to their windows. func reElementResiduals(residuals [][]byte, inputInfo PColInfo, rb RunBundle) []element { @@ -322,6 +388,15 @@ func reElementResiduals(residuals [][]byte, inputInfo PColInfo, rb RunBundle) [] slog.Error("reElementResiduals: sdk provided a windowed value header 0 windows", "bundle", rb) panic("error decoding residual header: sdk provided a windowed value header 0 windows") } + // POSSIBLY BAD PATTERN: The buffer is invalidated on the next call, which doesn't always happen. + // But the decoder won't be mutating the buffer bytes, just reading the data. So the elmBytes + // should remain pointing to the whole element, and we should have a copy of the key bytes. + // Ideally, we're simply refering to the key part of the existing buffer. + elmBytes := buf.Bytes() + var keyBytes []byte + if inputInfo.KeyDec != nil { + keyBytes = inputInfo.KeyDec(buf) + } for _, w := range ws { unprocessedElements = append(unprocessedElements, @@ -329,7 +404,8 @@ func reElementResiduals(residuals [][]byte, inputInfo PColInfo, rb RunBundle) [] window: w, timestamp: et, pane: pn, - elmBytes: buf.Bytes(), + elmBytes: elmBytes, + keyBytes: keyBytes, }) } } @@ -373,6 +449,11 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } // TODO: Optimize unnecessary copies. This is doubleteeing. elmBytes := info.EDec(tee) + var keyBytes []byte + if info.KeyDec != nil { + kbuf := bytes.NewBuffer(elmBytes) + keyBytes = info.KeyDec(kbuf) // TODO: Optimize unnecessary copies. This is tripleteeing? + } for _, w := range ws { newPending = append(newPending, element{ @@ -380,6 +461,7 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol timestamp: et, pane: pn, elmBytes: elmBytes, + keyBytes: keyBytes, }) } } @@ -412,6 +494,10 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol completed := stage.inprogress[rb.BundleID] em.addPending(-len(completed.es)) delete(stage.inprogress, rb.BundleID) + for k := range stage.inprogressKeysByBundle[rb.BundleID] { + delete(stage.inprogressKeys, k) + } + delete(stage.inprogressKeysByBundle, rb.BundleID) // If there are estimated output watermarks, set the estimated // output watermark for the stage. if len(estimatedOWM) > 0 { @@ -421,6 +507,25 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } stage.estimatedOutput = estimate } + + // Handle persisting. + for link, winMap := range d.state { + linkMap, ok := stage.state[link] + if !ok { + linkMap = map[typex.Window]map[string]StateData{} + stage.state[link] = linkMap + } + for w, keyMap := range winMap { + wlinkMap, ok := linkMap[w] + if !ok { + wlinkMap = map[string]StateData{} + linkMap[w] = wlinkMap + } + for key, data := range keyMap { + wlinkMap[key] = data + } + } + } stage.mu.Unlock() // TODO support state/timer watermark holds. @@ -499,6 +604,11 @@ func (em *ElementManager) refreshWatermarks() set[string] { type set[K comparable] map[K]struct{} +func (s set[K]) present(k K) bool { + _, ok := s[k] + return ok +} + func (s set[K]) remove(k K) { delete(s, k) } @@ -525,7 +635,8 @@ type stageState struct { sides []LinkID // PCollection IDs of side inputs that can block execution. // Special handling bits - aggregate bool // whether this state needs to block for aggregation. + stateful bool // whether this stage uses state or timers, and needs keyed processing. + aggregate bool // whether this stage needs to block for aggregation. strat winStrat // Windowing Strategy for aggregation fireings. mu sync.Mutex @@ -537,6 +648,12 @@ type stageState struct { pending elementHeap // pending input elements for this stage that are to be processesd inprogress map[string]elements // inprogress elements by active bundles, keyed by bundle sideInputs map[LinkID]map[typex.Window][][]byte // side input data for this stage, from {tid, inputID} -> window + + // Fields for stateful stages which need to be per key. + pendingByKeys map[string]elementHeap // pending input elements by Key, if stateful. + inprogressKeys set[string] // all keys that are assigned to bundles. + inprogressKeysByBundle map[string]set[string] // bundle to key assignments. + state map[LinkID]map[typex.Window]map[string]StateData // state data for this stage, from {tid, stateID} -> window -> userKey } // makeStageState produces an initialized stageState. @@ -546,6 +663,7 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st outputIDs: outputIDs, sides: sides, strat: defaultStrat{}, + state: map[LinkID]map[typex.Window]map[string]StateData{}, input: mtime.MinTimestamp, output: mtime.MinTimestamp, @@ -566,8 +684,22 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st func (ss *stageState) AddPending(newPending []element) { ss.mu.Lock() defer ss.mu.Unlock() - ss.pending = append(ss.pending, newPending...) - heap.Init(&ss.pending) + if ss.stateful { + if ss.pendingByKeys == nil { + ss.pendingByKeys = map[string]elementHeap{} + } + for _, e := range newPending { + if len(e.keyBytes) == 0 { + panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) + } + h := ss.pendingByKeys[string(e.keyBytes)] + h.Push(e) + ss.pendingByKeys[string(e.keyBytes)] = h // (Is this necessary, with the way the heap interface works over a slice?) + } + } else { + ss.pending = append(ss.pending, newPending...) + heap.Init(&ss.pending) + } } // AddPendingSide adds elements to be consumed as side inputs. @@ -647,10 +779,16 @@ func (ss *stageState) OutputWatermark() mtime.Time { return ss.output } +// TODO: Move to better place for configuration +var ( + OneKeyPerBundle bool // OneKeyPerBundle sets if a bundle is restricted to a single key. + OneElementPerKey bool // OneElementPerKey sets if a key in a bundle is restricted to one element. +) + // startBundle initializes a bundle with elements if possible. // A bundle only starts if there are elements at all, and if it's // an aggregation stage, if the windowing stratgy allows it. -func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) (string, bool) { +func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) (string, bool, bool) { defer func() { if e := recover(); e != nil { panic(fmt.Sprintf("generating bundle for stage %v at %v panicked\n%v", ss.ID, watermark, e)) @@ -669,21 +807,73 @@ func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) } ss.pending = notYet heap.Init(&ss.pending) + if ss.inprogressKeys == nil { + ss.inprogressKeys = set[string]{} + } + minTs := mtime.MaxTimestamp + // TODO: Allow configurable limit of keys per bundle, and elements per key to improve parallelism. + // TODO: when we do, we need to ensure that the stage remains schedualable for bundle execution, for remaining pending elements and keys. + // With the greedy approach, we don't need to since "new data" triggers a refresh, and so should completing processing of a bundle. + newKeys := set[string]{} + stillSchedulable := true + +keysPerBundle: + for k, h := range ss.pendingByKeys { + if ss.inprogressKeys.present(k) { + continue + } + newKeys.insert(k) + // Track the min-timestamp for later watermark handling. + if h[0].timestamp < minTs { + minTs = h[0].timestamp + } + + if OneElementPerKey { + hp := &h + toProcess = append(toProcess, heap.Pop(hp).(element)) + if hp.Len() == 0 { + // Once we've taken all the elements for a key, + // we must delete them from pending as well. + delete(ss.pendingByKeys, k) + } else { + ss.pendingByKeys[k] = *hp + } + } else { + toProcess = append(toProcess, h...) + delete(ss.pendingByKeys, k) + } + if OneKeyPerBundle { + break keysPerBundle + } + } + if len(ss.pendingByKeys) == 0 { + stillSchedulable = false + } if len(toProcess) == 0 { - return "", false + return "", false, false + } + + if toProcess[0].timestamp < minTs { + // Catch the ordinary case. + minTs = toProcess[0].timestamp } - // Is THIS is where basic splits should happen/per element processing? + es := elements{ es: toProcess, - minTimestamp: toProcess[0].timestamp, + minTimestamp: minTs, } if ss.inprogress == nil { ss.inprogress = make(map[string]elements) } + if ss.inprogressKeysByBundle == nil { + ss.inprogressKeysByBundle = make(map[string]set[string]) + } bundID := genBundID() ss.inprogress[bundID] = es - return bundID, true + ss.inprogressKeysByBundle[bundID] = newKeys + ss.inprogressKeys.merge(newKeys) + return bundID, true, stillSchedulable } func (ss *stageState) splitBundle(rb RunBundle, firstResidual int) { @@ -713,6 +903,12 @@ func (ss *stageState) minPendingTimestamp() mtime.Time { if len(ss.pending) != 0 { minPending = ss.pending[0].timestamp } + if len(ss.pendingByKeys) != 0 { + // TODO(lostluck): Can we figure out how to avoid checking every key on every watermark refresh? + for _, h := range ss.pendingByKeys { + minPending = mtime.Min(minPending, h[0].timestamp) + } + } for _, es := range ss.inprogress { minPending = mtime.Min(minPending, es.minTimestamp) } @@ -785,6 +981,14 @@ func (ss *stageState) updateWatermarks(minPending, minStateHold mtime.Time, em * } } } + for _, wins := range ss.state { + for win := range wins { + // Clear out anything we've already used. + if win.MaxTimestamp() < newOut { + delete(wins, win) + } + } + } } return refreshes } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go new file mode 100644 index 000000000000..af41e089a2e9 --- /dev/null +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -0,0 +1,159 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT 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 engine_test ensures coverage of the element manager via pipeline actuation. +package engine_test + +import ( + "context" + "fmt" + "math/rand" + "os" + "strings" + "testing" + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" + "github.com/apache/beam/sdks/v2/go/pkg/beam/options/jobopts" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/engine" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/jobservices" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/universal" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" + "github.com/apache/beam/sdks/v2/go/test/integration/primitives" +) + +func init() { + // Not actually being used, but explicitly registering + // will avoid accidentally using a different runner for + // the tests if I change things later. + beam.RegisterRunner("testlocal", execute) +} + +func TestMain(m *testing.M) { + ptest.MainWithDefault(m, "testlocal") +} + +func initRunner(t testing.TB) { + t.Helper() + if *jobopts.Endpoint == "" { + s := jobservices.NewServer(0, internal.RunPipeline) + *jobopts.Endpoint = s.Endpoint() + go s.Serve() + t.Cleanup(func() { + *jobopts.Endpoint = "" + s.Stop() + }) + } + if !jobopts.IsLoopback() { + *jobopts.EnvironmentType = "loopback" + } + // Since we force loopback, avoid cross-compilation. + f, err := os.CreateTemp("", "dummy") + if err != nil { + t.Fatal(err) + } + t.Cleanup(func() { os.Remove(f.Name()) }) + *jobopts.WorkerBinary = f.Name() +} + +func execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) { + return universal.Execute(ctx, p) +} + +func executeWithT(ctx context.Context, t testing.TB, p *beam.Pipeline) (beam.PipelineResult, error) { + t.Helper() + t.Log("startingTest - ", t.Name()) + s1 := rand.NewSource(time.Now().UnixNano()) + r1 := rand.New(s1) + *jobopts.JobName = fmt.Sprintf("%v-%v", strings.ToLower(t.Name()), r1.Intn(1000)) + return execute(ctx, p) +} + +func initTestName(fn any) string { + name := reflectx.FunctionName(fn) + n := strings.LastIndex(name, "/") + return name[n+1:] +} + +func TestStateAPI(t *testing.T) { + initRunner(t) + + tests := []struct { + pipeline func(s beam.Scope) + }{ + {pipeline: primitives.BagStateParDo}, + {pipeline: primitives.BagStateParDoClear}, + {pipeline: primitives.CombiningStateParDo}, + {pipeline: primitives.ValueStateParDo}, + {pipeline: primitives.ValueStateParDoClear}, + {pipeline: primitives.ValueStateParDoWindowed}, + {pipeline: primitives.MapStateParDo}, + {pipeline: primitives.MapStateParDoClear}, + {pipeline: primitives.SetStateParDo}, + {pipeline: primitives.SetStateParDoClear}, + } + + configs := []struct { + name string + OneElementPerKey, OneKeyPerBundle bool + }{ + {"Greedy", false, false}, + {"AllElementsPerKey", false, true}, + {"OneElementPerKey", true, false}, + {"OneElementPerBundle", true, true}, + } + for _, config := range configs { + for _, test := range tests { + t.Run(initTestName(test.pipeline)+"_"+config.name, func(t *testing.T) { + t.Cleanup(func() { + engine.OneElementPerKey = false + engine.OneKeyPerBundle = false + }) + engine.OneElementPerKey = config.OneElementPerKey + engine.OneKeyPerBundle = config.OneKeyPerBundle + p, s := beam.NewPipelineWithRoot() + test.pipeline(s) + _, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatalf("pipeline failed, but feature should be implemented in Prism: %v", err) + } + }) + } + } +} + +func TestElementManagerCoverage(t *testing.T) { + initRunner(t) + + tests := []struct { + pipeline func(s beam.Scope) + }{ + {pipeline: primitives.Checkpoints}, // (Doesn't run long enough to split.) + {pipeline: primitives.WindowSums_Lifted}, + } + + for _, test := range tests { + t.Run(initTestName(test.pipeline), func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + test.pipeline(s) + _, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatalf("pipeline failed, but feature should be implemented in Prism: %v", err) + } + }) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index 895122383857..b8bc68dcd1b7 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -179,11 +179,16 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic ed := collectionPullDecoder(col.GetCoderId(), coders, comps) wDec, wEnc := getWindowValueCoders(comps, col, coders) + var kd func(io.Reader) []byte + if kcid, ok := extractKVCoderID(col.GetCoderId(), coders); ok { + kd = collectionPullDecoder(kcid, coders, comps) + } stage.OutputsToCoders[onlyOut] = engine.PColInfo{ GlobalID: onlyOut, WDec: wDec, WEnc: wEnc, EDec: ed, + KeyDec: kd, } // There's either 0, 1 or many inputs, but they should be all the same @@ -208,11 +213,17 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic col := comps.GetPcollections()[global] ed := collectionPullDecoder(col.GetCoderId(), coders, comps) wDec, wEnc := getWindowValueCoders(comps, col, coders) + + var kd func(io.Reader) []byte + if kcid, ok := extractKVCoderID(col.GetCoderId(), coders); ok { + kd = collectionPullDecoder(kcid, coders, comps) + } stage.inputInfo = engine.PColInfo{ GlobalID: global, WDec: wDec, WEnc: wEnc, EDec: ed, + KeyDec: kd, } } em.StageAggregates(stage.ID) @@ -234,6 +245,9 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic outputs := maps.Keys(stage.OutputsToCoders) sort.Strings(outputs) em.AddStage(stage.ID, []string{stage.primaryInput}, outputs, stage.sideInputs) + if stage.stateful { + em.StageStateful(stage.ID) + } default: err := fmt.Errorf("unknown environment[%v]", t.GetEnvironmentId()) slog.Error("Execute", err) @@ -286,6 +300,14 @@ func collectionPullDecoder(coldCId string, coders map[string]*pipepb.Coder, comp return pullDecoder(coders[cID], coders) } +func extractKVCoderID(coldCId string, coders map[string]*pipepb.Coder) (string, bool) { + c := coders[coldCId] + if c.GetSpec().GetUrn() == urns.CoderKV { + return c.GetComponentCoderIds()[0], true + } + return "", false +} + func getWindowValueCoders(comps *pipepb.Components, col *pipepb.PCollection, coders map[string]*pipepb.Coder) (exec.WindowDecoder, exec.WindowEncoder) { ws := comps.GetWindowingStrategies()[col.GetWindowingStrategyId()] wcID, err := lpUnknownCoders(ws.GetWindowCoderId(), coders, comps.GetCoders()) diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute_test.go b/sdks/go/pkg/beam/runners/prism/internal/execute_test.go index fe3da83c67e2..29fccaeb238e 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute_test.go @@ -37,6 +37,10 @@ import ( "github.com/apache/beam/sdks/v2/go/test/integration/primitives" ) +func TestMain(m *testing.M) { + ptest.MainWithDefault(m, "testlocal") +} + func initRunner(t testing.TB) { t.Helper() if *jobopts.Endpoint == "" { @@ -585,10 +589,6 @@ func init() { // There's a doubling bug since we re-use the same pcollection IDs for the source & sink, and // don't do any re-writing. -func TestMain(m *testing.M) { - ptest.MainWithDefault(m, "testlocal") -} - func init() { // Basic Registration // beam.RegisterFunction(identity) diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go index 45223c1b2bcb..38e7e9454df5 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go @@ -82,19 +82,26 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb !pdo.RequestsFinalization && !pdo.RequiresStableInput && !pdo.RequiresTimeSortedInput && - len(pdo.StateSpecs) == 0 && len(pdo.TimerFamilySpecs) == 0 && pdo.RestrictionCoderId == "" { // Which inputs are Side inputs don't change the graph further, // so they're not included here. Any nearly any ParDo can have them. // At their simplest, we don't need to do anything special at pre-processing time, and simply pass through as normal. + + // StatefulDoFns need to be marked as being roots. + var forcedRoots []string + if len(pdo.StateSpecs)+len(pdo.TimerFamilySpecs) > 0 { + forcedRoots = append(forcedRoots, tid) + } + return prepareResult{ SubbedComps: &pipepb.Components{ Transforms: map[string]*pipepb.PTransform{ tid: t, }, }, + ForcedRoots: forcedRoots, } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go index cd302a70fcc0..d6e906bee59f 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go @@ -42,7 +42,8 @@ import ( ) var supportedRequirements = map[string]struct{}{ - urns.RequirementSplittableDoFn: {}, + urns.RequirementSplittableDoFn: {}, + urns.RequirementStatefulProcessing: {}, } // TODO, move back to main package, and key off of executor handlers? diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index 0fd7381e17f4..d3727b650860 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -26,6 +26,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns" "golang.org/x/exp/maps" "golang.org/x/exp/slog" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/timestamppb" ) @@ -110,11 +111,10 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jo // Inspect Transforms for unsupported features. bypassedWindowingStrategies := map[string]bool{} ts := job.Pipeline.GetComponents().GetTransforms() - for _, t := range ts { + for tid, t := range ts { urn := t.GetSpec().GetUrn() switch urn { case urns.TransformImpulse, - urns.TransformParDo, urns.TransformGBK, urns.TransformFlatten, urns.TransformCombinePerKey, @@ -140,6 +140,22 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jo wsID := pcs[col].GetWindowingStrategyId() bypassedWindowingStrategies[wsID] = true } + + case urns.TransformParDo: + var pardo pipepb.ParDoPayload + if err := proto.Unmarshal(t.GetSpec().GetPayload(), &pardo); err != nil { + return nil, fmt.Errorf("unable to unmarshal ParDoPayload for %v - %q: %w", tid, t.GetUniqueName(), err) + } + + // Validate all the state features + for _, spec := range pardo.GetStateSpecs() { + check("StateSpec.Protocol.Urn", spec.GetProtocol().GetUrn(), urns.UserStateBag, urns.UserStateMultiMap) + } + // Validate all the timer features + for _, spec := range pardo.GetTimerFamilySpecs() { + check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain()) + } + case "": // Composites can often have no spec if len(t.GetSubtransforms()) > 0 { diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go index 494baa5b4a93..ea4cf2c99695 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go @@ -26,6 +26,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns" "golang.org/x/exp/maps" "golang.org/x/exp/slog" + "google.golang.org/protobuf/proto" ) // transformPreparer is an interface for handling different urns in the preprocessor @@ -440,7 +441,18 @@ func finalizeStage(stg *stage, comps *pipepb.Components, pipelineFacts *fusionFa inputs[pid] = true for _, link := range plinks { t := comps.GetTransforms()[link.Transform] - sis, _ := getSideInputs(t) + + var sis map[string]*pipepb.SideInput + if t.GetSpec().GetUrn() == urns.TransformParDo { + pardo := &pipepb.ParDoPayload{} + if err := (proto.UnmarshalOptions{}).Unmarshal(t.GetSpec().GetPayload(), pardo); err != nil { + return fmt.Errorf("unable to decode ParDoPayload for %v", link.Transform) + } + if len(pardo.GetTimerFamilySpecs())+len(pardo.GetStateSpecs()) > 0 { + stg.stateful = true + } + sis = pardo.GetSideInputs() + } if _, ok := sis[link.Local]; ok { sideInputs = append(sideInputs, engine.LinkID{Transform: link.Transform, Global: link.Global, Local: link.Local}) } else { diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index 1ce102406381..b415f5c241de 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -19,6 +19,7 @@ import ( "bytes" "context" "fmt" + "io" "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" @@ -61,6 +62,7 @@ type stage struct { sideInputs []engine.LinkID // Non-parallel input PCollections and their consumers internalCols []string // PCollections that escape. Used for precise coder sending. envID string + stateful bool exe transformExecuter inputTransformID string @@ -77,6 +79,7 @@ func (s *stage) Execute(ctx context.Context, j *jobservices.Job, wk *worker.W, c var b *worker.B inputData := em.InputForBundle(rb, s.inputInfo) + initialState := em.StateForBundle(rb) var dataReady <-chan struct{} switch s.envID { case "": // Runner Transforms @@ -102,8 +105,8 @@ func (s *stage) Execute(ctx context.Context, j *jobservices.Job, wk *worker.W, c InputTransformID: s.inputTransformID, - // TODO Here's where we can split data for processing in multiple bundles. - InputData: inputData, + InputData: inputData, + OutputData: initialState, SinkToPCollection: s.SinkToPCollection, OutputCount: len(s.outputs), @@ -300,6 +303,12 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng } sinkID := o.Transform + "_" + o.Local ed := collectionPullDecoder(col.GetCoderId(), coders, comps) + + var kd func(io.Reader) []byte + if kcid, ok := extractKVCoderID(col.GetCoderId(), coders); ok { + kd = collectionPullDecoder(kcid, coders, comps) + } + wDec, wEnc := getWindowValueCoders(comps, col, coders) sink2Col[sinkID] = o.Global col2Coders[o.Global] = engine.PColInfo{ @@ -307,6 +316,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng WDec: wDec, WEnc: wEnc, EDec: ed, + KeyDec: kd, } transforms[sinkID] = sinkTransform(sinkID, portFor(wOutCid, wk), o.Global) } @@ -350,14 +360,20 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng if err != nil { return fmt.Errorf("buildDescriptor: failed to handle coder on stage %v for primary input, pcol %q %v:\n%w\n%v", stg.ID, stg.primaryInput, prototext.Format(col), err, stg.transforms) } - ed := collectionPullDecoder(col.GetCoderId(), coders, comps) wDec, wEnc := getWindowValueCoders(comps, col, coders) + + var kd func(io.Reader) []byte + if kcid, ok := extractKVCoderID(col.GetCoderId(), coders); ok { + kd = collectionPullDecoder(kcid, coders, comps) + } + inputInfo := engine.PColInfo{ GlobalID: stg.primaryInput, WDec: wDec, WEnc: wEnc, EDec: ed, + KeyDec: kd, } stg.inputTransformID = stg.ID + "_source" diff --git a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go index b8a04a7306b2..323773bd4cd6 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go @@ -27,7 +27,7 @@ import ( // This file covers pipelines with features that aren't yet supported by Prism. -func intTestName(fn any) string { +func initTestName(fn any) string { name := reflectx.FunctionName(fn) n := strings.LastIndex(name, "/") return name[n+1:] @@ -68,23 +68,11 @@ func TestUnimplemented(t *testing.T) { {pipeline: primitives.TriggerOrFinally}, {pipeline: primitives.TriggerRepeat}, - // State API - {pipeline: primitives.BagStateParDo}, - {pipeline: primitives.BagStateParDoClear}, - {pipeline: primitives.MapStateParDo}, - {pipeline: primitives.MapStateParDoClear}, - {pipeline: primitives.SetStateParDo}, - {pipeline: primitives.SetStateParDoClear}, - {pipeline: primitives.CombiningStateParDo}, - {pipeline: primitives.ValueStateParDo}, - {pipeline: primitives.ValueStateParDoClear}, - {pipeline: primitives.ValueStateParDoWindowed}, - // TODO: Timers integration tests. } for _, test := range tests { - t.Run(intTestName(test.pipeline), func(t *testing.T) { + t.Run(initTestName(test.pipeline), func(t *testing.T) { p, s := beam.NewPipelineWithRoot() test.pipeline(s) _, err := executeWithT(context.Background(), t, p) @@ -113,7 +101,37 @@ func TestImplemented(t *testing.T) { } for _, test := range tests { - t.Run(intTestName(test.pipeline), func(t *testing.T) { + t.Run(initTestName(test.pipeline), func(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + test.pipeline(s) + _, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatalf("pipeline failed, but feature should be implemented in Prism: %v", err) + } + }) + } +} + +func TestStateAPI(t *testing.T) { + initRunner(t) + + tests := []struct { + pipeline func(s beam.Scope) + }{ + {pipeline: primitives.BagStateParDo}, + {pipeline: primitives.BagStateParDoClear}, + {pipeline: primitives.CombiningStateParDo}, + {pipeline: primitives.ValueStateParDo}, + {pipeline: primitives.ValueStateParDoClear}, + {pipeline: primitives.ValueStateParDoWindowed}, + {pipeline: primitives.MapStateParDo}, + {pipeline: primitives.MapStateParDoClear}, + {pipeline: primitives.SetStateParDo}, + {pipeline: primitives.SetStateParDoClear}, + } + + for _, test := range tests { + t.Run(initTestName(test.pipeline), func(t *testing.T) { p, s := beam.NewPipelineWithRoot() test.pipeline(s) _, err := executeWithT(context.Background(), t, p) diff --git a/sdks/go/pkg/beam/runners/prism/internal/urns/urns.go b/sdks/go/pkg/beam/runners/prism/internal/urns/urns.go index bf1e36656661..5312fd799c89 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/urns/urns.go +++ b/sdks/go/pkg/beam/runners/prism/internal/urns/urns.go @@ -51,6 +51,7 @@ var ( reqUrn = toUrn[pipepb.StandardRequirements_Enum]() runProcUrn = toUrn[pipepb.StandardRunnerProtocols_Enum]() envUrn = toUrn[pipepb.StandardEnvironments_Environments]() + usUrn = toUrn[pipepb.StandardUserStateTypes_Enum]() ) var ( @@ -93,6 +94,10 @@ var ( SideInputIterable = siUrn(pipepb.StandardSideInputTypes_ITERABLE) SideInputMultiMap = siUrn(pipepb.StandardSideInputTypes_MULTIMAP) + // UserState kinds + UserStateBag = usUrn(pipepb.StandardUserStateTypes_BAG) + UserStateMultiMap = usUrn(pipepb.StandardUserStateTypes_MULTIMAP) + // WindowsFns WindowFnGlobal = quickUrn(pipepb.GlobalWindowsPayload_PROPERTIES) WindowFnFixed = quickUrn(pipepb.FixedWindowsPayload_PROPERTIES) diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go index 97250092940d..6ef3a81e6239 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go @@ -42,11 +42,13 @@ type B struct { InputTransformID string InputData [][]byte // Data specifically for this bundle. - // IterableSideInputData is a map from transformID, to inputID, to window, to data. + // IterableSideInputData is a map from transformID + inputID, to window, to data. IterableSideInputData map[SideInputKey]map[typex.Window][][]byte - // MultiMapSideInputData is a map from transformID, to inputID, to window, to data key, to data values. + // MultiMapSideInputData is a map from transformID + inputID, to window, to data key, to data values. MultiMapSideInputData map[SideInputKey]map[typex.Window]map[string][][]byte + // State lives in OutputData + // OutputCount is the number of data or timer outputs this bundle has. // We need to see this many closed data channels before the bundle is complete. OutputCount int diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go index beee5e896ffc..2859dfe2356d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go @@ -36,6 +36,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/engine" "github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns" "golang.org/x/exp/slog" "google.golang.org/grpc" @@ -412,21 +413,21 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { panic(err) } } + + // State requests are always for an active ProcessBundle instruction + wk.mu.Lock() + b, ok := wk.activeInstructions[req.GetInstructionId()].(*B) + wk.mu.Unlock() + if !ok { + slog.Warn("state request after bundle inactive", "instruction", req.GetInstructionId(), "worker", wk) + continue + } switch req.GetRequest().(type) { case *fnpb.StateRequest_Get: // TODO: move data handling to be pcollection based. - // State requests are always for an active ProcessBundle instruction - wk.mu.Lock() - b, ok := wk.activeInstructions[req.GetInstructionId()].(*B) - wk.mu.Unlock() - if !ok { - slog.Warn("state request after bundle inactive", "instruction", req.GetInstructionId(), "worker", wk) - continue - } key := req.GetStateKey() slog.Debug("StateRequest_Get", prototext.Format(req), "bundle", b) - var data [][]byte switch key.GetType().(type) { case *fnpb.StateKey_IterableSideInput_: @@ -442,11 +443,13 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { } } winMap := b.IterableSideInputData[SideInputKey{TransformID: ikey.GetTransformId(), Local: ikey.GetSideInputId()}] + var wins []typex.Window for w := range winMap { wins = append(wins, w) } slog.Debug(fmt.Sprintf("side input[%v][%v] I Key: %v Windows: %v", req.GetId(), req.GetInstructionId(), w, wins)) + data = winMap[w] case *fnpb.StateKey_MultimapSideInput_: @@ -458,37 +461,81 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { } else { w, err = exec.MakeWindowDecoder(coder.NewIntervalWindow()).DecodeSingle(bytes.NewBuffer(wKey)) if err != nil { - panic(fmt.Sprintf("error decoding iterable side input window key %v: %v", wKey, err)) + panic(fmt.Sprintf("error decoding multimap side input window key %v: %v", wKey, err)) } } dKey := mmkey.GetKey() winMap := b.MultiMapSideInputData[SideInputKey{TransformID: mmkey.GetTransformId(), Local: mmkey.GetSideInputId()}] - var wins []typex.Window - for w := range winMap { - wins = append(wins, w) - } - slog.Debug(fmt.Sprintf("side input[%v][%v] MM Key: %v Windows: %v", req.GetId(), req.GetInstructionId(), w, wins)) + + slog.Debug(fmt.Sprintf("side input[%v][%v] MultiMap Window: %v", req.GetId(), req.GetInstructionId(), w)) data = winMap[w][string(dKey)] + case *fnpb.StateKey_BagUserState_: + bagkey := key.GetBagUserState() + data = b.OutputData.GetBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey()) + case *fnpb.StateKey_MultimapUserState_: + mmkey := key.GetMultimapUserState() + data = b.OutputData.GetMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey()) + case *fnpb.StateKey_MultimapKeysUserState_: + mmkey := key.GetMultimapKeysUserState() + data = b.OutputData.GetMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey()) default: - panic(fmt.Sprintf("unsupported StateKey Access type: %T: %v", key.GetType(), prototext.Format(key))) + panic(fmt.Sprintf("unsupported StateKey Get type: %T: %v", key.GetType(), prototext.Format(key))) } // Encode the runner iterable (no length, just consecutive elements), and send it out. // This is also where we can handle things like State Backed Iterables. - var buf bytes.Buffer - for _, value := range data { - buf.Write(value) - } responses <- &fnpb.StateResponse{ Id: req.GetId(), Response: &fnpb.StateResponse_Get{ Get: &fnpb.StateGetResponse{ - Data: buf.Bytes(), + Data: bytes.Join(data, []byte{}), }, }, } + + case *fnpb.StateRequest_Append: + key := req.GetStateKey() + switch key.GetType().(type) { + case *fnpb.StateKey_BagUserState_: + bagkey := key.GetBagUserState() + b.OutputData.AppendBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey(), req.GetAppend().GetData()) + case *fnpb.StateKey_MultimapUserState_: + mmkey := key.GetMultimapUserState() + b.OutputData.AppendMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey(), req.GetAppend().GetData()) + default: + panic(fmt.Sprintf("unsupported StateKey Append type: %T: %v", key.GetType(), prototext.Format(key))) + } + responses <- &fnpb.StateResponse{ + Id: req.GetId(), + Response: &fnpb.StateResponse_Append{ + Append: &fnpb.StateAppendResponse{}, + }, + } + + case *fnpb.StateRequest_Clear: + key := req.GetStateKey() + switch key.GetType().(type) { + case *fnpb.StateKey_BagUserState_: + bagkey := key.GetBagUserState() + b.OutputData.ClearBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey()) + case *fnpb.StateKey_MultimapUserState_: + mmkey := key.GetMultimapUserState() + b.OutputData.ClearMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey()) + case *fnpb.StateKey_MultimapKeysUserState_: + mmkey := key.GetMultimapUserState() + b.OutputData.ClearMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey()) + default: + panic(fmt.Sprintf("unsupported StateKey Clear type: %T: %v", key.GetType(), prototext.Format(key))) + } + responses <- &fnpb.StateResponse{ + Id: req.GetId(), + Response: &fnpb.StateResponse_Clear{ + Clear: &fnpb.StateClearResponse{}, + }, + } + default: panic(fmt.Sprintf("unsupported StateRequest kind %T: %v", req.GetRequest(), prototext.Format(req))) } diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index 5f105597ba37..acf1bf8fa665 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -39,6 +39,7 @@ func init() { register.DoFn3x1[state.Provider, string, int, string](&mapStateClearFn{}) register.DoFn3x1[state.Provider, string, int, string](&setStateFn{}) register.DoFn3x1[state.Provider, string, int, string](&setStateClearFn{}) + register.Function2x0(pairWithOne) register.Emitter2[string, int]() register.Combiner1[int](&combine1{}) register.Combiner2[string, int](&combine2{}) @@ -78,12 +79,14 @@ func (f *valueStateFn) ProcessElement(s state.Provider, w string, c int) string return fmt.Sprintf("%s: %v, %s", w, i, j) } +func pairWithOne(w string, emit func(string, int)) { + emit(w, 1) +} + // ValueStateParDo tests a DoFn that uses value state. func ValueStateParDo(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &valueStateFn{}, keyed) passert.Equals(s, counts, "apple: 1, I", "pear: 1, I", "peach: 1, I", "apple: 2, II", "apple: 3, III", "pear: 2, II") } @@ -124,9 +127,7 @@ func (f *valueStateClearFn) ProcessElement(s state.Provider, w string, c int) st // ValueStateParDoClear tests that a DoFn that uses value state can be cleared. func ValueStateParDoClear(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear", "pear", "apple") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &valueStateClearFn{State1: state.MakeValueState[int]("key1")}, keyed) passert.Equals(s, counts, "apple: 0,false", "pear: 0,false", "peach: 0,false", "apple: 1,true", "apple: 0,false", "pear: 1,true", "pear: 0,false", "apple: 1,true") } @@ -170,9 +171,7 @@ func (f *bagStateFn) ProcessElement(s state.Provider, w string, c int) string { // BagStateParDo tests a DoFn that uses bag state. func BagStateParDo(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &bagStateFn{}, keyed) passert.Equals(s, counts, "apple: 0, ", "pear: 0, ", "peach: 0, ", "apple: 1, I", "apple: 2, I,I", "pear: 1, I") } @@ -207,9 +206,7 @@ func (f *bagStateClearFn) ProcessElement(s state.Provider, w string, c int) stri // BagStateParDoClear tests a DoFn that uses bag state. func BagStateParDoClear(s beam.Scope) { in := beam.Create(s, "apple", "pear", "apple", "apple", "pear", "apple", "apple", "pear", "pear", "pear", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &bagStateClearFn{State1: state.MakeBagState[int]("key1")}, keyed) passert.Equals(s, counts, "apple: 0", "pear: 0", "apple: 1", "apple: 2", "pear: 1", "apple: 3", "apple: 0", "pear: 2", "pear: 3", "pear: 0", "apple: 1", "pear: 1") } @@ -312,16 +309,20 @@ func (f *combiningStateFn) ProcessElement(s state.Provider, w string, c int) str return fmt.Sprintf("%s: %v %v %v %v %v", w, i, i1, i2, i3, i4) } +func init() { + register.Function2x1(sumInt) +} + +func sumInt(a, b int) int { + return a + b +} + // CombiningStateParDo tests a DoFn that uses value state. func CombiningStateParDo(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &combiningStateFn{ - State0: state.MakeCombiningState[int, int, int]("key0", func(a, b int) int { - return a + b - }), + State0: state.MakeCombiningState[int, int, int]("key0", sumInt), State1: state.Combining[int, int, int](state.MakeCombiningState[int, int, int]("key1", &combine1{})), State2: state.Combining[string, string, int](state.MakeCombiningState[string, string, int]("key2", &combine2{})), State3: state.Combining[string, string, int](state.MakeCombiningState[string, string, int]("key3", &combine3{})), @@ -369,9 +370,7 @@ func (f *mapStateFn) ProcessElement(s state.Provider, w string, c int) string { // MapStateParDo tests a DoFn that uses value state. func MapStateParDo(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &mapStateFn{State1: state.MakeMapState[string, int]("key1")}, keyed) passert.Equals(s, counts, "apple: 1, keys: [apple apple1]", "pear: 1, keys: [pear pear1]", "peach: 1, keys: [peach peach1]", "apple: 2, keys: [apple apple1 apple2]", "apple: 3, keys: [apple apple1 apple2 apple3]", "pear: 2, keys: [pear pear1 pear2]") } @@ -425,9 +424,7 @@ func (f *mapStateClearFn) ProcessElement(s state.Provider, w string, c int) stri // MapStateParDoClear tests clearing and removing from a DoFn that uses map state. func MapStateParDoClear(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &mapStateClearFn{State1: state.MakeMapState[string, int]("key1")}, keyed) passert.Equals(s, counts, "apple: [apple]", "pear: [pear]", "peach: [peach]", "apple: [apple1 apple2 apple3]", "apple: []", "pear: [pear1 pear2 pear3]") } @@ -465,9 +462,7 @@ func (f *setStateFn) ProcessElement(s state.Provider, w string, c int) string { // SetStateParDo tests a DoFn that uses set state. func SetStateParDo(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &setStateFn{State1: state.MakeSetState[string]("key1")}, keyed) passert.Equals(s, counts, "apple: false, keys: [apple]", "pear: false, keys: [pear]", "peach: false, keys: [peach]", "apple: true, keys: [apple apple1]", "apple: true, keys: [apple apple1]", "pear: true, keys: [pear pear1]") } @@ -521,9 +516,7 @@ func (f *setStateClearFn) ProcessElement(s state.Provider, w string, c int) stri // SetStateParDoClear tests clearing and removing from a DoFn that uses set state. func SetStateParDoClear(s beam.Scope) { in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") - keyed := beam.ParDo(s, func(w string, emit func(string, int)) { - emit(w, 1) - }, in) + keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &setStateClearFn{State1: state.MakeSetState[string]("key1")}, keyed) passert.Equals(s, counts, "apple: [apple]", "pear: [pear]", "peach: [peach]", "apple: [apple1 apple2 apple3]", "apple: []", "pear: [pear1 pear2 pear3]") } From 9e8a310f0c0faddfba28176df5893d8ad8fd10a0 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 14 Dec 2023 15:12:00 -0500 Subject: [PATCH 129/224] Bump Precommit Java Examples Timeouts (#29771) --- .github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index ee372e84d749..88e2a77b18a2 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -74,7 +74,7 @@ permissions: jobs: beam_PreCommit_Java_Examples_Dataflow: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - timeout-minutes: 60 + timeout-minutes: 75 runs-on: [self-hosted, ubuntu-20.04, main] strategy: matrix: diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index c411594c5c35..61a27004eddd 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -77,7 +77,7 @@ jobs: matrix: job_name: ["beam_PreCommit_Java_Examples_Dataflow_Java21"] job_phrase: ["Run Java_Examples_Dataflow_Java21 PreCommit"] - timeout-minutes: 60 + timeout-minutes: 75 if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || From c4c9bb82723deab622d3a49fca2b871f219fbe6e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:05:41 -0500 Subject: [PATCH 130/224] Add 2.54.0 release section to CHANGES.md (#29757) --- CHANGES.md | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index cf7ef0d4db06..b791dd806d4a 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -53,6 +53,40 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> +# [2.54.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). + +## I/Os + +* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Security Fixes +* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + # [2.53.0] - Unreleased ## Highlights From 543b013a548a933afa6886909a4c313e79d04b8d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:07:20 -0500 Subject: [PATCH 131/224] Bump com.gradle.enterprise from 3.15.1 to 3.16 (#29740) Bumps com.gradle.enterprise from 3.15.1 to 3.16. --- updated-dependencies: - dependency-name: com.gradle.enterprise 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> --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index 11833e818920..5a1136c4fa19 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -24,7 +24,7 @@ pluginManagement { } plugins { - id("com.gradle.enterprise") version "3.15.1" + id("com.gradle.enterprise") version "3.16" id("com.gradle.common-custom-user-data-gradle-plugin") version "1.12.1" } From a81c9e8e38b835d923367bc9e2d47c4364103129 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:07:58 -0500 Subject: [PATCH 132/224] Bump actions/setup-python from 4 to 5 (#29654) Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5. - [Release notes](https://github.com/actions/setup-python/releases) - [Commits](https://github.com/actions/setup-python/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/setup-python dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/build_release_candidate.yml | 4 ++-- .github/workflows/build_wheels.yml | 4 ++-- .github/workflows/dask_runner_tests.yml | 4 ++-- .github/workflows/local_env_tests.yml | 4 ++-- .github/workflows/python_dependency_tests.yml | 2 +- .github/workflows/python_tests.yml | 8 ++++---- .github/workflows/run_perf_alert_tool.yml | 2 +- .github/workflows/run_rc_validation.yml | 14 +++++++------- .github/workflows/typescript_tests.yml | 4 ++-- 9 files changed, 23 insertions(+), 23 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index abf583b4dda6..d329591219a2 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -248,7 +248,7 @@ jobs: distribution: 'temurin' java-version: '11' - name: Install Python 3.8 - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: '3.8' - run: echo $JAVA_HOME @@ -298,7 +298,7 @@ jobs: token: ${{ github.event.inputs.BEAM_SITE_TOKEN }} ref: release-docs - name: Install Python 3.8 - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: '3.8' - name: Install node diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 58d14c6fcd68..0ed0d1e776af 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -89,7 +89,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Get tag @@ -248,7 +248,7 @@ jobs: name: source_rc${{ needs.build_source.outputs.rc_num }} path: apache-beam-source-rc - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - uses: docker/setup-qemu-action@v1 diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 35c320086992..ca63b9f76a2a 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -41,7 +41,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Build source @@ -72,7 +72,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - name: Install tox diff --git a/.github/workflows/local_env_tests.yml b/.github/workflows/local_env_tests.yml index 49d37298f1c0..580ca7ed9f2b 100644 --- a/.github/workflows/local_env_tests.yml +++ b/.github/workflows/local_env_tests.yml @@ -49,7 +49,7 @@ jobs: - uses: actions/setup-go@v5 with: go-version: '1.21' - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: '3.8' - name: "Installing local env dependencies" @@ -67,7 +67,7 @@ jobs: - uses: actions/setup-go@v5 with: go-version: '1.21' - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: '3.8' - name: "Installing local env dependencies" diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index 166899df90cb..1c4012029d56 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -35,7 +35,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - name: Install base_image_requirements.txt diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 483d4776aec3..c02d91bd2e19 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -75,7 +75,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Build source @@ -107,7 +107,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - name: Install tox @@ -139,7 +139,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ matrix.python }} - name: Install requirements @@ -164,7 +164,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{ matrix.python }} - name: Install go diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index bc59bd945fe2..1f623571acde 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -37,7 +37,7 @@ jobs: - name: Checkout code uses: actions/checkout@v4 - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Authenticate on GCP diff --git a/.github/workflows/run_rc_validation.yml b/.github/workflows/run_rc_validation.yml index 130a2ce25eea..15979a9e1acd 100644 --- a/.github/workflows/run_rc_validation.yml +++ b/.github/workflows/run_rc_validation.yml @@ -114,7 +114,7 @@ jobs: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} @@ -193,7 +193,7 @@ jobs: java-version: 11 - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} @@ -284,7 +284,7 @@ jobs: with: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: '3.8' @@ -360,7 +360,7 @@ jobs: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} @@ -408,7 +408,7 @@ jobs: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} @@ -460,7 +460,7 @@ jobs: with: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} @@ -509,7 +509,7 @@ jobs: with: ref: ${{env.RC_TAG}} - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: ${{matrix.py_version}} diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index e71834f09ffb..0fdcfb070a22 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -83,7 +83,7 @@ jobs: with: node-version: '16' - name: Install Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Setup Beam Python @@ -138,7 +138,7 @@ jobs: with: node-version: '16' - name: Install python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: 3.8 - name: Setup Beam Python From 970a0bd9a9261b33fc70a03bb6696ca4033c49a1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:09:45 -0500 Subject: [PATCH 133/224] Bump actions/stale from 8 to 9 (#29682) Bumps [actions/stale](https://github.com/actions/stale) from 8 to 9. - [Release notes](https://github.com/actions/stale/releases) - [Changelog](https://github.com/actions/stale/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/stale/compare/v8...v9) --- updated-dependencies: - dependency-name: actions/stale dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 959b2009aa4a..e16aa75b9730 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -28,7 +28,7 @@ jobs: issues: write pull-requests: write steps: - - uses: actions/stale@v8 + - uses: actions/stale@v9 with: repo-token: ${{ secrets.GITHUB_TOKEN }} stale-pr-message: 'This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@beam.apache.org list. Thank you for your contributions.' From d297cd531eeb7f705f91c3a893eeafae2a9537ec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:10:18 -0500 Subject: [PATCH 134/224] Bump org.javacc.javacc from 3.0.0 to 3.0.2 (#29739) Bumps org.javacc.javacc from 3.0.0 to 3.0.2. --- updated-dependencies: - dependency-name: org.javacc.javacc 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> --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index 5a1136c4fa19..8a60d377f41b 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -19,7 +19,7 @@ import com.gradle.enterprise.gradleplugin.internal.extension.BuildScanExtensionW pluginManagement { plugins { - id("org.javacc.javacc") version "3.0.0" // enable the JavaCC parser generator + id("org.javacc.javacc") version "3.0.2" // enable the JavaCC parser generator } } From 1c7d178dd2b7053b1f03463593985aee6474254c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:18:22 -0500 Subject: [PATCH 135/224] Bump github.com/google/uuid from 1.4.0 to 1.5.0 in /sdks (#29733) Bumps [github.com/google/uuid](https://github.com/google/uuid) from 1.4.0 to 1.5.0. - [Release notes](https://github.com/google/uuid/releases) - [Changelog](https://github.com/google/uuid/blob/master/CHANGELOG.md) - [Commits](https://github.com/google/uuid/compare/v1.4.0...v1.5.0) --- updated-dependencies: - dependency-name: github.com/google/uuid 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 87c61f675973..2e3d61bea03a 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -41,7 +41,7 @@ require ( github.com/go-sql-driver/mysql v1.7.1 github.com/golang/protobuf v1.5.3 // TODO(danoliveira): Fully replace this with google.golang.org/protobuf github.com/google/go-cmp v0.6.0 - github.com/google/uuid v1.4.0 + github.com/google/uuid v1.5.0 github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 diff --git a/sdks/go.sum b/sdks/go.sum index fe76820fc059..5ed3a4e198f4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -285,8 +285,8 @@ github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= -github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.5.0 h1:1p67kYwdtXjb0gL0BPiP1Av9wiZPo5A8z2cWkTZ+eyU= +github.com/google/uuid v1.5.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= From 52f4fc0c378650ada532ebfcf16e665646d9593e Mon Sep 17 00:00:00 2001 From: Zechen Jiang Date: Thu, 14 Dec 2023 14:53:00 -0800 Subject: [PATCH 136/224] [Go SDK] Decrease sampling frequency for streaming jobs to avoid oversampling (#29774) --- sdks/go/pkg/beam/core/runtime/exec/datasink.go | 4 ++-- sdks/go/pkg/beam/core/runtime/exec/pcollection.go | 14 ++++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasink.go b/sdks/go/pkg/beam/core/runtime/exec/datasink.go index 6b39a2bb44f1..a58c93083f57 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasink.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasink.go @@ -60,7 +60,7 @@ func (n *DataSink) StartBundle(ctx context.Context, id string, data DataContext) // TODO[BEAM-6374): Properly handle the multiplex and flatten cases. // Right now we just stop datasink collection. if n.PCol != nil { - atomic.StoreInt64(&n.PCol.elementCount, 0) + atomic.StoreInt64(&n.PCol.bundleElementCount, 0) n.PCol.resetSize() } return nil @@ -86,7 +86,7 @@ func (n *DataSink) ProcessElement(ctx context.Context, value *FullValue, values // TODO[BEAM-6374): Properly handle the multiplex and flatten cases. // Right now we just stop datasink collection. if n.PCol != nil { - atomic.AddInt64(&n.PCol.elementCount, 1) + atomic.AddInt64(&n.PCol.bundleElementCount, 1) n.PCol.addSize(int64(byteCount)) } return nil diff --git a/sdks/go/pkg/beam/core/runtime/exec/pcollection.go b/sdks/go/pkg/beam/core/runtime/exec/pcollection.go index ed13c65a6f5b..6df2bd846532 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pcollection.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pcollection.go @@ -46,7 +46,8 @@ type PCollection struct { elementCoder ElementEncoder windowCoder WindowEncoder - elementCount int64 // must use atomic operations. + bundleElementCount int64 // must use atomic operations. + pCollectionElementCount int64 // track the total number of elements this instance has processed. Local use only, no concurrent read/write. sizeMu sync.Mutex sizeCount, sizeSum, sizeMin, sizeMax int64 dataSampler *DataSampler @@ -68,7 +69,7 @@ func (p *PCollection) Up(ctx context.Context) error { // StartBundle resets collected metrics for this PCollection, and propagates bundle start. func (p *PCollection) StartBundle(ctx context.Context, id string, data DataContext) error { - atomic.StoreInt64(&p.elementCount, 0) + atomic.StoreInt64(&p.bundleElementCount, 0) p.nextSampleIdx = 1 p.resetSize() return MultiStartBundle(ctx, id, data, p.Out) @@ -85,8 +86,8 @@ func (w *byteCounter) Write(p []byte) (n int, err error) { // ProcessElement increments the element count and sometimes takes size samples of the elements. func (p *PCollection) ProcessElement(ctx context.Context, elm *FullValue, values ...ReStream) error { - cur := atomic.AddInt64(&p.elementCount, 1) - if cur == p.nextSampleIdx { + cur := atomic.AddInt64(&p.bundleElementCount, 1) + if cur+p.pCollectionElementCount == p.nextSampleIdx { // Always encode the first 3 elements. Otherwise... // We pick the next sampling index based on how large this pcollection already is. // We don't want to necessarily wait until the pcollection has doubled, so we reduce the range. @@ -97,7 +98,7 @@ func (p *PCollection) ProcessElement(ctx context.Context, elm *FullValue, values if p.nextSampleIdx < 4 { p.nextSampleIdx++ } else { - p.nextSampleIdx = cur + p.r.Int63n(cur/10+2) + 1 + p.nextSampleIdx = cur + p.r.Int63n((cur+p.pCollectionElementCount)/10+2) + 1 } if p.dataSampler == nil { @@ -140,6 +141,7 @@ func (p *PCollection) resetSize() { // FinishBundle propagates bundle termination. func (p *PCollection) FinishBundle(ctx context.Context) error { + p.pCollectionElementCount += atomic.LoadInt64(&p.bundleElementCount) return MultiFinishBundle(ctx, p.Out) } @@ -165,7 +167,7 @@ func (p *PCollection) snapshot() PCollectionSnapshot { defer p.sizeMu.Unlock() return PCollectionSnapshot{ ID: p.PColID, - ElementCount: atomic.LoadInt64(&p.elementCount), + ElementCount: atomic.LoadInt64(&p.bundleElementCount), SizeCount: p.sizeCount, SizeSum: p.sizeSum, SizeMin: p.sizeMin, From 3bace64d16a5f2d709badeb425a78eeb8a54080d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 15 Dec 2023 06:47:18 -0500 Subject: [PATCH 137/224] Ignore testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers (#29744) * Ignore testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers * FIx spotless --- .../worker/windmill/client/grpc/StreamingEngineClientTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index 46983a618e4a..166b2cd0f031 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -64,6 +64,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -253,6 +254,7 @@ public void testScheduledBudgetRefresh() throws InterruptedException { } @Test + @Ignore("https://github.com/apache/beam/issues/28957") // stuck test public void testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers() throws InterruptedException { streamingEngineClient = From 5fb4db31994d7c2c1e04d32a4b153bc83d739f36 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 15 Dec 2023 07:06:47 -0500 Subject: [PATCH 138/224] Bump actions/upload-artifact from 3 to 4 (#29777) Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 3 to 4. - [Release notes](https://github.com/actions/upload-artifact/releases) - [Commits](https://github.com/actions/upload-artifact/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/upload-artifact dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 +- .github/workflows/beam_Metrics_Report.yml | 2 +- ...m_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 +- ...m_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 +- ...am_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 +- ...eam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 +- ...m_PerformanceTests_WordCountIT_PythonVersions.yml | 2 +- .github/workflows/beam_PostCommit_Java.yml | 2 +- .../workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 +- .../beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 +- .../workflows/beam_PostCommit_Java_DataflowV1.yml | 2 +- .../workflows/beam_PostCommit_Java_DataflowV2.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 +- .../beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 +- ...eam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- .../beam_PostCommit_Java_Examples_Direct.yml | 2 +- .../beam_PostCommit_Java_Examples_Flink.yml | 2 +- .../beam_PostCommit_Java_Examples_Spark.yml | 2 +- .../beam_PostCommit_Java_Hadoop_Versions.yml | 2 +- .../beam_PostCommit_Java_IO_Performance_Tests.yml | 2 +- .../beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 +- .../beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- .../beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 +- .../beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- .../beam_PostCommit_Java_Jpms_Direct_Java21.yml | 2 +- .../beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 +- .../beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 +- .../beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 +- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 +- .../beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 +- .../beam_PostCommit_Java_PVR_Spark_Batch.yml | 4 ++-- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 +- ...it_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 +- ...ommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 +- ...m_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 +- ...it_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 +- ...mmit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 +- ...Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 +- ..._PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 +- .../beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 +- .github/workflows/beam_PostCommit_Javadoc.yml | 2 +- .../workflows/beam_PostCommit_PortableJar_Flink.yml | 2 +- .../workflows/beam_PostCommit_PortableJar_Spark.yml | 2 +- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PostCommit_Python_Arm.yml | 2 +- .../beam_PostCommit_Python_Examples_Dataflow.yml | 2 +- .../beam_PostCommit_Python_Examples_Direct.yml | 2 +- .../beam_PostCommit_Python_Examples_Flink.yml | 2 +- .../beam_PostCommit_Python_Examples_Spark.yml | 2 +- .../beam_PostCommit_Python_MongoDBIO_IT.yml | 2 +- ...PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 +- ...it_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 +- ...am_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 +- .../beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 +- .../beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 +- .../beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 +- .../beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 +- .github/workflows/beam_PostCommit_SQL.yml | 2 +- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 +- .../beam_PostCommit_TransformService_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 +- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 +- .../beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 +- .../beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 +- ...am_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 +- .../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_ItFramework.yml | 2 +- .github/workflows/beam_PreCommit_Java.yml | 4 ++-- ...PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 4 ++-- ..._PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Azure_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Cassandra_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Debezium_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Examples_Dataflow.yml | 2 +- .../beam_PreCommit_Java_Examples_Dataflow_Java21.yml | 4 ++-- ...reCommit_Java_File-schema-transform_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Flink_Versions.yml | 2 +- .../workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Google-ads_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_HBase_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_HCatalog_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Hadoop_IO_Direct.yml | 4 ++-- .github/workflows/beam_PreCommit_Java_IOs_Direct.yml | 4 ++-- .../beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Kinesis_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_MongoDb_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Neo4j_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_PVR_Flink_Batch.yml | 4 ++-- .../beam_PreCommit_Java_PVR_Flink_Docker.yml | 2 +- .../beam_PreCommit_Java_Parquet_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Redis_IO_Direct.yml | 4 ++-- ...beam_PreCommit_Java_RequestResponse_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_SingleStore_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Snowflake_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Spark3_Versions.yml | 2 +- .../beam_PreCommit_Java_Splunk_IO_Direct.yml | 4 ++-- .../beam_PreCommit_Java_Thrift_IO_Direct.yml | 4 ++-- .../workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 4 ++-- .github/workflows/beam_PreCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_Python_Coverage.yml | 2 +- .../workflows/beam_PreCommit_Python_Dataframes.yml | 2 +- .github/workflows/beam_PreCommit_Python_Examples.yml | 2 +- .../workflows/beam_PreCommit_Python_Integration.yml | 2 +- .../workflows/beam_PreCommit_Python_PVR_Flink.yml | 2 +- .github/workflows/beam_PreCommit_Python_Runners.yml | 2 +- .../workflows/beam_PreCommit_Python_Transforms.yml | 2 +- .github/workflows/beam_PreCommit_SQL.yml | 4 ++-- .github/workflows/beam_PreCommit_SQL_Java11.yml | 4 ++-- .github/workflows/beam_PreCommit_SQL_Java17.yml | 4 ++-- .github/workflows/beam_PreCommit_Spotless.yml | 2 +- .../beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- .github/workflows/build_wheels.yml | 12 ++++++------ .github/workflows/code_completion_plugin_tests.yml | 2 +- .github/workflows/dask_runner_tests.yml | 4 ++-- .github/workflows/java_tests.yml | 10 +++++----- .github/workflows/python_tests.yml | 4 ++-- 142 files changed, 197 insertions(+), 197 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 56ba4083e1dc..881d5dedaac5 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -115,7 +115,7 @@ jobs: -Prunner=:runners:google-cloud-dataflow-java \ '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Streaming_test_arguments_4 }}' \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index b07e64b12cfc..3ed5850e5dba 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -74,7 +74,7 @@ jobs: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} - name: Archive Report - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Metrics Report path: "${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html" diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index 9cbfadbbb6d5..8b9eefcd8052 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -91,7 +91,7 @@ jobs: -DintegrationTestRunner=dataflow \ -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Avro_test_arguments_1 }}]' \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 4bba1eaeba7e..1fb81b9b1431 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -91,7 +91,7 @@ jobs: -DintegrationTestRunner=dataflow \ -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Batch_Java_Json_test_arguments_1 }}]' \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 5b368727ed52..c0433043f1ed 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -91,7 +91,7 @@ jobs: -DintegrationTestRunner=dataflow \ -DintegrationTestPipelineOptions='[${{ env.beam_PerformanceTests_BigQueryIO_Streaming_Java_test_arguments_1 }}]' \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 3e28f7b4d59f..dbe495f12cd1 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -90,7 +90,7 @@ jobs: -DintegrationTestRunner=dataflow \ '-DintegrationTestPipelineOptions=[${{env.beam_PerformanceTests_SQLBigQueryIO_Batch_Java_test_arguments_1}}]' \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index b3e606fcc390..b814e5d99924 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -104,7 +104,7 @@ jobs: -Ptest=apache_beam/examples/wordcount_it_test.py::WordCountIT::test_wordcount_it \ "-Ptest-pipeline-options=${{ env.beam_PerformanceTests_WordCountIT_PythonVersions_test_arguments_1 }}" - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 4417d3a21746..cc7d1c7d64ad 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :javaPostCommit - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index e20218b865ae..313f5c1340a2 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :javaAvroVersionsTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index ce930ba7a438..f832c5fab562 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -86,7 +86,7 @@ jobs: with: gradle-command: :sdks:java:io:google-cloud-platform:bigQueryEarlyRolloutIntegrationTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 78dcb37c8934..6e31094fa468 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -83,7 +83,7 @@ jobs: with: gradle-command: :runners:google-cloud-dataflow-java:postCommit - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 22fbd5d6fd46..58ecc37361d4 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:google-cloud-dataflow-java:postCommitRunnerV2 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 426291fac138..183a55a8e742 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:examples:javaPostCommit max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index a8e24f33adfb..33557dc6d2d1 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -113,7 +113,7 @@ jobs: -PdisableCheckStyle=true \ -PskipCheckerFramework \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index decca84683ec..f9e828106bf5 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -86,7 +86,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:examples:java${{ matrix.java_version }}PostCommit max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 9733b6d36434..661dd2dde2b1 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results 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 060fe4396d85..18fa9513ac63 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -94,7 +94,7 @@ jobs: -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index cec2acab0f95..34d17b4b20a9 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -81,7 +81,7 @@ jobs: with: gradle-command: :runners:direct:examplesIntegrationTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index a9f5d99bc2c6..3131422fe732 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -82,7 +82,7 @@ jobs: with: gradle-command: :runners:flink:1.15:examplesIntegrationTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index da58dc67c9c1..4bc8408f9a4d 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -81,7 +81,7 @@ jobs: with: gradle-command: :runners:spark:3:examplesIntegrationTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index f2489137fd64..a227144e6570 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :javaHadoopVersionsTest - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index a340f028fbfe..588da4ea61a2 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -107,7 +107,7 @@ jobs: exportDataset: performance_tests exportTable: io_performance_metrics_test - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index cb39dd2f26f1..0f39e0d02537 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index f33ba9c39b74..049b5e27e341 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -85,7 +85,7 @@ jobs: -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index f7b0f22cbfa5..73750453e5e3 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 0ae09bcbefc4..2a56f3e96201 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -85,7 +85,7 @@ jobs: -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index 2c918b354977..163169563047 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -86,7 +86,7 @@ jobs: -PtestJavaVersion=21 -Pjava21Home=$JAVA_HOME_21_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index b6a4e1c84174..ea8a89def8ee 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:flinkRunnerIntegrationTest arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 6e093ead474c..a8754ac254d8 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:sparkRunnerIntegrationTest arguments: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 76e2b702ba66..13d3384f8087 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: runners:flink:1.15:job-server:validatesPortableRunnerStreaming - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 604a37779d17..86647b6079d3 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -81,7 +81,7 @@ jobs: with: gradle-command: :runners:samza:job-server:validatesPortableRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 201016c5602e..58ee88e20ec3 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:spark:3:job-server:validatesPortableRunnerStreaming - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index d8a0672e03b5..a6061da45e92 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -82,7 +82,7 @@ jobs: :runners:spark:3:job-server:validatesPortableRunnerBatch \ :runners:spark:3:job-server:validatesPortableRunnerDocker \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -96,7 +96,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: SpotBugs Results path: "**/build/reports/spotbugs/*.html" diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index b2b3440e23b3..2e96ae63382c 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :javaPostCommitSickbay - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index e01ea98912d0..d34847d5340f 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesRunner max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index a0009b76f699..91a55dc14007 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -100,7 +100,7 @@ jobs: -Dorg.gradle.java.home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index cc5f86b0b5a9..e10a4db9abf3 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesRunnerStreaming max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 8b9d456ed828..adb5c9f9c777 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesRunnerV2 max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results 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 6bca7477cf52..2024de57a1c6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:validatesRunnerV2Streaming max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 90f47d5451f4..1d6eb82d9045 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -79,7 +79,7 @@ jobs: - name: run validatesRunner script run: ./gradlew :runners:direct-java:validatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 9f7897ffd84c..da9ab6419ec6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: -x compileTestJava \ -Dorg.gradle.java.home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 1f265ea5dc16..22fd277470c8 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -80,7 +80,7 @@ jobs: with: gradle-command: :runners:flink:1.15:validatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index c7b9219f6ea9..764211e26484 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -98,7 +98,7 @@ jobs: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 \ max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index c4583f2f72d4..76136e8038d2 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:samza:validatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 75a01cd8d5bb..b4e4e25fa4bd 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:spark:3:validatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 844082ef9460..55e3c8a9ea7a 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:spark:3:validatesStructuredStreamingRunnerBatch - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index d88ebbe28a85..510c797fef36 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -98,7 +98,7 @@ jobs: -Dorg.gradle.java.home=$JAVA_HOME_11_X64 \ max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 2206e1767fc2..d3c80473dd2e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :runners:twister2:validatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index a8b3678c6321..35d5ac738d64 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -78,7 +78,7 @@ jobs: - name: run ulrLoopbackValidatesRunner script run: ./gradlew :runners:portability:java:ulrLoopbackValidatesRunner - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 607ba6436647..61af8b12d2a4 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -84,7 +84,7 @@ jobs: path-to-root: sdks/java/javadoc/build/docs/javadoc base-url-path: https://beam.apache.org/releases/javadoc/current/ - name: Upload Javadoc Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Javadoc Results path: '**/sdks/java/javadoc/build/docs/javadoc/**' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 6142f607367d..2c0a2f0c49e9 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -83,7 +83,7 @@ jobs: arguments: | -PpythonVersion=3.8 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index a4aba3cdbd33..d15c846cf501 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -83,7 +83,7 @@ jobs: arguments: | -PpythonVersion=3.8 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 1dd4a4161eb4..12c73d9c852a 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -98,7 +98,7 @@ jobs: env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index cf760512d30d..ac40bee82659 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -114,7 +114,7 @@ jobs: MULTIARCH_TAG: ${{ steps.set_tag.outputs.TAG }} USER: github-actions - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index fdcd406df1aa..fa106bbe8a8f 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -83,7 +83,7 @@ jobs: -PuseWheelDistribution \ -PpythonVersion=3.11 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index d30c73c9c1dc..1d778c4cad47 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -90,7 +90,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index adfdb586ea55..f7bc434a2055 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -90,7 +90,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index b81323cac09c..0ea616635257 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -90,7 +90,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 7e594f39b191..49b337a8d6ae 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -82,7 +82,7 @@ jobs: arguments: | -PpythonVersion=3.11 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index ecdfda1971e4..bb3bae69806c 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -96,7 +96,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results 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 590ecd5ffad9..c13df624e7af 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -95,7 +95,7 @@ jobs: -PtestRCDependencies=true \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index a9e3e33b8e50..55e3a80e93ee 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -98,7 +98,7 @@ jobs: -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index e45ed572b079..adff5bd94535 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -92,7 +92,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index e861a0ae12ae..286d294e3e5a 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -90,7 +90,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index c89e1632049b..1324325ab70f 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -90,7 +90,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 295a9eff9b14..e5a2c1b39134 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -82,7 +82,7 @@ jobs: gradle-command: :sdks:python:test-suites:dataflow:gcpCrossLanguagePostCommit arguments: -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 7ab8c7a56c14..d4e57997da70 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -81,7 +81,7 @@ jobs: with: gradle-command: :sdks:python:test-suites:direct:gcpCrossLanguagePostCommit - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 3df1d24aa455..66402fad0b8d 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -84,7 +84,7 @@ jobs: -PuseWheelDistribution \ -PkafkaBootstrapServer=10.128.0.40:9094,10.128.0.28:9094,10.128.0.165:9094 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index 2b7ce87eeab3..4466633552fb 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -79,7 +79,7 @@ jobs: with: gradle-command: :sqlPostCommit - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index b9b761284b29..7ebdbcb38416 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -93,7 +93,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 2d6312cf2487..0cb4352c2fbb 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -88,7 +88,7 @@ jobs: -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 4b6fe15ce199..de1b995bc649 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -98,7 +98,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 96ce79b25087..134efdf0dfc3 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 9a0ce786cbb0..82d284a829ad 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -87,7 +87,7 @@ jobs: with: gradle-command: :runners:google-cloud-dataflow-java:validatesCrossLanguageRunnerGoUsingJava - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index f7f3c73373a3..02024805beb8 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -86,7 +86,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index cd10e823d63a..4908b34da18e 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -83,7 +83,7 @@ jobs: arguments: | -PpythonVersion=3.11 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index d76955772ac4..a4dd9a9abe1e 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -86,7 +86,7 @@ jobs: arguments: | -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 75fbb8eaffa4..b56872dd778b 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -98,7 +98,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index a685e57bb592..68f9cb93b5f6 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -98,7 +98,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PskipNonPythonTask=false \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index 0d8bb00160d7..a883330140ba 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -90,7 +90,7 @@ jobs: - name: run ItFrameworkPrecommit script run: ./gradlew -p it build - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 21898e5f758b..daee33b303ac 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -186,7 +186,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -199,7 +199,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results 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 5e0bc8631fd9..f8af89a14ef2 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 @@ -118,7 +118,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -131,7 +131,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results 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 97192a26f5fa..a693902c8eb3 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 @@ -118,7 +118,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -131,7 +131,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml index 241cf89380a8..4a56321bfe9d 100644 --- a/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amqp_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 4cdcfed4dadc..94b7c39b950b 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -111,7 +111,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -124,7 +124,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index dd15d18f26d1..daea023283a1 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index ffa8c9187a68..daa275086330 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -97,7 +97,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -110,7 +110,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 209d30231805..4ddac57a165b 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index 5cd015ac9832..e1b381d5633e 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index a2cf6c1b0645..5ca6b8afa68e 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -102,7 +102,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -115,7 +115,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index f7a3ca8df639..a6a17cebdef9 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -106,7 +106,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -119,7 +119,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 88e2a77b18a2..b7f762ca808c 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -113,7 +113,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index 61a27004eddd..bd4f98cd8714 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -121,7 +121,7 @@ jobs: -Pjava21Home=$JAVA_HOME_21_X64 \ max-workers: 12 - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -134,7 +134,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: SpotBugs Results path: '**/build/reports/spotbugs/*.html' \ No newline at end of file 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 cd0c92367902..7f467abc2749 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 @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ -Dfile.encoding=UTF-8 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -107,7 +107,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index 303a84c2df34..b7ae39a1ee88 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -93,7 +93,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index c7a5a84af224..eec0383c02a9 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -115,7 +115,7 @@ jobs: -PdisableCheckStyle=true \ -PenableJacocoReport \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -128,7 +128,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml index 2271c1cf8d37..2ddce986fc8e 100644 --- a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -91,7 +91,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -104,7 +104,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index beb7acc89f6a..52c91a26aa3e 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -95,7 +95,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -108,7 +108,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index c1e627025c0a..97096108ebd1 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -95,7 +95,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -108,7 +108,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index 5b8202d96217..604c72471fa0 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -133,7 +133,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -146,7 +146,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 7763ab690449..831a1bec53fa 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -94,7 +94,7 @@ jobs: -PdisableCheckStyle=true \ -Dfile.encoding=UTF-8 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -107,7 +107,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 1d66fd56539f..80869d74b642 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 60390f31223f..9a11d4e7ef35 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -100,7 +100,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -113,7 +113,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 431b5e0c6566..8f502223e85e 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -100,7 +100,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -113,7 +113,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 028ba721a4f5..ac63a39380b2 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -102,7 +102,7 @@ jobs: -PdisableCheckStyle=true \ --no-parallel \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -115,7 +115,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index e0598d0beebe..f76fec0bdd91 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -125,7 +125,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -138,7 +138,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index 47c360a62efd..8948bedf1384 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 4b9895f9cd93..f0647f93c52b 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index 934a428fa96d..17c4127cb40d 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 48a7da5f82a4..a4e0ac744f41 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -102,7 +102,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -115,7 +115,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml index 966f51f1a4a3..2cadc893fbc1 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Batch.yml @@ -98,13 +98,13 @@ jobs: env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH }} - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results path: "**/build/reports/tests/" - name: Upload test report - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: java-code-coverage-report path: "**/build/test-results/**/*.xml" diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 2997e20357f5..ae62fd93d977 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -103,7 +103,7 @@ jobs: env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 6f167bc89680..e2430ae1bef9 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index f10ae32d36ee..dd3402c79773 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -111,7 +111,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -124,7 +124,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index 4be46123ebaa..f796f842311f 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 6429cc9d7c9e..8e2483357cc6 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index d297059bc71f..a2fabd3debd0 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -91,7 +91,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -104,7 +104,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 5b026bbe9c54..620ab5ccec17 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -95,7 +95,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -108,7 +108,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 4668b3536683..8a3041bccbfb 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -104,7 +104,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -117,7 +117,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index a656ce6b1263..2071f894de8c 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 695800453d00..d22cd8cad7e6 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -102,7 +102,7 @@ jobs: gradle-command: :runners:spark:3:sparkVersionsTest arguments: -PdisableSpotlessCheck=true - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 51408f545677..0106ae61ab8f 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 4183ea0f01a5..442eda5a8bc8 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index 7f402097c5a8..407ea3218288 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -93,7 +93,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -106,7 +106,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 61160bb7b68c..dadf902a8a5a 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 301e1d26d4b3..244f37cc208c 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -89,7 +89,7 @@ jobs: arguments: | -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index c827a120b0c4..af0b21e6a94f 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 685d81d6027e..20a9dd1ff5ba 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 0eef876a37d7..fc33d6c4859d 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -105,7 +105,7 @@ jobs: -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index 2fd8add01da3..64609737cb25 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -114,7 +114,7 @@ jobs: arguments: | -PpythonVersion=3.11 \ - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index 61b916cb7a8d..0e325c19c8d9 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 27b23e0d50c2..7e8c1cbdcfc1 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -99,7 +99,7 @@ jobs: -PpythonVersion=${{ matrix.python_version }} \ -PuseWheelDistribution - name: Archive Python Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: failure() with: name: Python Test Results diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index c9b53244e876..b67addfee1c0 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -91,7 +91,7 @@ jobs: -PdisableCheckStyle=true \ -PenableJacocoReport \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -104,7 +104,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index fea4f759f74f..5c2face871df 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -100,7 +100,7 @@ jobs: -PskipCheckerFramework \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -113,7 +113,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index fbbda1ac7766..de962ff7c16c 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -98,7 +98,7 @@ jobs: -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: JUnit Test Results @@ -111,7 +111,7 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - name: Archive SpotBugs Results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: SpotBugs Results diff --git a/.github/workflows/beam_PreCommit_Spotless.yml b/.github/workflows/beam_PreCommit_Spotless.yml index ab0b28ee0b2b..4ab1ade15847 100644 --- a/.github/workflows/beam_PreCommit_Spotless.yml +++ b/.github/workflows/beam_PreCommit_Spotless.yml @@ -100,7 +100,7 @@ jobs: with: gradle-command: spotlessCheck checkStyleMain checkStyleTest :buildSrc:spotlessCheck - name: Upload test report - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: java-code-coverage-report path: "**/build/reports/checkstyle/*.xml" diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index 6681669f5958..b5d95ad3ab26 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -110,7 +110,7 @@ jobs: MULTIARCH_TAG: ${{ steps.set_tag.outputs.TAG }} USER: github-actions - name: Archive code coverage results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: python-code-coverage-report diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 0ed0d1e776af..346d40561d96 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -125,12 +125,12 @@ jobs: working-directory: ./sdks/python run: mv $(ls | grep apache-beam) apache-beam-source - name: Upload source as artifact - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: source path: sdks/python/apache-beam-source - name: Upload compressed sources as artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: source_zip path: sdks/python/dist @@ -167,13 +167,13 @@ jobs: run: mv $(ls | grep apache-beam) apache-beam-source-rc - name: Upload RC source as artifact if: steps.is_rc.outputs.is_rc == 1 - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: source_rc${{ steps.get_rc_version.outputs.RC_NUM }} path: sdks/python/apache-beam-source-rc - name: Upload compressed RC sources as artifacts if: steps.is_rc.outputs.is_rc == 1 - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: source_zip_rc${{ steps.get_rc_version.outputs.RC_NUM }} path: sdks/python/dist @@ -278,7 +278,7 @@ jobs: done shell: bash - name: Upload wheels as artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheelhouse-${{ matrix.os_python.os }} path: apache-beam-source/wheelhouse/ @@ -303,7 +303,7 @@ jobs: shell: bash - name: Upload RC wheels as artifacts if: ${{ needs.build_source.outputs.is_rc == 1 }} - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheelhouse-rc${{ needs.build_source.outputs.rc_num }}-${{ matrix.os_python.os }} path: apache-beam-source-rc/wheelhouse/ diff --git a/.github/workflows/code_completion_plugin_tests.yml b/.github/workflows/code_completion_plugin_tests.yml index 9dbb5f6452f2..ca03932c9944 100644 --- a/.github/workflows/code_completion_plugin_tests.yml +++ b/.github/workflows/code_completion_plugin_tests.yml @@ -109,7 +109,7 @@ jobs: # Collect Tests Result of failed tests - name: Collect Tests Result if: ${{ failure() }} - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: tests-result path: ${{ github.workspace }}/build/reports/tests diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index ca63b9f76a2a..628cf2c34462 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -51,7 +51,7 @@ jobs: working-directory: ./sdks/python/dist run: mv $(ls | grep "apache-beam.*tar\.gz") apache-beam-source.tar.gz - name: Upload compressed sources as artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: python_sdk_source path: sdks/python/dist/apache-beam-source.tar.gz @@ -89,7 +89,7 @@ jobs: working-directory: ./sdks/python run: tox -c tox.ini -e ${{ matrix.params.tox_env }}-win-dask - name: Upload test logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: pytest-${{matrix.os}}-${{matrix.params.py_ver}} diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index 30808563aaed..dc0f58d04f79 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -89,7 +89,7 @@ jobs: gradle-command: test arguments: -p sdks/java/core/ - name: Upload test logs for :sdks:java:core:test - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: java_unit_tests-sdks-java-core-test-${{ matrix.os }} @@ -102,7 +102,7 @@ jobs: arguments: -p sdks/java/harness/ if: always() - name: Upload test logs for :sdks:java:harness:test - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: java_unit_tests-sdks-java-harness-test-${{ matrix.os }} @@ -115,7 +115,7 @@ jobs: arguments: -p runners/core-java/ if: always() - name: Upload test logs for :runners:core-java:test - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: java_unit_tests-runners-core-java-test-${{ matrix.os }} @@ -147,7 +147,7 @@ jobs: -DintegrationTestRunner=direct -DintegrationTestPipelineOptions=[\"--runner=DirectRunner\",\"--tempRoot=./tmp\"] - name: Upload test logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: java_wordcount_direct_runner-${{matrix.os}} @@ -193,7 +193,7 @@ jobs: -DintegrationTestPipelineOptions=[\"--runner=DataflowRunner\",\"--project=${{ secrets.GCP_PROJECT_ID }}\",\"--tempRoot=gs://${{ secrets.GCP_TESTING_BUCKET }}/tmp/\"] -DintegrationTestRunner=dataflow - name: Upload test logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: java_wordcount_dataflow-${{matrix.os}} diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index c02d91bd2e19..a17833ead3f7 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -85,7 +85,7 @@ jobs: working-directory: ./sdks/python/dist run: mv $(ls | grep "apache-beam.*tar\.gz") apache-beam-source.tar.gz - name: Upload compressed sources as artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: python_sdk_source path: sdks/python/dist/apache-beam-source.tar.gz @@ -121,7 +121,7 @@ jobs: working-directory: ./sdks/python run: tox -c tox.ini run -e ${{ matrix.params.tox_env }}-win - name: Upload test logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: pytest-${{matrix.os}}-${{matrix.params.py_ver}} From 860bc30a110eb697dd78918481b2ba6c496a34b1 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Fri, 15 Dec 2023 12:20:28 +0100 Subject: [PATCH 139/224] Add SparkStructuredStreamingRunner to beam_PostCommit_Java_Tpcds_Spark workflow --- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 580cfc83132c..f39feaf64a1b 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -48,7 +48,6 @@ env: 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 @@ -80,11 +79,12 @@ jobs: 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 }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.runner }} strategy: matrix: job_name: ["beam_PostCommit_Java_Tpcds_Spark"] job_phrase: ["Run Spark Runner Tpcds Tests"] + runner: [SparkRunner, SparkStructuredStreamingRunner] steps: - uses: actions/checkout@v4 - name: Setup repository @@ -92,13 +92,13 @@ jobs: with: comment_phrase: ${{ matrix.job_phrase }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) ${{ matrix.runner }} - name: Setup environment uses: ./.github/actions/setup-environment-action - - name: run PostCommit Java Tpcds Spark script + - name: run PostCommit Java Tpcds Spark (${{ matrix.runner }}) 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}}" \ + "-Ptpcds.args=${{env.tpcdsBigQueryArgs}} ${{env.tpcdsInfluxDBArgs}} ${{ env.GRADLE_COMMAND_ARGUMENTS }} --runner=${{ matrix.runner }} --queries=${{env.tpcdsQueriesArg}}" \ From 6499c654c4aa7e36a9f74de5d402c4164f35823e Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Fri, 15 Dec 2023 11:30:31 -0800 Subject: [PATCH 140/224] Fix typos --- .../sdk/io/gcp/bigquery/BigQueryIOTranslation.java | 10 +++++----- .../sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java index 1056328eb4c5..a3a270a315b2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -353,7 +353,7 @@ static class BigQueryIOWriteTranslator implements TransformPayloadTranslator transform) { fieldValues.put("max_bytes_per_partition", transform.getMaxBytesPerPartition()); if (transform.getTriggeringFrequency() != null) { fieldValues.put( - "triggerring_frequency", + "triggering_frequency", Duration.ofMillis(transform.getTriggeringFrequency().getMillis())); } if (transform.getMethod() != null) { @@ -695,11 +695,11 @@ public Write fromConfigRow(Row configRow) { if (maxBytesPerPartition != null) { builder = builder.setMaxBytesPerPartition(maxBytesPerPartition); } - Duration triggerringFrequency = configRow.getValue("triggerring_frequency"); - if (triggerringFrequency != null) { + Duration triggeringFrequency = configRow.getValue("triggering_frequency"); + if (triggeringFrequency != null) { builder = builder.setTriggeringFrequency( - org.joda.time.Duration.millis(triggerringFrequency.toMillis())); + org.joda.time.Duration.millis(triggeringFrequency.toMillis())); } byte[] methodBytes = configRow.getBytes("method"); if (methodBytes != null) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java index 9de3b02c2531..c46d382bb298 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslationTest.java @@ -100,7 +100,7 @@ public class BigQueryIOTranslationTest { "getPropagateSuccessfulStorageApiWrites", "propagate_successful_storage_api_writes"); WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxFilesPerPartition", "max_files_per_partition"); WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMaxBytesPerPartition", "max_bytes_per_partition"); - WRITE_TRANSFORM_SCHEMA_MAPPING.put("getTriggeringFrequency", "triggerring_frequency"); + WRITE_TRANSFORM_SCHEMA_MAPPING.put("getTriggeringFrequency", "triggering_frequency"); WRITE_TRANSFORM_SCHEMA_MAPPING.put("getMethod", "method"); WRITE_TRANSFORM_SCHEMA_MAPPING.put("getLoadJobProjectId", "load_job_project_id"); WRITE_TRANSFORM_SCHEMA_MAPPING.put("getFailedInsertRetryPolicy", "failed_insert_retry_policy"); From 539f04233c6b181ad59e952befab792768f23d5e Mon Sep 17 00:00:00 2001 From: martin trieu Date: Sat, 16 Dec 2023 07:50:34 -0800 Subject: [PATCH 141/224] add alts to grpc vendored lib (#29763) --- .../groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy | 1 + 1 file changed, 1 insertion(+) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy index ddb317b89294..4b8a059109c9 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_54_0.groovy @@ -59,6 +59,7 @@ class GrpcVendoring_1_54_0 { "io.grpc:grpc-netty:$grpc_version", "io.grpc:grpc-protobuf:$grpc_version", "io.grpc:grpc-stub:$grpc_version", + "io.grpc:grpc-alts:$grpc_version", "io.grpc:grpc-testing:$grpc_version", // Use a classifier to ensure we get the jar containing native libraries. In the future // hopefully netty releases a single jar containing native libraries for all architectures. From 1033b71aff6a49509ef50e5c309bd95a19b6ff68 Mon Sep 17 00:00:00 2001 From: Vlado Djerek Date: Mon, 18 Dec 2023 15:23:26 +0100 Subject: [PATCH 142/224] pin dind image temoporarily due to issues https://github.com/actions/actions-runner-controller/issues/3159 (#29796) Co-authored-by: Vlado Djerek --- .github/gh-actions-self-hosted-runners/arc/locals.tf | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/gh-actions-self-hosted-runners/arc/locals.tf b/.github/gh-actions-self-hosted-runners/arc/locals.tf index a69d069ab865..7820ce3e8aa3 100644 --- a/.github/gh-actions-self-hosted-runners/arc/locals.tf +++ b/.github/gh-actions-self-hosted-runners/arc/locals.tf @@ -20,6 +20,8 @@ locals { arc_values = { + #fix dind issue + "image.dindSidecarRepositoryAndTag" = "docker:24.0.7-dind-alpine3.18" "githubWebhookServer.enabled" = "${var.deploy_webhook}" "authSecret.create" = "true" "authSecret.github_app_id" = data.google_secret_manager_secret_version.github_app_id.secret_data From fb8ae575eebd37353cfdc7c889c97b36551e0351 Mon Sep 17 00:00:00 2001 From: clmccart Date: Mon, 18 Dec 2023 07:32:36 -0800 Subject: [PATCH 143/224] getProcessingTimesByStep returns a modifiable map (#29786) * getProcessingTimesByStep returns a modifiable map * return copy in getProcessingTimesByStep and update name accordingly * Spotless --------- Co-authored-by: Claire McCarthy Co-authored-by: Danny McCormick --- .../dataflow/worker/DataflowExecutionContext.java | 5 +++-- .../dataflow/worker/DataflowExecutionStateSampler.java | 5 +++-- .../dataflow/worker/DataflowExecutionContextTest.java | 2 +- .../worker/DataflowExecutionStateSamplerTest.java | 10 +++++----- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java index b861295d8b9d..7d45295b2d8c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java @@ -341,8 +341,9 @@ public Optional getActiveMessageMetadata() { return Optional.ofNullable(activeMessageMetadata); } - public Map getProcessingTimesByStep() { - return Collections.unmodifiableMap(processingTimesByStep); + public Map getProcessingTimesByStepCopy() { + Map processingTimesCopy = processingTimesByStep; + return processingTimesCopy; } /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java index c5c3b2d41805..61b7e01e49a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSampler.java @@ -86,7 +86,8 @@ public void removeTracker(ExecutionStateTracker tracker) { return; } DataflowExecutionStateTracker dfTracker = (DataflowExecutionStateTracker) tracker; - completedProcessingMetrics.put(dfTracker.getWorkItemId(), dfTracker.getProcessingTimesByStep()); + completedProcessingMetrics.put( + dfTracker.getWorkItemId(), dfTracker.getProcessingTimesByStepCopy()); activeTrackersByWorkId.remove(dfTracker.getWorkItemId()); // Attribute any remaining time since the last sampling while removing the tracker. @@ -126,7 +127,7 @@ public Map getProcessingDistributionsForWorkId(Str DataflowExecutionStateTracker tracker = activeTrackersByWorkId.get(workId); return mergeStepStatsMaps( completedProcessingMetrics.getOrDefault(workId, new HashMap<>()), - tracker.getProcessingTimesByStep()); + tracker.getProcessingTimesByStepCopy()); } public void resetForWorkId(String workId) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java index 3dc026ca7c83..01951c2f83e5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContextTest.java @@ -176,7 +176,7 @@ public void testDataflowExecutionStateTrackerRecordsCompletedProcessingTimes() tracker.enterState(newState); // The first completed state should be recorded and the new state should be active. - Map gotProcessingTimes = tracker.getProcessingTimesByStep(); + Map gotProcessingTimes = tracker.getProcessingTimesByStepCopy(); Assert.assertEquals(1, gotProcessingTimes.size()); Assert.assertEquals( new HashSet<>(Arrays.asList(NameContextsForTests.nameContextForTest().userName())), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java index 1377148bc20a..b772952b74c3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateSamplerTest.java @@ -76,7 +76,7 @@ public void testRemoveTrackerCompletedProcessingTimesGetsUpdated() { Map testCompletedProcessingTimes = new HashMap<>(); testCompletedProcessingTimes.put("some-step", new IntSummaryStatistics()); DataflowExecutionStateTracker trackerMock = createMockTracker(workId); - when(trackerMock.getProcessingTimesByStep()).thenReturn(testCompletedProcessingTimes); + when(trackerMock.getProcessingTimesByStepCopy()).thenReturn(testCompletedProcessingTimes); sampler.addTracker(trackerMock); sampler.removeTracker(trackerMock); @@ -98,7 +98,7 @@ public void testGetCompletedProcessingTimesAndActiveMessageFromActiveTracker() { ActiveMessageMetadata.create(step1act1.getStepName().userName(), clock.getMillis()); DataflowExecutionStateTracker trackerMock = createMockTracker(workId); when(trackerMock.getActiveMessageMetadata()).thenReturn(Optional.of(testMetadata)); - when(trackerMock.getProcessingTimesByStep()).thenReturn(testCompletedProcessingTimes); + when(trackerMock.getProcessingTimesByStepCopy()).thenReturn(testCompletedProcessingTimes); sampler.addTracker(trackerMock); @@ -122,13 +122,13 @@ public void testResetForWorkIdClearsMaps() { equalTo(tracker1Mock.getActiveMessageMetadata())); assertThat( sampler.getProcessingDistributionsForWorkId(workId1), - equalTo(tracker1Mock.getProcessingTimesByStep())); + equalTo(tracker1Mock.getProcessingTimesByStepCopy())); assertThat( sampler.getActiveMessageMetadataForWorkId(workId2), equalTo(tracker2Mock.getActiveMessageMetadata())); assertThat( sampler.getProcessingDistributionsForWorkId(workId2), - equalTo(tracker2Mock.getProcessingTimesByStep())); + equalTo(tracker2Mock.getProcessingTimesByStepCopy())); sampler.removeTracker(tracker1Mock); sampler.removeTracker(tracker2Mock); @@ -136,7 +136,7 @@ public void testResetForWorkIdClearsMaps() { assertThat( sampler.getProcessingDistributionsForWorkId(workId1), - equalTo(tracker1Mock.getProcessingTimesByStep())); + equalTo(tracker1Mock.getProcessingTimesByStepCopy())); Assert.assertTrue(sampler.getProcessingDistributionsForWorkId(workId2).isEmpty()); } From 7db8c274dd50642a76dd2bf53ae34aad73127193 Mon Sep 17 00:00:00 2001 From: Zechen Jiang Date: Mon, 18 Dec 2023 09:39:31 -0800 Subject: [PATCH 144/224] make DataSample external (#29788) --- .../pkg/beam/core/runtime/exec/datasampler.go | 28 +++++++++---------- .../core/runtime/exec/datasampler_test.go | 24 ++++++++-------- 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasampler.go b/sdks/go/pkg/beam/core/runtime/exec/datasampler.go index 426213a12afd..15f32870b49c 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasampler.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasampler.go @@ -22,7 +22,7 @@ import ( ) // DataSample contains property for sampled element -type dataSample struct { +type DataSample struct { PCollectionID string Timestamp time.Time Element []byte @@ -30,7 +30,7 @@ type dataSample struct { // DataSampler manages sampled elements based on PCollectionID type DataSampler struct { - sampleChannel chan *dataSample + sampleChannel chan *DataSample samplesMap sync.Map // Key: PCollectionID string, Value: *OutputSamples pointer ctx context.Context } @@ -38,7 +38,7 @@ type DataSampler struct { // NewDataSampler inits a new Data Sampler object and returns pointer to it. func NewDataSampler(ctx context.Context) *DataSampler { return &DataSampler{ - sampleChannel: make(chan *dataSample, 1000), + sampleChannel: make(chan *DataSample, 1000), ctx: ctx, } } @@ -57,7 +57,7 @@ func (d *DataSampler) Process() { // GetSamples returns samples for given pCollectionID. // If no pCollectionID is provided, return all available samples -func (d *DataSampler) GetSamples(pids []string) map[string][]*dataSample { +func (d *DataSampler) GetSamples(pids []string) map[string][]*DataSample { if len(pids) == 0 { return d.getAllSamples() } @@ -66,7 +66,7 @@ func (d *DataSampler) GetSamples(pids []string) map[string][]*dataSample { // SendSample is called by PCollection Node to send sampled element to Data Sampler async func (d *DataSampler) SendSample(pCollectionID string, element []byte, timestamp time.Time) { - sample := dataSample{ + sample := DataSample{ PCollectionID: pCollectionID, Element: element, Timestamp: timestamp, @@ -74,8 +74,8 @@ func (d *DataSampler) SendSample(pCollectionID string, element []byte, timestamp d.sampleChannel <- &sample } -func (d *DataSampler) getAllSamples() map[string][]*dataSample { - var res = make(map[string][]*dataSample) +func (d *DataSampler) getAllSamples() map[string][]*DataSample { + var res = make(map[string][]*DataSample) d.samplesMap.Range(func(key any, value any) bool { pid := key.(string) samples := d.getSamples(pid) @@ -87,8 +87,8 @@ func (d *DataSampler) getAllSamples() map[string][]*dataSample { return res } -func (d *DataSampler) getSamplesForPCollections(pids []string) map[string][]*dataSample { - var res = make(map[string][]*dataSample) +func (d *DataSampler) getSamplesForPCollections(pids []string) map[string][]*DataSample { + var res = make(map[string][]*DataSample) for _, pid := range pids { samples := d.getSamples(pid) if len(samples) > 0 { @@ -98,7 +98,7 @@ func (d *DataSampler) getSamplesForPCollections(pids []string) map[string][]*dat return res } -func (d *DataSampler) addSample(sample *dataSample) { +func (d *DataSampler) addSample(sample *DataSample) { p, ok := d.samplesMap.Load(sample.PCollectionID) if !ok { p = &outputSamples{maxElements: 10, sampleIndex: 0} @@ -108,7 +108,7 @@ func (d *DataSampler) addSample(sample *dataSample) { outputSamples.addSample(sample) } -func (d *DataSampler) getSamples(pCollectionID string) []*dataSample { +func (d *DataSampler) getSamples(pCollectionID string) []*DataSample { p, ok := d.samplesMap.Load(pCollectionID) if !ok { return nil @@ -118,13 +118,13 @@ func (d *DataSampler) getSamples(pCollectionID string) []*dataSample { } type outputSamples struct { - elements []*dataSample + elements []*DataSample mu sync.Mutex maxElements int sampleIndex int } -func (o *outputSamples) addSample(element *dataSample) { +func (o *outputSamples) addSample(element *DataSample) { o.mu.Lock() defer o.mu.Unlock() @@ -136,7 +136,7 @@ func (o *outputSamples) addSample(element *dataSample) { } } -func (o *outputSamples) getSamples() []*dataSample { +func (o *outputSamples) getSamples() []*DataSample { o.mu.Lock() defer o.mu.Unlock() if len(o.elements) == 0 { diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go b/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go index d648fd89efaa..7dce38360afa 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasampler_test.go @@ -28,45 +28,45 @@ func TestDataSampler(t *testing.T) { timestamp := time.Now() tests := []struct { name string - samples []dataSample + samples []DataSample pids []string - want map[string][]*dataSample + want map[string][]*DataSample }{ { name: "GetAllSamples", - samples: []dataSample{ + samples: []DataSample{ {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, }, pids: []string{}, - want: map[string][]*dataSample{ + want: map[string][]*DataSample{ "pid1": {{PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}}, "pid2": {{PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}}, }, }, { name: "GetSamplesForPCollections", - samples: []dataSample{ + samples: []DataSample{ {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, }, pids: []string{"pid1"}, - want: map[string][]*dataSample{ + want: map[string][]*DataSample{ "pid1": {{PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}}, }, }, { name: "GetSamplesForPCollectionsWithNoResult", - samples: []dataSample{ + samples: []DataSample{ {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, {PCollectionID: "pid2", Element: []byte("element2"), Timestamp: timestamp}, }, pids: []string{"pid3"}, - want: map[string][]*dataSample{}, + want: map[string][]*DataSample{}, }, { name: "GetSamplesForPCollectionsTooManySamples", - samples: []dataSample{ + samples: []DataSample{ {PCollectionID: "pid1", Element: []byte("element1"), Timestamp: timestamp}, {PCollectionID: "pid1", Element: []byte("element2"), Timestamp: timestamp}, {PCollectionID: "pid1", Element: []byte("element3"), Timestamp: timestamp}, @@ -80,7 +80,7 @@ func TestDataSampler(t *testing.T) { {PCollectionID: "pid1", Element: []byte("element11"), Timestamp: timestamp}, }, pids: []string{"pid1"}, - want: map[string][]*dataSample{ + want: map[string][]*DataSample{ "pid1": { {PCollectionID: "pid1", Element: []byte("element2"), Timestamp: timestamp}, {PCollectionID: "pid1", Element: []byte("element3"), Timestamp: timestamp}, @@ -105,7 +105,7 @@ func TestDataSampler(t *testing.T) { dataSampler.SendSample(sample.PCollectionID, sample.Element, sample.Timestamp) } var samplesCount = -1 - var samples map[string][]*dataSample + var samples map[string][]*DataSample for i := 0; i < 5; i++ { samples = dataSampler.GetSamples(test.pids) if len(samples) == len(test.want) { @@ -125,7 +125,7 @@ func TestDataSampler(t *testing.T) { } } -func verifySampledElements(samples, want map[string][]*dataSample) bool { +func verifySampledElements(samples, want map[string][]*DataSample) bool { if len(samples) != len(want) { return false } From f02bf88fd159a23bd4481d4b0ff5c640f8e51fa2 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 18 Dec 2023 13:25:08 -0500 Subject: [PATCH 145/224] Upgrade download-artifact to v4 (#29801) --- .github/workflows/build_wheels.yml | 8 ++++---- .github/workflows/python_tests.yml | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 346d40561d96..85ed7c0686c3 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -204,7 +204,7 @@ jobs: if: needs.check_env_variables.outputs.gcp-variables-set == 'true' steps: - name: Download compressed sources from artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: source_zip path: source/ @@ -237,13 +237,13 @@ jobs: arch: aarch64 steps: - name: Download python source distribution from artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: source path: apache-beam-source - name: Download Python SDK RC source distribution from artifacts if: ${{ needs.build_source.outputs.is_rc == 1 }} - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: source_rc${{ needs.build_source.outputs.rc_num }} path: apache-beam-source-rc @@ -320,7 +320,7 @@ jobs: os : [ubuntu-latest, macos-latest, windows-latest] steps: - name: Download wheels from artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: wheelhouse-${{ matrix.os }} path: wheelhouse/ diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index a17833ead3f7..5596e06945b0 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -172,7 +172,7 @@ jobs: with: go-version: '1.21' - name: Download source from artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: python_sdk_source path: apache-beam-source From c3d130c8ebf9e5266c2d0cef8a45c2ce517e2a86 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 18 Dec 2023 18:34:32 +0000 Subject: [PATCH 146/224] Update python base image dependencies (#29800) * Update container dependencies * update dependencies from a linux machine --- .../py310/base_image_requirements.txt | 66 +++++++++---------- .../py311/base_image_requirements.txt | 64 +++++++++--------- .../py38/base_image_requirements.txt | 64 +++++++++--------- .../py39/base_image_requirements.txt | 66 +++++++++---------- 4 files changed, 130 insertions(+), 130 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index a82ce0050da0..d6b810725522 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -37,54 +37,55 @@ Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 -docker==6.1.3 +docker==7.0.0 docopt==0.6.2 +docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 fastavro==1.9.1 fasteners==0.19 freezegun==1.3.1 future==0.18.3 -google-api-core==2.14.0 -google-api-python-client==2.109.0 +google-api-core==2.15.0 +google-api-python-client==2.111.0 google-apitools==0.5.31 -google-auth==2.25.1 +google-auth==2.25.2 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.37.0 -google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.23.0 +google-cloud-aiplatform==1.38.1 +google-cloud-bigquery==3.14.1 +google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 -google-cloud-core==2.3.3 -google-cloud-datastore==2.18.0 -google-cloud-dlp==3.13.0 -google-cloud-language==2.11.1 +google-cloud-core==2.4.1 +google-cloud-datastore==2.19.0 +google-cloud-dlp==3.14.0 +google-cloud-language==2.12.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.18.4 -google-cloud-pubsublite==1.8.3 -google-cloud-recommendations-ai==0.10.5 -google-cloud-resource-manager==1.10.4 +google-cloud-pubsub==2.19.0 +google-cloud-pubsublite==1.9.0 +google-cloud-recommendations-ai==0.10.6 +google-cloud-resource-manager==1.11.0 google-cloud-spanner==3.40.1 -google-cloud-storage==2.13.0 -google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.5 +google-cloud-storage==2.14.0 +google-cloud-videointelligence==2.12.0 +google-cloud-vision==3.5.0 google-crc32c==1.5.0 -google-resumable-media==2.6.0 -googleapis-common-protos==1.61.0 -greenlet==3.0.1 -grpc-google-iam-v1==0.12.7 -grpcio==1.59.3 -grpcio-status==1.59.3 +google-resumable-media==2.7.0 +googleapis-common-protos==1.62.0 +greenlet==3.0.2 +grpc-google-iam-v1==0.13.0 +grpcio==1.60.0 +grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.91.0 +hypothesis==6.92.1 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 +jsonpickle==3.0.2 jsonschema==4.20.0 jsonschema-specifications==2023.11.2 -jsonpickle==3.0.2 mmh3==4.0.1 mock==5.1.0 nltk==3.8.1 @@ -93,15 +94,15 @@ numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 orjson==3.9.10 -overrides==6.5.0 +overrides==7.4.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 -proto-plus==1.22.3 +proto-plus==1.23.0 protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==14.0.1 +pyarrow==14.0.2 pyarrow-hotfix==0.6 pyasn1==0.5.1 pyasn1-modules==0.3.0 @@ -120,11 +121,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.1 +referencing==0.32.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.2 +rpds-py==0.15.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -139,10 +140,9 @@ testcontainers==3.7.1 threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 -typing_extensions==4.8.0 +typing_extensions==4.9.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.7.0 wrapt==1.16.0 zstandard==0.22.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 149368b15ed6..ac6f6dbd7258 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -37,46 +37,47 @@ Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 -docker==6.1.3 +docker==7.0.0 docopt==0.6.2 +docstring-parser==0.15 execnet==2.0.2 fastavro==1.9.1 fasteners==0.19 freezegun==1.3.1 future==0.18.3 -google-api-core==2.14.0 -google-api-python-client==2.109.0 +google-api-core==2.15.0 +google-api-python-client==2.111.0 google-apitools==0.5.31 -google-auth==2.25.1 +google-auth==2.25.2 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.37.0 -google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.23.0 +google-cloud-aiplatform==1.38.1 +google-cloud-bigquery==3.14.1 +google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 -google-cloud-core==2.3.3 -google-cloud-datastore==2.18.0 -google-cloud-dlp==3.13.0 -google-cloud-language==2.11.1 +google-cloud-core==2.4.1 +google-cloud-datastore==2.19.0 +google-cloud-dlp==3.14.0 +google-cloud-language==2.12.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.18.4 -google-cloud-pubsublite==1.8.3 -google-cloud-recommendations-ai==0.10.5 -google-cloud-resource-manager==1.10.4 +google-cloud-pubsub==2.19.0 +google-cloud-pubsublite==1.9.0 +google-cloud-recommendations-ai==0.10.6 +google-cloud-resource-manager==1.11.0 google-cloud-spanner==3.40.1 -google-cloud-storage==2.13.0 -google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.5 +google-cloud-storage==2.14.0 +google-cloud-videointelligence==2.12.0 +google-cloud-vision==3.5.0 google-crc32c==1.5.0 -google-resumable-media==2.6.0 -googleapis-common-protos==1.61.0 -greenlet==3.0.1 -grpc-google-iam-v1==0.12.7 -grpcio==1.59.3 -grpcio-status==1.59.3 +google-resumable-media==2.7.0 +googleapis-common-protos==1.62.0 +greenlet==3.0.2 +grpc-google-iam-v1==0.13.0 +grpcio==1.60.0 +grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.91.0 +hypothesis==6.92.1 idna==3.6 iniconfig==2.0.0 joblib==1.3.2 @@ -92,15 +93,15 @@ numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 orjson==3.9.10 -overrides==6.5.0 +overrides==7.4.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 -proto-plus==1.22.3 +proto-plus==1.23.0 protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==14.0.1 +pyarrow==14.0.2 pyarrow-hotfix==0.6 pyasn1==0.5.1 pyasn1-modules==0.3.0 @@ -118,11 +119,11 @@ pytest-xdist==3.5.0 python-dateutil==2.8.2 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.1 +referencing==0.32.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.2 +rpds-py==0.15.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -136,10 +137,9 @@ tenacity==8.2.3 testcontainers==3.7.1 threadpoolctl==3.2.0 tqdm==4.66.1 -typing_extensions==4.8.0 +typing_extensions==4.9.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.7.0 wrapt==1.16.0 zstandard==0.22.0 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 3bd996dbe2b9..2929ab6f1e04 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -38,47 +38,48 @@ Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 -docker==6.1.3 +docker==7.0.0 docopt==0.6.2 +docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 fastavro==1.9.1 fasteners==0.19 freezegun==1.3.1 future==0.18.3 -google-api-core==2.14.0 -google-api-python-client==2.109.0 +google-api-core==2.15.0 +google-api-python-client==2.111.0 google-apitools==0.5.31 -google-auth==2.25.1 +google-auth==2.25.2 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.37.0 -google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.23.0 +google-cloud-aiplatform==1.38.1 +google-cloud-bigquery==3.14.1 +google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 -google-cloud-core==2.3.3 -google-cloud-datastore==2.18.0 -google-cloud-dlp==3.13.0 -google-cloud-language==2.11.1 +google-cloud-core==2.4.1 +google-cloud-datastore==2.19.0 +google-cloud-dlp==3.14.0 +google-cloud-language==2.12.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.18.4 -google-cloud-pubsublite==1.8.3 -google-cloud-recommendations-ai==0.10.5 -google-cloud-resource-manager==1.10.4 +google-cloud-pubsub==2.19.0 +google-cloud-pubsublite==1.9.0 +google-cloud-recommendations-ai==0.10.6 +google-cloud-resource-manager==1.11.0 google-cloud-spanner==3.40.1 -google-cloud-storage==2.13.0 -google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.5 +google-cloud-storage==2.14.0 +google-cloud-videointelligence==2.12.0 +google-cloud-vision==3.5.0 google-crc32c==1.5.0 -google-resumable-media==2.6.0 -googleapis-common-protos==1.61.0 -greenlet==3.0.1 -grpc-google-iam-v1==0.12.7 -grpcio==1.59.3 -grpcio-status==1.59.3 +google-resumable-media==2.7.0 +googleapis-common-protos==1.62.0 +greenlet==3.0.2 +grpc-google-iam-v1==0.13.0 +grpcio==1.60.0 +grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.91.0 +hypothesis==6.92.1 idna==3.6 importlib-metadata==7.0.0 importlib-resources==6.1.1 @@ -96,16 +97,16 @@ numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 orjson==3.9.10 -overrides==6.5.0 +overrides==7.4.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pkgutil_resolve_name==1.3.10 pluggy==1.3.0 -proto-plus==1.22.3 +proto-plus==1.23.0 protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==14.0.1 +pyarrow==14.0.2 pyarrow-hotfix==0.6 pyasn1==0.5.1 pyasn1-modules==0.3.0 @@ -124,11 +125,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.1 +referencing==0.32.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.2 +rpds-py==0.15.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.10.1 @@ -143,11 +144,10 @@ testcontainers==3.7.1 threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 -typing_extensions==4.8.0 +typing_extensions==4.9.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.7.0 wrapt==1.16.0 zipp==3.17.0 zstandard==0.22.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index a6bb3a31fe0a..9d7e99f76eee 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -37,54 +37,55 @@ Cython==0.29.36 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 -docker==6.1.3 +docker==7.0.0 docopt==0.6.2 +docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 fastavro==1.9.1 fasteners==0.19 freezegun==1.3.1 future==0.18.3 -google-api-core==2.14.0 -google-api-python-client==2.109.0 +google-api-core==2.15.0 +google-api-python-client==2.111.0 google-apitools==0.5.31 -google-auth==2.25.1 +google-auth==2.25.2 google-auth-httplib2==0.1.1 -google-cloud-aiplatform==1.37.0 -google-cloud-bigquery==3.13.0 -google-cloud-bigquery-storage==2.23.0 +google-cloud-aiplatform==1.38.1 +google-cloud-bigquery==3.14.1 +google-cloud-bigquery-storage==2.24.0 google-cloud-bigtable==2.22.0 -google-cloud-core==2.3.3 -google-cloud-datastore==2.18.0 -google-cloud-dlp==3.13.0 -google-cloud-language==2.11.1 +google-cloud-core==2.4.1 +google-cloud-datastore==2.19.0 +google-cloud-dlp==3.14.0 +google-cloud-language==2.12.0 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.18.4 -google-cloud-pubsublite==1.8.3 -google-cloud-recommendations-ai==0.10.5 -google-cloud-resource-manager==1.10.4 +google-cloud-pubsub==2.19.0 +google-cloud-pubsublite==1.9.0 +google-cloud-recommendations-ai==0.10.6 +google-cloud-resource-manager==1.11.0 google-cloud-spanner==3.40.1 -google-cloud-storage==2.13.0 -google-cloud-videointelligence==2.11.4 -google-cloud-vision==3.4.5 +google-cloud-storage==2.14.0 +google-cloud-videointelligence==2.12.0 +google-cloud-vision==3.5.0 google-crc32c==1.5.0 -google-resumable-media==2.6.0 -googleapis-common-protos==1.61.0 -greenlet==3.0.1 -grpc-google-iam-v1==0.12.7 -grpcio==1.59.3 -grpcio-status==1.59.3 +google-resumable-media==2.7.0 +googleapis-common-protos==1.62.0 +greenlet==3.0.2 +grpc-google-iam-v1==0.13.0 +grpcio==1.60.0 +grpcio-status==1.60.0 guppy3==3.1.4.post1 hdfs==2.7.3 httplib2==0.22.0 -hypothesis==6.91.0 +hypothesis==6.92.1 idna==3.6 importlib-metadata==7.0.0 iniconfig==2.0.0 joblib==1.3.2 Js2Py==0.74 -jsonschema==4.20.0 jsonpickle==3.0.2 +jsonschema==4.20.0 jsonschema-specifications==2023.11.2 mmh3==4.0.1 mock==5.1.0 @@ -94,15 +95,15 @@ numpy==1.24.4 oauth2client==4.1.3 objsize==0.6.1 orjson==3.9.10 -overrides==6.5.0 +overrides==7.4.0 packaging==23.2 pandas==1.5.3 parameterized==0.9.0 pluggy==1.3.0 -proto-plus==1.22.3 +proto-plus==1.23.0 protobuf==4.25.1 psycopg2-binary==2.9.9 -pyarrow==14.0.1 +pyarrow==14.0.2 pyarrow-hotfix==0.6 pyasn1==0.5.1 pyasn1-modules==0.3.0 @@ -121,11 +122,11 @@ python-dateutil==2.8.2 python-snappy==0.6.1 pytz==2023.3.post1 PyYAML==6.0.1 -referencing==0.31.1 +referencing==0.32.0 regex==2023.10.3 requests==2.31.0 requests-mock==1.11.0 -rpds-py==0.13.2 +rpds-py==0.15.2 rsa==4.9 scikit-learn==1.3.2 scipy==1.11.4 @@ -140,11 +141,10 @@ testcontainers==3.7.1 threadpoolctl==3.2.0 tomli==2.0.1 tqdm==4.66.1 -typing_extensions==4.8.0 +typing_extensions==4.9.0 tzlocal==5.2 uritemplate==4.1.1 urllib3==2.1.0 -websocket-client==1.7.0 wrapt==1.16.0 zipp==3.17.0 zstandard==0.22.0 From add34385719e53e4cc728267bfd2d8e8730c7267 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 18 Dec 2023 14:19:37 -0500 Subject: [PATCH 147/224] Fix GenerateSequenceSchemaTransformProvider formatting (#29798) --- .../providers/GenerateSequenceSchemaTransformProvider.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java index f4cada661b03..4b693f883fb7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java @@ -67,9 +67,9 @@ public String description() { return String.format( "Outputs a PCollection of Beam Rows, each containing a single INT64 " + "number called \"value\". The count is produced from the given \"start\"" - + "value and either up to the given \"end\" or until 2^63 - 1.\n" + + "value and either up to the given \"end\" or until 2^63 - 1.%n" + "To produce an unbounded PCollection, simply do not specify an \"end\" value. " - + "Unbounded sequences can specify a \"rate\" for output elements.\n" + + "Unbounded sequences can specify a \"rate\" for output elements.%n" + "In all cases, the sequence of numbers is generated in parallel, so there is no " + "inherent ordering between the generated values"); } From 4cfe5498850487b2ffa1874425549a6145ac9b3f Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Mon, 18 Dec 2023 16:44:03 -0500 Subject: [PATCH 148/224] Fix nil pointer access in logRuntimeDependencies (#29804) --- sdks/python/container/boot.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go index cbf05bbd04a7..c59807c72a98 100644 --- a/sdks/python/container/boot.go +++ b/sdks/python/container/boot.go @@ -409,7 +409,7 @@ func installSetupPackages(ctx context.Context, logger *tools.Logger, files []str if err := pipInstallPackage(ctx, logger, files, workDir, workflowFile, false, true, nil); err != nil { return fmt.Errorf("failed to install workflow: %v", err) } - if err := logRuntimeDependencies(ctx, logger); err != nil { + if err := logRuntimeDependencies(ctx, bufLogger); err != nil { logger.Warnf(ctx, "couldn't fetch the runtime python dependencies: %v", err) } @@ -459,15 +459,14 @@ func processArtifactsInSetupOnlyMode() { // logRuntimeDependencies logs the python dependencies // installed in the runtime environment. -func logRuntimeDependencies(ctx context.Context, logger *tools.Logger) error { - logger.Printf(ctx, "Logging runtime dependencies:") +func logRuntimeDependencies(ctx context.Context, bufLogger *tools.BufferedLogger) error { + bufLogger.Printf(ctx, "Logging runtime dependencies:") pythonVersion, err := expansionx.GetPythonVersion() if err != nil { return err } - logger.Printf(ctx, "Using Python version:") + bufLogger.Printf(ctx, "Using Python version:") args := []string{"--version"} - bufLogger := tools.NewBufferedLogger(logger) if err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...); err != nil { bufLogger.FlushAtError(ctx) } else { From 2e38af69cf22568e002ebf5137d1282d4c4a27f3 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Mon, 18 Dec 2023 15:47:58 -0800 Subject: [PATCH 149/224] Adds Kafka upgrade module to the I/O precommit (#29773) --- build.gradle.kts | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle.kts b/build.gradle.kts index 59161809f37c..7a3686e5678c 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -350,6 +350,7 @@ tasks.register("javaioPreCommit") { dependsOn(":sdks:java:io:jdbc:build") dependsOn(":sdks:java:io:jms:build") dependsOn(":sdks:java:io:kafka:build") + dependsOn(":sdks:java:io:kafka:upgrade:build") dependsOn(":sdks:java:io:kudu:build") dependsOn(":sdks:java:io:mongodb:build") dependsOn(":sdks:java:io:mqtt:build") From 87794aaef03ef1596cdfa28cfbd51b5084825787 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 19 Dec 2023 10:36:20 +0000 Subject: [PATCH 150/224] Update dataflow container tag (#29790) --- 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 b43f615189da..1b7795916f0c 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-20231212' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231215' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From 2baea8b65ebe2952b317c2a06fd97ff25b095000 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Dec 2023 12:16:00 +0100 Subject: [PATCH 151/224] Bump golang.org/x/crypto from 0.14.0 to 0.17.0 in /.test-infra/mock-apis (#29809) Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .test-infra/mock-apis/go.mod | 6 +++--- .test-infra/mock-apis/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.test-infra/mock-apis/go.mod b/.test-infra/mock-apis/go.mod index cc65cfbaac76..306d9f53ee07 100644 --- a/.test-infra/mock-apis/go.mod +++ b/.test-infra/mock-apis/go.mod @@ -45,12 +45,12 @@ require ( github.com/googleapis/enterprise-certificate-proxy v0.2.4 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.14.0 // indirect + golang.org/x/crypto v0.17.0 // indirect golang.org/x/net v0.17.0 // indirect golang.org/x/oauth2 v0.12.0 // indirect golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.13.0 // indirect - golang.org/x/text v0.13.0 // indirect + golang.org/x/sys v0.15.0 // indirect + golang.org/x/text v0.14.0 // indirect google.golang.org/api v0.128.0 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 // indirect diff --git a/.test-infra/mock-apis/go.sum b/.test-infra/mock-apis/go.sum index a928e3dae2f1..13ca0c274e93 100644 --- a/.test-infra/mock-apis/go.sum +++ b/.test-infra/mock-apis/go.sum @@ -101,8 +101,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= -golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -144,8 +144,8 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= -golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -154,8 +154,8 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= -golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= -golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= From d6426c4fdfc93dc0c762963bd6fe2cf6614e1a33 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Dec 2023 10:09:03 -0500 Subject: [PATCH 152/224] Bump google.golang.org/grpc from 1.60.0 to 1.60.1 in /sdks (#29814) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.60.0 to 1.60.1. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.60.0...v1.60.1) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 2e3d61bea03a..a3faefa52ac4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( golang.org/x/text v0.14.0 google.golang.org/api v0.154.0 google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f - google.golang.org/grpc v1.60.0 + google.golang.org/grpc v1.60.1 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 5ed3a4e198f4..46d324832e40 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -745,8 +745,8 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= -google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= +google.golang.org/grpc v1.60.1 h1:26+wFr+cNqSGFcOXcabYC0lUVJVRa2Sb2ortSK7VrEU= +google.golang.org/grpc v1.60.1/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 60f040812403b3381e44f7d493aa2e9976767044 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 19 Dec 2023 16:09:36 +0100 Subject: [PATCH 153/224] Update changes.md with known fixes. (#29817) --- CHANGES.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index b791dd806d4a..cbfb005d5ab1 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -91,12 +91,11 @@ ## Highlights -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* Python streaming users that use 2.47.0 and newer versions of Beam should update to version 2.53.0, which fixes a known issue: ([#27330](https://github.com/apache/beam/issues/27330)). * New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). ## I/Os -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * TextIO now supports skipping multiple header lines (Java) ([#17990](https://github.com/apache/beam/issues/17990)). * Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) * Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). @@ -106,12 +105,12 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * The Python SDK now type checks `collections.abc.Collections` types properly. Some type hints that were erroneously allowed by the SDK may now fail. ([#29272](https://github.com/apache/beam/pull/29272)) * Running multi-language pipelines locally no longer requires Docker. Instead, the same (generally auto-started) subprocess used to perform the expansion can also be used as the cross-language worker. -* Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)) +* Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)). +* Python 3.11 images now include google-cloud-profiler ([#29561](https://github.com/apache/beam/pull/29651)). ## Breaking Changes @@ -123,8 +122,8 @@ ## Bugfixes -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). -* Fixed MLTransform when duplicated elements are dropped in the output PCollection.([#29600](https://github.com/apache/beam/issues/29600)) +* (Python) Fixed sporadic crashes in streaming pipelines that affected some users of 2.47.0 and newer SDKs ([#27330](https://github.com/apache/beam/issues/27330)). +* (Python) Fixed a bug that caused MLTransform to drop identical elements in the output PCollection ([#29600](https://github.com/apache/beam/issues/29600)). ## Security Fixes From 54d2c2df5cf6b221a5884fbe27e2b88c0781ac90 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Dec 2023 07:54:56 -0800 Subject: [PATCH 154/224] Bump golang.org/x/crypto from 0.16.0 to 0.17.0 in /sdks (#29810) Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.16.0 to 0.17.0. - [Commits](https://github.com/golang/crypto/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: indirect ... 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 a3faefa52ac4..28a049709ac0 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -180,7 +180,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.16.0 // indirect + golang.org/x/crypto v0.17.0 // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 46d324832e40..f0e8e65fb9dc 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -511,8 +511,8 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.16.0 h1:mMMrFzRSCF0GvB7Ne27XVtVAaXLrPmgPC7/v0tkwHaY= -golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= From 7827d1d1809e89c612b1299fb8069230fe6829f0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 5 Oct 2023 14:40:59 -0400 Subject: [PATCH 155/224] Fix Reshuffle implementation in Java SDK --- runners/flink/flink_runner.gradle | 2 + .../flink/job-server/flink_job_server.gradle | 2 + .../samza/src/test/resources/ExpectedDag.json | 68 +++------ .../apache/beam/sdk/transforms/Reshuffle.java | 89 +++++++++++ .../beam/sdk/transforms/ReshuffleTest.java | 139 ++++++++++++++++++ 5 files changed, 253 insertions(+), 47 deletions(-) diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index c510b346d5d0..c5946c509a95 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -310,6 +310,8 @@ def createValidatesRunnerTask(Map m) { excludeTestsMatching test } + // Flink reshuffle override does not preserve all metadata + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' // https://github.com/apache/beam/issues/20843 excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testDiscardingMode' // https://github.com/apache/beam/issues/20845 diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index 9643134d95b2..ab3f726e6ad9 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -196,6 +196,8 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean checkpoi excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderInBundle' }, testFilter: { + // Flink reshuffle override does not preserve all metadata + excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata' // TODO(https://github.com/apache/beam/issues/20269) excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2' // TODO(https://github.com/apache/beam/issues/20843) diff --git a/runners/samza/src/test/resources/ExpectedDag.json b/runners/samza/src/test/resources/ExpectedDag.json index 3165fc84958c..c61b80134d8a 100644 --- a/runners/samza/src/test/resources/ExpectedDag.json +++ b/runners/samza/src/test/resources/ExpectedDag.json @@ -98,26 +98,26 @@ "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/Window.Into()", + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/SetIdentityWindow", "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/Window.Into()/Window.Assign", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/Window.Into()" + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/SetIdentityWindow/Window.Assign", + "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/SetIdentityWindow" } ] }, { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps", + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata", "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps/ParDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps", + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata/ParDo(Anonymous)", + "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps/ParDo(Anonymous)/ParMultiDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps/ParDo(Anonymous)" + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)", + "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata/ParDo(Anonymous)" } ] } @@ -138,38 +138,16 @@ ] }, { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps", + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata", "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps", + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)", + "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata", "ChildNodes": [ { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard/ParDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard", - "ChildNodes": [ - { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard/ParDo(Anonymous)/ParMultiDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard/ParDo(Anonymous)" - } - ] - } - ] - }, - { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps", - "ChildNodes": [ - { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues", - "ChildNodes": [ - { - "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)/ParMultiDo(Anonymous)", - "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)" - } - ] + "fullName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)", + "enclosingNode": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)" } ] } @@ -287,14 +265,14 @@ }, { "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/Assign unique key/AddKeys/Map/ParMultiDo(Anonymous)", - "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/Window.Into()/Window.Assign" + "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/SetIdentityWindow/Window.Assign" }, { - "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/Window.Into()/Window.Assign", - "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps/ParDo(Anonymous)/ParMultiDo(Anonymous)" + "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/SetIdentityWindow/Window.Assign", + "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)" }, { - "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalTimestamps/ParDo(Anonymous)/ParMultiDo(Anonymous)", + "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ReifyOriginalMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)", "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/GroupByKey" }, { @@ -303,14 +281,10 @@ }, { "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/ExpandIterable/ParMultiDo(Anonymous)", - "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard/ParDo(Anonymous)/ParMultiDo(Anonymous)" - }, - { - "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/ReifyTimestamps.RemoveWildcard/ParDo(Anonymous)/ParMultiDo(Anonymous)", - "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)/ParMultiDo(Anonymous)" + "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)" }, { - "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)/ParMultiDo(Anonymous)", + "from": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous)", "to": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Drop key/Values/Map/ParMultiDo(Anonymous)" }, { @@ -357,7 +331,7 @@ }, { "transformName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Drop key/Values/Map/ParMultiDo(Anonymous)", - "inputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)/ParMultiDo(Anonymous).output", + "inputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous).output", "outputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Drop key/Values/Map/ParMultiDo(Anonymous).output" }, { @@ -378,7 +352,7 @@ { "transformName": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle", "inputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/Assign unique key/AddKeys/Map/ParMultiDo(Anonymous).output", - "outputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreOriginalTimestamps/Reify.ExtractTimestampsFromValues/ParDo(Anonymous)/ParMultiDo(Anonymous).output" + "outputs": "Create.TimestampedValues/Create.Values/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Reshuffle/RestoreMetadata/ParDo(Anonymous)/ParMultiDo(Anonymous).output" }, { "transformName": "Create.TimestampedValues/ParDo(ConvertTimestamps)/ParMultiDo(ConvertTimestamps)", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java index 42f0f6accc7f..a879667ff96f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java @@ -17,8 +17,13 @@ */ package org.apache.beam.sdk.transforms; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; @@ -27,7 +32,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Comparators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedInteger; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -67,6 +74,63 @@ public static ViaRandomKey viaRandomKey() { @Override public PCollection> expand(PCollection> input) { + String requestedVersionString = + input.getPipeline().getOptions().as(StreamingOptions.class).getUpdateCompatibilityVersion(); + + if (requestedVersionString != null) { + List requestedVersion = Arrays.asList(requestedVersionString.split("\\.")); + List targetVersion = Arrays.asList("2", "53", "0"); + + if (Comparators.lexicographical(Comparator.naturalOrder()) + .compare(requestedVersion, targetVersion) + <= 0) { + return expand_2_53_0(input); + } + } + + WindowingStrategy originalStrategy = input.getWindowingStrategy(); + // If the input has already had its windows merged, then the GBK that performed the merge + // will have set originalStrategy.getWindowFn() to InvalidWindows, causing the GBK contained + // here to fail. Instead, we install a valid WindowFn that leaves all windows unchanged. + // The TimestampCombiner is set to ensure the GroupByKey does not shift elements forwards in + // time. + // Because this outputs as fast as possible, this should not hold the watermark. + Window> rewindow = + Window.>into(new IdentityWindowFn<>(originalStrategy.getWindowFn().windowCoder())) + .triggering(new ReshuffleTrigger<>()) + .discardingFiredPanes() + .withTimestampCombiner(TimestampCombiner.EARLIEST) + .withAllowedLateness(Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); + + PCollection>> reified = + input + .apply("SetIdentityWindow", rewindow) + .apply("ReifyOriginalMetadata", Reify.windowsInValue()); + + PCollection>>> grouped = + reified.apply(GroupByKey.create()); + return grouped + .apply( + "ExpandIterable", + ParDo.of( + new DoFn>>, KV>>() { + @ProcessElement + public void processElement( + @Element KV>> element, + OutputReceiver>> r) { + K key = element.getKey(); + for (ValueInSingleWindow value : element.getValue()) { + r.output(KV.of(key, value)); + } + } + })) + .apply("RestoreMetadata", new RestoreMetadata<>()) + // Set the windowing strategy directly, so that it doesn't get counted as the user having + // set allowed lateness. + .setWindowingStrategyInternal(originalStrategy); + } + + private PCollection> expand_2_53_0(PCollection> input) { WindowingStrategy originalStrategy = input.getWindowingStrategy(); // If the input has already had its windows merged, then the GBK that performed the merge // will have set originalStrategy.getWindowFn() to InvalidWindows, causing the GBK contained @@ -105,6 +169,31 @@ public void processElement( .apply("RestoreOriginalTimestamps", ReifyTimestamps.extractFromValues()); } + private static class RestoreMetadata + extends PTransform>>, PCollection>> { + @Override + public PCollection> expand(PCollection>> input) { + return input.apply( + ParDo.of( + new DoFn>, KV>() { + @Override + public Duration getAllowedTimestampSkew() { + return Duration.millis(Long.MAX_VALUE); + } + + @ProcessElement + public void processElement( + @Element KV> kv, OutputReceiver> r) { + r.outputWindowedValue( + KV.of(kv.getKey(), kv.getValue().getValue()), + kv.getValue().getTimestamp(), + Collections.singleton(kv.getValue().getWindow()), + kv.getValue().getPane()); + } + })); + } + } + /** Implementation of {@link #viaRandomKey()}. */ public static class ViaRandomKey extends PTransform, PCollection> { private ViaRandomKey() {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 716bd8781831..e0e8a7feed6d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -24,13 +24,18 @@ import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.Serializable; import java.util.List; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; @@ -40,12 +45,15 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -152,6 +160,81 @@ public void testReshufflePreservesTimestamps() { pipeline.run(); } + /** + * Tests that window & pane info is preserved after applying a {@link Reshuffle} with the default + * {@link WindowingStrategy}. + */ + @Test + @Category(ValidatesRunner.class) + public void testReshufflePreservesMetadata() { + PCollection>> input = + pipeline + .apply( + Create.windowedValues( + WindowedValue.of( + "foo", + BoundedWindow.TIMESTAMP_MIN_VALUE, + GlobalWindow.INSTANCE, + PaneInfo.NO_FIRING), + WindowedValue.of( + "foo", + new Instant(0), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowedValue.of( + "bar", + new Instant(33), + GlobalWindow.INSTANCE, + PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), + WindowedValue.of( + "bar", + GlobalWindow.INSTANCE.maxTimestamp(), + GlobalWindow.INSTANCE, + PaneInfo.NO_FIRING)) + .withCoder(StringUtf8Coder.of()) + .withWindowCoder(GlobalWindow.Coder.INSTANCE)) + .apply(WithKeys.of(v -> v).withKeyType(TypeDescriptors.strings())) + .apply("ReifyOriginalMetadata", Reify.windowsInValue()); + + // The outer WindowedValue is the reified metadata post-reshuffle. The inner + // WindowedValue is the pre-reshuffle metadata. + PCollection>> output = + input + .apply(Reshuffle.of()) + .apply("ReifyReshuffledMetadata", Reify.windowsInValue()) + .apply(Values.create()); + + PAssert.that(output) + .satisfies( + input1 -> { + for (ValueInSingleWindow> elem : input1) { + Instant originalTimestamp = elem.getValue().getTimestamp(); + Instant afterReshuffleTimestamp = elem.getTimestamp(); + assertThat( + "Reshuffle did not preserve element timestamp for " + elem, + afterReshuffleTimestamp, + equalTo(originalTimestamp)); + + PaneInfo originalPaneInfo = elem.getValue().getPane(); + PaneInfo afterReshufflePaneInfo = elem.getPane(); + assertThat( + "Reshuffle did not preserve pane info for " + elem, + afterReshufflePaneInfo, + equalTo(originalPaneInfo)); + + BoundedWindow originalWindow = elem.getValue().getWindow(); + BoundedWindow afterReshuffleWindow = elem.getWindow(); + assertThat( + "Reshuffle did not preserve window for " + elem, + afterReshuffleWindow, + equalTo(originalWindow)); + } + return null; + }); + + pipeline.run(); + } + @Test @Category(ValidatesRunner.class) public void testReshuffleAfterSessionsAndGroupByKey() { @@ -301,4 +384,60 @@ public void testAssignShardFn() { pipeline.run(); } + + static class OldTransformSeeker extends Pipeline.PipelineVisitor.Defaults { + boolean isOldTransformFound = false; + + // A class that is only found in the old expansion + private Class restoreTimestampsClass = ReifyTimestamps.extractFromValues().getClass(); + + @Override + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { + if (restoreTimestampsClass.isInstance(node.getTransform())) { + this.isOldTransformFound = true; + return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; + } else { + return CompositeBehavior.ENTER_TRANSFORM; + } + } + } + + @Test + public void testNoOldTransformByDefault() { + pipeline.apply(Create.of(KV.of("arbitrary", "kv"))).apply(Reshuffle.of()); + + OldTransformSeeker seeker = new OldTransformSeeker(); + pipeline.traverseTopologically(seeker); + assertFalse(seeker.isOldTransformFound); + } + + @Test + public void testRequestOldUpdateCompatibility() { + pipeline.getOptions().as(StreamingOptions.class).setUpdateCompatibilityVersion("2.53.0"); + pipeline.apply(Create.of(KV.of("arbitrary", "kv"))).apply(Reshuffle.of()); + + OldTransformSeeker seeker = new OldTransformSeeker(); + pipeline.traverseTopologically(seeker); + assertTrue(seeker.isOldTransformFound); + } + + @Test + public void testRequestVeryOldUpdateCompatibility() { + pipeline.getOptions().as(StreamingOptions.class).setUpdateCompatibilityVersion("2.46.0"); + pipeline.apply(Create.of(KV.of("arbitrary", "kv"))).apply(Reshuffle.of()); + + OldTransformSeeker seeker = new OldTransformSeeker(); + pipeline.traverseTopologically(seeker); + assertTrue(seeker.isOldTransformFound); + } + + @Test + public void testNoOldTransformInRecentVersion() { + pipeline.getOptions().as(StreamingOptions.class).setUpdateCompatibilityVersion("2.54.0"); + pipeline.apply(Create.of(KV.of("arbitrary", "kv"))).apply(Reshuffle.of()); + + OldTransformSeeker seeker = new OldTransformSeeker(); + pipeline.traverseTopologically(seeker); + assertFalse(seeker.isOldTransformFound); + } } From f841098c54df1a42bb3c71dd208fbae4402073b1 Mon Sep 17 00:00:00 2001 From: Jing Date: Tue, 19 Dec 2023 10:26:55 -0800 Subject: [PATCH 156/224] Support DatabaseID in Datastore beam connector (#29815) * Fix routing header issue in Firestore Beam connector * Apply the lint * Support multi-db in datastore beam * Support multi-db in datastore beam * Remove databaseRef in WriteFn * retrigger checks * resolve comments * resolve comments * resolve comments * resolve comments * lint * Remove the PipelineOption reference * Update comment * Update ValueProviders usage * Update IT test to refer default database * Remove unused import --------- Co-authored-by: Sichen Liu --- .../sdk/io/gcp/datastore/DatastoreV1.java | 349 +++++++++++++++--- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 94 ++++- .../sdk/io/gcp/datastore/SplitQueryFnIT.java | 17 +- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 26 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 59 ++- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 11 +- 6 files changed, 458 insertions(+), 98 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index cdd003abae22..b94186b9893d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -126,8 +126,9 @@ *

    To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read} * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace - * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify - * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. + * to query within using {@link DatastoreV1.Read#withDatabase} or {@link + * DatastoreV1.Read#withNamespace}. You could also optionally specify how many splits you want for + * the query using {@link DatastoreV1.Read#withNumQuerySplits}. * *

    For example: * @@ -135,12 +136,14 @@ * // Read a query from Datastore * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); * Query query = ...; + * String databaseId = "..."; * String projectId = "..."; * * Pipeline p = Pipeline.create(options); * PCollection entities = p.apply( * DatastoreIO.v1().read() * .withProjectId(projectId) + * .withDatabaseId(databaseId) * .withQuery(query)); * } * @@ -156,7 +159,7 @@ * *

    {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1().write().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().write().withProjectId(projectId).withDatabaseId(databaseId));
      * p.run();
      * }
    * @@ -165,7 +168,7 @@ * *
    {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId).withDatabaseId(databaseId));
      * p.run();
      * }
    * @@ -174,7 +177,7 @@ * *
    {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId).withDatabaseId(databaseId));
      * p.run();
      * }
    * @@ -275,6 +278,9 @@ public class DatastoreV1 { Code.PERMISSION_DENIED, Code.UNAUTHENTICATED); + /** Database ID for the default database. */ + private static final String DEFAULT_DATABASE = ""; + /** * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using {@link @@ -313,6 +319,8 @@ public abstract static class Read extends PTransform public abstract @Nullable ValueProvider getProjectId(); + public abstract @Nullable ValueProvider getDatabaseId(); + public abstract @Nullable Query getQuery(); public abstract @Nullable ValueProvider getLiteralGqlQuery(); @@ -334,6 +342,8 @@ public abstract static class Read extends PTransform abstract static class Builder { abstract Builder setProjectId(ValueProvider projectId); + abstract Builder setDatabaseId(ValueProvider databaseId); + abstract Builder setQuery(Query query); abstract Builder setLiteralGqlQuery(ValueProvider literalGqlQuery); @@ -354,10 +364,16 @@ abstract static class Builder { * size from Cloud Datastore. */ static int getEstimatedNumSplits( - Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) { + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) { int numSplits; try { - long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace, readTime); + long estimatedSizeBytes = + getEstimatedSizeBytes(datastore, projectId, databaseId, query, namespace, readTime); LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes); numSplits = (int) @@ -378,7 +394,11 @@ static int getEstimatedNumSplits( * table. */ private static long queryLatestStatisticsTimestamp( - Datastore datastore, @Nullable String namespace, @Nullable Instant readTime) + Datastore datastore, + String projectId, + String databaseId, + @Nullable String namespace, + @Nullable Instant readTime) throws DatastoreException { Query.Builder query = Query.newBuilder(); // Note: namespace either being null or empty represents the default namespace, in which @@ -390,7 +410,8 @@ private static long queryLatestStatisticsTimestamp( } query.addOrder(makeOrder("timestamp", DESCENDING)); query.setLimit(Int32Value.newBuilder().setValue(1)); - RunQueryRequest request = makeRequest(query.build(), namespace, readTime); + RunQueryRequest request = + makeRequest(projectId, databaseId, query.build(), namespace, readTime); RunQueryResponse response = datastore.runQuery(request); QueryResultBatch batch = response.getBatch(); @@ -406,9 +427,15 @@ private static long queryLatestStatisticsTimestamp( * readTime specified, the latest statistics at or before readTime is retrieved. */ private static Entity getLatestTableStats( - String ourKind, @Nullable String namespace, Datastore datastore, @Nullable Instant readTime) + String projectId, + String databaseId, + String ourKind, + @Nullable String namespace, + Datastore datastore, + @Nullable Instant readTime) throws DatastoreException { - long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace, readTime); + long latestTimestamp = + queryLatestStatisticsTimestamp(datastore, projectId, databaseId, namespace, readTime); LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp); Query.Builder queryBuilder = Query.newBuilder(); @@ -423,7 +450,8 @@ private static Entity getLatestTableStats( makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(), makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build())); - RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); + RunQueryRequest request = + makeRequest(projectId, databaseId, queryBuilder.build(), namespace, readTime); long now = System.currentTimeMillis(); RunQueryResponse response = datastore.runQuery(request); @@ -447,10 +475,16 @@ private static Entity getLatestTableStats( *

    See https://cloud.google.com/datastore/docs/concepts/stats. */ static long getEstimatedSizeBytes( - Datastore datastore, Query query, @Nullable String namespace, @Nullable Instant readTime) + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) throws DatastoreException { String ourKind = query.getKind(0).getName(); - Entity entity = getLatestTableStats(ourKind, namespace, datastore, readTime); + Entity entity = + getLatestTableStats(projectId, databaseId, ourKind, namespace, datastore, readTime); return entity.getPropertiesOrThrow("entity_bytes").getIntegerValue(); } @@ -470,9 +504,18 @@ private static PartitionId.Builder forNamespace(@Nullable String namespace) { * the requested {@code readTime}. */ static RunQueryRequest makeRequest( - Query query, @Nullable String namespace, @Nullable Instant readTime) { + String projectId, + String databaseId, + Query query, + @Nullable String namespace, + @Nullable Instant readTime) { RunQueryRequest.Builder request = - RunQueryRequest.newBuilder().setQuery(query).setPartitionId(forNamespace(namespace)); + RunQueryRequest.newBuilder() + .setProjectId(projectId) + .setDatabaseId(databaseId) + .setQuery(query) + .setPartitionId( + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -486,11 +529,18 @@ static RunQueryRequest makeRequest( * at the requested {@code readTime}. */ static RunQueryRequest makeRequest( - GqlQuery gqlQuery, @Nullable String namespace, @Nullable Instant readTime) { + String projectId, + String databaseId, + GqlQuery gqlQuery, + @Nullable String namespace, + @Nullable Instant readTime) { RunQueryRequest.Builder request = RunQueryRequest.newBuilder() + .setProjectId(projectId) + .setDatabaseId(databaseId) .setGqlQuery(gqlQuery) - .setPartitionId(forNamespace(namespace)); + .setPartitionId( + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId)); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); request.setReadOptions(ReadOptions.newBuilder().setReadTime(readTimeProto).build()); @@ -504,6 +554,8 @@ static RunQueryRequest makeRequest( * namespace}. */ private static List splitQuery( + String projectId, + String databaseId, Query query, @Nullable String namespace, Datastore datastore, @@ -512,7 +564,8 @@ private static List splitQuery( @Nullable Instant readTime) throws DatastoreException { // If namespace is set, include it in the split request so splits are calculated accordingly. - PartitionId partitionId = forNamespace(namespace).build(); + PartitionId partitionId = + forNamespace(namespace).setProjectId(projectId).setDatabaseId(databaseId).build(); if (readTime != null) { Timestamp readTimeProto = Timestamps.fromMillis(readTime.getMillis()); return querySplitter.getSplits(query, partitionId, numSplits, datastore, readTimeProto); @@ -535,12 +588,18 @@ private static List splitQuery( */ @VisibleForTesting static Query translateGqlQueryWithLimitCheck( - String gql, Datastore datastore, String namespace, @Nullable Instant readTime) + String gql, + Datastore datastore, + String projectId, + String databaseId, + String namespace, + @Nullable Instant readTime) throws DatastoreException { String gqlQueryWithZeroLimit = gql + " LIMIT 0"; try { Query translatedQuery = - translateGqlQuery(gqlQueryWithZeroLimit, datastore, namespace, readTime); + translateGqlQuery( + gqlQueryWithZeroLimit, datastore, projectId, databaseId, namespace, readTime); // Clear the limit that we set. return translatedQuery.toBuilder().clearLimit().build(); } catch (DatastoreException e) { @@ -551,7 +610,7 @@ static Query translateGqlQueryWithLimitCheck( LOG.warn("Failed to translate Gql query '{}': {}", gqlQueryWithZeroLimit, e.getMessage()); LOG.warn("User query might have a limit already set, so trying without zero limit"); // Retry without the zero limit. - return translateGqlQuery(gql, datastore, namespace, readTime); + return translateGqlQuery(gql, datastore, projectId, databaseId, namespace, readTime); } else { throw e; } @@ -560,13 +619,27 @@ static Query translateGqlQueryWithLimitCheck( /** Translates a gql query string to {@link Query}. */ private static Query translateGqlQuery( - String gql, Datastore datastore, String namespace, @Nullable Instant readTime) + String gql, + Datastore datastore, + String projectId, + String databaseId, + String namespace, + @Nullable Instant readTime) throws DatastoreException { GqlQuery gqlQuery = GqlQuery.newBuilder().setQueryString(gql).setAllowLiterals(true).build(); - RunQueryRequest req = makeRequest(gqlQuery, namespace, readTime); + RunQueryRequest req = makeRequest(projectId, databaseId, gqlQuery, namespace, readTime); return datastore.runQuery(req).getQuery(); } + /** + * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified + * database. + */ + public DatastoreV1.Read withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return toBuilder().setDatabaseId(StaticValueProvider.of(databaseId)).build(); + } + /** * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified * project. @@ -677,13 +750,24 @@ public DatastoreV1.Read withReadTime(Instant readTime) { public long getNumEntities( PipelineOptions options, String ourKind, @Nullable String namespace) { try { - V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); + V1Options v1Options = + V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); V1DatastoreFactory datastoreFactory = new V1DatastoreFactory(); Datastore datastore = datastoreFactory.getDatastore( - options, v1Options.getProjectId(), v1Options.getLocalhost()); - - Entity entity = getLatestTableStats(ourKind, namespace, datastore, getReadTime()); + options, + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getLocalhost()); + + Entity entity = + getLatestTableStats( + v1Options.getProjectId(), + v1Options.getDatabaseId(), + ourKind, + namespace, + datastore, + getReadTime()); return entity.getPropertiesOrThrow("count").getIntegerValue(); } catch (Exception e) { return -1; @@ -704,7 +788,8 @@ public PCollection expand(PBegin input) { getQuery() == null || getLiteralGqlQuery() == null, "withQuery() and withLiteralGqlQuery() are exclusive"); - V1Options v1Options = V1Options.from(getProjectId(), getNamespace(), getLocalhost()); + V1Options v1Options = + V1Options.from(getProjectId(), getDatabaseId(), getNamespace(), getLocalhost()); /* * This composite transform involves the following steps: @@ -748,6 +833,7 @@ public void populateDisplayData(DisplayData.Builder builder) { String query = getQuery() == null ? null : getQuery().toString(); builder .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("ProjectId")) + .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()).withLabel("DatabaseId")) .addIfNotNull(DisplayData.item("namespace", getNamespace()).withLabel("Namespace")) .addIfNotNull(DisplayData.item("query", query).withLabel("Query")) .addIfNotNull(DisplayData.item("gqlQuery", getLiteralGqlQuery()).withLabel("GqlQuery")) @@ -757,30 +843,46 @@ public void populateDisplayData(DisplayData.Builder builder) { @VisibleForTesting static class V1Options implements HasDisplayData, Serializable { private final ValueProvider project; + private final ValueProvider database; private final @Nullable ValueProvider namespace; private final @Nullable String localhost; private V1Options( - ValueProvider project, ValueProvider namespace, String localhost) { + ValueProvider project, + ValueProvider database, + ValueProvider namespace, + String localhost) { this.project = project; + this.database = database; this.namespace = namespace; this.localhost = localhost; } - public static V1Options from(String projectId, String namespace, String localhost) { + public static V1Options from( + String projectId, ValueProvider databaseId, String namespace, String localhost) { return from( - StaticValueProvider.of(projectId), StaticValueProvider.of(namespace), localhost); + StaticValueProvider.of(projectId), + databaseId, + StaticValueProvider.of(namespace), + localhost); } public static V1Options from( - ValueProvider project, ValueProvider namespace, String localhost) { - return new V1Options(project, namespace, localhost); + ValueProvider project, + ValueProvider databaseId, + ValueProvider namespace, + String localhost) { + return new V1Options(project, databaseId, namespace, localhost); } public String getProjectId() { return project.get(); } + public String getDatabaseId() { + return database == null ? DEFAULT_DATABASE : database.get(); + } + public @Nullable String getNamespace() { return namespace == null ? null : namespace.get(); } @@ -789,6 +891,10 @@ public ValueProvider getProjectValueProvider() { return project; } + public ValueProvider getDatabaseValueProvider() { + return database; + } + public @Nullable ValueProvider getNamespaceValueProvider() { return namespace; } @@ -802,6 +908,8 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .addIfNotNull( DisplayData.item("projectId", getProjectValueProvider()).withLabel("ProjectId")) + .addIfNotNull( + DisplayData.item("databaseId", getDatabaseValueProvider()).withLabel("DatabaseId")) .addIfNotNull( DisplayData.item("namespace", getNamespaceValueProvider()).withLabel("Namespace")); } @@ -833,7 +941,10 @@ static class GqlQueryTranslateFn extends DoFn { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), v1Options.getProjectId(), v1Options.getLocalhost()); + c.getPipelineOptions(), + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getLocalhost()); } @ProcessElement @@ -842,7 +953,12 @@ public void processElement(ProcessContext c) throws Exception { LOG.info("User query: '{}'", gqlQuery); Query query = translateGqlQueryWithLimitCheck( - gqlQuery, datastore, v1Options.getNamespace(), readTime); + gqlQuery, + datastore, + v1Options.getProjectId(), + v1Options.getDatabaseId(), + v1Options.getNamespace(), + readTime); LOG.info("User gql query translated to Query({})", query); c.output(query); } @@ -890,7 +1006,10 @@ public SplitQueryFn(V1Options options, int numSplits, @Nullable Instant readTime public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); + c.getPipelineOptions(), + options.getProjectId(), + options.getDatabaseId(), + options.getLocalhost()); querySplitter = datastoreFactory.getQuerySplitter(); } @@ -908,7 +1027,13 @@ public void processElement(ProcessContext c) throws Exception { // Compute the estimated numSplits if numSplits is not specified by the user. if (numSplits <= 0) { estimatedNumSplits = - getEstimatedNumSplits(datastore, query, options.getNamespace(), readTime); + getEstimatedNumSplits( + datastore, + options.getProjectId(), + options.getDatabaseId(), + query, + options.getNamespace(), + readTime); } else { estimatedNumSplits = numSplits; } @@ -918,6 +1043,8 @@ public void processElement(ProcessContext c) throws Exception { try { querySplits = splitQuery( + options.getProjectId(), + options.getDatabaseId(), query, options.getNamespace(), datastore, @@ -985,7 +1112,10 @@ public ReadFn(V1Options options, @Nullable Instant readTime) { public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore( - c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); + c.getPipelineOptions(), + options.getProjectId(), + options.getDatabaseId(), + options.getLocalhost()); } private RunQueryResponse runQueryWithRetries(RunQueryRequest request) throws Exception { @@ -1045,7 +1175,13 @@ public void processElement(ProcessContext context) throws Exception { queryBuilder.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = makeRequest(queryBuilder.build(), namespace, readTime); + RunQueryRequest request = + makeRequest( + options.getProjectId(), + options.getDatabaseId(), + queryBuilder.build(), + namespace, + readTime); RunQueryResponse response = runQueryWithRetries(request); currentBatch = response.getBatch(); @@ -1128,21 +1264,42 @@ public static class Write extends Mutate { @Nullable String localhost, boolean throttleRampup, ValueProvider hintNumWorkers) { - super(projectId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); + } + + Write( + @Nullable ValueProvider projectId, + @Nullable ValueProvider databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider hintNumWorkers) { + super(projectId, databaseId, localhost, new UpsertFn(), throttleRampup, hintNumWorkers); } - /** Returns a new {@link Write} that writes to the Cloud Datastore for the specified project. */ + /** Returns a new {@link Write} that writes to the Cloud Datastore for the default database. */ public Write withProjectId(String projectId) { checkArgument(projectId != null, "projectId can not be null"); return withProjectId(StaticValueProvider.of(projectId)); } + /** Returns a new {@link Write} that writes to the Cloud Datastore for the database id. */ + public Write withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** Same as {@link Write#withProjectId(String)} but with a {@link ValueProvider}. */ public Write withProjectId(ValueProvider projectId) { checkArgument(projectId != null, "projectId can not be null"); return new Write(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link Write#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public Write withDatabaseId(ValueProvider databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new Write(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** * Returns a new {@link Write} that writes to the Cloud Datastore Emulator running locally on * the specified host port. @@ -1188,7 +1345,16 @@ public static class DeleteEntity extends Mutate { @Nullable String localhost, boolean throttleRampup, ValueProvider hintNumWorkers) { - super(projectId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); + } + + DeleteEntity( + @Nullable ValueProvider projectId, + @Nullable ValueProvider databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider hintNumWorkers) { + super(projectId, databaseId, localhost, new DeleteEntityFn(), throttleRampup, hintNumWorkers); } /** @@ -1200,12 +1366,27 @@ public DeleteEntity withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } + /** + * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore for the + * specified database. + */ + public DeleteEntity withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** Same as {@link DeleteEntity#withProjectId(String)} but with a {@link ValueProvider}. */ public DeleteEntity withProjectId(ValueProvider projectId) { checkArgument(projectId != null, "projectId can not be null"); return new DeleteEntity(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link DeleteEntity#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public DeleteEntity withDatabaseId(ValueProvider databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new DeleteEntity(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1253,7 +1434,16 @@ public static class DeleteKey extends Mutate { @Nullable String localhost, boolean throttleRampup, ValueProvider hintNumWorkers) { - super(projectId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); + super(projectId, null, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); + } + + DeleteKey( + @Nullable ValueProvider projectId, + @Nullable ValueProvider databaseId, + @Nullable String localhost, + boolean throttleRampup, + ValueProvider hintNumWorkers) { + super(projectId, databaseId, localhost, new DeleteKeyFn(), throttleRampup, hintNumWorkers); } /** @@ -1265,6 +1455,15 @@ public DeleteKey withProjectId(String projectId) { return withProjectId(StaticValueProvider.of(projectId)); } + /** + * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore for the + * specified database. + */ + public DeleteKey withDatabaseId(String databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return withDatabaseId(StaticValueProvider.of(databaseId)); + } + /** * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore Emulator * running locally on the specified host port. @@ -1280,6 +1479,12 @@ public DeleteKey withProjectId(ValueProvider projectId) { return new DeleteKey(projectId, localhost, throttleRampup, hintNumWorkers); } + /** Same as {@link DeleteKey#withDatabaseId(String)} but with a {@link ValueProvider}. */ + public DeleteKey withDatabaseId(ValueProvider databaseId) { + checkArgument(databaseId != null, "databaseId can not be null"); + return new DeleteKey(projectId, databaseId, localhost, throttleRampup, hintNumWorkers); + } + /** Returns a new {@link DeleteKey} that does not throttle during ramp-up. */ public DeleteKey withRampupThrottlingDisabled() { return new DeleteKey(projectId, localhost, false, hintNumWorkers); @@ -1312,6 +1517,7 @@ public DeleteKey withHintNumWorkers(ValueProvider hintNumWorkers) { private abstract static class Mutate extends PTransform, PDone> { protected ValueProvider projectId; + protected ValueProvider databaseId; protected @Nullable String localhost; protected boolean throttleRampup; protected ValueProvider hintNumWorkers; @@ -1326,11 +1532,13 @@ private abstract static class Mutate extends PTransform, PDone */ Mutate( @Nullable ValueProvider projectId, + @Nullable ValueProvider databaseId, @Nullable String localhost, SimpleFunction mutationFn, boolean throttleRampup, ValueProvider hintNumWorkers) { this.projectId = projectId; + this.databaseId = databaseId; this.localhost = localhost; this.throttleRampup = throttleRampup; this.hintNumWorkers = hintNumWorkers; @@ -1372,7 +1580,14 @@ public PCollectionView expand(PBegin input) { ParDo.of(rampupThrottlingFn).withSideInputs(startTimestampView)); } intermediateOutput.apply( - "Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId, localhost))); + "Write Mutation to Datastore", + ParDo.of( + new DatastoreWriterFn( + projectId, + databaseId, + localhost, + new V1DatastoreFactory(), + new WriteBatcherImpl()))); return PDone.in(input.getPipeline()); } @@ -1390,6 +1605,7 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId).withLabel("Output Project")) + .addIfNotNull(DisplayData.item("databaseId", databaseId).withLabel("Output Database")) .include("mutationFn", mutationFn); if (rampupThrottlingFn != null) { builder.include("rampupThrottlingFn", rampupThrottlingFn); @@ -1399,6 +1615,10 @@ public void populateDisplayData(DisplayData.Builder builder) { public String getProjectId() { return projectId.get(); } + + public String getDatabaseId() { + return databaseId.get(); + } } /** Determines batch sizes for commit RPCs. */ @@ -1480,6 +1700,7 @@ static class DatastoreWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); private final ValueProvider projectId; + private final ValueProvider databaseId; private final @Nullable String localhost; private transient Datastore datastore; private final V1DatastoreFactory datastoreFactory; @@ -1511,13 +1732,14 @@ static class DatastoreWriterFn extends DoFn { DatastoreWriterFn(String projectId, @Nullable String localhost) { this( StaticValueProvider.of(projectId), + null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } DatastoreWriterFn(ValueProvider projectId, @Nullable String localhost) { - this(projectId, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); + this(projectId, null, localhost, new V1DatastoreFactory(), new WriteBatcherImpl()); } @VisibleForTesting @@ -1526,7 +1748,18 @@ static class DatastoreWriterFn extends DoFn { @Nullable String localhost, V1DatastoreFactory datastoreFactory, WriteBatcher writeBatcher) { + this(projectId, null, localhost, datastoreFactory, writeBatcher); + } + + @VisibleForTesting + DatastoreWriterFn( + ValueProvider projectId, + ValueProvider databaseId, + @Nullable String localhost, + V1DatastoreFactory datastoreFactory, + WriteBatcher writeBatcher) { this.projectId = checkNotNull(projectId, "projectId"); + this.databaseId = databaseId; this.localhost = localhost; this.datastoreFactory = datastoreFactory; this.writeBatcher = writeBatcher; @@ -1534,7 +1767,10 @@ static class DatastoreWriterFn extends DoFn { @StartBundle public void startBundle(StartBundleContext c) { - datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId.get(), localhost); + String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); + datastore = + datastoreFactory.getDatastore( + c.getPipelineOptions(), projectId.get(), databaseIdOrDefaultDatabase, localhost); writeBatcher.start(); if (adaptiveThrottler == null) { // Initialize throttler at first use, because it is not serializable. @@ -1602,11 +1838,14 @@ private synchronized void flushBatch() batchSize.update(mutations.size()); + String databaseIdOrDefaultDatabase = databaseId == null ? DEFAULT_DATABASE : databaseId.get(); while (true) { // Batch upsert entities. CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.addAllMutations(mutations); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + commitRequest.setProjectId(projectId.get()); + commitRequest.setDatabaseId(databaseIdOrDefaultDatabase); long startTime = System.currentTimeMillis(), endTime; if (adaptiveThrottler.throttleRequest(startTime)) { @@ -1628,6 +1867,7 @@ private synchronized void flushBatch() ServiceCallMetric serviceCallMetric = new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); try { + datastore.commit(commitRequest.build()); endTime = System.currentTimeMillis(); serviceCallMetric.call("ok"); @@ -1768,8 +2008,9 @@ public void populateDisplayData(DisplayData.Builder builder) { static class V1DatastoreFactory implements Serializable { /** Builds a Cloud Datastore client for the given pipeline options and project. */ - public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { - return getDatastore(pipelineOptions, projectId, null); + public Datastore getDatastore( + PipelineOptions pipelineOptions, String projectId, String databaseId) { + return getDatastore(pipelineOptions, projectId, databaseId, null); } /** @@ -1777,7 +2018,10 @@ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) * locahost. */ public Datastore getDatastore( - PipelineOptions pipelineOptions, String projectId, @Nullable String localhost) { + PipelineOptions pipelineOptions, + String projectId, + String databaseId, + @Nullable String localhost) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); // Add Beam version to user agent header. @@ -1797,7 +2041,10 @@ public Datastore getDatastore( } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); + new DatastoreOptions.Builder() + .projectId(projectId) + .databaseId(databaseId) + .initializer(initializer); if (localhost != null) { builder.localHost(localhost); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 3280e17998e4..ab6f5567dd9a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -125,6 +125,7 @@ @RunWith(Enclosed.class) public class DatastoreV1Test { private static final String PROJECT_ID = "testProject"; + private static final String DATABASE_ID = ""; private static final String NAMESPACE = "testNamespace"; private static final String KIND = "testKind"; private static final Query QUERY; @@ -138,7 +139,7 @@ public class DatastoreV1Test { Query.Builder q = Query.newBuilder(); q.addKindBuilder().setName(KIND); QUERY = q.build(); - V_1_OPTIONS = V1Options.from(PROJECT_ID, NAMESPACE, null); + V_1_OPTIONS = V1Options.from(PROJECT_ID, StaticValueProvider.of(DATABASE_ID), NAMESPACE, null); } @Mock protected Datastore mockDatastore; @@ -153,6 +154,9 @@ public void setUp() { when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class), any())) .thenReturn(mockDatastore); + when(mockDatastoreFactory.getDatastore( + any(PipelineOptions.class), any(String.class), any(String.class), any())) + .thenReturn(mockDatastore); when(mockDatastoreFactory.getQuerySplitter()).thenReturn(mockQuerySplitter); // Setup the ProcessWideContainer for testing metrics are set. MetricsContainerImpl container = new MetricsContainerImpl(null); @@ -167,10 +171,12 @@ public void testBuildRead() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -180,11 +186,13 @@ public void testBuildReadWithReadTime() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withReadTime(TIMESTAMP); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); } @Test @@ -193,10 +201,12 @@ public void testBuildReadWithGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE); assertEquals(GQL_QUERY, read.getLiteralGqlQuery().get()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); } @@ -210,10 +220,12 @@ public void testBuildReadAlt() throws Exception { .withQuery(QUERY) .withNamespace(NAMESPACE) .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLocalhost(LOCALHOST); assertEquals(TIMESTAMP, read.getReadTime()); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId().get()); + assertEquals(DATABASE_ID, read.getDatabaseId().get()); assertEquals(NAMESPACE, read.getNamespace().get()); assertEquals(LOCALHOST, read.getLocalhost()); } @@ -224,6 +236,7 @@ public void testReadValidationFailsQueryAndGqlQuery() throws Exception { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withQuery(QUERY); @@ -257,6 +270,7 @@ public void testReadDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -264,6 +278,7 @@ public void testReadDisplayData() { DisplayData displayData = DisplayData.from(read); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); assertThat(displayData, hasDisplayItem("query", QUERY.toString())); assertThat(displayData, hasDisplayItem("namespace", NAMESPACE)); assertThat(displayData, hasDisplayItem("readTime", TIMESTAMP)); @@ -275,6 +290,7 @@ public void testReadDisplayDataWithGqlQuery() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withLiteralGqlQuery(GQL_QUERY) .withNamespace(NAMESPACE) .withReadTime(TIMESTAMP); @@ -295,6 +311,7 @@ public void testSourcePrimitiveDisplayData() { DatastoreIO.v1() .read() .withProjectId(PROJECT_ID) + .withDatabaseId(DATABASE_ID) .withQuery(Query.newBuilder().build()) .withNumQuerySplits(numSplits); @@ -313,29 +330,34 @@ public void testSourcePrimitiveDisplayData() { @Test public void testWriteDisplayData() { - Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteEntityDisplayData() { - DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = + DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(deleteEntity); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test public void testDeleteKeyDisplayData() { - DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = + DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID).withDatabaseId(DATABASE_ID); DisplayData displayData = DisplayData.from(deleteKey); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + assertThat(displayData, hasDisplayItem("databaseId", DATABASE_ID)); } @Test @@ -600,6 +622,8 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { while (start < numMutations) { int end = Math.min(numMutations, start + DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -644,6 +668,8 @@ public void testDatastoreWriterFnWithLargeEntities() throws Exception { while (start < mutations.size()) { int end = Math.min(mutations.size(), start + entitiesPerRpc); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. @@ -681,12 +707,16 @@ public void testDatastoreWriterFnWithDuplicateEntities() throws Exception { CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(0, 2)); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); // second invocation has key [0, 2] because the second 0 triggered a flush batch commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(2, 4)); + commitRequest.setProjectId(PROJECT_ID); + commitRequest.setDatabaseId(DATABASE_ID); verify(mockDatastore, times(1)).commit(commitRequest.build()); verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2); } @@ -841,17 +871,26 @@ public void testEstimatedSizeBytes() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, + DATABASE_ID, + makeStatKindQuery(NAMESPACE, timestamp), + NAMESPACE, + readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)).thenReturn(statResponse); - assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE, readTime)); + assertEquals( + entityBytes, + getEstimatedSizeBytes( + mockDatastore, PROJECT_ID, DATABASE_ID, QUERY, NAMESPACE, readTime)); verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -917,12 +956,18 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { long timestamp = 1234L; RunQueryRequest latestTimestampRequest = - makeRequest(makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, makeLatestTimestampQuery(NAMESPACE), NAMESPACE, readTime); RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response RunQueryRequest statRequest = - makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE, readTime); + makeRequest( + PROJECT_ID, + DATABASE_ID, + makeStatKindQuery(NAMESPACE, timestamp), + NAMESPACE, + readTime); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); when(mockDatastore.runQuery(latestTimestampRequest)).thenReturn(latestTimestampResponse); @@ -1030,9 +1075,11 @@ public void testTranslateGqlQueryWithLimit() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); - RunQueryRequest gqlRequest = makeRequest(gqlQuery, V_1_OPTIONS.getNamespace(), readTime); + RunQueryRequest gqlRequest = + makeRequest(PROJECT_ID, DATABASE_ID, gqlQuery, V_1_OPTIONS.getNamespace(), readTime); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow( new DatastoreException( @@ -1044,7 +1091,8 @@ public void testTranslateGqlQueryWithLimit() throws Exception { when(mockDatastore.runQuery(gqlRequest)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), + translateGqlQueryWithLimitCheck( + gql, mockDatastore, PROJECT_ID, DATABASE_ID, V_1_OPTIONS.getNamespace(), readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequest); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); @@ -1058,11 +1106,18 @@ public void testTranslateGqlQueryWithNoLimit() throws Exception { GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenReturn(RunQueryResponse.newBuilder().setQuery(QUERY).build()); assertEquals( - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime), + translateGqlQueryWithLimitCheck( + gql, + mockDatastore, + V_1_OPTIONS.getProjectId(), + V_1_OPTIONS.getDatabaseId(), + V_1_OPTIONS.getNamespace(), + readTime), QUERY); verify(mockDatastore, times(1)).runQuery(gqlRequestWithZeroLimit); } @@ -1074,13 +1129,20 @@ public void testTranslateGqlQueryWithException() throws Exception { GqlQuery gqlQueryWithZeroLimit = GqlQuery.newBuilder().setQueryString(gqlWithZeroLimit).setAllowLiterals(true).build(); RunQueryRequest gqlRequestWithZeroLimit = - makeRequest(gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); + makeRequest( + PROJECT_ID, DATABASE_ID, gqlQueryWithZeroLimit, V_1_OPTIONS.getNamespace(), readTime); when(mockDatastore.runQuery(gqlRequestWithZeroLimit)) .thenThrow(new RuntimeException("TestException")); thrown.expect(RuntimeException.class); thrown.expectMessage("TestException"); - translateGqlQueryWithLimitCheck(gql, mockDatastore, V_1_OPTIONS.getNamespace(), readTime); + translateGqlQueryWithLimitCheck( + gql, + mockDatastore, + V_1_OPTIONS.getProjectId(), + V_1_OPTIONS.getDatabaseId(), + V_1_OPTIONS.getNamespace(), + readTime); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java index ea00821f3604..913e05c86129 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.DoFnTester; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -58,29 +59,32 @@ public class SplitQueryFnIT { @Test public void testSplitQueryFnWithLargeDataset() throws Exception { String projectId = "apache-beam-testing"; + String databaseId = ""; String kind = "sort_1G"; String namespace = null; // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by // Datastore stats. int expectedNumSplits = 32; - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); } /** Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset. */ @Test public void testSplitQueryFnWithSmallDataset() throws Exception { String projectId = "apache-beam-testing"; + String databaseId = ""; String kind = "shakespeare"; String namespace = null; int expectedNumSplits = NUM_QUERY_SPLITS_MIN; - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, null); - testSplitQueryFn(projectId, kind, namespace, expectedNumSplits, readTime); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, null); + testSplitQueryFn(projectId, databaseId, kind, namespace, expectedNumSplits, readTime); } /** A helper method to test {@link SplitQueryFn} to generate the expected number of splits. */ private void testSplitQueryFn( String projectId, + String databaseId, String kind, @Nullable String namespace, int expectedNumSplits, @@ -90,7 +94,10 @@ private void testSplitQueryFn( query.addKindBuilder().setName(kind); SplitQueryFn splitQueryFn = - new SplitQueryFn(V1Options.from(projectId, namespace, null), 0, readTime); + new SplitQueryFn( + V1Options.from(projectId, StaticValueProvider.of(databaseId), namespace, null), + 0, + readTime); DoFnTester doFnTester = DoFnTester.of(splitQueryFn); List queries = doFnTester.processBundle(query.build()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index 249cadd48866..c9530ba43cb0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -48,6 +48,7 @@ public class V1ReadIT { private V1TestOptions options; private String project; + private String database; private String ancestor; private final long numEntitiesBeforeReadTime = 600; private final long totalNumEntities = 1000; @@ -58,9 +59,12 @@ public void setup() throws Exception { PipelineOptionsFactory.register(V1TestOptions.class); options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + // The default database. + database = ""; + ancestor = UUID.randomUUID().toString(); // Create entities and write them to datastore - writeEntitiesToDatastore(options, project, ancestor, 0, numEntitiesBeforeReadTime); + writeEntitiesToDatastore(options, project, database, ancestor, 0, numEntitiesBeforeReadTime); Thread.sleep(1000); readTime = Instant.now(); @@ -68,12 +72,12 @@ public void setup() throws Exception { long moreEntitiesToWrite = totalNumEntities - numEntitiesBeforeReadTime; writeEntitiesToDatastore( - options, project, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); + options, project, database, ancestor, numEntitiesBeforeReadTime, moreEntitiesToWrite); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, ancestor); + deleteAllEntities(options, project, database, ancestor); } /** @@ -93,6 +97,7 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()); @@ -108,6 +113,7 @@ public void testE2EV1Read() throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withQuery(query) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -152,6 +158,7 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()); @@ -167,6 +174,7 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { DatastoreIO.v1() .read() .withProjectId(project) + .withDatabaseId(database) .withLiteralGqlQuery(gqlQuery) .withNamespace(options.getNamespace()) .withReadTime(readTime); @@ -181,11 +189,17 @@ private void testE2EV1ReadWithGQLQuery(long limit) throws Exception { // Creates entities and write them to datastore private static void writeEntitiesToDatastore( - V1TestOptions options, String project, String ancestor, long valueOffset, long numEntities) + V1TestOptions options, + String project, + String database, + String ancestor, + long valueOffset, + long numEntities) throws Exception { - Datastore datastore = getDatastore(options, project); + Datastore datastore = getDatastore(options, project, database); // Write test entities to datastore - V1TestWriter writer = new V1TestWriter(datastore, new UpsertMutationBuilder()); + V1TestWriter writer = + new V1TestWriter(datastore, project, database, new UpsertMutationBuilder()); Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index e3057c4b90ca..0cb3ffac3222 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -137,10 +137,12 @@ public void processElement(ProcessContext c) throws Exception { } /** Build a new datastore client. */ - static Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { + static Datastore getDatastore( + PipelineOptions pipelineOptions, String projectId, String databaseId) { Credentials credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); HttpRequestInitializer initializer; if (credential != null) { + initializer = new ChainingHttpRequestInitializer( new HttpCredentialsAdapter(credential), new RetryHttpRequestInitializer()); @@ -149,14 +151,22 @@ static Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) } DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().projectId(projectId).initializer(initializer); + new DatastoreOptions.Builder() + .projectId(projectId) + .databaseId(databaseId) + .initializer(initializer); return DatastoreFactory.get().create(builder.build()); } /** Build a datastore query request. */ - private static RunQueryRequest makeRequest(Query query, @Nullable String namespace) { - RunQueryRequest.Builder requestBuilder = RunQueryRequest.newBuilder().setQuery(query); + private static RunQueryRequest makeRequest( + String projectId, String databaseId, Query query, @Nullable String namespace) { + RunQueryRequest.Builder requestBuilder = + RunQueryRequest.newBuilder() + .setQuery(query) + .setProjectId(projectId) + .setDatabaseId(databaseId); if (namespace != null) { requestBuilder.getPartitionIdBuilder().setNamespaceId(namespace); } @@ -164,14 +174,16 @@ private static RunQueryRequest makeRequest(Query query, @Nullable String namespa } /** Delete all entities with the given ancestor. */ - static void deleteAllEntities(V1TestOptions options, String project, String ancestor) - throws Exception { - Datastore datastore = getDatastore(options, project); + static void deleteAllEntities( + V1TestOptions options, String project, String database, String ancestor) throws Exception { + Datastore datastore = getDatastore(options, project, database); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); - V1TestWriter writer = new V1TestWriter(datastore, new DeleteMutationBuilder()); + V1TestReader reader = + new V1TestReader(datastore, project, database, query, options.getNamespace()); + V1TestWriter writer = + new V1TestWriter(datastore, project, database, new DeleteMutationBuilder()); long numEntities = 0; while (reader.advance()) { @@ -185,14 +197,15 @@ static void deleteAllEntities(V1TestOptions options, String project, String ance } /** Returns the total number of entities for the given datastore. */ - static long countEntities(V1TestOptions options, String project, String ancestor) + static long countEntities(V1TestOptions options, String project, String database, String ancestor) throws Exception { // Read from datastore. - Datastore datastore = V1TestUtil.getDatastore(options, project); + Datastore datastore = V1TestUtil.getDatastore(options, project, database); Query query = V1TestUtil.makeAncestorKindQuery(options.getKind(), options.getNamespace(), ancestor); - V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); + V1TestReader reader = + new V1TestReader(datastore, project, database, query, options.getNamespace()); long numEntitiesRead = 0; while (reader.advance()) { @@ -247,12 +260,17 @@ static boolean isValidKey(Key key) { return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); } + private final String projectId; + private final String databaseId; private final Datastore datastore; private final MutationBuilder mutationBuilder; private final List entities = new ArrayList<>(); - V1TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { + V1TestWriter( + Datastore datastore, String projectId, String databaseId, MutationBuilder mutationBuilder) { this.datastore = datastore; + this.projectId = projectId; + this.databaseId = databaseId; this.mutationBuilder = mutationBuilder; } @@ -295,6 +313,8 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc commitRequest.addMutations(mutationBuilder.apply(entity)); } commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + commitRequest.setProjectId(projectId); + commitRequest.setDatabaseId(databaseId); datastore.commit(commitRequest.build()); // Break if the commit threw no exception. break; @@ -318,6 +338,8 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc static class V1TestReader { private static final int QUERY_BATCH_LIMIT = 500; private final Datastore datastore; + private final String projectId; + private final String databaseId; private final Query query; private final @Nullable String namespace; private boolean moreResults; @@ -326,8 +348,15 @@ static class V1TestReader { private QueryResultBatch currentBatch; private Entity currentEntity; - V1TestReader(Datastore datastore, Query query, @Nullable String namespace) { + V1TestReader( + Datastore datastore, + String projectId, + String databaseId, + Query query, + @Nullable String namespace) { this.datastore = datastore; + this.projectId = projectId; + this.databaseId = databaseId; this.query = query; this.namespace = namespace; } @@ -362,7 +391,7 @@ private Iterator getIteratorAndMoveCursor() throws DatastoreExcept query.setStartCursor(currentBatch.getEndCursor()); } - RunQueryRequest request = makeRequest(query.build(), namespace); + RunQueryRequest request = makeRequest(projectId, databaseId, query.build(), namespace); RunQueryResponse response = datastore.runQuery(request); currentBatch = response.getBatch(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 4f794a03e58d..0062208630f6 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -56,6 +56,7 @@ public class V1WriteIT { private V1TestOptions options; private String project; + private String database = ""; private String ancestor; private final long numEntities = 1000; @@ -83,12 +84,12 @@ public void testE2EV1Write() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, 0))) - .apply(DatastoreIO.v1().write().withProjectId(project)); + .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, ancestor); + long numEntitiesWritten = countEntities(options, project, database, ancestor); assertEquals(numEntities, numEntitiesWritten); } @@ -185,18 +186,18 @@ public void testE2EV1WriteWithLargeEntities() throws Exception { ParDo.of( new V1TestUtil.CreateEntityFn( options.getKind(), options.getNamespace(), ancestor, rawPropertySize))) - .apply(DatastoreIO.v1().write().withProjectId(project)); + .apply(DatastoreIO.v1().write().withProjectId(project).withDatabaseId(database)); p.run(); // Count number of entities written to datastore. - long numEntitiesWritten = countEntities(options, project, ancestor); + long numEntitiesWritten = countEntities(options, project, database, ancestor); assertEquals(numLargeEntities, numEntitiesWritten); } @After public void tearDown() throws Exception { - deleteAllEntities(options, project, ancestor); + deleteAllEntities(options, project, database, ancestor); } } From 1a7174c68bdeb7819dd55031aaa8e4757f35dcaf Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 19 Dec 2023 15:09:19 -0500 Subject: [PATCH 157/224] Build and publish multi-arch wheels separately from main wheels (#29821) * Build and publish arch wheels separately from main wheels * Better postfix * Better postfix --- .github/workflows/build_wheels.yml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 85ed7c0686c3..82924c3abcbd 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -280,7 +280,7 @@ jobs: - name: Upload wheels as artifacts uses: actions/upload-artifact@v4 with: - name: wheelhouse-${{ matrix.os_python.os }} + name: wheelhouse-${{ matrix.os_python.os }}${{ (matrix.arch == 'aarch64' && '-aarch64') || '' }} path: apache-beam-source/wheelhouse/ - name: Build RC wheels if: ${{ needs.build_source.outputs.is_rc == 1 }} @@ -305,7 +305,7 @@ jobs: if: ${{ needs.build_source.outputs.is_rc == 1 }} uses: actions/upload-artifact@v4 with: - name: wheelhouse-rc${{ needs.build_source.outputs.rc_num }}-${{ matrix.os_python.os }} + name: wheelhouse-rc${{ needs.build_source.outputs.rc_num }}-${{ matrix.os_python.os }}${{ (matrix.arch == 'aarch64' && '-aarch64') || '' }} path: apache-beam-source-rc/wheelhouse/ upload_wheels_to_gcs: @@ -318,11 +318,15 @@ jobs: strategy: matrix: os : [ubuntu-latest, macos-latest, windows-latest] + arch: [auto] + include: + - os: "ubuntu-latest" + arch: aarch64 steps: - name: Download wheels from artifacts uses: actions/download-artifact@v4 with: - name: wheelhouse-${{ matrix.os }} + name: wheelhouse-${{ matrix.os }}${{ (matrix.arch == 'aarch64' && '-aarch64') || '' }} path: wheelhouse/ - name: Authenticate on GCP uses: google-github-actions/setup-gcloud@v0 From c5d53d61e78a69aa09f219f953e43440d1670357 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 15 Dec 2023 00:21:33 -0800 Subject: [PATCH 158/224] Revert to non-schema-based Sql transform. Issues with unknown logical types. --- sdks/python/apache_beam/yaml/standard_providers.yaml | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/standard_providers.yaml b/sdks/python/apache_beam/yaml/standard_providers.yaml index 0f1bc14c47c4..67ab8da502f8 100644 --- a/sdks/python/apache_beam/yaml/standard_providers.yaml +++ b/sdks/python/apache_beam/yaml/standard_providers.yaml @@ -18,9 +18,15 @@ # TODO(robertwb): Add more providers. # TODO(robertwb): Perhaps auto-generate this file? +- type: 'beamJar' + config: + gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar' + version: BEAM_VERSION + transforms: + Sql: 'beam:external:java:sql:v1' + - type: renaming transforms: - 'Sql': 'Sql' 'MapToFields-java': 'MapToFields-java' 'MapToFields-generic': 'MapToFields-java' 'Filter-java': 'Filter-java' From b9a62f0f7931d9e52e18683e20f7f266349e8393 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 14 Dec 2023 23:50:56 -0800 Subject: [PATCH 159/224] [YAML] Properly deligate methods for renaming provider. --- sdks/python/apache_beam/yaml/yaml_provider.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index f6078769c654..2b7b76efab82 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -852,10 +852,11 @@ def with_name( ]) def description(self, typ): - return self._underlying_provider.description(typ) + return self._underlying_provider.description(self._transforms[typ]) def requires_inputs(self, typ, args): - return self._underlying_provider.requires_inputs(typ, args) + return self._underlying_provider.requires_inputs( + self._transforms[typ], args) def create_transform( self, From f227175a4771fcabe8676e67dedc4a6018f5a88d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 15 Dec 2023 02:33:40 -0800 Subject: [PATCH 160/224] Allow pulling default beam services from an environment variable. This brings Python in line with typescript. --- .../apache_beam/options/pipeline_options.py | 10 ++++-- .../options/pipeline_options_test.py | 33 +++++++++++++++++++ 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index b02aa7da7de1..7e5954fa820d 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -572,12 +572,18 @@ def _add_argparse_args(cls, parser): class CrossLanguageOptions(PipelineOptions): + @staticmethod + def _beam_services_from_enviroment(): + return json.loads(os.environ.get('BEAM_SERVICE_OVERRIDES') or '{}') + @classmethod def _add_argparse_args(cls, parser): parser.add_argument( '--beam_services', - type=json.loads, - default={}, + type=lambda s: { + **cls._beam_services_from_enviroment(), **json.loads(s) + }, + default=cls._beam_services_from_enviroment(), help=( 'For convenience, Beam provides the ability to automatically ' 'download and start various services (such as expansion services) ' diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 2d85d055eafb..1d77c6d89f16 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -21,11 +21,14 @@ import json import logging +import mock +import os import unittest import hamcrest as hc from parameterized import parameterized +from apache_beam.options.pipeline_options import CrossLanguageOptions from apache_beam.options.pipeline_options import DebugOptions from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions @@ -395,6 +398,36 @@ def test_worker_options(self): self.assertEqual(worker_options.machine_type, 'abc') self.assertEqual(worker_options.disk_type, 'def') + def test_beam_services_empty(self): + with mock.patch.dict(os.environ, {}, clear=True): + options = PipelineOptions().view_as(CrossLanguageOptions) + self.assertEqual(options.beam_services, {}) + + def test_beam_services_from_env(self): + with mock.patch.dict(os.environ, + {'BEAM_SERVICE_OVERRIDES': '{"foo": "bar"}'}, + clear=True): + options = PipelineOptions().view_as(CrossLanguageOptions) + self.assertEqual(options.beam_services, {'foo': 'bar'}) + + def test_beam_services_from_flag(self): + with mock.patch.dict(os.environ, {}, clear=True): + options = PipelineOptions(['--beam_services={"foo": "bar"}' + ]).view_as(CrossLanguageOptions) + self.assertEqual(options.beam_services, {'foo': 'bar'}) + + def test_beam_services_from_env_and_flag(self): + with mock.patch.dict( + os.environ, + {'BEAM_SERVICE_OVERRIDES': '{"foo": "bar", "other": "zzz"}'}, + clear=True): + options = PipelineOptions(['--beam_services={"foo": "override"}' + ]).view_as(CrossLanguageOptions) + self.assertEqual( + options.beam_services, { + 'foo': 'override', 'other': 'zzz' + }) + def test_option_modifications_are_shared_between_views(self): pipeline_options = PipelineOptions([ '--mock_option', From 1986517d38a48af2f1dddad40819ad3340816589 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 20 Dec 2023 08:09:03 -0800 Subject: [PATCH 161/224] Enable User State tests for Prism (#29822) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go/test/integration/integration.go | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index 5b7473fb561a..20f6cc5b261d 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -158,17 +158,6 @@ var prismFilters = []string{ "TestFhirIO.*", // OOMs currently only lead to heap dumps on Dataflow runner "TestOomParDo", - // The prism runner does not support user state. - "TestValueState", - "TestValueStateWindowed", - "TestValueStateClear", - "TestBagState", - "TestBagStateClear", - "TestCombiningState", - "TestMapState", - "TestMapStateClear", - "TestSetState", - "TestSetStateClear", } var flinkFilters = []string{ From 91aeba43273e70f7d1a11518c756225a67ecdb6a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 20 Dec 2023 08:42:34 -0800 Subject: [PATCH 162/224] Remove failing multimap key iteration tests. These tests used to work because a request for the multimap_keys_side_input state type used to return the empty set rather than (more correctly) returning an error that this was unsupported (which in turn cause these empty side input tests to serendipitously pass). --- runners/portability/java/build.gradle | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle index ee6fe3b8fd8d..fa7af9107260 100644 --- a/runners/portability/java/build.gradle +++ b/runners/portability/java/build.gradle @@ -206,6 +206,10 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = "" excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedMultimapAsEntrySetSideInput' excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedMapAsEntrySetSideInput' excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testMultimapAsEntrySetSideInput' + excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMapSideInput' + excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMapSideInputWithNonDeterministicKeyCoder' + excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMultimapSideInput' + excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMultimapSideInputWithNonDeterministicKeyCoder' for (String test : sickbayTests) { excludeTestsMatching test From 881ca8c212aaa3e3ca2ad354e24155cd324e1dff Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 20 Dec 2023 10:30:02 -0800 Subject: [PATCH 163/224] fix import order --- sdks/python/apache_beam/options/pipeline_options_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 1d77c6d89f16..c7a56b4739bd 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -21,11 +21,11 @@ import json import logging -import mock import os import unittest import hamcrest as hc +import mock from parameterized import parameterized from apache_beam.options.pipeline_options import CrossLanguageOptions From 8de029a412ab3e87ec92caf29818b51dab4ab02d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 20 Dec 2023 13:11:25 -0800 Subject: [PATCH 164/224] Bump github.com/fsouza/fake-gcs-server from 1.47.6 to 1.47.7 in /sdks (#29833) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.47.6 to 1.47.7. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.47.6...v1.47.7) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server 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 | 19 +++++++++---------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 28a049709ac0..65d70cfe702b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -69,7 +69,7 @@ require ( ) require ( - github.com/fsouza/fake-gcs-server v1.47.6 + github.com/fsouza/fake-gcs-server v1.47.7 golang.org/x/exp v0.0.0-20230807204917-050eac23e9de ) @@ -148,13 +148,13 @@ require ( github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect - github.com/gorilla/handlers v1.5.1 // indirect - github.com/gorilla/mux v1.8.0 // indirect + github.com/gorilla/handlers v1.5.2 // indirect + github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect github.com/klauspost/compress v1.17.4 // indirect - github.com/klauspost/cpuid/v2 v2.2.5 // indirect + github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/kr/text v0.2.0 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index f0e8e65fb9dc..492fa999d6d9 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -186,15 +186,14 @@ github.com/envoyproxy/go-control-plane v0.11.1/go.mod h1:uhMcXKCQMEJHiAb0w+YGefQ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA= github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= -github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/fsouza/fake-gcs-server v1.47.6 h1:/d/879q/Os9Zc5gyV3QVLfZoajN1KcWucf2zYCFeFxs= -github.com/fsouza/fake-gcs-server v1.47.6/go.mod h1:ApSXKexpG1BUXJ4f2tNCxvhTKwCPFqFLBDW2UNQDODE= +github.com/fsouza/fake-gcs-server v1.47.7 h1:56/U4rKY081TaNbq0gHWi7/71UxC2KROqcnrD9BRJhs= +github.com/fsouza/fake-gcs-server v1.47.7/go.mod h1:4vPUynN8/zZlxk5Jpy6LvvTTxItdTAObK4DYnp89Jys= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -293,10 +292,10 @@ github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+ github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= -github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= -github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= -github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/handlers v1.5.2 h1:cLTUSsNkgcwhgRqvCNmdbRWG0A3N4F+M2nWKdScwyEE= +github.com/gorilla/handlers v1.5.2/go.mod h1:dX+xVpaxdSw+q0Qek8SSsl3dfMk3jNddUkMzo0GtH0w= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= @@ -329,8 +328,8 @@ github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8 github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= -github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= -github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= +github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= +github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= @@ -354,7 +353,7 @@ github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8Ie github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= -github.com/minio/minio-go/v7 v7.0.63 h1:GbZ2oCvaUdgT5640WJOpyDhhDxvknAJU2/T3yurwcbQ= +github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= From 140dd1458eb465c634f0bcd50622e5e807d183ad Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 20 Dec 2023 17:13:36 -0500 Subject: [PATCH 165/224] Bump transformers (#29847) Bumps [transformers](https://github.com/huggingface/transformers) from 4.30.0 to 4.36.0. - [Release notes](https://github.com/huggingface/transformers/releases) - [Commits](https://github.com/huggingface/transformers/compare/v4.30.0...v4.36.0) --- updated-dependencies: - dependency-name: transformers dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../examples/inference/runinference_metrics/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/inference/runinference_metrics/setup.py b/sdks/python/apache_beam/examples/inference/runinference_metrics/setup.py index 102699b6bec9..d1bc0a06c4b2 100644 --- a/sdks/python/apache_beam/examples/inference/runinference_metrics/setup.py +++ b/sdks/python/apache_beam/examples/inference/runinference_metrics/setup.py @@ -29,7 +29,7 @@ from setuptools import find_packages REQUIREMENTS = [ - "apache-beam[gcp]==2.41.0", "transformers==4.30.0", "torch==1.13.1" + "apache-beam[gcp]==2.41.0", "transformers==4.36.0", "torch==1.13.1" ] setuptools.setup( From 79d679db2185332ba2ce05a6c9387af7bac9352f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 21 Dec 2023 09:54:46 +0100 Subject: [PATCH 166/224] Bump transformers from 4.30.0 to 4.36.0 in /sdks/python/apache_beam/examples/inference/large_language_modeling (#29846) Bumps [transformers](https://github.com/huggingface/transformers) from 4.30.0 to 4.36.0. - [Release notes](https://github.com/huggingface/transformers/releases) - [Commits](https://github.com/huggingface/transformers/compare/v4.30.0...v4.36.0) --- updated-dependencies: - dependency-name: transformers dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../examples/inference/large_language_modeling/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/inference/large_language_modeling/requirements.txt b/sdks/python/apache_beam/examples/inference/large_language_modeling/requirements.txt index aaec7156dc7b..b5383f964b2f 100644 --- a/sdks/python/apache_beam/examples/inference/large_language_modeling/requirements.txt +++ b/sdks/python/apache_beam/examples/inference/large_language_modeling/requirements.txt @@ -17,5 +17,5 @@ # under the License torch==1.13.1 -transformers==4.30.0 +transformers==4.36.0 sentencepiece==0.1.97 \ No newline at end of file From 273008534af8c26ee5aa3a0d00e1f1a44a7c0b7b Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 21 Dec 2023 09:23:23 -0500 Subject: [PATCH 167/224] Fix javadoc action in build-release-candidate (#29841) * Fix javadoc action in build-release-candidate * Use relative patha --- .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 d329591219a2..8ffb91fc1f37 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -338,7 +338,7 @@ jobs: - name: Add canonical link into javadocs uses: cicirello/javadoc-cleanup@v1 with: - path-to-root: ${BEAM_ROOT_DIR}/sdks/java/javadoc/build/docs/javadoc + path-to-root: beam/sdks/java/javadoc/build/docs/javadoc base-url-path: https://beam.apache.org/releases/javadoc/current/ - name: Consolidate Release Docs to beam-site branch with symlinks working-directory: beam-site From 05068ead3eaca28d7eb50fdc3345fc4e4e478664 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 21 Dec 2023 09:45:05 -0500 Subject: [PATCH 168/224] Bump transformers (#29844) Bumps [transformers](https://github.com/huggingface/transformers) from 4.30.0 to 4.36.0. - [Release notes](https://github.com/huggingface/transformers/releases) - [Commits](https://github.com/huggingface/transformers/compare/v4.30.0...v4.36.0) --- updated-dependencies: - dependency-name: transformers dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../anomaly_detection/anomaly_detection_pipeline/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/inference/anomaly_detection/anomaly_detection_pipeline/setup.py b/sdks/python/apache_beam/examples/inference/anomaly_detection/anomaly_detection_pipeline/setup.py index 15a68228c661..72a65ab72810 100644 --- a/sdks/python/apache_beam/examples/inference/anomaly_detection/anomaly_detection_pipeline/setup.py +++ b/sdks/python/apache_beam/examples/inference/anomaly_detection/anomaly_detection_pipeline/setup.py @@ -32,7 +32,7 @@ "apache-beam[gcp]==2.41.0", "hdbscan==0.8.28", "scikit-learn==1.0.2", - "transformers==4.30.0", + "transformers==4.36.0", "torch==1.13.1", "pandas==1.3.5", "yagmail==0.15.283", From 840178b91b5a686960d8c513b93968388db990b5 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 21 Dec 2023 10:00:05 -0500 Subject: [PATCH 169/224] Fix input issue and quoting (#29840) * Fix masking issue and quoting * Fix masking * Fix variable name --- .github/workflows/build_release_candidate.yml | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 8ffb91fc1f37..8d27cea1d5d5 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -95,9 +95,13 @@ jobs: if: ${{github.event.inputs.STAGE_JAVA_SOURCE == 'yes'}} runs-on: ubuntu-latest steps: - - name: Validate and mask apache id/password + - name: Mask Apache Password + run: | + # Workaround for Actions bug - https://github.com/actions/runner/issues/643 + APACHE_PASS=$(jq -r '.inputs.APACHE_PASSWORD' $GITHUB_EVENT_PATH) + echo "::add-mask::$APACHE_PASS" + - name: Validate apache id/password run: | - echo "::add-mask::${{ github.event.inputs.APACHE_PASSWORD }}" if [ "${{ github.event.inputs.APACHE_ID }}" == "" ] then echo "Must provide an apache id to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" @@ -149,7 +153,7 @@ jobs: svn add --force . svn status - svn commit -m "Staging Java artifacts for Apache Beam ${{ github.event.inputs.RELEASE }} RC${{ github.event.inputs.RC }}" --non-interactive --username ${{ github.event.inputs.APACHE_ID }} --password ${{ github.event.inputs.APACHE_PASSWORD }} + svn commit -m "Staging Java artifacts for Apache Beam ${{ github.event.inputs.RELEASE }} RC${{ github.event.inputs.RC }}" --non-interactive --username "${{ github.event.inputs.APACHE_ID }}" --password "${{ github.event.inputs.APACHE_PASSWORD }}" stage_python_artifacts: if: ${{github.event.inputs.STAGE_PYTHON_ARTIFACTS == 'yes'}} @@ -157,9 +161,13 @@ jobs: steps: - name: Checkout uses: actions/checkout@v4 - - name: Validate and mask apache id/password + - name: Mask Apache Password + run: | + # Workaround for Actions bug - https://github.com/actions/runner/issues/643 + APACHE_PASS=$(jq -r '.inputs.APACHE_PASSWORD' $GITHUB_EVENT_PATH) + echo "::add-mask::$APACHE_PASS" + - name: Mask apache id/password run: | - echo "::add-mask::${{ github.event.inputs.APACHE_PASSWORD }}" if [ "${{ github.event.inputs.APACHE_ID }}" == "" ] then echo "Must provide an apache id to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" @@ -227,7 +235,7 @@ jobs: cd .. svn add --force python svn status - svn commit -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" --non-interactive --username ${{ github.event.inputs.APACHE_ID }} --password ${{ github.event.inputs.APACHE_PASSWORD }} + svn commit -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" --non-interactive --username "${{ github.event.inputs.APACHE_ID }}" --password "${{ github.event.inputs.APACHE_PASSWORD }}" stage_docker: From 6345c8bf681710f2ddd3d918b39449750ffd3bff Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 21 Dec 2023 10:14:21 -0500 Subject: [PATCH 170/224] Fix logging/quoting (#29854) --- .github/workflows/deploy_release_candidate_pypi.yaml | 7 ++++--- .github/workflows/finalize_release.yml | 11 +++++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/.github/workflows/deploy_release_candidate_pypi.yaml b/.github/workflows/deploy_release_candidate_pypi.yaml index fd3994f658e8..8dafd520b1bf 100644 --- a/.github/workflows/deploy_release_candidate_pypi.yaml +++ b/.github/workflows/deploy_release_candidate_pypi.yaml @@ -31,8 +31,9 @@ jobs: steps: - name: Mask PyPi id/password run: | - echo "::add-mask::${{ github.event.inputs.PYPI_USER }}" - echo "::add-mask::${{ github.event.inputs.PYPI_PASSWORD }}" + # Workaround for Actions bug - https://github.com/actions/runner/issues/643 + PYPI_PASSWORD=$(jq -r '.inputs.PYPI_PASSWORD' $GITHUB_EVENT_PATH) + echo "::add-mask::$PYPI_PASSWORD" - name: Checkout uses: actions/checkout@v4 - name: Setup environment @@ -86,4 +87,4 @@ jobs: echo "Will upload the following files to PyPI:" ls - twine upload * -u ${{ github.event.inputs.PYPI_USER }} -p ${{ github.event.inputs.PYPI_PASSWORD }} + twine upload * -u "${{ github.event.inputs.PYPI_USER }}" -p "${{ github.event.inputs.PYPI_PASSWORD }}" diff --git a/.github/workflows/finalize_release.yml b/.github/workflows/finalize_release.yml index 9a8b47ad3e42..d96b59ee7cfe 100644 --- a/.github/workflows/finalize_release.yml +++ b/.github/workflows/finalize_release.yml @@ -91,10 +91,13 @@ jobs: steps: - name: Checkout uses: actions/checkout@v4 - - name: Mask and validate PyPi id/password + - name: Mask PyPi password + run: | + # Workaround for Actions bug - https://github.com/actions/runner/issues/643 + PYPI_PASSWORD=$(jq -r '.inputs.PYPI_PASSWORD' $GITHUB_EVENT_PATH) + echo "::add-mask::$PYPI_PASSWORD" + - name: Validate PyPi id/password run: | - echo "::add-mask::${{ github.event.inputs.PYPI_USER }}" - echo "::add-mask::${{ github.event.inputs.PYPI_PASSWORD }}" if [ "${{ github.event.inputs.PYPI_USER }}" == "" ] then echo "Must provide a PyPi username to publish artifacts to PyPi" @@ -122,7 +125,7 @@ jobs: cd "dist.apache.org/repos/dist/dev/beam/${RELEASE}/python/" echo "Will upload the following files to PyPI:" ls - twine upload * -u ${{ github.event.inputs.PYPI_USER }} -p ${{ github.event.inputs.PYPI_PASSWORD }} + twine upload * -u "${{ github.event.inputs.PYPI_USER }}" -p "${{ github.event.inputs.PYPI_PASSWORD }}" push_git_tags: if: ${{github.event.inputs.TAG_RELEASE == 'yes'}} From a0081221f1d2c697412b81b36c9921b09548a9a0 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 21 Dec 2023 11:51:51 -0500 Subject: [PATCH 171/224] Fix tour of beam co-group-by-key python example (#29775) --- .../map/co-group-by-key/description.md | 48 +++++++------------ 1 file changed, 17 insertions(+), 31 deletions(-) diff --git a/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md b/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md index c0e7f9f3e44d..5400003b2168 100644 --- a/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md +++ b/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md @@ -284,42 +284,28 @@ static PCollection applyTransform(PCollection fruits, PCollectio {{end}} {{if (eq .Sdk "python")}} ``` -weight := beam.ParDo(s, func(_ []byte, emit func(string, int)){ - emit("brazil", 1000) - emit("australia", 150) - emit("canada", 340) -}, beam.Impulse(s)) - -fruits := beam.ParDo(s, func(_ []byte, emit func(string, string)){ - emit("australia", "cherry") - emit("brazil", "apple") - emit("canada", "banan") -}, beam.Impulse(s)) +fruits = p | 'Fruits' >> beam.Create([('australia', 'cherry'), ('brazil', 'apple'), ('canada', 'banana')]) +weights = p | 'Countries' >> beam.Create([('australia', 1000), ('brazil', 150), ('canada', 340)]) ``` -Change `Alphabet` to `ProductWeight`: +Change `alphabet` to `product_weight`: ``` -type WordsAlphabet struct { - Country string - Fruit string - ProductWeight int -} +class ProductWeight: + def __init__(self, product_weight, fruit, country): + self.product_weight = product_weight + self.fruit = fruit + self.country = country ``` The union takes place through the keys: ``` -func applyTransform(s beam.Scope, fruits beam.PCollection, countries beam.PCollection) beam.PCollection { - grouped := beam.CoGroupByKey(s, fruits, countries) - return beam.ParDo(s, func(key string, weightIter func(*int) bool, fruitIter func(*string) bool, emit func(string)) { - - wa := &WordsAlphabet{ - Country: key, - } - weightIter(&wa.ProductWeight) - fruitIter(&wa.Fruit) - emit(wa.String()) - - }, grouped) -} +def apply_transforms(fruits, weights): + def cogbk_result_to_product_weight(cgbk_result): + (country, values) = cgbk_result + return WordsAlphabet(values['weights'][0], values['fruits'][0], country) + + return ({'fruits': fruits, 'weights': weights} + | beam.CoGroupByKey() + | beam.Map(cogbk_result_to_product_weight)) ``` -{{end}} \ No newline at end of file +{{end}} From 6a12e8b88dfe0e0f635bd9b601bb809ef2919e99 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Thu, 21 Dec 2023 12:48:41 -0500 Subject: [PATCH 172/224] fix contructor names (#29855) --- .../core-transforms/map/co-group-by-key/description.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md b/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md index 5400003b2168..a15321cd8ea3 100644 --- a/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md +++ b/learning/tour-of-beam/learning-content/core-transforms/map/co-group-by-key/description.md @@ -235,7 +235,7 @@ static class ProductWeight { private String fruit; private Integer productWeight; - public WordsAlphabet(String country, String fruit, Integer productWeight) { + public ProductWeight(String country, String fruit, Integer productWeight) { this.country = country; this.fruit = fruit; this.productWeight = productWeight; @@ -275,7 +275,7 @@ static PCollection applyTransform(PCollection fruits, PCollectio String fruit = coGbkResult.getOnly(fruitsTag); String country = coGbkResult.getOnly(countriesTag); - out.output(new WordsAlphabet(alphabet, fruit, country).toString()); + out.output(new ProductWeight(alphabet, fruit, country).toString()); } })); @@ -302,7 +302,7 @@ The union takes place through the keys: def apply_transforms(fruits, weights): def cogbk_result_to_product_weight(cgbk_result): (country, values) = cgbk_result - return WordsAlphabet(values['weights'][0], values['fruits'][0], country) + return ProductWeight(values['weights'][0], values['fruits'][0], country) return ({'fruits': fruits, 'weights': weights} | beam.CoGroupByKey() From 61c53676a7b540b97fe3165c34e00432c950f00e Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 21 Dec 2023 09:49:00 -0800 Subject: [PATCH 173/224] Dataflow Streaming: Add a pipeline option `--desiredNumUnboundedSourceSplits` to overide the desired number of splits for CustomSources (#29704) Co-authored-by: Arun Pandian --- .../beam/runners/dataflow/internal/CustomSources.java | 4 ++++ .../dataflow/options/DataflowPipelineDebugOptions.java | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java index e68dde045763..fcfe3fe3ce05 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java @@ -49,6 +49,10 @@ public class CustomSources { private static final Logger LOG = LoggerFactory.getLogger(CustomSources.class); private static int getDesiredNumUnboundedSourceSplits(DataflowPipelineOptions options) { + if (options.getDesiredNumUnboundedSourceSplits() > 0) { + return options.getDesiredNumUnboundedSourceSplits(); + } + int cores = 4; // TODO: decide at runtime? if (options.getMaxNumWorkers() > 0) { return options.getMaxNumWorkers() * cores; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index 709009f8c806..f8649a1f0f3e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -280,6 +280,16 @@ public Dataflow create(PipelineOptions options) { void setUnboundedReaderMaxWaitForElementsMs(Integer value); + /** + * The desired number of initial splits for UnboundedSources. If this value is <=0, the splits + * will be computed based on the number of user workers. + */ + @Description("The desired number of initial splits for UnboundedSources.") + @Default.Integer(0) + int getDesiredNumUnboundedSourceSplits(); + + void setDesiredNumUnboundedSourceSplits(int value); + /** * CAUTION: This option implies dumpHeapOnOOM, and has similar caveats. Specifically, heap dumps * can of comparable size to the default boot disk. Consider increasing the boot disk size before From 7e4dc66ba208a8bc53c03f52945b95713b74fbaf Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 21 Dec 2023 09:49:45 -0800 Subject: [PATCH 174/224] Change pubsub message cap size from 10MiB to 10MB (#29791) Pubsub's max allowed message size is 10million bytes and not 10 * 2^20 bytes https://cloud.google.com/pubsub/quotas Co-authored-by: Arun Pandian --- .../main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 2 +- sdks/python/apache_beam/io/gcp/pubsub.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index f79299aea5fc..e281e559a544 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -194,7 +194,7 @@ public class PubsubIO { private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+"); - static final int PUBSUB_MESSAGE_MAX_TOTAL_SIZE = 10 << 20; + static final int PUBSUB_MESSAGE_MAX_TOTAL_SIZE = 10_000_000; private static final int PUBSUB_NAME_MIN_LENGTH = 3; private static final int PUBSUB_NAME_MAX_LENGTH = 255; diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index af58006d6e76..d0785fa1f214 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -150,7 +150,7 @@ def _to_proto_str(self, for_publish=False): containing the payload of this object. """ msg = pubsub.types.PubsubMessage() - if len(self.data) > (10 << 20): + if len(self.data) > (10_000_000): raise ValueError('A pubsub message data field must not exceed 10MB') msg.data = self.data @@ -179,7 +179,7 @@ def _to_proto_str(self, for_publish=False): msg.ordering_key = self.ordering_key serialized = pubsub.types.PubsubMessage.serialize(msg) - if len(serialized) > (10 << 20): + if len(serialized) > (10_000_000): raise ValueError( 'Serialized pubsub message exceeds the publish request limit of 10MB') return serialized From 44515bf2a805d08e3e9e7e2fe4d736ca8cbd069d Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Fri, 22 Dec 2023 12:43:57 -0500 Subject: [PATCH 175/224] Update google-api-services-storage to match libraries-bom (#29864) ``` cd /tmp; wget https://repo1.maven.org/maven2/com/google/cloud/libraries-bom/26.28.0/libraries-bom-26.28.0.pom -O base.pom && mvn help:effective-pom -f base.pom -Doutput=effective.pom && cat effective.pom | grep -v 'dependencyManagement' > cleanup.pom && mvn dependency:tree -f cleanup.pom ``` Yields ``` [INFO] | +- com.google.apis:google-api-services-storage:jar:v1-rev20231202-2.0.0:compile ``` Making the change accordingly, since this mismatch is causing some errors on RC validation: https://github.com/GoogleCloudPlatform/DataflowTemplates/pull/1233/checks?check_run_id=19897484522 ``` java.lang.NoClassDefFoundError: com/google/api/services/storage/model/Bucket$ObjectRetention at com.google.cloud.storage.JsonConversions.(JsonConversions.java:120) at com.google.cloud.storage.JsonConversions.(JsonConversions.java:89) at com.google.cloud.storage.Conversions.json(Conversions.java:33) at com.google.cloud.storage.StorageImpl.(StorageImpl.java:109) ``` --- .../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 82ab2593cb61..9576dd2c2f44 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -734,7 +734,7 @@ class BeamModulePlugin implements Plugin { google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20231101-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", // Keep version consistent with the version in google_cloud_nio, managed by google_cloud_platform_libraries_bom - google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20230907-$google_clients_version", + google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20231202-$google_clients_version", google_auth_library_credentials : "com.google.auth:google-auth-library-credentials", // google_cloud_platform_libraries_bom sets version google_auth_library_oauth2_http : "com.google.auth:google-auth-library-oauth2-http", // google_cloud_platform_libraries_bom sets version google_cloud_bigquery : "com.google.cloud:google-cloud-bigquery", // google_cloud_platform_libraries_bom sets version From 304f0cbe768791cb1f416246183ab1ceb57fcf27 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Tue, 26 Dec 2023 13:18:49 +0000 Subject: [PATCH 176/224] Bump objsize upper bound (#29562) * Bump objsize upper bound * generate requirements * Update dependencies --- sdks/python/container/py310/base_image_requirements.txt | 8 ++++---- sdks/python/container/py311/base_image_requirements.txt | 8 ++++---- sdks/python/container/py38/base_image_requirements.txt | 8 ++++---- sdks/python/container/py39/base_image_requirements.txt | 8 ++++---- sdks/python/setup.py | 2 +- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index d6b810725522..636d546867fb 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -33,7 +33,7 @@ click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 cryptography==41.0.7 -Cython==0.29.36 +Cython==0.29.37 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 @@ -42,9 +42,9 @@ docopt==0.6.2 docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.1 +fastavro==1.9.2 fasteners==0.19 -freezegun==1.3.1 +freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 google-api-python-client==2.111.0 @@ -92,7 +92,7 @@ nltk==3.8.1 nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 -objsize==0.6.1 +objsize==0.7.0 orjson==3.9.10 overrides==7.4.0 packaging==23.2 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index ac6f6dbd7258..c594f7f84870 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -33,7 +33,7 @@ click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 cryptography==41.0.7 -Cython==0.29.36 +Cython==0.29.37 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 @@ -41,9 +41,9 @@ docker==7.0.0 docopt==0.6.2 docstring-parser==0.15 execnet==2.0.2 -fastavro==1.9.1 +fastavro==1.9.2 fasteners==0.19 -freezegun==1.3.1 +freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 google-api-python-client==2.111.0 @@ -91,7 +91,7 @@ nltk==3.8.1 nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 -objsize==0.6.1 +objsize==0.7.0 orjson==3.9.10 overrides==7.4.0 packaging==23.2 diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index 2929ab6f1e04..a0c27c0c3026 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -34,7 +34,7 @@ click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 cryptography==41.0.7 -Cython==0.29.36 +Cython==0.29.37 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 @@ -43,9 +43,9 @@ docopt==0.6.2 docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.1 +fastavro==1.9.2 fasteners==0.19 -freezegun==1.3.1 +freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 google-api-python-client==2.111.0 @@ -95,7 +95,7 @@ nltk==3.8.1 nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 -objsize==0.6.1 +objsize==0.7.0 orjson==3.9.10 overrides==7.4.0 packaging==23.2 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 9d7e99f76eee..953b7429a1a8 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -33,7 +33,7 @@ click==8.1.7 cloudpickle==2.2.1 crcmod==1.7 cryptography==41.0.7 -Cython==0.29.36 +Cython==0.29.37 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.4.2 @@ -42,9 +42,9 @@ docopt==0.6.2 docstring-parser==0.15 exceptiongroup==1.2.0 execnet==2.0.2 -fastavro==1.9.1 +fastavro==1.9.2 fasteners==0.19 -freezegun==1.3.1 +freezegun==1.4.0 future==0.18.3 google-api-core==2.15.0 google-api-python-client==2.111.0 @@ -93,7 +93,7 @@ nltk==3.8.1 nose==1.3.7 numpy==1.24.4 oauth2client==4.1.3 -objsize==0.6.1 +objsize==0.7.0 orjson==3.9.10 overrides==7.4.0 packaging==23.2 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index d3fca2147239..e8c93ce85484 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -302,7 +302,7 @@ def get_portability_package_data(): # numpy can have breaking changes in minor versions. # Use a strict upper bound. 'numpy>=1.14.3,<1.25.0', # Update pyproject.toml as well. - 'objsize>=0.6.1,<0.7.0', + 'objsize>=0.6.1,<0.8.0', 'packaging>=22.0', 'pymongo>=3.8.0,<5.0.0', 'proto-plus>=1.7.1,<2', From 6a036699dc8f4b549afd2fa338eaf2e450d6a02f Mon Sep 17 00:00:00 2001 From: Oleh Borysevych Date: Tue, 26 Dec 2023 15:25:45 +0100 Subject: [PATCH 177/224] DuetAI knowledge lookup prompts (#29507) * basic knowledge lookup prompts * advanced knowledge lookup prompts * finalize advanced knowledge lookup prompts * minor content fixes * pubsub lookup added * pubsub code generation and explaination added * initial prompts added * minor syntax * fix whitespace * fix whitespace warnings * fix code review comments * remove unfinished work * fix trailing whitespace * fix whitespace * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * fix code review comments * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * added intro text before example --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- build.gradle.kts | 3 ++ .../prompts/code-explaination/02_io_pubsub.md | 33 ++++++++++++ .../prompts/code-generation/02_io_pubsub.md | 51 +++++++++++++++++++ .../01_basic_learning_apache_beam.md | 14 +++++ .../02_basic_pipelines.md | 19 +++++++ .../03_basic_configuring_pipelines.md | 29 +++++++++++ .../04_basic_pcollections.md | 25 +++++++++ .../05_basic_ptransforms.md | 34 +++++++++++++ .../documentation-lookup/06_basic_schema.md | 25 +++++++++ .../documentation-lookup/07_basic_runners.md | 23 +++++++++ .../08_basic_windowing.md | 29 +++++++++++ .../documentation-lookup/09_basic_triggers.md | 32 ++++++++++++ .../documentation-lookup/10_basic_metrics.md | 24 +++++++++ .../documentation-lookup/11_basic_state.md | 20 ++++++++ .../documentation-lookup/12_basic_timers.md | 12 +++++ .../13_advanced_splittable_dofn.md | 18 +++++++ .../14_advanced_pipeline_patterns.md | 13 +++++ .../documentation-lookup/15_advanced_xlang.md | 15 ++++++ .../16_advanced_pipeline_lifecycle.md | 36 +++++++++++++ .../documentation-lookup/17_advanced_ai_ml.md | 18 +++++++ .../documentation-lookup/19_io_pubsub.md | 27 ++++++++++ .../documentation-lookup/20_io_biguery.md | 39 ++++++++++++++ 22 files changed, 539 insertions(+) create mode 100644 learning/prompts/code-explaination/02_io_pubsub.md create mode 100644 learning/prompts/code-generation/02_io_pubsub.md create mode 100644 learning/prompts/documentation-lookup/01_basic_learning_apache_beam.md create mode 100644 learning/prompts/documentation-lookup/02_basic_pipelines.md create mode 100644 learning/prompts/documentation-lookup/03_basic_configuring_pipelines.md create mode 100644 learning/prompts/documentation-lookup/04_basic_pcollections.md create mode 100644 learning/prompts/documentation-lookup/05_basic_ptransforms.md create mode 100644 learning/prompts/documentation-lookup/06_basic_schema.md create mode 100644 learning/prompts/documentation-lookup/07_basic_runners.md create mode 100644 learning/prompts/documentation-lookup/08_basic_windowing.md create mode 100644 learning/prompts/documentation-lookup/09_basic_triggers.md create mode 100644 learning/prompts/documentation-lookup/10_basic_metrics.md create mode 100644 learning/prompts/documentation-lookup/11_basic_state.md create mode 100644 learning/prompts/documentation-lookup/12_basic_timers.md create mode 100644 learning/prompts/documentation-lookup/13_advanced_splittable_dofn.md create mode 100644 learning/prompts/documentation-lookup/14_advanced_pipeline_patterns.md create mode 100644 learning/prompts/documentation-lookup/15_advanced_xlang.md create mode 100644 learning/prompts/documentation-lookup/16_advanced_pipeline_lifecycle.md create mode 100644 learning/prompts/documentation-lookup/17_advanced_ai_ml.md create mode 100644 learning/prompts/documentation-lookup/19_io_pubsub.md create mode 100644 learning/prompts/documentation-lookup/20_io_biguery.md diff --git a/build.gradle.kts b/build.gradle.kts index 7a3686e5678c..511b537d7fe1 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -205,6 +205,9 @@ tasks.rat { // Ignore poetry autogenerated files. "**/poetry.lock", + + // DuetAI training prompts + "learning/prompts/**/*.md", ) // Add .gitignore excludes to the Apache Rat exclusion list. We re-create the behavior diff --git a/learning/prompts/code-explaination/02_io_pubsub.md b/learning/prompts/code-explaination/02_io_pubsub.md new file mode 100644 index 000000000000..de02bd58e4d8 --- /dev/null +++ b/learning/prompts/code-explaination/02_io_pubsub.md @@ -0,0 +1,33 @@ +Prompt: +What does this code do? +```python +pipeline_options = PipelineOptions(pipeline_args) + +with beam.Pipeline(options=options) as p: + (p | "Read" >> ReadFromPubSub(subscription=input_subscription) + | "Process" >> ParDo(ProcessMessage()) + | "Write" >> WriteToPubSub(topic=output_topic)) +``` +Response: +This code reads messages from the [Pub/Sub](https://cloud.google.com/pubsub/docs/overview) subscription `input_subscription` using the `ReadFromPubSub`, processes them. The code then writes the resulting collection to the Pub/Sub topic `output_topic` using the `WriteToPubSub` transform. + +Pub/Sub is only supported in streaming pipelines. + +Reading messages directly from a topic is also supported. In this case, a temporary subscription will be created automatically. + +`ReadFromPubSub` produces a `PCollection` of `PubsubMessage` objects or a `PCollection` of byte sequences. Behavior is controlled by the `with_attributes` parameter, with byte sequences being the default. +For more information about PCollections see the [Beam Programming Guide](https://beam.apache.org/documentation/basics/#pcollection). + +The `ProcessMessage` class processes the messages. This class is a subclass of the `DoFn` class. +The simplest implementation of `ProcessMesage` class might look like the following example: + +```python +class ProcessMessage(beam.DoFn): + def process(self, element): + yield element +``` +For more information about the `DoFn` class, see the [Beam Programming Guide](https://beam.apache.org/documentation/programming-guide/#dofn). + +For more details, see the [Pub/Sub I/O transform documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.pubsub.html). + +For a common pattern for configuring pipeline options, see the [Pipeline option pattern](https://beam.apache.org/documentation/patterns/pipeline-options/). \ No newline at end of file diff --git a/learning/prompts/code-generation/02_io_pubsub.md b/learning/prompts/code-generation/02_io_pubsub.md new file mode 100644 index 000000000000..90856e8956ff --- /dev/null +++ b/learning/prompts/code-generation/02_io_pubsub.md @@ -0,0 +1,51 @@ +Prompt: +Write the python code to read messages from a Pub/Sub subscription. +Response: +You can read messages from a Pub/Sub subscription or topic using the `ReadFromPubSub` transform. Pub/Sub is only supported in streaming pipelines. + +The following Python code reads messages from a Pub/Sub subscription. The subscription is provided as a command line argument. The messages are logged to the console: + +```python +import logging + +import apache_beam as beam +from apache_beam import Map +from apache_beam.io import ReadFromPubSub +from apache_beam.options.pipeline_options import PipelineOptions + +class PubSubReadOptions(PipelineOptions): +""" +Configure pipeline options for PubSub read transform. +""" + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--subscription", + required=True, + help="Pub/Sub subscription to read from.") + +def read_subscription(): + """Read from Pub/Sub subscription function.""" + + # Parse the pipeline options. + # For streaming pipelines, you need to include streaming=True + options = PubSubReadOptions(streaming=True) + + with beam.Pipeline(options=options) as p: + # This pipeline reads from a Pub/Sub subscription and logs the messages to the console. + (p | "Read Pub/Sub subscription" >> ReadFromPubSub(subscription=options.subscription) + | "Format message" >> Map(lambda message: f"Received message:\n{message}\n") + | Map(logging.info)) + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + read_subscription() + +``` +Reading messages directly from a topic is also supported. A temporary subscription will be created automatically. + +The messages could be returned as a byte string or as `PubsubMessage` objects. This behavior is controlled by the `with_attributes` parameter. + +For more information, see the [Pub/Sub I/O transform documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.pubsub.html). + +For a common pattern for configuring pipeline options see the [Pipeline option pattern](https://beam.apache.org/documentation/patterns/pipeline-options/). \ No newline at end of file diff --git a/learning/prompts/documentation-lookup/01_basic_learning_apache_beam.md b/learning/prompts/documentation-lookup/01_basic_learning_apache_beam.md new file mode 100644 index 000000000000..df428d910388 --- /dev/null +++ b/learning/prompts/documentation-lookup/01_basic_learning_apache_beam.md @@ -0,0 +1,14 @@ +Prompt: +What is Apache Beam? +Response: +Apache Beam is an open-source unified programming model for data processing pipelines. It provides a set of language-specific SDKs that allow you to define and execute data processing workflows. +Apache Beam allows you to write pipelines in one of the supported languages (Java, Python, Go, and Scala). The pipelines can be executed on one of the supported runners (Apache Flink, Apache Spark, and Google Cloud Dataflow). +Learning Apache Beam is a valuable skill for data processing and analysis. Here are some resources to get you started: +[Get Started](https://beam.apache.org/get-started/) section of the Apache Beam website is a perfect place to start. It provides an overview of Apache Beam and its key concepts. +For a hands-on introduction, you can start by exploring the [WordCount example](https://beam.apache.org/get-started/wordcount-example/), which demonstrates the basic concepts of Apache Beam. +The [Tour of Beam](https://tour.beam.apache.org/) is an interactive tour that teaches you core Beam concepts. It provides a sandbox environment to write and run pipelines while exploring different topics. +The [Beam Playground](https://play.beam.apache.org/) offers an interactive space to experiment with Beam's transforms and examples without installing anything on your system. After you have a basic understanding of Apache Beam, you can start to build your own pipelines. +The Apache Beam website has a number of resources that can help you get started, including [Apache Beam Documentation](https://beam.apache.org/documentation/), and [Apache Beam Community](https://beam.apache.org/community/). +For those interested in the source code, you can explore the [Apache Beam repository](https://github.com/apache/beam) on GitHub. +By leveraging these resources, you can build a strong foundation in Apache Beam and start constructing your own data processing pipelines. +Additionally, Apache Beam is an open-source project, and contributions are welcome. Whether you're fixing bugs, improving documentation, or adding new features, your work can benefit the entire Beam community. To learn more about how you can contribute, visit the [Contribute](https://beam.apache.org/contribute/#) section of the Apache Beam website. \ No newline at end of file diff --git a/learning/prompts/documentation-lookup/02_basic_pipelines.md b/learning/prompts/documentation-lookup/02_basic_pipelines.md new file mode 100644 index 000000000000..873329b821b7 --- /dev/null +++ b/learning/prompts/documentation-lookup/02_basic_pipelines.md @@ -0,0 +1,19 @@ +Prompt: +What is a Pipeline in Apache Beam? +Response: +In Apache Beam, a [pipeline](https://beam.apache.org/documentation/pipelines/design-your-pipeline/) encapsulates the entirety of a data processing task, including all the data and each step of the process. Essentially, it's a [Directed Acyclic Graph](https://en.wikipedia.org/wiki/Directed_acyclic_graph) of transformations (known as `PTransform`) applied to data collections (`PCollection`). + +The simplest pipelines in Apache Beam follow a linear flow of operations, typically adhering to a read-process-write pattern. However, pipelines can also be significantly more complex, featuring multiple input sources, output sinks, and operations (known as a `PTransform`) that can both read from and output to multiple PCollections. +For more information about pipeline design and best practices, see [Common Pipeline Patterns](https://beam.apache.org/documentation/patterns/overview/). + +To use Beam, your driver program must first create an instance of the Beam SDK class `Pipeline` (typically in the `main()` function). + +```python +import apache_beam as beam + +with beam.Pipeline() as pipeline: + pass # Build your pipeline here. +``` + +When you create your pipeline, you need to set [configuration options](https://beam.apache.org/documentation/programming-guide/#configuring-pipeline-options). You can set your pipeline’s configuration options programmatically. You can also set the options ahead of time, or read them from the command line, and then pass them to the `Pipeline` object when you create the object. This option is often easier than setting the pipeline configuration options programmatically. + diff --git a/learning/prompts/documentation-lookup/03_basic_configuring_pipelines.md b/learning/prompts/documentation-lookup/03_basic_configuring_pipelines.md new file mode 100644 index 000000000000..bf5ac7fabf81 --- /dev/null +++ b/learning/prompts/documentation-lookup/03_basic_configuring_pipelines.md @@ -0,0 +1,29 @@ +Prompt: +How do I configure pipeline in Apache Beam? +Response: +Configuring pipeline options in Apache Beam is crucial for specifying the execution environment, managing resources, and tailoring the pipeline's behavior to meet specific data processing needs and performance requirements. + +You can set pipeline options programmatically or pass them in using the command line. These options include runner-specific settings, job name, project ID (for cloud runners), machine types, number of workers, and more. + +Apache Beam offers a variety of [standard pipeline options](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/options/pipeline_options.py) that allow you to customize and optimize your data processing pipelines. + +Beam SDKs include a command-line parser that you can use to set pipeline options. Use command-line arguments in the format `--

    Handling Errors

    + * + *

    When using dynamic destinations, or when using a formatting function to format a record for + * writing, it's possible for an individual record to be malformed, causing an exception. By + * default, these exceptions are propagated to the runner causing the bundle to fail. These are + * usually retried, though this depends on the runner. Alternately, these errors can be routed to + * another {@link PTransform} by using {@link Write#withBadRecordErrorHandler(ErrorHandler)}. The + * ErrorHandler is registered with the pipeline (see below). See {@link ErrorHandler} for more + * documentation. Of note, this error handling only handles errors related to specific records. It + * does not handle errors related to connectivity, authorization, etc. as those should be retried by + * the runner. + * + *

    {@code
    + * PCollection<> records = ...;
    + * PTransform,?> alternateSink = ...;
    + * try (BadRecordErrorHandler handler = pipeline.registerBadRecordErrorHandler(alternateSink) {
    + *    records.apply("Write", FileIO.writeDynamic().otherConfigs()
    + *        .withBadRecordErrorHandler(handler));
    + * }
    + * }
    + * *

    Writing custom types to sinks

    * *

    Normally, when writing a collection of a custom type using a {@link Sink} that takes a @@ -1016,6 +1039,8 @@ public static FileNaming relativeFileNaming( abstract boolean getNoSpilling(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -1062,6 +1087,9 @@ abstract Builder setSharding( abstract Builder setNoSpilling(boolean noSpilling); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + abstract Write build(); } @@ -1288,6 +1316,18 @@ public Write withNoSpilling() { return toBuilder().setNoSpilling(true).build(); } + /** + * Configures a new {@link Write} with an ErrorHandler. For configuring an ErrorHandler, see + * {@link ErrorHandler}. Whenever a record is formatted, or a lookup for a dynamic destination + * is performed, and that operation fails, the exception is passed to the error handler. This is + * intended to handle any errors related to the data of a record, but not any connectivity or IO + * errors related to the literal writing of a record. + */ + public Write withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return toBuilder().setBadRecordErrorHandler(errorHandler).build(); + } + @VisibleForTesting Contextful> resolveFileNamingFn() { if (getDynamic()) { @@ -1391,6 +1431,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { writeFiles = writeFiles.withNoSpilling(); } + if (getBadRecordErrorHandler() != null) { + writeFiles = writeFiles.withBadRecordErrorHandler(getBadRecordErrorHandler()); + } return input.apply(writeFiles); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 2c7a4fc5d4f5..96635a37fac1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -51,6 +51,8 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -176,6 +178,10 @@ * *

    For backwards compatibility, {@link TextIO} also supports the legacy {@link * DynamicDestinations} interface for advanced features via {@link Write#to(DynamicDestinations)}. + * + *

    Error handling for records that are malformed can be handled by using {@link + * TypedWrite#withBadRecordErrorHandler(ErrorHandler)}. See documentation in {@link FileIO} for + * details on usage */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) @@ -708,6 +714,8 @@ public abstract static class TypedWrite */ abstract WritableByteChannelFactory getWritableByteChannelFactory(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -754,6 +762,9 @@ abstract Builder setNumShards( abstract Builder setWritableByteChannelFactory( WritableByteChannelFactory writableByteChannelFactory); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + abstract TypedWrite build(); } @@ -993,6 +1004,12 @@ public TypedWrite withNoSpilling() { return toBuilder().setNoSpilling(true).build(); } + /** See {@link FileIO.Write#withBadRecordErrorHandler(ErrorHandler)} for details on usage. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return toBuilder().setBadRecordErrorHandler(errorHandler).build(); + } + /** Don't write any output files if the PCollection is empty. */ public TypedWrite skipIfEmpty() { return toBuilder().setSkipIfEmpty(true).build(); @@ -1083,6 +1100,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { write = write.withNoSpilling(); } + if (getBadRecordErrorHandler() != null) { + write = write.withBadRecordErrorHandler(getBadRecordErrorHandler()); + } if (getSkipIfEmpty()) { write = write.withSkipIfEmpty(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 91d6082eede4..7359141c5b87 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; 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; @@ -49,6 +50,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.GroupIntoBatches; @@ -62,6 +64,10 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -166,6 +172,8 @@ public static WriteFiles()) + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) .build(); } @@ -189,6 +197,10 @@ public static WriteFiles getShardingFunction(); + public abstract ErrorHandler getBadRecordErrorHandler(); + + public abstract BadRecordRouter getBadRecordRouter(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -215,6 +227,12 @@ abstract Builder setSideInputs( abstract Builder setShardingFunction( @Nullable ShardingFunction shardingFunction); + abstract Builder setBadRecordErrorHandler( + ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordRouter( + BadRecordRouter badRecordRouter); + abstract WriteFiles build(); } @@ -330,6 +348,15 @@ public WriteFiles withSkipIfEmpty() { return toBuilder().setSkipIfEmpty(true).build(); } + /** See {@link FileIO.Write#withBadRecordErrorHandler(ErrorHandler)} for details on usage. */ + public WriteFiles withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return toBuilder() + .setBadRecordErrorHandler(errorHandler) + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER) + .build(); + } + @Override public void validate(PipelineOptions options) { getSink().validate(options); @@ -495,28 +522,39 @@ private WriteUnshardedBundlesToTempFiles( @Override public PCollection> expand(PCollection input) { - if (getMaxNumWritersPerBundle() < 0) { - return input - .apply( - "WritedUnshardedBundles", - ParDo.of(new WriteUnshardedTempFilesFn(null, destinationCoder)) - .withSideInputs(getSideInputs())) - .setCoder(fileResultCoder); - } TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); TupleTag, UserT>> unwrittenRecordsTag = new TupleTag<>("unwrittenRecords"); + Coder inputCoder = input.getCoder(); + if (getMaxNumWritersPerBundle() < 0) { + PCollectionTuple writeTuple = + input.apply( + "WritedUnshardedBundles", + ParDo.of(new WriteUnshardedTempFilesFn(null, destinationCoder, inputCoder)) + .withSideInputs(getSideInputs()) + .withOutputTags( + writtenRecordsTag, TupleTagList.of(ImmutableList.of(BAD_RECORD_TAG)))); + addErrorCollection(writeTuple); + return writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); + } + PCollectionTuple writeTuple = input.apply( "WriteUnshardedBundles", - ParDo.of(new WriteUnshardedTempFilesFn(unwrittenRecordsTag, destinationCoder)) + ParDo.of( + new WriteUnshardedTempFilesFn( + unwrittenRecordsTag, destinationCoder, inputCoder)) .withSideInputs(getSideInputs()) - .withOutputTags(writtenRecordsTag, TupleTagList.of(unwrittenRecordsTag))); + .withOutputTags( + writtenRecordsTag, + TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); + addErrorCollection(writeTuple); + PCollection> writtenBundleFiles = writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); // Any "spilled" elements are written using WriteShardedBundles. Assign shard numbers in // finalize to stay consistent with what WriteWindowedBundles does. - PCollection> writtenSpilledFiles = + PCollectionTuple spilledWriteTuple = writeTuple .get(unwrittenRecordsTag) .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) @@ -529,7 +567,15 @@ public PCollection> expand(PCollection input) { .apply("GroupUnwritten", GroupByKey.create()) .apply( "WriteUnwritten", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + + addErrorCollection(spilledWriteTuple); + + PCollection> writtenSpilledFiles = + spilledWriteTuple + .get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -556,6 +602,8 @@ private class WriteUnshardedTempFilesFn extends DoFn, UserT>> unwrittenRecordsTag; private final Coder destinationCoder; + private final Coder inputCoder; + // Initialized in startBundle() private @Nullable Map, Writer> writers; @@ -563,9 +611,11 @@ private class WriteUnshardedTempFilesFn extends DoFn, UserT>> unwrittenRecordsTag, - Coder destinationCoder) { + Coder destinationCoder, + Coder inputCoder) { this.unwrittenRecordsTag = unwrittenRecordsTag; this.destinationCoder = destinationCoder; + this.inputCoder = inputCoder; } @StartBundle @@ -575,7 +625,9 @@ public void startBundle(StartBundleContext c) { } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + public void processElement( + ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); PaneInfo paneInfo = c.pane(); // If we are doing windowed writes, we need to ensure that we have separate files for @@ -583,7 +635,12 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except // destinations go to different writers. // In the case of unwindowed writes, the window and the pane will always be the same, and // the map will only have a single element. - DestinationT destination = getDynamicDestinations().getDestination(c.element()); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(c.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; + } + DestinationT destination = maybeDestination.destination; WriterKey key = new WriterKey<>(window, c.pane(), destination); Writer writer = writers.get(key); if (writer == null) { @@ -607,15 +664,22 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } else { spilledShardNum = (spilledShardNum + 1) % SPILLED_RECORD_SHARDING_FACTOR; } - c.output( - unwrittenRecordsTag, - KV.of( - ShardedKey.of(hashDestination(destination, destinationCoder), spilledShardNum), - c.element())); + outputReceiver + .get(unwrittenRecordsTag) + .output( + KV.of( + ShardedKey.of( + hashDestination(destination, destinationCoder), spilledShardNum), + c.element())); return; } } - writeOrClose(writer, getDynamicDestinations().formatRecord(c.element())); + OutputT formattedRecord = + formatRecordWithErrorHandling(c.element(), outputReceiver, inputCoder); + if (formattedRecord == null) { + return; + } + writeOrClose(writer, formattedRecord); } @FinishBundle @@ -701,6 +765,56 @@ private static int hashDestination( .asInt(); } + private static class MaybeDestination { + final DestinationT destination; + final boolean isValid; + + MaybeDestination(DestinationT destination, boolean isValid) { + this.destination = destination; + this.isValid = isValid; + } + } + // Utility method to get the dynamic destination based on a record. Returns a MaybeDestination + // because some implementations of dynamic destinations return null, despite this being prohibited + // by the interface + private MaybeDestination getDestinationWithErrorHandling( + UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception { + try { + return new MaybeDestination<>(getDynamicDestinations().getDestination(input), true); + } catch (Exception e) { + getBadRecordRouter() + .route( + outputReceiver, input, inputCoder, e, "Unable to get dynamic destination for record"); + return new MaybeDestination<>(null, false); + } + } + + // Utility method to format a record based on the dynamic destination. If the operation fails, and + // is output to the bad record router, this returns null + private @Nullable OutputT formatRecordWithErrorHandling( + UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception { + try { + return getDynamicDestinations().formatRecord(input); + } catch (Exception e) { + getBadRecordRouter() + .route( + outputReceiver, + input, + inputCoder, + e, + "Unable to format record for Dynamic Destination"); + return null; + } + } + + private void addErrorCollection(PCollectionTuple sourceTuple) { + getBadRecordErrorHandler() + .addErrorCollection( + sourceTuple + .get(BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(sourceTuple.getPipeline()))); + } + private class WriteShardedBundlesToTempFiles extends PTransform, PCollection>> { private final Coder destinationCoder; @@ -728,17 +842,32 @@ public PCollection> expand(PCollection input) { ? new RandomShardingFunction(destinationCoder) : getShardingFunction(); - return input - .apply( + TupleTag, UserT>> shardedRecords = new TupleTag<>("shardedRecords"); + TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); + + PCollectionTuple shardedFiles = + input.apply( "ApplyShardingKey", - ParDo.of(new ApplyShardingFunctionFn(shardingFunction, numShardsView)) - .withSideInputs(shardingSideInputs)) - .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) - .apply("GroupIntoShards", GroupByKey.create()) - .apply( - "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) - .setCoder(fileResultCoder); + ParDo.of( + new ApplyShardingFunctionFn( + shardingFunction, numShardsView, input.getCoder())) + .withSideInputs(shardingSideInputs) + .withOutputTags(shardedRecords, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(shardedFiles); + + PCollectionTuple writtenFiles = + shardedFiles + .get(shardedRecords) + .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) + .apply("GroupIntoShards", GroupByKey.create()) + .apply( + "WriteShardsIntoTempFiles", + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(writtenFiles); + + return writtenFiles.get(writtenRecordsTag).setCoder(fileResultCoder); } } @@ -763,22 +892,18 @@ public PCollection>> expand(PCollection inp // // TODO(https://github.com/apache/beam/issues/20928): The implementation doesn't currently // work with merging windows. + TupleTag> shardTag = new TupleTag<>("shardTag"); + + PCollectionTuple shardedElements = + input.apply( + "KeyedByDestinationHash", + ParDo.of(new KeyByDestinationHash(input.getCoder(), destinationCoder)) + .withOutputTags(shardTag, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(shardedElements); + PCollection, Iterable>> shardedInput = - input - .apply( - "KeyedByDestinationHash", - ParDo.of( - new DoFn>() { - @ProcessElement - public void processElement(@Element UserT element, ProcessContext context) - throws Exception { - getDynamicDestinations().setSideInputAccessorFromProcessContext(context); - DestinationT destination = - getDynamicDestinations().getDestination(context.element()); - context.output( - KV.of(hashDestination(destination, destinationCoder), element)); - } - })) + shardedElements + .get(shardTag) .setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder())) .apply( "ShardAndBatch", @@ -791,8 +916,9 @@ public void processElement(@Element UserT element, ProcessContext context) org.apache.beam.sdk.util.ShardedKey.Coder.of(VarIntCoder.of()), IterableCoder.of(input.getCoder()))); + TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); // Write grouped elements to temp files. - PCollection> tempFiles = + PCollectionTuple writtenFiles = shardedInput .apply( "AddDummyShard", @@ -816,7 +942,15 @@ public KV, Iterable> apply( ShardedKeyCoder.of(VarIntCoder.of()), IterableCoder.of(input.getCoder()))) .apply( "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + + addErrorCollection(writtenFiles); + + PCollection> tempFiles = + writtenFiles + .get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -865,6 +999,32 @@ public void processElement( } } + private class KeyByDestinationHash extends DoFn> { + + private final Coder inputCoder; + + private final Coder destinationCoder; + + public KeyByDestinationHash(Coder inputCoder, Coder destinationCoder) { + this.inputCoder = inputCoder; + this.destinationCoder = destinationCoder; + } + + @ProcessElement + public void processElement( + @Element UserT element, ProcessContext context, MultiOutputReceiver outputReceiver) + throws Exception { + getDynamicDestinations().setSideInputAccessorFromProcessContext(context); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; + } + DestinationT destination = maybeDestination.destination; + context.output(KV.of(hashDestination(destination, destinationCoder), element)); + } + } + private class RandomShardingFunction implements ShardingFunction { private final Coder destinationCoder; @@ -903,15 +1063,20 @@ private class ApplyShardingFunctionFn extends DoFn private final ShardingFunction shardingFn; private final @Nullable PCollectionView numShardsView; + private final Coder inputCoder; + ApplyShardingFunctionFn( ShardingFunction shardingFn, - @Nullable PCollectionView numShardsView) { + @Nullable PCollectionView numShardsView, + Coder inputCoder) { this.numShardsView = numShardsView; this.shardingFn = shardingFn; + this.inputCoder = inputCoder; } @ProcessElement - public void processElement(ProcessContext context) throws Exception { + public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(context); final int shardCount; if (numShardsView != null) { @@ -927,7 +1092,12 @@ public void processElement(ProcessContext context) throws Exception { + " Got %s", shardCount); - DestinationT destination = getDynamicDestinations().getDestination(context.element()); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; + } + DestinationT destination = maybeDestination.destination; ShardedKey shardKey = shardingFn.assignShardKey(destination, context.element(), shardCount); context.output(KV.of(shardKey, context.element())); @@ -936,6 +1106,13 @@ public void processElement(ProcessContext context) throws Exception { private class WriteShardsIntoTempFilesFn extends DoFn, Iterable>, FileResult> { + + private final Coder inputCoder; + + public WriteShardsIntoTempFilesFn(Coder inputCoder) { + this.inputCoder = inputCoder; + } + private transient List> closeFutures = new ArrayList<>(); private transient List>> deferredOutput = new ArrayList<>(); @@ -949,14 +1126,21 @@ private void readObject(java.io.ObjectInputStream in) } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + public void processElement( + ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); // Since we key by a 32-bit hash of the destination, there might be multiple destinations // in this iterable. The number of destinations is generally very small (1000s or less), so // there will rarely be hash collisions. Map> writers = Maps.newHashMap(); for (UserT input : c.element().getValue()) { - DestinationT destination = getDynamicDestinations().getDestination(input); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(input, outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + continue; + } + DestinationT destination = maybeDestination.destination; Writer writer = writers.get(destination); if (writer == null) { String uuid = UUID.randomUUID().toString(); @@ -971,7 +1155,12 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except writer.open(uuid); writers.put(destination, writer); } - writeOrClose(writer, getDynamicDestinations().formatRecord(input)); + + OutputT formattedRecord = formatRecordWithErrorHandling(input, outputReceiver, inputCoder); + if (formattedRecord == null) { + continue; + } + writeOrClose(writer, formattedRecord); } // Ensure that we clean-up any prior writers that were being closed as part of this bundle diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java index e02965b72022..cf040470d608 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java @@ -119,6 +119,10 @@ private void readObject(ObjectInputStream aInputStream) @Override public void addErrorCollection(PCollection errorCollection) { + if (isClosed()) { + throw new IllegalStateException( + "Error collections cannot be added after Error Handler is closed"); + } errorCollections.add(errorCollection); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 39cb612f2d89..2db20b92f27f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -61,6 +61,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactoryTest.TestPipelineOptions; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.UsesTestStream; @@ -78,6 +79,8 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -634,6 +637,134 @@ private void testDynamicDestinationsHelper(boolean bounded, boolean emptyShards) } } + // Test FailingDynamicDestinations class. Expects user values to be string-encoded integers. + // Throws exceptions when trying to format records or get destinations based on the mod + // of the element + static class FailingTestDestinations extends DynamicDestinations { + private ResourceId baseOutputDirectory; + + FailingTestDestinations(ResourceId baseOutputDirectory) { + this.baseOutputDirectory = baseOutputDirectory; + } + + @Override + public String formatRecord(String record) { + int value = Integer.valueOf(record); + // deterministically fail to format 1/3rd of records + if (value % 3 == 0) { + throw new RuntimeException("Failed To Format Record"); + } + return "record_" + record; + } + + @Override + public Integer getDestination(String element) { + int value = Integer.valueOf(element); + // deterministically fail to find the destination for 1/7th of records + if (value % 7 == 0) { + throw new RuntimeException("Failed To Get Destination"); + } + return value % 5; + } + + @Override + public Integer getDefaultDestination() { + return 0; + } + + @Override + public FilenamePolicy getFilenamePolicy(Integer destination) { + return new PerWindowFiles( + baseOutputDirectory.resolve("file_" + destination, StandardResolveOptions.RESOLVE_FILE), + "simple"); + } + } + + @Test + @Category(NeedsRunner.class) + public void testFailingDynamicDestinationsBounded() throws Exception { + testFailingDynamicDestinationsHelper(true, false); + } + + @Test + @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) + public void testFailingDynamicDestinationsUnbounded() throws Exception { + testFailingDynamicDestinationsHelper(false, false); + } + + @Test + @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) + public void testFailingDynamicDestinationsAutosharding() throws Exception { + testFailingDynamicDestinationsHelper(false, true); + } + + private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autosharding) + throws IOException { + FailingTestDestinations dynamicDestinations = + new FailingTestDestinations(getBaseOutputDirectory()); + SimpleSink sink = + new SimpleSink<>(getBaseOutputDirectory(), dynamicDestinations, Compression.UNCOMPRESSED); + + // Flag to validate that the pipeline options are passed to the Sink. + WriteOptions options = TestPipeline.testingPipelineOptions().as(WriteOptions.class); + options.setTestFlag("test_value"); + Pipeline p = TestPipeline.create(options); + + final int numInputs = 100; + long expectedFailures = 0; + List inputs = Lists.newArrayList(); + for (int i = 0; i < numInputs; ++i) { + inputs.add(Integer.toString(i)); + if (i % 7 == 0 || i % 3 == 0) { + expectedFailures++; + } + } + // Prepare timestamps for the elements. + List timestamps = new ArrayList<>(); + for (long i = 0; i < inputs.size(); i++) { + timestamps.add(i + 1); + } + + BadRecordErrorHandler> errorHandler = + p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + int numShards = autosharding ? 0 : 2; + WriteFiles writeFiles = + WriteFiles.to(sink).withNumShards(numShards).withBadRecordErrorHandler(errorHandler); + + PCollection input = p.apply(Create.timestamped(inputs, timestamps)); + WriteFilesResult res; + if (!bounded) { + input.setIsBoundedInternal(IsBounded.UNBOUNDED); + input = input.apply(Window.into(FixedWindows.of(Duration.standardDays(1)))); + res = input.apply(writeFiles.withWindowedWrites()); + } else { + res = input.apply(writeFiles); + } + + errorHandler.close(); + + PAssert.thatSingleton(errorHandler.getOutput()).isEqualTo(expectedFailures); + + res.getPerDestinationOutputFilenames().apply(new VerifyFilesExist<>()); + p.run(); + + for (int i = 0; i < 5; ++i) { + ResourceId base = + getBaseOutputDirectory().resolve("file_" + i, StandardResolveOptions.RESOLVE_FILE); + List expected = Lists.newArrayList(); + for (int j = i; j < numInputs; j += 5) { + if (j % 3 != 0 && j % 7 != 0) { + expected.add("record_" + j); + } + } + checkFileContents( + base.toString(), + expected, + Optional.fromNullable(autosharding ? null : numShards), + bounded /* expectRemovedTempDirectory */); + } + } + @Test public void testShardedDisplayData() { DynamicDestinations dynamicDestinations = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java new file mode 100644 index 000000000000..41367765b920 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.errorhandling; + +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Duration; + +public class ErrorHandlingTestUtils { + public static class ErrorSinkTransform + extends PTransform, PCollection> { + + @Override + public @UnknownKeyFor @NonNull @Initialized PCollection expand( + PCollection input) { + if (input.isBounded() == IsBounded.BOUNDED) { + return input.apply("Combine", Combine.globally(Count.combineFn())); + } else { + return input + .apply("Window", Window.into(FixedWindows.of(Duration.standardDays(1)))) + .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); + } + } + } +} diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java index a65db5a90bad..2e4939560ad1 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java @@ -69,6 +69,8 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -337,6 +339,10 @@ * events.apply("WriteAvros", AvroIO.writeCustomTypeToGenericRecords() * .to(new UserDynamicAvroDestinations(userToSchemaMap))); * } + * + *

    Error handling for writing records that are malformed can be handled by using {@link + * TypedWrite#withBadRecordErrorHandler(ErrorHandler)}. See documentation in {@link FileIO} for + * details on usage */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) @@ -1427,6 +1433,8 @@ public abstract static class TypedWrite abstract AvroSink.@Nullable DatumWriterFactory getDatumWriterFactory(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + /** * The codec used to encode the blocks in the Avro file. String value drawn from those in * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html @@ -1489,6 +1497,9 @@ abstract Builder setDynamicDestinations( abstract Builder setDatumWriterFactory( AvroSink.DatumWriterFactory datumWriterFactory); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + abstract TypedWrite build(); } @@ -1713,6 +1724,12 @@ public TypedWrite withMetadata(Map return toBuilder().setMetadata(ImmutableMap.copyOf(metadata)).build(); } + /** See {@link FileIO.Write#withBadRecordErrorHandler(ErrorHandler)} for details on usage. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return toBuilder().setBadRecordErrorHandler(errorHandler).build(); + } + DynamicAvroDestinations resolveDynamicDestinations() { DynamicAvroDestinations dynamicDestinations = getDynamicDestinations(); @@ -1782,6 +1799,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { write = write.withNoSpilling(); } + if (getBadRecordErrorHandler() != null) { + write = write.withBadRecordErrorHandler(getBadRecordErrorHandler()); + } return input.apply("Write", write); } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 5b976687f2c1..ab6ac52e318d 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.common.IOITHelper; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; -import org.apache.beam.sdk.io.kafka.KafkaIOTest.ErrorSinkTransform; import org.apache.beam.sdk.io.kafka.KafkaIOTest.FailingLongSerializer; import org.apache.beam.sdk.io.kafka.ReadFromKafkaDoFnTest.FailingDeserializer; import org.apache.beam.sdk.io.synthetic.SyntheticBoundedSource; @@ -77,6 +76,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index b0df82bcdc19..9b15b86051f5 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -88,7 +88,6 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Distinct; @@ -97,15 +96,13 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; @@ -145,10 +142,7 @@ import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.hamcrest.collection.IsIterableWithSize; import org.joda.time.Duration; @@ -1472,18 +1466,6 @@ public void testSinkWithSerializationErrors() throws Exception { } } - public static class ErrorSinkTransform - extends PTransform, PCollection> { - - @Override - public @UnknownKeyFor @NonNull @Initialized PCollection expand( - PCollection input) { - return input - .apply("Window", Window.into(CalendarWindows.years(1))) - .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); - } - } - @Test public void testValuesSink() throws Exception { // similar to testSink(), but use values()' interface. From 7342b03ff4e62076344a9b9f7caabb86d2f78d02 Mon Sep 17 00:00:00 2001 From: Talat UYARER Date: Fri, 29 Dec 2023 18:19:11 -0800 Subject: [PATCH 191/224] Creating a Fully Managed Beam Streaming System with Flink Runner on Kubernetes - Part 2 (#29794) * Initial version of Part 2 * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Removed Trailing whitespace * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Removed more section and a descriptive sentence for checkpointing section --------- Co-authored-by: Talat UYARER Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../apache-beam-flink-and-kubernetes-part2.md | 158 ++++++++++++++++++ .../adaptive-timeout-kafka.png | Bin 0 -> 880831 bytes ...checkpoint_monitoring-history-subtasks.png | Bin 0 -> 170772 bytes .../flink-backlog-metrics.png | Bin 0 -> 580860 bytes .../flink-partition-assignment.png | Bin 0 -> 66575 bytes .../kafkaio-wait-reader.png | Bin 0 -> 399857 bytes 6 files changed, 158 insertions(+) create mode 100644 website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part2.md create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/adaptive-timeout-kafka.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/checkpoint_monitoring-history-subtasks.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-backlog-metrics.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-partition-assignment.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/kafkaio-wait-reader.png diff --git a/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part2.md b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part2.md new file mode 100644 index 000000000000..ceba71a4401d --- /dev/null +++ b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part2.md @@ -0,0 +1,158 @@ +--- +title: "Build a scalable, self-managed streaming infrastructure with Beam and Flink: Tackling Autoscaling Challenges - Part 2" +date: 2023-12-18 09:00:00 -0400 +categories: + - blog +authors: + - talat +--- + + +# Build a scalable, self-managed streaming infrastructure with Flink: Tackling Autoscaling Challenges - Part 2 + + +Welcome to Part 2 of our in-depth series about building and managing a service for Apache Beam Flink on Kubernetes. In this segment, we're taking a closer look at the hurdles we encountered while implementing autoscaling. These challenges weren't just roadblocks. They were opportunities for us to innovate and enhance our system. Let’s break down these issues, understand their context, and explore the solutions we developed. + +## Understand Apache Beam backlog metrics in the Flink runner environment + +**The Challenge:** In our current setup, we are using Apache Flink for processing data streams. However, we've encountered a puzzling issue: our Flink job isn't showing the backlog metrics from Apache Beam. These metrics are critical for understanding the state and performance of our data pipelines. + +**What We Found:** Interestingly, we noticed that the metrics are actually being generated in `KafkaIO`, which is a part of our data pipeline that handles Kafka streams. But when we try to monitor these metrics through the Apache Flink Metric system, we can't find them. We suspected that there might be an issue with the integration (or 'wiring') between Apache Beam and Apache Flink. + +**Digging Deeper:** On closer inspection, we found that the metrics should be emitted during the 'Checkpointing' phase of the data stream processing. During this crucial step, the system takes a snapshot of the stream's state, and the metrics are typically metrics that are generated for unbounded sources. Unbounded sources are sources that continuously stream data, like Kafka. + +**A Potential Solution:** We believe the root of the problem lies in how the metric context is set during the checkpointing phase. A disconnect appears to prevent the Beam metrics from being properly captured in the Flink Metric system. We proposed a fix for this issue, which you can review and contribute to on our GitHub pull request: [Apache Beam PR #29793](https://github.com/apache/beam/pull/29793). + + + + + +## Overcoming challenges in checkpoint size reduction for autoscaling Beam jobs + +In this section we will discuss strategies for reducing the size of checkpoints in autoscaling Apache Beam jobs, focusing on efficient checkpointing in Apache Flink and optimizing bundle sizes and PipelineOptions to manage frequent checkpoint timeouts and large-scale job requirements. + +### Understand the basics of checkpointing in Apache Flink +In stream processing, maintaining state consistency and fault tolerance is crucial. Apache Flink achieves this through a process called *checkpointing*. Checkpointing periodically captures the state of a job's operators and stores it in a stable storage location, like Google Cloud Storage or AWS S3. Specifically, Flink checkpoints a job every ten seconds and allows up to one minute for this process to complete. This process is vital for ensuring that, in case of failures, the job can resume from the last checkpoint, providing exactly-once semantics and fault tolerance. + +### The role of bundles in Apache Beam +Apache Beam introduces the concept of a *bundle*. A bundle is essentially a group of elements that are processed together. This step enhances processing efficiency and throughput by reducing the overhead of handling each element separately. For more information, see [Bundling and persistence](https://beam.apache.org/documentation/runtime/model/#bundling-and-persistence). In the Flink runner [default configuration](https://beam.apache.org/releases/javadoc/2.52.0/org/apache/beam/runners/flink/FlinkPipelineOptions.html#getMaxBundleSize--), a bundle's default size is 1000 elements with a one-second timeout. However, based on our performance tests, we adjusted the bundle size to *10,000 elements with a 10-second timeout*. + +### Challenge: frequent checkpoint timeouts +When we configured checkpointing every 10 seconds, we faced frequent checkpoint timeouts, often exceeding 1 minute. This was due to the large size of the checkpoints. + +### Solution: Manage checkpoint size +In Apache Beam Flink jobs, the `finishBundleBeforeCheckpointing` option plays a pivotal role. When enabled, it ensures that all bundles are completely processed before initiating a checkpoint. This results in checkpoints that only contain the state post-bundle completion, significantly reducing checkpoint size. Initially, our checkpoints were around 2 MB per pipeline. With this change, they consistently dropped to 150 KB. + +### Address the checkpoint size in large-scale jobs +Despite reducing checkpoint sizes, a 150 KB checkpoint every ten seconds can still be substantial, especially in jobs that run multiple pipelines. For instance, with 100 pipelines in a single job, this size balloons to 15 MB per 10-second interval. + +### Further optimization: reduce checkpoint size with PipelineOptions +We discovered that due to a specific issue (BEAM-8577), our Flink runner was including our large `PipelineOptions` objects in every checkpoint. We solved this problem by removing unnecessary application-related options from `PipelineOptions`, further reducing the checkpoint size to a more manageable 10 KB per pipeline. + + +## Kafka Reader wait time: solving autoscaling challenges in Beam jobs + +### Understand unaligned checkpointing + +In our system, we use unaligned checkpointing to speed up the process of checkpointing, which is essential for ensuring data consistency in distributed systems. However, when we activated the `finishBundleBeforeCheckpointing` feature, we began facing checkpoint timeout issues and delays in checkpointing steps. Apache Beam leverages Apache Flink's legacy source implementation for processing unbounded sources. In Flink, tasks are categorized into two types: source tasks and non-source tasks. + +- **Source tasks**: fetch data from external systems into a Flink job +- **Non-source tasks**: process the incoming data + + + +In the standard configuration, non-source tasks check for an available buffer before pulling data. If source tasks don't perform this check, they might experience checkpointing delays in writing data to the output buffer. This delay affects the efficiency of unaligned checkpoints, which are only recognized by legacy source tasks when an output buffer is available. + +### Address the challenge with UnboundedSourceWrapper in Beam + +To solve this problem, Apache Flink introduced a new source implementation that operates in a pull mode. In this mode, a task checks for a free buffer before fetching data, aligning with the approach of non-source tasks. + +However, the legacy source, still used by Apache Beam's Flink Runner, operates in a push mode. It sends data to downstream tasks immediately. This setup might create bottlenecks when buffers are full, causing delays in detecting unaligned checkpoint barriers. + +### Our solution + +Despite its deprecation, Apache Beam's Flink Runner still uses the legacy source implementation. To address its issues, we implemented our modifications and the quick workarounds suggested in [FLINK-26759](https://issues.apache.org/jira/browse/FLINK-26759). These enhancements are detailed in our [Pull Request](#). You can also find more information about unaligned checkpoint issues in the [Flink Unaligned Checkpoint](https://blog.51cto.com/u_14286418/7000028) blog post. + + + +## Address slow reads in high-traffic scenarios + +In our journey with Apache Beam and the Flink Runner, we encountered a significant challenge similar to one documented in the post [How Intuit Debug Consumer Lags in Apache Beam](https://antonio-si.medium.com/how-intuit-debug-consumer-lags-in-apache-beam-22ca3b39602e) by [Antonio Si](https://antonio-si.medium.com/) in his experience at Intuit. Their real-time data processing pipelines had increasing Kafka consumer lag, particularly with topics experiencing high message traffic. This issue was traced to Apache Beam's handling of Kafka partitions through `UnboundedSourceWrapper` and `KafkaUnboundedReader`. Specifically, for topics with lower traffic, the processing thread paused unnecessarily, delaying the processing of high-traffic topics. We faced a parallel situation in our system, where the imbalance in processing speeds between high- and low-traffic topics led to inefficiencies. + + + +To resolve this issue, we developed an innovative solution: an adaptive timeout strategy in `KafkaIO`. This strategy dynamically adjusts the timeout duration based on the traffic of each topic. For low-traffic topics, it shortens the timeout, preventing unnecessary delays. For high-traffic topics, it extends the timeout, providing more processing opportunities. This approach is detailed in our recent pull request. + +## Unbalanced partition distribution in Beam job autoscaling + +At the heart of this system is the adaptive scheduler, a component designed for rapid resource allocation. It intelligently adjusts the number of parallel tasks (parallelism) a job performs based on the availability of computing slots. These slots are like individual workstations, each capable of handling certain parts of the job. + +However, we encountered a problem. Our jobs consist of multiple independent pipelines, each needing its own set of resources. Initially, the system tended to overburden the first few workers by assigning them more tasks, while others remained underutilized. This issue was due to the way Flink allocated tasks, favoring the first workers for each pipeline. + + + +To address this issue, we developed a custom patch for Flink's *SlotSharingSlotAllocator*, a component responsible for task distribution. This patch ensures a more balanced workload distribution across all available workers, improving efficiency and preventing bottlenecks. +With this improvement, each worker gets a fair share of tasks, leading to better resource utilization and smoother operation of our Beam Jobs. + +## Drain support in Kubernetes Operator with Flink + +### The challenge + +In the world of data processing with Apache Flink, a common task is to manage and update data-processing jobs. These jobs could be either stateful, where they remember past data, or stateless, where they don't. + +In the past, when we needed to update or delete a Flink job managed by the Kubernetes Operator, the system saved the current state of the job using a savepoint or checkpoint. However, a crucial step was missing: the system didn't stop the job from processing new data (this is what we mean by draining the job). This oversight could lead to two major issues: +1. **For stateful jobs:** potential data inconsistencies, because the job might process new data that wasn't accounted for in the savepoint +2. **For stateless jobs:** data duplication, because the job might reprocess data it already processed + +### The solution: drain function + +This is where the update referenced as [FLINK-32700](https://issues.apache.org/jira/browse/FLINK-32700) is needed. This update introduced a drain function. Think of it as telling the job, "Finish what you're currently processing, but don't take on anything new." Here's how it works: +1. **Stop new data:** The job stops reading new input. +2. **Mark the source:** The job marks the source with an infinite watermark. Think of this watermark as a marker that tells the system that there's no more new data to process. +3. **Propagate through the pipeline:** This marker is then passed through the job's processing pipeline, ensuring that every part of the job knows not to expect any new data. + +This seemingly small change has a big impact. It ensures that when a job is updated or deleted, the data it processes remains consistent and accurate. This is crucial for any data-processing task, because it maintains the integrity and reliability of the data. Furthermore, in cases where the drainage fails, you can cancel the job without needing a savepoint, which adds a layer of flexibility and safety to the whole process. + +## Conclusion + +As we conclude Part 2 of our series on building and managing Apache Beam Flink services on Kubernetes, it's evident that the journey of implementing autoscaling has been both challenging and enlightening. The obstacles we faced, from understanding Apache Beam backlog metrics in the Flink Runner environment to addressing slow reads in high-traffic scenarios, pushed us to develop innovative solutions and deepen our understanding of streaming infrastructure. + +Our exploration into the intricacies of checkpointing, Kafka Reader wait times, and unbalanced partition distribution revealed the complexities of autoscaling Beam jobs. These challenges prompted us to devise strategies like the adaptive timeout in `KafkaIO` and the balanced workload distribution in Flink's `SlotSharingSlotAllocator`. Additionally, the introduction of the drain support in Kubernetes Operator with Flink marks a significant advancement in managing stateful and stateless jobs effectively. + +This journey has not only enhanced the robustness and efficiency of our system but has also contributed valuable insights to the broader community working with Apache Beam and Flink. We hope that our experiences and solutions will aid others facing similar challenges in their projects. + +Stay tuned for our next blog post, where we'll delve into the specifics of autoscaling in Apache Beam. We'll break down the concepts, strategies, and best practices to effectively scale your Beam jobs. Thank you for following our series, and we look forward to sharing more of our journey and learnings with you. + +## Acknowledgements + +This is a large effort to build the new infrastructure and to migrate the large customer based applications from cloud provider managed streaming infrastructure to self-managed, Flink-based infrastructure at scale. Thanks the Palo Alto Networks CDL streaming team who helped to make this happen: Kishore Pola, Andrew Park, Hemant Kumar, Manan Mangal, Helen Jiang, Mandy Wang, Praveen Kumar Pasupuleti, JM Teo, Rishabh Kedia, Talat Uyarer, Naitk Dani, and David He. + +--- + +**Explore More:** + +- [Part 1: Introduction to Building and Managing Apache Beam Flink Services on Kubernetes](https://beam.apache.org/blog/apache-beam-flink-and-kubernetes/) + +*Join the conversation and share your experiences on our [Community](https://beam.apache.org/community/) or contribute to our ongoing projects on [GitHub](https://github.com/apache/beam). Your feedback is invaluable. If you have any comments or questions about this series, please feel free to reach out to us via [User Mailist](https://beam.apache.org/community/contact-us/)* + +*Stay connected with us for more updates and insights into Apache Beam, Flink, and Kubernetes.* diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/adaptive-timeout-kafka.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/adaptive-timeout-kafka.png new file mode 100644 index 0000000000000000000000000000000000000000..24626421b9f0f7dec7450c3c37b57f8016082df2 GIT binary patch literal 880831 zcmeEuc_5T++jmK&R7fPUltK~7PPP_OwzBVK&ze03lXlyvB-s_&m+Uc^$}-6^31c^f z>|+u$gE4&P)qUUZ^WHJf+xtAv```Chy5^d5IhNn?JC5@>CsbEkefLhzotrjo+O2Wn z+?7q6n5Q;v+6H6T2A)(r#G8PBHhW!BKeMTz?cn65O?;a)&Z+9(w451)Jx#O=+Wy_@ z0yBdP-5J(*cl%Orhiv0id4KejitFtoH&P8{jp@S&O4&D?xt-Z7#Mj3eo@l%|oKGDl z5_@}K*QebsTOLn;-Fth$)U8}?U(4sxBX3F7Tmmk|5(icwR^^s^wCg{%UB0Zk1OFLCfb~5(3uL$F$l3px>=>qQZ2ON5Hp$VE}!`HZA%I-7YuwrqCcb(%q zKb;b7ICzb{=TKgn0n*uOV_xyejqbL5U;YT`((gi|)DDKptkBjYbo4`P%-F9&8*gD# z2o?mDcFv$u#@~%99JV1W{A#;?uss!2N$uJyub_~oD^$8HjgO+Cu+q|7sfcH8g{>^V z>69wRfr7J-bRGIsOJ{OHogMr2Jgr?z7J{+I@JTVOzd80><{gkBjI|{xoHOz|Ku2$C z078DQFShotIQDl1q$Yu|R;pKlkD;-=(;VCoVieJyrTze5%yMSBr ztKVMxCkBQo4o(N7L!b6Q=J2AKbYbJc^F{cw(qfR#?tkCR#xFzfJUR!YuvhIo1M@=$ zRc^#)8V3C@xE?SsarzIUs;OG^f0{~@n`<-;VU{KXkzl8@xqsVKtbwV7s>bktYJ?>6 zW~p3z<4vA81SE2_&yn?aC35mP7;u}Okt-ekDEMm4vTMc0U;PvSUNkMt^gXxn%}|?U3Z}o93D} zMfCp0urs^LkxQFHX!ONyIa9Z0u*HH`(zLd?Z69;z*IVmcVPluNwJBsL9R*)uC1P4C zRMFAvki?x2h%Kyc+g?W-pib9;K(mdZJ)|7W)Beo0;`;A`RhAmCYbn#ly^dmFWPF!9 z@+gg<@xS82x~eL=5BSgAQjr6rwr!&!<^OTcUri%~6M#+UE&H7z(kxlhSvDXDi956< zO1Bqj(b(|6Vtn*IN46OtR2U%}d12qTlg8>l0FU9{{^ZcFi=^>7-=`tS1CeXKI#>B1 zscfh5;=d&opnieKa$3^(UI5sedI$}0S#O^D0Gz%)1SzvNP-lrM4gc-C1Z)UK?02K# zZR@5>-3InAl^S;jU>*n_8fi4~u38wRub0<(7~K|N&^MH5jSGX!y)d}{1P!xXx6@8O z0Q=h{^%zP+-?r$qMidmlJPGBxLgySD9A@XX(GbmMx=1Ko6fIwq`fbO#1ssR-^|5E? zQdNO|B`@%c+*xN+f7s@(#~?l)y6p>i3yppL@n7YS1U9-k!CHf!6Jif{B;TJ}_s$>71f&ZPm&1;+zgzVQ8Dj%lt*$BY zq`Py1FU#ijy{?TT!oMqS5QslK8|>)l;LEMpMME+wsir`PU10}wepfQCfp!12N?N8yIR6J>!an{??40;e_rLp`ae>!E|8pL8MfS{wX0t@jZ!HDTi&w!CHm9`7eu+d)}hfn3|$*pP;9G4B2 zN-**YZSwOoIb$4}w^Y2KoUg6cM2gl9pnM|VbL_VnY(FH;1jBxzkesdbKKferbTmeG zbOH*}8dM6WiRwdi-!f_k3h+X^fw`1CqhD4v{z- zGFZ3r`@k5{U;G`u?l<|QT2ttBXYW)^@)G_1OR2g-PiwsA9L}iR&s8X(k&@$~_`zM} z5=CP@zxp4aZY=PUs>O@iwclSd4&P>+W}3KwI>ol5^u7vvUwdzFFK0p#tt|bM3M8wn z1J7t?o4@WZ@)(H7u2y%w(~0_#fX7~=&0{YB&GFY{hyCtGK3UUsZS&DgSq*(d#rB(N z9%}Uk(|G9LQ?Ufo_`izaU&lcHfQ%(zRHI&}!hTmlea67os>T?nE;wZG4XLJ6R8;hT zz^T4YvVTtZ7u};V`>)#rS;}qs&)kpWQsNI-Sdi0&v-WPLWBaq$3iGm$z@h@)CdlFaXL~)T{UpXYnV! zx=8;6z}N?2yMup?_RD6npaMeiV-{MI|MEKDr?H>1+N8=oDPnN=kr9~6^4JIsG*Phw zKo%1ju`W*hYNLP79wNOTUl!W;vZO?Ue3KUS*%a~va=JY zvCf44P$lO@$V7dC2mHrPl%5V?LDxk z*-OLZQzM`f6s`RI?{*4lbr)cX{DX7AUfZs``bRBe!_hgQWWH{iFty>Q>uiqYPz5XRs8+#WJf0Yxwy^e7un}h?*d_=Z_`lv zt`soZ*AhRw-$h$TNH>%5ojX;zGm`_HT2hqZDo?L>WHdZ8b32fx9)-}78I+Xnxq8RA zCcr1+pxi0xfg5#0R=p|ek(_M&Y7xd1G9(@M@93T)8yMVfUZ&SJ`opRcB`7UYdW!1B zPG8)>ZkFyGaZw9O@Fz{}-)*dkPiKHznr6k7|27++gdASNl>`JgbTdunLK=&uUWR69 zZegvw?~wnvnlVCZ7j#^7Xq%rL4RL%@2S_(U3+dKJ-EAPGKAycP3DZT}L@8o*-*K2Y zeqa=uhgl(K$VL&IO4tmohw=ZvtVT^LSpu#%^E=(RN|)cAoVc+xW@V zPY}eXxdJDA*=XT6$hJnbT4UpKd-Ju=DorhNiaP({F+U8BSyiBT&5?IY_wWS-1PsSJ z7lgV8j&M=0GAS;6l3o0AXJih|8})!USe7SX;>E=5tZ4789jpTNx3@A{lV*A+In%Fn zK$VMlLldEm@u4&o=#kwe=frQ9LgXR;6)`)OmBtf;B-AQ^LNsbG)9LRjGv)nm(<@Zx z!Wwq5nuROXi^Jh_?-|7wwgqV>$xCHuC9s=(e_#ZIN;8<+ej242$@^eEjT3ha)}!#o zPgL%J0WWlOsB$}tG$&=FdEuvh`>tnte>q;32Znm)njg9G)>K#_$=nt&IlQrC)=&2n z>%2*~<*~*}q#m21`8EaeouIKehv7Qj)f@6RVoagJSaUhHabu;Y?f^=cRE-vSys;In z{}x1j)a7B&-r(~IxOzh~*DfZ~t60l0pyl)}?yoP3)h=3}CU)wH+qXRL0T5wb|Bz*O z`TH>w1UPb^jEUK}0y^7*ke3hnqK zAhsk`xfcEkoV4VrTLz3Vr|F|Mb7#^0v9fNWK>?U2^-df zsCKEq_Qz;bk@w{1Be~VfQ@JA(X;HIA&U3>JMctOLqm^p>Y%HDa7Y1Lm(*QylAPhX# zgF@>V_18s*xi}OsozEfX zX=@>DQR!(H_+<<4!`a!{V4vEhR`*6D)N)tVl*}+lG$VW1oDZ+l5yV77d;#lk7fmYH z*?8&e*a~x&253}bhxrvl@{Livyd#Z~;i0U8jb9(|@^oZq<-$%dDX)CL1*=+H zrPz=M{R)@l-D`-3E+{IghH{L!+*KIdh0F}GG!a7AkcYT{mAI}9??*iZvxp`KFHe@~ z6)&+0XigBDg7L-8q8>9v=tlblSHtPE+)!sf^<-<}F7zc`^o=Z}01YoR<_9Ub#(Ep0^<)c_z5AV}_sJ^?9Ep^9 znUjLu0eLL^jekrUWnH~&Y{0f%g*QxLIIbCD!1AkOKV9iePR*BwJVmCTvbu#5y zo;ZAUW$~j??G-&LD97~J#aav(p?QSm*0M8`!|)ZjS{HoC8Koa06F@W5yO%f+v4BzQ z`q4T+e+@CK=ncJV!)^abAsHZzcz*lX+(L&= zw}BYr^qViY8HXS8BeE*B*|WrhmKUJx0L6(u=%s1y|1CzUtM=P7Giz*~$2XYpTzI~2cLM(VL+ylf3gP>Lf z0`VlHmk1i8Yyd19bm#fpPDPl0YoW!ZiBN45r@Qrc5zg@%6uaVuLF}?)XBT(s%nE+J zwt`tZ2zcF1i!c6Xw?IG-CT&v)6N)0 za>UU~&lVC^r>)lH%ln*ib3U$)D6iTidVkh9Iu630a;P$;M0uKA(n7tJc36Or)vYLPQ!uKF9vMI&Y9QX7{2W2S<&KmP*}J5`u?6&VWIh(M%Ri#b_s{J77Oyd z{gw}zf+o%$MH5hKs~+7Jeng6ij*iYjzX6Xe_q}aNnJK{m?zc25y8%hz)lsH;H}Q&D zR?Cw5oC^miSq`bGh=7F_T6RkW94ZTgartdMdq8^T6v-Mv(@1yE%`5bzQ#C0n9qnz`T@iFTi`|a&1LfZ>C{^$0_6TP$({f7Ld zLQ*_G9-G6v!;zx{psYaNX691dB*5nEUNPuvSEp3mbnna|;9J7jV^VyKy->jL_VX*8 zA8`pH4|wSm#49hpV;Y`MSqp3`g47jAowg#7OZHpdJfyVnf+;2+-k~MaFwI1L<aX&!6)FW;L~r$}c8~*< zJ2-^xFJLsdgM$%jipWE-*hHD@pVQw!vAX$0MV>JkqAV7^y|0Gf03skv!ok-L0$jPcsrqOQA$(JIF{pwIujW8M&^_gPsPSB?Tz7xCHZ`8Ez2H}iuY z^(-DA4m~&rYDNaSBj_lI!W2q4G)5fo08N8N27+9HeYw#3fFh3ftgSaGG znLgYDaEo*w4(dC_!#%|9wuiV)jM2-5UTt!MZ2*{yiG~H{2TT|m>zI_yJzGYilz9~J zrFy6o>T)+~7O$7x9V(d2fuqj)dEpN+?&t;4%qz0B4gGFAZ=$nH{vFVucie$z!*IyJ zoMi>#2lf8(b=+m0vi~p|6-EH)7rF=RnWobdF-G~OGg8uhj%mTl)8Hc8AUY~|A?-=G+3TySKXDxH zTen-L{@Md>EWsgKPzM1*7@t}cPm(i}No1a3a(B8$v~_Q>-5fwmG0gY)Mu1a4o6I-d z;6J*GY0;p+`%bC>1lZFqCEI;mfD6y9%wuA=#>u!_CIVCv_nqkpbF^uU2nC6OPh%vP zjA$@1*UBKV-|sDajSNR)&D;mw+hkBnt&3AesQ6s){~9)c<3{c`&JP)tV{&#i$349lz!?qo1hd2Ci1H<@~18v)<9Bdm(Sej$SFk7E&l}ZGRaIQft zxX9`%8p!?tRH|)~FRWd5GWfdZ+iiJm`N?$S*H_=&a<%~F-Gg2&;&W-dRH5$Vi@O7c zLj|n?m(Fuqg);b|lL#ocoClHYpvfcj5R3p&(L=rExpp!cM7sAH`#PC`Btn`;i0TFQ}GcROho06LYBeCiRyYWg|I^-1p3cjc zzMNt2@c@+{(l*}+ZiVvwUVwerZ4r8Q!i*horn7DfKnu#CMiljA#0rq+9*}HZT%Cz? zNDgJyIW4}5$F1hil&$^{TRWw#v^4r$-p8p)H`2wr9F^c!bqrM0W~CgqposbI4c9Z1 z41Ki?EA$_!d%UmqpC2CZ!sOp<^;G4C&&$AB_ge*eG%Z27pu6#f^0g34Kmg6n114WV zNk?%V;`EOXZ$Kh!9jzsu{RZo_4IqJ9v+>3z!1o9$UzzH3Z3q(^Bpj&?P!ZGF5g-9{ zSOE?$ELktGpN8@Z<1!}$cUiSc=CHq1HPC%?>07MLaDC`ev<|>dx7ww#9{=$q7`ztb zi8nz;It=okaNgDL54Kp2q~6&R20&620zoMMm?^<1Z9!n(3n3IqJp zD;eT`BcSpa{+M6gu-cOS(T_nwaK-4G4jpAhaDjvivcPI8sUIpu)?N)ZC;FT;8pEuCaI|6VxbW25y{%DjQjEhN`!KPM4Td2pR+_9cp+O(mzi_ zn(GASc<3k3@10TQ{x41=|7dSaF>@rqxPlz#gMr`Q&2^-#md+%K9I*u|YvK+Q2Ut3E zHPl6WjtGE57ENs#y=K_1xmB5<0MlL8t`WatI^1V`SAXDoB{C-{pISfjvY@xe0@cExR zx+4G%sy{ih(LVG2dt41MgpM!@!hzegcXY6>eP>!@hCJ1sqXFnt;i~M}6P{L-zV2ZV zM+%*$-rr;7XY9T(C;$EJmJvY1Biqx|$-?9cIHjUS_28Bos1i^QksuTqhBESygQ|+4 zEG#zMFzq6qJ(!sks|E|sTu=v*# z72%&OY0z-j)I{z)RP=@NFev8p_Eu(-Vz$cIc_G0s2K@260Jbl92{0TbC#jE%xp>gG z-$1cY_Um7X-|z~Ej>1aRNgZ^g%#g9i3@E1Y0%LY;@L=388gW3vewS@oQ&*3kS?_U4XEohWelAPrcDk;m$wACCLpnHR%vnxX~KfDi~Hgk`{vTD=7}E{Sgn zsRF#FQ!9&xx38B&X)!Pd??G2$Dx&sE)af4=%)MHrJb*K*gTll4`m`t|`P%s;0>qIU zioR{L?M?(FUc{>Qv|r>*72r8;n(>my$cYBIh7FI^c@BW=bnuoJ55RyPQ2X-gs`<0b zLTv=(j~BmtYcC*to?XXegcvzMMME!V=QKzV&zO6+p9gGQy*}%5nx~brjxieKLL;C) z|IxyW8Ul)U84|RL?46)%JI(FP*5Ahan4iqgO<}9gzAnQ;OKy15T!%12(wwm3Q^p-U%bCASuzx(I>KNEA!1*E|5E{%<-;yB>IoV6?JGUba@ZD zSeZO*0HV3rrVJ%SP_Y(VT^ubyf%^AzWq5G)>hkpCti^EHGgvw z$`qjB(ECC!Q#%qIew35D)wo>svu&GFK)JNe16Y3J7N*mm2;(Lk&450zz-EfnwaQ8~ zz?mti;Z4lU7JCW5!xn@DIKH4I$q$15t2T--xAECH>VDL&o-I zkwO+-{BFx%Bpd6hdnESW8xp67LzTwyLXj$9IyL8arPH5?> zPj2%8B{gtu=)<7mSGvmVT`m&Pd_$L9eN|j&)Hk<*n*-?7AIE9jU|qk8R#9F2mbu>G ztv*j@t5Om6a2aEG8iPK}YHw$|P*MqvE=5PEtup`9c^EAk_Ul9;s;oc7cL<2D;w)Gq zx8A*e&D1{1lT?JX0$(|e{`WiJN`Z~8KihBLRsXpK;%jb(sldWFgYf9lgfZ3Ljb!_3M0DB9LX9oCmVA z*y^LN(LXs zk2bcUpLY(pWN_HumRzw)8B@Ij8}L$Jw$95-)dy7hqEyPrM^(A~^&@FD6#cKSLg=UV zZ6B5p01{&g2YwmE+Ux5HL5|mWTcn)V*b%D-w|{l)KXELT8;Y_v-T2>b5cd3O+J*p1 zuUo2eS4LTM>^3iTyxfuRf0Er?Xy3Lp7gYRDL*J!gW=`V=f%>2(;{?e*>ED*4bchr0 zLrekCRH+ZRtWKooWr)hcwf5pCRZNxoI8;1TXxuJ4tud(RpinmPmdM$J z>>}_-+9oB18ItS=YsrsY#&s0S#uTok1-dc0%9#H=cr?`qJMC+9>l2rERRtB0Ea=|k z;m4b4sDrNp0NQ~5#urRe@2bpg?pNFK@&iwpMY&7s*~iCtA~fe7;i=@pkV{1wPnm); zdd$j`Me@Gd@a8JN=5MI$Gl{f*e!hmPQE{CxTl`(4P?_A2jrsMlVNJg z4~gPczt#BE`_`$KYf>~K1IqOV*7S^-8e#d5+kDf4)G{XGezre+dJ2)(au5}v0?;Uw zxT-gC(13zRP+O9cVA_U0u4|e~s9Po~fu_Vm%}<$1_n(!X@@JOnH!9EA$26TWWLCrU zGIIu=vpqgsEwZ#iO`$)V2dmS&*6Dy8qFnY+>`f|6Cc~wbiM<7W?%B&J_dgmLxpZm z@c_{Ae;V;G`SG`{oZ=4;cWsG0c8B4La-Ip;T>imxpZwg3vQnOF)4LCAbgeyA%YHB4 zcg;QW^V&@Gi^~L8BIm2q{n?SG{_ezql9KtJJ>)f)6yjZxbnukW4_;HNt&qC1Mx{K2 z0IUz{J@0!4u1HO1bB3G-n)o4ffJWyL{ST0pd>FUgw;{nf&X^v|EZ0>RPgMGph_5yv zOLH(yW2Xll4>3maoQtl}k(WP?Qoa;<0mz&1H~@2*pKzk9=EU^%oCn0|?C5-cV%_pF ziWOmq6Yr8)t!`gtnD_94kAPm6M*G@6*d>2n4I+<5S%$vPwrMI(D8)5<`Pp$5Q3GbJ zkA(8Fpe^Fz(01bg@Fj3H;osfPj~$XfwnP$@@&-S`E>Zacd`+kPuOeA^&z<1xqHOZd zdEi4)vbDhUY9BcFWci>SmUucZDXFg0IcMD5VH)T~Eifv5=c5T{eVw(fETfbGZ7Pq3 ze^g%PS`f~iSnbX=ckoqpg_P6UG?nM^XUCJX{ls85F|+b9Dm3Qxrvx{fpJFb(1-!jf zYhUMMtezB|Kd5?+UOshyo^Ah~K<~Lpc$ah;!F`BsTvSh%A#ytw1!R{Jt=RT8z%rO2lW`YLH zs)i-W56bjv)j5}_R#BJj69e3{+b-9TKRhL`y<>QpBWP=La%hcMjpJ+&I6144J?nZ* zssGB>%jr8;HHbB?V58Z~hJ>2@#n6-*D|vaRwH2A49X_0_mT8jB z3h31+&p;<>?{qIX#-2ZwryvUZP`vn&$0?5(lhYQ)#?nAFC`x`((~5oC)jVa(6pURy zj%^z2))O|n)SewCZ%4*f7_u2opG*(Qh0tsQKMpd+)>!t;GD| zi-%;|t_oc``sE!9u!mAzXUntC_+II1@=dmJ_vhJU$+i5XtA&yp zEY6TgSfpoCD|B_leSdysytV)OsD{aFrW3w_GzRm8B z$!o4GK&|OhbqoZ}{E7@u)J!1>`py>(G6_RJY8`}$s1_4b<FtxvjsEtA&)(O+WCwdFxbuq0@h}OUViq_MXeT=AeeF z&jSfX0l;V_A^UuH{mE;@I}d{x6|2pMxZAdD2%ikh|LhuVr;zTh5vU~9$=OmwS`!@{ zY0RS%s-3IdzrOc5FE5W>B2@c0 z*-GVy(k~}rW@n{x2{8LGc*QD?+?y%Ubxx~7{|^_tdV&7qs>0P8f*B07hF0O$n)BRT zYi4kay(lj5D;n?n2-iTLBl(0yp8~82iDGI{+rEqNfmG$v8((IRs)oEY75aHD0}lhK z#rNR$HKS;CYL47CR`9!{;97J&kojTC;7_dH$fGZSA3WjD&;0LV{Vxr5@;y>)F?5wx zpL_R4Ph0L4IgH9v-OTe`_`XZvjLq~mZvB$RS|6vNRQxKXn?BI?8ly?g3`Sw*kvp4? zVi)BL?g?mZ5j&F3!S|%gpS&4a{BBoo&rC@5OrK3-Jvd4y5BW6HD>3+g=#E$8GWhYe zx50+CtA;aGv2XDx?h9)3$0_c) zsZF^*lehnMG}68IY+jI65cYAdnn$qj+9Wdhw9mzAbhl**0W%b21xoK|es@A~XamI- z$73LYBA7I-CU#}}23viq{dysY6x&=ao7EE6Vv$Bzrqn)IrkC1noI9}E%@lmxC|zIY zB0hhB;*`4dDvu&hE^DUL;uQpQyT&C1EBtXK_-gm8ybgfm!zMGQ|(O3}AjWXvqO zL2A#*wsEf^HBK3Qx|)<`IlB6=X_Il7S|w}^N$9Ws8qoXwdV>3^rLGzd<~ju5VvFl4 z=|(Hpj*$UFoZ~t5iTZlc{oWNEfr9m~z&?q!G#!^vU|iq|V~!d|#npPw6gewV!rvYC zClJs2WSKtzVQM!h*V#C7c6N2etemk}3)bn9$|TH(t~n5b0*S&x<95WwGRimPIld{TXzVxnJkM>Kdy}tQXv3-c0 z;r*@Hg^#IPE$0l>iF)lUg2UBu1~Pla%l)W9j1+9e4@DX5KrQZH8$Ki9FR^Rv;QofDtIW4H(erKE z%%HM~Zrz_Dg)DEg?xfRvWA@(L0y%Z(mdE+S=Fz)raA=&=^x|j7kXEC5XqNj0qH!00+XJo>XywVP(d_DlS5@mxR4w%eQ>)b)eWQ$^`GSAxEC|lB!bh zZcTx59P<=>xz!zOzcuOWodibB`U<6Iocx-RJl^f{g810*2#xbSPFTi?cvZlWP3nOcR+L^u1Qu8&*Wq2J$ITR2 zQH3S5&K-!9wH5FArEzzA*b%oo=HK>0B9D5!NsT*7JST4lT`d21!dhKQ^6CO^ZSuut z_{s>cO`gZYgEFccXy(84H5&y6AfuB7&u;wImo)y@Hf>)~pQtPSI3-2u9pxSew(|kf z?-2|qB$g)X13aF^w4Zn2v-V0^`Al?5>?(v!pXSeW6e_N3X|xDs$-F$kW^6wakf*sQ z^b*sZaVWYQ17B4;cp#EaVB0p-V#B1MZ(+^M>I_@MsFfwTmOd`}sMqd4Eb2?`j?+@n zhv5b=$HesKd_l{TnS@|@@efNs;(t_6I-KI_ZDVt~y%27dn7o<>Ul^n6D4C3O%xj%? z*P11_r?tl?dHG_Tm9RkxF7+jzern0dWkeBIvVNd|17#{c^tvUxL=pzRB<5c5J@CS* ziG%ZdC`w&PCsN?kW`!vMUS9KVZa6=QFH4YJ7d!=WGUDXO*cu{y=+S8q#V#JIzoVCtJPZ%G^)J@;_j=G^A=ono2W zIAS!^Nf9%#W%#O4M+sKRDHss0QT0ltz;fbgi$KwCYZ28YR98ZBW?86-{kMx3xK_x4Cplj8q8iPf| zVf)A8rf1z($B1#QZKj17cz7)N=rnIdMMbl-Rf6wIBj|m@f7$v|%*TMxFYIDaxq5z6 z+MXX<)jB~7DI4axC1lYbuRagDq3pe}2-33Xx9*DNO+4-`b#b}S>s^081QQtZY z0rW%=#Xg#8Q~L6>EzBvejHyH%YY6Nhdiq;4B_EEkvVQcVs!%JLcROg|aM1huTFDB4 z7}QGa?{)s9t-pg)E=JOiii;7~lvI1zeLnj6&R}U?e?kY2QsdDxl+A*i-dmK=sOd0S zR54C5^=+J6oL+&i&UbXcS24Iw_1Gu_{34}R(BA8_yymMaj62#@eq?nRh2@j*&tby8McN{`pql7; z4MX=T6n>%k5;#U*Y7o;F1MN9h-5$>Mb1O?tq#+&9Z=+M8yr`Lplnf*iWL;fb9rab ziFaGi2#_6|=gElr;O>XoKg^B)>IFDXy47;k*@4FZ*B_bGdTB|kRlcgnq3qeW)yGm4 zOK1Dt3S{)@k8Q$(meVPPZKnr0icnSlAH0BZh3FUrxTI$3*MN5sq((&OiH?y zUphEl2R@GG>}{>90I^>WmULFJ3hVSu*i zeXiJ@AwhKG96o_$BuOq-h-2Dktm%h}N{<>j_=DmePh(!B7hhh?0QuhSE8|HkxL z>%#BJSJ#pTti+WMFP0SvRHCLf_aO)P?9f@SrjUAvyp3Q^2pi%HHBJS&XWyFK2JLIG zBPcVZe9B}p%ZPCoUXn@5Hy+(#o);0GA=QiT9jqM~wCajSPkji^ZSf;f3}!N_M>xix zd)v8Wo_6I4SaYXhyA?Ez5(dXTT*Wi}#^NoIUTYM@;NwMS+(KUxLXTLJRAbv$kvo6E z4}O|!LKYdMpiqGnZ)&d%T4?btkV6!lr(oKY^uMhh79b6}*wud>3$<@7TIP9h4L!gL1CHf=OC_~r~z6@36BGN8as5?Olm;VTbi7{f*u)U;^g&M!CbwirrV=jd>T$sn}kd20f^S zhW$GZ39h8cnr@{i41d-o0|fj`YeyAFWl>6QDXK*GE`&iVSBVHPJ(w7{^CKosyT`nEZZ!|f;o%Nx7FBQ*m)OOom zUliK?__N62Ye;wiA#CkwMbzU1x;+l+^?f{<7ST^Hmc|uX9vIcU`=WN1|MuXN(Zt~M zFG3|Q+6qCRwqn$Opp#keO<2@S4!mqqYng7@ceJ22g?CBo?m|H8N86X2xvaa;mG(Z! zy4)P(U=`8BddAoD7pK%z-0+USrS+H>s(gLv?iO{4&m!5Sv_5?XKe_Cebiglu@bI;7VTo zU>V7t0rQ)I6%EM?t2Kg*z6 zFgM|NGLp`jSJ0+^&hp#qo!SPAljOX55^GU;Bx_+pm=bCjV0-Kun2MJSl`&!OnY?}fZwyTY;4?R$U@^h9R zn|O$o?oS|w-q+h-5`Q7)cd4lq} zJG=s>o(35@Kif|FQRDPkVk9`-8IP?5CK4Y``SvQHJW>IFHb$et7KaF;JZx(0Jo4&e z(ImC4>Qql}WKhK5o_TURxhw11mF*sD@_UgV&9mi&&o?4rnq}(vdeQzh*IKIFU(8dm{w|DCuo*J6*jbkCFk4d^4NC`cBa{@%T_JiuvhGp$8e3&+c>Mma&s# z^O#F@dniNTV%JS53;J31z&B9 zJpQq_7bO$Eg|}kHn?n#$W>M4ZZX%Hv5*4a3~56 zUpT=0Pr%LgC*^2p)qFn$mK}IJzG;=q%4|5#A|zi)gu>=Z8K`#^Si#f7gYzv4 zhmBC~*7aZJO?K2vl-$qeIQ`z>Me2?a~l^Zw>=cG{1cE`A!-v zQ0L0bE$yyMe~STWfZ%`P-gkCds??R zw)V10+zUNfeM;)2F4DT!C2vrCrc(4xDV%r7j_6ogB(Xg*YR7)+lHQhRZIQtsH)q7l zuPrWzB&g3v&xzc6K26F47mB8GxP47lGGAVU59@bYvUv)=nZ-tjzA8hwti+`H99=Av z^CRL0Q#?LP;TeKm^DZv*dxWh>cqMSBZRZt3lte3=xN1$Ez$Q|;QxwE|=WRmqaQ`h&s~R_HxW~=6JrIlBA)29>$Q*=_8XJ9nH$iX)?`;VYLP~X5N=&bLbHA&qWDe zcH0-YjO!gR%HT`lGv5L~)}9hsIWIhB^5eXCiA4!NPb>Bm_t}PIQEC0J)&~{ZoDVOS zR(f|gSlurSlMGXjVla1|(ack{V;}c*TY1@oL#Kq1Bz!FfRt^Xkm0_c~$a%x4DJcW0 zpMO!#{)BLG&*&KLcdnxncHHA$wyP$*SBh38nT)8F(}PKsS6ZR~qjgTHNA>3xD`nd6 z9;D{3qPz;DSa&15SKO*&#IfGE0upjksmDU5sZe9k)}Gt6f;*R7=C0FrqD#qpcJ&1r zhMO9`?2%>tI!DI}=3CfSNABiG6DE(;_@Bw*K)la zZOGi*u3B!4S&^R~tS)D0I9Vam%13Q5n|c#lj#AJz;nBpX_11LNX3tGOw{5iRT744_ zzo|_O`m{@-MP*wF!a(l}L}FIt&j^jV1f*g)`4(El)kr0E zpPeE+xT>8A%lW`@ zmrI0;tFX9pwN*F!`B#@a<+G*OY>_7Dqr<%L(BLQ2RplS98+~`SvD-$cRCqlq%4UY) zfzR&CdAKgO?%1GJ16Ta>eYQSILBTOm>RWN+oJSS|y|om?!^yCrS<;c#1QAlNrTI7b zGfL9;emQMyGLZw5avha-cNyse&#oGb?9M6Vyp!bhDDvCN@huSsD}Di1hRA5DA$eOf zTYtB8fsUtO?tYw1RJ|qHTxzm6?vmYWi8!4uvhVWOF!+FuVcB55oJ6|roy4Yqk;)UZ z;T_AE#yBPA>0S?>4yrQt&bTl7b4q50qY00nLrxySVLBJv5eb)1Dkj{w$9l}8;ei39 z!ZqR~xcYaZW_&fsBVY&|djRsnbPE*6nfNYfve1z)fG7&Pi%B~~<4ovhDS)g3y>+XN zpT=9?bs0(pxN9GU-n`*d{wUPPuOVByQ0S52<}8fg@h=_YNJp>3J$mz-r3_y$X{B&? z50mZH_8C|k%w&qk4Y*_4&OJQ7IA7-N9Ci4LRk!}=b8)>Qy#B(0tyv0~Ze!c&Nsb9> zoTI0MTTIjau}noB=OYQ=pjUIn*{08{3WoRZQIKy{097Ok~*ui1AgaR27C&8W91X*sFEa zNcw?Ggt@sT@=e$b;+s=WS=g?)8@3dblKOb>x9q4GV)|r_$F6Tkg~p>1riYaY_c~5r ziRMaL^Tt`m2{$PIxDc#gBU+0qDr}LjCLt+C^Rk~yVzlfYb+jHU#%SVlg5@Wh6^ioP z6yo09=QNXBYw|ya@EL7+mefB}gYdzsr-}EKk{vV?>lO}PY0!z(GlZ$I_jesQS>XDa z*RVzoW>{pDU(4Tj0@Typ3H5nt{oj`k(lOt31m1bmk;SH#dGR&)bMbYxKPL_R{3CqU zLh%;I^2c$>-;O%|sQ$dULPv7`fJs)Q^vzh-2KomqFHCkj)a>1_eb+|*<<72^*ev|K zMZ4X;!OAb6&?yYeL6-8^b1q3c7XBY=?-|x~+Ig&>e3LI^EHL5V;Lh$M8*FEexA|MQ;nIPaJD zTdrLBz{P#!xQv>4~C9dB=!VU|}1O*<&JJ{8~kNFGhJ z&qd}s*@tKC$CeuOrk)O?Y_-ep3{1=Ptr@F`^q<%utaEij|7yEZZOH4p?cI+DJ*NLA z$88(o-+g16K3gLyeEa$RseZ1PrTV=JIw`r_rLGk0QlLH6_DPm>*(z+re&cuk=)$5h z#BK})4!I%hsdxA8?_s#Wj(wcJonue2pgv7>Jo9D$T>Cn<|7tDMN|da|Us?+-OX=nt zz8Yim>Kp}K6Rh^|o5Vo_T+_EH!MEl?)7QNL2-h2A#?9#6cv?}6AC#Ce$L^CaF*+na z32Mw*81&1E^9ia!`gGH7XT;+06d^ZgF9kb|$STK8DZ8+_9yfUHMx?1|AAtb1!5i*^ z(U&IV$%dh{;+y-*-nD>-!Y{ds@eCkC0Zh&L1CE(^4?{w^}Bu>0q zYLuVVRuSX~X!7*E%n+ma_X-bveeY}d zyE^-dV_lHPqFlC{72Rhf%D%s%QS#PSa>r12MY!jVWs$U78Fiz_uCTVJxCP60g3-mW z^vc%CoCG=!W|ypA*s-$4@U1cV+T4&?&JNycl(g~T zX0z|Zlit~+p}PFmw_QJAQlzw5nzL?tv}qWLA*i@CL{1tBU#?A+7$;@KWxaowQpt2= zZnB$aZEx_j44+QR)o=}}_w+>Q-r#B7sQJA#$Wt?$i_c2PnMk9zt_5ha)0^s`@KRx) zpH&z^8m>;RCz_4FLKyLuO|2-GsdJ~|j=jUcKPn5Ts0GFP-gl)f7<|{%Ri@i~Gc*n* zSJpPaBv;f>z^y@9Mp@XI4LZ2&&7rhcpi?q1XqAZ*tk`VcWyjl)B_F8*K>bSz_bz>n z?CM$ z38dcTv|s4)|Gn(K$&@ID7}du9YPfu1+GUn$Zcyh@L`n+E*8?TZFl}c2bTq4G#~joS zyrx^q=WgGnI(?&c*`J1<(M)3}eg(DG?g%3;*C3FT7zLJ$Vjkty_$6-Iwrk@i*>#fR z2l8OjYz2H+$8S60Shw6ee9tl`Hio-(n;*duBGMd&$qf%%(~{t+Z1c@Ixly`UHDGaj z<8qwl%b?L?P1Mi=q{1W|SC8?mb|2^W--Lz>TLfo&O~5h7AfpiLaH&hb*r^t`;77J2^DJ8>iO?#{%?pEM$W9p=3Nu?$TxzaRM% zhW68yZN`z`zhKbDz?!HZ-+{##sYlys>N{$~+uSl|>LsftSMonq?{^Ci=!$i3mK@B! z)Z?0YIo=GeypOy2UC5}-LQ#IVLJ;~xY*gGqed$1}j`t9`z+JjG(FvD8t)T}xV!wA1 z4;1VpMOjJbOd4d)cC>vvN+xOR50()9Qxf0nb^A#lo_1D^+_&35rDzA~w^PD!Z;xx+ z3yaQ8oe>m(ls6_~1dM$%=djIq zpO~zBaz@MYCk(>hywD?nM+!OmWY^D(1RZCNB;f)rD_xt1+-|Cz;hF!j_8ejW%WIfz zGynaM?O&m74L~!!b2_u+UUn#CBD!(_Nr;HSlwHzZa#p7+WDA@OrJS;IeJg+X9 zLfLi%^du2)(;AXn!1*(&&BV;^+*d19+2ie)Plgqds%D-DpNr}ia9{l_5d3+WmEBy= zBGDJ>**kyVN~UElQo4@f@W~l3vKo!xrsApX{!}+G0-3mm-*0Sf-lu3!Up?W#AN!;H zXm^c<1>v!gq#~v$=~cl=szmF&AtG@n^aY$&Pb?RTmC~bs8btx>nz=$T07nM9q^l7XoL>*`uP^@oD+HakmiOQjom} zx8j8P-IK)=vY_l-Y*V;sAaF!utyi%sY27OZH#c;gL|p0qP8scW7Lkaigdufa4W=<2 zIoi-LPOpQ%u@W!^SBau2v`Lu>Xu7Ifz~_%%{IWR5Day&o@hgNYh}k2YSf+$NdeC|1 z)0FtU5J%n5{cHGR^;DXaBo5ctkEFBJt%vE`~aI2P^}UHJBf)H}qn+L8gJvMFtI z)|%^qHK&NMj-;ch$DG^*4IBNCw2C`)zM+5h=jkM^Z!+!6fh@gX>3;sqfJEmq97ID8 zJ|O=l$;>{}L%xegWa0E&4WtYR~e#)0X>A{XDVYA_P9>nOm z$e%rs>O>BTUsmR)BU`?b@Yw!lTI-@fCJr)2>JkL5)u#JY2FmM(pvje`!!#5QH+;3N z1*eZWA2!-Mmt9^n4Z{hpN@6Vqv=)0)AXO^Vqyf}8mTU4iZjuw@rkiaj&CRA7z(-S! zVCAj>DZ%(dX*^Z!bVHz4vD7 z;&6z4Pm9oKS!Dj&WqzNhYa*4{$Yz1HUlL8;r9NAU)C|JN?KB# z|Nn-G_$$WPM6%IQ#{*Y@LYku+ zE{desdIbs^-hW(U;gdq>=4|k|`O`eSEj~;?FEL?sLef%0`z2xfHWwK_+@U?p{zb9Y z-8BD8T3`P_9}3N_j%sc*=<1pCwJ^@!4o5&hd#Xq{t<|-KhW`8T^!)Xb69&mSsbAqa zZNKg4P0dX9Ruu)Tc&y@ni7wLY7&3Ww9=d7`@$}#s!OBx93&+U?+2L7-GB|D6N&LqK zs|N4;ak|c^)7=x}0^Ud^L0}T3x);>@duGYw32#`?pCPzizW_3szvU?zw~%h(ZI&oT zfa!CwQ|}#dY>z2oSCBh}oq;ZxkoM#0YVem+ff@58#;EgG3!i-;-J(um_o?ua53Ln* zSi#(AY77@?F&5y*kdgc8rqZdo;nwCM*CnK!qS8sn;fFY<~ViQ||4TYU8Fgs0(k;S$T>a!GnZu7_BJcNFG=KC;8Cpxo z@!JJzt&bZuK|;->>e8}OWvT9kW#iGOw5$%8kP(Ras!}Q?DTZ>xX$480r z!g>BG^17y0y74G3mNZs5ZW&+T6(K1qi_W~9-WCO_0d?nz)9E!A>ebbOBm zRifHC!P62oZFsIDcQl7*GH6&htzaZda*`rlz6}HDt)l5=hU`0J&T2UGni`mC6=gxz zJ`~^~_mp>jsH1TaNWMdotMYx+3-m~Qkf$qb;$BM{BRoTm{M3a)nlhGv`xp6)4btJT zkDzDH(ccGQ-jAB27r%S4&qs>o)rf6kf@+(MN7}x6GjYj1jKuEx*TTG4{Hv`9>`#nI zfUkGvOJ_zM$j{Pfr3@F*$3TS-YJC4D!qhseC9)(2=$$i~!w|Kz^%fcU%Pc=OYQ~i} z*mJ_annND_9Dtg@o4jN0qbFx*6%wYK&FPZh2?NGzucGc?aQJxE{4hU72OaMQe;#Ka ztheT`InS9k$S`TRiS#+=S;qS0@r2zr>8Xv=0Q(91In>b@#3S(VxcT%PpUh!1Tcl2 zW3m%_!Z(uu0khL0Ym@(L{5v*^crwxC-Sn;f$h;eW^kye#ES|`4n3Ft;RQAE3j`)sR zQbJ1S&08Z5)|6QfMVwl~RCW2&@({0Jhp7s2g=8AUN`<17bEPVnX^CoES^ScrZ=G-v zf-kCo7aJGK9{ET0D=de_+y*`FkL#Y6X0t+7omAOE-<&(@XDu>YXe%1-lz zn7dUThHu{hS9wgMNUNwQNU0vSrk*nD`Fh1Sn2z3|tfZPZPFqzAMQF{@!LAN5w!7JN{PEd0i(V zf&3f#p1C(|jGh$JI^pwnO<_}E5~3D+QlF?l!F}3Nx%?*`mVN#yW?q90D}r}YHa))z zV77~!8kxf+Mj5cyNp}#9Yt!ZZD3Y0mBZI@%b%JowBb_Awz1eatWfTV$6YnlFP{ypB zFq${uy__I7ZrR}1LX;T%l5${96%4Z8b+?c5f&AJq!c~AhTSI^t`mR4~O!9f!$Q8Fz zl&S~P;vP9b1?Z7=E#e!_ep(X;?z+~kE&tjh@4XCMh`pC{U_TjQ0Vpo>dwsx`IU%f8 z>1KZqRKssw?{jvu^Lsss7`tO4O8rYGM`guSm7S&*(g{Os%qMzY^Zp_0juP^KME7EJ zJ}SiM;uljXa-~82t$uPg?H$=EF$!dN>I7rGqAd*?3){HDXsQWuKHXU3>NIyspdrG! zVb{l!_3f>h9*C(iXzI&U8^+9)vYc|w9Qu{2Lji`M4c{O})z0OVC~bT=*_bxSY?NG2 zL`4ow|7h%#e~#PvW3?qS@?5U9YFl1XJR%nrKs40r6rtEIm^S$&(P4=Ix3|SFyN-Gu z+_wme`lryq>fX1Ik~@d9Vxe7TErU>$kTL@ADV}e&8a4x z?GXpR$H4uu`wYuymJC#;;dKgb5&^BGM17zc)r5HRmULaH&T7J^B*vriPXAc_wHDGc zDmF-Pw-KCE`5oD)>4mK0)UVp7sx>xi=UCT~11S`Fm|S@3a1e=lR}}yap&I=IK_(hl z8#DrSIUbFe5u>I%R565__;y-<+lU!}x%g&VDF1z+=$~(AjsiDe_Nd>S#!t8}=k0$5 zFQ#*jL$}t~*GGbq-&#sfBR0NfXc~tf3(9j>UVv&1^ldxa9)XSs^{zgxb-<=HvxiJG z5L_(_p5uCCp$oh=_VJlq%w?T#RifK5#J4?@bAi#ikI;82J$&e|@sOD&u$#+z-?vsl zXInuA!b>#V#0qUSwVw?lzuw;!U^0X{^$U&st*sHe$(3}Jp2`w+7NoADe1Ub6j>~hvDO^T1BcM zbg~0_m$#?S$m=i@w*B(&oIFwPNxDD1KeD220V*gNXrE?XdtaINgF-1A_Z67x$3F*{ zu~={8vE~oUBu032dqW5Co`E7gF^_+P<$73@I3=i>s< zNmoBl?^%JcJYoxlc0S57+_ekxB}^E|X@u*!e0@h788?v4!GIp}o0k*<;w{S9Wuo#W zxUn^l&v`Jq4lJtc1sW%2{;i7&J_CPu$pkS+eN*XO(~&z0n`v583oQ03t2Z4OHJf(W zR`VYu$bb2GOPt&#R8IX2V~-zwLAoXGm514}o(89-R=w{JO=`O4e7d9D#5K>${HESg zIbQbg9(vJy7eZwrXIo6ob3=P$)b@FvVef#qv&xo}gPI<&W5I6lT>6y{m{d3HSdonFLi(SegzZ9;G}Yoe;gj%X@OW1(P+ z#TsE#_7|qI!4|@F-zt2@gV8$ByZoTsz&K1ANMf-=QFhUa{uqa4lxua@$Ph76vkkeP zA78+e=F**2VYUjMoLzkk>wXw6rXF-3>GhD zKI4y3;Xf(WcYfX6_v&5Y+)yzO7YpOXHRDU#u}eM z&pA{=Ou_p;Y%rYL|8%|u_{Y|_J3GrtL1XlV=7-(|eDO7-M%^c?e6Qgk@ZC}2st4S7 zx_HgVJ-it@d7xok9{y*SbH@~YE~Gv5;|*|cf)%7Za^Ezo!<9FUGX?}h$CYu%&zGn} zCX%3RzO&u@3%@;)?0zF)?+TsoU+Q6|g}k<=SX9!}6+};dY#p>xZKTPs7N9yzE_vz? zg6xp7B5wU36#{S0Jw{|vvQz(76~f_Z`@90OD{HBy=B`fBWK^bh?u;Xk;cxtkiI|^Y z`ZR|}#TL_g2V<{MEvCuj;nX+=(!SZapZ+Qgq*Pj8G!R)83B)r7Yl0LKaFpVAyTE_h zRO>GSAj;X(7A3j4A|7Bln36kz_tXF7#25C7?Ew3pM{fBE?5cS9`N%Ao%Q_LZC3HJlT?QXAJ6Ze9i;>dW(Q&WV&B}UVm3d~w*qst-Y z%lq#GT_qnRpE&h8#2HfIS@GSxy;0AaXnmnmax}=mLsXL3*(2Zh{1zx_AST9it&~hq z=}Yi3x~N=1iGa=7w0=LnMYiU0OF&9r%9%m#jzYTDl`77~2RXp|Y?fA5zY^uZq;NTX zw>sfbx43@Q8u?OlQpoMWTsslDMGyy;mY%#R&pP0>pSVC#nvlbd8UhUSC_AT2X&k4a zGXFwlcdc6m#nW!W&@oW;*vb|2<}Q$cbVCWHmhNH7!`>@2!O7`ME?IjNj|smm3-%{_9* zL|nwN4yPsm<@kSp6MsY;@GmH!p9j1vu=rD~Z-O%_Wf23a>i6tb*FNg{c-&sSu&4mv zbXZ@~muL{Eye0WEd>XqdC?VWO=^18XjyTFC_ExYkbW5LZ9b2@UDY{~kP@LYz&(n{jD>+m@L>iorL ztNtrGL3nGQbCJZ0yPX?Hv&+rol298ggk2tW-MwUJ@6H%I{q#f!aDO(_(W`FiX=Xjk z|J|=N7mV#Hdo-^vR;uGl%1ArQ%)XT{+FP#L-{@+l<^Q4HTo^^lIWaKu+BAImyxlO) z&}$|CX+lS=97$;vpVRSlrE`f~;SSM#T=Na0V}y?Z9yM!W;pr5`G1-oQ7hu%}!>fdK z>j3_x^s=RRCX3dfE`%N$X7)3hU2;qUqSj2Rg=O)`mZ9jdv4ivymo z8*BI!*+VVCN+=vY3@S)1 znE@xggmRm^+Ap(RwqtZf>Ow8eAHh7$41y0FO*!fr*EZlUwam&+pMRmD5?OXXW;zZX zLVZ{=fnj=cw>PT#y5)XJ7Y62}^Hge_Nbq#d7x*lf znsfkcu#fj_V~|<83KVQoIrTXsH+{KW*pS)TF}3m~4})CiEA9tu2QTIZyvI+5=TtI+ zpkSBVK6E#XdK1YnFI(8Ou=*yi0EY3~qfpWi$o?QonNB;IO*Na&VJQmo*3;KRqGna& z?l!Kr?j3D-2Qi|Q{nL{v&E&21#u=@41B38&h4Wz2TwHGKkM$MbX~k23K}b62#HwqxP~DGBRW6;A7NOD|SFO>R@61MwP^<2fFAj{P2jUc~pQ`!=pCZB&f%`Dj8xMw!K zxqT5ybxOINQpUH3iMSNTJRjN|D)NE%%(bi|_2cvLgKb}whV@@pq~#8V{sz`)$z9K& zo2}n0%(jV_4g0%&vlM6kLf_NU+nmMjunJ8T$~J(J`&lkHGGi)k-%Ka9^?q_K0F@J< zcdUSA+m*0hz+s!K{&LRCQf;7S+NHQzX{ak@|Bpei=U|MQL3dk$5lp}F8@4C06*@DT z=mPzOqj1?Q<5JY(GEoG}Uzjx3*7|b}o951ho_p{ATNk||ku2vpYWJ3Y;)Q5B|4#M( z{^6Px=})R$kY{qi#FoL_u+4ck@A+@-_3>3C`!SaPZ#KJ#W7<+(0T9oa248}<$9syD z$7nb{`LSkLb8;!@#*>P0pDp?<`A~|6uoo~EqHjeQJ*Fyy&QEO~kj-M4up%;=aiIHX zVJR?1jfQSHBwPuiCyb~}^Y5O{RvtjzT*gICVs9g^%`^S?M8$9&B_-og@1#i3xJ0sS z=70-bvKCw4vB=HL34R|n zM>DQ*?rW)Y0_JjQulGjo@u#TZ-7q5EO4WcQ6~lvD^&^~kj1WF9(q83`|0jKPK&}0f ze0V)iSkEy0A^$*w6~DqYV4W|GN@~Cl^?F$H`Z7!`glWXQwI5z=$NB$60AdX+JHOTW z$ekGRJ!nClR{BXG{|)(<#gj_Quet|*azpAjZuI%^GQX(YO~rNlP5XcwPn&iwhf|ES zk9r@-uzK`CkDi%8vD`W`;O_tN^EAD!r)7eLSHttnBfiZI>eeTSEcyv zVUND61!3fF7Z)=gEm+*`Lv`rlZbfvtcw%K;Afl%ZxZbrI?c_V-X`MF;x#j3P>0u_ z$xZ}10&&Rlj2#|Far8K~WLi9N5v*0Y)RHDmSAz0=dq5T(mP;mVZ#j5amqf_IW9fyUmAF^H2f3t0!cgsLDJE z83bg{e(^L+v@ZbiqkYGW_io&q4AlL->aLW7{}$hOP61hfru=^OKTmjIbLGtc_5%Dz zYN)^aKgs}H>@s|n4`9k=^H59mu+cfg>erP1yA>h~N~gmCxoUYIMhF*~>N z9D5DDA8&G5>HBu>LX^#qpt<6_FUY4J7(RziUvn2i(53`jBO)Y+idX z^NzdPB{%BV5nw@7hmN_dm&dq+h0TI|o=c+MY7F(a?%VpAGXru(R;P0k%Zq@)w`R$j z-_>FTN^6o&33QlppWddV(zp}n^*v_f*b*z&5R6K`z#$nYpD-R7Tt$eY=S(~yBjSl> zH|D~Ia5V#wc}{2#FGGj+xDJCO{h_&ea7VPF*K$aT-^lsKQAwtg3A-}qdnIadRC~~D zOeU=IP)q3JO^v5?za>udJc^N6etb~c#JKl(+uY42pXy-i6azbE8RQqc77(~7 z!FQi(PTOLxXW17|?`AlsDQBJ`Q&-aC0K+n3_^ax|dkoQb?}4vTWD0wjHyp;^151qw z8=Z~Hp(%k$10e32l8RGTbjQGH&~-=521HCY%#nwPbm)uKLFs5XsbsLwd33SI$XRUC zm;U|zzI_pGK$J(cwR-#g=jr~(iT?HR-b3K`uYUbWepfX7n_K_<4?VY<1e$%G)ZpxJ z*y8$ZjIy0eYR&$@Y?S7XN4&)hw@88cRAXAn`-+>y7Bi_`%O+?C`wJXa1p3I(g(cyx zW#dEd@^I&fzN7g4BbIGV!W9~&m4iwgg?izRg%MRJ+HJWd`0MzxwZ_N}=Z=Kg;f6Zx z!GX>Wh>kpyutfObXL6xtw|>3NLG_+O_F*d0*1<{1^{@#gcitIIIJ|Aslz;b97OWvZ4D)5dBA zkNtQsMCHYUnURlaM?tM0MJn<}H4l9Y-RIo}Jh^e-U?}KSg(wuZdp)|#q`O1O(-u+N zYo@j-@v>%8$Q-_GojjC~QbfMSYT+%@9M;$jZfY`D&7 zQ*q{oE?^@SeHSjKW@@;%CJ)OSxpr>3!-}2C2I;7{tl(N?m#^wu8Be!IP1BA3$Bx^`3=NC%_Rj^v-M$^qOifa7seC=~J_7fF}Oj@6&Y%}pqS zzIIz}j6AT~SkY)1t#IstAYQM2PvQK3>S zCh2qFo~I$?{)!u|m2?wH?9rd1n*$FNj|AnUc1Qzna*`{V2F|1kHv)e(^~QevGs#K){L6Y3Z;na8 zvHPjx^1A+sWA*}~xsILXBc@u7$My;Jb6CswojT4 zFt@gchsiKbI~OT^JAhhwS1aiSsOHZglV0caMc+94$IgyuW_R@ z9c#uS{%9N!MUP+zZ84E|U4@1NiUFFSh$`k(17q+Ae>Y>U=4KCwX|)P9x%E+9H>{`c ztt|J8hmPLdEHS&7R_Pez4STu3GPm4zJB8MCIST^OOh zdPgx@d1PZzC;Qj>hnHeh_cC$Lc$={XP%(_lBD9?T6$vx|nZzLLi0Eg&n56B0d5{O2 z0hQNeII&(w(g&9Rk>+qDZCqt*rr{+$SLHJ`(8g_*l4hj~Ba@H|4`Q3|CSTkeM%|Mf z5##e(qCFCQ?{RX-?I!9^_kWNo7YCF2z!|9qF#Wh$4rIx@zs^|1tqI!0 z5&puvr>H3aD-!*z>?f32jz;`H9VYqt9e}Td?XMv4d*MGKgEB*N!;5BF~ z@r;qZXk~G(mrAAPkSu+@-&)6%ER;&dK~)afK!yMa=p^|vS|uX%DO}mKF9iCba;)8| zY%-vmtQ~6sIT%RcHlS}YD^l~Dr(%(A9aS(}Qk$uJ#l!~M=L83RGL3w9!)LXvF5ikt zQ!O`_)%F&=yz1?Lt=yXVV)2ji+unLNQrTZ^l;vpsxzx!M`b6uvr>=SW<(RHA?kK=D;3(;fjP#HV{*1ZS8GV>8f!+?k{R5h=Xm zz3?SVOHJ|~rm<^+q)q5Coq}a{IRs^zg{1Wo0#*YQA^xXmtM6lZBCjp_y@pqC7G`cs}_fI?Bkv03I`8o@R4qH@Vs;I;v;z@F{% zf!Wc2iF*s;RS~M7>A%*ieUcM9n|IsX+Ox0TRm>U(2cmwaz5h)M#E2&uVy<~Zev(9- zH{A2>o26+=u*Cv8kqY&SuwCKaCuBOF zckWIDh~EO2Tje2B6m6q|$0eWiCH=$b72o~*eLstEmp;fA&j9PTLNwAHVUKT#ptCOM zJKu)rQUSvbrWaKWs3?oktfZXu-t8EK)y?JDI31;LOHa~mwqNG0m1$l$+r0C8ja+fX zJRa8~J?&MbU*=Ktb^?{5?y9zgxS>ofN+VQ+7YWM9tjT*J@~N4t;E8L5+D?!?;ZLNw z>NRGFN8apw85#BCl^?wqA4b)UuE6&CKl7s3k#0vzKC7I_m`Bn%okn(a`*&Ch%3@A6 zX-T~`POpf+%blJWb8TH{l-(ahU=NL`c*MDdx0`}M10{nP#r%Mt7$hu(pIfcHDS>k$ z{Sp5GW8tZrch&y9mksrPwljp&URt?*md0}nY_Mp^ag2#br$l% zN4n=y5!l+wvsgEm@%Fis&lRuLwM(oa&?*J_V+vr{v@(q<-E%ONZF05DvD}ZiaXLnH(ovagfYd^lFe_et*2eKLVun_ygglV0aR()dB|g zhsl9;7wc_#f;3Mi(SZnhn;0N$O4RDlK7A@^?Ny2gf59{(1kdq_JOB)gZjs!ScX>U{ zM-yA;GJ3KukX=nzZ73S|;^^v+e7kGId2w-DPgTJoRZw06@}s{Ct{wt-ntgu3!E<8| z(hUb=Z=iD^$XpX^W?2)se=v6cT#skTazyy!o+c)o?PUpFZIARu`U6hmG6F4{js1sF49 z^Du4q0>nAq83o(|=`CG%_CDjx&Cf3@vrUFA`6J(yzPqBAA$xrD(~ zZsjKY8dG-VH6wAc)B$=flsIbc=FhECy3^g}r&lO@p1YgLBq64H?we@3rDMV zP88vZ<}6o_6bGE%I_Bnw=JRu~k7jl%JN1HejAn*JGE*a}D)L(Cg(?a-5Sg!-$s-V# zV|5=NYIhj5TeVw`=SR+t_{7=YU(GB*2`^Tz2j!NvfHoD^SNLIp^}7b*)?f|QY{nY5 z^HDJ6r)k}1OXiskBJGgzqj|E|+YEhXLuz{kzi0t2^~R7;LFtlZ|M0k@M||8L?2Ofu z!hO6?`If`pgvk($>$)?GNpB9u^qq zZs^@mFJ_e9AjM;e$kwhG`yzD27eeQy*U^8y5E4L5xrnO&8Qr`$`S;bOs?{C%<|{V6 z60J46zvY`rhAwK}CC*exP7kiC_t^Q!Ye`I?x1HX>);_;UI>Va$19*?Ok`(7vb^`Sp zuX4=Kk`!oq>U#KwD0xuQJi=J|TI1RHL$2l*C13uq-BxaRYrTKKJ0-Bi^l+X9;UY;d zds{?Pez9S8Y}pl)oZ+-j)i;N&_L$ckpmvlvp7nOEmQ$D947<3s@uGew9_fPOAG1#N z%Ve8c>u#S>yt|#BVfrMouE3o*D z6L*5%*DM!l&ev%B!c9a3@dg|U+?X7is$`dw0=F%W5Du~2H8y%EfN+y;?yE4T!@sfA4+?QBwr%u4(4kcUSuI z=6~1Fj9HUZN|%|&tn`5LMG-E7K4%yr)`_P8HOu)DoVf`f$Niyb0v{%YQgoTXl zADjPsb@=;R2LMDq`0*Df^?(2DjneVIT-&&)zwNyfEU3@h&Jxr|OD*Sx#<}hx^>^Fi z*3u^9%S*L`s^~N+2Qa~aW`hq>>8pW3ejP@?pfMd~Z$nVaQTehLSHP}W466gr?H6Be zd=H7vLCcwYqPUATXX>?O9KX{1x!jt>1-tg4o_ddGo6xtMIoDIQB)Tfvve5%|PQg-J z^926QXv2%SY{UDvbtO=Dn5G4~p9rvZUW8Zkdw$L~2hWj$%V!xx6L$XR%eevGPd??U zEM}lfyaH7{Wu>JkU^iX8syJ(ptgrkYYyiGGD~Aff1~XygQ<#XG(~a6oADVhsy6)nm zTw(oMCeq)c=M@Emz0OP|p~dz%>Bh1n%hKP9nSoY9%vXqDTfETQZG4_zM~C|Ey`i+n z%_#*J;(!hu;AAE~qme+D5|@QNHP;W;DNKx>Z-PavP0FbjYAJTm-k$(V+QL4-zgK3% zz0+Q-`V*|7k9Rr#Ryh^Sz=bV)bL)tJ>}2n$qgDF>RBB2bPS@qP_!jG+%HB`}ZpiLi7*pwPSbP$e+QvE>lv!|SeiNEtHimcyewA;qe#Ui(eL zvD?zDiZ<#~mw6H27~`h$o^C#%K3kO(7&DTV;ZmoaIDnQswV@rCESy$kbuCRbb1AZ_xG_!%uvw)Zox z<&Ql}yoE`p<0WqmBW{G?sh-L+^j#ZDheot+exn$b#<&x}5ZT93;w*P`9LD=UO10=3@VF;)av`z5&gNgXbFFo z_qaK{sjQ_y0{W#OvauACx&0dK?Ey5@1bLUAz-+@8MrM4aX9ZG<$AOhH^HP9ZA*Ust z&-!QFY6e(khYnpC60LFP-^R@k=V4+O?afveju_`ldG^gF-0!^plVksT_sk}sN8NQK zWc$y+#(pu_NPBT|$NTC>(6P^uz?S_KZxvY;`LfvA$YXbz0Y(LnD3@B9M0GhFc}VK; z(%fnb%?ltUWB)Xk+i7ltWd+RDGM`5tkh133OI=ct$H3UJz93V!fF&B`VMEKy&vStuNQj^>!>=Ex~Srm7m_^!L8r z1K6*PdYaNqj@&KYJc4T(OeBvtV+NY{cZ4;6WIBe-(ute^D+{Fq(k6!|TYF@bhApZU zZDp@oPUU;D8Lz0SAlXD8d6u(L0Y*ch_8_NXw0c3%kEAH`278Td!_=zs`kN-srQD0r zbGgxeo*H21=@;n9vn5-ax_S{j?0H$E0paz~RZ=&`A&!Gps9r*oe;XwZWzw;w zz!_n_A;4Pjzi-F?0ZM>Tu~mk94j&M(Mk!GqfG(N*RvpQ{;3gE5%%~@s;@n*L8#di~S-jQ2vF+P27Z*{~Puw~qaC6T$xox6-$wyvv+) zV#H;#`z{sBse>$XxSQ1B*sk{)IS;a*M>NIFCik+e*i}mD|@@V-(tRBfK6veo4Af}?NAAQ+Ec|qV{u0_}p zt9oDH^0KMi<-%v$=qMYFOD_-&BQc+hBvY;W(d`-&EU(=m^11_e@D+e(IG7wIT&cMk z<{qW`N5x&eq{*7uAB$>sx@_3_!>(l3>< zz`abGoga*Awy+xTHW?K)cRCY%UMq&CY_HKcT+mAWQU=mJSwc=a7K20kqQ)n#4mS2Z zKey~QLTZd{90GI~i3$9Yj-oZF9{5R@WIdD5?Xcc~4^_!+D4W(n*7IP1Z8j%*x~a%NW=j7<9GC zX)XJP&6WR)ulEjTvth%BJCy3ss%mX*wX~=r zwO5DI_A#F#)E1;ht=KWz(po`1suF~vj}@b++Iz%^gxWPrB?yTif*9ZJeSYuvJHGFD z`X0yq7s)^F%QepPyw1zcoC&}75r;TC_$iAYdSjE!f;Zz>P?P`GsN9@GFnja`#r^== za=yJq-s&!}3U3FN3s3kczHe886X96E3$%@7w&wg{_-oVxcLTsY8>Oa8(fr(-Wsn+`iZ?B zQLs;@(e`;5=jsVeU9k_vbxrph{;LIO>q-pR3Hn+on7=oq-q)*vF^3akD}2v<`|1{? zn;0zS6)7sv|KZ-;!W`yFL}{g!?ZJ@p*Ty}`e)|59RkkBb(oy|yY(+~qQlgfL@#!%W z(k;_uyZ7_;!xV3+9}2Pq`M`)P42453*DkG#yO|04;!NZeRv4b=44#!mHOFms`I{`7 zmTVp=vE*-MfS`9}q}G@lXZmOs=m1DO9WVwS?#5jm=Q1I+###GJrF6XXp`;AIlI*=vP-{775`8q|Jzc1ruqMps6o?wJ_45aFFyBc zQ3%!(?Tnt&J<%!eCVHiU5v(V^R>9C+sJU=y$zrw!*k5ZHe%k4)`6d5>f_0b%w58#K znqK>6(&&h(9)jC0ebL`}@UNmLC6?@;%T-O=2`55n$qhbfNvNLi+G3_fHbO^Lnfql2 zd_|p*y{KF!;L%LFd)Zj0sCtf}E#r4lW!z^BKE#xv668M6NLxYRU`-U$=Q;C7c8z+_ zAkVF0Yj|ercb(ac`FMWT(I3m}lIcS=cIUn-jqXVal(e;49>k`l!H3i z8@ns`TV{mxLGudtt#y-}i-GPAHZv7VdxhiLvapP<5OcZ@I-9S;i1~PE9Rq9c{%o7; zR@UG`Gw1a;uFi)|q8<*z{dAU1NY?X9%=^Qq%j%~AJk4aD-iDM?kr;>LWhoG|j6yHV z5xGUM43utGTant^)j?jmjj@%}8mqP@XaQh~Awj&uY+`mw&_zMza(e?NO|10bAb1KF z`TfkT75bsa(#8fO#GMf2Rg>|LFU(OO->|-~vQKTAJ^+V$?7$DUsZ-U|ICQkbb{RSibXXvd`IQ$fUpwMrYoL~2F8E_`Xh78}~@XM+h+AzMGXV*3Mu$P0J#jFmsoxv9Y!NJ07- z*}=J?;}#p>U%|Uk_iAJlx)kkaUj`SaoU^x^NB+^2Vv>}q|^>kTT#5E$d~A?Rv^py!`8ZevIEKUb|DZ_ zk``$rky&wSx%_#N^F#8pcI>y0fJ3w1L2r5~NE^*m$}y71-#{oA2rR7Wk8qL+)L2hM z{b$)q)JAK!;@{n|$oOunv$nEfXVB1zo*<#;jkTiuQ!u-;KJa_(^|b_zc%u4`<6x$C z1wr|^!UI#@yITUHN;4xJQ>u$HzV4Oo<8qcb?}I)eGHiThFQGpE+0dl(!6Kz-&DPmF zak50MT-NdXUVkxdPfcZ@y) z(EX&SnXe=NtiYZ=l;SZ@$$LdQUL@tI?;g0)$VJY!J{UsC|*0U0yXC;`w>&x_q9z#g)mrY*DsdaVV+ zQmb-oQSlwYhyKSTs?%h>XWdIIVz(M)+gj{-Pjs}nt#S;uqQ*_q=5(njeu7ixileNX z@*7Unix~83eo8r4>yu1Z=U&xYyV~J~9o3=^rFD7L6g+`976@}YYhQJ9_?}67&U@X+ zlg_v;yfR;%M~>Nh$lwSiiht}c%*G_Sl6p5^bv1lg2{7DHo78YEH0ID*(}yf;o$dd1 zb^H&N#J7OIIUY_zaKa|Hi*Qd0EWDqcf;f@W`O;qO`Y|g=_r6`$wY%hELT-cQ769uq zFCqD6+G(kT(4>+vz9D~T3VBTZCzEp5vEvr#5mJ^gOJwNjh?x*|YU@RY8#Z-g%ouAs z>wqaoTX}rOaXMnr0wYtlFY}XowF(}cP9 z!DT1|^z4IawhF5Z?Z@b|8o7?N6rS*Ko{Pw)l29r^zj3o(15Vnp-rI!cTJH~?!$LfM`W$kw%Ck+9|33HppPvdx1Gv0! z^j*Sl&I>fLUHPOXNXXF8n4Fjhq7UjJ$Y5VP=1u3=tcj;Lg{GWg3My%vB8@k{4yb88;Gn2^TccQQU0SNA864nYjFXh z!@b0;fE1LE!1=QwFP#nB3hG2zj~OBttDO#tLb!vm_RV4f>o3F{QKkb3;Vci|=Mgfw zzJS|_bmfF$cDP`_-|#_}K}baoGJ<=q@Ll#;IKNY&T`GOO7z?F4lTC=2^rZPG6?!CI z$a|FJXoW92mdbis>KfG!Nyj-++NY>4QP+a_-ESb{gLQgOi*u9)7-cQvuZ#4UI}G!} z6vNod|A^WsR~larCZLy7llmIQTg&!-2_(iMAEsb8&9HLmF$F2{6)uoeT5R-uF6&t< zX#eBRj%6JV@v!hGxDR{I)0EBKrMj;w5lNg6{P0D1X>Ej+2{UBmV0M3GKDF1JZ0h3X zrPUuq(DBV;A1s*4$~UYWRy6UkqVN>rRm?)5bM5utlp{*KbV$}ilh@A;<=qGo9`J`k zMJAcKyYU-;0Hw`o0?JmMttI&F>xknA+8L!4VGFjykw^!~Cu!{O$DbWT8Ae!cX&f*^ z!$yybtoFq!FHiUY@L0-DZbrz?P9t@fditECkj~}bG`RoCC#wyILZJqB(CzzFZ--AaqJyn7B6qULle3D1saLsdl2fjX; zPUq(*%Ec8CtXJGbIUhwO89t_C77~IB+ZD0m@e}85uWb#>l+(mP3B_!V!B`jP3#ZXB zLxt1u!HpZtzx_#HFRVD9>rO{U8H-?bzL>LDi`6TX#dn;O8r?^$~!^Kj|u_9FpiQ^ zkq)hAps-?UhRbjQdQ6o0wq!EBv7j1%{VLv?AT>EAx7mV(afPKjg?7!J@6E&mKn%=W zwM1{}b9Xi(6e|DMinR=MgKt{6w%&y7TR~V*4AEz(L6Rac!B@!uHMv7|nzRp4SxNR3 zvHtFGHQalHk5{}L+SpgI>5?9&1ZkZS-MO7i1CHJ)DDNjEQZ@3_#|ERF^&dx4ID4D$s zA|RgWyUV@pM0j2V6F&u}ppW)Bi+v|O%8qqXcYNbi3+@e3EdzzvMYk$v82C3iqNWksYbshik!Tn^Jl3tew==l@le1&f=U@k0k zZtM%W$P8bnG|bu8UjZFKEmx;ZTZXjLy!S|h4YBZBi2|f)ly2Aa!=7QY1c2re)gu1C z{BOx&M)1GzGn3z^?I1Wa8o}o`+Gw4cNBhmf_dhQ`C>%iVw@!xP|N3_uL?N-)TQPor z9DOEw1kr-cUvQ_#j6ORW;9zfnx@1HY8e(GJgWuRT=5pOGCfY&t!?z5*QED)4d(ud8 zTesm%oydCy`EPF48K-^FsCZ*F>$kBTV$sVl)8cA1B>4ZBn~Y6lpZ8;V9~dgeLBPpp zJ@%D*M|^eqtDosu#Td^{3~``_Apo2c^TW8??H^t_B8kEXUszW5%doRAkij?UZz zpLLUr2e-_;Q1)y~6D9X2(jE5hoGPlFD6-2Dy)%%$IPxg7CMH5m>9Y@2>BSqSDFJn~ zo$>>8e=p9U9+d4Rl*rRI<=uwRTKC1yT!~4oS3p$~@$~sE$7YQF=OZUK0#eZv<#JpF){RsA);R8slatn);czyMz1OqGRQnJ{iIfd)#|2 zhv~bF23%;kDxD~fYL6K8Pb^mWRvr!PmMV1w;Wt~_KJ(|lnRrIPnTJ|u|H{-yFVx3ONf^drmNl+lmLy%G)yWj z{~^lbW8c*b2ME6ZsJ_(mH1?tUd!ckT;CNfc0mbF?-t_c#W$pnTtr56;M5zCiaE)O^ zM9aS4yA0r5r5K2UY#6KqR>U0x`A7bz7+L5pkVAyLw~(s8fpQzJeQ_AHt<1kEo^oP2 za(`A)%6XFW%eHLRk@5?l7E6@0ODE&Zv_+R;KWDmMX>yg+7IXcgMlwXN)v^Oak!rQV zC3G=ic7ApVP!aLE*f$ouBrIw+N;H>GPNU*Aj~1W)WXbX}V;s}{us&4$BncjpGI*dJ6K3_JT(|o5aCdb)FmJPvd zZ`Gr1Oqo9RD*+9VFJ4gse-0-M-Q716@|D97jA3j=ja6)g>MF7$FZyH!MNWMfYI9Q7 z>If!Tp&vSW^u;F8aZgG%1!$vHp09a@UhHFklpv0OEuOfp-RornH=`p)TQU3UE4&TF z;we$4TOl_gOmI_%bxH-;wk(SenD$%0n3~$&XN9uuFG$|uao(LDDLK=#o8sDFM(AB~ zEE`6kHeZ^T#3i=N@|~|z8q=f3ZClYB+^8ixxp;@9__6I8&`{Kz@f=ne)uv&^#aP4{ zw6FVlO?5f@IpDt2;`ddBDo(TNU)=KPq) zV#K4q`l9M!(CepdH9N-ZPY~r5S~8f9tHok5mI5&P>FI${Xh6e-P+#NJd0bqdY==&U z0=LT&kC~7${m-FCA6-zR&rhVed3}!xP&*K(eT&9pcIW{vAytaOYnV;LGz{z6M63LO z7qhb>!6L=Y2XS^x%by;Ons>Xj9Tw{c_c3Ji`M3Dm?s3<128p@opvr0oqRD`|87*V@ zX(8Gz)O=-R#|$d_X{4hrrh1fSl*rKol6zTJFNp!k>8{Zrf`L;;?@mf^eR3lV%Bf(` z>4xoUn}5eyeuE0+Pl`vplvI#2?Y+U(56*E{5#^%=W$!&1aS|va6x6p*jlg<;O$`YZ z?zLivC+Jp0X4vMysDzq3m@q&vfi^Dkq);0owA6Phi4UM}lIOL2atd~c&h9Y;YR4A> z9o}UKOI(I`C8&0ff|ZQgL8(219-`!{n`vFVn(N9v<4;@*tWxV6JO74G?@<;hxf;Q` zoh%uLQo0{#A$#!9kev$Zp4q9{e?S&>=9$OmBAXzfC4-I4Mzv8g zu%u8JSrrg*wgp9z{4ntwJmo*A3;6I(g~Z8Y&;GSn0aD$Kld8aeg}0v{!#6L%Q2V0W zN_1RypZuxs@zD)K3(Y1MPt-JYQ!bZEfKG6~D>@^xtTdG3JCagr(i|c-I8=QzBii_* zk+X>nVbh;nGKRITv5GnZ`k5J*-(sBsoc(4RuP43M5nVQq+WYXf2y2yeQwii9nbj;_ zlJhn#ito0NC5QckKe)5?mMUc*8??q2yGh8bXcjIavbHsoOFB9pRZ0z8(Ykd^vKFTH z(>+G2+eh_@c8n68$g;EnXkgKc+m5Wf&$2m%RY1~!ID&Iu@628CR#y18`OMg#@9ryS zNcz>LbRojM^-;sF+^x4TlGMHWNf|p~;3_Q>NrlTyp^?g|<-|>Kp&#oKi5$Wj!{PXz z{g>;N$@5Wb<|v=G`iAE&xTvZT>gE~YJ1MGF>`#(gjR|=CBTK3Lu`qhT{PKGKaOLdZ z(35J3Mrw{ezCHM-&Q;RA4A;gqfOCxBC>QrN(~pPWyDSkLx2dC{;2ZF4-g77$A~?Qh z<>LIS%r1`{r{(2q?MLa^DO59e86k(HJp3UM*VfpbEdu{Ly+1BZWaxTltcmD)q)Q}e zhO&E^px2+)0lQ79q)o`nfN5~=#x;xG*l9&IW~)XJ)ADk59Qyc~kq1YPB_A4A{T(GM z1PC1^*o4XrwGo4VBUD@`4nwOkRbdd&rr!1j164Q#8EwgRuIncUD3=F zU;+kLT|CMqxGN9MOhm=dQBNEtv*_8hJg>ltk5Amw`J}c2>p4^}NeP%-^&cMpN^%Tt z>%fG!af!TlE?HV?7|#uw(q>%a*TC#KNoiB28X<8|a$a%QRJ`bMuCUiqeqTwzO(N$z z(XGK6z^aR%oR;XCuT;m&3aC9`QLDoU5|$X(0B#CX7FE^uzd07k9l$HBvGt}Tz-_)i z(42-J&W9BuY-@a=8GZo+bTYp0#f5`zgeEoCVp3rFvGAW^tShy(q?soNg=d0u*5gPH z(_Fcv2IL>zgVp-O=-|e&#fl&9pl1RZV1?vSeIGTzx*cOx{~YW`jDn_AL8&+yis5GE zaQC3ajk&#f_>SdBc)Zj)&igX8&Hg4rRE9ow)Btg8wa$fhl{p zHwF0b<30u$&KNq}NGU?x7KJ#j(-{_#-$VooiQl6(>~Q6tqFK}32G^sPmjgWY_`uAG z@b75Ohj)hzh%#343Q#`S&FEc=I+gj4v_G%f-j(ih)*)f8jpV=RBh^Tl;)oZ>dM9v6 zV_No~mW{%zFPsCvaW4U9jMyWh^7S2GaAs6I!;blX%Zcjh?Phra<-mm4mu^l(S6iZ% z+Mt-OsAw7}k)vLLJf8wbKY3jDjcZq%!`QtO>HY33>Sq`+C-pj-s}c=I>lVg~Fw;7^ z*5vurebx(b7HKet&wZ#ry^neH#=FIJcb(X=q?(~)3ijod%ol2crb_{GJ99|YL}?mL zE`z;GIccPml$v7eU{O0&uuj;V8ilLo-X)0D%M-IUUoE?qwN605sH=dNBP2~K6ii|} zaJ9X*S}|Ter6JSP_Zq*V7O0i7Nhov-)|(8F1HD^SMW!?DrY9Hg`xXtSQK3IzVgc{z zy;UPxt!c<3nDmtqJ$w>|>NvTVXqAXQ802yP?(VZPX6D4l6lX{;E%@>shKRZ}P@mML zSGD)6tPiF#GD21WRD{z3wme^3t-8LMn}_t0OS&v#;K%1UyS$KT@2 zjpP5L9d0^txRY@-^32hHXoCNLp#oI8{};C#9C*%Nt9&0SYSl9Z$b!ZYIoIs!{lvGv z!?Qe=FW61ydl-`vcwM~b3eP-J{`o2T?n@yctTLd8uBR{1y)SPJeue62{)HKJ5IuqR zzstqpyB>M%kIO+qW?_?DZ+FBM@qnoIj17GPxT^&b|Bw;)b9T^@SbJ<(HtYT01#Ffp0r+FnfS1VN4ql+>8mB!N;gn|iUVb@NhpX68HF@i=Zj6%&`!Y#`8 z6L|Y|*6wAvyF5FNTI6$8Pl&};9jw3)8h&p*5X(IW3B~f*6EK1CjDba=FUogxbrO~XP*1SpZbSQOsPbVn)+IT4 z7`^r*zgqGjM~2|+)*i@xQ03khlsXt%K`p^uwhIEm3cGmC38C_gI}J#bdHYm8q9$ei z`Ov7oR|)yXen902)X^YCrQw^gy6-gXYw}*`ay_B=?-8HLTH}jXD-qtsN3Keb*0H9s z*>J1)ig_j;n_E4K*wBA#*Lv+d*W$;#f{&TH%1|oReENCgHnd0tj<7`e=;$a*-;wM7sg?vQRE8M` zjH3Jqy$;EaBctxhFXhd3bO;`TLQb=uoC%KWaXfcI;tzIZj~;&Wx6kVyQJd_z z9?@l*ZWD{nzdke3ClRsAqAe{;DSF-o)86>ma-EAK?A+z;fq>?j z`tJ0uVc>q@Sm%xR>{2MGYVxD`nY^5nxI48OZ`UN66*?9>?z!nAA!lpJeQBL-o1Jq> zO+G`%r4OPKUJn;_)L-bp3N)XF*}4stm+#kt2>zFxB`IM_kLmychS7F-U5qkLbIuso zbc!Akpvy96Q7+?t=~cC*k99%_1>1YO5(8f0I%^x zin5Fv3vfvpNU2?bCsx@C2JROU@gv6Yhumy&jcfCPnuP8hMghkvFN3fs^qziugCL%K z7hK!$P5HpVC!VZQo`svqQMN9s0{dWm{z8k1?d6XvfVcb6iCbrsifScEIgDbwx#h&h z)Z4M??9i=<*49R5?Pb5wJc#kVeH@FsLk)6KCUl@+JsIp@)i5P zKTTa+kq5cMe$&B>c~$CkW2mBv3X@W&a$jaTb9xvZwQJU76=e+3xkS#t7jokmB$F7> zFkgLPp?xA})|gu$U3)c#^yU_fv|ZS?Jq8_8&Yd;sMdGd;9r1=-^WD+itRv>Hf@3d; zTN(dtLPqj%v>_$t!lM>DHS^-Dp%1#ys`Jc*?Aiz0u4izs9}x%eiZUvB9pBYiT2VZS zaBZ<@%c$a9pKllbAoHL){q5QkhgR?E%Yl;?zfNtb7uWlTVmBW0zBor^nR71YA^=)r z+G+t&mea#=H!~Po0XDMRLr;v*?etf?;6s&j<#Dh}?klC8MT;S*`OwSdg={Z027>_t z5gEG6@6*T2VHDhG=TkV^SYReswdkC@er_Zkd;LE35jC~LdW4(&8g$*|Db2 zaIcyA(BW66#-(QSBK{q63vi6GUF*%>(k($k_nJ=ywsdSjoUKf#IfvI~WIiGNq2jb> z+`W~BF>PJtGT8W6wuSl>jWN{YvDq`-o^vi+lHDEF&e!A?Wg=xZZ_4+iBA@C8uG*({ zPpn=8$hY-U4!riBN-p{8kDfLQGWcNaPK2Mv?&^(FC?Q=}KdS+Mm=dHu09i-sBB@BQ z5VW|pvzH+d!q|sbekAxY~9nYV4auW0w6n zp3wrn4L|KmFS;6oj$XRDZ^?V>N{2wAjzAG?!7iiMY~8lK#=M-^Ec3up&U}p1^;-PZ zjgBC(H=~RgDx+@EK8gku@%<2!%GqhuapQ;WsUhx7!Xzrjf)cW7c@UG-i<@` zeDgwKDdhK|t zB6YCvRqU~AqGNR%YHFcI4B8kQbE3evY|lY9qf5P1`M7;R(>B{f{sy<_<(}~=pHGx5 zrWcjAQtKj$m%Nf1YZR8$vlu2WlpSZsi)FHx`WybH<>RjT-)RR`6aBodjEDL+rc#db zhRU0t%%RQh^aa>a@%SL)>EU`ckVApjLl-^Whh_Vp9ThR873AHYs!H?dp|t=V-yAuE zl^bTU&9CgeHZhw~i`Df@phn>l0Aqgmszm0s+-qsS*n^eG_OAS9%zY^!2eq<$SsgSrg zXZ_xe_%D3!+}L$KwHNbwO>Xp}P18FuO55XopzK8F+Y-&|ZLhLo(5E1fNi=5+-pN@s z69S&R#nke)B5S%@@+P!_#264a$zkF)1(E3}dY)ldH)QHsoWQOL+LSbTSNcG77pG(5 zUB`pzo)iyxPJQNP0m0e-Ae#jHroac}beZbdH*nUqYCbVgph0VN0p)8*&7RcwacM18 zkd+Lh+dv#5zcT-9%5!pKyZZzyNRSP8qsJPLEo|@?#KjR#RXO?zCG{U6h!oqdb0B}s zm>H`V;bjjS?xTGQeW4$oa5D<0^#Y7Z*`2mL=XKgQC+ZAhTwKuXhe?7%*-VlvRUaw(K+cS-nwwn~W* zZukt`SSc^a-U;TCeht`8x@!e??J*hFb==41w+JOEK#EBdBwh6&btSfv-5bjVX330D zudPWY6-|<9U~pgfVu&4T3Ef!ek6}B!{qvT0$f4eK#?#DRt@wY%k45NV{OrqYocdk- z=*+Yqh&TtSh+D-AYwU+rWpQcw>>9aFMim14teS)3cQGv8DAV{yuf>N?bGnMAnbS~Z znBoDtZ%e)2XJt+f#|LUBeMID+H4DvlW?4oy%#}yR$WoBdW8F`UQ1y)>$4WG-QZmMN zOlSB;FfXK2Q&aBic&TBMx-Y)%RaWj7jlo_0I#I1bG)5t9>fAU?`H5vN>W0CXooLjl z^||NVw6#6d*uzVert9AIX4cVgTZ1y$X~Tz?A|pHr;ZJT1_O4Zt_heCfiM-&=(SD%!{MED zSD2s`#_AoY#ksL)3I5l5pwtjar`l-+qL-*-zX!tXy8Nr~ku8j$A|Irz!M)b_0n{D*4KV z#`O$JB1RgQe11qYLwQv8;Eb_5p#h3{SAi2#b1phFm$**KthYv-(`J!;Tr4w|2l?=z z;pWxzqJkxATMHhfa$M#$uE5e_*kzIy(s#6JZV1~7`fXy_HLKm|09UTf;U`{{y*u_3N4MW)36Z4=J}b6jl- zY|{e4d7#HqJH~@N&I0Eul|OxBZcGx1!D%&V2i%16ywqvWz-&QtPO45vS4W?p6ACVo zz23z_=mzw}fR!Qfh>bG$*qZVv&m&0H(qr*R%?1<4Yv=9OkT~>-1tj)5o0NFG$*}=R zH%ZA@f|yTkb-aK6dq45N;+eTEMzIc1=LA~kQs#vpE^$t~5;+)}$#!1A7YI*FMaod} z@Gbt^9zrknXlb(`9;GkU&3e5~`yP{6Wa572nTiN1KDQ@Z+%Yi_&OiV^kWs_37)D>68YOAv=ik0dvw3etp zvs!qBiz+cgp95vHsbh3Tf?Uxr8XSmx14^r=r~pl9RRY>}d-qFXgn)*;aq6bD9J@J6 zSQ<_liG(m1ucw3|dCjH@)R<_E1IssiES@U>?_MwSj?cZFIT=s4}dWme9}jgqRvCG3@L8R2dyngsr?0PXOk^-T zA4nAynHnQ4t)J)UmD5hx|F_#&@oAXxK5VM*LSRWm!TYZmI=e#DZHM3Y$cMeo`bpp$ z#P&gn-wk0^Z&{vZb6h%FcBF`_>BK|scWB0VgYk0DgWHk5t)s_Ex!X`s_{z+#5K z)s3APf>=v=Rq=%&J*?LBAA9BFR)E@;G^f7L5fQ&VJDCHo{LT|zeT@?ihx%l51nx+CVI1B!H% zzeeOe(Mnbs3%Hljb1ayL#h7f#8}bu`P_VrOGYmT=FOE#v@F0x#W;#=~^q?|5&R0P^ ztz$WuF`f4rLr$VQ)|!1R*OcnvqjDyfRXFPLk+@?*2n2FUcqg|59)ALLG_la2ay?Ph`kfK z9kHb!NbtVH@DcSm%`V_F!I?*yZajEn8X4~%b7V|Qu=2}ee6@Y5cN*7>TLg(! zzglHQTXX&GOmKw2Zf*ryO?N$2ty z2{LD2j4x6sK3nLXL>husxs6`8slYFv$z=K`9~VeWGQ@tR_}n2=Fq;(rPzS9)CnH8- zMMZ%pNia?|#qY|Vr>7L!tJy)_?HtWG^pN0+O2bOmK~BdbhYd#S^vzlym9dfFSoR6KW0KE9 zwpM>$ipnF@`cWIINI5BoQ@q3tx4m{?5vKoV1)GM8QqbPMK1U<`png-?+G7VOVV*lO zOOF0SM8Q$e+;nO~gzxfR^=}H)e}cwB`>+X$Gh9#mZK&Nl8HCN7Ydm1IrXCQdgJ8y@ zfy?3WEmf9m@#xlaaEn3uw5BG`t_SkgB*_B~R+X32%`02w`Dxq!LUy>#;y&b??9smJ z!r9yoK3ZCpy45pETFkC4@w}WJuFK!~eOc-s;{NuWOHWmC4=j3P+&F5tJ7)E{yI8Z6 zl#ZEp@T47ge%voZN^7bxC4HVsN|0SZ7Wp2L>%BKo6X0O?@+5H^Bq!XtqxVrlnU$ex zkwO14RA<~KEXP%nbieodw2@*%_ns?9p7E4+8FI0HAaJf!_SyA~Px&*}kVY5fHlMfE z4tr0}-X1*ic|17@tSVv`7f8Wf0Q@vv@l2Ium)3R8X9ODH?mwNd*Vf+iNwsir__t`-?> z+iNZdEumK9OUrjiyZMT1+jc|XYj^DGOdlqR1U3?zK0{5kuFSmMHRXM_fcgO68N7IU zyPK%BAZRJ&>l0Ic(cc7CfJpKVmKZ3>e{I*H!zd_zM>m_1%TF@K`BJFE{b}@P*>OCe z;Y%gOR$!8LsN*rwFV{VU=Hhs^_V<`~IvQ_v2+A*cH0!X}?a%pf_^HFcTe&61UFxr> z0lJwHaSQXrU~X2SQH_@~rQG7yuK%lsfu90j0Y{IY1N{y|eBC3m+QIK$C$LK+gogO9 z>+fGzSZp$u)N_=%Hg&Gqs!si&?vkqSvGSxQOA=6Oa?V7{a$!5nhdPew`@*UvL&4VAhcAt3#I6yBwvZeU6Qz1^SfG&}r?G;i=LT{UN_{&_IFM zQb=Zp-p~|!sO2*UCTK3W@<$j*M3_>Jc8aHns05!T4>jN+S$@>aJr#?fO~d&1(H5q4Cm(w;!TrTgw^Lj}I_VD2vEZJum&2fC7c zmMqaZ`o`kG!;{H=R<@%@n`vWXWB4L5&!Sjmy=!O0)S^`M(*>+Tndp9jm)~Vtyo;pD zfCJX2M&9ZSvkF~)(;xQlsNX*Ys;S}zFyg!LE9+1E22gHdY0#Zf(GmoRH^#JNq2oyz z`??7CL@Si{>fG3upZW(Isrn|fUK5zC_Vc$N7B-0NJV&w4n4w>(f!-lcRRq-gPia2J zWLY_OyPRsa*~L#^Dq+14Y3^{{FgCV&ZoT{K!_&of#pEgOlGzY$g=-~cEd;yVYlHc( zj}01@2K}pcEM1s-0#q#0RyGbRuMkM|t0BFR zTHi&|o=Q%wK+vNZ3`}?0QTUcO74LPnFt_^IJi$2>v-Wg^+XY|c4fhSgcRWqeuSxcn z!`A=VD^eyBcI{>obR-87SHH-J=b3Bv765(TgY zj5Q@N{7;AO%n=88*uG^Jqe=3(%&)2U^K*9z&6Tp>?oF-?x|_7tGBQT;=2Qc|%K9%m zQx0J}2sE2fA?iVAaZB`!h*d6DZ!8|*`Y*mLpR)D{2gk8c{h|n5j+G5v!Tqa z9+5-D2LeZHg%}eBn--xjQyr|dyJd2_V6?E@}H(ZDu^ zB5GvgXkpk{1#z04L!0z@=it+lr-}Yko`#Wk&5k^eO2};~jgE+3-Kf#&<6A%NX(R9G zFq6Q(o}1<0t97Z|QS`L9#V@7=F%mB76~~xecNP+tkA;4Tl%w>7Ph)}>S?XmRliejZ zv2@g@OXLF0@-fB8Nj>yWB@_>L*01~pb*}f=tpR3|^=?CI{8(j|{%j8L$8#_+ynQ-a z2HHF6uR%T?cn4b#68ouCCM_hOvT0oBVf`^<+xysKC7g&Z0^ktV+fyF-Dz)yAkJ+x}u{5uU8DE z!Q)G%bI75ud@)XYUg$@K5|FD_!m0XKF3(Nl+H_)BOlWbY)abnmF~LNYDz@NcNCPNi zY1aUiRcLB2`gBNLd6HjNB+?bozhjnyJ^{9L-(TH#^m@NiZLI38R~vl5lbG%*HN>ba8r z$g>sp09o3M94a0dLvTVs@kL+aCeNlA~n2qg$3 zan2QSF$}Z_cT4u!Q7et~GgAhCJh0=AJ=K&!DH| zUs358PU}ervoq&@==qzo$Cp=uU3$aYu_udF65r`HrSEB7xg+qskP+R>TwGKC8|*D{f~kBs$@$iIB<* zaIg$vhuN(v#ziiFkp`KfH*VF**4yFD2*qBO0d0!Xn+CqSi0&^}G2@6a`)LZIxkd&! zxaXt1C~VefiQ@B6*WC$paBgSy;PO?~pSo~k@8@74; z$)u&Z9Uzy{i9sW(-S`wtdG}J9N>{&gQBu@Yw6)_G?m6lv{7bK?&KdbKIoqin1q+7{ z322lUX~C5;K^FBXs`qywW-9412Ty0mH6Ac8gUSrGM-JQIG||_2bN*@oM0ozeJQ~d7 zJ5dj3TnD5;gQZ;6u?P}_GRd(jR`ZnKC+&reUeI&-RYls7 zfWZGne|@=Plg*JE@!O7j;`}jfejlTjX0-cBwpPnhlL@ez7?;+1r5eVxWXEJV>85JB z2Iuy=kEx63d-EbO_0z+&y7ML-5?pL{f=MT(xuo^p(#?w3{darkCDvIiYdZ|Eh^qV- zwX)aE+SRyLO#}jp%&f{reXn`-u0>`f#Aa6i?RXny_{T`RGK?iscy5e~=F}Eu;s#DfSQ$+vT@(!4HG(EP>(oxoe-A)?MiB2$H3_iLM!kMX;OW zlGdYu-O2-3M7ZaBnAy1*_3j+@lw4!?h$tPbnmOPXp14EUeC2;GK;viob^<%GRsg{q zvtg+SL_Ag*#}puGbxK5mTvL5V9)tN1hYwj6Wutkwv?~};_&r2KwJ)5FaDvuer(`n3 z2Xg!2UYkC&GPVVHoIVfD%SCFCS_?w$b^PNbvIUpf^H4)+@nh~Fld*KbQF}-+$d?>w zP;vQ6=zF$0caz}91kF598wfMMwC&6h9{dF^AFAHPkQ^y9Eev%3b^w`hB3SVz>WpO8 zTN9Lb{hqX|ksk%Vtc>ukmr>C-A{1>-80$~)3d2P*bd?`!&SwT3z#+-n!l6LYd6K@( z$5o}pa^DWBJICA)i{}8a#WDZ%T2Vn;yeFwVH-cKf#dxxv zMslB20R}d>{34I`?3&b9L+pHM{Y#PZ0PVuPpREz#=p&+Lw7xB+{q>&Ti*AD>^3iZerjbrj&QaGb!{C@;)$E?J zB$@U$(@)(2`1L3c)naJoEgki$44d9v+|x5@?ETGAFgdQr`tljLajlr~Wh_Ry}(hsa+;U}%ywL{X|%NFgcl}?_NDckaLb6~xl7CN?WG-!dU zw82kWq39J-M9X3kM^~Z&9oZeE$v{KeH;2@7D~wU6+hB6Jzywxx0ycO`BbWTzOyhpg z`$SXyB$c$<6b&-c|HK0`XW?!&s2ozT4N(EC?t*1bxdcD-;kUH?np5?i&j<%ih z9n;?X>-PW?c$7~751GV=!fBUa$1s?22%zFPSVkyKx?|QUZWG><13ySSFN9BuN-K`% zPZwtB;GS1)n#dWToV=M25oTzW8bZN-+xjn({{C=*Gh=#xycI}Z(=l9!1o?c+ZZYm~ zif`Ix|8ixUN5*U8oxkNbPpk~E?6tHsRa7N=G0g zyK|qfOf_`@FcB*&z3pQgb6&;~OtOpv=-5B{7^%O5ji5~^iT`Q=Ua?QyJ%k9HIO&1E z<4N3Zzs^P|8%gtTn0^s%)Yt5k$P|k=jOsKoj41^_Amke)p9niYGFOc~wcLbziaL*5LkRLOf+AptL!I$r=~zx5hndGz`&R z?V0fI#cY+dp;WR4^llQ?V?5_hyuRzD+SB|rGKSW7*%Vn{pb}RdcNFyA6(=5lmss=d z$%^f=&Mv=_(Y%9ShAauVr$e>Syg|IN-zrcxYkm#BR=wH1!>W$ zh#m$Ty*`bSNOj8$)g@SFjrt|uUdYEH3UIjVrEwBgDBaVYzX zerkowA`}zCpWy?gJef4^P*j07IBV?8X{f92M1%r@jSB6pV6LiW79q934Lcv~$pqU8 z&Ns7L)GRCD&ENcM#4k;d>ujNyb55zsCb)6y*Xr(6J^A!HD}DmtK&10V?4Oi;Zwl*G zIX3c$T0e4~B|Yc26%g5U`)~*}=kvca{+lc}L*yKlU$S}46AnXlsv79(o-~;rRNot3 z5Wg#K1bLVc0(O{bYlEMUm!_QB^LJCeEYI=kh%(oTaI0#1 z4($Hhmk=A0kd1MjGbc!G9%f-5@agQvN zMOZHd6qt2C1?(Aq_^RE&h{!EB%MJXnp0{;8);i}E+_V+Go{ZH}Z@HMY6vbPMUyX!? z*qwYs25~Y97%wsk5f-z-y*-k?=$|d0-IwGBs!9;)zUu}kT3XKf80R1c3136Fy3eMe z5t#vD_2#i=1r26&Il1o00H6BG?g7hzSIc9i_^$Miy+`Frf^8}YjulxIhEaR9WD|N_ zIl&%!eA%5;bXkt~`H-4A1QzWdubY%pZ9#Vj^6Y&IHBuqMi8pvb z=+B|HN9mwkT)FZLLoqJVM*!xgLO`XGo}*l@XiaXrSDb@Z|II2ohDy{Os8LJSpnAyx z)ZtXO(ccAgOA{*jaEnCbgxb#593UtHdMa@E*8cLcD^%0k1%M46p8Rrh8>nFBm^YDI zb+fkoKCZTno|hvMIu7~TisJkyScY(rxBp5rK?|Z0N}l?O(bwR$B*zqg1X}dskChq2 zT(1_5EdmfUQ$V9d-G`?KY+3!UQLswqFfZxU2>mXOnple8H+?i@JFNhyR&nO- zMPrrmm`L4`x(mV7HQS%b(Gj_j8!aW5EA>u9CYiz=o;N)(+>?h}J(G{nu`jTzb=6ID z7mv89Xa`ijz%Nm9Sh6XR=lGnx=#|OJRH$7$4`ZFsh#J>?m1hUVLHU60N1{|=wns~D zS2fO3Yzlp9-te^RsUu+@kZh@?-Yu)ksdLS@+rDAIq6S(*PJ%jQaC=6rPfvBO#+)CK zHT$nE1uN;xuQ=IAQ=dn45x2`ousw%>?@=_2sZn(Xuf?Z0zq8lBXbV7wzsf#Y2-K>G zcRC}cWVC{^JRsIl?7pT@y^DUhY;Py}I173m*ugU0BrPRK!i+4Sn-32C-Y82ko}O3l zL$NNPPQnaZZ_2Uf>0B(UH-wi_sNCg+CZjK$%Cy&lzT$J4*UQ1#v1pK{BVJmL(M>#0 zX4g_>`!y?`AxCun>)Y&K^hu``8+Y3Qv9??w0}1*stE;TReL_|?R|+Yp%Zw$L}Un$OuQ+W`d`bOnep!eD>Xoy&E{XA{L?8Da&sSnO)cs)G0N8On| zV4gqrkl)AqEs3Bz<9p%NkpOTIZuY0(x-h@b1YR}9n9Aey>U-N~h~GpPvQpt`NyxnX zl0RK$^-SO~*Px2p(9LWH7+{&8za)NFtbm^vpF7I$W07GRkn`JK{>ycMyhWyX&p1iY z0&V~|#|jkP#ChgDy{BknasIpzhX)5yrXULVl560rZF@&i{~SJkdl$Q_{HL!&f==Sf z-3QW))uKM`8nyS@W=dT?P*r@bcVwg&$? zjs+hSAMFGd5dXRI_1y>aUj~&XZNN=3;zdhq6WUgsfeOD0*AL%WKv;>60?0LZUGfkV z2kNvp9!a&yp=nzNSd(P(-j@muVby>cyIK%2Ujfs4#TgMU8t(1ck3vn5mUVe7KRG|Jg6 z`J=k>kRoRHAxfm!7lgG}sj&L}QN*Toh30=gA%!9 zaM_KfYK~CQzUxzRtiU-*#D5`lm;zj1iuoEL^(!mHR{YG6PHB(a7pz2u#vY7OIJceb zPFUe;`;cJ&g|!@J3xwE4wj!*6m{i-!6amJj!QOnMxWw+zNPX27z;6Qyz!9ul!2VXo zo4{VWzP{g)5)yV#m+;0KIZF#{EX`liV+DyYpU0+0fNXJ zDmqbcYr8lswqzu7asQ$FvDz?2;{`+}*s+BIJI=#^G>Ow7#pz&f+`D#vxf|H}3kx}b z8V8=*2YKA=qh~2r9OD`)3TG3T`E)^r&#jfzHbOFAVvz-y)X#So`_xOS-1*Ap{1R^$ zo|wIEc=M@SCSPM)#X0TP%%wVUzo;0nHSg2{( zcHVhp{o7TRvzR#i%_R9!mCFju@q-w<~H+fn*J*Y)m?i z-V&0so4P6?AsX^&)oxWGSyZp<+Y`u>{^_vIaKSpWuVD}_hx?^;=I0+(Lnj zD69+QF1TQB6Ll_uNvKAY8HgKo{r%aXHM|L}lO-bKuZ5_vpVdxr5ArP;kaxySi&Vj@ z<OZn2ChW{uoa6nP7MuyE0A{2f#6i^C~YgN_)j#Y7?~_I(9S}n)H_0`_dmgwJb}< z&~by`mLI2W)#N&66ZmPg!w60(LH{XA`2eFtjK z(8E$Bv^1`HNA8vEZF;ZS@6F0#cToGNCOqNVpI85fMUa}D#wsrWmGNe@t7<{nP$RHy z&QQnm_;gm;NuXaPs)Y(fo^@E^MnoM9hn{la_SJC02iKG8i)(J(eD2PC!eq$!PSLHb zjk%Nc;MbWan>5BgoSZvr6U{ETGW+5M|6$2sS1D5gsu8E8*@UCG(pHm3vOud}b~$K# zh1b@YtrA8d)~9-vF{)jTpM!oNE7;39M#z$I#8&z1Ub1VgR`PJHm35bSbTw=@a7P|{ zrki5SbIrJD`f%1K7r(JBi$4jPnUrS}_k<@Og{!m12~-_>Q8w57^Kmz}JL#ut%<$?t zBi(&cSVttX6n-ODE*DHlCVNYvdsmGbc$h7k6+y0H$3*>>!p*m?nfqCtN-H$?^Sbbr zvW6*YPgpxNocfBv-&18WamKHcjKmBuY$ii<7PbZm)gw|TH1dvW0xLHBh#^)~Egx=|y`T5xP_*KLx=b`ab1S5eQoZmUJ9zLMD zfnxgAWej!HUB(x?eVPGTa=GICsW~`Q@AGPA1a#H9>#U} zQJ%?DB`@}26dOU(_Xryxd^>V|>97Cc^)r4B{2Ey1pkEwVmDwrRwsrIywiap?PlN#K zyvh@%*MFIcWjm8HMz4!mifQF781_Ps|K{?)R~3+jpAGCi?h!F6S2;P)gM_Vq|uG?rVyAH z>YnW}dXjh9oP>=c?^atYC^xvwN1vUFO@A3vZ9I;aOQF^2I;%iH-(KWpxC^Sr-MDW6 zYWx;;d&F6qQ!@&<*I-Gh5GOpv1QHk~zRzzmbW5N0LX5Pp=czJ7N}&^O z$R)dxq0U_JFF6+0raL02Rk1UV{Kw6Tkg1v)p$sO)`IzaO_J>!c`0(_#=K_kJ$?Xq_ zd-GZ~I>nqP0)}gJoV=U_ZGnt5$fPCNr>w_IIAD9#UXrrAtk{UUj?f2B=QNP^2QTi} z8-D<T zrz+OGX9MI>H;&G_9w(rabnaV!)k7ln>di2hE4wsSt<$f#q z*H?Nnx~i=rFNn=Km105!YD^}$C3s~g8j@8<+URG)?WJuD(;sp?akf!WYz{Aw8par# z9SJP96pWY+H{DjB=BKHCkG++}bx9^du~TQTz$xGN zzExB1`j_(F%`UgRm(XGJmSaX{wr6@xWDR>(XLpIucgPCQN*Giqd$1T8>4qOu>!OH=LRj3dQ(DBI4j`k@1$^M^^eTek2@3epb!1 zz10_2=y=Yq{B$0el@!>?UEj>WKeZoZJkfH zP6ko{n#dd*u0Gx2ms|v5qx%{zYQcdFCdJ_TOi2+&QASp;nJ~fMxtMIF+!6z$l;6`D zx11M_OG2OfKsx6!l9bicxX^5ATwZo0v@68$ZLro<@6T17h0*4 zhAfhjTz#wriqxj6n^Qeul9V+&yyQX(6!9eSK578xCi`n|eeUB1XnKpb`=U}($ND~xia(mc41 zXFhx5jX7+uYp@ZB0AFXSN*i$2f`^U8B8mZ)GwA~FUahd5(nQ?I#?FeuZh~Ti+V)yy zqfso-ctXC}?bN3`{=?i%8BrhHm}`Fo0zS|Raue~ZHwz2^NlL*Pe(Ka!j_kif41^Ao zB9Z??cgko2?MtROAtTNQqIp^UcUNcb@yVaEcjn6R=A51Mk-Kp`u|VdSxV*r(vTI&$ z|2#t!s^4H0FZ2(}G5bu_C{9KKxg{Cqx>f=U<^5k@UQUpi(1=^|rrxnhnuXrr+Y#JK zI~R|9SVysK{jwb*t3eER<`h&V+;6$s8=`=2A-YJTPrNRF8O?k$21Naz7wau6yF>nM z!JVq+Mg|`}N`L=+%y4oTwD_XsVT&X>{bgW5#Fb7R!MlDtvsQ!v%-A)ubE|2a&1%-m zEc&gU&KjFhF*}-{W`d>kND%q4nNQYa=og*Vuv*;+NPE0hQpF;BI$_|uh#jq^!{z-< z+P5e3i?&0qGS*gmKQ~Mxo^E-ory2$g%UYqs zGrk(ol4<1CqLGe1jE3{nMp36^Qpg~6hkF&Qj&<1)feZ@!!*(Y62D;WwWz4WEr<}gH ze^vqA`59hFj_y;4nRvG)wg{d&l}bon!1jmur7c?4yK21$&yM>zU1n~9Vobt&z9TiX z!XA659Urha*V2L#T=b>les)|fn+eh|x9O!JzfhTBP!_^ZXgdcc5`gQOp#qPbr}*Iv z&r&{vYgHG)$tR}(gq3w;+q@q5)9W8F_d8P03#fCF_(V#{$X@$c&|+idYPfy4)qYi2 zF_%2>lF!swZTsib(ibChPQ!dMFll~0C!c6{5wDhEt=z^?ixP>3Zj>v92h>;Oa4 znOkQez$JshK(eGVGn=Bt-(lR)Dl$Q|fjRVYM93p^gaW?WMth}Am5nz!K?FYF8{z@0 z5c$N&cWu6}u2Q#Do_$|+k?LGfPqVz%dXj4YJDPOG;+$g51{>CR3K(`ss6!+Tdr3fDr_6QU> zQo{LWHtZF3{xg`UMk zu8is)?SzVg32(#KC*sDgUE6ljK*e9DrN0^bBkW@@Gf1y8T2ue6ofJ~JXTng6&kLtu~^XJOQkT2U*JOFZ-2^V-&}Y;f#JToSD$wma$>OSr|Lv(qtRkyn`H`L$oexf*0km79h zVE*#!pHq&OUNE|bvC0A_(vJ(4er6l%*!3Xo!T|K;lI zIl98-ra)>Pet<%n+N^3GZV|+TlKo)NA;s0+=oXTM546L%|6BwE=0aX#d;fJz@m#jb(N33`lrXz%4nWc%GyY188u>BL z*U*h^PEuEcS6(vh{r6m;Sk4{d^Tb_5a)YiQ8*s3gL#lQ`i^CC2z=NGDsU5!lK!{A; zTkcfk0UoknBfEiDFN+2wD%CG11R6q=`u~OA(}O`I!tFMBp1o)zphnKb{Bso5^mo(E z%pl@074eE=bBndusHwWt@90~yKV=>WkSzC`8)1B!c90}y@WBxUo0Du*wC09>b{|Kj z*%4hq$AEEc0xuHN#9cPX$4%0rsfnM`^txcyw$4-p7ZH&tjzj1cCsVCn$KLib$VLTvty3?tEwMjub{BbxC6Lfl_oZk`{%E*e353gOe<1gLl?(>cyaL zHA5rpRRl%?I0`m9d~a24Qd%fT9q&$=30R%O@ni>;5HkiNOVQKXfB!zT*6*MUvbwfS z$h4I}hLB z4|%#vLm_dY_i*wPZ-lE%)nCnvXUo!tE0pt#*2oyTTF``D#`RWepS(vFX?%nJDe3X`1M^_ z`#~?7j5okObpOxn!)P5_0|5D$ZS}WLNQg**<3tEvA!znC%ZuU@3n#yT^A3}}mcwm2 z$3X&#B*>6O+9NRkLnqDfm8}l8Lr41V*w}K+V2Onum6|I9-iwA3+hBBm6TE}M| zbIRXuIJseQwf#A1ZkL>RmYy0f2Q9-7xA_zqUN7$}QZhA?%>U|HpFDw%Cl_$#pY{xt zS~B!6cABPqJ{)nh{>HH$*u3xJQ;Rq5Abu-ntu%`|EEMwmTjp`ibNu5%kUKu}hQN*D*J&4_VE60VGk)g!>ZwBTuxpiyj|U;c@=>p^ zq|*R{dJu#<@!Wo}E)nCr+uTHAyO9Jb0zRAqH?cu5>}|7|n5;~YR~ zCyiAsWY<$`Q%rq&8&8#^KUTLzsFIRwrb1EZBLX!y|J4FS5Hj2oBO{ZNG-}5GFl4+n z*!tq6;rn{wGpV(l(%dS)*jG4rBUr!YWsi#st?U!-g5JO~a4ht~nZ-X(2?0vte9-wTL$Qa%NWaf8et0IDkz~ z-RU`G;elM1jl#1UgtQg8aEdIdRi6Bu-GS zvA%Hhe-fi7Szv9oT4_mZ=R^r0026f|{FYM00_>gvw z|44r@a@A-U>!m?m_9#c%&X&o@SDEZ}E&E+qqcdW!DdTfq7OPjvf#jAk`}};Lm;u+J z8ld$gA?WNGTiqW&zO=ru+*9*Y$lLA%kMX8>%2DYg+xJKsfA@NnUqaH8btL_{1gZX% zIxK#Yxtna+l6(#AZozt%=n5Z=Qiq4pI&h!(BTc$oFgBT!3By?^e}^tx#a4^{;ip!~ z!8onWd5R6D@m#rf`tN&{COERjRaQBHK2t}kuCfAC@8h>59$a!zvR``3y7z&Fcn?1fKUOja}PWX#^sCs6m_>N@yUL#HFMc^3a(JhXMEf zr>PxT0=(uDwv2@Te*svTb!~g(=ePhPV`!x_gbW?G)9&r7n+RmQHwOgq|4;1h4c;dq z;KU!;1K-%+|LJmb{XTl#b?;9HIsYMFdzzaqtXBF|5$}3GIPuI^NL-^S?$~QA-qZCP zZ!e|_cRI&;kV3v;#BSNiIKvRhKyBqST5ldX7NfeATuf(cJ zLUy5AdW*34e?AsquAy1jj8lmzH6cC{h4)QKXyGY|cIUBSoZuTMs|812HH-=$^E*xr zC}*|RU2U9T!MF=w3mIN6H8DeE!6YY!2U}mZwHtQjhn-{TW=P;dqv?mU_yg+kGf)rw z$1StAQ_Qvq=%9be_(~UEU9gvl@9&+l$yvo>>}LmL!^{UC=JcxmtA@gvTB_Wj2Ax@R>VlY5vGEL*zMZcSlYS+a~1ipasxgAvah4 zFGb;)bGxnkPo5~^s<6Gk@BSm{dye_S(7|xjVud@~dfcN~$3)U&CHc}Nd&B`n=WwLp z_Wo!fe1_U_;ArrL9bia)j40K)Bb%O=q+Rp8aHtg_S4a1X>D>bRoY70k`F2I0eaM&j^PMTS3yCeqUcLURXCGf!| zrL`y1jo#Mw51NI11f{sk9vP9lqEKZP^#RBuo_wf-L4B z(^2P3If$Or&b7bEXPsz-kdkIAkbc?;)VkJ!YhE^f&%rqJ=fg3-K^;)T%UK!7r|OOk zm41H0V7sqM=i{MVW$A3#8q=YzL=9fs7Xd;`z%a;pS?FH(r`-KInQz{3JnHc{a zc|i=XR{%>cwBXV0tU}6(A}i=Oe1%pw)j#2( zbeaWtKjq3UWaT?}x;dPtIyCQ^UaWmNl11qZr)`O$U#*fh4r`uAam zq*H+5uHlQUG}KXsQm|WBQJ+M>*MJ_>+#@tIBX#)o?>*x|rA7q_)TmXGVQ_KUdwnwB zgyb*lG8s)ptO;6?Wsj#m-m^lJ&#CTAPU@N=Mw)%6Hab9qT{%tv3Q^Nioe_2^4;+$xjpZi@3{^TugbsJ_+_3ap9Bt^MS@-B<2%{6Knjht5 znS|rvlC_Lj1@6~BnVCY!Y{ufd_>c!?vg8J+dq92I-$Axrt&G&2P!OeL_qRRwB65AK zQ{JQd^)bpKs;QFWnxV*EOx0f{;;$dKzUfl=8W$BANz)gjWh;Jtchvb8;6I1`>b8+K z8y5E19*xy;JFN9zfcCg#Np66^fmR#WwdPZkExv!w|vKS-z_nXvqFl{vj%R*DLD zlBhDzJJW>YGw+&K`VjVBOD_LDz%*eRvx|O*0+HjB(@xeLS zJ%l=`nY(7#<*0I@ufI0Ht9kb`24s7b)3r1?XGdptOQ@vdJ>_Fm^lo25b)07mHEP>13^3aK=yO2~(bj3h`O8` z_Eb@^qkqhcc~pc^B9ji3W^P2wP)!2JpC_IbxVG4_RHU&8i6H8WLr25spGdtz`PjVR zGf;YQ=ueH1kEzchr-oC`{>*ttbRY272%a$k0N8O+;3X71*ZBE18n6(z@aSl6&yVcR zf9mQwx{u(v534Myk-{e|Zd9nX5Y;c+ArOdkf?A5(fsdffcOdP>AmQ7YgUE}FrfL3Q z)=&@+uXEjB!q*CWIX2DmMdkkC@|g!YLC@ldZ->Ox*%$&zjtb%kIfqkr3gxkft^jh< zIXNXN8UhKAW4M4E%5sr+wCT*-c{0TE!%R&KkMd*Wd7>~^x4T@ zlrm48B?C+C()$%|a$zAyNRzE|I>2_`*!de3kUOvQM~Oi)u89*HZTB+RKi+S2J6q~( zno7$T`HKj-pPRO{E;NRO!0V{g6hygFPLgAltn2s_71Qq^jh}HiL=dR+i|sJhDLL4v z4MP^ucnzUm_{AS!@?&F3iywT*?hIq&d@tCOoa;Ag;rUwLqwhy+?yQvcFBgflKI;`I zSf+C(R~@!fpbVMfeB*HF2%paS)NriEI1>|L$ao-rms+cX-`pW4>M~m0^>^`P!|GZ33W2dc^pJ;2;6M@e8Rh-wCLs+mML7EiR%#af)T{{mj)og z?Q0Lb{~>sq>D1-I)Romue6->DRLq%NC z{kGo4y~(9o)`;~H0PS>jrsN0!u`aQ1OCZm(?(PGs46hes2H6k45}n&#TwaJbPxK=T zGwAw&2lpUW?0@4a>|h6ISLPo|or9Dwj@ZTTgFjux4w$W1gvLaupH4pseC{ZSy}_Fu z&oawrlfUsQ-tt72=e5!TA2Y|`ZSH4q_6M^0=hLdRN&jrsYGQ*84DaS zqG9Sr>1mo%{cBPAK)y5H$z|a8k6izGR&DA-<`htC8*hm3v(5##AEgsN%vPz|$?4k% zXbIrP2k;A50~LA}=t`iXcEd1CKtMG<;!I=7a`DSOOf2?s>&y3anNU^om2%B{QG82J zzaiavN-;aP2b=X<={EdUUY6YO5Z+!#Zm3V|O1YTj>Uevo;!AV0S&)5Qp!=RCT%O^2 z+K)b*DtzHR3!eLU#Ec|-YG^iSMLJZA$^lo(_Jc*t63YGBFw_}!cLNLRgxg3)x9~^_ zq?5JMXKU-i93Pls(xxv-}hM({moq zGBAmH&qb%vu57A=&P{qzWO8-01tV9+T>e2(+nIR~Yf9t$E19EytmM@h@!Ue4&LUbrq@ z{D;qlP2xIDQ~B_C2iv>THbJ_zX-Z)5^ta)Jib9L@n}I`OO;Y^1Bt&?m?BBGq@gRYI zljZnooZ%(8=EoO$qL=PXP{tpWW=&cmHor5i;s;=Jg(OMM#u;_8W9A7yFr@9-y1H?nrY?r*UyvE*8!h3Cb~3i) z!dM!>%?M8a&}`at!?Z;|m-gAX3&`9NUG zfb~^^Hi@}+mnpw}$(UQ~&hT>;e22(%^CI`0UWqG{mhGN;Cn7uXG2}V8ZzlYiX2)hw z(3K+YEL1zmxUrWo;dup4(hvdzF>AG-*JNY3`b97i4Pe^0(A;u{x}*K1#1P8gbccI! zvHK*;k5ZEMIGXMW@A$}Qgf%uJJjx%=LsES(8a6pypanTr#jaVHx4=<}qg!|27gf|} z%@_;q^)z9Z$`qGK9b%2#(Gm~EB2KG(M&Wo3z!z;Vr^nuaYd>7o2aNHu*Pb?VX*FCB zFUj$`UGlbhB!e>Y5qYGuq5P6f^z{>hK+dZCso6BGsehAKBLMcnO0RPF2%rMo{7dN& zzYu>M;S}XrxXRq|ryKeiL31KO#J+H|Nw5pcW?!)S|0EiKFJA$1nz7)6@Ds4#JoN0- zeFL0{5qDtgs@%X!P{G&mC0C|9`NZ?oAT^)9={c z`ICQ@CA(=0rQD3|4-+!zI`TZE?bht|n`p5D+h*~Uq!2?+XXV$LDW7!j3SPoqy7d;( zFLpK1XXK5o>!y-G_-yF9bvw=GY(j-da3OmL;Z_=_e$^%$o7dUv3PDhg7IHAomT9)C{atHC(H}FqXG|s?AD#owfMQnh|@$Y*`(u^N;M< zJT9t=v%n?Bf9Cm6Lfn#fE}~dmT1bHy8lDbbK=mOZ`erxyk7mLKF)NfAbYdQwP?L?{ z3?EDvehhTY1IH>uW&|G~GjYziqaagL9 zF&FuDMgpJxhKcx}y$K9#*IRv^TKAwvcR~LR)rWZMVg;C&j z&mk{9`79>$D)B2D2Rj;B+U2cbYOLemp=h5yE4-#6P*X@{RTbPl$KoCxh8mF&93z7B zo^KtT*etsB1!Jx0F4x_Ij%D~QT-Ga8Z25e8!KC-isy+S`5Isw3n`d-=UV`dK zr_8ZI0*Hm{b(btgo3DC8`lsA-bIND*GuxO4;+(ct{!8mFw_qQ`_9)fCsU{L1@49qXQ3L-TlV8J!b|N{|OvizA<}` z%kXnLZmk(MKTcR!Jg;Rwwfja(XEq2Xq1-|g`VB;En4rT6h{d&U9>z#28|C_;krznI z7_oKIO2&ui)DiA(tkP5yCs29GAJoUJ?f>hsx1=m>ShKk8BR1QYGvmh$IF&#WVD|%oh4>l(sj_B z*uxt7xsBAKk3~56Q9v|x-XiYS5Ytprt5A4g^Yc> z=t}376T`8+5>n*f)xE00x5cdUSAeKl#=FvZ-}9eDXey6gCZc?HW<*Kq__V_}SVoIE zx6lmS_@q$Q3S2HQ2?Trda@}Y~AH@_48ZX(-|LW`-XgJiCE&Kego&2)YL^i6p@K$bg z89DO87r;LTij2!E@aeVD8w|YJXEen3OWg3UU z&H0?`HxVBnFs)Z(=TK?=Mm?3X^|32S{qQZy0(wYT&vb5ekcc=v$&^c17W5gRXlK}# zAaFEoV!aiYkbib!Slx-S?aA=X%3E+fe^UI3xC|9#HhnT%~uXecjdi{a% z(Izy;F}2Q(3$|BCGFy2yL~vTG{KZ|Oz;hpZvnvu9|N04DiSbUj!ipSlU~d~>Xg$xi zltkyfW!yMc7sau{6?f?g=?C8y-A@HegFbg$sxt)SgBh*UHMRl&Mt^3n5euE2)}O`p z;U;QaZv*mk;EZ>8^u9HgT{HC0RO=xqUDRVYrobdgXWUQ#F3Hq@LZKs*5tRqj9^h`h zw%_@lU0?nCpf!`HE!))a%jpy+ zH18wQqC)AMUeqS8>dc3lB8RMM7hN^MzcCIe$5Q0xCd4wrYb)^(Ab&`kO#H_xleV4{ zVqiYrcJgun#qEv~-`yaxjFS=NIx%y_TbZXJ#!sDuB0EaGF%VPsWD19#YP+#w9Q~=x zPUTichJs5(jGqf^G3}bgR=cT7aYP^uYU+SSc@|#O=z%~yP(5ebUOEd8)_`gxwE4Q+ z?4Nzt=`$j>!|bnj8Mb}5(G!<=2=?J&3>kMhX_1YEVKxOa;+FGp>g`K2x(Mm^X z@P388>|a)wp1mNScXXj5V`Orp=IxxqX+yQ#YP#9h4hVh}hIb9f*HlhDS1$vd@J?!e z^Qd4gfNC&r+|_DPSmHAnf375#s~r>U8j_t;5n`MQR%O@^iaNs;%B^MroAg@8+1Ob- zYExA%_4nXx4Y+&_omSTAuaMYI@))QLqBL%*o?xY={V)j7M-STPs0R=C?Gdtj2)cnJ z`yBT`O1LF;Tx_JMI-Yq-3!ognW-NbV*CEZ2JIgay!({I9HmRn^u7m8(ICeY>=Xx!T zY%r9L&k`Br&ByC~I;UHI@^Dd-hNoMMI5&T4l49ff>ggsIPKhSCxT<_E(ER?RlV}P! z+l1(yu1@PxE!78lS?r^*4bx)BrP2zy)E^8<&ffI!zmjRs z@8~Ae-3pe;m$Cv|Zl8TiAvyQ(iXalr1734n2amxKn@!Y0B0Y7J{34yd<*74U7OL-W zzw<~tVYrLaR#mK#s9DH6KYdP+z6^ezt;=#wrrL&fjKz+}!VHy<=G|8Ov0b=)vvq*X z1*w=hC&nN5F(@ps2dci7YS+GMAOFexhvEO0B|r z(#JQ0TTzHm%J=|dE9G~j$NY?@w`W2fLo3+ouABe+Fh`Z-p4WDH8~5hztMK5D3FCpP|PU&VcL6l12u#a*)W}N z970*{1>P8fPrz}jTE)=0k)K-TNr9OgDP3Na z$HgsjfYftYN`cZ$j;slHN#hR1SI&NCJhf#djMd>T<2==+GOHFnWlt6ZN(dyT^%7`M zOg4u>K4~%FU6gIx2zJ_T&X+WI%m~yJ9(jBq{7=8JKUTKT@(vy=(@tB61Q-&VLDmAt zVjg)&J%Bwki<=)`-C&FeV2o~B=PXs`WyO=qA!ZYWqsb-M7nx}rw^_x%1yTXc%m&*C zl|`hc0yz%@cMjEzpH5T~JjD$pnWtHV`Zkj8ubc?6iPzoO0GVx5;79onA9|tY`>z(@ zF)6Vvi8&s`{uDd#K+r+>!>^YQ1sKM4=poe&vo+XGnc7<;Ng1E+4JkpUm#hXojITBr zzJzys)^Dc%S7LXalyySSup7n3fcK+Wgoxh@$N06fIqirN@cDR9b)`}z#%ozbNi{C@ z+@Suph7Yt4F}^oluW>x#BeVbf2@u1=zV367q1$D2l+_~n>YtDyZCLz9O1wHFya(Ju zd|Z)&TgZL+Dg`nDP5(Qx+lZ(eA@M}{Y7x5jr9dmM6N!x@>nrpt}0QIvhZf$QSynQuo zM?4KRn zJ{Dv!HqM@%dHsYLJTEyRfq5zU&wm$;x792D6D_hjE!$bN2Qc{qc9~roUtZ@-asm4K z_R`=7GTyTsM3kJJ%0ks@5x{dCFmC>DX$(20s)-W++l7}qrY>>JbED{&6_}8*9dV{3 z!tMUCBIO&S5Lc>!WZIRQF|Dm#8v|a;_ ziQ5W1mKA3r*m;|0bgTQUvs`)1wKpL+-%9Y1!w(2q^r~z~xMt%ltj^2^yVAYc!8ZEM zeo~16#@h@3?E<Sn0U^qcnZN&9ue6#7%!iFk6Ee z43|+Z;{Tnj1*jqtEz8xWG(IE5_-o~Y%FEP798W99>y)G~-dKRIeyVT4NTU(vKjl-C zXB3)t9G8KCy)Xu@!V;$5r_0psp-h7lZ4SyqJRxKp%Wzo++Y7^` zL47qv3zP}p$OSSRwD~M3B34Kq>|u%Wa|Mx|v<0>h`nGLzdk(#*R{{E*n;sIXRa1*( z-h2*^?r5g}UT-Fa^Et-CtX|MP_fHLZ?o`|Uij~%cza`mIG0zm z0jQ%RV4Z*bt~>)ai&qX~Vy^;TNdF_Xh{sxri`gwL9+&s~)1T7K!v__JxgPKynzIe0 z9H?)7)+xoiXX3pGSQk6TCia@=f#lKMq>Dasn=fm@e4#S8Y%fn*yjM&qTKU!hsj*H! zIdyg*g@Idu-G`o1n$Q;LiB{rz?u~oTe+F~y{;>vT?a2k@vIufg?vOAqZan{k4*aw7 zm8Og{=QOB#KORmbWEiY<4QDrQha`$B62W+E1BY#I;YQw97-X=p3PBzp_?CNzm`S@0 z7kWDDkMDTer4c-bZ;+|}tG9H$ir=IB_DRD1s26~TZL*3-ACwVR+2fX!X4xs%TL3>V zP}!kyV!ON=^D3KIAn|KWcdAByXVL{jZ@|fA z$`B0YiCbxzbK=KA*$UE%T1*g4b#?G{&J6+rdtHE{gGJVBjF{@5#SWspC(zj5pq(RB zyPR{ft60BlCBsE$=&yF_AzHPrVQqgF&yyVmsEn@&9Fik>2@d)wb!}}C# z-|g+1D1BQVla#Pe)}nU2{r% zIBFUn&_8vwH!$XM$GlLWoNhA6LA(5Pi5ZEP|2USE9zRp9_;j@+F2+c(>Ef;}3X9V# zuQQWSw4T*-v2DG~|6#%7G;EJJP_ag+X%Racyf|LnfpDefU$L2h!WBUIbh`_?@Wa8C zXiv`MqrIFAM_~UH27SGjyF0Bh&@Tonj3nE&m#M+2w4FlbJ4I_>aPV%;6ZMy!uYxj1 z8@dw95$?w38>BhrirXHZtbqy7>~0vgUTu?$bTnpbfNI6D0*8+R2ej)k%G}eNJk)b| zSQIKv{rKk5tb*dT2%CZ}_Dy+;C{_%}t_8=SFI+VVM3;m$8SjnSiBV1cq_wPF=mxNP zvsFwzJm7d+1ufGb*}t1XJInA)6lb~0n8S9K2Z9$*X+LavH8(g+dr2|a%kQ5*+#eFM z9Q5FAV`@z&d6p#>7};m-)#ZlsYBO+tUtRewmpge zhDzR!joj_2Z!ynBmWThD*_|P^|DVD_0<3|W&W-((M&6-p6z9Qa#(H~)!#l)8GRI~6 zgA5+tC-=oSC+{zb9rQ{Xj*0=%5Xo&RECVvJ0@R_m-F8HuJtE}z!^75yFYk2A7qZIw z6&zQ)W>Fl(JD+obi*ZI8P0`1q&N)yTa%#W@4t_~PUM=)>7n#FlXmO@nVx{~*o0#0| z?K>`A9~Q~kpD>3Ysd2LoY)b79U0YY5ZRaHBxidD5-MD)ZAE@n>wdU3Q6AN@@`h#Mf zt-t_HSulNc6+xG@lIKf}2XU|nX^uqV=q=!%A!7$8Gl&r0Jv`gGyx$v`0_}zDEX2*Y z87}TnG*^p`4o#SKxcVKnwVzLMSl?X7?Nt-bJVq2IvAB>rl@{zN^JX#7yIW*Ut*h-Z zSJ1W`sDZ=>$=jlLd~<>Xe%U4a$2!~oMz3iM5~RN~R?WQ|$%YRuL9CsJ*2iS;rS z1@$73Z!1j@OK~Q+)N7`u)^EU#J1yksraLSub+>b)h0)!PM73Uq&IZ;aDi$XQxIptA z)Czoa8^1sZ?=3U05NFLw)G8x8dkPSfb7b(mpADg=q^!EX+dFE{?7B{ya9vhv&xyrP zsq2qN64wciMwODTk?t-YSfAwiUxViNpsA}f!iXJc>Qc&A8X}3ni|7ju6vQ#UtIK$-$N&j;AznvE~LwtjG=7YWz(2{Dk z-g#9i@3T_M-N;wK=qP`_)P1m-E?RYlu}xDR*90PTht5Q9Z7epL3wi4d96*{n?CS-M zc@GwX(f1Ebu)mj`jaoW+gkbn6-Yd6!lWU`cVH0{AJ^x1?xFhP}+&htJ$E z@Gi2Iq<#04T0$UEhqSa~*CQ@`NAO9X+63+D9XiwdoCDvhZqzWQMS6DcE#sKu-W@%t zYfhKRM1y=!^mmJu3Vy5dCDE_D3KH@^bjHbL`g1a~-!)vu9?JWsw_HuP;Q70;V=NYe;d+YJ%`|p!F zI{e4n>-{ID>-rVm&Ai9j#Se0oEnkg6HFtGCdghnIl~jX3P+bD~J#8v$>|4AYYEBpu zH!X3zirJv5$eA$9WVexpDp(ku`0*$UAG-F&{L5QfNZw4EWi~7^xxQ#1L9T4)d~6lE zY;Q{YEoiLl+~*RydCt#q!zOi2Q2S`vcnH(U`FYOHgK9@{sPh!^d0kwjY;~QoMJFdG zCIe$hJ-?RSVpB=0tF2EB^Gd8AICZu6l`}S^#7W1|mUcGRK+SVgbJCp_wS=4#*MvhV zJ#yPt__Am;ruZHXQVwtr8vzgQFV8w#182Mvnd(d~`f&tJmFhB{%iy>NqP(cge<;IA)@u#7m$5=H z|M+XuRI^JOd_LxiJ4GUYKtmr zMXcH*RU=lY+B0VCU4qzZ#`wkiywCUd$J?IsImbEqoSgjOI@f*Q*FCQ1HebKA2p0yH zc-dj1jvC)QB;Ip>5D|65t9G<)R}ZSSl@bY9$to0Rz@zzKhLK*-%>Ag<1Ht>XH3sY2 zN!2eT27ZbPoubP*kTVsr3Q;a7<55f=g)BH)0EflehF!V zNtV_iX0fH&6Qd-Zcf_=9{M*Kl(ars+jmcjN#k~PWa)xQ9#pKXSZZJ%yBYYrrtN^Wk zkh|u>*zE%WDi;6rzTKdK$sG^bCTzS{b1l_uTm9_jCnpsx5z_FMaNK;FLrk-XUVbGS z8>U05|D~ycIpPjKcgPI-e0oQ-)`sXQP?gtidis?37qav|lgL>sRHJ~64@abt6V%U@mMM|HTiygrTo?+VM0bGy66=3Pv3>p@2f(Q7@t^X>!fPEQ%P8-~ z+ss!x-@5nJgNxB1WIE&^_e#C$J7&wGMC&EeDl*IrRhsULSJ!P{HWw_c$PD9XSVCEE z?iWauzA=!n0u%;S*t1jZu8{kFR>HsX`0LSZW9Ec6 z>8(2!PJHiXOo6}5rtBXKGw~FbryqB@rUlZP(!(- z>j##UhOD1LC1ppF-#!hlH1W^- zb55((SRriHUKpV#BMf0Juc8 zwniEIEt&%_+$6vMDE~vq*^`hmpb-Gew7MT^zqT5wUz+kKn&^hJ7S<`-H5U)id8Y-A z#>k~H@|SkBl)MhwV5>{emw7XyHGI3*WK4oXO5ama=|ue62*{>6etkF@G+PfY>F)J1 zMHcr`6qe@r3w2EIw)!957E0SEM#=j#^=l)04}Z=iN2PJ|J@0238M@p*MRFL-JyGYoJNO8B?OZ9w5eVvSMwZ5{N_0|7rd?}VO%Dnf3v))Qw2`PfV# z+pEnXY`!VivxQwBwzkFUf9z5%N(yuGQbS7K zGhXtT)Mct6>JYf$$(WgK3E*s67_|1Rlp3vH(RRg2 zXI#AtA0o+Sa4>R!&+xv(kc`LkeD0+JbRWOFCpWF}gYa<=_U65{j_6cvSy^-E-K(Wt zt3K9o%>;8B%JfqWeBF4H#O={p&WPeA<{i&33rTiOtMrBxHa3R78_C@-7b=V$%s)qo zNN@w1R^h|z+7Q|$kxMBV))gr-W*Dj(f9u_qW8bg-bSU9uqv5Cx5I}YW;<;fk%pjYR zXl+&9)osE52XW~0<9xaL?YK*g%zfgy@kdZ+Yy@%KAJS4DJe+qE$~pR}x_veKsC8uH zhg2m0rfP-Mj&p)p9rs-&`3_Y7mEAk@UT9!L?GV#{#Z*-A&w)ISM>_LO@o$8( z^vMMbmq#@*tv_XI=S!=Q4AIvJV{l=%x8l*teb8Zx zLvd^(j2-ho+5M!Y&Qe?XMI?y&;E~48=TB{8zB(Ta7f+SzFuz`eZt6a*r@mQ6Cs(2k z4jry5tXoqR)(gH&r+tK^vn}z&5REaqpCY%4_mjL+HW%L0Z zeN~hn$9TPRKRT}W6v}p1w_E`Zz40wMeYSwioU^+<=kt++q%!G49rcWZOH&n`5N~&_ zEgF9xzX3tDvX+k2Xe%1wbuonoWH*1~?P0Og85qRIOgGUs)%Et=`#P={M+Gx2r`k=| z*6Fti?yMcE)YREC(8yxh1blq2g5q5c@~m}b@tav-uD&G*YJ}j%}tccnZs%l;B5~%p!?aeP0f{YP#ez;{7yd_n6<|E4zUJGN1h^BM75M(|lM ziM{;wv~ETpcfF^BSV|gh7Lctk-6>01dz)uBC8h7swLhlov?%1ydovf)-W)U zGrU`CT3X685cadoY!2O1IODV^b9%&)S_G1qr`cr!XUqA@U>CGSh+{q3QtBdhB+GAa zUqeO`bypxgL~fLr>66|G6|p`!xLDmiF9ANDC9WAkUDyA}iV> zAfF)2i&u1M`ZhenVBx^zj`3(w%AdZ45KHGmxFP+xDxR$scF;d z1HN((oVl@(#8*Z}>&AibMeiqC?&vC%+_?*eJ*%1y^RkpNk<)Iv3nWXqR&v$XJST(r-!F<04%_*&dvv3Rd-Nwh+sQEAqHlG^?M2TqBs&e^Nt|Id|9 zznooepZ#K8mi-ar)&-~zgNggLvI~1#b4{GoiJr|3+cSS;Q#_vz{wrh(UR)~_{FvGH zuVvc{XBcy$haT93+=ko1JS(W`{SL*Hq+mpkZo4$qME-T%Ptp8XU03uCs^RoZ8kE{n z$exW2Ge5&8Iv?u?t}(x6I;E{R&0CFhRHLfYn%_$*a-GO19$pWqym7hAWex}^*#SBz zD4+^W-n_i@Ue@hFYhiPTWG=6eRKLqTIWeD6fNG!5nrzy&zA>}E z(;(?e8imHorGD_XA84E$rpxO@^Xl=#$h^e9wnL(m)tW_Td-njTH#Z5mBuoJ;N0Eu< z`TOqsM~QpuB?+L&8fapF=k<>`>^fsNlK z8o!U)&{eWh6IOnvf~g8c?xz_BN=2G;fu0TV?E56fdC0I8mPI-Z ziwnnoPaQYia~FNQdJIV@ENrN~p*{mxeJa(3d3TSs!Jmg@YkfG?oAP!;ek(=7m4hHM zbTKLO^)RAV_EZAtZJ)H=`d@k9olEX;F1nN6lZ#cq?up#St1CFb>S3<`acr0Ja)L7=x>OpPzcKI6VTsvu9c(;Xy=ul!2pvzyBd(6~}R;t_S$C9HuuMvXIugLAQ z9DcM7zR|{ z8%c0YE!M(GneS?YMmjyXI*>g^5o2u;X|OmBC` zzoy+`!Ikj7m-IT7iat;_)xTe;yEe=eMe(i%SPY&%spS)0os~@9*e!fNk7*?#9*AYR zzQnW*JR#PVTw5Epn_uVKCoGzY3K>4NNs?f8DtEpafto&PI7epD?*Df=y9>(K6xBTx z$XX?Q_|)E_ekK0m)H7e>MhJ1}X7a_AHoAAJYNTV!3$&SgU;Z^5EkS<)m5{kxr{ z%|W~;{$_5dS6o%&7RQ62+s~2`J%kk}jlxXqT*}5N3tGq0dKS+F^MbhxiBa|QJAGCQGZaivGPRESARpY-q4KOqao5+1J zW)Q(ZVIF1BO8~r7UDPNsD z>HK@JT&zH%E|hV@I>57%RlTKa3`oCeO-fzJ)Vb6ISc2HZh#m;CG^F@TM;~Ada*_5S zmf`@K#NR;Blg}-{sE;`^8U)#q{%_NTd+y+(d6F`U9@7#58EsW;a%xPKVBs#JwO-P} zwdeGFE%jy{Vle(Zf$-T*C-4nJL8yFifnK2NNKKH5?L!abqs}1h7gOv;;|1)Rx&<2p zY+A*S1PZ>Xae$1hOx&K^ePj1XWuWkPdU9^KDS?N3U=1BMPwTi7z;5p`IFYo6n*?dX zj;R9@Jp-HBoNG`b)H`8HOq2ZMnP$cI#@M_n)uI9#x)w`;xsd2G8@GJF-mcrmm~1Dg zUU=`Mg_zc_X*Z&hQSfqF2Ew9?jesSgXbYOnz%Z=F5Uao7q&JfmO2E>X@6#vXZO!Q| zTwU6(^}&nU4L8LHn(rZm3Vjs13b((PcZRbMhp4Aq{r;{XBZIA?aJJT3KAk0yG7-$5 zWzLVYSwG+ZR!ejpW)=p~dGOl-Uc2_c{r$+S2_H5WTfxrtdOdv=AuL z);BMLu^sF!h?Z4$xFz$%aRQoC5-^)_PyLc2GoLgG-KFavo{2wvV?W*q!lrsnSKF|( zpSNLp5&;}fGA!eEZoT*SY-;`$_xuvcv;X^~f4&qcdE@z<*quw4|Nbl&k*rOq9?3DK zJh9W7m{YtmL7nvhK09zjR)*%5z~qj6z%i6c!6*i}y55sNE1$U@cfGHk!gL(T%k z9Z+AMKz(}^u;n+2x`&HTw0?EYLscwJ$(J0_ylvFiX*tA2%K1?yHIO|#GU!Yqp#Je- zo%}mf1}T@~Z_S~awK1b2)OtCs3Br`KCOe3V?cM1DmLyU6Our6ejnPz^W2nz^v9U3q z)%%lbag0u#wDw06%CSA8VJ5Z$1t4?#;x}<|K1Aigs0o#(i#oq^wZ5siu!~E>EatFt zn4XzYU;|r$)OnXWfl{9TOY;uc_1>+m+V7<)E%pv*F%Q6E%S2e1Gp&rTLJiwI=^5RE0vaCgp9@0Qc09 zQP6WGLgLT7tldSsj!};4C$V&?-tn?;JO_y31}M?<&taf3&5q+XVe5zR%Iwh& zr{cpXOg(m>!k3G6Ym#h#>ia$h(xbdeiuxi$+@y@>Pfw&ILNbBlXM^$Kxf>~yG(hY_ zkeyP(O#tbwD-SOHhXiWV~~i=JjDkNLu?<5sNcR*PiL)o+ap>2C@z=d zh1i)0wGq@p2=kqd+ZlCOH1{c#E6cOU+ptw=U2nzgCf-r%Wb+FVe@5SNVPzbRp6c(4 zXJ~s*ZQrIYeN7>HLOPAY`$$>D6d@6xN|Es<;@W~CD>y9-#(AZXiur`~rjmTd81kxe2ueqh?sC)LIJoJ)#g z4NgUW4s=)D9wO9-=?#@)mh~|hhP#Vczj9r}EK(Z;^%f-fhOl=`hET3j2+{^*1;kw5N8WpsC3 z_bx0MGpS$MlmXMD6O48*8{Uk0D{og^?!&pn;8xOy6LdnnJrEaatAiFa8?V|pm-I`d zf0KnQZ13BLeS*JUm*6fKj_HUkUs$w^69@X09Y7TAtI^NFU#svUo}G>y@e8_cZ*bAw z&qrRIA(9kGt*74$0y>8AkGQoRwSCssvk-={%n2np2r^=$Ou{xzV=*r(YRaqNM`2t0 zfX2i)0|AR~?q#U*-bSq*&iKZMmvh88bfNjjGJPczKop*S{R$R4L0ZN0b|D zEogZ>V+6l4me^(ClU%AOR7)#Jtk=2scmmB0`t|@kvZuAQwyrHHD=k$25Q+9#o$Pl2 z3|IQ*UOOHL5w+jmFL<=i3A%2q{mn*ajI5~iS;?6KDf_gy%1F_AEN(7|@U7e0G4c@T zcXp@jdwX4ar$ig#zI&?bVmNSSnm*kMqp|qeH=(muhoaCZ#9eS%<`rYv3c6V;LEh;V z^vmE6vAv_#)<#R01H_O7XTsI|vL9mN1zJLG9$dPc+BtW1Zz&ZeX~F` zpDhwYE1mNSUcsh@jgjvH?@pbEzi9ix!y&Qi^5Mjd}%I3+K0gTS%+jcyWF>X4>@) z(-WOir!)S^P|1((?T{v`xb(FPF6#<>hR$J@u8iv^OYz+r?(6$5of9DYmstBh(N~iG zGV%9*d@z6dS0=pp@J;-)fxMTDx}ojTt3PjqDa>{q=u2Y6YxU^B`4j=)jKonh8f6JFLN4XX#7 zHt+iDWL}cx5AtNCJ-RLZGE+xN6A_;bU2%EZt+&8i!@kWQ`jA*D)D|VShT0ILBMewYfv- zS8XVm7bXqUf}bn-SP^4yuf5N72Pt!EBm5=CrP4Ms z&5v#4;b*H^tyay~O}k z*z!@Lu>y4XhWE#?<$%kuV3htJ-`>X$tg9a6+|-N1(+la`^HouIJ^v!KueKt#Sy$y# zy+n=xTPSTN@va^DAR?}}Ra{SMW8O>av;#cmp@QA;_u*owmP_vswpR1Z@1JV}iIRd{7O*6EV4!N^avqCd* z?e1*HlO9bMuWFD7(B#Liuk&{emr|03`(rD>LfV3(`uuuFH zBkYYj#*xCL#tpw`YU)>Ivnuj2Vn3o6=kKqZo|p^;ClpRgh~9sRj(%M)kE^^3K2l#5 zEK5Lhc~yS5>@L=nf(=2o+~8?KWB!@rnr^VIc}tAg7h@nZ(jv|R|Ec8lkAv0Y`Idf% zz_hSPD@R68jjq%Lu*Xcv)PRW9hSw|Ssns%N^4#~Qw$0mZG>qh1EiU}66X7a16vkv8 zFF#r+7oKT(p|B}1_t<(ni!uw=Tfv|l#Y5^`cDZ!B`DZ;Ne*$^Y*;JQZ#U>CtFe2wQ z^-1>W?yX~-U@d>bSv8HAP)wn#^WY3)nqSoKP0p)HUJF}aCf;d?5Q*x+y$c>o$5R=6 z&0#*A8sFQw%t88Bqy-cZk*FC@w?X#T*TfG4)D?eXWoJoch z*(nsjAf=X$xaW`yOi8Q;`*m`^Z{1!S4_{Sr z#{982jfdFbqs1aS*1TgwJR;pR3&l`68KO5WKmz2i`O8a;3>WzPEr19qXC*A~~FHWsTG)V00CJNW881?PgDbKNR+-CBuut=Zi+dH=0^C$15z> zhLZPH?cIVYCV|R%2fsb|8mV*R2i9o-YBRz*lC8gq2%5c;rib1U=;dv5hroS0{>O{~#2Wgh}cv z+c(&77404FN%BsSIhv>(umF}~z!WH&;TcD@#k%O?JIoVz2u_RQSkl{+FgYl9w1}2E7Y~zhMZk;*mY~k>Kc?^ z=Z_nk^)WSh9G>bVyBRia@O*>s?b>4Q?-()dSBrY(p8RrCz_{E>0Y1;AO%;V@F+HXR z-WO1ruS4J%Tb(vw3h=jl8)EEv#{^DBgsj7(M?e=ZU5cJG)55dURzp5_u?Z9CO=D!cN*duHRexFpx*>e1w)U409u1-S4E&lpGx-z^38UX!sbhpkQYBTQ@j zxBRB-v~a}haGBs|t9s5;5C?^nzz!ed&4~WV^8Q#*d&Vxn1IsHB;*+DSt0rFO%P$(= zKl$f9Qq5V}U&wH}|EQ4>i850F+4>=iHf^VTaF>NP_K$3l%1O)K3!SMhR?$d%*wWhN z$E)p5TGlhT>HY5F@1v2|tR6c&vJ9`SVI>C2qS3^JWcT(*V||%=6P07>E0}sUf5F1S zX74~kFTGV3Why%CQUhZSm(lQ{gMvns~=4Hja7|D<+YHWlLiZa!X`B14$ zIVPF}O+ot#`OMNoM@0+3T%A5Hrn&R1!*nhi2y^2UKH?47pwaAk7Z>CY%dTQ70YwdZ z_^aCN(y~lY$;64_Ytyb4VU*_|I-*KaAn3z&wfgi+!@)jt5?H3YswXZ(?MV)#$H>j0 zy!q+Ri87xJPygN4jnq$C!DN+D?Q#5_@$j){&NHe>M(v|+tzy1uPPr^#(M=|E46wS? zn?Q*wjC!*+zSqkGq%;sAMUu+nD1ERDhwVvyy?kD>rMN#jYRcJVrhU zj5ykoFg{G6bMgkAo2-w3zvDE+G{chOS&+rXnp2_kE5)^s#OunOi8&q^2FyM2dJF9A zF&4?jj30}72}V6Vb+?C$7VcBrOy44%{}P4TW1(}`S@=}@{DH!S{3|A>2$b~5Pc&)g zZ~plbV{bk~G4)zZmXLgYG&z#Mcb7$k@w5AoJBcE*ik#!)RfA*I{{N9W+{LbBb1^yk zwx8=m7{(C+8Mv?eC*6?@t|6YI)YZP~*D;b(((N8%&2St9c`#CM8PlN?E zasJZO2+H~7G;sSoi3V@_(6K<(cA$EY}+z@H2-JBNlVf4mGQM)##qqQj| z!bWfB>ppzc;P_Qok_V$iMtOoIOW-S8yB=BjC!MaCmPg^}-{5@jCoNOq&c%;^41MS9 z^q-PN^}w}AA9P3TJ$qaU>}Pv!@te5%>|PbM zy8%_EF4YpQtdh>c^~=@p*CEnrg@yTvDvS`JSL@Gtu4X;b(B{&LS&7ll<)r0=ev3lG z=uxqjn}9O7imI9x;RC2{zL*0F3%jMrE+TB2z&YZW0z^!l@QUVdMLJro=XT4ic>(=| z@=?Oh8hL3EwQVy9B`aoJcapm?L|_+ZF~FtBQ`su-!{HGwv7Goi(%Pxr3lXCicNHEz zoivubB2S1fNsyLJ0@9?+$#OH!uHnA z%tCxJIKT9>3`zlnm^^xj@>EG15!Dzk`Za0j&mma%eK^l@=#hQZ>=mK$!|ppDo|9=^ zA^9px+-Q$j0J2r|(2c>g`{*e18F| zzOT=8X<(C^B4XKh18;x-d;LJ9K!%HuDmove`SG72_0XETB% zWB{nX)_pK09-@*av7I2wf5+7;|DC9)Sr;$pIEEUzE%z5Pw^@r^Fm%|QOF#-DFoqlbt4YgEt|$xUltVBMjQkXNR7 zVJ=oR-}PCuu=RJ2-RQ492!sZ!o@;;X+q(-d?2w;C<}oYObGy zMW0~=TDnE#soqM+B^0K3o^CfBC7DEZwV1WO8E62;Y6@61u{u3ta~Q@342^2fd!Wri ztK?9MRW1&KF)2t;=0cQ;e=FeYw(G`D_Ia^dB)^-xNOp}XFpbi z@ka%A=`whLREQHAlN#~vyT?RIglKTbFi%RCrn8$A!ZoQ!1fb0$N*g25UbQ&G*{ZqE zV%(JNyrf$5O+zTe4gad`8SZ2G=gE}+-UUdc^8w}QP_Bwqxd4p&mY6h8!+M_sodz7n zN@53?J~MD12~>!U->lmcOj{Ar&;>^fF{t&lv^JI?;g>_(bT`QmY^HW=)~B|zp2)CP zH=ynN+xE(#$KU)xzp(jj15{ZsPL;3jtJQDc{fF)Bu5fZOOWeZ2+(wpl^&e^Br2zhbBF&KU*|6KiTf*G#0|yRl zuKBNn#c9t}8M)#@WjUIzIP-CS;7dZjr>UkDC)(0qb`85zA%7QWfu$l(RA}#F{}cDv z#3F{~R;M50jnLlG(Ic=5-598%%2<`{ki{wTK_(&3EII*X5i3$Tx3BQ7qnaU1xZ;uS zPJ8%9mPtE#D|p_s>*qelmDdEH@_qKJj8oCvHRZR;(`8Y5StnROqx~AWdJSA|y?!@O zdAk^OD{)umE8Bs?r}L8}P;1si?_x|_4KSuWbW$biwWHe5o)Pw~^J(@x7d}5IWbM9i zFq|s<40tnJCZGbSe=OlZs%<3V1nov!!Nl%s+uRmQ(3YSY7W3f(x%E?(R>D%2Kok%E zcyV$u^H+@=B^B~{QpO}jo0gqy#s#Pr*x(blX0$~)t+5#mT@_g)DLpe^&@(B~`gk{q zg<@!}TzqB5foY0(b=5>!E?u3$U<8~^Yoj~EvKUkOWrUNEoB+ZO{Ss;W0TM;ltDxK0 z9e~EdX#;*u5q`ZVoW~Sj#_`s2U5lDlm2pNRoJ@5hxu$a1M_EA)#f$?Q`TgN2V0P4( z8ADHCrRUc*;;EkC*|iLdf&siNZJ6ZDdHw1L*~R^-)!-_IY?E{U;|Lh8o*xc4G~g)!+* zK#mzR5RCu*w^GQPXSL$fade5B38I|c?!avj_EMnSZZ}o=?4)s)m_Is^IA>TfK+|PJ z$(h3{b~f&vT@L$AG6IpUCKB{3@M1$30}9X6W;7DayL7$(JrCTij-!lHh6VGED_%o| zFS=l3xIfWdARby?4KwAguJ|JIn724}I=QDsC0cEt(^Tlss2lrH``UB2gb~w~BmruI z5&;G)UU~qk<8|P%?<%RDlw8{MDA|HP)UZ-hK2;ljiKK9Gi9fUxxwWBG!URnP^~=Ud z`H%bV?mN4baDoQmW1+>SKsCV*RwOBe5VROCg8#mNQq-E>m560TD~*qO<4Zg19q`#h zH<0ST_c_jhjLnN~faBCSs2~HG9Jhl2${9VL<&Xx0#?M@mub4rf5E78dZ-{eKnc!4? zmzqXUa*Vj?*~|_3J6MW$=HU;%cN2kVnUrB_Lm@6ZF zYI7UpB1Zr;%MT0!h5Z!2fRtCOyHz$SNFDnT^TW@hGeqtB*R0{y4CeTUjB?8uH;fWa ztTt`TlRcKwPQxM{L#er@P52_}_=8kWlB^&Z@O$-j=T=U$iGa1+LR1AXmXh(M6mqEg zm(nGJ)cg3r@4nch@rEsEJxsrhU`ue0a@ zH}K#X=c(}9x@AH)=u^K*MV039z^Tv2ZQh)_`M3HkJT#b67UIKwI?9B_T$oNwJ5m~t zWcfN5aEE1IB*M&vsLdRx3hUcwf|S7fvMBJ^%BZ8Z`Id+kkQ{+t=Je0t6PIb};}QQ; z{c+-PLc5Clh5&KmMDRN4eC zDB82&`k@o$QY*ZG3PJt}=%4TElS8h@#8TS#!cJgXFjYl2kVL?3@o4*Lai}g1#S+`Y zt>Im`874pQYq(@)l3_@cvtnj+8@FG;gtzq{s0Lc2_IDcGr|`)oQIRG@v)(q(oRpvF@4vXhu#|SHw+#*Cb1slt+*yqS#05Bf$w|1?qB# zvw?qh)6U_Q*~GFGTHa_g7v8W!+H`| z9Dg|Dg-T% z&m0mUXkH}To&)MP{yqn z9r=3gsrC!6w=PFYPTupr)hL}yc7}F}({MMJ@DYo*sVYVf1m@B3SJKu44;I9hSdn!* zH= zxYMGC2{K~I4Ub<}>=9-LyK!l`892m*O=T_u!kY@PeN$+tRIZQ5sNj*JIp1xb@@zm( zvDyw!z@%hR@RBNnq`NJY76k9FL{VRctRM`nS+Yed!nXr9X0S0js!kO9s`>1Ll>6)3i^K)EMA*|~=ozUN6y_xpzOwT-VItPF-Y*RG3=`$mZ-_m)&r8aCbE z=(>O=zw4mo02lkQG=BZw^jV6qJ=J^WBd2tX!T<}#>?y>f5$oc2)}9(&H{OWZdT*fR zf|lt(Fbdp`&XCDn8GXl^P0A?-S~pT_K{neeL~A2a1xsAOV%6X7wnH%!_w{~lf=QXB zt*JbVg9h8;o|%85`JmZ$#3DM#mm^ zeawyBUt_4tFXFLU1i;CkM-zIlH2uCNpSMkbd6@b5jg4>X1q-Et?TsbgA? z6b?jsU$VLZAT0ru@2TYyejfpJHWHJFsdDPNaeI+sr(o)~?BR!C3Pwc@*(0k;%C*Pl zmrQ+!G@qD92qbUpk)VKlF`o@*jCFPL)$VIa&?iwC43dWB-&b--{387L)uP*LmQgy0 zw90B+iI?VSN3o2?V}87beF(S7fm?D~tn@6%&7mM^7>b7UU-g<%yLoozH4fe~ANQPa zI+d-DYjPy=s;YWp6U%wu@rWliRMBNY0*^g`kn zzn=`GqZN!{n2PT}`-ILZKyY=EZ>*5H*BYOkp^$+AhB{VFxawrzu6LgW+t zd32a-SI{?8Q+ za-NpIqlvq$yonY8h5)a^K^Vqmq5a;ci|@*)6FWDDB&`ze13Yxa20ON8<5^{Oe*AAGJ2*g|NVb zLH6G!jFZva!h1n82yc}`pVPST;ezI8s18in2i4{Up3C34f6!`Lg}q84(2ydkUo>mU z;kK30Efc-mpwF{mntKKn?XsNM3NC(jtj)CUbWq96Ga>NyG-+DrWf^Xbs!3@?7!CIT zvCXK@nPPjg-0$ALQH#3KhOSGX6^(6YdU=4L`M86YtVhH|{IN{(`AGDiwXjlqNF)H1 z#Cp=Ttwb6ZeHXGB>?LPyY;kYY7r62;tFs7*CH-2Hg^dLTqn3xN8z!Y;(Q(`#R5839 zTGpG%eSX+RE1zKMNl(ElE8VM~DmPyAvdSm;DtPHh1!w_tU=;h6*LBEs#G^#sS5V}S zJ@}7rd7CSvMZxC+)Bwmnb<}%&&z2)=SBdc&z@eTW>N8#FCAsN1ZInghRP#dqG+O2> zt$@*xpo?JjZ|dXu`wgSmr;1D8@SeJ{lq;_{)3OQKQGqNT!=lJd7sSi+tmqttP4xY; zS(iGK9XCM1h=mP@Yo7<-IGHp!TFb`scEZ`8G;Ab~Fuw1FUxv=FDXCfo5AG84b+G1! zDFZ*dWuT5pxU=(uJg-W=sghr1ds}cP^xki*bM0*@h*j88CJG$ zf|x>X)te8L`o192<6;v%`XV*`*{S*f)Z1>uL_tB(+(317-WtuiIsciUXQP+M_;)$m zo%AB1xTX1%?%ypI^()i<-&?Mv5{v8#7fljm%vP5O*JnNDYw4U$?^1Xgzj4zofrvv0 z!axV-H^X!<2X{$YEB)%ZIe*1V?56p;#EaMANJeCcuQ{~};DiMYuw8y)X z+D^+p_Rk@LW{x>f7p;+xJ@C?xJw8XMdxA9mDfpt7walK0eC8MdqU0tWb7nc?m*N~D zf6xz6CYoTuUL)-0c>QsU`S%h!WyaGvt=;F73=P1ysC=LfDi%e%EmZB?_~3Fpc{Jw| z;!d4Tof8F(^yONL8W0`(FDNIrL?)+A)8exQ&S5t$v~|jxW9p>KtWu>VH|>2gMA^4a zK1!b8i$uNmuG8X{)@CFa@_>yG_P{aQFT>s#A*nF!N^P$qYKvpu@aC$t+Dl9vFE8Fc zu)|mCx5X_|FE(F#@(fU7<4VhCPvtPWh|EVU_(sJM!>3w9XV zW2v`-ON!&=pU#2yf_H|gWmIZ6Je*fO`ZOEsO)cGMA5EdGyQ!NBY$&EBS~oK4)O}3y z%S?3A{@~8VD{o>XBb<+0geH9Sw^6`k{cW+ASVZ=IhfAgTYCnLk_o zm6jiTcO+_;mSZ~#_`km=%A4O+$?r0H#D(vd&nib0$@{ii2-1GYAu&Kyf4JwC-3(Ig zh6TzkcrGfFzu_1R&?1$zh>-|L&0v3W3*=Z(>Mg}Zl*5JYp8V9!qcJtn-=O`C2qG8ZS+)L2ZXCcb{6DT~E5wv@t z1fzRy#wr%6XXaGF)jC;E$MTz`H6|nKzCPMh>zA5CaFdfAO$>Z%Lgf4FmKhP=&lSzT zMtOt9cHp783RT2$oOcu1q_We=3ww#-0maOkPK&O_S=U?y;;<$igXgDa>wh9LeDX>ErQQb-7b_$@TIm=+!(0 zZ`+gjwj^1D2dP%BldfunC~dwO-xbc9EjV}?u`+0^C}aNfQq4vU-j*gTY{*Z-!AEo? zQKo#&MGJ`stb8W~FEOmsLr%jI>!;Di$AgMwkXyR@&JG3N>@Bv(hPz^hs)s~OLibhf zHOx^^{2`e4V$PSu%arwk2psXe!>i)zyxwu6o&}^9%3}=-x`s+lyuI!th&mrV zTfFtQ$EB{u8FRZ*akVmQ?+s+>|D)@y!k1%yga`!T+Ezj+2_kE1fS8@|xd*~JI*gez=IuGm-K3u(aJJ~t(xc@agx!-c6#bUqI*Gp{WR$f=ZFVUOP;S^>L9zg3`Er8wNU%bLv?Vi=I z7B?|r;TK!d&DZ{`)%bIW1GwFDd#OwQE;5{7`Hmbf-i<_3--NM`nZMz+ zUFePyqS#h{w8vES(!aU!B3*;UjmJYfyM3=E*r2f<^5W4#pMFs7 zt8?dUrqo3~>Af*g^03af$#J*oth{q5OCbckgl+V|TgSf93Br$2E60a-j*<7zLPH&$ zCF3%IDvmdHs(0z44@4tJZ!ravpQY!R=5EnD%7;nY_=r~Tpbo&e(2Q=p_JUhDExTgc zD2ETQ>9}Pz#%v0H$l{|i*O-lR*V#cYcYo6wiri%@1ox9mb(=L9bK3lr=lPp;RtYkG zB;;9EX&!mZoh%^eK))nToz>LM_;4f6aLA!yOZBW`%s@URL_C_zi{f`&^`KJeSbo2E z?h(Aihlv%ov}?LGepI?44%Ot?T%qgncrzvp&qXrsJq2&&*~-4V!gf#)7MJxHGi7>a z5uT~gNR%9Bg&z6WnRYOo_$q8*y?cn$r zXq)WNmi84))+Aw_h<0i!mH9648dNsny);`-P&-`?CExVW9uNDRq_d7bVs1&t&SOjT zy>tqaWVHL#B@f}ao>RrQq5<=8PJH@|$cH)Eq>+nlKSulb*|KB$PmVfbk0bt1IWc zBaL##NFS9yGxrl^ptt##gdzh&^+{U^PAbm%AHJA-KLn||7+eHQ7!|s~q*yXEd9+WjR%t!+3BM=c|f;&}9*C$qZhHKeSM7TpEy{G$H2MErtrbW=I0I`R*%E`{X6B9M}0g#12w zM_7igq#EIA147^LO97_>o`7C}y|N^SHW{>$H0b7qdcMO|k6P~7X%_WCGgk+TjG3^fOj~Fj|{#5CbDl2vviMjMZM!64^`rqIFC$Mq|WS2E}fn z*6NXmvYp1b;axWyF%gO~@={jMApLK;>)?}zzw_5gP@^^J82+G@{rvPiEmr3j3DiD~ zL=%IB*;8bMM+`g{t7RyeB5Z3MRnb7VE6Dx%iJ;ML)dO z2sCEiRLr{|hl=-S+ATst-8RJe1-Vq)ThaoYHRDsJJ<=->)T(uh1l3Nmk_I}LP&xrN zn}HTi4*L;b(krrW+kpcJtd}s&#omq%_QlxCUOsY|$kRXNsW1NSFFN(jN;OohuupDl z>Ek&!*B+snAeemal4wGhP=B*;o8BV@;K-5)I z>wUy=ZeMX5qY~ceE~J`fdVkK(8CWtotR!=&jXsJ_&pCmkgLRH3I+M`hS73EgJvW-G zw{>?Pa2UbaEVw95OktZZ-oZxS=nl>?yV-Yl5@&qUn-B2m!{8ve2`jL+bq|>xp7rL8 z03UlZV2|FZv`Wk5I^E+ka6*c#{%E7fZU$6Xelt;-MtCv)6^FS7_aO?DKTaFLE|~PA z?so}@n~qmt=G&i;ZWWsun5B4HUElzrdHA!5)(@rdnkzTBGAhLFSi_Dm9zwfJzfJQ?g-5nOwzrE;3n^6_dwAc?WXwfVZ=FH^o6^+DLWGQF0F$FKUFtt&iKC(85?OcEji8ZF zDZk~{m>yKKzyGkD@USZJg$fdWg-uQ(F**egs=Q5KBpR!_dCgq;eoB>q;hn^;cV&B1 zk!ZO;k`S%xbT!6E71MVgVng5}72S!!xli$QP6t$p6?%ka;>?%sXhV5rHi^fNoPar| z9UY41(cvow^w_C4=6)YPe>zV}N+YRr6={quL^ZN`1V-Qb-4s|Ju)YVlkK{|y@~sooP_t35p& zgRAO_jL*?q4`v6>|G__gF0AEj^zd!*v(R6ikt>>A`2cifrQ=z8b!7@Ja<%;VJ4PG3 zvHN)!Ih@88Zf9m%U3g($ac`l}*5D^f1ofLA(xd>N@sT;%I>4@W14sBg_=|OhW}00l z$w6M0iuA%;m7VREIq%Zw4&e4N{Xy38%x*c1oSHd+Cp+LFheL!qi|W-`ae)u-oT(C2 z`Vys62|$O}b^Rm*M93&Yf#BUwo^_f>Yplqe;bm4+oceFr{nEb1EI<47(AvNy_Km?) z6m*eFVtc~EFzvUV)NU91NP`=Wu<4!!OBH%FX#b8W69!aP%3Hw$`VmT34h`-wm%ITt z-uLJOd)Rf~d{Z>?Ffp0y%3Z%()NP^zCHkKl-2c8N4PO$Kdrr^$r$AX*J zjY=!L_ZxFH9E#qdM!kA!9ZAULg@K&Ip!P#1%A7cD)4DQPnmM^YZmM-_NsFVCeq<(}I4Ys%-VCq4%hcw4%4C5KwynhCerJe-q136wQve?4*X`a+=7 z+vC@ae#Fq}qNd4-N`=P_YM~&h<-ul}{ms$Fm}h}v%PGD_4m22#z(*k>=yF*6#-o|~ zCD9y0KAGwRrAE+r8pd9|uvpA&ZYjT* zO(8_qH2$+7Ao655-2y@YHc-ytj=i41uH-8xxXX9kq|pMP=8$z7nQLD13xChJ3k{R_ z<5Twgtcmjfih^^kJk$^OJH*EiZTJ2#)tF!IzqIoG^DhA!*(UqD)^m<7-oB)gO+jTB z8!Yl(lWl6KRgCuvFVzr`Lh!OIcspkCptgO}@$zmu#dh)?o6l+J1?8V3R(O|d%(yQs z5vu5wW5r<*#WM4u77Mi-TTRE{G%U=nROMQ_w*=^{l@z>%w#iSm7Cck(>_df^u~GQg z+UM}6c?$9ZjWoBVl#d?_+fcuyo&f1*Yt9 z)(B>`v*u~n61*^2>Ug_tFPGozQSm)?!c1~A$dat2##(zbE%EHnHFD7LwkVO+=Xgih zbwdw9o*E!-LEcT(v|KIVk2LG%PBmnk3G2I2wGy3Em)_iom&msYiqe1-UjT^V=n4?dJR-Jyj3X+ByxkvRzx)%Wp2d-w9%>`QwkZGCfT z_0gigpm(t-)hFxB`t-Pq_3V!l+A7@(z0?Ja23oD?<+D4gGZA7k6dew7lqpQV{Ypy; zVyE~ZjgQJ}Jra|k@0S3Sk%fwJH+%QJqRs@Hqr6_C+LtTBj9%Yun(&Ct{PnYk{s;p> zXkb>MmgpIqpBIv(9E|M3l{OmD;P&}9sK-*YcHiFQl0%SOPv7fr)UsZ)T!}3*Kq%_q}>)$17-7&iNFe9>=x|Kl?zM z-hjAOz;R9O0wb#a58sn@Q06T|7@8N4@vBJ~2hxTHi>asRMC4$thB)e4hgmi5>sL|5 z3Qo!?!yf1UB;H)}*XD)yE4jaa=O8M>#;pBh;3RizAcl;pWR_VguZx*Clgd%+P^yok z>1r%WM;askTk%dRp?J1el*z;U#qfs|Z+1IJve1xONnEj4eZ}>_m(L&5IrI31QEPnu z`I2pWzsx6vcB$pRTmYC~6F=@BxiKvy$x-SFg4~kh z{`WC2+rFK0f1u^|83zQ^1B+1lbG}m6it9pKct6YiP|ClhXm8)M17R(eH!FeiKb{_0 zINL2?Wl_?|ua@A7MU6fx4&KA+<*Uc)K|gw6a+Q1WLR51{jbg^@JGSNdfjtK>tsFY! z@XbpiT-TI+6e^$FHi@$#0s-UDWvDA z2gBO+MpNi&F5!!6dp0!OoRi(6I*-{^qo>i?>&JnE67A@hkz{nW*>!lQ2QvP;+CCH_ zdZt&qRqs3>#5EMHC9U63trWqs7h-z+SrllNKTJk@Z|#tp1D0=d1`*pk8mZ_**O?O0 z>6lW){7v+6WjffnSJ1VLu;v+dWkGz<;oc7{6C=Tv=hYg3pu&cCWA{2 z(ei8tGA4{O`a*Xh{ov*tpGkgIaz}eU#dWm^_V+`Xov+NJ2>p`3@>CGu(WW&RB;g5buQxU!_rE)#uW-O| zD%iAc5+9u&Ij36-f_6{n01-+d>!T7lru3*lv8Opo(6hV5^q|?fDT%=8xxn~O{7H9p z$qw)6=s+`u9%$j#*6`ezCIDu4Ha-{nCCV)>heXcWoJ`NSD9%W{6kP#)-)#eL*~L0z zT9kAb2PZqp0Bq&|9+G-W- zs_zBHdwL%!5gu+)q6W(JRud}LzBUhPs}<=D0*!`rd+zvkW-IjSG;%KNe{tmb^LTZ( zCQ~8yIF)F*9wjz}u_{sK}E6%)9c1%`yCN{L)ZC>T?qpHZ;jZr9yi#Hx-VyBvXTp8YO z=%51ItiFp(mqCyhrMjYnI57$G`pbzj6FZ_ZN9(3%^|UfDFK9A&rIfqO{`R+Zk&Pu^ z(TB>TO$Bii$GUz8n-Cv0C1#h`4Le={tS(_WQlKNp#M@6>TDQN@KEjaijt}dT*gFid zZB7j;feCUNFBCs9a=K&T{~8=7k2B$ouvN9i1wOSU88qocL0b>4DT$#kvYPaexcZ~5 zY=}2L+01yWvYS(Qb5P04Dekd|SZ6n*uhrq)=&d7rXU(n@jnam#df^ahFKcv_jYbpw z42L4Kebt~-s>_Pc89~foZw7WBAI0yFtK&-bM%(NSYI*gD>EYEsujvzm7)H#u-;O6N z^_}sFsBDj^+mhvav=MV1xvH04N`{FXgpPLLndQ1o^7MFxlCL>|WV6X_7`;o#v_>qW za^u#^Z7!$3w73o^3duj_Wz-xyiY^}~79UJaJ~cp~14|!V+c+&no#tA}py;$hEwi4k z?&#{feNPz%=b7IsN1jj&Z^D-E9ec%!FjnhEXm@_T`kI=4+QX`3w2Z}7FZ!+Wy#C93 zlHH(3<#*2Fte)NZH*v$C*Y4P`FtLi1NCm6pFghpUle%ZF?*HR&sv9UtnDw1|7Xf8} zhq~DM&Ogy5>PtJ8_B&Kj0KC z)-Dunopzi$j-YuE^6dpY`-`UxAi4mxQ}YsE{BZO3RS5tF!vy~>eqm63bToU`Oz8L% zRn#f5KG1h|3Nf%NsosSveRiiZh`ZSgrii;v99`*`KC-VrAz(0}@bUWfA?k6{**$tJ zMDf$%*v6#u%E!*Z5YoH#cos1Srp-otf7RLP5&Y;h|E{UJWJTaO!^nh`O378QYsM^3 zI>0&_7ccSeMQfzjb_@p*R?GKV*KU8TvAWZ?ERl9Ua4~RG2rbu`09E8L zfxRsIY+w>n_I@+*y8Jr)VCGe6Q{2|cepy?!PzJ=b9+#IqWC?O*(%CSHn8Q#{uZzbg z80He}19~3;QZjJ{2I({Zd12mV+>uvL)+b#oq8SM1&L_-nRUYBwUeXQ2npF#NhA$E1 zVmT)12V^DHr-vHEzEm~VM;IXbA0p2o# zI*jogGNtF=9_PiTslm2>$gQ(LP)q2QZaXEM5ciFo(M3SyUd<`=^~NqJi)AJZQ7 zGnPlI7{R@M+?zg$r1bPB9*&0D6sINHxv8qL&a1P1B+18l=5|$&K;@I1WD;@GZdKPW z^`1WpaaTv*&vG&vY%e7m@Epxxh(8q>Jjpj2?pL3JAbnT(mdqmI|msl(B2!6qpYUiq-BJtM|k33 zHdF|yL8*A%8FxPli6eU`eiJ^OI17NB&Ql*dtB*uvYsVVWSR!B&AqYr!`jIAy`2ajsZ>Kz*5Z4kfjK@O z$Bp-5Wv#t>x+0y&G6QOixo1*RMJ6~BnRXTaxG_;9qBWaZbdko^L9H@UCE$CBh!9N!YfOUs)z3$n zasfkqi9!~%4V#M8T$=U=X+#7yz=q^ao}?iCw(0_)l>$l+qQpKLO8H2iY@Ht2s9La} z2%DwMlW%1SSgfoJW@8v`Y=?0uiqt&oZGPZQNZ}y4YA^9(w~<-y9o>DW;QkBRTd;*n zEKftT6DNIrN6^JU^V0^zX>9nP-T3I(8q1DRIAj=v9ti53r{8xk@u*|kRCp9kVF-KA zcWl)^fntnDX1~m)ZSwYDNi%y{CAQ*0D$}e>ELor(4KMIImD+E}2&T`n%(Jj2zp6F~ zvs3i8$|cD1>%Kd!fGl~JdAKB^+;b6`8+e{#@4dmK!K9^k>7a#78%MT*y$M@0*pRZAT>IWz^R) zU#}uYuSIkRarFFi82G3D6{LS014{NVkTBPBMpe(E$;t1W_{ZYn&k6?F|MoxuLW0S8 zI@ws>()E`%&p!{8>^-%AX!!}M-=)lvG?+JDZH}r^v`7;f_Dkg{7gdD$JTzzekY2&= zZw@2W8TXRQ*)1;G_=)e-DkyV)+&&zzn8zvf6}BfrL$BuDu6AgI7vZsM2ka42QF$84 z%TWj*wtQzLa?dL(dG>6aR*e}51G+0)boeX8HZZu!_u7!sFFkgvr)&`F@kYrgxuNu5 z6`yTdG6PstBY@+^sFOZp$ymSM!;K*&ss1y3^G?u8oyb^5rV`U{!J{dxv=ZWQv<%(8 z81qUqvoaj`Z-(w`<^8JjebQu{PtYJ7k}x(XXFxjH?KG3ewykuIR};S!xSwo`Au=Ie zXGfdo6YL+@b&K^;os`qv-Xy8w8#{dlK=`Aj1*9fSsy0|l-Mckjyf*y zFD_LXmzguL*=Z$jUYTFR`J5f?P2;@b(|_E(h#tggo#^QBzFTf}L_2ipA{BHf= zT2O)@QlCSKFa}LLE4kRX91R9cnX6w3X!iAW*&6%<(9Vu&}Ud0iajfPQu*SIR^|J zS}{0xY*x=#^C)t6&`pW<3B+CfU5eoZ5;=#BCyxL4+8wS;g!=KHAZ%`47G0d1E!<5W ztMD~ba_gn-yZblc;U8af7v+B>fxy>ny|N~_Hg^DBCOP)dv|cTDyiyYT&Gw>EsgIT? zcd7#0)CxU!&4?v~U^tXqmhD14DCqK8A`Aw5R}s1JzSHb0q8`GA_BlJT3HR=8f}Xk| zj+nhd71~AJMkY+XmgyNBXFRI5_s7JdpbrTrd-@(q{*?pp&-`0*QTGi*+fV z44dH)YkuK%I%&_RCo_Ru#F_VZ>sJXFua>t}%SoLA=QuM1oy^|aXvmBw9x3{ehyh;U zFjZUQPr83y1mw>miPIpUi`y4phFlkWFFThw?GDi&-+_||AlW8blvp=|$0B4--5DfZ z?Twd$mi!r4W{z->ao3Qyvp2o<<@wCPRFiQD`YW(keWS{NF!a2BH7^+* z2HpbN{pb`mot3m+W}9xq5Q%_4I4Ki>`@gYl!!)4!m&3V_k77hFQ+Ns_GIik7~ry3kbbf^wd47Bcv#X$3#-zd48_Nc?-yzNozoI zmm%pjc2C-~gWt}iX(YJBf?k!)rh#lKPY7z2Pj}=ch*M66TONCZN~?ewCHkPsqLMZ-!Pu`JMy@A**{3;Z6lOSeX*4Lq3rJWg?p?n8ZJiYw$anagGFkw(_C4Qv z+GQMEuY~nYh+K3OuZF|&NF!GR;auy`*Y1|KLX$&I0a7u+UWH>ePHeE!aZZ6L|M1Ea z(2KX>qkIX8LZ+@QJ>(`EO)2V2a;?tA@0p-s!8Zb`=u@k!ZB$>YNi9S_wA5xqU$>9e zKsOG;!yYPk=-21dUdyx)Q-s`G*12!HMrSBReu+Ej9Ef$k5~q1)Uo-d=zUqUpr5o~D z6%N*^KcD8{e01l+g?}Cu$17R0rN<3#O1t{9eaVGC{+)7>6+1`39nvl&NdG++eUcsL zd;te4CCts+PFXUPv9wvS*HKNP8_8J*EL!hqg08wGM+Oc__o4zbj6%p1+>x_N%kifnr#kcDN(f=mgms3o^^)vqKe|KkA3N~AQcButQFx^{3yvU<<9r|aDdu3;;4XhtNCA(2dZC?OW z(^D<(_80N)>Z8*3!h)z*{V@)9*hsqU91QN|{?OjIWr$N~6nhrM07)ruK;lzob0-#g7TZ9iHo%;BvY z!P(-DGP$r%@pi71s&D#7==Qv0ZQI69V-jytH63j=j6`7LYdg|hXqkO_9%;C_`@aNM zu`9kX(csk&5l!PbupKwG*;)}trK;hYPbKyxce{i`$}NJzd>5e`<((@)&P$$74}uDZ z(F<`pyu#g&)Rq;()0Z`?Qgdn+H~kiL38NQ}Qp6`bCFTNce|ATITT$=Gy04{PgQqDy zlpu!&0vrBXV*aP@5`EO$OctS=WfUUm(fkby6!mMR4N>JuVi1;+ik`o{KDCW8ewbMn%v5o?j62SYs#BiLUGHE1WRXzNE|)sF@_gp$#^`%*_VUKWr~22o zIRGU(|GXU0WKPR!(fAiH7io(*sQAwZ`7~48@(aZxt(`W zO?vM>+VkvWn(Ewp0wiQ(V>xoa(&YDMmXn7gO%HCCKM{9fcImqBJlGXk)svSjx)gd> z?4U1+%K7zns?h2;Z_w}Wbc_!zBLCNcf1Uh@$$8+J^ZuwV+}jc>40$HJrro2$1~HVi?>^95<44lRo~?K)~$VDU`hVvLK!bd4cn% zWYt87i8QxQEnmFH`hP6ISL97+@_Erk)lm`pkf~_uBUNy)x~?;MV4rx>*G+|m5W>aZ zC3f$-rKy7O>jI&3<)aN)^hx(t(3`OoW3{wIRVSEzbJesfhQsdgjjpkR-!)1HQ+y+O zo}Mpu<6=E$_eril<9@M|eG9|-UY^zP`% zW8XkW{ zxHNIBMB4DRz^$F-huAr2sB8`hHnZP<3(6wyR{*srJ1Q9Rw;H-{)L1KzEm$EcR1v5x zy*F>`M6?+Wa~x1F-L8x}G~4~oP3I!t%8}+^C3;uI!SN5^s>onI4*{DwKRB7ClK}&=IzN$LZ(2e=Cc6G2fK?0Ys2D&J~x^vhf!STSf!mI#O~& zUFc)|TY_~yMM#ah08`M`%nX7W!sNO3No;x7)Psgp70jm;m42C^|Fkk1KCrD?SYJu5 z{+i2WfW_?@_Ll%7fS7hqIF`voy$_LN1{a8#l!KBe8Li9V;vHdsE?#4j9rq?S$OG;M zuQGY3kaQ3KX2;)O_vTnnzZt1r+pMJY!hNCsCrNGsrl;6kZP~3uW*2^j^onw6jsepO zIy_j0Q^cM7a^r_y3t@p162_^SG7%?tN0RESunu(kXTiDaen6E+TA14@0gf0HLP-sW z<4E16o)wkBO`;P9m;E}`=a=bgazbCRwOx{AGt`wDP7G`kJ=zZSQJ+~YCC=LMC*kKq zHvbs8o*oDTaJxf#JQPVPDd}?*V=?BmGEu%qhnh6nlpjlbq{wG7m1lKp@kR!7UP=hS zoI`{vnq>AL)dwMs|9GdttzufW+|6AvN~;V}gq#`W&0TklM7q$UpvU1{TZFZuWjiO4 zLSp@9ig-ASsSOyvyg1rm&ALM{ns$3vF~3n>)jqU@zHzUE-Ma8Qs{|tf47xO;F9T4yiX>I$@2pTty&yUW|P}&s)_LW3x-ng;EZmzJDDXa}*<#JUf~GE9DZ>{>y z=xo&fZTg%Q0$^-5u0-o)r&Zfwd%%mKn~fmd2~`$FOQ=K=Av6UdPs{9$TQp9R)K*9{Cfvg#Pxw>!l3DKZ0Pg|PRr`@GC%`+Aas+A$X&pkT%N zoy)-vOrU6rH1hNJ)Fm-pW+Q`>Y@c`jz3@lW?!BLAK|q>je?HkLh!;)!D<-qjb6%pv zW(So2JyHOrJM5ARy^A$f_VtrLVViIIo>Xm5Q5)rlN1TQ z%4V?MK}n{~Ctl5|DC(>03^a`=batY^>ix5|~)-H=)0v}`_`cBa{Yd4%K9#?GqNFu(w6+Br0P*bm+3b%#6~Z|!>7 z0=N2>39Ju`(wQn%2$2_F+wEPw&;E-Mu-8qy&3bL>>;3)_NTDI^x!rP2*nZ*(bhn$! z5M2wfyDO5j2jO-j0!N6`5U72X_R;>G8x9J>MPJVb>xsYmIXWcSN@M9?)2yw*5qRjp zrntylJ3w|e5zxIzL_VZB5$e$7f=TIJiXV?Mh9l2r)<<5gwtpqXfChp39l!%zhp`f- z?3KZN=;G06hjdk@`e_etlpEL%p42`VkW1EtiaE z;Tr#Z)wvtEMMT_YH>GujZlk@A2z6i$J^L*sELZHTMJXbUQ!8zQa9$(~(X%Vnlw2k> z{?f|ZeXP)0R33T%W!0(kX3SLBn$BZaLc?ncs1aDBSdgR%l~nz4>;OBQ6TnLGh=Nr* zg|A_wA`b`3W-;tt^kl2%u&T+?@Xy-6f20{Ca5Z9BNbfhwgU2&=gT5veYF_3v&S380 zGN5>0P-uq`@$j>GRoiSYj+9=vpGh4XoZu|)!8rOk=6(n>;A|;MpoUf>zPS9-is+s^ zk}T3zdSEhd^E@X8ExgVAzvg99d5F5iJJDWDarVySxS_0QKlnt_=P6)`vYjR0!0d6A zmdXJHnK73R-J^ffAlbAFXEUh+7K)OuR8LE|?=rfskRtr>Mjx;JElc{@^Us7aT`2*^ z|0u?N9TWjcOG)uKMr7b^^BXeI%S(GHb7R4sFCot1^J%x2)(L7~7N!DdPGWCH9O!cw z;Ad*V7`XX<%-mC&c;&v%1zVQRVgM?kkXq-7UySGo%SHhW#J$ucd~Upns;;S*i{t2~ zFmULhgvlxb3FEEUo6_KoTvGvwhf6HLwuU+`q2Mq1^3Fp`@x%OK+DgD8$MZHE*u&jv zXMW;%@*Ep^l?On}c^Y=J&LG?+ww3@&p170cI!R}z{arN6raxN2>u`V(htOhl zsJ&luGG+)?z(|NGyyHK>(c|U4NP`L?x^PS8Au*zb<93o=X{H)qU5+sY%RdQP9kxa+ zcOXyf_;qQiHEk3EiDz~nQmNWTiedJri*)&+02_erkT-u9=Bl2&8R}lFXMoKyzJ^?Y z5>JTVXCswr7{_(PC4&1|q$b2xzcql3M;7ABVXZr_X$l^@!tE4)VZ|#cw0P{t>pM}U zPE_TFPBAe*>qoW~J1p#?b)y^WnMWNX^p(aOfHJ2wMZC&QPOhRE@u8Es3YsomB2qB& z;K8IVK3>P(!~lM<6O*VHG-onfi4@&yCFLKOf#FgucUnoS4%dp{-#f)L(=|=gpR{;w zq3z0YFV|p7ziaMosFanKlOR3zw6lma+01|^z~vR&i8_ms=h5a~cKYeardwZvfG~7# z#T_-5UzpJC-ZU|hYYTq2p#wQ?0*P7OS=`y{w;s~uy9@H;-0%BtdMc>lvEA|{y%OHN~@BTOuw{<&_h;xNhtCI-wtAF>LGT^s9$M4JiW3mx#!bVVq%~~vbK_iMfMSi zSsEw8IH6tTe=+IY-coO=?xe^J)v-rw7hzrKEZs?%fNdB$b|?nNMdhBS4E}ql>0q4X zjq|iFO7<!kk7Y9d#x$^Mp}Wl*1IZ(I}<{{Kx5h+m2g zm2l)_9@1C+Wze3(ZdHq`_kiZj8QXCGZ(li_UP3ycM1FKUbzjTzN_w zb~-e&5Vy;7xoXCn7Jq#{Gy#w}MA;C@Y?;L%&j_>E{-`r*u3lg$sh$n7J+E(N=#qr?l%|A(%y1fY7D8Dv+xCs#}-&?a?Wx-6%MRBNE|8D(SeP+>? z#e&BUcS4D!H;#1;T}CTT-g-)YsPy` zxH=S&`hu%~3T;@%)H|`CSuEVaXC*W42|Fa3U(A;noYn#O{2Xulf_mHI>C1Ap(7q0u>O&HYR#v<=brZ`>05x) z7j4!?j$zdB(0YoE7PU$N%Cu=9g*Z@)8(=IdMT_d#@5M?u#coW>3%jhnK9?hHeU?jk zYbHJ`hBK*LWA%LEwU^Bb_Gwt=>Qgg^jG2V_PM7V~6VW3i_h;zk6&X{{oPZ4hjk-Qk zepQ9|vE`RjxLK*qRb72ZG&sGZyNkL!OoLKPI_u6m1osqkX+ zHQ!RL^Qb4}|0Yx-i$3^c0MQGWKKS^~$naAwetlWPGbxE0dtag>W72`rz zJa?voee|ib+Dwr8%`}_+H4#9vQTo4Tp=2EMBDgf=4VD|6ugq0?RZ+-(Au4`a|1?*$ z&E`K1?!7)<_&{N@1Db6+UEGP2oLNGc-iwxD7S_$NHyGiQ5{T#Js){c+Xt7r)ck9jI zaXa!SyuA2;>cn>F!C?mw7LRpJbI*Y@>B%zIlkrztqo?4qyQZFzI)>otya<={dPL0i z+3uZZfGCENI`eF3p1AV``3!B;&-fb)P;t|{+ilJ#T0RUN8^9zPsgspfQcUocM~Vco z@uOiI{j}WZMC`H;{?G%i%fALE&TOT4`%Jrt`1+KX1d0bYS3Wp(b+Gjeff4hxJm;w$ zX0!dtvq>pr8f}`MAi9Lv1kpAQ`zJofekR5c(BV;%-EGos_zZNtW`|;J4B+Wh&eZ|2 z8b~T2oV6%2bHvm7a9yeF&f7?Wgtw{5?+tyvy0?Z$4X^v0aPyi#Kn6b18rE$9Mz9~x zlNnJ`adW?7`3jR#vwSU>crl2^$HbOoX(!s%dB$+3CPBm_dR)Ai$lE;5P}+r8v_6B6 z>ST`t3MX;gFoM=+k-Z8Kya(2d?4*P)LdZBcwFu(*4t<`-9~q!qb!LJ54I6?C{mV4y zpR)b5H98MMOD3)W6oW(N`E9hZ{>k_9>dFZr+ZW@YRz$#QiBMefgSH3+}8aGk98{!*44?%{S1u)y<0f@D7dUc})HlSRDmbV0pZghS7@PUur4yS(inS~dRw4!y)(Itytp*g{1)3$EgjLC zJ!J%coN6LunUdO4;5U0~qv=sXfb@sCpGhj${$Z5fE}biirS!As{olyh?&J`cj>+<8 zUCdUSlk@@p0_RhMWL5vCCtkahW)^VFLhu%9`;jgA?&|XOztKtY=b<06A>j01xp3?! zfKCda(OOcM_@ee8fyZlRchnr4GU-2MSaw14u*O_X%hLh7zW?a6fEKV4tz}j09VuM- z!rDhHN_8d>NmPJj^HcGhrRVJfW+WuIDlpwAXOn@7QKM&^VtnUoWRwCi%|OaWTcPEDH+BVjE;m zsQMrM&U?Nx-gCgQD$+I82&r;EJnLzp0YK3J7O1sdndZsCeiL!%WBC1RXV)su4M5s_ zRRW2FS~11+l4TmAnTV1Gz7rso%#n0xie&*xyA8UUSw}Ur8_?;)O}ph^EGiyyM(lw# zSe=U>EF)okfWdYJpw+LH0M%nva8)*p*k|U9q$zjre<%;P2^HI5_{asY7+%|0nPH;O z;j4P{p&U{8-eH&0N?e&&*eMu3ZO|7Lbpa&g8&ylc5ZtjkZ1bhEIR|HOk| zkWvY0ik%oKL{0&k?YcLf@o<@lz)2H-UZxyT*P{UancGOBd-7IxbH?WspI!~xIy*s} zDM6a{X`4xRriTsQCo6s~W8ZfQ62skHMwUOp$F3T*-nlo$_$6Y5^Yj%X&jYeaMbCrYk%xAj z&L27km4?4xW1Z@Mz){G^>m^D=xMzp0pSZVofYCO?UmhV57CLiJ*H)px9QJh&G8B*1c|h;~ zImrA^@%!^8`>hkj<=IZpowMa^{$mFdv&`q6|39V({7Y{Fa2Lc0ycMSW%SG|;XY@wN zjum7BgZSvHXuK2QpYNhHu7v1JtlX$6zAnk2rs=+MW4~%(7&b#a?UBdk8XD=24Bm5^ zf0kB6Qz(dkd9lL5l=04HF)e0A zadJ9owTkjNb>gLkh%zSa0b|RcK7ZP(yisW*cEH)mW<7nvd<^Jd6o!z^5OB441g#`~ z?o#=7TQt{jd`GB_80s>`(@&*I}#WxQ8zstiZ@6Cl%~y9;7QE^GUm+g0VO_&>65$*ym_~P{dh|Q zeUNT|WlRQO(!=(tx~g~kqpPpd4__nSy+5C!rxZ@^&@5qlXIf~~r$H#0^<*Bn%m7QW z6f?-~R@K{!yr*-1NsoTwmXX;9fI+XU`S5WG@Jp%M0bqU#$qm^)w-DYMYh!Mty3f&K zv~!Hb%&*H&0q5LmvJ&&?SV8_`eLn`*6c}kamU6lpB$VJXY&EE=8Lx%;2}G!!>;54w%srfo%ArV zJQLQ#Tl;I1(V}Q;cJ$KE%Rxk6(e;$h&D5Pe`i#ZUJoG_tlHCvx>ts>VM5dF2>hVCZ z02JIPux|_)mp)b^t+8^Hxz%H)h*&*51HC73EiYE0>L0M4v`|_{Yk6kbfOGrn`t?jx z1pG7+bC=FBJeuqg=Ob~T=u)wl%}u-_0$J`$6n0)2s6X6cG50IVj*u+ve$kzF(Rt$! z4tiE7BJGuQhT=Dc3dFf@+PUS&8(ZuZXawN<$o zrnkDn>;GF7J6ql27CqQjkez&mVfcLXz{OkE_vh$cyKwJt@Vy+BXj???f6xR!d!7T; z61Um|f&W5od}Grtkbe0yF9Xae9^7@4{CG;4@<+zmmffLD_s{#U_To-t{5Qv9TAEvC zV3IB)5j2;Sh`H(#Ro8jnkF_{`XG0HPdLi)TMb-|jueC~Y9DB%YEicuje07rIxspYV zG5L{!0a+QK`+-*2j^i%P03FCA8UFa>P>X9mL(L@qT-B@gIp$*fBAiR7(Iv|EyrwH- z8y&Am@G|Y7Yapw@`IX}g^*QA`BqrD1JW#N()9!xP-4&HXf`d(?oGkmD(wz8Fbn@c| zo-?R)tTa{vd)yB_!ynDiHM^HD8NeWD;l?CTgPLTLTkBL2!7%Oa+W7Ey5^|>ByE_YX zA`VPCn-jnBP~x9X9|^y_!tV{IPOC$iG1RrbDk0bPo|%UF00V=_ggm#V>cY;hUb!+? z-pPU&=I;f6>nLB&coujX8C+8P6s^JKuIklFEtz)1Hh&M{O!M|&y+YrDFGO5KF1M~hk8DO{ zwcR-9XzsGkTN*K;cRvfoot}Io*#9zZwx@6KTIjf5q;@q}vQ5=H1yA#}kTkzoS65J(taCe8`F2P-bySoQ>cXzm7_Fj9v zd(S@W{i+tK2rZ#%%xPZ?EDb=pPFLXW$8Flnp8dGAHWl{gK5>F73GvGy!Yvj@Z+=JkTU$$XlLeKy{ zz^QY2r6J)%@JKh^#eLAkxxnT?Q>|IwvE>WvTjw`C$eiG?1oIQzSmTz50Ws)tpU|Is zv>jPmrY)%o`KOZKbv0O7XPd&N(K5XaO?7v^OrSdJXK8^39a zK>WEC<)sn(V5`8&Kxi*IsrB#9q|e6^V1AQOd1ZO&2mQbs{`Yg8|Mh@~t}h=99?JWlb-2{8V z#2RpTRQ2>!Qz4syGFE1Nv{0Vm!!AO|8WR#F|X`*(buu0Dp9;fb~nMrX^be zV_y7}lFacygi!TX9|{3pc!g12y#$JIP_|Gk=^B)E?GuI}*I+MI+<|zpVn5?RG?Do6 ze7XJJC{ByUPfy(rFVAfva9lmhpZVa(Mcc_-tQ7)J)Uc`*>1Eufuqb7D`dxm5c*s7j z(R@KQfpBEvk5`%Um>?$EQGn03J((lSz!V)_Y}Dmfq}TE(4C7;f$Kf;yLCP6Tv7WqI zf0RDW>eF4{Krqs?dLFI2Plv##XdM56nV&AN#G_NPGSiCgd$wy4C}d);_5NO#8XaEK zOA5z1uTmUTPRGhA!RSZ(S>9r9xK#0uMyuSsnK3aBj@PDHIoJ7;x=lkOgb9p#xn1wU ztft85s9O?hERqvx922aSEpmyUh3yr6;PUZj+;2ah9?j_-E9qL2M(ez1U|U2J4eYX- zGC8kuw>EmU<+VSgnp^#pHPi0kSDeDdybkh{aNml+VPyiWUh4y|*QXqLd5ZB{?Ozr9v3SywqPG1Pj@eM#(IY4?mLOiK^#c)1Ec*F_&DyCH(W(GuYM1s#3RbPMI4G_^xHvlymn}Fr&*cW%MQCG3!9_7xr=5ePH zLBs-4o5U#R{0bpFEbd;GKKnMjy$50;{(KihBJ5!?Q)4`Gf$Q5uo%k^j7ULs!HW535 z**v@s6r%LBFh5aWo0daX0Ure?Cud(LPFim61|f|ccC=QDO|7Znwm8vazVFPa8yHyC z&Z-MSG9Zx?qg;6|c46b%JK5nrts-9K{D{x_nA;`0*0^)6S|?6uTkD{S(RV~xo_Sc} z%=VMb1%Kw2^nL`kxu{|4q@DihRK$g*(CX!Iu7cpRr*^dLKIv|ri~VYn_aRQ3ZFnX4 zGiv;G4KOfz@_6w41GvKBS^Iwal9t?cP-(URK z_y6C2k|2Q`6~sr^^>U_~!^w^sijRlCH+n&(l%U#UR%l|fP4SFeWcY!05j0M{55@Z1 zf*6l4ka+@jS_pLRw*}tZ6(R^$sEI&5f^rrUbLj=R&c34$LhJ;*MJ_Jvu7~oEqxxwv z3-O%KfpwG<=I5>crl4%N+_T>#zSr4A^0Ql}rs0=&!hEL9NwRtvPx#Q{eEPXmwcP4w zfKJf2-QlkroxZMz1xeZ(L%h_TRJm1FOHS`xMr*uf>kVTJ9$~zd7t>O%rC%^+>9jDT zk(GPbiP+OP#7~CeX;3?a5 zUJV8g1wYwPW#2pb>O4-t6xxxI%)i@)XwqW~r3Q3G7u+_-^oSkAjH zmfz@QaM4L6LMa2Cj)SJ<1R`x^v-$(s%F6%Xh=^m5)~NvPBoU{x0fRoHtQwy%galCDZKd_itqCk z);D|>BbYabGXu0@o4jtwe3qEJK03{{YEP%mQTQ-ONYJ2gC#a`LUW+y6T?!V9Ix~eN zE_WxA`{{AMAaXAGq8(mPvy>k{eyI#pKffaU*kwp*wGwhD^~0>i%7=USN1t4a^CmkQ zOh|^R$%JGNO1UOUk%8zd=iwRcbh?237R1pEuM5bbE0?!d`qQmi@B3YvvBRGhcIh}@ zQSJ@U)1kMw2Qo|5fS@1);P@Fz?4%YyE|5xMIG8O)n=X_&Mxa&bvTF7mTLl9fi@;@v z1#(RHO)@wD#2Sc=y`a@uN|x&j$6BbE5M?#ZWm(OmF1AM7|M6?0<-E3EeXjBxW_oi* z9RY9egS`F3C1Zn?kiTpO9eeh(W+X2A#frlO#ph5|RMZP#E^o~s;8oeJa{s|>xMXsf zyu|Uj#;8vrO;*t9G5#r?=DcSjECCYMA&8iy<3xIEc#bZNRDyIK0`}~^J%VUH>*fK&_#I)WQLU&K4OoppKl>Y+aaWz9!ooD$XSBF zLK50RV$a&YL&<}16_yDYux-_DGYAH!!`Pu{HpXm=p;m3zqV3mL<3l(3M?mam_v%OS zAYT$GCkPR|XLoHB{tKXV!meOJj;<9ScIcG86Qr+KpN}Bh7yN%<Mei3wd{;A8mhd%G0e0+9JN#goWX6CbfrXi zq5d`+OJ#?FgOi$yG@dWh2nuCmJ{Gs%V7O~0hS+gFQ z0bJdK$zmEGhxt3(p6$xPTqZplB{7p4!@~K|QvCgAmY=Zd!IL7H(Ud+J&ZkQxl$2tn z!oiyLOs!`evk_75-8}f@FLyk z9osFXYL{K}<(zIu3D)UiiL{2=`{S$_*5W{t!K7370Zw}NUC%!xQnQhvLbolf9|7;FwLz_1(|83m zB0v2!iO3s&u0*9L#i-ly%4i}gW4RHJ3AFYOYdi@FfO=G`%w^*e_q8aN^ZIYYi9Za# z;06T+5SxaginRlQ1`G)H6b{==bML~N1K+30wnClbBu2e<2l_%8)@%wm)ynxOW70Yi zuEyS9@LF{+;h(Ih@18p=bX=eM5ezdu?YE&c9gmFM=a=Zi!YbRt8dKR>pECAVn)01K z;$6=1M8h1lduQ;vp#pTG#!AXrKmh(^5|lk0gLV8>D~Zu?{qb7;c1m2m(I%rI#SzMF zHE^zWVV#IASgX#=S2!4P(P`nhDJv@r(?_AkNRp0#0}+Q&_XY}nNS?bo+UzXXcpozMMwcIn8b3LF%^GHJPcX||*?N3*}2 zp4jeQFThnr#SK?8`R)u0T9M!hxRv3EQGMqb^rBE9vTG!~-*3>?v}z!>*WBgYe`eeB zx?W_vL9h;n6WD!@sO0P*r;i*?U+LTRFgYhcyc3a%$*Mfpa6Xr9c45(5o9UGpv_pUN za0H`5VRn3uj<=6D@du`i4o`q;r48Z-$7aC-kF8hD`!iWn#?@uii4CLPebPTn=7PF2 zJIbBs2gE)xQ!wIij^zhmM%6pGFREl8mV`n(3HsmXd<5e#uzf(jCD6pkJk`QY+gHX< zCM<2i&F<(nUJI2)3lSh~71^h0{2Sz)1%YwvL$|3Zg@uGjlubX_E|sdTMpD{I{}+V$ z8{V9Sg25mCqGEBN`2U+fNV#Cn7YIj6UqCpOt(A$P`4({eip~Icbwo;Y#Cvw9S9+H` zIx%!q?K+7`GA#4K?Dv-dg@fT`A#M?zmQN!llB~#qRu_5v@~C?JjVx1K%sa)W=$9wN zWXj|4(!}lKnKWqaD-8&@+>R3SECq~xwrQLwTzaluxYIc^n)9N0!ph3%Kna}TB0~qA zhW!4VTI}(nsnFggz8qg5e$%(jq3MEZ$3sQgsU#=&9 z*O{Z`=HQG6ra@42J+a-^`3t;p=t$ycL2t`SI0C;Im$_-i8;07(emc46FD+eKIn)n=_1 z4!2D-+w=Kub5SZu`?1fB>`oHrB$o?L91+{cjaJpQ{_r-AqS{BFPVtdmt07zIr>w5q zIs($bh>(!kVks6PKL3e2V{A3>S;gjct-{LppGPXcy**xd##dsv0LsS5goEIVl#5Yi z&3$MB;V?emygW*7+txbo%cTaOK(sPR;>`RUm=^LEoWxuu)Q4O-SjvTbayi^u#Jo=~ z48aV?BI_}X&QUyHB7i4`WX^)qLlEx?VUGSNp;5Gje3&0IklX$IERdQ|DWVc=g2f*?^COz*t%@G%|e`@|U5cD=Il4bJYk#LMc!e#aS$$X>Z&J$VPdJ}iz@w|^)ciMSQjCT=t<1c-6()8BGhmW5)niGTAE^X=RDs-=U zHM$x7de^t!+x%JB6Jh8Pm>wG>s(=9F;%)bVTqp(}%<&-(CJ06IUKE%+H6ZX7(Gs$o z1Yn~=yNbzWWYR@=imr;rtgs`d=P41VN#kIFUnZ|Ci_fMohfFVT+g$9?!G9o{Giy&!Aj| zD+HlzCIv*|-p`H~+~OgRvisFjip$!;%4NYbLPy6p3j1T0KTiap%!gh|M-_DQOOO6o zN-4v=r;e}iqD>CAYLD;u&S0RRu?WI|yH%?DnT#tF7KQ{oGquQ5SYOXR%o#Mx=$%E$3z{G81~3~G)Z!U!!=ze1LVj0vZDwthn3_=Zqek( zapfYx&MwI$22fcPEENHfU^QpMygCSTIgd>1KP-`;w<%dZ-I`=PgG|jtXQ%mjn zzLS+E)&p;iT9aE3x(S@`2B62p{@;3@Ze>UaAi#hva`+PdNAl^PQv1`oDd!LzB=I#2 zDz1kVbVx{Y-*@xrR z3ftK;nb-yfsM}yV&gJu64rdn+TPQJvlvuR$m_n1TeNd4;)~>YGS}PY!W*(&>pN zqF;*zgfLU*N}*pK$j0@W0}KUT15Tk4iRX2&##R~IAHdQ$Y(K$IHltsRvOc)3awi>T zjH8EpvM?|p?hYriTBF_Wwvhtz6x<@7Zkb$9lVMn;W)^;;6xr_y<{y0bF! zKBNG;a+>Wn!Im1Vk6|F#%T48G>GjqO5tE(O+bxIoJE%8o-mAX$2v)(=qEDCMbOVb0 zM_hL<$>X4>WBZ0LkElWfxWBsqNo>;b;i9*x$WnMc2Dw!wA11SqRWJ3qKWlr-j{g&8 zW%9{H-TBFK2T@8dfn=S~3HX(!%RpY!t7(OHkxD6MJK!KF>A+xrvf7c@8BT;#Gcddy znGo2HHyl8|UNj9gr4r(XsWF7=%-M*IjV%&>E=e`QhU9yGZ^pyK2o$wZ_pSH}QSt!} zPPjz5SUZ~CCH(-6YBDifAaw^+#Xjl%a6_ndzOChx_a+!*)z@p2eA$d8WvXcBkQOkX z2dzfClM{{fmnX}Y@2}jjfaPd0Uv@|!?kTQ7LW$`fOC}LqG_}dcZofKm8uOK8*v(|6 zMHH_ut9*l$dol{gV?f$zwe`nB6zug(n{dc)=WH|{_24s4DVs)uvYu6}W;9nHZZjga zh1y%Xo9|!FXN?f2U*ACwUem{g_10%Rt-wqH*~b@J9+)#LZ9-(kP(tJ$fnQe;I0sgb zcVEL-TwfR%t-ZCeJ>GF|kmpCUqk`?^o233iMJ{DNI90-)SRC2u@WZW!ZCrnA$A1r} zzXsQe)*lLKd%L&ve@jMx$thd##C;Ft4&V{LMn2o}I^;4N^PiL)yHeNr_%&I7?aA=b2+8$~kPhA$ZFV`%zqw|m#Ny({IxwYNYfoM=1~GDQywpVQ~z!Bwm68UjLpKVPzl%M96oUb@>N zTeVyhN9+*ja-45L)X_7pie!rdnkAu2R4O8rOvlZZHQ8)+M@ecQ$I>b;l@KPBKQbX3 znvPZlq_%cA(&)PFrH~K$#yi33Uu<-#H&~^pEjLJ~hX{f~XTPQP#rsm!K%F_`ZGceJ zTP>mj<*xHbLMCUxtaRmjJ_5L3qm9n;r7Ej(nL3KY-e?yv2Bjm<^SB)auQU13gm`}v zdgEifT@z&;f06aAA4$rC=+amzUVnL1uXidnF6wM`tFCgf-ElplQRC;Kj$0mWGr6}3 zkjbjZ3Kk9D@~<|$96ay9yKErf^g$#cS!z`ZMi6ep(z_{B8wF>LuW}!I+1wAl>grmm zU%oAGoXpZkoW^0cogj>eh`7;M%H(s$eaChKpon^vzHh@zgJirsF)^|II>jV}kVJ(} zOW{}+uqp#^X$(M&xeIV=x^Kbrevj;^H5P8xZh|-Lz>xO&_?y!PKMe5gARd`DSRp8X z1NhNyZ02o&Te?%^3lkxNvqXU;DzV~HYn4`rRbOsT-m|_wV z>eZ?oIKDOptS+-Z$Qa$I;Y`aHGwEEPfKDTQEK;4Z}jdcnA!kP z>AmiCD$m)%{&>!h4@~~4lEl@NGO4Z&>g^pS;}O<+o`HxEO;h>3)ry&(S6$F%V*8Pl z^thbJiRJ_n8#t9#luQXC8%P*T{_lhL;?3?ULOj$#HjHW$?xoI(@V8M6f}c(FS-Cn8 z`6=(j4Jz@FowzTH9h#liylG2?<9^Lek}f+uPr7 zu{_h{U& z=ffKjD6bc9)?X{|9ex?>}ea?a^G15{FH; z$@N|cpWF2113s@SvXRx@m#gn6*Qy?Ky5J?){00!1$;o|EfPZ4_54~ND$Dut}WGREY z44ad~Spc5>3orSX>CkfsINDJ%vmulFomHjH_y;8M+q+Ypj$>l3t8J^`b%s^L ztC*0rMU!wpS(*8yI1ZJbeknx2`-FYn#ZYgvio3bFDG^T@sWVTW$G4ks|N9pwM5AT! z>p)aWf0242eC9|xju2x<#Fc0mF`v-)gI4H^+X@}^m$zrvb%-yY-Ei5=d|*#cx=V#T#1k`mB{1s616|H~_eKLA~`G_6zps;a$Q( z>d`8FVXkLQ`|z%Ek5{{~{3O0$E&WF;T&9K?D4TP(wYP%znFOq+q!{%RCu`;Yjt6H0 zTe6?)!=LlNd!u`+9Z#=@(?nICQ|In~ToJQLJ6p4Xx>h>>#PWO8Ywr6vq|<3?qT0kK z72(xqzd4?OvlQM1Njy;g>5S4Es96&Yf!mZm&$zvXgd%F>vbgCCAGU*xCD`wh)#v^0 z?K33%9)Gm12{nLU(g=vj5);55T*$ciIUqmm)NXfJz@eUy*V&M)M;1t}{wE9I_s!-H ziuy{6V)_?wdcdL;Jl}@cNkvvL{_^HE3tfDmi(SCw6&l~V?LzO zu;o>L3DkvsUI3Yy3nt=D(Is?S;xO}R#N+5E<8t`!g`=g`m`ql~C3hO!AzavpLYN^ZzoBwm4qOyUrMb~?ALSnvIzm%%-P!E%%uph15<@UTNRdjdEM2en*)c% z5OHu|CI|B&vKre(1J9{F6+|nQ%_5-f`HqV)3%D;L*Zux1^U)~=k@;S^cDLH_VXOB8 z8 zgsULt5LYmH{ZI4V5ws~yw(ISYfEc?0csKJ=;9~^<-y%gRBQe#iH9_d^?#7QMI;^tW z{2Z;`?A`eV6i>M_3Wg=DWnkmxblDYXrLaDJl1pJRTPZ>rUMN3ic^Gl`bJqpR$&t(! zYbazhxQ%ho+stP!ELNNQuXQZvvlAtO9k7WgdLZ5qIzZHz&ZvtK>OAEJs&8aOzGuC1 zkZApIv1v9}lE~~mU#V#fd-LFYZ`llW0(`sm{OP@MW?N5?|b!cR*z zkty8`Ca2>`(ANZZ4(la0=~T8M+U@`boykFj$fgW3=r6Oy{s6YE^r%ziuvrPA)2JaY zr^k`a^RXD$Z4DXD$@%5{+H713w?E`>21?EXDykv$anx^ymxMDCQ8xF-2YC2ipvvQ7 zkSv;8F{XZ4viwmEpk{H$bOAKLw{6(Rjbfu9=$N(uGFgr>jOIEizC3WQB37O zw|lrA0>6%dQnfrW7zBj+Oz^W7$=M2>16IQz)udPg?rccVe$NMSv!w(+kz6p@y2w)Y?e!H6E3QP4`V!|aXu$#o>Cw|O&B;^V-ThndHp7! zm`kAQ;<3tkaJC`)6)Cd1Rr_z<>-5)%YU z(4d9!uMq+QeFcsg4p8zv1Gz32>G=V)G%aVGW3jm@s^O4xURCGc@NSOH? zKdvGduyIN6!g~iObADdKalXG1*8fZF2u*-db;%UXa3A^4sR2n}Xq-Bs)eFdmoRKWW zcp+%wSZ}2q3Kwv7QcIGp8QZI@A)_7Y;(ltL>lB8frveMw?RKjJ7GG3XWfE`*qmU4c zd|`G3CBZrL{n>2mM7hB9$)x;4j#@^Kne$o1$MD-B=3@nK<(YH>Vi8Og@BuSyR6pR5 z;Y?+*C~UM?=uh4oi+yU^*Y<2=c`Lq_+DR0|th_%Cw)KAKDy8>zyV74RQIUAxj+Wdu zXW{?w;X~1FKOU#^Rlao=H*ll@mfXRB?5C(OiMSzk=p$<2y$>_0Pu|oT$h9_TS$)^t z)AL7uUv3Y9_ww?BlK0y{5&LEkm$-oIgAyB?TnS1l|0@H?z{xc`N~VT#v-M8_*PQuA z3^<2^P(i#)WN_DWl=bw+G`pQ@I-bpq&^orSG@-D1weD-qjy{X{*E3a3V-E(&7wgg9 zopkwtf=jndb$EL_vV+Errm=k>`zcc!-|4DbqD*60_Cb}HDs_gj?IteiL!0(yw>qXD&wVE?SEsG%@PA2OGfK84O*7(aP9)T`PL|jn8Yj$g^#g) zd;@O>ZMN3hM3Y~jH<5Xu+TNnIRqhYYC(XcA)}#f7eNPMLgdIR1}YZV0Z_m? zA`l!Z`%4CQ!|CI3We1Z%FU&G4v78_XCcRdWHke4BmedvpC|Pa?G*qr`IOfCE2qVEQ zJ6B4HN@-Bj>2gb080)O(o;7gc74WZ_4T^+E5GBCi&Cg6cYicoG=XXryoKm>HJ6#U` z!|^}aRyJ5y-{nQOUM0F+ik7AGU=_nCQc@AX0}xX0bS<(O9C~S99LKDKKyrgS}cfz|dvR@F4n01%E8?GHPLts8u+=mYtQe*!7-d(m^Mj$0(}`>h%5^xfcsF z{h{0`_U(N6V@S-A!?&QJbC|1FxlwN_W|8$-b=kGroi|N$lY4%O{@IGQ=9Z49DB%7K z#QB`V;|w3}(QMOMwk*M_?`ALp#ILlP7tU-EqK=6&SHSAu+K zVfJ^KTEFLiNGI|p|6yWX_#8Iu|HQ;zq%eSyVb1}A;zHVjVdlS8GNE)O)nEb)&(atK zi-p%og)^SlJo3l$t1OkgxR>Z8FCY*4ys6m1ZEgI*5>AP?84xPVo-m|`mxOSs3dPgJ z&qX*VB+1T{3d*OPN0>-8lZ9CHbzWS@ir~R#I1Ojf(WHJS`L+nLmQgFMi(Ay zWqfP`e1YuDc(GFKexlka!b~$`2{`p^b3GO(iC(+M+Mdno!rX5EA~vzv+HdJhUK{|jJMzurd`kS3 zwcLzRAPv7JMJXp1<1;Ymsdmqm;j4mE`O3@;(~v9&A(B%|%34mt-q-i_;whOvp zJBiuz`-x^7?jlTXj-<|VolqT^7x6{^c)1qZ!d266btM_gm2D{?cqTq*TE!d@Kh_sqG^;`my%!@tNk5+Nmb}Ip^`wr*d_Nz5rqFy zG%8f?ah7vT<*c4x9D-v#Cd9U-?C{x9PJ-|F+2=m0A+iP`%J|LvR&Mfo{T`6qDd#AP zRv+^x(|AO{vz0vLX{BvOjf)o0J(SAng@(c`wu~_ zXVXzZTt8JB!4@*{sBE!F+^DDIbZItkXFwuaBo(c@&xv8ssq&hktei{?#~L8@a}RNE zgVl&TOXC+}AZOAfRS!4d@r5n9Ui*MAf0}}=)I>lUP4KH5R`IL7_bsa0y{z$FCWWwq zGICO)iE0(1MU0M~Xxl3OV*a zD8q7;{V$!EmlyV6ZDAKlDI}AnY7^2;XZ%&me=_V4Zu(!DDm^oih8|ahe|!_;MRb1RE@|9XT5e*MNOf#~DfQ zRfUv~)T*ggF_n%V;bHPzJfcrSEPjQ6MR~OYL1)?;(c4uW;hyUqUonL`&VDjaKWuv7 zZd$Ix_TGUB`{{G((7kM%r%-LcFvEN+4x-?AJ(N!b*eqiE;~8y1+x|%Wog;@cRP`BK zO7M|gk0*<@)H=;|r!m7gB>;o~sxHkt-1T_7-tYjh72jN%A$q(3QVzJ6=C=$1&?c?uMjK zb4w1UVF81&pCkj5lAh8ZFtfbjrza&;Ek~?jtg#8Bz;8?`fYl29=Qs-2lWxfKxA5QH z9;K{R&Bt$7(-gBaW+ZAHjtBB}6v9nI?Q^D=1|p*3%i}{XJv|DU#01T+#0-enCKLS< zEW)&h!E>dT5;>oPB%fY3dt3{gt!J9OmRc4t^dQ-bH!zJOQ*q`2nvw}qR|Dqsg1-Cq zxrPzS5`zIeTPutIw}hvB95cu2mbj>;C?uf4!vPV{n>W;uuIev=VBl@UCSG#9zW#N0y*k1DqY%?V*%^R+ z=iABHo}VZjoNHxQwczpV*iclkZa<3y5~sdI4XLI5WxABMpWq|i==Dk5gzeJ(*cpOJ z_h1Y>hw}K~(N24M+`ZQnzd|LvO|AcRXszl>%kH(DS7i(_I^j3U{&$QHjsh- zpP&4Hsk@zRihs5zzL;11V{VyH6LK^g@1y*dmoRy{&!?SFC6DGHV^8{!6ICDYiDh9k z7_O2==JuOkXy(p&z?@*r0xVJ+SZ%}md6?~nssl*YiY29};&7uQtes;Jb=(gdcj|g1 zUyl~@$Burk2j0n!eoqi+dQ|R7)8_D*5O~M8ZGSWglOB;#P;gZcQS1m{NINTLOl2nC z=EAX=LvQiqVbN*cVLATLLu9Ntx`)N6D-w=LkBo(nfrh#5ouOkY`ub4fjF%pAV1|x~@y*$c9 zHikkL)H@ykM@q5M0PSAfP&~@bibKWvSi7wE?MGAtLuR`(?XtD%o_%{D!pJK9pnHlc zFnA5qxs}1g?6@aw%_l~ex(pUUn#O00Ruw9FcOg!cW8Y}=hvu6L001Ba2)nuqg9l{3 zy+QSUz4rzGp?PDqjQREbQBIN2&GVOGA3k50vvBbDzDKh)-W|`a4~8T!jz$|;?&I}k8kYbx*9~$~7y=}Bqn9bn*-hGfVIPW>Z84WfKmfITZZMl7 zRj$xpbiEs4>`1QI4eJa0dbn)M1NKQ9IAQnzaiZ2jV1guV&_QO}o|H<{3jwF#e z48{GfHAR9EbnGk*Qa&g9#+^=A}s=(yp7!b+YU|Sq?S? zb0uZBJ7225U{oMAw+51b}HMSAaeUG2-;2< zH@jt!uhTKEL$5?CD^RDY1yIs_OlsC5cNa&1Lv-l0g+czwrPua5^N-+s?6|`%tg{ZM z18X&!4%+_!pOeo(uc`0j0hO{!_2!+JTWNlGyfE>zLBRxRGNQs!8$Xj#ZTwhrZb7$) zm)oHipQ&kmt=)6kr=L!%L3+92WU(ekCW9NB&%1r$y3py^V6@Jhuwy1uu|^C5h%BEZ zzDEs|d^46lTqn{v@G1a{>s<#{dU`VX>h;Sk=1RUY_;QQ5eave)PhQ2FB`mIjerzHz zl7xQ9z1kw&7~vv6Qdx*X^9cfJf)NZwR3*7rg1qH)o5;N@<}N+QY;Qd;TH*?9E#Cf# z=MLAIxe)NFg|>zL3qz6r5sLBQYx*7HzX{38zmk1ux2+?n=((4_bZGDr@^E4hfNT%g zQg(H6&6@-%XP^PFX4Px)lnpFacp!(@{ykxAAP?4w{y4*g-zT42p3YGZJl|~ITwDl| z|5;0o`?H=Amxr754?+pRvNS;-i9ACD-eq?^_!W8C?CN9z=19-BSd z558AECtAq|<4Au#^HCxExyOt$;Y+d>>--J>E7VP&E7b;hE{oo%rG?(Xxk(}GE!=#K zZjHTMFS6bWW}#e0sC?9B?*VlW7d7Ewfj8Z8O#aGpL2hEIB}44fucR& ztn%Cs60Q;OOw6bB|>6E8rO z3ibQ;#&j2D-m9~%_?7qlLfvj!+T0ZiB+;zXOr!8aUvOBrZgtzFpS3GQKF}Sm;+em` zfYeqtD4AFDe@SHNgy5DJ2Toe%8sT8_!HB~7fB=X9r$fKnSpDu@wJUV6|Xb&hiZqbw6cl_;T}S@n9_u#@Mn$ zt6apfQGJM*1V{|MyblO6k*EkU4=5vxG1yoamac1;^7-K9wgQ!l&4NR`llWv9;9}!? z)=06_s`ZaD=+B%LL`zoUH#-)C>dVh;{Ggx8(WoU?2i5PCpiE2d((`&W{!^4$!PM-Y zh{(ASMEL@oOT-QbSGyzh`|3&GZznsh z0!^i-%hk@OngFg*)T-3`L1zXeCf9&-!_8a|-38Q5KZ?v+Nbu z#@m<`-N}5JtIbJs`61##J5rJR^26mevoLrWii&j# zCyJX?;^(QOxi(@6h%FLiDZtA|-5&tBZNky-DfcoKliojou#Urqyjz$-uO+BJtI^;z zlaG@o+E;mLVgd;j9!Z=f2OLMJiTjdOaGBi|O>THeJ6#*63 zIoG$2IPlk)347+VcApP(_NdQ!2j2R`g+L5GBm`zV2|#Zo31@8S|B_mNA^q8r*cCNZ z7aunJq94fuS%VW3#RKXXGLL*fR%erwV7}5p4h!stQGk!F8vWY{|3wRnTqgWl1k=NP z7B-8Y@K4&T;-5fJes(f`*gw7EEd}s9F~ zL-Di<(kW(9lHG(G$n8KKf#AtT+K>iWq~@(>c#LUc48eMBl4mY1GnNf!mdbI+v5OXkn$Uhm$ckcsKH?}eYqmQ3z`_^3w+*X$ymI$Nw0EooSy z<|9GJ#sTisnl2!at6JWjw?EM$-4lj#8BgRT2-H>D^@9Zo%7?QhG3IRtd??g`11yK` zKsexpkILu8ggOcYwgodp!&|pvL(_a<2qjHFVLZW*mFJE{1N>Sjo-(AvLt}Yw4$%G` z$J5n;`+b5Kw9S?(bKR~DR&EZn-^}9;wy0)oSDT&cHSQ-e!cC=p;DGD@z@cu>qXCC0 zrJ+EjRMl*`f!<=a=&QvjRXj_i|3aqC)$YR&dM&?fNE36_^R=R>s+^IR65ODI%Qls4 zkUIjtFkpZ!gWu@1(q zKjJGl&|%&l1&R;4+BTSu6q+;!u~8nooK&%Gje9&>&(f)m&WYWxO`1*;i)aXbQf1H% zsi|9oxRK%}xkr31z(h7;C?Nai(+HgY6>`_<0vf4`%T3e0$j! zkf0w&b*c!v{IT%`*z~ntZpabz1T47G(a&Zw zO}>C{XW4!AnV|GfH?dEWEpm5GE(GgOs$;-~gaR`5~+*o6~RA$J3YzAvj|BTssp@jm}uq^BarT zD<0opXsJS}X4@J}AUkpTIwiLmew@su(P|*>Mu;+y58l0- zK>8D`W}pSsn-;2zgj;cD;pH$JnMO>br?>tBiKvHBy_G? zZg+8GZng4j>f2J?o50T3-?POMwVFZue{^-=?^U zfVcO{({Uvyc0w!n_J6VfX#RmCKYSxjREuqY${qlzwN*jWf|F7$v4*}g=YCsnLIn+SuIu-+3G>;4I2xh zc*!Ng0w3OB1nD3T+%9QYZwtJoX!%plsx>L87D&5JJ=u`Pl4e;gC>Ur9eIf zGCafs2#1UQ+|W-Liv^d9b5oywyt&-|UFAsg!)DQs_42YI_)9-LPL<7-5&FgTb>>Ko zkh zps-DLdfJ(i+&BbSHPVzyVcqsWFuO_rG{)8)hCwSI<->vN6U4n+2k(=WARtJ#dR1z*^A?8kiLn1(1|1*u`9SF+U;FquG8HJ%EK#ep z>C^l(r7Q3_ z<1^^N(mNO^vC)yvYZIGaB{}F*79==`l+NP&zA?I3yJP@iE%|(0S#dXfJ96E3#3vHy zC{KUI1M#ZEuXlLPVm36?mOh+(LJ-<9Vj9XwESM9`5{C&@=rUkiK(u!ii1noCWO%QZFi;PF|icd=U-rEJ9cuO_gwDad2+ zZYJ4Cw5)3xy;z>#KpLv>#WKv5uqi3xwNvKHh4i6Hm1|98T4xI2GTXffZjc6^E44#a zF!jQJs?%%MfuyJ@#uqEyh`tT~>OXn@vqa!}x%JEWG}YW;&_}A@sg(EIrmjv4X`LcP zl6}GvuQR+YkY)W*sb#vjFN#K$%Yt(#I*m=ay|pC|P?btfLcJulg=p>xBeHyKcaC3% z)2(jFCZJ!4XIOBOX>@pph^FTDxvxDJm#MK!i{g5L7@)=vAbuNI6oKUII$*ASEa&(nUoGEg}jCNUzdcf)EmV zhXA3s5L$o)_&4W#ciem5_uPMsJs609CVTDmtTor1bGfm`u(nOUr&0PMq;Gcuko@=s z1lr3&%*@#Dk5{PO>3<+F^o9(z(SZj{s~H)6&aRrLrD=Ww0`pns=)SM>Y!QblK0ZTN z^K9FI;U1%;vlt)a=%PwZhvKK#fjUtQYTo{ zeGyd`rkA?!a0!*hCVIzI>&IM2BJM|WY)}AcRDulPUvkMeFURw$!~*^owi8S631g3D zUmBT2psD*MAJUmFZ&v0I<(B@r^PbB|)9JBTzmNcXFrAupE1O8~yZv*AW*J^asr#4D zNv{p66-w$qCp({LXlrM?j1-OkrpyewaKoc93SeB$5qDS4?25wQukVu&+uqH_0lRi7 z^DP>pQyjZ{-6!RWUjn;5BXnrPjOxuK&h+%AjR2dQFSc2cVMZW|lxS|nWGV4c8z_~F zvb9z!X=udv(_Wvy{MzPbsiMD4OR990XXYUeiKC~W^VZI3NZi+v9y-K8uZ&gwS_3?U zyw>;K2KMyL>+f;zX49#A1!DPhw1Lrz8Y$fTseGtH*KpAWArQdmmJ42K805480=h?t z*h~X%;a*DC{;5#kuNS;{oVpxPNI+Tq?5_H_@Wx^x&Aos(M1Yfxw(GE18Xqb+xGQ+! z3}mfb8+p7Iq32u zFF;9^`f9S$|HEg&XU(`lpO@=NmgqS zZrPO1(cq4{n1#HDctvj&#jkON#h2RpR&uJ;s;?Q}`)bKKp8|22Sm)Ns(3cERCpJDm ze|$Y-A%I8t-UEyD?!30I40YyzFlVJ0&)<66>tzvlXk>1D1huQ*QyKImUOKwbCZoYg z?aNQVpNH$YPG?Zcp&VGAJ~oH8Ukhygz;N*shVPO0J#K(#fgmXh{ciwF=dW;EZY9qD zn-289|1_g!2*C1;T{>Qh|HCZ$-+y<;@&@oQ4GPv)!cPajB)D?tl;5R6@owamxQju% zc((~RHSR$!*5R)a=A+UR>ubv<3x%k3fcOO9?_+eg=&Z_0^{2dA1Jyn(BG$}l>hKsMXCGjwKn__>Oo<}CD&lKnwp##BEM8ggE98jJ(gQ@a%}-9w%~3zzCV&`xzg z$G)i#>Z@{lwqErd+LZISFUv&q`^~T6h?1bSVbOVX#vsc1T}+HTCa&e`Wju@$bg{p} zZVN!%;vWRnaF=OxGN5vsP?!>5&uH?+&jgUt@$laDPMIXhj9J- z_bI0XH>A@)(+_P%FRQ@$wHa`+c~6QnO?&_~bO{uoHXh;_JD-fe{o?z*OPw&5?u&HIJBQmr>ENp`@m)#I{Y_1dqKA}};`S#YR|Cmj_T=%$8PO?# z8TYF%E$1!+XC@8s_WO0PC;(wa52ynHYO~c|MWH1pe~!LvST$F{hbvMA1}*j&@k=Y= z`g-TC3VyAglYWu1IWq8aU&B&iZC)Z*-g7;r!1_0-HihlZ=Qj7Lw2VZCk%!-U^Ht`^ zdJHr7&%c+C${H{)45GO1b?L@u>F@0K1B6l!zEyLE*|)QmE^Kc$j8dWbsAYq5KRPu= z9LhT$cxu$BnKg`8s^>vAzIT3Gxc+y?H45YphD^Rwqw@}dv}*FV#x&|Er=2JkR4x z6Up$w5a5q%O!;9z`Gf@c7e+rSV4QEnlFEWh$xq$9ykX+CPG@};%P@Qc#Lm)VW#Z*n z&RN-m;>t^7f|M{+((BFew>E2CP<&XX2h8k-syY`BXH_^w;3P=e$cx2=xQFrE5-{9T<=+hO-6?Y3POW-*`;Uz zmWls@@qYCuS&wff6IC@@*Tb*sNu2;vgnL?)=uW?G7b}_7-%T{o<0HQcKs`oE3Y8KU zzmg{7)(Wgco*Vls2%bok2q=04yqa%rX#vPTDi|IA92;LMCL}1_nlt2vhDa_fgr}+GdW!niEfS(yl239*8U~pVs$#Q}h8% zL=u3BFLWmpnAO;&em?DpPI*N7(WDwiCV5EZJsasBoNiR3dc(r?V2aTd{Q~UCpr*bI zElv_kc#^9xRG08=rVOB5r8=ioh{Y`R_4O@}R~DEm12(nV*v3$L_P;s@f0}tv%BJ*Z zUtGN`_c?s3bH}1u|LLn68jmKOrZ;q9TThYkw<*^pD)OwcMo*rID)?()tFbco=&Iv+ zG&i;4DkR8cvZukc^^p9%;P;aO{aRG9ABow#=ybK7ljBMfG~?@a4ga-82_moxQ_<)K zd!!)nYA|vWJp9cc*@amR1ZN%$-rc(k>?;3F$Ob=3{{x6UHn$hp0i_;y57ztbWqXJj zd)Z%{k7r!huHDth+pK@`LQrln?}qvoAprYoaX%vm7@57gr@8_-KVQ1}iq##`8UDhf z{*_15>2yJUv5be-@cp+o@+R^=b~qF)u+Z5MpOasr+-2*>Ylt6UW-kat#=k$^kp0Dl zm>$keAX@Zwe8Lzo^-y&7WZQASY2VEJ;Zn<8f_bsEu=8~H0UBSSe8>6WyTm^(v38{G zFoCAaDI2_c(_`}l=FomMt`*sp%wI`*(~Q)Eni*6wm(jT6YD0cDrAIziQ!Bq0w*N^K zw}C{V_;&Nuhtfnzq?s1fsMggr6L;1ewfkSfiF1Np`=0q5o?hI%;YXzsEIq+CNf$`%id0X-t$g zwdF$`NuI-k*kI6nZ7?g9KO?nt9?`uhA!+%M;t~7CFS<%S)brKk?~e9A#4YQ3HlrZM ziauH5VbLmb&o3?v!M;1ny=B=*zh}5Uw=8AYsN(TrkTtXlEG@hN@B@=>kJ_w{mO9_h zd11*pH@7uGz~$}iPssT##%_GZ1J2JcrJQGIj}0bPGd6|hcbW(Av!}b__*;a6k2ap_I)iN zcgH}+7}E7-sQXJ6zR=vNv^z<7iMHy>T8G;vd9k>ux5!#Go!MpRTL7YFcX>9${xmH8 zH^4@FT3t1S5K-6ApyudJ>3#rC25hH#e>`Gyov7-mDysEadaa9zdLR+QLs}840IBjg zq3ck00#a!t2XZ~EE1nNr0O&K@bV9{9LH$`FOqpNB0Z&k5GO%C84Djx93v?~c>l ziceuIv*m5>7kzeR^CY#N=7W$v&-!Eu1&?*@#S*9PYrYKhesu1o12N$@fl9&~g!a>V zP7JXwal8yH%XDW$=PrpgktG2X`^{f-7FT}UY^x)D3dh$V1!l*bm}(*P%cx!HJ_}A@ zgD!F$(7;Fd_sn<0SR@Keg*N4}C&VVJUM@=T!SP{kB`kyEC+fyg)Hx7!-kXgP_kCB-35sftW)+Q@QV8W}*%$}e zaLD&59nVBCGDGtXa$@&mU9pYIt>=5SEFWzjbYrA;?psElD*Ie`tbV_KT+byykV4~o zT~8dRtU;CU^Ct(CNjH{_SAiW)ONI5PD~rWr4ebs7EHq_x8hij4Pb$@dNZ!r55nGIF=~ zrO7~9O9TP1AFBa|r(Xjm0x5Hy|C%$S{g9NdeIffjw#WA)o<))jP`+4KN02{#mMZp| zSgySrrmm5q>Rz$&gW%cqHg#81nS6TLN~SWJRpv?c;q*V9WN$l;uoVPcoSI5QXm4gu z1-~6Su_ZQsjjtFBzPZ41G_hwoT|RWcfJ9#&Wa4ry_9-(JXZsttZMFc0QDk7C`#)&0 z;tPRS!s$7{!0RT2b%rk2aVR9<(xi@FLzbVlhV#_@KexLarGVjBMDF?jl?6U*1Nu+? z1?h$N|6X(eOeF8W$7vPwI{*G1%~ZwVbdz6Iv0_&u#hU4*@l>iblN2V+7tf2RahH}_ zy>sKQ$jAa1sKmEU@)wTHWgJ!-x7WX`uCl?;yQ&{^)UoWa4GHPKW1zQ=dOQf)caV#+ zJT}pKW`0rFJ{@MOFJrDC0e%vi53?M(aE?1yA2~c#SIJ))O7fq_z_EL}MB`l?lCWoL z4Z85G?z8hllekDpvS#X&H9^;pQybSbDyO@2-!r zP*6}T`U&Su0np_{IPaB%_B(EqHB6e(R|%+ZDIbbe2-FBs%S*3q1!P$Pe=avpU_}Mx z`41;6xfCD)f!F!gpOb~3(Cta_WGMKx0**p-%E5c93sX`c1A%?YINQ_VjR|M*K=DTM zVp=P1o^Iv_HkNWhqH89V5>cNox>6- zj`jWmv|IE&R?uTVAG7G*Kd!oxPKfCkHUhwIcvj@sho1R*y~4TooTjj;|Hh4nPs(dyms4cD__W3+#oq0%NBP^@9ssJezJ}lc_!2WKdtWZ&(=T^_ z@q&BEslfY`6QkU2iTk@EQ^a3{L*3JVL2i2$q9o1MN3@u@b=9LQ16PgcGL+0YLkA8B zfq*o{DL^0eHRrxnNUhjU(L`JBtJ>uOLe~G<7x@DpN1w3 zIyC-=?XzFb-P(n+KM#*zSBSk6V;Z<*;I*C-WNjpEjLTqYgLCoMS}%X;{*$8)#+6R$ z*$Nspy%e=!@#Zbm{cK|0GaScV$v#Z$6@A=_eDtAX(|=5-jR$h2zWDg# z5z8wBEcV5Ke8+L6VDGc9){{SzXK08>Tx|P+dAaAtk)PX6H85qNWW6JxJaCThu2sD= z>h1Gu5ce}fvqsb8==ro`C8;`ZqOnz0P!kGhpEA`p5*@e3T zC)d(Q?w1P5%>EMG<)@BOgZ%JKT-sdqP_b89QsoKQw10JiC{>r8)wcHIb64x=tw@f1TfYDG|T+n z*qT|%*0mX3Z;Hoh3EYbB0KU=sRNcq{+>%e6K}{Z*3)@5kBFN>KTRX^30iyYO);Hfp zR1HiF-gk|%(V*_kYX*0FgFgvy)GEGqoAUQ3$Zdn>^L5`G$`Vcwa0zDeP?ah~`}IBh z1Y1P9{H=Qnz8n1+b-#DC3avjigaNoBV2LW53hZTZoC2n%&YLMe0MeeV z2_`epJp8MMmXM29|6>1Xi!2t|F$>5hU+@218!65PQ1@GaQj7+sFhsBr3hbP@CfZwL zrL43~!awM#GK;WFn5Yg_9FOHprdhk5r*2Ig|9%euWP#Bcx94*2jpR*`l)5=m?YlBI!vD z;;S(@iedXiM=*k48ZSHdeF}hKRofAByH4=gk?v%#g>h+nF%Pp+zA^`ldploe_jtB8 z6B82!Woyb0?;nq{pNtd%uGN*(zH9m{%WZ(|y)S~US3WDJ_53B9=S^W-*_DhlZF#qL z;b0_Hr(I>I)v4+mh*_FUV3A#iK%4{nt3UM*0T4iWsizGA>{g^Z)%2Ch>`EP4UfST2 z?_GwjH_(?9JDE?DsG0AdGt^f?Txmn-yuBx7aC@^SnL?P}OzAJq{R4>%um1gtOu+Xq z*B_zqF^-#C|5k05hru!5( z;4@sTNsVw)*y);mal)GFrw7AFP52!Z4E`iAGpVv@4Z1#h&=hfu_K@Qgo?!fD>OcLC z$1!v}f|opVu&k}Bzyt8J=y75(L&DaFU4&ySsR0*C#6r=XnBe_#N3aiF zS3|ixC#c0Sy!=obqMZS*N%J+sZ=fPDE)|$eu-nPlxgObX@az;uH2g75T2Pq3%A_h8 z^%qFMSiW6LcqkAC8>TFe--%Ti9=7ApGEZDpX1YSbG)hS+xjIVuenPH2>U0~6h&uJg z$f($QFp5URZ^U8mXmJDf?!48L-rf6py4f+|H1C0{S>^im^7zmz8lq~{8u$M5+P^^o z@>_5F`r-lKb;|#^E+&U_dL1<3f3FqMvpe^e8vFK|wViKst-7%j_v_m)FOBPhp1_)% z^1aaQ`9HSSq_xi=KNKAplEWaf*hNiD9L?u1>!FN>@4De?X#DudN=~<~-4|mm9kUTR zw|f!w#wqa%qpG8deJh``Y*MpR4>eZKedK%Fa2uR_+)M?aj&*M2`C@X{!44+OpX>K( zKB2m}&V6z>11q0-Fu8>2yRY`0mXSBmhTH`uC*ZWP2Z#y_;Dc;@aua!BsWCwU{{C1u zy&EwCJswHORXtvLSaB%*zqkOnDC?vCY&%C@svTse*}?)S z47z(lOu*Kx6?rYyVU>A;`!sRqM=WJgzPPdC5Xajpp+w-z)B8EHaULhXi#f4}t!yeU zdS3}w9TIZu%a>W2s)B)iA%tGju!Vyi#!%;w4tAV3OY3imH)--UGm<|KPY?SN;;j#4 zPa$OTGwe%$nGp-0o`a^agGl5~(`o_F!jr*R(DRUkHr{D**@J_rfQ0V54Zd8zH-8gk zflpElk%$6&-PgJYFh(VDl@lSTWY|VPnA7BmHdIRGxK|y=3E!_E5Uq|^3DsH1j88g= zf6fnXvWe9Je(3O%l>?NM3ixDO?9@AC_OkWp0rCw}Kk#6dP=Qo-Z?l-6Ya(|Ke0zb0 z*Me(`T<+=>NqN``@@!m)6@sj3+kT_mq4WX$<{P!xi6ObJ0rlmKJf%_5)KWGQG zqL2`6@IzGKYnXA!o@qj^W$=ca@9`e)r+(;a&32bC5-^u1-+B?x8r|X6IFzbNtR-|1 z8w2X`18^QF;f+wN7_deZ7Xv3v9zcuhy)BMdttB#dx7u(8jazRCex_FDmm7sYF*v;@ z-5<1f^E2dK-Mvy|IjIi0z*v(zTV5L$A)R^%C zj0zm7oA$F(ZCnKYD+9$1{*wniSf!IqAGqS(hWU@$Yu7aAp9?{Ehc^Pj4 z6nJx}ibED^sC^dl3lOJ-PpZIvMakeP-!^)jH5mt0Y8-#Lhct$1tE62EtY^(Jt1&j5KNXc4a^7KKUO7TRf`T<*$|E+7CXxzWbx1a z+jZ`O-umbl!^Fb)o^wmFPdFeESqX3}xhqSq_v5LLytLe1s5SxTzP#yc5 z0gXEuxD5{|DR5$9+JAWTyvK1C@4+hqlt(qJD5w(*mp>tAMat}R#1QRyc2WqRABk~e zw}AUoP}CG=v0G^!{PMjkCLaxTnm#4tcx0!^Rn>7=;MqXJL+AN%E7&c19Bc8*p@U8Q z$tK}IKWy8ODyN``<8&uVpUWe#jZ?-u^|1yjWMSM%*~J2SyvLbi1;x2EF7|P?TFf?d zsP%>bDy)41sp*u2g9f);M#!qHJV@ZMR^Do^A@0C=WltiYy+BH~V&Z&8Fem=e=Yd1F zS_o*;CMSU7ohX^8?yBQRjd&6wpkdV*hXbN>Oul|kD9lcYcq%Id%HUy!*=mrUf5xPr zT7c!^d+v@$M1Y6oA-Y3{+?ZbDOxt(`QEaZ)>HIV%HiWBCBG{j}#E&k&sVvl0(RsCF z*kErf4>}gw2XZc}Vo@cx*&p|KOsBv=G{0bm7t4xJ_j87_Wi^4+`&Iv*7I&x#+tEYt zMh&S6YBw2*jbM9uhi&)J9c}wbjvRZs_Zsckq-vqXO;T=s3jZg8zZC=$_+@SiW$XXU z-e>tdoVGJhUH-;@CT}2@2fuID19_FuUJMi6iBtY8p1Rs7Brd%gh!1{b|7z<~cAX%% z4%=HqL3B2sM5U@dL~d@Rv?&~kW*#799lcAB}oRc;OY`n-7uL&2e|PjGEe0voQ^ zClg#$b8<8wRlA?>XNQI$ZaM$%U!gTWCN+R1M0Rj;qao$;8gmA9!`tK2X;I-*yRUqt zIm&H+ml8x0w1qAt&Q!L=*t@r`=3lg((k#8N<|b>yqCu?NwxCGx8&z1S#u;(e}t)F;bIx7*Lz!ls)!)qb9;(m`>C{6+6c`|x4 zIhG&0XIWkpy5Demj6Xdl#9_BZapu;fN?fldTU-qMXfQ4)H4Ck+vYkk1VGQ$ZOIfcD zTdS5qz0OaDv8pWG)|b~l=MLfE6v-sd(?{gem(x!2mmNWH z=lbzeitn5BKt@%czNVu+Yu=_^_^HoZf2Gqy3~zDZST~~lZk?+|@N(~gH~5@9_*B1< z*nm@q9z^1rLV?@F4QxpZL@G`j#uJZ=!}bmUJV4Ky4JLOTJZ}UOFj~6LE4(Iu5E{K> zeY$6z0PH>Xv^bh&Ob~<4CwdY5Mmn4}olln(EXx?`h~;qZ)78fw98E?kGowE^OJ8;f zBg!-P3cUbWGF~h|{g(t{%nZOB6io0z`rHNh(-Q)V>Olm~2Oo?L+zFlrtL!Hx#AX8- z=-^#JaoFzkSY|@)k`s6{6qL~Qy$#ty;@`Oy|3JoC=6gAT|4E77#&+%ZV6D^*{ah*w z#CrW@!JsJ)I?$+@tGf)T_TmV*AOdCxlwffzr?aZUbMs#wG;T^~ge@O9_>ibTG##nwLjcHt%-y|W$7aLC0TN#ciAFb0K`&!f zzebWdCGKj}oZ>fWu1}glk2D+}xbwH%E-$Xp z-r17=>N#=#XN}5yYG~>KsD+rz6XZai&IpLdSa|=bA*Z^?wD<4d-T{^+6KY|fX=Rc* zu8rn#U1n1&^P-5!FJSiv?f8L9l`mtFYZRix>&$nzGY1(|>92CkFTxiP1k0~lALRtsiMb;Ih^kGrNY&nI*=f_t)hW399PtxKhS|E&E)Hf$R22q$(k*gE&s_ z6sJ%&{VKG##!*>H0cX}&{-C?x+<*VreX5vQwvej3=^_t1bnlOTJaqe=Y-|9zg;(Th z4+~J|C>?e;x`ym$MIzK0l`EF5>JCWOC))%TWP<98>-N}Ky8JK0N{H#T(KQszF^6F^ z0bE|}+619AV?l7NO)iZpH)ViG#V1F_NY%sY(<;ZJR|7$FEbtmzqn$CJD6IqxfxC!d z_*#hSw6@|R3ra*W7`6W)Pg`*!O4e-`g=W!M<( z=8Je=gWlIhtr=tVUNMdK;gHBF;$(TLU6i}Rs}X4+e@|!v$Hj!E{}Qr+)TAo6*@$o1 z;c3C{BC5Sd zgeQ}h-RZS^h`;s$H?3^nXfraV-PgF!S6bX?MuAdr zKp!bzR{Exa7nEhRDBy4HaA1pF(yuKZo#T3ZFN0Q*YU3R1PC)vz)J@6Ra0KQKj^O7BQdNk@?U0+E%?R|*a&sJgU z5YCrQ==p2*6!p1%v277F#3j#I9vTiZ8u2QucQW$h-di`1Eg;mG-biqzSGvf{&m0t> zN$OMLlkhPquVK^_8GA3Z)=^IW8S5wci2=@;=2w4uj3b53Yv&I%e`$aMOE6=x)OQEVPRptv1mU261@{ht-Z_IrG2M&I*?&P7GoSQ(0p(=`4Fdor8qCX zLXX8x7vzp6mliKb%bXitW`2_X-rRo}=aTJoH0?lugX(9#W&2z}WN`-ncH%dwo>mB3 zvc%_K@fahW9>Grrymspv*D55ai&Vi%Ey_v30-MEsV^mmTD$XM17ag8>IJalHNiKgJ zz*w+v5e$`yG*74^k=GaH%|UJ&!7U%*6Zt+Y3B1Sa;UqSAHM!x|X4nQSUGZpqBvZO9 z$q@M^srmvl1(Vq;AmN#W=RDJx3BU5e5J^E>@#Xg4xvxZ+#t6bK1Q7E{S<-cOt*LMR zGlu?dI~xVAXUB(vOhh04S30$kz49I<=W2Cw+=K3;mvc=AR3z!g{8Z!<2t11f}qy>Fr<&0eH)SwQ|5;4M;iI(;0$|H->uODBnSBNNj4#+PAo8u40*o{xNID*1> zvNz?@RqHqN7Y~kg#I2%WV>+#|w;=CiKoPD*+_Od4ZY{OD;~5KJgj>~cUQ=yJMhBwr z={LW+!}pe|V3}w-o!**OyrtcCcqrvzAqTHD)qXGTf2}o2iX5_X!K5u*Pc?gtzHEK8 ztc@%N7O@}nL5cV(X%onIASKq@)~`E?#b8v%!W@jhb+z8Q>bt}y{c2|p*i16p-~v)c;xRq4nV4+GJL z@yRW6Vp-y5b)Du~#zMKXK)Uo;f&wtxh^w|h?hOng|_&F2EmLB6#IfmXKaCDkOC zn9C{@ejcd{sf0bg7E{>BU;6uU=9n)6{_1cY3!auS>6r4t$_G!DI*m#{qTW`yn1=tW zJPP&sP0t&MJZO(SQ(MqgI}G6-Roneh;m?uarO%?9Qx@~-^H1}(&%v@D{DoXU4`+ov z>WKw=T7`)rudmAX@w}>4Vu{2coStxB3)S{gY+HMjpd=Z#!+7dRo_vN?*xXCqSRd<< zh8%2hV+q_iW9+^gPSg4@2`9m3d3D6z>2$$KbLlO^sWTeZr^U*)sb<*p6KzReH*bX3 z?%SW@pt069xOhSiQ1ptv@6T4Z2$=H24FNS~;81~;bWzP*v^I98@y!M5eXR4jK8cPN z^8gb6rs9b%iV*FytDEj}HSz^-Yw)+7X?C1Fl#Il&g%JagY17+Ze#H%eGbOB!Q%7P= z0pOr!?&Bjpd@@sVUXvX>Humec>5L5J5b_VNgcnYK%=$n+NueTLX=CzIV2M~=16P%D zc%)|n8m&`auXP~|?XjESv+6gL`NuDKu@;EEe#sFVAokL(0}1MJ^T02@5)9Bo*~2fI zb+i0^R*oi71Dh!Kt7U*BH<`3E{QY0UaR1^Nbe16Zf))5>|8Xg5G}FrskBrD>-cpa} zrYuGRk^P3csy&aNc~zUwJzY)P4E=3!oh99J-8k-#Prj~ne!8t;s-<<*&5cx@^;#pj zQHdxszOdq0uo<0ssg}Qz>HKGjG+I&1niU^3N+2ZuB&2%EnmUgzGQIW;b%k7tyP zd{=hSjO=f=C3|^Pu}HG#ue%@jNK+ydZ%e@Z+)#+=huBsNC`53-a2SWwNq{p6^+I|z&3UVJ&tOk9ST!G}tLr-ABrOrlh+kO@HX+Jy}e?Sti3Z>6WDX<99aetyBwDs3|{eveSklyO7*5B$MSB|SnWXY(=V#!Puq^}x8*up zDqWmV&-TlJVYCXT3GoCURiY0rE0X6aPr+Idrx@uO)TMswxjyIL&>?`r?Mit}5M7Is zZhuiuwC|KCpB`MPv~bBtU0}t*S}>R7HrFScbIhWiYtX{znZP(Ur+x{E*lpo%dFu)fW z!dIe7Q)oBnHQ`nr$LU^U3q99+=YJw|hLQ=$nCF2&i~hgMxqmV$9gt;d|0Rg8U!S?a zwU4jc{Qc-1R~IwT30Xd{8PMXsdm)C}2X;^|yen-76Hh`e$N#~Jvbxw^r6Ozvz4niP z>6>6ZzzWy2N45Ka2QITbz_h=VsqRVf^`sj!>X^n1_w2;KiB5u1Pus_asJMTgL0mnd z#hR;Zl^YiRK`>1{bZU!ycp8_OuH>_0+5WymGOFN=Id5%kay7C4$wc^Fshe(96 ziivNJYjGFgHNqlW8GVEl!PZnMT?^B%v_c1k4a2wsv7~8P;0K7YjA2_5I4a(Q=8|F{ zJb#nBn?fvt4i|8+Gg$Lv48wz|Jz6CcvNd{IkOyK98B~Cjixpz}xOBY1l<85#zm2J{!uYyfv?SKOB2>|K=GDswj%9=cP(2G=6*#vMb zFN`uVUHXkURg!Q~ja{4ELR!)97Z%XHAzTn9WU*%QAegr+PXtP ziZhPMQa`hpV4OWKl`!^SCfH>&ovMLt!_#a*m^}2lIX#J(cwA^bCudb0hKCX3d4N~d zxRe=J>@X;Sy|h;Z3@Y}Sfk5Bi9$MxrNj^)+V%qvXZ2u(WwX=kLbV?Jf{5!SjUk?6t ztFwl`p|lV6-%;-WK0M94u(;Smsjq1|8p*2?5@^p(>S$^pul-_wW`FjCoa6K}(W6~>Ip4ovOf0V>6w_tZa7wW3=CFZ4F7#-y%+`jC* z0Rv~1VB{s)?KkRNX-)ubvs`^vG&w0HTl%CQmfUKN&*Z_eT@IxqD!D0QGb z@i^?xn(U{D&Y~`DHm)Nd{vFDzo4>{T79;sIt(N*jGHE}`^O_!0AKuC}+lsCI0All` z9~}{~)1r!uPZB?d_H+$pSXrH|k1YKF(}MbGkX~wZ`&6U`+ZQ zqH|?$6sHrE3+)BlMaZ=o%HUPbOrk%u5;`~Sv?G%pyBymqpCBT=*ZfrbNtWJ}ol~kqu z+bWxAfl?wf8a@U;aQN6arCx(Cd?AMveAYMi3S*(7tyFAGYUoKY-;p!M&P$Ulg7CKJ z6sXEYo`P|Fc4&BT+LF3c;rcpOzCcr1B+nG&XOvbw4Ft^S?XzIG-TyM4W?lL0(_o2# z*nd!+eKrd6lzg!+gqtKh{50<0BI(-MPlzV=D*a0&y=nGO^X8KGo-Y))%HdLE{!3Yd znnCK-7sE)}99@knW4nf~UcP9Z{iu1%(jN(+sCSCrB+^+_2Iz$h=s2U;f=K9zunu@o!GZgNzSQwi5o5>0vB-aVwG|qH#fF#wY3kQ zbiIKahiOylkTRRfLLACp!-8JLy>+ROu+$xw_?)s}m%C^-8NFvnSn-|Lxn~vi?l99i zE-B@U<~Ou_jO9)L4JAhS^O1=WSKlXl4DehmZ=5^}y7)yEGbl8W7q&p5BKGW=5M#DW z3C8?*qM-bujff(IBa`Jz@-)s&1Vz#szMLG5+ZgN1Tv@9m)>@@pF7_JYDslSt(gP9l zC(so#PuNdgZl6Y`>^Y;Qn~-u&g4}(z0;!BDO0r1G11M!u}o@E&i#N+a7|700e>UI39b>h-k0r_jp zam0OLrvHLY&Ze+8&LZSS@5|W#3ZG^)%ZJ-YGgX*N7xAu^t~YNU-~IMmyNwc9jSY7N zDmgY4-uT3&=f*xjxq=DeJQJAIDuRKuZ#v_}cjS!$ucL$%ZG>V- z(fh?H?9M3Ku8YoTDK4)mIMjqBNVK-@X3jcsZEE8J9_T;_?r38gV{a2?x+G&&jeJqX z+k9rY&klCbj!^PdPcfX`c*L*EM z^4(j0^3&;lHEgR!5vl9T9`z~iGV;S)#G{d!LPb{#oh}F~EO4?%a2=Egx-SI(AQ5xN zsdZLI#F93^D{~(Udgf$8s|$=+c_YtbVS((N+5+)XJo~LAxAliM+`m}ywr%M!=TB+R z+B=u#eHO@nH2PKi9fxh#v~eGoUuLpJw<=?tLND3BlLi5h@p!aKZDYPLJ~HDZ3c+!! z!phFGxjxZc$`r$2_#`+ps)~?QiredXvzbAx$#mkxqe5=uGC?|lCn8J_CMNl)C<@hO zO5fYLdX)xF1RGmAD)B{Rn{RlXnx)!v?+C@pE`snkCd(C<;LZsvnMn}L+U`{0-IC;h zom-NNv&wY}ArL`+^B%2WcC_s8gdLf2mpc*Hc}5m(1zkg|+Zon^AU8c=*#E@^sOtU6 zD`K0p4nA2H$=j%Ga}Z*rg3F23(|Y#Rp=RrOuCc)^uMwd7Z;^_`)rd1_KrOR%$qBntN)>>IVW zjPNOj4)uh?L~}|k(j_ITs+3(a(PH;lR|K7N^}rQ+Erzr~Di4XCW1|fmqr1+GJ>r7A zZbor+Y-WUX5w#k=f+(%9SFbZ%McTXOhxi#n1X6ar?$}4_x}xvz)zZBPV+a3#JLk!qqnVve0I1rjumJp)1 z`g^E{Gx<^Iw+l(;4~D8CmDb$nAPq`59@>7VV>_8hon1_;QwY!E0|-Yurr^j>FclxY zlDrPPMXjBkj-t6`w=wsU4XVf@yH=Vh0p{422)dgQnnt|p&;a_xFv(V@oa`ez?ou#n zRQajFdP-KY&hwW>j+~szu#+z{+y z>YNv*1y1`798&2tp$ubU=7{!<9f2UOJql~wa^bhyxAKFuKE>}_O_MwuIz5d)CF!|- zJC4LfN}ItdY@yL;QIJp}7*nj3ndvm>HSl-`ej{Jm8h^(9X^uWa`T69g1Sv0u17I%0 zy$_Wd{{)b-x(#I5>z_We%4zsp)W3F(|9?ew_&E+A*B`edm0QDTJ;cB(y^qdN-hJj* zRfgKy>D;19EZZipMY)7o!eCdrr^`olKS$bi!fa$%JjV}CtZq)W2VvSf%LSs^8~sB8 z>LAV^XB?$Q2z8QWadN`5+6ik*2ALxm!EPbn4U4rICB>Jocc42SergnQ%h)OJR;!U;4Wr z%)b!J^7wNG8VBw`yJ_iUb?!5P{c0E*oYB zk-f;Bu5U3yhHj?yd;_V!@)XxWji+~KEr)oElMaoP8&@?|tiFIBp-;!gB#&w-i}&h( z*|CGFjFc}PWY`aS?ey4hAUO4#>IAIHed?#ru&-Bw0H!6Swj#+S7Y;;^*SUzIoB#00 z)?ZK_oH7S~10b$grGcj%oEhk66rDyN;L;GoS!Pci1F==Nkl1S&J9psO1LIoIfpdm8l)tRB%- zalK&7`22g3?dqV3Zt863{TY!veN|I2-@0I)u*V8Di+uc7QG0q}qkCh1ATVYHHj^x` z0ii8-3g+UsXOX24_n#WkZMN!(2xhdxMpa`~E*cr>6Sen@{zkMa1Pd@w;!SYt_S_4zXiOTV; z8|J_Pc6W{|B;ECNUWJM9^e>+=KnxcmyB}f+(VHLXdFkfQFfV81XS%mxjzO(X;eS#S zVPg-6V)9eAQxz5;qSA>2tK{SiHP1Mi1QB;8r_E<6Sg^gM>~7wwT(okL4}=d9d>7pm z9c*;bb!PaNd>wkEMKyOc_|Z!y&ic_~=Rkynor+R|IW%@va{O7g&pBeS!%UFj5Ka8- zT08iK>^pya|M4DOvQMnS`X>*M-z&bMzQl2JkN)7lZ}wSMxz^nOgU_=Wxb3*`^A_E# zf8X{|7GU^TD%AS@!M`My=FFQw#iY!s@Yv!%Cqw&euQntX0jHc!M6S)K>dUCJG#18h zz~QGv8nd6t^{yMOXSN01B0DU?bd*`FcEFN-R)Xx9H!;Rnz%9j7PIyr!-PD=OZwhMte*Ia z`tqrQpjhHprm=C0{7J8CVZ4{D3P}86cMlOD$LRlN^ppF3UKA^#)D#>gisHltV0!T2MD(X}JOm)(bkN!4^JO1sXQsz~&q`f3whwkYw+qpbajL%?yw*Dhsv2VJq zvODYIoOZEiwItBGu2{d@s>{;+`y6MKV^2R*kS>ol&F5|Nxs-_J%8TpZTZS!jEpgJCI?4Y*P~VdCpgBRCzr%pu!d*w1qufHT$gr%Wg!*cf^+}gk zN{C(k`t30G*V_?`p-EIB)shkIhwLL;tFXt?Ld7fzp$^+077PTvtu@K^3z=zP#a5pr zZ=5&0E7nt~b1pNgHM>%yvf7tna%X+*&gM|&^PtG!{*MybN<~Lnrq|<-!K>^ME% zSi0(b7WdBy{`!>=F1C@%_L+9%dsLW${D-{<@TMhKC(#kINPdPWNP;nx`8Ezf)nNQdBdPP}>0ydL}9pYfT-@WEDL$AXa^_ z+ua%QS=}PlLHIu};4r@(g>kAH|K!+?rC&b)V#JA!eciW&H(K1rm_gFtiH82HlA$+{ z{!X*bLis38!M$eo-5CtN@oC5f^gs8#B$yV3Qdswh9Y)a^iTztG{C^I$|BK*b!f(?= zwz9Uy){Mm|m~gJ_T0#&v?Bjc`3^oA&KyXI$U>54dqc-{@O7Q9l051_6_ zE3>%H3DuZtiJqL~YYB^tuBT>uTJ6Z8UkJXbmVnqjmHkk%_JPW~oT$5=3KN*w33W4! zy~8B&|55hc@ofI<`<+y))TljLR8^G{d$zUO(&5t@v1g*hj2)v&&7j&EA*AR~o7CQ0 ziKN;Rd$wYPP$h`*d$gZ(zUTCu^T+S?^2bBs`RBQxdtBFbUvHj-m0=Zc*21%m(ET`_ zi(}HlGu-G2oT|{YqFw9mrJ3rsy2sUa0;}diQ~L#iDv^1jDQRBrqdQ7T2K4Hk?PDxo zQ14HZuvWZF7hEO34uz3 z(#5*XoylF1ys)`~$$}iL9!Y{{{p6@7iogoLY_5@BI7r0qUdXdk*Bsu>^vzH!vm0n$ zvxm#7w+<3JP7#h}R4ekzrkXt0?Q4js{xTNnhlXL|2Ey0f2w0Pq-M+1i&V{cOGG8;| zo}sS+VW|U2eGGVBptIF-vA(?>*WpPW{<=T}WzPX4RcM9~P;oMZ3Q18-t zToIC?Kk+cmp8c!Qy0NYeSIxqLXu3!FL#tw$9fBuACIVf9K+TS<-=hUA1I2{g4 zlI}dveFa~4-JkoV5~Du|nj9Lvw%BN1c;Q)hj-a9SjryV}*k!GEVuO_=pJP?>g!sWC znZ?O_KDw5hnX&ILgP3M}v%jLOKP|^qjquz@&u1Ia003nEU7oPUj=d*MJ>47-U%*zZ z;oSqzS4$~Ua;H~H#iyY5KGXj3)%fgTiOR7vZyq2dc-nV-@TvQWld7zl+rzcq-33B3 zERAnvlDj%=Z4gGFU=5ycet3I-w)VmH?z331Mb;-G#>S&Gg?heV%Q4RI4lO`f_!!5O z(Js%Le2sihBr5B+eFX^QvQp*HaTi?Q>!e|4vM!K`NUj~!^cTp0PBui@?%u`3;!X*9 z1VipWU$t*NE_=wPH00~$B_6X_%9V`r$p#jWnO!C-+qcYqEMa7vaBL?zT^-6910|GF zba#}Yk+}ndlW#^Ydn~~m%RWTfF+%DuEHDq%A7d zVNI-3jmN`Mej>(;m3JlcQozB44NC7$K^s!q3XsAOv8Jor7VW85%n;sm8X!Snsex6C z0<gxxKwTSfCV{eo(FSJNUX)4npt530Sm2WL(da)uD?-f)GnI*@$It3X~jU zY@j;V;;(SM!~=T77Xg5*H;;|6e9U?FHPd1C&|Y1UwG>-UnWGGwlgfxAA96z9oS<}u zf$Cx*u~9V8b!6g#&<1M$a7V|i!`7MlWG=;XR>fI&CBP;w)S9l+ZneO-=>}J!`+NE@ z6!+=<4DcpS)%ZI&`q@@it{G0}IM zxwDv+th0MH=1xbgGc25r0nD7aj_u|;M6E)45MLkV_Pk<@Fia+@L%bW;a05+bWeESE z#^xTlF;kktWBd@o-qAa_lKGJWrM(Aqs?F|{BF}-M1YA*_h9UIb9P(?u!icYQ$g}Hp zJsC6$WIt@wB)GHt% zK$-s@@c{a3N@fGPn`Ghs$3i7;2lLTH_+O`fL(7Am$5yC|Rr|0DQmrq`) zRhp~Lh@PxHeMzQ%#lO@K)rtEd48pF^l+<^E_LcJwv!6mV7z-h4<7v;wUU2tUMe!1G z{%G!Wtu_3Qjz!#_Mf*5O2J~=>q@}|L^uu#O5Ey&AbpF#JMH8$cA)b*})ktiQ@=b$e zP#NEM|J93-#H_Jabc@Y^V~H?Kvz6+5F#U=oUB;}*N!Fgc=Krl2LdZyR}jHtrvJzq-eKG)C49{-n8_IpN)JvNHg?BGL42c%&S#;o%P5tF!C zaM>J`iUNnlHi@rtozz&ZmiU_9%RrhWd|w1xFTg89jN!=V5yzJ-AbsW_gLI7)ZBJBc zCJD^&K)6rr8zJ+P`;$L;Y z!MUH8@tNG7cD}zd#Nw=jdAR&mYOla)&)~{R71(^$G9C)iv8HNG`zWY1g%I?Q4*q=F z%L?qGF@o|u+{ZR1He|Abb(`Xpa2|6Tg0;-2MU~$wIf}~Wd*h7x6h?%RXFJB0b=k7P z8m;Pv(o_#f5L^4cXhX4OU&GLlUiQxW?GTZe%=K2J+6Hr`E`+0*5FG$7*RTzIW@h|4 z9v-(a?kSoUVn1vRZx6J5Ly@T!4>p)+rFgG#F7{9ecqOd#mc+}Y%K&A1k&x36H)FV- zyu@7P8e40@Sttk6oUB=DD@xNl+vtd?sM^|5fk!zZ;~4Yy9#ODUCkCF7h^#N3g8BS| zgLUSM6c&gL&;Wj@nurQbx$`b_ZYqt&uPLa`HZ`cXvE0jL)SJ4pg|-Vv+^_U;iL0s@ z;+7h5d$BgqM{L+AxL%_Ey+XO*SD&+Vpt)RnonNYL?RJoJ#^W&%sb3S7yZZ^i=XshO z$+DVv4^Zg4Uvqr(cjNr;0h2BY+~SV#*F%4P3fuu!u5@7c$IPT!`0yZ8RfHbeV@YI4 zS_Ym9w&l&n#XVfO8SsASKX!4B*K+8}W<}z|`_-B7024gv?7?KwcdqkUd#k9|;)yyJ zw7wrYn(2;6=b9FaedqmF5+S=q|C4g8g~)iO2b#eGd{rq{DL(8*keQjt*Lba|DoFuh zHuJJekrt#DZJHZrf>HI$XqwAjA68geopio~@I#7CawN#{(mfN z{OwXQi>r$e_(xo{GCb;sP|SJ(3BOI(V2n~*skY(3I!?se4oP!aA8zdQG<ULqT_qlJp(I3)AuV5`MYrBJ5k~oQy{Rdo#x#hDpZ3onFcBUB=>aw>mp2Q zidt453m+pmsuRdKv(@;3=&=_lJ!vdzYW(>&=6j$KW(X@io5>8j3tcRc%dXa!vnpV2 ztbqGO*`E12+!wOR?C{m#d;Tx3oaKOvV!;JOogm*jxO`#r<@N7Dn@Q~O&NW6Y>yS?S zMkw(YzWbk@uQD*1^<}+w{_~T`Lu}8KIuA27BufHp)gl9II5$W58alf4{-y(Q4B;m6 zkPse~`0R4FI@|Rb9`bm+b#iBZTW}xxe%Gw+{n{c{=WUZfjgzV10t<%n-3Q%J!g7>W zj)(FzZsALLP_KbNiM0&dOFIG29r0xeWv>BAo%Jp6@DW8TPRU_E$!p}aq)lw!x7Jyf zLagu;?dxU4X58RYlnpF7X1+Q;N4ae^$Fi^YJa2lOXx=7#Z;Oa4zbrsq_t+bx*4Bp~ zCy82i2B$HElya8N-@X`<64L{Mhu_7_)L`Y5Kp1BZsyIhRK#$K#lvyOTAVFJ+r9Esb zr%%6fe;6w?BWmteN%CI;SaGFrZ{z3>9+G-AZ|@&5xB$po76T?I9TRpflK1_fQY7N0 zPsc~wC-n^orfKho(oG)1SoDQm+h}e>c7*)n$$D3gwYeQq1PqvO(B~B+Ws<>yEtW3u zmEppUj~kh}mMLEk^BQPvML9<^E@i6?&&ZB^N%@u)_vNFCB@GgAqGNaiYEF9IAlNUs z3{^FHsgh`*UAdR9COsVHjsa#p(=vE@5atd!In_%GDoLr%z-qpveQ*<01L{o>L5?fd zQ@51=YOI2H^h>MxlWK=?h8A8WmE1#>r0gCU*+3*tFg)1DMP$J1Ja^1B6}!^=wQyz`qdaS@L@4{ydUI=J;%$Orq7+xsipo}orgqwXkrla zRy-p5mhQ`*inq6Sco~AAj5QIL$I~I^Nvs-6 z?T{ShVju>H0ff0Qx`2jVACgs0#X9!l4#7`TUx|m51tBpX%aJoWh*XbulVP~|%2IW+ z|8OB^daN4T@o%88CpSk03u|LW3XBUqrQ$jJJ>=dN4=tQrQ<@n;g?yl_a7V(AQqyrF zVv7!Xca*cM3Y{vNmA(}24kJ30=!&y7CzJTgF0{MrA9zl>zP!cNoG=>_eHVV%U zi1X}g;8aUksGTLFK0)gBi0z2h=SYsCPrMKtzqG~uppjYooy5;dXXVn) zauqYd#>?CGFAl{U*q^W^ShpU{l#206G&Fk9cQG3wiBPZ2?K`ayILdvQ!*IJkskD4&lPFsmq`J*c3qI)&6=lIL(bJIMLr;C* z%`Ey;vNRXuUB>CF=Ly@lH7e&% zrRMW^m!OVQrPO9$5F@_?X>}96)%Xp9-gl)F6lIooIdOq zo#oD-&JbwUR7Pv$(EVb>a_n}#M#wGZs0{@iVQ4RL$;OYFB5kOVFz6BI4pYXJ?T~Q^ zB)D#`WH&K-ZZLnR3(CMhEx;0@|M$Us)gpK0m0(gO!{@S0oyv64%l`k42Z5V0HG zs^0$D()Fvy<3y+VgFBbx9J`$p57nRrCT8@O2=lBZ;j{+CrWa!>z#q!BuExNjpZS~J zFN9;I^SvKNuqL?&ing>-Wo*nZl+6r6h_z7_G78s<{1Qq{zkexLbCoc!@P~GY0Gz{S zet04JGAb&wy)V$JBbi%L&a#;_Ia)S|(`u79f>hX0ycgPvVm(>H|9ee{l8D5l*sXdJZhI2v-ahmEyJm+HVqCwg8y!@SMZ-A+iVefd6pcefd3aYq z5EVgQZ7sS}*g9nRFhBe}nhjgrFuU%xrk3xp6I@}$Zm&zHq_?qD;JZb=LI-Bn{c=uL zb{g>AmP}i$@_$z7a7)8>N{J-TgVv(>GhuI?+IWKkKX@62MQh6s=H%`55K6a(sgQ_U z*x@y*GVQi3(*!Yt9-{)G)8+qiBH@m`-)FT4I1WAY#!P3X_>Ymk#qJQDb`-988k z#DPAI+Mxsbvo?mh2wT(fr^}HJsImy@NSso|6@8vxi018sj5O6sirGjBexiy$$NI5$ zutBD>$-{^x4#cqIFm8v3yyDiHZovQMr{^mJOh6EL%T78qqHTP8qSxb=t z3`1oPQVo86;}WUk!{yWuS=slrt;EB9nyn$YN$2kHT}NeWg3grN`lUsd;+tpiTXS0) zJ=ZxNZDYmN4GKLcO5|$&K7cGAA`I1J(!mo^>T2VttZHQR9nf_zQNB1vJs2bJ<-832 zGZ<99y?+u)NHIlX1H}+GL1gLD@lIF+e{Kbr2y*j#GYVEom8A|3q^uk71)q(LGJwWs z1Qc!R(&?lRRR-M!Tlye#V90KB5rhVB$fMnvOah|^p`^MjcNA_133yC19T0}0aj&75 zZ7FtIyNsBzF{nSKg=Vxg!rz+aTJPP_QtvvDlFXF=W^^g&a)c$PWUTRSWVsRwox%wr z8Rk*?Jg_~iY35S(U>Lb(fjmVD8(I_kau(f+BMVRSEg9hO&MLM78HbIR9t_?~zM8OI z9#p)&_4Z51PFmhZj`Et)X7#(_PGy=$c~SN_UV6c=*STYO(b9K$3-Ji^b^X`dF9e{8 z1t;!S0w5z=7YLo(C;u*7gZ#(HZn?Mu;eZSPoDShES7~_-{zFdo3t}odnEWibIqv+& zwp~SJlM$EK;6jqP$KassSzkKPtTLNMPb)L)CYD=GoSteIRSdW?O{bz9zieqo zFsNRfeLNbJF;;`_Ka~FPRrGCJLUS|P(;Y!6>0Eg3 zFwT|GZGB>9DMF%eO}RRwU%8`w4Qf=I+JoA<-*1I>fP7|YcNJY&Ca0Ku@!ORk4uv~i zMkZc0ve-zvslLeKxcT9=AR*P=LRSfiDD1BxTK2DHkG81e$t4q4z?e92zX0hWbj(md z2b$(Ij;=I-)y%TW58zD1fqF=Rs40vX*97@M-p-oW(|e`bNGSUBqojE?7QP$qQ~kBx z-pSbIm`R}HAB=^-Q&9^zJRKUE;gKB_MKmdm!v~CO)TKO}NW zK2~kl)oSdo?kV zAZo;}|2=tGQ_nO_)Xo%;DLJ?T&HurFMI96-|NkgVH1uPf(uV)za#i52NAFlC8bl^~ zP8sWZb=V6xPd@h$YR*;5Ip3wMa+YJ}$2+5ur>MEH@x>E-Irsbtv*RBd-cU`EgAbc$ zzddP=x0af=x2?WiGRX2xxbw|)zkXGM?R}1~2z`UixhEvlCwaET0Nmz_0ckfsb9Nr+ zjSvn#gsbn<4VE7LDD!>``_EH4XBCU%KhhC%Nm)US!m1>1Z8{mOYPY9Sz8XieXC0RI zjGM?sED5q$f1YbKU1hj5Su-}*>g)zlo?gt)6!d6P>eW}?8AqXx7E6I@OG_cPn_!J1 zBueN(4C>Lfwjvoi)Y4(DqvQFIS^ADvOUUw*VsR2gdYxJT$x&0te?V^J2k|fG_oGoP zR1y5l&|V74_2KU4fhe^Js#!eP;S4Lv&stx3Ew!KWoL1_FU=(Icaw1q=nC-JP%`o$` zY6|>Wj&tguMXl>l2o2+1BGF2^&@<=3yotjxX!- z8Ka#IX)MKECM`wj8FJb6u09Q#%lxw$-ni{|8g93Ce^kGFp;0GAtIH|PmNSg-8Wi3Q zd`$E8l1y#CGCbnKxU=_d4-jd%Hux8e`OVC8KdcuWzPPS-kVOWM0bv3=lIaR?T8fbX z8u;?e-?gTs_5`0=fY;!J$!ZT$`Ddt3?Pa61|36KLb08oli{qY8={4iy zI(_k14ZQg{P8l}0zP=uJ(L~V73?Tm>Neq|@@n{}S3As@pH_B2nqknVpef;h}-RRm%k`nB@$HZ>ZCx^LP*|9)V8PbWm*K zgBa}KhwPz?-*Q;t(IcK*RyAP?$Q;yvblkF^{LUD<_X`0q`pRQ#u*cfDP zqTV6s-s8)5g9&|Z9!VH_TRPQVQV{je-%Z{$f)j{gGNuW zB@o{`SdEZzPa#3U0xLC@#jCLNe#zj80eh=8FurrXE$;L(Oud&zkX|Uk_yXFWpMiP7 zYWY_6zn<>pgQc{SpS)sEL|wZ%nW!tk!u=0`In+X{X%F9xQ_dzH7^L^<%boqlNB<;K z4~F}(8$c?ny}+6l`DYgmJA9b-rgRrhwD5h3`Eg&Z#H8RUFh$+- z+fCTlM25ziBS!4SBPL=`K%L)1xsmPnc(>r0h_}X^uSE zF2=i%><5|eVh7N05#*Y`{gch2PYD`R!^>-YsGu6vSUr?patI4O$1_@h3r@mH*kJ0f zK6x2i8x25!Q44oZbnkdCvmX^c0+z%h@0g%fdfrGx>;`rnn0RH5HxhHPX; zUssXC6$CaUUE?99e^*W~kvKaFw^~T$a^Jmt%3JCihhrC-j_(^3Ebb)HkSCe0Ofwaf}wc&wGmBsh-HW9!nD6K;`PXe$V7a z@fj+e>AynR)x#e$Sn|T(8VUvszNa|9M;3Q#&-DjpS3#-;)<=gddE>%7X6jvUO?}=O`yq<8UR;*Vp8Ht_yVv6xOYA|o*jKfq zrQygq4&>|e zKt*Fx29g%vPsUH%zo&E`;ZR&bYcOXBu%R`k=+JxXzYt|wlSdAw_v7Kkx_@%4O-3vw zR_w*a#XV@d{Krmwzw)B+fd(;Y|FkSDwPuNq<; zKV?1@+A5l49x8Q|WE*jiwhd$FtraH6hfm3ekc^`R!G1k0yN(TD$%mK~{(e)DfFkuo zzBTsK>{lZ;3Q$T%)QOnNhmX4nGj%q1CB1q**}=6ZJ`YHzGfkPZix&rVhOJr;MsB$5Fd4VCD^cu8**RIQ3e>DPCji zy8F2F^=y1*m(D5450un`6}U$McfMGpA~l(OUBE1X^4F>~$xWULR)sNv2}?g$-*g<3 zc)nqxs|Fl-AMaH8g4E}Kw9$CU7}tXk=TWO_z6F&Sr`FO*@h>Mu)2XMEV#+fzbp4p~ zap6dQ0%9I~ad@o;2Z7!RNuZK(&lsI6M`#`rpzAF)mHR#%Lt8B*llHUluLKWly<@T- z4E7Bkg+FN?(x-F<^s0N~$ns%JH!NHix*vt7jO~T$=fVGyPk~Wzidqu{{i3{JLN#x{ zi{Z#n1E)07gb-X|4cWmmW=1Qk$;ULe1S!idG?}gdhjL^V9dM_icMdX;=;XW+aE(0{ zOzsSZ?QACOS4I7^Jtp5cqMIOadVjAQsBX==wO6gvp8uI@fG?Y0190!E$1CfOf2OHH ziO^myuLJqm3}@H&g&&ZdR|oDCT9Yj-0LbXtu(flqt79HAdouWrsCiFESKGGFF7BL- zyuhi8mstiOZNZAn>~YnP;n9!f*vO@?zGOav*M++YRRQDzf=PyMeQs@gDc2Asf1sOo$Nu>j>BSO{!B5afk`7O3MNwRLPKhUO z{S~UZpdVsfx^jk zqtf$}0>;vL+Z+_>O>dvKxQ&pRX+Crxj`1m+Yt8o2sIVj&YPjEp=T*GhAG+ZUBN1(5|Knl1^MLOvp)YSF`f6V4H=GmNcM4| za!c9T=h((;F`w{j8zk@WfZ!|fQ{?1=i);(D%?fo+=@F{Ti};%>91Jo+qd2h(w1?|X zft+g47Bvp3ZB>&x5`lozQoQ)gTeOZa+g3eex26adfK(eZoWKTr{2`jJf@5J*(_KU! z!x#yI1?-*UkX9@H{d_s*xM?{yw=zW@B5S|y2Lblnk=@ia1AUc<%kmM#N6BsyRRkxE z1-+zKz8o~vrRS^fojAC&x4OlJ%y>^ZkDS&>o)wM{TT9SCtw=8vx+^~Iz)_$}<4fzF zr0L<9?$5<-S+2hayJ-cF)s!xljmv>@nxSC)AXgtqrL>33=f^v{`;Tca(|vIp>Q?k= zNfPt>V*%Y8Kl126ITvw_x5)*Ra zCAvvnFYO-1!|NSLppXXQ92xCsH*5;s>Q#F|cx2gWLKwsFU-eDqQX32cs~eM;$rkty z%JrE`R=z~?#}3=I(%gH))J;-Yo)4gV&IHwHyz|0$<>QVM+qRwu#rc1Td4GPNuP>U$ zbu>=-Cqu=U^*j(gp-Vz5+r-A;)Go_kBbLe2rqc?eS6yqDDhyrw)&6d*&mB?GIuyd3 zkno|g+SJT!Bnvzz{m+m7^`bW8fSpH#%WC|YdJY|tnY*w2JC&UaeQ(g|YzZ0wfN=BX zfQW>r20j_5M3x#aH>X%psIPL%;cl|U`l_mWq55?k(aXvb;)fl6-s(pIa*dQAPXsBr zu7e1fnF-AI$fl}voT_88k05rk5Z#d>xpzLpThc-=+`;J>j7(W#X=>PX)1dKL)$xP_(4>cz+#q+!Yaujf)m>? z$TAAuExIH*o+`l0_HZ|6SY_1YFmxj;VsSct0<7I zC%Y5N1bscli#6-=J>CjkP4EvH8G%I`yFh0_ou5m?XZA#hqGzxI;p+?VUK}#JdU9ND zgc+)Hp-VE~8QNC;qCfa$Y@&hQJG{r3`I_EF7YvO$f2R#iLQ2n(`_Y}_AuHs;_YC%- z(Bvpkp{5Q$wq@WzD9`!iF-}?Xn77)4V;ky+QC{R-E~n$a!+<`r2{4xd~dglN6%Qb_N4 zey>eu%98q`LnV6#^%xdyO?c{|PFUv4@f)h&ux<9aP^GylbMk|W1bg7Lw{ z;V$%uvsvT!>{B0ioeA?|t|rWuG(QVnd%}#XyswEwy-145oM69iGW||aL9F8)0}Zp! zwf>kUflnW)xec|DV_ewWz1>|`iJV~GRbXpK86i60OLS~Uz#)!Lw^&iJT5_#+)$gB8 zxKSU@-jffJ-D?@tr5YhFQgd8K1A_wh$;dW+NNIYnO5RyQjw4|?Rl;UJwy4uQVW4Zb zm6(=sC%gVqUJuwRcU!+=0#Z6t{{n7f1Ym@^93iSR?&@X|0JPI9S3*}xG2ZHefcmy% zx*x>%wtfXDtSEg}c{1xv8GZ8$`skG`M z%&!76!-u!tQh{rv}|t zTJ!Dm)jh}EPgmq}_5oYMs<>RIx2@F(UyU42E_Ta^!F$PcnKyA$j`dRUa4|$JqW3Qo zi41NIEnvv&%(&N4$lE#>tN{6u*|`-3$h~CAi~6Z*i`OMKzi;_&|0VJ>0cTJ}fY1ph zJyy<}kn}C!{Pww8L*{6coZpzs+YMzo<8`xdGcyj;exqifoKyBjsDTw)D!qJsZzl~7 z_)yQZ*?CwJkTUMOt^yjea(Ds~)#osE(oHkZ+XfP7Y$Tt5?Rm z$_>`RIB!{D6DPW-aI+*8l34}4y&O^^5M$ZScRAkyssbA6!pXV2y7EY?^sEfm>I4Q3 z%%kgbFn8+cmL@V9+7aDGyyad9^-NR=RXGLBJY$Ob-i*8jDPuvMcUZiEy+Jvg0q;On z@dq@Y!dn*JD~fBH|CM6WnwBaj8XQkHQW1@nbKB(UH-xe0kpe5%N8N1{Y&Qqob_d23 z3)g1b7^w&W$}D<_jUneB%Q;QQMck*`qDvbZg2x$u?u$xV-~ag^z3>5&qPH7dON_d%{;$?~01I$vqr3fRS!EAn>2>wN)TYn&xJ-Y^N63 zN~D8fT$4wB-hOQQ-9nZgZzdpY7cVL#CKd;%!^DK9<=Jiyqd$FX?&x7W4LNd)aM5y2 zyp-Vi1^!C-33Lj|9^0*W2tJYN!k3e(J$tetA`b#_{w@t^f!E5jRhCVTw(sz-HWISf z5RDZhQW+r}mpKBAS6dpUMlYA{;0ki4Gf+ZPbs5qFt`{aVxmCTlLqtuAH!2+|t5s5R zwWi`f6Sj4YR1Ava^x4osyY1!d!NfuF~Gl|=h94cgrw^_D2Hr++i)9qM7cN$!8& zTp(zq7EYayzd23{r!z8;LRPJgf23{^L2UB!M_YjXJwfyu%=JEM+kJc*l`o^S@G}uC zn6R9H#|o_BZl3I%Cl|U7w#|5qt&B%Fm-1B<3uEIi0BP8FIc|gOG|}v>3zZLPMEJB0 zn|eYBc)e-X;+1n4d{Ljn-{DnRp=x>;OcFF_(y}t>y(78s( zUn_Ag4PadXVfEo7W{|I?d2?U5v*trlsuE|fW1}Ql({+wKV9)IZ4(YC1Mb0_faa;6p z&9*kD5r>0+x$8 zNjgHk1;J#RNo!&gLi8`o;bQ-K*oQur5?vuTj>~3hc*}e)5r^tj?lauR3WuYR_k}oy zwuL>s>|I-(XVp43qKmrJ^Et^$eI^D>TXo3=C?acg1C*xQPXmjMvseNQ4QgyZQPN#XCINho1-Z>SmRl z6N|Rfd>!aTOc(6D=>riRk_lkDPkWaG^=Q4PP6@10_54t3G0Gh@><2+kIh%>|bjsP6 z;ygpOX%bE30yVa|5fh(~Hf1(E_5p0-?iGFQ7f1&FQRubY!@Je*)CkXXbr|)QG z`^tm5wQ3AqSJ4>=BCK_-D>JNEk|OplkdOT?R{HkCdr7Hvd7KnlCIz+`Muo0hdE^HO zFRP8NEo~*PYRN$=mfz4I_Os!3x`+WB7d@Q*^iiX+N?|y<`(e}?241Yf8E^JsImSvf zxft()t~AWlB^4RAZ@_M_X7Fb3s>%Eng)bN~9YjB&p(l*jv&5lGL8A)!ffXcA$w`4T z85e~fL^w8f1;nc{ugS_Rm#ji&Rr~kYms~6x9d(HO7nR)oVOzwSBrHXX0itzJ5y zUTyS0_^3gKBap8)AtUWlyBaYWuZp4nQCI*`#`3}5%@{9lul~QMBEXmQV0FjD%F3;H zVDH43deZd}U3O|8_~A70nmxw!J~3kN(76%FSqvr%UAJrtg1mkft%|yytkTuK#yqo3 z&6*52yC;vi>CP|xqZ(QbVjG^R@$XH13^H}DXt??C5h2#|mGkS8HZfbHR&a27ZW8<$ z{FA>8x=_m6yvEssLlU=+9_k*wp2LH%)P?bWFNq5i<&uo*-L`xPO=UK~m)xQ!$nm_; zUP=r-+KFqNRVuk(C6gYtJ^OhFw?Me_!Tdz5QHwZvim?DU+=CGTIFjAs?LFwu6_v^enXnp@kE5NT6FAnWlj$`aXY90R9 zY7L)0fZr*mdJz9~?)Cv_gxAMGb5ZIZ=hHF|CZ$imW$?hHwDJW3D_#j|*UuOj7{Kq_ z!^|~1qgsn7Cr)0TGKAbLiE&J)lq%udN)1L$+*R0$b*Z{SO5f(!fVaI@=d?V%tn!MI zc8f+P?CXR*2PC{CsNomAdLL>v_ z?S6-V3|o0HFZ>{CP6?%Sbi)>^7>lw9>o_VYS&Y0{wr+1bFV0yVk)ym2&(dEj!xeR(*Zx~TAXX^<#^ca(h51@_|B%Z{AtX(d4UFrZOPv+1 zvAj1+w)>?{`lir4?^nYONbywYU$oKvsgJ&|0^2PbO>tUy`$jgi$v7Jm4W z{gWUL867*_mW7*s0u0suaR!nPh0F!Rv|qq-TZp}zI`s~ab2db%s5tlHO+Lz&T0UX`OgblWhv0ghZ9AF`C|$$C@G)BdH=c)eFFxL*y-n6tB9 zDg)xlR7pn;((uwVPu`wP45b^eD6bITP)#PLEd?dcsV!bIxQELe`}p3m*i77=TJ3{3 z;Zqpr1!1GWyEB%gAY4cJe*DHzw^-yvf})PnS?+g+v!)>7XJ9ob>795AL{3mTb^%v} z4VbFO<|!c0Q$|cEuGhZyzQf7Bt-);R)aBPe=&-;>^U#?t6ufAaQ0jpU;uSId3{lV|3S~tAq)4R0# zhLi*G>hHR4c2e4vbTBuqPz+l2?_N;qT+Q{`RclgCUlGreqVTFi?F6Hf^VQD97$jp< zISzK{v;s?@HWSqG1cLokV{tW`0^WQ1o*`K#e#o-Ket)h_H6PW*N8gyjeOlX}-2=HH ztI+;_BOlfn=j7I54H37u>s@Z&$^l0f3(}-ymVersPdwkl^xrbf2Wg&?N91~*M2J%3 zFUWy2G6oL`%ZU64LUBXZng$&Z{V+;1wP3k4VQ|n;B1e5(uDxt)du;{xeSeIAf#pOhlFo7=vIGY zHi@v&oVV;!hkC@IUYvKu);H6jb9H8B1%N@kP9yX7{znzkdUgHrzTnddTOEGL;D1+r zHZ>pY7>Av8D+G49{uwdA#ep^J^?zEkHXT0qNq*rfN8((u@Jroz{ywvIHCBwa?IQuG zsF8zksAHWzjx$LoYXQKE9%PpqP@mkv3nz4sL{GUqO zpSOqJOU&#R(6Q!bS}W5dJir56s$gX9>c0J1-p<(yR zUi*-TyZE3V`5*{O`|z(HgVK!wH+sRs*^KY9oux9m*Ic<;3yOgWg?cumF)ajQUb0Xf zM{ufe^W8A1jW~U!%n}SVYp^74i{2kXX%8NTVb|shCPqy9%y+r_mADt%1|oN57qj|P z6)&1Q_amUQy8N1Q_HokDwqt5Nn>&4uN8+^n?%)fYlS65i@}#`IDyw1aDq0Z~+O_bi z2T?)wWX{{Z@8Hxw)l?l>t(-4y+4(#(`TMo26Trc1fHaV=VSAxE@iUC@3&s5}18#}X z;ho(&W*brd|D8(yXH@*Nc~#r`JCk@9O8)Keby)b>5@6 zRczdYpzSh?7tsybR5C*BgVzjQAP2sdbsIAMReug64J{?JhVlX8sSEP)jmowL0Im^I zCRRfGI4cj?0Ms*QlHl*)8CI=NZgDI@o_NWpq_zvpv|^4>oD0JX1yc0UOQM%7N-xY_ zw^#3z9&F;mu4+O&Ta@^@>0YtiV($?jPgInbVQ$)JKZDwLQZ?D&&wo`8j#`}v!?yA`!W{Pw&!LMrpt zsC_#T-1}ZR7Ka?KGLm+yU5`f;5m0v&R3mqXQJ3g>Sqg>bXQ7NbgSe}axvW%st|2hq^4HkiMh zb@=zNcMM5}O!L14d;hl3@@oz@xiM`fjdKcNOf=v1vU^4)R)fQh>;G_tyKszy+gnRW3t^%g=tOjYa3ndN;S7JGzw>&o z$hslT83O6svR{M+?Od9axY@d{E%d$X{;QLoBKn2MQ)SY^Q`rnd=8neF>mE=cM-G`E z7ay(FOWJ-Y1$jutRw?;HFz&81KWfw_b|xfd?XXNmDdL2!0Kt&xE*aYl>1m3w(o_AN z;rLgjtjGYju1ziTPwpZh+FY9o+e_QMEb9SJ(Y!w7Six+bhfq#9uIWNRun6DAVp&>7 zv=?>@f(f8ZR1RDATfb-B`B8&Hks?U*T6GBgp1sGc60};KZBRqwoVbCd* z*AJ=TR_Ki&zL57@H_AJ;bQH%0-7~Sn(fCUl>Sx#Qs@h3#w{}eL*-b2-T+Ru253G1< zWU46M1@{hfVbk)Rp3MBUu809?e6bsso^LAGx_e{4Z&XL1uc&a5m@rk zr7(UyPcHaX6-^yA48`t6e6EHO-J^a2T^Dz;#ShRuFLfW`JQ3F{gUH>Ghw>#0ZF>K! zRS{)a+0S2@Ixv&`nCM;mU;5doCIEN?{4Ch7^cwyVDugp#T3`B({EP-FY(ZM?!9SC4 zA92ZvzgdThF33*GPVGMKB>@3t4dB{@0xT~?#UAT46fqHrvh7)TQ40RxLZIFw7HK)7 z?$=%p{^_lq-EcGT5z!dlQfkr$q2iAkeG4@NQAYZ@%Zev{Ep1;rr zVqYpdZXlp*Cvg_1^Y$?RYIQN^OxukhBz?MqmMFj2X54gy^O~P~NY?;i$AmA=N@_mZ z)#v{DCEU=>s@Qrb(Hh8G$BL`znqOmOqyyyd+6zsul`08W-d0>z%~Mkz7K9=oo$d}v zYQ(kPWd|&imEJ|p9KH%#ZJi(GW2QyM`9bsc)5>Kv)2mg~P1?ZGJUD9^dY$BYMc27~ ztCahieMQS#a9?Vol4g4>vB}$HEMaXDIM$;cyaNh5t8vt4?W&C6s)>@i9e-~c{D$8C0l*zFiM5vn{#1DaXM~>F6_Pg{0Jx-1z)q6d98-Sm7lapj>ee~h zhs?FT6Jl)oSVbvTA-q1FGL`onM??12jN>#(dke#>8%aZ<7um)~ z)DmxT%%V|(TMggpkKTEjH#k+7oW^8N)e$|WNB{?U2)30!R4YXIOz|x)_uz@km7J7l za3;aljRGu{Dkg!iu@haF;TAgUk?^9bH6?C;BZTk3e)c$K-C9*|b1e^4O#%<(*CH73 zjb_g1i+x*fWc;UsI!Oi@U&_pp`aaJLINo)YbS$<}-v6Lq6i?eHLoe0&LXz3ISv*~J zG-6k3axle4W7z5mydTtKMp|@M>|n{M^kB*9KWot7qa#C<`6ahzvRljFgv}pc8r(l1 zd-m-L&i+xL$Y&1Ij*|hq5`Ro`Q$nRxN1C5$psl*^Vu-FS5p^aY`v1{&6;M$vTHBzMP(eXjX{Dqa z3_=(}xdf$8ZzW1+n)>*@v!<=F9 zogLr)_TD6@1=+*aJ(XJI!$H`+gz@$--4nTCo`f}nIMz*bC9WlxHt#6|r=geaMACSb z=Rxh1X$eA_Y{fHgmn(c)HC|yFkn1aBVkPbz`SV`AW1WcBFBTz6x?cxnQV+|Ml&l!8 zcOqSHbmaHyuTDNIbP99PC@nMn&ID7~J47@4qpNL`wL3!DsK_b?0>@}-^-4txQc`x% zKxKOVYB()0tE}pKqh_C?mG2R_kvj=DBvVzfPRN3(08NC;i1nxi+Sy22sKOgPlw}3X z#q2*V$6@+W`wXuSgG=~Bco!;7lSf7R&#!ook?IHg3!7tNcAAzOtg-OHxAz!9kePb)(5-ido=niyHiMu)wZh=V~l8!IH|Bz{b0rYn?Bu5DMUPeYqL zeH`PEF?&6EaQf?}zZATH(Wp;u0oI}k2OQ)E`k^wKJ52~sQ-{CoZThnOymMK z5$66o5MRnDo{h1k=U^P|Ap;g2kLCAlH~sN(m1i^BOkG^r#-M{WeL0jMTT4+UF-pgS zXZuEnCwE`**Yi_;bRUcAd;9j(cDH4m1+KE%{l%(TmWmj;2fsB!0OPn&y)&Bve&s@) z!irPr47y-#{Y9V@GgVdLwPj#@`@SL*pLO3Dz`vHT3iWLW2wjZo&11{&3IC+5haAN_ zG8(RJLcRMKe{i-tRaly?27p2hGbe3?;+*7yLzPQ&irO3GCWIXN_P(GL)++)UM!vHx zPw0mIVBmgzwb^6LH&epv#LZ9%`aynd`XnsW(7=4{z3JcqWlXZ2SO-Su-Bpx+#qjy0 zBew}9U#F^Z{ah;BuN>z!)n-u@;-0;dg2fF=u8oS|cR^CgR^H4l{v642z~Vd(!IjRf z3L{^zza}bE+HC02^WM&zJ-*iNW*>a+ca*aH9GxHqi|EsaSc6{$y)BUwbT7ka`fXKc zystenQd5aI{}1s2NVzq>D*@}#+BJ?xG*VL+ zYXT2zB5InC*Jgi)@bht=cCtou-!tB$Ro%m1Q?Dnd%9lY6iEF3)3<9fU2fjNYeCMr3(PyWJv z>PS{3%rMyeefkCF;5rv3j|l%TTm;&4UC1SXon!}WosMD}RoC-m1V(D}Q$jqJ`zF3I z9N8SL-3Xp2Afg+66*22h4TC&4kx{DL zu!8B!F%Nyj3Jl-B>et6px6|X^1I`9|;f{Ww^2h>8MFl+eR{Ig$f)o`99Y|ss?huTD z=VPW0Ho!s8_`6ieIl+ZM0)F!S3X=#&VgLJoQ@s}BSG~GGW`gStxPIsYZb7Ws8)bdU zTn}_^m>-XN9nCTaqIIuGZ?t%IYo}>V7ym~|0hp@utJmeUz9r$+UispaM~eetAABvOj3)$cm9zK)Mw!~GE9idKA(ku0}Y%kywGXf;zp!Sa;-RTqrJ zOOLI$cMK!oHfYX~6S74_Zsc_mXE6%M8Av5ryqY);bhcm;!6okrw|J=A0lO6pdwSpc z!mG`tAe_`soBDRM=}D_WJNDf$*AEI`sRNTcRHc(>j|wfvqDJpvT-w=t%JOpH(rxH_ z1*W0QbSJll^LJ%xG;c!i`gj*q?MWk1S$F|Mxj-qFKlQcDFs<xZ8ZMD!iRJ^inj>`2TJ2X>bV@@q=7|TDbklsyd`KB+jXysy3w_=eKpjhF;`Y6Aj z>AlRk5{zUe{@Vsz5-u-sPpUxv(Ad5Gr_mR>?*1G`?NHfs-^;OhxEnkJVMUgzFxF4yS z=-XLW;k?%%s9jP#F}{k)o&y6d2iD3Nzh-Bj(f??cjiH1lx3ks+jOsx`z*0hcpV*8E z-&_c=mZx?u8q{CE-UKX}MtZdibsxIF{I|IRhEuG*7U|7<#JpH(I`Y2_8O50)V{{qd z`KKYf2^1mcOT=L`|WDaM(ge}m} ztv+N0Y7kTYvH_MJ+r7n+f|PgQS5azY4gpbIRot!%B5=FrdRuU-rOaV~zMI4NR&E%p zFAZFOw(N-%6=|jceaWX#OF>q`US^CQZI1OyUu#;h-G&4Dz&}-Ds|tie6(dzO4E!dp zIZS#1B;{vpG(+#fyN>n~#!&^fo3=mvt>1l-347aY*f5~Ol5c8;<4MzOa@FYcarP)_ z`zWaV86_%eCp_TagzVV|zq@Ey^s`R19t;fTb>;7Iw1(|)iEb_SOHh9EE!tyxZ3Q$8 z_ZS}q)U4Z9p_khHfMH*r%(KVI7^Flia{kyx54y*Lg_~-632T_(qIYb;@lLCm$8|Wr zUMn&A0xxL4fH}i(@s3Nu{&(?yc>a__VA1YMp1EZaFwdg2(%e>&+lwP5jFxuP%kSG9 z%aqjf(}>=8M@>8dDZUIb`6qsUwx1UmgeoZ3GaQDZX%D_5($Z&^8okh^?&7$+^WHCo z{?Y=B7aSM+%9Zb+XMLT;P6h)I+pFAlsle$uJ*&R?>q8OY&9 z3a3&0Xg~tS>!rIf27eX2W)XjGkkdwKTyb{t zXjQvxO@u0}td{<*yGKdm%#CG57X5T*ATPcVe41Y+@9=Fgi*oYE&4PAP?=t&G)&?!F ze3LIG3+rT>r+c!+r@8ca@fiBr4wPoKA5C_;y&50+WH@I7am(-!dtb}93Uj*kGF=im zdmOk5jO9%m9z7AeWN#UI$FdM{L*Y~A9g%951q7Y^o!jr&-hDs4?We<~3~}ns%+XPe z5^kph`G>)7UAn82*?Lt)`KV;4e8_#Qehj_hjg;t8_K_POE!b-EJpwjXqd6t;$BSjp zG8~nk@XTslu9CT37b*fGF=@+sw9DIJ=#E|X>mn-@$1r^LR>Y(L+V_0!@zj%Ei6_Y0 z;IWLuIRS;OrnD=V9}JHsnB(9gvn~mirdV?cj23#VUADX=+G0YgVh_rAG28pG)1cpc zo9HNhVW{&2FYZG`nD(T}n))L&JWVMap<7dVg1|*fEFBvdd2%-oimMPo;=M!HduH|7 z#TF73)g;{tjRbU!Al_(0iMu{%<289)?83@4Ba5#h=AI;XmA5`0@$wwW3Se3F17XMkny0Or|aQvR7V&|Fx{;@xP*Hy`B#t;Ib zt%_T}l#PXBG>T2!kV3>|GtU$mFB0{#5WvHS9=ygrI7e_FI6gX3ego9*LjYx+OtxM* ztPaXeaaFNv!S#umn<5Ue&2o;2`S^?jE`kfU&iy%xlv{idM6~YT2{nq88697rBYE+U zqa}s?96Vg2hq>_|Z}!>{@Z7FUyKoZa&S$JcMsxNZ7S3NN*6c|04Umxn<=ONZf2iOo ztHtknOvF?341137$)s(RvPp&oeJ{AFGGI`dCvd<4$>&xW9g2KmsN`kLQl;32F6T|6 zDT=vnu1MIab=2i3KNc~bj<^cyv0$Zv3M(>6B@7F2VKT;(+v;^V@17gUROF4Vs4*g_sQaad28Xu1`(un7CS+YZPigaqWvmF>#)G3i4ZU2RX5U@=VQf$RSBj+yPsrP+ zHN0A2vFonplnt)ZA4D|56| zrsLe!6MH!$452@VTD~>P7!gC;Ejg6e9dZR*xV!a|r({BYy};*m83kIP&fcl%54wxc zD!Eaj5yt?YKi`wbPA7NZHz(67_kOWrJn6NknUUPoW2z{v^%_el@Rp7UdLP4TdG!-y zE~~@b>-L(A-9VwU#YmJ?-Uj0z5LB98kXkK$1Wb^tPa=EX^5Xmw5ZsZHPXaAj$GcO* z>QZah^CKPtSUSCZ&wY8stzQC@6(M&b?pQrxYY(i%Qw8F#b1YZSoxghSrR~*Hvm<~i z-1fEhIs@x}9Sdp~?ky*Zjo+xa09^kchaKSs_85wTj?t6j!&xiBVg zeNdfESP;)8NTZ^)_z9*ZL(PJD1J`fAmszQevDAE1#Q`=%0h;xq!Pq2%KQb;{DBNhFDs%-X7`7UG0YxcRY z%cr0V^Fy^><{M~>&yH*~i?#UPS3C~{@Wzyj+7h7cf?4CFssRQznjWvW2XTt`5bhZF z-Q7yGTv$+QQHo}rxZQBrP}8u$;-zr7w?=z-Xu^=Uo=A!D@y1C=(o3&R6L+3?)8AC4t^vW<6(&P(*Im`*~`iQD^_c}YWa`b_UKOj~3BsLsqmuavnL<MK^`pC@ z-Vb)%?VUcnZJOs&aTd~mPK>@nVO+Kbtb#FcI>kvmUQ<%@D3nS_QMxm;gwsl@>MUDM z=gbN=6rPD2YqIWT>#&-(Qjtq|wmVa~IdzS7Xdu}+ZHG5hQYC@W@5@45-3vwUvv-t(ZN$uf`$$IAg?iRmcV@Gs; zWoB*jORnuX1>L{l=r^Y9W(h1t2HR;!zFzzfi~;Z2(G#$6FnV2lIAy>cdf^P9C-j#L zoH4K+Qn_mawsYh?S`M6Ti$**eq)!woi`s?|gq3f=THDOmMpt$Vf;xEOCp%vd9;SWQ z{~*jOM97;HLh=dfoELuiQ`LZWNI2pO_2X_A!%Uk6p4oQ6y^pRwki69p)>N`!suDY;3lTu`_ZzE8`_EO0@(=e9(x@YUWMU4JfT9LTr{Jj!dcQQt!f-2=2r=^_RkF@T#xqZ6wFDrKI z-qa ztvs>MSExal$l?*Pqtk5#@A#twy=?O#Qbx+g@1Ty6Pa1M2PnGNZ-d>gzL|Pj`E+7wpD-1U4eJKZ>>j_eBwGhTY`# za*T70u;%aSPT|Mc(sCO(Zh(c;?^+F*$XxS#1L<#7T;Oozc;MJ0(}8#|i*H39gpM9^ zfwe|aGT_YwBExLyl{M1|Cu^j)w-%Q5SwbHLPnu*ruZNkca=dP@CJ#}Rgt0B-pmcVa z$XwFA{H`84K{}6df!@@YA5a0w*1SVCd+IGUwtGrL{GpDEzTmE`7D?6}iLCMWvjc+g zA*AkEemVcaNKUpQdfN7-YGjH?Pf|Iiye;;S_i)=(e4urQ6V zUJqVyaBOA4+&uQapJcq@eNIU`Y}k2}DuPzY05{YB`SAfOyRw*h<(Y;J5La_!AV^7hu<>Q2 z#(Bol|2lB_-j$c|2wBP0z>|HBX#S=>mMZ5NrrO^iNqW$7P;;mzI>9#BJTv>RJN!0R zzt4#^BrSPXXd{hYqJLO9!b@|il8ekwmey3aPyY(@EYzuh6`upQc8u$nD)PEsr)9Ax zg?w0ZpU%$|q60zYtt2#anUe7fdnpDwAty2dPj#}2J3xW9+Ck9*3Qp5JpY!~_))aFq zWHCr>pR&KqcIcqwAI1w3sc?`OZTGDu$fu7@`Yc>9dd@xjqV0vxaYJqS?gvh;%GMCG zdiEcm(AZXk0L2x>D{W77zUN_vIwzsM&sSA8y{sAHH|68Rx8Hnqm3uAl|kaq#ATn6hCs@@56a0eh9x@y~nBBRq+j+5$zyuA^z zRD~kFql3OJN#K6F>{wib@7o5YtwhiY@F{&}nVWN|<|!R2$Ht!5y@?Fkoe_!An4Gy* zOoq*y&%V3~hmC%HYjNvYX8vZ7RJ`nz&%S^t>%`_)&bDb*k~=q{r^o6bgW#D$ z8!9*jWi=V?)(*1VO#SY>{l82b@8j-p}O;!ehp? zE_cenUq?$2ZSu5z5((gTf`8Gr&$?wb_@PIO%+MDu%=}E7RK~t&n<k7BDb(LtI*ep@M?6?rxzOCwl>NPdz>_xXx9tTuq9UneDfzMR70_p|IDt-?NGnY6 zs&^=&(HX|Dk@nGNM0DP?;FR)WV_K9y9MMxIA1dPJFN2SMQWtv)+E;DoqSuWjW8i^v zAE>TpMBj-?@nl`#l8nH8D`@4e7(UGTfY2K~;ur^Egj4x5_rD5KL>?Qwqc?NF?0)P> znc}C_9!zwjXCTfNuj-dcC7pxan0T*u5+43YF!!{2-F`%mFJb9hhK$v8GpjcjxHxKx z>L8qDJbm|Nit$o^N3YHLj%5t;<^omMF~Vd#s4UKQjkY0f{c!Rld%2n3`k?1GJD!Dg zrzlYs?{=vYnSeXll5h&2i$&w=OX?y2A~gsvH5{(j+b26#D+Bm&h|%*zxpcUpy)oG$ zV+L?S-{iw&Bgd-Ip)BQ?B-Oc%uU9mH2@;wvt}&b%>Sx>7O%ch?O%iJeOj_Fk7vLB- zm@kCz)!7{+uFtgN_A%+Xgfu2s|BX1GL83;o4=3czSN@9$0UT~i!=rPGjJSRyOOAh0 zD8FCu%Aez4RosWr=Zh?5~{_2i)iza(2Qqq9kwN5ub|-Nu`OdB<_rFo z$>Q4+Bv0o&cM$c;jZz!(=uOb~kq>DLK^``~V<>Sxm7~0r>y9d)h5cy+S|WEosjIt{ zv|;0wFP4zD-AX2&Bfv*V7+rY6dkc9=pw z?{l7heTQK8#jmG^OK){nekP(PFiOUUxGfE_tDR}T0zV z%ubW>rCx!b;boivKx*np6-1wgI8^fLonN7uD?#Cm9Mrse&~-YK{eR@B=Uz@c24Gfk zO|%Rxns}Rw>IjZ6{(Cm3owI07Gj)SHriJgjN3kDUi*=Eqe5%@I2GHfl}UY6VP|}q z1^Pu1ySBg%XKaj6$+U#Yx#nD#c5>XfZQpqyeG)zRmYreDm%m)2u(3eD8`445q3zFY z-m|&nRVv@Bb+7j@Qei0Dgsa1$&DI{f!QeHmvI$29-CcEGRai0F@j^vO+Y1bU8In2Y zc1uLtzn5#;4yN=%;V!W%@ZrGvBkQs06K({!MmFb@1vl`cx5nL z__l@K>7>Lv?-h-EnT`NbN|BpA2;=pki>0ejJI*` zXGmLoyrC_ua#YWl-6!Ngk=`+O%slXT1enlf+2~!=k{QC6B%34k!V4hWD&4y8MM9UcEj8GduZ_nfc3(Cla^3r{5cy9K;Ydi8~}66t+;s^1R?H1z(ryw`*g zJR9Y2mJo(sEKhka=ZIDsd{D@jDB~I(CSpeEsXs1Murcz|QxMMSdG^3+J8W^Q4>LKy zcGoB#I{r%Tpjbsp$BfjBy(apjZah2D7Lq|jLg;!3eDvLoKrOOFUv9*@+_MW`{TF_= zCI@yse`#gZ?hoJW+y>E!#<%yC)M z2m%u4ndPcG1#G}^K9sdDirJpaGHi-!)HvPOTNM+qUT>Xg*@n2$;mYG$5BPJBnJ`*xkrJ6^qBRw0Xe*z*!sb)VU-{u?qHhUQ5r0~iM1a#bf;*W+wh{Ea|~M+rks zJ*k7}Iv;SPs1B~*DMV;9(<6K7@p}K$stoAe8vVNQ_%YLDmAFJBff9W2_&d>ws2qpr zFn6W0Eie0ZpX0{*2lo9*GGX)C2~8)4e3mL!VbfAmS}H?%@W9iPPv*@5h{9LcKk&4u zzGmRkc4K(1D~5hKB2>y}doijS?X!|}UZLJ&jqVxPDPMTT4>wAV2RLESMItgT_1P}Y zbX7Uw6SIYqz+*_?5dq*r?}wfZ;fdUx_B5~A*xQ^PHi8u0fRJVXzRL%|5Gp{;Yc(Cn z3I7$n{pX8Gy0Z|k>c;24l8L|AmIadYiEh2uRu`}T!SYY-eMMn@5lDBepbm+7*ftJa5E`e~Ws9%vWC_ViKRv?yDkaG6cWDe=AmtfY z6`eA=XdbYiEl~pEZxm+qS$j3_Yc@B>EE<{8c*ZWBd5F zuUw)bqYU=^PksVCG+cD`j9KVC8T>r|?_^FYrXG_Y02xFekRISzxqA*q&?4svAY zrxv}*;+Kg0^(mhizpBf3#_%bL4l`jw`=q*ms4%VWIJKTab&mW(uTpH?cQNB z8#3=4S!&NzXhGuSTzku5!Z*!iVGIhTvSK1A&^7%wbN?PCbNro`!kwM8+KVQTH5&Va zuYMrBG)$kxs;vJe!fc3(E#VN#Ara+bAR)ibh%{gk-T+SMs6FvdKJ;D6m5P z+;x2^KeXb6^b`8lgcw26qW0mMlLgt*EVD*Q6N^5Ap1!jmj*jk+zrVNjmlj|xzJCve zu5QASiMbthx~5>Z6QOH8>2Y~$Lld`YtGB27>Y%z4klyxS=6xLBNt0;#XU*TdhK0v9 z@y%AM;qRYc8&^u%PnNY&8)xmVP1V>PLi5WsWcS&40b0d<`U+WIG2QPPHBM@!nTt1!MuLqBU8g z$J6zmC2`3TP}@wYf%nV-iCh#GDFdh*^8Ot6--%iK0h1~eLm&IDmJz@B%^p_XI=?cn zxd0L~D%B7s{5xa#$Nl@BkvqH375V=GcgNUss*;n7Un~rMzY%w7v1U_ve?Be9F4ngf z#{LAB_f3toIn$XXSC(hc;KpDXzWrc`ega_*`fd-qXCTcp7M~Zd;J$6zRhzl3w-HUE z8_`ACwsioxU4DHvVayC~Rb&+>Do&l{axU!s%s?HYx14q0e$2?!(3^m0#&RdFm3fd2oP^LfLe9lH~#1ddl?|Qma(I$B3Cn{Zt zVqnF(uP2EZ*9mhMF2_hmRi*ik8Y^^dJgfnS!9N(7RCAK&OIu}ed9I+4)4|p=V@7ei zfpLe_32}-tt+&fP5V)$?S?LPO0ySqIv@~x_q1NiWs_UF-Fh0#@7a04;3JyY@&IBzc z6$baaj~%m&N4Rsw;u-CqWMG4+J?%fVf^CowSY?M7?$J$#xFmR_quB6}i z58q7D!%mkwYmPGNu-An`U#H$5r16fUKU`dKVhaN`qzkAVst1%{L4(q<5LQY#9rF(U zbb3KKE#sM0-<$3_h6b^OzYT6GL5^B-e89$QW*~p&s8Xr1o^)CMl$#^#e~|T-+9*%= zAkj38vVc9_AhWn0SfQF@>N#B{9$LR%$GukimBM|#^QINdCjQP?iP4Kt!IKkwb1sdU z*KL`NmW%e^N}n!S)ty#tL3by>xj+F$HjG(L)<0qW@r<-pN4T2Y@sfry!c$nTgz8qz zO+i?fjM$7HbP^CdW8AMNGM|}gAdNnl+5Ck2%|~MXA3n1F5&+qCGr3Tvzq3RiY0t7H zGOxVLzmPt#K(Yu-Sey+LYhtpxG025f8T%o+cnZ(tzROMM%wrql4B@XDHTePKq^o&T z!^9bpDkfUc05;1AlWPzICo%_nJl~tjXT>G=X zRh570XXdF3Ymb`a7=LqkmzKm!kw+_WC3w1Ov64;4sXt&C5n>g&ptLrGVkM&WIaQQc zs_H2+c3Z3LQUKVb6U&%btp?0)a{?=?%a1U*Bv58bgSVJgvl~&Y<>US^oa6TDkUy?z zh{Y=$Y>%3ocR!J)Aw@LkcN=E=ndb9jP$|bHD=J?jhMKyj)#eNVl|C2L!k%HW_m(TR zdpf~&6|P7o9U$6YwlIJ)0TU6&xgMreQD!7wCm58t=Ig%K{k`)dj z|M=UQ0L{_CkrV;^^>3#~;sUw_YmeMBs& zO;b&8;hgNgE%TSWuhR;9zv{2|KayX)b@4-ty5t>GpQY>~Q98j@@s+}gA^yPqo{=)E z4!xCJ5&8i9w5Qsmu)Onajy2P^bUv!-vW-YTV3MPvXP*1F6&gR4HVcOU5iB{TwZ_u%cgbA~=*uMsjCy z&}u!M$M%DM(DW}LT>)k&I4$~RLk4w=Boc-^v^wvD5q7m62x^sAm2 z^$XpV;@p2O#X(wRB1K+T7HvbMQ^yT%b_~rTCg1J@+wr=wX zxw>8HG!r!6yiOzDzSnil+YBL{ML26Vf!e2@5E7IPj=J7%Eb$oEMN8g^8cj)|H-%0s zwT8-%{n*kn;U6#V_Om`RBofnHABt6`^INA96<>hTm_v}f-b6?4TflD zOX0CTyGLH3yP3WSCVFjSvg>Bh93|F)Uw{|pW{{F#E_>|J8YB#Cmzk{DtDjnAKmRAG zw-tNoG+mcXWA0a`TW@L$vl*fVr$kYRe}7T={S|5TGjrbQy(#DB)OfJ~S+BV$B`y5= zbjWnrO#v1_@@D2fZ`+#~wU8coV(HI!Uy*?=+JCjIW5^#rEv7^+(eS_=+9zTU4-*FP0GFYQMFodx0mG}0b(r`q;HfzgofrH)C3kTWsBS##` z4$+;37ymb{i2|Jkx!B34bN}%AeYY$cKfOr$%I_m~@oxU_l4P&T1+`Ggw7?U{10&W< z&7@%J7kbyiL7O*%$)L_(7Q|k}h{rKIQhv4!JdkCf8~%BZ*Ooh29sbG{#8yD~jww*% zV>elTlAFQmV#Ol;r2BSttY(NwMYLT+hs7oT#=Qa-ov~wx71f>VT)|Ny{QDd}cRWIa zv_YB2ci%A!jBOI$%(zf>5ALOl01Dy5sQ*HORhw1k}8dm%gg(}nSrJqNKl_j5$> zan!@_cM;u!mmJ;3EZ15-(7a9j= z!K9{y2(NtKZQP03e29U|V!!3x_VIuy4sB$skZsoTR!~j6w|z8UrTkv>Nzm%43A0{W zrfidkCR^H1)(G}) zdA*)bj8@Z%@`edPurx4C;5FL<&`t;6&t?(|1sRM7W1gSz4I}+FeUEQBP zSfQT|_Gv)l5$QdJ5+UOIYw0Jk@622jigZq2Q>YxPORy_ln)y>(Zeh3`DYU!@M3s`5 z6zC8*-Cs}Wofp7@4&`aWhiYAB>F8E2a~DP`hBFunK@*MB0EaD9R5_&L{*|2ws0!gj zhHE$5>4%Yk-lERc+kEc`UITs*_&fX5|+<^n}UM(b}c ziT`d{fAazs>ui8l5dodP_w4VqDj+}GTXPPSS|JZDr%iMd{uh=jFI^bQo6ULfdWd7X zPM0QZOSRG>N;V=WAiEYss>iB$DNS7UL!ImF#+%@= zBWj$=)$A=J=^=NwMf4=`uAJirKMl= zj_(Dj;2bINkNx7t`s$Zl&<;9wCFB?L(xV5Xg{v}d#)H`-Dq=ed{VfE7=3bGQBX4Fb z!&auD`RG$#@A>v`XO`P~vuUqYvzFfDG zf&C<_S2$g_HS3fJ=JWvHwj1_~ePZLXZ#9)hv)*t0He3}U_xtl^nNRnWj65c6R^Dv^ zVzS(`X!XpK&KdxGh&HA_6_aWBH1F*D-NRq?I@;`@g1L7RHy4QF3z-#vho--C zufNXJ0O@s$?>~7k;PpuXNlVsj^D~5i;Tg8~N>~S4^Q<}}E4wrT530OkLP@f2)yXS{ zxPzQq7k0D0BdOjSW*V&oSQ z=kV+*I5vAjGd_Yuqex_={Tiai4f2%JWt+@MTJt0I9u*ZGP4bgPgRvE(q+A~4ibIhV z8&PFvoL5CqY(SK(<*kox;ab~_HzK=bdfHia6(W7s?D(RyMG_R>>}4N!`K#QP>Wyo&TzjyBq4862P1_5!w>ZqXt^=+Vbql^rdWSO0Q$()Utvr?{6R9hQ=}r4 zYp)_YQ5sCGoe!B;?P|yu2zS!x&~}@#ua=yVZyl4481gpk4t7o5W{SqeK}TJ%JGZe{ zBy$%7SZ6EO=^L;OLSH5;V$uQ&SXQQr?z@ERQwh2 zX>l`dY2z!{LvjNN^$k^7kjdxlM~|%3Bq| z`exqQ#Yon0aTe3(p9$N_8t`aek&1ADH>lWcj#yeHm%d0$q57Km3Po&6g#w3aV&s^C zRao7F$f)+Mm(BISxRinwRb@R>_fcVBFrQH}z@XWLP2B))CL{+-MY4qOXbFuP0L0sH zP18;hP_P$sA+-x0%y9PtK@8`Pc=sj!`>dMSr+_g_r7LnxlFs?TtiD<9G{cBXiQX6T&Qa%M<3erwVwjUZ7MUOljxu z-!bC3#yuWfOrJ0g?F@$O)aH}y^4=~_Yv!3PD}Qsq!0NLeulXjNxQFYOn8rbP2W{Ac zM&?Ci{H60XYHNywb3W6oE{<)L)L0)DRS4fC*H=N`0h_#3twO!v2$WfDnLN%iqre1o zfK54vqGVQ5)3lbhhf>*_eGlH5M~RE79ClgtJanSZHSk$#use)dk>VTQ2h zO2dbZ1l;Hfjp3;YQv_Yp;^F(Ii^6whYg7eYA?fv?rHYi(-gokX)#9i2CsFl5qD!5$ zvUBTdKrAJ@;HMO5YxO37!OoRj!inzs(fEL(!O}Ol+{&@>JkwKwikrBk3QgTE@jI?c z7=uriPqJvcj_H<$g+il=IL|+@rCJ!Wc!;%vG|qv?{8%JZF@92qu}kVeiT4%>wr4&-6}i)9&NhKrU#z5~E2>>rpx1A++rE$L@wa zstJ`G0~m{KAi5Y?KKqp={pJJRvebBF9-z9RrPx}hXRM26>!TX1-BIX7;1MdLyhY+rjImA)2Xg3 zi;VwNM1-@$$-7dQ2h;bTW<1pqzpKVHN8gEPL`~XP8_ner#^}C1EXSsJp=P6mD4q_M zRMG6Dz+e4*oT42VKae;sz1B4B665E0Bx0v=q-uf-Z^F|Zke~Q*QT$G7b`&ZL8XL+B!?<$L z_)IqnJ*dt-jKgH6ls4~I zn&sDf1FUMF8jfOn$j4v`Yl>UB;hYOPMuCtfw2WYaoVAq0^UqfNYZBANqIZ5s$+LwQ z9T(s;Jc+CN1G#q__hVfpki$~^+X2;~4;w^EEje8k7hDbB)VQ3CpY3u(?5dqM?y4M9 zNOENlNf<~bZ^-uS!|DKbkQ@Ff>~ussHew`5Mf1}kUcZm{T{H7^xAP! zg-uiyhV+%cQaiJjBd z2|LcP)#81(@q(A@8t9!zbaN=g{#8z(dNj0iny^Ufo$$Ay1+_lac7m{G>ks;MAvzQF zV<?v2q=$e%p9+Ko)voIfOZWVI^g>I_de;}e!fyvlQZoxRaCqtGHxOi6|4l3 z24ZVvVut7U>7+agrx;*TJY}tvyo078H9*}(Bk`2u5QsG19!TO*&|yGDJ)Y%&Op_S~ za?xk$QHXY+HuA_(Ged#ekZrAYHvEkKP&0O{(lxTrjziTI6jd7~3EOK;UMa#z>LdS+7hOC)tynCa*gY@b4#MMU4Htz08c3u`5#ffvZ~+8lCWJjl|ihgK}R97a8Z9iQ}K|ybHGM zFYKV?fVlYb(Rll+xE6d#UdSdq7?BH$vHE0vlNBl3KXna$6LMwAU*AY>;$2|@@9o3olkx%E!N3~O%#fT$4H=O^WW)Bz zi8}t{>n7Qrpij3gc(e7-1hDt8R9?qBoBI}TNj3v!`Hsq%Ru*Pr$L zs-~m`94sY$`U(^SlVuMU;b-PIdmbPSd23LvRi{x)kK!<)Lj8}=B4g7ytMLD$>#M_} z-oCv})FX<3g0ysl4Bg!&NTXeam~?j|DUHN{v`P&*bcfUoA&j&P_3k;x-+k^q z@4f%{;NuMCcki{=rLlO;+!bY~)t$Ou2Pm?f>h~jhkx$vw2q@uSa-Q?{qUriQ)~*~1xpIY+AKF3u^`_C?U2?Txv9BgP9(9}^vx2vrC3gnCgo8> zWH=MXT@fa=3w_!{^Eh<*`SKWXo)5`ff8SBAGtQen zbmbyFTNXA63pXrq9`_{|uc+sAko@&Kw?3=u>oKD7_R2u^ti-QyEw}CQ<OSQ=E z-ly|^d>+>E18jc!i=wve_nvgVvHE;p7*lq>`mlKAu~)zk@A|!=hEHCV1(@o@p+FJh zBZ=QX3KXawn@6V8v5Y9(0)r#zdeNs$73Y>?YwYjks1b`W&csBZ6s_qbchC{Qv(0^) zZrO%CS-3FcMd3g~2fE#2%|F+9v7YQP3Bxa-AHuPa`K(V17^d05Z0SCxJA&)b>6*jI zc$sj6Z^(J;erY`_bscwS17(M%5fCY;u0HNfjn4o8WfAL1N!R{;1wfp z{`xO1z&>b;Dy@fqba(>beS|(rjL6QZHi>pGJ>%7Mk#cKI3!v%oBUUFZ!Nz*JM_Fxm zMLA5tPX>?DkUZOW1H5ReV+`%h)gTb&_8vR@0q408LFuX~kZ;r+PIQNj5Bzn!zcI#s zheo1lh>4>7Xjqjy9}EzmNYLJD4O$j31Z@xlM0@JlyF29DBPG89Ov>8~eii)H8P`b+ z9t~lx3|^3!y!=LjaWHduSvU7ewQkwU1sZ8wnFUwGk@Rf1N7fYPi9Gy}gKl-P{kv#f{Z2IXn5sc7?(yGh*cPu1cR?_fwEDNA_4tE@wsTZ%54D(l2V+pj4)V_r2pCa}^FLBxwfA`ZTRP9Ml zdh3i8V@|GlpPy(bJHYZS^^-|sVfWhyz4wnYE2X$9x_Ol25`#TQA!Z+2@(dLRF3Bg3 z@jN9OOKL$2_Z|-g^fE&f!!p9J3m6qnCz>1;d&zcvHW9>fkt-%5Bq1c4uetk~ODUyb zku_PCTt~igA5B7mG(RviZUELqqC1h*^?-*wMy!TVV>>)zg*TEk^F|kJf0k`0?u6Su zb}8idfnUB#slz-U33(%Th7oMSGCR-+xt>kDk7KLhJ|0gxR8o}^_Q1G+h!VnvqeE|AqM$SmhSd?@h3Ag<~U z?p5x02#W0A!Ude{UylqD{t*IQ0}Z9>du-)j0NFKy!=p7H#KkCxngPRrJ&J#qc9}F3 zJ$0;>{FpZ87CxD!j-PG+`T`dszF3$bgEYz9->YFQCv4Vbur*|VYesMjM9LHaE+<$uag&!b{`&-{4ZlvFd=Y? zh7+s&>xlN{Ut8M}#%2rt{9`@E70=gJ8VTZid6!Bh4QXt`9!mgDW zxpi9(E+4jc3FlrR$*`m5upV>1SAi99BQIgBjgzTQS5}DRhtCyWGEEcrRG^<%sO+QL z_iSo#75C&^zkuJ=4oijQNO7NYz^)oS^`1czq!E$%tW3XY6 zyIzlbSZA!jr|9LqzR=R-=s-^F?C6Omdc(xOd+rv@FY&N^fQ+z>Y_4D+Euy?tBCasY zmIGw(Z>uumxgvegQ3;|}&FRt~0G|88pz~RM^=qsp@jZ)nJg5BhqM1=kcAg1Xt!L7J zvIC}CO9?jcB2-kOO~va=e1NUk^l3NmSJn*PSRTxs<|DXa;)80lyN?I_>_#sSsW4pr zsAf#}Yp09m(>s6_;8+MT4hTJt|I$7=+-h@olO`%%0K5cl?D*?I{Vfc_^LL1cp0VRt zX$$?k7SPZQpPyjNZA7FLuI+oRdjU?^9z{8Nm&YmOQ}MUUN5YQNHRRloYhQ_(rFc|O zl<)BF6xRf{UYub@XoDz;Ff93Q3w}rWAe5m%o66J^;AQGG#W+92jH-BH3Oj84rjSeb84`|sonC9U!=&Bn$~b>*aX!hL9g1j zzY?b@&{sM|tXX6&B}>UkpK5zjL%JqLcQvAzV2j(xGBj&V_H2j@*Ci;H=&&g=X+aCe z(qRqSL7WE3haEr@YE#_-yl3JWzPsI=lN71{^qee&(E$tJtad~l8SQ`nOA7RlVGKvC z)RY~3wf(OK^q&umqF*x_Z2zOlFu#5Hw&2gde{{T8_+s>V^P(uWz?CFjpU_i9Ex#_; z=Pf5^Lp45Ejcxb`$MLjAxRUk}d&IqciWT@JF(SOgClb2?4yr7l$n*#a1dcf861#(( zM$(cP#cJmb%F%r?>PtUwS&7FKzz>_uGL=L5f`+WG=?GRinW@=oiF6s3t2;8Zx%69( z6fo&6B(7!k451l!DjD@w_gXW;oX8${|2szueWY%ASao!3{ zVXr7FB{g(z_7`(by-LxiL%5|dfh2yRy6DLDY50q{fbIvZE5|6;>T9;mPMP0+LKFUT)M|4N`7LU`Xd+*wsUp@AK9QLC7=xtCOf`v-0TB#R=3l z0>@uZLS?Bn2ZTPx%QjWVs^hQuupaJpLQ|f;^dCOvs8d|^-#^T8uZwokORW}I9o#W; zNA`$uAXWUbdf_E^ii^|FNHnL+SoMw2>UPMDdIKgmz0$Q=ntIK3p(H=vPbTv<0~8sO z#Vab!^U#OGO7r8D8e1y4;o6kFE=qBc3WryC;cC}am!lmJoy=m?emC}}+0L32zzuMd z@dDAZqeeD138Eg9=;ShLrJmtcHHP#G?2~2+=O!MISW;v^w*U1iK^~x0aeoOT!iK*Na0W@#J_w5yqQ?M<<41#*sIEk&x;fz3o? zCuOi=Htpmrna6S%yu>_3wdgx2&}lfiymlqiLYDVNQONuL{piQOZFdCKx~P3#Y)n<# z|@jzbnI%W_T4>Z~e@EV8Bp;TmF3NNm7Z8S{2nsFSN^H`WmHD&HCe5clXzMIrq zW8XR})Pb^2bJXLFwhIXgFIijQAu;n@d;*KduH43nKtBEB^nmg)~(O0IE+psjh&klfP^3gyytiyCzJ(kM2!HR z$U9c_P*dwcne*74woq0sxxH$$x5Lvg=@VQ{{k5R>db`-|6<|&$4i_j1lSThO?EUL3 z&Nc;GE`Hwk-`ipEF~BU>S?FZ`A(GyBg}-(`oCA33A@>KB9liaso<4TA_NZWm1LL}b z)U}qJD;?=ygqY>vD8!P0aF&9aNTDN0XJx2~Q{(u?*&lShaHzKCS9xf(R`r+yX%#2( zW(Tp@20SK59QA}q$>B|Y2LrS3kwvrwx<%FJQIau-bnboEM0^eCj96WvfjNez7+G+8 zpGsWsLO8n#p4B8?AbE5cLmH)btZgpF%xcfwagX;9-;wU5k7#PoVC@2ARINw!y3U(a zgxU%w3Bop+`^}*PA4N#s6}wn$5jWTElY4fGRB<}A#585i)QMoceB|r0yJQxsl`oyu zb!lhG4@19-2+|8H-~J)Iy+2!1sa)h7mS*FvJ1t7hy^*UkVJw+}p)XzWDnm{bAL~WU z@X}RVBya3e_4gQKHL~xd(Rt8%)q3MmdS7@}$~oFu-b}rkb~|C)w~6_;*ixmu+Z91#4YZw+fd;Q;Vyxsq3JF^l7dGGe&Hhmy#P}aIPK$LTo&o65 zzNVn+1gX%$4uMD;)E3rJziJ#}rye@-k<|KJQ5HA&Bi(X3mHo0QV77GUELru5HM! z#>XqKN0EEb1d`vmj|CoBn=&|uMxY9A7XJ95Gt2;1kj(Th39;8fYMytCn2DiB8TwU9 zK@G^z1M=+q)vSRgRs$6K+X+!5IXw63fQZei1Plp`>)>-m32Y`(@Z0OiqLleAz&j+V zV0}CZT`xh})9AqRxF=3vNihOeoMt-u_w!GvwFTZ$#kn<7;o%=2G%ku#ViG%Tn^MS?>NrF-#EN zq;U7VQW?7z798JhrB=ppGxFw~oAGd;ROm%cC!Y4DhlCk+*#-|>;19$TgJXK-JKL-d zDA21y^mMPBGw7|UGCvw4Cmc#bC(S#9eXsh-{YtX@4hv3AL$s_W=lDoh4P9|`jHO@j zUPyrs}&Sw(Vz4Ve5 z8Y4-vtNZ9-fB1M9HZN%JCR~djZS>YWgLHp#KNIfy9_~-lh{zn;?DR6OFx`HCuq6K7 zEb}R(RXa_0B4g8mM7t)VSCQnl{nK*Ms5+>x%w{6-@6N8N z+g9m8iS~qOSPdD`ZG@_iAwZPWfRVgMz5Za>UHHQD^dLpS(KztNxCtk3PWrb*KD%E7 zl*rGt7Q@+lhOqJZhNxYe7@zm<3)$EgiSc1no4$Jka`q#Qm=)~jaKf7PI1kdE)vFnG zt_nQft=$GIdt6n4-dudn5*VRO)j&OOkVU1u(wF~gg?h#3-3!Usqg6^*JX@?um1`xf zM#Y`3Mnyv#+ij<3XTCZOxrR2Lu>PB6;toGSM`rMA$IzJR9*9fpd-J}>54+z@!pOrD zvcJ9{ZL}=Bi=f9jFrEUPK_AK)Ks+^|fR6eJ0z(=T6HP@6)}C(h<;oOr*^U4Ku%3QA zN_ja%0;en^fBZ}eyRI3Ks%PTTVGDmEY`>T3v(3{h0X}6BaP(%sDwP3`sC&c3`H77m zzwO{h9K<6qOC54>@w-cs2|}7(8FTq2SyD7%FCflPwGL$LPZlYhzR+%ghdQBaZ}h3= znA~#2!!m=6m^WXKS}*)ODBb*P;$0%P-&=U~CG+qvzXQrQAya@+C3w}yeYhhq`6$M) z1Dst;0l`mu?$ACXr-A38Zt*Vu*Z1|==4-XXN<@qpeO1AAL}qp{SYuU-%jj7#LW zx(yIe3Vmh_3akC65q^7(I^c9?x}Kiq$cpltfEND$q&xnI?+(F8+x62jBJ*F}wKef& z)xy(G3g^Cp`Jp?1j&mQ`724dps?nT5X~dPf4;;N@d^ReIe@sC9S;&a_(CTY_@2^+w z--c_an-U#u-243ALnlxI%{*K{vQqUs`TG9;#eNF2Ek{{4GY7g-kF3H;HT6ln++OCl zD-rbHyL~9?GXp|`B1Ofwwf(4WFvi_y{kh_$@{OhIqm7C5nW3L&j!8+W72Vf8#FlZd zoYZd2&zxD6w_~xdGrAmsxGA#q3-^}3^;TJoHFbZzaNJ$$xB*P!U{gFrA!l!sJSsW% zhuNWVe&#Iq__U9LfS&jgcf?C_VUNN{lIT5Z#HtFFpfK@WGUi0WXcGeQOzlSPSx)s> zv7tJv>93olJOZD~B?zywl$hCJ6w3|0o!7+>n=7kVHk-zNHl0!QtLt&C2(XU-ZdE;9 zHLx?AHu)HjsUeFQxGWr(md+~0u&r{GzgC|WMxPQ?qpUmjn|)54GmZq)7w0%6ly}Q? z)Ie#Qhm3h1?b=c=`kGQT;nZ$5M*GxI`02bLMaX48f)yUQZ5r2$jT073Mfy&}X$q~y z+IO$aJMUGF@HhLGJ-fFQsW#0t!joOsYoX;iJijEuN}FG(Ms@%8x~xDIbM>Cp^j~iP z>JOoE8IdH(%A#~C>P&tFdj71x9W=ljmZo0& z;q=js+6DiUd@v4|cN=I5ug1Gg+kN^F9lic0k_MtZ3A$;S0}4){!;*tV_1(>l1d+(J zpxB$L2WoT@14`88ldQ#p_|Yt{dZySEr>yY1!`!22;clGrcCO<7JD^-~&e_Wb!9F-u zmK})7gZr<~(&a`CPwkHzJLiKimh$e$h?~LFoZ|68h0dv}>$J-RJ?~#!UVnTy$-JO$ z`wZS(lTfp!`rlPs(-OE5`;7nl&40zCu@M<~ttO<6eyrocMW9)*|Eb=WuE0n^SY>WE zS)u&w{C=tHP(wpMVrH5?IseCi&q>YwcgoqRBo2!5mPUCq!LMUciNutfm>dfOfwgC} z^tZejH)m%WRrl(ytwn?ulpV)-a??qTT^Uao{t!;P9rti>Z&=m*qLo=}UoMUDZl6$t zpo+8WQF}sD=WzUo2%@Hz=HGQ+jQ1us6;4%TT-YpcC0iMf8_%eoED&QAum2ovxnCo{ zjF@TE7JIS9xKY9|Jk|$ucw*Hwo}A+BcIMfN@LwgDS5I`qm)~?TuWGS+rbVQ@M;+rp z(=94O4YTlIS6t%{4zA&M@EZ3#$i?HAsg5o%m z_wGj?`3+-_%Z?-~TgL21E7<;&8abFA*Su<9S^ci3a5(?2vJwm27J)3SCru>+p;Pqxo+E7cX_VCVPtYmjC(6!Lp3tZ^0h=% z=STN|h;&O4eZvA*?A>|<9bB+Cb}gRz)U?9=_fmDO%rjiMTl8J0jX8UBgW7;t0L+Ca zcJ0B0>wwj9D~WM=&Db7*GvIjS)ML}em4(1{B8B=>qJ z5C*sO1{^)(lfk>yUJ?`Ze(qvV^5Uuf7*T4RS12Fr-$O0{p<{r?(V4%f6-?;)K;=G8 zsZn9pJp601EzS1`2Y7SVPSS~$U8EJr$7Ov!;A^Q6bZi5tK0eXF;#PH` z&DZu10efi+qv7-_1S#QzW_i_}-YiMlCVNjel6u@R^UaCuleQ}WM=M)x2A_J6f9uiJ zGLVhf#5cQ-Yi+_~=>lasI$Kz!JSJ#m1LeZfR@WB%3o{gGLbJsWC+Ua$EV>z!_|P|U zBNf(jsBwk81|K-72(XO=$I&=7-hVLce+f8ea1orf6E|}I(H#Akva?X>D-mleK>Z#V zOS}BR%#%-U8rb|Ot=m%Cc`E0bFFzb380~`)`{sVX@_w}P$Q@R*1epdk>|4;WLE!C?Vt80Cn;PlWQt@w_#vRv~i0bOkHbW%pHTVKKF4vP( zkNAfJGQ`E{$Hn27HNS4lLy068>y+X(9&d$9)5)@tQ^dlXwFjADyQo(+!hc1;H>Q=T zO$6bp0l4_{-vQ2P!VPU6=^du=7>22g3AF%FmxgdZv3lD0w4F z)8Odh&ZwWQ#n0b#WOTNBL?aAbO_IwJ9LmJGkZiJ#B^=Cconl;i{7RM!WBi=Agg5c4 zu?M?p68cRhB}W_yah)77{FMWG_FX4pNk>)zO#~}O-j1l<#<469eU4Q&jMxt(eSPkz zg54Hur_1Icy05T1vPIODGTxOOZEr+B7bw8QzQt*{Fx@ILZsjv?I#`^0WD9eZ8vD}Q zL5DFEmBrk~dXoU^MYj z?Bi5?LWG+1?SUKk52Hc3vDa7xEb(FolCR3TFVZ#d@-3q@7ZpngS`*lQs*?w|Tf|C?Ok&tf5tc zu06~M%>QAlB_s$kLmYR^fB6*9Rp#(1BQK=RyCvDQ;frmL){+x73h(IO-T*sr>ymN6 z>OXfA^4eXI19ib2vg`Td&5cmlDYzT#WlljbOM?|I5{L`Lhl;!-EPvJ?*Lng*<&m7z zqw6_AKvm5%WKxQrXgzlq0ZOJYOY{{S)kU{VMxN_yt0{RnY1tpj12O&B+QvwK6SDk> zmkRR85Xe&s6NC|>zcy3n?2-yiwOB0&5t0qHHb#52aTFy$ z3+tUXl&e^0J%RHpRPYXG-|fDnEO|c*PG7T!sj@4`p)gL=HIuu-BkE_DIp#805m0zSnz%}s`Y`T z2Pbot+PpW&ou7JuIV~HZGWy4R{9niZpAT$laat@fu*IwM0V0^!+s7p2(FW?b z2RVn5Nj7#USCl!mqyCc%FjFJmcCUFQ#!MA^QTPmL z6d+t+t7M#m3BCkpFiLdMC2Cjbh^y!Sj4H7lB#&Epm{P}8u@Jsqmc=ZRxTA06Lw2+q zu@O<)7)fI%LD(f$*Qwy$~C;W|JW)I|rQtGM3ZM)X|Kq9OF_qzJlr z1=M=EP8MNf2W<$;Fi1spcbhAP%~(9!Qm$~KchI+A0QMl}-iHd~JAI!E)pY0Qn~wJo z5+e>QFbOmv68gF+|dXpX#G*EwP zlOkB$9{EDEb_g+Fr?vx-e(Y>grxs%_8(WFGh)_$8jh&sNe59!tQrt$K`@BUEWG+~1 z-22_Z?H^O#dhT84Kx7y}DO|;bf@k8eVe41VQ|P0Iz5z{%o_<*<%Ly*r9-5wgN5(A| zI_a`qk8UWS)j0<$Z#B>t43`#HfKSqYUsiIIrhgMO8nF84q7KmZF8OoI&zh_O)szOq z{U`~*l&OH@o^JPw$Wb!5_8 zMF^w)_j8)_UqQf$Z0l#VYzMl8fqqCfEpIT+z!dxpO!EA%up#ZwDtBbw;KrKTacl~e z0u>Hs#0kkZu}n|@N?~U~3dDL)gKun~{&{dGmHYZ|M?VDs=k$z#W`@l&fA4kQW83~U zMv58_7QvDEi``!Ckw6M3gBkOA82AN2)q@JDQ(8=b6vkaWK3oKG-@dwE0Cp$T_9FK9 zPD+|Vr{U-#@LYAJY6TxeOOs}p>ShI;ZJMv&ko+y}4U|E1#n6XxHkVh$8n|)2(mL=^ ztAjoVU3wg?d?Up=#txU>*H`QGWihY@s0ehZ=z9K<|JOWv1DKRXYZFGArsU%5mvdSF z7b@NM|4?B*J0u=-CjH9oh&<<)iu#8UfJMHx8Fd-?C~X(#G*_JB_KPoXty=hdC7Xrh zwsB}9N8DxA(fqF7z^mHOM6g6-aFDlq@Vhs*n)z)RTk^h|dLaR8WDP2VxQZ9a_Ex++ z_d|g0bo6rG;bhb6^E8xfxbl+<8fU+nuiCc@;4E3wV=_~*_dai=t{?6vd(Lac-R(F1 zOOW0XJOq6vp_XMjR<~*>GTxE+4NDZ!$tB>4>S=sLx(c(_jf->&3FJ@4pX_6Q9hM}( zT)W;1cs_rcuYF;bHceSrJoHw8X}I#~v@9jJx1@qvY@w4zX~hR@X0}uU9GSwOV4+t2 zCNc=@13(AJTYc#)nMIEclt)AOR}zJE$4DBbU1r?RFq(~$&(=AFsGwT z_-B!%w@?yVt;*HLDWuWtLLp)Yda6t;(l52}&Fy0M=TKi+@@}|rsZat(lS};WjVS%B z`r$_<&|#1#(?y0kLzwp42Vs1JPw;I+)=puG(S)E-)8k@L%8oOL4CNzP}nU zh+7dxe@SldH-1qm#taNe!yzZ9C-J@`&t5wZUaK;Pw~5)YWF??AZg!N_hxj0d4u z15JqOjBdY(62cb?Oj)>Xd+t9cD|ZdtKU}_w_r;nCYLau&%r;Hy!c2c_lT6&(t$$Kl zB2lXg+%0Cr;Ulu0)L^c7RqMn0OcE#j2OtCql@o|QEzt&z_uMW{!eUPc!;k?93wEr+ zKVb_4UQ*>d>BS8ls}O$MfC_5#$Nphd;?OEsLHFJbELC!3n1OZ$8%& zP+;eulv$`tIDPa<7AWQzegKS3jX{M|9Pk+BWa(=`66O+EUV|cHv=4XJ)4L}lS?8nA zN-FKm%8vmBZ%w9vPX@`e{yvD1?IZ?jGvl~oXkq1`%(u(Z6S5OX@>`;Zg1)%Rc$t9)ht^&g|mWAx2v<)9YQMT`_` z*88K)|8mlUiGXM({(jHDL^CyZR}zKDoJnj)R0F^L^KUheFK;G!6o#d0brqx)=nzeA z>?kej=vE5CS3~l}vkQJoYAYPkf@NJCxsb#s{lxoruuH0*1=p>{jpyMkl??Y+>s51a zbj!A;%n~oGSN`%XzImhGTyvj-@Lai)<1F@Z*kS(7cpOUK+2@ z=VYbqB7O-6q|ueO#K%HIOjI0899x#EzuD{27DHh(YyMCP#>INL4a3ZAi{;|=1Fh+W z`Y#hG0qs1bWk|cXYH(ALq$5$AuwbgF8^!bsgk`>E>d~b!=FY_5Ld?YnyDfba^9HUy zQ=y#Xx%YpMWEr|;7>o8!D%H`fT8-r*i%}+2ijV<%7nY;KKB{k7d z9Th|_q-)89)DO^CHARADm2&Og(?1H`26~cfw|klb>yK?M?{u>^(5%eGL`WBAq%Q_i zHVS;I-X@66G7}|v(=EhKw!h|kuiIVNlG3O|J95OYr?Ga{IlYmdNjX)Q&v}jCGMnq5 z`==|qFf+`3A@p&=76O*FVLu|=LgS^t1hChsC@aL%9si<$@#gh{v|$irV37g&jeiaG z@Xb?TUq8cKoL9_9rnp7l{IVZ?{DCAr4sijXKY34}?&yDkOHBkZ z#c9@5;8EV@crHZd_!FeqBw!yAUndIk0D=HL3|NS$1|n>pn^JVq1T?WX`GZb)7=$uq z;N+z*7od~U-_ZcnpL~#`NWRy(Nv0?_oelz+rhWJNM?k8!k$(EMC9r3}$R0rZaTpF@ za>PD*7oNTz^x}PlNNKIgGFQlpqwr$D-0-a2h)Y2Kyy>*sp zD)56`634kC;7X8pj5w%9r8+P{Oq0P-pz`!~KmZs20OaG6?%T8V`8)h}7yz59-xtMBy{PUV1`9?cZD6E8y(ERgPwhw(O-mdC#yQcUixKh9 zI9F&aR6&|aq4l`mxvC!3!(O3QqhA(uZE)WQZ(Pmoca7ct)3)A_rV?1 z``J;O^5){Vq9tVrPx$ZTPR0+eLv^w~&ev^f2^WjZf>6P=aN8PT=VO#h%%pK-5d-Cf zMS$tN13TU5gp3v|L*m)WIFbl&gQt>E4tnP1vwLSt_u5J3x9erHZdC3jd)#A5hY58w z#pDuev(GyqX;KC<_T}95#yPsqQAw^*brYUyr+(16JVa&$f1$r!fPt5On18;f;Bck@ zcF-a;761s!p|<%?rlz?`(7|86)Hv})4rlJUQo+(r`MEWH5Op+DCv=i1dK^l>%5_##l!KXl;tAq<9*Tj48>HV%6c8>SradNYS0&R zazQdqrSll^lI1Q7TbEiCYReqy$wu_4K7EiVP|uax?3``xSJ^BD|Hj}RdrCxir4sC( z15(GX7a}z%Y>HG)n^}|earlHGf-EE)w$!25_l8r_EKT{3U|!7u;*omz>tYPO@j^=D zSQ`Q-=2%%3pU19D=-u#%7Ly7kn%Ic&H+X6xHPXCb8O*J_3&>knB+qxE)M3}jd;gA) z1YP#8&xp}?;pB+u=j-`NxFpmO4(TeVJccFEXkMfu-e(IValqay@bEVcJW$7uNHDbtOoHe)$c@ybOX(J5_W(6>9Waz zFtFT>e16`xQOvLSC2Ls-B)yDQn71~Zi^ZJd_-x*m;K$hyg~9jmx8Uc2tU8}2kA*m} zWINNzCbAKfsRC{72slTE_v4>7LJuuE&_ElXic^s{l$ZRqN^iLUM6)AbT$VRNhZn}2 znsz#4n2hq(jJpY3U7;c$`yaAs;M7BOS0M(+TxX|w)R0GOK(|xr5Bn=x`EJUuwV*yx z#AptG7EV-)QxCYC0+nsb{nc=CTgf|Q5C)(cbi!q5*a27;^`x7j2-uyfTY9(VKrnmG zz~jRDy`w{ia#M&YE+BT~2958%9vDu#PuE6&CI;XMep$B7Z2nI$H-PYQXv7R`A556H z8sK&+s=DDRP(#*t`{cP=elNLAJ`a*lElLnxd{1mL!ajun<}7VMGPG^wJo|ho%m%IUk$RV}n=dy*LCRgl3IK0ZmvI>7s`WV;n z2hJKV*89G%8>?k5ji~+95eTHJ(ZbA#!*R6%TDC;T$SZVjHA59E7 z-;WAjIgU8&dp84sxcg33PtrHEC^%&5I$5WTuP9NSB(glM5XKmnywDgmB*Jbq82zxj zB>|%wG{IkWjhcsLdVZ-v#g_zE+&>ho0&F;l^IJ>~2|j2(7}*QI>Ds(wd~ANZn-!DT zKE4~CeKCGY=ksC1?u|Qx;HqoctZ|I60Xl=yhY#(&s&eH8#)Ko};=n%^@4;{(n=4oM zv7UGxr@!u(P2Shj>=};vjlm=lNdttch1^{qF*oCxK}r^6ebBebFZ;;9DBUA)ZPv6W zX7@$z#W)lFczxc4ewMx#44?az(1jePL4?$0ewETBl-U}!<&)CYY?96{yRc5F>gVO6 z(}3Qx@YVdEKDmwW)qF5tB(?tBzhgNA7Mv(>sypCD*1^%6uR?C@~kLC`G#(u zKR~)eLQX6tD=4b``<AuDViDRco*Vf~?#gw$ew?gEE|3pPyC{0=N14rtx}f8(z%MENgixjG zho&)y)F}jOLE*BvbD(k-|Ccd}SO*PI$fB@4ZDY=3Kiv7%z^&VxyH5jhN#%I-yb-?E zp!+Cc7qILM=L0`nQnBFX39xna@_9vjeKw?mvf;$}{7-4m@IlX=hC!?ZZ$F9GU#GQ< z0zBTpPn>SaZhvj4x`l$rLN3R|TAbHrKy6$^K;*(>!A~$|UBN*DH0f^~Y$3pwC8wY? z4lG=NThOz$RiyEATWeYXZWDVH&DJ!Wy|xi>ET0zJf+iB^+p=*B0h{iQnve1Y_emnI z;nI4+f&7}z_!-<|g7u0M=VccKX{etc2mgPi9eCkuTx8VEyZ^IGe^Z}%>1;R07g!&i zTMS>6XR$E-?MMN`H`x`#+`HISDpth7wHWPv)UEj(3rs8`29?t>#`q>Jv-Q4;z3kOY zOXbI9AIi02sG~ZFHita4zSU>OF%F_2gtXQ7SCtoi+im5mKaW3V4vq>9^pc1mXE3O2 zqf>Qsd(uO${ldP?u$|@2$zYCA=(ltw?MU@BiM2`t7gWf1_#*xReXr3KP3kZ+m}=o> zh)4df3HDsci5)hyDN)p>&@a@J(97&j>(#T=wfJ_PV!-@+!+_3!^gJhqm;By=l z{yxL&-CP7*z_kNDyT}-E`rf2n4>t_2zT;qg8b#Nm*8Y@;)Kba<%(d~BqiwZ(6Y{A# z9fhi5i<-rZiBY_Mew*RvV`ELl^YtsKM%JuD4|iQ{)(4c%jBVn~m@(UrW};9O^{>Va zH7rBPm$m%J-|?Xq56c9aMAhi`e!14o)71NwoSZ2shrIehl(+BhwpC>>oG?UsvndhMlx9M#RJQhj;FJ~`EuD_bT)?H@!%P_P zncsZz9XEmX8)dBFGC~Rfz1G6?Bml(m8FWB9x(nxJ=9=Yau}7IBKb zj-Qz2EhjWs)u0JBi?z^JMxzm9+C0=xxXR4gqz=NR(GME{jvg za_6sRRIiY>{+8#wyFXHGgXFUxEhM2JNpax-*{#+-BuONjLW56m2e&Zs+tsogWvc65 zK+!%7!hOv3!_}Nf%zbyX53Kzpfn0brUzM~v;?KgI7Q-zx8;+*KO~VP(7=Rey%px23 zYKW#~u!4to2K%!bsT~9;js>9Nv4Z>BigQv;KyzG1wG?$z%@e*eyzmE8@*ICo6Bg`Z zALm!bBWw0P_{&}1D`wYn9}pD|L@6)hsUcA~8?z}7M(p!+u6ur^A*`lnWJ;rxuZ(OY z-)q_D?-Te&hj;x^^<5n8mf`S!Q1jHdLfg#$E%;yh9JYIdt%Y~uBLDjJpTwC=x<>(a zLH=4n=Y5*$2x!Vp+bi0h8^LP!x*UsU?Ki*E&EtR5Q!nMbUds`m?@#`qccUSOWyqNX zEkamLG}S_~u+DEOZJ{&hQ1(<#%*f)|Ek5|7jNGZ>neEPJ8#@u77%vFH^ zTRO}`Sj4XW`PbP`re7e2ogF;3-9HoOJzGIFshy5? zl{sb3^rSm`+_1p=w^lYwX}>J%505kq8gN%R*~2FJsNI@9YKetGm3h z*Y!bK(e#&=xMR^gaMegeknt9jp8@Z+Fh~(_ymR27n+j4raS3sKhep$bA2&YY=HO*B zAk`&~P?gYDObYxnA7CI~qR(_#_=_8fm zNpaDO4QxJo@c9imabpv!IG2E@E5TJ6BQ~^oRP|tK|AP*4yOsoV^R2_!#b+frE z*)Ws6H7Jm;Zz%KgV2z;olX&B+eu=wOhT*@r$bUZmI*VIA149};&-<6W9q;m4RNQZQ z$M?7Xso?>^Dzr)yr?Al`}z#*V7;%7Ev;$sxtyXAT2!_(2Xf8NcL=%*7QI>i;%L_b@0X4>&gx|5XRx$REf zeS)O%A)?5*$BFV)O#yT{5LKz^!j~z_uPuW*m80MuzeOFwr(Y|i1sR+e^NujFgK%#K zZ7XCCC5lW+F5Gn`Gwv7x;rq2)AQ6_cH+a%RERl4`e)MVYQiUC>KUl=F?K`;nuv-%} z_RpU>l^Z=n+T23MVM9<3;)>lL+f^svos7%HqnVl2W{SOQh_C>Vd!BQmfc&e!WK<(E z<+QxUe|jGw+DPwXo=GMasD0s0eUa77`=+*pZTOd-HBS!Rk4p-b*Hc*_!PhzzM)BG& z8$P_k!WO9Is3z5@KXQeKEeAg&7;$U=A6hN@*E#vY(G_ZxB9tdF6wd zyRtaQ37^(InLWYtyfEx#tUpQ@_tkAY^zS~oSWZ##vU%w2xbEd?Lx1KwzS7QHKWu|6 z95-lM&D1WTN}&uM4}FoFMOkAn&?Q0KXe$n1iSuv@wRk@E5cyj2HRF_-0~y)9ZelHG9-1i_HfFU^uJ7lDMeIY0uB= z@?Za$uY4qB;j4O2-4B^2SV8GNs+B7co_a5!JC@N3Fv(jbmzl3&y zT8cIosFqbg1C)jYj_2!ES$>B+t@K*rWPVUC2+bqe2Vs@I=b&r<8S?F53OW3ifx%U#Zyyd6fA=Bm4@Y%*4O0*bRF;M`44cg2>&oA>4!1$?^O2eO;#)jXly3WN6=aT`|ChK`)f zA4v#$;5_KA;Acq*^H=?^aB93i!f|H^bJr536&65#Oe0hWXQ{3qLFMF?En@>*|v zj(~!x!%8!RV6A1kp&CRu$@Zomj5`TfSDDjIYMuT^3qa;$COeh-sRK)vd!2h+Pdyh& zM_4^!!*|1Qk5|}!t1*3#Q--=bEz{<*Cj<6yH^qUoE{@0k$L}M{!+RYf6oMH?-ZZ~Z z^q835V?SAj7=JEDjk+=-lBL_8T0W7c4I=b zKFGRD8fgSK%St*f%~{r3Cdr^RS^`-)1+TTXCM?I(c`GO2Y7|ZIhi+c*1uR)d&jq7u z&nOvhpx@QpyA@A@hj;~AHmfI=z(9{y^37l7id&VHN5y+Fb8yjHq^ z+}V31&3nNSY%*O1J@ki}i4ya(ja^(^6xGbWfPdgnUXvtI|` zTk7HCdbAg}0N!I2!9V`6AxTZg)`?GmJIGcHOt}GG6OHo;0xxKn@gVnxNYDIO$u8k< zG_1s`2KJlQo}eW!ye3Uy?glN;Oc$OI{7YIO|h)vH@+2pn+T0R++1MfJBvqld@ z<&SIbjoIKV^GP^e{qvI=jdF0^RDrc%I+z2niii@i@-`yb&5}Uumy{te6}9a^oxT9C zT`y1o@Gt50J_)jC8?!7HOuTiEU*@C>NF^srp0?9BG^OVlsk#@v0zbcOvmYxlMMZ|u zcJwU-xYq=hw6Nkgj~fZiW^f3}aXYibqQ)Yy&2a+Wz|d>FI+Nc*GS5go99)F`H)&h` zS*7w&{>uIy9tawBSnHt7o#amaSBl~v(G4@m%Y9|-`A^>Zf87wz-rkG0fRYpWEBm~? zhS&DbYaa?;adpl!A73>8LAH}fr3y_Wd|}AzK-j#2F0-L0bYmD#^z*cgv(QRCsQ4WjhkkZf!o8}1c_ zWEoB>SH>fakV@Iv0(GDJSW44}WlS|a?V34F(ET6Z@22p-=txH?W=*tL#=apjR$X)s zyQGV{bXy~TM~RO{b1=)W)#!yi8582*`*O~4C9N`_1gd8@5CO?T$}>Qf>PT>nu%sA%joRo z_@%if!yv`9gdWXx8~oN=NpNvxOo*O%9GooyUd$)2XtBwL)viaki8)w$U9r*%{o@Yh-#{F&@jk` zMhMOeQ3f&xX7M}d9RqDFjYL-SB7nm-Lq2y?zV5@%3BwlfAT~iuVC?nnw%L!=7lXb5 z`?)kJfe%3A4J6y%M4dJjAUZT&K8z&di@4>ebdEz0A%xr$JCa5H>bQ;O{PSs>9$Gzqf zOSK^HzoZE*jtQa0Y@Qfp&=cq2GGQtX9SAT0N?$cz!0Jt&d$gX+?M_d$2sA~FZ}N0e z@-9gjIpD(mFna{PKd!FgjO{ZyM!QR-twKMUVQrrz4p^)2YM14vqD`UykFGb5hkAX( z$6J*Wp+qFI<=8S~i7d&MgCr$0wqh8JCA+bPQnG~_#xk-*W$ea2Xt57t9oe(*WZ(Hc zbLxD)-*dj__lH*#;ni!N_x;@WbzS#$Z7TC$C@jW&zWxuo1Yoh>F%zo@~Z$-C@&hGgbDn>)xB^7Ik%be8o8UVE{LSkv2fo@67F1OweejIMcz)SnU~D7y>;z60uv>k@p^u# zH~C;v&i4Gm%x-k8h-ATWF$W&U!tkndd(rh}4-RFm808dCiOl2bby=P)^B)JxB2k^6Hdq9j43rj% z*-Y|{2XMeSl|Hf8Kq=0pFd+uNcDc*j0f0naxEDwp9iUb#|&+ge%)jp3K2?&In9Cu#}(bZFcR1b_~nh3oW2|Na$th zx|*ojsmNYDO<4uv@FM69>ZyuW}ksTWTv_*FP_*-bakLyPk+B|h%A4b zBvQGz4q5s6^@YChKK*sQiYH``v*6VDo;^jNTK2_I^%~hfi)Tx8WT(@pOE9q!4q!$f z2LzjEk5y{emkBDWeT(J{uP1c@zJKSVHd9FC^ikv04Q+(MurX-vB~^z9_kd@mHWj`K z^3hJez*}D=x?P@Q3f9TTMI6wYWFxb+u7?bhjncb7bt{yS;PmGf=∾y4?+>M!PCY_1$&QSJG&K8k8OKgPP2eQ2}R1h=HRWzNA|6syy%R$Hzx<*2uB<(%-4IP+kf>ZN&|>N+v`d?un2j?(%b z>3EcF1rO9b6RB?&x9v8Jw}0Z2x4Dm(QGBSN`&PZWLevZp!RCk<+LyjZi*>DlCSz+H zFpr|h1jisWL)Wz?gDYL7Wz)dgQv|5~dta&pw*it<_2CmP*7?(HpBM-CZ?DDA^fsBS z=bOY+%{-dGf>8J9J?Q6mULX0X328)GX8@gIsCq$J(I&cAoVk?KKKo$aDW3!XOP*e; zi-#A#A=oF5=^Iiyap^vleoS5@dVdFEq(<;T`Nk~$OJ+xhJCt~MRpzKz?nOCd;mN<& zYX8RVe^rD%D$ibZY)S5)#iP39EC7i&fsZBaQQ6--`=euNx+jRRuy3b!Ln>A$A3G39 zr|=are-6Ku!q&h1p4Jd(A^eE8Jngo$P~qs3;g#mwC2YZ}rSoxjJ3YTjpZ-I>Ab?JB zFMR=T5K%jam1}i}eLZR2Ssh?*AHP2oA1FZTLZ>&M6V`QZ?)=8g=A_70OW?@Fh9M}S z=j@8U-Ko>70m#@G1CAuU*C- z6lNP|`{?fC_w?E83yUJNB4r=)tk7FlTb^qV_21wWI#}aK?af|k>;r1cWbw$QTT&M) z_Ob~Je_AewM$3r7kfdzX=TBtMaDC1t=lhBT>lbo5h;>_H_PFmYa+B-FF!}3ly4hZe z@jnN3Ly{~tH=OJV3)~5*6xa<}9_wr)FT{f71|!zb=}ckgYy1*N9lv=ycXzJkSk}Q@ zZ!}sJa_LMhJ;9)Z4#s#O*`%P85np>S23X&uHz-Z9ix8-JENeu^%O*sjGZ6hMD09noomm@kX(cXx8-; z1-d?gQK^P!f$g-ie-(&rTi8^CwjfGwl58%pCBBY)jpQ|x*L01Z$*8P*iT#6<~hvHS^1)wh$ zL)^b#UT-!2AvOG>zMpjtUm*W1)nm93Smk+6Xcm%vCvOZ1f*6lGu|kN6JW!75%OlpG zpe`GhxmfprH|B&to82#DaaHN&exITwbYy5}$BI4Tw=eZy?EL>=NV*_k;Yjn9G3$Sy z`Sl`{`>vI*%JoZc+)*|K!Yl(#nFQf=$Ex1)<=4l$YT$#&SB@FiTs z^mUqP2$_o%1mHCS!$Vrwnm9gRbmn)H)O#MM@nRt&>=6QoJfng4UVqk&o#8sZ-ad9k z=+UZ5rDQU1gyzLNt-CUqte`WU^brqR>X^}gib~=jcY1#Wt*Evrn&&?b_C?EKIdaXk z%@Esi(O(H`y~`CAh1Dn>3Z!YZo?Th{$zs=++qUzx-tUvOEC*%nS3_9@)n}VCw~T~% zK9|&pl;6op$RHzHtXINAw~i6GH*^F|HolF@z8L5HRL47895QCyHBi#^=qTQ6}@LO%#w~!G9wp*BD}9P7#;In zs}NadV~!M6J5lClxqc?m%V}WNr_-=_d-v0$^tjDjzY@X zbnKwtU51wSRwf1~J>VISYo1ebYOeQg-leM2Z;9*6|#)QEZ3;N`}{CL&{c7`KwGFO9kc0S_x#F%)f?eTLG z!|m(F(EbmSRCgK%DB(5&W5Ci%kG9Vc5a&>B8AUR1o}ae* zf#uCTH(+Y@cqp<24Ycl4-|oBL%G+9s@H4ujur=lc`hBc9SI0K>)>R(r3YGmpq}oy4 zo-&2byz4gv=$jR%ve|N}^JF+HyS%2Qqzjdy8$aXV5h&v;6t1oG=b#aEW)oBqBM@De zgwR~;qqf%=W*xb<3Tt6HUPv@6dHY=*l8x9ZPmQq~ko{t}3x2&Y@Es_Uv4FH2fW;%o zb=d*sbU7fxE8D_byzBVLJjx9@;i>urQwHQYjUeZ_vy*R+{m5RQe`Woy|I84u`C5!q z-~f7salo87%#h2sfErOaY4e)^DF>`HY(G}hIy&8I^^8u`DvN$3rA*!OPmD}gox*w^ zj!}Ic^czHw4xMn1O0W#tMF!t)ik%(`ssuGC1~d=u-^{5(0T}ZTKn^iv-}M5KfgdKn z*h5{ecLyejZ&e!+x00;>vbp~kM+gEN!1LehF>L=Y!k_l$O?KB-G5(SN`MzG2Kg_)M zh2D5IM0WLb@iJLi?Gl`el~zg8$a?v4uq8C5O7xZtOM#l1qY&G7qo=CtE)i=Kq=ON# z&4Qi~F-!A(-}B}2FWZUTFp5<-NuirDWt?4%xeqLoO$MbxEBmo(F zozBs_dW)U)iQ+09J2S$(P9{V{<=rDbS+u=2EodsN9;( zZOeh>Sp0VaYueN#-;Qlq^z5AiJH{ROYETkN{ zow~s~Bh-6g^|}Sgkn(4G5OUNqYcHGZna-y_UqE5Zv*bB%yl7L4#4Tyu&z$lqy(70m zpL1!eRO)u_-ro4F3Dln6>4xh;aidwZ+uYh9Ytm|YKo&-}XdXVYk84^JV`Yrqv1$%gUetk@ z@|gc&jJm1=*@Bt#SE1w!+6rcN71!S8aOlH+ZdA7&J~k8~?N2?-6+oT7VB34sl9n2C z(*rsJ&jTGAR=0fHVQ1Cr}R?*UM$Bm4U8BTaUA)$GvhT zaU7T`r<4c9h#L3yzDFx|oq?r}X%|>F&#;Mx)vTpU6F+)`A~=V zx^SGcOC(!6;2ei>)9RT$Z@bok_pL7A5yY@GS6mLL*yJs*R?s$G8(9Ks?3ky zB7x70uMgJASzo@nJn?R0`e!Tu*L6=9K)vOa7=Jeg|GVo$1rKl4N^L(c1HNSu)(2C6 zt8MxJIBeIcRP}*pqj^wlp_P5T@a`M_G*vk6shrUFw@1X8KISRW_}~~bpa>C6D%lX1 zu_NMq*xvM2-4)$9i;I!kG*L^sX z8D#CB6H1goFU=%9C^obye$WBO6`NHj5l+5-f45>>CefXc(EMr`HCw|}sHyh}-s0t| zI&mmo%Yg8;>9Bmd=q)=PJTB|-Uc%*+m8XP^6^_z7t=G{y6n-_QM0mO&ESeq_k++^P zSK44$Ew-z$$}E`aJFF0 za3Vo2)dl7N+2}zC^@Y3J+w>geJ&nDAh3of0Q8X|J#>j6iAq)Df7^#kQ!U~lIwYq!T zw~WybFT0O#->obK^G;yF|5scQoE z8vWMSsh9iJPX(0G&MyOy~S+W_?P5PS%o+TaRq<9^b{MgHgjzrAih5fnbJwpC;i2*w+!P#I%{ zRdjHU_qdP+=<+!nFE?)hGc4fHmEV}VC|pF<@T2?s1~K7XV69J|y6}IdJc)FqUgD`J z*r2b9SPeA?d5djO-!^ki1}?|Z5R=s_-PfrYp6Ht!fh9I-M+x3+DnlZ6p-3OF;#ESO z5_p2{%&t-+WY9BuN$^ArvPF4q62xwxl-gYuO%Jxa^MQR&DY+j=^U@=A^#p30yG{Lk z(9@cxGg4p`e>eSMN0||5SYlyhP5`AW`!je~;}tu})U6@vGO?&}xJM8v%SvY@TPeG% z_h|zn*hZ~;7VIsEfPfm)&FkZgUA?^uU&HR&WmNDxzu%e04=km?)oWEEAD_><3>j~K zsOW{R;~xDLU~o9_5b=%jK3-Lkot~I*^xMBL^}hiB+C=co*UBGb{b$4f^yX>*k7*U#?NgRJH1{mF7ma=?(nhA=B}YgMplwgK1mNy zo4(Bj{45wB1Ih4GHFP{IaBwrl^L@>>gkttEVG}(;XIwYz7`hJKO0bwZH!j+Zvl?it z5C+~IwK!OiQB}ey%#!o8-F>E-{)*bvEJ| zM#;W%fmgDVFfUD7!m7wPNx8t6V0ytthZ@%QSAX&{Hec+T$HQJc+(1;oM;6BeO^UY@ zKwuc;GQz}?t{T7gp_#>Ihp<*57lUvnH(s*)WSJ)dtlS$!qz#njq1z+*?0~ z9mF*u&B-duWLu!Q9;a@Lqc3*Rxeic=co4@S3~(yA=^ho1 zmhYh59-;<(*n~WrL9i-)Sq7^lg`@{+H$JNA53T~bFNTVoef?%7fhkbY2JUKLnqLlg zuSk7AUAZ7nol=1weg-g%ml4@Y>FklYaiO&J&-THd`BiFmqT21vDN6@7+CZzM&7?=<}+J)@oJOY4=@-10hj}J zPvKW*nvM4jO-=p9Ir){($Hlv%ZOcN3skC1YSSTi`Q_;1U?urRXD(=ww49f_cZk7k<_Vis94hT#9%`k)Y8;aI!@i7!g=w_UCA0 zP-^EpizNLb%6;Qy<&(~9!)N(;z88{c(|T0Yo_?C-agM0HdEs;q^vJ0mKgyi{=m*uN z)d~}G8RwsY$IGM(Oz;V<;vZ^ob*P&DCU$HlrMJ7ZKi^)&7rPtjV{@-K$L?<}Kwh@g z?d9IoN#RHcVSunO<0H(Zq%OlmyKkr+>#=CTLyJ#E zgpve9W_p*6c~+2zIY{at85@V7W*yF|*>e_?$|g%~lnI!UX(@OTfi8R1lN=Xeu+}Xb z0blB#?UoI6O2~3fbG6J}9xxR0uC4DZn+hF(hv0kTVa^1oq2~Vd_WLvXfZ_15;`7?8 zHP|(Nl67*lJowM2-&zm`j^GYwaCzB_SCZbRB6Z*vHJ!$a8-Z>%+4`9LL6CGs@ccXK)$;G?Qj+7Hs4;T_;LIzju#q2&Ggxw^xH?K)a*NeniqYj zXB(`i-`(FEIv~ETMo+iD9|!mTyblqKz3Z%tV0r&J=>=5@{Nyg;&`lb?bLx;&T=c!M zqhLXGHP=sw=PMngg|&H>O1e>H1@gQ$RlrudN$A@U?6q}6_>BdF?2?gF?yOW4&U1ED zPzU|zWO7tby_oM($gT9FLi+ft2-cM}0r`&=^L7qGR4B0&kO@Hq?NSx<6fYV_-Gff@ zM&$3hr*! z>tgn;&^HG82n`3j2#RX0j)WMtiE3!pMD5Be%X!U{H-JRqYcF@381}?GFhUv!F>-9{ zm)PhAz#L)$3g~RPgZZ3lBB~A$;7UDZUht%)6R#%TkoOidzaK4yUoTRJPEf zYHwYLx+&;zeRT_%DH*-YanAx^0MX|u9oA$yD#m5xC!i75LMy^$CRkU{7n$d|~f5I`%VU* z9O)!kVxH9ZG3#WWK@)_w;pBAT6M`k(qu}Ay66;W>{oV?0h+^a(>L{&xtQ1{{cb_Dd z&m-pCbsN32p8QOW)+}(mwq4;XU+kr-yRi9Z%2kKDEHyvwes9Xi|G#vUUz^v)M` z9 z0pBr9|CLC2@>ix=8)%Z%08BEfX1=W;D&xHLz5+8eQ3&?9@IeS7EHxo|$*hAHwYTit zp3nfvs(%t}M%wG>P4gn4)jQQYeJ@OOX5gh0qDAbi1K(J2LVQh#8D8I2ct3GVM#^tI zKq*f+W~q5dUN4Ylkac4z7T&1u7yJ}W!3nNhjlZEU5{-&6f0$WHsm$zA*&3ZUrC{Ib zE8FZ6IjrT`>j}}~$^|wSc6+Lml9~I?70^ipN=}Z!M!=j@V`mj8v^d9vb@f0&qc6z% zZubj!#}OF1wu=nVv$%yYN|u81ZIflXHog-d=0%aLGn$YNFLTX(_>T2`W}!(pky}_O zN*@c%A=+Mo5Ie2gM|f=&fRNSmfJe{YSBxnX#Ai)^w7mU&zsYnzpNi-||MnRneABES zsMQE^^VT{-z=rD%6;W=wh{8wOI0!pjNsR`QnM}D^7}=?ux(eO|HgjP0tHZv%4rF`E+=oS1ITJjPX(7y$B6hXXMiEBkyLFi1P=ln zkc^?4;)#4Y00F-S_0?nG_WXkiX|cIO{*c-H5$H8ib7P^z-0CD%I9z=^43Tzo?`JPg z++1x7@P~IPK!d~EuvBg*H*>U$>(vJ6oTjPFP`3YaJZ%tA)Drk(A5 z#e=2G2afetmOz{!Ol9(O!tZJ)UpEGxh05A0{JO?6)bOTp@VsUz+~L20NXY}5ri^lK zdxVqA{?=stglOk$i)s}w-uXU6_^q`-(zAN4XX}4{f(4d+ubI~h`H0lBqdaoq3TJF@twFI=^1m4ht66VRF0o;T*$sqz8qE2or= zOCp)t<#WqSFRu1BJn5LXE(nZM3Lyo?JyzD_tS?2Lxi@xwbw$Eg@JRUJYBt)65otHC zii)ZID3f;;JK1VMoRTX0iJ^m{Q&CSX^4}z$F1G06Xv(eqq=@@YVb)#w4skVIU|1dE zO!`0rIQRL+EktZrZ^J19PmqYrqprkIr6`U-#3WV^ZFyQUnvTdXUunbxr@Q2lz!puP zI9j3h{z;YQ+&0WTpW;}N@A}dMt)>FYtCZNuZ85?E{7@J{Qb^M{*Zsi3k0Qxy=q{qi zF`#d`XR&E8Tj6He(xQ|UoK~7B;a5-kD(j4BN$O1qWus712}x7(J+eFolZG=9-8oW! zEcv?qAg_H#?YufKX5E_7ES0Y)HXYX_$;Awn;kU+onXu|~E7XB|TwqGBM-5|k1tEVw=Ya9p6 zy2Y%y&eV<{P}#ku0#O3%mseeZ-1I!thn%8bn?U<*+Mz+P%%ebhIuw5m%kWyT8W`gD zFefOEvXNQK)~G!#TeA^R!i@tiUTx8@%@u?oz0uQ=Cm?Narn6D+IzYPVW2Nlit5z9{ z4QGE)MLE><^v2UBKLhUAX%)Qy`IP>*Eaecw1wdyfp>qJH4~0YOJd(_`t+F_IIipm-u%yslW3i%PMgHu^GZ|pl=mf$r?lJFXOhwvtIS5l^X=jcMeQ}qh1HNK{ zY>*o-j(hFe)mU1O=Yjj-U@u5#gwo3Ak${s?<*R^7_wIBG0na;A-cwuMm`q(DPwR zxZ^1DinD$Tf+Xg$_meb&msiTE!uTlXxWl|`ko*o+!C6WMkS z-z&1QFh@2tneTKSFEX4_Da^18z3vj$fxg#sBkeFBgWuNjIQIN=ubV#{)qenas3sNt zqVBJ^Lg%U7WIuE%Hyh~ZN)o`0I=%+Bb4KlP=8}&_Jb;z@n~pm-vifxh_z2X-fV&vg z!Jh1W_@VR>YOX?4-{5-Do6?}U$Bl0UEdani&1UBk%){>L!Y+ve#0x9xh)YW!5Y2d6 zvWs+6SJ(7Seby!>OCsu0b=6N_?mzw4f&BUH%FUqk_LrZghjx<;u3lsR=jivBOJnR^ zV9qF>{KfzC|G?j0FFcNoi6OtH9Yr7ceYh14+u5sDjrBrjK&P4fqWy?~V1gOrMM4zg zXUa1t+y}i^JUv%=AXi(jXtHQns${dMfeoJy3}4WS%JjWsg2$iimTtH$W4;xu5Nu0c zGhXpZ0r3rNwA)tnm}?XpY4iLP=in8I=u}VPZ_l$Wnu4OkwWL-PV2^|h8^T}U+P=VB zthYs9B??%kBME0Kr1(vCB;1>>z2pB-Z1%D5!+2yuRox)O;I*YRNnzx@^V5BfE)IuF za6;;Pk=ZRR(zpj+iZ|ve19Xn~9qnqovaHf>`&xNctf?8|`-*g>VrhyDE&WP1kj#2- zGV3LB*uzdvb^+7FxS`v0n3zp8Fq}1AFpPYuTWfbV{eDk>$oj7BAPjXUPV=^NDhb}uPo31L1g(Yq{Caw%_rDV1psney@w@Nr=?{@OzBV+}} zlT@m54(s;%5xVBp`EfwJ3X{<-&W%(eftngG@a3Gk4?W_nBUN7nX#7vRV2$Yysv`;2 zAc^GLaBc)RwK-BHVV>z6)Rqek#>XD$tLiSr!>?*UoAp4HxZ1wYu37YRi|kiPijx#V zOWT9G7kly;0Aq{t!C`+3kYHD>AZ5YA)@{T@VlaSHb+>P{L|)1Q^9%ngKS|sdVhkztl}~3$(X&JH9+WeR?k?d{>wd;~8L_ zuE(6Dnj&uVm5Jt0^B|VIA{Nb}lO0)BUMb|b(x(Z`!9!C~4xQPwZMVJuv@_g5s(55qJpGFyRGSd`tpmOmq})n)O$ zm|_#j9an(JjAfN#N;_w`!nk=JO~|P5pD&|9IO#yZzAx<i}q;;+^#uKJCyV7@EH-)J`q z-SD<6IiK|yt5`9a+xFt2P?iB9LC57xM6ChfC&CHQp6I(O{p4*ayqvIwQTz0ob7x?nR*I}o?cCKay; z+7gi2!}OO5iOMr9gO2$vxRZ%^Fj!cCW<9=8uGVRUL2>tq%xEaIG5H0j?5BaKpZip0 zj00fK)Crd*Kt9AZ*<;l1!3n6DFl6ruXllpjMt$iXMD#Kt$0y4c@E)nX3>SY%<`4NR zG3D-8KJ#OE^m^}|$bS^aL;g}!lgiQO<^KZIYW=GNj-PJa-CB9!X;SIEbv^Wm@oyPT z_2`SI{WrGksKao&PP{$~4n1Mcf5g)U_oRvGji;HZ4V1}HJ*8?2En`pSO;*j-m|sJ2 zJr3q9xOH8mefEm%#`e1OXiy1x07HaZ7hyA@>t|aq2W_vwU%0vF=1|~R&?_4&UyT?kZQO0EbJN1ZE_v92S2y+s=vSd-TeumvSBx^h2*RTR7Qj{0a&Tpi4 zn=}nR?(%+h-UkOe=uP;b`Yvg}i?}ur#p8*KaxPRyJ1?)tYwfQ2_VF#0oq1)bd1mR^ za;H5)maAJnXC)4Hbikpxt*?C56JGEEDN@m6D!%t3ZFb7=Fa0r@omi?Lf1lctMs$dksoT^ddkOpyVVSP;AZoE z{n_0?W7*`}p5g7JTg4X=*YJCU%TQLL{Ww;r+@xZUl4(cq;aXWi2&x_!Uy}^3Vra3? zr5?6J$M~iE0mYp~6_AdD4HmYkn|}$*sHipa{w~-~aerr*3V0R&Jb{}B7)PKu#Q?Kq zmv@QL!DP@N8iReVhK&G|l-=d+pA-W`P?lf*fxg%F}&$_#-_em{`?P90WPi zwsNbZj```O%Qf-|y&kJ*bx@9Ru%_-91Po9d>r2}GR#qaPCQc3q2aR(G=VcjP9^BY= z7WuKRvP&Rj)CnDbzg^|ZVY7$)b|4#Jl^~qG=swUWqdr?>#-(kQZ@xr21$PRlv+JUT zG2<6gU7IAMYI$Z;@q#^a-7FoNjw5`)wIB%{^ecR#}%hgewh};%?~7 zze=MYpGBi;l9g8Ras;`SLp~xIeKM01wqv2mzKk|mgspQrWH$GHPff^b7KitFh7PiY ztZ+rlDWB zT6{V!g9sDI8ZkXD#0~6B=;O8dK6VT4Z0GpmvDu+Mr+iMQ8 z`X;zO@6DQQl;s0J^e^dSn8*(|aI)%-`h)nf13@Yf*kqlNNU<&fyN;U42gG$kjtKDJ z5~7OgGTeTi{e6bzhznrDy7;nRSw+vR=c~7e8U|c@*S+n-)Iiz&H;J$k88W z4dK^FyL$Y<$zE5&CPi9P1sSZg<|B-=#YXTQYCaVK9nqDsm+Ef4^}uXA3>Kk!s&BfK zC66=W`3S%EF}R1du4I_f#LWxCf1GwLu)O3cGFAMcAo5IDGc%+5h?%^Ol*VK8V2sfu z;Rh`rI)SZ#v+{+rSd-Qy%qddtN0J0X!Xdu#9D<{#$c3QAa24*DMoM2-+T4kf4-kY zVQ{n+mu4@lH)#=!3z@irdZ;;P!PzJ!3oJ(%%?-`Mfo>bU1H{@ul^N51yc&@NWO zd3LgYVA-~6t**C2I4}pU9kq6E;evNM!RgEB)yeFYTBzNKv%`I9i^SI1v?f^aPpq7pXFu4Yj7D1KLKpVS!fr9`HnbcB;yH$1$VfDz#SeV{5?d;_BG z^8P~URJINP)+>(~6t`%6vz#y~mw_Bvr3zz_jd>v1_zeJ_7MElmhh79+5VJqfth}Lm zzf+YmL{BBcNE|Ag%_hXA2FTRzOAWYmm?V1C2$|>tHNIEurb|lLi{CJi>|whnWlzQz z4{NPDntT7>Skmv;KUMfZBPU>k5EuJLk@EmcW>pDcrj5P<$If^B-Z5i7(*8>_>M|8o zRRN@9i0^IX2K}70IHnCW4uQthLX-lXnY2@SVWbo*2fx`csnVlPR!2^ zrZwv&V7&f}ld<==Yb(m$KJ+DWsT@=ZH_uqmV=(;sulxLQsKYXF5YW7f)YMiy=#5akk$~1Z(U2<8x(M zERjCVwgP?Oq4D?=?Xg@?)66>rycf|vt-eZgf2tC#&}JkYNjNualgggS*=_T5!Z>r1 zAh%ZAfNog{BL!b!4zHK^0xyp8iTMt*Jfqt`b{${QAsMZ*GWZ_Fb$YGAX- zE2ZVO*}NhJq8HR0pJfR1GW@O=?>XNjqdsZ3KL~->hkktpC1@lemt%J})>X3i^nG&I z#hR}1e6rJ>Zyj2}s?;WIsS=WK$&)hD?>`60?O1#_Oqaq=URRKysWhdiZlU@8R?F-p z&n&gns2Pos^W-E>-hn86k^hgk|Im?W|EK%q%v)ZEq95`Zk?9>N+$;@B7!0gOdO<<&yY^6#&j&3Xu-P} zhv3OY9>%zHy6}|g(3kQ&GZRY&BTDjqvu7sRxQqB$vS`h+il25|=dq+1xxg))-(@xg z;Dn%^K_ZgF$O*gENUrF~c_OSITKjr*1o=_Kp#o#j&lGK0L%~#+s`3hr+6C7c_O`3B zh?x0Yd~rTC?ny6uIY)uF3A*Y4m9%KaUV^xQ22Y-y4+xegzB?#nNMr?2L##I@G2Jv!BqWsMkZbSF-*eg>;`7=6A z`@>ztxN4JdZart>QsFn9Y|~fVr+21TwB}sG9lgNhHDRAML~*=!K?XBdnS;8U3A-8Q zqiN!~VYxonD^KYO&}KsH5|ZIzNeUT7lsPy(V$?DbJ6$lx$zR9i{M6>oI-OC7LsO;0 z8nUad7ac#MK3D&;ruVX3K{!S-wOQC>UKOB!TaE9oul?xu>(mdMh3x zZC={-2F#7HF9vAEH4VRXp|8eu9T0~4C-2;N%kfh>Av6{@`g6ju=XzE)3sS0YZ)S$^ zlOsCRvOO+6^+7D{Povuf&-BmrF z!vOV6^^+65F^Tc2js_R$VN+SbjD;Ire-hs`7l~4X}rMcAd+9Iox@-I z4&27Q86jl2BfqX|+`hD-r42u}-k(m`PhFZKDi;j1s%*uI!&%d0Fw!~ow(s;!Q+mG& zPbn;FX>1q-*-!>^oTbX8DZQ^+(gq6#OT`+K;Af&HR4XjAI9erh-E<^4Gx?iJ9DT;} zU_or>6iZ_~6mwQl;TDMnDO>0zb`?X4Hv}aGNsnAwYk4hu!p5ukVMw8d7GJlQ--QaR zw%i+!`i+EyH|hVMt_5%ok04C1#U|cut$>R~jU^WPA7vv{8h`+y?#Egc|M|tInm571 zYfOIR)xXcgotyhn;k(B#ZEYpM3L58(-F*rjt>w-JD@^+eUVrmNeLH*n%y{IKPO(5U zXC1vj{NGvt4-V|UN4T)c%CSJU&h6%Q-h0`>(ULiM?!ytPA<;Z}C`)>b)peoo#ou~u zmv5=@!LK%rK6d0|KmFte{6lsfe--)VsqtNTM~B+9z`n>UFl)-^A!I0%O~9;Gv&YKt z=As{lwxWWN*Bzjyv2_5mbv6 zuPe}!g_B>iFv~m4YS}g`LY#jt%g8!=(P(YUD4>utiLNcp3Eeicl6qBmAJf`IHZ;;g zOiE;uZ2}{n{&D;TNwKBp>IsR|tLrwE?N(7$UxsMa1wwT}}1Lnc7_dnm9IV{(fAh3z}EPT2Wnj$lxpD7sp7S=wc#D3VrJgPrG zowdK1k21Ch2~d?eq4VtaX!_z+sX-25-CP!fd#3Q#I`WbMk7FL6)}-6?0n|*hIeZ^q z6V>CE_8JxkD|}f*Ojx)QKru7L2DZhuI(&?AZgsHR(7Ip3P?{c`L%M`AF_~|5Rjt!F zoh^g$P9ms@MA}ctI0&@a=9hj&XKIU=B!wFE*#x2^O>321L_NzP%^mVVx$fucQ*~MJ(=apJeOE>Rs7QMDkY8L1x`wdr8G} z=Pjuy&Wg&8cCFLPFUAG?_d8fuNVqvHE>o;hHEuafQA}FP*6{_8EF;=yO)f$;3erWr zUWaPV({X9bZX&tS>-7sgWfP5xP0nYEk>iN_URhAVx?DHR1|=8Cf3=U)S)tIP2{*cB z{Ix_z&&BbV;pM+nuhj@%u7j+k35L}0hS=l(aqm>CaDnp?eD-+g;eXy+kB<2>jtut{ zDAE4?`FBSTmuB2aQXp<<%DPzV;gd}Vm%h2)YmH;vy(a*FR2v|TQ&+qCu`iM?pWK8L z)-_af;qUwT&F$Xi1vYH1!JA$?ta>eiCr(FA0)Kq^-gPbW>|k4Y1v#?qzK7$iS$?%$ zLjq!W;f-MF+i}_m8Fi@Rn-oXSJM8$*@426!cSjK1)`*yCe*yg*+D@5$+;osYcuY#) zRUOHYP?&#s8XwEsfWFYXL!XpZIS7g3^-Iq-JF{c-SlBJ?y66(SF)w8b7UUd7LS_N~ zJe&j*2((o%NRB0R8YebEF2kR<;+~Z2cfg;m7IqVb<$gi;aX&8rM}J=~{tb7Yn$#;*ov4o4m4DjMLbLL(GC=)z{&~FM4CJMSRsJpof2$?#9a~l8 z|FWj|{EYPPjDn{c?}~Cf`8icjoqa!5@ZgOWPyVx^RdnW$Ow+u{ceNwIn+7Vk_i}%d znVtT0KDt$8Z^ndGGvI*JU4vMjz+sPttU&{e8#G?+8irm(sBQ32#@#B6NTfQ*?cw5b zWVb4yU*}tiPGPq{+stb(N8OmLczfAtPVy^V2)1Q0wC#mNkriXux}joW>2YYVN)cpZ zm37QJwFKmam%-HoCP-ynKCa>qKXqxq`to-~mINp+d3O~G23>Y~Z3AcR3L^}Tcx=Jm zgupjNXA6wT*d?psq6aZ@ZPy3b!yZ5iw7lTB?pyN|`9A2oU=pKCzbihi=hhqbNTpi{ zN^&=`g)oICVp5Ckf8QQX?$+|L3l3_NyB{P0_pXn*?b+P zALY10ktk*!e^<pCsi3n{(+vL>jHe`-cD%r<#y7cW9L)XtXAyktr%0Tq4cPt7- zlB_7#YAmkxT{d)6k|Duf6 zkNZ~9WB;a~{_ioxoGOViCv(XEYcKZLu~jiEG1E#8(*F*@HrEd}r~A!$g$*h%q zSb8kcdhGDAJAUbIKm~)DO6Ya4&R}7;5sHh65yFTENs9JVv1qH&aFU|-6?sZlYri=p zPp+rPI4>B^K792OH39LQ&;MS==N9MVwhJF?PH}ID?==&}Jl3eSO0}He3cHme!f zG0^T^ww!O1xGQ|IDtpvvAYVzj%JrGqw~^7%X@-?zCu67lV?PNca8#-kE>7@ib}>A3 z5I*Z++q=uhv)nB{SI5=R>&!!MLJwiKG zvQ$7i1KEiWFSmFdH6SKj)Q$+OikL88YJzmB(U!4@Gh3@|^$*7TG6&9ih4=>jw+<206{^+}+F{4;zGm+{KsrsxxFa2{=_`CRT9vfRc)M^Yo zT*VL+|G!;M{`bC8Up$(lPNH)LsRI8j8~(h9r#BUmEeQ~>^|`U=-x5gh4b@lfLdSk; z$$8Ej1o&-NsOG9}Z1AiTaNQ|nl7|6rdd-B0<)wQib!6C;k~nk_^3++{yQm=)^sMG1 zd)mpA@f6iIZ$JG_BrOAU#8XC8UK$&-I@F0{v7SM9sl+q7+XyvA*5j4e7xO~Q)JE%_ zVnSas7`JozZ#UxECcfu9y$_vOvY0@uH3wZJpr-v`f5Y}ZQ`Qc=zW|-O6GP@ZR<5jESOTxrmgEVK`}~ry@L?siHU5Nkmxc^3?zL&{ zt)kS85)47CZ*1325*F#2)8vbi&pC5DrPsJA`U8LQSS*K7gn0SF8oX`9J6o)WkU{>l z-`8zmdhd@<4kjY4`~p23nE3{1j8yQeiMKL~ciA~F%;G1yxU4DM&P0o5gLK9$8LO@V z$_i6)vi8;dtx>kl!cJvl#Vko(_6P}BBn|6osO3yzRzAP{paSg1P!mV@q>TcyP2Odq zHAuW|5S8ejPcRL~HcP^1Bk-4d(|Ek>J7svuCKR+UA2T`aXiC&oYoK)em%icGU*Dwo zojBx|5rF@F9sJ$={i^y@x2!y5IOo6Ezdxvz;OmE@(|?zbKyFY6rhrdY-Ud7XwJj`U z)_F^cWl;5sF|KGKNfjk>O9h>?X1Np^{QTVKwmshZOxmeJu4d?|$l$1-S;mgnHU#4C;OF)QKcdC>Zq&3!jgmo7GDKddxee2Cl9vm?JXPg$?n###rW!g0uXnDsfm z%syR{CO5=h<2MS#Ba)y-R$mx(x$NAO?U!j^{H+BNK9b~Mbw(ZUA}3scvl1meGjML1 zb%XBQo}Wx0ZFdXf6|%f54)!5yd}88IX1~iEm)e*9f8KhzKW|pjh3r>68UAuB=u*i) z9{#^?j9RJ-LTwP#^IvGxZ6+XiZi*4pQ3(Dmj|AUtbF7g2MZfviy@w$S1}9l)TMN0O zpGUCeacZZlwrXq`PF&6p3(ccZezQ>&=kkOd^GKfQ5QTIx-%NgH2rAL=a&2{!P^!lJ zZ1q@fCw<6snMxD-h7xoDY;k_ki5o$8&m~(NX;*Rc$V%_~mAHMzH%S>vdkQ=l*;>7e;1orI|LS1p3$Dt+}@Ajh(0cuoK*)X;``3 zyVhSIbkBQZwxfB3qSeIe2j7XlqFddI6OVn(jk7gy8bENfA(pioI+;{m*>t@zp{Xtj zpZnAll7xW>N$xh`g4s22__TA8#p*_%Lx+>dx)G+*rv9CIr@qWp*CKWZA%Hs%Pke@r zwMe9WGE?oIs6x0=^>_mf6%E#hw`PM|1}@eh)pI?XS<7Pu3$uFsg1jeWUVf%qNSf`7 zm`H7FvVwNSrnywxm?@)OjNCjm0L1`vEv^w@lTfeJ(&FyKNe%9qn3sEEM|mEczGm7! zE81GBJ~)ws3J}DlE9E!=36vhDJ;X_V9hfTiQh2zV#h0}k6Qe6hdi1Rc zdJ}?RvuLp$?}tSXMtcP(UBWHiGPMIMRQ8b$@30^QGGsc3vmz%Sj1qNDH7 zk?z{@Z{RtW=t#}@y3mYU()?qVh(&bSRlX~r?w2r?U`(1=%VSl90Uw?9@M-TGSFIaL zRVd|L471qzuE0inOiZ>5>1R|;jE&^Yl0^Gwa|x!dsuG#>1YinrC}Zb?#YiL4d@oh}so)C26B|TUNmZw2Y=;OuE_- zjtn&t4b{`0E!~UnR9qjh=#i8wM9r7cJGG~^vfcaSeM?0)ZkL-9yyB&4E!Ou8Qxck+ z!;3_rXVe#3P`J6oFi{>Y&)gn@VYGBE<2x}UOim6fZt+^OEetRBeqF^h5>?l-)f(Sg zWZHARrK3ZdZn`Wtn-hp$28jZtt3H2S@P|?OFZ?UzE8LSfLM))(rsv6QI(=9J_>MB? zq=7SxbJCc}U)J9bo^k#>kRS0_#CLb+b;V*xpBm%OchB)3Ap!*w#92kgQ(AWH(!$up z>LOCFTzbc?jVmw@r@l=&D$|+yDw6(bp}WxOqcHy4kJN^jn42nMx%4m0sC}J&dL5x& zeRh!UHr$54THD(F+ZVM=t+^Y!V15uj{n(q;>@#vIj~y=Ecm!BZyFYgqnUB-YW2rXSE@RsyShLh_|qrm zc6BJ_3NltO6~8I59zL9Ihf!al7$}by2ZaQKe-^oyJDd_g&}~FrTC{&na!91+-gJ ze0WRFR;7d6M`{wGxHvk?&|C%}%lW2pRcbOkv|bZ`^d%HyQ|DQ~854{P;I*zimcZ3j^{;Y}zj+ zHs?dfgou# zRc~Pv=Q_P)Uc<%h9QiTjp&whLI|kkvpm48*Nf%?6Jf}`qT;xr|y-!{+>HavQixnDL zu|Cf1If&J`eZiC~Jeh$Rkzq-HFMq|ovnozB*ixmNdtFAZr6|LEZysOC-FEY0dlw7G zyFF;tOP3cLF+m`F%1cg>UECr^p)nQ%0!lN;gPm51DecYWcB-#dd$$J(pf9vl8B@Za zH!-c3oT1abj>F~hyM^Knc^Lg7ygGCUt+whx>6UAu{v}0&da6**^&}r5H_rLi7a`mB z<3~1rI(z;xCx23Pttshh&C=+CK?Li+fL_RT|8T-JyOZv)DWdRH;pXL&4}ZPk&;3h+ zs4xf^>Nmac&v`3rK>4cA%8w~nDH8Ld=&MfrSuO{T=3C7LUmBG9zz=$=_+jsp^vhydLu6n&{h$W@@^$wK7Ek+*HBqpe_TU6Dr+81r zR!QuJ<3^}!0ia15BIK9svyQ|$A=~3#qLp}9yA(tU!$!Wx-u#=L*htcK>h;%bs$j7r77Lt#~{OP(AU>M$k!*%7h`Y@^fWc=D!D3B+=>*4 z_#5O?_E@!~pDz4M$hrmS{PnFflS)e>d(K&ww%_YEPkNDCBA-=tU%c2^ARntzZngP)A&{JI8awF$z~S0IXQp?!fHPF^3;s& ztW|nc6~&vmlYFy6E^R;uwzD`YT?<_9Q09;K0=Tvs^-Rl_MTQ3^!{1?VAXqL)pP(w% z9t^XvLwR)s9PzRjCo_{ZO%p6bz$A_68e3K#>FGi6IZkgZzS8DLn>L@5om|hBE&P<} zj~lHVk?Q|+zQvjo5-79lX$OTu;K^p^opcJH6QB*P&C&_x(}8yd#~A}xs$GaT)`{oD zoF(n2L{BZ`KffB>YRXnKo4hOCuhNhn7+Y`&k#n=p#j?j1!RQb^9g(B#SY_Vsg^~qH zaeAO269^ezb660{m6MvU#@LH6XFdVcJqruYZrRFPePzNOmO<2Wd7ZB+;Ohi>`k~E_LENz1}q4sktuW2@RTPc1%2bi>oA98O9B}ZevibWF}*EoFVDW1Su>4WRhq$o{iW=X zCblh$oP6DHmL}u1l0VC|7Tpz<4w}w_l9>sm396fc!p=_;G!oy9Kg-ENd1f`%Z&);( z??<-r!TBRbEu4~Us(J_vVZ*pan2LII9>k}?TVF-mU;;W&J`cySq@r}EtKc*|R7Q?! z%cj>vp*4MmMblaqH=75RrlEHr$R)^H1Cy3(XloBn-Hht$#?Ccll$se(xP{%K+pmVK-R|pusf{)y+rzt4 z0y~M61;_ryORbU&xE)lIg+2e^Hfl8Abp#{#5EGJfc0YX*en)6J>@h$x+I^0ZGsa!n zY1@SX-ryqOq)00yr6%M#S!28C*`RG~KK6p? zw%oV${tm@HAJdCin0WZ499v+!bc;p<_~^3SgUk1YBg>{J7R}iUsQB1LZT0g@COQl3 z#&9FW;88UCL1Ad9WPnffO?rty5vS)4QTg^_3X3OQo!gMeeyhszB(>L0Z=o1Wjg2(y zMOW_CU=%xQB3#U_3_3PsFB!J^)T-s-H-?SUJylGj9&c6$r zl_?CLl}qh&lk}go9AkR1Ws7W!mkb}v#!{5LL9?!E*vfYB~N&qfS45SYxOyrR%tGf9LUfX&~v*A1D&5L~3opvfU>A)*z7E zHJo%g(w-|C`}iT~JR&KhntvqsOqzj-oYtZp#U03SI|!dk#gI;pJ}u}uJGnv+RPkP* z@DZ)kRxkZ~Sg3t8i-HmMxFy@Yl!rWQIeT4q6HFE1UHtJqz4W#~y-~RO35%@?8)?9; zpJY&M<36J;r~G(UEl<$YhC*Y!{3XxnWzb@$AyORtBrCq;l@8wmc(#D&Q~{N(K=eNp~y>)7QS3|36msUKVDwehJ zrB%*A)L>~7uA7oYvJC6#Q4(Mi_}EXjtxfHBo2auAFyXhU?aNj9$m$}FxASA9Y7 zl&Rb(5sI3S=-T9+?tfplyb|pII5RLzzJl13!db+fZ*A)Hj55*DOkIvpU6}blg|&hr zg}I13pBZ+kOLZmw>%^gmR*l&21BohS{x>k62hNzO)5-rA%-_9?>>I-O|N5yQ2pIrv zQhPg1vZgo0ge@Af)7}eJhTMaMz^*Uy4r{g@L6a}=ciK-#Cf(Vl;^T~5%z!U;QS|IBAhF_W2a4@wgr1MZElYH|^_6;DAV!OiRTfF!P`Q{>rq9C+ znwxZqr$s|N(!x%pR<^IxQH+YZG=S~-yX}qmbd8!P!hs-LlQk>Zhlmw#paN^Jxrc>@ zruBx>gp78WjYksBq`fQlwfd}wq2ru|zUmU^e4Na((lX0lw0Py#j3`1}`*R*vMM*5` zER*WTTMgm3W+s@&S|m)ID4#fzx?9FtMOHLh2-RG{11U@K7f0C>2f@~b6RG%QAZq*@ zbDVt-CM0_h1nZSLYDVQchUTydy#Tuoo1OHD50v3F$K=ShSQREQosuPkwSows@L{GB zy=!0%sqT2l!;z=vyFhs&k^K_4mMuV?ETA9H&GSfahU9-Y2mT^Y2ab?9pHlpDHvGG} zP^&-z951Cvbe=m5bOna*$CI8X?QhR2(x87@77D(kMGvBf!Uq>n>kuG`OIfr1{JGc_~q3U{DFm{Z+oz~q@4k0Fqow<;FvBmE?seg7A@G!Jt*I39FOi^ zHfS&oX#-7wdzq2mlJ->?muh9FHH@wLK$ccwII?UM^&Du`_FTq_+6Z=^fwrn$w&mU@ zW-YGc?cFvx{oT67c8~#Z@*2iRy$$c64@4lIT>RJ$LZ$0Ui@Jza*qKvBWcN*85H*J4 zD@%&zwWa2blJTzDYe4gyq!78r?&T)YsR+V3(dC9HBrW4>z&lXsGE}|`TY@168CP;~ zK5ZWTTGXElPpy5jtkEFm=C^c@g?>QfIAF@#LVIAm3I3gO5v)KkL$XVnQ<`CJbXfop zXtZ{|zmvzeQ|{X*39wG*M&$Di(P!E6m&u@%LrmR-Gs!DixmHPkTTr%3@m-!uQEe*q zZ~E~Xg+a->?Gk$l4PO9%f5>Fcsd9qfU4` zlaAR{Triuvkc6ZUA&spfGge{Vw!`y|^O0gHE`zsyMRt)BRv3d8X5K%7S2QAPAS&5ddEWi`?dc#VC0cJqNkwz7>Sp) zPd8|U!e90^vL~lmP{>1`7!QPSBW%RLc^cxGL^Z26(+G6b*RUrKK@FUa)X@e{#pEt! zzo*!{pnZ>PwS*MW8nBz(CJiR-*PRtryqaiWaNh>R38Lj3OlH;M>P3#<;%aCk_Y~de zCXEK~Ft!bv44sBEQrh-@jib_ET6r?sC)QHdGIM8x_S5j`8Xe{#$ZVRV`)CfuwE64> z*GRFoPbTBhQmPx^w<@64s`5dTE>6=%3v!$i7w&0xIsQ`-Ci-l0=7cq3XaEL-U=Trx zcP&%PlaX;oCiciGtE$gS`eGq9=+MRxEgrkdhBU7z+s|d!?^j-?%yWkmuC;yr6auMH zF+q98-M9M`E)SF4co2o4|3WS(;OV0+=Xi3}yEBe!8`_9|Cy3fALScP{tWF1N&Z zgDKpr0{pkzND9^Q0og)C=%pZD1;C%PW@yE~Ll0*&4e5wXVtir{$WW2}s54kw-0m`& zZHI_?vng$ehQy>MpS^Rn<)`gcL(uW)(!@{6t1QgAW|*7rdwU`Ew^td%jEi5Je%$Jg z3@N92VEAFqoqu%)l=kFecB>Pf_cwF>|148&Op-=WF^_o=Se{NIu}j5wR3y*^g`0kB{H-UQru_|Z28 zH#Z;tTO9gjppl*`oxs2!*$YByBoCL^e`{>-&v{F1RKFm-^_4XCjLB2ZyjG?Q&rvpa zYeaPL%guVlB2?k^)p5kV^4af)dr+6%Ujmhz@Bq7C|;5p z#fs!(^S#NH{LGnmt}muiqS;l%r6xMCQqO{l%xw3wWr2ouehKNE zRLs~)-zI#GL&shcYtW4*ILCOt$79Rq*KqjK;^NUnd67AH7eP36u(cPcx{u|{tL z`;k4)6qCL)jl(UW0`^5f&iEfTnNZojX6D$)UU(=H5D?XmO8!Rh`8fac%3YGT28}=f z@QIoXjZ5DT2a@G|GpT%F1^rfJBiOuRnNB;di5Fq+bysJ?CZjJ3atf?|Oas+vQ{}Ud zKx-ZwtGmaZwO*DfYN~4S;$QFU1yTi4H_c3P@hz_kF8DIK5d&Pej8uf)HKscK%~>>I z8EdRpPv63SmHC5y0M+vjXfA|L5Mrw36z%mUUn`)(Jd}gVqQJf|!(p|;6)7&_l;;@O zdqXEJ+I4kDwBejgz)5L)@9RF#uRn#_JNh7J+-N3vq+F3=F-hp`aNMO@aLKZ$iIU`q zkyzA=%%R%}+OwR6)IM@8n%GU;rs%*VTTUEk&6Os9VWU#nG=gv$3!*2^NIh7}{3Q6U zrLW2e5jDz!vJLC;eCsj@S&JqFPlvNb4XFb59GV)4_}+dd=Wm8pbVh{XI+qG3l7*{6 z)*0JAi{OIR4QF=HTW!K&4IEn#%SYg<#iWb9>3 z5ZlM>JV5q8VEYcmG}XV3zx$WNm3=P$2#E8h9wR-hZr7I5c97ije1D;AbM#k5``B}& z8ik0y!ocGhH%s2xkLD{SFy;$v=+b#xC?zs3y%LXzNG;awd`n+eP>WK2$ooZs>A2ZP z*P%ir5j%7`nU)=Hg4bwurxsg}bG`84%El;#7XYR?#}tYT;Eg&l7v1p?U^|$h%{wi>3d5*16y)M`l2<` z?vfhQnW}npe_{z79b=SMZk2U)D*DENGPy*?|#! zfl#p;xk+Q`<4vs1mPU(5OU0GK;jVK9ONH}a zP`ptaN%yZ=gh#=&=T&XCJ5W@1CCQsk0wo`Z`>#|L#@-FF=DE_jjg`FJ0)c6OM!Q{Z zGu<#>ZV0ZL=(Lya!y~~BVqEa!&n@p-)DyAoU>W?y;>e9JNxJlgu6;uFD%+mhMGwr| zLmADtcL^J<(vx^KkC6p0(FFOQ&CLGrfBoh6fwp#Gj0(XG{spXqNBo$8{GjvlzkH=X zE*ipWyR&ox*Q&g>mK~TWrGE8A%%iXpr7r&=gm2U%CJE5@aaSzAcqgR@b8cr-sKFF{ zWNiU!8W!j;w6&W7N2*8Q?t8u zOw`C!a3yJ1ny;XJjC1@oH=<-&?S!4=u9q=Sa@Ym-5K>8ge|`tp1) z=CR!$TheA(RA6(CvsOMf_Y^d&Jw%KjkyzXQ4 z&7L9$GVNpbh{oYL5Bqc4z#iSz%1`k?u~Bq9mwUi%qiH*N9zM_9i7JtAZ8H3vuK+#S z5j`#XMb}x&Ed6<3;kdg?j;k0qOcLMIZfMkg5;|wq2b+Sne0}?g<@!Xcm)}V>_i61SDp%X?fU$rZUg6dHokDUjrIV z+%YZ{mb#|!{!bna5WCQ&{rJq=PZ*;}e?Nltd#0*TV??`mUSi4Q3GF|27+p}S#5=&# zU6LdrLXe@-4H+EkM@e*Tsifs@#6;A zrqaXwBaYF}B&M|IV{qJ?#IvfNwp5BT(-f*{y-Tr0?nWXbaBE4%IN@%`{FljBu+d_o znJ#=~=1y98Kf@B_amnuJyBn$Qi{~!fcjIbmYsleg0%GOM<&%6F*8B2R#qQm*paPlL zvPFHm+i>~Y>HP5Ji`&qG!g*Vu-a$XIf!>kdlUUMC-3BdAI6Dm|sxAe&i*Q;8T3?q3 z!lmf9?FV&Rw(7u~sQ6nDC*7*h<$?EQaaC>1A#AE2j3( zX~fxV0oc$94^cG7Mb4?yz9;ezHBXw*-%Qook#{= zRUa;#EKn}W`qb5}*|H+#x`W5`sxMun4v`*^4$-M)kHT6!et!pyS{O zZnUTNV&M2D1mk}u)V{c{33`FUJG#ovc-=!%L_ItCb0UsSC@6mLw*3^8mA8ycdF7dP zj9kC|R#aI=ac=R)_U6y6Ws2bt`TDnKjJ{#RuSUv!S=eualjOOS{JR@9B7d`E~LN0T8<6oXhBEdW6Z!7nw&_3Qy^1l9 zj2H*?PpK}d&#CzU)lDqUMV=*k0_iR^dW`|QhVYpxvA3;EU0b#kA%!bF_FW&hm7pPOw>up@s|hRAMfIcyNI9@UzJ!~&k}DyA3Dn8VQ`rJrZ} zQDEyi`{(-dQt$O0nq$2tB__-A^=n>M4cV8^_6-@Nin;NBdDiN=*sjr6&YBo^f$l}7 zc=z%}X_XM}4Zg5TfO1O)!ON+| z7ARq|u!N0wnE|&dK9`R(I!7F7d}ZVY`jT;TCd~1}Q-*a7sj*|C6Bn2MD=0RUYCp%8 z%5wF89U1?r7jp`^On2Y+22vxu1{xkCeg#Y8$dxQu<`ZJ@--#F@T}W!Xq!T`PKhCBo zNDHN(5K5*mqgG{h?1^qN<5de?vq_4}2A3J1tz`o}cS37>M6l;o;w+J|)=^Z}fvqA1 zF;|i-8pwhtTiSL8uqLpekx9mi_&ZsQzVnVK%W8R=JS4xR^>avgt8H!~ zCf*iR?-8AK7n)NdG>Q_~C_b5SF}%EpCbCgTNN?v%lt+$stc1J6ES%*N8Jq`Ov){LRWVvctWDj|$H{`gq>9-ah=XI2M zIe=txZjIAIhO$-sdlwFlSrecZ&UVJT{3KE=93?kKAVT4uYW^a8IAtrY3kiv!px_ed$uoQx8w}N zgei!!xG|VM1Jg$TX&ZDb90V)jR|xe6<7;#;OVV2jV`6Xv8MgS~G6q zRYDx99z0(XUx9H-w=j-hrLRTU4omGh;f>U#O^=O^yMK}TJ6BZ)X5Y+T9TV&W$#w96 zL!Q{YuwA8x#o#dmUN6{S?fu=Dft|26$O0IdxaWk8F5A)?Ha;EERZt9>XIH<*X>Ga? zw|A)`I)4pXvVux18TOoiIm2yK2uH_>Vf(%0S{`PfIp676BT&z4l1zTnimLA%^EJk3 zae7Rr<4~*Z7p4f&5={s`ca=I9=IRIsq9JO_y(&YUPR}vp6v1z2u*}Lji@;w z5vJ*D;_IOjF1GU?^I5jcjeC=64BnFt?`7>vmM_d!s4_eeWDmrxm;XK-gmLy!4s>w%3EW1rfkympICrK2@@IQTx+5Z3TA89 zVIt4a(9zY<>jD?}YC0_y*onq4qV zUD|T6mml zjVC>ebRSszI z2jF_C{0a*TIebpNfa>Y#)#Eb-_Koku zO&t3+1XSu+k+xsH4u4aJ%&J9PB&{H>j)S$K&}>OPatP&=8tZ5I*YIqioHzRL?9w9M zbk4Obek;S}sevqHRzCXX2t5ZgN|7jGXWwaA<_!y~8yILq6{b&KL8c{%#iNynGAh3o z^kf$!JX=?sIFVJWsHAV5;C#4!{`{R92;Q|B1hgrgo4_c0%y;K-IuV(r_nPa5hdjbv zvGhAJD=!qqJtt~(XNQ;`3AqimwV?-gQ7g9D`ZYcB?2w|G%Vwr_Hcr?%pgw2B%boH> z5?4<}-DV;T`yxLtAi6B!1gwcTkwj%Zw>m5DI;d&9K-GH*pW`w&PHf7R7Ooz0(XUV` zNhq8Dfk({Pi1%RVC0?cv1zEiG!9UB{K$6UE` zg%#fa;wMJj^~)DROxlyWrt6*Ye3=^p^k6EU+yVZSORsI+_*8G|zd&{TBIT^-39__P z<4e&2Ha6O;aSVDY`F53_Cjxao*oaJ_V*duxXIsyTwzJyjTy(bE^J>#rbGhDW$(Gk? znzUUcZpDEs%QYdEdGMkA3-o>FU%mRfP7uA4uQ--eh9)pOv5vx9SBhU%B(24GSs#%^ z&V{yV!SY>)_P9T7BOBu=4IZ%C%k8P>x_tJYkMDMj0;-YSZ7l?z| zsZk9vo%!l62mw}va4_$ei)9--zxWQS=>x+UCCUW|bcpu4$?ik(N z{OU$5dwE#=;DF2J1z)iAeTH23A+4F**NHO=Uifv(m==3qc`6XH$r z8O0AS@>gAxpx5Xq$LN^Cj-I|P4ah|+PUDKDg^T5L27xd|%L$BK%MLn`ApF*AcO`W@ zKUXVJ6lFqW%LUoep^{xjAv*$o(FrB~{5 zJQeKyrO?!7SY%IUrrLbyn?%{wor;lxP^}wbdOMXbcPm&-p4fim^Vz+zmsm{OA&ffj z75)nrKQHo(Nh~cb-F#G?U0l|13;ydcB~SIk9M@G9pu+jqr%4C&D{;baIc#XjVlhW? zAnlfj+IQ9J;W-i=KUVFN^!Hhp-W_xOh)qbMn>fMHcyS?3zvhj8?yQ!97u(YakNacY zx6kptwV}$5Eeu)6c>ED84-TPhM2KScvV^h0YcXfM1wi!egyQ0drL)(H66?gl0!K_u zClh1xU8GCoZg*O>l%pwUOl^?fEYij%?K;n$wVr!f3!dc@G?4b#(~zXBlD+l8BDMFaf+PRVYVvgF$q zoi%N_meRFt?XCH&czI?n@?ppr&IujoX<^_wm@{|<{8+FVb%nk_WbI^!l$*}V@E5EE zw6Y6lC%gNC5b2a7voJcWoGm$D%4IiF$0Jk>-)?wydZ65xu7Ji?>ud{}Rl3@kGEe_@wlSah{3T2> zrSmyvb@afSTTf;3GF;|t9CpCWQ{^RNm1w;DS=fd|PMO`Ce2ul5weqU;Te9r%g;<^Z z-S{$NcpC(_=S4LRlTY4@Uh%Qr!{#I=0SOvXU8m`V#)ZxfeEAk*Tmr`!Tf^IM%n)hy zoPLxgVzS?`35VO=c=F-}2QwsDTAjqP_=IIJOk00l=1PvNTSgKixK0YZP-co&85gdT zrz8tbPX6YLPL@yJG5DKUDMsH;JIv70=_p)C_1ngNXN~uKk7;UZLS3M@8h6{%lIZ>s zgZqS4J5NDA-CHHu$$4O>kb!6@h#nkvT&u&EN@XWInOVM%C}HPEJ{zQW<}uPk`ME)5 z``-GnPp5{~2dLB{F7FYzb38H?{pJ`J`P5fz5_!TkW?go>sK`om-Mi^AK7JVYxE~av z;`Zii+1e8*bL8{fL@`sr9A=%XS97W6(!!x$mk{iDg$2Gjb}VuT8}753swRo3NZ0oq z-U!ccJSRfPDGTw|p(UKIIg?JS+(kLDyIPp+jodR*zt@I)b`&(Qw~k*Ulv=Kz^m#us zA34*2gHPb#L(zm>*1bqP66FKKSM!dUO()f?dpJGwW^rm;fzH;f58P}t-foRqUoev1 zDmQ>7&-Ybfi5Ps_z{OKtlPGlZ^E$r4!n8d>BK)zWNy(Qh!z=>rX!{L8-wh_)u%}J~ z^Keh8?#Hs80)xfe2kOF4n&@RcvViSREbE{<*jxZ268wNt7$}yka_txG+`}JRezH3` z+t$|BzF|Q2!+!9aJ0?y!x7n-*J)0~R=xckosI15;zvN-=m@%XXxAJ)wm1PWej>CZ= z&gI2>Fhxa+iJVK9&!24(q<~xu7SHuv^skN*J$BalIMqj9RE_7G6#rSe!362&>tU=G z>-GgUa(Pq~G!nXg{g(b@<#u6A42`}s$P!$J>4*!VM^yVnsevQ<8B}vto-lyAtJc_s zMRw;5y3qG@xGMsILc<}l_5y3hU-nkBOS^sMa3Q1XGJ=SwHmf*uVl^PfN4B;0 zoZWXv0!z>uAhhz-t4gZg&D9KJ2w;-22-_TCor|%AocBB=tuzrat^`*#s1HR~sHk0Ula z1e9Xe9rukb^u~s(Z6_W_#Ga9Duwr-E>nEu{bGcJn$SXrAKmXpAW}cE{3*E7Pka7eu zq=D^+a-oo0JC$Hx-D`eb=#EtHM_FGHA#NZTWwg;r9X&I?n^CjVBB&_mY5ZxW%jV*Z zzFoOM=dsre>enRu+>?{z2qcgNV4hzsgu&h&K5n-iE65CcgAAz zp0^JXLv-Eihm;urqx^_;7yYHaxhc|OMwEZy%a4m}I-2hMcJ$D1?>LjcxXQXaiS@J8 z`5_&-NFV{OFU(3f*tA!-EPcga?c#E&>RVD5^OJz-m|M9EBm>dK6`` zCyO$XWiiKcmo5!MNvW0ZtuqN}PY#+b#u742RaIl?D{`my;=R+*q695^T;r}szV{ig zjcTTPKb3^Gb>Hf-au2jmA&c+2rT+{uJCcfg_GRs?Z0-n-IUyA4%V2oLcVuTEDojv)Dvi@l76NfRQ7JtcaGfN8vbA8EVQ1OeeOmj6MiT^`i zHWwbcx~@-eT<^R`>T;P_{V*yG_T&(%kh8o6JSz3C^wKW;BF?-hl5zGO0Re%4m6esm z+4q`1(WrtW;<5CFTdAl;oA*h2!?E4kmx&$3LF)mWfq>AqMKQufM|Qhuw~*slR{rLT z4R<8@-=(bmk;KtnZFgou^~uHuv_9r>ad^DJE>Z~GBX1|g^y!Z%}to2n2o6YM)H*ZJS-$7Ez4&$ z$^X(T&RB+U`@jh`y??aj102~VC3RW#=!-wRQ$^(D(^lFAzYC89@>j234Lnf2)yR|0 z^uriZ<)%6IAnW3saGf;b1bp;^0EpacrtL}~M=;y4dz$3rNPE82;|uS+2R2NIVZnp; zAw8zlp3u*BwjFg{#0~HH^{PDxvWlC7Jzo?1q_GgmK z3y=H|q(Yn#!8cn{zuY;ABRu$0i$4ZV8cCH{Tz?tt<2%AI!7Ly@Qjz~v!hRD~{pRs_ z!|3qTST)0Y`<^{+4%Q=ak55M*2c?@3xCw$6qkV1tRQA~UxVW}Hm3xPx5_Z@Ko9fu1 z_>_P=tx?}>6!zg@{7G;S(R5A#K!)A8QLQ08>~Y!j(VCN%)+byqY7NC)JvBI@D2RtD zD1Dj3sgME^`Oz%|YaIFDpG=f|)=yz;s45{LL3R=)aG1yX4W0!!M}qzZZc~Y z4jwnAfA8i5EHF3V6~C;{s_Zt+)g!d$^2SgWJM7-8--2jY^H$Fijd2@|GCn6tvq;69 zjeMgKz8`Q2390jW0Vw>uMqoJ${y3gMO3s3+ovYeN$~7pg6>YTlw1|^!`gq)7I0)MX zpN}_ftOkCh$mFXKa!H;n%itz}*`rrspqIqMKDT}3UWa_0owaH5bsN7{ybhOJOFD${ zY{q~X8AbcC9kMS(GC7Ad*nDSKeld(vJ|VroQE{knut+&xP+d*332Lc_&x>Gzfz6=P!2fVynMV97v{;+Cr z3s4Qnr1Ap}Sp7XLX>$r9BqU^HXU7X$o+bYaWOO95ui0e#Cjw_F7_(elrW|g{d>5Wy zh3i-&(bocGWEv#cM`H;^e@f6_JYijL->q>Mf6QfeL1r3=#QZfjfLH+l>l}{}*zfROcRri%Evop~`Nr_Qb}-D;eQdxGM)B->apH%cyJR$GVR9C)+%#Avz& z(~jdjya@;MQ0sjM(w?S=i;LK1MdL_&$@4?BTuVGbK3!^TjQmY+UA&!ga(Yo!P8~QW z`~8~_KzZ=dKPQ$6{ps`ZI5k{fDt+S6d;caTbO152nd$_1e589+6Myps{UL<^oHOw6 zTh-wUtELOFVvHpnbi6fJr;|spl^qK;eJJ{^M5dt>Z3+ zMn+T_85w4OJnN?kOq6)<_wP!1D-QfFX2h zR{0$un)^9F5kGkZ6UELFncSbi{6FyE>&SP|xEmXLddLsg_eq~V^gS$yeXS%;o$~;z z0IftTS&mgeP;g;&ZeqfMm+(&Kj}Jfev4@uXHR4$Fso~Sz4o!u&HizDNXbf1^^Xj@4 zzx_G@&{H1Gp#lDnr97&&+gB+8z+d|pTYe509*4BKelS2S$#*yTZe9#@H?!ogPJ!QM z{=bpo9;e^3w>Lm<%&2U}L%4kYAGD9=4L0-)ijX>7ik-8$V#RKUPr$V49Uv!P(zHMhUnq_@5Lq|Mt`Wy{u9)znPgC zjykC>K1Hd2aWPu~ntSJj8Q*VTOi%v8w>5-^YeDc)5#|5qiF$9H-=h(@%F0}49sFx= z_Ad~M05)rD6ka`${O&A%8CuEIF)=Z2)B>5K4ik0S|0^{A11}U#OGryIf2%L;n#mvP zXYdY6q4|Rz^0WXLtv@LK&04EXqUe4sWoKu%ZgZym5BJ`G%(MUL)avI+L@s1=R=WLqdI5mT=9nkRqNCKMk4{wny z0NsfFL-FBzo8fw__=VlUnhNW^ti)s)4@Y}jSnSWOeAu*)CsR{XH1}CgZAnQ=dbpG} zlArKs%9o-)1R|E`eTt6PO8bqXb@tyi-Qlp$P;d@71e-tB?w5FGlA_Xz0y+6W(2u{* zl2_O*Q%8q`&FKic>Bt2q!d%Cq%hR6&2=U#jZd`aRAEbV(arR8#0p0E2cfA9^9HAL% zcOXC|Nd0!gt%Q?)=np$I0Tw3@w(O7Z*k6(JiK64$MWpvXzJ~O}26LPw#Cad`MUc65 zE6Oa_p5kDq|7jrP!G2Lt)7mXSa-HIylQ=YT9tzkJq%unN;2UfC08v4I^w7>7<_Uft ze#pmM;G+T!o<-^G>g#a~`N#Z5X<|=osU2ajlSP<<&T)T#-JSmsfsJxN$Md~udaq?l zA3rp_4fpv>Q@-nSzfJQ{HD!kr?6vax%lb`#P->7k(A?OieV+ej~Ut1|By^uZHS7>QO==}#hg?Ir2So|ZI(;Muy zDs-+__{a}tD2Fu~{zHwB<^cAA_w7~Q1DLZUDS$FZCnhEqk6SAp0^J{+Q;PJ7v-7>a z7k~L-NuRjrpZW1@Erx`)ix!aAYx+OL155uO2WhkZ00wx+a(i?A8Za-A#Jt((z&8#7?D1Yj?VsyX5V#d!v9Ey9zH&&;U5<6g4kRZpLkmARj6I0`Gn1uO)brn% zzR-6<*$s-WA$5^CS9A1$H~bH0oXxCds9thw8R7)I2Q!c}=8Br}kN{R*ib|{{NWs z6wmq0(GQudLA6&Y$)_bIr^F<_YY2;0B;$nLdFjj7ZZ)ReL>)auIJL(C!7HxNP5NuD zBP4Fen$3ut^WGUV2}gg^F%F3T=Z01Y0BiwkskTQx8we09F_yRL2eA@(9(sGreuVFJ z&3rdy5Cr#^jW-%g7ajsBry=5pR{K=~P>)#fmCOgqxgQ*GFgWB^GX24ZeqVd94gevz zlta0iH~c#cU0^Hnpea`}N5gm9YFLMS2y>s0?zc6*$x8S- zToU+~S4n=G#6zu81szqlhiQb9QJXz+TSj$D>%)C=J0^Cl0{@S)caLZKfB(lzl&GW} z%CYkaN#%S@2lbLD9WbXXIn0cf!^}v9N)DaKSt*AyOBu~DC002s(#D)B*_<{p%nbWJ zMz2?Syd=9G$zx#Muod z>VQY@QfQ=5R4=dSjlS&U?E zpf&CEwLe$wUz>K>20vB67*x55i^e$vEp5_VylC#@U*pVF-2DChN3zZ&`>6;&4A z#a`;2#g-`WNt_T5* zx_T-3IyUpGdvva>liS;iLUyfP+F%XzHjPTQ7xmr#`-Vd4e%i1h zFMIKaZNDWgUM%?|+{_XUPtWP@QvBc4*dGhnsvOak6&F-F(fAYJ9p(|VG;6!4ct;F) zGxYf{JLcY3Z%4%5g3WI=Hnn`Obmn4=4k&ZAk@N+Ll!$2Hmqu)w<9;V2zaACxMEu!n zU+=t7*tTHd7U+7_V&O7eIksO^gbQAzN5w{ao(NcF^EZ|J{7xqq&H1<|R^!?Mu(!X8 zXm9y*1(cG(({4`ntN#5+ewiZfx5AVLzkherGc_eqs=XINoIe%-s6&;rP|st$nvaP{ z0d?$o^FetYUJPNY+S=My4}Z4A@k7I%Q3c`yH~bPboUa;`*t9GR5sZX6B63TR&+jK& zCiKHq+gkh<-tAA@EUF`<(0At6iYwi&k|&MTe7h!9*>04v-8y3hCBqoj)fF`_)v9me zEnV5A?pKcNKBEGw>j6uartP)m50QfuEZBZz15T~gKl=7nMKr?%W8moQ+~vhHS<4?j zFP`!nM*lB#@!nX5$=40HvmR71oHl)v^1t_U82neR)p-rk8aj zb>tSVX3(UzsDw1}FDI=Ho`x#+hul+ziIUr=4*vPff9yr!LoeF_R&3GsFA3s4Koy`% z4q45qO2p>1JG!5jp1E=4LomqI?K-r%G6;!$<=|jL_BLO+P@7f>MHzAIlh|)LL~r_O zb8II1ySvi=aceRN@p@h}+C7&S1H0?_;HC2XqbEq-->C4*V&@#~)haA3j1m{K=j;OM z)b4)#;nnX`D%+A4$W(mc73P-t72tt;-J&&S_voTs6&F16!tbtbT>=&kz2{ij(XY}% z%O+BGaL%$tELr`2iDt83Nf^l)bz6fj-{QY0hO&@>a|P~G`j3mh5blhI9om1Cg8%Do z8~j}7Sd5>MnxT(ir-wsX>EB?anG-YE(c|oZ9 zg+UBiT^$`hfoXGEF*Z?6q?UuaoTgWre+V9A0d9fka7tc2^IP72i{=1XtXrMwl=7lG z!1@2LI{x~LQoeA1rw3PWTWIO3^32v#iw-|L682(rNjbH0`<~%p5W1#k-NC~le_BIu z?XQo7Yk9x8=l6Z|M8HB@?D-Hi54iqwlfF0#w!tPHLuow;;fJ*vmd@KI_L-~|pKmO> z4U)fBarPw`kaak;?tSw)l@@81ql-(o4nCmjee-PFze?8rBRy&bc3jFspZ`lj*TF~P z{F+xLn`MKAII<)47v9Sjkq(rOP~gGfSx3B|`=H@jQJ_M6VO+TC0C<6o^^diFzjy0m z;2o!^vj=>aMLm`LKemLZjx;b%%d?M&D?O>ARF3R(>Inf6ZB$ST4uL%YM~~<03Ixm zE&h#67mJjF6;&7g|GX*KBR${~1*=3vYr!Mh{R*F}5=GqeYUjz3%7NAHi{1;l-;EnA zo!ue%u^Ll^C#Dk%0MTL*OYn8|ldq)yctLP_&}=Y6%0q5p^60mGgs&eVFT6i^!hhW# zFV*R=wm=%A)_yx7&J+1TO=``}z{jj$6zSU$BgNX(N5b&mQUsNKhfwuYUA%8hF*Cc=e$DAA;S1_5(7$UIy zI)@e=4FNIGO6k@n{>S#4Gq$NPKCNA{ajg4McojtESPZ_h&qG%U9SK)|a7ot_o3%%eSf{zRLRQ_=5 z?<=no8mz@uIgkERxbwikM;NXnPpw>|w+?)N*}ex$=Q+s0Q?Adi#IAGXnY;n{ezun6 zK;O1k3whR|EMS3@(`20fNo;R;5kF_h10Zi_Od6k=1ALp#oCYbAN!Ngb zs?+@8LSLxYlJ3qdfn|I}Zy0H6q8*|WqzSK>2?`r`rq6s&nhiTg1V*RpC!07N$bv%W z7vSq_v0uQZ(){)7*XuxxOg#B9A$IGqw36h2-I}Q;<3)xGmmne*D5H;P{vSyp@@^Z+ zfuGT@pf0ktzs=G==;`lYgj(Fu?QkP27BStW>njD~`*`_>{^6Tj_C`d@-`JB7yj&C& zjJB9ZR@|*NQ32}9+g?{zs%-MyTyetMI@&Xrxiv4_qORn(pn$u1@f?UfFyP|YZ^jek zIsklmwfwWw0zog_Xv<_Lr)VUTCvUmbB_s|$C<-?-VTest?G1F9ZkCTkD)rVYjvZWX zgc`jNEF6HN+~_~=+#=9&?+@-xyzX;|JIa;V?Ug;@THl~hO+mr8vO;(?u zjC2WZg{llT6|kV`s2*YUOvG`Zq!V=qj*=&G9JYX8+J#BG+8>)Hf{L-l}zXJ>*SQG>&MQMJUeJ^+Q+pnD&Bo>J6-yO@#e&LyxJR8E$#^YZAe99jL45-mD0Y?yRod| z#IXLPK)UUF>kvKM;L=Bu4oTY>lFLyelJ4@$W zlu|5MXn5{aQ24N7wQd+!v0dkxwutBvF+iHn1K6*Af1~3a)k=)Di1ulkD;1S%cFq_VVA^u z$1aLE7NIJbBwzof;!C7=R2v9EdN9SM?s+K5br>yX4eFK+k0b1kTAIRt2dHxj#I^;h zh#B}JJgm9{QmDl-=ee^ZtfP4=?#=6)-nIy%8*Wwypbwv3W#Gl0KxFs(YL{q8HkGQh zI}BpMTI{DkvqL}UQD_r%HBG8~s6gCLXg7;SFB=bnyE_kVBzmDr5 zw^BPwgHS%OLW}biUs?k(aD=u1RuROa2fnB1K(H+6ls(x_$RoHG0gb4+{6#r@A~v(F zz8@RqVNsvs!}hO$NjiX52=sIFF+8+AWY(0E zeBRFXM(B$3Dy#Io|!nFffQEsc(etVG@}B*}#?FHy1^0176DN zxX!%GL2q6ytn9esY^>UNCMpzUGdjawpR0Smf_DQW8c_VcZDEo$@EDNq?uMw!e+=2` ze;DImK*i{GRpMbPBwD*b!@T2Vmj7T8(EQGtqR!~1vo&;kyMwd&kcRE{D%3T|(lLC1 zVN?7b2YRk})=iTdHF|JQY+K2ZnNx;MncBFP6AJCkC2l_c7)bkxq0Fh4ulw3MAF#?p zIGzp`u}GcvT$w|~H3kl^RPF_gZI>;^B@05Wb}K`=a+9Di%-Ug?6DfYfT|D;#7T2!T zr~OS2(%Bx!Yh$b)7C<7Q@b!mAn=CRN1+qru6HB3cMX?V!r#g3zAmj8XQXP1#8S)8Mh%r4T&|uWeK3q1 zKUB?~^x1dhEUTw5(#L1LXHnfoL#<=5jMC3rqG@yvyg~os-1S;_0>x;zik|Ow&H`C0 zi}=L(hT5vxx$FwBdn?3zAE2QoJ1-8=M%T17J-(p0z_YdnfNKy<4o>|hkX@yKzaM(D zrGFl~>6u529eF&x@-QxP&x-1FU8M)d-5>pcAS%XFE>t%7$>9UuSlMqaM1 zabj4Uc5~xDD5!XOx!k6gKakFn(Ti`S2beq<;@{oYp5rehIVvx z>|a?1WLAdwy!hWirHUEY3rjW_{!Z}#nePyMmFuaPhfq?OzS>{wFm%JL+<(Vxl0>KI zsiF7p`f5)~q*d}5Q`WJFovm=*4d{{X;GUZD5+zcJ$i=DCYE{_XPh`d3;qWx+ zO_zy?UDtQVWTD&F8y;lT>I6=|tRMv5Vp%M?`%xcep9zJwyIC>dKVX+W$a)+Lucgi^ zBoz#^zgcc;HUTd@d1%91gHh(q$;?oByv_)2h?a{qwW;Sp{J!DOt&@ziEo4=a zc->K4s1K4~%dHz>LSsr!t@~gH@nT_SIYFiG+j$+#0W`TNd)RhR9gAx*uUy*9byKO0 zyE733&$jLl_ODOup@n4B_q;}-HM*djZbL-V2}0O+m@(`A`CWtnOT!Q62g+Au7X`@h zDcGXKfR^vhjp1Wzg%HcaB;GUyOH2e6;)s}Q$C@j)peV^dnC4}}Pd=qyHcl?ftg{lHU za&i&Vq~Q6APS53BAn49eubT_Np2|6(zdcii2EU{QfaEohCih?c*>m-rlwEVhzBd?v z%A|wV-`phhZhVboQZl>aa2^8v1kiLdec>MA@kA2D7d1_o<<0s^bS&?BFBzmr?8L{zI|5aszKvy0-=m>Ty!F4-e&Hl1`O-s!pi>ZAWcMYuRpd~y(&ljz z7*eoi^rT|=dY@N6Ix;2Vz4QfjSCUm?_H&z#tiQBZ-urY&T*GjO!(Zg&SH$7LKvbu9 zQTO+MLk>MA2T1GzdV*U-d)d7g}g`}&} z%{1por6<<0nY+9*Qlek>As?sHBsh=t_9>g*GU0?z(sUQx1x8y=PdxW67zoZ}L&JDI;z6<-(jR_a}ow zYP2^FnFpYFR!il@Wi*Z7m?=|;{!)w z?7K6ZtjNBD!+6p~7s>2(5K|=5#UY7R-+=Kk)h20xMi}Z<>5vH={vM-}W{iJubl~m9LI01YKDZh6hJ{?p@tA zx-=!@z%ClAJXVIG(^jPZ{`;4}nti8iUFa5wx=4}i2+ua9%JD&x34JPk2HKf3I8z7v z)K8h|=;TkF{#4tlUwI(-xB~*^SI3bubyQ|ZAGrQ)?J6gVt|YV07woIKyh3|J`J9`* z+yNX>9C|i(z8a}T)cb$}7-W!5+PACQP!A7RvbKp1`0Y!D7&{uE%^sVJ&3eDlrqN@w z<@64(J7EOOxLo z-8FOSdb@#riJ?qH)7Se{PR@!m{o?in=tFrsi{N9@p3OguZ3|4>kEk>z%o4>7Q+zge zn|ANP7(121Vt-(8PTZF|zRZG-ZBjRqiM}~+$M&3Pp8)7CGg|#gLTq`1TUs`a7CL0y zh2n$mz!jz5$*<@mu$k~SDrY!6Jd{_Qp$l!-W)G|AqGewk*tUmXKYgiFh|P^Pu*26-!a*0QsM>oDDWHZi8tl7u#3zG2&-cW zFXGB~{N!Y+x@^@rzrmKz)JTfFaWZ&4jygl8D@blicHr|mo`#JT+eW4RIB_yhU%qv; zT6GmoTIQ76S;Wrk@iRD_O98DY>uUbh*`)4Nb#cnfOTX*w(6hHHAe1-8#4Y{vjT5hf zBF;Tpi{JC$&XotjiP*}IW+_RE>7aitAR7GWpi~5JW1%WFGF4BBJB4D@>)np)g;(BU z4$5q3nW{g#yzsU0>Dl83+9ia9*j8QR(p;?{7+4!A*HlAIOBK@VaTotax#%pEwe!Ab z@``q0v?eT|C_gzUch$h*sgH##NTfx}{wngjy&Tm1bn?*Cc6Ju)%e7<&q{(Yq&NnQm z->7;7u)E&<{)tjkxZLt2a$9-+KCqk#q6ATe~3>6!O`nH^$hR z^j(tb5T?cig}-{^MifMYb)V1dsjDE@>fpomD;@eQU`-yyXCyKGZ6uwNKHGvva?dY0 z1&OAdri`4D)rC3b|Z9T+9UB^8+SA0e>+@P^V+feEV ziqJV8t;*(|apnU3`gv2>3t1AG9ee$Bxy0=dloeb8n-`h+bWpHoMJ{PX5odi;Wrdd9 z?FZed0C^EvV_nF)gvhaaDw^WNNxQE$X0o}LpGb*M4&dy2m%@BVU}6b2#G-_F`b|g4 z!5b{MdX2@tUhP|t;AbyS7rfZ^x_WzR*)sslT`R!;7>~*q1Hps6cmHc)xd?Y!(Rj{t zJzUebyk$qXX)ws>bnpx@fms=W4f9Uz9}=$BT^KVM8BYB!EpW9K#ta`0_0RKAts5hP zIaV6M&|`WQm6X<0jBj*?yz&v7gXIPibl*~(?5dIyTf;E;iQ2bDmD~z>NU0n=vF2L8 zUB$-r?uHfoqzOy@9r=nJ9fxSgo-h{*?2KMaXc3SvkwC2?lqq8lyy2w-a<%zkR60HlpB_B4q_MJ(vbZ$v2T!J@w zl8{)mC-J3TnNPNMy?a6W*tp6oBfD+mxm%AjdP_je7w(U!bHOi4CIp;b9*4pgGACk( znS3VmU?{FRlO`fx%kWhX_W00Y1+WVJLwph?Jmq=D%x3lPV`JvM!(jtT zyCKB}?{S05jHer`A#hLAi_jPA0?%Kz`VY}C58_l*F8|r!!R4K-)EolsM6K|#?E3AD^ZB9?*b3IbeuiI3-nXRl5 zafA0wG_bg1g^(4BC=)`}vr+%$=IWQXm}`4)RGOPML2glO(ZbI-s$w!1z*BJ}Vfi-s zZjj^ejk|0MV3=PM2>Myl3cvC_FCJ8#HnWgH*F2Z|dPh}oW)JyX<=!-4i13k8EpMex zW!u)I@FF-_H;Nt)CYoI#ynUm0v90^`ak}Dt8@btWwkqa>--UK{KXk-hZ%5i zgu!-w-4^tWS6uIe!tgtoCjE!&W_g$`Z~ZQ167V!PT90u6yVMz$TX?`2FKzc{aj;iJe@vF@QrUP?J9*dR^-DO z)W#NqB5bE|cB4}!qj0e?t!}g&VN7FC4-E(Wh&C->Ido&g(>g>LvT)q6TfVia#$A|A z5H_;3`fi+N4`ZN{wZmcAgg1XyU&I{-=DeuB*nd6=-zzdg%$`&Xo88r|Xty-vUH91L z(Ci6sF9Zs~`lEkEs2D0Iz&ESy|LEF@xF(bLltDHV3sCxTSy5)r;Ml3}0V8Tw9`QdH zdJ9=zYx03N20CG5G{bo%edA-a@R@sI#i8C)jvK?g;m@+L_�wu9#P7k*$}$Ieha# zd~>y5x1L3jZDU~aSgPF!7P++*ZNjPmrvKbGN3H1&Zgxd;NL@l~OISR5$d<0Tb22vY zRFlte2XlA+uyIYjV=5t_WiQv=l!Nl__aLai4v%7)b*;Qi^HU+TPc3B` z#FXuVmsP|s#dy1L@8cR|1w|-D+mPg&I|z|^ZJ}7f;9)MrWJT?_u7G)3wOX<&=^?5b zI45mVVo9@(ZI6nfDU-b(k9U#G>07Rw=k%%v24+^ce=?^iD7}K~C*8Gdz-wQ4#7!;; zbn&iWC(`ILFF)_2UaAMis)A+A^ggeF`bp`~E_FO;S;+6DYx4K&=Hw4vtV6ZCnFe&T z3BI%vaNwyemzCksQN-+hNDLnDTH;M(J)OzNvhsPj4h}4G4YJP8m7JC^4BWuiwL5LYlcN3ewTB?nK5Au{ zZ#lw94~~sOmQT;tAZ8hUQ*D$u`Qz5IW2G{3j3f9BSGOJ}p^99mgUUm0!EG~ZB~C&Z z+Cj&}GQCJtksqV?n$Skft8xx>@10YzTCuW0B{H$w?m29{Q$@G@KpM&)(kfl` z@#=6!fZj#=UW9)dm50OD5HOJ2A(DZ0{!a-A<8!rzn=L2eOFj~Yonm;C*-~I`kb-YN z<@Ea_sLC3`NUdVfIk!6CGRVf9b7~3kMYYej?>IlmDXA$7TtzeA9MafXG?N|MXA5ui zro9<0)7ACJj~%T%sR+hWU-;0WxpJ)nF;P98h==8svv)7wy$+(L0k>)+Pv;IEb+AOO zK3s=#L6>~sx9{WjVz!Ov>nc5>=5FP!nnL2+j*r@kW~oL$+) zDSy!2eRn*?gZ1>yIw&(d6tlbKNqOPZLFd`>&b1|XkAHu?bH^^kbFTj7q2vn%=`Dhf z#``3;eLoZC0H)!GC8~wTrB2;%p|h%JLJqB9j$KCmx4`o%EMBZYXmrc_1q1}}(N9J} z!tP9G;6D&d_{)nyaD_lj)Y$K`j;c6eb?wOVX5%^C_<2KQByII!C%i~u9~GGS@_gom zl_56rP19Gq2xurSw+gGi!C9fWNx^fEjs@(N3CtHiks~2Z27s*giva=!!M|-yygK|N zsq}%o!9+c#GVThHdr)JlUNR9Xm<+OZ`FNF?h_b}sLSIg$yBp||klm`Ouf#(= zJ9URYfhZAQ6!f_SCy51C7pfPjbu}YMs;kjd~hrZwA2cR`xw_u*>dw$_}bi$ zMT19&%MTm6x7rRHUr^86kVz^^ctF)Y)wP|IkWRI^F>P?SyWN4pFG63i&O2a&(B@`b z$tao~D|nOdT1*XOWDs%+`ABYq!)p67(tTJEyUgD3ECprCs2D3V)}2V?2eK5}IIj-s z&(cF4rjY0XhKPOiREeyc@E*rTX0Z~tMSGLf@}1X8Uj}+duI+ChK|GZ{OidUlXZ<%Z~GPS)K%zxju`q>#!%V7Z}rhJPd#mw@_4f3tgD z^WCnl2P5*`3hC!hbcZ5ePmU%kmPCMB$xe^UStU~Qk6td6(k5fC%6Vr_bnQiWrKTW5ad_9D!q%|eMsDhT zS?<20@SI!eWOF&UG-D>5ka()q{P~#%qso#epG+pyBvY8T66*r)w3E5b!4CX=y4;h8 z4Bhvi^D#a5sTbb-D3JQ`)_q3m-k@#$RGH7;ZkSceJ{q(ljEJiP;-F zOh##+9qZi3r)ki}3)x1dqcEE~2k+%aQehqg9SwYWfm308BJ3cmE0LSe z%t&Xv4DHHL?75KEu2N;T=3OMKc-rs1&OYWkw33cDec5R%zyFtV z3_UbsHRwZ~Zd*6q)a`mF=$6aOHUGZjP+d$RM8e|p0cw=`lfYBo+@^`8i~Q9?vz~&{ z&>MmZuDpKg`fB1>H*YfM)+)};g7H^A=t5ULeCe4cL@_|Y!{s%z?k=ZPA&lK6>1y=! zL0iirjFLxzDVV8;gl)m{sUwMlPXlIt!1C2^vvwv>+ILcBWHx+4#;RAVgft519pj?m z#r2q%$%fPxZRC)`XWO3;!ywzLnhphAg9WM!Ur1^>i~UMsc4#c|W{6mQ)Vkf)%Q_yL zA3!YPz}V#!99+w^zuT)2)CJsULPWKF{Mi$Y7*=Oge!h!KpIafwyBG!QSkwZ0Sb z*c40o%N?TCmCygHPM>(0U7A7KpGY0FdHKF{7?rp_l}JZgv6Az~%M1gp28@I1I692L zWAFdsm47Ll$YIYIbK9p;YS;?rn8bwJ`6j){y*jh`FJco3l+t4~RzC9zftgOPcscI4 zlfmvEo^JHxVG*7B-&qu7^-qU8QJZ22H%Bm_FZE(n1)D3oY<-;Kv}NDd7KOy+}fviQ9bKr(u#lRi$Chr1`p*v^L<}M6|Zy`3(^-9rr za-o&+Mzvl-Dx=6fHfgpA=ZRy|Po`EIDdl@~J;s)=GcY<8-t3=Kd#0>%q&BAFQnn{j zebU{RT`MsJ&~~MZLu`XnT>uzphVJipz;qhcHjLHC59{?D38|TxY+2K3RQO&=KIpw$D#1rD{D1gi$tWFo#+Qwi%_i5$F3S@l4`$ znGVBH{`eWm!|!o$BrT4P*Kz5tp!noh>@~{?>KYw{N_cu>juc`^Ek`IF@y5gToiTOz zeP4$2@hu^k_t?>dCmBW5!ggamt+5BsoUJo4p2;><*iGU!(Bf8`GHWQ2JWD%rhu9ZD zC+O)!bX}eNDcy3?Wh>w(SwmV~{55?QyO$5V#wCB6CxTj4BPM68LxU|?L1w+tAuU}N zJMy663 z<8o5eTR9vPE7=5VS;taQuVT4pw77c{tn~=5bjG7EMkOwa%?Xql#{-;}_4~9D*u#1F zgxCy|aT%YaNCCFy>G?e+i6Pf?MtRneFR^EP9q+@fWG#~TjKQ#Dra4z7zuRd*6*!2U z7vbr|qQQgKMH6!8_kJLGz)RE6sG{-atl09{L{v;zY60gUDas7YC*qzZ~QO(mjPEdl$3Qen@>JCqG+4EA~` z(yoxz#nHeM_ixxr;wEjBw9GhgN&um>5qN494cd|EyKoP*xV(TMw!yh})AVpG zMHdMjfai|Qe3Oc9&s;(p#~NDP%H(}Nvox^9#5iC;x|%EW=y@U)Vk6%`r$eOzAZ!Ng z-5jMfKB1k~fcBq#KZ56Mx2vw0_psDa)u9K`!NsBhlNC+AjYZB2*|l;IqoX!M{!V6t zEAIhKt<|Hy*c}IRgA@^itC!fH$>i7MTdtf<^c_*?p4k57sk>4#)LIccUuW!^g0Eb1 zZ1awo``Lbeg)fS9rX1`nVBcK5aVs49mU|tAw=hg9Y0XJH`Bd#aXZCci@qCfvYRCZd z>86;P!^mCJJ|iD#2kOewiS-XmupGJeFVq>wJ8(+@gi&CWR6FEA1BcUSw*Q4x#q{r_ zHZn~nVgu0~#JP6bZbkYz}@p>$IR36SzL1IXh(m= z1g){HOWUq-{I0g0Aq_&cX%jfuylv@d6!ZH5E8lwI>DC`lOj$T!0Soa=2;jQ)Nz31;qt5p<$!?zWNB* z426#Wl#Nr3bL^38L4W4nv$o{lD&5?+E2HgQg>{8 zM}JFl_++63Ej(H}WRb$(Ps3)e19?|`7+z+OcB{a4(sH&UQGcBp9o|-YIaM+t*^t9% z%*>dn6JZcGQS&P)VtWpe7>r7H9}V(N=$FQk%hO%vJgVCdeSWj%II__gpBiuM1VcA= zf_dwf)dWHV^Fm{Iy&-TSKO94H{nEN%PN)W-bX=%$OW{V4=pEFbh<%Yo;KK=lqb&3= z)Qj&x%BAmOB0q)?(K9MdN&_!(idM8fJ`hbY@^b0)iC%p23iHb%gJ zRZmGLmWrj;uPg=2Zo zY21zsxI7c8$2I!F2hwOgD74<|9%sc(9a8W}oC9=0h8CJ}(SZ1I%1twVu6U#)kzI6;|s5?kTlNrH< z8;I`C_h@#~qjpXH@Gn$}qa5oenVgU9w@Xghn1y1%jOHPlgR8xwwe#0x`Yi!MncP|g zVLaXHx+OF9r8uE?T`$4Kpj0NE;fptCWgx?zVW~*>g4q2YwikP+Vg}DvvK<%~*rDDt zVT8c#x5A#ESKv7ow*t>T=zxoT&UDOg%f;cl_BBu7@tNJGOo<7y#LQ;bXIqOs2RdLC zDtMVBCi&Q4@py)dq4merd0S~6TgjTZ3OqWzPGR@3X|H~FnF>eiJ`BMwGr%KI6;|BZ zQ6*S1e{?EgMK)F@;d2N}qqBDE4AJ65L+9P`wX9xB3Kppo;KU-x#Wf)6%2d!%yJ8)1 zQns|Nq8D++yF&`wp%g;DeW`mFjIyQznMA?Nc07y@IF8M4Hl1}M4#$g$H3B$bj9|`W z5}h&ABR|Jd!Ng497|xZQygmZPEw2>XjCYX%#k&FAI3b9o@4ER7sQ6S1f2bw*UnxJ)Svqv)|~UBOaWx@JjEm{MFQ3KhGL|NgCfgE7nSY*yln+FJ>nw{3*V zBM|7}ZVf(TEzNeF%ZcDNrp(kQ2O|fyvH~&P+j!kG#clfv30b%EMVe3lEW2FqwYx_E ziRrDo61!j1I5N98GBFfQA0W(N?C>MSufe2mu_v#eN08>8ZkdIGxCvv14gOD?xZl@#7IRj)e=V- ztZgB5qxVkKZT-6zpz%SE3!1W<+}50#597$uSeM`ojJN)9oz??Ny9P!HwplQB*8`i6 zZtBp7pu6nXd^*eLZ5B82?x~y3E#9Q>N=FD@^&XObtV)K4+@T z3hh(1_4GUI6{h31rU87?VFw}Tf zpgA2$^5b_)i;i-@2>o`e+ldoGqPWW{dDUFiSVa2Thv;FmsTmui_FA=Y(j)GAcE{y` zg*m9p!ur0dmpkWmY_8u02Z*Q_yXLGN7i{?}xwhSj;HoEsPk+=O&QZe4< z>W-n#u|i^-5x>3H3FTb^m~5MNKxZ%zAJqzWe1W@AHxxoV&PAI_h)t5ih+j-9=`} zeHPE{+j509DzK`*e26+qn;Wdx7k@H{D-Ji|YAplf{62j2_EO_oPvC!ED>A!IxN}eS%8L56(NMxUHiWl$jY9npGu^F$c z9prxHZ3jJ>$c%H^+)QImXs)C>BkDM$UIoa+BlQVTNpYWf2E~qdT1&nfeqk4bV$xbT3*$-Ye|Spte-i(sAf!-TNA`VhoLBN`_Hx$z1)znBRgrQRkkEi&eVbH zuI*2e5txp1+8#c75tsPI9h4u&*#UV6-XjekP!6AEhKDIWDU!sICTO$tpE$udNo5we zzd_Ix`%U3giZa3CPr!2b!+}5YVu6BQ5o7H;e;IkHb{ZYR`kgFy5 z6U5tlzP(_mvg-6va$j!eZP~aQKiGLOY{bfw*ApcsAh)(4yBvxF5R1Rl-;R|3)D(+5 zq{^JqI_E>2;Pc849k}3};1Cs5FMGU1^lI^K{b5^F%2*?&joPuU_C(#@f=2;Ly=4?D zTP+_;2=u%pIB`LnO|Z_?r1>gTKm0;RZlycfUP$M{;BUg|rkX~+$+(tiU^o(8Q(J27 z-XFFvhGf(U$HbrCiexsl8Vu~5#5y5+U>|(h4(AGUJ3K$24CRaxresWPahKR3F(fiW z50Sfd?4s`wo{*~c9fy5z$+#C5-uNuAk-;ToTuG-Bx`}}w34d`*GOoTJEr+F-Hg6h6 zYfNWP%uI*u^EgMWt?!g~z;#xzA(5NEGrY!Himq=9SiUuVojpiIWPs&m*4y=4VX@?}D|VWPrIUNdqgOlE^$hs}oSfgVajwJqk#nwpx?V5ZkO7t-1>J&f*zu+KwD*hkLF z@$bH7Fk2VS(=?R5E&XUCpscDXf@umu_m~4+QrM&oX80h2!PFqGn~%Y-T$52jm7|@7 z5u$cb*dtERI$(%`dxb_)Ahvvy=zD_xK6{kly8p{Din1zMwVt5c7Z- zGV%ics(^5nQ*_hb*2qq-BBv@r56Ijtb-ERA%~XM9<68rLh}4C33SA*_1fhUZzW?+Qjbk zyVllOhuF?bQ|pyUIs#Z@8$IN>3rydGgWLpnQE0&1N;bbIw_OTrY`~zpY7Kmx&OJrh zUsvjmA2b^zn9`gn1R|KWFZb*d(saGBP^zktslS+)tX*o&|MDJu#@HiQc&`L2CSGG_I@MPr=hcDb=1&_W6fVY6w%c;+ z&T@(d>30({=nBUH52mxFo6e!8S%wWi<54=-GO0(y;KwL;K{VgzQAcnwAwA_~8giK# z_Y&(d#rB0PsFH+w=~)~i0CL0C26CqBn(*I=XuCWY=}<6Eu3{_$nu zKbKXT&XPgfOW6}msSWdC@uGX*f0adAi)~8N4jbM#@fVnjWiWC1YQVy7cxU0*jgxV0+}i5)uziGmxmtWrb(YeOVUgj@v#~k{&0Y+o0d%&K^eZp_Bdw8;8?=ZRnQHUk@rH_Uf z>tq@Ov@vANgs{p}iKSMZq&fmF^5kun3sHsfXeW}^+3{#h4F|`=j%l2Rv+0SY`$}~_ z4%+aJr$twR^Qsb&rmF37Tvn0ItHi;;v8~3q2jA8f2|B>JS5PSB671T@<4!sf;(#`$ z)fahmc*1;V>-df;g|yGnm#i4RAi$%jsfHUlnugDY*pW$7=c+&8rawNkw89EKH8t-ynkpN`rHPn^h^UTQA9VpI5P7{rMrPV}t12R~FJ9&D(a(%^ zcob;;$f@`Y3{a(0$0qVkVUn|TS38aNZ;8_?jdDvZy4ruXJ8{DUzkF;R;W0S0YxJ#P zzX$ffN)+TN-?7Mp<>S0B2=CT6>ELisW3j$Ax7?1H8yh-PD&uWo3hAgbyH{>qaX=EU zZ+W;B%S32g$lboq%X=k04c2S0i}}K1WecMWzo^okI6ZqRgNWRy%+rINm%-8!TN+RA z_;43ls$PN+5^rIH_>z*akZP5WmCPQ5e|me42mD=d&NSymVn=X)z%!$82PU>dGo0G* z1+CA~e*L(7m`Tep+Tr!gx>uq8+O|>H=xNvL4m^55<&)xWTL{8^PixEPODug|c=x^_ zD8|z zwVs>{xk0td)CCoD5OVWO1s;QvV+47!NGgggmuuP*^uK8&KI5%m?aWP~xU)Cit^N=!mr%T1*881N#E(6dncQC~? zgWB?8cM{t4Ooe)))naAo7tQUtFDLia+P=zoiEF@kIT^|`ByyO5%i{aj7hM(Xv2-?` zg@iuFjHzi-eKlIr91da!{nWQSakRC?q+Fs@EHjyK zCpG(rP_2h-AzZ94FC|XEUg>ZYqmo*k_*f8{N^i|=98BC0<)Hdm4{G()?9N$k5el}_ z?&fTZ`CbPKp@^fSY6*kjQG@!bqvN(jsT0XRI0kaHXF&JBTYtt7ZK>Xu!o{dPheJsL zGxuqOiD(EbEGd)E<3gBWyIddlI@_i4M^%!e3%|TgWUwJThvRT&56tN~9&I)N8Qyn- zHI2~L#Fct=F{lXmEGKLX9D#@VZ2VkDs6{*5i;yqNo#P`bpWr`VqG*s)&5N3iaa+ZG zhV-+7*g1tWmQ*BQdk&aG0hPn+=I{H|+G_FE;JZfzsFUHGiI=%}+bRyXc7xwhVTi32 z2xvK3Kk);~UKa%H?iu1Y(oCs9XS&6DrscU{=7;s{4_3I~9f*lo0y3$G^jQ!tjFiHr zdsL->iwfT}%>k9@E}+>)(5JdmAdJnwf$vyH38pgzoS#C|q%fGJ8p+p_B>-fc~lcGO3xKQ_XArh@0tzVyrblo}mNUk+u8T!y0 zdzo_DEmEzCEHUjHAYb8)gOYjwO< zAZzZU&`_}u}QFOY2;*;o0kH5`oNG`pZT%{V3~;#Cvohp|^vHd1#1x zETt_V=%t~voB}RFVwmQNG zQrf4~n=Y}BcZMg5o_Z2cJQlhhO=VU1;+ES#n@N*)siVvgG6VjiwN~nuUmP&d;u;wQ zHsBmLQ^E#IqXwl+L&wvfbGsCGlV(`wTw<)PW9NrqE4?&B90;`&AIQVqI4Ch-_lf}q zunK~OqyAu4(`3PcH7#1(=jh3Q3O_fzr~_IRhzcBw{ipJ;N(F@C zt)PMC@SHJwVdJ|0Ne`Gh1ezvXWc3!Inx1Crj=S!8C2$3ixzg^-Tu???ZDBDsKXv{gQ7-WB_?bK;|Ip3&*yA)zgVv`de z$wN1a*+wiy4n8hci%%ADb0boDim80h=K-^)NK~1yxP~wHQEcEeYZ>suhsdFI_^Zuw z-2As+so%%wJnE&PL?kYt^EkFztx#%t!O`3yB75s%ajsMuX?ki#(TH1+1vM?)=HC8C;N7%z1^ zikz$;VPLD@bT|y?n~s)%Q%v)Znqr?Pa!Bs8WCnKYlkAE5GsHk^Y^+0;S()ZjWY7o9 zuJD{Rq7EEYkzR%0WwI@ zK`Hj9$eVbXKr%Mxyj_%=(eu$H@zDis%Fdoq!lPc^HWxMIBhSbnje?$`3a&GJYXOvf zu?LXNLqwtO+&*VxfuWKnpK7jB4gJ~4*eZQqZf@REn2S!RV1^rQcdmDE&NR939=Nn6 zo$q6AmYAcSjlN&d-#=%6TK7dHz5vu9S|`73VMaMfn0c7T-g8OAe`OwW#k{#Z!^5`5 zkuCUyxP0rW-KGyv&%WxOQyTnick*ZJiXV|asHQ&pTMBii?t8U(P~XlLpPf@NMb@%2 zj7UmvAT$!3Xj5yVk9ijFH%X@UWUi@O~m;D-3ozHAcr|v~}sN0956j3L~ z1RldqS*(I*#No&+nbrAujxx8e{fyYQ(Qb!kQC}bbvG(Irw?v}P-+FxXi}RKPCyNS7YI44jsC`=&Ej^!@=>jw(e$YFP#+KFN=M$>9iz5x;eDlsNbk@A0-XeI6hQi z-PlVa`(1y<R>KyULyQwIIl+D0ecF%3+7g#rNroG%td}Xi^eV)R z+VR!_SzoqQuFh^d@nXM0OhwwcEh?0uI)fwnIUaFax@i}cWlG~L*}M$>I}bE3cBY`8 zcONNuT>1aldJ}Lc-}ZgHl@{?q$*vH|mOXotQ6WiLW*AuqL)Nh`l?vICEMpl(q!^QB zvNWjd3}c-^jO@$U8DsZ*ruw|^+xP!JjyXCWlO7%Bdd~a2&hxzPTYzdt3CWLjssf1^ zbX|Si@KZr5kS+7DGet8YR5>?VkDNABQ3>@&xqMF?k)DzyfYF;YqNN@)Lj7|A1TLMu z*`3nAXPse;%z;I}gs^+FZLIYXedS51WvwCGVPbk^(iKewI~q z`uq=4X!=Znfk5#!h?~LEuEbytMeU$p!?|zlh^}%ZcX8dG@Qqy9s`wY8ow`$O{~)D2 z@>G2{TUcy1NoHeDO08hW?Ak)=G|2{b+(bRkFPN$)mf1vB=l>PyZP7+VKZBD~STXcM9@$)5Z~EV0mB^Y>x-oj zJ+@33&v8HrkOhbZ$CTSV-AWl&&kp+O^$&2Ody4Q8 z@X=*?Nlrvx^E0DHA1T^!XsG|Jai$7GS6t}MnCI7njQXz^@FWG#dVhUv~ z_ez^Zz0~!9h5C!DaH}Uqj3n13Sq$|Q6EIWc+M5h!Z z+)UkK-bPv9Feqr=={xrA#rvq~xQ0Yq?6D!N`yJAus0%Y(3Z3W>5yJh(q`YC)!>s!3Cd;u4A(qkC8@K^Eh%^1HMTw4bBh8Lf?iHZI(2;WiVk>WTGn53kAfIv zER8hI|8@d%YLWLnDdT=Nav+L`fqp@&kBe%G)zNGZD|q5NWc|cI73vbjTCMj z(qIJ2rQlb&7y{+e|D0YfESfv+{kC!H4Oh2j3`4=|!=d}lmG=hAouG{Zp+Ob%pMQV~ zH~ShjOxo2}@zw2#<>X-6kbSbH50|1^za0EVAL(KP>=Uv2NOtUU^GVd9KR96=c>wAC z(mXr_fT2tFTwVtc(u?)Kug5bt4}kc(yKMXl38xHx$^z(&b%sb9xkbn)$G(LYu8 z)k!ysqTC5K{Kq9q{N2#msm-@eCn(PR zUr!FZ-1oD>0VQ2g8j+9Ky&)q%VNMXAJ&1ZhViSZB%aXM*anl=c)rjaw+nLnHkYk^* z)=%ejI#b6gldmBH+`4n;k7VJRx6v?j2aW)I^fx24v~$8G!yQ}dKD5m#E4cZiP(HEv zi3hx@_&jrHQnRuZyvR*Ks%V2kHez$wn`fnhdmfMNGri32=6i>3)uF` zVt!fWYzo4lMA=G_A5L&phU?_o8L_;~gtc3ksN2&nMk!;IHeqXlv2#NM67R4KxJ|o8 z)(UQw?YVe2;|!%hA9L@OW_!o};`oo>hvi7j-WyCuu0AMcb39Ymt!CY04dVgv{ixV! z4BgS)oFCqsmwCBIwc5w>`u%*IfcTK&(QOfTWi-N*piYb4+=*3Pxac*BpJ@$R{IXar z>$@;mY~Pv`zg$?oHS0MVDY|aI6RCKEj!BKzThERDDV)d6T5ncsv{HDxwO{`QIloq= z?$-yRRdSt1WnBNK#18&5eHK{M)}n+P{=G@0)`Ja5jAyqp1!u(OyyB%zj-CpmGsc~nEq{LlcUn{sVo?Aa)S&f7gmsGcD2&s(AINP))@q)6N z@v`wY9!^){M_T+REn52|ALC-fmHoVuvhH#Sc4LdB)JquQ3#c2i;;$xT0$Cwo8i)#S ztCyV_p%jnxKAr0vpb-nOiW1fFZ+%tzNvw7Fe-YAMeq8=uTb&55-qAr`p| z?1<^m5*2dv!RH*C5b6vv-q9<~k=(iWY6Kl~u-wXM5Dm9lg*Rtd$f+@(Y>-?1bcFgU zxb9UU8p%7I?(b4;>*w`~rn{X>&Mcidw4bj0qXR3iTS3qvV5l$W_&?aL>dVTG&G0sF z#ZRSz)%_`?d}(t=t^}qEQ8_r5VU$)3sZ6e59Gr;gt~BhTLKg`IKkyo_Gvcs|4%^k~ zVTFO>zz(n%UMSf{U)U_2BXw57=YJ4U{cMJ=0xiU=u0yaD%mQgRw{rTmaZ8z!%SAdQ z<&xIEIHE&G<$kSUSLvX1S|wU#?#OpwYBiaW-|r1T)XBEzh7cr6`$;^KJ#QK z3ao1#5!T|2ClZLqGh~na;#7YX)5~)QvSW`3Tl|Y+fBy0(`y(U9kB9#(khvU>tLNsE zerm7{24tzQ>40cpvFS_CORs0|%=e&&Q=w5m~uU)?l=KlvCNHi6E(~0KRofo z3bQ9pt)J?*8$2@3N)Gmnnke5kkiRr#heWuOb1o~3mw$Fv-vS}vZDt~>@s1WJQ3^QWf+cg=nXzNB7()i+}mNR?1OiMWz<-`w03&A7A} zUGn>*t*pt@UN3Td3@E%g7c(1(4p)%~3UxoJI!z{|nb*)x!gIH)ifg&tzVNlR^ZcgT zeu1^I2rV_iVNeBhR){l$(Olc5ujf_ZkQ{`RRwg3r-|e-$;52b*`!m0sZP4U$f^DZl zn&PfQ+yiFh?ycBb3Y0s?E|6~sjvh+dfovQ3RDlUosQY6sRJPW+$K(VH=hc?+-qsk! ziDLX}0e@|JU&M>4Gl1H@=k|fmZP(YlEGVuXRnZOO>Xxcl^gMMPc0QyUc5AABf_mz{ zz<`8jZ?XL^4El1uua~=*``;`;lgn&50{};|taLaUd;q`^CmoLbKzwP6u9pbyI~tmi z939)h78RcD%2O$<36C3rq9U!jJOq*-o^`CcDPu32`#N<6hF0Z71`eQukAwKwCBq^Z z|9svej0au&PJ|Jg_#gV7>SunUwlD9IPMqW+Kjtp>7dqdK#MH8jZcIDWXPh|; zdNL2KPd%Nj09v!iz=uRMNUp1dugOKOI6t|?9bRa=EE~rYd2yx^jCS-oQVEUwU`|Ad zHI7lt)>0zxN9miJ^~+izbzNUR0M>~JGa}d=&bD_Ws|eE zQ6UpC4b9wOKlm^hQg?4XTVEZ_psaAd>cSfgm!sLPego=*e|~#$l@11CsVPYz&ZcoS zm@wN9Pl55D^4e6tkf|!4o`w$e-CEqoVliV6Y>&csb-jqTGrNkF!>YqK5>WVRX|EIo z`6XhZeMIty+UfKJ1?P_C)g3uFgm@dh4sUQYH)gFMzcT{Ps4m{C53(T{di&+Tk~7fZ6x z|F`1))0^dr(YvzlI@^K&`~&($2MfI`W7LU0_%F1De~spnYdBKkzd9N8DCuh4Vr=z% zx&}#vFUakzjAE1teAH?edHQpwg)HclXEeuSaRM6P%lI1&kFt`q^zP`Ad}U7MamDp` z<}G#)&$TU+R^nMLbPqX+1++(qVDF?~PnNW?YD1!u*+aUfBv)TvbK`g!K!fBe zdM=o=2G=o%k+60XCg#BG z^;O0-z8(s6olN-P)f}}4Np&er<(cWr4sQSW7`o~4*;pnuXAIfQ=Vm&yYyaZF|0h>nXtKD(yr zPrQ0T{l+LDVHmQK&}c#5G9gJQPEA$;MA6aQ-z2mqnnKO1mt)*s3+ztoPM`u8#2}wt zxcD6aCMB`GSQFY0Y`)v7u{~$ZthV&^+7o@*;aMv6ElY5o^CbfZ&bYwAF%`|paLpZS z<>@zJ`0eJkr+C9GV2YF!nn)@9QfLu4kFB+68XN;Qb>^#xX%Lk%3>@fgkwYye*a{WvItZYD~P zBP7~@D@4XsUg)bxD6XM6j#36eHcr9naj;%wBc5&6_l&t-b~sT+4P><2;*U^h{DjJ2 zrL|xusV6dO2e#T$4x0I|)wLtB4%DG0yRY1)1iceq<*+}wKpiFE#WK8jma;tw`dX-J z9f!@MnJS(>UmNCF%|QV;AAG2ZmPO8npM%@js3)&nFlgSu*-@MH1-hS$s(r>LV^f&y z+OS#R)%6yRY$pGQKNav}4w(cJWYWRt)U28xtV~~msL{Y6+)u?&3{k%DZEL-umMarD z=P8g|=qm?4Z-Z8@##Trx^ro#D$dkzZVO}VyU+=R|R&eXsJQZb9D-qjGnMvYGra~K- z8dQgyWl5p988l( z@z~7PR#eSK)Jl9@P#tb}G)h|PeP&1i!clh0Z-@d_vH`z}87m))nRuWpg zgiqwi$iYETx-%O6?9;2@4F4dt?wCB$fUMmDGn4kT%-%~~Q#*Gg)@+uY^q6*B^p-NX zbPFRnyqNm;uf!2ooo(FQcKxb;d_3sTXW!PT-Zl6>#|pAZSV}^e%f-ILUiB~v?UZhn z_{J6w^P8>EZQv&EWUG?~^2fQCY&sTOQP&mpl<~#1XjZV7X5J3u`2&+S#{}kT=>(;G znAvmXE&X~@X^MA~hE~x`$tg^Y$({*+_EtBH$w|L!Kyytcfk9Q?M1%taD_OpgyxTEg zjN3a;idW3Ys;ZpO*QLJ^E@V+;J)5z4G_G z+IIP9mCs(A^dMC6`(@SX+1~tJYv7Z$ zHc|m@Li!O{iJ>d0*`kBxT>sXa`axIXtT_crBw7k2Q^YOfrxcaP6|5 zkWk4t8tnACXrkc-0jY64BywrdmIjc`1KU-bq!80Ff=Vi4df4D>hUYDF|N1kv+3jgc zn5*USkK^C~l8C50HqlaRV+8T}m{dnc_oTKHw* zl5}8yYYQN*_ILZ)`7M)a)n(}cvmY`VH))~!v|@tBb_&hPv-!eAFD3->z6e;p2R-*T zr_yL-Ac{UwFC9{}*$?Qk_in4smwL7-kh}w=hOaw_y9NIE(V{$+KF0lIZS7G@+ppFn ztmR0<(ces0BZrfM<;o znJ7Q4XYh&ZxCSD`4^;~CADl~_>E+dWVR&JYh*+GEQjO*Kbc>nmZM9tsRF(c;t4k*Xr&v|(0IBql%`+Ja zTWeW0p?g0WHT(QaCt8z}bM%s9EA|cC0*p7Obu|an7647xbp3~$=4v&&p<0neFtv9Q z#-g@#6{#5-vbo?}^J7O}mZu*O>Qc62l_yg?YfmY?7S&jeY_s*Ldix6@#hnKKqo;g5 zGCh2;V;@v~<%lEj9ObG#RwcSgP!wGx)?o99zI5|!4d`X>uGg@zz(xlO;Oe7rw_bx?B3#lbA92+@4XIe0&#NOk#}==&FeJGj^U+Dz|tsbwBaI$xu&PS z@_Rfd@rvXVqma)Rc<~K_UfDXM123~gnjFHy3(6sKg!UioH$V4$ftI1MwiD=9PTmrZ zQ!wIT%UasaNv_o+`45FdoiwhJh;;=D->?Oe@bnuu3IlUs?k^{FNan%lsEBgLqJ^bW zHSMPbTDM~?K}Rs8t9Gx?k;HeEczo^pAR^-TjCXDwOV?d?4z4bn3J^JjR0nhCH$i#Y z>H~7^kn=*Qn>T%KD}?dnhpcU0&CGNby<(ZK(!k-c#nfrz(nZ~2qn`h$>vUa*r0ew& zX?@)V&Y%#WNo#nePPQ?NGSsNit8yTSR9aEwZ0oL+#BNFq+||YTyp!RitxXS#!mZW^ z^$AIV)xE0r?eCo8qMLvCIx8$t_e}P-d3uS?tXkDR4J}HNLoD3q5zVbS16IDTq4(ic~JzxTDd%?J#fhR_Tb*Cm14{2C=8ZOCjq$(BE^K`ga5zW1HJ-bp7{{2CvVk+je_FZ0yg7K-+ z6B<=2O+6K-FY1*%O2W}rg5o3_QD;iloqPT{?lZCBBK&4EU{|Z%h{y2LOt_LmDcpQc z&OgrP)n?q&6_w{9DaXUL*i^g8BUKWe8dhoc!}&##W#M+2Zfyoc zKL<>7bVZiN#lReLGQ{&0E0q-B(}35-xCV$|;vIv{3MSO?DuIecp2Ie=qZwF7CJi}z z+sNoUSygx2>t~fyi^$2c+hK}Aqq`B0m&S0ctuUV$@F(Utj{0Fmik z7PKL=45dZoE(^%^O>&f}DQW0{oz18DgV-$b(*+?qi=A?nYnD}vrKoLhHOauwz1jtG zrVZ|Q2$qCa+f_}{DM^S4aVM@RDMZ(@!gJyV_8}Rkta5PVw}aa8%M#-PEHN>_L3JxP zd(M)%xAwl(>3s&#U%kNc%V*rrqtX{^vR0AH16~C6R?VGS&DZR=ue_bB-d>ngFttS7 zSKb^?Xd^U@kLIR5oaN1g6)yh_V@ax79}rhxP#{?tLziEerjTl;)EC7Fv7jA*2epdn-l*<-FFN#e{CZ?~@o&sv5R=X^rOEgx-s~>!%M-%#=L(sk0KR&;2f< z!d?R!bNvaCX?0+L!0GZ234F@waBOO4tUa+a$Jf0OjXgU-9=K;G-KBnAL%Wd0FmL!G zk&jvQML*jvZ0kjRGus82O`eR~2ZXty>{%okh;3R1YL-k)=`tKk@`zE`2ZxmQKZ433 zxnGQ~nR)EVf!I8mszBgb=>d557G@%2;lkc&!zk;p@kVUIeE8|OLAR={xOU4Vu~Z8I z+`gW+_(x+`_xt(5>Z1HLi1}vqgoVUO6EvJOpeo1UO0I003o|;kT!AD&t2h-L<(Sk> zCUFz!MSZu9s!9?H6R4P$T3P^!ZC2gKM(6ZTBc$Q?Nr^CMEg>%8s;dj!Z56MET6^dD z?UYjVPN9k$8E1WM&9`$6jCH<0LMsqDNj^*`aFMs)-2caL^|4WS#(hr2&hseyp5@NC zkW$=?f!l1UFn_@GTe{CiH{eoe0*D@?iP?ON6eF+uRm2WZ&t>VMdz%Xi!#k5ot*e>) zJGW&*=4=JT-TZ2K8V0`vP-m}!*pMV?09selr_|oN06YLc!N})*$Gj#an2x3mK_G^U znVWbRUU#m9pfLg5Bs7~YK61oe5+S8kWOns2ES02&?NI4B>(+W|M($E zz%wuOSZz~Ew%XYlZk@tdOtgm%J&zH}?A(Z7ZhT^F$Xy-stz%tt`Oxpu|MLJWi_pyD3pF@S(shoX(PKOy?!eL>a_plCR{IoGHkY5pIYcK7Wd% zO`OdVg3USlqf-vIbJNH;L^pY0Q56zirAdg^qE_%IP0&~tuHIDg&_}ZXEv}Jv85ZTb##@nTyt`zE zR`_V!IrUD$m&;Hyy3VgnXR~kexYK!i=M@wExFWt{Ji6LP$VyOUrVdFJWEOvF#l0)!b}0jD0@4xk{|=(IW>ia~%F_ zT|aPa&izol79-NLBdSe~IZZCVW6`-aNs?!8xGP65ZSpE1aH3hemd3j?*S}C@H*l~v zR;09?^M6Ri8j9xuT(yFscdpU$$fyP5L7j!CezVJ69Ob5SK~5N5-#@_u z(A`%LdX{;l3dLY?T3pp_a0C6fg+Hx97{O=v`M{9P^c*k5;k;cJtTgXXPGW!#(s#zik7o70h zIWaWyP+3KGK>RD$Q^zsp84$CfJcBaCx3iDnASpEAbe0+PN#3!`M~+ zZNGWeZLUCRuZfD%6SN%1bRZk}Ua1CO*_{QqxoXoseF2Uv%qa&~dq4}hBu^)Kjc+=! zGjO;Ni7!N%05got8SnA*f&|)wVn5_heRamMwOp;pH+Rv!&?&m``!Kkz?D9%LkZ?oN zq*{DOKl=^#@M&m12Y52Mq9z&IyxfZAMb_AnUYjPP9#|3YJKQ#kMP`s0*V66VV0aa~ z+h9^uB7sN+4DgH0PF-3upTx}b%^$V1mf$1S;D>^h3pGxf!SQk@^U!DcMQ3NQd-EBP ztg2TUq(J83?$prWy>ki3Iq=d}NW;&jfl#|S1w~dRzM8>(tdqIkV$vX$3+wPyG>{Q} zW`{4p6mUvI2pf5EnxHF#Y8~Il2@2C zhh^NC!YzGY8UVUDo4y?(S>h*#fL)4}l~m<)mW)6!HQ!27I-oOZ5Ei<>JuI!uhj{Ju z+N0@w`0F1Jzn}s8fZuju|NGhMJGa%AuX_y=f)~7MH(PP^l+a2Gup@_;Qw1;q+bSn3 z&-5ex-z-4r*$nQs>bzfq1Y=+MnZNK=MC_n!W_EYnsrf8c$Mk; z$@2-PRlhU*53xu&La*@Sub%$%cZFvG&Ozh#Hm1=(EaY(t-FgLCmI9)6vHQsP4G`so z;nhnq%I_k2x5FnDJl@-H8L_YEfX;G#<0W8w#~&|rUxSCl*~IE|uSPI{hO|(kWf14o;|GxE9T}0dm}Q-Y>mVIK&X866>r|Qqx@FA6)a+B~ zKtd&t^amy34XP~OT-vGn-ZHUoU@M%1hf8Okkdkp&r;Hg{TQh9FfyXpAcT}5{Gh#MF zV4Oc*q3?w!(dMO=t$Cn*>p-Gcf_jZTlk1)DFnF_R9BlXDwbl4lLdslIb$X=v$|!%@ z%&x2(svm@YUI09!ErEyu7?k`+2tu3C#ZyCH|+y6^L0QU%pYq3O zbzpR^fPA51@{Eaqtd<#uIfkCgssU98~Ie`EdI3F)k5HNgHIJJ2EvO2j= zw$>u*2BdwE@4)44)e!VfWMRGTGq~>}NoZ)SevV7njNvji z#5Qugq~bkA;-KuwhY&x#5s8@ANlr-zo>w)zLb9guK@$AM0uC#0Gex}?fvk1`k2p$mcu&g#{@E{|8wlR|Q7knQY+ zeNRG6I~Os%d_UlcYXv!y9c?W3X$t0JzAa1UC^-wqlx)4S%c);T5;J#LBq_hmtO_w^ z@9PeZzxf1O;z>65x{xnY+N`a&uvRT@pjaIIHLyBeIxXH$q0QEBk{2o^lzq@GrimA} z%~Vg$KYp4(XbHx^G4^fdQ)rX0Ge|4hL+>)}SqMFE?Sn?puKb3@*$QX5*c|jRvPEE+ zWn~=UNIz9DFjhXt`Z>~UEdu>{z56EGclIq7iM#CZ7Q~VfH|Ao`7%MGz_?V{_wHQ0a#{m3!wd;C5uaM^bY;kG)&~Ha4kXjX-6dmiE9XU(l_%}aJm)>RrC-&i=dfLI1%`fYfu%*K@oAOk3>xp`BGS)IdU z=PRQ|hdhC9%r$Skb6UE4>sNOuZf(IrSGmJP-GO#6oW=iVwd+J7A}%p(YT+&lw|IMm zT7KY&^N~E1$SuC1s)z8+`r!ST0hHhcJ(vMbg!y3Wj-YLamfV}F$Pm)>*0#230*mZXSMgQKb0^$&EPucyhDi|c**Nt#my*>7=DrGr{P`hqXxR~;xp7j2*-V5KB;^{b7hVNd>G7RdMP?N0$TR>8u|C`7s8~ha zAi(awYy=$MKI+7OdG=F@a(|49&wv<56Mgt$f^X4Jce4c8yyuz&FS(TmP9dTvJmLZ^ z0+e4?H0;Z^my#@fEtMC_$G&?s9UD6Z2qB+5bSP2R&GeEaoT^;=dMGc%3pu@pjdc0$ zKdhKY0zzS5qJw;Zdnc_DG>3y4b~aZ}wv{xrj_$0N)l{YXlKnrFO9Q-Tg?>^i0*#wK zD($DW*`QS28DzEo9OyIeJ0{04l71@~3FbOUbDjsqu$CxVoL$(j7BDu*boNN!KY4K zG~qMM|4AC_6Xbn&<(<1?>yw2NJpVYZczB&Z5--W>Bs`gfJ?HPPFR_zD8CNPGd7c}4 zEh4>BKz9f2zJYz$pn3~ercixJnej3MA7bJaIy#G55r#$o$SEhm=CT4GiMl+Lnd!6* z2p0Ylg4mM;VHMtz4zEUf=F*}HSR<-$rv^>qmgMZBp%4A%ms8ia%JAncyQBJ_gn(N4 z3%6@HzQEQ49nP<&wPh;$brk6a_ozs_t4U5C<_hD<2vv0JIus&JLz!Q%``W6d)4|Wj z^IVe_-BhfAXX#wlD@jWNkMmBWj3K@jFz*<0iSzT*(2Pz_B1+fm-LhRfvPR7|fJWgT z?R9Jofn~(QpIa1PbgpoVeB98pZ9+V!O*-e542SZhPGiqd=aU8^+mG&4XuP*KP{9R1 zU2wCP@)D!GLGZ%oJx|%U51J4}ZyS;2qXq&_!(Y^u!HET2-Q>x(G}ESn!_xHbEXC!Q zmaPeB+2*J@5a0H~7dN1R)cg3f`O_G1I3sMc8`win<~eZo`pS(>^zfFXS302hO-%s3 z=&e~U?|4|xa;{RApA58DIMI@HO_rqt(gS2}47m zf%BTeUz$QU@rbW1a^{<2_!Xf}!PM6AF{;~6$+hQvE7+es2`x|FCV;iSQ~wZ|-}U?x zVC{@A{!MWI3+Qry;NlGm=lR3%AA|!A%_3_y^NUMD_?0Z$k$v;GHT)#DJ(!N;O4_`+ zMe`W!S@zwhV)b$0zT6t?3{(QLWL#(8z$?42$>qGNAu6H%68qs{x3F(Ym zMV7(*Tp&yzBt~i!eBEN4N|?nQ-I{|99t?YaFPPu$SLw=i#Mjtcw#f0!^L6^)NmE&( zqQ0?ZV*ISK(#;!AcBWI)-+j|l3@t`&;-~5}XL=<;=RMUHaUyLuvcM~zbI7jgWkD}V zS4SHz$TE;Y3MXU-L0jjhwxSJa;O7oTL)WS43Ux0UI=u=e03oKO%G^qbl-rw34aD+y zyv1PpKfV1uLnH*w+u#L6!mQsjC-m%@S1}LpyCA-LTc=81j5PMwvXzQvIKqV_SZZ#W*+(iM_PFm(JV1{3G3mW5T1*~&~?@av7dV0$m z3(Tpkh^x49N~aDdSFhzboF}zf?X7qM>9$Hju16KD%ivdAd=fb1?vtyZ@;1fT_5S1WzH%EYl))pq%dV?3sX78OJ8UkLOB+lXBz`=b$h z8NLyEOea?Mz(dNn*x*Vp+RH`9y%qdY`bttaI){ovZ2h-KC$bN@Dtw;qeV0wz#x^(F z=NCRPZ>*D6zkZC)lYYMvQt1W{V(TBZzxsqZGr-{KE81E1{-fSZU;8>xqvS?b3&<`?kK*KA9}Xz?O6~d=IofSK0BsO(Lz}+#rwiN3rH=k%hJsduaYd zdtuw1oIEU%fv;ecW1=C=rG+RK8CPRe;?$tye(8kU}nuV{Pm zxLu-u5Vm6SBP}}^vM===WY;ATwwG2IRqhWPm-3$qd}pmosf^N(&XTK9i)u-4EqQqF zFh?jH-9s)ZC#{GWEL4uqa}h3T!!li8m^AK4rCCSNBS%0;mK{h}8kYm4B|#A&E$hHM z3w=K7_L-Xv?9%?tdM%jHU5d3A6XB^K{OUKYL0~Jf2_XF@Z>#e2(cu$ZcX>lLMrt#G z{RV0z09qq1u-(_#l9LPtXD}kog7#K3rGbn(e3MvKeIEtjzA0wEPE^CF{mNLhd>R%` zN}AH3Rm@N(o{5_jTwDwweYs0(2lDKLfJ-!qHMQ{2@vj$$8u^g@r&gMT;$EvR;z}$+ zHYX-(1@_!~g`bv_JzFJf2S4!B`{HZukIO^q{NlUcvgn@smD4*%x& z0QdeT*Fc|9_TfD!D+sRio{2sAfLo(NK!gvcglMzE&;msSZa6?r-N?1R z`Jm-LMGsuu1nT?7lkfKb`yOy@3xq}v-~TrtpAh#;r8!#V*eSRT*#JMK$Y*iDeKvDF zWA&r<*OZdF5%C8sUVIzJcJ-X&4nwg=E*66hnS5B&vvWI+#lDR9x4UI>GD6OQUA_>X zKt)fn8WIkCWA_3t2HLDfA5&+Gx#d#{LIBeXiMHm5e|j_8vpE)Cy}>%DoqI-ooo|rV zaKbXjEExquGTva808^h1U=ezmP9CD6UV)_zvNqP@zB|yf+UNBy&i7)_Vg@O*SzcUo-z~x^e|&ei##e5tX`u z%V8)$SbWK(JcSF|Ud|yg(N0@hVJkbQ*e2_2d;H9UuagCelGJaZq@5{318&%U5OMq) zSP8mcQ2)l?IV{5wpQp510w1&6dWzCMo4`()OP<%{(B?Z#)89&%4WnVWLzSP)fBcXk zAn(=hCDARlI44RtP#kXVU}<9cdAzMsF_7FP!aBHGq1oBjPlnG6ArWr9MR?yf$4di& zmFb#*S6e0hMFhT6HJmW6jvrFROv{X;M7AF~5bk+#gFhD(^nbEu>c5|Fk_b#Q=vbbm z$LAk?eJV%~f&K)h4d1&1$R7kuIA6j9&%+tPnRZzolSr7N`WVap5=(1x&5k{dAINnA z#$=a>(CQRHnq=YCwxqtZ1sqXj3Z;HDy+0=Sm^fM=buyI4Q$F(ONsiZqkoA&I`)mzO zfb>6~x;4p-Ji8k@wnm%5p5=1uiB=r1@}5@$_L^TQfT_dB=MB;n6SLzqDxDWRc9t8H z0s3EXM`~;NmsB&&i1)7GQyjyAduNbmF-o14lm?FHUH zt5RK}8+WIqpv)z)-XM?j{{!7ulK?y~e&^2jKkw0h5u8B1Up4yE!hRhW{N&JDsGVK*Q1*T1jtUx99=hpb=sBjdm}IWWf}2a~n;_2G7} zzhXGvRy#;5-yDG{N~!bLzphYX!+|Wafo0nKyAz);CihT4gM-a=@fO_Y$*{w=pNWJ zdBk`ODhFAL&IJ>XSn({!Q&LFwXaTQ59Fa=EDnsz)EH|xTZx*6}sRo zof5J#({j45ajiYYgMoH!%_Rx20b*qcI`zfMT<T_{j)9%ykd3z4S0Qj!6NashX97f2PA9$QuhKDz6_ zgUnxnAoIAL(rn*nSIA~ZxW+wt4{0s;f<8So=<~xhSn`K&6~)rN{>2#sTe5ez!85$V z*0P{bbbevwM|$MM_IV%>%9nGgGYZV`Y!Ra~jvM|1-6)Rd^Z_8xKI`!Nr=-Tu*}d`r z45Z}u8q^OXeL>%LX~V)gd0kp>Y-7i@^e>RhGl-mYFfnFi|$#YpZBKbjn6$ z2tjjnqFxs2;kVN$BrQpZn7uff{vpg{QgWGPbTa^N0;+0Ql&RUAOlyk+Zl81O%aEL< zzDf!+UZt}4_`?7C`b%{9?)?|O|6qFuJT^sY~>rdLp%mS5m4^J-LXMb~ZIYa(f zN0`G?NAP8r$SaZ_*&Lk39Wrcg&PCRC9B-%I)tsuDfMCsqKg1~bMMobf6lqAvYauZ>R(K1i4 zZqr+lvzpuK(N0$9MW}n{pM+1;PF);wg zxKg zawM#$oRw}QZQc4RH|uz&Y#2}+E8z3CLPdN*8&dX@7NLZyz>^k`FWeawkzl|z? zH#Weqb0jlD@?CQ?)a%0jn;eT*;qg}>QU@bchPRvWerhJ6Z@4{cc^|V zKgJ;CdW!LmLoI=tb$e=sqINNp#_aZ>!Oi~Sq`pUCbtOrCNhJOuz3+GTl-YJ>2X8KG z?)%=$f0Y;=wjdN@UVEz2glstW`NcCd__e~TjHtONG_7^VxrCo>8uVm1)#x!U*%r?^ z-!t1QYx*Ewo|MpQTrd!y<%iU4tK$KN04+WLA7gJG4(0#0kCzrHlqJ~-q3p)KmKYT! zDa&A#Z7?JuJCznO_AGU_Xnk8W=(%S_bovf4U2=w7cR{M3jv~I(3-#XTURi6 z1qL^69r@@Jrt+p+tA9Pvg%}WE`vb2!*r5>U zwSK*r2qJi4>Snt?r~`HOD>02sKdzW-H!iT93GT_$-k*Qp-UPUNnBGyF&shV626Bj1 z8uv>KZyb8hyUN3l94{Kg*LBu?C&AA`#JJ`yK7%- zsOAgxU)~6=j}&)dl*)c(v$t5U&jqu;El@pm#jd|6f<=$rggitbJzLt|m!1E)3I{7d z8M(Gt>Jx{);WX!FsijGq4UeivcfYY2@hNVaHlE(GP6$q)+fmoSQa$&}QttQWJR-?i z7xJ(F?;!WZ0j&x1rJ%gqgh5yEQOb<_wUY~SmYUQ{9@A940TT~R*te^9A zPru0F3KpE@D^AjU-m+|4X3Vy$Po&N?2CC*l&W7!1mbOogfwtF3L8P3@lI0Vn;iy4~rr-jPX&_fN(#hz3Kp~F46#1NAf2DB9^(vWXOa}g<%d>W! zZ=Gc4&~3Pbg5GYft>as&QDYtnnD=>}n6#+(4Sl)qI=W73GP)axTb~xphtk?TzggG^ zpTT%}SY3X!{eoEOWBJZamBiw-3eycd%2_ZyNq+Y3Ci+aGjEsa#wNs}n0ACE>OQ__a zrR8TTH1@&?nkklBT6x3tf4Bf9Bl*|6cK{4hhQ>_n0Eo1!YZ61SP%<34p0YN6Hl%XV z0z_hL$$-^OU6P>*Yv1uJuc>PKw8veV&Io0`HMR~^N7c0p53pBg5NRQG*dOqn1KM80 z$Qs|(i8U0p7S@TQ!Jk!AaRwdGfTaZP)|$D4-4z)HpugV4Y^VTiA4AoZ+Vu{A5xKXa z>2(q5U%QEM$BbSV<+~k1#Dd{V2G)AMx7k{o|4G^xUj#?|kcvOytedGVa5nxlfj9JP zr?wk8#<*VkbLV;A#cL^czd$ZvhJD$rFclo9qL*BjQ&2E`$pYcXU*~zTG^x{o{7Boy zD0+UtSo^k)i{~ck@EFxi;+r7s+~(*CG_n11(cqn*B@fIJ2-5W~$Mp+u(vMX3-BD*6 zs|V%>j%a*}*M|S$+rzEw=^bp4oyc;Pr(?ppsiZTC{=eo3M*!?J<0SC^|NEk;E%jku z|D2b!XLITWDpF0{#XtnhL?>U`*I>dQx;#>LS*$u2UX!BG#%cK~hAT7epv9M?8GyWj zS0?bg?YFOXH)DBHkxs@Jttkw@%WlNKF3<=DJBcAHPLVMFYfX6~+}&YHn%NhOhA9JO z95x-W_#5#q7M@E&?{noOdtn=0Gl|cU>**}z6h8{CA@?Z$ls4QuFI{F`wCl`VwJVcj z(=TMQ{mM9utXr>fp-x6VE2b);sv~?@%A=IG6~y>y(_P|_9cH=_zhs?w#C#UDZbV)y zbR4o}SOPhoE-8{<)a-lC0Xm8@!i<(u$nwh078Oveo5eb=3V!TjA<918!K1=zeKaKz zw}?dh?$+0$y2Rxhyn@OfT@Q6V@b2Co9-u%CRFig}!Rz0*7^a1~GC^jg{Oe%g_k-)+ z`+A~kow2)mE%&c!UEgx!ZY&HT4qqdFY8fk}O4rU;p^SCx%P&r5c!=9b;Tsq1Y7^1H zkEgA-+?0NQ-f9^dLTov}yZrrJyVO_X7KLUyueJ^w2+VEP!UwXo*85#d(%a#a1Eji# z^?y6ZnT5a2_P`xEnzIie-eCXpPMrnbsn>n&`JoINzyi2wxXmu4YPp)yK{L-RdA2lU z4CbaCBo{M(JbKGVN~#7V>mg7Y>!-xF`Ls`M(AHocjp2S}#kW1Wi!q;dXF0^-b8c_U7*o%CV2QR7G%RZg|n8dL& zGXoT6Wc&ueYvWTI%9!;D>h+uLat|(bPNDgDyytmbnW?elY(!Dp`=$IazD;Tu=)Drz zi~f+Lzp`?^svKdZ#LK2bLpdk`WkSbdzup_w+KEikx);sni=q6l{-#(kJFXoE z=wFjppf$L=i+c+l`Gj@Mwiia&Ex*@=^`2d-9Hj?GJ{u3XUy5RvQW~8inX{7gh)uJ0 zx79e({O?)Fdfj8=KG3w?To`yAtnU6@}6NZlv}X&x1O-6X|cSj63wiS<{z7QL8DTkGa}=i$>uxF{EIb2c}ZP&$q? zCY{K%U&B+p_V@*uL9Ed1i-XQ?v(h*n3x2HW8UGJAi`@u;)5-+~#kK5Dd{{E&iH4e< zApng^9`{}7J7(Z|*ORO*1mYc|CK(hMt9d0}{c@N?EQpM95Hhi>fYx`BO1wjI9}fmL zMkx0glmHcMh`|Q&q%u;ik4mU@SSI`R;ETKw(APUp$0oi*cFizG_J)#Wm(9BJzZMjx zw1+#8zHXE7qJ4Sj?a$YAMSX@A3lsd7&^jva+pwdaHPiuOmkLb#!@usorZD;BzAIbrvRD6Wd~os z$(_5tg2R=b)%p`)M`^PO6LA2!Ws~QNRFh<-S*!W|`btmW4_Gt?h7;Hq_g84O*fP+< zmv~IJJTeKVxfRh2Ljdw}VT`eUJye0F_*i|qze!G_DV}J0<=2L4QQ8N;gEg~OCVQ0i z4*`oeF30syfxQeG7nBD6N|wTi(q7L11<0xC?fCRP`)%4-Ldo^}UDSHo?0cGx!yn}- z4HENDG5gci7*%?JXfd@B`6Z)iX^2yYUPHXNj=n|S{CmJyYXb7^iq#zPTD@fUNq@(B zgt*Hp#i}Q;UA)2HTQMtNv)(KofnaGCOvkovA)Fk$M^u}=&f(mV=qFM1&a4w2R8k0L z%ilUQbio1NqByV;GXE|w%CI;>^~7u>;*4fvGj}8Oww6EMCgqh?`9i_d!D+L|R{?+f znp}T|hr$1DVuEN&Dy@d5yZ`=DP*j#i;w(;aH@)z00-mCg#BrRRVsgJdGHsddt#({eeLDw8g(NhMRES8uSO~JPOZa8yP`~& zn*CVBGdir93Gaq2n-IqsIa$7xFg`);OAE2LX%L z=GilaxG-)auWrRk^Lial<4Z{E=>4zqmF-0~gFYXNAC#{^!1if`{IQO`o26%#2L?AS z3;FO^-`S6eT3GTMh2{0F5Tb8nOUD$x+|l`i`HTSIITgLHw6H&_Wl7n#s*u9p=lR&S z>J6R7ZAdyA5!d}u-lT7oLfcUl1%>+nv;=Y?0rr*rN)*muEhz0V$}*gyu`_PbUY9f3 zQ8(}YI^}WioD+@gmdx&~2RL!j^*#S0*t{e{B7@mqd7T5Fz}>fKIlbmDufUlD7z@?~ zLxD_fj2!iy7tWSOh71)ASVDl_A{3YWs%%OY{=28}X4r*z2i->ln;9tg=4oW_UGyHe z7lu;lwZ!yv>1;`o9^AD$xG3^+QO(SnDEptvCj=|XN1_3;P!eWLrSs8pn#FYOd4rVX zpSJ%>%rE9|15I_G`eCx>-@N<}9=k#7wKc~ZyZ#2MY|-(u4Cme~J%{mz*KTs=jz<-A zBAjB<#r0A*_#+NMLZ*H))=v1##E4l@XR6TqF4+PH z5^JkN>!My|k)PK#$uq*aOY&;CsBr1=u8)brMQ_nJ!SG1SZ9ocN(c-k&mJHUiiE4Bd zh>KLZ7Bfll^L6B6*}but%Hj!o+Oq0T=Z0*}KWnd>Z%n_{G&;Chu#2&hz-hEsqU(&{ zH|vZh{dcc|U)k6ps~e$S*x1!l@; zo_jjcNvo9P(51JC_Ld~`u8?`_VWi&8z(?=CgWlYLE}1KG$>mSi>SefZmO#4ks^&q4 zvzkBB;S>$K_bi)M@UgfRhvg*CUYx!*Y%OnB`cRVWRzBOFwpz!tN~%hlwd1;8MsE&t ztA0Bf#z-jfSV+lRogXag{%G;WM>Ib7hJWxds1|DcgYvtSROhxSS9R8b40vmb9M0|i zT&*CLIZC824b+Z`p+VQ!u?|ijyK2*F<8RXMKbEj*1oIIR2bdOx9@FN_M~=6|($FcL zP(Vs0$mxx+8=o*XN%`ZD+WKCa-jL>h!pg!~R;dYhk`=d4G^W#}6yc&H{iiY*kC%X% zcB;`VdQG=epW8=r?<6QY_xDiJB!$C=0ylV1#!dqElS))$W=kzB@*AV=0q(m3Pu;V- z9=`a+E_bVYS{ptqkH3b&*^Zapq{*GGoOyHS)^OE+9Q3x5XvaRlcaYu)V5(m>uP?3* zT3w4xx2(!JI-&FLu)It&)ijaEB(wilSR&2>#X90FOyfVczdP3+0%WP|H(YWH582)! zo^-MqR)npJpO#v7*?t2HlF?3mZO`QqXn*=k&W^*a%-5kJJCp8d-6G1u1q4JR*}Ng( zpoN;E`kG_dQ+l3=I8rY9J;~dc2kY1tbN-TJa|4uDYV+}cQbebv%hm8j&={*b!@wp` zLolJ)r_T=KU0p*X(1k zEq3QJV@s^5mM{O)kgy$9?dU;R*C!2+>c!R)0e?x$?(?|7J{ZQkkHFQi5L#<$Ir*zo z=KJxB1rFFe$nNWksO^2jMK$O$pS5QX?r>o!Hy!S%3q9OIrO*3mh*B-PYL(s;X-hog zN=@LAaaUt@i^~iMxo;O|jTv)jDK=~@nKc}_9g*-rKQt{w^rXMmw0#UEBnB^8usQ#G z+rUV}+kOb+$Q~*?C>Y8Z%%=o1RlF4HTNjw2M5$a(A>`S;d)WQ&1DOvvj1QG`C$}U&B3>?TD z*xIpbtsn5IJxFt}Y2@ePn{RLr;z|Cib&sH9*ycRsPQ0D@#DjttB(hj%SV8wCMw8*;J2_x7d)wZYr!L5Tp?7^0Nt(yjPeWdn*w5d3aXo|DkuN~- zWKT6{my-_&8?BM2+`;nIiFMeN`@md`(|w!{?%NsHZ>+g#@;jqF*%=1kQZm7BG+-!7()(7uafe)6GmNaPwP(YIWV?WVAOOhf$V)0y^ z@GwdInQi!#EbsoXY(c0|73*(9Y92en;O?VE{mpslB`3!Kv7z=z|?m>$sX> z^X9bmZTTHQ&AYj6%3q!AJvVFV$ZhTUSniu&A5QtO%(_NP+B^L`+gN4xZF!Zb;DKky zn+oy41Lf4cYg;8w-F-{z1XJq=(lvVpvCb8o#>0W@U9W?I5lGFE>FtK-z_LAUz{y{u zMeR{f=DMs;H@YF056G+B2fjU`e7ucS5lPWaBo}-6kp+goW#m6($=^&ge1jXf*?(XE zFTT-|@EivNRJq&Aa=Hz2Qjq?aM*Npq-#Yi^ZVsp1P0*VZF^{m_I_`8YpP!*kve>7^ z>Uyha2+2{zm~o8+_Se$=8b-K7 zsq9@Qq(mSl5(vlf2rzwp@7TmGrl5Xg(Qt)JFs}?NkI$8#D^w4M{k(K8h#LM zO38lH3zn#kg4AF)yVjA(Q`H%D7jK7Uc@wJ=_QPM7ZHSE!aXs8;FgynLF|F;V@Rtk< zZOT7Wo9g^O=lH7k_3S}b@>VwvK4|W?tFES$AmA=RWXnTT?@4Mq1Y=n>zV~Z5*`%D< zldK4;E=HuqRVh_G-BV0emdZu7y&L@flx>!VW!g-B#p=Vu+Tu^r!eu(GG8kb^DLPh@84qtq;o>n%F!*Vj>=@{IN zJi&Ee2JRrsSM*PpRNDM1>q2}K%`(OQ+&$1t*mrHa`n*J=z5TI7fC-cUn74qFqPJ_C zr$Q%!0B)GDU0E_xf&LB8zaur55r|(y ztCGtjUCHt1&`&v`OptZn3G}^RY3G?2s)6oE$W+tJ`Ef=)POIQ(m9+0L;xe6$L?rX? zLDXGkVZj7ZK6#^i&n34F?0B#q=9D_Jdl?!2cAeBvr02BQb^DA~O&H$U)!_433ICL5 zwxS#6!Jy|cRclr$vde#xue0cZPbTYu=i}lG@PMQ@wOg ziXd^$){_{J^$+92&)ugu4oT}#S~3pEfZbEt_0v>#KmG)KJu4vNFLoIykO#PW*r8P| zT1jUp{zqoY4@mV+KkApuyv6g?(3b9xFI2qB;3+;J^WI_saIPTz@XgDr)(CD05Su45 zB3Y}s75jmY#OUwTd$nBg3D>m7!mVdheA(24nTC>eT1A*_Thvctw^}5C)gWG&rTk~~ zy@V{-omOJI5}bfMP7dKp)MWgaOF?~wzNX{?+uD4JC1{F5V|nR#lsnV^r* ztJVN9FOHoVm|dTT=Y;6_jnM;`EL% zKT5^CMR;;{yN#F)5MA=w_xe;Y4c81bZcXZXJn4+f?5=RQJ1$k^DlM5vsx9@loo-C2 zao`nmxc)Ol+mw!$K3R$^ooeu;DCgma(ZXi7 z;^-Y8Q+w-xI}tbE7F~A!*=Ji~yA&&hPb|W1!Qo@;JLu&f^1=9Jw13IWPcD1sM>VxY zj>9aNcdiGU-k5z-!MQ;=kJS% z$&O?%b4r0@(=~EHr?-21GUI@MzC+KqxSCGnWzWY{#HLFS$#OT&ZGy@{S;=|S&*E0{ zhmtGwr$}7{id4|4?5KD52g4lX{}!p8gWyHr*_yn&THm~+4ndDK=Y*~!<+8$_Vk5a{ z(#zpq2AiIm=_#wa^@chwM=cO4xvt9rbhg8;&ALgE6*u2@CopB0u9Ai_pBU6CMleVC6``q z1sLHSj~X(rN8c0`pt(>rh7^sd%)}1c@!T@8ncHOed%87XB-l_Rs|T8@J9;|0*fI{65z+rb2Ts1cabEyW^-|62hc7VaVj1 zyLmU_Q?rlMgmzRb+eJ$vS;rtKQY>ckxNq#;Ue*M#j>(6rHm@wby`mE}duYVf#G*|` zZuzS6E8kcZfRz@j4ZR-yuM+MB7V4MJ^u?LDkD@HzZZ55`v!$Lj&*}=6tyn;HdO_A@ zww+@y1#uvi+EgS%dAWrPF5!cdZ75c?-s0pi#k>@`ldX5zGF-CHU`IEV2Q0^|(zlg% zQ=Bwvmf?6MIA^9`>7XHdXqxcK*&+KcL(vgBjK-6 zgvGh_8~HNL3r$Ad7uC;>q|7Bsr@{Z}3dkbmav8kKKh6yui?QdCD; z@z+!8bOSyy_e7s$)4pD_;|{E@gg6jKm^Jnm^%H*_w? zF)EiB>@UzpicY9;txu%hs;iDduSeD;1GFnpQQ+77 z##^j3FO}r{Kl+KYwDlN!h4sPiek(Tkipv91V@HM+WTj8OxM>Ki*1??~5tZY9^~X|_ zE#<0qq*^tnVi~8gu3(LZP4r-dg}#jFMO>A`?4?NVtx>ASbp8cxEgjOlX9SC8bmxWN ziu^6?^1fZcG3yl>7GTae`Vr6#QREQiV|ljDJS1S>{7^$iejXU5z5g_-MQ-6DTYLJs zTpfi$3{3##PavY|B+^E>|GjnUK!j)Ptm$#Y_vnH>CJ2FlyELAKN27GYCYsD{)q?ue#c zQKsGe>oc>lwz(G~o|Lvo@J1+D|J3M9N1alaV4o?@Ir{dJF3i~Z_Qn#!k|h~~4nk?a zYms{7e7_GRM@Re-aeL-=xV3PsRx2CV{4Z~oRPpH1y^ zxXw(D#uAI`O*f-5Zztpks(3{L~ zHv%f@6O54Ya>5yt*~yLNl6Agb1Yw@S4V}w>mL3&5*rAy|%ZITpq}QF{HyxX<@{%+E zMZ$V3h5Of`qS0ecXWNs}zM2gM`<*-MB#Z`OUX=jCIQ)tmv6)kI?`DyZ5Y-yHk^MT6 z*MIlM5_)nV3O$~3G}Qhf*L!CZd3U3zEte>hM)xpUQyj za}ij$l67BkB6;yRWS13C1O&9OXan5C`xkp;vZK(*SIwAo+aUh>3ns@2J%ZfWq-~f> zyZp--FfV`yb$XVNV;D|o_W3V4#WQlT$z}PT+%A}|6TtB`kKPNgc2V~a#^Npv02GZ9 zX5a{LD4-HQ&g-`Pe2MygHa-fRib5a)A6SI$$wI^32#Hx}bH93(7}Mb5IpCOoyvcP=3LKU@HovLm<8bm8vaB@VJZvm86c zzq)q(OK;Gcr@`lBIsRVa=StiumVom`o>9M%8=zSswzrp-qmO5kG<#LplP-a0e6x3A z*ZXxI@Q2P)-qkr^dWURdJqm9?qlES9C2L*JpM{TqbP&vJ&hM>m-}spF6J&Qh1CA9< z9$5XQ*0G^nM}kP~S>(31`)*fRjF*tr=YlT(?(WRnG90N<>_Z>XhSz>s4VbxVs^N7O zi!_b(>e3MUQ6Inpp}>A|_hbcG8t5bXSd3{Ld3|BzIsaYuZ zr3wXs13tlk5Vx_%);>6Vy#qSs=uNMa3Eug{VAU0+;`^VmTh46zBNB(AP-6h)A=+Io#qW1_W_t%P{gr1GLa1fA zZXWytdc{zRsx){%0}=#_>oKyN-baz%R?oH{2(LgyxveLXqBF;$+s;kR%^s%5RuFB9 zb9ez{3|?8u|1YHAVahqcB|_n*LhgT?vWC+3i(I?g{72>n3_*HA=!Wv`fRWYaTk{5N z%3UIN7n_}4)-`T})6O?@zU%2i*rqM%%B07@GOz7A-?oT;eFDyxsM>5C)+xz8T$UG; z>dx`_ZgpwL(XF~%G^aECEWCl*_DWFxef~Q-DG-B%<7JJHswPE)BB8#}tuR@5`l5 zNKY~7%*gIMS6tPA7|!^PNPpt40#$;@m_xeYAju1J9I+U-$6I=CbaP;Weh@4$Hsq~c z4&f^w)}fKM8{6Srj-*SiSydg1GL-K_*S*}G+BI9RbUDknaEy*h1lW|j6L5)&DaDAJ z3howmIOqh%Z+X^#$tZ_v7zC z&aSa?#5voi9?KKcjV&B|9U6H__5%Q$x=F?^mU=kW`V0^g_Rwq#Ma@ zUBI%V*hlWjvt=`=$-b=Nw%^x_zE+m0a>W0|lPVAB%bh5{zvZ_&*yA9a%6 z7%H1rKF{{f2s7O7PnkZD_O{;y`(qtTL{4nkpKPRiQH zu9uvw6o4^-RYnoT@6uA2kG}V`xNgi3kgFy}ss$`|ixk+iRFcnvDj3;o`GvYd%T!-F zEC_QA?9?yH@!C8a=>ZAF^Nuw7x{qt>tw*m-cNKmNc7Y`)Zst(^JI-LoUHL5crUB;F zQPPlOk(v()TFBIGkFS`5dUkeOP~F_(;s$pyo*Z~FA{sh206vyz%uddPHu@Xkx=HNx zsfDZ6d6cPu5}PN(rcjLEQ&-&3uZi;RnbGZ!97Z#)*((*@Y9++AQP>AdwE2$5!Mcn2 z(=`Na3O@)c!TIQEV#}cb5T-Tx^mciXBS*f>%bB*duWga02nJR(J}49nJV_H; zvAv-Uwdmdgc&!%D$RwA51>@8H!D~f~R2pb&;pC6|D=zfdZqwMx_dukuFBY~z#NV0e zKpk7Das6d5&v{+f-=0LYod%PY%+5YKj$kJwGSQdv#(hyuRYwlh=hF4ga z+r9SMK=8ADOX6Xt;>ur)gX`C?3oDzQFtivq1Me-q+b%PYc=d7ZQvEpc#KexDq6dZ@ReGd@B%L%{wg}Lt#_AW^G1(Np@6~fBKWJBpA5soy%&jN zY{>G{AH8dl?%owozf67(Llwo+*&5w9A`cwTX1KsyyEy8Aj&7XoJhSn$;qGR zjcH3Ot~y4%So1jNxdnHGIVHof&2GR54zzAp`s(pP=%fR;0mPJGmzwzzexzJ~r?+I^ z*+OXy~Ym5?1z^=g{dDM(be3>|ZJwSbYT(Ofod zwPeG4SvxT>C5bzf;41pEB6UjbjTLXX?VbPu>WLlDkda3Gi6E_54~K5RXR1(X+S`9D zvL&O+Z&zmqk!(#h9bQ9U-@6=lSF5KjdeV^F!M4 zcc{wQq^igA5g6u+@YsQ})Pg%6p z6DGp|)=)|hU=8;M!t6?DSc6mpzPK~z)u4qpLz@)jYmTY2*4w?KIjnGjCiURsK zkXBtVyFKX(c2zbvCY~VA&3=v`{sQGq9%=<^mY7X)$Q-yX0qmBrN^cB4cnTA7^M;$2 zshi1YEjhzb?YCcSdpLCu>TmJhJrLSK8;>1t^d?(D^-5^$$&fmLJ;`3p=kV?8*ZJAQ z^~0L#T3}B>nWS~K!uk5z8>wDBuy6UH!BM{zwb_;b*0F$`3xM?gujtsVjQ0G-T~n{% ze6b#Kda193-J73W=RU{uu)VYpg87|o&e5ylMv1+TXo5kKubl9kk?fEUcdZo&te_FeWeDxG``$0y*fuk&%b0}yw&#_(KY}P-hf7!rxsOos+Fj|-|X-# zZ(14kt#<$A4dCT);h61=Vh1LRm(Ep=QpISivrP$5AfQi5NU6@MjsN!Lx0tE6(c4TN z3{^Rq!nVR=tbv4H!hYg%>H&F=l_A2un_r*R60RoUlh&nnfjx45I!lQt*iyx}pQ5(X zvzXRzoB4i_-FOGt*hljNpAjcH=08C7$riWL6-QU zBrVo1e1ay{24YgZT&pQRR46$L9Y(c>C!`vVt@&DZf&%b9kEcqX5`f*lyM70wDRFPE z4yCn$z=hMTL)3ZyDtG<*>%IXYONK67tGb#dJQF>fsH3)0D|;3)vzPdnzNgaHN^1+slCU}?1%>!C@lGqn9!f*3>g2S#@yqXgA<#bX2#XelFI{eCDG>o>3tdipV9k07xRD;K8Fi| z_5sWfKJtx`Z56)Ov;FDM{Iu8OL(flOIsdm}Z(gv&Qv$|IDc)w*_E@X9G%ArfS zepI6li1E<2)&G>f>%S|H{=4*Dplw{~C(1egOPda)>0>mbofnzSU;d_SV6(7)J+#cp z7*G+6e_t(qX;9tA8U&ZKQb2LUeXI0}<|x$HxoV^$+FIfS(Q)E{lsxZYZS@!BOFW{; z+N-DY*s&pN>qgrQgP~@ulxM6X+hFoGPqjrQ(N1rB&G1_F=^Hhpt*b6yMeszw=%((W zyD!$+<=$_KlNos2&1a6OC8BS>4NQbe#J0bvHA_{BDF$AGcIkzOiFXrPBNnV??`Rjk z)*kuM?Akne5!r8eN+{DmV$Mtp0AQ%j7I4YOj+sv*k^K5(<3iPNMIVBAQ{lxRZR;dC zu3v_9b3sre1=FUSkF&m-T5#DM%5&RkoPIGxXv3*Ko}y)&*Shc^e+0gaLD>yxv`W36 zPb0VSim2823I%k5w~C$c`l@|?9cg^E`vxS#h--0a%5v`K_bdb49Pt*e8++3`n9Y}q z!*KYpd_)G!k=nMH5-;Z~%2dlQ?|9esUaZ`Mb$_ffbun0GR(i50sEdaPDqbU9@5*WX zApytPPwTc^(VhjBERuHuA0lha_wyVZ&KKtqF{L(L$(MgbXYKS&nmhVF)ei3tzN1B^ z!3~gd*C}Tp4YQy4Kcfvl-~BlJ-Q;tAXUte8c#oNl!e-|de(v2S{f7&LFk20PADR6I zPYoCrUw?V6C0@>Dt~pGo!)9ii#AzWMeofORn!;tQD=L9cdDTAy8vs_GhRQR1YR;LJ zzz*ap$S#BxRP4?sVX6ygty$x-euP3bz}PKz-UFJq+JFqr(aTKNAzccO0f1L$of_QQ zF2Cq;y7E8p7Oj|g7)J|K5hFob{qw&LIWPEV92)gFEff>2bOJ6_Iy;0~w@S98#w22? z!7*5`;XcI}^10;=lYM^-pHawKt>6f(H`c*0o~0M8bAyo6 z$m#=sgr7>b77mPdkdEt0wZeAZoDo!yvVw{5Cu)~INt?iQVMO~Me>zIoxL9595X8pj ze^dDMWu@ur3dYA?uKdxE`hj!{xN<2}GE=roBGnU)DfW1Yo^{et6LOog~z0s*I&pr^Q3OPgK72AJxTO$2Qs*~J_LE>rzUD$&{ z=}Ho=_Z=;qHxNZx`?)~NhfgP-z~OC5B4-3PyP?bESqL@ocwB;f*L88oB zc=FO_2SfGRcJg2MEQ!DWLpj|0mBP^W(~+7QJFPL>jeO8;SuC>u9bn2k{~iKL3?y~< zuO&!hIxgx1Gu?kr+WcvgwvEe8|4PWTTWO`)vPRiOi~XmyOME_xH4ECCbQlG^=C^)W z)Pg7S5N)@#`&n~v!Gvm#!(Nbi<#{f?CmM<0>y*+{9^MY$6j$Q-Zt0<#RXqMVXRbzc zrt4{R6Do1_hLp|F-bEhz=ug1xBAB3(XrWKoN1ckdb>?biYcL452z`$BcrA8%AnUf& znpkY+p6CuoZzczFeNQO1zDSr*cI|WsT+VNX4wz{q=o7`WAL#OPiTl#oODS!p6c?RX zxHKgAVwAugXYOqcn(x`~l@DwWL*umwp6x}7Mi1S&qy{>}%6d=}@nCUeV2|SHFy7bK zo$`%>)-3pFWU-%4%|m~&ju=25O>MScxLf3jS9Wyybb(ol^5cgLTxe$~_@?We-W&U; zO!`6r&bALg)9;wQFfd+s>8D*Kj|hjYqr1&RLDtj{+K{eOZY&QcPCj@ccuMxx(WKld zzxMZB_vO^ZCC%JjrtUt^$}0;Mi87Ht;c}ZNmXS;So`BG$_&ant!CZProSn0j2d=!l zR~(Lb?iXihd{Rhz?(09#_vDjn8WfNKDr}CbS-A9514-9AMMqjh^8j@hqrE3Q`2;# z3FtwiyaRt+MdP=YEV&Hh4_a&kXW@6oo0qtrI|O(%S?TS3LTqLQ@N?fpOaG4Mx zV!m`72cKA#2U7Z;UWRdNM|=D79myBH)e2W8)Qcagdaw!}W&JMuBFgyfD?!^JKRlK6 zc9+GCkfw98lkvC`!l{0@Q&yKtPxK7D$(&13QZcK|mRD~6^FD&p*nSf05GSV`P2wI) zwQIF^h@Z4GtR$^%i>>T7iL6iz3?EM6-D+!n?X3_mfOW5yOz_;*MVvw3G1wgePS>KF zZxvf&m~DclcV)8}zU=?tc5{r$1F#RYDH-HfB=YmyMQMB6<1-jY4*WzkaF( zH$nIXC2AMLH{G^ek>T9tKbmzVIEGcCfbDUl0H>gO} z7v1{7xHo4Znc(g$DZYVkxlJKN+CkF0wc$o zBH}%9^;neaitp-oZ?5q-u!Mu)vS?LDrcu?`yjNDseOfw#-yJ8#0;YsUiscT=(>7%* z8m6S2tb)z*L<;h`Vx@3NE8vW~IJ_NT)uUmnaF}Bh5H)AfuNkUwjCa0KzP&q!#g0K? zzo%5UrQ{Int=S>`p|qT9um+0ng@n!CQmPAfWUAhZo(0@PMqs}s$d7I13@G>>Hu9OA*;CVM#qs{o^t{Jnpq}K?kvK z-(MmYnB`|Ha(P-3CaqzI#PB+$Di7XKh=~~D;z6EqYaC?kq*5Lf?+J-gIY z-BK9**4j^GUmu7p{ZUF?lyB%OFI$Y`_dvw|#`KM!$>)IZm9w~UC-h(6K0nSJdL&Yf zIaIAx$OT2WFc(vtpt-i{5+7#83F;do1aba#Mvtky9a>k$+q*6r8k0o3=TSK>-k0MyP zbw$3~LMD^hx(MrhrOXZ(_@KjMYPQeGpWFSP>Y@C)v&pT;#!7eF+4cik7?sR^NrtVp zTUsvVc}h?2IQTHCC(hJWJS#VSu)Ts5xj&=gNmQ0Oo5=0z2b4>t7wEaAx<0k~*T>rk z+eu<=;0jrE2i^Pqqf!b2;Mu6%qJpENnz!wiIU(bf+9uhqV~mdzX@=hR=EDdJEe z7WKu+I?K5AodL_8NDJ**t#L2ZQK9ZYXMdKa>=u=?^YJdmvLa7LJIGt$h2ny_?YE13%UK?;Ke#*e z%c+IH>DK;c>-4GqiQ$|7O2E8y3JVV*F~ITiud7q^YIfA%-1nHj;ad(CR9uJ=uhua@ zR7$*wbSzlX+q)#g;tF|GS4%!`Oep=9F19xiyur1NlPE70Qam3u9|&&V5l2u`kh*(! zD>YzuJZD(n72pHg>J+tDyQ9$oqVK_(u3NW%qUJ78_1>RJ_(^6f$$kWWZ-9)AEdTWC zePZYoZ@6fzi5OP%k!X3u@rn-z-q>3LP6xJ3LIqbAfSt9Z`&BCplZC^b6RWRAg*}T6 z7gau}*^+g-ICIWg29#ELziKe%e|B5V>kjm>^HPHEK+h+MThz+&(fKq`qM!0xTx3P- zq8egjmZ2U$wDK-n_-mP2%_Y(EeFQAhoI8>}N1NEz3>y9FP3@j7%IBrL;WKI;8mb5E zI^xW{>2=~?bz4xlB%Ju8(G_dOxdKkSeVU;;AG*{rM>{eL@?x)9Txx3FY}__$F;`k< z4p5Koe3+ktO-lXVVkdz7hJSut9-+mcshFfM%S`SaDoQPU#!O;tDwG)5`SIJ8sdE{3KB-rwHb4wzGl*()vGk@!Ebol+W`iE~zwb;D zV?tHY13eV#!{fanY>*eTw`X1leefMxs^t-)rCM29sy!L0ru05%ajS`i6JwyTpyF}4 zfdMEm7R6sgRq{o$mR}}O5v)x=uN~jp!d)(nD!SRiDjY7USUqVcU@aVeMQS!`WfSNc z@8=4@lV90;ZXg7gLw=#~qCXVodpI@0JA=7TH73}aOECKa(@=Zeb$aaHjtJEtaJAcF zX>D))f_tq7-gkc*{}%1h3@oqFYTrKuDulkSROPqyv0H}=6~XTH9~6&pExS&i&qi6ve;D-M9$~f-S|J90?QU7%%-v%m+(%#~We9CV z%`J`{MYn`av=w>0SQf5XR-!k4qJHHrkx{gI;wka_k#%W8rs5Xn z!gWVTO!MTsFQq|PVs;Pb33P%v)V4wM9c6hslbg|NA);GdKD(_`sCM6RmKUQ+lI)Zq zo^x(`c*zpWa!~=*hdO25B{p+v84ntfk0%!H!#!_I;%3-;EtW7f;p4bF&$_KsDbI|) z^OiM)sHfjKpz!CDv^%&`v(FpsqJ8oTbhp~rDUn65eY65L^QwhSVOJk`*6gYKD5urn z8zcKK*37!@C4Sh`F8ZkW;F?KM{u}W2&krw1pWBLTOr!C;E~a8v%q%ta|HB3FL@ji` z+I@iZrOaaX(>#?^jS@mEx_^|HFFy-D8MZ9&VSnYrS(Vnd#pA??MHtgcl!m`jyD#8X zn|oDw*K{Z4k*-q+NmByMcet1yR9ld}tJUFNXY?nO1!i zmUe->jdtu@cU^4*-Mayhk`vHMn0Z+4RKavF2DvQDEY?oJNElXB_1)3jtec;lJpzmx>7{8(A++6BBqc#@CS{4Ejw1hh&4 z5a1s7&7wSLZNUFrUzllS2D2pH^}lid;^kFB{`H-(SBc+49KU4}9ff4rQ}pQS+5*3@ zq4cwq*-LK)3e%|?wZZ+oUil|tqGTAWi+w4n$g}ox7DqpQNL|AAX|drS@n@c5vuoY_ z;x4DZo~Zg#y;FaB%tD);G3%%7Zp+h}XKhg1u$hUa;8PUlT#3M^{n+oSg{7?Su&#ogN3Cr;5r*PwV4;L|x@9(ES^;5X=6#bZ z8DqBvjAU)|e>-7_-2zkWnyDiRB?quDB6Y=KG$>QvykIMY?CNsope-2Ny|eh(IZ|(< zc6m{`)8Pm4+mOHra&gFhS5r+`xNbo6i>0>vp*@w?eev<_@Zhb_zz*=KOWuMJjikuV zSK{2DRQeRKmkU=>!XA{#Uqownx9xaiOAhWcoIBLVRQ+VPA|bTfY8B;Rw^$y%9Ge|* zC!N}TXnQCiyYPD>FOO%fEA`o7bjoYpsun8k=zZuwm;KV03kBvo5bBg$F!xg()5|q8hx;^F={EWtS^R zMHCRFqlBiQbdatBiAoct_YM+@5PB#AN|Pd@^dg{u^d5Q>2t9xhdJvF=mV_EQd?(&} z-}n8^d@~Hg!1*JGv(MgZueJ7m1btusijCH63!_z*kWUs@vbizJ`g^M@j1xK~{A?)_ zFSo#?`RhnRs_EuSt0&ngr{Pa+?=QkM()B-{13?*>MhZVqX$yd$&KbwSy8eD2PuDwS zY?WnCr$tu=6=G_D{qyDMS2M22d4*oGxmJsCM!{u_m7s9041 z4Dleox94f>e_|8(p~Zu8FwasZRqE0|F)MKU^%#PBwuaU-fokKMHaBaf9Q?89^+bKp zS=z#d%gvekfBAxDNl|T2nLzXz)D`FetO<|EzQ0JU) zR0TJxEoglV8JW@Fjh1?*1^MmwYL>QpUOu_vq((zq$Yk^y+f?zl>QQloRHhA^v_i=Z zi{sB#?UNx6;+~88Oen)sU;*^5eYI2i(7PGQY-qH17<6LUdBDf;26J$9(c0i!F2f3Y zKOv3n-3JQ2k_AkY81(n97EWnf+04_hM~|yLX8ao%^CuUX{FQg#3rLpgtjf2H7r^B5 zX))7Tz614UF7n zlt(Hw*G4DjVX`e~_l!|<*IMD^mb8a0A+2f6ICq|H^h$5#BrisePeixdUUM7ve0%G? zq@?qyvdts?#+aH3>2IE5vMJJi74>L4IR5gM22m1pJU7Fg|AXhGK2qBMbXP%QrOWLH z%)1LL>F9Q5g#oJJeI#9cAJP!f7&T@Z z;P-K_BUMTD8QAo6wfST=C*Tk=I}IiM)2~H&wBr63`bJBgWp)RK+OgaJhkP)L1F~(| z)4H{kf6y=bwvg4xZAI|>Anjak2!U!w?zY8t)X3!3OX=EPCR72Sp}g(Y@J`tcmP=_x zdK%5#^>jbI7osBWe16B!0^w9*E#cpbO0PJo2Q%5Ny_w}RDN5_hpB?} zNc$klgYrlYFPBn(nxoo@0a9Yyy>bk zvfJtXO!xQ6%OWI#7#-t-9#$L0@ z?EI*o%`(qu4Dgal#vk9!v7a@c9nwqo96GReaU&Xd5ZaVFlVQ4Jvifz-RUC!Kbta>7 zN>dQ5`GI$-Gf`y4YiCrwYF?C#312UfpC5(}NIr7$YvJd?lzj}lQIV~-e?NPB3bt$- zv5R{4N}9@5N`)nU@AZ+#8MU^kO3Uqc3y_c-+Rd@F9d`Ah84U>g;e!TCv?h~ykmS$7y;eSH$%S z8UXW3mEKN+6uo+v=ytS>aPF5_9^3*}!6FrQmX|wL1&L8LP%ztUojd$s^QZ2vMMVg6 zu6o)eoCxe0`|-Tv_d;&-YyX4#03vaAl=#}#;eT=OYX_>qyoK@m_jtkogN7ecr0@c7 zYQHF!@*jv?`sodWpmBqjOC4aBpW%)e9T9tIkpC16so_Os-%^WYze`(m^L9k@-Hlw{ zn%?$WzV#7CHzVkC^Mx@fo5%R*E&Li2EuW+gJwUfiIn$*J;bI4 zQ8FPMprDDn1x0z9&C7BNb7j@OayR&xqmtWN6DN;=BS9r6? z^15qR^~U)gOoI@>*-gn$!Y5>{X0&_@;S{^ZV%55+-;G^w6wzPk3?LyA=~NoXYfo@r z=k1dU8uE7#wfT;p7P^{{e%XD}RY+{K?X_&q^}bto=vX|r?#qB08a|4*de!-vVcgSYRd)@3g*=5w|?S&Pb zVeKL`q{(N&N?@iaY5_6Hf=1KDE1R2ywIMnAdU`&V7*p-p_Poou=$$Wcr>TmsDT*~V z_8~*!x@B927R6tWm3UZ;E28-`fCW`R%80R^F5QOtY)-k-~Es8BjX9{-2gsV|!IM zHH+<@EDIW?`ycMgi|CdL zVx;&t_i=V3w{w^i1WnlqBgt-04s+9+q#ONKNhS&?lX@|^*VygX87cN=xI!Mz16v_r ziC8KS&62W}iN3y@p{G1i(jnbtXa9?2Qw7KXLVvY?;^Y4T5c%gZz?A9J_U65h0oU)H zK1de)x5Eqk!7Btn-L`?IdrhE&ikG{6%wPcbHplLX(ocJHt~~iPN$cZdaiqFL{PuQ{ zK0L^AIw?$$PTKQb0JAA~GV4=EjWgrn7ypV8!9Ky*Me>~}g1cmUIi@5;R|m^9&)YA= zFxqYDnfEZ=9q(v6;Q=Jf$*yS6%r>lz{(H-{$wGA@-?f z)S7VT6a6ctK|1@V#by}#@}38rRjd96MSQ8dfx5N+~#5M$^Wc^Mj&2LJ4d zk%}9=bLeaJ;pd!7sdgF&@|E6-e%38XHcK+ z<>N8l_FDo4ewd{2qq`m$eX}j&mB*wu-yMRiHnkV3H5E??UmHl+33N9MASS6Pdp;np z4Z>#9ne=kk*q0OCyb8s5`VaFlJzHK^Kd~YX!MiS_2?+`+yN$maI$uVW$gWMQH1!L2 zzjtj1#D#OPmAX(x?2*kF%rIqp;*8Edw!&)Uk^^6az7mU{nXfd04Tp^W0Y24w&tU)1RmjtE)!_P4$!mZ0liyop>Wff z0_m{(P=3yJW@L{sF0rtyZXzMk;%GnPIN@9;KQ!fS`>Irauw^|Bb^DcMPVL!{d4X{XKI?D$ zQ8qb6C77|D$$4PuOtddxI|CP>kt&dkRyWgTPctvi|s<^y8Nrtkc4vF8l0i z;w9ifn=%`=18-iAcxFy-9x4(4_*Kp0w$u`8r?m$4iL)=U+7T7WZIMX-9-|8r4ZdEVAQk`$j&lg`vaJ$N89HVf{jt{9w)G@R zQ&#ki)O4dv;7ll{cGYEdq-&|Lr3%k1_KUsUXl;Z>rdqtGYm=v5RF5XDUdxSO>kjIf zqK%_KeJD1BHEt=w;9oE3fzFAGm*JdvGaNbUB)Kja!w+i_IW=ID{@sHxRq68MAozJH zi#`u_FTt`}Is4bu+c*~JCJ6eqN^`6Uu8Z9FoSHbsrQIA&UPxCQAgLCw7>M7%gl`z7 zy##Ubm~)q$9%ba>OnqMY5LQiR5AhxEZ{`AqdMjRmzj1Kd?LQef=sy|QjDXOL`5%0d zq=~#7;8jF0OU@~#RYo%_U=&9w9BTVudAuMVlf`h$kw$3e_1YaIuvoo$NAPllv-Zdoj0! zAf9Y_e=f=Gi7BKbPf=C+WP1u>lj1hMf0~_f(#9EY$&bwaKZuu;jCeb=E&G6X(2=5~ z`NZc?Uw)4^5QZC|{p+7#t^`0uc2vaGe{yk4Cas>;NSW3{Z1^%H@*L+}$)Hwd0)ueY z`hEI3pEjfGlpg0l#tv5eyiq~X;c)jlBC9`o2~B^iM=!0qM+aFMaut+iyaw@iJ}kFk z>$*z^%zT!oSxYI^n*9|mAt__g-PO_p;N~UMceBhDwludgqOj^*KcL(BqNvW?P0ZL^}t=h2{JI~JLM`iljPp%%{gk~rk1L*-RzapB2? zns^TdP8R<1VnCXxX|nt91iLakm|4dG$J;3SZRXG2KB7UlDJK{0AZ>-3~)uYEd~pVN3ky+A;|9B~wi`kk8} zs5vuMQ2=vpk!DU?X1Zy74yC1Xpi6(b{$d#kAdwc&6$fb7SM2KuH0Z5OMLTUk@p`32P;-Z)L1y0M(40?HrTR)xsX7)DaUaTsuRbbi+cpiu-Ga|1!@u8S=> z{bg#1P$U5W@j2ZWO-OjyRdVazqC;88?NkD=U)_4yA>2G{9;{+nVX7R1dFC&%*8q93 z{fJCFgf@NOXcQlRyP;Tx-t)F$A$3V_xY3)&e`pb@XDOvw$gX7c%YB6(?KY(7zWe&}7&S){Pxdq+X z0?yL&xbe2CGei6V3z>ZYvM^vK*}D=VQ~r?pU-d2LRXLbV(rG$zAYE>1A?Pt5)Ix}J z`npSECBvhPOJ#TG5|r$eF$afFgqFhnWQ6k{7#<*a=+bXhX2nza6;zX8R@s1_o+HW& z6s3e{%^hI?YhoF~wKX4vQB3#bomD~b1+L$`Bn=iYP zC=;1_$IAxvqmbkUzJZzes#;ncv!+`#TYdFaiFh$IJ`i6}F)3dnZqS(uL)+Tor|a-( zV{AY8`rvSMudOya7zeG6kqeG@K>5I@JFC8;pUQvN4~_iU2!pD=cJ@ADnN3Uw4pVH@ z*vAb`nEyZlq2Un)NT+DySd_x%{&JLLQ>|0(TGaU`x42*92H5C0|Ino4XDOf&Q@Dto zp}aX>`Q3AIiSC6P`F+xZ+hzdD<@1P%_4D|dbDc5aCMt(%oKZGAgrkl)!nIm`<^Ifc zO3IWzjbMM{T|~igMb6WZNOAjva?&0YT^=#>^uRYfv-A7QT+(yp-4h7ApD)K^L5a~U zQ=vq0GqCJDOnol^-onepymj=mqjH8PmTfz<#R% ze>0|lsmW(~SV)KZFiU@Qu@fn(Y%X% z+_}Wkw7CD|;TqN6@BQfKqVzbA^R9^kU7h}hZZ;9WxYKM|hmzv97pY(nrR=*8kQ;T- zl2vZhD!<7052&JF!^z*Qz*p?hbAB%$B%ifPsRuRmPi7D5z$VJuS@tvM z^Clr;-|_VG`b}tX;Y(`^Aaeacr<$!i*kFb!&x1|U)RQZE<)#J4f-YS?J&fsr&#kJz zV4qY6pv2k?HKuq(Mh9y~!gv-IskVi2RbX(H{NyW@Oi{0dOd7GKDa=&ebPMALLi z*{NX7W7Nt^ZP;%ZzW`$YIi8KBfWaP^uMP5;V9CzmBNdk0#w0uQolColeNRv!IuMeL zyl~v*DD0@P;d1JX=80E`+>p0s5yyZwPMnj3aG9*nJZZ+*VvdVa+)T=k*Gz;ifww0=f)9{lr$*orzda8eP9>tU4*vRgn5@$48BRB86*lKQ4IOSv zETruqGrMP2$ZCbW)e`L$=IsSXW~;8m&#$_8&}~_Hk<9VTnp&5`7j(DS`tkNh;;w?d zc!#5JJhwGxO`PCZ(a}lWwFcB!u{F%%L9`>5*?lQPf4ae2bc9=WXM1!RZ;03}m4oAl zw_B2ahDHw#?4p`?Gjb$n_jZ2>zq&t*VgEc(M4 z7jFwWhQt`k07#R#iN(Xf6yEI&e&&jKV>aklLF*BQRsEcaNhJ%%sU{J`Qdo@4!Y@{w ze?_(Rb*4rwo;&G!rt>VuqSs^R5xs_%6w9XX0E6y2iG?_N^d+K|aBtePXPh23sWvlv&VvVCG-+`!wrQSE*jL?jr{#g zhd(ylnD;k(QnDG16z9Zu5v4MzTux}cE!X`t6`SF>#mv`(jicQVjjyIrQCltYg&Tu; zZ`hz=Tv zjYi%0M9;+7sTJ+7h0UdxR*j<5X)YHkOe7`l;F5lu-P<6T ziuk)HkOmmwq2UKn_Dz_Wio=IGTRWn}HKIt2{j*$uMm}@ z`1<=SL^&7F3c~>rs^Kgg;^C)zP9UVGK+g?zy8A>?lE!3zIa?<+uB`brNq+W3dR{p{y_5cnVQ*c_Yq zsw)B1d$g$Xa?zIka8lXwquQ3$>3{R@LkdeE0SkA&*NAIDQx1N$gyuynYIgDqFG$(_qTj^6pvnVe*94OdcNnW>)~ zKo1NxH5}X)?-srCT{R4h?DVZZq=ZRIXMp>-IW_$}X3HHp?~|yP5~sgB3Q3h>i@PZc zZDhWM(#m0oceMIQI~vUwgWu0w@KmMFuaEGQa6|p&Ok;8-URWfKqJ*cg^-9z|u4wfu z>zkv!-|epqy;TRM$PeDHGT~FyuXISS^w+}?WefTti84MmQ-@#C18dPt9dk4EoLDc@ zF&W&FcRd=@VNm{ix_a$oQ!p1-Vv?4AcrdmLt^pRqy8*jC@6zu}k^7dl^FZQpuwk-8 zmoXKueLgR6HrY*xgPe!V5R?utrlW1n>wHic<1?p{Llr6>8uki&Vc{sBuwk|PWERDY z7Rl;*5LtZ83W^X%BAenxn9~Aya~r0k`xMY(RNWR^q|>W2p7tMicdBxpcJV;8q&FRHkNUa&%YJ2MRpxNPBBj;Sq!*Ljy1s(1XwWd6zsUP{l>Pbs z;5xb3UteSWbl#OhY}qUXxCsI7mS$GdXCLrD&sGP7?htne?w?fpYn>9H{5Y>y%&)Dm zq`O>7*HeE7h$%YODz090(*?w=zV(j?lR+Xr=N?($KWHG$Eyy3eRg%^mC@Df-=|Ooz7dgSRy61^ z@ahNMSph>Pc(Dr@mrgtC$c9KRA&fog<48NJFQ5H1G;&BE(qPJma60*g`A&E;B%cm_ zFgI#UY|_Qw5lh+_pPOmNT&`~@hdm?Sl@R=pAbFKYNYm|$QP`REh~j8S&fNveo#RfZ zV2%#qwkr$;Tt%Ar@Gh6>&>-g_4oE!fgYustfci9o$3ZG@Wayr77I9zdu{^1ccs{j z8s40>1vb$%n(qGtYS}1KZXn<}=S2uwe@gpht-4`!JVYB-IoKM z_&6TzGgYjK40{?XQs$@dHx2gVuwvpriFOd1=(d#c(`)iF^qK4Umh8%Y*WQ0gg3(cmUr;m$;R%3GpbWP6b{&(D~nj$i=IXd*u5FR3l%NLsb#*51`Of|MkYJ+rL*R zVhj=`&LyVNa+jhr`Za4V)(_^#9a3r0D;1;=-kO5o1@9>NN&7>tkJI8Z!_dh#O7p5l z!9chTOcg73u#5@Nsr15Tnm#o%&`dfXv*RJ11j<}c9Hk6$g^9s(#mfX^cf|Lg$Q82i zkoF!)wA{JKYs9r`YYiLIDR|s)NI;HAuk9>HI_t`^QZ+fk!@HT!j2|U~(EX~?ZcZRD z&rqF`_%O59l4zRW{OD@zPFr23qa5hd($eGm9`OFD1TUUfcR)GxYStud8>==7jA zR1rDrYfA)gRS_-wiEAL|Q*8@BVq_IO$5Je4D+&5>ZWK@OelI+}Rno&Wg-)*4g5Q!> zTrbuiRrhVGK##IlW_ZHR?1rGTV!g6z_2NHwr^zXBUuYE7!ZBhMqUn~hks8{BRMEJX=CQt20Oz zF=Mb98AO$-?a4)d^l?%~(*2`pl-kQNFQJG8;vZ}J4a&ZK$w#wZJ5|e>q3n~7-H@;_ zNsO1}P&;Pt>a5WSJ4*n}Tz19`_|;5J{Q^4@$L_O~DoLelv- zsF0vzzEl`ld=d)Py53to9~Mc@(|L)JjN*NOq|u4sxt$%t#RiISI8sZr zGRv4ql@OL6kQCLQ=-gvJwL=}Z%62U~nSY=A*iiU2_cL9Wh5=-6?UL`tq+ZT<44wSk%@(z; zG)9n=a53T;gN;n#tN2=aCPC%hyJGr-Uiso%ea!J;r#MUk=gW_=GP3gTcT1K8HtnB# zVBnq$JzszHUUlYJm;s3}Zq$rdWY>paFmW}U^tYW)$Kgf49VK_mD-b<~k=Y`4%hTrs zpoWNg8@3_+__Vm{0fuRsqryX&f~4`@5$-C6$gM7mD)2#~$QjM-GPz;@tBW=0d^BTV z^GR@kqfYyg=c9_!GFmdoU}d40qC>zfjy1DGz72~0xV8-cX~Xs(S8vmpe|uOKb3i*_ zH0Kx1>gq&#K(Xn_r4!UNtmUl*&0&OX=ikO&sZ|v$G@yCe2s!>5zX)S??=zmmBPw`q zq-!FL;%L&kUuu}9VVd0GL_{oYSB`BXSIEGT<7qa}t(0_`!~ca5<7`0aji*(aadDMG ze(j8zX8aah{8Kstznyo&G)Qfv+>d`@1qV`a1i+x&eS8Gu=07n@lkw1gpz-#Vfuvvk@2@nEHt-)k{&|5#rZd^kuo@||aP8Z# zoP%Y`lkx_|Icx6egr-8Tm;R@OdXD}{!>K3h$&T6@VRlz8ik#(4rv2r=N>C8XDSN!= zL^CPn6XXK5(hUEWP`ZP4=qU79>uYF@IH~Z9?Br|9Ov#ndy{7;@cpmN#iY?sEg)LD! z%e9s;P}}+`1zD!O)Pa2njHYM8E4g?*hO^C9hpT?zTu8AaPU04s7@n@~4}Y(a&?pz> z?lE;nO^C;#ik6sgS+MoDLnMLC2R^oi0G~QtG2O1W)}$TB*LxAlpDf_6dY4nP9yk|W zZmK@%_L~{#-N^y_+_Vz$RRHY0yKrq(9-RHres=jmHH`%ubI~cLA!}Rer@4*k0qo4}MVD>U1sJQ7AxQCY8ZbpxQT`Jk{xLB;v=6igop<-3Eu* zxu(yvx;#Oee(1@BQCUOWI<6%Pd^{d=JbtqBN7!F~CphicCtgPyGQl2_(^n~(>Mx{> zGmdQ$CJPppND zmKim*7D~*AGc74WVp(~clrZ8PFR|z``@yuRz9yhsB)_$K>n~w?=4Neue zoUpkERXrS++w97YA)=Q5F*srl8|~wjY;XAl4<%HdE(@k+oDgOY^>!;ALGS+o8P=Zu zqrTFBxh!;X7wG&7oc;#&z^5gDwNR1CZ2G$OR1yK;u#tq~D=SkX@A!Y76C zhf5}74d^Y2{MH&G-d9tLI7$J1@|RmYtl_H)FDUJn^*y;`F)3~^ z6v8|3dqy{V$>--MMsQiTh$w-v>g})rJfb#N)V@A+pOM8(|X(%u)d4 zZ(vJxJ3jW+8zH@yg+$?^NID5ew6OihvTA+g(jNPdzObyG1@X?^&X&}OO*^%?4-Uob zpz!5Jg#5HK36O||!vm4kCJ%NUV~F^B-Oh&w&{OAvctr-CsJ*V4&-42EdZkguc?{csdF#=kNdGDocwapSB#`CX_9}+E`Jdj zcitldxD$_nD8}1VtQX0#NmI}z^>;M0dQ#lWrV0E&bu9ZN*6YRLM^n+W_PV~|LB0%m z+VeUZLIRwKCxxlpLA)RFlp*!g+qL%U2dtfCfIf_lSE&l{&Fq9382#>#mg_zB8ifTYRL389q-MWC>^fGQh-8-(MTA z0ZVvR(zuP$)w;WmibTELoRM2~`EJjJO}pf-{QP|oQFV_cDOoAx+@xfu(PFV}4i}UU z%6%jv8mxLII*3Gz6}bp~Az}ap1;^+4yD9a15Vf&PGd?$&?tEElwCoJ>e$_9}g_6#K zYS_(~$H0wX0|P&>wpzPcqK2+z(t@~YwNFU|Ce|ODOU2&@`b7<6QC8I6QOU^d;>hr=vGeGOXSS@ zn(}=0ZKv5By;;R^VSB)j8+*^tn(iZ@VryFY6~6jN%cMDd`b*B75*kgs@*=6w!N(Wa z(4t?40$Qv@Ylnr;9pzCEL`cxz5+afI6DawEf}hUEgc8T7LkToVsB;fpH1#tfqp41E zJJ?oT17%b%ww$5TH_G&-8%g&wQc5*Ga!h>rL(SR)5cn1h+b;3HU=@-={-mS>;R8#u z?Ev{&Q;M7J0T%`A^bGz(+hQqjE#wpVlBG6bhrcN=dgT^o_f$E!N1e+8SoTzv%h}Mo zZ79~+Zuf2JjI(~i1L>@SZ$S!Qe}_XBlcd@dP_MtxW!TGVvJSI1KbCVO^vt`DHMM+|W zc~CmhVmNrm6y0brtA3)CL#fA+RUwlcibuRW{G3tTm=HR857TJ@%afL=sE_izSiJJc zUANd?71tEv-iB#*VE0Ix*z#%T>L##-KXn~#%9qKvJu}i6H|et8^bI;P!OIR37l@r6 zJ+_T%%M!gKQX59_9@o&m`5|h?P`kfutDUu?Iipvg?xmJgn5WjU-@=Jkf+q52(|Xw> zJ>>Lv7M6@Zez4DgT5oCnWi9%A^jVszpha_04wjNEc}$lg_j2t+*x(?l5*v5s zPPDQfgx>hNFN94Kn+lN%xpEbDRdLozHf`E)sxTPL*8kgReH;`c-!ywra`IW`Cr8uK z&T`LZ{-2Az38C;PRr&Zt+sN}9{|dm)A{JjVrHcT|fm&8so=N;v#n^qdtEp1P;&ZtF zjb&bPpsFvm!6xfffLN}tmxzo!AI(g@$tniR2Cmm)R7zmBZ>Is_v3`VrXa+!6g z6p2cU8eF4*y$wwdzj#Tbz`3V-cIm#0Rgi|pIxj6faw1lr+N=DFa*rz|9Ek+4yEQkOACzp6?_3glAVDy2c^Hs_KG50ZCsc0c^33 z9hN%0D|#_C#+Jtn_1#X*QDLao40zg7McINs)bujk<1C|Ouc~SXp`|kkK)?ezU$7#=`IS3dCegY;Z)Ea-J1vUSb%+O92wzwO!z6()l>D?^OzOdW{%h#R& zqjs?(2~>iIu@Q&POblAfiL8%qmL=hnQqHuOO9hn9}-t7YJT% z+U}@qnqN@m}*b_?zA_h0q;=PV0r3KCb-Dw>E6qs~kmeWW#Oxa0vE>L{rbF zF8U?g3pBAV??jX&(5bgYILeRY=1Tlc@bupIMPGHbA4`PvRGNv=m~*ZQmE-V6Vj8kP zzT6EhuXWpE+`2iHDP)3M4)2v&%W_|<{!B`5RY!{5>OEIfUl%_lUy5enQc{tXq(pkinb1!C26GXLL%6-WTVi&_ysWSriXJ z^SKtXt-QTT&fX7$-!Po6Ed1kl9wsZQ8bLYIZ zAB$Z_K#eX9L|g!Yg8J+-`K8-&vYNi53D)@=Q_d^VZx{Gdqt1Wr>bXStU7j@E19&dcy1cNz?4Tb23w5l7Io@r9g>b`w-F7|qwY@41N0VLkFU~=rRxcJeN;7(+pG_4Mm@f^uB0q+KJ9JMa* z<1#YHHlAaqnZ5qm)Pa+24c*a@zc-nvlfAO)0_TEn1*UNVZBv>7&4oV;As`_wJneWG zK=snCUfv@c)8+u*q`eCSso=gcVr=Zg`8p=n_OY=2r|12=hX5B5yMrhJMm0RUuPHj# zM}|nVd5s&wU!4H%HTETLQGF&xLNTPiR#FmCFpkAym{+UA4RrcHmOi+`Dl0YwN#qHR^@KzQ1jdh0%DQwuJP?)dd9AQ_ri@1r9YQ0^22?_RtJ` zK2MVXyy-Lv3Xp%uyugSS&TN`x-|y>&2kUR^57ai2UW?PgeJpgM82)-2d+X|7e(7N@ z_k=U<)0J0p%4_K5!^&o)5dOyBxg(qTYEGP9FHeJMWT@m&}0iH2)X(LHO+B z6{eqd%^n)IxZ>l!;<$m{;%3{@g|t=@b?j-$rf7k_U0oI~20Kp~;AjY3DVu(0_{oXV z?x)zQ7>ngT!^z1l2^ocQpWjJW#*JF5p);BdF8px}ecXn+BWq`gLYgPayNl{YnYP03Z6>g}BrpALj zu=z}#om$KwqcTi?d_3d0p^<)ODmCL=`sw5t4YT`njc~ZP*q=%C*VoC|gs+g!k_+%i zFdpwdWS&Wr@vLUygZ%9*7{5aSeLsaubmRLV-hN=59Yo;t0$;Z%zt#vw#+)>s*1~W+ z!mBMeH$S6_$ee~T%>)cnx4e}i@^yWFV|)pgp4rU|L-<*(jdkJgVuq?X{QU77WzF9r z{~gXK#xU@lUh%DRwaj?$@*^SRB&>q*t}G1yq97QUpPzbVGd{%3%1##-qe&kowJ}wL zitdvy8-#0F{i6nRDK9X;E*GU{`}mVkc*hz?^06k^g3*2&R23NxZS(tQ`8Z7 z{0}2XCgrNhkv`Qf#ZGk{z#pyGZ=aH`lDwYe5}=AN^QfaR{9-p*jD>EpVAhPyFE%&FLD+4 zJTMdcEDH+k6XUc5WkgXjHa0!~QUoJ4QJKgqT3+PO8Q8q`SN0I10Yub{zReJRqZ5ouT=&rx5bH`Go1Nsv4iw#m!uCn2uxELk$h@(*|Pi0 zcq?xrUKnL}%Mc+DqAsThVql1f`CTz1dUw4Fj)|TtlEbDK1(5dbwUw2VFy{oc?ge$Ui*WH|q!N!ovLS99t1>5G(ZEV>?Y%a`=#t4ix zRt)yOliH&ncjtrQEEX?uv%StMLkRxvCn&|FlNPXsL*HIfQx<| z>n>bi_7Zg!g!Oe49zd1c{28k8^*9-8*_MtFx@7Ce&iN(n65H`$ZsrR(+ z8vYE?Ara>4nZftO^%k2s6+)0ogN0Qsn(@vnHti@7SzeCq{y~&Gsj`|*Ov}KywbZ9$U!0xGV zx>^o#j}y`@S5^D?3Wb`N(RF`r$$N+g7&Ll1}qOMux7Z z9I-;9m!1~=lA;d+H-I~``55sLJ3=xII|;V=Ua}{kRr`f7^zfxhcecWo=a}BeTZMPd zY(8o==Xx0V_5?I-*S*wG-x?)xt(2rE-NWb&clRc@09oF-{!_sz46AR+RMo&Q?uz`8`qR8OB^t?W@S{s{p{=NVw3C(W8wrf zI*(+FQ+5koN#%%TQuctfN}Ch2^$PNLNQ65u)fa0g?a42bW^XB&&u>H%Ozj?GXAo-# zkPDY58Zq@CLq+cW(*dTRwHY1+-`M@cZ3T<%uNz9cRz<*RG<@}Nr3*|m1|eAGGy?&Y zQmy$`S2DqhM@qlY3$kf%g-f@G`6uW6XAFc_B|AE{t%E#6N<-$@$fHW~RBP;kLjhO6 zijqZ%B5u$hr-LJ%-p{;qQFOAR?+2!p3dxgr3J&G#lb1PIm;6V#xZW3utS&K%D;+~T z51)-hH#nZAh<#hf&TYivsu>`k;80+(iuY81BOt|0?5VhF92B|MCm$M+E&8Z1?8!x< zi{j8m$5()N8MQCks!fz?Ag@vxXn#1D1Wg!@1{Ndc+iG5n?GA0fRnnt1uhD!wz~y(e zDR?~Zw!gV(eg+13l|TH;dSkrEW>10O4|8TW#jZu0`XA&^1t+E+oe)pdUvq0m*^9tS$N;<&WEclEcj{An#{(Sn_eb?gMJX zLZYBhF67UA>g(TB0wOC9HzxAV4AG|>l;scZC>bXO1{u@xKSw(cqvfsFqJQZgZegV@ zoXr2q7vbG+jOf>nWb40K(MNym{@3Af!miqyaqAbP?yRDwG56Ofd17%%jwu$2+rQCBs78{vO;?X?vAW#4qW%zLMuIq1n@bW+5&HO?0GLchY^nLqbQ zLU7{(M6e? z&dypcI=J`k`8H=pF+Yy8RmHF7lOE_2S-&5I5pFZT_s4u!t_#gReC)yQQDd-m&M;+= zkFVa@+Pzp*H#Haw9H9o8<*n$7T1h7Fvk18xykO17IuqG*H*Kx~q_! z7u6i~3NGrSa%|91x|brhvqO271A!f6V!57deyA^}iwr*R5!$lUd~vwz)kv8PwCl+q zj!}pbVBi!EmrB%|cj}uZ{5G33D%+$T%1#V`rBzmPisyv0_A(FE)kK<_-X=wn;l1r#A=?a^LJY14hwZc6nPh+p!o7&6Ar7{(PSZ6KHK1D zjRbHr-Q-ZD>@xZ2`_!{ z8r4!dckNufg75yfWsPP(XmSB43F?|9{Dk1Ui)xZQ-piHuyLpq@!$D5_!L5ITx+R{x zv_j>J6g8C_5WLwc5vbUfZjjKy%`MpJo5kO;i;+MIh$qXCFM=HYY@-4GO)1>KOdp8; zf)1xu(nkiTWgz=}y{B)?~jtJ7FJFB*OcruzeRbL*2Xr%FNuVnD3!ZFoUsTj319fg49zM@tE$! z1y)BI@DZ4l@p|dEamS=c`PK$CbKRP70_Tr7K7WnAe9PuT0)%Oz-?PNzj?7G)@m#TN z232d>ZxDQ@StPx|LEZ=Xj;;6okn0Wh%@^|MU$>GoY>yU39sD%rQE!htj2jvgTu7U4 zZ{6LdEv}Rq)k#@h2?I#klk1_#^Q)HHM3*#&E6Ybm*ODT~kCW5pfR4J~!$ICg9D<9g zC5&R#PbT&OX2-K+Ku$D51w&l#+&L<7No{tu_Ss7FBq=nyEOfe&)+PjCdrai0Sfd6Q z=e52ULf7cVnJBVP*`J&%Io~29`se~s=*dDmEk%-MXrjBD>J5*lg@Pa2xw4@`B9|DV zXumc_auZiq0o|Mp@9{!dTB6~fy(*;&g6V3arL9WP=6G7=zsghKu>VKcSw}_HcW+-> zkdRV3Bn1TN4gmor1nG{Uhti=_q!9)XX@(G#?ve)S96%Tvq$P$F2Bh9I-uLr7zyIE~ zSjrMu&SCc1XMgv;uFsC;1kjti1+;z-u-;}R`pvbcsLtM)53#alFbV*^xpV4RiDD;H z6htV``)uyZcCcUp31RhQKFHy+OW@*HdTMF&qqWu*nwKP2V@H5=|Euhktl6C4uhbG| zL&u3=WUGM#g1>Fk>Dk|xjDFRZ=%VBj+xo-l{j1OFD%}5{4yVgUn?XB$Decl zy?7wCGsftF&w$WVceA$Bw8ATCf_s#nHQ-pbHoPe2*0n7hl?4y9sM+$5QJr7z@|{ag* z#(WtSQ?tR1u(A}y0arx4mRGw2u`=eH1&AY0@LqnlOUzQ+c#9q+qO_u)DL0zAxWIc~ z?|SC+1TmVPZo#VvxJZ4h_xAQW=uT2C%MuMvuYRB1p?L%Jgd0#Tmnznn0&fpGYsx?q zT8lr5=T=}Zydyq&>`{SMOgo2A(G6$gKsO!yQFEi&e~JJ3$#9ZWn=-B{gO}93-~K~{ z(NZBKi&*vxKZ8H<_OKW9j0LEC_wMi&0-ICE- z+mv6?H8tRXIQxKFNN8F^hB00wY+qGe4gTe&2EELKIx$}WvD{IPeo&W}-Cbi2?qWOa zrFB)y4q9I^d(QL%^G;0{>BO9&nF|KGG@1#9;%4) z>SAe>@}zhY zNrXOmpC;|*<+^V8GWOxxC^t3%a<&0;F*^8?n3ppEM41ni3l} zGx_o9C^56NBD!yGE)N)!9zWIix0n}0V56I~9=o8#u$P&f))JqUxvxsQ9aJRA=4_V> z>G366+a#7FdHH>o?>{++yF2FY!a0acck;%4JW&vR!qcpELLu4bXl9Dh&VeZXGm2QQ zK8rDEfzjaS==P`LD?0~`r`xO=8Dg1ctE3)8z4#K#!CB|yJS{yJV#b(t`1Xkc#l}Ve zRx{;%91WX=%Z?bVHA3F`qTcJW7yum<>!)cVO0>Q$H6fOkp6BwG7jRSqOSN^&&VbM? zagPmck&a_$ft|*M-Ykm6L;5DWOBfR(XE$So`}#-Ono*yd5|Oi|o=RKVD5vdFQ}I=q z?`K+zv2)KceXVW#iq6`R+S4l3p9@XlSPmd{4R=EL(?TnY-EUVx9A0^XKWL71*d4jz z8~m#!QeX*=!ve1%r?bO%I)+3mc)^Q1RNdYIp9fUyahIt10JkEgPWUT(8LdUEmOrK& zKAv_vn=n`&&FXAGjz_+BbL)xGyzb?~zNlYCof3-{W3qyKMfAR&mdwtdT;}An#Io4M zp(*+i!MdHLH-HMj&)L&VM;tV5mQ^-fRVyn5osd)%jX=Vb|4c$&Uv5uo+Qq`NWrKiX zb1-Rp4-}_Po=&an5%o{6kr~}ib}1QFQ+4h1{qBUb=@9TfqN` z?+55IU*Vj199)Q^XQKUv;AC@lNeqB+S zDO#;6?+oL(^IOq@+MlAr_H#w~eK&_}PL(0-IWbW{hYc^JdO~(;IQ|%r`^%Fy3=Ez9 zAQg2aO#4O*stCHJ2TOEVScPkb!!ez^i?+2 zOBRMcn}SIyGvx(~kYgDFK55I6k)v`=?dg%tjNV);^NRreZ-8IO>X?fi-v525-QLxE zcgcZqYE(rk#!vC$m&_+0TmIsw=7uXqamQyni8NI1JDH-4rkDFo%3zMmI&Fm(D)(1< zn><>hmV0JU<&&BtMS#M@LxY#4OF!O_g)yNVk>YjZOV~QMQg?a={uHq&IKNx78vB*j z=);&)?DJmggkGes}58kXWBUW;@lvypQ+TF2_mC={Cw&uTg zv4&*!Ye;(2dZ}HT(vog<(gd+P+8@k=^qx#-;U`t8xyu+rbs5iyt$mDLzAfinXKz4l z!uVHaS9&t*l*7kc*>+j1(xwa#7WNn3e0g=yLRa)~CL*?mRzZhjN-95b=o=o(TJp1y z{^r0@$~@?+APLd*VMn2VkONQ9_2vUh2q&2sCJ@4JupdM*qwV<}P_q7RWZczy``y^5 zO@I?iV_r~b?54qag0!c)yOgDA4cLI3>M`>W%7@GBX!3}i;H(Z`#D4M+PymFhO7}%R z8p;rUTLHLVdY40Ag!ge8HK@lNjED*tMgc?6*8S6sw9t^KOde2(o1CkkC-Py{J+jqv;-0xnP=j+@n}R2%UoFs+jE zvM>g(^5EsK3h}Q>q>#{b5k`Ky5^rfGGotvd>Y3WuO=D#PFhqN9S3)P8dqxxyJ2=B3 zAo44Hz@pqZJnxcd#98sz0;$M6{}VsTqBtt091HzX6fOBn>M|`s}Le+7L>Q zm0WzLSeFBuSXq3*tv6*VkI>xBmYL@ltGqNZ)MMu`yBajTtlaXbi`ZLyETH`XF|kA` zAXwof-CFm1C)LgXgEu7$F08KEve0`Y2{u2Xy^s>@O9(_{266BXOVcXo4oO^}r1@Zv<*zc{gk-0; z{q8ZR+jr_N?A*PooXd0Q*5cp(PeF~+%98MKJh(;r{4nkQ!cGsF2klG$i-lRNL>qVh z8_IO(P+s`>G$&C59m>*JVnJk)z;Q8T7AUYQOVCf567H^V{|+#JvS1czMM!Bz6*Mt% zzj2<7qc`sv{!_})_WX@LTmL9_2+;b*3TsQN9KB3Fd>JGC=&B~5NKR*!juL=5wz~e= zk5(yj)4)zyn)86z6XPN0$=5spSG$u!5Fe(dq$ovbT&labPyX)INas%D-u(8H+u_Lk zW{p24^=7&)o;($vDwjMdj}$_nV$%efY_8Tk1B)-mq5_PAt9mn#3KXAr+L8h#E{VWt zwFk8y3`)fOx>>ujs2%04FAusy?4S=L%A){bKYqfNlb?^-E+jd`o-h%=tyRKenwkj2 zLt-K*sX<|W+D17wFI;&15cPy+!F`%kU9WUD2C1OVzur39BIrT`wwVZq)HYhe3!GZN z#O*Cr)`lzDZGQ9}oAokV%K|SI7{W;GL1Ykba;1Dp!wINGxF~W01qV{aZ8}_ZF zWhb-U%=y-bZ=*ax4qty%Sf2EOjfBExQVnZDLYS0EDBpncHtjpv?_=J7sdJxOUR z3odnnVc*9E>XCL~LWMn}*`w3jR!L0`W#QY_h3>16KggR*05I#z_roNi4N0F1EOq^x zv7AzYC#eWw>SAICDO!ojQg1<+@>hu(DW|+(7Ii8KM>Z7wWc#z0m3^ zPj^eI2l#t;(Ba(i=qWZ(2BYVasZ{8LE&7x=kNq-_7=M{yN9MHX2!G_l)`Pl-fH%)o z7T#7y-4mg;&be3HOJN%F0eWAUEOoZ=vzRaR5Kge66N#X_G*rquG?zIzON#RNlFe(p z4hpL?lhKnmb%lz=JxU~m`*#_mNXZFk??qFd)-<-%wNj>dV#8r3<;8L~W^#{Pwk1loD3_Q{c{TwAM#FJ){+Fe&(FdT2BaNcX91qX4#51$ z1A97ah5^+W@YT=O@4|)1Q)1yL&=OJyfk!!NPE!n&c#TW=!fnN;irTe^;$jk%cJIo> zh2!?mWL1!5ltzqHjW*6nU#2WMe-`ZBZ5SFkPJn!t=ySUkkfDC)3b*mP!&lA=Y>a&Y z&K7QvPyFB%$f87Iwlpuk4KqZ@oe3bs`Ji7mrOc#j*adWnEuYlT)gA1##iAZLi@qf} zo$;>gm)zZqVQ~(PZ3T;M3EyO10!No1p>!`7uhGjw&*K2}31$ZUMmpW$YGw^)LF=ZJ zy^p9Q&}oZK;O|xts;Kn_hY994Eoa~Hn5)!j@pOPNPW4bk`SWj|i?*LC|3IJ5WN%0P zl>xP&j!74%Rg@tfv|VrCXLs{7wh@IE4))syLBmzj$A*NMnUZWeNr#4TTK1j6$ z;UxU6XwQ{e)_o+P^jm!bU==Zwi`r&e`oUD$qDRm39!JKQM_sqvG-0Ya(Qe#(o7||k zYype9wO&voTuQ4!nnO&tLV6$1WT!-Y+=9J%A}EA0pk`X_p{z&U>|}13V9$2g?XBtN zw5wcewnU}7@3U$^UqTL1cB7lhE5GtT*O4<=^sWs;ustOj^fM;EB{` zW2|{t4I)ROcL3O!TJt^nLb`^?^*{-vV@og6HtK8R!$+ul2rGj%Y|C)P6&z@WA1^U;7U>@*5bNcH?{uraZ6d;Bo!dUlNrk3Pxv} z;X0_O(Q7G*R|$2Zale*8BZW#R*s$ULG-9E6PIO!fxlrs8hXC3X;GEM5V652nMTveU zd*sOoa(tL@3cu@Cv%@A~0qC zjtpqgP=dc`2{&x?Ea5IMjRHH`PfE``25cm7BQY+GTlPa;WjWS4dPxdRFX?J7TIxny*XzVres8dm))vlGPo zYqG|p`ozg6#QKgnD z%5ItX-}3@s)K7K5+Z(Sxd6^L%#xRtge@i^ja(;!v26_shq(Y&AMH&%~KRm?qSIGNff z$(S~it^y}hKl*9fQSMM@UR5T{rUgmYfK|0fSyqmR$h<7Lj2iL8?;#bx z*pcIFcM%JjSgM7eEgtH~ti-D-)QG*Qboi7s<+Q^9y`h-Og6FNLI<&g^-E}EQEf+L5 z$KfPV*R`)|odT;^k{KivID|$EBht5iE>4x)o4W2<1MkKy((0NH?AWdC9|L=eG>+{i z&Q8%u`f7fsbQj_JZ$P7~VWME0239vkbU;V)(slh4&`dx;N2+026Ew3T^z*G&tG*_D zFVyNPBn?x?p)3RWUhU11y5Tx_e~`gxY$K^X~(aapNt z*8h)RThZcSKZu)#?J&CgB`Rr+RaXEk2M!Y$|E@w- ztYSmLVJ%sV?}D>A*$0BYjoIORh2tg`+u`2YYvWv=-cZZ|P7+C6TD^BAy~HqA1-Ml| zb{`Gc8tClIxav=DNnTPxJfK?9Omtab5`zCGhz>D(qtW@V`{HDwe50Z6(A z8a{T~A;cfZCo7oSs&BrPDm)|R-YPUUx&+^}oCc2`#i}{U%w`^UFL*wY^jR*GWZ;`` z_p;TYV+n~V|F)RLG1v%cTS<@UC9UcJL~H`FUm(i^ZwHR2C4N;EZq7xCmps#ojoLSI zL?lxSJwG{f1X|*8erkcPJ-2yY+9N$&TaqZ;*yr#ZNA+U3Fkh8au7TYmvfR_gIg#0{f>WjD8SF0PLXn)sD*#TV)Jpybjh z^uYXF1>zb-sxJC9VS!vjzuY9PyA;d(_y+J5GkRkw~VIrb`~^E6;H6CzCn; zwT7p63}`tR&#zyfH7{h@(sMgmdH#3m;bxBxR{LM#Pw~%z&pL;Jz&-?=>mM$}HLCLe zPV*q>k>k{YB{ycmX^gzk8<1wyi7@Kbe1+IMSe9e6F+?yJRjF! zo4_9x4GQ}}$whz~m6#2nz~9uxHbnwk3SJGUWd=8`KYyOmZcVGf9cw@18K@#2Hi z!wQ*vbcUl!-OD!gg~%r0nnK{|A%eIKAMm%8DS{DNAHBEQ;Noh1k%azl66SyxgHX9M>p=f!>OPpsFS>M!R6*NVQqxG@l{;M{5n z;D*ng!2%-eIx^4e>Ma+VmLDbJEu5TXRyKaKW+0=yB<~YA$6=YoS?WnlTfgK_yU+|S{ z`}PvGINxDRt=;r5AMh3ke1G8@z{@7k6^%aoci+wbEggn0P;wF@l3x3Coz#N=K1%EE z>Hw@Ct?$6Du!I@q7@AGsq*fIZb8spgq4gExyleM1(D(P%a>aLIS)gDLP#_0S9-_^J zWeyX$&wOnuB%oCCzrat{l8B{*KF5p9)0y{vP?MadSuz*fx zH?XB@{<}uZ55`zO-bI%f=cqAMRQ3alCQ9mGAlD^qms(Jt@$C$FnwoJ$BEC~1fA-%0 z`VOV9fc&~EXcNa;83~jH4^}`RGO>E>&8wcP1=&dP`upPiKxq_g?%zi9>MEK8xOI{8WS;<#Xpza)8ILW-1UHk3{RY*ET|K;({?=TZ3LK+D_J~Bg) zQ&yxjIhk@e7K=w;bcS#i>56O?d3|7>mb{J5JAiTwhTM(S)Cq+idB6APQ;zr3n{TOk zy;Rf7M_)sksbDR%ve9OvKUJ`od9X)4gxPZgyZ*YS?BT^4xXnG+zU1?xF~bvXXoA`^M})VdN3{|~0s2ar zF=GaK?BU4{!9SF5BDAI~9HYjMOw0;+fBl$XkAO08#!Aox{iYhE$+5fKPQYcQ4fC+! zg_em{*W&Ed;B=2+>Sg1s{z!RmrHWr|?TGfR=j6EX(9Z&n27;8_+R^H1p^A{hf6Ok| zV*v%kd&*G(+MCRa3)zYDu$Ht$G+#D;abGZ@ss{}|Mzxpv zEyCN{n~c_RjK5F}c$^8Lc*I|_Guj&*1L6x&McCs17~UPe=m!Xc>X{;D33^JwMKz0$ z=5Il6tOEBLd+8*-Gw%aU>|US{0VCndY5n<9pp1x}2BR)6M z>a?gORt&e@jRZ7Tu*uSAf1-~2qm{la<=ml!ERQbEKNQL0bZJfzR)WX(fi|KNPa8+g zodmA?yw*>MAxJz!hk(6dU=t5^yRVB*I3*1p0TAL|LKQ4|pCg+$HT&+$?Qxu*c_K5Dpj*8yzcZr4Rsxo=BXbFO?qIBV9hriiNi#ZzmSvY42( zqUR}Mb5d_hy(rw3S#=?1<@woTspNFZhc1olr|F>Jpj@)&lMTVeY+y5N_s*iaGs8RC=z;Cfblo_Gi6|0pP3I+HC1ZdgxOWzoQ#4vrAh0L4r zwcEt`YhY;@3b41Brwt6!<%NDZ@@D6rv-{+y+V^|T?oI!R>`^0Qsz74W7k=Hw2BC(M z^Wvkb>$%Gym;Ld=JfqpJ3zK1qZmCrOi%5cf$P%<0QLucwxJ*s#2ptHgx2_GKxc%3i zC{1_EPMgCYZO8hVT@wN)<-drZ&dg>`N%tRixJC00Sb`3Rx3>Weo=7A1{|o`|upgjr zWvfsC6odH7@Oa8Il7;W$}KG!=JL(Mmi?_qAa1Xb*6j_87FZ=HBmq3{aIM z=v406=PYHgTrR7dvRtKxT9ggF?H8P>s=K?(*b7qUx7N@*oagU|zlqOliQwVZs_fN8 zN>ljH3s(?ArL@Vx#PQUair{yvZ=?mA8!6Hl48AtGN@Muz;c>5}59VXR#jgV%#HAd3 zvpw^#4JRCzW|3q}$y^rEbCuoQnws6(2%72z%~Q^Rjg7sM__G#u8GM~;{4{O8hwjD+ zOEkx=d`oD=LJ(X8qv6FJvAcVpiQ%Mh+$AY2y_OW zczd539&4TqZ|3XYYZ2GIT+SdwRv$*( z{)W5_=6LI|9pm-Xi<=+%HX2WXtt9Ge0MBbC$a*qgfz}o+0?kvxnKJnef#3NKY$rIn zy>Rt+@x5GhU7EU1H*eeG|A@h+7hn_qXt6O)%5K_hZp8F9fgf3u>X&_GoSll+Rbwsc zURv(YEH2D{GF0~R&s;XT4B9^@BZ-yRv|aR^i!$u{Fl{dLG}VqZ;klq=b#OpzJ<4jy zrX~WAi8i@LR^ul^^X!3E9b2<~Wi^bxU+dq&#ye~%nsZ8*lw_635#CX#74hz({r31) zoZnwei%hHt9pqBCuW#~g3AfY%rqkio34HmKD4eV_mR9s@jUC)YlnVOa)g^B4Ix<|_ z#G{Im1O6gS+*ZC*J*Ftqf22C^$KG9Y{j;HRUPdZ`o*-OsJmvQkshVe-j}#f{AXUQo zkGAY8g*Om3_7S~tf+d|oyX=GL86})bSn6S#__Erv4LSd5cPW{^{+Q#|yU$9^y)5kX z^{<8ng-MGLBjtY9_s60h3ldV7FyNuWYGbj>S-!GPko5kd{%CDAJ?xJmThoi9`=qn3 z=KI$EAD@_#H(!VlJ){sqdO-owmx$pUI=$jcIcXfU@7lV=-y5Daxhyz&i9+43zmkEr7i0Z{86r*t%2Z+>3^6JE zJ*wbFKs)!fGl;wv^kKja9YPjE=yN21odGcU!q0`64XmPnqTvZ?erLmsEe}U6ofGeF z-vXyYS)uQhM{Uh!#X8akC@v_jQ2k^c!vu*PrH=id>Z#Xp+bc_mB_EN{0jlv_*!81;M}8xw^ZIP| zC>hjo`(VJ3?%G(B9zSZQG}57A z1s*!)mdzVFcQu<^pOIhltiZO@nA2N>Z2S<$US+eMu=U1-Y)ISXqQe0^Z@h5r8dc*6 zMj%@Ih>ey;`|aK*7@20|J0a-R>~^(z$_ zLxyUX3}2vRnT-4#?FY!B*F8mU3IVEPO3FM+*87#jH87x_5C=q|g!3@?7L{ivFuHcd zgamqSr~9jzSWO(AmCfT>L^XDiZ!u;th(Ub4nB-Z}gTot1d(P@fYfx?L)7pyZ+w;@y zl%VYoKa#lhzYeAFkqDYN9y23vcY^%P`|ZS-ME>c0p#KaeLW@vvSE|8esR}@lLkg>i z@acj8w1`#UD5Dd7ParM;B} zPE}G)F&Q$~{wdj_&HIKz5=vfZmnM>ys3U#ymP2BPQyPFt~M12W2-a8 zGl{rySG|V4rd8cx`XN3tPv6gK$Ud6&VN;X}g-IGpg%Cy;D`v^7yPGmqM0ztrvgyT6 z*BelL(g0>XiB@B*)KDF@3BPOB8n-$v4Fx+oy2Y9h0xLvq2M?pP{op8ml&JwibK2c4_A z`&(8?bAiM!zi#3hmm3?N3loB0)|#&b0C&W4ftGS210!H-oWVhtHetFJgshd4z@y%e zDGlRReQdv!g$3O-3+`fyr}l^}tS6gM=4VVckd5vW6p|zVI9O#NsV`M{>_?f`7;r+M(&3_JG95=iL-1gYPI$JX z?pLjT_yl4twahsw33gN?Bjk1S^N%WX9vALEO(S%76QEC`TSv|ln+B|7Ou%QjtTOVL z9P$6A`Roqsl}|4Kyb91K1P-b0dE$O&d#txd&;Ki}_5tXR0I20DyWA&DUAOS}WiM3> z@f#j7`?Iq@cjvr%&2Oa4`tXSYTVz?X>+ktS=OE5X{i;8A$&q|kEZO?tEonh3lN|n^ z4a4(|zL}>PjT?>7e(Er&<2S7+cJ8VM$~rH_v%wz%JKg3R7Wyr5@*~6X9YZ8B%Yrgh zbxiYEoIhi*x5lY-7Y}>u7bLBM_(<}+*JpXY_KRWS)Z2f07f!Pzmi&cMX}+!W=WM#M zd#vy0^H!Q*C2oU?7OEuSbb~@_XxXf3O~ha+%X|%e4RLH^aiSPhIO=s(`)^A8+bUeo zUE+NVdi@iM5i%Y=@JnkV>sojW45_?j1ES{dKO~eDdiXhTY_g*_?_x4d*&{A~PWtVp z(k5F|))Wi_{I(W%c2N}FS28QImg6N57w%D4Li1%2FZ zt}b2#BI@{PJlsiW(UY3WzbMFiB^^}gW9cy9=oP>Ma{Z#AveIAKKX80aT=k^*S#l97 z3a{(_L#-gh8gDlhX+^u{wOI!LOJyK4OJk5zljtr5Z4a$kT`trbD)$5ez{sg{n1}Iq z`_Xf%A7nL9v_lf0@!qDhGiQ|AsG_Lm`>@BXF|#RZV*XlJ0H5MLfK+t;p^I~TT|td) zDRAqo7PVV8?-C@iiDjp8KpRF#(zJp5H@Qbzz5R^Kp{mYXeU{ew!t0NH|9d0?dRqjn z3aV(_n2b$%Vm?|ug}x`t&A-l}A+%=;&A69ai0%FEUD z*8=c_bf)&(BKhtzx(-+4{9PBW)p?g!h&|q*`>C9{s@g+f5vrZJ-%3bxIv{KFk6*n0 ztB+1@al1l;803{QtR{s2a?z>`Kl%ZDUPsepfL&szB&0=`A+Q~JR^Q?ZnICG6HvD}7 z0H_zf&6F$45lXI~uAasPo|@%t64?I+nCvY#Jvj<$yukMC!j`dL>*yRFR&G??7_KpQ z^DkTq^~Rl0B9x`=kv2sD@wBINA~ChH)h&9BX80$n>_*S+LLR}j|1Q!NdxjqVynm3P zElq>|Ob8LB^%1bK52LNRU#{x)qt^hd*CVFy%Ub}pqTH1)Q(@)v-EfB)emwg5fA1N& zmrO!C@eomRJkPcbL#1*>Kl(SBG9gy&&@l?TxcAHA0W&fz{7zGR1XrbQ4UQjntw4v} z%eS(ATbM33y2xu%d+YSCsV@;lue7yDpOD}bGTE3N=Eu`KuzL9FYpTiQ*gD?r zcTKYuzMHF8i=DVXdODQWFDA75mysI!iyz37axPXAT zc0T+h-2Hs?L#qD<0){qZVBXk!R8hR`ghUhewA_Bu60sYFr#T51hJ}9qMAd2&o8XrC z>BwN8@v*n2%ZNQB9B$A-4PK7F0CnPV_%;WG}D;W;CZOeShS9sIZw z&P(&GADS{V3u)ly=xDnXfSLBcx@@Bp5;m0kmCnRMw z`{IcXK3o{`MZQHym{it;v}E0lw0MdUin5R2W`7|o2O4${d7vt!zFMcrL~$3#S1b6A z$M4VQ;~t!2!2~+jeoS;Puhcd1H0(nu2tqM)$k`|e=f$)pivE`*}0>YzuA3nJ4dAW9p~%o=>)%`h0zTV{WUnio0+?8 z$Gq42ncmG+K}r+GTd@w70@zugq@!Wdx7D<_IIqW`kWsM_oQe1xJYt8Rq@+5ZD=;MN z3CTGa=Y;8qn{%Y|22s8b))nNR6@n)3uduUtnOrYVCZ!ukw^POwM@MxTwKVEvbsi~l z-v9f$|Nb>ac~?`}1Gze&nPWp>Ap5CC!sKS*4*>tmZ(*E| zImC1^HcThqX)1~EcbR^e4SL%Llk(gTR`zzN|6=!&D87Zy zcBP^)A*7>{^u2|I2{S>l#NftmG49x=$%5%Ub<~N>W0E6jhvQG-zQoA*wGHQq^v-Yn zKl`_eyYtf~gz?5#HuuONz!G_HjrBWtMLsE&nVFXQ z&u9M=0Y>MaXIynOHj&igsg2az=D;g|si!Nox&p$gD;{GBdx#@q<>d#V|Ni4E_2&&AL6D376EJ(P zcfAP)wj`4^XrL7(%5c^}DKr*>?O`G5z0pCoLE3PDk3ahDC>%@+dzJbW<9@xChwB&< zBc0nb7DYK_!FS|DSHaGKryuz+ZXT}jwZ4rf$-D(D@XUX6QY2tmaeZXT-S&!jzC67fd`qfvKxv@JwS~~HCDovs|Q>dEW<2=4t zc_@Z@`^3}Iv?OhTd<7{An*C8D|d56WETyWZpa;G|r8dv!IWO z@x%^PJ>r9%42KLw*|wNd`-x-3PdL7bkN?3wp1eBRf?VtgHyXLUH*pwSt+jagh2481 zW%H?B;6~j$cr%XP_}p4NrFi{og<4zeh8#BkYYY0CJH!b8{tNE$Qi^IOP`n4>GRKb! zyU{}Fi7Yt%lzY~R9x@3IkqasCJiN+H&4*?+y@Wg3I++O`-WCP4dFgH|jJ_!Eb*M8t z0|%5%Y2K^;oR*=w!pdL}XIq)#hu5pV8@CyZB`78SQ6j?|{-m0HNSYg`;OKYf*7x^y z1|EvWL;g?J&uFs*vG)h2?#OsyPp<^{!i`##4kb!pTa^KA1#8pkdRa{@@L0r^&nUQv zP7S%ySYzeV#y0cb@!Qt0%ZBy7N7}<1>#omYil z{od*YzDmGMcL0q!x@`OQd6z+66348Ih`dQWT!+*A~aCoO|%{;=c(hwy%7 z@O8sVPr%Y$|1{0SNH#If1ISKuFeKlel*VGMuu{psFX8Erz^=c)z*?aC?-l%;`kN_$ z3u*oY4U`YY#KD}@V$xfnAt1buQ+#99#-;1=uAioh5J&^^I4$u+jv_eohqaWI(C_%s z^M5ZB$PgIMF&-9TWuK(JfDrh}Cc*wx51-Reu-I532!|=ax&{$_IRmF1=ZBwP-DspO z*4C@W0Nuld?#P258^R06RqI*f#~lLqw$vi^mJHe74uv{WPqMuWeY^PQ_n8jV7_^=Z z&6^vxPBts%5SJH5vE*f`e=3vmr@27rNEn+q7ozN-XV7qZbCAvXUw^Fq7ASWVY z^Tor^HSZnho_;yxm1+IE#|Rni*UfJTTVP!8SPgd_VjieH*(^b5CTMY<^jt8wqRQIjP{ib z*dxU~Nw_1Ov90~5AmT>Br=OXS&7+g187Dfyj*v(xVJJYxmgs3`pTG=~cPWc~D>COu7z=+nE3qY_^4 zcMl5iKIJT)T*@vlJ!J;#52o@Qpnhlj;VxGcYbD)COFS~cIpOpYzK35mH!Vl;I?wZ& z1g8=a(?q!xYS>WFhjsoS)+l+X&byrfrzHnUQJ5?HUV>dirdHJgTSMsH_E=U*JTd0` zNx>ShaS;~c*;3;Is$t2$*e_<>?Ee-q3_uL8Xg%J?$C)dYdnYl+4@9>$wA1Ds>kz7nu(yt%O|B3k4L%UiJzp1 z(P1p}@y|J;{0JwEh;{fc!Q!PGJlQN@Ue#Q#*Q!qrgdu`)X2Qm-JE!p)Kt;>(igK4p@N45dtXU=&-AkO_j_5kSlh zPBrV$$Def1fA`4QXhkqiFAYi>dI`(Dh%z)sJO`|?-ZIeuLmui_ zj3k7{8i;f3%9?X$)%Di7KHy^AC4t78H+rzt$6k#MT=S^yGWn9 zKKZHP_-Y-Q2rar+c3hbSHxe6_=jUKm%W`|Qi1+uY`gEW4)S!&P8qhp!CF{kK4Xy|_ z`y(T!Y90E~kvzMD$}?$hhI{=<#;%PXcG7XzxwzPQ%<)9Mu@zfFV{;o|;^jV(BKlX- zFOaTRY8@x@+BH5)`J+3HMvqZqf%U5wZMOS~ImG_Sa3FK|o-ps<12UOr4lX*+g+z`k za*oAO6#hZUaXuUEpNzAP&ixzX_X#)&euo2PJf2RNj82*kzT=GD-2E{t-(BgcnzF&*RinIl)b;rok`%FR!E#UY!lU0~dgD;QeqwGLl{%AB2- zUXP_TT~1qlyy$@#54C36j7|H_YZzgW7%<~(cOWil z3qwMl+|N21o4BO8{A9;CHi=-OC(9b=Qo-x#+KMz&xD&M{A4$3SuJ2%C`E6(3s|E7^ z)$#$#=#vVpFh;TB2JG!T33-96R{ndKlAw`u#V83bhadp+z43oCq zqXsv>%g`8+?@=DugB=6S5MrJ`pWNORVPD~F*le- zq0R53=J3|!ylzXo1%fLvbbnX){fn*~l(=dGT_lsJmo?owDydrs)9H}L0|Ma~DbeFS zYLrj3DB@>W`x&q@Z4J#gfp6TK*4-?#%~aE^vK_84qB~C)o=#rT-DGwZEeNw~$!hEl zyvy~IKWI>j>7;7w$4{T(bpGVtY$L&HA_hS?KJm-3;p&+b+uvvF^^>~a?-x3|3~xe) zvbW-ii7Cc+s)c@^_;#uPc`M7DC;;pSQAUeHA{-U zMgVDpkm{%Tj5JIyr@%U80S!KTR_mWT-G6uV7X0kpy6*=rzkD|S(M&`e)W>)jsAlU zS#uS2kQkhMgU`UV*K%1^b=M2p4cfcObNMH&A|!N?@x{|ir}9Vr*P(FpeIAbNjk+H- zb`W)InWW6?TETET`mZ}A$P7f5%LwUIsSfe6k?Q3=4xOB>%YaVqaKqSz3@Gnkub~vlfMOh7qO-?WA@eEe=1;hO8bV!)+yWTOggep z9l6AGif^WfvN5hWDr|3)&o(c(n}ws|<0sipjjDf(h#i%-jwruAN|KX*&HqJ?f(=x3 zZ*&!>y%uZY+1fM{_c_BThUw!R72Y`2<}GBB35mttS>8 zw|b3prbETuRB#>b4y+v}&5+Yp^`Hgg9F%BRze7LX;Y8>O} zj$#6e9rU62POP(%GoB!yY*kSp=?uU7VQ*}Z`^GzZj9vSe$KGOp4@{aC@3=GxcjRC^ zki4(C9u>Cvz0>)fLtvdBe^zXz1U8dY%`o#Vcmb&2_H5&))l-yJmFN*1y=pj&)bFk;{U84wAlKOy){GY`z?CMb*_% z%oNbe!W)$`8TcaO=5&y-vw3hHj1cGtRii*9k-%QE+pWv1X$mB$#oc))t3&mn8(E5Dslt;o`)Egp4W60 zXZ;o{)As!Yo6ol+nU8P{BE|aY6f@LyqGVuLIc{eY7wevJ!v*&qB37qnDV`TbuJC=D zil{s%dhEc`AnMJgW)`*fY45)FguhWoKvShc%a+(4to!npF+aE&zR!-+h zmP+T9dH3ZNV-ES|HEl4@d^9!~A8Dz41zsaA2oME4Bs|AMLQd+)8=hg>e~=(~0NkIU zBwz0Rj~0Exvw1L-geoiJa2KmfNYw--BYAD?*1N`6NIckdjH`WHPXF-XM0Vs%$QLc< z>G?Ax=^9jCjE;0D!`&NJ@q=GJf%tW8W1!#`ePaG2)vtO&X}$oR|Es#3o&S2`aDK=T z2sUk%s06bVhH|j-RwxG&yT8ozvq@=CI35Ziv-#0Cm#g&XVbA;Z@8q51yyPg+v}j$m zQ=!jpS~%H)fXkX^kJ0Ec;_=1Qj0R5N93uI4P}Y#o%#N4B0!Bx^#}^hc?K%Ac6@4{R zSW|MxNte4HrQLWwo}3Dr|B9aJ+_A5>4U+v!iGLW1I=}@HcjvnzIiFUevrypH2@T6L zOxUN^g^v^NX9x%I+c8$umc8J36+%fduz^3ch8hHosGcN8#;_>mUudH$Uy1t72=yTc+gL|KHk3Vrth9J>599)9y1J$avH? zCUU1*VXz4j@uHvDR#7EYt?0Ex9w_a~$x*3!mf5AvRd0S9hIp&eOk5h z0x8Vanv7!;eT-0GE9tK6+rDQW^2t&)%KLH`=Mv_@BqI^+;_68g88=N_>y9<4@oSp) znQ&Y3)_lIm0&VLv*ckIc7ROC9jjTJ`Z5B`+CnKjtO+!KCOmtreX3!~~WxaJfQe_T( z&Z=^})|{&k?u1*3V4QdqsXyh#`C7EkdnB$K}D)E zZ;3sj>+3tKM%5|5oAHf1*a@se_H-zrL=1Jeu%dlN1Jq-fx)K%gH`g>vj z8jSChwf(c%M4UEO7mb8OdC;(N@1Ni986KHYK8URhy`}I|Lcm+`%E(+$9?!cB_aW<> zpWVl)DXNm=NwIBbQdId{vuc8y9!nnOwYfp}jz5+h_AHy4Fu_Wamk*+GSUnEX^_>u= z(_4k;by8IO1j-aecZ$$dpum>xyKt&@m(|ZvUXzORC39^36HC*_#zx%Yfu=?ssS0e* zf*E^V?Uuf9{4B%>f|b*1t9=L#Ccs~mdP8H(Yib=_Wzz69VOKp(sUws zs12{H(NO7EgpoOk?JvPJj+l1oiEOdhxJ5?D?p?DYd)v&M-Mtod4o0>I*(@nm$9eJ~ zS7|v-NU=SmCA^9nqGVyaWxDBq{YzW$_4LKzbcHFl!8=hZ=`m8~#4`_5$q!>HmZ?UI zG;37KUNeidYSVb_e+lgK*Ak> zhOS$}Gy8Is+uS-0nVp#wsjU!^)%^?I(AT*yCuO^9&xer@<}1Wfqa%YkDB#k06I=0q zbsVfL+QT2&-bXfh_}X1VL2AcVc&)=QhCBW|dZA0+#UI#e{U?otU~&@fY#f4sK|=71 zZ8E~$VTma*(rk8*yEO<%FL|Mqprn)eIq5JlHZcLIx8ggUD$R(DIK2$nlx=QAk% z(}*MP%^%#j9{+9c0H}0_n`($O@NEyXM>*&4?FSp>&ivv)w%pI%%Tj}yCq@51(S-mw zGQtF0Qek)h5%~yuyqQ2mFZNi`#L8bIUt5*DPKnG)-Q#>mW_$jp?wT34p089nkzN zMn%w-{FVf=IEZwehY)n2r`4^jz8yL*>!evV$EHdyUQ8(RO{V0S0OCnO#hd0^b|&Pv z_Wrm%Mz6CRz)I(kzg1f11bj|cqw{Ug?wC=wo?WopFW$e5(#KLJp6}fs$3LDSUGu&~-k>nk zfO}AjzX!yIY?9&bHu>+_2rhT6&lj!x3m!`2Woi7MB&QUkURR_RQ~rSxuwi`K8@qDx z#xwnTSu@`X|K?b>dL$(Ig-&|4u!BSCn0!_?#^CFT4%6f{$@?0$qdcjitagS*T!_pU zCM&c7o1jQTtR~UzRrxgXo0a$8v9XoGWHp<+sF&b=qw(7B@P`foCkH zCy>R)v~x_rJMISVX$~Nz(G_5>NqR{ROUYyyRBRi@^bFf=k-VI))qF@qt8M zv)mM%p8h_``8Y^9T=H%xh8FReUf|0Crb_McXWgIPS+6Fhf-KB!K6l@)Hm-x zL{XqJ_S(i;>td3#eV3RH#^0w|>s}gxHHfHITi#r#E|=)i%1Z0QDr=+F3iT&MNuqQ5 z3@WGmAL-SX9SQ1dm&>tmBV+g9nZ<^_@^ob;r4kgyJ7O-($fL=u1p{7amf^@^_G?s~VvmPZWd6c^LJL^^Iv;p$L@XYJ4dg7b_{EXmp5P7JO*gnX)+!yjrJKvXz1{ibp9!MZjQWQPM#J=f#L*v z=F)T&4&RH&Q9{3Z&NX>AldVBtyXcd*4@7#q>YJ z$4R1U3GGyaZ@h&3*Fi7Vh4o!d1Vce{XkjWlonGod`0nujUc%qJDIHO`I<@w$&wypo zT^GoLpb}dVJ<6Og{oNyL`4p~q6-W_r3WbI3eZTlHb&5?WM%yqYT^t4qxv=prhh>R; zNKQ8_+kMx{wh1574bWjPI?AQ^Gv;ZR^*(#qoC0nLWkgdn+yj3VD+ULBO7xy47OO%+ z@>$7Qz19l`9BGVu(#9>XvQLm*9hitC>dXf$5vWh4gN#wXb3$P;TJmZZq5)A=cA(GG zYD#x^q?()%o$@>Jn|bSx5WIaFEgp|6N@2gipJzQ?5jMU>1NY{v9Lj8CY3#3{0k);u z3TY)SSL#_RwX<<;q;Q0-QyI(Bb0pyj%xVC&?&d&KD#w|m=|a8dTb8Lh(GoBE>8*;k z)>SZ737Z4a+Z` zT!v!UP}8DZO)4SFOD7FW=l9T-_J6$R{T~B9G+SED#+)xI0AWyUQk>l73)B)6s3qyu zl^~DYTin)6kkfmSEb=tP>#JYYLrXb!f|c*k4Qt;a z>A00JAh+@J72PvFGC^GBHelmr3y;ao&5oJhcOOWo543;z78O_mHu!q!?AT1Hs?}Xz zVRmg5#IwunwAu-Q1l+{8DRLYPhtK~sBJtX8$u3_I#r4s5!NP_T@qYa-TM+Nl7j4Hk z;y<4`u04yT^}A&KsTstHo13@!n<@G#H0$|>*0k{~-hm;08mz)*LT2rEsBJzKb%D*xU`}rdgHIC6k0-J-YAHYdRoy+BF0w0+HOq#b!Lp7=$F0S zl{O}A8Fl^`Q_kmq#xo6`+gwJ9)%``?&JE2Tt3@8{r#gU$;p03L(aD(X0J+8yxj8r{ z>@5+$4VMXpSf}5ILp=G`muQRt9XFo)vW5M@He~hfFP#mk4MC20IMWMepw!hP!HZ}}%5;XIU zey3nMSl0LyOEr&BQYcLamtD+W@@dT2R3Ej4HW(Ol@3?res#Mi!s zFqsE-N*1I-h&7f%7mh9Ln_bds-dd3xbuuF&hE#7eR5)U$=Y~w16vRNz`fW@2axBd!Kh5c|@w6=&VQVtOGpMS`)&-rU<9#uVb_u zAXW0Obd}L{19T&~RpVY%j{vIs9D zc;mZ?hqkrA{uM@LyB0Ah86y^CPO!f54bGz6{Gse&v(#Pa2O2R+ z>S#ob6_3Q4so$%9UVhqAQQ|+_X`XW)d@3;KKGCp47)yb!N^LX5Z3z_L^$@)~Ah-G2 zP;SIUc|C=QXgHncYiMYI4OLLdyM+4wM-Ekm2#Quf#H z@p_sD%Dt4@YG5mQ#~32rgc*qUm)fERY~XsLYn!POj;4D7pF9TALH&QM0-J>uSufnJ zER4S^g79ocquz_}e;R2USsSbP3BBTjZJ-&^i!Z7lIHmt68cS#2R@B;?Qb|LhR!GFS z`^}UHI~+MR7ofJ4FkSC*ZfMN3M2PuhHWl%%jT;hz6K8_k>J6U6Oy0DG$M;(49tT!1=+UO9vx5Rt`jlG?E}A3tGdd7(6s z5uq;vcCtvdCM=!|3;ML&YJa%Z8_A5v4|__1#!YLdxhA2yGKF;;Jh~)AXApMQO*U=% ztAyk0U@|YbOB^(%L{tO=qbk{fp-yS7%Dkf z@fz}Gl|Cz3X^utyeWHIVB+%02P>xdq>P=N)gyoQCFpb*FwAo3^G+-9rrj zINsj)EcOzdZm_9}cKEu;K9iU)naSVOqh)A!u94=ud{-26+S3 zA(E*^Kum5AeO{C~jC@I{+6K~0a}!kjoLQ5|Yh-5^W!CZt-8XT=ep5U1+(2DrFjolo zHBrehG%Gjq4=+b>X((?Y$ugN-0Yw?dShNx z_z_F%CrPK{$WMyii+BA7s`Frh99#V`4=A^zod+f zljh!$)0kl|bx$eIVEPgPmC5!ul5t!F^YXC9gcAp7S@+t#=f>SEBaw~#6oENpHbH(T$CF#cwG{5ix zDI+<0&ojSE+zgn7#dp9BSL2rR$!!5kxXzO-%yx~GERzUvak0I#JH_3uqp2{)Z!vFv zak$v?fF`_|D{?#OGG`3mfV1K?E}2G^itmcr|IR%J01_(QHk|Z4uK^92(+)2UZu3F? ztEc|ce-9kY)wY$%9jPqLn!)I-aqenC{YrwJ7#Z|G>G^++3yd-q8jX$-5)F8fkYZUo zO6BENA(rm_@{H4oih_Q4z`IFq@R%#gWiX)8y^q7}eN%#=H)Q!jecyO1p#t_fS4m|@ zo-xMipnj_zBP`zk@`dw{7t=UZeGnSgU?8})^gjZ%`YGd$UHV{6q=|)bp z?a<2?n|9NNhBjJs*O!zx3y|N%;_?T2U1XWeKe-56X4kzu%?4r8>c<<^m84aL)>H$98FExE4`x(C zIOo%g%`b&<`j*dvo1rta#brawEyIn*i-i2TfyvB87AJDj@`HpnSZUJ$+VrKhrB7aT1GuBRkel=)ixX{^viyr@Y7GZu7xJ7nN09rz$`b9-1lJkN1kC{ z$^%YSay^Q7tk*WL>8ornzdB&~CkFBi@(3h zf1FkQW;SN*T1d)xjCZdyF;qFct`D~pRz~_b9byxod&gA0CQ@{9PD&)AAnYmoIbKjL zGZ+!IqwNYVyCLX|kaJ2aZ0;cqVY;iE@%U=v$xXCHN1cTodOF8JuhH^g<@5wTx&#pAN_t)_oq z=POdf1D6QM;kc-JgfZKmw6=&tnL;sVw)~0dAlA~ZqGsSyDptv1JE^kpChKucwrgl4 zCyY(a4od#$;dh}us>BQl>TCB9N>XJfXE%|HBPK@I_nMx9hlyuKhU{y(WOtJ2p*-~t zI&^GlSa^Dmd(xR|IoFE0q8Lp&I|aFNRuf2%xYi4jbeJ3ihei>&Ckl)av-z+c~KjGMsqM)<2yrQ?>VhPwP2)OnjJO(^#N+h|v9@`j0aGgz`ye`!I z#<|fS=||L~!O$<#O7umN>MJdLUw)Kd0T5Qtx7tJ$5i$5)^5qCOeJRrkfMkd)Up|IZJS$c%p{o^J>Hb30f^^GcxUVAPPn%#r59Ih(~| z4tq9P#>XaS$FLjz_^2#1{;M-TjehhbSQ}3hi&Fyv|G(Pkxlk<3Edw6pHB!SIMYGRt zu}m}6QoK61z9)_TG1wxk{p~5sSn5r}s^yR|N8yc** zs?%5!9fOF8?C;~el$MW)rSo2n49sPa2(^2Ts zCROfW@L{R-=AKX6JJ~###*Fn2h^S*n?w+NaW)!FOa55Nzp+b!$$k10b>D7Hr&pQ&N zkYC)~m0iX@TZtWzDlv66S!PltbgG5#4`slRJXYTMJ)7>+roA|ITCt#QSHIR3-*VhJ z2d;nC%lnay`wg@GQc{Fp-rlPg%tTZ+Iw&hs_jV)+WFM)(D1^##8JW=)MZ9NW;s}lr zCNDutl`jYBUVRuaZJ<;Ty`dP5fs=o0hoOf0;U}nd@{ZS%0y!UO@{l+Abcrh4tYIhB zE$n;QI@;E1o9Fi}hRPfrS*GD%4){#=--1C1cJJqpf|dY?|9stEF}LNc3;z!sR;OH$ z?h4#r8B~8BG#qVEV|A#VShgu{O*igCtEB?QJT#;w_0hp&NB-%E=f8irzY)-me_c81 zz!ei5kA1Vav(2 zoIYT3gTay00>a;uQqR4w&f$ql?2vn|xeR(V^5$~ZX*XRfqZ6fWpeLv@kp*@S9(R`1 zTe2i}*{7+`rm8-^eJ&)=9n7lBW=B9bSc_$iNWOlTHvG5#{-Z@cPVZ;9d}OULcabrt ziP(MxGJN3e#`SG}LYCol#hiSU8(;TBbUsc8JV->ZwZRrDI2Vy}{`7QQvX<$s;eM@g zo-k(mZ`BQ|piU3%FoQXvDCR451m^VB^8MzN(5xMCufYWHAd`yS=KftEj-K5k-GO^S zQ(U>#SyyYAyZn`m?uYwloO)rB_1wwnLZiO-H*_aXyX_WS-s{newxCj0-BuU-rM+Hh zib+1(o6RDAnHX}jR{mtxy9>MtuMKqFl-uXo z4k{?sYC}mgfOCe1OW=EonaFub(?^5Kzo0J>_jekVIXdf?|qHJhE#PjvM8$yj=hmmF|C`e+cIH#3;>+K^wefCo~AFDR#PK%-67 z;j5DLtEzae9yfMV>0gi)h&QPQORo&15u4V?lY^iy*auI`8gmWU&&C#5%huUvG%lhp zdp5h<>noZ`>)50klJ2>|%7U481|5YO@&qesq}Z7XzcbQ3d-barjv3s1bLFlqA_geK zG3#4q(V%~$8|g=eFTKoyq+R&)wp+{?#@m&sx8+vpB-h5rnD0~3YmqDE-3Eq2R{K>= zE~v-jLfxRZfi~$U=-z!=Qp2*gb8rhO$`3nJww14a<~X@`=rj<=9-O5TU3lOpiyybj zssO4$IxzZv2Vo)b+j<%h*%{BV0CcHDJk`1h`#YxvZKgc9qe@8i1Z;N-}Qqf)iq+of|LBT8_wOl*17Y@w}n6+*jqi` zGi>{8T~E!wC7$TmC7vC44sv8N)stb{62!6ZeQZ2>1|rkk+L!_+0qKf?p(y7r4Fg@ZCJ5F+9wwQA6Ix|AvLP1_(6tj@fS zV#1q83%+bwt`@&>-c&5w=%QAa%U93Eo9cHuF*pYN% z|9w<@rrpMto-q#Z6*(D`&+an-2j&L2Dw3B)t6u0}(;(EY*hi$#MuFJrz*8u-=0&{s zZ%hoaZ)sgA%fKeAy3QBLs-XPW80?+DDF+v&gT%b~Q#T!VmWPY%#IXg;A(o4iBhDF3 zAvTcO70sih5V-R-b;{HS2<>V|{%|>8*ep&;zpL-w=4=NL`C9wq3G>%pk2CVUof;*! z^Xoo>?U*X-fw#Ot<0BKwJEk6t182y3pV_L9Q`K9TlK9M{A&c2Nz&Q|nOAr_a^V)si zus^N|pZJoVUWW5QfGf*m1Xc+mMcF~R*2geVu*Jej9GxvWa*3B?m#K_}m z)CJ}FuBK*N0VbOw(C2UOk>}QiGaBVU!aX2MHrLp%in--%1YGL}ocIb)vEqNv`j|2? zMB1To*9w>}l_QIkp9@Y7k#_m@!m)MQ|2PUS$8qy&#cy+-WSlLka@*2X0e;F0Lfhd# zR^uLz_xEm{HIe++qx0L4ePCg^Y_~GIvKW+Ccp)70pSftN*G9VD#sh!%jX6qgg-ayz zyV5YY6qXq0L=`i|pI0r$;WL9qcy}*@Jodccad*uyPDGb%Oa{w&jP$+dYm5a}8_O24 zUEpo*be+uXyDk>-VJ?kkOs{nEk63DhnQsa{;9u8%&jD;g1QAO`3tC6#?qXJkY8>E4 z=`WJ!gb0UnU+zq1Go+CD6vZo4CoiI^S+mi15yZ5mY`lz_I`D_?KS1E+qWmhcUwxp3 zK|=WLhn2xEd*Md{J+JHl+kLSjWeHkpX}GnFOf+VuZ?*b@P?Gj4O8`d&YA6!gU+OH7 zT(Y&3cxC%^+pXo>3I&slmS*+eYN-_8Id)oGWMm>XoDq(y7ipbt-szHXouAj z(6N*UAGTdLiN$%FGwzjsM*m<+EP+q^VcH(cY0`kr7PYSNE4YBuu@P6Aus6{Oito=O zxew7&X+-%w!o4hs@XXMGol~KiSA)Fl7BnAo2eXnCrcaYVO6x<|R*W~iMbemjp1+8E z`hw};_w=n=(0WmLHLd0u3sQ+G7x@!(byFlxnjH+*T2bC=$|>XC^d&H~MmD0LPph|< zj@YrjtoAQ&FL38URAxZ?E{h$X#S4kM-;$c96sPUt=O_RTA1_P+dzm=Ty5~E+wSLI%<9+jotAEdcRJ+E&+S${4+6NURz3hPeFxuuDhJo$ zk;Ui3BT{z(!`f?V3#5ln8Os1DDpO2SxY>6Rlv|_1GV+<|LNl0GC*SStpz4xnZ9(*N zW%P+z!;_1g!65g8*39S||1g-2GLd(I%-y_Z^g#FhxOy5wZqxiKc}>#Vyyjt?m5@l> zd*5u$@NN(Iu8`%m zOb6-SMO&tMtIHD_i_*F9P$F8#z2q95YMl9Kb6>Mr^wvA=@D-aChoSxzS>Serl!gEjQ z44bp8@s86asJB`f? zLYi7@@60@F7W+W{Dc^=i#-1NjphDAAAQ*1c@Q?_zBx`HUN?RHV?*IM_c@kNn*)qSp zXST)KbhNypIn?T*+jm$oq01Ipu+;-!gw-v1pR#>X{<;*9QEbf;`2fL?Ers)I{k7lU zXGG8wBJCx}|GCKPsI!3^pkUL*1eeCwg=lMYgo0;E&S`*|BYTZzdDK*60&R%WmA{JK ztdJkZ$IEtOQeGk&;8O+=xm!cB*a5Z9m{vs5L!*Dp=DT9+>$dCbZG<@nS3n1|LAJ8+ zq;{rMNQs(W9e|;0c0GiT4N0*)!6u0v(pudtl{vYA_Ek&3SS=~?<5ji)^`{m}y6YjI za=X%qMx0c?eDYeV1+Vzq55j!^fEyQ50h|~Z@HndBgwg;SK@L_(D~zS~2W4)~*kNx} zn)UW4idzJ(ZD`E8kkrUjiNp^e#hFZ`DSs%&0z#7exO8}o;$1X413_a`Ix2uiNcoV{94N~pZ?Mkl6egMT9r``4klRR z_+8Ij%`oe=>p7u-G_EvPf}B5v4fW-J;rgFX1g+rHB3W-!${%>GP14^K#`W0YpbO)| zB|ar20e#&nCCVD3DGCH&2<9?Q$DR4D1|JJp96k}L^^z(jp2yX_wL7jk&A;s3NXw8J zBp4k0%jw!3)uC;jE_5y=aVI%7$1-Dfvgh7CsU9`EC?3dMv)u$=!(tfo0=V}TR ziP&dZsW2d$VzK%Di$h3b!0%;4ei@KAJ~J%aUn=b0F(y5NPKb)st8WVOgw>h4xIf9h zB|b~X)_`Ga*jUL}ggFL?Mlws3@6tXb2;C-4)O(9_`>4Jp3J0Z5m+&~M9~qaKI>0&4 zcUsOCeCN;DKGqD1S0d!k+B|0Lt@(>5h_0V@7{ytqjZO>g-y_u!?wT5g9KDy`HrOwk zI2@k^TLsz&!1nd~A}b~Y9BKAWzxfX{_STr~z|r97mBC5h%M(OkN0Oc?1@4b0`$p6L z&`vh4i9AeR_>JNA|78IV$L-qa$?x|H@4&h-Um^A`XD=z4INcFhx0cSqXRVQJ_b^7(P=MF&kbk?Si%<*Kf zRAvoY9$+?YFVN)Fd1xQ=K)Muv%@?oT*8J~wCP)mDR;_p?j@5}Q3nO=;maU9tRLLT{ zMpuN6*P;MWN+BU4<^3<(vR{F&D+6n{JxpRF>A&31(=h>dIP}aS_X?9P`cuWCoLZda z*X-`3lAVnOab=&EJlL+H);Noj2qax-A}!2$lq6G;byJJ{q#&34d#fU&HoA~jfZm^n>Jl-q$JgC1dLbHKYYLp-DBiOHy$#y`65 z`TCMXj#{K`vY6sq=-xOJscc(?BS}Qj(RJW{!SYTP9h}_A_o*3ihiS?X8UOwSiV$ z`II+8H2iItYX`A?w({MHIJQnc30YSipSN@&pnAVY-hv(aRc0G875*Q)bADP@9?C}e z+dQ|2>$*y-HGJ|=kQg65%{>|K(%f~uMt};B(ahi<^aJ)kXkS-Xu>UCX+%MT};&0*( zk713eYIxliEIqMgQ&ya!nq%h^cZ`5yu(*)!Ps72(t#_qY%U8$bk3lJp5EX2i?#)uu*pYHFPq7f)YQ73o|(|7<}b5F>6N7E`a@;+f0 zYkppGGCmZ>-0@lBPi{wgil9+`0b#iQvfZC>k^M~1OC$Ag2LcCxUGc~us^g88eH_91jeyfA3Zns zmf#-cH5+}G?TbG|Qkcjw^MV^Ph3y)6Jv$uORWTk!wHu+#xOy1(z+=^}q(YHN07`Zd zS!maSHdw%1?JK5y;LV!JVgy#Mq(_y|j|Nk_5cJ>!)Z{61*B4D}aU0JAzbj&Rb~b7P zL%q$KNCI~I&D>vU4XF;x$YRm#p?{vY5}RJEA7*O)w#VzAMgOuk)?9y?hVj2;uK}v3 zob+>kVxwTbz=!k-8~-a;f6~>MTf41p)Fi6pQ4H@`3tpXlh|zq*EaxKzESZ4fxOwjX z7mo(WNU7qDLgf$%?JiNDm`GD3ceQ##k{Hz9=)iwT%(n=Xon}^76{YCTx9`e}D>DwE-2wZ|ldjv%@46{&HO3ec`|o&YbNRFVgz3U> zOL4@3o@c8hM_1FlbBKMN_Ix8)cGw5%AF}tkBH<17q6O5tmSte#JBvUNPC5TWQAM4Z z@6r2m!mi6%SEt|gEu;k|9tJTCu3KC(D*hR6eEl0~YC+W3oKMfLEe;MwqWYXKojAY`Q@vcX>RGDw|DuOz!uC zFH`l2pc>|N3FMdwX^K4~KcNc$=)GrRM2y7UVoH1}FNE+bl-Y80%RBn$9W#!QhU9i} zpdcjiuRl&_FHDs@9HzA3H2@#07g?Iv2q+SKSvJc~G^JYTHp ze%G}N`4m{XLGugC|3LX_XIC`8GaKy@subGSBC*PE5h)4e*ztNiS>MZU$+JX!PR_a! z&wHas`r34%ee4vZg%;+Rmys==42qK7iM%?Eq{6WrtYSOgLnXq-Xs#a0dZDg7#)uT; z$`UT=YCLN=eQ5tT;c%CSl0H~^_}4EDYo2&3*&2CgDa(NB$b%0L2N(-vD`|vsL%CWF zh>NTm>lKOMpAyIvMQB}Ji}U01As_phb^1Q>Xv}&ENjT&W_!hwv;2#*p8TMV{-f5-l zV2R2ezTDR{acA&59fJ%F`qkqwtiA$Gp0f=wK;JKSu?IFA91c(yPm&=-%QRxXD_H6; zZ#$cROP-%*g0*poWxo-OL*MfvO&jN}fPoT3e(tf_;NZOK%Q5vG-PF)P3c= zS>~~H?qu^70WFOP0*WCwgy)IDPRXw%^@e;f0d(aVa2v_AyBwjjAvHR5*Pd zBkj2`=iB*={bJS5-)n29!u700R}PlWarpAB3D(;$pQMRHV;2y^WL~C|^y`(bj+{+d zL#Vd{ug*{{l$;JAIio<5_CF zH_oj&m|U5?*r{~Jb|=HI?xO+jHvDMLXVHM!8AuXPx|I^?v*Rf)@zN0S^TM-KVGExN z&u4D~$l0OM(HhOs6d(WelD9*zT5^nJ`61^P-M{VxPXfsDp!&gotyM7*=ZeRmfY2i8 zU6m37GN!`CdF!o-$Gqsz9A9_Q-JQ%5Q#qW9q_jLgk}7F5ujXVI?C+HlUwI1=MUPWh z4lB~j#>hg+9M9*@w6)T8mm3%|7}S6Mt#my%VqziYDddR0h#_ToRANHPULzHzyCi^C>*; zBfBYq_tbQQgWL`hSNGZFCCpi;2EP8^y?FYQ=oyqooywv5$*n~|pakuwq{@LRNhKUH zT*{*U@oOlHpt0!`dmn0J_cVWDE`AS(x=vp$eh1ZqyT!HxQp6(cv`4{a=--e8$nl^8 zgZrTp8jqk$4rQs0d#asjgz*^~*nxvO8{@+9o@fmg5! z^&Ih*2dIZ>4eF?1F*b}jqpXC>ioW1;ikFKQ{h9^bq}uz5#T42GrX{M%KxYgyf~X?U zUDimybYF?exbt70G-o}((~5IsxkQTpHvYN}t|Av}Kbv<_PThx9v`gH6v=QeEyf%dP zHNX8*Nx zj2M~Es~L;3^|JA=1&{zHk3a5uhhJ~Kk|d0lJDQx>ez8i>-<)!_L+*84{&V3{h5d8= z)=#0PU1~@D<}TiO+rcpuOKE=EVo`qIiGWmn-rhGv=qKj9y`=56w@U955p$>SqJaHC zwvi%M0c?A1-;d>7*x%GPu4Vf!Sg@a^rfbeO9>4b@S9q|rqGuF%v(I{2x_O9D5bl#^ zr#^#F$34!znlrB3>bLF2)@3UD>1OvYGht5FNN>4Bc|nkkBSZc-kCo6oX)*>S>%}wK zhJvUp68SP)bjRoG?|`WuIgxVm!p_sn`zg48adn^x+re2LYnCwhIbHMYE|DPE{$HWX zL(&D=gUHss7G+SK>K5H~P1~zbDZCGm8L;aH9AH4$HweDzdN#QKGqB?TAkbpD;BW&o zV$f{EaVc4*vT4L^!F59H%%d&HEF&Aie$?HqV~R!Bw_PMXdh_F)a=mjxc+9A51gogEq~fv;u)SI zOSnAx0?D@!>*pxk4LrR&yZ+15gb2P&V zw)s0EO48gIwaeS;F6yGVa4j+Ike_s`uSu>=TtE;MEZ95a#u{*wOsm4URXmtyZJdUq$_xXF8g%ur^M%!rs@ z;tze@9}goCcCMxWQRR-pRn3`nch%VJu7Byb*MJh6o%t8c*3FI6Zt1qeW}Ip*&@f3R zIVfy3pv?i#nk(00qWCQ}@frwaOpK7-e~GR-L>bYx1YDjx_BlVEkXk1!)Atob`HjM#^W+bl0XXU2e+}>j#DY^7FuEWBCvzZ06~u2?tb;K+1ZZ%6Sup z>PQv-OY6FHuJ6C+y*D6JF@GN5xQRjJ4w)3yc&=~ofWdZVb?cD>wAk4)nEhbuCu_QE zJv!K0xs=e)rKf`y-^F)YYjxEz1JVzCj$WpF&V`jSN(t7=sct~-%(>nA=iR5xQjafw z-t|H$|1&ZDbqUYmcTL%BRAho#|10i{F#GGq#O_VCGx}bOVc415%5HBofV25#OfdVu zxa>~GjH}g&Zu%iGaPj2-G4+;VO+W7UINhOigCZazASGRbAd*Vw2YJ@lp zy6Z?goNxx;;mLoE1_HOOdc6L*x#+rS3n>`}=(!Lzv<@&NeTiHt+7fDkr`8q*oWP3f zXI2>v;SzNsweEa=?xdc6$V2Kzbot%Uz6I%6RvE_nx&7M<_h6Dx_g`%c{uIM9Vm>1p z#BwgUJ$0MQ*V^S`*s~$7Cig-yn%YHk&HJ+y0~W?vrpVw`|BQeh+vX$@{|uV07-6FY zO2UImJdXC;ma`5R^!A3$IPSPcc_I4xp7{)V>aqZ(>YStYZz*VZC_`NxmEv!#u1E_K z!$7E)?wrmY0;r2+@{mKFyS zVtxeJi@^81lUw_w=l`K$M>p%T&nR+YLox;UuFkn0#K$&mlTC*JS2Lbn_m(pM;0hwF ze}`XsWIHknGv9t_lc#nv9oLX178g{cR2AKH2z7v)cz5dOuMmM$Xxd-8AJ_%Tfv>>*4KwCzR+fZBof}{4 zhgOtrf6z`?^4N<0n&MO+o>1vtrn96SuuT8pgFvI+6bszvn_WBVG`ozZl(f1YQ)I?Q zz{clO-VV3Ehd!$Y6|wps`o!8C>sy@zCcm%5s*IFrn+r_H_CHS3&S<48G9}h7>p#FD zF8fx*>;}!dG`BbXJO29~LeAIDMc=L4Ev$~GHvv7D0}?@}P81B8E3$r1`xE!vT%N)@ zIrB1p+KplFzOdT?^sv~V%K_a<>)yi#t|rS~MzcJ`Ok}e{OsJ~M^84!f!bGyzZ$%dp zbllXU#j4ZmrK>}v-9pks3(4wa+g z@CW^V`q&j+z3MNBNfwT?Fsp=;LYp=A=V_A!VSeCkUVt8D=tMWHudK1Y%Q+UrWbb+Xv<3j*rR4IHxU$5~jUhgp*&>(i1R! z7wLhg;#Lj)hIS6PAw!q7pZ?e?{py8I(U1Rqa|4qTs!88qdYW=@W$)wf6*r&SlyTm7 zVF+)%qTiZlZ@PM zlG3|ihpXG@MxEEE6kiC@yw*0wJ?FDlJIz*b0$#pdTF7)GnhL&Y*6B?clvK?Xd&I8o z1{pzs=3DcUroNJoDb!jIQvD~?XGIpbBa#FhgGTfW@60!9#RqvOD1{$=!OkHLG9mhz%&<2Ozpy# zc@lOt=?6!xuAaIqri@=7{tryV)DAI_WwiO0(9=LRL8}&OX?~FsarwmV;9J+vR9?~N zXx9mR`hP05jF`yNt@o3hL?@WK|ImhSaReGhDBIrT`u+lrk7>5nSvwhnmd3>_3BL6N z5h|?lLG9}G34-(lvc9+8ayz6Wvyexfq>)SBD~wrzSGIMV3CgS9t(E;(k^8OE8Scon z7swcABSOV7w~1#iZijep!XU{*%)+YXARBY;RPRBFXRznErAn!z+{90cU(*R|DmbJT zWmVfviNY(B%u=#yhi2;;^eS6Gs7~h9VK^8R*BIGyUVT)QeM__|1o?|%n)$5K>VrGN zn40Qtk$9Sfaklx=o*Q?Qt~bRp*5H)Ko(O8M?=6YQ7}__Y$Dkf%`fZzr5ggB6FO-*> zD*7VWR9Y~u-nTQ5+FESAXYG|XLiGr@{IGLfT#M`+D$vxm|1k-#Z9Z^38WMnR^0ywNkj7pi zjuE7h+{-6av*w?nHXxeG+G9`CHE*3)6F03nN=a64`o}`6a**n$LJ3 zA%D3i%B3U{Tobgt`#e&=?--neZokcgef?gRQ76_uVEbI3E_X5$V&P2hFWq(dyLzNr zsr}mLcC3xTp!v_trpdF5m!ipJGzxnkw*?VRLbX|N`uBTf6b?_Jh~L(dU%IcB+veq} zJ+lTFeH9X}-uA`dEE?v{eay=yIpt|AjtnK%h_r6@wKM(aqjzh9egLy{8@*3AYnmjm zQT)Er4rqy1@+>DXjbKfLrboW83me!+xDhFD4C8{w^_a}wy^MLUDkceNT>yLpC)gZ1 z_-*AoSu`M!G%}9lmBlJ({odOu-Q6lT6^vuj*D-vm>qQvD@+hn6C~FRSJ4tE=JqDi# zdv3}QXVElW*p!iAeY&@p0`#!s_J@o0s+i(}Zxk|t3!s#$MKI8k(W`$BrYqLBR9#5c z&!{-%ncJ~nd{E(h5JM^Voi3rsQhS>v%hn_$QC*TTX%QS07lFDtOFUW~zg|tv%gUd` zpAIZM{!RQ%Mheh%8~37%KT+A}p)ci5V8P~WkJwC6Uf?C@sY~r;F?&MJu7Z})Stv#B zk1CgzT*BVnaXfm<=;u4HSMSCqak9hjb6DTndCH~JwsP0P@Q)6pUnyg2G8>%fEaQH8 zAxM4!luVrvX4Hp=M#$HX>OJy5peWCAQ3gboB9IZr;ki4YTfYC@W z4El`4_iorz{AL19vM&TR|g>^T104B$7Jl z4y*@L4?2M|3VWZ_;|AZMHkjA;R&IHS0xvCTuKx(PKf+}|+iOLX*j8r#uKoi-;9nHf zSAOY9dlaZDt}cn7xY=t(6^RjpkR8;k!I&gFe$L&|lrGz;)*V2*9k#u03&HJ3)y(V2 z(QPt6khI_4FKs*wO?PU*M;*rnwKcBXIx(i%4uLwqM7gIL3f%bO(;XFyN>Td{!&K(x zuc`7o)AAU8g;7j>Dd4Yd`s3~(fScGyseIXllgP63sxX#oo_FN~`Rtl8eD~o!svio^ zc2~RbAKi}aOv`s};?-8Hn@h{OkQ!uF9VSOWZ1Yk(IjQ^mz8v8?dG%gg@{YQDz8?{1 zW(IK>FjlC*-Qtwn^92Pe_m$JtN&Um2=tb1jAb-A$FwtBi#mdCPYhTTHOCtlHl_)gzqF$&7pp>`X1i*T@Fz zgHyj(j>pH;8J9Y@TfAimp2n03_sgy1PVsxD4Q{w+G@%kJP)j;-J1^Rv3;8lECT=T3 zKw{A!^5toZ)plIZvsT=khBZm9{xn(9Zwnk-UkqW4P?Nz6Tau}PfNV=yvErsRp~T&k zFAL+UBb3XL)UVPrJZJc4?iIaL*S`Jr+=dS}BL1r*rmF4s04mV`WBT^`*R$3zfEfyc z_H6go9&U!2js~ra<^BMke28O_Ay9ydC}Yq6D?qITzlIL(DKw@Au=dQVat_u3|DNHo z=IDpbX+SD+ya`mE;x>2jo=ISE(OhL>^nx903eW|N-@NFs1>ILad1~WIcG0Ck2jKWZHtt4MkT+6o<=@!^4zY^ zwwLkTtuv|ptC4~@&Rs1LhfjLA>q+z6UVV4jbG#!C0<|2_Yk#G4`^=Kqr!gkvxk1Ow!Qcq6Jg@qYfO zrcZ6>}L6wPp@HDz6sg6Sydz|-`je)Yid;ztvxgO$~@PEru@p6LcS5h*yyRa*C$ zbZ?bz%PrOZ5nQ!dk@f5L2hc*SOouz>e;e;XYMB-RonP_C%*~pm6xyg!(Yc~u7 zs+9hoZ`E>(#5MME;((@qTz2GJsDgx3iIuN)h+NduWsEntv1`W*&XIhI%#Ph>aKq*s z4Ata0zD=2*+791Sz9wqT&TLebp@%1CXJ7if2eYqi6bltqv%$jQD^WL{o-P2jc%Cw5 zNl0B9INWHx>GpW}9w$f73&x!fv2F#FP4IN;`=7i{kuvQHg!#{ei>ODbgr_P1U^t0H%9v&@+(&EAE_bFn1TGA|0C)q< z%Q^q}TKZEd^}7?ZN!Y#gElEIIDB%=1@h>dCvtoG!*t3cu>}=7{8SxBca`{>xod0uC zI@qJen8vtCF2XjD8Ga>}77(~_;2l~^v{J5Uqe&`R$JgRzL>9V^t@=%d=L7QrU3R#~ zQK0hr%yx+M-RYhF>9rTuyCoM#@joRqS28TaH}+H=p?glj6d6{xK>Ih_UU3{+0CP{@ z-%+SyOSzB0R%6A3-f@PLHo;j$_HQG#{B*!Jf1R1a-FfQpvV2Yo4CWV+)%5Mh0&yX= z2mBQU=T28++(*A5MmW4J$e(i~@sA+kfNU3yp)3g6>g!N=3Yf>z$Nz>CP{_e5C$7_=k)Jzjh{hoI5 zOaI~Z@y=_Dq>WdJMp2x`_K{aJDksacbzcP$wi5#!KeCZqyq%$jPJh3CQcmXih)CkD zjQKL2Q^y{$C4Pvf7_`ZHq}go~nfYbeRZ;I~3Q{^OZI3UAKeIDdoRoN@*-vh8x5*(L z66e3kDw)=FHK}QKaeFS%>o&+;+ZHK(W9I%<-4`alR>^!#L&Q{d9@n+(@#X?V1z0GY zyKKY=cd&S==eCav^~`uZgKJu?QalX<#4_2%++=}m_GGx;PjaZ#iD)reZ#tPxi%M!R zCf_y->fuHzugY;&-lZ*?-1M;?bpZ~jf5Z>48XX!q9b~ed-Z!Wn2lCGkT660W%w%B- z13OD3lONtPqBJWz*rl&=g%}oeU~m73Ttv;J(v96WO|w*;;M=1WIr1v(pKUKS!Jme7 zrKFiQ_(05L7>NU8lSxAxuMdz+TOm3weV+yGPGQP`8$2(_5J_jN4i<1ubc-W`I>6`d z`dHVN!EhmMpQCo%H6XCFlo9N-_)Q$RlTzNC&Y%hj_x!fMq9a9Pe*sXGAHT95NJslz zpD%QFOw4gmR(JuWc;$YGJJe#=4s@O^GkbB+hPgKSF4y#F`&%1hSg@S`(&k9?ynfWr z^J($a^fMvcZaqHxqtt}UB!W|c_e$atBnGWiB136Ypc@RQ%PGieXZCHeD7=@^n_&pR z`4QK2H=)@}HUWmG@u56Ey-o*m6zSv*qcm|z%1pBu(&Xg7e!nF-nO8ivfjJSHKk79_ z*K)g##{Qk~;9+za3|;nh{_;u9`_YF*>fGeM%Nj&k@Xo55D)@IB=+I=`lYCZO%R(dl zjdozTp|K&bkycNn`{#!^4%~`IZx#G2LllD&IZQMjy^LjtzrrT@fX(Yv^N|c=rnKTi zvwr|eW??YyWXMs-J21r17B5X-8uvjy#6l?$h&|0=nNgU zTYE=eSwS6!lS?aih{x^VB3DQ_K%gcYmjr=280mx@iE}If)sUd6Gys75UHgXmv+T=Y zu(=eMMCj{1pdif;3yi1&LdY<#1SE6`($;)i&a!~mg9lY^i0T=k4tqq#ce~N(>$`jL zWTceW^BV*NYL(7h(@o{FtnY4?7z0Xg76zEct3(XmR-&Wh4YlwUsjJ3yGe*0E2OVLGoOcjTCBYlvQnRLE#M9dp+2XcO zK)JmF>CPm~Qs;Vgz!aZohdRrX{wUmfIqchAa#{Ojzn?E{nZye!Rk&EbXrUevU&qsfUBc+$$^h2pHK8hu2-kRn z#9H17sag#X&8^1Si%-*(vh_8aBK9}NEbCLRh6T8W@gNY3?qzQMWFm-YF4Oj5Veh|J z-y#0k=S#Z&1i@BS+d^EriqktcAK^jFBX>Q*4Oa$1fM3REj%*_MlU(0Y9G401npZe_ zjyygpSj_F$1*C<{_B|n@FKx$vF7eT=IQtdS!4DgbqkJ`0j%f}-= z!9uDd0Y?faiJpsQQQoQk5_I3zAy$&O{*r=4DFq=V%yA8dtTby|C@f<-k)N51$rmev z*-Nh?df9WvxNOcUy`+W18biJP{aXa=IvFeL7ty(!pg#|pz1i`4x696t$TR~7AqSf0 zS7UKHnD6jpDy&qBbV@=2pIot8dZAksE{Vhgk`)WmS(K`?!9tSdhnbu=Nlq0CgGSH; zq4~bTjMS=1RomqwiQfia2Nx#OPO}0EVuUV&P~ep?sIl*FQ5drX(p2O&lyLem&Yx*( zE(V^ zXv*hU1iQf&gA}vn56N{Tjf6* zCFGm$mWA>EAe$o6QvZ{M{R%-aQrSV_FpChh%+A3QxNkn%a14i=%894c_)~_ zjfIF5Fy=x@Mt%MKbzVjf7d0rWDiM&h$SO;GqL1%%6w5)0N!NS40{C5TU4AJKUK{aVKep zk;@@lN$Dp;2#-Yy0$>~gd}_;J(h4g-?n%?EC}8LN|49}9$D~Yi0!)fuHL;#zv!8l}6 zYGX3`HNre|0Q4)mPxR;I57*<%t(;&X_Q7T~iHA%#2hQLjx>tCIt=YG05mt+NH8V{Y zNU^=P$&I*3MW+6VjeCMnOm9~yu0Fd^N{+;)S{j~co{QoMK9Zeg8<%0kMyDcIg7mf>yz?2rO%&*~@={g!(LE$(9 zCCD>#vBd>Q{lWWe`D$vKbXnQTXiqdX53g!<&nJc!(5pCw*-rdHe1e~;2M|ahr)x_C zGGR&tWlyT*Zr|=PLf<*Wb?UtbQ@plsf92ESGrwla?oZWgY-gHuVdyE0`?CWj>CU~t z8I|1m3F&|8FJLzXX+3335fr#plxjtG^O~Lx3xm=%>u*&SI+ZVsO`L(4bq8KP=TrYT zDXYBAXYbnr1{zS7gWPN9;)gpX*3+BY6LJx4b*>vKBIMPg4zn-X)K&Zd#ObYfnbSoa zBOivE$eG5hXXAlRt5fFnQ|9+pzQp70o{yg@{bl%hD8bz9K25Y4GQ%gjZM1A)uePmD zK-RfR#4K_7p$;+A`Vw||6mT9Q5G!5^&ivK*J#EDzXktA+7$34NF+fq(J(2Y?XtD)! ze)Qm)b~!tb{m_h`K-)huaZjA8J23e&X@s)401+S%}93cr+Zh{S0QTJEV#Qs3{1SitBVxGmZUbzBT*N5g6T)N^UvNW@F+xM$}-a2 zdzsDf9cdFy~H%-(ar}@y)VIQb;n}nJ&kl|q`unNz@!1dwG#E2 z6Th6^MBqI58c)fjxpZs4p(?9Wwb+ROxdF!k@+DV?ul2Ay7({-_bujGWaDP)+*+mI` zQZPotQw1?dKie|0NE7~_17a^@Egg$b2>(p$_QTkLWp$YDp?nIC+Va86h;*|dL{H-h_#DA;7 zWvDkc7+^(U`Yrhzb)pnB!T%N;H9w;I(|dPPNq2y4{r@Q|c1go=YQ_O&1xrb=;H+9k zAd5;ypiJNNFATuo<6jolKS?v-pc9Ynx%%{ZAC54sACVE~gYe#8iqUT|=|VW%;$tUI zvEKC2jua3(3HHwN;gG03r6sN5Bwj@tloHz(R`SQLL^{6DSPZrwF1H%dlES~83A*J{ zx!O^j!b5o?UFuK`dpcoQRiZhI`sAI75I;%=k0HJzV9*|^jz9s3E1}oCZQfb)FWwOd zLn1bhD}rYlS28iBF)>bzp!Bkxw4qwcV@Cry1qqxAC}EqP%nrH`7ugAbhbf}+gVz57 zgY_!w^**LEop?>}Q#G-_Pg0};u6AcM73@+sJ}eKXCTKPszqBnGvoEN&MikI`b07Gw z41GYIR6;vqnNekS;jPH^&t`YG7ocEm)KdUWvFno!jGOH;;P26v)WDNAEkg)eAE z-o`U&t~_;HRLD54(%>zq^a^6S)uBanzaCJ6?4|I%aJxa`>|mMcd8{#ONE~;OyTDb2 z+wY$1W3069@%VJrC!ZkIq}TC~t1Z{1flAx;xZ`W+;x-6zT+YP$H|3k#U$VnoR6Mpj z?=CXXeTogfO#2RD@I5UPAP;X_@VRHf!uHpU3x*hJXupS8FBixlnW$G9gSar4# zt#J?V@%7`;B6WhP-YSgS0qIlkg^O9ScK}g`>Ag!34{aKww5(KVgV&>bYaq$1VuJm$ zp56ufm)=#dO9}0mB{qNDB`t#8w$YVp7}JZ>l(=>vIblMUHA>@E$Ta8CLZ%n5^M4wUaEuZi83Dcm zaHb&2+;map5Zma@DSJ(#fS%oB8ysjtO(BguC&}Pu4BXHqY*OAXLbXBKEd20F(1xGx z&sEuw_xP{<5Qg}XZK;WHAIN>j|#vz%w-5q zJN{)eqeV)j$+&Mj>{4$irS=p^7D#4uIE~_rEfe|d@c!m+2_C46oB!s3d!m|>RM;#L=WVQs`>>G9Nk7NM=*9x*l3>4~ zpS=i?Cjm=aC9n4_wyQ|4P6o~cJf*RFRvq;;NK);acm^-uq%QsyE%sfkQ$7CrH$!DC zZ7h98IeIY7z9D&g0PPJ0H;BYnX_K@)h^OvYp>F47Fwt1zq>uFqS|PO2Q2cp(6ON7t z*nXE1LsmG0Rh|J}rqfk?w&PEb2uP5lU3o7PxJ_HvcJNt9mOCbUlsXRo(EGUR4ZixsWyF^nhBHBvwQ9U`l&AfNEqY-< zr0Dr^VIUdz#XsZm1vJmuIRTqdmbz{FWQ)dvkW6L@)VZgu9nP_3n-q9z2>mvpMP2DYC$0&5XEA^ zH}eI>f80?SS>k=qkfe-{Cu!RFd|^WMeyUieP0Wr;Fzq^MKNHtSt9zyg#5r#N08cxO z;$i$x!GgEs596P`*7Acl_V-lwNeS)RSel1&Q@|8w&&-sGmm-J2sd zGqc{(oBX|1RwQm;rrWlp_4e}8;wgPeu#OB@4&^6pf>|8T7e$sFf`)9?p9QuP+x(So z%x^`Ns=gH*4jLhU34A|LGi9vUG%xxkp+Ej3GruCVM31c?>U2E^U%-0g3>g?I%<<$2 zRtYEHSGLOvdC>H7CPM)1Z{8UH6D(=`2ZVEzbdx2YOPyPyZvIXv8YgxCAhK%XfNR<_ zEt>vDcm6_)6+E?Z?S6ewV%B*?KG*$y{I{~j)U(60_mwoO=G$E*hB;4;3~QZy`?VUq z9F6K+&?8A0PwI4lZ*y@rUuWQW^>d{e5(b5;zQ`xa5Tz*ltZuSaDodr&=e8MaeQxFc zt6lh$%>42Vq0^RfEb>}<0R)ib;Z&Iwp@n|L{5-EUF;m)dg4U^he72BR^^l$a$T#2PYRy{ zg_YxV*}N-#(~4~&AADX4GnKI(zOJ|po9`!UzVwZ{V0`$h?S|(Sksi^)m>%VV#AGB} zqw=}PxW#IfEz8d5LZ7aNVzSohIlaePZ&aL;FOw16hl~~1wa;aZu~*&&r#!Yu?eSbO z9PrwU_?*ncS#2|0_Cs-e8{llea;P)e-sTX|mN{N~eHi$rG?(3rO)`J-LYd0{?r)yn z`IPSL)48Azj7dCNc9p#3ue^X z$t3k#=~NoaA>puN%VG9qA9aSeBIhLjivB&W#t6|d-P-j`X?=5h7;V>N?GEc0;v~m#jIC%@jAL3Wf_R&) zjZ-35s)KHrgn`J#%hzIh{MbU!*d9i4R#gAI|3Vx$XJ{bnde$E zVPY5?-|9$q^=h3mzkV^(iKgHTU95L!^*z~qlPM;2uZ~-64Pus;Q%QJ&pTMqack?4i z(`bzODhPPxOsJXV|7ii@`7Pi6QcfoMJCP?m&#s&_v=do6wsID^`=LL9?FGqLbC(jf zU_Pr_gDO3vIz_y!nBNdQ4j|Qg5@~x%Z5^`enbNIGPhS)QworQI=CAkpwG4BCPxk)* zH9(If5W(@(xbsTb9@>9EzIzkm`N9y1^? z47$%g7}c^$-o6x1lpCFS`AKQ-@a2ymKT^cKODFfv7Uy$hG2!GbqBd=vigBY(9a#IO z@ffdfHiy3xG7qPOn0H6l+oiBw44ioq%#@M$7RozK>xif}AJ@4e&F%u@vWm2eA^=e= zIF>!kQ-c`Y&SkX?ed|IlhF-X5;5%2d)p4(N_F93*UnA+fCq_kjZW!5LZP~A0CAIa- z5%2G)dFn}|u}xHBIndUjwRKGu@c{u*Pc?GTYSgn2$9oCJ+xNe>zv&Xvtb0KPF^qgf zCi028Nd^n|D2G`5I%7weO4?t#1EtLS&aeA1cQvDU{9|7?-J6002g@?DKQT6h*)-5z zA%r6(;Y|xvj=_f|IK{`_x|LF>Iu!Rsqp#|RsI`hGrso24EnJv*Wj?yHq&|k12o`qV zg|TE%(7J&(Fch9Ni)Sqd-32(Tv>4YX5K{Ae;Wn&EDe(|3R`@BP^^Aqh8&9S{# zYht@tal*i*{wGx2Cz->ZZC+8&ol*2KT+-P6t>tF$Jv;twKE9?il2mvmwpE%x=b`;q zNpZEqg~(Lv5=|JaO^- zrz2W{zOYog8woAfrKN6U8x`c}towakq))4{kA%lDAq;q} zp;_PI%nr@yNYphvMw(B5M|>NpwZ7oLp6GmLWS)%SmBTm#0pUtA18>Mo(lOnnnerbj zT~7UN(+Sa)MmN58TQX{nVp&GlG5dL|f9?B$IE7C_|H0VTV%|bK1W&+4Z0EZ?|H%so z78gqyI@cKS-s+=Ax4qMCYZlWh{Wt)Ig~6~tGKK%?71h?6)bU8BxCPeyBl_$R36%HW z^0htlk6g4$R4Co-vU*gvnlcLu^MSah9O@NRk;D-v0=d-(VKF9U zzMSgRqD1SqlN!VSyV}d1UhrK1@M=McFWoR_pV|VV7G_m*v-W|>^+zcrCsy~_x6Ygv z$@LD3>KFs`R!ZcDwfFulR?QoM1k`>cW)_N~PB7;%Ok7e{huxvzHnH(vN-@hzVryIc zzg31$C@0??)!(E@EHlNAf{Z=DB{Kf+fNHk>nFArQ{vAUH#1oykRN>2I;=A?T=eH~i zJNb+(BQOwl9WFIuKKcj>_%M<|d-fPV6Q@wQLlgK-vMea#i$x>`{~N{35$B(3Fu(H7 zHXpl>5uRO|y}P!y_k}0~t2Ed`Bl5ZJ%!ut@73&sBwx_3ewQq080=hZ8%gI|0o>W9d zxlHmZ)8z&kW)`Y>?$17BFFwH^_abAk#7}p~@=7=xHa9$Dozh0vlsEL%_-y#uq%+b2 zu{$;X5U?uct*x02d&k{FmNiu|k#@zZCThcFv_ z9g#?B8!_Q11zT|_r(u^N&TOlgMKtJ)P4IuM`&AZEF3ce1Eu}C2J+~2$}zE|8}u+SO$Mxs*~f*f+~iUBNEF7wIWD5hEpjII$nMf_Biz*t||70 z>DfG|@~p5SxWdzV8%)U{gJV@;aQYK)QqbrY5x(=Qw%-_qWbtMk%OdQ$qf=@>M9mow z&>Q45T%;{c_##8`4*2MeaVRWCnTi5}dtBv6oN%#!xK!KyIb{Hxf97i(>Gff2osoUX zyxA41$3I_}2TriN9}FEXYz95+ja?i1NSJbon9-~EDzOcMP;fPRP!UI zB8cgYUI@%FnOpns8lT^Aeg;3RV5$5PbeJG1ULb}0DX`qYdtEYS&fI!r-S1sU+>mZ2 zOj|(YJHcd!pN8zu;}xw-mk?FE6k$*Af+i;TiLMQ zRA(LhhuPnhd2;PAZk6F|PLH}ISOuM!ISTChuGFYr#8NQePwg|TgB)qA*I{DCS}Hy> zi7G^L>rsi{$b%*!yiDhNiAUj4VFJ@Ru_8Nyjurnc4{QVy{hNSWWAKM#4t>zKsgcGa zavW#5@>ky<_adm}&1S0kH9D*GAJw2!q0q$TH85nK;Zc~?&Z56dL3G!Gn%F5|kO15h zZHH%AXE_}xs~YW6Bhhh{V>;(n+D{5iRDCJ6MdF~i$Z+H?t^hZ(Tq~YLr&pId3n)G5 z>o3I|T?zgkdB&YkPp)uR8`KgiR)KBZ-7AoXbj&@J;fjx^7|n?d-*l9m5y=h4@xyXS365_Yy$&m}bCuTV zBY%9pKGr*#vtEuTnI2GkRY+;nS(@D)uJ(P){3OS4At_UAZNa$}VWVB^WDMBR%9Vud z@IEs0+#2}4vrr4s1ykl*TyOS2jc1K4YSAYvO1XhfGtu|kHAfqi`y|K`7CZddyFqn! zXH#UbvR_t^Z`GHRGKd6UA@-C#rEmV`y`g5+pfw(uL9_Z=<)|Z$sfM?WL;N zA#lI`_3>+h86hdFrKr0kDU0S4gVR9rMA%2ka#v^;QZ_h@VkDc{uqRf#H=5uZ%QD9Y zu@k;mGjkG_uKm~FwjV8A%-vj`>qNDzOLc@v@1)^jJT#M40ae7we#$2y-7~eHK14lQ zh?^C8rbhq~zBRFRGG+>`c#and{8oYIlLN zr);qAe6&YsHZoskS=nBg^%FMsJc>5lcnab9q;2!?OWtzYq^^ex(} z$#-+G?B3{l_o?|PKC5&8jp_Z(Fx(x1KnW7Y_pLj8Me59xJccjrc$%mxtQqIT__R9$ z-g!yao=C1hc158M4%!mpbJv#v!6m$(!L{q`4?I^M9!iUQ?YA|;TkH!Thv0;2=Dn18 z-V@kmDpb+@ZaR^GEN@;w3wpjTseb` zFsEZ{wPDY}%*7XyO8=YI%?&~TDby;^WXH{f@>%~hKWyx+4E2adT$CDpSxOw8e*G** zmQjr6eyRta@;2RniyJWTZm-!_w5Z^U!y%<6Etteu0yH zcQ_k+Kyw?@N)q%__dS=0rRHrsHaM z_K4Z#a@k|A7h*ixU^j!eWHY3%tj37hT$tGR9$-ubPhVtQ5*{BM!d(Z&}Hy2*z$y4{`{ ziolgXIr!qR?bE|xllf}%HI8G6C{korKLin-A zECTOR1X`oO?`Mpu$ zcNjhaZD_Xdy$ZDoT+VjhNxW{Qu~N$YR(5#kJyPRHql(|l&gNWdq*v|4$Zs`dBPCb4 zsRByecr8755UKy{Dkg`WIq;M7`5_`@5Kse!c7LWGgmEYvQ0>pR21(Psbl4foN_YSQ z%!$GyBf*G9x5=kX@=@_t$N6u)sK_<6?O%mAeY%5{R3z#R%aI~!zGoitbTH5MpM*Ny zuYX)otqR(OEDobAx)p+y97-KOGZU(%d?%l(qzbF=!UL|)AK6bAzXNlfk{e17u~$mU zPtL1V)HRN9NrKlIT4CtvenUL59%lAYgo> zh=mz5GNxmVwUusyr5+b`sfaTEVP`%UT4`>Y&D598tDceB_|k~WR51hq04>H9 z1zh%#_~KKNyB-I#RGxM%yLHAa^rKnQW+xu|H`8|+V|iH^5ohq26HCwu|JREmQ#CZYoYs@w)`%?a>D%8=ZI}eK6gEQbisO zi`N=XMOy;fk88U)6$_NLe+RSj3+hBPUz*Qg6H9TguI?h;=3)*#EVeFMQ<6W8;3jS9 zo8V<~jx?U%t2r#iCM^teS>O@B{(=O7IolJj+dMJ+rF}skfmZw|08V0#JCODrzXHrK zqE*vh`d&P~`dospGm+VLwKK*lXfuiifXmVQ7C;E`Ju*x&S-yH__DFjm$+JYXRWZ<)9BSXdgDivD1 zuJk>JJ1pk~&BxiiL~y}u5~%SP@d3C{-glXuF7ecNj>p*Lcz|-NMG|1h1pnw+9QfdU z(NAa~ieQO-Jd%Cjt}~y0uyG^Qc*`Iidzz=7`wBMl=Ym&E@(55T#}tx33~_$x*dd;) zsJtLlsEG?Su)SvKsQDDz-}#LvqmRyriND7xujW+?Soy37`;YZ#J(TZgYAV#1rSA&p zvTIK3j@?~;T50^dx=uk0%zJCJD9(1>o2HpJn#l4#nv8-XJUf}kv&68{+H=ap4#$(M zTF8*M1ufc#SZ~E^hOUTL3d%2o)96Gc{p{AaaevkV!C-;C_X5lgyHG-zbl&tkR9ocj zr_L#fyD~XATrKRnQ!$HIKtJ)DMo(qQ&V0P~o*!T8gk7cCy$P9kQ zf?0y)5sbB{??2%ML2LgqSX#Z24s=i2_}E5o_}jcf0CN$=z_4zN@W4_VoSjU>@$6!E zwlVxSpU5n}XZe%3@cK<8p52kK{)$R~IJ|s5#1o16pq%ip=_#IIapzy8*H8U3&gyrz z6i?rLitwI29!f0G{QZ{lL9yI(FLU)`3*1ffYMryj9aJwmXD~C5tN$+lT!AFHW|$metXTK54DUvbyZo*5L~t z8`3O==&;~W0*eg|g(2y0%CaJ$?#^ltG?^ITdXDZ(p^z3;_A@*$LXmTXA#%VN0kh_9 z@HNr|rVx=X(aS4KO`{2gg(6D4Ul`BcD8^mhzROW*mm&Fm?UoZ9;VJS!S$FC`csE3^ zDA-X0S2^%KTYYM~NXC=5X$V$!NGh9VK|*&wxm6C@HfNBUcCzonO1Ox2LQCc^F{?QJ zlbU>S@vrGU)#wd25R*J-7nb3Fc*i3w6bt%mK0 zl~)r7mB{%-C!*9od{05^e_p*1_x#7yZDp*P8S<6Lnz(MM5fUKBB@r4W*ssaN+4e@g z1z;nY0tqd^+7#uw(bF%>NJvG)tLf9`I=e%~Q6%kiga-VQERc5pb?wA5*}33o{wN2p z!KymwNI(~zzBq@l5V_s^N9>S%{sPnx4uArF-KK>UoGv~n|0hZrBD-mftwpOC>zrqE zmAOH8D4yV$U;fShq9^0C4l1c$RxJM?QC}I)XWYK6yStelre>HnZKiX&nV$C3F=M(r zH%#X+-QC^Y9naQb)DC79(BStH>0t3^u(8Et~7g*2JM-D*t>kXC*oba z-=y7wxx#n(dEPX2w=vVQ(9%tk=ff{UMBuE}l|~FDiePF~j-P_~nCi*g;G+ede&aCr z&

    Y2gLld8cFYVTb8W)eebaNOHln(6sejn<_rc&~e z*gDA4(EQDh5oj3ezB+4s4O=gHDeYoGdq4bmwuMa{VWz4agU!OC=hRaer<|`q!GUGsl<7 znU~fvLY|DSiK`rL{tXE4B$sH+H3f&aO=CP}9x-#iQF{by&)i$06~ssFiqb@%fA|Q4 zDZf-}=0CC-dbfCgsf*a<7mDb5RJba(-g&Qs@TpSM^-{;dWJO3v$R0|=Zn3o{l_#3s zRMw;~lTr(XP%?oh5w2cq$upX zJ)3>8h6wxRFiz#!Tm>E9cF!6^$)yox>`(f;+)BkCQZvB35zgEv{x7|l1m=w>xi;a_ zMbhUX^}fcueTB;AWLvnnB%XiFmy{|Us(NQjY7A5;RS`$ND5^)W(1)kyp8RJP@LJeC zY$`Obkvg$<+r$Yx5=Rh* z5}YM9w`9TlEcBR3BjU;;99WDMM?>$qjb9<6oQwKWY&_xw2W>Up!XR_}Am?yNUd9;u zaWREFIaYHGrPJs%ccz1Ic^uJS=0>@`UPrSe!i@C;|7PPbb9b>SQ-NSOg%7#@>-Ony zn=De@$CYPx%YHPnyRQ2vn0(_V%^ae;>%&l|y>UtIUxFT|-xV|O>>gckxvUN!kJD8e zoV);Xd7*T1Li7I#Y{dO%0dnmEHlWuIH>ZYV&Y5WOELxl!L0z38xHK=Ir^VC(eCRyr3;hQBpK#iz7D zG)03Lozh8P1#4p&eHYwblaOtfg;0c$zJN;~;>FhDbe6risoGdK>jbL)Nal<@Td`Vx z*W)ZKpMIX1mEewVYb8cqIw5;MsbQP;74{p;K*J;DvF!xfR0R+#dQ;lk+K~39P|LRZHtvY$+MrxQYuh8dak2AxyTl5Nt_`^vIRVVV6FF3T#T^`!H9BLglM6zJC-d7qxkK-|}j*p|+ zQXY^NnjqPHrs>sWgUMVdK%KB)w>v(dlP?X14 z^;Xkd;fX5w^295;q+Nuku;$l3@TlvczNI_!zK8%$$GFz^t{=I+n#XR!o)`4XZYP444Yo^vv)TMgZI?Uy zPwg+$IXZInPFQtEvpwBuN8*qb7gK3S>ukK@nD8;A#0_?5E2yNu7mp@!xtOeN#2-DY zUrRw@QVMlu`aFezMuUJ=5n@ggsJ23XfB(icB{}^fNX69KoGCBqt1wz^()GWauS6TC z)$b9?2=z4@ea<7ke{XGfzhJjs7BCe+|1yU|Zdt~C5r8(`HZ9bW`BR4GTCF<_iCUTv zpW}&K$aO-99f-Wv%ZTxdxkTxiaXY{HCV zYxYma3;}NQap-t%lJW$04v?d~nCs{8a$Vg7na_uwd5Sg8<(Nbl_?ZrF!qMN<4-zE& ztRI&fkvHC!-e2c=wqh}cw~DO`?Lw!$4VUy%`p*jiCkgHURkQ8AMN7^|xa-MsjbH)5 z2MJ7i>eZ-wZsjic6p1q-L=JC|fb3i9E(egn%gSaH2~)Uvbxa<-k$V~q{Z19`eUcwZ zr->J5r+ztXKp>FmQSOY@O@PFO6{O`OF9Kiaoz}`%fa#6lN3>_ zbn9wDsb%2jb?Wc(mZL$HnP_GanY*~+5zsquGH^GO711j7OLtV%$qzs+MpjWFX$2S; zth9hO`3$a~9V=(X%$s6xxwNX%h$XN5@2@C$(ORAzhKFO$3nq%Jq#D(i-g|bPiTw3b zUqxoJ-yb~LT{e$WAQ*hGpw1YIpUD3+Hnuyw)PNwjQHz`ZaNMjE;Ne#GZ-L@5jD#O~ z<@q#rqdB$lAN_pw<*=0VVzg}*zBCmO-lbq1sx!F2oJYN?^Sk=$9gJP(W&h9qbMj*U;l=)tU}6`~!gP=s?9FZvs4@$ac`4CnZj!2o zlY*w+CIW4ek4viB7huF$Ux4YeM2$TgHOwmB%4Dc3=dyu&gLeTQO%eAl)F*9vdGR$A ze4XX+@2&r;EtRT1kI7S*C$Gz>_9ko79)1qGW!@m4!B?4zGQ(ftEGt)n>- zs1R3Z{KFn(M$rUm?+t`uz2>kgnb`I7q0}2Nl*4GxQk6z`cQ;?tyAF7uCOjv+4EAX*mfhrr}RNQUgT(nDdhaEK*^OC=y2t^ z-keGKaKP4hyPZ027Jz{Mp(ldeVZ9T%@oL^`oUh(LfaB{R(S`}LzjIOlP%6(Ex#$ld zxChH8w~SU8b|B!30*4CKB`!4d;g5p!?uw>= z1Z{Z?|z2>->+|`Sp{&n8(9N;U|%fZ|LzO1ZJ^WH5^8r zNQ8@BDC4g#-d?<$NPAKjT=;Qc!xB-PG1t|1>SYh9QySVc$NQUKAvk1xg7Fib>C@au z$#(0>tH$H?3)p`hurems62w0{Y=RHpvg)2gIct_^K#F8$#{aXo{J}3r-N{;c!%+}5 zaqSP#-5>sSwJveU71dcMJMBDqd{{xsYOc4}S-F0u+q>sLPGage=gdHjex_3&|NmaDR z3m8q6uI%w^-KE3odup*2H>tE(DhBH+%#~^6i&pjaGmVO7iJ*f_LZ(+oUCL$CcQh>H z{Hi**+o+)){a#5OJ39i%e@U-WG5b*bH~tQH$m%@jIEf}}%_r|8ytVR30ju8fg8rVpip_}-@$K(WZ(juj z8=PjnaI$1$)k5OCBba)3oNUeuYZ#+zKc5jtmPoZPU#~5LOpn+{8>sG&b0Z zaTPNfdH(nyPRAIY&d}_!(j0S~vt&+FU6JUi({-DtBa2^Q}z-tuU! zRDM_6J!=flGk8vd*Xw`jeOVe~3erXR7IOs!Trs-7OtzNryEb;0-Hv}unkdU;9-Fgm zFTsM+hri>7J1Q68*5xx8R;T{voDHAQ z8$18hBMLXVpx?^IDc<$d$fSJHYGVt`kk1qf&lU?d8lkvnL&js%#Jac#4oU5WLP2$S zM=bLW;f-=4;}RHFEq);ivTbD!=K{r<2} zk()eCK0RX77dz4`%*D`E7!8p$;56(bJ2Ys&ynq(?oB^+?!H)*b-!> z2j25^>4DL0Elk-*FLC-B=4mp|T#f+M%$J74(oL)&jaJ)=ID=*hYL$EMp^0$rKk*>_#6ef4?+G6uAHle2N-JT~{jcBB?e!|$ zuxL1OR)F*7WfK*cYi^=RCPqX!3TY-*>m=jA-K4246ToLwVpVPUUsivb*#KdmN;8-| z%WsO-m{UvN><7{wl{7*d=V9ZFEb=O~uSi-XzMJtu4uryN%6jWj=uiB&b?3@-&mA~C z(I;AqVBsjN2A4tua=-5k4cv-?Z!M&q=G!06HxiY62`cjkt(1HdtP-C)es^FB=809{ zG4f(K%WV)c#8*2&bl?N#dkG4-a$rH{CU8ZqfNxqN#v(FK&U+zlaQ zGtMuwA|`zgVH9iAru__}p@wpC(oGqRxgx2Md*YK_ytp0woC4FerUta>6C{4TGRFN#aH7k9J+FQ=}ET6 zgfxMWwt!rDrs?nOB zXZb%!R(pU&W9*Kp4oZ%BLlLB~X##=%H`J5KVyKbl|;4DRJhmyNqR(4_KQ&t^uD z^M6w-QeQi1d6sPRhaW396npW^77Eq|d){>s5}oxi$gT`!2$F(+$@AvpWrl6JRxV9Rt;HtNb7oB@e0=?%Eeve zwp+Gt6i}};k%6r|zts0}KMv=yTVI>2+I?phR#Dc6LRY0C3>svMuf3GuV9&_wc0|vv zw-WW&uXCjw)<%T=O!}hUOJTr1DKTlV#`PzSF|%4IylM9(@>i+{xO>>HB^-w4?3!X( z7{{Uwm_$Wav~UaB6JvNZSXqb#h~_MgVyCII{H_t(5b8b{tHjWi=+9TN>NkhsMN-o6 zdU-TzWm(wJ#sisAyC`6g2GV+4)wp z^jg8E4%v7tZkJ-KXSEVXY+2YO&wvoZp&(Q=dn@v6vKqxgaO z^V4HD2~{iV?J-zYqOah7eGCSIlnDw-``ZyAtIjuqgcEOwFAM}0JGJT#U=EI1i8w^B^n=}d(+v=X(<2HIY9ImY8^qW2xGFYmWWxy7&)jsRC zIHP8x60TQ~^Lt)QePr`H#zlAt%n$~A=t)%&>c;-#y~SifbTFkS$>dUjWz83&u_k`y zrkk85TfF;HQ)34OSCl_d=v3{OoshWVa5So(6Akt9SHwR9D5X3z@J_ahS)XtH5g$UH z2{Mg`sL0p(xC4I8L1CwW!?N2>UBB@=M>*M@;QzBhQfc_Mz29Ye)K({o%GD^-)3e8D zZvGF_r7@KF_05Tv#m7ofs{On+1LWxdBLH}Q^(+jM0TgjVvq)iJCMt_H@fss2Iy-7! zU}_#=f)!zhLb3}$!NFmqEPODZ_K|B!Wgbqq4wdKqM!uqwt`eNH3SiEkc+3)`$oYHR z1@(=45(uZJRP%aBxL2#p8uYuTT{)6;q08RUP<$;i0AX=Yl4iDEk*lg8@VSj=A^3Y@ z1+E8RN@mG!G4Xm{&h*OQLxfQJvtYPxP?1B1qA?#U|AG9Ii(x-8lg1C;MB>FidL*7+=U%2+0rPl&eZ60=Qlw6_ zD^DV5zYe$WIEs->>@{B_cAYJzn>Z+6&mE8j+{AJ0o{xNzn8g10q}-YkWi!qx`kTN1 zRKh8Ye~`OD2opa}3^cK02vEtf2Ul&Q+4O}rhf)ba7=hzyvp*iSFZQFi7Hfr#jvH_L zOzA%o7ceio``*TRHaiXwu%ru9F8JK_g}6sMGlQ9oe9}w`PY~T^UMK#^pX%~xfe<{z z&Rlx&oTlZz>tXtig`HYLXUdfX&o>*Gs--$}RuAp)s73>SY?fX649Hb~1Pdqm+1F77 zV^U&_>#HL;@A*fl6^g?sJPk!bkF?wsN~y^%FE@KNK*xu`;r#ma;}eio9e0KkQjQmD zF}149!V7yYZ+)WdGAoSFZh2PR{&4`#ZgMzQ3Y`(ht27jXe;Mf6f0h3o>2Re2wnO+9 zPO2`@V<041)vdzb&~vte<~-r`#(0=tg4-#mRA3Cr)LV|h7eka4H$S?8Al8_By_F8L z_i=MOCfgoL4se_)%{Hm|;#MU`Ok3~t`U_*NZQP*LwOR8hpZ08i8kh`R}$ zfB>0uC?O9OL2SaCi;tK^-A*wksU+`mpxzxO`;&woq%9zpF4y_x(MYat-d?DSCF%KW z#cQy;9_##+j&85YKtp0UY|EEVZ}$ZQKehv+ZEw#$U8Wxi1~+7Z9-^1)@O#wJS7Q7b zw27j^l#HegqAWdwiczyeSf|gJ5PB5dCK5+{A>pTuNaFK9%9X=1*>l2U3mCwA*>ehgoae6U!6*)>?d!XdcZ!+{ zw`s*7_Vu5$+2Zl3$+|L|9}vOATkvC%_p~DvU+WZN*ciC0LCsN`+hnHgw2n|e&=25C z6aVhM#D|{qRm>zSU>EL9lGfD=s>VWLg4-+C2D}0c!l4B)?JI!jOHu4&AMGlHLW?|aNiV;B^^VF%pj-k^?b zZ2o?`J2vug`ff_ZF;77<;;>C$bs(H`QwC;jgUIP zXDE>U&c|lPqmu3!m7ytj&l5Nk7vnPuO0_0_72xbf>o&hHq48g;39L{egNy)^Umg<2 zW0~>zh0AgCzfLY)@Uy;Ih{UHAlT#-6#a*S8Kvu{HJs)E0$D6F3Ao4}ZF%OY$ssDk2 z{)*$dZik$J;M;GNCZ99@YA;ANu$raS+e14eVswIKKTdS1+U!$Urr^RwRsFuIJwDQh z=tHTjVZdFi0(nK~O(8f+YH+^Ur}SrT@)ig?j6eB878yR~kSHlL{Yw}2@!n9V|GE|R zMw?qiT8O;%D_y(r{h(E~%h5b>c_eVoe5TL*5pH=%|0#tCE2jPZSE>>>>UwK0(~$@% zttzATKxC>}zcSPNwD*iruf$YiT9Qs>E)q-lUvXmqo%cncVSD=mw6!R`iKc}010tn_ zw3dcRHtw=JS=3lqi8$fZnABhxZKn^82rK(@F5NflHAT`SRh=j@zqM@2n0} zOg@*HZS^Nj)L2dGwVmvxv9amY`GTcl9VncCdc1zy8?vOnMU+`wjX@{n&SDOyd(Gf_ z+&Q69Dz3X0Gv~t#Aqc;}ivN@P&J67fA*1i7md9_yXtt&cA0YTY2w1+$*evAO2ccw$ z;7;q*4AW5CKXxL+jyoJ4YC!_DCIS152Py@~M?Mo=uL0B591P*_Z$PvlO92wBS2CqW zAiSatE-xY7F^*gx^3OtGz1bCGcfM}2wQ%DFd(ft-#>*-gWnkfwS+R8Rd{?4v!S@9( z0v;)pYya{3Og@&r6q59l1+;vK<$Xex*7=@3DkMHa7{5#LMO`%$`cSB9AVbt0Do$53 z7-_#Hz7=il=(N zyRDNI`Vq2(EQ}(DTsmdv{FgV)Q`PtQ?9vPRv4lx}jjWhb-y5+^l>CgNUtmYPHSFiV zl}bS+pDYrzVbOpP#VAB8jWvhc^FK$U_>Z8FobO>q@%lfRRCIWv&ZJc{!M*Fazc(em zT3;ap2x1QVqhgU9haU*U6~($)La>mC{B@!HmOLId4_o;d>yw*v&HI)(so*WegJy(#-12-Xa0I zKjzF3v_N>a0ozC~QiSMZLW2PjTpbbN`Yoh83E|zWp8{|C?e7gKrFvP@LG;YyYD?P9 zj+qYW#~{z%`64drXY3?^OFH;1o){t?*WHc}A&mO3D2toQ<}IazjT0rfXCEIii3Fd- zddNrlaW}+*b1ks1v*98jqv+Kz75aYYskPLe$DTdk@}#55R2ae~uB?HWR$bY6rXaw&&58Xq9Z1L~Zh|8jL1{{3fQ)$#r@hajjTedQiKNU`FZrB>i>pP< zYfse;TwvdijddxT8)-KW2stWj&*!O1M4<4?{hcQF9_aU-N(o!Boi)ZvfFnt~w*9Zq z|FkotIG8-J%ZNKORSiN_sYKrfauk{HR(azg*FK4WPTf@j#- zSseYRO^-US6AF*vw2%_WVfs}eXA+JBe~pD7hFan3-cK39$2Sorp}(pvD-IUy#+bM! zt&%W+?r%Slw_jt9JDyYd3|N8!y{5n8l)=J}czcEF;f+sSF?74Uv-h1aFt4j2TllNp z=!+wXnCK}yJpc|>9hu7-vB|dLBJkC_acjvt5bA#2URYeV&b=*KRSrkaWs}`xF!=n`!Rz-vk<{`uqU7rotykpUpOEgyfKG*uZnyMzi=#;a z0Ye-#-lFSQZf;;(O|MS;`q*Tm+2So2O2}sW@UR#uoTFZ(o(DzS4EvcMfPR;ws+@Gy zXpz_8nX*~;^}-)ywXgEq@*|yI0-G7XKU29VgvTb70uF{r zwRiMq8C;zxRoEwMOq|%GiJxtyL#e+Yc;WvEhIk*m5g3etFHrKgI&FU6iVC{e?&`R|b~t}pLqFtH49UgD|_9 z3e%5h9v7cxB8xM~mpqmly-445hqb*Ty^a9)?lwTQ4k2l=819!sWajpuDDbWzLk0Of zKh8jThCrxR^pECilv=#KV8s(yb@@QVfe22V;bsINq6)V&BeII`Z{}N&(pcXZ9rk8y z6&t8bay$8Q7>5!s8zd4_CJbu z5!OF_3^6WR{ldA^m(jvvT@L3wtHq{{stWv&7VtR5Fk{F&s}ldfAzXmGS~@uUfJT== z1n*OwQo$?3d!gU*3 z4}I|=KqQiFLK|KynVT7|nzbuuMWQxzzTg1PlZf3~brbV5mz9Yxh2|MZ$-$ks!HTKA zrNV+|X0w0Jb1!d)WiomE?#QJw5b{{*{b0lPfp3tCenaeAz>LXDH$gcD!-binVZG&L z19yJc%B)gNqUGwqV%ji6>D2SqE{7u8&3c2Y_B*MKZ^tVf-8CUrkQx|nyxF@g3{Nh{ zi13sQwQ;cZ${0&^dnh74vU+V0F0ezEVR{_bvNojt&kJCJ&iPXz^0%R}{m!s1?4-kW z>H8=J?$K0dtK5$J_BCc9?VkW&uh+f^d@YYf=7dr&_J&qoc>u2Y`bEf{I2oFvNHhCE zvObJZ!1R}NG&L_Mb{ENOuwVONi8Bh4Mxr5tXC*i1F;P!nxtb;XG9p`0(&~H=0OD@j zPCkbdLf<>pSYw!Z%5`N}-HcR1c69nWABxYMktXV4^1pdMiXx2%o)|?^{&EbC+fTo^ zGZ*PUeyfVz&GYx3t?`@NW+M}euU2?mkhwH=Nuo_B32_dM?fZo@V69BI74o@i(>fM+ zjmFsTj&=dK9gCVgUgO%lZks;5yit9vyjp}#ilvj~>){m|5|=}d!rKq*93sAcA_=Fb z_F%5|O#MPHIa#cS{c`bg(Q6!Sq8Z7Pv)N96696g$AhEvroTa<;SGwPT32%kvHn!`p z(UG38H(AMFr?zaBd@`%7(9H256vMTsSJLtCtE{{fKx@s|-ST#K%|_Ap^!;2kBZ{6H1=CYIivR_ht0N(!`VOEvywywT2?Nwr&V z`4Gfzv8%}9h!n85R7&nKS+GVjKt7Nd;D4%I^CiN|WQxjexv@Z_jDm#6HjM23S8=tS z>5=}E7O9nT_M%dOQWZ2+7F`kv|A4d8F`Bf{@2&G*_x9V7!Yuce$|WS-u7|s_^!e>a zuP@27nW7iJV_tc^{>Ywph1M_XnDYTrP+q2T&TC1P(5+TzFrLK0s$X~g*FBZna56vB zV@mh0(`zQx0^d{y`7Etq4BoHXVOu)4D3@AsLb3iw|Hw>MTde0sy*H(;tDrj-(eE}Q zg-H}))fSZ-^o1VrELDD;AGXsvq;cgt8>>rqWGpq%Nc~optZ8|=Lp>mT)4tgC!{KaS z&2r+kN?ul5Q5nfKd#=6Jnn#X!!CXzzYlHs5jK;RM8;^a}-2TRAM7b~$;pHNau}1fb z5bxoUbZuO1`ScILqURH`X5qvE^KU*Aaky;tVmFc!Y$Hw5RHZ|)+clnjgFcq;hhCh$w}t69E6?R1jkBTmn`y=Cn*8hb;GQMsb1@FK zpA>MKWjpQ|?Oh#3Ov9_m$8*xzo}E&-Vme{wIIB>Z%AVZ&->@+G1u5~sCAr0la*=5x zG)@#?OV(faT(9_0F3?>ZO#a42aM@cY9;v*KUdXIHUY0l*%RR$g;{PPOVm5SYo6aY# zTS*&59}aExxyhrZzU+^+sQmc166RWR5W#}yb2%&^v+0d`WwFxSRQ~}Rd$z(5R?_nv zc^F$-Qn~r*&T7=Fr|7hTMQA596LBzO?Y1c$VeQd&=WQ5>R-h%c@g*f#e=59^qbTUP z&s@1k_sokrf*zEw8bMjtaWiMPITETpvvC)$xKHfcp`|M@KGunQqrtlWej2&4Gx%L` z>Wh)$Ov*3G-|;l`_MG1hegC^K2g5L}Wk7A*D%xCe`Uk-MN4ps%Z4t@ehqq9K+$LKp zN)1|-O1+pcGQwako0fn@_g4UJwHPda+I<(wq^Ka@r)dp8k~Hmn+icnCd;%^ZW_eM> z$zeiGMAF3S^^bC5d~Ui#1s$);0}jnw1F&f%dB3LO7qp>Nu3NCnnC4gPaQzuNbOe^! z1F1YEKwgVh9RAd$G9JM&>FU3Jx=XNQ(|%694r`m#T5egGI>}kk$Otxu1F9+EqyfS% z6+2=ZL#W`%-GXhl&~L0sQTXF>fBqZPIV2}Yyu8`PDdP024<8P{jOpKEAC`?{&{zX*PtyC#cAR z%OD?@k+nPGR*_x8#)iDdCS@G3Q;l9B0qyT({cwzmH7csk^^@k0TfZwmNL9EVIeq*; z8yhjawI5$Ep3o~LIg9nO_n#JH=zh-!shs*E$g-HqjJr?~SE^L^5vch$_mguQpyy?% z(qV)?&L`upd=uVEN5k@>r^n@!N!pkfl1s(`_JGs(nmb{n!6`$j+O-;8l$q>(@@Q_&5!G-Ea`%zYgRLNi`sw`6lH5M%ceC=7#+Sg zgWkK@$@FY5b-9*R#8A>tKygsIYjDnld zHlB723CH0i*$OAiNa@_@j)`v^5A|VD+UM)98-Il0BzxO!yJ5+|-q{w*#L_EiEFSzF z*zB9Flk!$BvtA5&^~$0>Cj7kx)|8;f0YP4gCbTRH5A(@nVUW^E%Zb0!=rZ)Gp&v}% zJ+W7z+}v0b44hVS9+9`V;Al=Un5`S%c@=2ehPj18h9Ns!a)eB5$ws<^xQ(5^WZu60Q^+ z&}Fh<%nV4s$YxsA>*iC9M%3Rk9g@k5gABOy61J?Je9iYz)U;4a|oD$FzP zwf=8LKXG1E@r%b zDWep+KYVMM3M}9o6AHo-IpXweIgTTKn3X1N{%kFs!G?xk-Dhq z)Jzmq=>>MV9m}_x!7rvg8li>xSvH!Y?@xsHl^*Ib?$VRp$2(la77C5enoz^kL(wHX zPgN0{jxjisZCPp$9u*hw2X@qo0miF_anOQpu#fn<)lZ!HYj9cYR08q8)JiiWdBegi zyv1piJhuyJW{1sBI^~Y`c=rx?HOtF)GS-NI55U|J*Kz3bb_%B z2XK*6atdTQtWPRbG(AN{8{4-AZ{8&i#I=8Kdo;SgtoHg0or!31ea#lH=E-HT} zlgM(ZF25z^Dc<^{yurFWq2Im4zKbuxd+*iGi1ixqiy}*SItIpxmPFSoHqt zM>&Sqc(B)z)2II%?A|ZA(Ox`!7C*WSewnX0u~P*;cgaheZ1mx=D&_9)yIh>YyUkiL z8C-hrDszL>*0Yl0%o21MKJYPy=q6A}3A@Iq7NCf_Qu^q(L?{dS3*Eo7izr1Wg>k7~ z&t^Hi{CbW}cf_e*Z}a)~@p_lpkG=di9-AH+Xr_`(<}}r5l@A&2Fdsqiib#xyPUp*Z ztoc&7Wbnh@b=S9WdO@FK9lNc zWfkNHL0v3u2;M_0(+IH;uj-0j{7fio`6>%S;~;HYdAM7B z7&6&#?#2pHqTz!vWlI$^py+Xh;!4PUJzhhvZT)OR5b6yG$VWGu2toSs22%w|g?qE! z+oldZ$cyAOt*w23Sn}8Ac?$&tC z9p_ZW1Ti1tG!W2}sN4D6WqkT}#9(hUyZ7Uh@hCy|1c+zmJROv6%>Gu<>u|Q6LV?|% zy7=o;EbM}|1Q2O6s5owkAULl1y0t7w*5Hn-5AH5*>PbN+p605~x|+RYXC+Gy=gvxy z)6B+x$`BPoIm@N8L;pi^X|t4M(;M{)zrrpWuM~w1blLUIXXGBb`sCe`uW1OWHfgfC z-6C|_9Tk1g)s|#-wQt(0CV;~86og7BB2!vfNhLt0#%cOh7w7n0oIN)P7){jk6sw`8 z1J|VEdqC;7|(;{U8s{}z`({L{5eR13E zGx#|2-VO2aI^M8TgB9oX*KP6OG@cujY-Qf1cu~ z!SApKd@rDGM~IRnmTMfjfpCKs;P)S|h@~EMM7&V$liw&JPRz5icRIa0`0*d+T*((S z5SgDWH+8R6w)>J@LNY`;-rt;g-jX+hpj)D}(@+2vNeDw>=8{QO4eFjf4AlSF7W|gZ z@PMkZ|KC7_g$2LMuZLeAfG~QptT}K9H!0fGM}=zoh4vkS2TtD|Coe+kySZO5q?*hd zh1ug$)(R``Q?G@Ix=_^4_fxEUH3TYF{v^#==Ly0p$)&&RXEQ(?iuS7I0j6+F$F#fU z*PvTOU%n^tpE;s@9$%k|LH$WhcW0+m zl)Slc_e)O)^9$|L%Apy8F$!Z>f|h28NVT>SCb%aBi&U|ADki7f;_aHJ6Sv-Iu737ymQ z_)PPA#|O$@^m5Q=NFyl076(QPF*vt@Rtb*grgh#-YunvEu1X>^RjNqOwL0beX1eW= z?;JxNd*Ni8Mvx0ZeqW<%$C<;f+7P4Mq2G<)8T z{;|*q#D1C=B?@9l;il_sM8B`Pe(K8#?} z|4Igu7^yxQg~}0W)NCkN5#aKq7rdeOc`q|>w80BSE1p6h08u;_gbiZXM0{87FWtK> zazrHeA^ygc$8co$L7Xc1-pRwb*(v;v3UWgYL+>eIct2TTTp-Ud#Lsd>!reK#J(T-7 zIh>8YdjRKZQSkBVVx(X!%Ws?{^cqA~UI7D&k~3{q{7f)L7vQ#Iuf#&SSAv6SjvPno zalF+k3cj1mvb>>{VaflgDI4)CcD2f+&;I~83+wDCsh^%mTl5>h=e9gYWP-o))brzr zvdx`skRH7Q3j8Yz5nwG-D%RwwKmmc@HETxcbBbl9jH5Deq9> zV50HQAHmuZO)G;lW9piT$fnnsy40Y3r(Gr-zPr>=TW#lm zRZPx<5%Tmnvun37U1GuW^=I|56s1?-9bEWY&$j0lc2E0!3s?u465jkMnkj?)`;=wk zk9O)ZqWcFe8$~)GSsGkxWN#ipM@R1hCD%XIYJ-BZ4N7&Q@31#lhj>;xv&HraUAr+! z0_nl7B)&K9Uh?ivC=RM41c&YeXd(@J>_8*nXf&2;)NlIBER%`?yHXB`gJ4o&RiacX zrx_WpikMJCZfbgmZl+7LyIu1jc)UXitUeNf)Oz(sI)3g5E!#S?HsS`SUYM<5d z=77Wx0>dN|JKTFqonI*3VPqlsnO^F)AmA$jLtE#(kEgdc2HMGWW3MExmpF7AsQZ3> z%$TWvO!1Z)El{G-Y4T8;@40GFV=d<_-n zb8E^1z`O<})2yGp&%^erXk$mmTR2$cFfPM(xJ_C&B>lzv^>p*ht^miKsJk(m2lp)$ zCg-EVG@2zb3v}FF{z~cN8N7vrW|Ft``+A)tGfQX5sDz^ylZ-WG-ZyP--`0-m9JkR> zYOG(^v`pNe7#oy)F10E&x25Q(g1snMSA!=~WqR!^SU~X{*8ao>`LnY|B!CxLV>p88 zB79K3Ly%mep82T&{n@(6!>qzzrPtSIl4qeTU7%q=D024p@Ktw(H7iO~h>c>fmcXvn z%eTW{**A;q(mN1kSkwqBj^^(gS>tZ=1BM<5=PJxf68`K!&I*{UMt%A&+~2dD4DjmV zzwkf*al;V9Bh89Rz#nDpa50VjYqlC3KQVCflLZezQ`|4sy&Ls~d z*l=Mu=G8TLr#D!uSaUju{vW|@eYB~Gy?w(solb7-rVUKx5k}{H)OCgzDO=I_ zkMQp_MTHPBNVniKi6}YA*zn%0-v^TYQT1c4H1C6}T+xsQI|L=0Msa9Za--wNhcOv$ z^O$f%mz#rok=n$rP!#rcb~qeyXHeklo-q{EEclmdq^ce_P~Ia3S;21!a&34S*cZW| zSEDR(dGPD$v2g>#I7_rM=tmNlDS|iuo>x@()M*FeKZ&_S*?YA8-niGjvmcR4fBpa? z)u0qGIJ-z=D4ic(6b^QIpcf}bfRjNu-VbL|m`XrLwY&2#Dx;XKw!{Tz4h2U;3gl7J zH)ZWHeNdo3pn%*jH2m6qF5xxlxJ89#BII1eXFr{bOxWU~JcV6iXEgOaVTdz-nkL+sA~Y+~Z57?&9>nUd zhm2MJk;g?GvL=Y#Esh~ec2SLNlR2dA4|#Hoyx*vIw3y6~%sqXfN->_#QDYQ2FDs0d zY)oAWaE!EIXb9iNvCAXxNJWrE&rehsOQDf)^8?1@7K6JX`$%;YOxYSJI5?{kmtbG#Dy16ZWng`Zc?Z!e@XxZhOn3NPemLNCY`BYPcdRzrwpC_A{#d@O_z|~T^(6fXXA2OkmEjx zuv{Rbf2fI}quCK%LB^#LRktW%MM4!z&Y~J|XPzc$3)c7gr>4@N&-59Od5>1;58Js% zeQe7$h-I45D*}UfRj$Es8e;|<>OJ$bZedf`oU{3~S(Y=Wc^L_oVk>9_6hk|BPHC^_FcB^sPD;Pj2M?HdF zdXpuM(0P2jvjxE-_4~22?R94mx$YjAq1N8sF3?6Y8EmZp5&(iyk=-e}4RSLHc$EwS z!k*POsWoN(0qCEQGvr6GC12Uku61ah|39YQGOP+N+QOwa-5t_hlG4&4Al=>F(p}OL zB7%~ekdp2WDd}!0Q5xy4yZFvI_ufA~{zBd0T62v##ykF|U0a@6(~m@&$C&P#i)7uL z$Z2Zz-l~W_ythHl?$1V&5i^hwl0MvhnnA)Edv}}3+Im$RG%9*@d}tdC{UNr!^pM$e zRDHjVO1g|mrS?n5&hY9X`i2KrnF&6JQ^{8sxiWXwh99R&?xA{|^?v&mtI}aW|Hih$ zgBw&DV~=x%`&eJr6rZH|x8~3nIcQ5(2M4QZdelLQ!~Jr^EwSD~Q?60@ z@I|Bl6h>sii?|%DQmkFIMSiatPW<2v-y8|@nWHYz6}6JQ+m(36@^}ESz&3dH)GYi) zaRLQE&$MGWJk0-9@1asPKPRI2`NNJf3^mM@Lu2rh@U34XnEepR+{|l zJ^yv?qaV7X+J!m=_&*ZoUn)gWc!eCN0{b)J0P%fb>uWZO&Xi^B7asYv%;uR;*+7wZZ zS}d{B;>iV2opgE9!CVyATrR7Y{n@HRB~2Zvq^q+2eNx__8T9@ z60)PhWh%s#spzw&6f4c+2k|P3q{T}OHV>z=YmCQG%6(wGss}?NF(bzJXi~8d)KB*s$KeiP%0AUAKeNN zsPZKvP*bI5Um6gOmg>m$^~ZmtlR_N7b;bw^z9%lfyWoL9*vLg+5E%P(N3XajXZM0C zD_U}40VZSv1|>=kN;wG(Fjfd^~W2J%EpuaB^9QD+t)5rg?p|1(kQkF zrMNq}?&caBFRS^k>nj<*>czv=H+S0N=lQ+uvY>;<+LS&{N^vUZ{`3EN0TN-r()I-I zrAn62EDUV4m3fh}t$Jt2en%=F`)$xY7qTcDYX;-nt~1}$pI1Z@F8Y-S_X4i#A>aCt zI`1$^-ff+yH|8FWX83%U!@8HgYYWHCe8s9>@Rm8^Q}hb{SQc4^&8Q^JKY#06=SIyE z^(McI()r@%8k>Rs$^4Hm1s1-M5q$FB24+C;Y0W1BGbutek0r!)Y4R<)^hN50JcgVm3H&r^t|(`bj;Oi?Mz0c__z?l8fSnNT!)zxz*ona^+fuX zA0A;+%En>e4Br)jFfQ)pTPn#A>Ej4EWjsOXf!^T!wqZ3#YmSOEF(tf1(nqh`_05oa zf(<|GDoh^T=mV0BgXGuqy09cNfzsE>d$m~I?+)x8u6PoV8QdIJ>bzY7g3pU}M)r@) z$erL@YP6}0S+JM?e}lN!f>4BA)uO(HQ1X8SX*H-a>*uoDT6@rYllbi5c-^vR3;@QV zP317~E2U8K^HzA4jt(r+#@U#fRUL3LQz_1kf1$@gCdAf|sSE8dLI~1^b4pa;SPa-D zUvMUm3N=LyK%pz^AyN$=aREzGiT<(}$M81?*J)AdAE?9|<<)>UeMy>;t_QP>R%jq? z)L`b#hYRuk&*V?BA;k808hp`7m{aCwt6WppV6|%CUkjk~A3p==n)AJI5)`s{OGHf` z`d&qMT-QhnK?l3a-1*vLzzBE-3C`Q&IikuQ-~G|4+n*sI;jy|T=5JZYXVFE%_!nma zha;>}G%g(~Iqq*K&i4^>QxYz%^Up%FTV1FaG(oKgG0$TV*Ue*V$+0}78BXjW0J}%3 zUof>qJAQA60!f10OvC2)_sUT$n2#}Bb|Y`_nGGj6RORLqoYaa6Z@NVj$aBfSU6%X& zA!22f7U5qJxTb#)vpmK|r{Jagry|nDQ>;})*zqv;7#__*Sq$&^uFgZ}xhP|`eL3Gn zsM-UdnPEv1jnSQFOFz4%=Zet-%u&DMHaIw>z|JjTFV?uF@RLm6jk?YM&gg-AX}ESw zyK>sds@%X)W7P=|XA;1s6-b)qBFzz!NGY}|Fv!ag_rL4fDJj3Qg@TYs55UJ05j3!@ zN~parX;T^Fef>Yc@Cj5pRmi*TvBOvH!w3wb)I-W}tqKOIA$`hO)~e{GF7MRaC$qkc z4D_qQ@*ao_`3T=U?rw{qcPrC;ca8P6RIeOzZarbV9A46+r%BHmbMg{u8hQDz;9zjG z-)!i2pI7-+v7G!6Xx@`WRJbI1Say^5eNl@`f+4f{Zv{N+E)OT$ z1eXF&WPAdDi8ws|O0<0Q6UX ziB3)3@v@8B2i_{!UFa#heq9$}ojP5Fz{$X)z|a(kg4J=wZzI2qtq22BP4tk0 z&8_U$Ow-Lg6)R0<%oYGjhrWyx(^;wTc_>6irlfM}TMq~Y9{)h&c+Vf&YNOX~fh0;U zti1#}$i`cH2COC^->ZKb5FT|J&Ic;OT9NenI}1ZGtmU-l4jx12MH9oHQQJ?%0I=Kek`a-hFW%E=utLrBf@hsen>>#z3I>om6Q&a|JS-7coKo= z>GSH?le%2|UK4~ti@ncX^k^37(Ebzz4@CYMb$Rwul8I~^ks(ig{3CS}%41D93ME|) z5u;!?2`lQm7>ru=<+4`D?(6LE6$Dv@cv zmaq#gC;~|vtz2j?)HEoVp0O(-A?I9fGd*wxhURhNCz0xVlIXBo*>SS-fwh>`+FI{J zaF=`eAN@m^qlSNVO@>W79ep!ObQn0PImb}5op|!6z^K$qTF0QbTD^Mk4966un$nM> zrP;>!4SYEFHGG|T%lEp(@+8c{6yyq5)bAj=OC4@4O#qd8L89G-IkZ-6Og;_Ux2iQM zq{5}8j0>w!jA1GDJ|!t@XR2F1V^iH2{M`fn0wQtyRudFgW&v`0L3@2-BY$UcZB>+T z)wY@Y+eH1R+IFu7GJ*&a#^5t@?Fue2sI+#wm^e9m`YXvptcXBywFMm~eO7FpXMau8 zp5V$qpK_qiTo1H^Pq)INME%<@emA&&*_7(o(ND@VSZHWc|APQ`X6*aW_8f!<>Wj$G z88BX=?bgtEGVd}wRZMd?ldwRBLpnWDj>pty8sAK7pkpHEl6;;@@SJk}-Dnphj ziP2z&lH~1QTMu8kdAyA0C{jJ6nQe|rQ1Z)q&)+y;o?MvmSt0CXrTQVXBP0nsDfAzx z@V|lr2K-~xnQj;|w(HT|nE4qsf~p_Nrm4gq*ii?N90o&Rv{lipRUrGO5>MK&{w!5t zT7ut0pdz4wt{tAi|9f94}l{6Fc&!Zvl^v-~+6aoZPY5?=gW^{}@RPTCK ztn~CkIFg3lK!ISP&J_0Dgx%o_k?x(SUM&2wUpecre0Wac&?}T7I{5De3K4Df_Mu&c z0l?yr=IPqFBOF&<7yJ|yaJ>=5=Z}{iA3}ZZOsz&UA1ON>y*;ho1Ak=V*j`a3 z^>TD~|9sQU%Z8kqQJJHh{b4qPuX%AWn;wlXkp%B#@$Yty+~nTuXU_*wPZnc^48C{m zoYURkx=W(4PJ;4f)ib`W`6AUCjf-1D6X4f^;&?HsPy$>9OimJ)8*+PeOnmFUfnmm! z@nw3blk&l8yd{YdBiUeBg3NGku5PU*%F;X1IK9XQ=XhCwj&~Re`yNO53WkiO3&kbh z=j8Vn9vjT;Zx>2$RGfcxksY;d!rD@+o}^)r62pYYVNpPJH~T59rh)7C5RZY%?)H3R zGX_aZ{`Ll`8$rmc?nM!sR6OY`&Y*jr@%!9d+u0I4KA#hcMd!ace0E{uC_%Iz*1$Uq zhg`_xm{X(P!!BT@qrJ!%B47%Bpm%n}<2FnJLJ>zZT9>gf`X)AdxGn}Lzpa$2)zI}P zz97N$u7<^2q9lza1dP#8wgAXO`YS0##JmPrS)cPAoE_4a#={4cJ z%s%Q&_(qC4%2~0Njw@P*HVYR~UFw#@%0sxs(Mkr5H1fEg@T8=qCb#1_IK6uiu|7%p zpNr>v9g5tuXIZp*<<8HPd~w0Mm6UyU0{j{II=vaRrv@}6)f&s0+gMW$bT8UpPcIi* ziC#wytFR=8IUI_h6~EX*GXz1&H95{VSa+-a_pYskF;_V?ch0QHNZ&Jp5r5Cyl!hD0 zId*-f`Pccg;=ng*)&?T-+`578y-h#6CvasAwqNGzR>ev&2acKaSUrJ z)L?Y|8Wn%T?U%Jk8UVFDClRcBDnlC_LNg+cKO^h6bqyR(@j@76wq8wwBP;Wo*o-#)FmmStrdp{f%1tCoq zFHv3QT&$$kvy}!A5FG@m)&@!QpuEAsPog*{YvzYSxGf`wI3YbKLU zKD)1=QkI%YgQ_F@l3ly<;@7t6fN5?2g<54p?L&;$!JdeI=yf4Dv3vOw_a1xEAjI*L z&NjE_|H4<>Zx=?hO;5r7*Yth(Gi#L659XwmqH{rSqxhpV-Lj=;OOhtC0Y zXsU+m0uWRj?@>@uxs_qGBRK#j1rO9;zPQBU`A1UyJ;X8lE zd$6k@`{wzzsT8JLP+3&KMqDT?Be8dPh4Guw?zs`xAHV(jrX{5nXAXR9&=}z+6+%?A zO9OuRo{L4XAHY)e?rR2e%!>$k!9#+7^pJ!;_6Da`xfTWWN+N+ z!7O@nz?103X43H@0+k>ZU7!AG`b*M>a1hugbLl)hD2PTSU|G*_>EJr*q_eGa?TKt+ z=ps`ucoS+d@P`cl1FGga(WjdR_gR)1yzsD4p8&nkVSM^Itl-tbL4pt z?Vk3s*$z0_wLaL-q7G7uet2(kEQ)>0=d&RhP-_{J?*@n~ENv(KWZ4}Ithn(^rg68- ziev-vQWYa#dpc`Si>)NSwdEQ#x;kwuPuTq{;xJUb?D4pNADL%xw0hm$A7c%SpHlLE zIekcn?(SO|5HwDo9|(zF_tj2hxJGTZ0YUU^=q~ln<&OI~V)yGM5sPgefmhB03CKA8 z@LIUU#Kb4gW9Ty{jF5?1K?g{@X*tn0V5Zs5SCDHzJVk17UF=RX(kui6zXg?R6dOAQ zFS5&ICbS_bu+}s4{^{<>r&EzgL*NHgG;pHMwEO3tAs;C6JE#_ocA^m&-U2>-@ld#T zIfqLTTRo9xpN@_B%ts<3e_vl77SEJt$lbJ94VijW8pEJ9@SX6qKb0HhBaTJ97f7$H zo107@D0)`7){*q=)pOyp6+a>J1i%=2IZwi6-CoAO$98*2e>fWlw!<8`+2;p?(Hw(* ztxs_*CnDwx0hq8P0eP9TKU`gJa6-s+GCCGp2*Ry6@nHJS{`Q2y!v9##3-dg_QUF_& zQ$RTLIM4<2%s`#*#ZYQr5=V!4vW{`}LF8f^`AiMH_R&IUmRzF#lA6_*$D3`R94c~= zfaqZnV9c4g^ArUWA|~1Pn9S00V~^O@ybFrz`u2qC{Za6f`%FKq)AH}&*ZxDpO?ye* zAB(#x!L7Fcy;63J9gzk5NM1R!E3+?da(?eo(pCo8btYm$bnhRGvPG_~j*RKq0w$g+ zoe-GSA)L!N1IGmMfqV=koIn1CmweYY+nLB!*9gWvi0@3$52YL-?M?f3Gnfs(@Js6h zJ=Uih8;se9lCswU!V7_nWN|eYGcOCmXX$IlJ8Jz9HOjZSt1?%%2?)d|ey_fk6qhfa0e@5 z;kVTX>=94yo)Pq6e=57U_le*Y+aSP|~&t z69MPw9MtH;`Y`KjRPc~<;D3y*+wH-MtSTK>S zksa{L@msJG$GnU{9qI0v55au$TOd-EJNlQCYw2clS8%`V=cL4n?TMNxc!wn9wg0y% z?DrAA%-VRXz)PZONW*;tyanUieyODl8=iKZQo5zUDY(H;^k;yBU8nYXrcz_tx?Cs& z&|Usxa?M}0Y**_7Z{=~KJJ_un4uIIIZzic6nWoLimyP-fGcz+Lla4m#0t*nF=$Bnc zGxn3$9|u3DNz57yyWR--&+yPBAy50trQ$G$GNaZxP-~ipw+5{AEThH^j(VH;!g@7f z9&HzM2E>Bz=vD)-jOdgPZqzd!aSJQ}3PJr2Krp3LW+Tm7vueMNg9#t>O==isv5&Ee zyCxiZUy(ilQ>MVt7}IWuz)RjTS_R8FS)kheSI8s@D+j{RA*`YkLoWp+XGOoVKk^8N z8vK?YRi-8{b1H@5c7b2`S+P=+C#oEFCLJ8}M0CN0_U{t{jwmX!_?Y?zBgIYn4!MK{ z>S@8cGU3e653v#n$wSDgTs~FXCE8%O3fnHlnLK!1V0s2%03oNsO5jZ5g((=KLkTJq zHHE=cfW83f%hYiG2&hAe>^;g>#p75u7yJX=kvEfWr*XTV?POj=&EmfGxNNrNAPsz0 z`IeZTkQ-d3X4#He?V%iHE@S9hyI-|1#%`Fc2f>|ocz$oQ4`wi!^y`>{sE!dAaa1hR zMw-FC1F(WbljhEpAMbS+8eOif={e?V_JQYpp@}Qu>87ZL5?}QZSfxWBFWlQQowHFz*{6EWvG zet%meT%@vDZa?d~z52Su$gfg%5PwDj8$Jlb)LpMWTXAlmJdGni=n_+J&F8^l%YD$VmFvrALej79wE_Kn`~`unEft?IEw_?ursBqu^P4( zeB6d0<gN zb+n@dw}leI{$t>*{gPC>)*qJVCvJ$8&3N8pS7?*_RmI-`yn*S4%1&9*mdKOC&8;_Q z&MQ?hF!++U7Lzn|dHbmJvlpDa_^cgI6y)LIpASg|U8(%euS9Mb6;-htK9CWihbDbzDUWq>i039cP2sjK&09hwSC@j zp0hyF5P95D3l~(S(0iwG*&Ofh-RL!xU4L*L$Nyu^1qnjJ+29sO==<{o43pt=f{DFL z>vhJ8-JXcg7laUgw2CXHF^5Y~I3Qy05W78=(-u;b16%derKZ7|*7G@~C1GR?urhCc zq7sisZ}v0OM<3mJw4pc?h5z+c4ZAkd3_+K+mF!~}XMNa4kT$cSC`Jyrm{c(?tPJ_YAGRXV#53mW?I{ z>slm#Xe%UybaTL@@Ri~*#6$I$bh!}a#_HzrS`4x#S3uIX%WJQPjq!jy%*VRX4v?+9f30i_Svh&$AK0JueSFxi@cC=K|w-p8K?e z4Zlf{ZwO756nfZ`=Inu`kW8|Yqu1x6_0(qfZG~EJ_Twmmls@?CN~1WuQUsh55a#G2 zz)knc_>-Eb>=f{DG5>mKYB=RFS*fB%VaG}XG%UWRVr1iR@*B_Kuar|mz2#E&(V|05 z{tD7R$!eS*7Qt2(pv)?c(ri4c7-4E6>(2&Kww_QnE#(X-{Zdauk8R-qxbdY#3D zcmi=?QVAJ;POvn7yiuKa0{t+y6Hr*dV?ER|0TGZ0;BS_Jlkj29YrxmVfNsx}Os#wF zwo@GmpF$R)RFVn-6Z}W56RN+2y)h|0juPCHq05Cz8F?;i9f+2nS1Et0x6G7OpSB)h zOS>Wlq3a?f4$a}CwyoCraAh|K3fDX&ds<#zpX~5kf6xhnO6G$FpAj78ad$-ln%&Q^ zC8NWlDe{7?QfV=v`n-mYJiIuX5Owo%!5`lr`5W#xg`D znwfKCGl~jF@0iwwTr7&ZZX)PN{MZsm_F42oL@Q>U2|<1XFTN7kMg8l%P2&h{`@4o- z25WFK#8;9FdogVggQ;CA{Pge1qyG^#Nhqf#Gw+NKk+5GkT;hwnLKV4|G!w+$bw5Np zWDJ>~Pf{wPj?;+q_?)r&_4Xv8Ff;s%FZPz}nDwjo@#~?sY6rjSP%dmA)ad+B_CyFN!}tCViZ-xC>xn%^ zKg9Ze1dZsL0cT1xtOkTKXImIidb+<%2F1x|7RWUUsqkhwetg$eabEec`gd6EyNk)N zVCj(R)x#B6sTQlqCqZ`~;l-uR7CGmMmRAjE>y?IpJ-+$#V>ZLhD>h#I4DAZ(Gqau) zCo%xP5xw1^t#e+{zisV}HS1C3v{B}NU<>>^l|I)6WLfjM0{MmDy_7nPI%n-N-N}$Z z94c0v6^r8#B<{3d8LMjlf-zb_=@z#P33HqNIdVrcW{p+ z&LvIyt`ZTmR{*7K?9_M}HTn0&P%heY4(m~ys^f>CPgdIv2C!G&zn;pQTdmFSr#dN~ zOkm-Hcc5G9AeR>{b6U=h^Z%~f&@zDX##a`XJ#F(s;88Lrnq~}va;Lm|GWUIh^%Yr zfhyyS=gV*pO2Ot1zjMiHy27LURqy(a`naTh^WdcOrAw zsI5Heyg+($17p6kz5d(qSJrj3;(X_FQrp#DcaE4jLso1`{A0oywE|UG>|O>Yjcen) zrj)`!VX|SJit5=vFBWUfeth}ajsj%0p>`cqBVoUP7&WulS@Nt&AIZ^49T}Lf8}$l7Jlorbc5u zUdDd95Cfctg(ETzJAX!b131GztIc2GfA|O<4!`n7!wvJ4zI$-zXYsmky|s{PVG9Ir zV<<+kx!?KDM>qX>pg9%%1-}o)Nyby|8<{yO)2mJAb6@WY-Gf0DCa?QC@c(-@ut2HF z-R3;G=3FuMqq%lH`~Fu6f)PjEyZ(*VrjaXYd0eIO%`(*f`H`SZ5dEW6DWFQ|?_ZT0 zFWhSxD;Wc}kfgwhnj#^ROqbS`hG-$=&@5+(hm(2u zjEr9GeDluI@(7(y;wqAoULJzLl)^z7q}%aAek|J)=~dJ1-Ev>t4qrgiF3RaA2Bjah zfWo>oFXDT;4gy&+Aag)tV8e~oVN{D~f-146nH1K95{>;4bWDrFXmE=jJ)L2tW01!J z-9X_2+|^`Yp&J5nVR=HK?D))JTp^Z~xSQb+W?W*O!{>fl*_mXz8)BVW-v`A)VNe{f z5max8;AJRglaSTyUe#<021XVMS6w$}`K|pFW)02}I;wQuJ>4SA?ACWO#)k)KOthtex5fzs~fzza}y6?<_&Xn$`ZuWQ!j9Bq}96vcB4cclfYRi1JpW- zK}Z7D{xp&X{<;y%Te7bpdBuE%Ber_>r#rpeU^0~oKel%ra(>aLQKWhR>^FjTXOyt; zpacmYrO-6=C%R@&z$k=GY%~Rh)$2$W6q$W?`Z805Rf_DO+o&SehXptBg}=)6w=_N) za#BLn1sWlAzRFCUDY^`YIhnp^BPU4G98N+Nyshr84!WZPZ<&@eS>3J6Xa&F|v*GLW z1{Hp$N#UsN9pqxE{;cX^HUzf#EBH2b`VmmdQ;|yP*Gp~+dH#WhPpD?D)T8AnWq$}~ zRAE_e78D>}v|G6j-jV*M@Yyvp3bk3`aHW-Kce*ICO2R#Z?>5$MzRnIATHgZ^`n}E9 zzWJOB0B642-ys472!t9q6@Gf0G2pH><2euYdwTsyq9$hd#cDSjA#T&IHn*H;h55F4 z-h;KnXPY#&UFSU6`VNj8?`J-Mic&n#<~Un2`nB|5Hs=KQ`)3K2T&%PS-aoV)kebo% zK<8!V-Gg_gh#TD1RH4k= zuV&gndTo|)HrK0?bAu4%Ei;Cl#t6sp*V~wP*ximEVob2k&E=um?kgsoSV1dRSybhN zPmWm{D;zQ75SsPgtw8Pf_xFC%XAwxw#p1MWN4pz8%RaN|D2v8g#~m zr7Fw(EX#{{-8)}$J2q6f!nZs)aCDP88tl@|?KP17seVK`__ser7~8RUbp3E=RO8)c zxj4aDhsg(DG$Oe-q|57g+8u=Tq+G?p2%2H-8M(en<66pitm6*j;S$#^nM(;{_aa^4 zd*7boVnPK@JF|791oNf^D_R0zJvHy3G)oQYlvFtIE0bFGko&@rMrgu_L@4n@t6V06 zr(ydo)@^_wK9MAc{@y~T4u6}*`Dk&FMjV>BEgg%PDC~qZc z1eXZr)3su_&=1c7r5pU6t|E~B{9V&Gn8VHStI4fL07nDCw5koL=B;j1=~{V8wRY2m zT1#es+XLOC|7P&og*7Ly!wX69GiC|dBtPDQHU;p@)a73gFZI7X;NmlliwUDd{!Oju zflHBPYl}dkTLAd@q(DC4AC;l{1pEhNLb6t;)}N9%g5ricY?t+6gPFix@uhv1xiJtqTpF?AUMUuVJgJ#5!+b>hQXBT- zSgXEN3yt{`UAJzWeE%70d7i9Hg3tF`K3ZGP9>jrRC(W#YI{1$$;a-v1~S z<;cE>_e?so9{mM>U}uQt{_1?*M6C?~Z$_iVH7LY?Y_Vc7yc^Y4NyGwFO;3^fK>WTjm}8rnNI%YAK4wYFA^hVH5@DU57=yfCk3!F1~UaCv=Xv2-pGDRnwpZkd3ehK-_ z(f+4{a476gc$h1XmBRBnhN6K~r#OTfEQ|4gfr&Af@z4xAIQD|+GRlawp)Jj6a{ z+qyzv3s=b2Y8mZ0*AO9Qm*ZlJ-N>)~FFnGWi{}zdh}b3PY_d_V2q>eeH% z{t=`srn+?CoDZ}4Fk%+(oN$N>QCn>GiNQ|apYD|CMky5;SK8sv*|-=MoEG*-1QjO& zM8_!SOV-VFQNHxvj8CSB;1VJdnL`^34*Efca#qqt9}z7z3Lw3C{KPIxg$V|9p6H5; z{Z=Vvn9R;VG9$n#rkIlW=(B@nMNK8pt=a2f6Z>bd+UCfZP9^(SRl&Jdi^oE{KR@pD zMIzGMs++$4)Y;di0*HEd&Gq%bfrdyWle)8#b~4f}#j1IAL*uIGSy_&&cJX3;CZQB? z3%{>*UfZA*xngKX!C<@#=ToEj?wPy@&?NWw_=j=nHKNfSzXN*%hD>a%IyRQO`ZGwN zNsVTwjztKbb+(~n4|lDVIbI=DLyIMy&&K{deORw0L5NzZvZ#$XX3OJ%rvOrA5r1bl zpw4S1zIj^ohMR@zmv@oM+t117IW0TAnixuDrxntudyWT8zHF{geHOK-P>ZuE^{!rr zB)X{*wRRtR0Z|LZoJMzT2*QFLG5yX(>kS;MK)8l3Qhl!5XoqZ7K*D{@N_t(;7<}WC zj^k;Y)~EJI<1PNxFC0nOQ}5~h3S*R$-p_~agf$SX!@ANKMt%^8@DzZZSuh%Q^k&Gm z`LjV{qx?c9|JIW&c`(8O-+j@Dm>> zJ(fNG%x=?;h<&ZW&BS{09=IJssn%qN(sclB*%AwGMr5?iLR91ZH3o$M8T|uN%IY?)T)lB%^jlB~hCvWC^+hhN=$#h1JdJU#_70Yu6pL93Nez5K6Fa=0}vP z*1cjzeyu1Dqi*#xgsjmeBaF(jun>bXAr>Io63|4kQ~QO9W`Y9lyfpXbl$L0w68U;$ z#LfFW;~@j@x)RJB=FNv1*ZdZi>m7*T;NX6UfmVu~2K$IvVXrSN66>M!#{L)}E0_%G zGIc=g=mSQYw#2dJbv|JI#EQQv3j|T8zkqL4=44>iUY9!IFXG3M9W(cSb0A(x%`7yu zDc}#XGH`Xh681Ll`L{x7jKI_s)cw2f&E<79uXsB^jZS%gOO0&*DQ;oETx0d-a_1wC z?(4fQ*mE*&X?te4Kl!K=*58^ucIZ481 zGiX5PH6Bs_zEnS%w)*sr>bnB~0l9u=nYu_g;DRb|z(HIO`!Ny&vV=S()BG-I;BQ7@ zFl$2mzwdLIiW7)FoNL(4Bil-aM1auT=H=ls;<0uA`EJws)mowc+Z<7Hu$$~K$OdRa zfN~m-SDqbC>3;8ZNOM-G5TOH(`_$0YZZcJ-RL!7|E8cGVkZvZ#hhuo_vCNVK#YRl> z2?^b}5vb@HaA*$>eeOA|u8&5_h>^I16CZqO8?o9woyn>AM_AYaxAvI;n!>EhY0l1o zjQFrhBbjh5w42WP-P^@-JcT*W5Jbp7JG=pyg^4^n4Fu3&p-4@=5PZwa@Y2YB2}|Dw zWlZv6V|u+i9AhE%OQh%C`$(e}V#T}vdyCe39T|pHQwnxWJBt0zth*e>z zzU6IxG0tFqz(DyAw}5KC7P1S~DiA{&XG1#SGhviu|#dv4QC)yqqlhUSb0I5{UyL zR1wAShk8nfQD_#e%q*8*I;_Mn5kLDDd=Qo1b?O&+Pf?9wh4F;j!P`7IBoe z9J3wpQR4HvJT~JSyVI`g`0y$a%aIW0tYuo4e>yAaWfJ7oec>(SCwgH05Iw#+8yHvn zKkEmgGzoGK8cT^KUKV+enp|}8T=jAwK=;Q~n&;sufF`K@H;%VzVGpj|L>jW2- zK|qrb@C0L6bt)DHG2!AT2EQv$gNUjGET_4`SMG0aEibd07#Ugw!IQK46rl+Oe4Dcn z2Gfd0kMO&w6hTIqpU;0;9-!T%kv?g-$&fKBXL5!y=CSzJ%D)L0${QkEa()Y{Q#S-N z16;>!f27NcnU59_Ol^7GRv^7=C-8LVW_UH>c~=48%hwgT4gVMT9V1W;16&|VDl5k0 zw`Hr#Fnrc>goX}?OI4^l=3Y1A#styt8?8LyM6mXuZ7F48n@oww`$@C;VG%zK%TZH3 zPz`VntJWpV&}Y8usQBiwzjC=g-jEGpg`CJ0OsEF%c(Ndlr&|f>YS1Z0GSoZdo9)j< z6Xz3(QoHuPN5RWffs;p>EmRD%c?2erxCPb1(lWW8NL#IXmTXov$@GYI;>8-PPeF3f z1P`0>$lw8SM6WQp=4x)>A3XnvaFi_XM^xi~<;>st3+N!DU{qgDd{JFa33>Vnr8f(H zhyV}F`G!}%{cK-(EdDjQxtgiayh1_!0gjhRs092p>GN{rxrXelJ(@FB2yC4}Rm$~( zzNZ^%O*UYL`H0swRZQu3hrcp(IGPmpKJFf%RE(bzU{pmBtE6#^KAWvyb^29{#4h{c zxQZX(yv)@^-ryGN@zPS)sF&JTiIs2p9?Rlx^`{rt;~}8;NOs z9Q^lR#U2VoCZ89!J;1{D!dc>dTeO#vLnKSmQxOo7E(_#8c1l7zZ(7_&(}{IEMR7IA zZuZS>nQljN5d!XGD$L^v zUy%cE=;}a4s%M5Eo>-YN`=M74xYx^+$h;S_`T{NK6azG+9Fg^1Qpo9JOVe$Z$% zw}#@7sZuv~46ozf*LR~g(cC6D4ti|Y|K2VuP8+hZ9=ERHizfX(v>YM<%AI^BGWo}w zlo_hWo?Dm&)B%;}*Wc4EegMpL<^5yMc6LSrPd@ul;Lk7eZ&gdbAElsYBq1MhcZ4hN z`CN@YXi`JB_porQYq;YTVz~y|^Ysoy2LdnSR*en&EkwQFcgjYaXVA zGFdvE+B~en2nz*GQBvX`kxyc@wg_G%vUR?tb$K{a^0z%0C4Q<^JD)gVhqg?2q#!GH zu36rj6fG2m*snG6pRF-N`|a7|0`2=8^|yrifpXLWE)e2+^l;V)H<3ueiUz{#e5!d-4=XXcGXIO-hvrRi%#*1ZY)gb zOK~EFSbz8(mh0UZId;IG^<3H}|09Z3RxMmm_nR4Y2{ia`J!)k0sKoyr*T?|=Tns*l z>j7prk{Jocd!=jww;9oSntmqPlEQ ztU%1D$c0uq;_Vds`r zn603&%c~ZKLpTZ3I?9F7f&amY-&GZ{&$PSLS`~K^DbLt^9r_7In&en&DhKhEz3E*j z{L~~iF>!IE&-+O8g=m~FdB~`Q7*oO#&j(mATSGir1z6rQ?9frVgk9YTF`~bS-WyG4 zmb(G^QT(=lm~;9!54&meiS_OciJjybK$I)fuOHgXcx4hCKhIgoe~}&$2bqi)ZSd&NK#g;O_?9%p4wLmCgLi znSb-+6@mjd$N|gW6p82n%@{X9(g9g4&B)>JO0CUgt{fKjdhfC{>?Oi45Ugf<1TxQ` z@f;KTQuE2YFLUj!8&_wE6uH3cPqD8%e0wpef>0hk&()ypzTcf%Yx7NfdknZ6Yiq0~ zz313QK9)%--@SFs^Lphl6uu%Q<#w_zf$y+9*W(xp2RWu~-^9d_8CW+hyFTQ%o2?V< zU|1|FXz9>{G-v$Mx%b|HQvi$nq~WwIuNB&-%v>q8*wqr&idKY0sP~<@y~%gN7QCu1 z$g-`dBQf9MGNTh#mZ)tRGfkDu`v^6=p`N85m5jeG&Q{y~`#le%Egw`qiU=m)7%J{y zg$f}rP%GjbT9}-60 zb@R*;@tq2r!IKIHht zO>+;{f%cA-=vPromKIa$(rU(r{fQX%Cf5gk_}|^QuQ>)E55M8Jd{KfdFdx($(-KbPm7n2SrY{_iV{{cYf znWm=gGGlqY4s>aWW#0m0oA};L3H#msxY!faye}5|$ukRVWcfnp;)~y`I^u#ZuHxhO z4I~<8xFRqZ_VxB;k0!!@gS_JhOMvp3J6&WjHw|9X=SOQ|B9vG$u8_X9 zw^^i49oKNnPUkB%+&Yo{1T0XM@$5I*v{7xyYnWcAuleVZQKy^SEDqK>#q#bm1>IFk zHA{}4G9V9AS3gL3{qKFw@z$FVT_eO%UXm|}6kR!?=kiPN1U)&&ok$JUnv$c?tUCNrDAyJ!)-xBd6R(1f4-hFK1_!r z#3X+Lv1g_JGozav!m*27m-9Wm2 zww>91;JTmaPs=PtV{(==KSNxsW3X*@MO-qBLd9cyq+eQ{1?_1<#gEH#+3n$x8Oz8c zex3C(l4wTqJX0a$Z1d0AWFg&A{q4KVnA!blud1+>XRmXjq~J_59Cnv66w%L!wqCT} zWfu>nj?Hj-r@=8wY+CpPo?{BRA*T2JvnACJ2$Wx+K_?$VCPh*YGD7zio{n3)i#j8l z&&Z2W)p8ZZ8}6R|oWnQRKK+y7yTlRHOuXnVz{mgeew@lW`*Fm;3&GweJW!s3n!_wh# z82;^&%-RCDZS5t}FYbfIa}KK1{BxvsI`5`p{B#ToEcv;!&`KLgC;r+jevP_A8GmJ- zlob;Tk=Slme|PU0UyoR)d4+l<^^ozM5#W{~w<@Bf1?wQ4}pT zU$k6bmLXi{BtOg4ID(D28~(7~p@eRVp8tp0%3z9kR?rZH{8`tUd*bi8whh&dcs^H( z?5O%TQ|I1k>6ly2V#db|{Ar2FAWTaY|mKpF?YM0l_eP z{H|N9B?gTj#Xc%!3WPy{$qBU7%YA*rWQf)i4pcGOK#|;&LMsx;g(x_tY#Gyv=7GLP zBb}>65|qr1+-c-UFQR~w731ue)?K~-f#2x>K`9*)2}Ct3?SH~N9_zQ`AO?aPecvcU z(3%3GNV#bG`%(L!P;JV(jV=E_f1CP$ph%YtxD-?zVAC0)e4|~nz(^$*Kv`9v={rF< z4jMpInm@TVYLMKRWyJu+1tEYh%IG6~fDZFHi|gm@^?=LSB7ndT@`cka7HW2?Ot)cnAWQ;!1 zPG2rqoN;#A?p1+5R%}cOfTkjmf-LsQ-UH_8f`b{iD`lWMjukA8#O zpv#yv!prk|^|Bbf+A`Zcb(ai|_ED*$_oY;i(-kk@x7-%oFde07eA2WB5>8-z%`1w8 zOT(rV+`vk6*vzHz{$xlz?%w<<+jg~Mm0B-a+z|Q0LaW{n1c}%B4&2A9#GGwsQ*6RF zemJ{-qscihQ|%RbY^RF838@6Y`;J^=qhm_B9l#{#uoxgIg(ch_`z0Q!SsdjqvP&&|g7PtT zC#8seZ2=vv1K{$b09Q{F_QV5W7@SJG+<4ehYuhZTm8@CGspo*5l?JN(vZtOe;TGQN z)yy(HP@+e_4?g^Uk4>!iE(Cvx7aXL;ZBymCIbodR(@ zZ6sPTK(ZgW1eNjEdY=PFO97;i<{Qr$?aWP&4R_jj@ExReVld2K23devllnIEL=tR_EA*9f)*Sx$2%haImk176D=TCna z_3LZXoS~`=JUIb4xkI}w5x>lKSf?X;`9LIiq!j=N$5hj-NIH}n_WZkcyynjR5jBO<9N6A-@je?HO z4MTCi+k-IA(KagKHw3D^-unIf@}4EI0B4}r|GbR;R+0CkIgw}P=+%Nq?|Pa*H1taX zCj|)!N?@cGsP`H+Y&n3`s{03J(H!VkqX9V)HzhTgN4leq79F>)9o3H)=i#$HSUTWt z{5Gnzko(S-Dga210)ITNY&WQUBYaEzb02s99wD*>_d+?%p|UQY68aVyvnB6ECjb$FD|i zes9S}8yM=lWK)Q;!v~Njlr=LTm3R8>7?IOxE$o`Z2nRZ$nR&KstMleONZRQyRytE3 z`ANKTvp^5QpXw@fvT|-m*JqDb+gCW3*bhkCG;{_&?Y#B4Q$a~#=balV{<3%aXYv^-{l)%=knm2UwbqR6uGfhN&$ zup;D=oFvluc8nb$3wtf>kk6fGySbfNaHqHv;`<{2?I^c7{6qoyL_y5{5MB6`a?D0Y z+>|DO|GCwGV2~_~+A2$}Lv!VzUUVcI%3dm!LLiK$Q`vN&Fk>QC`YleAW74E!dEEY#lyfx_Ykylr{IoT5x#=07IZyF=J$K%cXS>_usXDk7 zaKuc3Y&|B#YO$fZ95qR)|Cf~bON6nw0OppPE%GYc*)dg0E)`@_ZIIv%rpXu8uIpWx zrhjfgE?|=D=5XdR3R=E?eZ`p>C9`6xSK8xDHk?&i*Du9y?+dW+G7L+k(t2!u5FvWH z>`x8SAwS4T`gk4HZ~w|cg&S+~bk@fSgkBFs60vaw`T~u-c>!%Sso>~H=5rha$YoQ; zw$kIwc^2;{o}m^lUEb){xuz(^Pm_n@O_7W={}0a@&Gx5LLFVZ}n`(#49|+u3hy8fJ z?G|)s*50S2Jh^q(Ddmob!hkNUfw=o!K!eD|mQbo2?kOKa1jF!EwOZL0{YHsUiV;<~ z$p=J;PZw^(&Se?z0eJ)sNITAGHedfksN%gfK&oD8~n zYbYHG1EXD_RsJHB*FMJbhT2ozw9TaDV5a;Jj2aG3@Yr@V*(#ta{(wttyf>8OU=OWs zp~5_Jv<9qW*5_fDI{`AR#@~#HwQ(#nggp}9Fo8BXo2Jrw7kM8v+tWbA(XAkCm0TN* z;ViQjH^ls)+cKwRi0Pj-9@OiXMlIecB3`^>i>;+tAGdEEL=6OixsK>L>=75sCi~2E z6f80yAMZUHU91UyYmZV6bRd&P!j33<@v=*`#ps-{7=>QZEz9+O^0@~mQh|DJkF+rX z9GhA0ifux*1>oSaFnct~32q9=d2LZXVUxmY;J~}wtXNT@{H+d}dwbl8k|^$1^d4v~ zR9hTAB(mt>nJ;-^|JLDXEwbkdheem!dUCrt0K2h_-VWvVDLk^n91nX10^(}3NvFyS zJ>Dj@7UB1iBLuy=PLDgPtZBTM`|igwYa6}dCu@L`Woz=PIPDM|^6S3snu6%Gak~4Y zJ?xk6Y=64aZI(ify1Ru6r@o5*@D1iQsKVu1VRTa~w3$57i6oLc-TC5`)zUfV39`K3 z6sR#sw89BEuXTU;^P+1R_|?TsWs^|KI6TkHa1c7=Drj8Dr;o{rU#}c&4PM{fy6OXZ z`08j`cIj=>Nf)_&76_HD=@yKxZ+W=Q_IoncA49dsPOAH)= z8SW)()Zf#cu*fp<<{&Z85XG+g(OzD{#5GU9UR-qL5frJMnY^o@r zOCXziCT7>K5Nc0%3oI9YNI!=pBt9}5G`skko zvXHyO)2d`5`+EcdD@sR$#@+9P6cn}#e);c#cTKfa9DV}Sz@NCETC8hO=#>b#ZJe3^ z1EsUbFU9N{p1$$f8(v=nMP)ZObyuEg*GgLNMXWuvghGI$&RVVMk7N`5NqRWff;)n% zW$NRu7hEz414YBF)ox)RBBx1TqMT90qls%o1ByS%AFb`-br4WOBXvZ9eh3(xWeGmBq)frY?TqVA{~`?U ze@2?}jmmdvsD!ex{3HZZjLh3_i!ydXT~eweS_)@2ZWyU!m_=uPr(DjBHEOQ=WXATQ zCE`qitcKgpqZ{(6)!6dqn0=@5TLJ8amb2*@{JWQ_<3GNtqUL|Gmby6gC&=56L2{kR zykC-_(Y77i8cb!ICb!=&TaHWt*{jQ96oPoZ`Fr8w52rGa;UKNQQvv}lDmqjs@uEhC z7}S58D`V_Ao_7ztwB~;&pXo=n2!Y0~*|%<1pujV-L^+ORb zw{_Z=AcqYs(dcO}&uB7~_E^=onLD ze=+7o*g+(*l0;(04O(Zci-RTap*l~W-Mx(W-%*f{H=c1{4uHF18i`jzhmQS1XPnpz?Gv)&`PMs}sDveh4Ga`9G^DID zw5m7gHsFkG`^K%^;LcuiG#~c&=^<>bv^n&XdH>o){L8_1e3;7;y)f!zcG)lB*i6ru zp4HC%#5DS3F8-T$3;kh4Dl3{E?4+A{%5*OiZ@%4xp*Q55iQbZF%B(Vo!scfO@xSGoD@z*9N1!< zbCCL~fWz8wt}5$FS@h|?a`hZQQDh_2bJb8fCRkzZdG+?gc;mt6CV1e6tMj?5Ifd+l z9Oi3wRt)*$1@X4R(zWP?`Y&l=K|NFKj?L>neaM?gl2}+@ZDN$Fxbw|qRaQ%-33ffAU8|$Qt{75VgtqlE4%Y9&q}f`f zYv4@$5|Gug;93Ys9weMj)=3b#B=VhCS|;$g%&f)&4oTSGxEJ^i+CgV*&K5RE>i`j- z?HIRl=~aLXgIQ9y8$%ctWw%}9ZDcpTLi%@gx{dmTI{pAR2@Z!M9^kgXr*`w6UT@436iEacr;Ae(qL*;4j zKjW*nC7$BsB`EPy4ipVYy<*BCXB+Kj0q`Ovu?UC^C?pTr7vGQGIprutA4K4<>ySa4 z+~W6>-iv;Oh=ORm{o%Csxn)1TBR{#HfF2_~euM|QD(8l(ylR`(rRJ1+@A~r4hzPOw z2=DXu)j_F2M`>7vr_x;6K7{m3QG&}N9X}DQo9t^f*|@?m;@Y<|Fi!rwJ-aekb4?TpnrZUmP&_lg-sTnJ z9hlyY+4 zgIH`>BO|$A04CtvT<3KVuQXMUL-i4QaT{B$l6;MA{BZjGga+YAC2p&{sq_Att01Q{eoGeB85E7L)5#u3 zN2OY&AC1L6Kt|FKfV0nEp)n!tQxN3`umR5+%brlctB<2H;?f10#otHW(Avrqo;RNL zL}iAEEaEE9)W@0xD`#%I2_7ilMY7U&?*2d1FX0WV?B?UFfDDW|9q`D64Tcq6@B|hH>IeHJ8D1YFIk3_4q(pbUHkv9J?~CSrIl}4F8`_tk)T~p3pFCi7I^luE3$|+rKnWS{RKzAq|}Lx5T)gJ`l&h zK=O-ky}lf6)p)R=!LiR`EEq2^I26cW#MK4*3ml@jojpErT3(RorWk^ritX8Aq;zkA z^lG&`pH=KV2H6HIlV*EH5nt2mA_$p+w^+%HOre-s()+cvRo4m&iNh+`-pRuC=-MTTr9i~fzH}jvuXEp+38@SlLt8UEk^B~#(b4d9TYZ~h*(%-=n9&F3 z0)eL3v8&o*eGtKP={!8Ts`QKCFrz$el{$=~m(X9f4`0R8t}E7fJ{v51#zjpE$5)4B zvcOD%#+s>gK^^%>Rv=r+S0rsR{DtC_%ow0Jfko`G=Jx%~^?3(k+H5Pe9yZnA?~9Eh z#+)XIIsmeFUHf|yJ9ta0r{)0@kc4fvzVw!FmWM%uS-9R2emb6^+Yc5N3Fj54RIyo9 zKaYMpQ$LK$78YvqP+@K*Vv|^13i=42t7CvynG!hXu`=;XPQCAzMBs0ox0Xa~MVt_T z3$FZ9<16b-g99y9Sg`TrZcuT?fAN<#*1H(O>5_BM6Sn=xY2&0AN84Oa;ThF;_i`R`$i}&W--2 ztj65E=s=rngw!M&t{hmKifOp~!$>D2CB@R1ieYJn{*BdF!>Q)@@988?%3if|!X9wI zyN*x|QQY9mCeeo)_#yA#jcJqW2h4gEO5ou%(JWx1*&|{pzV5My- zHJ{=EjxtGlEuy?EtlZ)Rp9kuRDB&uX{{S0eWGBG@D-ZZxW4ynXnCt$j>jwDiT>0xj zlsr>;4DEtRVNt+Q61{viH-7dh zWBTM$Pde_^V5A4qFwJRY1gFa{iUO<2%um&#Qk7k2k`2XuU6aS_g88P)0YTa=PHdM) zqKuSuOSd^PlYLg>Kfcv=x(XEgr-EZH8#!2$hUht7=h5*kKyTvU3bjgsd;VM;o6)6U z7tpTvbW#0i4{;24r{hIO6U z!WA^k)%1~_J}0uNGdALiMNkNfM%r~Mn(32ryWib8W2}y$q1lUXt{Q< zUiW<@g&2>?b5xMHFa#l|AWG_{FR-B#@C6x{J?e^MsJB131znj11hLL2O+p}bX`-X) za|iUsN!U@uw~kMbKDI{nc-tWRd(I!mj8z3D1P(4PaA(WHw+x?RIQAuH_#)Z>O`w*x z|BED3`P^m_GbSYX^MH_*r$r_o*_f_Dx#9k(@D2{PA!Evabegwdi>G;?1n+(NbQe@pDEsI&XJ zzhBa70M6SyU`q2`o_EQ-CC{5VGh*chuO(X>-banXECsXFIZ4E+5kkNm^~ zvYyi2``fj{+xAi|;b5tzkEbR*Q%c{y{)*(d^u0Ue146}~(?ezBP>i`ji&8re9ZH z1hphYn;H^N&LrI|?qwKMyOle34vPvw>t=bh5a|=5>3Em&F&dvP-G<`XJ}$|qTe>ZG zUTIuLnY&`Mk#`qt^Y*dWK(`GgPY-0pGZ9d6)D#h=1v2;>fYOeNu??ANECU{M+u=wZ z@J1y%`ix(GS=iCf)ht&0*7V23+$&4uSTYaa2Py-XaD0?-8|*aFH}uT&RP%OF7&+Uk zxQ|A*8vaKi*3GKLYzH_^JArQbf%8H@jn|&g8pTW6Y3#!ghv@g_J(A~2JED3RTE$B2 zZRg{;F-}r!+5{BwP+PS7LH(0Z76JneT4jx3NN~z*{0%1v=go~v$`3YES(ZWx<&XJM z;rmU~LX51Bno@m!!Gzn4Rha`}A}5CWULq zN6MI}y#Z9sfI6f-kIGTyk^ff<`hIezSqmskJL-NrhPtCXY%$53;&nD)XYZ za`<+>s)l$FZQ=Ur)u$`!;B#6(X}!UhpPQp0B9O?MTN6oXM*uNHdoR-)%PjuL+&^3q zE;RVeAHTO4h@)!%8T!I?Pu;h+81t3zGcLS(qkqMZ^&c+m_#Sv+!kKZhRsk{6rSSMt z;j2Hd{dGJbXF{y8_-5YwfXpZv;1l%uO$LTwp}qiUR;~SLkU95LoD$NThu)0~gne`|{LEB*+;ul{X-kxzPW4OL zRI%yBjk$oL28JmeSn$D9*-<}P2}Ovfx&;n4Cf3bxzF__i9u5>suJ8QcS^!E}X+vv7+391?fq}rgFt&mA^=-PPI?*m z0-j3o%YnQNcC`0A=v@{*mYzk%j261TyMTXKC#zN`jsx3PC7O@B4Hhha21Q#<{8YP7 zO|5ANoc*>Dm)c%ZY#xgTMeyE!=I1>H%iff>OMQXC8fECWX>9~WMYCk>a~-kfeaaE- zY-vM0kAg26k9z%8im)cDM^g7g>kMZxYlz_wJ$sP#f(WP8>_1RyN=jV#T(atv*_;^v zLP3G?D2w+H6`PiS*y$XCY3Z^~3KkG&UX5zkUVNBr~TtptK4c6xekM_L5b6q{>r$jIRz*b3y zT)yem2!mozLpjmyxy;_x6cQPI#0zqsqW2~v*bJ`pgg#x8hfB)FkX}C9DEQ7cFUk(l ze+5O&UL-6F=3sV05!cZ2pP3#CfbKP39(1-#$~tZzEizsyjLTf8v?>iUmda}3RHG?o z3J@!RTD7^i2>YJSt)ZiTeqh^@6sN1H6KRQ2e&BoTd<+UJw|^2p5OpE>@L>o$)Um)l z>Fx1PgH$4=$lZHgSZr`INMfTEi?OPMuAKYz#A_t5pUt-Vi2~(2Cv=LnJ1_^7$%6rc zHw#Y*A69Y+M9nD31^0pMPD)NdOAAAT4EcfX{@>2%pNfcoL7{uH*exfanB*#;vO&#j z31FzBK#}H!*+o;<1(c>LzvOWHAx2FeqyDaGLheIc$Drvu8fGr|$b2jdi#=q#b1mw5 z&al)cJSH#rKzX{<*%0*KqyZ8jPf#b?k>fq_g9nf+PI1uROAz&#BTlanR>AR}w805E zAho=61F;+;CZakHFD^!vMoKUW+TmFb5cVT}Dp-u~z4=rjzDW<;5DEak0%Hdu@wFHE z{s)l^5ry|Om+y$GUZck+wc|=HN?C4Zj~IOsLyTi#mX+glQG{{S-{*GgU6pcDxWo^2@7P5tZT?s%B6 zD0%La&C~ySB0Sojc{n~DngkR2gVV57ff(EM&Gk$#!iXopa|1wsDyV>+gI87a`*Aq` zW#iA>*DmV+!ucav&S$I7pW~r*YyMV1(rGWB)$--bvPyb8EC^DXOV;FTdcK%25%yHk zMh~=BVY5q<`QC=nyDM(Eq2lc~|40>-f-cfyFj!OqMa2j#>?#V3G{1VgZLOFN81T+m zyXV}TZ!?A5*DH`N`+#h+o-d;1rj5XIn?!3wL#yPH)AS-(g7*?-2UkNLvD^4UnvY&` zDr3hs$o3$C=uOQHDBlYA{k#h@p}V4652+wd7@1e(Rw;v-?Chb~d+f!5&CWFW!+dd8zgwN*cgkSr$inXA}aaVAv#?m zTUhF3*8jO3{u7<6%`_~_sBUA)0QG+}#Vs~j47X*m>Ds+2b>->XugRPO9*Lswx<2WZ zlTJ7uKl%jUyTV3(W$msPef&G=R|0HWJ==ZxM0|shzd1i{;Ft}>K?(}TF4w_gFo8Qv zS=30(2a^3kQUhB{kYLK&?Ck6mK6@&F&w!w1DRJTweype&Fx23s6-jgsOzuCGSKz zsyqWXK}wO`?;!sRY{<*MZ--DR{Qke8q6#hz4unMg^=GpW!oF!^ISsbsWb9|w*7;Y2 zHbaBdr-KM)hp|J1Pco3SM#?5<@&PTGS9{3&r3q0=}9!RL-zF zE@%(RaN@*0iK4G?qq~@afxb^K+O*N1Sf|wt!f;sZtz6?x8Dw!j*{hA$j;@GJhF{_T z_UrPH{Nb8DkXj(_swCwMd-jCz9lQMHY`_0tc@)uph#~ja`=cc!T)ackQfNJtI?P1x zy3N^FNv~QSLA&fijc8y1xj1!JLyQa~4j2BPiqV19N1iJWv9P3fr(_a7NrM$^W7sA~b)(28hS zU8mfZMI~f4t;%0sAOfL4AeaqEkAGtBz`rE@6#~&RI5#iDa%_qun+dUt2smQ)MS zvyekLH}zrBik7+EpB~2ydXgmGaR$(9C{m++`KP7##m$UoYHec@cCWkOuG%C3U!deWYxFuJ5{kp1-EI{+m!Vk>~5Dj0S@m|3L_ z3H7g2_yJ3E>RR(xu@#iK%0HL5YQA;5s-N4HRz$8X%{*@_{zjN8j7=)m_OYxb9a1T* zB!;;Z2s)tFF@FWknwpaq{V^KBvdHwOOnjUC8WU}zHsSB1wz%dGw6W|ncAudjMl;i4 zB~hQ=pMkF0)|(3H6%`gEaXbF5DbYUmurD~9zuwHts+teK&JQ}g0g~1y3ci=w;LZm% zF+A6Y@yB47Ss%?q&nF0v&WZsIP$tXmk~3wF+>b5ZFT2u``5j+ikk4S>oT0U{cWXx; zXer*&Q$f7L{p^=IsJEZqm^4>d`XI75#lgwP(+7-Sv)%E4WeDNZ?$RAPh!mpNCI1RB z0A6B(sO^kM!4bVGw;mbua@>j2?i(BmeixEd-^I_B+e7m^(i7&p|6#?OtW!LDx~9uq z#V=#2MBoAvlrkWu2hn{LiKy1&zr^F|hh>hED(v5H{}X>!#)}IZsI%Z`{3nM@e8-rD zWmpWticEILa`=HQUnxU)7s;U~Q&cn*i}Euz1(w2V%}R3cYw% zG$5?K4rjoBhYEO*K;K9Pwi6ILb^!-z9IYhM{3ZBHZ0>2CfJ8-}_2>Mz_&h_*LTDa{ zdmQL)8$t0J;-?k?Llah&{`ON$jd2*9I|>*sRnYEfV^JHvWqTsl-dj_mrY_{3Qx#0+ z{Wl)bec=v>s`!w&#rcHB-&C9{T1a+f+;W|#7Tbxg6)fo2m4(w!jRbsG? zyex!L^swd)_fPvP4MmOGUbpGX-;NAgw^w0@X1xxpTfxs2<;O7%f$w*8-BXuxtZ`yN?+Eabt1^VQP!eBE z3nCb2T~;0GQ+r3B)lY!=(d7L1zw}Hmk{~PMZ3$DT0Og&r9)yotMItiC0i>E`}#FPMcWC*F_Hv7yGNz=UeSnMo(lJUb{IHn}L*{ z{`q55I>1Dc@nT?62$M9d&eDs2SSY~e$ML#6X$)#Y?=6?~hbR6e+5e7q{(fMTh8AML zC~Ew=tvMfNVygN6Y^%EA7~*l1`7}_Y`BSDn-!vuw9$Od(Ji0ZTB}~J@gAMydQ6f-* z9px?SK0C6a9zHeFyDHHW`(z_7;zoe^ByA%t*OwgXV1~XLE}p5kIbU}dwU(XND`^yC zrGH5{uyeAj3hzpPY`$-dmI61K=N}+gVE^vyEdLOdL1NKp>(iiJ)AfrPhtrTH-osDF zIGgXIiWw!R@_(R_@XpsbZut-L!M7^_I~H}Z}sMe{JZ|^mqm0Ihm{9@@UDLk z!%0+-Cus&VM1;sD{>UynpKy?HSXr&+s3}2El}P>Gr#bUX^dZvqqK^ge5FCHBf=ACV zR9D$-fWGP|YYv@=li0GUW>R6xikU2&ZWDdFV6TJTE`|i&CY@c)Gt)1a6Q^VUl%>?q z9$oxX;(2AD-TG0a$e@-|eJ!TTehuZn-9_@-t9|sw<0)zo-8Z7{A8Q=@alX?}+l43M;_61ZeKVoe z&@Fkc7K$u%R3_qmTJ#h^S#u13C{z{ZHgBuuCfX-pToV^LbzO@;TjE|?I zD_YM9bqmLKSva{WprPSb;TX_<26vWTVckgZ zyi|Qns_~pkghSfvv8>mn%a>#h!}$zQrFP!=mCYtSc`I!>{fDd8Crls`4BXD}Dsk|? zI-Dm4$26KY&Q~-~hvG*tA*{p*9Wi=$ya?nyNw`oFK9%A>tQto%vQpm<@}+x&=p(}+ z2SQpt;P}ohthA2+oN8gg_n^ERZ6y(z+Ph3lT}rHP8*(K8x zos}0|T+qr`>y)X5TCc8~LDjb1-Db2zp^ddOyh2QgW#Nj@+D5R)#2WLXSZ9Ow0?E(% z(){@g>xJ^hxNb@x8y`j1Uj*M3jH*oJZ0RCPJyk31Ar)4c^PeM#=M)yOF3;Nqy%7`1 zSOf1zK1+Fya?T~;z#J5Ie)%UvpzvLZ6Z)>ja@Lgg;Uxv#8|12OIH_y2vrR7wZU4(U zEV3DiRN4HIrKIs2K^z>3AjbMj%uYUWR+Be>AP|nvNDkq%&LuA_Cg!Wd=yAefNbx)* z&lB%dwuDO5aVlHk~r-FQaOfbXj|ScFuXKu6BHM#Ev6TTc&;?KR);_ zbsTew-fSeCj480UN3yqQUuSSXzEcL5qj4Ce8>4ru>oNE;GDR-^X&>$+C4GF(eF*06 z#;#I4e)8@VLjCHu@-`I&&rJfjIH8@}v{!=2WIavmsiMscXMb}k-4X6v-uOO|Poq2u z-yKw*%t7@alfa_f&UAeq_`&C{1J>tO>Nd^xEJC4`a^h5 zmi7|qbv&AGc=LP@_uri5yUUYp{#Z6c427?+Lx>&iAGSqL4hl;W-=a?u+6Wjm16W0a zo)(LcS4JeU+1#x5jZ6^twuVA#x-)kua~vn+v_;RU>{CJ02q}lQYy@#N#uny;D$*Cb z@Xvq`64N)vad81jZ+bWTP@~b_Bu@u;|3sY=icm>oYME1|rUhxPtnt-Gp?1A|l=zS4@pzN^3c(3mlJx z6|(PO9HI@S2(G^bt|+W<8MTD)t(#~S=|5PbZJN(1)yk)|xy4A`S-zf~>QU4)U7HF5 z_a~i(88l?cL`=}kh9}GkT;c#)73AF%!m)r01(cMXrAqhq?5}7kBVv@E5`rf|O6-ki zHsBBrsjmL4k_D`TIsHq)qEF^j13v#8wVO6(QDXb2f6cqiw-YA&kRo9(RC)&?$$WCY z(JNco4JorajSsZQ3C1x_r;-r7Bq<|5XgywXn7|F2)c@j>BSz^>TD#nWeb9KtF_W!q zzcX3Ma8Nt!urv0uRj}0tb?9vnI0+3g{cg|oJpz@qM> z5NT+2$diu~a3F%`BkL7`L!e+8hq~`sdEttB>Tk_~E~A7@H0lcR<2(YWV`{|-i~2$D zosIP9weM%liabKCHLvRi3uU4n>yWz{QJLhF4kb*hcrIlv$Bg6gI#c)0QA{BJQ|&gP z;iGOlO{@VG`R`1Bs?XNc;o_0!E1?sFi|Yp*6AjU z&J(VbBh07MOxK1Gtxt;Uqh4Z#X5DAOQhA@Ckus>;vR+ZJPi))i*ALfXbrQ|~u!qpd z9S9J>(VwjOz|qRZA>6LbAHF7z3pb(-a0I!2ysm3$-WP&VY=${r}Bt0@q#lunfc`bg2C6F zWzs$M^ETd%n z1SMDesphaMw_=ni`(iNC%;7J7$=H;)ApMx}xxO3O95L>@)1jp_dwUyO>Wm}ZlY@Mu zKVQ^4qc$JXfUunGyf%WYZ|Z&G?$m#SK`l%cicMxfg7+k1>>;^&yuEF2<@A%n0d2Vf z%tQbFAY-&&DI6*@UVe_YGx9b^oVYegjkN@QjEjfh0Y?1JzZ3UIk22ph4|a6GF~bu{+ACZ#Z0fPvLVE1isn8xSrg&9xePn z$(~IYbYm4$MmiFJ!J4r*+L^Lz>_GlbHvnT)+L?!!6;lnJCUWYH@;U;c3m#-s#d^rl zsU|&*GuOQdT*!Sp-D=cu+IvD@Fiae`rO9q}3>ZBiEonDkoWSe!J*+_Zz5mgk>?->< zSvbkp^EgF=4{f7^dj5yN7|f@1nGWFlfe3z2nkp9iYJ|t7BD){+%_h@__1F46wZZ#C zHO?p90Jz#TUlHM}>-gMnWJNy-SsUJsKEJ*Jc#JCut|JVe)f`I-PiD3gSS@~<6wdq@ z6h>&HV(YUuy&sWx5K9ENQPJ zH1mrNd5S5A+4-?;gUa#y6c!-EsaEXJBRBC>kA+k1DKVPztZymYlKn4dlx^sa*@QDs z(tbt%P7lcT3XNj;wl87Yu*S3*Bk7FVX)!>VxSY&L#glG&Bv-|&3;D`hEyCVz$^;F# zlsJ_14id?1@z$+z>o_qQNyxgwu-VEB0 zirdcJFUi`?K(q~hd^@PN)>Un(R^{%~oa|%?_{W3hQTNM6r;1~&^OjKU83LeJaFn?E zlOp_ocXN;G|GmI5=qe^^uFl&M^Xtn0Vk#b&ys{iOZK6y?kSz6@&K~ zn)@!`1px<&9PV24@+1sRLnweaG2w*Dwn4B`U?dKlYg_;UPKf;o*kNk%^@^y!J4~JC z7+_lA!0Rv);l=-8lhAZn+!0slz;3-j)-JnV?NAU+zp0MXf9XS=O}RryR5Q*Ekkby) z8>pv1b}{MP5KE1uZ-WVY=?qphHMy6L={%pWCRmW7W^0l5P^8Sq$ii_N&a#gWw@!+j)))vC6xRVhtCF&mSjb1MM(>y3 zzwte8u09z#5I~$6R$y;^|Cu)4?x^n1ej;P_{#T)Q5FkX6!=_TwVtaaL>evZM-?wDuKNcV$Hmwfbxv(&2D_TwAVg2) zqzq2|?@ktD25)tbiNis7sdZ-+M8`BTCI{1~-0v-JYH7;1u78B)@^Z)*5o}w|)ydL- zuKgb6_bT{DHPjim^9Z+l*H%+Hq!0dN#%+y>9wt<)nTEw4Gf#9@|FJw>8}u93?Qa}+ z{eQIpP6pSvxqp8a;*9fPvuRdI7?$J|;w%!n&l!%RL4WroefBmu{W&|nMX6`Y)Ga@n zI;_JIe@O^Eb7a!8Hjc!C{qD10ct+|mK@X2l9x=^Ud7UD3maD4%I-Hp3i^wiVAbZT%6pOY;oyv|o=40~HAhY=T?Nn15l z!js^O}yW<(=kqRwg*->x3Zl~r`U>wJ3mzz=N<3n5zhBkq@;NN{FG*Y zyQ;c2kBLivw^!FDDnqk~y(sM(Rr*X zur||uN{WQ zZP1R;1p~R}L3;e18wM{#-xQQZwPTNRuo~zxq8}N37-x}PjonOLLH$W|(s!v0OY0!y ze(P8vbK?;VA5GuamH6v_pB8-^M;CzcvlQJwKJ0&VAFONC2oMx}V;g9+^xzDPVQw76 z)`e2ivv_A8vSO}NYQBCze2cl~0qvYWnbKo{xe_IsC^f8nb)ZhF_wQ&D{~20n{&W!> z;DzB|PAv@A;J%M^ap7wI&tyLv-2AJXT92sH%lCHI+pT|(S2QLn2ZI3Sja$<6ufOc- zq`9DtycWG0Vb5Xkhr;&INR1tQmHjeR&0-DuK@HT8gv*NNph3aNpVuzSP7>a>^4&b# z%FlS^-{O?^rTVBWnnl50N6pk!erLzefC;m}VsY2eWTh-@$ylWTcGXL21)KF@fE7-j$9fcPylsPzYGYKfbC zg;OWvt8R-XetFhm!nzh2;4-pcijFLac)|qD?fjxiF&N4JGKB`gG+f7vG>Aey01^r@ zPxoGvp5QMjR7oe+vA99-GzxHF^nEi}EfRi;56?c5Md*z&h!m!Wvmy@v!>aR&mIN(& za2>^He2XrU$NE7(q;`yYPKeVOpRMFZ7q^M_)l*#-*^aVeNg}7%e)|>PW9nI%MFO<* zsrc|En{q=Ik3NHrZLxKJv5pnC)m?X<&|asr=q7xL9!5&|oiw=cljrm$I1rZatGNR+ zYr#AUjsQZV^Ff6mSGhL1J|C%md|J_)yB2|NF(QF&3HyJW5CUR>}CHPk#MA@#Oo_xQEYh&-EgBF4>xs0I8q$TO2bI{lS z4_jXy6;<2rP0i4tbcd)kIMOwghzbZ2BAp5{(k(-$AfYIoN=kPRC7sd^E!{&6F~qm= zdERy2^R4s0xLEAH@B5e6Wv$y!&(;-x(yYM};}sRS244!za7U&X)`@&!tP&aD1(mJ6 zDTC#l^{p|E%zTwHbljC(Gm0TI6sr#nGS{0{#7p~pLMb80zh>!4C}HXvIP7leW|}EK)5zr-_QcUxwF<^?;4A_3slNnJ=)9L+H1mRP>yn`CTxWo zGLufcSnU`z69>RD+qc9U@OK|Xg7aC1j)xFS-ddSp%Kz#qt9QqaiR9L^796iDOAy18 zJS85>Ksd@DI0fQ~c%FQ~A!HvG=2BW@M2owf?9>R5tU|vr>o&Y1&0}Uw zUlmx(|82P6dEdjQm1eedz;BVX6wpjl1=ysO6ttGF0J3gvhqweVOw4{ zZx*hy=jMiz!^T%#S}+r0@>VT6>BIiu_!E`lMf<(fPxMX-$HolH?Q+Z_6eJ>l-sH?)pYeap*XC(+k&)`4M&d}V5yI)aUSzqnzn?L;=LXt{!rT*JhK{06jy zkTM%$<*{X0l(6slni|%6{nqb4J2Y8{O21$Qv`x?7ovO^%{h1t#BZBGi$ReF;;QTBw zbW3)r;*BBNY&k3R$*u2p9nUBOByKb6{!o2gwtqCk3EigG=k_sCadpK-uU~?hc@t9W+ob0`S8{5!9@5RrYXw#QQ7islfMv*wZ1?P>Vfoyd=^E291JQSGddpN? zA}COK$CjR4e><3s)|x}b|32$9h%t>J;AG1*CTlX52h@dr+wyXIKvB5(C*NF zD~4OaiY%7w)x1M*_Z5QEGd)Ijkc;&u4-gI$;-*ixs9y_-^$;iP28$< z7FZ5!BuBl9ap&G`P?w4)L*T&YeHP{DceWQ1)^cD8BlT? z1V=2+p3D^xRRL47fl+^RgpVVo38JosKMf;{ysW3IGkM^#&u{iJAg;elE&7LE5o}F} zg#OVX@t$SYlt5`9mR>hr-8~Xo@*(3 zP%|5kyST1ns6UBY{JMX|yXyZX-k}G6O)v|Mi({-Nn+Q}U5U(6|;zmpGgIG{j(YWBY zGuiqq1|zZboAdvT8iu+*$<0Ja37~9clTPH?XxSvkk@r zy|-f#{EWof@r9!AL%7w5At6Q4ai zugk>}P?I6()cn~ODUnfxkFRQa(GfuW#5vJfcsR6rsI(bR5mn_#UD0>^Wew~rD?;9I zP*S~DSrxaIv6pg)ZDQ^uFc}?9=?vXR2*=r%bo$|(rUskGpud2{+D_nzdMD;tb;C7f z5{D0+uCv~iuCZ%XGrXrhWfG(Dfwd)uFj7sLgLK$(faKd-AlBtas~rh5>OKb-Ekjoc zLM@;!j%h^0hmR23OT#l6n!@`_A%o)dftfOUBSObMoAqL`>Ffjd3Kb%%=+X_h+Sc-S z@J$S#E~s*WHJd|PM80FFe@C6^tJ!3E7cF%&Gvwh$d_`fh2L`W(XMcUA{FggCEBu!` z{EX1NbIAUx>)dP4zw;AE>i5v|2FZ}^IP3a1qTg@8Cs&HA5P%tIvi;=cM&c=bR~vwq zWZoaNIj~)54-oXSrFh@hh9A4L8qXKLo{PDazBH1%>%EzQLA#R^ zavRRYg;v%Li$-^0v3TZ99lT?!UBs`E)-Io#H20dAv2UqCcpbkqNA7*IXEup9@)Y6o zc>iKCvKji2F;38nLxKQ6Ne{<9vr~HdJyD1u8!Nj9OJH)C$g~E9W4pR+Phk(ro)F*M zz8)7Jn%e_%Y1>o;uPdz64<6oleG6fb(3WBLK+^>l+#mHBENhJyMbr*`wl(ds*RQZ5 zBO^U~G#*M_Vvd6eO(coYOr>5uom>;Bk=OQZe^`6C{bIdpbt!AB2F zyF^p~;vY>-mS|>N#*I{C@bD}e&jyZ(7=|niSMYvtnw~{d&cfHc^ z!|w|A7w&mjUuji()i=nf*$z6?6Nw`~I3N2HRtA_b#18f8s6N+skTECzXfYvA10V6o z_g=YwP?T7J(hs~JWIhafAGV(k>m!ajGc@1nK3;Opsu~SHt-9l6tI8i$T0%^*q++9= z66zB5o0GeYql$A1A7T7a5c(jbAVHh=JsI|E!}Dk2#Kld0oV`nB^=Ai`swpb z$eJjpc^h{uGSsgO3V5WjsY0(OH#-PLi zMZv_-OQADlYsP7%?p3#DnP>Xe&|X)541PZ6NyKzykC}ej=ff|K-AjT;BK|7QbyihV zi(WMIxu@vV(fl%16S3jb%o28#_d(#GW}1Q4uIfX44<`iWnB(Cn6JS!(p8{cYh(zh3`rCtDyC!$uG80|wa_)(!RZ>Ah%Z}SF zBGn-1(L@v63l3I{(NWP-WT0de)J7P-G2{F>@H3IOm@P63y<%bLKJbXHdNO7i@}sKV zNN6|YMh|q@`x2`jpw*fFW!Z)#3zrm_Ir{V^j7cK{h(jdV<(1%DkAg+tP2sQ>RIWE1 zZ4uL^`U)IvHE4~uOFSviy*lGD^jhLa^%h+JsDwXVI$5719h6#ArK?@TnxgS_FCa?C zSjk|`H!N|MIxcnc92Fl3hzO4<4gsd{%+z<&W^Z)JtlR9pSS;;#p>nx|a02=w`u!zll#=`W$7IE&hu&Xx;D^qMTO z7x_GK^%hb@tbu|G(J%cJlWd+3ziyX(SVyjAZ16TET0?PaZal zx{cU77NoEKVScLqFLULD^Kic|g5Dq$rV`rPZjIg1KH50X@*tyZ2`K&2#%t`5pt;`n z{XPtvM=(awOPiptm3W=NP5Htx$LNp;d;KQeEnZS>8+Q=sE+>wuv#0N^$zMd}R5LkHjfz! zNHZD=pz;e6z_Pk$81paz-6}aT??o;g#k6ts&2HDw+Uw#&k8!clxxN$YmIgf_8o!{Q zwblOnF^B79FtD+4XON|ofPEcW#Vn^askXkWQ-BqgzEDv7G4P^g`#u(PYd03G!)^_( z<{j6MAt%@XxHkAk!pF(NWyN)+N`}}J$ofk@V}?}@0e(0uE7P}6AYpkCU^VtoNIRL7 zDsw_K1YW}Kup>gb#9Vj~*Rf7Zyy5X50{+GN?YYj`LG`PXaFDBZ@&gWXNi68$Mr`KZ zpp;HB?71%sW;zxW^V8SZtLY6qE@~I>x9PmbZab{!-PRGq59*#dpJi>?yB5bcyp{jM zxn6z`Q8XQ*^IBO_+}Vn*e==jn+!M2*lih&L01bgPAjvP-G;cgKmUa4 z9n2h}cJ?l?fJ=JiPoz$Fo3SE(`)v!dU~(1+c=I;iYjq*uuV#jAMx>qj_mg`;^523Q ze7*%XYlGbG%l88$wEXHw9XpVn#N~UB-TixN?SOAK!)v=DwZool;5Cn!I!(~#OUkZIoF?^V z5PO?N{q_*2j_OJ1M{MJnD9J@drSAOtj$)Z;&+|$LJ(rzi%8pdNhs%<|L=ce&fqaIK z`QaTg@JmCb14Nq%VUa_&!khrHWnh;;RBJ@)>y*$DNVcTtU;n@2?}LchI%H)?z;;Z^Tc2=nkwFR9Em_tQj(j2zsp z=`kaH*j#tOA5G5EAU%%~qSHhr>lp+M8T>?Lpl)goq)!~V7w(NP z8GcM4eaDPgTA+G|;07XPXV(w}y=dv#X?~ES8=oE59@GG>J&8$+3d$ zE25X7@#nBD#`=bk1C`C$i7NXu@l1HTzuK5O`f22A+I4QKc7dIZ<0ipgJ+ zirudR@YdZr9^{l9GxgVbu-IS0WRpq`tNV0+DbQzekBF51As$L`e0d3zAwv3yJz~EP zw87S_Xi7w@dY4PH69|ITxfP^kgK38S2$tnMuw@9kTh3q%i(4Jir>=xh=hc4fbFoXK zk49St(?sy|AYQC%jqR`RMd_v!DsD}D`rP+yN4E;u4C|bAm^^AA-pvk>BoW4bup1Pc z9OpLCzwxV2b-qi!mT=<9X`1;27m^(|^@GZqJ0ptOeTwC|IGV%ZC^RadzFt(MB82>E zQ5_X{-9!n@q&E_RHZT8vuKIR@BVSGKdHJ*U`tngr;q&O#m#?mxj2Je?#pbcfEpJi^AnXChXRi?wVq3ewD z&xZx%g)JeaC4QoAJD!Vz{=|k0q*w3r9DrNW{QZVkYDR=n3dPX`<4TIIE7GcSxZ_2{Np`dQQwM@Tx21`Tp_h!o6NCi021IsDhW0 zfdT)}h+gh*=M{^5$WHRuH!FI4{VUWCs8>fgG~t2+bGllza?gZp1Z~9SX5p0nu~j1q zJ*!mJL|ef4dqqQNEE@fY#JXK)c@K$?5MMphSlzjm5fZ3F5Tv-kMNdi59s6EAQ=(Rm zGm|OnxC_dZw-VT7>bsN1MCpa7h@HPSM=8%8w2Mx!6?>K6cYC|T6 zR!LD~`0aPKrwV&?`K4q4n<=+^4KdG%D(`+5_|FfF6S1XO^nT&?^KD?kb6yB+W_@&> z^9}a93xBhv8v07Pj-7eAVU-f0DQ#B?o%JL0*JunN~dz>98@?mYS}e!cw^RRq|*IsO13uik6fk4>2sWfW` z%O$juYS{%>ZB6JznvCyu4@&K5U@a`Lz^ttu>J7F45loQ@x_V1|Bgi7tw;3^x;8VL| z_gM80eM7s8&#RA>6-Yc^H0_4UdGWrKwG%H|Os3%q8N$@6;YxFeu~vCD?OT2FxAE{D zsa0XY#}q`jiLo81-!IK5dZm|OBY*iZ7mJ7g8q_NMt!j*TBfi%WQG+Gv07upoH{A|! z=|1kfhACx32NadA%TfBH#+{8paMZqc3s@ix?30=Bk=r!hk90)Vt@(u%k94Q!()q8m z)<^OI z$<_nA9`Bd6*NFR@vqA9DFA%4KNf|(91AV%`oelryu8JBMFBVdYXxG@iZ*al)O=858 zsUjU14TPyZpg5WTY9Rhdao2sGIpTo1li!}xUy*yQaE~PZe7`#Lc+Iw|!I)qqv%|** zar`^u#QcDrHU8pxqAYrCjUn`1$LeFkq<#6EN~YL9C59za-Z>qgXKKoPtWN~4A_5?o zm&MI)x@2eH&(d*7_x-oH@VQUusK+4Yc)-?1jyGNUjDNu*Pw~H^&!4`y2;6@|Bign_ z%_l6@OcEnf&xr~F$E1!=Pwyi@82KJfv;Dm^wZG!vEqCLrEl~ClZOSpe%8S+@vJo%r z`A-($%4JV$l0@?b*#i)(NuXtr9F?a)A($How|ADmy|Y#_FpIAz&ri_rUnYM&sP)}x ziUH~&Y|1y%zDNaZZ}{B{A}kp;Pd5LZO}N>+NR1mJCRYQsbxPh17ZYtgBo-Bxf8gK) zBtYkdhn#So9Et_|?V(oC-%h(yM<99Tn+!_PktndZEr9781h#r)nv$a+IlS;%*O>Gs z1YeqpQ$--D)e1JO`A=B>iOTA*= zR^vRiR;!n&Lz6jOPpbrc0-h=FnyXTy$>P$TYNHIRligBO;=NLbUqU@T--?J49yA~6 zWqxcDZJS(9`kJnm_>f+|LJ2osb&f38RZWmJ?9>ydT*~IQ0}7Njp$+|*zw`N&V^>Qn z{;r8uUxVGO~nV_K!8+a|KdgD*9^4DiJSQfcum$`@cVOjcu{-`ZeXE7)`EH_!3bGRXwizRoNo_hDaKh80lf^x|lekqtu=XLKFH50`^O^~;z9SymBU)LVsra?(sy?Cze zQbd=y{gcMWnQ83itG{#uxM{8nZg?oCQJ`Ftvi#mx^bD z?qpBa1J!j2ofpoM;bg70<|ia)nJf2LvjT^7a25BbR!n(%g^<5{s=6+}?aXR0^@=#m zCM%o>%#lxPGNBIJ*COzo_fs|kU2AuhDZcq10=_o%pyD~MfkP^{;31TW>+_Eq!L(Ck z(9z`f6IO4G+UoBy!*4sRq<1)+^|vp@-&`Tj+9n0?wtnp{A)Hl4`8w6-lX8#9JDzjV z7-b~#t$lD4c-ul|>*EyJZC|(dmcP{oN95V-m#ZrRepc*PS$xRP;F>n$-#Z!niB<@}#W zE&rNJzknMW_qrdL9a2#8w$Ab5oQCzHl;ykzh5|egORYeQ1up`M7V}Fd83wWln5dJ(jreqoaR}}HdVJ>Y3 zh0%uMV;jr2lZBqNb`6)I4={_$&vqSw)^BF98spTPWZMj9AQJ*CU!BS}F#*Eb_%~AU zd*E7+j7-K7v7H{qIwn0HuG8Y2^2}vUh9|$@+CeR_zOg!}%0#drWNmHC3T@y(t!8rJ zlV^0By${CA!)xW)6tE#hq+U-4db~cz{9JPJyF>pki%f(M@hMqqp^}F3iQRPTn!0R} z!(^-P&vcDfp?PY(6C86hQKk|vYsUXnHp%*6^I z|Dv+Fxs{>S6?NlSsUa@^#GkZA?_TJ=F==*P61ySMZ+7Q3B^mO*XBTD-d{^Q&&4_}> zjs7oq?sGfCwM+G$mG89_IdzEG6Xkolt(^ijIQorCf?1(HuT1P@ZKeO79IF1k+!}x| zWfaXV9jEmr_}iM)i)VMh>wW(!C1K#y@w$ z&G^wqWoy%7<--u|(ht?ANsHH>{vLb|cZi3BO8j5m9a#D}l-Sueb?=E>w>nh)qGl|Z zw)mS#p8rQNJ-8}lfOu4bjT0~4k^0HfDFnh7aV2sZG!#`*o4B=;99kVBp40#O;h+Zu zI;k_g1gU$OtU_>f`@KnDU0Ko#!+P|C(7XG8tQoX6Z!yHRq3cdqox} z^d`=>*Ig-?Vy3%`bj1znaTcCF{S5Bn(9VoKf$k;6(HFbEq;fxZLU+j?zR_c$mwo%$ zU((RC#~7f&{z)4DhE6C|7}MY$RrCR=>nBfj9W%abiqpYUhu+gzZQP?K{F#zdugiRL zR;zz8e@;?f5}GM(arC293drdWHfNb!%#Yi>O!?11HiTpTF&}Q)6j2iAio*zR%#X3r zwXuVmmo4O1C(jcOtz_B6wtfP7I{u~m1~Fhg)7QK2fxX@52+2-cpuqv2?@MD?u75%& z=Uj-TFS;A2Rt>78-W{&*>NQBnej$iHVoYqEvg~y7yN~`U7f(Ex4#|N9~NzlV$t@%ZLz+t`@D>SHn10DZ~AzkSJn@M05=QlO;pE5$r4 zH5xTwlwN|*!3~h$0c%B^d$qH} z@82qw7YB4wHB7zqITxXZFL@#Bxzgj9t9r8RDD?mEB3Bs>k5__&84B67`JTt%rpQ?X zH4@|;`#?EWnu>$mx>j$MHw_QNI2JiVhSMSm{u64gf(x^jZ?On=!os)JU~zoAhZGuv zzMlW=`*7EM(x}f<+RhMEXBa=67U{o)s*3CAdN|a}q=IZfSJNzMvf8Fm++7~IgQxXJ z|Em(CrlU$g0NNkfIFX}xq*MBE3MLOG)%w}XsN}U74_jQt=}Etkm_iK z7W|`77<0I6q6*xPf>uLr+MTKNTPtGfk%0E4kW!IgzEFT!UpP=I#-1<(_a0X|sK>pQ5y{ z&DL7|Wat7N#M30rS3vG|w4#2A_LGn;7(@Ni(*`8;@Qe|u((yA2tp4kdLLj=MlX=yE z_LW8icCXqIw>3iM=ZVPMuuE|)E~>R>yUenl52foJ%!FQ-#Smh5qW__T#DLdG-{>!C zWA=I!?gFUP0^S2KF7Vjx@=vSzRRW=a#KRTZD;n~*hPd*=Z-EK=0RR1G>cLj4PqcZ8 zZ6ppKov629F=Rnf-IKp^HRY%@+2|R@*uak#BN~TKzv4LG3d6mAOn@)LG@Lo0GTL2Z z_YfUEF4V~xZmYZfRZ`2k%;?AI^A9q=tEphET?=J^Tu_;}^nid>E|93HtVQCubq1J? z-_Ra}^q5{8@vEs^jxd=8tk@#49(0m;!Q|!-RAje)1TSmnehNSoca3C(5zQ<42;R` zm(OH;_5-pSC^|iQRN?H9&=tA zw4AfNS7QQ=33)5U>5M0~e#jpoky4-<+NhC0i^$BpRoHU8F;#A=$qRO+7J_Xp@Sq>D z-t%bHDcnFh>Z_uv9;umQLc)uCq6!j{w(_xMgk>x5biTT-lhIwQBY((2EL5fN*7h0( z?fO}7Sdef>eG>T=^1;^eV~n7F@nUJXMyte0PkgMo`iF!@I$Yw=aXbuNxF$UE^Tiz0 z^eZ0u5+?5#arz44C4Io=LMfbTv^E{ZcY!-f1=A{JXeoV#i}7-^QtaXCDo?`gqdVJb zvS{+M_xK2By$qd!)tB8&869s`3vD1o=cccx|8EjZ zd&`DqEgrlz<*~0+y7-4UHN?n|eDl_(NlMp5gZ7P8tlgVr^cTzlfh}x_9MwdeG6%AY z)HLZdX*p(wSN@ono934{jV{&j8>tQLOX$>1Iq?ZJ7DMz6Ra60#&`ksv0)aJjgB-?L zk&A)Kw8jijYvF}vd=h&<%1(Rk-XY01e3nXs&0osycPYHWKks3s_tINq9Tj}}oz6fH zXn|SIEE8Dgl?;uGGI(vQ__k&18iNX6776sC3A=po zpzg@r3HNKr+(`P+$LVlxl>2eE`KY!@rS}typM%^vc}ZKW22c80Io4yhIpry! zzEv_sEBgscJmB&Sc>s}eRZp29kEo3fU0quxD=(u<>d+4-=c^_)TR0j~dbps|@WCoa zF0_%Cq^N8Vn^=;a@cj;M|8h8l9vAG~!7;!(#F%ffeKmgGL9>>t0D>!>XDJd*-4&T% zWXVwQ*;eto^O6@qhB%HMNMv9OPWKv#n%UAPBkJH!5A6R9xpi;Or0+HpMQ0|H^kC(r zH*6tL#DUb@Wl}iegT(=`7P(Y#R?EC<2#4daBW<-=a42}U`(TSPzL;X0mRQsY$cRB< zC*}+hCm))+>W-p6u;!B%sXQb3AO$>Ga{1J18Cq0B`QCU<*MF)9o;clF2 ze)M9@Ej+QG$bQ^%HrtHO62$S^=N1VvsqNAL?bDM4hy~I0?)2sG8;y|0NxHQYAiptS zN(U#Q&B~e*IX*s-fBOuS<#5JFp?w|)(DTSNxMH53M`rVC0JDC`n33)NtW)GSCbJm^ z+BC|kqa-r{Q1+aebLP{®rqD)*X->ZrT@RH2oY?ZZ{?PrP+X`VC!=g;=Bvg_+5 zPg^VVMQY+;bKbNp*VFThA^G5~;ec(ani(u0NHTr&QY1hUSv3s~A!ET?OF5I6atz&3 zsn+c^J8c5OGP&0$&Iz8^_NJpb@WaF^P#t=ZSikuX{__qJNXlqQ@v{lzgmxtQw%m~ zboeqPGd4uatrsr#+TW0*v2`V|#4nZhrG+mKfjjm+!{Dq;oPCW}OKTrU`jh9NW^r<| zJA{!&g?4xX!`s%VFgoA7g@=8J5jkv#A6EKxV@5LaEkmC;fw!h~2NOEhO$(h;z=a4i zA9w@#Vy)g+NM_($g@uxTJ z`P}vjoN#*cZ18v`hEC@6YQdD4?8h(RPhP38)r zCO?K|1hUdNx{Fm@(M_$6Y?8?`>i6cq53V|OA28|jg?fBBdeKQcq~=5(yx@|VLgkf) zGVB==O4L%QlRKyU?#)G>{qr`X?}eI}?J;);n4gS4qv@u8GoJOY@}Bgvv`#QuKo1-Z zttdqO30exVbgKMk8cj9aCYN|DQ(nfprSk=c*=4`i!$|v7o)X|N;AsZ0`u|1M?4VnN zHw;=q2*A=0*j*|b5eGOe|i4YH8N*B8nzix z?aHh;7P20ArYV!>6>d9{ryEBmEixx_>C@WiX)F8F^r18yfKtaNtHoz_U7_!)UY#G; zeRN)4{Jz+BIM~ziQX=x*yLUkiK4{_upOgM?p|+zrq$G@zbZ$sT{Pn3a4z`vogpW}F zJHXl=o|+=K0nhF)ivq0Dy|9gaE#3;41_GTFl5K>6Z|!DyUxruBPSRYFs76;9?n!9b zKb20_40P3n>AQqUs-uo1(5hXm1W~>^ zDO^AY(B~g1$3mcId4FB+$^Rv*{cU*mieOe4bz5dc)Tu%+e~hh!8Qp;RW-B1i^T+dw zMsaKC5CH{l3b|MKzJNrpqhdMu4;ym>RpYq!Ct!1&qi0DxS~zoR^$zuUN7}y0!^8Qf zh^Xsru1=v4X`SOSg_nA4xzxMg$srx-ZdmldxYJd2vF;^Re;1)Vv9;>eQ&hg! zoNksH+9JYD;#E*PzxfH{B+2C;iI2424NhUAw6?PfSG9hWKuKap20=bdUWY$SzS-5v z&wjmpVWiZaDU-9oWzmD5vv6r~AUrRopZ0OUbB63(&JQH1>}|z8#PJ1J0dzLjMQd^) zbI`84y`#^V|Go0|W~ei}t=3Z5YR{5MgRJ$LdZ*+hjRk>sZ@qohp;{WQ^hx)W(~iN} z`%Rx;a)3Jc#eOeim3<&94nChV^Se^w@R&Tc=74F%GpP$N^xnSV6RU{o8ckE?gIx-* zH~UQeA}eQ8HPIqd%G-!hr#BOa2Ga%*SQloW?%6ML+gg`lr?URshu7!mOx_cWpFKQk zMjdHIQA#1@zEil|QWSGlEJS&!-zdHdqq4ouUS*>#4cRqemwB|ydc7jZnAh%&{4I5p z`NI8|P5*$I-5&HG8pF_t%1ffIDWe#Zhzl+LRny*yZrB96#DJHV!~C0mz&_dZGvd0A zO$8)20RP3X4HcWXGD!li5;uNIU_^RVtqgcfzN|n@zC5iP zE21uA=y=BAY|lB^+4RL(*3(#`_TwENYas$KqEns@k`Xd2nT04RV*Jg;nT6Zsa;7OD zYgiVq`z~~16;RtnVGq$`Zi51P=F?&bL3H_Q`7qqz07GD0d(>M%taX>`5jPui7CV06 z0&JXYD<*~v?`aSlh4$T)Y%osqkn`$qlK||%5SqIY^d)2ai9~BCd0I^@R`5=@@9P5W zG5_I!S@#`s6Y(YC_0e20m(3~2^@o=&ciNbNy>uT9AKO{yg+@nV9>5PMSpx@CC0ERa zESCqb4D>sH;hFWGcK#NImyk26v;|^@>#kW z1)=)YS$DJXALFGym+z-a&2Xi!|EQpV3EJ%i*H>bNhG!qoj+wCmQ}N z=X-(XSu`ZEaUC>1y;o&rU6YzLmyV21#>JjCN|SJsc*F?HNkPex=l8di$N5Wxgd6mOBilRXOy9!8Ulu)}YTa z#=;bIp54D zf3pqUtLxgoOsM$0MvS9?a#fUCSNQ=9V>Uya$9W&{#=MymLdxJELQpj3qu$GgbTXa}zsM{KcDe=nd zNgtOLiFv?{T|`_w_N1^y32|(tEuDx}e!}VbWJPfzh29+)!Wl^!WIG5=zpcqw=SJs+ zOpE&MwO2zA>{Xn|h{{wn>pW?01~vzJ{hhg#=k#w%=BtS|ZAzfmOOia?ow0#bg9nr1 zPL>sfOIx9m*%fIfah+SV^T^kiZ%ryk*Af@h$JA_^1 zSDXcLLYu(r_PolL?=StEA-WZFldCcCp5^w6q=Vp)K~GIvn)lW3rSWy_Kw`+C@r@D3 zT3zw=HgbVf-3nz(uq{4Yw>>RedLV)>8LMk71cm3+J=MF2T%HseJ%FB`A z#Pj(&a|7G`I3?{IK7xp8sqfYS2*mg;Pqw0Ulg-X0FgxinQ?2YrnN0`rbwBd4Bo|}f z>+>4U#W(}|&>CGDa_tr-hrz`3KyqIcf!q^?y0jA2M)0a^)q|SFD5+KJ(~Gs|f%T%c zP8E7&ioRY4tPl2+U_zu-1ESL{StZ6Km|Yn@E#IQi2*DAT;_sMUiSQ(zTB zyR%>;0njR`i(C|{oG!jxOx88RHmY~AnFTk7BK!8*O&S4N|2xIR1S#^!;$2Yv2#6(FmJo z($J`$g%@VsFJBOBB%GU(qV^-E|`4-YEl=&S2zuo<@2}auCUIip2rWkRzfFwGa35+unDL;2+C@R5lK^ z5($(B()3BMnrA$Q(gnVJ$A8EY6}t7o*5crcp)&pVVxh;Yuu6`I!u_h@v{);V6V9q9 z25IpR-_#CKd}*~1aUFnL5zH$ur*AQ#%%!xIbs+@OOW{i9`T8E#kBCz76T4jb^BayufoW+42R)=-13ej1y_wN1BacN1B~YR@co}gNjd$W)^tsI$>)eG zb$Kpj>u((jDhg-=`5{HSnVr0WrF_2qzXKt48c5R83M}I$=-cy zSjbXbAO0kAjr5!2TY5&X>;TuFQLK#93}KzL%Ir^`*a+|{_fNFZ{aXNX-zdrzICp7!QROoUC#m}qv)KuQjX7t6xN=n zme(3rc>_^eQc8h3rmI8bmJ()4!-Sj?;UdZeAHEvS^u~PK>0oGiU!*l8jz6bgcBfjU z)X2z)vG#a%EAv{wJJ7Zuzg2cs7G#ooajYt^0}O=$b@%#UNQTB_b4I6muZaPjFp!i{ zw574@3Y<_8FW_|(-POVJTs{^S7H%nSI0`J_GzJ@OoXjTd1HWAe36k=@i4V8S?o7E; z53s{>@iT96NW8Jnz4+JCJ6cPNwb3j_6NUj%7En04j@Bs=mp;+ya`L7O=TxOf+zH$d z#08enYRSKF|NCdE9G}5k*y1XbX&)DE7J? zSLpqVjXA$f5A!nLuT@firF=T;uL?Ew3|lZSP+zN6H?Q{js4m75oe&^co=shTpq@&0 zmPJhtTL{xIOD!R|?LAI(e3!|OHpOkDyhoT3Dtf7m z&RA)U-ci9WI_eJ0ez-jYI)P`BhL%{zwU9PYl z7Z|QI0tv2ed@>EHY(49us_3M82e@RyCXdPMB?dFNBJz=oOCXvy4$iI(eZ5uDd|ss)KWd!Iu!{g{>vr%vYini;#lNR(Qm%47qcjW0C<~1NZr) zpW8~FVARxe8F#G~a->Yy;4PmhYpd`$E2BEy(SFcaxPN#?&P5?{YJ~un(X)%QT~!A0 zJ7{R9JpPxiO0Nh}1aGK@!Q>^ki|oX1cd_`GQ}gy-gc)o9hxtS6re8E29I2Ct5`4Hs zRQt4oiPn=#t-bEC`D|Uj$|vWoivbXK{sHpd<6`I3UGW3h=UZWYxu3S6_*gu77C#9R zfYRB3zo9r{Z|m-STGhWN#!Y#GE3;N2@v&SNz*oR#)Y)K{1s4XpTEb2HTE~LK|BTa> z81RUoPr>v|QlD_|S&|+n`t(esAEY&m^0#CFcUH?CB-M>2kwXQ<3iH4ZLa8LGYAVk=nzX zuiwxILPOP6d}c!PV(+F1bPX>_U!lg@zO4m$t$!n2YcmUt!a=GlF-8jep+bcQCCTT! z_nNv{90wjpOVSB6tY6f2>WtW2X%R2-QkL{AlLkoI)$KC^CD$(1%ZQDmKYb;@YT7H} zmscmuud3dQTpW)3w#w2-lM;MxZDfi~27*h(xNg(b;BCa&5IcUf9~rZYsp>s>W&nwV zOgDQ|zMHIqK|bpk%_rL>f_+a%+v`qU&-2rRD5EY3*c`G`hLBZT3)_c+0h9BulD0~3 zMrGKYc3S3@Og2M@TuXT#zR8H<(I;{4pkli@In?>G0wfRU#|r^&BG*p~%K6M4^Z$r? z>!7IL_j{P`ZfQhOq`TW71w}<*De3N#T^dnZP#Tt!kdCEUx}+PHSbFItl#b`c`}6(H z!wfS!!!Y~DecxA{>zs4Fzzp%Qcr$cawkb&sd~^;9sFk< z#0#H@Q$$TM^d#Q(_4GcP#i1RtbHQnFYj)9~JN-r1-5pghmnYR2v7I;aRwTW=W*1AW zzNc5HM%h=smiOyW@iI+s9Igk)XhgxX6A#}fB5`vwLhvw^prSYk2msv|23Ek>8%}4~ zH?(ih=tl8{XW-5E{7^bmKa(KIvy|g7=MgnS+1pXNp05H5T{OG8L0*NbDV-pnBmUFa zu-(H-pOU_2TKC30n%>Lf@F+98-FcC!zIlAaVYdkUN$*$|HRIp*pFC2{VdV;JPv{&d ziD?dLFTytk)&fOcG2J6Fs1Y-59}xzjt2#WVD1+t3G5M+0eoy0`%#h<@4zmZ041x6h z6X^iq`6}ctznLb1M~qbL24eacnW0pT4$0H;^F?R-QpRDp$JbhwopUy2H`6qVsJ2Sh zzmO48u6%g&%l8lSL4+sa?MfTo9<`ss&r@?W6I`y(Ohr_{-`Opr*Dd-j8p`;Tqvl|X zc!a)^oxzc;Ccmud3)-pA{fhvq;UJfLx97xUPakFIO!`}CmnFAA0AGXR8@X&3S_Vh| zp%M5joJRF^rFA(mbEwxgrPK`JP8-_qIGK1+Mg^7sAIzDr_>BfICHxl3H}1fjjvda8 zpM~8Eq=)F-mmNKk0-=DwJTkqq=?aJ)?YSC>cuJ3j$TTILo@z?I79fh)Mj9Z1=_9P9 zX6W8y^jQ%`yVN0iG^Ewgy&accQLIC%9?cCjDBZ^#MpbmX;fjRWt`PKqukC2~I~`@A z+vSg~=|`{50ZWPY;O^}gh97xNc+*ub1;loi3dks`bjn^C=H5e_;$Utw7G|o|RKMw2 zGTV=ud)-}L+`UP)V$sUGbTW58Ky<0@_I^+m#p;-Robf7yxnL*#OJ2#KoO>MK91Y#mhqyNu8~CCydQ?UXGYg0ZS_Kt+8Cg z6*Mpsu>l|!M;N&WRmeMFO+8S~6l4gq0%XKl&E0p=8Fc){SbC6~ODv0Dv#5|)%y`z2 z+v?Hm7BYSAX?XeFn)WqI8PtuZWY~-qoPN4L-%lgT)nH*60jdPt^0Huo;}CioJKULOMdGV^qL1e6ETZ`zO4Jb-LoTpaO86RDnUZM=XQe0R@;F7e--D!r(1R3Gy!F$V9~d{mqB zE|ZM9$UCPf33F(Ye=LU=)s9lW4BbRW@I82C6H9Mfu-6o4B5xX!_zLXH!fpDLC+CA4dLzT=l^+;psnxlXS>PgVf&kW2?TQ-;L=iT!NOb=Mgyf@CqJMx@O%Het8NjMwe< z-Cs69oGuumvv7G_$3y?8;m^s(O=tS20~=|0+b)YwL?x+eqf_d{$tP1nIKYC0g#B@X z@eB%i(WAyw(;7|ljjIbu-eRiZIDaqNq~%q^^GOwE8d|S+F;X~{uMC4Ns}2lP@WEG) zYS=!S2oz$^AIYI2<)8|ung2!i$M1ka*JEIn<~EJutA=c9s>#Zu)kxFMZhEU^OL-%m z;#W}p4}#+V;OE+`kc*gD)S4o3Ex_ubYpxd;13J>eWI%e5ENn>>y3CBTUWhA+PcK?q zFIW3f-m0>SW*3D?cyz%9Srz>9R#`bu+h?R#ti4q?^KF}y4cM_|OD@zi>%%WFxH1X< z>%(vQ0E@L6Tx@35nZ@-k841Ecw<@GFx&Hz1qE+1~y)y7@_l4CSjd&C(j zPU?T9!XL4qS z-jOFV$k6GERrYX{t=fb1RBE+C<$kPzDg7jv<=FxtCD&NaDcAh7-VKoW25gn z^VQ(U?IwkDlk+KOf4!)0IhO|wy$3E+v`NZ(*NY*H0TjQ5S5^mzF@Ao6a*{KVqSAfu z{rIy#>ZV-YT>~88yQPb2O^8YcFkg`b`EMujV6hWkeE59^ywzuxLuK4L_v-Yg%Cgf_ z|C?E82GQK7_?$$T)|=&9Z`NSL*R8CXeqc4-y8n@eY^5x^Y2->ux#NBZE-0K2;8JsG zs*0KK|Nq*(M*y}Uke@YiRx~42L1qlnc+tb5h-<5j`$-75+)WDveRoYnDZZ5UM1{|XIrHO~u5^M5cr(u^7mdPxW#@DxL!L!i zT7nxKT>1!VzZrR#_Cb4fiDUx<8l~u1;;hGUZ3H{ z8Ot{^+aqVHBtiu@)+FWKvj9>aKmbEK@O+eLfBqu_h?74 z{M9cr(QKvoL3Hx)k^%e1B|}L_hgpxg;ed&sKzEO^o?V~}f|jxOHO}_Rz7m4D4`Cz8 z+u!tkX-*o~|Ay{8&n`9GJEs~~@_syXu^y?0yg|VDk7Ei-6L?T>4a-DEq!=`k$=4xo z=FgI~RpR*c4qjg{_F8ssB-o&FJ^Mg3*l9-@um0a#z8_jYrVz{RPbk0BrS|Et&RGZ~ zJO1^53iVM*yGIVq=gNSRNO(;XlcK5}N{Q`fS^I*eV5#mY(X7zTcqe=8QG{201Syc^ zBj|fU_r5ZzZ!vQ0rAhh+j7FjbEKz=KpGo+?ZKRYsSBa{^bmqEY7@T~cQI!I{^mhnFhpv~A3 zqi)4woApLVT*mb-E3~4F&yS2m05XEsi)E`7yF}*kTv*$rLV}+n)VNa`SVsbpL+PGCqR?`iMHl8 zU$GI00;)5EpF?SVycQz`E&ser7TbsC_uy?4>{{_@V%~TCyj=ewN;Vlpk3yt%@^#*@ z3HZzqZTfsK;{A%kwUMNwLxvHZfx>cyaML8e;6&=Uat-}d=A0%cByPX0gpYpYe&J9k zv7;J@s~42X8|v9kU_n_7a=#8xC3c_={#8%(Sod#yr7Ao>nC|>^CgSVG6L{0~`1@b#B!WCbt)n#Cw$}x+&DV#*SyePW(@$L9JS!S2Rcw zzs3o0y#0K*u>LG!Q%W!7bG@<3J_y|~jDIXfV-JTdqsMLugXqtF7zMd%V%FvkAB-}=&5IVKgUL0qJZ(!-il(InYpqX<&tW$ntM4R+)d zXrPH>$b$LgZo)tBUk-$U*r4rHAI*x8`+z;r=dQd%o!T0zIm+DcJ(G-(i;HDxGASaz zlV;Z-X}tb+q!P(QCPQeuh1GKI7Hg%u$-lq**w-tIg*31IAHm|A3;6`0*YaWD6?>@O zV7SLwMsemsQ1^$m-dN{pCWRZHo*@b6;W02MYG8H&J)re}&tB>oK;dWZ9<(9Krqx9o zew>PnNiK3d@5vo;z;l&tw6Lu%``c(GK#rkCvS-niWh{6T-}xzj(rzAjI35CTjO^mnxj>*Zh{<=>MNDnG6LsNe~nGY;WB6el>?S}G`;HA}s;#sqY_ zvW!kCUbM^Txx{1FQb;B{5%i9`G70xN(P`Pwlj@uNpCv>kRMP;Zr`>toAq zpf9d;_=!us?E|3L6cmVizlxT43tYaqH-r~}nPr7`<|r9`7AM^YC1h;B!EPK(;j+ba zh_6^4MYV2=2Ih?S1^Zn>fH11FYj|b=hA8l^~ndA9NyW z`Ef;6ds*>GLQeub)OM*ltn7)_v)bHa1u#QB5Ag#q?;Zc7K1xvk$MFX!Ex#4Hbnsul zr2DMv3jF}yO>I6^+tMe#gLO0|Yfo_@YjTthybA+0J4h{7q!8-dpEfF}p=^~<2;;_QnQ$AUqLjl7s+E;y=;e_&tsAZe?r^W5Th^zcoA~4 z;+H|6zIM)RxB4tI$^Y=*cfUtgud6DOz~jfUi6KT;nXyCd_zc#S2DV3`{wpdvToLoh zt0qI_Pyqwuct35syOz9|N z+mH#l^51NRow|{)++BR6YJOf@ADtW=;D{c`aVB%kZlO&OP;q^Av^F8<(Q{PIWd%pv zZj^T;;?6iTu}6&u7o*XZ^5lDHgs$6N(*buS2F(mSrr)RxRJgB=ls-Otdyp)A*1MC5 z_5CCx&EQs3?v?1W-WD0h2niu0amE~D-|NkX4dgW~yym@i-l9WB=Amu{&NqDp?@Yjc z!?f6CE!4-*JzG@jn&x{Y7M1XhsQ!crjtC(k$Nw#Q>C?X-AMx7YxdRUzNH$&a1gQ){ zE&~EXt9RQEfDf(a`?mX%9@t1!ASoI0Fi7HU?+Er}kUtc>2yIpYX4lQafTg>=1&|Ja zqJ5c&0#F5#$q}sEsJW2Io0$A>yURVHx2Pf1Ce$MmqA>F5W>yHY52}c1Mxn=-Hkh!a zU9|4kKm>-dTV%h}CYX!^YIaM~Pc<{se6%IOCe|y}tI=AA*TJL}rbnG&cezW8b>-Nf zktjdSwB{K;Iz;ep4_nmZ$q(|*ElUK=d_zz~^aGI|sj!bGeSvuT-^~HWI6%LP{e3@l zv{(nU0n+W{zJYXHcX#OaRVc6{T+Dp|cTe$a53n$c29d8UL*Wucj_9#&zq&(VLjl3i zm=~dN1z_}MF;=ZC4VXX&{2!&kNzBWJ?CtzVK|;2|ANFjVTx@J&0269c>{l;+ozSVF zTiQL*NZWr?)5gKUpX`VGaE6R`@NTO(Va0MI9oxc5-GY~l>^c73S-G1bDU*CJu|NqrLDfhjaxPG-q4cI>g z(b6Ivr+hoKB!PQPWK$6XtVgf;s!dc0`_C|?jb@a#k7P&DoyzI^@wdtKm=Czs9b=RdTN2UF#O<5*?4ymmVh0kG2ud3al6KIy}xOs-JjY^XK1p_pzQRK0&S~ zgLY#Ze&KW&FwSF4^zZv*2Zqvr7`SNIEZ!$C{W|$xcF#~(e>q0}ik&5O8X@man|(P= zgw)`s*nEN5s^hgF$K(Uq$LJL92=tU0S>=_V97{dtu8>l10;GQzvk z6n`nE_vz)gL$XrSnfL7F8+iIpnV!W>ZNWg`rPClTRfUvd zl{m60Zuj>#Xw|k*kPM-8``JcrsaWkwTf{7+8Nj794$O42;ZfdIGb1xeVkvFMfP}L7;AUkFzDs4Ue z7w3i767a+|SQ>tJGlcpf_g~C^*8r4)h1Xaa;2UUakl^v@*AQ7y?Fo)k-lg`fR%_n5 zwwnZoNdE8t#|6-N_51R(-~agO_mjfJQbjsRd}V2af}zAQ zN@Rf4?-=xC2-*zp!`iC1M7$rZyUHGFP(>0V%)+C^!rt952NAn9BgM%Pz+={c0pJH%#kSm2v<4Nyyw3 zzjnfAG#CEZNGX!y0Nz-52PF93qr08qWI?#Z5-L)M9E#z1n>IVuGjUj1A?)lNv_3T`qF+k$-KVHH3i`?C*j_eqWRAd^Fo9k~K}@+V;Ie4cEoML6#toHWnl5wnhlhWX zN1u(H4|2P#9p6$Xl?00n_S^25Nq8gU+KTEaXrrOJWPYVukL%hBZEf;Qhw5mHnSQ zkwVl{-fd1z9nr{UqS`sDw8B%^03jMceiRd@6pRV zaTp+GupRty+x*ppXWXM*>iini4U>gi%G%*3O41vH(P`nl9-g$c?h2OQA6>8*?iJC_ z^w8mJe8if2*!N34D`GmNh%wcjjKk(3E-dRInJ}2Q2aE15F9z;BjVmnAn*Cko!!a=Vdg3%MT8l9SKy zdEJ{#7({1b?7pbdz$Pib!$wWduZjx|8F~G|CrVu4uhc+>#-_2>!YJEZN zK9`uj1F*G9+drhduMYI9YmA)B|EKrzycYx=Yfxb)6*ebz`zi{?09`#ces~naY_Vi# z&e=Vb5Bz?1eElDe?{xCy>3lAbImjaaXF6K@0i&JyTr))&cMHj%LD1(4K>0Pl$$z`j z)&>0Md9|!-r;1vh|$EDgF5izcuLrl7j;Pi&Yt%c}CHu&=yOK2Bu!$K?LaF}%`!syd`X}-m8ZV}&PCD9{Ru7tem zmqf#PN9kt@s_Yq?R==+%5hbeZ>J2^#Opwo$Vbf4H{uH5daXmJ@+AX~hxXHgJi+7*| zk?IPZ&ZNHypUwbxW9iR;%xAPhJOCrr9FwAAyqRn%cgRKN4+ER4xY&@>!@Fi18Q|8e zeV~J_UD`r*V!PK7xk2B3WP97B>y7p_8ra3H6X<4RFI8IwUxG~W5#?n*35`@gp7(?g zn80giztsGoz%VIkG{rDj7&VSqRLBdUYl|)(l@|4SnpmGG-xm#zo#qlJ?{C@1dSvUT zfz7$xtSk<;_SA&0t{t1isE>i@?(C0f-HmO$38WJ~omNgFVF3{9La)s?telcqvBVM4 zz(UBMPmVk@p^s~Ny|p*4-eXCF?aMQMy0tw1!6VN4$R!wsDNnEGpK#{+rx$)*y2xY$ zB6IjEg$;834dKhO&B+Ktv`pfL94)?qQ2a=4=x?GzQ#U5nua5rn#Di&HFk)OCUd1%` zP8d&sT^Ft2n)2Pnwwz#0{Fc5vSJkCtdX_cI^fZD?y$j1uQ%8*^8kcI1_Ip(pjZ!1~ z3p1tXpR^_)#dT4{t9EsnY6L%~`@uC!OF>aW16XS|OAhP_7A*P!-ZhJIXTIP-2s%Xs z6|kA^?!9=2PThRH!!^SjV3p-(bam=|HgvluViT|r+jtq8Us9x7fcW5l&HIdk_A3!h zG#2IL8&O?M`M7VZ{k34OjTI(CGiLn|NzmHam|(Cp=o!{?op8fkpI7hLpXtcae07Pb z@IA%!O!wVBHB-`E;NzfVct(28AxcXs9VA583a{jS7Y|nUFMwjO45ih6=*^_FF`anC z?V(buLFAo%0j*x11Euo$gP0zE<*bQ_T)Q1bVrvuce9MY)*fdgRA@GUQ21Wv#^q@&r zioU~YVu4^O=iw9Ew+ET;d6GJFaNW%gmz75U^$fS~Pbf=|*Lo|-YbsLXc)7I1D@yC| z1`|CJqAe(8fu2%MukddJTjAQ50pUV)!^&%h-oH`okX{w2*87ZV~-FsSvc>ZpGr2l z_(_fxqgV=RJe9>UUuNj}mi@u{#kgU85i>}UYNA6ZWgEf42h}4^lvm!C1L~fF` zm+F)JkF40Zi#4eRyQO!#3t2H|eHS$;LM6*zZdMxOu*?*(CVge$f9?z?BNVon7_y8%8%qzdlw=sNMsB1Yfoz5} zsv8|i?N1)hB$XzDtgH7^Gz`5FGd@FoAmw2@-%Qo4uf<-XiyMtD889$^1)q|tVA5!% z4w8!?qc$IQDI7!v@>4*1uh$!SX4~4155puHPMP;KV!qocW#ZOCUhfQI8fIVR%3k^U z&D@$vmxDbjG9usI9m@(0MUuv^$=va)hJ)poM;JH+g)2|3M>2@_@yYoMUS7}TJ5%;v z7Z&0%1F9H7~>B0PR#}B-L!iWH=ixP z820;pMQRO`BhK`5iYyXaClY(}r7Py_C2$$u1SxY<5TSsH^ecX`Yx#;sfTH&*QXGbv zO&r!}C0}K{|BMjWAdGqsb-=@UG1_fLt)@hiJ+hya9>B94;4|05>+ZRT&VBJ#6~B90 zky!p;2;)i^?CV;ph{CmRk2Uxxv*Q`XvEMS6D!%1j^(|*1XVb;LjH%6o1xzJ!&T!Jf zAja1+`=V`V7G(jp(O|xOyaq+VPG(HzwsIE8hI+In|EKUjf~mjt!g+i)hg8wMED+|td;8i5u%$xBwx$*m!6}%NwpQJCj4&Za72IP z-FdS6%Uw)}*&T!@cI@+!3|bj#7>5O!LJ~BkCcRMs%dQ2= zKQ`_qkmQ7gZTYWxx@^Y!GfWo-dJFj2Q~upcpqQ+5Hctl6WZ(|gt~PeGT(^e@5XatJ zl+z}4dCQ?e<*a|v6WFP2QzaelM_S7h?~AkZ^c30@5{&B>7FY~2z@Nr9Q(#ROiAsXb z?Hz-w7kE1^D;F;PBLcyUVnk;mM977zMuA}Is<*;D--s*<6>&T*o@{=d5AQsr?5rkc z;Culb-%H$80gSHIK4r{nX!-tpodErBN0irS?MZ`vXsi%$yqMXFp`Kmw|9koGO@bz~ z$ec!%I}1gHatViU9G80(MFn$|qLw}mn_-A^rVSB9O(b{or+@@ zfw5(}H9nmt;2GU|dFaW)Me|k=;wZ#9?C6j0iIC$!Iz%(5*}t#L5sT(fNFyt-n5?L@ zh}Gy{Zi7SzV^B6L+dUM8`PtPc%*woDX{m=b3bkT?f zN>IDy<)-72@wG&d7Wi^H=3!E{lr2-5nISgr(Y29hl++s%>W|$x+K2ih`Ja2sH55Ze zqB?DJo*IH_y*iq2T*9(2&&iBgvh5Kt%@_g>4b@}emE5`04qygU(q0|$`yp!kC zndrvvyErp);f3duC0?v!B9X;0>zPc)pjCs;d`PdCv8g8gyBG7h5a_R(D~BVHn2MvV z$zf1&^01-ZE~4D1@npovvkw(K?HylvJ{euBaB`7o|Dnv&_w`tHQ|Jd($tZ27YgYQ@ zE{|ay5=5PYy|xUS7R(v3J07^DDc0xiy)RcgqM{TE3LrJ;#8jqI{{+-el15k6#i#4c zeJI7x#zZM9i4&OgKP9eT>V)qG#zJ2Khuxj0Shzy&jz*R%{B8ObAVivrKneSqiDSej zYDEw?QwmoSNN{I31#sG`HCgG5I~A`yd*tugC=o3Ed%~f&^8UFODpEPGuWoe2F}^m~(`i|4hB8 zO}NMl^OzOQNmq@u&h*XKq&IZKYMe8KHsE3;N!3JcRTo4!EB1IA28H;q10vutEHnpzmm}J$nYbiqMS%{1sk+X&_gK zwCqI}n__u9zfQsHjIIlss+IU(1IwwykH0m^o;|T{w`{c`_uM^5mV8M^3cVX~xX<@1 z1Lg2P4?#Ln1~9u0xivHf$ou`j2mtq!u670eGzcQt>GQgLSi)&2XI@hnm>-BHf1C69 z4Q3h^qYHBCXI32y0oS-^*3Pdg?77uwcPwFG6V327A5jp7c6(*i9gZ0byJO84`6W zBEFr+c{u78S3!_=itz2*jqAh)9TJwxJ3!JllXln?lCDI`B|zpUDCdj^^vp`b*}GB4 zV_KQ5@~E(VUQfBCz;K~cUHU^ETxt=bQ&a#HoPbCyA`OZqBYi$GDl&1}7cn51A5 zd7=0=-kR!&&Ey(_#Q+DSdGaa2d`kO@0#$|CZ^sGNl2Q$+nVNv{>MGl7x8ZfE2aUHN zQ$eJ4KD|jO!OkHq&l3H8`JH=Ten(YPxf8v|P1d9YVF&P^KJpdDR(8s`ftE7 zZ8XMcDs`8gq5|oYznBlQUQ#_*3etjE?@DqXS|BF%KS+|hA29 z1mz41trE; z`P@pmOj)vseEaDcSBLZ^Z#>(`UU71Wu-8J9qAoY#kwABxs!9<~hz31-wD6bdr~7w_Ss_} zrzzR^OY&1YpWG3JJFLbz}!1V;aN*51)U0MI>ZC@n3+@0XC5+vDn19 z-5e&R)#Z&Xo_bk2DXPecRyXwvs&k35ne)vcENk-pw78xp%(3-G` zubFx+!W0!eLMd@c2@XyB<%nlq#)JQSd8VHvDqv<1TnmE1vLE<2e9yS7P7(Jd88X%0 z)L>f<9@yI&*TNFB8HE4iG$XHPwSbNCFS!vVG?1gZYL2%i4@r@&*H^`PZ-oP#oC~(n@{vvztzEesyh35xL`?ScW4U zgTTKmc{9Uf1%>S&y))#MSs*lj;xF|h^sIDSUUORK4sUIW$ zmOCN4C~P$sIroOYB2y&(i&L%J>11-w+y13^w#d`DSer6ia?&9cyY1Wz(&?3P@)Her z<3F{`HESp7meK;-yYfQHBd>hP4yz;3+J9v?MDX(&Po-2~)$zS+LhJ6?#N$@;X`-`4 za-C)}P`O8r`*WK}>w}-Cl#?qbS=8unK+6{K9>@+)r>9hCLlM`#T}%*G^=g5brgV zmACkv!>o9}1qdFF1@dIF&vf``6cu~xUDy7!d{iWBwj4-o5LiL~9z_<1Npj7B5K;57XF?)QO80^M1_sP@JVXm7#O!7jB`=@ep@*IkO)6f{ajs)?P`Pgohs5j) z)V%&^u>af7FZ2hv*?~v(VU1U{i54vzr4mbq#^)z^4$N+cL8DEt^k$urJ%=+7MBFQ4 zYG%yBSxGwa|Al}4=cUf{tbCtxVbr@#(8rzC&-JTyY~f7(Se?=$i}zoL@6dVp<&J&5 z<*1+{!;2Tjq>OvG-cbxBeg~DWUgT$=-V8UPRn*Y86|UwND0b-cvxQuFk6kbe^WBX( z#HoDpjJ+_|`>tKk`}RY*YF{9Q-8d@W$>n)M=gP%hGe&OD>p-nE@vzX@P2(sjw^>D- z$qJ?4l3_MrRH)kFWQj}p5$iwia3Rhw@6H)!N)5?#dlzx5O~4HlZV#?zJ!gzeU0Rn$ z*AG>AL}S68tV%2wZ^e6}iSl#;p9XVqJS^mVUV4qZO2@{dN^N%^31{2NhSVQ;W4;FM z^IV}yiAg$aJ^xIzv9Dc|vsL1XCj3nmv;Tl&_lQ=Ymgg(hxFRD53+{J0FDlxn1@}Ec z@c;D$x|p1WD#6U_hf*$z%F^d8<2JEpe)kK&B zChB63#`3bS8~eLnon2rFYT~&jsA;GEa@@2Xd=Y6b!fMau3bp+8px>9Jelr^N=Sqbo zK~XyI&6ov|*J8nghgzFHnG*Q?E`gXP;nzc(2stk5Lc(Fimxf7d2{R;{nmR$EicCX! zd-e=J0RZ41WT=+m75x2DEvuGiX>{1m#4oZR!)xwU$; zcHh`__>+T5%n9v;hoB|SzMIRlDx1cPv0CAHMS< z7ibk}hTe5Or+TI%4PpK98oKLnvlK!vh5jbt@CJYmK0Ut2f!@Cl8Xq5L&Qoc6n9g}q zaup~Iurx49fZ)lPU9PY2juLrm2_k&pV^Gnxt%nlagjtLk)QbBc9nQIgol5dVu61buQgzA zPGeVV2;+#q0R{2L=7&~F zDnJ9bk=fI;!R5R*9{AyH`dWF9`~ZK+YP+uv#H&kV$0B1@L=00fu-&c~1l6qq8lO!5 z4cpfo#b;Yf5-oR|Xw~+U7bh%j8>#>HF;%muw48DhN5pj~Lsa8Lwqc1&S8RiOs2vQ> zHW+JmrOt_E%{Ry?znW&>C`(!APSlVP)n1iLy+~8B5zH(tYY6)Bx8eb$;UY7tl-~s- zoYPc3S>$v>m_~!|5J?raVt@R@y<7rwPJE=o3N25`zE!YQCAF`TtizN|H>A5W->`}I$)cA-_RHBsEp|( zuQIzMp938C9LOl_)X2JIB)y4UJO5_P;#NBsx;LN~P}YT`%Ql3m5W9s2D;x(= zC7-G}8m~*&w9{@e&rBz6&wYo+=6lLPTN?UNB~|-SW~`Mf-Uq_gW!C&iM7Cr;%JXu; zL+_b<*_~tNf`S@Rve`YfFkTY6e_>r{)R7B4&#(lfAl*REtdDFzt z=?Fr%GGd7Q=i7#f|6`!avf7^TdF}t>0$2(DgJ0FAuN+1EFD#VZhlL9qsqtOGOWWiY zQ(Vn$cg7lrD!mO>&EP!<2g~FAR3$Z;I|=b8s=w6503`mISca=0a8EFF6>ncNbVwbZ z=^Q@Pc}2t9fW_tRSN9q_dtdG55AHCd1qD(6q0nylQ%XcD-Bl2wc6(^QirFyuERtr- zBRY@_ORewLQHK=iYRj0isTIsP-0b0JFM+exmdcHj91uwq=h{ZWns0 zKq?#Jd&Cp@S#Wzr*RM*x3UrqZ3tw|7y#sotqT55iB_O%~eIQsrhz_poo7{4Z=%xfZ>r%z?@j% z=(k5R{d@r!H&=fL#TJ`=MXooFo-8lnv%RMJ4Pp|VIzumF`@3A9>GuB7!-%qUDKTT`8#$I;4`NPw|UmXcZz=Q9bFP^PdhZt)!zi| zWVhuc(RG~j4BRc_AfG=7P|Eo{7{F|KKkE^r1;#%v^^7H*xHf|Qc@JVCT8886s*v;A zB&o;!-4>^(V)Dn#npPr1KVEOzIiStQYaVDoqW#^LDxHNR-P$nx7x6R%Q-&O~Fy@65 zxSX3M^6*60vSRbaZAeFE^3w)?4*tl4PO6J!+4Z;5Np#X7tmLkE(!&creJDg2icGF=o4Se~@)%onVqA|$csfdPiu(LbL*|;#y zG)^ru$i`@sx@1V@1>Y^sEI42po3hTv;so`f$i~BtPvtsmT7+xxYUF@epb~Y6NVi7? zhRQZ%R$Aht4=j2bcn>sS06Vd}_riFxItvR19@aTD(ra!TEwn18!$>H{zr%72 z*8l_j+nVU_xj>eP(8CCji1J50x;;>!+sF3AyX@Mn`WlYqw1dLmMd|7tbe@xrH!ovs{o5#rXAW7N$ z|GK&yTsQ=xdY5lo%+N1MPv*4R&0p!fYJVKd@)CD}=>Lsgv!w5beN(oz#WNqRM>0um zm^bv!!asMK@uj{K;8?ZE#d?ts)b=!UpkfMKa?dy`8OVKl5%~ zz!p6P4Dwvg#@eYXOWsR)Z{}-ad;{{OvKvLVO-084v}iCIJuc~_H_d7XY5D4hvQuqid(?8uc zRxX_N2wC#jFl6}RNsvlNqLn$mHo(P5(BKpr*V8oNj7UCLNt?#Voy${{=y(y zejkoYIWHV?Y!A#YXWZhfQ|*n8MmGGH5uHF$7x@_4w4S+&g=t2X4_lEr6D6B)7d|ke zTFBkEW-Q_CKcOqM7*n<9iY#tnIlOJtml!T#>aj} zqJCN|GSlQB`o{3ytNUkIsMtdknv7MndwX-K(rS2VovucLDh|XeEOg3>A=o}FO|&=z zUaj!rr{e$NK8v)6LUkJMvdXj+dvSl`vqDFGVZed>fWp|Hm2GTWm@5XHFHb^dyRTH>KhUZ$rv!BA*Dx7Mk) z8sgPGj`(-r$JRDF-}@*S%f~bBIG?Gkg=*N=&+5<9Iq_B$Jbbxa#EOgPvc)`2r8P~% zv>;=L%g#ifQSLgeNN^)|f~w#uWUUp7&2vc1p-e#iXI1(<3)^h)u)Sg}rn@ewM8Aoy zY62qD5prBClz)K^YSRp>5u%+MbMg@ng`iIq2w@20V2T?;-L4k{KvC;Ki$kO0Lt zZIS<;%rn#b5$-43`#uZ=pgUjx>Aa_nb8fbuZ(ym}2RprIabnhcm-UvL-HfHd!~-e~ zdmGKc#RNdb_y1u?;}R^b`qU}a{Xt7!J=esOC6xJP$5mCZwrVv)&*m4?l-T}-@|3Hm ziME;xINQRDuDMJ-|E549JLBN`%Sk$1i9xqgu(_=>>njK-kUTSzIl=1MW5XO9e|Bfo z;*qx_J9uFs)ZQ;aZkuQK`wZ;^RxM|jf^yxvcM{_ih)Kodx)f2!^zUN1l%j;54zGp3 zwrSp^?;c)FEnSzDI;K)H*?gJdo|(b_+i^15Ce6hVRz&9b#a?fJ6;;*J4|4j=3AeVl za-LaoHuO#%rm9~?M5LF``S2gw zfV%dk7bX$5=VuUoSWejTP6z+?`mMaSX^hWYrlZ4^+W&EF)|KvuJt2j@B(_-ZXs8R# zPdiGTPtsaU=OwOw)@Ynf&k=0*nFm4WrMU|^izxZ-I3wilPyIdyaL^oLdYZE8r~b2+ ze|sCfZf)EMIn;3k-gQMyTpUFpZpYIeq(~lf=tE*1rLP|aOR$3&7@I>reTf?R ziQld!S@?W2M2LWWH6p1SWw39lKd(P&Y(Nps9WDwhzAnMdiE_(-^OX=!l|L2Y5 z!ylfjJCaZ7)@0h6X-&b>y*o!eH?y1ijoS+Mez*s}a{7kQ`~SFl%c!UxuML#$lI{=$ zl#=eDlu%Moq(P(^K!%iVgA{3DKw9YrNkKwl=%KqChMXbpnVIY{qnL-)^1i=h)RV{jCjJua$oMWzn?Z)w ze>_@F+YFb>{0_pBpk+kb#F&q%dSXK(NzaK~G738I*#b%Osdzd=>uK`|^(VdJkiR+! z1!BV(MDNlxg+djny^ubSAAyYvHivQ6zM*p92K|{kUAP@ok zZ<0s_lx2Rb@b?J6^uo2(L=q3AinTb?-<#B%wiDZ614f2{A#Sr0%Sp^~tF6SKW$c^h z^dSPQk~i-XL6S8n`js`4tVV{IO_fSsZ)APnoMR}JTdpBNo{A`8uB{+;MKZLXepCNa zgpJNQ^o^scI&wgqmLV)UlzXgAKv43=TDyRr`!b?`&VE!Ei?IQ7+7@ek1M0qXL^utU zrxZ;TzS5Zqpft8Ju>$S@;onc1-!R<~6?jZPB2|6f;yMf2c1igka=p_@FcR zj8pVBBN`6s)*+2nsnwB2jpyRrSeih+#_?-6vk4Z=(@oZyPp(!~Ax63B1qA_zBQ6*xvR+G6zXc)RZO0GVO@*elqRL6{e=%Sh=&Toj)6NddOwK?u*GzSp znck{lG1ee{drms&J~aL%=iLamG$rMt#mXIZPjs5HyK(eOb66a8Oe}W-FILI#PObFK znb{@5%y}UBGkNmuc5RbcFV{@NF(`cmj#|cQ?`ALpwXa;`f&Wf-9!;RUN7*`SPX~W1Dv$KH4KnLbT}q$l7D_&NH|=>= zQeD8Dm-f-sMAk#)4c+eV)hfkioJF0dBugmyvR%`U|M>y*LP)69v339#n7Ewq>PM3I zaU9AtQb#`&=rg|7IN>>P&>OpvEwsZ3TZR3WPu%OBbjkFqG$3aY9{-U4izy2!vdsQB ztpJJk$=B^bBUnp@$Pd-K+&|tqlc@F+2qXQ!NWv*ix(me4`wA5zKgTx z<@%0IHpcPt5L&ve?_{Lxg1`$t=0=^;6ayi2Au_xQ`mAzv@YkdMEcBU4dVcAut>WLr^HhEos};`UT|%4h7j?c&vw~(=vRvT_OjYr#_C^r{D?jI9KFmLa z(t(4;VTRNG?eF zZ4ui+DA_sGQs>7g{r~v^vgdVRf^^oM-7$}s5B!#k3!z%fCK7V1HIr_go`xII@IUT1 zp52;0TM3IDO`f}y`5v%KVYl{M)C0$iypg#pV$1PND#{Lwnp+A9_sw3b3gD6`J?K3@ zzwx#9mzr@20IQ2N-0t*$wr#yTxuiN;>THW}ZN1_ip=WChqTH)gyLkfUQ^}<>N3xMH z2y?&uL1*E{YWlA4l@BGZ-S|jM|7lC3-^e)a#`zxNF2X_{Df{|d z;|e;6B6|;PAj5m=ozLO^VK{RafMC%H&OOllR=($0Z{{6gh3y(L53~<)FW&K8UIbYX z^mYgC3ad$mwMu$MnR(AYs7|F};0={|^x|g9+n_xg;DowH(OX3W01WT#=KOd_!NKwp zth(y+qY4X{AxUye0ph%F&Crn12!iDQXSR_r;x78Ic!SGvLMV7Hgj7nhRz=`bZ z#xv*78}cbv@tehI?|tt8+mKiVGF|?mg2T>N)o$G}o9+fHtha zaqw1KQD;bYwC&%`_6Oh=WVHSP&)cfxU}m`SADDP3<#E);C#jzP~$&!~*yinT$ zQ)4qR`zw4^0w`;_E&W~#y~e2uLBt<$g`5kw3*~op0~vApYhr7vnO1edQF|}1H4|VM zpk0ee+1hs#4G-(K`3dF2{Kb}})wRH9LeP&_$n)tSGrQ1I*R;Yqh2iRYr(0*I$q5M@ zXY0@T6IF+E4t5WAV9*QG3gN$62Z@xaPj{b(Me4Ci(KV1?mYIMLOs!L3b5nHZ+tsi> z-*|}q>*`m!_;JhnsIsO{{~IFFG#|2Kqj3kVr7YB+_7|97&B^+U@-iTD_440d5=cz|>?L@ATqb*%wJ>7E;wcsv z`_y<9;o8_3s1-UaQos>>7K6x%kFTi~>WMv`9KRyhOqHWP^81`e_ghNSl8jX!TfD@$ zj)8(*DMjd%L!%6o{lWjY%$Lb5odrR8t~^ey{Rc*OM$sJ-s-FNdyzE);0faF5b*;N} z^U*bbDv$$`2xf(v6Jt6)UQZzJRU^m?+f2flO2DfM3E%<8EfEiR0~?Be!IY2As;b}b zo5aJ&JjKlTnQg#~Xw>>Y1d{sb;_v9?x}ka4a&Aut@@$u?FgiOnq<)S>AQrVCA)G9T z!oeeF!(|Js;dy-^XwT^af@a4Tyzo44DAbJ~=Gp(HB#@+bs{kHF&0xj#q!x zqp#Ch9wbWhs??li?v+z8WU+Qke0W2xa1&%+d7|HN6Z_(Ig_|Dt@x2;|SB7Zzn7E;0 zsd25y$29KV>j+n+Hlz$ zM5#1KH#zKaM_tir!#`gB5)|Ngpc~YQN0)*1+tdWM*xPAe}5VbC=btM zvr>KjBch62h;V@30(OhnQFw~+ShEXHV5B+emC>6WF^i)`>!BXH+7i_)l4EI~y53r@ zSHw`L<1=(SZ-D>eA`&;G9HRohjF2Cai+|S`dwgb{&X!d^nF3>K6@MK9V_MK4cgadA zP`kqOaZc>Ef4HlZ{-80C+5L^Es3YEsy`I9rOP|V^Vm3mmoSF7(E%EORjN-XX#X8`a zUO(~Gmcb?d3kHz_r+7*@UipO5*87CBuLUhQXARM-1+{h}s=pjlg33B5#e6VQn@4QrFl zX6}GXg@)6>aTTgQ~uRX2Ly?j!lX7YrJ+an9MyCT1s1 zOYt~{g6?;xBe2sFPAybfE&s&>a}&YdPZAcUu;DAXdu52TDJ0vo*aE)j=rjt*Bgc#NOAXD*k?rvweP z2~}VUtW{?d*W`5cUd}a6mX$7;r8_{DWMnbg33l89+E(7%M*nYeVDSQM6fT8yuAoeG z;0)o9-lCtO5lXsh8oxM8Z?3SO`M;rYwoGcy1>8$(RPEWQ> z4+a51|DJ>bW7DD!ioEeDg(Z?~52!HnQT%=mifO%E zd7dhaU;~JXph#EBjp8%N3Sm!YaI0hrY71~Hg__^sb*5P4|KuYwF4Sg`I`HdJ)JWO2 z+I)ZJADL0_M`4dZ@Rdb*?oK|~$z3Opg(-o5%tS1uWCh^015PV_a`4(P=J55Fts`@I zh$R?2*FfWvTG!c0E|2s6Yzqr8Ny>tYG-oUV&FEey==SW_FS6pDw-B~^J0mpRM?(FN z*HeyGo|m_GS%BiV&D}k!Io;#$$?0q%JtiF6Hw)rY_y?Wk)WqbOT^bv4&gXIiZH4*n z>*9DCgzr?sih@iztN4j+yES$=;)dozu97YK@ob`Uvalq>r zPsYl*ZPM~Ea{PJv#|LXqe!4E;ZT8|-3!iV+e!Fs^tJ zw->mpAa6K5NeEN(>wYI+-nayubmOjWt;3%>*9ODRgm)1)X0wn=9jH`uS*@HIEV?;{ zQ0})Ciy#5%BuL@TUSO(wWoA|j>2x(JO1H6u{b(7Leh8_O(|H@%n*Ll78$Kj{#}Q9r za^tutdW13?78x^A7Ots&(ft(`eJxiVLVPSb0Nsem3JOqUDzI}|Y#f`+JTw|STSxI{ zYCK~G&Z&FtB_IEC{3CI9T>X#M#6as=829bQ|0iIqxjuyP#%OjD$?;jXU96q9KW0T58SZlV`(U|Og&&cS_E9m2Rcof7~%dZGDSh8Rl&O9 z6HtHR&lZc0mnRWQsWk~-W)(32u9tc|-u+XEBB=u=3x^xo*f;x`LR`J6Q;ipX9PC8! zIj(Js&kjGpSR<(?>O@8FVu%$kLQ34!)p;$uc<=*tE|~BDG1Nrs^3({&GyKtZP+PdC zWk1D|GOxXoZ8Gg>LY;h+zdgoh3W2pLF>qrmBUe|@20MQC@7jA+AWdnGZ=5)zq3@26 zgI@)LPWlb2xvs1gnDay&r`jzmG(U9H_zEe@+yd9}mc8}a!o;261={&&>*U!o$})<@ zOV^r%{>@*u!j7*Bn$PFy$b&6E^9iAzlC9__&?#ffn0kR92?!yLWE+~T{fhLgi4_I= zch4j@ejl=-M4ORFCZouRFmf4oJ0Ca({^!L&eB_)AHN~4R6P}k=QSDmuGM#}aiJ?bt z-M$TGNUHSk$#!gBKfuA%)2;q!t%h5}duwucTyD3Nu&5mY&>?@hrtx`nf&r$6-g@x5 zr_J&@&rfvwbLd{RckUdbZ@W~dPt4_uCEPgfnU_GxoPXClAH9$t`GAIE zP zA2?7H_znj%r&sDT8iQ3e$FAMO%ESs1T|n>C+Og)=_OZ8@Ao&baU^ncASAMd;%12Vl z@vl@N*Ca7<67xP_ckE&uL)}(x3ak!8F7-|^GVuIWs+f(yiR37%M6^|PP3DhjfF-W~ zfhHhwnwt7i1RPL3|LKsTVJ_=ZbDS+`KrR6(2k)l^D>f@&D3hN3pBDgrjR*)iWHW$N z1)0K_xs{@f?U;#_AWFuU!A17Y&0gEhOUKT;gZSA?`fv~`T(0e1bOB^s z!qt>X+H2A7fjO$?Kq_^hHqMfXbR{?Qojd;wLtlu{$zj2!pfnyM{^U#Entp?R_aOpq zB}g`=&Smo(L7%iNYtwkiegBbet&4lvVcm!NzY1Cw$Ln|Z&W(OizNPt|1}?oS!rlmSCv0cHLX{Hc zF(v36mknes9^IA}MHmXT$*bzB1)jx>{F7&1zGQz`6n5&Cc2^GoFo z!N0mRXWOF79p3)jEJ4A$opw2)@e~4dW9E~kV~X@aFbcTG9~jl6cKa9dR~~+3 zSC^NGNwbPP{vlL?(^_uPe~34f0jz_SrIcex*^7^FsvDFOOqNziIJG<~I5@|tn-2>X zP-GhjTB``jIeGaBwfNtrvbB!}7H_`@G5!uh4GZOHXUcT@5J=EJUh{f=J@hN>R70r( z_F2dtAHX2y06I1=k0azBLM%WpW;7H?IC#9B5uTVU{{WK%i49Os3$wd(g+U8o` zmjhn};VK!T+B!60lIE>%9%$aIiuoWL@7iZbg8AhnLP_z?c5T9K*_+qU5KK6q%4&aR z&yvOOK#vW1x`#Iy>ztmZ%@IQ34zZ2{9W9{P!o~Q0dBfx{f;n#dI+gHQcRNv~*ZyfdPk46f-sHen%Yew=mtJR{D)RjA*e#RP1gpqmHV*6e1!?PoWyC z=}2HdLl?T~{(PlXWKu6^sWtiIC115ElAJY*@dw1Whiv%vH$tS}Vg0Ypva609VG#6+ z+W%_8FzwGD#Rj^ImPT1I`8kro*VpI29{XI{zpj;$ecdx83lC#8lTqqtV}@?~5Yh05 z$`Ivy?D0GTYRPdtaX!$v#t<1`qd#hln(pHC->>th56I6VJB`JP@i^mDqKOOyaJwy} z8EIQ;Dx7uyhogRo_e!Pz`yT$Gj)z$tw%r9z5#-;KDR!Xnn;6&zManNElNf3&PzViz z8BaOH*al+Sw9G`q9U@g``BO=9(#edOrPb1fVLc5(q|?k2E7qe)kRj6IxnQv9^xtEL z3Q`M1w(aeoOAt3`kiNL3(o8hl$rWd_0)y1P9dUnN147lM2y_v1K8`GZHPu#PVgZze zi4JmULk^wj*KJ*~@Q0g-&C16mEvn@#Oa!Kuumo564^>PS#T1OqdebkA{AT8?$a2#4 zSw@YMcVC%cH+xBajkrDdp`;%$1`2^9=qcmF#NUjXi0Y5gJCHR1&7l+7G$L#wT~U)Q zUbe{bmSjCEniK3S)M_SUCr;z#+uz(L z2%u_)`xzT@r{C_=V}}H(XT(ddt*!)bL`<76-IW~!F0x23Z%@=6YI9pKbc z=bC!X-70xmW;EO69#GEdnP8VsI@$q`*(FKXjD!VpI?47S>ds5{e~~+&QpxR~$<~AY za7rsPy(PkAly@gQV!O)GQ7^p^xkmuD@}-s}&k4Yw_1%)FjwgV(_v1~u#Ry}`h-8pi zXWy$Qk6Px8#P_B?p zF=exvX@%Gr3|{pI4`l#>N@bsmmQu?I?DIwPSQjrGQ=bLt;daYja+=NKHU&cOMUDshItG6FsPP_|{FH=P?jo4aiKR#$*_`AEzOf77@WH|qVZrP*I*7bsB2jj_eU%i z<)koRO|_q{vYBA8WY+jc>bxP!!}H8$P3ItOJ8)uLOMfj zk!q&`O&3bBmKz4tvGDY20AM#4?(^8qmEHL8-ik@uaBgdh(TNsP-lAvT>bYPye=OaF za&W%%7U6!L2>LTI5O0dlmRQ!uUy_z&6Ey0fz5Dm=MqYCOK#XPi1oSt3jmJ(rNY|LP z%XBzy(97Bo;G)G(YKVp#-$~?F+Nz3AU`b{=K?FD3lZgXV1CGxgS>ZGfne8TBq~X|d zL6SY6ZvgW2z~EOZ*XI`v=d9`EqfbR2I9ZZWJ^54HZ6oi%P7V)r`kQdrrV%|VMkk_n z-Sm^J=G&$JWSy)s?rr_L(Y>o5YrhRAJ*xz0)B5MJL%)~rfa1+uFJYHoVrHgQ(IpRO(Ifw; zV^2MHVqJX4_0Sg-&AGZwNG#H~AW)m%p*0y?k#GvW0+&qQ9i?NX{E-2Z@)_7rtBKsu zGQq}DFFWv3wfCN#yBkepq6}gDGL)eitfD+RnML@-n)2adJPGNO&RyQc8>iK}(mVf& zh5=4Mhw?l&T*aW=3Vxw(|CBZxcP=d8+zR}tYUj?5&YXl`o3}oW)srilS>mxPO*W=L zMK#3~P9`Zq`tO?s&pzovM&;kAl-0z55m<-6qETI=!t&R#oZBo?_j+~@E+}MURMG;v zc+-!+bj)*`V02kM^9^&}rD2kRe3`-ChfV%m@bdj23?~XN32vaXMlknt2xv%|uaM_DzE^LV!jR zWo!`AhmhZ>iKO6W{atFC+db#0ijG2uBYA*cBB(!o6rw^7>ixOjt(9RpD#R8XH1vK_ zhR(cWN(-RN!->ogKG}g(hhb3e~eT zvUtN${nhw($!E(?U$RKQFjjO!_OLZ2BdW$}Xk4%5y=+IU>;Zw(WO!gEH6 zDdGcWX%dS2iRa2vs%=X3`FPgjKVA}RI|YcCjxOI<%>Hl~p%_cUXZYKK6-rp%3gVG; znfq1UdOib*W6oRk>tWt!L3+2{QC&MA7e(Y194AXK3u{5RWsNJ8Q%QkBowB)~ehC9o zmNse%V$fcsxJ5qw7~qh)LO&51n4veSoguI%y-~P_L6lLfbxAA{kc;#Db|$rfHojLq ziYF|>#)o#X3q2K8Vmkt*9H+j~NaSJgHRBc=+!GpH0=fE6k}S`?RWp7k%C~+XtD2!* z_U_rBRxm*`4A8K|>KUxTpPz_kpwg2c3gSO-)5SP69GkNJvgVsslVod^vi;>`1bF!h zuf?fy`YiW$3`J93$p1C2FO}ApPuzyr$5xULiZNpeWF!B8@^fCNzph2DT6eUq=1*i2 zJ1&dV^-IG~&xG$z0__iY>ki+q%Um)JXgR@g+K)yS#_Jz)Zz#IqnbJ{~bwR#bQq81j zBCjxTAA|NW>srB{1sxQl15mGXWOsGxYCPnLEW~WXZm>ve*D&l+UyBwtt3AZtBqayN zFF_@JH&AouaB*2vJwbKH|MsT8(g^{U(Wxo#x1%Ja66rA1to<<$cbbboVj{|Mg;$FWYSaY}w<&QV+RC;)KDy-e$qiv71gzs9W+tDhY} z3e1e-;c!#0j4I-dmyL07eIWxa%NGEuqt0|3d8&mQ}a+>NVEGcnJ zdx>gv6XIhDsdzf9v@()HSnXR8KrN+sglC}rnC(%13 zmW9CIX5K1LOclts;Zq>FUsk?tiq#J?%9^J3Y*{vz`W2(l<+NiBgm_Vs;F+&RWE$9+ z@7_CYO`qXSx%@h@%;Y+DIgljS7_>;u+m-;6fsQbwkVTx!s2&N}P1IwurlWFbAo3w# zbkXub&0knbWe_Se>@d$r+JQ+)fR6KthpJSnQ)fXA4r;$17TdjhA7z{ozTLpK#7LTpB)c* zYQq5ph8|E$O|`tBxa`>PsgbEq46>WCKnjn)rUMT|$@e0~iI>2KNNtqO<)zun)m|c? zIKzWld3{RFAo~sV>(fzX#n<>ZT+82X4xcDcz0n{+b1{($G3) zvDeD#WIp9$E>25?e?y$WBAf{SEBgG8rs)v@D`{`6%?(U^;s%d?4yMJ~y7L>e%uW)s zu{nk~Jsq2=&w5~r+tUvJFc{qj$&r6@-cOqEuCwAqIO|kkGiC}jIcHNBNXhY#l~rt~ zjBg3Rv+-ox{glsLq=QDYRzrVF*K$0wWX*m}>oON+J56p6qKV_BGW zH`s{v6ekwP0w1QwDziPc@_$~bNpQBp7CMzc^!>FK_7!9)MswoYV1J2rl0S(T!~PK^ z(yU<%1WNH-aQ`uo#xT*&Vh!vsLNko0*Y7f&LlMtnwsmGU>p74$8_n2M_G=y=@SObn z+JAr6(dZ`!()dMi`1YK4htk_N&|`kOt{C8>&K(KO$Vyz#TE=$I*(8Mq0P*Ljtam(|Hj>i^v}68+%%pt>+@XPmy&?7>3tK#BESkA znA+a~m*n$^g9OvCJ-hh++i482(4P!S%@>NFs_Dvl=jHu6k`=a%z2{B-b%`G9*@^8N_(94pWvwTM+$vOLctG9mL(;T?X1d8{sN2WWYvp z-l=cWVZ$=364r7)AXi80wq2!TdAU`21Q=g6=%*qcbqIDWRe8WRi!x3kN}KUXbdhE&%R!e_txlk`V-)I*&W~xvRuogTKbb> zp}`q7b3O%x^#tJ-q^P}lUc{6UKHwQBp@}*Ej>W*cgv`mAd! zxC?Avq6Mwt)&NwMADPn#PLsm8>FErOL(u&`78PJbD`+s z5aBJYQxL*d;{;=C#xq7MSp$W`Rj$SOhb9MU zr)lMS&~o82J`4PU7VZ8qn-^a4EDh)Pp^Yhih2t4R(s0oI3$|8}sT-d@+1mxl{Y8-F z!!e^lm;(5&DpRr!(rvfzxZikC7oh)>urZL(ZS@D+wdqf@MW1xNMO^Tbbx#`tqH4l9 zbE~?4*__4>GR1COOwVd@ZgPmn32cLz7f$@z8PEqqDgS>N{-?N)w9&{~YTp|6&u}Ng7hl`h@_i7=ts6%A zWEmOY7jB#_J>_|&DB}~&#ZDxEZ+Kf3(^qtW{k_5={y|%8^wIxRsK+U+WQ~fF z^1JMU34c7_c44&?eDkYlZ$MrGTJ~<&Ax`Fp#r0#_=wxoitjn*09egX*0DQ83rNJMg zFUuFShxa3Q|6^^$O90`iBRHqt?v>IIQwy5@6k5)v&I4rBi@ht%Z@E)fp!sUw{}NN7hs^>~FJXIE>kptV>_D zCYVoBg$S5(dxB>tNV;-%Nmt!7i`byZk>~~cRB<4=>C|mt;?d@6HawH!tHJQ1t8hG1*;(?Qa10;;vH3N3 z^PWz>>spu{CdeeCj3P0JQdfns;mOto4#--AXjfg74F2pm1UNMvl*Sh2Ibd|;d|(aS zXohbi?I#erZF;1;!_J870hwdciL3w2v zz-62NYFRz>ly}*~4H2NfB9~c% zoyC2~ls!Ja@u-FF>=1CaOz+#>6^VPaR93qps!;aV%%dS}CzpbxVX~!5c+XP;u*}|{ z_K}Pq-&kjKg2Rvx7$;IJ0&(kOSWT8~PZ`A>fh1lNdL`EsBU{)?+q@;s?OsTd(XPZ% zq<9W1lWf-mO8DkmzJ6@EJmGt6H5yRic((I3p$^Z(EUYr4#A3;>L zu&R+-Lvx}{C@OKBgRZ%j;Qcv6Jo+b0LE?~ZgGnjMyNNmjT=`F3+pz279!A(e`C(kF ztTT;yzkba>3w<)#;MAFVBavKM*t z$egIgb&-`~Qps?3Xg?dg32B+>&OpzBzlH758bKipi3KSSSc4w6o(x(INCD4C(F~FL zhZ)piTiYK+7g0=zMos;%)lHDFSZ=w7OQJN(VG5q93#dBnVNhrE@p%B?tNjZ6c+sBmR7osRE87?@hXK0 z9{If0JP(mbZD*9Q{P0a@5I7pF?ISmMy6*&9fA?0KW5aMSu3W%ew7}Ff(%&I^a>ZVp zGe_MgE?mrQ(0O(3xY)d*QDOU=!)7??Z_BVXU2~4cqj&FeCE!!y3I?*w4VEIMv)T6x zS|;$GCDdB}RlK=qP)IoI zhxs;2tb<&`f-OzQfZtrcpHyH+oOp{vw*>t)g!JFP(@7TwgP3MYR_p{!{0e zi*~FGotq_tCOowt#m6g z!xvt4qOU=#CkVl}2Q^m^2X3d)&9o;!t%c#$P=Cr9a#m9>yX&;P!9J&_B#@#=`G|~OV6;I~4i~T+RDx!ISDu>Aa#d!;t59aJNJ0Gx3 zxCVhP9551S!S;piQ~&1$aNNXTXoa}$swsUUn#uvkJ{m^ua@NGgwYlJ|=xwJBUtqfvz= zVxttJxs5_B5^2wkJv zgz{ovj$nE%f+e>`YN^nDK%!df5h=I$xjhR>)4tt_Ds4AHf~^>Wt=VRJzkyN9IridH z$9Jr0v_*1yi#``Hl6*pM2`_D{-bZr+iJcy~9z9juJU~bRx>w10A3pr3{BJ=hQoM&= z5qO0XN3uYh8{&hf>sSJpsWFYJ=|j}9AzGP-~F0H?eVLrUDKY~7M+^S)U#zAG{*V7r1kx_-|+$U=jAuVc*8 z>&shMZD(WkI)}no@Vz?3jCaeFd&PcGRjjjkUMW6QCPk4OjvULH_K_c62n~w-mf^p6 zzaiXdFcrDJ>h)(h>th364lGb(cLry>B2%kKD3*M8u7=0yDz#mKdD!L$%Pf2yf}zZA z3L`5J742*8vlAp+8#|H6e934J^WGm}fWbqy;uR(_n?V_y~_VJu- zy)ozhl*s6?i~ddUVq;GoH}7}8$SUuF+aneG#cDgF{2H{uWWjqeS)8`~-nP`XRmS)`nl~ zBG*@zPhHLaZf44vohUG<_X3k(OYt?tzn%OZWpJf&yxQdV-;b zX;duyyNH@A5eNVYFc#;Jf1P>5l)|vaGL`~ZTg`n%JkhMYAl3<0O1z& zevdpj{-f?N&Y#l|f8eapEWj>g3G;-1ur&YUb`>nMRYHI(e$^$IhsVqa_|Oy&rH|75 zSpB+d9!4;~GDjI#xZOKt6TzA+Gs|d#c9O}{A2bK>0?a?(%Mxg0@68E=b+`6OrRC}{NhhGy;H`;=nWv+Kd7S|t0 zV95;K^cgkYF-r83@6mH`8k)j!Uj;Tj8lk6n{n$fs@d@~I{=Hb1U1lp;hw=cXLu~6v z@<3N=`1~pkmmj+T5D9^Hb zEZq*DMLLP%-wtPIy+;}~K(6zeV~(T8OjM$XVNugXHs!#uMFA8XJ3`A=YXBh~o6*R(k=Ve!7(J$1- z^&fMaUKhz&ju9M7v3={w5TJFEHek)+yVP=BdUWZUa{Ro%2dH3t)2O*JSJO{g0V;41 zkkk*lHV=%$ON)1%=w`lxn<8QZ`mXbr>F@TfFU1+ii^&(S61g?XqMYRBoMBo&40Hrn zdC&eVy2Pq#4QOl#!J}}*E=!*O{cQOdr)jv`Ar$*4u&4#*pSBY{nJz$Q1e-}G$BjHP zUONuhjhs>El-*FW#Bb{1DX0S#(`q75YVAzQrxfs;)uj@C4%uXmeoLEVEVmO&n%!8y zDU62*k~ZOtFWG|GG9Q-Bc^BRCM$Yv&sut98JN-?L!=Y(d$zZ?N+U+yTd^ujcKSyZf z$H>fyW8ZaQO10EM_f@ID=eBTr&s(Wm)9-7Y*Sc~2{Ho)j38GGZ%s9Rs(jAfzbjy@t1W!j4>2_VQ}N&2X*<(EduQGvM- zj7d_*7=IRdF$5rmGQvdWu9O0RzsYO+8;ahFp*4kYJU9R-2dlTEZ)d(_z>lOR2E2F&*yZeO)Afsi zyL?6mAl6?1u@ELkzSE1aHRqJ#0Xn@s>P|yEEGfjY^+^pL==Q)Z%0r%)9pLQ3gI34? zda9ruo8zx|ez_o!*PU^o$ej8`OrS1f$1`YBGV^SeO}TyuZbM1SK?c|@9s_MWf3HL* zS6jVFJP!R?_5jKtZ7T;C=>NK`*Ixz_{2QflDr6vDy9$K^RL`2FNUh`KKu!6t!vG4_ zJ5=JwHxkm)MVjRD*J(2O7By`Yg4;_$k+U+_Z?A}eBp)+>AS!d* z_W@zNc=)=`*d5-*-OdPCv4Ia$xb#^|vFzS_k=25`QMH$V+!f5oZBZpoaZ5DD^##Fp z!`3w<=M3A+TSp3;>@0V~+D*xxC;057+%O=w_O26#WjfF;}w+3jY3#Msu8?fnONopou zWgrnO#0#$P`Rc`jpWhrUj>;7=^fzeFsdHQ1lPL9%4lk1)$6%PDclNbk?L_q*?K{m) zlGHP<{r>v7kE`#negY$n?4bo4cHO|AH}~hb!Gp~Go(Sp zty1Ez3HY+nKM}jD`TpoVIi`Fc;>LNaRFo3X^Prq z)n7pHAx(Je&%t|apJ6j&NY9DKY>BV%jr~mIrzZ=XXW7_ zzA5u9^qDbrLRiV-*NA>OcOhF`uI1YNavqpf4<=uKuTCmPkbZd8YeiLHC$&G_OE<~^ ze-tG-Y*$zP_taE#)27$AbfT8jG5i7sZOcz0lu?CheE1qJ>#!NS%}+8q^t!u;xeD5N;+`pQJdZeJF>6dcx1G%@lDDY#<;CBz zpQbE`W%;!pP3nv15hEM;PpLdX`~B9G#<3#!GeP9f3Nse34M)q21c4&Ys{oTjppM7= zX285+T?e@AX_|#{>gOi?+nRi6O2ieAbjS@jm9MFLFwuOHUPv=!o7g7=O76rcAYDiHP)|@UaG3 ze2-2=bv7g8++uxb3bCqat4;5W_vgiZ=`Eore|yq-wg2$Bz9JY>WDg_#rj*Mh?#$lM z&cLmU9i^K(u1t@ErDPglkT>Jc4D=&g8UP+wES!x?bB%2J*2=NDO~!JsZd8dM8PL`B zQ_Pz8tx34LM{^aGzqU}L`qUN3Pck$I8wo5f6X?&b<75D~w>V2SO`LHIqD@Nh!C?UQzfQg^=R5jNf z$8N{4nU2<&i@uWb6boNcJTj&<-UUq5A-?wlhU?&K>U?Bg$R%`Q=50Wx*T*l`0U6~mGdL~Z5TnKO;z-az5_ z@3K=rKHmYy)$SNK*iBn#8|>PS)em#36^Pp8{CVd#g1f&IA4WFdWKp&R6uG|O1)hPT z|B?$>PvXo}aX-BV7+~I&zK_N0)ba6T*E2vB*M}W+x_`nT%uRyoUK0=1(Vvl*1;`Rjr|gH7*HLb4Mj?$as&k%LcSRm-&?ma5t74ORLNVOPWrrEsXV)Dx)D%Dj?Z@)?5??l zVcc2PD&GEazhv^<2P*z>wsEp?zcS{ca3YvE)!JCJ6;-k;-Dd~wr1yt)U9IdY|I`vt zpwbKr$~?_(=Au{2`jrEFaN?^}#$qDaEJT<-dalPne0Yreso~rdIahnG1?w)W-Ix)r zxmr)x^>6fAtH#-Hs|%1mjb4LQ7HrBv#rdEQAJYacRAgqRA27b|u8S47C)jVCX|zip ztAU~NlES~n(}MgGN)G~B$vp;hUn$IpHclPZVR~w2HVJ%qf!#B;!xM+BW_aO! z<{zYFX^WVj(Vs^&ew#$>{8;6Sn$stR&8CXDRC(A|>;ZTr^W3bV(npWHr0^dI_HZb% z6da=r81Owk_S~$r6W=uWhcKQ(MC8cty;y1bVih>eu{=bWxKs;YR8O+7x5kks*@5%bq@CUVm(^$TKN_pEQ;ylM<9pp?;eYiYx(#KX3S0=v4v0&zxgWzQ5sYr_3_%^H*i2== z1|s!bhRxECbm`F_{>L&s=HIcBo`&Szsbs|o-NLT^ip>5$WW8lnRPXyeETMFFcS(tW zgfMhV2#Rzo-3>!YcXu}^-6-8iBi-Fy0}L?F!O!>iy!pRiEnct|XXf1Z75m!P-ie&b z7?Tr^xhr-W;f#s4Z@4y}n?*pWuk4>v-@nc=MhnhX1?i@M1V`d+%uYG|j@V!A4HhgS zWdHVa(#12V7sjs&K4C`oHX!G{zFHR5PNxF}a;_MYl?D};W3U8!)? zI9x>}$emkA6>}daBSL!bb)RJOLb27Ld=<<#_zSVGmiODLeK*gTD$y?X8^0;=-Zr8xL&&ALkcUS>DaQt{oUXWI4+G-; zwdT2_4tQmBXtGFAjc>&T0P7Bqbkfdyvz0pMW_O9mE$3%R|6TQ!s{43b!}E;DW=>;k z1Fsp|I_taGKgb-lqlf4aekZPAXL;Af&X0a{ai4!d=U@j*Il{y9957luc zBl&RlRZNv1DgPS$I(g#DVVd5gO=1BngL!syY6{{WqL7c{@SYo9grvp$ncht2DWx4%unRhpBGVLOs^J-ki?XTk1srTUhaKG(;)&&p|ixkr@ zCWFxzfR&52izte>$Cf~{eRC0YewIz7_lL)3rZdXSgWu`bJu3gvxPL9?F##u%DG4_w zA-i|~;S&!Zh4&ndilB#E8unLfdrlHyu;_M|IE&j$2G|EYtlge6S!lCGKJh%-+QBL* z6*H88efzZ&`+P%ycO*yNJ#%)*Yq^#SIQqamb_6sUZM^jEW5jS=9EV2c{bd-H+Nhqn z;$nRYqnt#61!Q_wzuTIwA5b<1$PAxvY^JaQKN0e&lJzGL>vRr! zB;bKW^`kH4qhG=^9wZNEd|4t2xPtG)Vs_u+A0XX-{&3Gv9t=Nu725be)PT?8-*T(t zOJE5E8+)d+l2X`1R8HgJ_WWqOMdx6c(B$`*R}C9TeD_04Gi;C1m`&V8r-9r0FE zZhQw0ard{IzA>-iqJ*E{?(@NgDAS*)(mJ_N3j8mMyxnw?Lp9+Q>8ZQbB^do1OCyK) zE#6w+1Cyj*=QnlIbpo+kO*8ckm3QKTB~c%~Y1_QGnScKmo|N7SeipTyB7J_z^ms$~ z9bLJ#xAhAwz*3!Y@?eBV1jO5SNO2w85i{U7 z$GPlyeW~R1hk5hKmUmI+%N9NJEUz3U2cM!})J-8{zb3L^^cg+7&4USO!N<2f_ML>_ zFuCw`1ewW&LFdE2BKk}f^&QZutS>qpFY)NRO;73{XA&#T9~h6pJ31%UpZfV`)3b$xz?EyYK^wyV?W#&V#|{ZDG;)iZE`o8xslP8zc+*d`-YugQU4hus z4Y=fYQ0X6Ab8df&rYcLJ6aN=I65-{BkAZurlTq-^Ykm}QDyI#gSqY-?Vh6s$PPvnH zr%OS@#nr;D0JBhv(_+7IaBk!l|1Aq{uavI9JcFhJwmz@Ul=Wo;Sidcd;k*$eg1tww zXt|2U*rp~gT2Rnhtl*+A7q(u}Ih5W3L=6}}(apBSn{YbrQ;2Y4z6ATB?qzn>zqgzz zv&L1LN?y3$4KME(XJ@@v0HiOUtQDeXKR@eo zctJj}b(o?4?bQxn4eD%+3>pK$gqpf3T;|~9ZCWUmK<7x$`hIISZ5Z!tS27S@3_t9) z8yqh1kMBg0SMyG zz^w4a(3diFrP6OBq|Xas>!!!vsLrlSTuolG2pPqzA~&X@F%oT+??eQJ8)d$E<<8Ja zCwH~HM61Qj6kNf4Z-uog(jcJYx&3v#s#C9Yvk;}~zIM!D?%1^p(Dmpf8cr+_SrYMq z{PZ-09v{-1XZ=z{bS_AG^C|?@q!OqaO)AYJhzUxDfmcs;0Ukd08U3<@$~?M6SCG%Y z%kT}*1!H{4qtQQAs+J4bKlCPi8)o=#@{IQKRy!!tvzo`vUM-);()>s4`6o{WcU+0A zefHrN+oSN|{Yg>g%QHXcThg1q#7{i=WWm6%NA&_RJ66f=DxUrg(dNkz>$ez=P4L0a zX7vW#HotcQ4n}9MOu6UpN$(2m6T5N17ZdlLF62#}mWx+6E-RIlesR0IXklku!^VmM zEdsiIQ%;)@1Mov5V9PsIG}T}n5bwskN20&|-IVhvLNqPNgyI~g{c`0D%Q!Jq3-Ie=I=H2R$qApI?EX$`9{ z7g@YtOS8k{sWe29v=?J67+tspQ)6jStk0T&k2B&0h6Mh8*sE~o+qcpBUF1v)N9CaJ zxNYc5b-y86(M;^ zb-{}>Kh(QH$8{N7m%mtT@5(yX89@to8LSQDI-XIiKKDW_CUTyH>z73LOG=xuj;BAq zf@~3ybDOKY#*BaH{#{74LxUNGChi6<*%BNhI=gxxfP(YITXIRD7Z5pyHFjC;~kdC<*4?}?`Qul8V2r=D6m zc{>y=PqR5~tAT(`ZSYGR>MF*$CBSTmn1U4S93U<(eob9l{d1XNcQp;zaIJIbMSj%r z7gj;XFzedHi$K`TA2+~6SzBAf(3jwgVyL?j7sIc(lhyV#r)JfZ5Hu@mJ6x54j<*gM zx<0hHJ*yKoiDH1Nf;UL21@IKZ0d-(E!YnU6XiZc&%9Nj6y>PofS7X8Et`-MCR$NJg zNq=RAt0bRj0UwSpu20YIxGj&_3U00W8!VL=0*jdCYYlRwA7G2C>#?g(lrv?z=>^TM z=@pq3ezju{v5-d^KH9R)(erqNPVI2o zpX?xjH3-s&*Ay7n2gmqJ@8tY_gfE4JuFh=NfvzEr2V&#jL4^q0+Pr@V=X^#o&g8$h zzNfk~7%mE?dUZT1K&B+TT>y5*A%}!>whwm-?u_bVFU>Tdn%MR8fZae?fVrD9NWU?dtmoR*upa?9>G9^tQQjPj97Ol<&>84iIl<-2*C z?x8k!i_s`32BW;;+_EIw7UV=_KDubg#$%w$zMh@J$>)^kT|kh{mrRlb=@vtoLV7a) z{(fL#`%tEBI`eSslx{5=F>%i6FM7?x@#R!wQ{y_))e;6&%7xMP=AATfM6tj6*Jc;xBcj=i`d&r4z9>v3;u86a#bZk&6w`s-p z+b~=q-b3q-_I)uLR50O)*8E951vL56{E|iAeB$XhIwJo5Ui)Q7m~3!Q3G!2tK$;4| zXgsmK(3>J8O1B)^;g6Q~n8OTDoe$K~x75-sa5xG_{EUq^0~Mb`Xt!l1t_znFuoVZe zSa2QICWKWv(pP5@CEcvGrNkXPQs)Jz=CF|qfKNwDK`d!Xa4_d`$o^szE7P7^5QdF*a)S6JD<5()DlX8sJ+FhkslJO1Etw0T!t= z&-{)X>Xk}8mH-8cwNz6W+$P5ZoDl2*?epSabgP>4KD!$dCW-d8-%~vx(#KiW$}Wav zJrtj;>FV#C>8+4fyRG%$4%1gy0T_HsRo~7Dc_WM{jK|uERh9|}$M~WiQOPd?#}dDh zLdFtjb&y=2Zcf~>1odCQyGf`1S{6!%kBBA2o0{+hD-`d5+9uU7dziOcct zT6ZN9gpUQr4^kolQsG%apGjKBez`zSme(UrGY~zoZVQ~{Z<>5yQ0Jzb`}DOeDCIzw z_D+D}yLsXu@-q^C@E%d_7lfNQ1_eF;Q^geIOG36`!QC=q3hzCl^|{K3I!sQl3^eBi z%u+HZ#2+%iG5OIy&dv{?iQ<VUk)C<8hrp!ks@4B~hms#Y zN>qY&&4>acmC*GMaLj%Y!!rU~vW%qFsnHm*!sZ_=Kv8Ho3cT6k0fsg4v}}&wv!#VT z#13qcD_w=-5Ek|clC8}cW(g!R(g;K@pjkp0BA*nAo)&M~^?Y@{ zP5<{3Yz>MyTd3@^>&~o-P`c2Ng+p`U5K_gqxlStxriw2K-9aMiU(ZJ8tp0$)EVopk zg3&K)WV_5%15GrW;$IgPveg|Oc{JB&YpWyL9D)wjk_{jj5+8=kh}Ouo0K z3nUy~F|OHj!eXenA6E;U>!Gp-di|SGq1=H-<5~Hw(dPvgUuqfb2di_pyFFYsb-Z^r zXLWLhCjZ_(BvhgG4))@RH(5ed6>~-$R)Q=<7!&Yo$qwLX;F>ikQ_1sB)0d2NxM(lr{>~o{$I=4>+ zfR{=~XTWfAPJFZMNVg@wn*_M{@o*)TX`Dmv^V|ylP3y zUt~If_y0$xL*6FYi0(vpqyb>#<}SYtdgK16@>_rRE#;3iVH!tL7SW3e(=HZYzWi?c zra$9SmbE>nh1_FC9IKqW5O1&PsflXq(|(Q}g5sE~9@(@D<4&D&ou@>sP_C)q7WbFeN~@Z8R5+jeGhpOikhulFWfv{CBk z$V5}xo$X=;h(S*f7Jc^oops;4RwwIulQ8O^8@nBy&o`^(Z}QFMFzI+so4Ab?r6*ZH z_S>JzX*s?du_YBR3LbVm`6G4h;Y9`0X6Ox(UX%M92{IR4_KOIA(`xW+w0RCPT@(6( zlRRxlVzzD9gbz0rtDVZWx0?I#n8KbnzAUr)2mlMhd)}t8w$Wi*z6tWQ+Fhr=j57A{ zIGP5+`eZhJq{Wt#PXB9_&PO@y z#s#t5*zCe`e0LQ(ts+V6aTK}lSR*to=eB66QP4^+`DW=)4ROom&5dDJD&7)XkNE5F zB>Bgd&#RSr8V#Zj&EKhBlhIOX^?Z%V8Op|2S2ghqIZF6D{($eUTWi_*fo?8OS-qw~ zns{WNq()jbz+6Ew5FB>byKB}|xT3tx^x2r^ySMBc>2R*kp5(vBA3jmUVYs`DgG8ys z>+>@exN_-P^so$(Twrz$%dJ&M$E z7No=pt*SxgZv5ul$rgVnk%n%k@4~|(qg_@UeI_WQ3WX(h)#M8#Hp)@x?axETGeax= zufM}9f+;KTa5Z9^EXC0tKGeYiNdnJku_P3eZNssKAGr-RXf&?1# zn4KA|BX6m@Ox}INSVQB9jZzyMf_q(=z)aOkZ66-;r&%S~Spv!rTxl~NM4giS>bH}> z&X)B%Y;~)4R@sBEcm+a@C~iOeGOjQOx4z2d@An01cc!)L=H%-&eW_KNSVg;7efi^+ zbN0WtE_z94K+~vjj({&OTH`&h98wm2$d*=LV=WLa+qc%~pD&KXZ91x>uq77G#7M0o z{h%Hjg<`sb@#sof!eQQ-PpYpbjuA4!S-gV8r}&=g+nBzDN5u^d9A(+8s+xS`w+llCo>|`ts?I?6b)rb7I4`YS=g)9!m99jm6>Qt}8k`O>PuBj+;X#c~zF_ThAU|?N$$79zl>qP?VXY&wU*W0T=TFs9_IM!w;Jrstgc@khJW8 z_h*SXNJWZ_QCot3ay&t6-+WsNJewSXg;h-coDDhyq6UIK^di|}PZ3~`gbUsKM0YFO zHj!lctoL3aZieW(&XctHbT;=*Ok+uOMh#Uwp*)-``vmc=f@?ke;n13bKL%{&jCqrS ze8x=;mYo6l8DR@nY8CZ2l^pIEHurbz%B)fQ_Gh;D;yolZ$8U> zmiEnE3d-45KH6emvxW@rTdL>kG)mMQ`4jt;UzTiM=|jLxLSqKc50rOn5BTQ^GbKTF zoP$-8v~-JHUB!$ZG+rAfJaRnYI4=Z9FwWPv-qo;hDaGFNwm1YR*4@1Y{fJD^Kw!#N zJzmp6GtB6>W{?Uxh~0^AmkK~OLUhsLW9-1Dr3!jTBS0Trw61BPBmIDA*%IE$94{&X5Y%;wMiRB-g zh#bm8XlYiGR`TKwXrT2cPWM+``CR&PTQyAH^H*{uNjD6!!cyWh2a(yaq!WUTt_2n| zs+YM>A(h%Y`B*jHY^K(KT)`reYsIq?Q1aDXNk>Zxtx@erNsdz`F8i=K@YQ>%so9%+ ze3l&U9*5Pi%;=i2fP1zIiRKO;%Z>M?T9b%h3T%^VWwVMODhTkXs>>KI$qycre%tIt zr3)U4-{m=ueveg`U|BL#9;^+wP+SQ&%H!FxEbAZewrp**bnZLUKouL*3G|RU(EBkG zUpcrW2S;Lw=`Q57o1am1lR7NC>MC{rZUif|vB0Bgo8ynwS+-^br7)?MmX;A8zR%dd zUL;8DxfkIep#LM*MM^(RBWxjH@}*V-Y4ljO6BB>41IeqeKTtrhqV0!f^M57AXa8Sg z&WOOF$hpI*idih&>-3YEjcTHk5~J@T%neGomBK6)pr@?bJXvN1OmBo(xNz#0eUkm` zrp9X*B?Vr(!!xv z@PHAGy;^3`~FIzZ4Nkd#z6q$`XkdNVOuzbr&S zrM+&U!`|uX>{2DnaBJ@8wZ_S!ITMBcIn|~#%#rFxW_J(;ify@0@!Qr~RA&P#WzaRx zlHKK}=c>RrkL|+!)N=(vpHhGv^I@2DB2L;3xQ&i)HnTj}M@b+r!dyRHQs+LA^?Sc; z!;&S{vJJw|Hy#3YkMTLnXWHy?Imhb&Wp$-R{VYbiz@&1fo`m0WXKe+K6w9Ylk1NE z+0XZ;FTx@}LD8w+;kUhYaI61TvujM{1D?S<>o}zfDuy-idwQF7Nt0cFe|mHYRDPJ1 zVclJWt~M%~Hym2;*@kEPt*&@+KG^K|`x?nfWse=TQ$^h0bB))Uj^#1Rr|ybV6n5-M zllwW~FgrB;UorA}7N>Cshzl5gq7zgDj=PT{Y-F^FipK$rlUycKG;d6%Iu$9td~tai znO*U{!x{@rhCR`;xf`Oexvvvwv|8|5)jz(g8o@mEFZZxTSiG6Xd^hLToz_|Ba3_B| z1RD+ZdDzx;JMLvTJGiXuctvLW;^&Iq%XFJ-$~lv8MZ-($Mqd!Zy?8W+CB%Ot?d4w3 ztawMSU<>oV5Vl8*cw3CFoGD14!d5z-@Aac0@Mw?-E!IU1QL?5pRQr~c34UW%Rx(~v zhotPGz#R=vP^yPc-!4OXVK<---#lZ$iPTVM!&hG$5HwLVmdYI|PAT6~mK*QAyY0qj zb&uK2LLQo_|LSc=$;1=K86Y-Lq3#n$9QDp<15VwDe<4;xKyj>t2% zI>5>756hd+>zrv|n zUvdpr7%kKtrr2P8nWcJY*&Mx%$(02Ces31#7Q}cFbAWN%2C%5Nd0cF?0>m94Q9B5k z$%GS>B=0L7f9^#zaHn#P8*`J1&s-aVW=HdL=`v}S zF4J`@dwVp;kP7!Mwl)5&%lT||&Sxv_f`j;=-GO}dy#ORFWn61GIfHjgBO(=|*E1F7 z^)WUePx!r)jh219xYLqe$xOXoKg6z#Z_xW^In5=`!S97-)t%SHGG|_SBWUxJCvsAw zKF5?s0_|?21^k~Y2C}cA$>R`xmKJv(4ululU}x!q!hAQRJ}u6Dy3gF0?NGBXpvUa? z>F4%+TJ&4UH1gHfkvp0msOSkCta^%^s4JLfCV0|ZNt|)gDK(Y7diE-;Z>Fl{wlsJ z4MDY9h5GlR&)N~9!U;jgb^%1i8y)TG!X!7*<9Y$RP0#_6{4BMzYQ^J_pyp#=0Z`)n z#O;2>ddd~AfsOe*Auvdtu_mgfM?O_gCKhG8*dd%hH^%!& zV#JwE{VW|skzY5j+z$5Jiww*mPd8Kt82TbRW^?tYqrPs_7w(tkjcYUdD{zoyn5WzR zwokqGfj`w8CNOeJKU=40s7%$6WjC%}05~cyg&+@?cB=^O!N9PLa7C-3Of{R#x()~Y zlryI8VNM6)gdGwzUHzfzhCR9`z1B#$Ac1bTzG=ZY9}5pCF$t9Q`pox`I7?uy0mvSLILj+xzdUypeADYpTIJr(<2)U-Z!&cR2_C zTq89s61F*Qt)GW9~}r%jjN&+KxeYFN9E1>hN|*Tg9peD@g1{geX!1vF%bC$*n= zV!OEs0lwi*35X^LDPo1Q^N>dptzuzB48p zSN7=>9-EhO>0op;#VyM1typ%zQ}&Za@zWSK9^=b$&7Coh-Ld4k2i0Whf0hH^#LJ=+Q@sx|9oS)|1!Z&>72ru+#1GU#Z=~?6=TRp-}Tyv+qUjP^WDx zge8i_vFFb^h|#iC;2yBonBr+>#0RQ5OA^b$ku9t`U=@VSrr>(V%XtwsLO)8khl3V&}{DxwL!IEn4j8C6E%4wUp`gGbQPpA85wa(-{0uXV*#%8l@msPB5?V!5?5^jTy`dM}9 zDA0|D83{Mzbm61dzu-a&4@Y!RqD(5ALnlLz@bZlca{hMA!jr?>`)}^AuJ-w9qEdY5 z7?)O+z+1K=!d+}Gpp zchvJO7Ll=~O+Wo@bL6UC+4OmP=UTPw2{C6K9T%KN{@C+cA8-E_%G%C{5)p7j%S8%1 z0tn`b?T)jn)jkmM^v^)d3N1^Z!Nqla$5=z#E2QeIkBrYH*BuhHHRD$Q6abc6kHMw* zF~ga?YL5(D1nb6mcyEVmf_ZTaj;U%!pE~8bQ@AmHQ}W>~?#j0(8|TIbjjw2QE+6t` zhvLz8eDO)D)tSx@&s^6EUknedcb9zw>rmj_LJljtjeW8(8etxS4__zLb`nJzpf+c) zC?WLwYp-1i`(bWN5xb|Z_72v{7F6b(8lzrRjd}~`uM^H^V*nQO>#zb%*yF}GOW331 zxnu#o_iFkZ;?Og$TeHRM_7E!Y#;Z>lu#@ki9&7^)9nVUYz2R>bU2W$Kp(|CrJL%IP zbE}v5t*P9$aCJ}T%4a=GcW zKN#MXg+Ef6(kU2JYZnf38>D|kTLaH@vvkW z_AtU6ZrTOi{xyzn><;XYYQ2fizSWR=fg7{&!Xdx!?;?o$WT>#jde!Np*dpgK6z3Y!Kdw1$L`am1X-H);2=CH?c;@ z0`LZ2pL}Lvrh_7$R_`I);Q!!`EU83sv=id3X-Z5eyB;Jr#8p_Sn^ScU&y<*1sqB@$ zrotOWMVT~1v^uW;lvyAdJAO+3kzCUj-_>laut(4&TJhMGJ(;EY`=|C??r5vsEg`3! z^GlI$%0}6^A(IZhr|g8W{-ZN0E4&RszNq2ZinF2^q>}=@Po)D7LwK;(zba!q+!( ztv?x9lJk3Gd>=vT_V@0?4zg3u8zc~W@DXa{>#zw~lF2HmBMWw`Nm=5l$z(mUT_3w3 z7>5cw_eKN7bAN9JMl=$qV4e|E&X2!VgKREcDuo4*yIrLzD<{#S-lh?xC&VBxh{J_1 z7A>JMZ;2|)=j4Q7OO?GXY&F&QLbgeB0a5ogE5meOsTbw!4OKIFFYxH%p|HH36hpHh z*6dYQ&7&;JNf68Bm#F{ZL~$ro-eAu$-8QYnyUDXoay3*#1k`t=TV$RUj8neiXadTn zI0qdm!RQndg!fnGYQt}OBn8Kji4KSbjEH%{>;i{j(Urg0SZ)gZ-nyNw{SlC+rQ(<} zj|g1m0B+3N{d!8D&+OoU7~W31X%OvuW}%{eYj70vy+sD$2(ZI=e>_ky619 zZv<%fEXN&V*{b8CmG;=MHu@I|5fnY2?bUh5dX(TS2T3XZp6ne-gZsqx@B3d?CPv+f z4$Ve&e$4{-@HqK%0%-6KH!0D9kEd@Wq=|LBC{5%Ed*hh<>Vi%ZJKQVoJ(H3AMYHs) zU6zJ0RoOg>(QfBECTVT?;LXU*W&?3d^JL@EP&+d`N8bk3deJRUxhokmLVf;|0EAc; zbMFu#ia&gY#6KnE?rURS@EW3paWhQj-bvEW#>|D3y~JSs@mWt>r%VEmJ;sPq%7U3* z5ceW=_Ir=kkg>cOgB0m>k05<1fBmay2I;KX9%@zS9IY&NP3L)jYaH!|VhcIkH3`pT zD0`ry5j?6;KYbgHmW0uWREVlLPh5(kqy=rAwp1+75pPXH{g}OH>4M@k=JA&mh+0#C zmFg)JjS;&|%GxrT$LEhZPcmr40BhTa^>?hg}dUB=;cKtajuKeKFgq$p%IbMO^D4(v}%L6d+iB= z`#ajC(yW|iZM8#l$m^%H!f$oI5i?zr?lw-;@AOQ3)!NCuybUEH$9f+n2bGlzYA}kPNNwYJf>;d&p4E?ZSU>}q4(?C<61R^KH_;1HjD-jMM1^PK{2`h z+L*xzQDmyWbon%>5OvD(LHy9yz{9Q~Q^UtIDt-^eV^Z%Req-?BL#E)hE^Vol2^>?9 zY3w>+Vd_R&-mE!zl2EQ0{&ly8~W>@eWB{ zdOzJGm$GOV;wUsBgu~5PsybqHeOeZ8@coaXCv6`V7RK1?i7q~)UJYcC9Q@YEcA_*B z=baJi^+0LsHy)~I8C)Rj;)lrKdF8eA_E3pGYU7zqBEG0n9(o+Gw%GlOUvZ*MMNd zXQ(aSD|r;-Q19u?w>K&rt1wQsnQhSlsZ2e%XZ^8LR> zCY&T;B1HX1fnMT@IBtM7lwmKBPs=@BHC6)!EW+x^ToDDi0dKYP??9z3(P*=5@OXVG z;@iY+yjkr`4%Ey4eQHr`pvyYjcy5!UfQ#uS?1TvaK%`^tg}*Oajp>ty$ZTsvyZLr&7R0s7(VdGikyX zh=Yd1X29~H1qCx}cildlbaG&F*DJV9x;P_DG4&Y$L~JfWi%J3BYMIbMK&*s0t&F;tzR7gt<%iLZXnPY`IJz}M(`Lp@d`)H z+WbK}Udr_n6&TvDZ8>qzP~D@ikNa~=QZcTe&~AV@`d`~l@Sa>{#pl=v_!w#PQ3 zKTz)MaH%!#S^#ZdyMpzS^7rAuCDPa{(rj&+>c`ZD-xpp$!XNO=2nq!d#-&UC>L4~( zfqCOStNdERv0R;{fg;A5k0w}_v}n!ZmMw&i8xsLRWzA#F#SSnx;Q$NW-(#{?g##f3 z&~e?t8Y~G@6`RVbd*uW4PDp)iT)T80OEcs|CzkWL;)AFD#!o}K{vJe^H#XjUn~*r2 zSc;)vTKdVZ^*++u+19Z8Cg6U(P9=61DbRy{P>*d zS4S-j=zFyUWp|!K7nx=b;W{Y@66U&9KnZ^JLBK0!Q^R3Yw+~L-J!D2Eqv3}fqmzPkqm*_Ogl-o!qV&dW)_+-@>ooBw3qBERn)toC?5-s zz_2pUwIcJ~?xQ=bqM!y(eZ0Lj1S48j9u9`7nC%$DD*Qsxj0q$U3*Ey>DMkAHERb0} zPV{jq7q#+43Ah^hx#1(kJWRD#0WtMPwS~Do8VZg~|FTo+sQ* zxSn5d4{~7dyFihO%2FR8Zy@B&WHa?isj|yzT#H5ImEbOwDrO) z)tG4Wc#*xx-Yju>@APpW>_2>P6D<|)@dsn#hYPDd z{Ae@kktKN#5eO`-vn0?)IiCMS9>6}j>n!1>v1{2cg&sV9j!QeGJ0CCQxC3x985QGi_zVN7T*rzI~S4+Vvl zYTEwHdkuVmXp7?%-XAyzIluV-$9-(iUqXh*pQgU+$#A#37qCS9p<0cdZFPMxUBu9A zonZ$&aFup@TT$~@tPOXW#bzT?u2@VG(~9F7V`tBQXZ#;qXAK$q87-<`L*}7=PJ_O3 z_Q?Q~RPuRxi~YdSR0q&~q6*qu0XHCG5T!~P-PP49%(!Z3<#!hn)En8OwV24NX&SRAaw=jqzk?f$%YEfnxgD2BB#n{9@idZXNs-I-rMZYv~rzYLD@OU{n;S(qeT zv_@IIpD>%Nfy~IA!5%Lx&%)7s^l*;X#u}*RF_qe#*_;luAaaOWj&@{72_pIH>Y(^y`Ft9>8ZsjgBZBe(<8{UQ4Lm% zb1pCV7K+1^>KGeD>y~e=bT3mTA^hb=-MZrFWvSp~fVi26s;wlM2_L;E-g`cGYpr{4 zt5b7C&ze-$NgUiiRa+nFU_|@NX;`J#cjSurH1t6@HDrI34>;(@+COE-m^L!L@^vQs zfg_N8YMUWrYWpWzW$eQ`2Nau#k7aUfOF%vH42=l7=54-P;j&}trJ2RDkl90c;H-#{%l!R@;_awwg(-9pF?mCE{<`@7g>H^1I!mp;4SQvJP1{iFFld$ni5%ult| z!vGKZOyyG2-Rn`ywA1(uXoHKs!X5UZ*8>#8d&l8pe1~wk^N9}eC^B~=2q|^SD=h?rrjW6CIZi9vLF(*fyBpQFU3YnL>88-Q|zfr;g>A^X(cxr z#wmmA2|=YW$+Z?B4H{sJaUgk?LZTa zgiM_x*?&oM*ou*H(oZ^UCSe|qYVSE3{g*GJM8%$SBefl91gC`IIlw76EjQ4OgP)tm z%k45vVVBsbZzD?!-msek{qD@<)wUTjVbALTgtjO5xl@JgAWE_Y?>E;A;RViq(6cbK1LFP zHS%fRJpR$|ykb2y+a(NhTH~C9ZvOX>D#GxpB@zS@aoN%%f%Fa_U6SSL0TY2O&PdU3 z3{mn;1?*rEsj8%RJR2<0!2=dcGLc99)tz^e!*#cGa$aWPJ`T;&3!}Xsa7D^#DFTui ztqchEsmZF9p$##|;ynZ8Y&>M-XcwR93RQIjLM}+{6VSE;ENEt4$=3OzG4x2jD=5d7 zqw4Rqi_i8@hNogFGpg1=o+wRX<|^_^Rzq;$3<%lF^)~K}oK>yJCXE%4dYfcosyIcT z=t{{=G`tpYOipH^eB!SX9o}2AJt%&Q8R`^pkGoNV!cd4vv2~Y1_r2SGc7|m^rmihp zVqx|kguDIK=S@ucbUDR>0(d!vIjY4|!=qNco!s#%bR!f6SHpvXh&T5#4()TnMsh32 z34(m5DoGtU%XLrmGZ}kjiE2K$4vHBWy^OFWoOGhWLgPR6laj{dTM$;7YRqyDW6$7IANR^>EAz_069NI(;}}te zlhmJ~R$EJC61%TOf(uk)dgy1!#NGdnioGz?tw9X(c|5c}ZScwt_wF0bn(~@lb^SLV z#C$$Rchu1-b|MMv@*mrhM=Wt(3Tk1_69A*|NqoexVt^ND^VMopDE?eYXaL9*_71{4 z|EgSz0AHyRgyfll`>!a-OV8}z3C%#%0dWdWAvy35R7Oipe>2f%W;3}TK5&&{#Rq_|LGT4ot?#`U_mY#z z@xuY|%2!HCOb&{1d|Y+^$Bt(TtFa8IIU4zDH?tORxKF0A__Aea<#8CO9OEx7y;X$#cDN&2~hn@VGm9d{gCxCk?bq>PQOiD}_f*$73%dAmasNcWR1 z2<7rvjNXp1Gno_799(GS3)12cb7auGB;5kB$0>L+E$H3x^9vY!vxk|Yu1Tgz9 zw>^9O_=!I74eBu(TB!C0Io?v-jrgND9#t%~-=RO3Ye6$-T0$Pt&%^rQgvKiz1y{3& zv7qC{Iw7vi#0f)9&=xSg4Zt55_h_HVuIDe!a=dBZK@%w!H#y&eO_vKPK20S}@g(@xTcTX_kg+#}G#_t`K#py{|2Y;JFh{%nCGb z3aq=TEa4H7--63$nlz9?Ndr%ItF#l1Q}ab`G4ak(HVcxH)z@Qmf`B9?5G0Qunm=@rpx+VJTI z2xMcPAgf=ghtnXw0**Q*hPCCt;X*Z1Fhcy8+p6x~*^9lT$E5RX(#nPZ8SOxUkDK~* z_R@D%O(Tm|@-#s=)s+ptq;w%Xv`oVoJ(5+`Gi^ZrV?nUDvN9Da$o6~xA5~u+74_Ert%Q`s&?zuON{N8dF?6R$gM^f{Gy~GzDbk(N zC8dNzNJ%5zosvVmU-aJJecnG@4r}RJI_vDS_x{w;1mvW8FX-RLtyDz+(E|RdOMshD zxbJrg+;d3~75w{thaY469kfYze^D{Yn)(NS3;ndZz2@}U#=c@s2-`Q<&J)l3#o3jM z9X9WKy9=KUZ`3apy6OCgFf$gTzD9~gp98nX^qYS{QFoj1U!{x7kU@W7ubejWo3A4?TV$`V^7WzJB=@-W(>h*qMyN&KfPffI+FXWNu8*WY{#@_-iYzO*fxuFExp9SuX zGPGY0JV)lAcJr&5zG3SdW)#-#@mDMI-PK)pG~})(^TCEY8mFDf?wTq?+=7KcvvQ8% zp*Hp%uB^8EB;De|x-v7*T~`Shm6uZ}tB93`A!_A3u5qTUvOAm)AMQ6xy{HYr?b;SE zWecTV4u~eS9XSh;8XlzAvAUIWz$(=WmTGKRSBztp{qSMZ$jUIF=O>@3k#N$)I*-Ka zEa7098#N?*9dBE_P)xxTvVk%p+$|i)H-lTAnM(1Cg;UMz9fyV1iL;DR=y_gBCTA?Y z4oh&Ja$7Ju{C8WOVc*QU(CkcmU8}Uv*J>a2I(7tM?C0LsGdmY`T?rM`+wGj$cpoX4 zuRA9r_J|bq#7_)o;x}*&#RWFX+$VRt_W1yg<90;bx;tPZKF31O;;CtPW*z4@WOvu$ z!`gk3a$Jj>m&EN8$@X6CE`m|u6Sg|@y_@WsNgcGACXSP&FYQ`a3T1|e=Vi;!=%TVu zGx1!ibmY@rXWwojDCTRL>_4q5*ZH~3Oxq0bWwHb=Go2TE=DIAzI9R#r^;NV~=pq_i zY}-d&85L8jkrYRn)eF6+IKtOrE=XVcKi`qRudy zuVUgjehYWEVu*k{A5*rT25p=Rqd)Bq*pwt`q>GTLY$U30Ym2#-lGP88|Jvd(Gl;)M zTS-D|yRe@K@;?0_2ByM?KbL?S2TJ%oD%5v-@FT;nnPu!aDl4>Yj2t;Cnt||WzI^gw z8OALkQI3p_mGz#bF_pmnJQZGPjxtYOuv2F0sY^cYaD2F`Hkhhbc-kpDDu-y~;>0n4 zpF0*OqOu0VIbfMtJP+AZ%SRu)&vpicRhHS*CyMayf&#V7p# zjgratO5HCCB|^h*p}?n9X5IZ#h>GX^cXCGRdP=m)Q44+|E=7d|{F)ulw)}FTDtqAx zY;hZzSLUA%ns?`LrS!*;arwvDlJaP#n8<;bJMU|s4D znVGKQ@5|MFykGpDw?UNLxO;ZXHLh-k8y%&!YreBN%`RU`GxWg?o6RqA`Hr37_ccD6 zJ)jq^pw*w`1AfbV_yq)f>6^>wkEw+zb#!%Em?Q%E%es2;GRTi6YZeF4O=PHz)^0%07fG4UG(9Ga7QQx>Cyf>oR&9Y13abh~ z5^Lcw+}HRxSM$tcC$1kf(c-nYd;Vd`_5+5hW(;UG%WZO8-ZA09`R@B_% zc@#YLOWQS~dO4a3X3Y~IPozU4qD`QtZwKHhM$trfvo!@h2BTg}hyn1#R})FrkTe!l z5cG1+WIM6P|Kr^QryL@qJDogrU+ByQ`rXA9%Guk3v)2vi$kN;XFR)5DAL4ym&14$RE_lT)XYH5NgYP>-UW_g4fRytH*0yB z|AhHx=SsX#{JCBqSvowmJ^S+7dA@@6at!6U8*cT~;znW@+;PHEDAb^4|3+oL{C3o1 z<|E^|{61BX34Vt-=Cjq1%kHxXNml>pDsRK=^u(6rdYM9d^CE=!MnJ!67Eist=XY6Y_U?dApj&}JA zk3Z-k+NSxDH!PYPbLy^SA`A?dj#hL5^wYfm05nex=kKS^0$jl_(t~Q#CPxUg-?(>5 zLjI4i{i;mhNVodo(ga&s&MF^rbO5>d5NCGC^8hBbosEf%pk&DqhE-`YKu2)*8iJ4C zNyG_#^N?^SMwt!OT&?Nsyk88cbqL+q@N@t_-t_&V_AmlW-rlk|(I?ELbE3BDZz6>i zNy+(G3Wr)#$vBXRIcucd(6G_sn)F0RGd3>G<2lDZ@*i2mA0Zxq_G8?G!Wc3ySI`qe zML~{RZ3vtIvQhJdt)O3kxYGC0eQlD{fzrpYNwzliLSeUC8;jTeL$+8}D+4clx$H!6 z4-cv{4V6+b&*$u2KTUIF@De_vI@RX2KJ2ah9hv`a4Mzlq@>uw)Cco)q_u70h2QqO@ zLzo$Fcg4PYID(t}b^&{w+IQ>qEoOnCJ6s2=SxfrHX^926A4E5xAL3IUJtloH^?K5S z7XznPLn1`VI6kWeoR+HbLJN~AwP@Wr+mY*}tPd=n-+Q}H-u`fT18B8gnGGKpPs`vUk0CYf(P zmv@Kyr?g+5*yKE;#di<0P~KV6X>q~J9n+&EAlMpqOvR@j={eSX&JNF*lq>RfPp7fu zk|0k+(9utP>s_Ix#oZR$q!3m+>?W@%bHC;{Zx3?Rkl^;4#6MNCWu&{s?@-4i=;{(E zA}%+^t8O!63y9K_-7ncoKgbneC_7|UQR8lyA%;R`ThSGsEEp{cZ*1&_jIKcp0D z=4UZq4lI_LQlDw`;dE0HT=h)5>L{*F#0$RUc~~_p9CtoDdlTZq^X~FVymJR^Uyq=en;z6p;CzOoCT?C2U!(8+!VBWd;W}>P*eKZlFyo#* z?xesPxjbAs+J#()XfbQIfW_@ICS5+q1!a?vY*!MN|E29Ef%CMTOhy>Hh=gB`W%y24 zk07Zi)4WbUG~6CwBVL(RomJRqCnD1g`bt50mH1#Pv`f(?l-9?+05-Ua@&5eJ>D_#Q z5{BuVRjMZXK$2KRl;*T60AJ0?ax{?QkB^kJa{ofV)$xCB&b zW5Yc{oEOPcOu|spD-vbUW|EK?n@@K>MJ6l%Q+R)V%hJ5RjLKu2o1;XD?mrSOUTR2Z z5r6^wDg9Y7Ha@<04*%_Q79W?HT->zxbC%f4+0C)xqC0y&A3hDVx@w*IO1TslyMi<Knz~8k zm7Q@xaXddu=swCin(wO5vIQtcvWAiM)Aq*VRw-vTX!>>q$prA-F6ykRhjbNRb{@gc zkF2BY!X%zzq~PC8d6n;TQ#Vb86J{);Xfw}%!jrF8JDzaH;@ZkX(tjd~VSd(&Gcr*6 zG7Q&@PL0pQQgRv+{cJ~{3=$?-@B5o}8SR7&MA)l$RXCBHm#JD8x_o7&*ut{Pt|EhlMa}eirFNl6oIA)V@ zW=UdCBNKZ~HN#zRDleq*mF>;ZlpB@0fmOalvKwx`sJV)%|Vq5Up{2c!PvTRdAR2!wGq^vb(&8pMn1j4D)cnSaC1LN z{5pBVV0$+4*5X8cL*!tRmzZ-E#%OL>;8^x1Z=Cds*3(c1U3})pH{G-BDYAqw%JO$* zt-ABrkR04%&7!b;?ZG^igR1>dt;jqN)4>~P^DhS#))XfYr+aYa58TteufjWX&92R4 z;UpYN1tVyGV7pXKiB?gV;x=5Ovu3pUAVI;DC@=AlT4HWKQ`SCq` z77*B-m#@X>>*wW)z5)K$!(p$mzZ@W?uT}3}lj%(k6m^0d&=Xo!dCQ@%0D|7HF|C6d z;!NOrqi!U4z86II#+3}cW?d-8gx|C52R4ZnlRV&_)Ue%mvKBy<*}wW}P*QK*3V5yi zb!ANR6=LxQlC1!fwTMsq3SK3;b3gz?&ko?9{AS{@wnp;C-}yBO{#MqVAtrE{8&ies z^nf~zRuA92A_{ujoCFY1Nsg1nPgOE^{|BP{00LUuGo-anN?#AKNEneC5*#NuyzCHfm^H+(2;Zv@;B+;3F!!ZG z^`xS^#(3H><#!@RrQ4vP;`-1Rm|)96=fTU!0kYq#nZzBd6-q@4(`EPBi<(cd~gW_~0;XoeeyM8!#2+uCUzdmzy7 zfH(7r5B#WB7_N9lhX}4y?r&v4sE)YNzbHc+>$1w2ubEnGggl{p8|U9#&tYtCVuh=A(B&W~&HON6_(t6L ziH^wlOG|^E=oqb#GMKeb)tRDV3b@Xd?-3=n`#xc>iCh9r(wz#M_D#%5UFyVHAj2?G z-={SNr8V{d(A;R5t30`M&U441yfbn*3mr;9ch25JgWw4Pfh)#ye zUWP$~d1LxyNBIIpNfg?y**Ar-At_S4w@GaSoH;3Fo)XQ!HfDvO8&W`IYokoDSm5A#y(J^ac=tWxeSNeA*X9ebX_! zyikRP${$7U@#6P_JI&Zn>fcMBDU$1)*1qO?)*`+DW29D^r1g-_uj1=hEI+^PS=Y6i z-0iE9^ywM0n|>>r08?$MleMxYO8c~Xqi20U4tshpa-Yjd<&}`jL&mo7!?VE3wCZNq z4(^*_!6xI++3OC6NBhoQt@&^$rqzs>zLpkW0+V1WDKX4?`gW~gm{QlK;`dPfs zh=IeF6x@#NI5d#>uYZG{U2G0b2sS2XxPt~Mj^|eSGjF$jpZyk)W`GlsRc`OSvR~de zwEN_-Rc3AVR7I^zIx{}{0Wc331$z}JrDFn-06i}^x&H3f%ji7_!)EVPz=F}m?}(vA z`v%zyh@BbjsU{Q;BUN<%8u~@TptCMCg7nybJysNX8uBwOv9f z7+ZPvOT}(@P=+%6T({|*)zxed@kwjWk}nz-Es0G&j1KT)4S2x^9PFKbiciUr9yvC8Sje?-Xe%yq#X_AmS^<-K;SiJWP!k_uBBI zpzH7r-#~ZhN^_2l+E(9!QjMEz68}8{R_F31_hv_IL&6)`)le!eqrvqp=Dzd^%tgE$Qg{~m2=&MjT!8_q2=l=`aH zKiu5nBJcS#K+_$THTvTt*TbQ*F#`OZEcXD2_ngn9tE9L}3i3&KG@)T}#NK2BI3*6k z4K+`H<)(He44X;NXuNIvufTYb0HXs?OgK#(o9KD>^Z>V%a0h5p?!L#r?sK^@PRx9< z0j4_bY)fKEA3m6Rabm6{fIHtl>QblodEktLQ6Ugv+bGbeIb0=ih^OI;khTT99d;5coJ^BEu%BVr;Nw8RTMIoy@++(^~t5|&2V;rmV|jtBqtkY^tk z^oqy$>7A+V6I|;4P^Zo4TIyd&vm&*Yv{X)>3J8MuZnQbPTv@ zAL2WIJKwB|^Q9l*qHi-EprAIVotHZbNo$eMc}Iq_h&RaRoCiu_n0 z(#7PJQ=ecgKvXGzcURZf8#+7@Htkn?`J9@0gpu4Xl(Qi?e=v=3>62HygdJOZfTCOk zB;d`q9`@QN1b8R*Jad+Oc+kf{JxOQyjFlv9*uxb)co;@kcbDHfd_~@-N9pJBrHgY6 z*-?^zRXAoNi10DiTKqE8Q;Ymd8@|q3!7j(Q#T&ynk^3B80~%~gS_U=8Ug4oqUCWb) z47*KVUu|cU?h%D?xgPv%0V51|K28V_f9~JmCgg$}ge#tig-K{5-W9p_3*_ z6F*wY+!Cs=5F(jhX$kt}c#?ewU??^pU^G1~2h(y2Fw>?E&jP@tqu0;vFX(5?g5_1q zyuXVTMiNu(%+9KFNH1E9(X~+cxk16e0{Z*$q}aERPleauZKg2AMUR1F`-gErd*Te& zUk!V2FR126rMYY1eks|wUeZ4x96zTQi<1V2+169)9{qOsm#+r?2iNztak4QC*$lY< z#Hdgbyo#>j3&_1R9%d-N0>xw&MESF4I+f>+ucEUWexX#l?Rv?gn)U4axh_Za)4OD{ z{R)PKN3MokLKerHSW$sOxbvky=qIb4CwW{uDj4sokFSc=2ZQEs>MOQ9Y+HoUvsBP9 zU3v;5^$e(AdJ+A>Yxz+DUOE?Q`eik483`Mo2un42NH(^Btw^4N7)dAMf(YU*Urokn znp1H-T645R`BOM5xvl?__tE5PW=~{=YzG(~8Bby&ABVQ7P1$Lo9Tf3IcL%1mk+|W;(@uoEqrk7fC^0B@epH1AuL&>_Y zT#6ox(Dn7{7s74x zv4*3^aT`Y4kH^m*&2(mD302h?{;Hug2Nya~HwjI1x{&LOTqedt(=l zU5a1Gis7}{IeOS`D*V_pDHi&6GUx=JobFz>xy^2 zsVO~jY?4r8_vDmOE&M&%afOj{@lzilf#S{atBvux4Z_Im(t&V{_rDK-9x^}dGs)xA zUl^i-+~9Bl>SXDl3dR;ddOP&75W2u>jHCAf$)VZeI>nTEEi6sOpZl(HrTrD3vf47E zqcG-s!6qa2S(lx$y20GMlLs*@&%uF`1|P za69pw*J=0w6Y`F!VzU8hhL6#98K3TMS)@?nWG%_^ODW2wamrmQ89be~Jh6 zfO$C!lrNswMQBMq83{Uv8dLCcNT;Aoe&r3pi#&<7h?3}}{NeC~Q6sKWjQz5bX(@R^ zYQgLlgzKdkR&e-by{U7O&oXtx0RQ8qUeOCQJ*a;@a4|0|gf31db}XSCX`E;WDBZBEpcCv*@#fq-jk5+k)NKX zF5*=ROkilduUvJL%|$@dyZF`51HB%z*aZ za<)}AZrtUNU8B5zr?D4wC3G;!3`*3x3yZxlasJr%W>c8yK&ZGmY?edsv77)&t&^O9 zwL_ZOknWk?b{X!{&`^U0Yjwr4n}I(kW%-Td_ivM{8@`kzI-e@S@76sBW}119Xl-o! z3}nhWpAP$T%8Lo^%&$3_4T=k?8UK*Ezv}&NV`)R2&7&%P! zCt#A{!|s6J!@!Vxw|ZnuU?VO782U}1a(v4ZSbx!0<9ufSbx^kh$c1VWmBzMjU+g&e zcFulbkYq-uN&E31)iZDsLq(^dQ~}1)v8QIl;@*pmIUNs~#Uh-Bgc`s>vG{UK;vs&^ zikAT^JryeXB!JClFV#wdzsFV~1YBp*uVtv)`LtTCKt={gbPfd8X4@xvtY z#PCo5vC+Pd?q@%8L9u;<$$OdPm$&mmR?;;01?Ogy5c)MwHr{BtE7XR;^&iyGjD4w| zA042}ym1CaE_VC2Z_e*Xug~)t$GYP^?>5*|&ir-#tf_puLx=6!=dAbpGY_r^^gR1s zB|1l4-o9L1$wZhAO#VjQ_W!zS!f;QD=~-6+nQKUxE*F3GR;Uk^7ONtnBbhrVAkl(q zT+j5Z@!W-J?4a{28%vTQo-rX1vODco5n+vtBS z6Dn!Zc=CnTlg~5qSAx)F99J5LNj~qv{?A)g=^3m1HhuEN$rap}C&{Z}td?Vg^r5(@KuJ-Bs zJJS$xv%l1Uti@!)Y)aT})$ntM8%zJBmC3(1y9x)2qYsD00t~GqVvB2gEu*3nFXcU) z+)wT`@g9vs6pxSH)~PDw{@$1R`bGQiWW$M6hrcb_!|zu%q@IuQCW{o+2go^6^K(yg zV_gwB+^#Pp6+sfVeaZWTZe509p~wWhQdk#tb6!=Z?5| zUw-!dhdK5*is7(mc{jh+bbs}O!*8l6wi#Dgukdnx+S`mko1!a{0pyM9oUzd{K9TJE zG&dQ-Hesp3I-&2i`r9BA4aO#s-{5~H*gvUWK(~^=|5qn%ezB3xqCsjBiGiD42Opz; zJF|=uCAhz66l3xj993_Tc4Ab6zs9Jdy;Mcnw?g-AAJTnHLfs^#f0>(k!%O+nL1W;X z$4T8wqaP0ozx{lfRAPO)-yr0*v0*M2+pPHc1b5+#=K9_4X`-cJ`r*Q1D6Qvu>fsS~ z*aM)jwSv7=@{qgvukVeFt{|qT6EvKxok|~}gk~2j&N+~0rvL=>^&wgr?K@p+SxO1D zL^{Xdca~0k7(bv$62Hi0m1b~YSIv?d4<^w;?MJQPOu9kXcb~#tnaA|gAG=;WrmGKQ zmJNbXm$}JM%=%z~X5=Q6F(ii( z3H@pkaQ|zFJ(P5mJX>e%OlS}T^qnUbg|Y(HA+W$m>&QtR4{i>&3ONgRZXZe2^m8w} zJJ!j9DX*}#%AAphD^priQWLZ;bYnTMQ`hiv++zhu!mHS$VX}B?Rb2Ytk&+yfoZ#$8{f%-)3QS(nVS&>Mai#+50{lS}LO2D%3djqHQTt4~-Bv*%;_f)m!p=f73hv=I3&wpJ|W^-1ocjV=zC_F~Gs>;&FT zr5-5$)q`ir;`Q82f4oytF`=t>UXmaFvX~#nbYGVmv8Wh4I))6~bTOdP*0P*uIBh%M%|>GD zjtezge~{MWa?#lT6-V)alYXya8fmcnHX^#TkotxbRS{%eH^sB!^$|Yn!Ie;l`;RZ| zFJYx|Up*SUb14uNoIlm`>L&Pj&FDZAz(VhP5jiKzz}D!$9cZvwM%rnebM;hys0Dok z`_gu8k4oegL<+mv;=w&kUdY5e*Y|O?Pgi|iC%OQ0Ep4K1{AyQm%QNeKxtCZcr7uK& zwLe#K9^-ZsKGw`tb9ZI+KGG5(LIo&>v4ZzqUvh zXc%C;CdW`zUBV!H$an~PIQVoz%xUCphy^L+<5CNutP+Ie#MvpNU9y`KD)ACa%%GGs zvL_1V1vrJsgZEPkyrR= zv8$;N@+?>mA0tko5noJPkBQ_$Crng{cX__x`fwgTL{c79d((wF4Q-ut9E96% zwECUjXaElq%eu;Uu$}+tJo35$pU?*pQ(S)*=RBmucPT~$PlT_wMWxFgr=h3oKbKTQWPJ@G1I#ZJnHgnV=Bg>bDwIvd=hgT!GenLLH>MEGr(<2ma! zgvJq(O9$GHOebAKjQw62XyFHP^Q%LA_C3pdPCtbv)R2Y0|JkDW!7&I=G4phH;dF4*6?QFy2WikG86j^DGTW><05GI1zj z6_q+xED*y^kOKtFN>)}BAP1mW_Pmu*$V=rhaS9(({n*w63;aKi+FnvIDtq0gnVIou zqKy!z`I9v7!Bkw_yh5NyJ#)NxHdStD$i+=XOPGz}!d)`u54=8iWHNKudUuO>C@BZX zfl+{LQf?C^vqdVVtSpL)=FJRBp zuF>~wpQHd@Izj+J4`;;-zKMkyY6w(V;Lm$l@F(qF^&{YqIThZPPc8(gi=6Js#pu3k zpidK6$w_^5eLXt#?Gi1MO3&n8&VjYxjxw=TucUZGvmzpV1P6^fBqPEMcYhi%-okcr~oysLcG8=w4d|Vp%^jeitjl zN)j6Lm9OK2k6Md2(i?)flf)^k6E0{3Ix}f^?w7V@BDOpGmV=%*$*Q7zR5UFLlzp z(39eNn5}6fNa3eY*w>$AP0Des(I@Ia5FMWNT#Qc@EV++jI%fGy`P-xu6sYWTD;hY3hc53|U?+mNX#P5_%7fbz0vfi8ORBmVCzHUsVcP+l7U&SBW z!N18kE~>O|wvbI;eWvJrE8pViMEV6;2O(?WU4nKvBTB)k$`Y=XVCG}($s$cBxsyoT z(%)!q?a8kpO-|hMc6F{L>rlKqe|S2*&u8~@SvL{EvJ8f)tfAxb_;94C~r#Oy|awiqqOZh}p0x`YP~{AeJ+?%JD3bbe7aTUKaVmx#ka=_TCt!kHN>s z>G)qQKtpf<`8GXG3}h?@A`y$Y-R?^$>L-7xtISr*S2&Ah~%ks`bwWO z!OjCDG_9EN@v*>k6RKF8VPNmYw1N74P0+jIHGEp871;$!tYm{ZgU3iV>3=M4 zyHh!DVFrol3H0wrRH3+}-rAfR82Of0WE&Cw(@yKJzUPqk0ZIHCw~BW@5`Jn7CI^p8 zH1l$2E+w5fPF?^IdRir{?0+G2D!+S(0)4lw2!ba~bB=awF#v?emT;qd5xRa>=DN9L zV^`jI^Fpg`(f~&@{+=yXOsah3LL&PyEf02?W=%1r>&iy*@H%?R?!ZOwS1lHG3i`bj zyB9o}^5%NY@Q#+%&2+%$ad1Ysw+O}ykAv@ztjDq4cSP*I6(cKIa5>KFlXR~CvQmcK; zx#rfCS8MUX8$&*mr-ZSFkZ1>EmOBYEMZE{Ft7BXo(&+YeLRJVv>2kDtb9fEI&}2kp zfG|_KQF}2F`zLQQs>LLD_eB~?L$wc~k7K==m6(6fpsF~AA}B?aF6!3x)~VVHrc82h z98bDK>WK}xs~*c^=l8@-04Ca}sbBpjywt1%pRKrea?;u3m%dWe@VD}vVE(D=_PDs! zxcs5~tvi)`bJC^SS0o#)69!MxXx&nIazlw22?Baifw2`wv^b4P{WVprjw z^sfYxxX|FMA+Uf!ANKh=hvmEETBqo%*!;pgiN~7?34BW(%-IIHd)T2hYZ(Lrzt&(p z8Uj@}=5rj`C!N(mi!)uoUK+G!7%Np@<|hZjV-gcSJUV47KQnQYrNu2JrM>d^^t;M? z<;MA5S}A?{_}$8azdfbXRWM;vHrJF&@Uf>(+LtLH^4wP8Pe1}Witp?KmLuTskWFh| zLLa+N04SnuwEgQvQwqCBFYe9HRBC!f*%PKP%LQ zmwPeWzAoOQ69dqys5_ViLzdPf47s(`cT%yq+MW>U$Nm5qvKF*|pX&3Efc`6}NZ?*K z@7e$xA%sO?Ue3b*qQpv#SISu<^1a#Z4wIio)& zbG8@1+!)k|{@N`aA)&&Ewag>Sv#s zW#fpm!wH#V2^jgjnU9Suo?J1@-mMD2ljDOzdkSN5k9Z=v2RCwWD35JG9aM=CbR&%yf z)KKKO+DLO8{T=ZgkANC$;CYdjAtr`az@T1}i^2nOg52e08LxxYiyqIoUF@nF#gkx^ zq%s)0%f2q6-DO>OG^}cgSK=tw<0t5d$1(Kr!_k%^2X(LP21KJn_vU?g3s%0QTi{kO_ zw9Vzej-^FF8*|cmks;*SQSiyOL<)#YMDVDN?OUO>n?Y;LW7+Ss2f z7rdqvi^U<_VM1tMeOKeXTdcXy3`teU1XPEWqfcN8CgG2D0C=ACJg{cw`mzG~{6odf z;RU$BOUWPdZ-P3-Z2wHt>-q=c+QtN=4%2OkHWe<~4*gN0PGFiqeyP8YM;;6bVTKWbhU7Mo_tg07sTk&*HS+)S{&8UfGWt!D$v`ecB4F;*jt0 z3vsMr74V*<8OBnA{RC;O-dF@M@u7DowUXffs$5qYZ0Pd!cKFFeB1H?r#~Mk3lOTRy zk>ZeR>y#F%=#5nvyJ?4h)HgvQKK8NFJXNHr_tJ;8lT#UCpMxyeJZWH&!bbaiAiWcL z0<)M4?QZ_sCpJLJ3C9|*rRR$=5;s`D8NZJ{rV}0maffg^iALdt^LE)r9fy;lhF)$2 zxm-6s0bkq>d|xABGG$ZM0$29UL;R$X!7`PhvB840%UN4(SNakPcGB2w$P6s>;O_ZB z__g=fn`KXWx?0Fk-`MoPdHQU4HT>$zo1JouJX@C0RIR_N052Awyk4BDx zd){B7QTET>pNj5V9l%egEkd85@H(Sw)gUd&IxypS*n7f{9FlS>StzK+_o&?ewXH1edkfs#-UP&SJ^=0b3i<)Wgno%c zwV%G-=F;#UE^fnscHOx_rSXi7_6sk2IN0dsjUeXx011Bn7f6P!AUx4*6eatLIyWe! zYnBL>a(AfLxZKWRiFK2<;DuX%JNzOa>d=4r@|(YUM`qt>&1+@e-wBX6fJN--j%~{{ zgZ+B#iP6fyOXhLh=Vzb#3%*5A-xLg--BBRSrV!{P3?&#-RPVUy#%x zF8rk4hW9iiKL{yqYHWWrnBF}m#Glj$=TaULO+@d59~+MFyA9|u6nvzk9xe~D9MzE_ zuEE=CIuwrp;xv{QEMC%2WXDFD@Q^~}jpsu`NWeEiGAoU5H%EKRZ}CDolejFyuf-PB z>Ffq@v2mXS)8oeRD((b0ikm9wz^e_RP<)+8twV+gzBj+1q-*tjVj1BthTGL00?~W; za&XgH$iTJ*%VDS0S{%!lk>yk+*`%?88XGf(J-U+oJq3hZa!Z}CCE;Xk0lG2K8)-t+VwHENvVpsTpsNi5LaN)lxU8$M>`L=NolR|H!Mz*OCIhB$EFw?Zg^%)^lnjW zH0T`kYz6zqa*xC!W~IF|yKezj_~a9{+>>CtZOd zYIRw2vE|M^u$uL#^}_>9F;5l1e zY5GUHqB#CLv%xy@m<`z;#!|f3n}gU7fTYLi)j!||pmzE@6~65)Lu{cFN`Jj-7V`?s z>1tRuxKj8#RRIe5tjF0=Q4Bls*#QYMGU#_Pc-;<+utzb6RQw{PL>Qst$Z(f4KeVM< zEF5&4ihLO~LyhHuh0=VcK;lLQsPD@5nsm__cPvp^e>F9r53_0}7bCIcB5}gb1pr7K zrkOYHt6qS3KT8wA3B2xs0}#iP^}{)+PD}9D`j0Mzr1!<|HmU&C$82$8MkEwhvVBr^ zAlVP^9|2VFSxpi_323JXSi{OL?=g-^C9c)Hq)V4y|D(&`yRWdpSn24T_&@#ZF#zH4 zYl9LR|^o0@*HcxP8ZWzPk3Wy9k&mnh2?u6?C}v1 z8I*6wTHbQr%i)dL=C$OIPg{2w>ux_i=|fug@;WWosG<|CV+vV0n4Y6WhYWGAi6`*( zdj5XL7V#!xd=xl@Sg>AE<*VleJI@Tl<`Hl?#6-Hf*)sFN;0Wm@ATTrid#$i5mat5@p2~7Sr@g4}Cn=%$5mfROkp_Bb~IMj@bK<4F@*W2?=rchRmJA*WTkXA&P;Ag{UOh6#qHRWhs=+U+HnU0 z2$V@*q5BX{i&jNYW)SA$e{9@ZNiGJ^;qc zV%P`*^S{blL{EqS5Rin&MW38d`b6&$kj4VwXJ4<4+N%9?AN0BQJAapNt*~uZqqX9~ z_k$k<-e~ZNL`YXfWe82)D&O3k{i+}Uuh6d1*pDf<;Q(&rdYdpT=hdu9Jrf_H%Vc{% z7*)>)o9$9P92xR%*`~QyeduxT3cB4S`k>j^RLv#X4^|8X@9&gj-6w!Fp89?I_1*j0 zDKmh>(Ei<8>+=jg+`*tyX+>M}ZGy}Nf;NvMpX4|5Hb7EPstQ_MU##AK^goCWm2aL3 zTI$b`cIJ+ku44GOJR<=K{hW^GA%yQ}W#J)PGQ6YcKAeX1+Bk5qEQ!c8MdlQvhkRUa z;4bBfBz{!Wlpl`|`Q;e&O1wR9D)2Xt=k4232dQO z##!xD(zXRtN}{8NI>%QtTMGnp|EMZ_k_BooF{Vn@g0j=buZVef^?aT2EQS$o6G)IM zjeE*WW$m`)pM~#^xA+|CRE0#zWM@+zA+^19A2@0aLhy_?m^A9jwiEk#gcz~y82qw( z&On*F_okF7a#tP@4pLcN7EW$b%7qG9xPSLjvx8(mx}JgdC@mQ#%(Ba_&1m~_ZhX#^ zw=KN3fIhlz!N$m!>EJtX(Byn-0GVm~n!@BmE0}lZn3jJYLrt8GYO@fqj6SYi^vDfI z{c^+F%9U!z%X>1bg$Rg>R7o~F=m)l$6HkdU+-`}z)ti8Wa+UUtDIL^1HFM|9!CY8$ z4#sTP_LO76YMSiC%!r5Zcx4@R?}5OoKHIBsueKKXVEb?U`K0QuH^WWehX}|`%AMUO z+HU!O3m<;1dzzPiXj|Wdm)`iB-?zl-HRY2!hE30&-%l>kr+S$36sw zh4BzNK>LS08w?7;X&sq5K(4EGMG>APX1tY^GNu>dx6MCt`l{5K{<%Ba*cdC1``7+N zB0mtyY1JCBXE8~DViN!f<7}suB){Xjd=*YOo)r0jqx5Qsvqm^f`p2XMKl>ArM=m>) z4SFQ?& z*Pd;$5Zs%XrEI+R&=x-2@o&Z_asnB1nC1sQa%*-TSNqKrUVDgz4;d6TDpSehrJ9|s zlMAQ~cZU;o@6!0!-fhm=Av)?|#wL&~3}?!ly^4;@1-^SwM;w6|YoYOiO<(200(wF}a<+>%B~}D?J?E`b{|v!6(KO}**^E2^rLj%HLo$Fi z!Aa2jX{cs3;yVfb*OL~w%;oIs)XtmuJahJ`tk@Y#wkLMm1oLJ}+{Xl{&cL?r1BJk6Dga~6X9;cmP7Jyl2c&E^mPaM&mNZNhkmu-0qxk4 zZ%sZ`%H2!W7@9aEA2(I);ERqP)VE&1UMuW*5`r;rpC;~yLl}(e!@OxS$8{Vr>}a7a z@?J@&yR4RfIOI$-HXvv9RKqYbJ@GgynI%Yq)M8m7+W?N>8EUFCSkn%C;W#mQ9k-Em z`eQ9e>3Yy~j}}N3dgQpTXA!-vIMJXPe0)nr@EdruW+7P(D$42Qbo@CT8K0{k#_>LQ z_^=ZYvppbJvkS@i3{h+9vWLf7b&^Rxk<9G+8l1}I&KDjRJFpKP5ER; zNoWyHm5#kvn%MzX_~6M!DB-DArWfv7$b6|IOh`~Rq1B>^)t-FGSsz|b$hhJk zQ`f7SrXA}uN|22$iPC09e<<2`?V`xb9! zbD@GmkK0?Jc{K-*=)2)>M5k31$8$sDXopBj;UALxKYvX~|0gsQ^}lyTVyBY2mRdvZ z3rV6MkUqH#bmdG2@Y`R9{j&3a<+-=^mFFRnq&F!?Cjh82$h^nazn=V6(a~vlOZ{(t zO`ja_v@H2rx%I6kx5?TfO1`3G3l}XO9yA;C!b1(g3vLVcvhk1b*?2oiTT%u|}`5 zGCz4kUvcC>TkR4dYC+qQNUJ2ul+`XV(GbGj^WG8vDTqv+u9kE<69uC7q6b$|=HWu8 z)-bi#LY#ms?|Pm?bzv-*cn7)8w7f6s<<=~-F8Me0u>8Dj>CTKvzgOHJe2_dIXR5Ty zI6S6=hqfnOTX*S|P_y6&4l#FF_iYT{hK7>K;0wd$&u}v+B4@*#)Ex*0o3#1`<-m?g z*gdT**388IKboXcut6ReT=GNl7jmkoih7d9bh7Oh$gqA82>huE_UTtczF}jKS^HAs z@W&(*|CRqo)t5&@-M{Z6S)!~>DBD>_+?(4qp`^7($m|_~)!7Er{2Uvz_ zCa3FEy`7mThh6TVuY2I*Hdrj4frUuSP~Dc{fMh|HROT{dlh)QbZ1()^4*|EjO<4@y zD?2esB?a|QyF1K&1u3&edgGjZORjO>B>6vm5Zu$CX_$L?bKo`NhH+Z?bYk>ZR;C{9 zXf-E7BjX@5P<#nVR$eG^@};rTS`~?=iJ+8HU6DB)!m4J!HY=MPj2TOZe-C5D@`M~< zQ-eQ62N!Ium_Z8WW5x}B4HnXRqXgvB?I+Lth`>v0`&V7`diE{QU+eunn!RbicGQ~~ z=JE>AcH6%U>tZZ0Pv5{l?pYBi6Kd9b-VJ0ztyz{f^M)oL>JmjZdCpfe~M5@wa zwxO${F@vUPGlgHU=KP_bbqQEHBS4{t{`UgdhojJ&jAL%jU;U z;rxUx*^RW5t&JeOGKsXShy(uNT0nH@m5@PY1PQN9%#>XuAVhiNw6^?R+IHH%@tY%p z!i8OK`;;_2anq99sBd)XAUm|@tDecyJ&*ucQU zX6g%_p12@}fwUI_%A*$HZ^Z_ufyvxKHiU zs|I}f+WIQv`rG4cVm_4CC%-5AOfChwh2EzI_U4#UXRoRMLSPP`P6gl*eb7K0_wqUZ zvz7&Z-eVpLz0SGk%T0QEn5A1LZhcZoUF@kHxPX}}&sW50bc?eQ=DKr)Ted$ymZIHB zCx*MdIi9z2OoG?r%dYn)T5e540}q}Z7Du*N;g7qa!bIhL4d2;`xt2%48zIWPoL8yt zaZ#k+@8(u+{MdL56Mxre^GovVacq!GJjhGT*rcJ%``w;7h;Y=0=opZsAsxGExLDl_ zeit3pdJA2;+wf=WyYcHs=1a~^^#0#>b&1SOr#}<4z78<@^hU7O^DtcmE!q5L(uu3m zMuK#XFPGi6i=^FuGWksy`>{ix<-T&-`9z;+X2Ew)_s=+mhj@{SXRbH`}Vy z-gvcv{pp&=zmaHFJ5ecWC;ob;Qf~70b^wHEwUYIGK8V}41mj3MjkGLg&_Jk6kL|b1 zZK^3xn!>;r*h4QQCSi=A6`D;-7>}oSTk1|Ln!>taa{0cHx8@WImh(l?gMLJ{Eq^ri zG@(eSW)FhP%=fFCsI=2ov|8brabF73*vd~;`bcEzZu5OJQJLc!ITR8dyTvV&8=i4H z!m&GMMt;aEacCKXHK=B5%AWeTa>;JU<&G?t>vHEKqNQf@u1imYh| z!pgTkO({Gbb2TRvr;#Qwwdx7R*+F>2H}(Fz%N5Q)KCF&)BY4OtU$mNnhmQy^dB!AD zox>lf*!-eVDYE~S^o{553;oQrs`|8LH{u+fgdFkP<#U}q%r9?mSMi02i;9d%Xh({c zU#fwM-nmeNTZ%-#)}}sD8+zYi1fIPK;%EH!O&~?~kpA4e)M+eO-7vPrebWIra99R4 z{`SvqgP-sYY*_Q{03xQi$*JrY-Hm+9Iot_C9Mh~K`kNYas5w>XXiz!Q$vVa3NL_yF zs7&ZyI=FJ}u%Ub52FLGc@4(}uL{{dR_p?Y-Eo^&;vPOtO!@l$7Lmp%T!J;{+O==;= zR|LeyK>5F;Mo#1Xp6W#_8}Pu;MY(XC()NjD9F-X47Dcy}w>~U*d|a#U`JW2i{s(E6 z6+gFZPwr=mUi!}qKuN*4uKgmM_Yq~Y_n++w0@3-kxGrbtm>H^nK3pem3dXz+kIo{e| zawP4;u3Jkwc>p`vKgWV@x8Ubx-+X^3d#A|W&|i91nLAI}$hxl|5B z^Lt;>u5oXQ<ZW zl2%keS6oAMqmuB$Mhe#Oli=YbQ zc<4a-Y{QgaW3%&;WvsW~O0~v&f=hnv?ey_;sOe8xD`vnA!a+JKN}i8*1B^xMsVDJS z*|_LLzxnHiS?XYpsncI`b>vahd!*?`a-`HpDPfo52m1%w{rp}&g<0^ITKFDu zS+Lwk63!PtRf0-^Ezl>a(6l_Sk*VavxsnLP=Iql?+%Vij1?HAiPxHN;siEMSfq{W* zOG@GLQdep1frw=@0kFV|F&^`%zUa1apf?OYyxbgf#j7IFuVaI&a7M~T;u0qJ|l`GK2q3AL*<&{Z^Z0(}l zGDeL>&$MY*U@BmO`{CwzxRW9`c!Lk)ii5;e_4G+h)&>?wFAV$XQhtbCz0W->QOgQB z`UB-ee4X{_FMaG}_KIVVB8+j0=T2H_o>=4QyGhIY!GFdicemn;{lWz%?%X3C#9G}~ z#*Z+_uHXiwhpt8UP}6!P;Rt9#D+RagQ&W>$rPh-hvVvsfXKCm!9NJuCdU3z|_T%%) z5dXhWc#bMJLBLvGkSzOtckX2Abpv$pRh4&Slt8v3*WvKI0xMeh-OysBd@#W<9rfC8Da~TTS_gHN%}$9~r0+M&*Y`iZd0-V{=8uj|5bsYD7fB3R*xo525#0M_ zPjgzX6OJcaw8e>)q-E63a{q)$H?HnCL%*8|I;Cbx>Q6_=_Rh{ArVb@)uAE6SoSLWW zy2L;Zxv4IKyx&a}53=?@-C4QlZfiF+<{Rj{i4?jNxw6swnSg168GUM{(Ka(jG_&?i2&* z+jQ;YY`-YJjmB2T^seN4Q(?Z?^qL<=vJT^@1M^vF>KGi5p2l z_|DHE*B9Fy5u3i3i-&qV#p|v8R@{~?Mh|>@)8f(sHv>ZcJX<(P+2mXIZJL^bJt(p# zp_{Y@WfcFCGFtWb`?&K!E~R))pBAb@4Mf}s`At3YvllFCOR4BVG#@~i4TYMA59x;MuE**VzssD zvSmiDSR&dPqsrIcxi&E1nMxsS+otYAcZ%{ErstMRm3-TeJG8j!TLSmn#C@gILu#IP zu{)+HA@DbcxMh(H%&+~1mS$EKzIt@0?({wSc(Yq3hBnnkt9Vtz8krK;6Dc%&M^%`A zShDmW0UT+1j)sibJd?h&n$|3z z!V_}FA+P#2R*ORM>7;U3h!@}{--*^G1s%pJ;JAvo#pSmJmMlrzO-ov*y-} zO3~Z|225>bkGDsX=?|w+FplPkrkk&1>!hb+6U_taM1gs=8?-cK3h2Fm+k*3~sOjHBqPc>u z&9ykuf%?VL;-1(&-RT~hQTQOdCTDv{UUnbdyHPkj_u}Le?PHy`ix3*S=%=d)is+j# zM@M}!?MW?s^SFjhXZ@rp7zAr0UuysrM;*xwmC#<&o^`R~ggqBA`1xU}vFeCF?IQQFNm--LsVU4Ulzt7Nxu<%eczlr7U{5ovf zp1~LsMM;P_jVdSKUHVjVir6@^aCwj%xhdCKI7-;kjD^c#Ty-m-{1HJrKRd5)G60p7))jtgs8)<(l6H5^zOKMjvEl+nsPtIN3IU{vTv%2EK-J%MD4g~IIV%SdT zMTI_l(!$kG+avE+n*aDOjgpRWeVzt>RySF3AIZqCOR4`xH67?X>@MpsEe!=sv8qQh z{>9VSu-4`vV&Bm|_KN5|Fem>6lX9x5mI!$~Xzeb1S6%Gqb6gXDPo=h+`q*`b_u4@ttX!nhTUv)hW2xYyy zY81)jJj)j{soiKuN>UQ(YpKSikBRAgVSi^CE({+O&80q6eit%lBInKpGR%~}Vxg6- z)FfM|twN`nopjEgIl9Lwmpre}FIwVL89sNdjH%S*qv=rII!f9~^zBd;|2G zvO7wbThd6N4R2^K9wCmCDr1N4O}$R&g-+d zvPmmM2SyArprEiV_qW(mr>Qz^;-T@56P7fkq7jt>j*e^*w$?=4BDM3dl_YzxL!%V*n(vn@6l{Jwy+M8aNzrefYIX6#erlAt(RyEh+zSMCOq0Lb* zRspWgDJp&h3xlfyG^RT`4&E167;-n9fTnQ@Kw-s3 zpl@pth(3cO$0VMya{m58j=gdd%Ll2}-qmtcfD-aZyo4ler4oX1eeNo*)({Z> zx$mA!;ctph({?6`NpAja(35g2AAl<{{tTM?*uRC{GX1p?iJO6;4>Iu_nQ>9S1^#_O zireb2I+t<7Smb?brtEK8EJdPHCh!h$C5t3Vlj}Ra5_?$63YG! znpoYOm4(&-^N4=_6+|cVoB!1fsRB3L)q3dy=niZ!4eQv!4!3er%22~NiLKlHDO8`>?KAXSm2^AnIiUm*j!z!be9z`K8jA=Jnba%RuM>Jp`Htz{ zuzg8DCFMnz4x6$`KQP?{SKnCE)(k5hOIq*aczV?CjgZHAfvI{=va}#U){|0%!d3d6 z0EPu_Sr9^`Q5!7+jjr#k{Uf(^I%C1*=Vo<{*uwgT`Opd5y53uho+ics19HrUc&2#56>L z()xpHPZdn4OOPdjuP&#g!ESS7-1p+?RJ~_UbLq*h3$(H(A!~^AgnO*I-#7BfML*)F z30-rEvLz*8SycJhmicF~$BvHuKxt*j{ z1P$+ners`|n>Bb}KONM<2;$Nuy@wjHAxW*8fJ290-LQZ1i@H>W9cR0lb@vlnbz&UX zbt>t;rb!|9u%BVZn>yt@W@-!Mxe0f8aw>T>gzICxyqE%P$fhbr{a}KI} z5%;^p1)s*vKF+=55e>lfm$01bD=nV@sL7Pv0hQ9zg^C9)53Nutoc>0ThMl8-1mi9@LK(q>#-7YAihL^R^0dFb^UXZgul`+e%vSLa9G|94xRvY=~Gd- zTYHaCXnk^YLpOfiC6q?8~A)abYo(0Rt(A(kaAIF zI~f3T>IqL$wt=40?{qU;@Vsj@)?ZZV*2aq_TSpx9|CCU+3Ir{49hxS5KjJz0byzab z#I-HM@|yJ`PX(CejyIR6$93X6Ua|@8knl(P1#0syw5Tc@x8Eh$qC%*;46IV1+%Xv% zKa`B}7M)N|>$qRW<)x&XQ_={NWXDt}iw`U?h4xwp^prihB*P@b%=CHFx~^Hk>lbp% zdoUgKF8&Vg%G3jMn#Jta=&f?lT6*_bV5)ap^@sE1Y#5H`7DW4KGKcbAyt@y>WPq=F z@QNv9o9Q~9alBHiZ*Z&H3Dd76-l_K1bm(~%&4o3Y>(VGL=JXEqYCm&fLK}|Of52^k0CvlL%5@D3`soKYj;kEbM zI?wUJ}}FNr~@vI=D{lbV|DM1_xK zR=eoNB*8r}wDgY22__6zA7`11t#Qmt=1-HXHpgAA0{6n8v@xly@826XxVL=r|80y+ z(C)|`59JOSq3i1|BWXM4Mx%NnS#Q?;TTfNh*9$LP@80u!$+?#x!QZCEyyX-f>p{w8 zu$Gl2e8&!od(QJ*9=S~8C3=_E26#5ge-O=v=}f}_zf3F)eG&N=&=eTgRrH-HjWhq) zT}e5Ycp^jgj_0hTy3@z{K1RXEv&Mok2N8#h;IM-x`|MkAA7j` z2c8xha-bp6sbzy{4)V0)Jeq0=X~AU*`iyEH=>^Xy1{w2#5%3^gO4Vs3u7UBX08Cje zr6ydtV7Q%!<{dkddyxCsE^7Q1ai4% z$M!TLF?0IvUg}pY*YJ(oHntyKNBgrhMg=9qpKwK$uoi zZXhTkUx~rp;|*gkrk|TSCv~gw7+9I&ApUtS?$4v*pJ5=Gs4a^00H@8SfEL19Lp`Z> z?`bi~GrNM~O18GE`orL zU+h%%F<&>F;I=Q-0EzyNT)pPdXs%wCXAN_Ct8p^efVtl+{gNr$eIlqp&}V5}AKQd! zw+_y(BL-9(&tTGJP{;k%c|`xs1b&>XA=(@LvU%xb@b2T!Ew!4}v4p>Zgrv%gUS609 z&lpnU!orJII3IA9dIz~1ssC~)4)Vm=(}Y^L>#mc>8Eb>WJx4&};0~?f*q5hFE~{Ym z-zA$%VO{PsH6>5x>SZ2*k&GH}b^BS_l~l%8Ldud)ke3qXSYZBV{E?5KD8n!!tRW>D z8fRE)aL4>>PXK!j)Ns=u!J^7JQpz=exIR|?yuH5SIUu;S1D6QlngSa{7F)rWQqK|4 zq}JK)?4Rl;T2vO>*GTbWB?)e^ejg<=eqMStC~yS6t*DYvLtq2_6>q0&Vz ze(>MfIctYC1Nf9L{_-sYWyA}hSd1WBXTIsb@n|w-$;8r4GkCSyweNK;riRimefCVa zsH?jwG!%l19{S#}fW&n7S|3OSQZ}EMU~X8G`m9r1h%+rBA%t4Y^$-&9U$-*$=T1Hp zKf7ZPyivDwNZLEUf5mox_sPhHClWAc)Zp+gdVps^aJJA zQ$RdS6dyut@AgNNjth2Bj?iM9(OYOJO=Hlw23?~dGQs+I&H9CQH{~+3*g2rQZKg5B z+}jA2F4y{+Qeb#0_~gPAxcV{ehYU6?ys0pc?!=sW{--W$LntRQZH?MOf^O9Ut z1v*nM|9t=TCOPt?wrAV;Np+)6vRH|l`a{mw{HKyI8Sor}-?psA>vS)WSGl-j~jwQEldF<@iM07* zu#eAQ>HQHB7@QW{Lx^ylW%5@3v-71h$Q4LU(SFv6ZEpvq{>8K(Q7^QfhV<99Sgms7 z*acxwzOiAGsJzsE5b{#Z51`Fu-hlTdt>L<>6|)&Tx5X`tWqt^pqO94GZ?ube8;l=Y z@ZY^-iq!5c%DxlR?6p1Br!zZ@G$`wq=HXh_ z{%!#!o%v6K7#E{F4WktP4Gvh1=EPIpjb8UEMicSloR1gLLQ;@6a|8sfDjxs~6!26b0DL|?65ZZW2wGOLoP9lRxe$)B26)Z7yn z&DV?WsCJ^YSUuH}>z)0C3d+2Ex191S8+Vo(?%sQ7tzG|YjQ42;aG_X&x9mu>sehX9 zxM=f;r3r?%iRk5J>`zE>%Dy5SlCAWy!~)-&l|91|*}U6V-EC>W{=k0a2ZPcXVUjlj*AKKVTA6f8S5d8Sj7c;9U2M z);ur}+#0XAV}T?^Mk#nHH10k%-Q}bgF9RbqCjK^iC(fF^f~A&#dact3@T+CVaN)*7 zD>0`)lk!mGxC&L;pbJeA07}T;j>s^LG(iN-6PC^F#1(y5U-DMZGTe>&5Ml1Rxwf6t zcyjXUbi`n?NjmPDLB0NUpAqo0JI^uRla{Ye=@;Up2Za&%jZdn!(Ju6NJKBJ?^@WcL z!p;TamS*~!2uc!!h1!-w(<#C$XvUt@+ufX33d^S_UV)HML*=!pufcP(Ekci43j`j= zou~L;u&Yo3`W*cJH#Qy#yR(3{YyJ4F?y^whSB~+}?$dw^`Po~5#vnO&$m!q(h$#TS zQw$`S^igAuLeKZN?6Tm*gtCHnge||B{ryT7^6P>fU`~J|!Tb*^t*AQw<#{h?{&Kd} z$mXoKlnTo^^uLW>vp_;4ME(208xAWKKSmuZKDIose-*cU$U+P83)jeR``n10nOd;lHHR;+ zwAA)!s+eK33}o87JUqQ%*>kjk0Ew67z;>OVq#od+IpG26gZU`yLFX}G9Gx+?YPfl{ zAjIK za;Q@eTDosJ7q&RGA#N}H3JiZzKji!U-9T=&BRI41v~#<5Lw2W=8RR-elo`pLR^n&A z`TF)aWd%IJ#rLw@AWKS*jcindhwcX2|9cLW__&4vMoe&f{46QBoFT47X!_%(*-O7b zw7H!r`Z%to-e6mPqPfhrbeDTy3m3RxmG(VtYy5~%3{pU60sEqeQ$wlgSql36cpQfN zkpTK{pm%WraS<03^NWYkl$A1gw>CKcxXb-iGhmY*Cgw(e?r(lqv6*L#3YRZnqekv? zH?7#@hU`i^0qB$~AsqYTKQBOpF>j37%cB^tFPo)yoM;RYCvwxq7QR&SNI2J^99yj& zHGgmgB9b_}J2vc6EI_#F!#dBl@%j;hgMmXgPi}GYea@4uh6$fVcb_0Pt--MSTHzif z!kL8T4kv8N%CIk<`T%W{bwyq6g+5MmsvC#deGd~j9QHq$(nP%L^K0_RJ+^mF{SGn=h>u5x=&5()8n}R>0_!TH}NOab25FgQu4g>d^gEO zL@HEVO({F*$?pwT_vBed55rA@`=u9t9i|-mxShC7RJ-F+^B7WpzOYEtc{LpzFHv&x zrlaWz8h%R44gP6#DSJ1Q?{Hu_arv%?z|?OcK)AMJAEmBfK_!UN$-#|-=D3zBMYRuLRjI|BP_aPNEd(d^~YQj{bQ^l9#sdYC?4KY#k zl&}Hi^7ULz)!Hssl;Wt{L&Px}OPrmoR0`kTy2~@+n7#)ML&{>>UD;HPj23yrLMCT> z{p&Hia>?J{)wUF9UnsGShWmTUEN7b{!XC8dX`!c@}- z$w|VD9=(+{zd3B1I^8-!i;ls;ety~>Q*D#lXx`Tl?`wkJ+OBVLJm1Q171UAQn6WaH zKdbT9af>09EJnE-tkByq-M2gBYw$%b_fmygqW9!rW)R!Cv+w|HB7oxBL)*HeVxxCf zyS^2qdJ4ci8D$r1C@_pbtDfmMy_2uV?O36#Fr%TO-t7&vTjU+TGw1Oj&*QdtR2GOA zypt|MDDULx#MNIA@S-Y8v7?RJ>%~X}t{G`r9{+I>7pWL^(9iyHmQSSOHh}0iU#WKeKO;h(S~P)&X7-e z4htx^v_8XXXR*!89p$a(d& zxI`*-o9SDhY+R1|shY~KWqh+d{uRZlWh^K5U0tuH`f)O#x5W&HJLnvh9Kx9lAs$%{ zYRsLkVCw8YYU&&{J;kH7V(MDav8brjMt^0m@;j2lYW`!YRLGxH-8m;UM^O^O+h1rl zt1cnE1Pmo(Ub_ z+A;~ZJ&}&zrM&f1ldf%UT?ex3uit;EDHhMM^1!Z-$or!bV&RW53!g6v2J`0ke4OC; z!5q(eB=)TxOw9*$R?S34?M0)A@N>t@N%$t(8 zCDIxT=t6fa#xpNF*Hio?`I~U{DQDIvtHg2FegtSLMJDZ2{$iFnDdEi1c8Gzkwa6l) zBCRm%?g#XLIZxLa=ZSlLZt_Jd_%G+7$_3p&>yae6lF^pfZ z{)#fugdookl@feU^A)Rr_(3V+UUjn`Wpc+f2;La2f9%%yYD9Qa2!k>c+@=mTTQ8!! zWw@yU7O|X2@!V?s-Acj29-47ylVpr!6gRHe9kiY@(?+R5!_aC$+u37>Mq}TYtNzT- zOAGe3r{NIq8&`HVcOOSUM%(B#>^R5XH2|2>)BT^B##1-pMyXDT^ufimgZgi%7bXB@ zacaPMp3`!5An5JWQ_FpC@QwawT$93=x3=X2hk*LC-7w8DnkDC*taG$Cqh#=A=L&rQ z?B}_&6Z4Sii@JQ#l_sF!PQCcgciTkKY|PY_0WAKs=(2H6fD3PV!hN>(o$p|0Nf=a( zXRP3>C~xT-Q7m%*`Uj5fN#?4g=j9~We74p>3g+%pB(>7cXQ5MM3E$!7&hh4MdSiKz z0bD@$DCMABn>yryk3RpM<<~l~0s8L2>O12<0e56z?R?=$uJ_x_j1q2q34`rqCGvZE zM>n{-W}*PatFKu$ZpS~$v$B37z8y!{3%c zd3kS0QC#<^x`W~7z4Vhu*fqHL^vYn%0>D`cRrn8y*H~J_I0f@peuO_jQuRv&0*C%3+ds)#zv!$FRE6l9{|M!{XAlk(u)|>BgBdgZ z1R46RyNa%+6l(wSpE#z~8)J8PtAXwz&#=!C4WH?m(puA)5qhVz>`H~i=r*#oHL^4$ z;=P#LHRV9RU*qNXX@PiA(Ra)aWQICpJ$)i3Sw;s#7Wj=RX`zXc zBZ)LDS{&1&63!oMjScuezD^#A!?TmV;~7Qs;jB5YXKNIB%}BFgVc2ytMaaFyttY$n z_zNQd3}2Eg>wV5N*E0iWz%ql)zZOn9GXE7JwB8ttWO(}M=v}7o)ujta^`iOwo4~}xlTcE^QcPv0gMkE0N0C6xe+~<{6aI>DU@2g z937CXvdZ)r@=}pbW;g&6j?E~cx2od$-L`0-Q2XjY%aflAt{cM-rzyy2)w)&qOB=2h z>jjU_Nmge^7Ro$NS$I59;g40TNZns$+N=!z0*O6f{|z}=y*p9+MBRU({tG@;YYW8` z?7e9Q=&~JZlL|OKP{b!rnvA^?4Ko*Yry4ABlP|ehHm7%G&%FW}9rk4W%kOGYNpPv* zR+{nm$Gcrv^*r@ijJJFLF9=q>Oo(91KWf--t$T4_!1afjSpgXAip`iHT3sj`!#k&# zWVr6%n!CF9Ru8iq@aU!qT|p0e=q2Y=wsY3h6e4mU6qMK-qvSkW)vB3xdoNQjVLyA6 zIJyq0_Wby$Kx2jXGx-#*fE(7hcRKP74)JNMgxa^W#Kv?)!JA#M@DBN-F|FlN$2m8t zASmCc?Iy--6Xt!7XnCR8OG$QbPN`zy1Z|h)j;^y*sI2C6R?Dv3!|aXWGavCC#d$0b z4GyNJr>7rs)*j}?_6-bX?Lu)aOQ}y=awfLC{r=ifOerbNUde;nI&;OQz>3Urqc`-J zr_9%J#3&Q(jV_HIyXjrp+yv(#iTS@-a*SzJ_bE_8U_yZTfaO<$6<`hsnA1$yUNT&?>V8koNpH5tq^0kh zQsBcywP{D*q(X}aQJnu%bO1PLJ!3sZ^gI#~XW>u(TB=LT0a);Saz=XCw8$sH0JCG_ z4JTMK9;`F87&m+Ta@~CZ1*llt1E5p~vv z!`fZ){0A7`1JQogWd*rLTZ>0m%`^GF$(D77--0lX%$Er^X>=IT zrU*?iGHs%+d0#gB)03^oGhmk+y|v@GlNLM}6>)!KPk+urR?%ScWtWP5I!w3p&bE($ z95$eDi&wzIPQJ`Bgu4=MFTPcp zrw%uG-RK`bkiGvUCCAaFd;*7(VRN5DNPoo!jCosE+V0DN_-#Y;P!h&7MntJLSu1-){k79M4b*#5P4Wy}ZzLRy&-i>*saLfAHowH`PwB5_8 z2C1%p*g9iqq1c#Y#Pv{J6&vsw3s(Pd@!ns$Llx1=)3a&Wdt_Gh`{0V0$j2P-Srxk< z%CH)0-iJxv6U>D?Im%uq_?_MvjC+P0F#ItQ`0+W=Zu?c_9}Yc~iP0)EP`wz&npf1B zRMJ#h&G~!sBVMXnX<)zFt+<8{c{yw4&e$AJ04#jQC`kO(&0GJtdre;DX%vLfmjG7A z#-_Lv+r>LE$fO|Vb-(Tz@h$#Tcqb<2w}f4|{{Ln2T4zi?htKaTH!6z1de70+zaSd0xB!x}ioflM9Fba--f81=W!a9+5!ErGdM9wpeB7Ri zvEkqs5-+WO+hCPfw=XWL%7`)_81q7s6*6QeXmm(F=!SJVqCl#C;YreRV(#MvRmyUN;7Ye z23>{HICZ}BLa)jE@QsqVeb@wE9kiMy@MkgWAZ&eR^kb@Y51%;8B~h=G%h2*J7~D%I zA`!3Dok!61$#I#5ge;=JFIyAGU)JsQe$EGRAH@WXZQV&`b>Gz2y{1iYPM59Fax9yy z&1=X?^E4Rfm6zUQ&&B3C;C$vhkB*a?Qe7p@&&g*SmqXIk>O!Q6#Y=k2Y|a-0zShFG zYmP_kt|NP>h`;7AmAfU`5La1GhBbsq>5^QZ?(!h4KLk(UZ`K|5J9H|)XwXyO)3Rol zznr)L+?w=$jYzX?TpeBpHdnNNcg?%hQYP)X9 zJo|^ZGgDG&JOyt4aD2sl^ui`u^b3y)cW$_{vUBK5U(y6@2~7c0Us!#m%*Npq^Jiv^JYwd)UtDDvVeI8yk?q1;dPD)Q7&%~r8vt}aN^UH8czC1_Q zQPuMM09Z;(;X$(oe7>|{>&Bkr2OdE1(izpT===4@=u9%vo<~v>Fn@%XVs^M zLEA%xFK2@z82N((i#P_%$s5&h|3ej&{pTNvzW`m9?+dZsAZ>1PP7+70E)P-?cy`ti z6Nn=_oA`ZpnM}FSt%NXFXKo`5l{H7u>4J5zsOFMzyHx)BFZYe@7}q2vd7t z#Fz8>ySAcAY@n&?buM-jmU8bnwCvAH1F2fHthafAmp*>Q+x%HSBX>#W#TbdZ&3#QH z8WJA_e=^!Kb(TG^rJ1SlX_W8Z*o@k6+rJ8HnBo3g6PT`IE*P@VGU|yS z!s7!s_BnJ1TKm#9#wJZQPxP9Hl4_ddH)9m&GQ8GiDjc`7T|Ys)ZcIbd6+&mPT9_N^ z89%*A`!4vyPtm~q*??`GI4POMsot@JFN1fRLK?=9kD#;t(%$g#i#b7tTM&{%S<*gy zzW^BPaMjHA%x0rJd_6^)s1NiRa~W%h?WvTi$Bb?9msAT6DL_=Zlr@w}d>s2Wik(Q0 zJ!T8sp78&VGq0=zw!0LQQu$7=20%QIL!IT@lK86=Z{o3&5DtDLJ-(Q6k5S-i=5jy+ ztg5oJ{r?iK9xISV-xhhbO%=X8t1It*Xz2W}-I|p_k4#|IW03^>yDZ2L(LP2o-cF;( zv)rGpjz3i3=RF1GHE#p+RjmiS?}-;_&sh?3O2x%PCi=UEV1ATme-cwLO^>@sDlcxj zx)>F_rklgW|2{2aHI26y#{V_yi)p(R%f57n#!&WNe3u`kvl51-{MU1K)hxk_p4VW! z6ms`i4ARLqn0%I2WGcks?dQMpYwC`2b?VtSzJ)xek9VXJ&$dS@SIY48*6&>XRx*wE zxaa!3#Azh?$*J_qi(GL_dIZqwL5{^u*z9K7tua2C6<<%-lf|fH^Y3hn(+Ss z`jo(yh*}L3RC1j~)#J7ChE2f5t5VNDpq=9eJg-P{M2BD+%OP`w3MKQS(X`u7vk)Wc z#)z`r&Upr_Eh(Ax{a%!C=%l;!{H2lR1p?$7IVhNE*?5pv;d&`A(O)xZV10W8y)2#F zHS(Hi2n>a(2=m>|!@=r5>TD-&oUWLbWLp;BiKbrhVWU9~Fn_rCm-Oc8u_LEO#-^ksE=z zWYHZB_7d(fxCeor4Y5%gLwsfY@w-8_)e)$M8=J6?LoUCynU{vYalX6dwv*Jp@{h=M z*5+hMle$e$*#!y(OgD)o?*C+CGRlb9R*9ZXCU)!BhmhQeJDP5^9NNUQfd{)Ul-$bx zL?Rt9_gADLZ<=I(UzODmoj0Ad+dyNejwO^dX#)cfq`|9~MA=4b@rt||EC%MmDI)US z?acPu>c(as+`pZgxHZMG^W8TGk#-z_JE!*|vQBpj19>{@?OM0}!F**St7bjL1F*cs zUdmw=aNhm|I703?Jh$*-{PmwILjD&}^3DK7BaDmg962|To8QYTQi9Y9?5EI8o?F+< zDygDj>StPK)GzVeLf(n;@Z|XWp{;B@HqjI(zuOT!qmop0p>)7sL)49upXn~2Bw@3Y zi)jvA=zrgZ`5LksJo?#D_w~M$g-CS#vgx9h;GPB^^&CP0M*dV)KP5zcWV(~OU`Jo6 z?rdd{oWbsECjCu!zGQb=^|bOCA3{&LtaAbBR7q)e7{6{jU^sYt%Sp)q9=dG)jaFv9 zafxLm&jGVqZ%1)xmxPyNVBe`Pm4QJmWGj56i8A6#>y9I9iD<>KqaQxN4rll7s?$n=Qin8WgJUg?S^+INNhHlda7So{Kdi$!5-a03y z2R{y9D|;H`OBx=JCN$&vh{l4X9Z`GkRW93;gq^+mv$-l@4erqL_jfC&O48EOxbazB z{y$BL2jM{a`XHQ6bDl;NOwle#DQ@Rv%JEON#7sjUJKJmu4-$v#M!oF1WX`XClpl2a z^i}_bpst}NhM;DPy@U|h>aMLKdv18MDyvM3G!}aZQDtR%YzdUw0o9x(7!&@2!kTKD z$VZ5W#P$I2^Gw^QaQfo3i*)%q74Mkd{(|QGtaKd8{B?gBsmgg(7A10ZvaDb4>iS!8 z3_->JY*uHY=Ysd_S?6;eQ{!enRqtJl&0!_~&u~UB17iE@<{|qV%mC6To*|9DTv)d* z84%5)q5QHD+IH;yW%VCKiStY&ajSCwj_E{4)Zt*6YNdyZAD@PqYB{PC4Z`nhh0xpb zMyl{nZ2vbGCuh%g{wym$V+6#%Qli0RTFCyiHQD2{^AEIcsAy0Yb)$PPdyG6WHmK+@ znQx!3BszrYOxzSwJFU?E*l0x;YMR`6b4>{xa(Cpy$;EMYsd_D&A7B0X`=k8s-j+s= zF!S~V)A=tNSkIcT^f&&g^^iVgVye;16NKg+O087Yn7A@Y-ZG1nM9QTs-h~(;r=+>o zYJnTo<0^>b>*uqf?C%0_4%=g=yV+?7!x*>W6_9eWsNSvUMe$ue`-LIz#`MhbBa6q) z9ZH{d%B)?V6yf<{wXgPy3D&L{M{WV~bkDn_LHy0KT33=H)63N-!`@wNy+Or8^kqXX z+gCUy>m6mDS|Un2zQ%HMSXZ>ov#=wWg4uZ)`d$gF=OIc!K^0pNt4mFlu$X>0-aRMV zEsNxFQYuqm-@oUIoqfOG+dVC3)`0K>Ya{*cK6JFo8Hm{!3wVlc&fk_wMBa4(b6-xD6CPLheJ5cCu_i?4e7c(7imfA?0U`#w&% zu@-K@Ue4tE9eRU&X8sS>Y?QiP%b zc_Lpn{DH#@b?F3R;qIv9KGRwe?aRILcRpO_EP*f&oo5rEFhVVaxu2-`A8eJ0ngT0m z^9QIVC@6kJlI^puLw4G%YjT>^mrpo5+V=5eQ!U$cptA=)(v z1K!ZW!?Ty_>^E0d2etq#c-Zn>Bp}L2zVt;_@Jl%Ty!Z-*ka`b2yU%T{C=G5}pk3F3&Cj)Rf}3$B6y!tL z=$IhQ7?Ml_rd+0ATEPBAwlsEseE2?~t}^bPWIG1ri}vTTk*>RXTI#yEbRBw&5xSE! zCgzsg7-p+DPx+b=GCdiQ&F z`adne@~GmIynBOfM2z0sY5PC7Yd@a1wzgsSy*9HOk#2Z^{|=CoI5Sre|8bUT;W3Sl zL{N7D?*tAW70Ev!87u$omrRpLVI$tGd*TnryA6_KoE|mH$H9Znro1f+>rBesx z52cln>DR_m=0H;?qFpLMutO(+0 zEWK03Hfl=dXe|}#I(#a;6wFcZqJc_ zxW!GQGg@0fAc%P&ny_d2^}j5X;TjZyKOZ;Bbsan)|%qrZ1%gph17yxV{01r8Wx*9o6`Hf z@nl1vwj~Xb&(Am`lRXw~QhAYz$EXKKEC)%V7~7`DJB3BpZJIc>1|>@Mw>FBsB&|7z=cim5Op=NflqR3q#{(=3u0Xz?0cnkE~)-aQb20XLv_o zO8FPulr>l-q{+8nVI}1Oh!e6qa$>6Av1>-z1S6qJ5IgQNtWCP>SnoD?{&%|@2pt7B zzMBfTOp)Q*JeC`isaBb~JN3xY_}fzbGlu6b=+#Tw|$R98O$2%NH@c?us6Uh`>3+50lOUo+_+ zNt%zY1{0vq0Nerxrvr~v=iMI#Y;KovwedSe|2P0^OQFX5ZOVdKHT&6;Q{9RQ)3JM& zup-Qy-ldym@l{rcZ~&B|7zCmBVGOt$`DVb=g1~MxIV{YmSRgXpdMUl4&0Jr#ejrT8 zv#vKTl`S>7r&hB`SJd{?3*FGeMS)eS+@_k2UzUU)B$_pLdX=JKj3%ETJvWd#@O;%} zJ&tx3sHMv1#!S|YDxFwLqrFbfLm%~uoo0znW9hC4<>Z?<{A-$0H#e45*#<8UZ)+@asC~Uq9O; z7i5=RrIXggkL_$3Eq4oN*uaL0VvS9+qb^6fqaR~nf?Q(_)+OsP#RnjMD`TFe`}Cn} z8p-vw-Z9O2-^50a`A~zCGcGG$ow+Io8Y8|{Hc7~PVEj)fwPs9hew(I~(B(<#6aMB2H*dKoU)15=4rD60`SWcu>hzX}6#<~Y&ii4Y{j02+2 z>!#zv#a2rfS%68xEc{tu=WA+nNVJnWhY2aL+&$F~rqv7P(3_M#hb`X-w=mCY266p% zW;_QF4^Tt3_*2Z!aKD>+teq}BpPfG2d*@U4b8Kcnwm#G@sNVmm;{}5Kq7h=~*s%y~ zH7B?C{6P*;5F-e6qAKYL3v&%O?mAoM^_D%EQkIqoXS4A?jj)c}22|n+-Gr}u?vNciO79n)A(Vk3$-wLDNfQqgCZoOB%aS8%1Rke9)vD#n;$nL8`I(lTJfVm~dfqWrQP1C1GDaEvnre zF>q9K%V6K`t(dx}^wzeIn!1&?R#)MS_NZ4b3|JWCemj%SAQJKm6#Avu~PH z>NK5S=9a6jW;W~D0gFC}lpLt=cIfk=7oT*mxh`b#!G+nDZ@=(fye2X>>5KAp$88O@ z3w+h;<5QK$p684?Bu&kf@CfIEHNAF~RNC^>hQ`aD=G7C{W0{gk9j}g%&4Pga$Pwe%6sg#q{%BWINNN#wlZXOKzJpT%x zdGtp~LL2;Ezw$|poq4BbBmeVkZkI*i5@yjof<={edDLxN`y&rb0mH%;SSt+ld9D;`+W=^^fdq%L zdaR~MQO>4I5X4oFesgXjC1zBI$X4XceZ=kaW{}BcIG!=a@0qZ5iR6|Mm5EddibwX= zsbHw!kc7+5aX2=VA@pfoJpfkpX*B*A2TTcn_CMPiK^P$vFrh<<_g|Tw^vEA4mr4Cv zF$u#Mw#}11dju`1 zw3R)HH1FVtg~2dKXq+J4wB0QW{;jHG{nO_U9qXxBDr?YLXtj_aA`W{eQ({7J@UCKty3;Tm1Tp=v~eLQ0>3-Vw!7?-SIJ>C%cAm%&o>@(|F61mtd} zvwPK;^T!6!)_+HQTtl9-ToYI~O!jm&G&il-QAh>H!($JQAY4W!W*@MP^%&3A$HD}A5rbl;Se^CqGSDS59Un96^$G5>s2*K zFD?wv3H8bM6_{c_EKxvQTpvJ^S4*}5Qo0VV9l;9-5|okG+p$Wvg4+QwPyBF*c@}J7 z>&sR>;7&~<$IVO$IXW3S>mN!PmWeate09_Gx6HYkh3|=T1N%51ptI1c)-!mgASaO% zQYuFYKf>G~*P+e;+}^=opXNR}HZ-1Pk7591ti2Jio8Azvg*2URKA_I0^iApEGm-mK zD^2X9Z8af0wB#$kIv77-$5Ue{u!vzBm!iN%a%h@K)&=nDZJ(1u9~-s~shb+f8l=Vr1BScZi*`rm8U z4LY_f;NE}DbrOZxZw{7{r_Ddfe0H^|Z)!IzW2RJ@!D8Q-hn;~_J3=U~MZxiZ6LDsQPPXQ~8eEM^@iWy%E6q+}uNObsX)Wlpi=5 z-P0jd%H>|-QbJb)8{oG-6Lh0oQhhW5?2?d@?)9nDuy=7hRDr#R?2e0jYld%QfApna z30%L_(&&5roW(#HEbuL3VpgbYgqB=d^)xA~c2zx6zDJOWt5gB0@6~r`hn^R!`YB2| zU1_5ck*u)huEZK>A7}jhX7)qv$wn&HI5$pA64gX}6Ld9>M6WTj%Utnlu3p40BKPXg zST@hN=>siPwQ2Em{6kjv)MnUFMpRfl-ExWjNwID4W>IA8yT)Z_!~KP5=fd;P^V$L@ zRfC2X++!Y1U}MYD&K#;*_G(Rv*CXvbTIxz5CMsWuqWzR*NwWF=>il_NbD7`J99C^K zcCwZ{-2`6yD!ywKmWucKCU%f~-AMX|?C1^I?Hmq$I}aLZBbD|KCQQR1&!y<2PNAMf zDojTh$-}=wB5M#Iw=hF4+h7yGM)p;9~O3|AuRvXl9vO3mfXccvWrW={hQt^V~Y7pLw~@H zRS-+sp4SJ`3DjqHu)$P-fEo`(udYiy{~FMk6WhXmva|wu*{&Lbh`(G34&(1P@CvS0 z%T$Y>^fO#s?Efx+Q2zD#>5&CsxGb1#aNoYKkS(^kM_?^942sf2@QmJxwPhn2Yh1tE zZ|hZ|pk$-qW8VilAEo9rl9g8eaYv{NbXV!;25{>vFQ>FlUiBRe@M*}83Aas_zNv}{eyPcrw@dwH%H0&uiwtQBMB)MWF9 zS?poubo=+S&ApG=+o!@R73xh7^p}aut>5cA`)FDX)G#QOPSFaK3Rt)b4 zkJN@I^Ol3{+{BqYD7s#R|2=bC@-I}nucZZW#e`);ZkjaQlN$w%@<%lui8}T&4mMCK zf6jo4U3yEQM)bgL)XgK`|9DRfje;LC#>aU^SnLmUOSd}_-`61p}CieK7Ivnn4wS1TFQ;i(+{Xr=G2rzm!K@x9)LYy;{$d}=Pp z^(zl^z-8oAtzu*;ZEn--{?PN_=o~uq_8V9mdV9XC@*S&li#z3Xib$K3j-2c5#tjd( zQ~Cr(s*S1RWItU*3(|1Nr}=ksz>C!`fBt_Z67o)__)(a9?yD$s4q0j?K_{M?bp>@& z`OxaA=9v4AI{1HOI{v{}<|I-TQAo`KjniW@58}vu94~hZqDadv8<^H3M}^@CnE|Vr z2Y2$V(|kRh5#f2tyhx%J*WRH#8A~T}qb*Kv`&Mp^{wAos8D5i@Mz6#hi7>c<`uOH6 zHieYIcfENKL~oVpwc6OzePH}n?l;oRL?<$GL#-ysVnhKDjar!}NVO8RgK`K{RR zw7(*%PM9I;f3Rk1Vyu{gxU~ERo)}AO%T0Iw-6rDeG<869gz**~weq zIFfrf({{eW9Q2vtmWR-RaO55iDUi}mCs>S{5up27_qInmTdB~6I;=DgQ% z>(O>;R)i|mI}lyz=P>4P4X_Pbs90dhqv;?$ ze3^k^OPlfqEBEcAuPWwUVzq2ay8ImTEGx|`nQMfq8+1pvzr;bJz>h6s6+Flm8n_)2 z!kXx;ccSGgI!u0zc1+OsW8e6CqBbE=aqMF z(2LG}bMt`wU5Ck8P(iG+;m-2!&E{Y6{`uxLAq(FdXK?Yej_lsk5~Tj8GD1y-kE6=0 zomKXPngDa_ZD0(PEKFFH^J)HwOMjgtN*dFlg9<0UIWdm1deD;S29JLA95B8*&soE7p&ud4}J^(6q!S1?WTgqjCi&pZn{JWBqZVh+Ey7kL`W8qC^Whq-l5nE;r@ z4FZ{bGfs0Feqh%vA16)8$lw9^B+DtR3zQXMBBcu@BzdNOXjQ4LE+IcVYNqe=El$i@ zGBzcPnOhBd6K&Yto28uZs7||ezqAbr{$(X1qn&fB(vgfwSSph-{!0q8+c*By^u2mb z`btK!>5W)`yL0itzN9G2dCe*kFK4l(#LyH>FWg^-c05> zVQ;83hGyd?Y;72-ac5>k-J%KIx09Z4|>SZz=BJoxI?le7piLx$atm}febCB<@+fg_3XWv zdl~B7$cFo?x(wj(vhFNr#{z+*ST&GVYST*6wHC*n#g-PfyVOM3;SU%z{_gvKD^(T5 zr;-y>3c}H4j^L0roUGl_!^H(eYzEAG_^4h2HU<9i1zU{X;S!c-PVoioDW}|b2H|`^ zR3mxf+k9`npq9t|IkAupG+*=TaU(`%+M!q}_@u|>fe-lxFoeJ4+(zLu7v(65O+~`j z98x^q04^-JxJW(?;B5!j9RvDISVDFeHYS4E0f8Md!V}|q@FlYvH(aZz9>M_1`JV5zsAsv(nrGyFuFgO29A3I$#gZ=%43H^6i*Xb)%E&VAeO;& zt}weYHmlK)d!$yKOstlW-Otm#ab|hv+8wjDURv~)wj7{iCDFmS5?hRZyt`;Ot+J$X zguZKAu|`>Icy^@&{bE`r@342;p-r~|!i>wd2M#cP%zXtWAtpsmP@GFntA ztJjU}YkB>dANi+UI`kUU)E|rc>?mMT&B6!gda~A=x}HR~>>8~(jnr8mJ<6P&@o3nZ z-AbMP5xG;slYgm~DJYzcvS7_-0rPKn<1tJc)&j6q3dT}+KbOQ+IH;1<><|*B&mT}&e%&m7!*^3EwW4KLk3Amj=2|p zCQu6Hc?pk*5osH#-SE+cyqGAufH)sqyOXHCmSv2l7kjDF8khw{>n5vzcxWLI@Sh>z ze@Zn;2se;?lR8L62}t}=8wb>8X#GyHkc#HpWucmpfGmw@fdWZgqY@U?MxM#Ecis2* z($x&)uqKwhf!O-0*()PSKBMb*`~W%sGv&lj4k*A@HjzqO<=ny*v&Q=-1#jxoMl=u&WQ&CH>PbV@JzV*9R+@^*O70=&_! zmwv#SjR%>Qgo#2{PTbkJI#Je{uQfDHX9-qtZ>iMK+FH1P9u?UZvY`+Qj`LN97-dSw z$nQ@Q8`hhmauBvQQoZTp&aaQisqXJ};xy`gL)X~x?0-Sf4nxFk3zL$Qrd$~le`ss& zn$LO$s;KEL^&mPt)rv@zwarc`htY@==|LrhDHYXa@Tu^CrJatB(te zV#I3s9h+CG8BT`p$K@XzpdIv)^goHFKeCTW!xmVUKdta=dZaFzRKih&DvBQXYeu4f zt*mQ{l|Z_7X61z;>OK%H`vDv(t!!s6_rEp>9Bh-o!BwLPkR@28wTB| z6s^U(<@@7RcM>4Kk#aOmLF)T@|Dcx8!X!t-k&iM2D+TW}HqPA5Csn+aoHXb0X`6$c z2WhLHI>WGfnmQBX9o|VPN3U5^-vo$|%}s#r`0>^RpfnGo+itNT?-z9`=l9Nhovk<_%iE2eq&6Ly3#al)6&`AlxSaZsKGfTPYU@2! z1;ew zshlVW^|xUI>4K+EyN`+L?oj+1a#-AnaP41kR>P7Fiza~PcgWQ^>dJpV>;HT3k2wQL zH@{#r<;x`iWkOT=1TL;{rx8Dlyj9HUxhy!o2+X$SMIqGHd^Nzh#oM-S zsrIt$M8uZY_k}KaPqg(0&_&t&aR9_XCY}t09d2ZpWER~Ueb2q`BmfR+9o_S}wbOUR zWH}Hec3X9e#vIcA&vc(AcK1qw*MR=<0GoL)TM#LIE1&XyK zsAZ7S-Dk!WZ||;^cp|-Xvl^y`-Rq~~rurIL3F2Hjcx11su3DJFYQduTNvL>OUpsNq z>p(W2yvc*GFZJ$%8MP?is(lUkcDR;~)CBs8lDXpG8ss*#{3B!Lu&l!Pn#9{i^D&ms zlcy@Su@zmlr0FFLOlEIxPlvsY*nAd0)DU)E$88a0WU>t|F10n^eiFNfcDKDaxMRmn z>NaHdM8MYe%-koj;mKelh5>l3@wwtMw>%(s#=rDPYDem4&T83JdML`>uD|!uQ<=!2 z-gtN@>OIwV(UB3vG4`x%-tENjrJqEp`%v$p1By6t+A&o;DQbcOZ)wpjR}4O=zr!{c zN^z2vkzvTxS@D-{$f{~a4d+~I(U#j4sdxP@`TrR~u~1Dgfu|fj%5yepzb&?zH%KjE zMME0-rN49~?4K4uuALUbsAeuZy8}ub?&hWE`l>3mD|o}?#zqTUJpW@x?wsrUC@46S zmhfKc%v~UhHEX61z`|_Y;y^TponqJPnr~ZGUzQw6V*{O%md5)DFR2iQST}%dqyGG@4s4A!M-TJR_ zKnAdP0_7a`y?*J*^Z&O7GOUXFsWvpY*5LjU;2!mkV}ovoGc z&)wbBV+!EwCh;zm>}IA*8Ekr$w<3$wQI$WxrtwZc>IqS7wqtGu^2ZF58Ki`v#JYjP zxL$YsXjsbJ1*7|mXZsE2)qbtnT^!B4?5NCS5vRElmBx4DM4v)z7w;{1JGj~d6cC-{ zUE4E5srEB)(ApS!tEN!Vxw7Jo?Cot0N{x~omiVTH*`y%`sRbIkZJG=bG_#n#()(Ji zit7+dCyD7Z+nm58@5H8#eep33K{+DfECF0mglI9wJ-Mk^v9~fs|3U$R_zcKz#=&)N zyyYA(Rv)Cc&Cf1Q*CFcc=aFv=+!x1fnA_EwC*9wiOM&QpJfcoZq);VdF`Il{L!a(g6AJO&y z(d*{c_%cG{(hTVnw3T_Llpi4$z#uN6Zt}oAZy+V^)O@MOfD^d%GWKXS#cXwzUdrc@ zfloRfx88E~yJfL)HxYta<`iliy#Jbx`S!52_v1wv4nW}ntyEtrD}3#X%Me#z^A{2N z*Q~G+Q{Kw=?l-R9!TvEf?V-+>tGK`!Egaec03lWS{!bOwuk7EVBi+^;gv7S!03|%j zT1Q?%agX|{zcJz4`tMoIX}>P+Jh{wZn;p?nE&b`}Q#iy@nSq7{zqVDh7_hp0B(>j5 z!MoSvrz>Eq;2cHZOXc*u>(hDOS+8TZ<9U)}Wr-OwzP1z(9HUMF|3u7*p!!jGbZb*n zO2STfgJ1pzeJAyxq9sZfY2mYrHb9CrsEH>HKYsV9zV-IByk4g%EJgpU_w~JQox0u| z>Fei1;AHU28skkp_J+AdqrHfl7o8Mj)>W0ym^kl-!Y_Z)ByK3Ctq(nPGg!#{h7N1S zeS(Q(OrVElukW*BYez@RO|h9tDg?ulG)0w1OD`G% zF&;YsHX-k)+5v1D23&!GaOICEfn&oRTdf7|oR4;ks~7zctsC6cXffo?H*Ha7W0+F7 zpN=E$+MndM6HaKk!(rrShAH`?e-c|h3f5YzrC1B4+R%Y)ehTiS zTVr4wm1W#sY>^&2=l`87^gmDf_cx^f7wf%v@j*TqkLVXO^d@;9Hs6sT9kLXm$t_tM z!ZyxSk7E-{)i|)pVlqh;)#*Z4fk^OxYY&Jk(IDu1lkgzV z6nzkvGYsYegqFUaJ#V5M`!|0Ym;oTDB{Rc|zB_@<$NSqW3jpp^MKg_JP=pjvj%GuD z=RR_-I_m!5Bi(!xLh;{bo^n>~juihy{2hZ8c5NS=-q@M_N5Cco0Delic=OkVe*lxU z{Y?gG?BDYh4N!K&V?qSnxPl8L3xv6HM@z6g#;!^Jk*yROiPIV4K>BQyI4dPGJCz}z z`kJ@8j9FXEO`4||nNB#c$yR!Y2&n*UtO0$NVpL?0ADNPFd+p(icx6E;=X4>hgfq4_ z$w-b-im@t&#T{uRQJN)Exie5q4>@Jdv2JBe^$OG% z^Rwl>=Y2?_8==)gLkZZJBAc_>ZUL6!X6QloI($FtHf8zNt`zfKK+N@J%+IQ##@ic6 zLWl|X$yScQw-_Xy7Z(I5u~ZX6zKPDr`qh{UG}@WRGV<5_sG}{WU}}a=n1|+{TpZDBp)>lMBg%&s-~!E?;-!u3Rw%IrdXY`X z5#3q#+(p6J`u9rd>ob=qNujha6|K^4W#cWs{+}C ztlAlr$J=qjUESQ`8HkyOdh9|PP|L0r44DRFb0{t3dxCtZF5Su#5a8mn}E_UE~b;fsIWJ0A-tu`K@o z!C!pQ*se)i6_Ij}7TYylPYu?z(|3Oqv=SS+TYGI*s`ZNknk`{*ti%7W!|V#(x<5{* zn3#VEr_Aej|3$7+`+f3#7mu&~o143;BtubAeKJnN9G+~b;zwnj3|wbH3n`IbvYte% z*3ifq{q$2OU||_=txy38EK`z2l=#@<@>3XC+lAn8^u`1k6Ul0wj)ewMLzDBwMs4*w zd#MC!YK;bzN*=f_z{GmVL*P6oJSYQwG9xW|AjoOUJPT-LA5p@fvQW-}HL=yxp;aE1 zRc69^^Q=rMQ<*5If|iXGi?Un~!~If>W`QzdQ~AfDX&=b2Zzjg3LTRQn+^%}7p-97m zaF+bwE4fm@U{rb!fxUvQ&O_HN9*91wuAg4Ko}HxqdI=QHGW2*Sx=l3yGON{weB%pxhNUI> zy|Y5T+1lWV%8M2*rK9NI$(>hzgC_Cvz_B)rQQc~q?YHyeMdNk-EZ+L{T<1BbA<{S& z83xGV$bJq9q)6i8*YZlT-$^nMlibd$@M8D(p#-3ufiqp-p|gv7H9QvEX~z2B1qn4J zE}}bDfN3)Avhm)1zIA_|0Bm6Ap3V;nfk6{Y9zWKMPV{<*3kaarRcGz|Fp1UA`aJ_i zO@7+(Q!ypAu=dY+CMAw04iln;E^%dN>oa!`eUs4wZomZt6U*$xW25G#Iq>3Dmq;Bs z3P7edg}hwWUNCCD5LrwoF@0%D6-c_lR2VC|nj*SL!YJkQ5_CwLIo$hJkl^>{^Y2%V zE$0Mmkmo&SwOuDw?3TZwcKmOH5-V3RVBs1kn^qMM3NZW4xPorI&Ap0y%DdY9H;K&| zUo8a}cmp0W<3MAazFqhEy;cHqXjQVI79bItb}M+?8vl;ERNX0>mUgf*0ITDmfUxyUx&BH3|$m0K4?NEV=J={bC`B2qjiCTuKV-cMU#cuk;M zqaNKz{~GG+CTUKP5Xfq4)w>C&GMjK^jd?x6H~Bgb_Fq&b)SyxxZELBig4)Ot4tONDVBYcr9%GL!CPgEfWCeM>=(Dj zuvVy8t&&OqJYU`nc?8(NCU&cQz8!+Y>q1`@bnBkPcAnXQ2g$pvr!VW#dMN#U*b3dZ z^E`AXa5&<|c?-@-|2eVe#mP{Z>zs4nyG0u2QB5wK#;dshKpY!_v(2&#o4OwnT&tmH zXG51hfM&o$6g_1o|2@p-Xj8kTehzXxe_$4#Qe)AV^!CAZoWEs3ErMtB6J48*K@us5 zR-+W=*!N;IC6hc`%Y*sN?UJ71HYNX##q=2k(FaMc$VdZWgt-#G#{v0Dxmz^Q;5XY3 zK+M7)w0OKc<1;kF%-7GpiP3&N)uit|7KY^-xOEXLSuA5dLued;KUW zh@zP11(phRJ|lXX;$bsoO0OpqNVNmXm2HphoSy+M&xr!zgfX~B|0K6N2Y-2~m4I#) z?635{J{`ao3{-d0%X+dcR&2nvYR~fzvAQO=YG?`VV2_r*=3EUmYABHK!q_i8x|u?U zof<^z-W3v&m0!Gj&?)weP3Cna84wz4zDDvV%_St{zb1!S=FgS5)#j@0s!4x(N!qh7xAd1PpAJs#;W1R6IG$*^?T6Z|iw=qY zfZTB+EWZBJDs|2pB$m+BTgYvlvX|UBaB~4sns{LwB@mrUvQewFV?Uih=0MI~r|b!d z_`m>K1cO!9G8HA?7`ZQcVWh;3SLK%%9+z9?)$>?H7(Tsjy@c$;t)cmd65r-tylM6GG~Fjx;uI-VNsE=5sn25J^5zW##rH zrhD`(^L)#SQFs$+eCh4gt7b&5{^xWewgk+AWfU9$aFxT~iFD&Nlk$?75OylyWm=Lz<`0BzXBhK$ZDmIlO zm*d{{I2yO)+0E8$7xArE0r?_mRif3T=lMm!l$+FvD)?)vH{Ox%c97QqMy`5-K=U8Q zK%mYEEv=RW$MgRw&JC<|WelpUz^{lMCL-cbTse5;=%KAj2rX7LKZHZWPXX}%UeIZ{ z1d$?BR<}SkA49NMm0lMaFLuMsrVp5&o^I3$RZenXUKZFJ1~qxh;Y!p5SJObp^us>{ zGF={>gg87dNgw(&7#}NK!}^Xnm>ncxldr+~M%LFlORxD=h7ZUYB$<9(`o%iFqUOW= z1F>V>{*ULd?C`Y&WgrWA7fh^o~7Gg%0G^WKAu* z&7KunIkNSc`pbJ`sA_4UP!vb%i579yF|loFOrgc1r9z1S{{$Wls3PQzu}eSvB93fh zcWant{#|OntddL-;;H~VBbE2q8%?~twD2QgE7d3ixI`gjyp#2KC~dyEsefy(4ky~M z!TmkNjB-pO zF>E>Kc*6|*UJ@_Ka6XdFrCK~)HFIgv&T#6Yu>hj`mV6{zDe9le9lz%K0+zP%xZ8AvPJ=?;Gz@a_}(WvY&dM#lqg|tQyU2rhq&+ znXj~RU8qNIOM%2`FV`Jjh#&B@OGn4@eLAw>Q83?*Rsk+E8Bpl%?f3{d@PIl{0xm0} z?DjQt(8hy6DMxr7#?qa4L~o=yd>?ddKH#VU%lWF^bjhPB!H`fgS!%^L)JO12LGpvk z!i=8{F7=2cCwWcp8F5bcI67vf7$)hB8Yej)6cH4BvA%c~_o#c7N1thwECcEJe|J_7 zmA=EO2Jid*3ibQaItT}h2P&l8-oyZpn#61DJ6_&^YHm3Dsc~U0M34HX?vJ{SOOr#) zlh}GCDgyS~wfMRVNSq@mRg?pq&$B-V^r&`%hz^SAsyCZr}y~y@9>26pu+|eP~6VaR_(OyWhQAMOhD|jIq2`^<)f8zkpb5ZOZ zN&>`Br#vZoAbPsIenXxIseZA)Ho|X=hZSq(#hGjEZ)`YPpm^d&_3*V8^qk9F6p&+9 zKqAqUB1X$phk`v~v*_cRUNfKBBW-_s+sCm@1cLu9m~NB+E3CmwLXqZ3%V>wFqoC) zdYdqlJs$!tF27B#ipn3Bb8qoFet#1I*DE;s6sn~_6mz3{_Cu;95Q!sr1pNfIXMdre z>ig9b%0zbVVYztIbn)?)J~-|+n1}(MzV7n%TQpn9q%-6D=F!If1*lY?ffI>VQwRBa z_Q%x=*lxU9%;9F?S9OHrtQLCS@6reF*Fk5|-qQc!{c(!lZ#jC(*BuOs-?_=2yH9mm zwc6>@&)-caycv739=qDCN&C;_8lWo2M7~+Ba63P{t^Dg%P)-Qo9ETggfe6PED&~lp zQ|L30_(gys z?p>~@o_jJ|`BATjb+DUS#bf;RjQ{{o@b(Y=1YGwf{m%_MBCW+1ze|u;pv7OY?N1)Y+YtcAm>cjz|~I@$FWXF9xzQ5g>~wXr~85n>{QjW2#9?{uzW zlgEr*emxnZ{^LUh>PU6NF?mjR&0d|gT^zqUL+fr5<8)IQCdELU#eibvK?0*_SEQs~ z=)s)~et0v$K}_kVc+@%eO3uQM)!MF>R@jm3zkR$PN`T5C^pRXtqR<(N32DF&gb#qY}$ZTbI}3V7?W`Kiu9^T=A8>=yk0! z=cughrh)3OkZ8r1V2G{w=hO#qU{hE>-45;)J4_&Z5$4m5u_E>S7fW+2tDQqcF2Y(m z1_-9)pd=cwKuHkeFYM{K8-y;q9lo1q5UfxDxWE%T5FS-6)jtAqBHwR8t1LE{PHZy% zamfCD{Xn~Y@(`O-vvt;n;l!q%jwO#YwpM?jDLm|ihLLTDL02R)xth(%F$1+CsYY!fEyUlT zR5@{^-O>F1m+nD>5(BU)moHMOdOpiv<+M(3%$Dt;U?e$ws@w*9niEs=m4s+^t9b&ce;mXiLKBd%)x_kYQE_Iy zfzev1pG$zv!B>r!N9!G}8#@CL>>uX(1l(Ts8M`{v646qh+#h~ z7MXVsDq)Z-sEOb^Xc}rnh(jDBt6T6*gWkRj+rx+P|w$2(TJ!p4vqNOx*=G13t_QWa(gi0q&T&gxH`mS}NV1 z0?ECko9lYvXbJLm;N0%ecOO7c5R4~KNs=W>10}XjP~crZxttQG@ci%@9V=a$$`Pbp zbjN8Rv1d5@iUr!pT!rKPr}zp&#cy#%VTn2UH)#YDuREH@2n?m|rm4mo8C+gg!BGuE zQot#6!~!PWWvHp;=;TnnU@cdg#6J?S`qqr}3MbYy9O5cp(x{BDB zgVO=Mk5MBvuUu>}GA`TvBRF_6|M_Bqkogy}KuSa`?*g}rH^TxIM{n}0XezH-swS2@ zzZZJf5_6%fhkyG=B#r|89^5EQ>nTnpu_((pPJj-De?AL>%YRQ6_}7@W$=;q`w{$5O z#d|ShKJDmLg0F>YhNXn%6Ey4d_d$*>4=NwMqOjafyjG>CrAcowfgsLae6dgzHTF8AGg=ilzlK)WE0gL zk3G6O<91q(vdwU->TX+3Tihw72vngmd9U;3h)sM^p!c0kJA7>3a$fwtM`A{8?=Xo+ zhWK+=)Hmv0qfHnPzmhfRGnX$|>1%GK|4?Ki$i8qAfBKie!kNJuENn;OA~h<3gLq`V zj3V-UZvSEpljT=B8f$zd#Liyvn4;Ulk6tVM)gdi&12}y!`siX!tp5qyb+8}en9K-a z9KVb5-pZJYG@3Q1+j7x{Fn;nzD;&B0pln)5L%F;}`4rwCEWGhg3*dr0I>V6JO&zz~ z*$bkOzFm61tj+<|{c2`*GU{^nV&VC5|1?T%9-TYsJ1^Bw?KuELoqT@9A|i`_xzm+P zjGRA~8Op4-oUMr)6E*9K8uqT7^p(KBLW=YE>l*HA?IZo-8oTNEUrnJfgGNuNfEBU& zo0W&lUD0`i0P&QzEK1-H7!XqU&DO)&GYQPvN9mH*vi)%g6f*rIiaJ!szbh*&Jro4D zh~*O8CJT=GoVB+NiBzDhJi%io99il!jUs<95YO53eQy9n?$z_l%j-eB*;{D=Eu*ec z6SI=Lw{$~i7pep!@a|?b-8np|&AL6`wC0*DJ5F}U$?RQl$u!gG)-b}~#~VV-kS!Ee zr(`r4G}|J&YJaMIK$VD^!R6%ta%*s}LM7j{uJZ6F#eA3x%$1GV@^Tdw2jo<7o4$|W z&J;HcC%YejLje5Pf4=e^aY^0sQAgcdFPN`{i9KJ+#Up)=zzxoCVx9b~dS`Xey`=E| z<`+K4hisEDPI`*lKSHrJUZoZGGPd<9iNtH4?~~j|4G(ip4euh|-pw%K;093WoS|{` z&d_IZz8`zX0+a)YvBi&rHwwB^XwHi?2rd(aK0&OuS%epTR&9A2?({kL+Q=i{gd@>= zFL2sjCR8sdRf%D}J@-G1Ym63hSdZSH{;u;VXZVv&x4YVun^|rG-6nc_^Zu6N^D7fU zCR~Bp@$mCfzh}Lt2sj15vz}k=Pwi9*Wns<5PmJ>lU`^b*%_ftqmh=9NZ%$uYm?%Qe zA8RQfc^j&{h#kO+HBz53+$y~xD)&ALXw)7})}a(V!cGL|^Bpk@uj-TsD(29c-|Bk? zYup@nv*tT>?jO{2=L>3cxj+*iTlip3i-Dly;{wLskg$>yK{DI*kqe}~Lr<62? z1`79xNL{XYob8G-Y(ZLfS-el!*kQyp^<0*4fytU2`Hua$8B#eqw(T|=wO8xWb-hLu z3Tt04>Pz|_uMEEM%CeXCFxs(4B1wjd-(`-*51@-rRUyz zrkeOF9m`(^xUBEA8XvseJ4 zM1SmXV<|k+ex^FddaU4+M^GB@3^NIP6zD6>y?8-4YF`J&nTMIv7lg@OzaO@@ZN`@e z`S5`3a+dq^eu7FbvqJnehQ9k7-*u4izWef2Q!Y>#OWJZII@)c(GZz|XmV)yNZA1#9 zExcBBTvB=bPT~@MG-(JfY}o1@yt*Fk#CxqqV}uNFdCz__zLe$U2)V`Zx`%J>F=;$8 zmCqC#BVDBKL&sSM0?~fc=w8$>v3O$fteTRv7qj?jX2}24G4?slPs49|7Juj!B0X@vX4MeS`YL_YcLw%NPaaZ7FoI!$dY!vS7^k! zjq`3itW3afx{Zo$ZZk|TJ^siIVJzhF-c5790YnyY#~sxBZqrC)vMo)*?&3-CdpG*P zN5^wnb?*IHD8~7^FoX*Gm4%YhtvNjuiHNU2Z^U37MM=vJa0*uP!cn^SVzv2M6F7b6 zX$JTM=50LDr*wVfg3F9cr7sLTxh*_tecnXi4CCMP8krV>`=D0FjPOCnt~c}-LaoNhp$c=TaWjr%E^YVU)~Z^rDn#w zZ|d!G`A-jS!TbF3liXbqhnpJ(x>c4!R)ciblZ6^t0NlSHG;vqZ@}?USp+3WK8)zjL z%1r$HPKbdgDo{E=rV}pCRmlDyzTN^V>a}Yh zRzRgB1*B_e1W}M~kS+l!0ZBoW?jZ%FK^mkKY006aLmH%|OS(f~X6Wypb3D&;-tYad zcP*A@4hxjwx9@9T``Xvucbd#SP8tVx_ACr?oSJo4LVs_c7@7r|)R35}sIdH2e0=E4 zVBQtUc)IdE;xVKulv|!K$e>8bX5YfDPhc~GdrPUh#$tJ9208^*bxSeap$oq?_!CoP z)R*Axl|igVxtAV=kh`CmbaQNkyN#U%xL1|PEUXg1m!rs7{zSV}yW;4fzev0J2bV_EK_pW(O$>@|ZZU#(qs zuK>^qn3K<-%L?<*S)%Pnr zV=&M;hvE%S>mgz`yf%x{<@-N5+D#5dT{fJ?6y;-^%B)ZHPKeZ8*rlJ46~`2sOT#l1 zEH1U@odnW-r;x6eJ`0%wAX+E*YeZL8_i6^_yqdqX864A$dzNUW=#QHpw})1G&OFht zcxy1pWg&SdoFBPh=|JRsa76ytGH}erbE<9d?}P>du?>q#)H5C+tc0-Z&L`83FpYE} zmyozcjW(=!4xOlKyX%D-Zl_52?#yUBn`ZX6WT)15x4v(wWQe2)Id9Otjm8E~RR=Ro z+&2UEYLKhl;e5Ad6tGf%@ObI4F=l0`?INI|v+bntgA2&^Oc16dCWVX?I?HLWv4;QF zv#Bngb@JlFU57B4doQ}8A+iZURTtlR+cCxl5r8T*>Y#+(m~ed?=!-Orq~vhmzhf=>Y4A-UF972WDY_p)B_!N5ZsIl zSB2#WN^@sna!^A@AmXDb!K~KL`8?DU=~xXY^UO`ox57I$68tC69b@@6O41c!g>T4a z&W9pf&07>w8`fqsRbgUXEc)nkCvMYrh1=YzHE~6fawGIYc6F~h>N0#{t9B8IdzU68 zN4isAnQaWZ$coc@&$bnx@1QYn&sT|8WKwFtntG90N9%}wy{W^@#t^X`x1@oyaNXo1 z-U;J%5f44zsoKQG>K-G0^KH8l|D=f*bLDfUhjmw~u-iW;-sT+@cwSkYm~Ve&pC)Lm zA?(36FO9pHrFe|^wK_XB*c7GpqbW~CFjkd{p~QB5YWM25qpfzAWBs+uJYsSpbA4Gh z%7UaBx`Em?wuWcv#_JDqbV`kmFOCK^GekW6dZj+egc4PP^gf67>vERHo)3lQaRQZM@wA83`E%F^S72$d;G(0O9KQ?+#?(&)B4 zJ{@b8Djc&((jJ%Dtd*BBNm9=bp@_e&Sp?Ae(^q5Ko`tf)ZkudGg^ zkm6SElT8(}^W%}%0lfv$;A;x(R0uT8F(t+c$xAj!Fw8O1Ak-I;Df(E$F#iLpKOhUi zKqGcVs*FEby-ij68r)!Q`&`41Q&7LXo$Gu^;cVqfrpE&Yd0|6ki&eY!IM4j~T(%m` z#B`Es+U&R(O;?v}Qc8O2L{$w8L5aREnqge>LI?g?(Se9MpLzS}zO40|A|N3HO zm-EB|O=)eE5!~-}R_&+|hKubFZ$>2;NxEL1w+(#Nm9~fIu=te>fX{X|Temb(n#Y@Z z>+_X+Zy*0M$ZI6er0ogYpaH)`SHY+)!`{Qbm8T8Wvze(EGULHBXJ2=yx5di#NfKj| z-D>u1N={A^yAEJ*p;s!EnZ-NGszPmdQi>#>u#o#sHFDb6HP`IgWJBGijagi>d0nr3 z7VPv8SxFP#!Upvr?~I$%!*&|k2P4C0YPI!*Yff|o5%80my|>f~!Bf%cwPbta=iPSD zrmr69?p&}6OGBmMbM}N2^+-ckTWv(st~UwGjc5wKUUwB;6w>aQPdx>cHSn5yXcNJU zDU=p1nDOVy`my_mlW{*YH8o9kZ+nx;p~JO&a$4R+^Mye+EZ;RXcbkM$_d^toD8-{L zaJ~8P&}ZqKqxpI_?gUP~M@92(K{y4H0k=osfEUu)2ekmpcY(*zniLg*%gZD`iDOI| zhFU`i!OK&r`FB`tp~c3Y{<^v^No(qidM3Nd6lsgHZb!v*MLRphy2vO#{l!bmZ!OPj z>OjngL7(-Z_{-k>D2YaQC!!vqCBv)AGdXj+YQAt?TwdYMAP$v2P?)x@*ox+~5H^<^N?>5Sd6)nYvt@e!!wW3$9N8Sc0;XfansUSu- zp_e@8S}45L`o?oq7|FhIS|+0ol%9sKQDp`gkAz7vocc5Mi_r&h2xt^e<5$Ay0-OK=|fl>t|<; zf)EL*q648~P6Gs*ihimxS;g|eu~`zbQ^y3=r?2fOjVl`T-hyy`-2`AOg4sJ?&b;wwl$fTvzv-U zMY}EoUpM9#;p|1lM-GaUYne{QworwW0T`^hVl1V3RCGxeW*R9i$GlA1JdpDEygcl7 za@F1yVdL7j!|Ja6&H=BrO76=v%JU0t2)Rf`%H+VV3&sPusNLS%qxHJkV#MuhJtm); zMl;1aXZVK&Z|-{j-G+1{H}jn|c<39iVfS$aqmUao6i_|ky}`jo6W<^F%bFx+f;Bnw|lQ$$mr zs-&^rb4u>pg3LVZpvhY3q1&*BnuWdk^CUmX`2`VkW~j3P<4(?uZG9xN>YHK$d;4`d znRmz1A8TppTkGqlYaSLAN7-%0lj1TXbrTE0)k zbGG}+k{yO_bFyiJD<^J{ibk@YJ9oav5)8O49=e2+^#p5=v@YMJ6!Bw^=l&|pyA|+{ zx&RJvwi61GkM8=9jkx)s9I-0K<6q zDvU;x|7Lwrz!&e=j>g?W&k!*BIswQH1vx4%3aQ&bxAAv=>KE7ctm$;hVlTHcKs?*% z@}o@&r0QJd?096JOMkIlkE<~DZbc*Kukp|z#kQu`BYCP>F!JckY_-y!rnO1-&=}a! zGXF0_zQYkVa1Jk-IkzEiX5zX6^u@W?qdfGxCPc-35K#+os%B-bcC(P!uFL+J*K$PuM`LOYe2<#vrocYM7{-EEgBc?Q;G(jO zLM)$P{Tf?p?P7iHzssx=0{4rP)fuq)qw2D5$C!7h$j}=|&mHbw_JD!W3pyt1HC=e} z>{U-jN1oa<$MPrD0U z&%+Kf>%|uTY2J6SUhWBj$;}*vic&kVQGetBn?cIE|E8)EH?r_{0J>R#>Py}NhsfXl zQRICEa=C`JNvA)+*|p9N!_VvVrM^_dungq-A_*YYO1<-bLFC`HR~G%Y1bD}o@BQMk zL4=6|?eOJoN$x*qnt<-6w)Gu+y~Y=Ug&H*_Jf6`sqGMkU;smZsy?q zRb*&;c6Z;2bM_R$$aP^y>Xhv3>Nz;j%;?}zCyP@!-{29Ztj@f?_=KS#_>5aGAM2mo zwD@#!WB7|!*~?Z5y7GfB;Qo-YBE83jFKS_jD(%+Ae`}}%R8NWzm%AvJZs_Tio8FoO z9X6An)wXkrt5-hFkZp8SB^C0g?st}PxVrG@{m6%GHz@<+EG84d=89{cxp$Q5er`Bh ztBka&3Er9xjG&zc7EYo2yrrn0$eQN4Mu2=gfx=TGCUT>hsR@pJpEP*2G9A$vGRr%k zVLK!c(!Qx2a$j%zHq+*!7^8R6Er-2PM_O0CTYV@q{8MP}nowu>3VJNwF4I%=e>gKr z7bXLE^%CkQ!OjROa(06cm_c#>B;NBNj|#S0OO>v57+m$P#pP40x{W6{(0pX63tcB~ zNXe4X5wHSz39Tb^K!Pr)CF7)iBQolHkc{p~D{Ye;h$xis>_4U~AvfK6v^Y^s!T0*U z<|E_vHlzIp=E0Ot7h$CRaHCi*!Q0{rrIJisj#NfV(H1*Y6$@4Pg^jkbueAtgB01SR zGj=9t9;#{*2k^ukcK`c6trNa$;9dUWN=ZrT8{R**>dK|T(<@-#?nVWHk<=z zTXYfV8gdOP!<0|6_h38SxY0^u->rIgzPibl_0pHB`5g((f>(2|@3{AZDfL45O8Y+a zt$69F(L$;2f?3J}^RC6-9sH4xIZBbmysyVOm-)DNM1#KHeu6>yz<{E3YHiBE>8!FG zUoMhjbvnAXG{fi2`l4AX2L~qs7Q}jEnZoIL zUT$_9`(DbjsAXgmv#1WmFv@Kl{QBP6eA;}v{%RoY^JtzL@nHIOKa*S(Rl9e(C^Yfv z;z+GGLsS@0%h=w;2THp$HF^3~ugOnh1(AT;9@T1p z%4k)pbTFRAXQ`kIydXG04ltZS1^wZoozjKyuR70Xu|5p?>RA|bfpWwRDc?43f|4IrqkLjH+~p@v!`T8r|v2Z0CgPPR0!wN!j}- zZ{MYm`SFqOOJHDNNF;G&lv=uwET|jqQobHY6YRV&w;0Zn9V;{WUiO-^lU4nbUr{aG ziPYlacOQ4!sm<6w5eBg3zh1&78*0IZh|4=xwL0(gc1=ZolClB?Sr>V0fwNT$f(tjU9 zLhM(>NWzUL6)uc6*C);pks{u*@I@~ATPi*pipZfpq|1QxkYLBcZ8(j5FTr(jcUbbC z4bapN%RHxG{^1mrv4_eUXljXI55MfM)L(`jIM`UJCG;Kyd{&3UtNv&Ish7YD>yLi1 zv)`pYb)HPvVEvFfTVK%Zj}_)mG3DDnw`tygyW3o9zaZo-H4u|qBg}l~uY5)zhoPH4 zRBXV1cD7y0Na_M;l6Ch7`ms-v=)E84rQ170&AYmysEM)pv~JxyTKhqaEjo}PswC*N zc1(2!5}Y;IJ68#SWg2Xw+nuXV0;05Rz1u!~HgDlQxELm@KjkrTxPE^uAY*4hy=7C& z5b>#(J9k!IOn00A3>u8}L;~JtyoTN2Y+y6Q5;rBHFLv23a0Sr;&WcH8o0{)77t4-_ly`3A{B_c;EQNswOy_ zv3UPwKmI>l@&<;f9AHJ01g(g>Ebd87vZBq4ir;lEI#=z7>%BP>K?{dJ;3Dit6Nb&7IuVba zbiIk9E!{o6B+8^NJv_IzGdzD}VfCZ>ttT{kc}=Xp@$1A{Z^MrY#!0tlpZy1+98B;R zwN_PovmPbbh$d&ex_)(W72>imLHDfQQfcdt*nh(323kSDjXs*+>w~n1FAJX+D{?nq zJOam?f645YXv=5m;QX>h@5`D6A0=@6P%Iw5%{b@lDVT+Bqf!!=hSWRHSo&}Qq&Koq z`|HNn(x$M258tgv_M5N84)qOWsb9E((r~QOT9b_f=x{7(=(lk{0idWyt8+hOu;@=F zBmg9eL7DDj&#tRRfXi#`ZA(0e-U1f5GVLW9N~W+kU^fIT>CK{C$tU3^D=u3G?5QVr zO})@G>c(?1Rh$82_QcDMIo;9hYJxiNLKPg=l05vA26fyu3nGEit^ zJ)uAO#|yxX{|@~x)HslCQEt#u7Ffeq&x1Us5wU_Ozh;Rz3nhCydhe!MXryJ^gE;ls z8u}qBBG)QeoY?}~ERmWR2&YoYK?+}7M`H+4Kb$IDCXs*?r=lW^)cMq4M(69=Tkg~> z4DQ{_FzkM0V;IfFfO!64W9Pn3<)WNcOMS;6^7*iTQsejK^HA~pD*k=yx%w_<&!Hya z%M5C9n{~I{<;KPZXwrHqTQXMMb0p#M#e-AbqwUi96)~>T6wX*pcI+&?y*n^GrjMH? z&rTdyBM_;ao2v}onTdOtCT)Lk>_Z(UXuvLaMdyLrc}5E~^Q^$)m-lq0fQ>sZw%k-B z2A{%HNf%1MCgt2H@aCkr1hD0Nc`^}vx6xtwY29XSYBs$}rdT?vo&&=lt9>PJ_4->M zeXwgfeNW15q%0RrlMqF+jlJJ=UP8}Z9xjvt-4mpEWG-Ly{X5b@c@fBKjPu5-ITHS8 z&MVz90vYsDcl?p`1jPW8Fu${Nzclrcx%X%(Q|*!?Td~nq@B2p?DdyIIZV0_<`F05F z>>BM#13-p%R10EiBu|H9os-K$Aspl1WT9%+zKvpJhoQ%M6`fm^aLI5h_kOE63!@1J z(dyJE%fug_T*k#0=x`b*PKynt0YxG8cg0_dS>h!Z)6%-mZI$RG(l4nF`64a( zk1}oMXLI@@PF*tcnORX3Y4$zdW#(<+8*NP1x*r=(-P&N~Fd~KbRwrM{9(c+=Z&5g_ z3q70t%%``YMC&7SOQgvX=6)aD$;&d20}D%PjGzx$e9;iXd=bX5WLUZAwtF$K048`# zz)h3eI>!Wy^E~Ev>pIpOtwSR5XtMccYM}M}OpZ+F&Vj!%gAgK7x)`M%?!MXeq3YsQ z!~|ij0#->Ap3UnmV!7IrmJJI+PVaFQf|H>9Rp@eF|3uq+tZ$)0?9Yqj#V-!fQ+m{& zdx|9FNy# z8XHXG$(o%Jr{$MJ_riDfn?r^L>xdtNu73_BfkP*EBu6PYqU~K`j#AQ7dvoXXaV%cj z*~xGGY1j)oAVn%}-+w4hAWwpk6*Eio!`z!ti&`Uqj4)_M$}_<^*}D(~RcCY{L>i)( zuh0K@^+7TLsUoCPN6$c&OX$xu*9Zh=&a1k)!y2t~A>WJ49CB>2S`my*mf>w9_r|?R*C`sIbc z=)FI@^~Oznj0*_ok-cSG)gt=ANVcd!TWMvJx96*Yp)Ed}k_#=3d!Iz#-^sQ@LmP(B zk;DqzT4vEpKR+J5+OOs-)WLr-l+|wZcuKrzP`5gIbrP3BaT(_p&YBc*3JNrVF1Y)y zhEPgA1{xvbOL-dJdG7ZZru9xmL|8d~m+~F%e$>JxxZh*3$qG`I&u=CnwnmVbz5XFq zKUR8l-j~Y{$1l_)6Brd(LQpNqzdvFi_Q(Shb}jjfsZ7w4*8q`_^L>A&CR*%#=dtuG zIEKdZ+o>dL1L;a&o11nC?bk3UVlO9_N$LGvUw~8WMRY;wk1ViVb|!hUy$_c5>H;7#`?XvKEf^CvYUu|NI7Kd07pr-*g%wyTq>~If-h|w!X4y<|@bYm_21= zR((^!Qgg_!XI$spgJc2a*jTm#F^WiquD^O9GGNzy$sFG#5U5kigbaeXGYvMH&|3iKB)$KoSG`5(?u-1}@CgRmn5;8eLW zy3B_&ED&Y_j#n>9fuHAZ+12k3+8um~g*W@BGX()aC_8n1W3xoVa>W)V^o%vdX+2PG zhM#9z0h_p}R+Pm^F_yWO+XVW<98s3-@( zA@bq{(JjEebFJDKUVkuA*&HloZ7bc7T)xb*SM^!2Ie)v4X!KOyZETo$^=S=hG}HHl zBI7@5uuOalAUOHYi}l-hR5Qiic2hX_f-*<}Joe8Z4@J$pFW;SRgx1&`p|aH8w`II^ z`YfXfg)xWUD+;}FEYZ9)G5KFWj~(3A%=~2+ShMA##m@38GwC=WOiYAbE9NkFj_-pmX4syp@GIP&cj#(S zZkJ?aQn}hys+PtgyOF!wHm~k&9%{p5^yE6()gQ`0Oo>hNMnwI+$d87WYRHDhYH~64`jdGA;+sPs&y8Y$LCnAC zRT;rUVzNf&rdtUD8!lerp#eT|VwVG=cr(EvV!RNhw#Tomn=PRm{LQTC8nQhK$+PV{ zBA)`+`DUuwg2EM1Kj;SvEUQZz>Y+w!#j4k1`JbcO8;)v&+V~X>grBT+@1D8@cf9zx z8`%UY^>`NLfc!`c6Y!b0bCC0q#n~P=#Oex<|FMr;JD0sxkh*+MEiTpIz93v4Wqg?^ zsqzn``ePEJpu}ZsjI9U()S%%K!+NhsPoN`G2syJln8s`NByf-wzo?C6*UDeCDUyFO z4{i#4D3)P7UZj^J9Zf5qQRlo#^lTqKW0$wumqaPPywG^j4Gq7?X6p5o_j+El?4_i1 zQ_SO6s-`jWqq!V-~O0q&ni%(A8 zV80?4aevr&(1t^I3yqH87Mf~S6f15dJxb1^Tf%7QrDCAb6{24JquH2LyfV3yRE_`Q zv_|ugW7FPdAF{Ei3(Rnzn1~hf`ogXMbBw?-M}j&mSk2E04^5I7IFqfi4yTO&rqbH- z2kfzTQ}U%{;e?s(BR*q*Z(s6t;x>XB{JXGlrE)5U?nq48Dca!_K|)@p&qvSWjhH7a z(RzQpgA%Wq3AWqob(y_yCsc0oZ9X1f?(*C<6ZLtmpj-0%!RmW;)o%x>MSC@kbW?_c zt{KHypE?e=A`tNY=y0;(8#Un;Q2ZU~l{;RlwZ~-(TR~DFqO8yj!zy`_XfVkY2rO@j)LZmvK`O@2pK}<5Tam zW2}N1Q26VAv;TbfIMD_;e6$LZwdT2^8PNU%6P^=`IlazPKNOe^WFpfcrg0BC~y9IrDyE3^c@0F!Gxfpgtr^y z(%^x%M}ULOBfT&1a9xmE#4#;b~EvsltWveoLj5jEM`;?Url%vwa_j;Paa12feTlFGMe zFH$#}cC2yJW&Z%#@V!@ewHpZO0FFm?`>KGS4!q{pT^w-Q&9mW^GcqvaEMJ@ak$h$l;4}x&Wa<@zB2av%z z_(H4T)^dic+2lq}-LjSsk{H^m86paZIXBv3<7T7DRj>`SAou&LxOZW#HsU}w=9_aG z6s7;6C)0YN9me{(77(CL!SyA`Xc_#66y93bb;*m`M<5O~EUbVa@bN=!p}CATfkDp7 zin)o`MGoGVr;)!$Ea=`cIQXDp;ij_^cBc#TUk)^X7mNo!M~$C_+6iY;OaSM6LG5zT zg~+F==04INhM zhwu?$5LJ-oJ<|(|yW~tZlLX@L)*tcr2F5)sq(UNp@FISbrviKp?$e1#{kFls$s3YA zsLR}!+H;np#9>RS#Mg|~HJbohub{gXwSSU2r`d-ziv9V+VnxtBy_vR0P95=n!u%A7 zG#_{N+`O6*iC5P6Z%v~1UG{G|*zHY9t}y9|g+TiT8_CBSTRE0w!HpO@ zZq55{rY?53TSe}+;1_C4KZ!b)7OYLm zCvQG6%0&(NUY{HHCUMiQ4rXSefF!LfYAOAOQ#zkbVoW89#O_8tjiOQjw-Sxn2H$Qa zM1pPtZaH>lG!7Zp{iO)dV;$AjF(m1xLuG)PdCzd<0_<6#7TeZX0WlDJ!hn-%OF!M> zE~f_}Q!}l&&#b_@cQMMxeTj%KF|kjdtIMx zbK5KH2&2VCGTju<fp1ME-m>) zK*;Vdh~F$-AHVL_@s!_RhN8Z1Zu^^M{n$ev&bU=Qt=7Xv{1L4!J;=_fdhBX z3|L8z*HZ09Ku@e{dYVK0sQ>Zt*4NUrE%e9}tTbV_D6uP~nb}=_yLrwXKMTBQ@hbtB zjUj*Rd-iGv4q<)>g*axFV{}MXlCe+9+41Ibas#TyCtvS2LiM(Q1aQR5%)eAoZqY|- zW_L8l<9+6AZ46rApKa&r6t+`HblqFb4)j)pmw>10y>5F0qi*8wLEkIQTJf5&h5;~9 zfK}iKj4G7B84}u{FM*4IUiEpdMdqa(0EbDMJ^J6s>o_=|i!AbjcYDNcg>!H>re zegp+kb52Zx(VCG;-6DGS!rj^jD^!a48d(2^Piz3eA!z2fH_?Q^tD(;cXIuKKRPYU- zk7I3`Eu7ok0;UjP$dXMuJI)RF_6$0XqkvG0*j1Dzw;L5E4XFJj{&M}9O07lHni1gl zW*c3RmK;jptbt&%Vq=)c;$1x?^sOiUPYHJe)1ARDTTN|p2=iY*#$UM}yn}ZPWp4XM z(s*Uw_4+XM=)TYmR2fuCp!<>sc;*=kJV34*X+o*cX%};V$1Js<$|nM$c^EV@+KAkA zTgu!IO;M=fj06)Ogx3}3Jw(=qKe6o^Pe@(%=MAgsG`Ix?Q&iIgBstN$6Cb2LpE)*4 z$-(AaUxbt_oL-x-CA!Uf1k*rwcsiza3PH<4$9caI@w%#9$1*)w>8)hmwr2>gO2kAm{5Xkv@NfJnlQA|6K4&W zIm|XR=$=>foZEu8Gf2TUI4=D+JELjsFdr8MoiCqguw8V}x@LGKi6Ca3=zjp!N*Zh# z!gu)X1T2KCHseL?bM@mrfD%rMtwNos!($nBLFGxxOg~~mDq|HRCJws~+Ibd5Jo~gi z+uR=Q9~#&-YFC9m7fn_eQE9ttPjBSkdtKe zh~}KcHBBV;&2e1KODSc)BV;*6DoeznS)Vv_DqcXYA{ysLK6s~v5qBkfut7>iAk#0PGJPZgwVy;8c!inxir3RA&MA2rVf8G^ zlb0}6{IWB-N`l^m?2ia_=}J4^^z-0)sz&x1Ud>I$9{PjOTo@kcz*ykCvwy~SGp1;{ zpvSww{AarsH9t_z!G(952U2(6{+m4hKVH*`VS1geCLNdhUMeX`y&TwCO)}11&-68d zW~KQJ_J9)q1Ii~;4Jx!)_|8N_uy&$-rD@$sN09)I?j>UJq)RR^Z3gVipb9o$l{G$yP zLT2RN>VTOz^up-}U%a@l4~oxjfUPfgEWng6=SOEN)NTaYD*b69n zJ}W@B7KL2_1QAkC?1)gTcfDocG+U})0?wF>weyn+W6CyC@m1vE^7|pn)8DJd0AGi% zF3&ZWemQlOQr^NLg@~=ohLbXp_8w5k*1ego3CpK1ey$xF8P)}K6Z-3 z9pO8A(wt1}Ue&dwfq|0{q})fcvpfFB5-%6ll;ZPVzEAFXG*ZvXD7wSbPY5c^OTw}+ z;Rta%?YJS(YUGuj9D)0<-4+ND99g((jrh;}f&`rrN`B(z2oe~*3&`b?S%LNnl}OxH z&Jl4RTOrF@ba(ro@_}?5@kdwikM4=ZGZYMF#l#doPU?)#4-m)bppBQf>3=MP@(5H; zZ@QH{Qn(YB0>nElvGzD0otuyUXQ=;EzyE3ff_8T|f%|eKe?DkTGzqN-!gY+jwQHlb zzdj!(RueG^;d;lKW+h_Ld#}J*>R2zu^JI(sHy-p?;2G$t&G0pV(@iwU1mr@K6XZHz zUB7unRi)N2*1S%C4}%gQ{kIxup@nvhCD@4k?q=k*9?34#1@21=Ab7{$rF=3JOzUG6 zLLlV2lTV#GGnvF?$e=pj1pFj{zi&ckKzmmcd@6#L-)gAk120lR${5s)rEp6xATi|u zZ*i^ZTJ{-8>bDu%%JAB$+`V<1T62I`;C(*x^PpQdIf=f3MMG55NN|ku)iT9+o!0s# z+E#c@eZHY6+&OfwcD-$nH}PTF3;CRoY!cA;YofmnH8&{;s~sPA0Nx<;5F!RW8K*VT zAF;O++8nbRp@wFT~l)_zj^ z*W>X=LRv=fGz1+k^67h*5k+a&e*dV0f0MgI;IsQ{#v>WrB0Co2nPi|n(E9k{3RTXL zobphsnT)|mtJ!U)t;MEX#8+w!u>v<1k8s>9JiFN;c5jq5z`N8}KeLf-lVI1$Y43w` z_KBh2>3>HV|K%n9BX9y!sVrJgPtRDrn;rP&Xwi)>%gkbvuz!Q2WFXUmu0SkeH1g4T zEW}KKCJSia2b5j2WO~KUdu|t(y-J%MAi}Ls{<}9;e2vX%lh8TfWzqi1;mFOdls=nC$xH zE_S*?{1KRsyuk#|(*>|x>5&W$NQNQp*e_59q+Fmm6ezP~4PU}ab(tc zR3*wm;0TlmLLtA} z&K8}(^pN_*(aR@9p`HRIV5HNKts1mrh5)umZ~gWj0^Q;qxDiLXpi?Ko`J@uB40YN{ zXQo^wDG52jLUY3tueIVJ1c#ELZPTeGTNDT&4LVLc59F`l9G*h0mj$>=a$p>(u~NSbz8Ibenk2S-;kzU;!{9HS;V9nB;z#~yax9t%LA~!1m+AK-Cv$&|}jh{=xuvy8Qi(QML;>71^oK<4Vw?n(=wql>*CF(t$k zQZjr8s45aasI*y)J2_}ER`K=1up!hu1~j<1|EfyG`i|sw)25G?Wug)3*%Zx3kykf7P3qr z4L$;V>=BbjR+H5L%1WeT2C`Punyee9UsfFIM@Lh|M=BFnJ2Z7^Cu8O#`nRVFz5!zskl%$kfSIrL$ON- z!6{Ez!L>XGJMJhH7N6e?CH@4qcTqy0R+oAh!`{effgrTN)Sp!Z{F;r8Efio@Vyyb$ zvp4`K6pn%X^X}jIZj&+>0XYi}eFN~LXV}p>ci@-b8G?XFuT#)Cp$PIdaYl?)h!Dk- zH}c`653sS)JBUFa#~}^RcOVO7z>?qhQ{w6M*I!MP7@8ctBvc12!R)>c1OXVA2`?%` z2JA~Fw5>M-w3YbOCGw9Kpc0C(P1uTZ(G6=WT`~jH+-{xcck9K=L-Tc2o5Iii1lSBpg$*CoC}`LM%KuRP{~tLMAq49~*Y64j z7TG(BvKUtcE&-6>^|^4DBbNZWq}Ap~P8*k=`cS!<+>|~2px%oXgW?fMlfS6ZJZfIFr_k4*KV}H}OoGIw^G_U?cUH=YHL-B05iC&jcwX(AJ(jo>&Nck1yVi;s` zH!*Q;KeFhR0r1}Lp!|`SO?mGnUhi8J9WudHMI*0(DQmTyT%9jmj{uYp2@=ej)Jtw@ z(BmWuZM#h3HqHPtogAo-d9tZE8+uatOPvKQOE?Nm%5U`DK-r9S3|$iFmGpqJYcD>H z{?Y`^*Op5ygTq*tjvYuZxoYBFycqwXPXGIbzdOpPB~n6RsyYfP33=+inb*kXyT1a9 z+Mn_MrKW)KW5cacC6Jn>?pElo*IG_!6lUnQxUcAZIvfm^zK)#;T~&+I?|!&^daa}m zrk9`mB}8tcgh(?rI^#}}_~a^q!{*vaL?t;r1T-T#Wc~%A|CN3I^+_X4d$hNpg<_cc zyJp&Hq*>Gh2qH($VCNF%t$S%7E`TDBeQv>q1YWs>@9ChcQ~pL;OF+;A`0&~lMr?CH zYx#KGz%ED&tTDi2V!GV#Q|u2m_BsrrfA|gb52@nv<~p~gDp(?^1RrH%E6M6|(sOX$ zeQ7OweRa7Bbdf~u2_G@{gO?N!U;34$hs12%NK-caW^ZD00=|2IfK|7T(yoD!*)LPX z;}OtF=yNE-F#x|4y%-8ya1ORQul? zmfJ8GEzyTXe!Sv*hblgG7XYsD$p~b%Nu|Ha^nOn2+3Q5wj0+l#A;bE*t`AMyF<&n< zH98u@!#F8_xfH8^e%D%{TW;D(^KGwgn-%DZ8Ry=h^vnmb)4N;CbxoZsZ~y%~HuLbs z9QHob)y|73D#5^?$P5E8liB?^k;56x$5XH1t@6H+<#KfP)qgz+$ez+u$ zr-E8^nJgcC$J`^J@!)D_y6UT=2&Y8~zgbNb?1vCTl ztV~dxy+1K&UY7~T2cXqZA+ipqb@A(X z#ECHNNy-QA2u0$P{~wC|`+rA>00S5qd2PyN{SGuS%8j3DE6AOUoQBF$|0F)@60!Q~ z9uUB@O369@Fg(n?U5Kz5?QT@T&oW=7DI~R=kOV~m2y_ZPd1c02j2+W}KMhVZ3gX*$ zZjzvJuIHyL`_g_4%$5$K*!ti6<9~cc;(`DPU@Rigke{&CELzkOlPisAUrfdf`z%QJyB-e>EQsa43(`_M5-1^D{6*45aeR%Hkw! zp^@8bgWHSMx2e!^gpHuPRq>ZhVc|jIgQAGEhvtcf55;X#c&%gU@js9xVy*xzEc)0> z6Mymm+T^nR+^l$)i6SeC%dl<;l*ZpXE!kwHq@;QW^=QqB)eQRT<004VTM^OVQR0U83*+emw9*T7tUsr+R(U=wKz2~9vS+q zwxxVvHF{l&B1$2~EOXd6<+?N-&kBL7juJ5nIKjrzptzcK=XO6LddB-lX$>beYP#v~ z=J)^OPgHT6g(_~7ZC-$;I%9PoujiF3g{W_89-Z!NYmTR>{|ETm;BLD9En3&%_f;j@ zWhNwc_SA-_?+4o75m;RunVjD7e0N{&mC>O-87P?y1>4i7!QgSs^mB~EZs4h>XAE5b ze%{wF@Y`DLkmKrru<&dF#7=SSagNE+c zQlZ`(GLlXVkE4MEhfuJGaBFI zs6vD;hH%~jd!s$)FfoWVe^COI2pE!-AeHvtsQOou{_iIjqftAAa2jZ?69PnSC6)JuYYBWP?Y=?@6FmJU*NRMC?gy zQe$F!yNpQ#$fLOq%bgxua3B+F0bu)l;b(Ebfow+>C~tx=x&RC8i&uI)vea2k=5IZA zL4a-P?t-<>bgHUu?3_ZZ&x`JZ-K3W&fjT$j&Ns0^|2ROquD!e90HGNZdna*l-UzVG z86ibr&h((rgB7rebL*DTQHy%=7Kvpxg3^v1LF_D>Xz>k;Q`e`(Mzi1!DqLsI?C| zae1H?>~NSj8YpgjoROLEmUj@p%Mqv-D7})ZgZo7u$irIIZ}gTIIE^0d@1xXv&}p=8 z43<~%{+W>z;!vxiNZpVM;Ka5Tw+sTVxI}j^jbBH7 zM@_rcGx29;<@a!MXBBzj70=3mP4K^u`hPut4HSR8XJW=mpiU30CfGUM8gm!l!TDqYqRX$E#VXXNakUz zQoD2be=9m5zy$~dpm0uywjxavwi`;+T77^sJq4VPFYs2{{k_D%J{LWEI(DEBe+I2~ z4d8^<4kjJn{`r-x5#&yjLETS7;If3}^|T9=Ot|Wh;SHZEYj;X?#l8QKt_WZCzAsQW zK*eqU3(Xh)!i*^zRCg}+3vcK z$s-V*bJ_%zU>0eq3Zdz~D|?a3XY&+jhFIGlgJh|ast&|z*E68mCI^PSQnXM(x4pDj z8A%~Or66$hV#3I{X`Tn{$AIp`>8AcP{2yK6=Q-%iNmslYRILq`r9SjnIzKnQJ|jbo z%2k5?&oB2seZ3r!@bOcgJI)k?cUAq-ZbekM0hXHs-2)O6laFC#5>RBx+sHM3@vS?> zavO_n3LS301!_;uNz>UD2daL|Ir;W(#-Sl`$TmC4e?yz;WZi#cjwuSV^GK6mOcSUndBNm|wV}-|RnXh7 z=LdRaC~yRQnXP`=`mSx|005l^5S|jwc{qT3WJJ5SO)6G7{Ra6Vx;^3QT841SnlutWo2(dX7-G-B0MOo zYzd*vMD|F?$cT(6A*&}NGbEBNyHrMINR+HHLdg1Gcc)IzbH3lx@B2Tm*Lj`i;JNS5 z{rOz;eZ8;iW-j%Z(JNLCD~Ag1fje+WpQnW8AVBv#|G|5vU9-0vuq)xpm^bS}d)iZv zx(`~>ABX?&e4b*2#cmEk2tXDQSXkos0L0#+4}!GgL_|cTpX$<~$3}th>%>TxiL^ zDGjbQ3yd4l^#LkJxt}4bMh^0dhy!15LL{m&CsX@+n)trA|1s@5hoR&=zCSj%?8M*G zZ2sGEg{y)yZ2pkX@kv}0Vzl1s8V4^kDzETn7NM1r{l(WP3Ot(L=9o1+>&)kERcX9m zs5v@=c@4sWA2&%@QdE+g2Bx0eOS%eYPd_NLzR{Mj+J2S?~? zVnzF`##0pMuSClH!EpZdtGwF{pf8$m?l(oB0?FjK_}(GSUe~R4S0sFPwo5G=2ES6` zFHN)+Od&7o|N0sky<__eD0gZCA+}OS`q}euLNCr&29~=tKGpd8c<2*U-yT58`=-%( zi-ithd99}xIMgbfM#Fc71i3Zh(NKuEe%kkETSVz6fPTChNR5xP3pgDPREv}`x+Hda z-W10F<$RDiSLb(B=rhF3>Czl>dZBj_WC+k(A?6IgHy|2g3zc(#d8@*vPen^g>bAa~ zv{d<~S8OgfvCP;p!84nr5Fjt%^m+HBs8Np_(r}`AyM@GWcPLV=_u1UDxr2g8w;P0t z*H6oO&eEDe8SaPVz>qAXWOe~9YL;`jCQKA8GVx%ajUUb%(L)t~P&9~WY?#e|{eDUD z@>Hij;7rjMx@8oO(VrAg_xq~;aAU^cQ+?vON0yq+rXF3!SV2r}N|W%!(kxI*=^snCy1d;Lq?0 zFIb+MV)hAbcLY9hKbb7T`zzS@m{-uW=Xe-Qdoa>Dk>AhCIWez=re936(CBfNgIBI4GORC5*uG9>?Dqh7k0eCk z)EZt#e0BkZ*S?>}Ekac;NXgl)!bJI$g9ytSv}Et&gc4v9Xl>)(}X?$H#MhzHx$& zPS8;NhkY^9%HXBn_W1Jlj*RDQmW=24QR|szsQL+i{s8UVkOtf@xzIl#HS}J{1kz_7 zgwAIuF?lrd5=>=NMF2so1N2hB59j}U1;R%=FvuEKk4g$4zC+;U?C-YOfV}e&6_m-G zu=$g_P|HSyC7UF;1X_fOGh(0@JXroL^RTm?T75y--C?4w<#A&@KDp%hOmJ8lbN3tR zz-*o84Mcq;T--Oej(e^DYwyQ8gHn6Tj|3+=+0S#b-|~oJl}iRQPuEI(i~JJXpKcG1 zOidOKLHM~EZx9EBeS9su5H)DcDLiWg&B%|ul?>=bOx1)q)wHx8Aa-_Y9a0x*V$GMW zLOTpc-K^El3B6v2)S3^X;2yD50Z^W{SmzQ-RJqG7X%Y>Mcs`Cx(8IHzNx{EnaR^jA zy4mqgW6uqMc{K{mt31?`yo)(!-+x?|zxrLRZGyNa=SqMv&GUWGM-4d_sYCtG#xyvNyWmpSDC_o-5SjCIs^(c59OoH6d#W_6oy}gDC8uUJo&>JDzr+VJ*fb8` zaO(}I!!ZMoC6(XwsY`R%1>+U~?c%>rU2VHY8oQCOA(R+oEiV{&p%k(3uC|qx z`c7ZLnFHtipg)=1XY$UO=_U#V1IzPb9j34+6l%5aPc?|lomb!2A)|w0XTtM<`DS0v3$Tets*J!pUjB4TSV4nA5aKJ0 zX4wUV?w*fR1rozH1=sX5aAyFKDkr=zzx1x=E+G4y@xqHWPn^fJkO&=UY$w}h;Jv1g zJ}MeYTdYGnAxS}NMl=Ady8Wi#{h-Jr4h2lG8aK*B&b+_k|9vJ$X^8mbAtM!b0c1iF zl)!Xg8nbB|aBztFiqD9;5+a~KSF)!`sAlc>1kR@`(0l|#MI1Cn5ehIUrDO9G&}$Bom>ei-^g=KM`F|Ak zA$qA2RI|DC(mnh=igy4rB5pw%c4dXn)g8;VIJWwM$7cQGZEZe&{^$DjBhx@sLRxZ7 zz$cYZ$V!Jy+%iI;9g!C(h)v&y9WA0PKt(QnN+@KSNNMsT|*dC*Wnd>3Q3U z@6pupBHf!Ned>)CX4j!=z^falmfNI9L4S1VD(9^|OIaK9@FqjmXx|%G+Hbr`x1^#O zJLxnDxCO!>&n|bJZF5{$9I66tLDt&C+y$spzmX1r5<KUI5&}eKmeSj;q_28 zp$|%b2DgwZ{z}Poy zkWIG@O2d3xhD@T+$WO@KFvi?C_QZ<5U8Y1iLvatZK*21Xe?&_&5qmnMH`vt3-@M7U zD*F9P$}v2otfL1$jfLjAu~rG7o%gMuExZhX+D{dvc%h>skjSfKI4UV!>lZ~JgbCMb zuP@BQuTqcRHYt1hwQ{fjt~tQhZB?n#+n}X7FupL`Rr=~DTM*Bc_rVkz!v%(e70zsA zZ1Sl*r;^aUA)ypfqaONdW?D>#j7zhueCe%9NI3v_x9~JFTzYw=vnxyec6{rU1_(+v zbe4qmouTqvWVI$Dh$JYg{KPbqH0HM)Evo1T@o#emiLV6_Nr%zPi89`*%L`gydlEjZ z2W&0vLkdT+4U7IzMdi}sgKmKeaa&(}Cqe1}m#;f8b);<5-M#^bDsDY=QclPr!lu%1 z<1q9!yp6+sTnDK{x$?(Vn=8#w9S(II?KcM8qdM$_aB1T)=gP^15_^E&vH)>5>K_?c z2LCMc&O%7}DZ~hVs|)=}Hu6HNad(RAzTcRy+jKIPP)-(idEZdt>P zt8~AhN<|RLqoF`kZnO9Zy=?HR(QY*{48pStxJ9gI)s;Kh1` zCzkaDZT_v8pV{9B_xn~163|8wa`XNhI%sU@>2quID}_0Cengc2V3DAhIL~j_rr)l& zeVG*3Y8yU-39x0I#?gxC?B5_~Jlss#GYPnp4k_q&7F2VHHZQ$-KvoNpcoGb z`>7GH*+MAjlz~u+E?vKoYXw0OGJ0u3A|XNsp%gFU+R-LVF-yCkqOFp|4N4$dK=7j`*M+K*)o9Y4 zLU5KrM(-Ohgbobkx!e=_>Qd$Iy*rqV4}LNOFeXEK9LJ&3(5C^u5v_@1i)ObWp0kL0 z`5|Z@Yi#R6IZkA3Qr=lc(erz1S9(rrC`m#Km@S_yFenZ@GcNR7BC zs|nM4hSxnbzAtEiWvqj8sii%pV>v8i@8IAQ>p(u82NZ0CVS_~Wg%K+oP|nDO;g>P1 z(vNB(@>XFv!U3MT!(*bEXyWP7rFE!@xYB7Qr7dr|(>oj4aB8X7jXVHH`xd7Ta-%Ll zYm`6fLkJ8LR?iGF8#A5P{A7CgxeG#~F?smdYj|#cF|lsfV>zC>amg-1kfo5$RYUL? zX3ckN5@3m#9=f{y>;B=#$K_tW93hU>@}`i9IDXu*<7b7#w;AWgJ~qEDh9gW_R(Oc1;ODa8^@v)NxqzovefM3GGtF>RifbSzCu<=Ma z!y6CAgR#{GKM{p@(bu99?<7xQi)Qb$%?NHT+l+b`^1jvrByv6|$Q0?1Tf5mh92$87 zZ8EJljguQTO$>l4t_|G|-jq*GH(hARN@BCYiv=||I81tc3nORI0OXd$npq)jl-j;a~jI1UMy>1fq{gjWx+y!(Ha`N zWS#m5plpK%;Fhp}mz$fhG*c}e(qerujHcB)fV}E)oyr$mOuV-^W;J{gVU&SCEXs&s z@1*G9F~7Qc+F~$6_ME4`%qlXgOlW{GW3jf)_nE-s!<$!ue@#Yp{p|RFG5ukrH3?pr z(~;Qu<_c}}Gl!0=k+0Q(ks7{W>uaH?4Roq&sNg?J5zN(rpsr7Z4CiDqB&p%|?p&rZ zXx8g}WIOPPHr&C>d}BG;Y{GdXH{B@EN1T6-QHF~pz5Ak6NUrh$My9|(XPj8ibrt9- zL#88t+iun5m}#K@4G?C+S7zVl1kQqmw{rVluTl7^6 zBymKLh?{cXj5N#=s^FbTyOcLf`ism2>2O?Bg@`McS`Rcm_zrYBy&SEysC$pgf?LQk z8WncoCygP8{0Af6Rz=9v%xiW_M?^#*;DE*Y)y;3&0Hxr|NqpIXjtPsx>-!aNzBn($ zyAEdI&U`+IDN`exFIAdzn&&oD2xef=iZ0QBej50A*?HrH5$2P$!wRT1t{5dW_QXSOP$yM*Io^{MoEsq$eN^c$8`Q&X!?>Pn&p-*}Q7SLg zFdLoLcWl(DI*kpD!kUY#k;*KG*3wz?E0c6S--a%*R9g&1PM=g?ze>2E_Yg*EUX%P-@gq9`>eHctzqVABHo zRqap#MQYxnO~9IRdad|TJ6rGMox4kIPC^Q3*q$_6eOapcK>q}fy6D`at&q?;;0QcG zXyOoBIc%k5b3$SPmW2EJQX5;ewRJ*B$x3tHOUN*caybN$3ORHh%W51*{cbI`kJrgP z%GM%Ah{dSh|EkerV0`OBN0pwxJkUNhG&G0{+KBoK&J-hPOW3gkvR*=BCmk&MnXBzB z_=!5y3qx)83X#2U)Wg;K?nSLEs!T6CXr<3`F54WMitT{2)R+%*5FW(sVLN_t_=zx* z3N{vbR1#3#)Mp#>-+Z{jJQssRNS1q1AUiCRc;Do&j1HGwq>-e(7YsdC-O!de$wUxI z4yg{MVGj5|q)TH)OR;hXn$h^7pcJzTZ1jg*0=le=3pdVIIAueEEiDNzGDvdGxxxk{26&cNc z7!^n>_vr5P!uxx8lq=zNZl;G}+r#=LLJKAP>odr-g9e%MUdR|syb{ZG59s&yN3e^=>yag=`Zj7^p7Gar8-!!TDnibV{bb6? zdr#nB!@q`SY)zHPnT%#<7ietg`hru+7^~RD_(g@m8jt9P29shmU!*A4i}OrWw(%TX zLftCXA$zEB!bv{*YKisfo0;{MdyM9v^&DG1woO^}T5c-foqM&6%ffcw6)Pp$*@qnR zEB+&{ZJ_{vOBX(BWs+cF4HkH}T6B4G(|_jVIVL(D-8Xwqr0q-v?C|cQyP2@h8c8G9}`OfC$G=?2;lK!*~A(~*#jHWZw@-~CkVQFcGo$aw*IW|3Nw*Er{ z&NcY)Boxx{+V{o_3Y$Z5O?9geokN?*WeX=nmrP7%B%b8kd3uE37jbo*G~rH>31gf2 zey7-vd-t(kksQ2>*&;Jq(%yRTcud5?$#Yv1`wU%k61M9VHIu>|1oyeRCpsPc z&PDIl$^LQ!Ug)`Yq0mIGe&dyMqr^zb zRHEVDf87cBFiS((t9vi;-QPb+>k{*;^DyzkZ8pw-Pqpe3F; zYpHKuEaA=3PlL-mvD2m#*~aQCbo>KD%_E>;hCvV?CRMi-eOg&(0(TSu1wtrO4=9sb zL7&(o71GygrDgyQjkj~&WWJJHyOL@OHO@ik%5RffQ&`Q@o?Lij z0w4^Nh(uU)v*>Makz?PdGsh_c)|KC_#wwE&0VU&{QK|Lw+(%ZqN`92J{dEb{iyRG-I-gS7343~r>ynfrp z->hZoD+nx$mcsT5QK#{i{8@%e-rmxeYL;3zQ0S}x|A|zwcsq!$S=Ww5E$p(bH|*nC zao>aqNn*{SiO9IEW57}32y1QjWc6Pkw*ZpC#$?+ z9xMK^@sJASa8FPn5p94=B08q?`Q(`>*&c7JUY%|p`-o77f+>SPZ^BYQ1`@r7e397A*%dzl)Z61(Yo}%eumP= zn;5%XzZC5p0upEY5_dcL)Z!QC09j_A>H88a-v{b?JX$*d`!eHeQflzDrp=$~7bc_=O8NVcw%H zA+0eK{%!l-eWWp+S_~<=sLB}zeK7uGGu1Kt83JUZEvPtN-rY*(0^Ox&M0@f}E zNX`0HgN1L@Rb@HdDN42)tuVZ_rb?O^?l9^(VCJxMh0(cjUH(8YBXN+{S*J=Hq;qX~ zfz!42;rd3zsu@f)8VCJm8Q=-~+)NqGa(U#DZPPS;lU-5G*B7PZBChBZrz(E78r_kJ zh4%GL-ktfV1var$d|?sI_d?gx`wHpX>OS5&sYEL8GZDets3rR)o`O8$nQ^uwv@~pt zc>*}nl(bg!)Ol$HnbGpKK8Huo%8xAxTE@eJgn3h4lyh3zQ|Fqa>j1gq{x?=R8qkiqxh3ZN&PReLu`ikDAB zIG4-waaC_G2g_?B3=2tErUl=gD|~hQmjZOj2d(S)`LXZ7?vp5NJlAg24*32?mm+0A z&lsdlzNETaxvPOOfY31~Y=+Dtb_K1q|o?`g(;v4i-%_m)rX`VeDD@T0y_% zI*QcU`@?}T2CqSj>$+2GRyXAmb}Kg~4+Sv#uVk`K*5{Ysy!zQ{T=3k6hWsGAyvGgz ztCMRkV9biW?39b!T!IFO2d%KPbaOlqmz)r>jia79UjiBWm&SIt$2KVQM!kj|4Mg<{ z{QWIOTT{liJ|(}aw~B26w*E2)N-P!H5S!_7&s#2@@GC}|o0gw3FYQX4b`Wbu@X5Za z?Ag`z$IDPb30vRfCzwTgJ5I0ihy6Gmk9<@m#r%!&(@ekg`L7>Wz?sdzJ68Zh_!U{w z$}g6O7-hMhsC1Nil}Lvx=_Yq*yVCT4^HtX<`?efcAl|+?eoY37oJsZVg;eaKu0d`Q zQ`LSRHy4{p1bOA3QdMc7_0%8^YMW@>w{Ks{&dgaee0#CKy&ZhWJ;9^3gA z5-qHEhoXog_hu1gx2Qk%H&709=JqLFizl3AGlPaC~s#4>9D7^IpJ*XCTbrg zdl!AsI^}1`HPb1Eb8}i2^UM293g$l$yL(vbsLIKI|7mv1iQ-9S+9%yMd~qZ&MU!&c z33h3;w%%|c2%-iV=Qox~CobWV%&XfhwcKZZ)Ub5zsK$OBkz8ezs_zqH9aZ0vR~qP4 zLp6rFt`A4;i?O9M8zMGccm(9&uBLR?$!j2UX4&f79^0r1f>fO zbKJLc6&xy%uM3o+xVV&bM#h9#q)f`QgNV z2^TmEJWuh^Zg#w6UiRF~D+ym1@+jX@O|hDMGh#5Q%HX!4ZIpJC)Bn1>Jy!25GrsX5 zdBOx-*|{_AZ6Ump4ScZAT7)<=)Rg^kMcZm;C8{4*=ns1Ls93!akaM=&$H8)OSZ?*V z?P2fALk`4qs#o)uWTDXRc;*mxBjuA&C@^1#^i#I*@oO22R}urCQ}I1qvJcg*TdvQZ zsb`9j`!;y|3>8~V{nr9SN0bzIkSm^m+}830`5_AJnvz>)VfY~)*51p) zX}?4Qf`uo?>!vxSTQWwEf$<|EJ72C}Y68wBL8Er(LO(W1TuZOki zFdhqqrh?T*=N=71U4xycZk7-aBB!V9?S7V$vry$SEDw%k!0|G36F*fYQ0{ehu{kkBMwn`b+Pd#=M-=gwj6iiqs(dlZ_46 ze828l)=p=1lS{GP-}E8{Y*{9M6ahzA0;O~HM3WFI6Z5Duyjzd|8aOwWm?)TzdCZdI;V^`d~R}0Ru zOyGI|w&pG$Zv|BxiWH|=dDaWmK|(}7Vs3y>$tmR5&+lv>tUEIT0R<=CQmtjd!zaXY za&jAM5oTUP+@HZOD*2Uq+%gjUar<0-^hj-Hi}E^qv5cZ8?=L?N2Z6c{;x*kBt?#vl zeAuc@TMj*8VK;LlE_CJfTY31mXnDUaz9Ynl-G=_RX&BC+SFW8DNhA>~{BWUjl)sRZ zb1Eh_ChAg|4Q3>34n*5mR$MMfnjFccgXg#^M{vAHa@k6wrftUu%9z>y~@ zVH>=!K97Xv9ixL^>OCU6!7f^ax@e>dR)6}-7cvFk`PYb-obk+Scy@aY+AQ57HbR&0 zpJWt@{4F4RaUA!V07_WQ&W?fqlGKhZz->#$uH~;iYJ0N zYbdPO=yg{qoyF^^{n5kKm;1kM3|P6S7F~G5q+0Y&__diGWDFtqZEi-(@c9>))fzO# zcgZ?b&(ciC5eG%!c@~=2%FsFJ-~|a0zJxb&H17*0)-+PRyzt#=?hCDCdRh1MShcb; zNBX`NO*Pfa_x2g8{@zuT3^;%+D>ehg-#`1uBTcIW`6$Uue1||mStM~zh1lV*O#SEc z1nq)!jT^AX9)?}#AC6_}$m7n<^vGpMpgK$XlCP|BO=FY3QSyee^b35KUZeS3C(z8H z-uJO8_AaM9;X8fNE9|lKK^hh;14UIwg}L)HiE!i`v`O9!Bum<7xlx1?&Dh~Q;lMn5 zonUajOtBsu7`pI}RokhZ2d+bAOe6<$3)^QynOxs}2Q0lPG4)wS10q;e$3)A=>|h^` zT&|2m$CMR{hXub>xM*wgLRS?ZR;NXVlg6nt^9=21yApCw?(&5xq2t+f_%A{|nqS{h zGSR`%8xZ}kAXB%bByllsgSl`Ndn67nE-u8o0lhs|l&UjpH)NaHZ$5kWOvugn2z(%kDx_(+A#y5mTa0=~YCof|6Sbm-LRz9v&z`lf$K7__O% zhPSOKvp<%6|4`K$bIDO) zPFpisGzyKl@8lOlJZ?F=U{|GL`*oRMp+T9 z%y#U`J+a5ocqZGtEh5gr;yA*tkD=(X#<0bAM}0^zts3@8a`A`)Ib7Fj|2&1zwtpo~ z*!Al)1sx4o+SKI_m-#Y=OGnAT4ZF3h9e(tj5p7sP(R9+^hR|yEl|3?*&S7x3p+c%` z3qlPkkJMm7%A}{Li?@8nabD3JDkMeJKSKL_NCNRnp`7rka}IBFj|zTqJxOwe5%Y8i zKjP9_#ag-^U}h#=s&B!LJ~2HcJj*I^{5v*=n+tPuB(=4*_e}#;m-hL$)D@lX6D(4| zyr|4ckjD#O677#5@+d2L<@~c-q9yOs4G!^Ce0jhT=JC--Qqwc-hOat!zav4YDPjX+ zx2Fr89ZjVQ!V4pzHid2+ER4f09jwi*dcU{;7#JG*100XXNCJNn_6}oM?^yew9?=*j zHP!4rH+Ob+bmypg<7U5{X2x75j}UvVRbvPNdzUHf`m5DT9%W$Qzg{O7Hlqqlkdr)o zs_25obXvKYW>afP%wP}oAXV^D%k4wk`qHYJDM=YxV0OzGH8{Eg1508Ey7WBSTW?De zL*V!(_r?n|J9k~-6Dbd9F~sjMgr=PCXPus5ojIoJ2Q}lp80C~cb}aQ?agW2DYjV_r zJ^zppLs+Zhq}0)%Mb(SOkL$*SN zWD;jlWuJS0IX}xFWC{D6l5Y{k6ULh-@pBc7txZi@4Bbmu)_8;>oe8&K3FXP+Y)Q!$ z$6UCuD9{Bcab<}gx;(aipevOAuN#j-iv+o?5z^0ZMB&~szx`s;jl;j|%tam5@l8+7 zA}q^5=n)0hFJ?#})b<3Df_qh^yTmA+7Q?gJQ9r_KYVGQ+%6!Rg$coddpPF8e@I&~% z@*}aC%*_qgxHr0P!|NrF>7Q>4=}br@01w8c)cJfEgczOud$ElN_Jj3u>M%7^-4rGdKScJCFT~q`cro{a(^bqM7zz79 z8ef~|iF4va6L^lej7D2PJ+eJq??Zn-UsJWV;lNkP3 z@VW9UvnT%5+5JOz@p^=G=9_AW6`4b;9LQ$FwetJY?ET1u4_j7!DS(Le3|k zxY_&1dQR8c|7lw>6cwaD$Olg#)P=T{Z~sT#^@idl^MV|Le{As)WQ$wn^A0N`X1jh! zxx?@yt_;{khRcf#La?=)h!@#7;V~0_=TfMLct8`rr}$WJINn=f9g)Hhi!?EWVui{M z^{CFbZyHz1A4JpYeVI7BDQLJCQ!@U%l)BVVM-d ztTQ0VK5HsAj7X@u6&>*EdY!&!7eg1Uu{wXd^c@G}6o$g<);rIS;HQgkSX1?OTkAAY zsYM{6TMhf;S5d4MEc}D_f_WE?=$)w4xdRCdtwA0hhBp3rnW*y_IFm|}-@kuXnyI~( zt;kYoD$?w$U{&B5712NIjddNh>!5%hpMwPmqaaVPn}03}CoXT0VbbAPeX)J@)bq=c z0U~9lox9tWZy83v`EbNrK};@M_W4ob&eV8M;x(C82PK(&B8hKR=$O_K<`e8+aPEk_ zVD=u7MNGN2?}pjgb3eCya)380)1hLh@&8!YzU}(lw9)u#IveIn_7%#AidH9@Z{zgzsXT z1#J1n6rsa{IwDz_(t()_wl(OO!QvH92n*yM3Z?$OKoBXClQr<1wY&g9v)#P^tBuBt zuwdTg)9S9_sunNZEZCx7OTixJUzcR6)AfA6`-Ku$27%oj)+n2l@O*k--dTWTLHU@r8>DWGa`%EKJus|Va&Ttu6}U#7_UdWCP_ z7+zd-spX#0{(Tm(d=|u+<4USW3{itcb*qI3y@8J z9j7#L$D1Xa3V%$$&x7{NQcB0W;na-7qBqbnq?by$slyi%A-gco;&r@kSZy z{rl&eAw47)?}$7tM>FgH`^jT11KBtt;#1JE-#h(b^oY?(9o0&CKrKd4${0Z-3LTC3 z@Nj2)oJkO1E_ZXnIlB=Fun_$}EZUWs&ebxFZ>}}(jEDP<7(CdNWa@PV!WB~Nv3>h` z;;-ZX_SpzgG0*+m>?><)H)+|kGUa?N46zjbpWXm-v9V<+;b#Y0B$_EKBiFp?O^~cQ zB=hN=>V3Vn;pz)nOPH$G#fk)|u2O(=So@cM~CqkSwA@;(|)#|nP13SJ30N;|>JuH3^ z_5UHa)HpyYWs$g4j*2$;y6W;Cc~4KglB@?MAZJysFL*@v?zLc{X((9kniKi1_;fx4 zpHsDx&yW{A|35Dl9gIgnK%jE<=^3G`V97@4dZ~M<_VIn0N^d5Y5sTNr(6JyShwTl@ z77Q{(%vh+~@+UsLtMM^O9wn--T`Ef>mFb^CoD66L$ff!5k`2zf7hZu#setUuDlr^J zDidBQKYJnP#?}2jaW#(^G0zw?#{3UWLnY(cla2EAn$G*bh6IjMSz2Ql5M<@g=SK2F z1GAp12xC30P6{}Ux2Y=!yO2a#I8#!WE@(nDX0W^;730?Q?Ju|{W&;nN<9- z^h(C7pte`U`&idrix?UECc5mhbTn?LUbV;~%v=33O7R#|DKIwVKbE>DC<`BUQz;-| zw_W_|)vNa_DlU&)Jb5!BO)%6M5D1@r(U?Xn$OHCKtxZ1nT0=N2dgUG|6o|yL+W|e1 zv`IDD$7oVzd1^puU1dDf?W#RXyS>6stE=l89zI(>-?J%YRia8d#y5cHX)xLQcP#)p3o{O|`+`rp z4^~0CrdoB`-EQcH=+ts#kC}B!LF8VR+Q&~QD$_NGP1P@J9?H$E{`SV6@YHdXQQPqq zbD3{jnRS%&^;Sdes9UqVPWJzM9209B)Ds(HGzYFolN(0+dK%Q*?`2#4R-8weh9%nU zr(v`7%2{r3P%3dMviN>p>UU}KlzQ(Fha#N=?EfHmq?{8+yjsO^WNwY-K)?;bHUk9R zcQ3iB@S@O@INAGMv{EnY4ce+`cg#&U6P;9+$u|?4YnI6WpZz`|C4;cecg2gVBML^& ziz+jwNYq6^_XcaFP+5U6rPJI*JrU$`x-eQO#EVqeSBULh+#P!5qn2}NfGa50b}rRc1THbWTdRol36h{kxX;;~IUCsu)Q1 z1fx#EhqCx;$J;Han@c7~n&dQarHPk`N9=~bNgaEHjtQG#Rzo}Bzc6b*3e*IP6h=9- z>Q5^4hD<&1^j>j=aaWk(y&ag3k9Bze_*K?P^on^0f+uP)5smV#SzHPZ4(9N0Pg249 z!%;+eWWRL_RH=dT?00;SO=(vOridq)DFy{nFvk$U*%9i|>l^p*q!wMrpZqlTj@_k! z*CNm6N(^i&o`Vq(tc=xFQT!+_33MxWq3G5MB>!FkDxUmBwtKu zFV008%6>5)RX`_ulP&K(sXa}Bbuhg=IzE{;wfDjdhlL<%{@2BL+(2DySK%DEnM~I^ z&f2PV)S(v>i4S^Itw!$lA?&rwLw9?WK#XNX+ zq=pw!!W93nY6dPXd+z~6l?ZHlhj}1Wni8n>Kkd-2TI)HB>Du?=JVXi?KJG`T86fAr zP7|^xNCP$`^#C4WuYk^cW0&L+pT>w~t$fe$n9rBD=3-d&FK6)_m@d&qo)8v^MF`{bZqhL$A8GfIw{0a2rUtiyQDYW12yxPO%8H_>%JlaDj z7IZzVAr94sQXG*8ty`!K!z{ErhhLDxp_imqaQ=lw3& z9_a&wS*OrYt~pRtE%q5&s`<}GxDis?J%LTLjbyrsdV*KQUzmB}%f<1iCT;!$J>v%r zpS*S`zyQMoQIC*w{X?au8mDDVT}wQRQy`&wp%iWg!h5r7>`|OpX?2027RKHNK?!*l zDIp7GBDSS!DM1sBkOU;yXOsdPo(v22N{sWhuL!h3^u(lf@oH4H7`>dQ;x*}q|daF*+;F02=w71 z749xFSX-6F{)a)RO?(-NLQVn?C~uhL8U(~UwuY8}TZkpba@=TLc!KB`M`JM8g(=bO zAoVQ+5y_s1EM~TEM90XAi^tcmJ?cWDcGB%G?2A2yD8u91PF@`e@?rde33U(;;yb9b zEC?wHv!5^F-=djHqXlDpTi|{)uOqFcn(9_gY>a_pv?b=1Vr5`edBfEB|L(o0LCJYb zjTISQRej^iLPU6Z~()4tTX5wP1d@N&-?WoXQMo|(>p~BuPk08+bjK7@LaQtbB<+u7v zazHmk^(Ns}|2gNRdvr6EJkew}HY=2(fISyd| z1J?sEPePCPt_!3cJ7qP(C;rCmM?Qpld~~a%e6#S6l2q5r6z+?dJ_SR35j&D~4!t7Z zK0ZGFfL0F9aX0YM1pUJ4s5K$Uo*N6n9%ELV*J3b}F4T8W!z^cQgLnivVpP~Gbgw}g z@guBwgj^gt!iAi#hi){)ASWql?4GH+mt|7KgYLjnqdU!v7N;;J)uRZ`4H_wCd@+o-|Um)ikQ-l72^dR$J1Cia_O5~X!eIGx5};Sk(ya_ z=Q_SB+R(&#K=kb_anpj$4OuBIVuzxaD)gp}=JsEe_!;KlU!oBJHHIx>G&h*e)l zRllO+Jm#Na=y{OJH5sR;^Q|#Fqd+2+sq$~RT+smC2g%9!>F)DK|2hRZtWyX?DWX?! z^v5^*L`dn*tqaugou1ijnGlQ0gwp*fHPz4h%)Q-Fk!D`WSSnwo4`*KeP4FMO1yFqmzNn+|q2wp2hkK?2qD6}z1Kr;75W}G3qT&cp$cjy@J z?mXBV0|wPzR%T}A+xwgq z=|Ga2Ad6H)U|dA+lo5uM0!tN9&#UG>G-^mZI?>F-4t5!)r6SD}wzjqO8i6P`oDcY3 z%_Q2Pk=kN2a^`9`9r$KDY2*r) zxuHN==YH(|FD`&^-uD!gBO&$69FqyOC7|+JN%0kgZK8Jj>CD? z$_e7VEKBz$TlxKKg(HvfW8K;V$sS@xBv1UiXUPXyb6`;5LOjc?YqSR+`J3=IDfglt zPcAyip7tshY-r=ZEvN*N@0W#rN<@@)H&Hd`Y0m}(1iZp;ri?MWJSrh%ULy?%RP@oJ zyZg)RMKn~g_I{c;s3(Qz18+*pE_ol;PGPI8%vrQ}iQMzRNUo;w4_9w5B^%zaOz#_- z%AUoDNZOI$_tHXrUyi}~;@WS-VkOEz4n<|+{EF3AT-qRL0-ixn$XXTo&e-dvXL$@g z<9qW}R{nwD=(4{XWT>~4wkC}@HLKri_wV}$T>$76)|M;;B#QRa<;#4xhK*T?8iRD(Y zgpRU3zM-K3HNCLFX6!C|O$O*nDfZ^y@7uj%tm1WV zm}P)cQ`L}CF~&Q5P{(g6S6Z=dUCFp-|MSC2j(0H)!5P z9vrsN!NH@Z1fJ=hd=3m<_~Y&bq6q%lp4q=5^OUDbOIKrEOS3)+o1dk&V2@C~i>-b@ z)#Tk96h&x2yc*td$p7JbC*%D`r*LfW1wyUjH~lMJqNQ^K$$0ng#W6-GujMQi=2&X} zr`2@%1;IaEV`B=37a^os=H)+P=^CiwcQg+j(}RjNd|1pK&Vv0FVWlEE{NPX49?#osV;=dR)`SXdFbbMz zh{|d_{Gu|Ez>}1mJj_&NU`1Gf?4Y`R`h)#WSda-N1ArFe)*yB^P_#Mb|Mm90DA|DW zuABD76_;EY@8+hXBJHoj5RgvSzI>K@RBjy*TA)-BRm}aOhZT1I!3F|-JPma*yahJt zq&+i$7EAK&6A1=^P%}_}^@dvdHl~FD(Di2?&hCKoif|>DUZ98gjr}-%bT2z)CY3}R z^2fit^8dUo=MUnvD6;%4pB-%K^Se}uj^R>PLUc{<>fF)r}KvP=^~69`q4;KZ2%6IYa)#cRZ|FH}Y?b00lZK>7;PLjlVi zI1)WhXa6FOBZnM>2iBJ>+yM){`_t{%QS4$tJ?Iz)YUpH68ua^7c)M(D=UQY)aGUFi zC@!WqVCO_Oy0kmtckIc25ln+R$$@*n_W|2L{-@}vDWTU{+ok>gqF`%U=5HlI(LYl{ z6!VKF2oIL~C5AF|A-t=qfSV7opp!oyr$;8LAXp~DVks30O(54^oDU_i%#R=fPfR{B zq5b>TLH!VX*wt1t{?GPhPf!HO4-NzSPg$A3y?Laut4l||FtZDxucmdr$696AF~pia z7N`Dw(^Dk0=duLqj12f2INuo2|2wYuAG?{Qf)>xU0%8i_0LSM9$!6_ZQX@p#vF_L& z3$X&JjRx0xx#8K^a zfBWyJJq-L@PW@rWN56mcN2Ld=B8U(HkkWN-c^PLeC%}|YZ7&in$BJV;s^N*8#w_N3 zD_|Gm)r7A}lKs7_|L2D;aYXVB32c$7Ap-XmW}n{6q1ki_ZEG)BBPjSW*0&^b9Q#_c@q-W?C_Y*^_q1SiX*1!DaRXebr0gupt7MA=GW65ZK(B9-mJIeo$JA~o@F%PM7BzBaS z;OJv&ZXhZL@hD|1`iFvu2)CeHj2LRQ7VruJa-@TEN3bCJgyH3PRfra|t0N}rAF_&P zPsM<^l@8w2zKrZYM4zadC*nZs)un_)<~;UR0M-i-7&`=Si!d@V#=)iK4$f1fV9lVT?)L_H}XjjY1j2#i9Q)L0Lu?Tf_;T35fYMAJ#EsiR6Pd4lL) z1tTb)B%Z(%!6S>0T*UsX^N7jgS5QcYp1l%e+JWr+Q>&~V7#tJIdGpxe$vc9F9C2XD zb}(goC#m$^?#|kE=c{vPFA7R^`xtS(aBxAMwY>zfW6pKi7`lOC$H!^eHDl_S`HOJR zpL^=xR|pkTIzF;lmuUWnve_b-+n0rq@vFXBos!sHzq!jG^7Q>Ru`%S8hJ!9mWcuc{ zUwgQ^-n^lK9@RQAZkW4y1YYwjs+rAWP7>K1q4`7m5iV<7cXY)itm__!>uN|UF=For zhSLusr%nPsAqXQ=o)NIl9nF#2~hcXDE|HkFnyBE@lZo^w-^v9twG%29u9H&k}!$ zd&y_FOjOiwJFb6{s%J#TF(CcFt`%-D=XD&Wic$CL&Jh(O@jqdiCR}oOS`!=C;`r#C z5hbNhZ?cv3cs=3*cC$IwVeAQV`@3W5Bp5eAtf$jKiYX@XZ+oZUMNC9zFFk5W8lsSX zALs(J63X)g-gsp+h_x}()P3(OP7sz0dv**asHZ(G46}F2X*sv^H4$`4Dg@~tNBWyR z`vPE0UvVuHfg%6&E9Jr(h@^aJD#O-$LZ8=Q`|I5t^Q!zNP$B{RsjmN{`v5 zIMRC!*(8?0Qh1M=y|NmE7kXM$e|d7?%`S{xA%(f~1y`6^h(4FS(Hj4MoV|BE*Z=xI zUgDk7@XCxtq|BsbWhJ9XWN)EtvXhxTvMMW^>`e$6WhNPAlbvj`$@*Q7&gpZ!-k

      p#nZ+pZuy^X~>!!KT#zcs2ZvPch8Fpg$)kC!{V6I9W*62jjOyySoc} z0+{jN-YsB7O{Xgz%g3YsWIS{v-u;BcgG&3qs31v#;n7UQ&AYJZ*BmT4IeKjqA^K6)Yvp>`uENBylRX7W6QxtKyVBP@<_9p(5+{vz7_A$9+oeJb<Xx@R2G=XQIWocF#sm3%nB;xw%en)?Rd6lyMx=<&y<++}VDA z;HGK9Nk2O6bF!!Y@E4_Q)oXgvIc3usKjE)^o96Mh_S1IBtzCIHE`_7Pr&Lg!b2V$ICF${uuW{a z%8Ha#mE(%jvC=Zwkybdzfa2VH$EvY$onG??LC|Y2b9$lsV-u8;F#0uLC$woa|=EL z#RaLzj#c5?%C$e;bOU)(MxF*9I>Aj-TLww2drR&9#F5$q*~rCg8-5gpoa3>2@MOB( z16CVq1$oT5LUx=D-*H@eWV=aLd+~EUGvixD&G7ONe&pQJvx?IPT zCF#K2WYpmnHH7y#R}F43FMZmF1a)lFFjiKLGQY?QcR!%s~B*I-YulB3op7zvz3XNqty z3v=t;d+<}OLGEyV4~v=7=ynGk#+kKL@=@genat+II2S!l7kmT);;SCLXIPCnBRKd; zw#LNR1>OF4iD)w!abTyQZc5LT!|o0T-*p86D~ep_1bWuNlPu_Y*<|9gPhN%`?knIy z?`*gJZb-v64Gqic`gt|@aD`~vbSk_K1jk}aJBSdJXWLzSiF)9QRD;#gJ&O@=K{Vg^ zaMC&WNQAGlNR6E%ohht=8Ig|Bl4d_XhCcTd*pALW5Mm9=6j_Sd7E|PfpXO zg6hS^nnMl_4%dw_ST7r_IgTVt|L+MIO+V;{b7SM zn1$BZ&t!*xEH66ttLts$wQumhUYE`1)tnd!uiAdd>OFIlhAj_Hzb7Qc%tSHAjU8l2 zbfNHqj>WL+VJ|Rkt<*~;KWpA`PaL`avDC(&2oSn>H0fO0m?xPgetW9)Nr?Ask{jqn z*X}XqU}%C*z^9$+Ijpchv(aQiJS!+$hDHh2UEde<(m=$~_aZ*<*Kw--?Q>Ta0{En3 zFXOf!Y>bjsaqF=Ka*z6;GmNmy zf|*p)HlPInlZuhbt@%s_OL#Ee;Ni_rq(hXSW@~crve#6!Mg8d!ST1k^e5q_NsIwM0 z;C&_R$OlL3pE-{!GFg^j*X8MXC_(dDcxS8^E-0UNJ%J>D+LS5k`-8OgD|oS&`emB- z^WNB>`8CHsCid6990&%^Ie|*PwK{6AncJFS0J`Jh95_igC>@?qjNLdnI8CEB27L z-e-><5tc64E0G6Y^cFG5u;pjKBS*TBt=?wbJW<;iAqqEMTiCH;4)`&&sAAZSpc>3m zhiv9aMdDu~W^rjgeV;qbL7e{)*zJ3Z`=Bg()7ht>KqZGc^HTL3h4v_KtOvoXPD65o z3Ia!Kwc6lP7<&j*4JVLYsWPmaBGTQ3qity=9e~5Zh+I?@m$Ao z<_2pXaG0!t%Az|g@=o6?oKBnzf1LTOe~eCN+0&9D>4$Qm!M)DYd*fQW^p2xMacO{a_Vm}` z0aK*hc_kY@4@*F5kP%!!G!d*Cx9)lHQfg_U8thSG_xeoUG*45=>Em?b7FZ7+nNG9i zw8MI8xGzxki6#&CUl6QoF+g{jgM)aqD0!5VkMA2lLE*pMkpBHew|+mjxZRc18gNbD zWPipF?twLUWa_~T)nIc@@uM|P%Qg@ld~NsRoVToZCEWL*)H;&uG-upU?ueWNCzCO0 z3iku9b&(31Qm=Gr-S5=W z{`EcD@)P4VDHrh2^M-EdRpbG6ZM8eS6bE+W$a7CbTUCcDX}%Jq&MZP9!*Tc!H9nl* zRuA`t(bo3`51W>>@&n0n3c#h=dt=ysTLo3sirLXA<+~7+=xc|7Ls`0oVAj$NV%R53 z09n^iV=U@18O;)mi{nu-0&a321?L`GV5|0eewd zmo3!z6MJ_tdn(ydaRU6cG`T!hKRn=WWT-yQeVq3ms+6;%qM_tJmf>FU?Y*I>id=PI ze>M>lA9!NdRfczejxG~|6isH5NdJ#74p$Y;OX7V@*?$d$!U^D3>vn5AKE5buDcKKx zz9fTNgXGXxO2AYV!?9 zCOYKas{8A=PhW&?V7k5ISE{p)KR*ywjrRQpC>t^xsMgcmnO2(EJ57Bncir`8lCjzN zM!>woscV5`cfs&f+yfR@J65;vBASs*(@whtEc=)OS7X=9FP;HsR&A)mSqYdKSoLU0 z0VBrM{Kf{rMRv`S)OZPiH{K8GUuHH<)2)+;pIZaRb#nmzT~d{>!o~}cU{tm}3G=$t zVA*I6q|U@m`q4vs`hX~riFJ5#=6Z{6ZH+eM4MYGQR)%Y7>FRrT> zk#=7j+*C7`X+V~t=HNp#ar34o{8w*eh1b`i4P}`&Ksb`EPTt37hLKHH1b9Ms@-*3O zq`Le{byU9n#*LDKiPqI!C+#!xD+Btb>F4`HC%?Nl?hfs3xc~#}35MmIuBWvqmH-Gg z?>SylDL9Yn`%R(Orp3J%W})=ehNV-ICWE7{V?!FK0Q{K?W=SM2ny+gFw`&AFHr_G+ zHP7+2z#cBS!FLl2?1G&GF>r6r2kdWf?LSKZ=gj`Vxq^>&3kLkuv^@j7``v9IRusWxs$fFg?B~S==k<3rkAd~^s%Jg3va{eK==jr}`pnf` zaLQcWUT-i)VW8&2VQRjVCgjIz4l4|#({jX42A=E?d%0n5>?l;G8pWn| z$@g4+qwWLj=aF;o(J?$$4qoRsK~Wzox!#45$6pz&8UL-QgM(Vr4jVr>_Gqk1^P(i& z>z8R`hsOAy4w>H=JJWOQ*@osKN}D;W^)E(SXTd#=+_uj!^?VrT6}>JOv9YaM@4U1BClN~x?2(I$n(ZC{{^KuIX z_rq~W6wTgcwFh$a3Ly{EI)V-4=e=@yBec;KxI$%aIAKJaPF1j8VY}6qZ^77f(J^3R z4>>HVo@`u~?0=%9v`93|lBVnS^aNJ%>v{x;fe6Al_kYd^uP~5DC;!C>zf!5f8nAf! zz80_VYzKxq(qH@ZxT@m;-bPzk!|4sh>)TmkWU7fU1K)JfM*1>4!x-A(g=}P(P7Z-D z`HzP|RgmeIPD&{mb*o!R?Yb^WunK(s1=0n}&rPt?=AqBYg5rWYp_k}lOG5&EcvH$K zon9cAiuytBy7tt@&&}(Di-FG>M{9b$&sg3X{PB>@L4{RLxyC$-gY0Xn<_23q=Sq$4oAV_YG&lLS9DgfNDIysoeT%~>Ix7dN+3hC$lAyE zBd{NDYmgtb-#{EP$p**iv;>~9R;}&R~3+0>UHLAY(<(w2+qFGRn-%!VzJbx;r0MJ~)*Lr_;!a`)AWi!mHaUlFGMajtX2 zd=6QWa`rxWdGBWJ;TL1>jh(8F%3J~wYB_b@w;?j+20fVqI;KUiwm)O-P23L4p1EnV zck)a&oq(eQV0&-iFEhL^sTH#zko#~;s^gqM2emutsC)b3x%u^~y&Yt*c;>27=6c09 z9oXIxA+5KAjQnzX4<0@fEEqTDKIFLu)viB}AVF>ny!?@scIn)*lgU(x0~ql$&s|Df zXib3)IEKO*>}#-Tu1RVfg;k5k^rJ1rd``M__v_e6IQ9rSi+cMOX6^Zt&gvL^4zFBE z_LT|aFP2S{J;vvAj*RHZBRE7LeB_(fFOJ2_QqHHXB}{@%H62zx5tZzD)2Iwp2KVYu zfkdr~K$_*v8h5(!o_+HDxrIw>9buZKY5ADW9=Y&}pHjKbH=Ex$ZE@igbR{B#rKdii znEab2=LF|mi-v%aGwdIe2k-lAXr8U%j~sw~BNS_(O(~o4wW(dtLSYkS@N3>7wPQNN zeT=EmFHQ~^B#F*!yes=QL^N}H0hM=^Yre(QfUeJVacCN^KjHdvjdKjfo||PjU{GjZ z4axzn;@9_hknj!*p8n@c3g~4Vt8~>tNn$cr)%l-(JT$&fh ze_Aj0^S#A=Mw$2GOtkov94|lF+gYdHRCH9{0vqqsa(5jgI0fX@3}COjO9b>Tq#m}y z72h+rE+utm`Q55wYmO5XjQjS=S`GNmjg?eED#XoSxT0`oy#_u|$;s_{Z#Mv;9ROulV5-a~uNJkZ5ghye zVPj}#{xOL=O$PscEwis=T8``pnWSf?EcylAIE?6S8-fFk4hj^VaQwrYo2-*dyjgC!9`2Ogw{^=gS75HN{nOcZ2w!pAmXaQdLF`LG zF3dgWk0d_3lu=f=@op(2Gbf`j_xXv%b;oT7VT;SM+R=dS&h5RMA<)_orn@;2u<@wlBy*ya2ackVFO|qiQM(yoA;8Kz{5IESYIW? zgV_C1hqJ!e$IDd}7n0fU?7QuMpZQ=IusL$Q}oB;b_J%AhYp@7~PTmf%&irLScW*4)i+m1}A~ub6oJ zX>0pa(Rys_0JO{;{8v##ePC+I%C(7qm7(a!8uZSj(K~(@$w=CRXynu$1M2aiz1_#E zPOF{mP@Bl+Y~^(8zO*z^Y+;RqwZ@Ua91s+<>(u%bLX$id78dgYa#Uli--JVpwmq|L zUE~aOfvBd!U4uf`O2vW`+ZO$=4ZMxyFx8Aii#qkO@CoDmBNg)A(2;d0gM|Dw_}tB- zifUkO-VVt{;b&3fk?D=e*Fmx#g&tOBz|7x4p1Wl`D*WC2Z^wE(sBc^%x^O4WpxKf+ z8|Zkno~Chk5@3&b@vf)DE{|r;YuMNI@WVN1<(QRb^!`}wX*&~@*%XSO#K>9CU5sHp z+B^l7wG;|}xm?K&JXGryI^8$T;&Q&sGZQ9AydAJr`y$6$9S@%|cXp7ciRDcaEJ)xK zjSyBpoa&fc4YrAv1=T`&VG>GCq-EThx)j1F&UsxMSB)Bs&}3WJ=>E z2*@GG&2+8%iIT0GO%BA*mf*G1?QDv%OKl>Pi0zrvHmYaaW^Rzna|g z%GTQY*!=A%Dy+-Yp&mcv&YJ@}J_?&T`&S}*RP2Sv(dP-v6%jDGNe)CpIXdb~R z%9>o=hNBeY>AR95vLR~H)5f-sCUT3Z^vWBhc`Zq4t`56~(U89&#j-ILe1eD#YQ69O;vo9S!gNb+O37+r4iLTUe z1sC3!`_seb4U=2qCcpMjVaM`q6hr_Yal9q)xV}LmczBbZw*jtyTnhdLMluHT?Zt*v zXm6A-Kc)Ra){RoDY{xV(b)SWqmzofw;ehIqj4wxCrU81uuU?Tb*SYc1&Kdz7%VMXj zPKimncj-LskF(v{`LageVj4syJ5lXs4Sw6C>dW{+KJT+yPOW#~pUpfkMDL>hMKM7p zxa1-tY^P8=s&oBi%c1%}s;E)kw1>XN)upF#dC>`tv0_*nWew4d zjLO&~#12?``*O|cN|J-zusqgICBx2(l;AK0>;dN;^`ZMz{6%F1j$F({!pi)8Y_W?FMGXUzGAx+m&TW$&+(ObX7g&8&OS1$!~d-8ygA+u{FvtA z;$r0mVF}uj3>mvy`Ztqf_mW%Ig%9~!oofk4^)y-q$Hd!(|Je6HhUEwP#gW_G;Y`Um zRZv|<6#}nCa;(SqzzgWD?BG=*?zpFKK7gJ2vEmyRSb*cEa0FJUm-Pol z)IV0~@9)nvy7_QFRt163!VRi?UoKAlAK|F(GlYt7>Lm#iX=Ph)FYE{Znx|b;wz~6G zk}VmJR5`xEPUoiaHjYEWuy@O+{QQ;DiCAXdaDql%|FEirEFKYA?*d0;Y7c<>D{ znua(0I;U4dw9i;%3)923oJ$jhA}lAy_?ADvoQG>t2qh%R9BM5qt<89Pay}y?m^Ki3 zHW+J_Z^$PsuA}RaJPvtU3id=u1x-T=?>0!6@y5?e`+hbw=uFhgm^ze1`t3+0K&*ZE z>gUTh{et6k=;wVc_83-~V=$GnlLE?2G<0vPxGxsND@@e&anRkyv+z$=_MH(Brac-e zJLWU@3`r6ebnXEWeA}XxQDO8|8FTlEMuP47suRale$AWY!H6v{Fs(FteLc4Wv5D_Q@9u$5H$ zBQkS1LWf^>(4sZBxVC~P)-*K4POtV2yKrb??U}K&#Ot{_<3KXql4LEC4+^I{|GLAX z^za#AMtdF7&fWBS;BscY5%xd4Gl+vmmvI@i44+l*u6Apsk|?R?Kj6MizwQ;cGpo<& z_IOD$KR(bt?r~_Ap-U%BsB10#+TUjVTe-jIka7?2>=2ewbj2>`dn=paGfYPHAK0@9gQ>mC~>p7!#=?)F&V&Lhk7vQlv zD%hN+G=`J3?$%IS-`&%3eLO@?FSkK&t*1jX4K<4lVZn1w#zM;{xGpNBey1pNmL(J3 z``G7f!3u~1t6AtLF!8=!ZWR>lpBe zScVzgZo)FF$g@uKpmdl5P{C;E0<3rvZ1A?tBwAkscD#Jp0`aEpa6+_f`6WH&{Hd;U znj*HWKP^QVSG?9LUB>t4Gn7^a+|rJAKFdo65SC@|7PDdj30?7OR@OW$orQCnVQK}}hO+u@8|9F9xwa%jm;A=G%Jrms7AV_|Hb zv0CtGs9g}5_W9(TPP<)?UW7XG0+8;a&Y@#lc40Y(cGx)sj3(lSZ6yDpM#nyezt6Ay z`QR;$25w-eMNOEnVz5xGP}N}@k*ClIGx?&ct@CmmDHkHyluM6y*ZpGUY>_?WuazGp zRfmZr97m!mBybD5iJFOKl){-{G8>Q@Wm0e(0rG;M{-{8x0G6uv!YWj=xk=h46jhK4 z%cj+#mJ&TAnkb8V6@|O0cSAd7xm`bCliRK9bA@srboYR=Oj1x(p?Apc;MRmp#IaDm zyq2-|8#sXMg28SU4rQFFM=R^9`npckhAv-zJsfze-sZs=T^Bf-?eJz*I{U~n_{_eQ z6v7?{5>tKHY9@sujjd*_*9?HS@=P<6pMlLYRX`)iMz_~}0tO|VROP{QGd8Y7(cHTw zrw)hq-HCPRfnNS|Bt$M1lauhw`2uK$UDQX^+1KG?kjwU1B#T=5@r^G~y{iY~Jq%&S z9LNK}Z9~BCV>}7`+-($J8*74~fn)udy@5FUs!2~CM=RfQ66jO72ogUYZ-wz-l(c%U z76mJCP`I7~#z^tY8rrqC0$alJ8)hu##N+RoF-~-2TMEyj_H!5vLL|eYeybUnQQ@5DT&qv9t~ui??+`?fq9zcLkI{`NGa;4_^grH?Pzz;?inj-N^`CE2bY95Dd!ct@HnOKWDWXH=sYqF7?Akvlv z_I^(zF9!cH6GtxSW@wI6@fJ=#IU8UK%|Jt6`mm3?Z=e zf7%%IbEFoZ&2G|j-mBMZRSk!0o$@-6H^bmf{s=ec*@a9fi&*bV7tmX>#I(Aw%&)LKzh%iX@#*zGV zd+08<23A3}|IExIP--~;+~Y76J;v+BvLvY1aHDjiD;b6W@c9HQ$s`mgT1kc}uTrMj z`Jf*<;C9}U(&zX)u0cE9svHNC?F!JY9BpCdTw<%*ULtAU1o6dP%=;@AqqPOA@WKW- zpiM%WjkLTephL?wxn4>#%I9ky2`0BnI#!Rr9ZG@5teFDkm&9-C_RIDj-)p^=(I(Z& z5um{C_POlsywiLfuuI(uOFT6>hH2;DBB{fpu;d1IOfcrgY0pp2TXW$pz})(6`4T4M zAKHMjOV-2Rq_|h_VGS9`rz$uAPbhp7m-oN#4Jeo2fm7a+HSDO7&l>IJb8bXSlIjb& z{YlL zsmiUg>!U4WV~#<~0#zL|<;(>A2S{Mf_dP8+&VNOr*yWBqjbIvGBu2qOs& zXeGAK&B4P8B@D@uGpShsln^({#+|5c*+_2)g+# zLHQ%=u}a+Ue>Q3iRKSBd=Ssx(9$lkd(E)kEr4l~OQV7 z6eR{Smx35My<}TA{Qinon)=118b|j>3ow%_jZp4~n{*}_FY^4jHxTZ{lq=l!JrJTZ z;kdhq*}i8ab$v$ZR0PHb*8EIZ1CCfH*TcOe>Z7EezT(JAcH-2W=>2|KNmhm63)O9- zeX}f!vFt75l`N&MIy*LltK-{7owfiAzq}TIjJJC$HaE1`_P@p`ELaGKVf665<#icr zYis#-k20(?Bt%0rxehw!TBzK^fV3+;D|za}k)ponGp;m1epvcR<JHb3RjSoLg17(FcX>SHI~T{HYpjjXrOYCf0pNlLgCi(<)fLx<8N98W-xBDIjv~I0Gi{rqM74n z+QxV+T-@~DN&XJn9HkBqsm%sL=nN!WL&9jmTCHX$2Et9Ph00IxdMc!I3(CjP^wjFq&wpB(-N-D+|EwnJK7y?m#ec_`-_{fie(iNd%tb zG$A86GsR~Ptw203QE{J-2p(kh{SR+CHxswFtse-Hfs2jh+VT~DeAqZ_D46Hmc zP0nSej&=Ya4b-0P$!FxO5%V1m26id;eg!I!35e5^90BsHxS&RQ<8*#67w688aQo~n z2oDj88aAD0cN(;CK(M!?)SE8WBZgZ`9Hk%OIROmM(Cci7N_=xbMh@)q2^-QTov#v! zZ-9S`)`J?%8tzjn)MS?sAM>!GTjg9Ju*WWOP&&R#Y~XlN2M|6SZS=IEH2ypSb?#ZEw#QFjNg6GFhL;IyL{4Bej1-Pmur;0xV(3)*M&LknHXo|y zWXLwDvm7nJDOtm^-0*C~OeD+QCiohP@I(-NhXw>Db*yT_)8B&qa^6psjMZ77u3_uqdai2 zn_}Hpc^5J$90e6fJHAN&Ec<_8Rz4m5ei(Y(@@i^oO=na8hSENg8Ti2Fh3(VlaOyZu z?}4yQVli{MGfY3uiL;I^x(fIn`jau2wT1nThb4ddlWG$cbjD0@{X+WhoS_sASJv@8 zI|oI-%dPx#@7`n;d}0#4NjA!c(*oPiGq@9X&v~Db{TTrz-sK9QvU-gnsiKw%(p*(# zI9oA`D@&MUxWI97D8!5$#wcEa0`Ebt6I2L>ux$NOUpG)q&(Sv67rMT|fpc$%1Vj%A zm!A57K^%9G(W>e?_u+0>Hw;HLWvNG9Geab=399htF)oNn3_1{mZ>Iv)p1r)Uc?kaP zMpmSoEx7EreTuT`B;U{XW-gAv5T_yqMgX;7r2*j`T#9XIkv9t)NqDx2)C4@hj0Kp- z2-9;qHfVj6;u^3Xznfc-E3W(nu@W6@^8uK`F^W0ofI-Nuy`@@dB6(lMC{5+NDqkGM zbe&3I^^{OQi59HeQ(aKV+4Ce-O&h+-BIt7-=Q?fUCdfPQ5?#E@(qr;X9}fM@3h1@Z zzZ)VZ&|t7OUwI2NQel*4LEU4q)HkkrRu|9;G0ZnN`TWZ`-@G`_e0^;xc?8HW#aCNb~mCdtS zg|>q(R%OwVb;bF7pU=AC;%5-4#gqxZ_#~B~IiXQ%!BIT}i`%tH{IT58|9fC%0Ct7H z!A?WQd)WhvF0^MxQIvfHQGRu7U&xrNKp!EyeHHqmAu-0d0?}|G@wjY%P^&jy7`9Bn ze9^(Jc$ZYpUzUvOcR;W_zmIR6g{Dq6p`#hnN6n{+YG@k*W7e8Nrd6j4%@^Vj1n zHJV}y;ba~ZfU~Rh!oTWsAf{!MVyDWO zgR}1uv|RA!E&-iDydJC?QgQ+fZYE0A8Dn(h<|^{P^`Pd1+lVygFU$@3ItdS3kvw_z zXPD?bxVx#`7uG4*x*zY(SOleUrZKvO(;gT_^AKxya*DI^s*mh|(e9)CXlu+r8;s`NwYx(I$;)rbyk@&@nYo{XOGb`uAMFT z=1`7L?mKgSu%;RAW$&9S)fP-oa$gC?*~dI4VKop0ZPu4of_R7j>*OvJpC&YG^^tW^ z*LTtV=}~M{kfs=~?Z&I~Qh!@^3to<_^@;pOPUmw1!nitg56yg~7La1*7wlhGv(z=7 zXIE0U5H0>tsqXvJ74=38ieNyY_`BHF_923uX?qXHTCp!sDdb6-Mg-W0YuX$3!J~K< z$c%GZ=g=%tV7`#^r*T6>b!H2SY-Z5g0$w_|Nj&}|1RbS7R%q~vtGpN5wsbNnzNAR& zlh}4)9xD9vo(k!fkfKuYpx2_k}v@*{}@YF#9+Y|1x&6aXHh+c@q-UXP))~L_PNgv zI(Y4X2>M@FaCf6Tl7VA>;jxTGYx)KAi@!kE(@?P0hZ@8z-06L%v`G+EQ{??nPg^NX zL|O5XF!|9Ug;{~qHgq(j;)1{K>8&le>|K912=vRR*-iOrr@J0ewL9m25RMQp9%JDQ zpM*@f)+j7E>ikkGaSx(mj!qSKE*flLA*ZXVUABPx$JVFlQFH9{mGk_+CSNn+C}uD& z-l5>Pw+>on#)+f-sSPC#UetAX#_dRRSfR)g@*ReRrIBdrTFsQr0G!Z@v-hb^LTgXl zL<5Uf#tMV~t8!L2f>L`W25TT47k@?U_u>3WreS@r4~SXjNO2Q}0P|-c*I=xGCr|HA zbM%@Wh#pfwv7M4wN&7Rgt-h0+o3m3atsdxht$S+srhv1MUYuFj90&S;hV&!a)SC(| zjIsyso?R+4P=%Vd9YIYzg<7#*KzLjP9~p=XgNJ&g!3he@kX80m55WwhA;2i+zDFkI z0>e)a74XJHeWddrhJvGsGswFeN~M>0sUghI`FODJb1k$QQp$UXB|J;jnlFLV>4D6OL=S!giQR|Aho5h z;UA5LZfKJ%AG4(--_=k27mxDPTJZ?`>72ntTZ7mp8H&s@1{GXWYfZ|r`T$9wjx;;V z+1DZ(pu`HpqZEDG1LMmQ#YC+$cQZ>)*6z?$i9BtAhpV<~XKy7`(eiH4KYOqN+!WuI z?EbIA^N+dMRRZ_I-ROv&V0AOtyXPCti}jT({rT_0FLWKcXZ5eNwQwmA-WRT7Q=c2lGE-G+_y;Vh*$-ev?YGc~02;N!rdG%Tw) zl|TuaQgu@Nd$)aqy@eA_j0XnLyujCg1)*22cZLoeGh=JN0XfTWbR7 ze7{4?qwnz}r%l^#r@!~GbIFJD=x(t%2jPf3B59=h{C);Qi;|xe+LQc!&*J^L*Cj%M z<)OV!J<2!3Qa3+#?``CAiG0;!>@;H8-+aCzsmS^x57b}l+~;vBnVh8W9hp3GF*+Ej z9CI=-dNv@&AmLkq%JTD|W23#pkAB|msKf2IWgu}`qK z&N#gE7Mw#%O`n)^qtW!wY>*`0kKa6RxQ6YkS&J~E_~$I6_5t@}Dj^eH^;xE=R+J#= zHjr@HsG*ki_(sf(q2YkM*gdNr;Lrey?l9WhrEedCZcqY|CVn%fo+OI5p0O0%YTUYy zYa?=BJYfkUz0wZf%v=fNk={b6r^5Li(R>V&1qNM?!@tc$8XPh-36x~a=-IY@S*FVg z+1acyz`RtLmNi&_PEPgLW~+;U%ABVPE^rdl_x0rocutM;Ry}#zQ}ER0NtbK@@u$My zm3qG)4g4Yfs)k#zMUnD%M> z&Cvk(m0Dz8Dq_l?;^*vWxmcqYyol<{3N;ZDY zZJYGY3A@nHv-r~7*LmVTL|jRf-6DajlRQh|as(3!4YseF}+$R#9EK;V&J zJ*`+|L(D`yP^vIy9A5nAYQZH*k5WFE%hqsFZ?y@sGKM&xX=^_)-Jmy*-^-wcE)TKm zSO+QB=Q}^Sp$*VW*0ezW#K=EsME^Uk!b&pm^4h64!G1 z)gE`$mL!3|BD>>?e^O7hCKzvUfWt&+^wJrt&(CA+MtzC*qO^eP55e13hn6mVzmI+m z@NLXTEttB1VPp+K17U7#NCbZyDc@PcE*XzOw*J&L3o4lfg(!@=1<*W7*0J?F6F|2y z_e&ca8;7k_K>ArDSDM)JFX0@m5PCGEUq>OXEau_*JuF|hAqBEvK?6Cug0oH<=K~ku zJ0DKSWC?#}9lUu5)Zzmf$kN0RI+x6SX4%~pGUBUz3--sAQx91Q*7~hK&o{g0xAP>% zE1B~DA=EVDExGerR!#JL=Saaez^foM_!u7;A#UQ2+O+M^uN;_p>eq9NBI=L}*iTv$ z7*2Z1+1}EB`VsiPS7^M>8sOksarz2?iHq6!^%Mhcf;!Pi#Yj-R3H@g}g0+a~<`L95 zFK*=GTu@Yn9!#t?cu}Mxn)>U_qg~H}Un12&|NHBpYF#PI?vX#U*2=%G-ph#ut68l* z+nQ<7s66I0K|?`|oD>sb_Ry5139?X7gGtO-YiL;|1BQB+cvIJE^Vdvn+=V+4ZP~j} zHN2?YnR}E9P0K5+FeN7%L#IBB_*WD^M0ZwGWM3&*5A?8mivkIDLL%7)i8ufGB4%l1q!T+iiGr?M@~mX4+yh!8?raJZyWa zXoXt)6=KxQY{G%6B~Bo{r^Aot&{-IgA`74{e|SUM6GsK9af!P%ol7B=y~E)<3j>fW zg~=}_?DVf=E}zoYz+95e0;WV@3Zg07FMECETi2|zQ_Lz_-x<{_E|fuMfHg<(|KW_C zaviDhe%xO}m-hP9H5;rf7#|^2^OueFi47zXD^BO)jIIBx-|}~ir!*_kRz>L_;8)j} zQg;)TR%^GF%RvFoIZdPpf|tR@x z$c-9O(9MFHEZ*QH=7c36pj7;$5z$M#TMMZ`7F~cojPVrkv{FdRgp?6qK|R}jnbh+K zm*Ll>6?pP`F89`fQBCQ|qE0vtgzR1T0_bD%)bfzf?>a%QmO{NG5aYcnaM2-hicZCR zY*Tq;c_wO=q5^VWJ4o~f-RGfw!tiAGZlKLXzC{jFD7Cb-tA=gAk1wed_b0zs_zYjz zZ%bMI$yu@(2d$1t}eP*_qtD~Bw3z& zX3YY^R^h&ze`=xI87Svi71U}Wy|nzAdWXSb=oNL~*YF9ATGPh1i{tV3sn%P(Rew!POvAR3fuqR0Ja6jT{Pj2H5*HWeos4(4H+dge+XG_; z84s4B#1}_0-CxiH)mpqvdyM7>DdjSnnRaUsl@T)Odqn)q?_&1>lBhX~fcsH&f!v9b z0)mr?x#8zqI`Ol8mp388Ma|BSQUp1O==*Ltcw7oO`qZcFVvMz{k*R63ur?E;#a`fn zEPMgKEun<9F}7H0Pz&e$I16G!7LM~cwy5RA5^*0mfJ_E*#>aOSX9Fu=&y}=1hI>$O z<5=kD^fz~L_iB5rCWxI@SGAHF$`cpt#Okf=GtyH&#nxax6S?GTWd;XxUqmz)x(%e~Zu z6!V@2Buo}L;%#jpj*CDF!dZ2{b+%9s)A6^cYo83zu*pLN0TIsv;(74FA{zS5Sm=^b_yFi0DO8LJ5=Y93@|Xoy}ZYq60hI z6N0VPmWZcjVM(#leT+N3y&SjqnKL_xT2Md(IG-tC2U+@mpk9-yc`NIV;XT{;Oe>l1 zA%?{ua;)$@4#9(ioI@ZJ{PmE}A`sOFD&*CmW-hR*?6ZXTlzj69Pg1|O#Q^H_JdE;t z>K?cK{UU-gk}#li=dCe=aE>O7G_!2~06~Z}!f5um2ge^1&egFW%6CL`+kxBPBuArt z^8EFf*oW|sHL^l~uc@U88tuUS^+AAi;GpiF08zz&(tvFd_QOt72NdSU&~xx^r({1tMuEJaGQL zDzF0m3h8A~f9D718S02EK%5GK4iHKMS%y>Jwq~`&2bSF__;4L$#lr*-?6As<|0)4E zf@fJZMBe_$(D3%$+?*7H_9v|Gm0<>nK+%hi6RtN{+WPYX2Vby&5z-V$H;w&*4|=}8 zPqYDvv>o1346V#C)D+SvL)zvb+p{|uQke|@Gdx$QY&!1Q(Dgz;6M06dzumQ#zwQb> zD|UVWU;`B2fT3*%7D4^p-6d5xn^7?QXY%Fs&Mg`(@KoB$8rt@vn=}N+Eh)Cj4IPbN z{xj>4E$^a6r^YYimAjy>`gQd5Z&$}Zp*->?&8`4#rF#0Gf4~az|GtsWU76N9)QoSJ-BP_5M^S}XtqFS4-~au(bSS2*!l>}iI3s_mGK@bjdmz4 zJ15TJHc-1b{>GKSYw*aVN7{eO%qoyTC!_4N3E*Et=rTL~umo_Bf`6!y4H<)}LNx=Q zQusL1miBHs04QEFtirI^fD|hQXY5KNQb%6yo$|Lu@Rnm3{X=Nh+kV?7)|;Oq2Cy`0KhQ>n&38Q<8)Oa z1sHYZculkA!B}qc#(2-S0&pYIOE~4EfVkm8M9CNk;M`PHHB7FYE`IP%`!cXg95B%v zxkjCE5y|cR%+nErm|}Wzr5@Mj*a``U^L_+-%*1t@O)eMl<8m5LBRo*1SlYvj%lzSUKM-k^vH_|aBE|z81dKaPx#QXK z(B$sf^$EaI90PzUoH&Q8w#V7b6SZ!)>+bcq?z=Z#mkjJ6g3$qB&07qnnV%(?Qw{MURgR%NR7{K{L zAS+YBapoEn!Iqpn*In>CWN50bpm+;L1^MhR2bcqSW4s7mnYn5DoSF@SA3!e&WW&6Z zA!vZ^fcb#=aq)z7u~R*8lD-038Zy0wDu^;*(zi?^sHmt=!LaaU7vN+Rvd|Tnc^m09 z1bM?6kW>nqsCjpn_Wc<245o}|-I9Jlu1C`=4_mX4yc--Ij11w$jIwbL1iMgJE9Z6gpVTA!ZkA2`DodHMudyu|`2lD)g zFr!n0)BhDZSY7(BcVPDKD+IXi*E?u#3w*>9TYq($>%uQK$XdNjvHwJKM1jGVo|?t- zpC5|-J5f#xA^W&vrmJsygKB)B1M8%_enrI6xqlxB*Bcj*LC$aLsv!%gm`sibAA<#= z=KQa=M8IU@OS3r`Fm6OmjD*S;ykH&Yn%`+My_D$N%l3ic{~bHutZSRrN{Pb}zc zj936gr$T;mGJVn8x8Z^^f8$MMgu#ii{f`sl<&U_(0SE-Pm55G=880E)=fhtH3jGLF zgqzJRz~CpEo|yXAUk1~TIw2^~QDmL}Rje;>UFWyP<2|NIV%h+uTaz}Q$a zJ3G5*Mo4o1rJ-pHfsY<(*(>#Lp*Z|zNhQK!0g z-3Ts4Am}471I~qD-2XbitHxmeuWvI!Khw0XIeh=)#QL=|nlyV?&%$kM0wxoYpNI}axBu#_WrW}JjeG!+~w%^i(>mn ztQKD`|^ANt7TlCL|beFmRQi}8TZ^rnQ1`L2qAtp5L^9I!K(#Q=p zs2EDCFDqa^pM)Z~^si2XG4S^`h%UN}Wq`)<8r?v38>>H`bpK!cuY*qvH&9bopA~!; z$@gDg!Zc6#UR4I8KL@?-J`SCD;2m))8~LDn+0gs{jhf;-^ni&ZGX$K&%)kznO(CcW zxFTMRbg05Z$`&oc?iqB#UzsK*se$#I_TT&W!(`;pNu_|%>~0Zwb+Qf)Tf=cBr?KL{ zP#juaW~D>*=i^MXA@i$LNNOar8!(}I8wu*-ju|14HP#Tc@MY~!2S8v-?sWkDW$Ved zoP*;jEo58E^%NCYFY@VoA=ur*9<0>b^Urajh-}{#T)6cN>n^leqw{IjC>8Oq>tgVi zE8_f`0e(+`nrWVW0CgLYtAbNCGFJXwu-)%Riac~peeDgF(}Z!9&pS z-9}yU2gRBjKLNaRBNqZZt#T*{+M!qT?c$D6p#?hREvn>LONk?q_xPLal1;3ag!RWs z5PWxW{_b5L&5re?{s)f#`_9CFz#aS}Q##Z6hcR7m^Z^s9z3TxK*D}QWF%*_CVZMGl zj>7Z|r1yX~q+14^X$JRz9?`l4Ng zJn~VIP7wGNP%^0!spJ2p9*gHvLRT58`;O3YSXJyXfRYAl!JU-bU}BO5WCbHX-z_j| zb}Kly=xb!nA7OF*JV^Hg57lvorTwpxN7WO1NAECidBtT>XyK^VZPQ*TuR_g~FABB& zH`6}^V=}5Zjc$O3-J6S0*aZpR`$|IOo2>RwDFIA$nA~G@Wn@pC^~-JTi<$pTd2GH8 z;u_$IujRts`uy27Pn5^XuwP_c0UHNamXVwJf zCMbDQm~Jok-HPxgelnCwum+Ti7B@70Zxh}o zp}7g_ZSMpV&DQp+#hei)ea&ucZf<_w-P1$;m+p^-Ok}3FPquslOsrwus)KeQSAl`> zzt1zD4p&$mQvOYI`Pme=PH$U?Py)2M;Q>*X&6Nfodq(=IGr7m0Xk_mB2n(WNyK(vu z%!H^i(Yr9aZ6(`m3AH$Y{+d~b)7$KZrGEpFS!bK|qmVNL2E)Jo?(9|}Z&-CV)n1pL>|kULL3Bbk{AzX%TxA1C?$ zZ6M|YL81TDdNUHN5DM|=3~kxm|38A{DM23*Dahzb9Rgv+*6!}^!!X>jZB%jqn!^6j z^KrwLPsM`mwldvt-UfvK0^6^BDS$SLN4M=5=?01Eh&->&eV_INo8r;4YWe@?kb_4d zQFa=f3(lYh!Rxo+m>dQ3t5@5g{BI8TX9{^vMKq$wHj1~VlaLE6Z0R-9D) z5fnKj4H_O`5lbqO1({zCwUl)20h{v4Yvb(=O?wd-*-yux~ z0NJH;*IsT}1~k2mR>j0R)#MJ>Sp?y_5_#sXDHO;TF91TYwZFe#tg-j-r#Ed9KW3OV=X(BmV&g~I4l3Vgfz&W% za~R!w;T`n#RD0)g%fNZ?mw{hE{%K?=WOyS1-y}g+BrilS`@`Eg+s>_R146%HdpZ}$ zBTFMshfQ8;+5;hH$FKz#>+!b3Tv+1Az=7zb09qlT7-NFBW?8-^mhAp)b21-8-;w3|NjUr-VS4 zkJF&9h8Ks=&JEFE4WThSx_gs_fxrfJ!Q_rtU%xrD0Xo3Lbo;L|x8eY40-x~Y?dwXL zlmPAdvY*1TbIMwOn&L&aEmtUBm-$k|ngNWZseEn&)B*wf=$@jk&xU`&s73}U!f7Tg z`1DHo$GqafjVloJfENxdnaDBItDA78NK*i#lY0+;_Mj3siRdibN`6gQWUD2)HzO*` zHnj&;a!A<`Z8a@a`04Za6-4)|MW0TIO0t{${l+rRBak~(sOJb82!7Iw-fo7Rehx6L ztbfpKQ|bt*lH1GA)vlD)JxMv6QWe0$LG2S-LEjR*U z^iA)z9dIBdL3c?2LRHwE!()^^bQTy&#`w^Q7UDA_(ZdUf27) zp0~D}vXu9rg`C$kam%^^1o{9ap5fYx&Z4rPUq82i2t}?FXmX}~>Cz>_6sU66 zNI$#z5dhEyV*nqG-CO=fQ39~}vGmiRN`e65+YMc3HzTP{J`lH4U2oZnT+YjvL3V(4BWMay@5fVsXR|W17O9gV|9wesY<z*?R{TN}Tcv;1GKVn_ECPNc>+^om8#~-C2M}c9>my?SWCI^#$ zQ&99gS2Aj_bsJgiK(2wwnP0DN60?Fy*8Vx3WYpC!irEUtn!PE>mIvKvt#Cb5PxYT@ zUKLew+#5!le6;?}^QI75l{npu)Z3Y$=E48cdV{+JB*0)O4W*K6k91BnMhT=R*@HZ4`d0n z>-&yw5o=b^w4jFUM#~Ec5`ggBP=gyfQwR!_9Z0o0w7Fs+Wg;OdGg!~RFU8$`^n^a?`v|T zAG7a9Q`PR5l>y-_EY#7E{CllXOb#ez|0zBvdi;eL4{d?XEdxv|Sb$}nbhxsGc916_ zHG_P+UA?wm<@AyiI0yUAvF)(aLlA4OGBMDa<+{uQ;E{i~b=~(7`blnozSnFHY`UMB zS@_SU!4C>3?u{(AuiPA|1Skc>WPg5RWZrt?Rv>^_(tSXl!X^=oWqaqM${y{dZ3Yc) zZ(JjV@~FRiSGrMGf>Z`>c${f~;JIEPhgj|=S8fL9AVc9HkOx(qH?C@Ls@5h-vZ4#~ zMJMKe>AanKVGEb*1z9CNSu<@H0owpqrU1NDVJe*(#=WiAfr#`(&#znir-S}j&J?Zd zmNWeRwxBpz21v(LB^VinZpJQ~ydn98HQLZS{|1SN+TgJ-H?HS$3rT{zc~r3L6TSaE zN%C)bhl7x5!4*-FupPSF)EWge;CJT-mnRwt!Q}7293-F-nSo~H2vp8pCu-!M21fuzAB4 z@w09;Gb%!?gENiZhh&D_Q0h09zt@i3$qT%0Q{jc6prD-+dvxMU_- z0KSYieAaU0JjQ_1ey~`tA0KCQfpE|t&9t49p^Ac!1+ND;WoE)zV2M%Isi(#Rcfg-2 z`1rb)dREzn@qcWeoY?@7idkkbj(Yeo5F{(FGS;KYVBPlKs{EBS`11i%ux*yxuTn<< z^JD28n^IAo2ZA}{lS6FXZ-K*)GZ#txUkwHmoWaK2h6az5Jp>aSt4xdx9oAXWfA0^k zJ;I|4?5`cRi9GUAXzG0bx1}wtdz1|H5Ks%z=Q#F!6qa3@3+$`sp=JLo`eqG{C`gQ9 zh6x=9;`V^Q#p%84X3s@r?-b{tw|sVO;?+%G<`;pYAzjEPzGq7e{jx>^Xf#glQa+}| zT?UgouDy*W|8hI2j)SFtW;%CYp%f&zhRA=57*I~I5z&u!T95sSQ*5siL3)8C>|f`% zA6|n1W~ev2DN6rhBjBP6LdZOpX-pOf*b#NNqjEjPhb#pIkCVvI-x^%rfgFW&&(K@( zEyC{_kafA&WCe1N@&Vb4hfGoF&!)GE`UFX^rqfkYFE>MNNH6N+^sON; zt?<9)JITUBYTG`xEFU6zp=S_|-?WM7kj+DU&H+cSe<2X|jD#*{yfzu^E7o{CaC_T@ zP1E=f9uqir+lP)hXpil7ocsQI_dNN5KE1y9o4^J(1=Fv+jRK<_HTgqo(@$xV0nbOn(;NQsbhdsDxClU| zp|ja351a;=-ep@6uwD`q3FV?gQA4_$f&pylUo!Qet9=oN{{T;iX3WA`r#p*@bmpj> zx0#&(1*O>2j3UCmiBaEl1rA<#C1N7T`puNoWPap3PxXUpMe6FaJIJW+?;z*gv6E8azkafj zF*-iurn-Nh_rBI1$(`$WmYt-|I(#_PvCjGMfVA(2f4(8<V8=e=TE~DcJb3#P+0W> zY8{Y#km3BstB^`uCK%`A4&O?Ap3*Q4ZmfBIqbBo|Cau|S!tjc%dp~1KE;M?^Hkpa? z|6I>*=y|fUmVQ)>NAGJ@aSEOC*?4soO%`4yOliEYq>vRHVW`UR@%yu?4aha}fQK4` zIC3|hf|gx&j@F9FaU74qR1eKOHx926Y;f5Mxs9#;jVMP5J8b{J&O53z9t(H$3=Dd7 zCqF*iQ-|EON5^6@jrQ-o(FXu5?Wd+`K+4Gow6xlIs0DybIFKd%Z#pqSK(G|MF+#U(W$)t^8j?S+ zHXneyH}BqdF#cFoj|aiRQkK^`+sjo*N}|x)#ThqxidD7DOvV^6M5$; z@SqKYYvtoDJW%FOm4#uIw*R5pRpLCgRGY5l^MN;h`oWf4{_E!`AR$g^ut&`us6+N- z>kCI-<1*o3=0Q{@n_c6^N^xyxHhvxeD|*pW_4~H&^V0bQ%htP~)|xxMWsiY`cWoh@?LSTAz&9fgSEWx5 zfYSo_QLg3{{A!Knn@q!k{emu|E#bBoUQHiTX1l~-Nl(81sDi>55azJ}$-(cm0aB{8 z|G?O72xHFVj2v5+bV;LZ^+Elby$x})k0n(eIJ+Yp`xm_fx-mEl0F<6^F@D=p%P0UNs8alm@qaF(-Unc9_Um!9M@NQFL#C)3 zV9jFk+u+Q-J!nPXl=+?P&pW>C?VRGQg|8F&jw2f@{PSTG13=8#=?hG(e-I27n0mIX zxC`kfJzXn zo;?l!;lH2!C-Pdr28Cl|ZCx8!GEZGMc9`E-&X-U71^G?eKW>tC0L@Qjwi(&;i=|wG ze?Yjggfb@@APl5OM8&rPQRh+F>I&&;qhlArAi4`X!uM{{eHLu*u#vyypzzcqCZ{8W8uVy3KD`U@J>&hh_eW4c zi+A2IPzF!|1U{ng*s!<~$j5>4CcCzBUiF`rM!{1+f;~!ccQ-Z%@nP0Ea>cYF@3qx= zI(HR!L@SpDk&;d4xUlZY-`_OZLsK1jc`l=d*<^013%L*}vauFm9>h@cf5N;Hc;H&+ z|7@adj$QJ8VP4P(dbyTom#@0DKuTcJ}{4=Z1hw1{>9FtrprwwEWb_ za4jE3mt;yhRM4?~QkT~CC;yI}=&Gi*)Uy;6jQgo#{Isq134?=^p=kdz_Fw4ssezWw z=7$e^FDPg(Uh>)fwkkM2rtY<`9O#38Gx|A$e)hpriNc*m@WxtJb|DY zKtAV%#Un;ZpNS9aDE-8KVWG?%Y*J0x`QmxN7oS6nIEBsuR-o3uv2N1bzYk1I!7!pm z%$c368|3ka?aR(VNap%fcJ$AmpfB}8l(}ZXq|8>w=B~5nH-mOsli9~}t?7LwyxaGt zg@>NFAB)A8NZ2Xf1v8$e;$)NjNA}b+6Xcd~ybTnJ0aLpdML2{3C|by$BNVgnA`Z`1 z(_ek$f3N}uPXDNuN{Wk8;KRXhrLK&6l&5{|b%6Ki)Q_rE3ovRy@LIRhNbtE7L|ju~xc8>|{?b zDolP&kMdb&qYYLHq@RT(b=DckhSB7Q3>g(SWEmxpR?1d)fja&z7cZJXCWp`ueB+Os zCja6EJyih6Q}QKq*s8|L_5ll1C^v(!p4IW zMl&Z{?1I+uhn0RL$`KNv2i*W{=gaP7Dpv-bd1V3Z%`hFMjxI7Un_TluI|hfQNo zevOA%q1$&Ahvxi|(us-C_Togs*Fff0UmisJ>xuN5O4Vie>4}Mk@EV_iYfsmbQ+$Il zj5{nGs;aP+@**cj*i4<&wd>_%3`!V87JL2G1vAjm<72kwIW2Vu>jb-pqQ_tq3?VhO zVVyO-AM~^Z`&S)$i~@sl#Ey!n_%{}G$GDBVPh+EDZA&8on2~;Ub>lpZ(WrDWnYw-g z%Clyb!!zY~;A0M=^F(Wahut^EaA9lshw*;_L~vM)*&G;Q?@1yXS#XVL=^!F&5v|y>r-;7MTR5n>f7X77jy)c;0LcIacAg zA(D>oT)Qt+YOH#Dt0IELR2es9N{77ucw-x1mXi~v@R-#+cUQkuw7sO$`51%-bG5pQk2u56yJ` zet>TOe%xDcVf|;<6VZ)TmB#%q#hkQBkr6dd0&^u^TkQ=QI@Uflpl2J?zu<*gSV+~C zGZ?a2PH*zcLlevzLfj54>wEOQV+v_)tmvNaT7Xy9wNBI*qq3FF1X*pC;le62^YRF5 z+Ssm~Rjts#U%Ncb zT6@$>KkxS5WcsB36{FPKV_r3bK@)ZHJurAFh5?5OE^io2D42zV=J%*ibDdT4MnNkC zJG(T(a0KSV>X6<+`83x(Ppa}ihY=wy#xD<<8noXnL;*9?0cK|0DeM~(9rh0#lBk4S z4+95o*WcC^y&q(k9XU8iwrS$*KpunaVVXS=Z`ZZxpsKoG?Xdx@c zWh=hQ^zj}-BZ^QfJDL<>pw;JFzJ$u|@Va(K=Bbr#xBr>zgU2$*Z`cG}y(7cDWSfmG zmuD%7)0OAD7Z;(6pXwQ3kc+TP?+KA?uD-#Y?IFG@i+XIdjOty*6`$Cs1xSs>31}lj z=1;w@d6MC3&ZgDZF!{DW>SdremNms9M~sGKm5%Nm8wgqP&_O-My5?{AF5)p&0JY5j+NzI@hJGjdV*!G8~rTMAT`ryTxx}t)Z}xq2=L{2 zoO^pi#oSOr2uG&SB;%`!e2CD)e;M?C#~`a-RWMZ@h|IPi`PB9BNX#KJg0zoFKgfe? zRAmMH70dz5r3zRJ_So;~R?gai)Z5!>9PJ7x0WZz&9#*i$@|e2&Z*^JLcr<~6Em!Vl zccvFma|6q=?(KnR2&;3N!5_77uj$6%O5P$@j`rq0sEmuuwz$_si>O=BLp=fvWMSi( zQ@mVMsKN2V)ps5~fwGi&Cgz#EH!jzbO%?|iEQN4ILlvACycA`9o7=Oi)54X+nuBQe z2#cxz;Ksue#e5ksD8W{TAKz1$(+-%g%I)H~En2&hNka>?;u+TGpICM@f;mK8OT)h_ zLFWfD3e2g+&Kf3UdU_<#aQTT1!?9oxG*!hG#7y!6RF2I$Jegu=Y6A5^Njml$CRaD?s%3DwAxOXGWC z?p-Fk>uQg)ti{zl3FyGW>a`T^%0kr55VGeaPjj~i68f!9 zb*PEwNJcSbT5>kg9_Y|Lfue>bbg#w(A-kgFm|J)q^vg3-lvr1&%sAXpSc$D7LwQB5c!m8qG8V_Gp@Zl(!-fR$xW6foB2*%X7COxj| zTSWg$z|luQ{ zOM#;(^yA&i2nzux*xje2mQUU`e2>=plw6~netUkY=7&#ah_?D}5BX2}c$NY7`L`>9 zy`!$5eNrT?Iz+DcSW}3i-Nqa;#i!z!9tPo4-EpN2ksf)C6DO4I3ov@>Ivy*GelHs^ z)%SQDIB@UiqCC5LdYpoA9wHMf=oDG}Ts>Qt=&F(Q=M%MDp}{_OAMURB=O6Y+D4Csy zC+*sE=w9Z)?|_G{2f$Z7uQK?@2Zl})2ii_YdNqyxw-(E4BOibG(VRa(sv!Ez%2o{h zvQLgfhOOUS*=;Bl@>8e-Vc08y#>wiRF2UA$`x9+slB7?spl=huhtj!=Xd#$Y{f!pp z&-;G5z1ED1U^rp<$Pf1k{Zli8iTY5GkhR52F-2BN@(;z6 zYC#sADl2Nj)>*D^9O?t6zMn~TX&uZczj5$<&UEI0my_n)OgT%m3+)Rd((zTNc#9r` zNhjNx)vG{{-6m@IR-2oh_DSnBdQ=6Q#`@+PxQ3;aq%Q=QpE8wg?yvmRQ8x}IbjDPe zS9Hkg)v2<#k6Jq63H5$%D|3YWoOkmrEcjf4qruoza;KbBt|;+9daF$FckR2z>PD3^I7E$?wI<}`c+BmUg6QmJ-JCXca0aV4(F;lrY4 z%{SfWxG`zChw(4ysGcem^V+2EWR1m4@zRV1`|dH25m_-ma9PB3rCuz@$-B?jL?RlY ze6QOy!?fo>>hV36ei(+-o>CF+R@L!-f4J@8Y7)*x7MB^+)+3$2B7e*&Jft#1!s|$3 z9F?rL5PDdiR&=7!Qif_G#AqUABBeggCL0kY7SP&voHw_~tm8H*>KLIJK2<+jfv@s9 zx(DHY&s;t)fX0oa(;hC&`>sO11v4;|8G&uRY&jYiBBvawLZs3rW!-%)r>qX+Mm;#; zD6NmN5Qx8Y@aLea9*P_v(cfdy(=pi-))gk|JQol^$R1n`TOi5mn_W%iNSBT4T1$_< zA#Ly|w^YO|Z;9y^$Gm9WN_b{b?qG1E-5U;LiItd1OoWtEhoMp+jl0U;#2VRi9)|!b zOoV^w#P>ik{+mwJ0TFkxuG~i-V_jYD zS4psJ;;Li;?Jc|{!7+m3`}}U{FCBW<-V6mMeVcoP)cb0FG@#Jd;#^nKOM~nLe{+>` zr(zdrFDIM*8Qjf1M+0mv{KVc-eSQ`)->MjD)WjZ@rh?K7zbpK)+pAZ$y5w-sOV$=#(3uYy{0>T>NkP)VDFG<#|2%BKk|#b<#r;b&vGF z8>Tj5J4uG;7aQ+;;Gg|Vye|aGY^mZ*PHi}ckWZ*<4@K+oRNwVchx7_oDwrpUm*Obb zW0-P-`Ia{{YftWOE)jc~>B%oUfj2&Z_(Bk7%9c^oX# z4O8@EYO<_1yUjXCG9|><%r}Rk#EO3AYb-@tKhu`Jd+m3&z8n9g2AT&~8v}Bb%($)~CSG6*67;k@R9xiM+3dy-WAR9iCT^slI#_#x- zC^e(l==*Q;*TXt6lm1baH!3TExnp)!uVnJ@$32O-lIrAAi~kG-RcT(l|Z zVuB)l%#|=A5_U_^=tk@G5tav?GMI4b#<`VpEd}8!xNeb`Z5hTx!q+~Wv{6x)w|?D| zMy;edcjAD4QP{0aR!@0UQO1hmG3S~ zwq>yN6*{3slD=1IW|*5}7NSph$J4kQ2P>0)-fi}LZm5}l+tMM&*-$VzCp)h#4DKt<=wx%D!=Z&>`GubV5Vr4I%tE3qpwPZB{wE4R}7Kq{KEzD$nk5|mTwWKqF zJp4;~n(#qVnxptX%%m6OAGG%1d3!H*&mG+2<^TGr4L$jSbz+Ot0u=a7GVXr5+(WJt zAn9K%*feLMF0)Y4Eu_!99t0|gy}$N4Ktr|B!?niqzIUu?(<=_k$1ic(sJFq?)A0;B z7f5L)nnQ@)sWWkuBRStbh94J&J(@otQxf2Ens523nz-_AOl{RZ*?F`U!D|9`w*w?{ z$M_wn2u%G>w5eo2Now8FICvqIQhhF|!ddQ9O<`bpH){6h zvi`T>?u8kzBPIRaIYm0DF!U;7wvBb>0i5~++z1w+zhcp0JodTrcHZr}`QoYZxLT2v zs4+#y6y8C*0hmCQp*%OZn}UUte3*w#1WQ9Y?WQ6#qb zOYO;AE&YS%1O-L&jWec+y{OqN$LnKG;)MJKbF=1mFPUbu>d&Mvh2O##W1Fzdqzr;X zjJTo1_?bIjeqb;B4!xS(gfcSmm;XxvgUI=@u)82~p8eu8PGZ9z`XirBXn+ySa}!0q z?TXN2h4$p{Up_wM#q4O@NR}ZV^KuTA$oO@!+^u#!l{tK(!L*<2K_)ubYp!O#;nIR6 zcW3@NRX#%4qB+U$k-J`YK>)2E+P2*NlZ-7s+Tqfp?lC+(o=(rY{ztS{wuO27Gm8v| z=xaZx%sNE!e4SnmSa-xzk6rYZ{pe^OaOJ5=uyJaDVFT4xUOlObS)3bG|>3N7;M?RL^PxiDsOKb>qNFOPNEEBz z)$KeK#}XSbPqdVke~o^TQm8fIH>!bUO^0WFH@hxcSsNE6*Woss8bTaoM6+V&MBg4R zVZX`g%^|$x-;Pf$T&>TH)kDwXgIeCXi?yRy1!k+$%to=Q`J@7kW&I~?k&Xc`WPfEw z?nps+)F7)G{@b{?;stob=T2WfEmMX*Hdco z>^U{=k}-NApV7kleBg_m0_B5xxsOzj%3DkYWd)7;OoG&#(DU7Jo}>XvEr zut38no~PqOitQHNce+|G>l_Th9(_VnVpw8NWQa3-Qb=Hpx;Ok@JV}2@b*#SehhxED z`%-Oy*3+&dW}*7VFp{m2V-NmeGA@{aud539syk&Q;(%@Qzw0zrd*dw!;+$|MDLp0o zWm_6dgv1neDR68!&cCw0H6-ptT>tq|><+Qne51G}cLQUl+#yHvb}mngp2A5kiphwV z0JD5Ujj|b#+kc1o7iO#f1GCr2mx)v^!yP&yiN@QI2T&XN0w~&Cnm2#BsETFGzsfOu zO|Ba;Iv%+B!*tzEEui%!zhPfc7gbtN;8eRTen9F+FOl0>Y`j{|;)YwnH@TG($^{Y2 zgZTyBMvvW^?bLlE>E=2fy<+;9d~A4k<17uLj*;o>tJpf30OmLHf~*ad7@pdC%3<5d z>UbE=Oj#|~&-ju|G`oj%F|@W2 z908L1w_|3dSiJk~DBF~>j=w%D*5Mmx&L(ieEKiFzH@|yS*NAn9R7?43;sbivLGcUY zycwQ_ZsJ0ZUip$ybs7j6xwa}|n7FJ#titSNU@SU=el4I{jK(Y2S6QsxOYW1bXenME z_HDSmKNM^3;W|?KLY}Bt9pZ|N*Uk)9TXyu4QZc*~?pc=x>+#`enTeq(iK9YaU-cw0 zqjcn*yi=MadNkFK$5Vf;V~iTpzuQ9S(jWL9lg<{7Pa`!D#4hy(jjDt{oSd$Xd{t3W z`3nAwcH*E|;mCl#tz1-ZO6;15v1>|=GB&}2@0(o*-mK}}VyPv3>~ez@$}-@~P{J%x zGNO+opAa?MfQK)MzS5bpNe@LN#uatRK9BP0x0`QN?YTa&s+~D9a6cA5bJE>q89umUpOPHwEGwG2Zxu8T&jl;Sm zF$!BQ*dP3*G3#TYK;qeAeq}cAITi1#AfmpOCo>Zg%zxN(`25f=A0y6ZpkPvk=19aw z!GVxsnKuV=iu%yk?|-nlVO^aG1jKMAFpFthWgOMn)njkN? z0cPLC5z!#WkamYSZ32hXz1rn-)8U>ci^N8#ZW<{!St)rhy&3BcZLtv7X^Om+%I8b| zM(5{=j(PH+@F#R!@0xBa{!I0B&0djNs?I?1F3o;GbVUmzk{eFIiqrg5CYD9dUB|k; z`*Qt^Zpez&WIZ22&a2%77PB=hxmCawC9nt$oMGc6n+|mCU%E{{Hjs<8JfEa*3Y3k0RP9K#t8eL=QWBVbV-q z?6zF6s8|WZ_x>S~N~`g-HJF_p*3%#PsFXJwbqU3WV9|3Bu%`Hw7-X4X+J3aV@P4` z+l55*qi+t7q%}|hvHZm>Lkd(7ekmP)5^7XO@eDI_ef8pE2*+Z`Oly){zG~cZ)8UoN zu4TK}g*Z&+$dLB|rM%g_VP_efE%WlZT^dIKl z8PCAzru|H($Lf@vDbuT1J5i-O)}%x`73bNctvCaZn+@MUr z-mqY?yl2E5O((kw`OnhtXqa~`+TZl2;`Ykc`k#fXYNGC}BL}53t zh3JmdTyb`E=e!)zT?7$i?%@6-u4?jby6u&YoTC07XYFtDpQNQS;YIj&Z4EZPZjOb{ zJo(l>#t2)DDAKs^O9_3Zln6OH72=Inui6!zBknb4(U~@}ePVG0`0){pqh=_`;JPTx$Mb7d2|A*#K~*V5wY5|owq=m}eD=0#BFS?*2@5t4(QBuZCQ8OvdFMzgPf zv~Y6P$BiHPS`=Vrmhf%B=2ZK)A*QyaI$t@2@9sIu1jUzuV% z!tdH?Fgd(53Yx2Dw=S|ixL3K+&^xouz>F^(*WYj_v+b2p?3_c?fLC23T1UcQwk9vw z`c37}BpE!N#&kMze1V7~JRU+V}VP%<;+ZY8!0xx2^nn z=jio`4{1Afn>*7izqouULxxPUrsf;(p<{HSkX>rF24@^6T0iUROU_1BQpGqDMP+y@Y_>O z|M@iP#|(iZc=t~e!?&=_mWro`9#@NKx!PU5lls8oV1=r?{2spMI$uTu9!5QSfseMD zX|dGH`y%3m`bFw1jh5()v>OE8tR#2#+7*Io1pUfgT{1Iw3xvn~f<+~p-5z*~uWB9Y z?5r7z^5~E~IZMbi>R%W$aOXA9feRyko?9_2Db>b|FNmG;T3s#f?{Ana()S$RyNLL3 zY_hmL#s*QV2Dm_@y(I6fWk>LPI(FWVE$2MvKI+Z$lgmWAvx(oG;qR)ruRatS!C9UW+ zk7X7R>h-M6I0kNHCU#@phb(O6^oi~hD^U~JG4`b418ib2|D`OVjs28Fjx&O|+=cFW z>S=c>opqWdCDK|QX`^V#)_?Vl1UA|75{qk~$l4m30YpK)-`#?(g}+39{vicP6bCMV z-y1^VvA0{Z?eL}cpnZ7bo=R3X*Uv2dm%XMLyzd^8#Z_oPg*qDk^bYGa`{M`5R`Idp z&ikD!_eCDtsv#g6NrPN^ofF+@+s+s4KL?x!-41t>qB5wxKdP(tyV64^TEV>eE@(Rp z)njVEkC{2bV#jR~UvIP9lKaOb+QXQeHH=paPX6emOWrq8Cc&2W##BNWF@P$Ar*_;= z%yBtR*C1oePeoModM1FBn^Ah=_UR`p?@=>bLmfShJ=JW!zTG1CLmH zh`8gYXr+^IvT)Z6@}F${YZ}t-YO_17Mxh50S`AiQ2HxWf(Io9rr#w41OEx(|{=AB# zdHEZd6x<`WZj#g;)z!nB|GvwUYm9ZIYW5oHi48kQUo+(D3j_}Kjiy>2Y5ION{?hH9 zZfTm5;Zk){eQ#=mJARajZ>^B{4DGJ$KFQ~9(Brl;A0Xlee}|ruA@;fyq{~W~=G+{^ z3_4W~b49m@)?UCw8(VAx?%cQ_(-mZzSTG>sU(qJ?%YExPeagCQ?S-7 zi#LPeR}L9FUehc2`b1mg0dI`uKKc$Qb`}ZQlewZ{>`uj6RiJZ1ZRT=qW<5j_U>27{ zE$Hhl>Jv9?$_tfN@hUwnTK6WLD0Y(2=St<@i<(yQZ*wF7J%~W`Ai0#rwl$91qgW<| zCkfj17#Do?#i;Kz)m`@?*VdFA5edf>Kz-i(#){~B)eL4oJQvqaWU4Ck(w!#pB0PxJ#4&veMk3v4GZFIuAnJorwx>DGkkg) zKKBr=iApe0uLkC1RNM~d4J?dv$@t#(4R1C@^o@pfI?fT)>e5i*wCSVt!czk->GBPp zD;`G)-IYWf?WnSAW|1^^UWeD~1^7};D+`A-Q;SFrVk&#qA?Jw*x+6q_!4Z#oX$Oc~iWN3Zd&&dqp!s|W z6D7+MXy+i=gKd4^RvCOCe0kA46IUGjm3{z)CE>j8u|+whqva73mmj93Eh=OtT0WXX zqv2y$!a|(_q9re;K3Q0vYE#GX(2aq#dpA+rG^t3)lY>{8S+1g%K&eL$mZc|W;>O?L3H7+(j zFBD@SjG`-UBgf!ex_uZB@JG*^jR`5!>1tDZ6LU+<-`!O>>ck$Ut!X1mvqn#Bk5MMa zgy5b8q`t(&ElW{#hUE1V1t z@r?Idp(JVD&4-(HWN@#fkot#WWwEc(@qPI8g*m94R4g~BzNo1O-u{ET#4Z4yEi7|y9)707s;`~lQesw!Rn;G zin(fXM;qbVK)ozsIyhoQ(0lQOXjK}jFk}%r+f#NAIFV{_xWTE1T0Ut*nB|DIE6^a1 znRGkP)EF>_dH{LY!^HLHLyiyAYkOpe&$4(&$k%!Q@=?6kzI!hR+k+Ozb|Ooi`H1U9 zV4Par+M@hglEhfyDbdGY3Y&*dAi)4goA(Qx%irH`F^V`T(FxdyrF{q5?o-o609K%BjldaYXLeZ|O z$zlX5G_kXGe|e(sWmrfQrOABF(9#d7UNjHWbBcpEA?voQWT?zOi5gpp_I8L?MKB&AM zrM$m1IHZ%bKzeU6(=~fJyVx7^VvpEI$4m7Sv!xN4OgCLD(MHJvwu(PuYS=JU+{&;O z#R+#(x|okvwpYU&ZbSm7!!0JZxuxuQB0)WGE!PplP-rx%1TRZ_J*8)c|Bn0K^)xV3 zAUs5ZX?Qv{a6sKO2c8}PKUm@R9f8T8mC}}2?zQO;9$)N^k{#&(*wHucWfgUAq4-9s ziqi_Ux3t9Kjj5fm*?g^sM@BRojw^Qoo5SL}gub08`^mz3qrAoIPjf9YEDiF}I8i!8 z1E_kTZ|Gt{S`csN*(rCA-mtE`lHRRtW=B+R|C!==+q^+a-%LrY5w9&CfD;<4<{|5V zB29lRl5oK5I?T{)AB@ZKMKp@$3& zHDh9^CzVW62`?Xa_DOvJ(i- zPlAI&Z%z$`)nuymjSgtEs#GqdR`k!r=MZJx({IW#ST8d6WxAF+VHFc?QIg-LF40b) z;!jXto1=O9>quh&RHzlB*Gh&83cVUf$V++wCULGDQ6v#2_`vDBsKZunde&s9`C;go zU17i5*cH*6cQ??g2K%yR=;y5+T^Rxnv#)bHps!!3b^}`GuA8eU^EPMZj{Ht=;L3Qe z8=S4Acp2@t=jVoc>Xafg3=Q%m)gnQa!!a6<6s!8DuX!uy*^0<+|!5g2p!ddVl}N zPkWuTND@LJDoIR3_Bz>4$U)hSvS%zs)-k5tPEv%-oG3ekF=QE2$vT*9GZ;ptA-lZEg#2{%Lbr!(wD1tn`{sWTo7Z|`nG-i_e7 zFP(R+fv%;h?)=#dq_b4Q6pBR8SOMCQ(*zOD(kTU&dmTq0YGfQCyPff}_-KW0m zg8w2v!TVxF% zQJG;}F++Qh?4r!8mcg|&ri1OvA9@agJrCe*u6)bpQI?7YJl>L?(-VY+B ztPR8SYJZ4(bxHUlI!>kL3fII8nRF#4nB=|YDyLR=;!KpInA}dd)pZRRxYo1r%ed44 z-K=bVm2$U|4qB17*3PL@U?d(5HuMaXGm^v&)=Zl)qUt?Kx7Z~a-`489yR`1%iypk- zz~w`DlW#xjdhy}5`R&$!VfGL=v_k9Kjh-wV4gPGVI(@O721bXQhsSd$=y2TPOY~Vr zSgZDW`}-^CRTO^-kp{#o6m!~9oIk@mtdUmTI!m2Wbi_m5v#9#fu zhxG!Nu_M7no&WF@{sX3$1ETBw&$s-mngD*A&B0mx+eAP;`}pzYjrzO!L@>9)+*Sh5 z$pFDDO$BbFZ^zqGQc=dAu8Ahb?*7Y=to!X9!?^48brU;K_PpxiLnCoph+4AnH#GRn zQDJnuUCJOOt`x>K+__JSRdTRSaU{Wj88s)Tnp$mbhzH-rEDyQS#`RS&J0_vm9}@2SjIEiJ(fU^$Av2dsGQQ#TsdD8A{~l4Z-`I;l;Z!bV(_jj9R0 z*1UXX1Sl$aV*hH@w6oHIyq2++sJEmDe>wJBfM>++1RfWCxi4frHKV5tQzG<$6)-wS#~xJ=%ucfJJMk7B1#4aQ^E4wl^HMRRil;=wmdgByDH2d=BD8% zr%Sbqx^OUEm+_pSt1~0JRVc(4T+c}`q1G96)Ms?pK#Tuq6H}d1qOW?8;UBA@HDP!Ju6;n)LpyI;!n$lS`KH@iWnB-rezz zU`XZD*$5~ArHXKQv7P3XLFMr9p1&j255Xi6?6-Hr^_4CyKXgG^<211bcwJ1A<;NNq zviHbaxI9U^{e&dG0PV|}9Ul0R)2dZ8j(kuG$Rpy1k@T)sGCD0KPV8qANgF@u-hGD3 z=)#C%d>eFN_)r*@T9f=;UoG8aJ8JZR@`=W7jBO2kHn-OJ>9HuH`}+artVwn%RL-qL zxFXd3np*Sx$+%Sgu_|N@vq3i~hif^L>v`naH@8V@ymQsHteOl^M`o}$w;G0UCmMJ; z2JFMaXU2#n%?xEO`{&T8+NI)!%G&d^DANoI`-*DJ6fSaqSQEg7mjHz$E`+G_Dnt9f zddf%pO#0;=z~g<4B*Eh-GkCxN(`=N*G6``GVK4_-3F>MMkun)8HH(EIN%E|@ieyB? zEMkpdA$|)`KPe_22fVDsoR06#tFh6CZ_?RGhSjPyJ z+|-m~CKwRJuN+X3_K&@m3#Momf4ppQ!J8fXxs7z{1SMVUjj4?I!kg~qxPvsD<@i=V zIr{8->+dpM+U))?RI8pRc+1-Lk2TATCtY$Q${O0_9=%9?xlb;5=4seiGJ1))EGm9# z_5F%LMU+hS>X5E~7x+S&vh`Mdw0)PKT4aRRZTEiAbNx~P&WY`oaT~Hly_HcfI+O>8dk7GjlhQ3 zK^{^*=f5vi+M-v2V8kL8v_lb-cZzQ;6fz6QdskLqZ(9t=N&VcP)K5stP@Ny~0NAj{ z&OMA0-nJ5vT!N4u;_F=-uIbLN$zziPf%L78;e8sfpWmJFl;Y2#;FFCPvXVC`wvwCA zTst6c`14%4OH_FHQr1}H?{Cu^ew?J~GQx?y^g2b|sXu9bDBivIWoS4ZN=r$O{Lu;J z0kNen-TOL~$5vZ3%Cd6KQlM4RxB4r5#XtNr`7E9Iny96&Yv!)k^kuVk{nxN`b51}d z(YDfh{IBu4a=U;#cDInfBa7Ra8RpKO*tk6iW~}w?fEd^1GjLLN?Wh23Y?r9bOV4%VPne304JRRK2&kg*?i zfHT4TLRPa=KGs+b+q5?Z_MK(M$tS_H1C;$2Yf0T^osr5ZxHDdw6uSI_7liZRM|N)szf2di>~J?<~XH$a&H9-ofcn#kTm_ z2nRsCi2G7hXrC9dKZLj%R28D`MRX*X&ZNu@oaPvt$@;+B)FfgZODB@cnp!LdjTH`+ zhOZo5p3AS)l%4U(xauAVuG>|mAcg~-(K24f&2k+|4`d||>BJ0@ecYA#IYC77_*Q`-TN2*X*`ZaxM1LV_!888 zF}E%cVE&}xV65(j2|TxkA5sgX@v5&ZiRdqg(17&ypPgJ4hh4cVy8%sp{PCrK93g;v zf$rrD;W1ICg!kwF{jdQA^~;VYM(JM<;R9vao6R4o1-UCjN6e%BHz#)`!w>QA8Z3=h z2r4$S-l&^M(b?ZUR9MTJqL+R6BbrXx%fxWt zv!0^xj@J46FF7WoAs+^y(j=ek_=&64$YCa~j#_*Gw`L^k?hI9E z_4^I(Li+1Wee~q*%3G6Z&Zefb%3LhzO|U7k-eGi#_CW1t^EoX%9e35UD@FBFjQyoj zX^)n{?V6tr3(EbG(<1tz)biwSu}g72U0T1uv?0;6hT26l`ymRl)7C% zo9k`Zm=Z^?XwnchsvO>L1~E#E&lw(|P+qFe;#a1xCm++g&O{=jPRprE#O~OzKe2P& z9HiVo2nMOPf^i_`^qkDaV5hpB9joYVLLJ()qe&faig$oU@0Z*dvo2 z2im!g^AYW)5GyKxESpV326a|H9`Kh)G@B0FpJ+5AY$O$Q|C-pZwe7)4c|Zw}dEl`9 zCoD$(Uqcg6TZ2Oa{CQ}>jrre8-eAx6l1p~TzvMlo)`1z{I6AF;L$CK?|M9P=t8V#D z+{O5~vi=uW%oPmYrwjXb9Irm0)VBD?50h0%ae0HSe~7on@4nGLD6-A*pzhs3qk9xJ z^^^*OkyoNn$Ed~0&ehVDQ~BY$cfUhJ^i%#cmS6AJgq;(=>*{WBlij}jX54sUJ6QEL zlt@u0J#)VhrDo)b>Yq4NA)awXez1CLm{iTnUCOCK0i3_tS0<0@4veA)Che*$jJETs z$KIu7m9A2Pr}TPs+9ckEF+UBGM$Puc8eko2JRn%Th!zEVKmeG4QiWEw7=tTn7>Y&g zN|F%E?3@R2-J~Bm3d0`jtY%HV8pFs1!}_Z_kVKz*y&VG%(snyDoLH>iB=XN<9N0ZU zHJ*X0rc|@S%edj}A$HF&h^-F&d81xqWgnWnN_?q_LX{cfjnN8e5}oOv3(3KNZ=> zY|KeZRu6IL;~aBX=%O-6-GIcKQsLGb$AO{YrF__YJvDf-@#!xbfspV~(H6IU;qhic zU6C7^SQ?NIue(DKxxIl9zG>)$oXtCiXV>E$^?NPTf`0*hc zkc)a6cm?IieD*gP-nYQ^lj$$?`IG#}keKOU<=mv1vidTokRcCH;zFElLr%j|HbaR0 z9_V!)K|qE2^AW4j2s%1K%6+KgE7+jpadm3=SOCyz})#xAQ^JzezR# z@bOY`nZNoZtNFiKoi{6X52|>>a+642@z~!zH*E$1JGut@wAc0PK6-DA3~goAuBYw? zQ`#hMddt{c$d^?KM=Pwx3F*m>@0Uz687bSkR&ujU(V+A3*j^)gY7ge2g_Kv(e9aM* z@OYf}{xh$LyLqKIH5@|aO9p0)ChSE99UrXk4uu+=_bLA?KC3}rZC9;crg1eYCbnA{ zlnkL7dTxazycVUCKEL~<2O_6^e3dBc1{LEJTq8xC(bqj77TT-JPzJ1rH_+zB86xUr7{lNa)u@7x!!bY zOxuK|LaR>$7$W{i0nzP4IlWV*bd@lCAsx1McogNywDwcRuG{-=~3Go%D1d1)P?oJ!j&?%{Kk~e%uT@<{qRoi91IYP4p||0(lQkewa2AAm}s!rKz)VTmL; z0N9qhny~37X6L(=S}+bMTH|$ZdH%Q9>KDdw9_XaUrT+a$`<>k2`z+8^hJiLqDnEa5 zW5YeGx`l5jR^N|CZ`%p?W*DjeE!o&YUf)pX%u>ZTb35NYD&=Be(O1x+KiT14 zto4|-N>V;T3q}h2V|~fBQ#wF9+GK>jgI}uqZQpet^6-p%B)(A?^Q3^9Fy|LS@GS;OFo6%#w;aQNo8NYBz{5C(CiN%v|x3$UIW0ib^vIL^ypnhuo%2iSvTN`Kk%wQD$QtPHUu)q zemNr?FMkbqr0wE9YD^1iF8MbN8CV$71a>fl!{$H$>3<&f2T~><^#1J8zg*$@y<%H> z`-{!laPO55NOY8y@qVc@uB3`TJAXb|?=4rLR?7CawY^~>uXXIbVd$Oj7Gdd-Ja<3r z?Ccq~``WOMY?MvsD^azgJETfRMceLe9#nw4176wA0(S$K92wZHx2`U0XO)g(gOWNW zx^-*PRr8~yb+h3H4Fx#+ZVsJ%(zU$q1%}2@=8d{1T zq>gNd*B#*OP{fO>NEUtw$o_MO{MA9f*4qVex(n?s8 zX&-~ky88~Xd%Z~KKVvN2G@;M`n(6Dt-C+2%+`v|VX8JKU=7yCO#RK%hy9DH2960ec zHp0)(#a+lMQfg^oCVNXrLc#;9&l(tXE}Q4g7+6H7xeZAS6yY+~#>Vv2k__(mXb-M# z9Em+Q$HKG!gRez=7eU$S(fG>H@1BQ)L{zt~HM?+{V|63*hfbzDSyAVoYPT_%+e_z@}3eXRp_Pd<{3(-tr4T z546kttMcG~$`K%@m~90{29lR{5Vlid^C8D=m$iJ+?aeh+5p#*O>O7?)`po|FtEe<# zKS#@w-}^BNQd)&=M?b}*Kqk2=FOzp4#Lb&EQ{-G-MB|I@TNy@%Br2DIE%QtC57CEM zembxQweOZwxeC86|A8-j*ky9>;vrD|H5q;n;1Pwe;+W`zAE2|n4)cXCBMwg{cvmW{ zCYvb$M7D|Ci8o~+X#dwG+eMSG8D7RG$mBDyZ_G5Y;MqXtOxs80gJmi87-=R!JjbQWOeFt$Fy z+V!?MoStEn_!wyU^}t(2imgIaX`}3T=2@~$SJFktZ4&za0mQ0=ZECC|V9b8|nf}!^ zczU8e`f0+lwRWEa%QA$M!d8Y+_keVBBP!zOMk_rJfo+&k1oBHN&P7E7M$eTvuG+_@ zCG=Pk&YuzCcV?u-`o%l@5k2;WaBzvvww}BW%{qq6ZJCVVu-ueR;ziILrA?<-@}ie@ z@{HiYDmk;c+~x_E+xS=aV(g>g*`d^MT3P*EW_Ubz2*tHuTSM{J7P~_mgWjW^>qA}! zbl~D(v2R*RYp4$HN!AbnP7*hgYyd~llw3}aq4_&6ep!RMu!nU`Vt z%?oISc*`iJ7b4{5Ti!bjYM5L2$Gwm*_&e8Z69g~r#CD|r<|}IY#|H%B0Yc8<`G-oO z|1K#2uTwGr=-;BhtDF8eXX=COq4Mrk-c&59U|TE4FhM$K!|LThw$a}n{eUZPY${>l zkITZ3rmX7>&t?hBoOy-(8%SeHdi0^AEgStc>aE|AK>uFT5@Yq^ZV4>o`uVE1m1TpXI=kwL+^wWjNv&g~KV z_A(1pRR>Zh`eNEn z7M;iln+Yqe8=J|mVQ@xr0192Y6L#{F-w3>?3w^b^X7V3rIs-qy7If@NEpt;y) zFBhJU^uTgYaosz0I2p*oJ=mkeu*0 z&!8KSi|l4qrU|N&046(Gm%aZUXMLRe;Q;5-+9$Eb^L@-isY$jC-$iLpCH=jbORKH= zjhuc9Zhu{PivVs};R|RFfh2pcmy`PZHCmGlZ$6JnjACMdF>*vLJPOYt;GG2B)7OU& z;C^p)07y`Bn#Dr!S%d@p`2*2@=Wxrti`6waWzH;cG7Sg^2o=M}OwIm;tbP(pli_@n z8M!GJDZLP5{GtZWrPFSpBL?!Au6Jag${irH7vBsdxae%OQ7^MiVCRDxV8gNRMX-OQ z*R2e(uHk=8jKDj7vrs_#yZ+5x^)Dd4P1y_&IUYJL;8XE11h=$?AQMxG-B zo_coKC&xy#3cJ0*=Dq^Fu=XiOQ|VHP_?a+Ek5W8PP{fkUAOyxu8dM~-AXzBdnVMie!V}}>Eix^2Y%2)y zVwoy4bAi@=pQ`;6z@_zzxV*FNp9hjBiHmIoAIV>k70(!M{RJOe07HkNI9!x^Fmb4n z9)7xl@Cs_+6HJmlZnFw5?rK7g~ zDJ0v-TR~e;16iY?F5zng7wUznu!VS+2E93?F05!=XYOvYQs@{wbahQCbi@+L`Jrf$ zIW8!;7kDO_H=I|EhK>?AzY|&_}XkcQ|UFY2LW;S3W(6_vQifd|jjD1$) zYR#K>dsX&s8fN`-CMiE8c;{o!V@&?LDW6r}!Zy`lB#f>5R@gSrZq{|Y;qeHcith7~ zYKdb0E06Pg0u7fZ6Q%oo-wdEMYM(V4PxguEF*~&)=pNUwP#*;;NP*{*DC&o%I66#L zlb|Q(*`4Q^Y*~uRT)lI>q?l|WY(N6Yi6gR&%9aIX^@`KP>hRVY0mE`OS4UT^1F+l*;|~(iNcJ%10>0TlT+3$?Q`EUEw6;& z2Wz`z2U#m-Yd-uTE%zPc>Jvjw{mhVtg!ZTC_Qg<@F#nZ4f-Y*CW!cYTm4FU5pX8ve zsM1etDOTSF$EQbA-!@6SZ1b{zBrUGu7V;OU_h_Qay(q7xvvrE%ql!!Cy>HXGDXVOb z^`obNl||B*83J|5+xcaXG4xFX83g|cNarj(^^+6!M*<4w?T$6T-iOt7>-rqgIU!nK zFqb%y)8!Sg{H%~tI%wC2{n?qwwCXC5twT!=VH%$j>*ta_M=JT?(F1yuAeO3bY8FY; z&Jh8X9^-gwG69H(8@=BzQ_8};bVtI55#{TvH?JQ}wlscyd1J*+t7&-GFabtu?s)#n zY@}Gf&wTK{#>ky3;X?$7jo%s+&xFz@qvj?vbrIu|W4q1IRc#E5i@@fB&pKak#T(qv66snDY zNid-k1qd}ST-cprbO;YHCJ8gpk!kV6pISTiDy1<&2X`^utYDN{Q9*?`HqZsE>BrEh%Z!=l80BD_AjK$KrxR z=~@H*If@*>&F4Ke>T-kP@2O8bk&R`SmWPqY`?n^1es@MaZH@If5HJLxa&kFKEnBP~ zKN@Bn7Iu?geaZ73aR~2Mt0`C5r`!|`;mjWGvHk*k=t9(u2&_$nPE7q&w*aL3)AZC+ zydJ7l`vZH*%;OS zM&pyLYwqt!l-4x7Qx@1bcCN2qsG0P(bIrQmq6;u$tsSikC6MUD5p_MOUtboQ^r+Ynyqfui5 zBvK6b)39sCzn)&bxSX$#quAd2IJTyZ$mb9cYbqLW`JE?S@Abf>^rISH1C#GJ4~xj} zA_6xC!ZrYwx7Op6Y8tUT*t|BQ0p!cti=W>s09NVyoFfw6Jt{7%E8_I3aII?4_9`2w zW#`q`n~?egn&3=${*E(pu=3D<_e-Fg77K>h8*%E;e+{t@50!f!gPa{%_r09nRJGYh z_Zojdv^QV7(vo#*qg4LMKVn~S?sqVS`{@Z=$%qIngi;VO86 z-L=Poi-&qVj85Htl3*jBo&Bh#t)K;YI6`O2`b|QAM>@Uk-=&la3wcVU6S>5`X;NUo{*V0n5Xp={lUDZ>&2O%h{?O z^DAR}$1!UT#q8X>tXe-e6~x@0R>zhz)`7-P)=A}W$QH^xu*jiNJn8StJ`mG`)Oa{! z>C_l#>k?H{tl5GpSxAW@CjR)#yge+>fYWxfMAXr-hE_K`;`mfK1IK{k zM`ti2s3a=L2yDT8)lkxYnRxsoQZg#pGrVd732vL}!qjMhc{nY%)*dVcRFhtu#%@R` zoqcV@NY@pEjinUI^bj4iqR;N7Um0>dmtr5CY#=&V2o)ANJb>F_9Y`b_UOzp#gvoO~GUX8jqUag}4)@tv9JjJPp)p>!%O_X=aTL zkYn=KBY8_G7d=3as)5FReYm$n5$LAZ5xg7#o_y0V>!CvFX9{>(kf;f-sJx*`6mJSu zCiyt5Pt|X^7BICJ3=Wk3)_+EgXgGsXgO1T`7x#MI<4mNC@tA>A@6y%LBA65v_>?IP z);I~P<8}pp+~~SThO0Jf@vPVu_TR!hbMIPLg3td9ssBH!wyT#KKfiV9VB*7|@an$_ zw>DYC51z=~#Q$(roZMG=)%4k@$jt55lvl+HMd|9Liu$k<<=K()f4nxlBff_QlZDFM zR&hwhvp&aT!GCJQwQ!S%J?xbAbcH?p(=}8N3RgEoe796Ooba%<4KU3IpGkGAK61Ok7n$^# z2qJtYOup^90uJx!_pNOqnbn673U3^M&;0~kjg!`=GOEHz9gbaZ?{#Mc<%zLR!_d_{ zGu#U6Y_++1bjj(IM0g$!&2rcVS{X(YCn~^uper5>^u1XU*Y2@wOJ4)`qg#@y+rnG5sVUP8B=EbD3>ey8FGvH*d&IYQ{g^hLDiwGuJEL`H zZ;b{=vMFQd^w)Fs9rTtmi1*OAO)(7_=byIMXeK2=aEtF!llR+1JrCM#-O6O&uY7Gj z=4kg2T)F*!I`BceeFBw)(QEsENjHyVULqzmMcrZpaQ|N15rs{--|oWJP0RdmRqENr zaz=ZV<#w{|yL_ACLp+`?BK?Y&3S&P+%Gi{Qf?zQ*tBSo|JF>Jw#mDqdMBbBTPA#i7 ziaL=etVJ%(NI-My9V#96BjM(Qn(NCUQjVmzIRW7+gpuwmYZN`XTTnk7q(oo#C=>3f zxf%j7xV`U4ghd2VVz%v7&#-KoM6}64B~)yziQm4(D5D?KU2=JMYDP*b+D%+i{v@_F zV!_?rZ(1mU#j!T9tusep#Id*#E~IayV==DtQpr~hOf(ehGTbe@+LbUmo(3?Dll8}B z^?D#FOy)4Dv7}TfmrIj?B*v&^w}gtvvs>gicph9O3O77XFrxCGqJ40-jkKXj`mE zcOxe|Oc}NyuXi%7Xx+Z(MbZ*%xv?A@YTQjINOLEmPR&X5QKL$m8$*(_3hMzmsX_pT zwOkq@mJw|5)D_GO*`M+tdGS~*u6m*64;Cs#R)J}eGr6ZI!?gtrsBMZi>)EUGn)>wW z#&{s7zbHqWt7lNF-jjiGmo{CDK8Khia%=l-*1Ecy z3$q(+K|0}++GF+*;6D1;jV|cGQ+CVi>tsA+1BXSmbBN}dUz`!#6#-;$BcgQUvOg}( zdcVAqE0#o(o07>*vg~l$l#ko#ZRKeT9j!U%l3@RDcd*}pbT8oSS!9qI{I9d;$31qS z{d|H3`-xwaYR%UN?hzZqarvF?K1Nbm$g8Q~qXm1~WFNS7y}fce{3B8{VJb30w%lZy z3>7gveQ1UdEWUtpff+s~neSIBS;{+^^7ES3?4EU<8RT&_P^ilM?4EA!Cuj~s2uexM zj&4AS+8IBXS5OY@@|GT2O90F|4ID1N>%7u+P}WG7SE}2qHucxFl=pp@_K90Bk~_}S zk417ET-gwzT!{;Zv@_zthl^+-7m_poG%nR6k$PuZNqf3b!dGHvc~Y@s4b#C$|D_?7nY*y`IwDw03zIAH zJh6V+w)g>d&?uY?xM=I~cx5#KxeJd|8~Rt>uF7COE7yq~y>-rw7zm*(k;-slpe+3! zuFXsWW@br~9FI7Rq5|}y#@qB7PQ)yo>r9{Z&d%WJT~BPA9Rk&b<_0bD^f-OiqC5CI zkBMHQ3Vg`tr8cqu*`anLsX&{?`6ntR{(Dsy!e$)D9FXh%`5`sk_0*NJ#l>`dWM42N zid9blR{dUo80=%Q3wK(8Qqf8yqIK%K?MZ2)4s`KKkmyYG#g)8?jS^vWYaBCOZXy-TY?(R3R zP-_xF%Qw~i z*4rB{>amQU@TxtB!JaI56~k!hqMCQCPmnTGivd9JPlnV(tbOn;PEe<6Pt*FgFD6u( zDY>{&Z?;289xK%OG56&@hE)j&5ZV z3TM%K(N4iidz%sD)d}-8jjxc)G+=$ZIk3s@1$in;1~+i&0V?T`h})q`Iy3gfGq#X$ zj}3(Zu9}qo%FZcjjW((}yF-7)TK@*G3%{n3UnSi4jK3i*BHHL9X=O?s$UpNZ(M%2D1eByG*lP7#d8~I z$^oa*;ayPm#1%Y`%s+$prowqL*T5Id;j|%QR7fpAV6P{y@8xmvd=^a{klWp_@q-oa;zd&){n{UoNpHo|Ei0L7>JySClcRW&9Ty$y?PK9p@Gd-q@@ zlnu6C7gwxL%LF_C@3XePA0C4(6^b7=V89;WmvvvKGqk zY%a!YnUgZ(l?X@sdwyO`^YG)hU+hf>`&BMLX+Bj(rI?{XurQ8IylhKt}xgh*NT|lf|znK-MvG8umLMTI~eiA9` zDaC6}y$;f_5um#NIhW14=O!T*RCtDK8K||4G}J+me1+ZmSp$~ zGAej31&=FCe1+hoA{H%3ngCe~rJ@q~Z7;zK7=X(IhEfG5jFDqCp8X4G-T?l0#uHyJ z;OIu~NahYEzZ9H0u0Z?ZiNN$C8wq-PJa=2nmNp=uQ@tMu%RAxpf2~?;W4;JJ>l|bM ze?IM-)&2jbRo&gQ+_ z+;8%R2%G@6-k!jWoweWMYF-W?r&dS zNtwTAFi&SahFjslGi}S_bQOHQ4}uIPndvx$yR-Y($PE8?AGpVDRMl3t(1 zb!~_56AsdyKF{7h^z(2L*kjLA1*s{c2ar)ond9-A_ub}R;Mf}yNc1d z!l}`xAX?X|{T&}EWZ}gXGFHQFqFmNjnN0H+Geg+!It$7k zSL0&$1ZIpKm0UH*^tl0uBYd#Vgc4SVCA}KRJ;`!$taiRP2BuNpgoZ3nxG^667{QQ@ zSsFc)S`3?R@?0&Px-yqapRD#=A!QWS=fXSXm{?FX3uH1e+rHeRwtOcT9(r8D=&kDW zazR>Ki3A{&nGjjX_U%9M-AGf(*u{15#M47STADREizm40hOhUbAan1>lgym~bWG(8 z12)Ef`m{hinDa@6^M^oX-qhF#^yzdL{))>+vkoc|6d!GU27Ms05c{Tc&6yEltGUz( z>RaLgu$nWn|2_F|Qxb3-Hl?j}dg);zXWe9!4n%x;l<5{X)vnKp=HKZ9Fa8!Uh_gC< z05IhG#N;!_c$CH7^u{Vc^-Qfjn42VY zvh1H``azHx8F}Ct=RXa1vu?8mF{s(cbSE~QHh%T$lX5q^E+!q*wE7YM=X&KIZv3I= zw{B`~{z$EGxmR(>naSeU?`Bk#^aydf^jV*+_f$T`+r0gBRotDa&K*Fo$S?*_ zziXC8()fb(KxdV-lslsmyOtL#s=XW*O71QP`Ni@u3J26|N@Zl&3772GvuXxi);lE} z1_p;RpQ#$p!`ica6DF>83B7rVkaGoy;+LoMx&~%>n1vZy4c;ZgC+W}tT-R#_Gt2Qq z4zGB*da@psIM2NC)2=%Z!B+2hFv}~=kQihcFFKP7!!;z>-VDR=vEtgs5SGyA$oX3XTCud)Ae!FHp|8oL_EmN`eo2Z976RSWN0NP9qo$F0TFfkJoUZ{u~9~ z2!Q|v;VD;#cG}`J3OJ@kx2$du8j?aAgwfjy=ZQ4xQJiX7|zm3nssNM2<^Q&n8kNTCC2C%_QB^{3Z zG9moux4`&skCt2d2Kl7OJsj+v5tkhw#;6~oq}F(n{K8rhKlYb zAvq;MOF4#M!WSaUQ%cU_4p~pG0x6;M(@S|FTXe{=0%~#G-Z8a7WqeP(Wz#OnaZ60v z5lX^Qw8HLVs{5z*g!uV)INbPAxX}K42>s%jt2GN9CFDvb?3^TwV*Da{G2Eu=_E16C z4Yo{%g}ffU>y4O3MzqJ7rNRv3gqTOTk59{LG+eTZdt0(rOxT^&^y(DxuHB4bWLJxx zYrt3^u7@y1s!dTnSosvpmw_>k_}mV$;o?7<{OD7U@&2G^+ZrJB>CxZSs}Zad>>_GR}W z;=+CIWMQ|pZp3m>CbV=I%YwLFHW_eWXpdj_1a#t#nEu>Jz3#a7 zD~CahExvZDU2;^%d{@6@5$5p=rmqs3RvyB8CPfqb#8XR_>HaRLs75OdOVspY^mtW# z%Jl(OKA4iIN;^o8_N}y{ou81AFeqQUe3IEZTsDzS`{2$f2&>ghLuILLyQt@uW4WEI zKlvv!^v5A_-+Ky*ubU#1GJafU3h9|rBIMM=u9d7Cyr3xQLk&uvLCH|foyGd;*IAg` zZSLoKGKfUsRCc(<1!j~&CNt#}A}8pE4gM&*tz-{t;S{ls<2eCv8bjDuWQIFS9=WLb zBGsjw6}QUX@!js;fA>`k$}2YsFt+6nHqOm7mVw_7M)zMipj*WS@nCH?%k{Z^`lF3& zS@_z9MS#pb)Z2hyg(5;>KT?e664%iDbup`j@WT3UasJ8d_(wDI-Zo>$@--2H#!=eTnZ&&S^19rHh5 zA2d%M&rJ%FxM=S9&(Cc9QVS8_L+2dOUU*#$fIt!W_3+K!3Zpjo| zt?9Jbj!>_-nm`jlw=A;lk0ovWU=~Bn0cj9o?t94Mt{r^+F?oA*x7(RIH&qxIfi!M) z*V-i`X1J8L&6%UU?QmzJ_u1EoYX6BQXy)%Bs>VI@TXu9&wX;Z`e z+;whtf*xQ;DFZ@F*)et54}xOCmcXg&lR93v%JsdT)Q*aX)6F!7yk#;-$5@@BkG7*w z0Q?i*A*bUV%LH4`5V6qL9`%M(>^rnyu$n z5A*Zc7X3@M(2gad`PMts*K!$y2nTlcgm?1mIjOc10;7zV>-2dgxMFe@Mzxvr>fSTQ zYOtNR*Has6bA7(z8`ZMhd1&CG9?a*I82#>06vkAj9wvKk&;=v?azKfp+sD>KoIh9p z`EyHr%Li;lczPEYbYv<5TPnHloy|H^QLn-oPH?QI0A+NnNNxI+0=i6Oi||GDt;I%C z2lP2olwDN*1GlFBGM0{Huujlp-V)sl!St#TgsM{zTry&#gSYNHp!}K$xtVtqV7Q_5 zah$o}F{a;G^$>!r`&kL}V)cijeyupi=EZ^ZwzZWRP|drZgN`QLRbu_dHyl7BRGs`nkt zl!Pd#_wFG_{Q7O!Z~hz0T!vjHOad^QSq(Mg@0h}$+9J^5CbJJTw3%I-YmbR{=O5=U zIF&CbZVf%crMvcjg=#*&|ExJT6@1b9hY!PWA#3+wxXuMw!TawTP>Te+^XYcK*}-#l zgId6Eym4yYAHeHe>$`5(6HHPskv|NCkdhX>WB$*4M*bDmx`JYI>+AvEtbb96@ zR&y%kBEJZ4P(j#EQSW|?A5DHZFd%(M{QG=O&$S85mC7Q2#aV4kQjtPN_&w#hy{X97 z97dtInMS&3#T^n$*+v~-#*<6kPvDBC`QEqOvyyvk+RWi%4UP#%H-jA**~{=bzm)rU zt@%62_FYoaFqx*OZO2_*?L_HA4a8xv^ZpxN$=#7VSwdo+wk@_jkve?Y)cwe;HF%Iw=zK8mWxTSyKNc=y~Rzx`&5Sg~boU8zjR&CZ3P zmNZeA3rX}zPf`VOqeprwNR7?!8+*2YsUjKf3tK668O=CF)n;0QR~sdVFu^bs9|t5q|r;n)os~icYT9z)hJ?z2kF; z{03dld=nk;tTV;mV+*DxdSXE2_>YQjV72K%@ofLYTZ9h+Uk`HS@AG-N2$p8Y;V(Ic zznbv>{rUBi@G~IbZ*Sod`Hgn^tEnE}I+^ubT-=(Z$bvp;igK73`Wg)T^{=r_?{}d& zx0(A8Wa;o9;+6=ngp@tSPVcqWLhc~$cyfDqEj_T)nX?O9|&(`b7U85M=P^P(AtL@Jx5M7jKM*+>@gkDz`xyyv8zI8DCgpt-D8U$iK5Bw5aXh}u&Mt2C09J0wH_ z*Z=r(j#vYe#cY!)&e|m*xzJGm{_cU&2pf+URj|72m4h&8(hdprl2>Q%j*VjZjA2Yg z=z3-zmLfiKy~W~~i&N-;Qt>crjUDBXK{hkf;@GFW5z=d#@DF0b731_-Nu9rWdJsLX zIpYjx>l(V52px^V*LP7XE5=^uWzZn|7^Vh8qg!HP}!fq3FQ|N5E;_t?`Cg~{Bx+4HgD z&0m?(dR5Nw*^0><1wvVvymu_%b5F{+pP;p0gd6YK|M9wj3*;*Rt7}l~S`@VFtZWW& z4aYYQA+knkD}tj;=)U72;Ta!42XI;fd_84*a8Y~~0#P4GyBa#g1oUODu<645;hEWy z1^O6|4qUM5V|io;E{^UOK!MCZ6ssCH6dN240k@o2U?BMHnt=7Z03Q^$-6CqS%)V7X zg~yV4qkshk^Lcv1Tim(2OGZ-KxBC8h*Y8wqpJ#1l@a8^Yp~i$i_KN+g=KuQh{ZYXb zr24(!#=jb+w9zhd zg`LKsur+JB6M^#Q8QWvjsRsQcsvaLFe$+OkU3HgP_^w$|9%R^Q-@J|dX{+^-fe>PX zo>z!<&uHV_|BtRO4~Kg1-!CailI+<+p%O9nwX#)59^vAC#}+_4IElm5xl{fExtBwj}n^v65RH zoz6x!>2XgMMwvRlC8;iMT96uB z&T$9%F;M=Nc7W`+Z9d+T2Le*OOeHb?y7(eXicdPkAy-1z;0 z&fJ|@6d>RW`?Ms++q11^R^gxdK4jr(vDoSb?DHF+y&hL#v&%5v&F$kp(=}ap&a4G= z6OmmK^V<>DHI{)>ge;tp-RPui;2~9<Tzz^IQoe5bE{e_~t~n#{T+-z#|;c;iMrc-Q8+8Bz-1Q@?VfE*3#VmcL-T&=hfLG zKXCIu&57JrQ zfwD?SHf+%<O z-9BCic1qYr>F_2ehA|1e-!@_#x;iJ0VelI7d{433WF{wm#1X%B&^zp4UIbyAP+ch^5?|N(O zxyV2;ax2Lk)$q%7T_S0JJQQ&}Yvc78jQketiu0kJNic$Mnmo<3tvkm?G%+gqiZbX_ z0TFC+3VV5yu_xV12Qx3D6!ozINi;QvE2#4Vk=Q#|3A0UeP*LK0%zIE(@AqpeNk(dK zR8S5yk}N*J-Bz<|7*E^0O7p4mE8J~(y;?T5oj=afy!nW_GP1LJ1M$?4ri1$QJh0O@zO zM-W(#Uv=28J^@Vk3g#}r78HUPrXb@7Ke*_|B(NY%Vf!f;yd_f8V8{|McT#Z~W%45CGVJ$6?Q@@7ERGBzj*HXVII7T?s zgp@{L6nG*55RvZI-!EBzV&)0nKWwx2BVn`1hwXF|kEXxB&g`m6w;L)0DNl+2A_=;> z&km;WU%q@<;&yEPrNsWB4#2lu2|%imw9=~-{w??war7T!+d10~42hrK{XrcOgY0?n zJKeMT5#Y1x=UVnY33~0(n%wItz<*`pp-1-#I&r=jv{3n`9bMIjXR+Fm;pe0HGX^LCy>*n$NPMh2BIhA)|M3G+2ZN55%lO@3W_XkixD-lkFStKAVr%_)oS zbc|qRVm5pcJF|t^jL06oTM?u5rFIw|jo`%}Ri7xX#EQv*aXPo4&=BF5>RLpt?N!qD z`77&??&5q`L1f9zg%j*0%8&42$bjbM8g1C8r@gkL2yev{&8G=k^#$25tVTVS@EWW* zrO@Ep;Ag;`U2e649hiTK@b++9Q?#76Ck!mRnwKaNix+R38r10O6Po(kpG>YQPt!^b zCh%;cY=bcNtHobGTsC?rl@wqsrxFLQz}gP@G`?T1q>OD8a&~GE-hEb^Athh{S=I+t z@4j9zh@LH+fT%%o@qNhx&vI=(qo5A2?@N=M{G3YJ5$uoe8OgP(y&%ggd#(F_CUfow zwL@B;Bs|WV!XD*6fX#D${kzcwk#8Tszqw3=a5ulV1wQnD!%_}hM+Mp)F#N_a$-^V> zz5!(CNceZ@^FG&W-y?$qX@_dR-aTZH3pPlp1vo@x=pQ+$>k&K;Mi2JM8V6pW2QLms zKqgFJ!zTPIh(ncCzc=E(H$q8o-cI+6Yi`<#Y4)cPfegRO091~vrD)|+^-=`oj=v88 zo33f^aFmWR^uWLs30T=105iOqgLB}CdAFtzY91+)|Q#aFQf^E5mqrA|AEOrh%(bX;E&`|ZZOaI&6CJ5 zpj-MI>%cb*!FyGvRsQ-N-|HI}UCPBW04IGyo+ESq)qQ`+t()xq_rk*xpAB~*qjW)Y z;}2V$u&wTItyy7pj%?cb$)?u;@ICG(e$MA+!$8QDmjhX-0uNQtnbcfo`|#2_QM}EG zlUsQGlbMXZX4gj(pWuLh@CmQunEyQ!=fq6-ez`4GtNT^>(aqF4V$l*&vhvB$+BK5k;lx7>zc5RpXG9?{GRdP4osaiTb zNlH46r?BS>z}MZs$V+^-c0h<;?MqCC z$t*K+AM~n^4P2I4S?=l#2+&B)sz#Kk_1}9RR6I<64^vS-xaTb2M}~Wv_)^B-V~x}x zZ3Qt0Z;ACvdfh!30soL{>H>F_yEe`QOpko0fnDY`u1AV))SirLNj#J=kLe@<)b$t= z?>;=mZvK#>pRjU+KKEbbZ~E)b)8flYhZfp@fYsq+kC8*ZQK);H3^)~s!SM4dROBMS zC1<6a2@~uI*Oe;le*7Zz^^t>@F;k`Ftll;?t^|GH8t^$$s}1)B9ie)dF}%n0MR)Fn zDZG5J{Ss@@0J6;K)12Xla0=T4wk*^hOgk@=@v-F%<+w)|DSn@$dX+_O8;HE)A~%EW zViUjK7h8gO#p}l+)gmK0B`H(yj3t0%>T_NJtI1xMuEc`r9g&t>K}0FkibyBG=w}i= zMsZ*-Fe*L|>%FqZ<5FtttXcN-)iF~q$B{DI;?UF7Z$e{@P4SwYjck)Bn`(1GQ0B=h z)U<~5x3${QO8NNo1G_G%L91s3rB{Q;)VuLRtBDrSNpb_yd%=UWP^)!9Y3xq;J!O~X zC~o^bc2WYR3G>$ZrN-c7PC)hhYc)10R%V@8jR|eXW~c9=v@?{Cma&i;3JNAqxD#yr zVFd6?@9vSU>ig0WV>5%M1qHRkqu}OcjA~+So1J>%2g{BR6AUPnFz&XUmJ9+iWZ*VrW=2Zcc%)Oqkg>hhW3SXa+U{8q>19j?STn-X|tBr zDZ6$efxcq!?XCdZq|S_mEA?Tvh9C%3&=VfwYYiEi1?$f26Y1PI&-wnP5L~R0L*w+e zGbwuVa!2BAj8?djfL(T@UR1Z&veqtI+_wEWpQ?2{Pr9@o7(3sxLdqNqD`M1+bJQ<8 zN7Gzni7p&#v6V|0^lrm(#w9YoQ+h9^8h8AZBLd8MwVnm1E?oyl8yjmayYM~sGL^3C zbvijivgItB>8!X=54lo<+<>p}Ty2a-JW4Q?yVzFO*>H12-62`GezCfnD^6O~&)C*_ z)h{#P=EWCXawMmC_F+ZpYTw&;%js-6&sSVnSKlDkLISO#pO%5xdnwi3T3XqDUmCX*h0K9gsq%$T^!h*!o69qdPY2jZ zyvn|LVZ1}HjMuVK3t%KqyaC&c>BvL{R0j^AVN`}&T;55{iwEJ zh}k0mcMu}xKQqJ){Ug9+KuI0oKDq)MG5V3)&SE>sJ=xqOE=}8pm8{%R$BZPR~56{k6{GQOVyI!f) zb)4;}=S5XrNyL!`nY*T}K@;YpxBM^)==5Z(ZZ#}eomG+6xa%UTu-=g2Vqv)d*uB52 zZxoWtJ!qkvbDDEiMiAS22cxq+yC_OINUEE_*@P0}sZ5i{YE8a3MzUutWD&3*C}olc zt6mGP7NF2Zyivs>+>D_cG*Jx{oGF|bT%PV@W_=2b7u7d+hvmaaBQ{NytC6$3@ibP< zH*OV{72`N!xPAl7DQgn5+>Y z(dpH~p~Jfn-)t=PD}%7nDY#Ehx=q%2lK$K)o(WGY{^>aUJB0wox74BDvl4z65dEK( z4j^sip%vFsDmVWa*L}@xUi=eL{TG*W z!K^gEyG`mnEBx9X>O=1>(F^h61xD=owGV+|Pw0cA_OgNvTnpg=E$d)ZXq2V=d#A}* zlUGUNd@JZMH^VnV7lCi(-6>1;V|Y|oUI zazq^TNEFnHTN%che9t1dM(}jMSa1Uy%r8;bQH?43op^4p<`g{l*Q4qhGHiW;yzz0^ zD!$TD6Kb<}cjDMPi0ZV&aM7$(AEh{(e!#8Zlk4+j1E`2+w{^iU zyDF6NsQmJ*)Jj_H8|Pp3uYXyuhxT&eIpFzbaj}b+exoG1`bXlFM`Od9jSLTtl6sP~ z`+vS)jqZh=#mBTYC8#~6Xk7j7MNSB-(WQ^$MlD^Q8K?Gln=G5=c#u5sNE7Xa%=K_J zDRYxfFD_}mm};{x&d7RkKkLD)8ItfE$D6|SmfFIT*OaW+t!OSLgJP%L_IQ#k(5VyL zk~PIz8yIYsJ&pzvyA{;@yt8_cJJXCItxmqKb-G;HLUP~orkx)lyv5sRuVdb}BT8$aI2d%M##3@$wN9YJ8UDQ} z=AK<)B2S$+cr568{A4!>bwQT3UEQvW&*D*2iH%@%pw+Mx%3{Mx>04z6#7y1S=TT>f zy}Ym*?k@in!Pfp__;x>@Ms$f-J7pCtmxHYXhKr|~q3SudAv@wS~Ssa70x?!i;OY>1E`-!bhR z$F%!#tOJ8YOL(jLEz1P1wCvIkofX9VQG)@7+Gkx5oEp9G z=0RnXIM;IZSBMV1KvB%&6!lUAmiZ<}vZXY0DPe5(8CjW4Lj{FKe*i(2)O9Ii(~C+F zoc_RN;V?$L{A#^va{=u{@zrIa&pulCvmx%~XTBhLGVC1qu8Q)Hsfu_p)W61O@!WXa z+^XI|lbUvSah>vo*@?=|J(mkB%a|tAb_s_du@xk;qNCP1Pc(+faHW2HC6+H67D+vm z5H{dk=b+y0LoVD=z)g325mN@wLzBkiPRC#OW#i;I=*1pE7p8gnx-Cv?4D2}uCvEDe z*>NQ$Eu9`5zWYjZP~h9Ps(mZuo~p=k=|n?a>akbqz-L#{_XjIIacFA(FPMmL8Aw;C z=2dW;tk=`w=a>Gex_?tC9@B?W)u63M{!O@siU>c8-*M?0jP#%>*>=fI!2i~C+LjO{t^u{i*qybP35n}btJi0 zP75k8s*7=%=nA`HBcerk*+$i}sG=3Ci%*VcjtUYA)T3eUZ#y#nv9B3R(E!3Ow%_o)$ZC z#l_-fr#h%LHDj}sBAtSouK%;S=cKcJIaQ?@{-*P-Kux6HvJj{?^}M6^Za_%W|-!{wD!Farhq3#hv4Rj~0E# zSp_kxrl0_nbr*%}yw50fha~_n+mlxH14UYHGD^fCOgE}yJd(6S3T9#j;FWQG(Kaa& z6{39UClBtYuG)ee=2`PkYuvA~gnwocswl}ChRcB-&{=?i%>}V>w$)FY<2wI>hG6$Y zgl)dkR!Q|=q{d7Wu+2!iu2C6X;-h6AMuMQi{Yg3!m|%^ zcQ4w-$KK9AQ=Qa_ED6LW_| z!aTxU8BbUB{apDJ+nKeh7Bmh;?KcuEf*QUBT;Ezjs6QFBP@B}Iny=DqQu z&A5U|?hU^~(^$0hN^Hp19BEX2ZJ&h?RPM1VvwZ(MFn5sLoYg$O1?WJ zqF`5p#9l1m7phGrAmXQaw*oSjQ(!C&OG+WliQYy5-h)K&`b{9t~vl zY5>gik&)Z1PtA#yMD>!(k2g>CewVLKnrge=x|r4BRb zJ1Y;XhJy)6Qrf^s7ry0={wlaMLSv$)_v9kjUMqFD5!ckoNx}wm&Ut-oLtK5cspEc#^>dzC2uLE56TXK%fENLxH8s*!4c?&mn?_vMpo}YzUtYw6A;y=$7 zzyGb&^h1VZNCIReL(*OC-fvcimQCUjWde$aAv$q0W<18m2XNR9C3M8a79IYay~N`= zY^~g%!L1}aU?X*-aBN}6sCu{Fg(WYYGRuvuxFkmJ^*xS3tBc;#x)KwN=}J}-${YUo zng*u3mc#vh?^O%P56!xZlC*}0@xatJ;0LxXji8<5zI!>9vclC&z`(3Zv3i%cK(O^i zPOL3~YUioRn#W_>uRKJGPD>S#MMxVecrEh3`^-=XI{=RfHOw;b{^aW3wU za0!Wa;v@vuh1j^tFma9R@@=o2?R39XjZy_akeK8xHO?XU$D>bcASt&6lCU*bE7K8% z+Ku9mn5ZBEq9>F2`Q)0UZOZVCPWe7Thu$uo{p}&(Hcs%Cg*3zjGrqgh+m!fKHEp73 z(Qt$+s?lz<;(3OXERIn^$FH2a_BE8%bX8a0kjF>0)Z^PfbsLX-`q8|cy}uG4WM&G8 z{-#5I;!UK`VL=P9R)5Pd7wD+)Bd6lW^1_laUuDBhJw_VnI(P>+_RM&H(W6_>209q7vAY3NEv};y_HpF(75{$LdlScz1pdB{C zHnHIz{?^phW$~Su@W#xu6SdiEa_N2-k5tf5R3(Q*=@TGIz{3?Y?^J}Ep7L=_Z!y(f z0U7mHdQsk*O|S00m(eqG#_6$#Z8$kOlbfxZ&@;)A=l!=UEF8o4JJ`RXMFeARGwtKJ)BfA8Ug6a4BF9xBDrba3Qs^1!k{ zdZyfNEa-z}tn=`(@-lzyfZpx$%6lY*EF*Q}_4HB- zdm^a~(VZcwh;FH3_u!W!ry^B7-7{Bu{qM1dIjts^v7R@-hf#Lfnd39YjQ1Q-BEHm^ z7AqsZ_xiRo{RP6K-an60NxGy;8by_>wee`5kO;;UQq+sBr>Q1^_8$$3)hVOCv=8rz z+JEZs8-4zCC}+EMY>c?2a^e*=0ms56IcIa4g|1+)XAwAm8Y^SmbhpsL{TJ6!+OC8sJ*=h(%%C*y1XW&Q8PLi{g3q$Nza)NUvd-cH%> z`o6?lA#zhN+EpyuV_($ySi_Z9ncQsNZT)cW%2Vpoq+9iv7jMre;D!cfx}H)pZB+Y? z1zBBsND7D7EW$?>>N=4w{%1l9qk9txKWL=XuB>;y4 zgSdf|*c~qH(URU#Z|EPCZAnPDOY=r0>Cfi+j(BQZa#nxYLY4|oK|Tk~1MOD(u47{j zL2-i$b$*^GY~rBPE^72%`0F64;Hx`Mldgr1#LTrR{IkED(<4FE0W~jb&jo+?+9F+D z6dP0)F7vq5P;y$koF*(HxX|84$P-Yts_z)cVr6uAkG{VY|CeLB82)iGOX@LsF$Mn8 zO_N(L84%GwNr5NTCu|`&FH*kNRKZ`5rZs(6)4MJ&Pke5QDT5pf6%QT-F3gN%R#8rh z(-WUsZJktDA(>(O(RwvVJt#-_IC$1;d6b++ohmW2b6@b8+(U@VP^r08UkKkcTba|P9#eF2l1t?1qlY- zBgWwO*;0>Ap5iccQQ~M`=ic{Yp?YWNDUMEQu+~Dd*U&xm$7?v#iRtwP?ju=IT9zvc z97?>8I9KaljH*9^H+MgADm%F0)uul2h(+3dT*&joU9I#Oq%4#s*%Do-s`2gA>B zC8L}6#xNnDP-?l{sN}TX)Pp%KaK>aYGeXGHz45YEPhvB%}{PiE4;w7*qYP>&6gNn&mm1$D0p ze4N7WMN^Ncs_4aXmiy?XlC7T0HQR2nZZ5*t`@E$xhunOvX364|8xDf2j7F=on?aaO zS}sIUz7|25K)-OZCoPgCO5XLoD;6lInXTo(I!*=iz|^Hj^D+6mV{e1Y)%>1HEq;{* zu4(iT7&NOU9JFHNt2H)+cznDke-v*A&;8`G`pLRYgv3=5!R;_l0n6H__Rx8_SQn0P zi&gQ&VT_~#V8bn=>HM?B3h-05-dO;nIWxt;*3*^S6~X^k1u@+{Ea%IXn!o-nPAo10 zNyziC#R^F9Mmzx)7tff-{CdVifC(*p2l{M%2?u-4kkhQYowqvmOH<%~h-E(rakyBK z8Qta4N;~PRa;CYo$8_1FkI6LpD@Rdqvxu<2R5{J{% z?DFg7VE#)xLf@l0!yN3cTe9(r`mczrtK*NR<&W~M)CMyz>Wb#+7q29w#MmXz~c8Q!_Cu!qGx$L@nn< z^)JrdO^<*$Q?Ia%A6%ssY)_6%Z~fB?z}9K=G;4l3M%2qi_?$F?n3Q<=ukI>PObegA zWN-<}T20Cxa!D2C3)az5N!ogICo(;?to2XoN^e%$FsT^EMLnxwJqAr`;@Jx_{Ax3T zTHDKG@P_4#AIqhVb2l6q%ujE+u4qpAYscl}wF~r$V%RF`bPhF)q;ROw;|Aa!GzlQX zXcwL-`c}aRI4bE#;G#v9yMbB14jB+W`;EujRg6}VDYAi&tP38?zh=et)C>HXAC z#9)j&r{(o8MrUwe``hDer_-+M90geKj#@p>j?kXrO91O>y~}vwZ*XxaoB(VZOmQ94Vo?sN83Z zoXHf*0rlHm^9J<54=lnTCjCOM>!Em{5YXIY30>etO4nR(O=9|}CvG3o?ao75{KWI;L#l4} zXHCX#WpWhyZO%F2?~|j7BIoi^iX+DzB4!Jo3tANiI9uorrf0q2jbW z5b$B(3R;||FgZ1!W!AjWZw#cIsE%m|#|JEsudln>f0>;W^CUy`yZWCVA$5*UAjQ_B zw{)}jlFPDbCWMjW6PP5N#$(Q5CaO5`&Tx&P2ASrGv=d2*k|xEaBCWjG-uO9`OMi`P z?6F;O(5c_B;Ooc5=87bdk5O45N=5H`?jO^hHWOUhk)w`FxHsSmxj%1>m*Q+OdQK(F z$C501cJ{wmr8|3%Ohjl`r1;?F>>{X;`QYMpqaioHN|tK3|GA#?Lw>U{3#;T?>IFVXIt@w8~j#0@nh{ocGlWF^*nL&fF|ln!`%? z#_@jL+GP986(UM`b4NgDkBLfR#1)>9F1dm_gHE`xzvbR0&(GuYu(#-NFeDz{l7HO_ zxOmDJNK2{tI?j#O^oPsg+z$t+Yh?itdhPMe_TSj>(ckFcwHLANJ>xI3{2!HIiNhM8 z@O8@{jduT**lRW?|K{ob+Mhy7^OLtW7 z8l!E>tyaN$rr_8bya7p8o^8d?!Tg=* zu9hr2oZgrpKC9OXzt=i4TiH~1pxfAuj9buA6C&{UyesHlUU{}o70B9@ML$wi(ZZJXwRi(Zw#x+ zv<_G>cV)hrA6mV_hjxk<&@J*mn^CSG-Or3jN@BZt?uWnspPd8%f&SkuiWYo_T_^?f z2QP2{q2W7c4*!+wI_S4jtk6S!=ptftyX9Fq4s)FZkGYlTqsOCN4L71~+wPtwv`KRn zt?wGLcj+)CdML3uPXu0_L&DTqS9P+nmh{LTjq_etYQU&SqI!sAj8hKn%4=3^4_|9> z|6+yhiO{y`rF1-K-3IV>y~uS4j~J~_$#A{l9)C6YDtNlzhnd?ok6@ZX_Qih=TRYUW z?J;QYh{yVMvFd&igm2`eqV5j?`|PMgTr9uCfiZq;yS~X z@@Y$i{cO{pJewX8*=n-UwIG7O4~^j6jNI^OaCkcM_;cdzJi%cB{E_ZfD$0!o{yi0c zUGB+T&Es8H^~7~{OntOUS505-oxP_Rzu0VP6nWvUYY4Eq1(wW0`YL(yZgI?d)Qc@eh4bC@Par0t+M|(asZxITbEXyKWxF?=Y$21jycNy zgH(^`O<<*YrYn&eG<}dm&g^pQeJld1{dRg9o8_&!Rz{*`Nhe?)8~Yx zAz~RTS~HfTEJA!8kW%rN%y+W4OFu+2iLZ{k?Mb}S_PmzP^l*_Oc?r8X2zTDwcWY|e zmk=vZ+G!NOzEQgk`Ha_fzJ+3-dD)4#u4_(9j%=Kd4<9g6cd(@tI-uMic2}+|8=t=3 z5WKD=&&3;ti=H`!6}^D&S>)n5x~@=ZFmTyJpUvx7{83&v{{!2F(^)LIYsJnlyM1Az z?h~h?EPojBBML%`mN{WLKG-M2yYy~cX?T5}5 z@pjM3Y563r3RzSKPXkH5CIlSv9cI5n?rfS|9iwI865$|Xpj-8fN;TY5yA!nB*kA>Q z0sUP$TT0C!Q#dAj3JXpRVHGSSTyfu>?y-|5B($V6UoNo*SVg^W(YV>>4-c5{_%BXe z0mOshs~FAz&~4R2clQ?WSO4OB+E7@XlQ5UDpQ2rL$lswSA~@i1OrRp$IOALt^0_$Z(D2RliU zp{}N^_4!vJj7om3#DSrIyiZJns2p&!M@l-&R*-3jWn9Fb&^gl*k7_*hpl2TATwPnn z^Jhn0x-R6EA^eMUJOy7!$rivjX0xSvWzCy9R?Xt>EsvDgmo+q#7994t5Op8bH0?I+aQ2im)(q4F2tspvcAm$$YiW|a^#Vfrd#QII}YlS3;9}MT52PW*< zep(QDz$$AT^^}eHL1YiYKnEzFnqW{A(r52ds4k~tBhcOk z>=*#x2q-*(?1uA1`cYR+iuoR$a%#jsd$tKj0XLui*0w}rwP_ztN=n*0G4#uI`7Ka9 z&K$-MAN`#Nzb2&qNp1emj~v)m@%W?)MK*7F?J;QB$Nz}P7HL-hEW_@Fgl=1Bc5@$L zS@}~oe-D~o{p%jDR9&#q+Q7o!vjY(y9(<0xz2&@6QYv#aqtW6lVbWMDmVe^*5YDro zeonqwQc5Q2gk3@|Bb&pn1x@9Yub8bN<_@R3{sKnn%Yr^hxAkiD8)^4^chdxV%3$^T z^cOX3cID4FwUK5pUr~OHhzw$uuVvFOphc;^O$}t`vB(xC&?LI>8DY2808=w22wXjo z0Ga1K&GI^Nd|6`AKJ10KpNOX%@3qmOg7J2x(aI`)`43AwmloM5y&Pd2g)r;-DbnU# zHVxxVA7`6gV+2!SOY>ViJ+^~@%Im*QaNIe||13LjVYG;%8jPuM2fLx}^vQ<2&`77) zJ-@AJ-4w!=b3!ly9KlzvsI!;-2x@o#Spe@8mR3(*3`A9GZD;Y^Q3TT(2%9~7XHQ(_ zRzZ6C#kVSHHD=+5JB7z{##&BMWqGk0_=XP@h6WazRN6(oZB*|aJMqs&D&x@iFp+{3 z7i8=XH?GC5Iqo}lk#8^de&(Cqc9D%f$;See8TkwfEC2kFV?q~%Jr4V*nO=`Adv;nC z{I|Ua6vBZw1IIUw{7#4QFOEdN$T`T1M4}SAl`M*LC4T8GFdY`Oyfw>EeGMq%!&A#- z99qoYc!X>|8iZxsRwW{R{%UnOEzgu1)ni)o1YCb)cz-2ZozvVwRYYjqPR%GNTYBt5 z`=Ys;*tr^G(Q_Dod#fZLTLGiLaGo4Nl(rAw=o9(_qJ&2!L+)R&hHr?3!t8)m4`xAA z=pzw>hU&()>v_h$9Ok;)<<3ni^_E&W63ShZBt52;aXoQTWkP@j5GtF#%;&9Zw&V&* z@;H^G0DY~npzLxc*-?S5XUD7FqMA^y{w0@xZ2G}4AGHV_E4;8kj*mTkg|z_}z)o-y zTt+L_VWkgNu$dpzaRbh6e6wn&NS&%x)WHnfDQ^w1-&g1*x$*NIti%p)GGh=#RW^jT9X=7W z^nRJu5UJHQmGv^ST}-Xm7Mba@9mM+e90n10E4Pw2^U|2CSpP!rw$nK_q_eO75@4W_ z-LSFqv9xrgV4Y`qFqmX8wqSPL%WlV|Sq$`H*-2eRV;0tnWdXK8v<%Aun9zM2Fy$iQ zK;imKyJ1dL%c)QRSX>NL*E{;I)fxYjcxL&|)&!6jeHXgoprB;KZl#lHx&Mrw{To<3 z&K?@UCv}*G|4VzD)32qVq^z475IhV`byx%EtiYeJ^6X@WJmBI)QF=VolT!yfogb`^ zJhV#*eQY=xJKG%EKbt+v6kzf?iiQj4L~OlxYPNM#pzGjjIGET-TMVw?Lw5 zkykp3GLDJc__QBMk8E@=xpUBJ-m+7@o=+>Nc^qdKAi!9>H-3A!zd^L|5}2Ro;+vkp zWZrVBXWcys(C*fV`zkSL(Tg*t-m|56QM@J?D+rCJ-WF6Cm&%?j(c@Kg2PJyTbw$7V z_;5|5Xil7(yZL-DE>>Q&77t#lQ#H<9Qk5}pDN|fDn_1s&*?T-(2>!a`*hpD{e>ht# z6f^Ud;vu-6hAI=QZGV2uv3zK?%+O?wR;FLC`ehUX^{z_Lsbw38qslBSoKP&lCoi2| zQv0~y9L-A!jAcbVniCvewL=yJ;-fvaUn`)Kp}pZM)=7w((NSMlAN?N8!_RzuP9s=F z`RX2Vr>DkM(xt&URqK!B?wXcjQOzc>E|kmHn9O5&kvT zBV2k52%W1H-bKGSONJnw%aG^Or%!`LOAx=cR{s>&EdoGmZW)8>)Ro`E0}$4;Qjm1R zNOnF_|wHTqp zBVPiHKnz}DdWllZ;a2QN-M*f|_$SCy8gKY2#8LnHW325|@|G@nPPJ6gm>e}a#_h`z zT~Z)jx<579O@4olwt(6*8{9NhHB}t&q(X(;>CjivQjV@5Xk73AXW=c-K8b!YYRYy^tlqjrRt{*9hlZK~QrtlQ5C?^ySSSLSR+- zMH0x;;_UH$Oy6fv#d=R{oXcs2MbmHTE?PAN9yys!vu{&|tGW&_BYpc5k3R3MN7;qb??fVeW(#P$?74Zmzf@Ms2ewh5#nvHuzU1yjL{W4^zNA&w zsk1rm-zX3tuvV^c;%Gmv*R}$EJrtM5_)G;v$M>G?%^QzOwimXOyF8Mk>8FTw+umjw zjNN;@ZDMpCw@;fhmK7*zx4{-4!Ze}DyuN}=D(eBsW5;l*Om>E)jtdi4&ozHf3&^bR z@U$-TPsT}356ri!mdfv8)pa1@;KuF}d)6vmKaXqc4IZ)L&AU5r@mh0Kf+eF-{ig4& zDWFuh*G|l&9AG4DY+u>}Q$M<8~NkH7RHz z-qf$?%aRar;t!*h-2k%$t8Y{OZl#c&rdA!b#Pz=ha%{WT=d->5v*3{gc#YnGYASE~ zMQ;L}=$FHz;r^Jk{5=}_4u$McAAc~|kPYsm)|<6|;agVvjJPq2A8jskQ})Uf$L%!^ zqm1Q8Uiry!J?gfbnS*g*B`9ViEVU-4K?PoShmWnHt{7y4m=^<5W~#E5&_ti-ka&3S z-70H+Qs;Oqc-pqoKedrC5#=G$Y(!0nY2tGawRj-1z6;-eHRY8~{dxy(De zGduN%AOSSM9EQsE|8J<=oLwA9pNQjCAl{ST`BX$MJ2&v>KRM_>O4orLSl%3J|x zLa?A65XTH*Z=M`EP+AsBEIT(HsWjlckd?Tp%#MRjm#(GZ%sXZJ;_ejLK{J~*q%Y-l zrB2U$-H|0)SWkmISUkgcOAv)Phi&<>g8)IFGu(dArgT(kS*Kki;$~P0L`J702CrPC zQpnr?O&qWR)&;slkmrwRJ=aNAy2}wV$uPC`X2B34_Ym4Xq<29^C7+VlJGOduuN~RG z7xIqf9%Wo2Qqej{Akpv?71qqj49nHwRY%^jZZWl^ZCAL#3YXALwx8HDkJs~IpI%wM zleB3=T5B2wbf%XUCt`hc$gW^w;wl)mG#Q4@Zrn#TZY_foT&t0R%fw-TqsU5GiU88* z^pBYi@t-pt(I*Sf`mdCASLq7EyuWf>0j;)@L)w7 zNLbLh&jbIaEI&juOZ1WGL=mxlBAG$XUCrcIz!AAZ8>vl};jzhUZnl5fT;H0gGpsO% zJ%ZVxTW*;~^20Th&a#zd%y>DX&SdmItXb_k>su0Y#x+y?Et=jh@xHH-))Q>zNqv#l z#u6Nf6#mXvE$y1eNB%#z-%Z4Jy=nP5ZR1i5skE^%RC7yjH!eZ>mf1WsUG*dlj?4og&9>Ni}lo0t;edWCoHjT8QqP3<+{QLOF`1`5Tp9fXb}K`pD+t@NwB; zJIiC?J>Z@~JLs2sE5FoMK=nkmL;;ybC)N9(?x>8bbSaXSbqdyd69cuUGDZItMgJN^ z_Be6~R!6j$|GSd3L%))Oy7$7Rq$jSKwz~l-6#F+_DNQkCIMw5l=4Mm66E`+^PAvN# zSAq6ld$&gs@P|_k!sk9foGX+{PD+XlW|u&Ptu~y+PB57D*)57DKAY)k2{5{4*3^8C zhJr`q?{R4&nPMVPmA>=AoIZZ`oN&wcp5UX|F>)qP#m{}1P^2`)_`TmAjpIO^{CiX} z5TkWFE*983hO{Z}!FE&`HST~)Lu}k_7Zd$CyxNO(*^&w)y|=;01?8Z)9-;pC5_psV z58T19!g<~6Ge?MMIy`mmM3W-KbRz@t%wN)|>+XI=R6W(d*&%JgW$TO2ypFJ=_eY-n zS-y`S9c8#=Ilk|Cz9XS-kK7m6Ij8Hht>S8uuX#_i+;@V_WEh*l2>E2UZRxx53 zuXNaI*bF%!iRQ!?JO1c_kUlhP=d$;=S@w|a%(;m`fytciP!sun?MtR8TmCyn)K5+} z1bEeZE_5~aW@Th#B*2|t{};L(e(Uig@twWR44~P+UVQlT9r};|kFM_kYBKM>Rzwj2 zS(@}>(7+1PLWclKRVlL4BAo<+6e%(ECLq0tpc1Nrt4N1P??^8RNbd+jXrYKCbiapP z^?i5W??01q9`Ym^XL57zx#xH8xe4Eg3rs!f6gG$a1NI@qOg{+X*<*%A|BWwjUSASt ze~sokC}+~}P1u~0Um;B_q)8|`E!)L6^ZY2Nsc;Jubsa{zagL_LOa29L@)+a=N+LM= z#R{(FFGi9z?cP8=_I_#s=rNn664V{qYG4rZ(ue&?g?qumWVy_1s*GGAtx-Ix2j3%u zWFfFdt~Ip}N}1PQLQZcOh$$3o+~RS~pNwhw9@uZONU~_# zd9oeht`x06NK@{v9D^q84i|3WU3t=%V~ZephF=ybcJn^8)D?pI^@q=J=nUC5eaeOSufjj>IrUV7IRxACH8V`mNH+9>y%sC5;&!wB(u%! z)?R=4mb@(w;u)xu?yBo3Mtq8{^qBn6=DTr)N&<3j-C5tBqP~`#2in1f7ztHIf`?u3o|g0@`)od)ra@nR5Y&3sj_w-@W0_u+tSsiot%s zT;g;}x?`yl1{(N>5HQ3wUG6LccqG+q)sv^{ZqIbxKdX`n;9M@I69PtLe8Y>N=c0TR zm#H)v#BqyKjqmE_C{X+2mG*Pa|ANB+V%k->uecSQ@!E-VcIDTr^wIg`Rvf?>Fi;3<@~5HR~a6hrN^vQ(>Jk!ZhFWPuVs}!#JFg1 zMWZ*g#@^nVY;gC09#D<;;PADkL(sW&>e0co3(isnEcUVb=vox_Xo$B zh~WfwO#{czUOKd`AUIR>%Q|7CCrqIbS*qu)`Ck-*<_#xo#75-KF9+8#r97q6#C5g)Qx(ykG=dm9N z1s$6d?7B8D7jCspx5eXP6SwPnsmNYMA0NzoE3>U@vFWd?-d|qF$AtI2dc(m#z6zMc z5!V~Gx^l9!i||06h>JOCpEzAHXs#FUG?*So1Zm4BvHtjiUANB90{ahJ$%YaUW3pLAZkLi_Xqg7>BQ=^<-mDvAj{=UkliVa zNk*#|n562qZGhNlgh|$bxF*S*&CVD5WgM`4B1{Q7EncLU+uyj|TtAx&*b|lFpK+iX zV;R?~wS~Lj#q1%3*dlCHV2k5RDFaj~-Y7IWp{iw2_6n2F{&wDB<0Oidflr7GGS6hs zc+@R`23fIC1r7rbLvSm&=N-I~!TOsFzO7PlP4?o9i)%zFhCBNgyY zyj?z9p@E1=E0ksA%;iz|Oj-uF*Hnk|2sXTEO2;GSFYZ4Bvd z?4;l&Z?A`|`(RXjOeq*0KF?RNG`%tz0=^tz6hqn?f=pi#IdPhfR zlChcT^2Y}$)S&Jj6Z?h9+UB772o1~1b`RQIR3DIWfX#KpHqqm`sdvk%u5~>+ z;{?k14ff`Lv;Z;=^4L-oIQKrHHmh=9Smy&W&eCGwbf7`(uhQgdbpDlo_;K%IzMA^> zzq%~oPg&=a&vTypCE@zno!_8N)(>a*7!{J3YsB}y`B>Qh-+{x99tG*!$Gp8D_$npY zS#73Ek;M$O7AY%2qkE;2>?qJIdg*8!`cOXDW?hSRjQWi;PkH(6Z$#9B3wwP1=c%Hd zD($oL3^kFB4cE?JQr|C21VHb=cK>k$BVI2G%h-k_8wY>Xjfv1WEnUAr$3f|uCUNF0 z^LbMx%|+JqJ&LG|8W%fFoRC|d!zDW2wQ3Q1i4rKMxxw7dlIe1ZhD5&yb491WA$LiM z?vR2<=Rl4N{JUF|4|qJzXXpwNjK(^U5_T>I4!0EZ-KUT1Zj+g|oJ_N^ImbeDv8VlP zvv+MI+oY&1lkgayx^@FsW*Sdeb0})v)w3bYW1F5tmm9}iDz-FeqdD%usXS&!gU`c+ zs~So!-xw4HA7aO0pX2I6;ZQjU5fYwDs*x(b!9_CRtWZ$i#I(ad#RAhR-g=#LFW5B^ z_=$yDsWC_IjYfY4Jl^p5v&AYBp}1-g2Hu6)21!Gs4JU=LaXq7vBj!hnpVFxXkd$fR znd}0}E?ZcC(bUQ4?XLYo%YmIB{{RoG#EZT!GR>=fnky-N3)OfNV@*eZFO!QCs8Nc$ z5rJR$dA(mH?sj}9Q_+9h^VwZy@YcJEp{yx;;()j%2h zt4#-t_YRLJ0ZCO(pM&^65YJyBMsq?0p1Sk$UnF4~l)~BGS$tOT{&!}6b_erGIr!rM~jTf%A*m%%gm_D>%)zfRU(ig!g zLlQR~jVM3K8Y5=mC1PT8nbcQ6Jkn8HncW@hTJrG)_`>aE?`xkrLL;`(QlT*EJKZMDmjnOiorr zpkNqHsj!W@#4XTvf-a>p!`B*ExOuk*G|d;bmc;`bx@XibY){xXfvOtpC|IvowP-L* z(o~mtnLnJgntw#rD6_9?_qDl{h?@l2grG2KC7V6%V7_{3U)-a>2#2$=)ZqNA`sfXv z{di0K0e|7h$aA84<;cE&z;yms*>R!VW1d#I@on&1Q2j&1!`iq0iN4GkW#m|H{O=R1 zest{Z&mqbqJnMV?_xaoO=@)CGJc)$k9@x&}l5nux(=UJsOJdW<5)ecDW#W;XT~rp` zh@wt@(8E8oQ=ML%tlyp;uT(uy_bZIlW-F;@8%Ee70VZeuT-Hy|p#Rcpaa}(FQ_f3e zu0IqLeHR?Bc-e_C}8#;jjIeE8cHuA-bB{F5`aBW5H(^uFGZUAIX#7 zznW_Qmh5@c+ah`@d{L=wY#RIWJ7%>!w_7WMYiC557xX&ds@nQjW{jr4XJ)eZjHT4b zY+Z9345nF5C|ZN3-wNT}&?Y@Z#z5oS<7r(=OXOtwB2EEbMplwcEq|^7=RloJ; zRW}A3_V(9=`-sn_%gMj@q%XrnR-m=?g)Ka*f)ZV`D4XXSFnSgKgu;jK2)8+r4$lDT z?4OHiO-lZPUC8`1T&Z}zO1*J(!VP_3$DG3}`FF*UT952<*DLU#ZEB?&G<@wIQc;@o zRtZd9`elBR&b~=mysaC~1Vx-bezFyhds=z!0A z;h0CGQpZs}ilp?A^`EnvR64H!b#dowFDG@yj;!mP-Dgeil{Ca#&k0>Rr5hTkXvu`C z+MI01dT*hHnwzw*R9t&XQfu5Y8(hoO0mPW!pI52bC`k9ZTDwbUcBD0JU~p$!n=w{y zo7czj;;A=2JF&uh4sb~=sEM{OChE-=!af!ugsUkwkKnsr>06rZN?qKX6T>b=?_p;O z>3emn6&KMybt#p8e@=UIN%vY|FU0ebf}`XDRQ;}es&E@k;)gg!npmgF!|A3~TnhlV z$$0g7%o3F8<9KC8sv+-Jbj^cM%ha9Uimv18au&tFJlKvN@G4<9K$2IoI{Djp&bkF>D~`b1G~$U1+GdwYfpUhNFy zn!vQO5GuPvjE^Z#bF16Eak2H>l_@`0kx2;12orgMGtTSS!2da(N0!{RmYo9GZr-g) z=yT}RaGdpj=x*16J6%a{xD%DBUVH2K(j3_rXUiAjt9eX(}i%BH0sBRhSi82J z1u0Iqh3eRqvM8isg_4n-(PO8#J#O`8G>YWgX*>(MEdr2XE0X% zha!-*Ie6d08o#2R7LpvKphdas%v#le;#ap)2rwQboLQ5WZ_}=f+ETI@dc^v%BOmV2 z(yk1*nqGURI#730gF8{=&574f+*6=giW6J9UwZxu@U6Zc{Do^z;H)9^f8p#&=762Z zAFy29z5kxd)_w(A8v3 z`o=qj-&&@0^nwQF9jK*QE}V(-e3!MMwHAp_j;JT@ccKL;TUHfNrw$bvCFVHp=EMZs z`({LT@E^;vB25ay&o0(PIYXLAlHpm^pE=S<@!FLN462%2MWLTCY;t+1LSv>bJ<50W zq^^Aau7X^Wvo>Nai~%f9>_JrORpnqD(VwVBW%+8}ang)L*?46-CPh2-&8GDqHcx-N zAG`Qi?i0pLElhcLQg=2Er2(OiV<)hT=naaFZAOmZ;Klab2jSnVx~@voH2LHtKay)+ zgbZY?d#}}1*d!4H2aw(t<7Q*bh)*(*>_2a1=F`RspA#x_nn};UoICd5x@&;vo&~#% zHqKU`K4TGP*Ed_%PaGQKk+f~y>X}oHgrPlpaO#FiTbS^919qiHQg>VZN;mBM&jHH2 z^(6BaC3#id{qv}F+~bk5=qiSc(`$G$nkm@0Qajt|z ziv<}4VE%R^RbKgw0>)W=EQuF0S!uq5Lc$IEXQHfVM~Eni8)?xC0$G6wF6uyqeot`a zJQ+AlUZgg(u@f)#U4!@%8Nc7YYY#U9(;Yi;ezbY%&*@iQ*ja%`lI?;YG}L!rt>slO z!hLF6ZUdwb)h`_@A;#9_5A8qakD0vg7LXQDfydxtnny6y75v$uG2GEQ{^kRfmsx!a z>x@R8E%+f@8Xdo`mTb&QRT@cymRcq~iVB1{;-4&MVAHK+821amb!x5kk|!*ZV%c}Z z3ZLU1Md<(&m~tfAOSg;EP`gSYd ze5R;R_>F{S1nDu(kFPp>o{f0G9^H|2=E=dB1xq@U>r6%|Ap5gyL>i*w$!aAVE6RCe z>-AA}fp%MQ`(|wk+u^99l?wgqqrM63f0VJh;m~z)?O&_Qk`2M4u#9yFhcN$kLW}pk z^fR*IZur!1K|QY!4)XJ;HgB(~{K4Yb5GhWg@fh9|^E+#1t6yQih4dJ&(bvFcB{*`@ zXhl|DS+efV%%}Tj@BKy737sHI$1i0M8N8H$OF^L@eTWLBZsMt9g;*#Vuv?boEkgbx z>;6+C8ZrDp+JLY!>S+CwnX@tm07rXE4}1i12-T{$6)=;}T(rY+x6IZGz6 zr_Gk|4Sm6I3S>94nlvU|DUe?wZ3Cm&T#s$=8vAEN8^w;{Y z6~(SLws*S+$6g<8Io)1FSrfpE>It1l^Taf%u_)v#T=KE$Rdci{5lNvNO*2kL^;W-E z7f@#r8Br)H9X0M{6qj`D;Z68eCAUcVWQKV=8r7hY!egm zjdIJ3&)rT>W^=xe}CT3GLJnyp-M*k=5k8FTp<3aYCVep@VVyJaU;Ya3YIF#}@&BjLJLwq(8J zi8>5S`|7J6cR{8@y_R_4r-j-wS!L=Jj?52=l5vQ+b#1Yp=7U>Pc3=iUFAto#($==I zdwXWnInHEbLwYM~i3{P-b(k{}+A^cLofxU#`9|PXRU9rJrr9ZtK~r# z{#HMxb0#(IsPwqW&c&P?{{8Fd#Au?w@b$MZms{et!APc+f{5e( zH%+_hHy?6}cAMhwHN(hB{EhRATUv)1 z@}(l;Mgi6;go)Y3=%y!g+Pn*2PI=%%3Tte7mbbx+OmZ=1S+Lg#G;nvh)+N z$gse}S8&}ofY_4%WIN#iVH60vs1ut2QZ2|S@+637QW2cW$nK)Jo@9A!Tqmr!%X{Wt z+SYCQ5=T#|@g}7ZJZV}+>P{!EST43#i4g5HQ`B?Ai?`7~4dq%!m>YWpOrdUDdkh@C zTdkQoZUs)~1nrabcb?_6Jyo?Geq1Lu;j+nb|0U4)(%m?*%H`resY39XdF&#!oB#GI^dgK%=_X#cAYu~IPc$KiV5F9|C=|5by`lTE%PXnx-bF{ZgfKPPU-=`J@=3yP*Cm!C)WXZdx=gZxl_TKCWHgwjgP|Sn zcA)0U8p$ItgWztJV04^W>`))J52%TW}Dx2EhKie!4F3$tx}Z6k;u*?AzA9Bol7eqcf@ z6i+8x+2*${+*eeoOM9wv`tJE|3d@?|Jw2Zw%Xsnb9@{JvcYK8#uwi4TS}U|++_yjf z<0{2`)S{pgp(jmngzjDlIstU5@^>?pI85_F4-{9JIeskKx-Mv)$qt>o6i^xIZ-NB`E^n$fL zcFd8NVY?~RXSS-f`ke7!%G86gk$w2Ko2NlEPDdz9f|?hM+FXD^G1Nq1Zr%|Sa8EW$ zKK9Sv?v|o^h4sTdP9gJ^*ADG;1C$9CR~Os741xz7V!C8)3@b4Oor_ftdioTSumPm%4562HwiqU@UgN)9wtnsShKcQ@k6B8;XF2aun&<*%NTM z-)RQ+9~!B0jDst0S8sAHtsp+l zn$ljS1O$OUzuzgoh!*WfHv|*03ET3#M`eZwHU_}X4uAOq{2e8uk56&DxA@_;eKC9K=K%e$7y>RTy-p%*%O;oQA6U1sp9z2` zvw?#*xTqJkxCK%-Pb~CG zHd#Jj)_Cz!_)E){mznfLF+LBixyWI!CN8x+JE^{y=;Br3UEKN6z>It7=B4eS9fLLo zjQDLkuV84?aa!bqY`Lta5Pw|RZ;_fpstLj_VlG<&($sUuVc^q3pwEJY?E2^(qgcyG zbhiBLac3B-ep=RUe#8Pl{6ea!kpSif(}mDF98|{0M`p1ydQ#wkxvg-e#&Bl(rL82D zg$a;JR!ENU)(VTX%;|>x^pd=k+M}nY^1=&Az|oXFBDWAeGvipQ7VBne6>i zb&{7qyIOb=bBk)YzV-e1s{ir5W3S<+QAm5?I_RMgIy}uYQhAyeqom_y;F~(sfryDX zQ~L4PVByoZY5;cIU-7+^iuU47Zb`))V8&wPc>>Ke0Nu^2zvif)osc9FYxh5#4?RgG zEx*D*mhLaX6dBmfo_5X+s`}4*0&uCbwI|9e@5rRZpSDy#z-+PF?8;DHYUB&zbN>O5 zMfUfI%1~q0zvr&g#81XW0DLn3;S?dh+3ON<44s}6b+%)duDieD+5wjTMy-EXaC66G z$XSw*4bXv3MDk~wU|$WMwP&(WvHEVLh44T7>=s9aM$BobU9}2T#<%=jwJ@74xd=hUO(kQha*nU5HeSRFTHY3cO9(BBjN$Au@Fn-xvlB)vc_aPbs^~IxHd--ZM0+cvoY2F*CZ$~-#VP)H_7kd2s>`sW5jo>goc@!)|l1CjIwy^ zMucTA?#w;F`Z1_vi~OE|1YO)P%-Q2OTshN=+#1Ns(SFqniTca&(WfHYO~%N z+YYCHTeG8!c70wq}w`=+y0xdn1BTUo6Pt93;4q1anj7C@MC>Z~to<|bBhq^0D+R__&3QcnUXCHs)?eq|XBoo7E3=tj zY_7arZ;2nPR)7TIzO@h>X}$7=Erk-}@^$YzakAf2lVfD3DYh@mtI8o%jSgXadvzSz z=+JNRt$q=q10=b2LB=7y&>|<-;4sa!ymAjq7mu02#>?H_W};E4{aQH^_EC#Y`1{LK zOscURAFg%7*H|##VG9~|6b3^k$e`bb?hcNCvfnS$Z!WbfGucp~?bShVYhT=q`c>CVwjt0EA;)P^5DcMN#)9pUUn7Y~I-Yj3k| zyZrfLO$o7n?)vd$SAGaSno0HAV$j|^VKX0TYp0mE+DlRj4E$8|bm9X^?nPyW=wUN) z5zroZ%=>TD_&#spGoM@iYXjGbVIe&7!|?AEPyo+?0zH(#&?N&4&d2R?_E(_Q!8gvk=`rDzYT`yQc52cdmGreq&3hS zm`FyF4|bFq4EwY3FGt$vA05iln>l4JTteki1{+Rd4^3I35B$WC?`9C0t+f(bn1Fe| zBxOT$jgk4oQeVhrVZx0MU`v|#&o_r25H}|i^o#$BC`_XSW3RTS? zpr1F4dDWyiqU}zPGVKbpw5B}Szc=uHMe#*}oV@5**;EE^Z2NZFAaAO0K}!ZfNs1~D z$Lj_DTBS{1?cytx&@3esKe%P5#gINWnv^{unw>ncy*}w}_f3Q}Gs)+lAo^ax>qPSk(F>I$R&aKR&r=l=Z-T$`@GFQg= z{A^jSnRb8w1ACL06Qz1lpA8+jS&y{+Ebu>qy1 zGY!g9?O96l;!#_&h;z2}7SiLLB(4n6Oo3b;$=FQ%?-GKT^$8f4fQC9uq$q>zcX4_{ z9wND>Jh|aKT_5fyI=Mg_%teF+Xf%_@-tfbLt(`m-LJCNU0;WlYO^ZxD*5kDSd#x_qyuAm*kz$Z%W>fAdn(Z<%c&k`^8tc_ z(G}RMc+mEH`}ngFB3#&mcfqEq-6A*r>;5)9r1MRO{Edz<59ajW_7m9c4P9q??Q(@jJux&_#zs4P z$8P6&WF2HXTz_w;=_o+g@3EGwW-WBF*ed#NrWLrjn&&a$@=)9E_l5ius{fY#` z`xra3E6#XpRARmnC^72ROziqcvjrt)^{4GBi3x=kxubhXrjVBp(>mGu19)2t$IsM~ zO1ZY~FYmKuDp$VLICs9^H=*GHolR ztFbo)Ml*-?jT3o9l~C3^9>_W2hhql^Y|VymubCga*7j*z`*<|gHXWrnevr7cj2oKr zj#aLZ8^bu*JWRrPq6t0uq^DKnr|U5h$S$1ym5C<)`S|B4t!|z~?lH_+@_=wr?*955 z?+*8p_wM#X(nbw7B%$!c%H?tN2?fV0@B%PrSv%p=4j2cIegz8q=_93d_y0w?DwIAz zVp?KfzVOfBd5qGvnp;;E|0YXdll1@V-eVeVKtr&J#%|^@$!gTP(lR^gw%zO>$=y@tF4~GG2;HWYbM6e~-jQqPXw2LrA-8o|ba zMqtH_9z*%hBKX+$?zvD9V#9Du9ur^iyTpvj>n$}`^T;|Pd-~?+Sk~t!t-Tig_d%c0 zwp?PcGws|5*W9tIqd^qC`dNyrl!H?lkG(U7E^K%X)d$UAmbv8Yc}|B*=DR`)8eu19 zR?~l6^2r5@RZW^7!eZqS^Dnv#a#kRTvg-aL+XL-wX>QwL?>jGS>`WFJub$m#-~Qxq z=sH!7Gsjy@%E%8Cba3%0@VY6f`?u457USW*x8Bb@XmE5rw{F>@lTglb zPjU@dQt6SdIrz~Cr#|KVr+lU>bi%CqC~kXlvD)Eo6XEnm)#y9GKEBTP>T%kcvHJA% z?<8P`(E?Rqt4v=R8E)=UwC2RxDm0-Q3CzV;zxo@VoGb{1$5amsQ;46O^gw=b^S}&H z=f=cT5z^7@pI`T18RP$0Bm@i^X$Ns11Er>HtU`1y`Yf(rN_q#=$yN|Dx{jr^sSAGu9^Tt!4zR847*WA{%)S6;G^HxEFD& zP8ZJG>i@@T+8K5x9_2hhrQMXt!v5Gg4zb8q&vOu%XSjiV_>^&nE3G0vwe^Xwdz;e4 ze6fHIxuQeu&G;%7rF!Z6Lmuf2vTygqQj9c4FG&VggbWY@TUAnN60QScT7S{~J;clE z&uQgI{t@AalV`l>lR1QK>ky3J^9AnLRgUSjhr;jg)Gr}vWGGij-T4iYb(VkwGsOx7 zND}Z98U-F(JA9o24~U2^+WrT*9`Q9cIC=u}&dNO5;{U%EXYyqEP!;{_OFtpVkL)6L z*>z`Ss5jvKgax2?)ZjWyF@t!&__?VHID!V^!|HPuO^Qy3qPJ~U!+SOwbvxm~a)Rdk zY}&ivRdd1vlvO`FF0wW8`I7WOxi?dBK?D)h1ir8iZl2M~cXl+7ROXcQs#n|Uv{@!2 zY<44I(XrwTeRiNJoE1O3HO-$cZ%#iSdyPKJd$ytmE2nMT(DQMALyD#MDwE?^MOUX( zR&t`&mqc_ zz+arv_qQd~uOMR|5H)J6^!6sBxt{53KCk>w`}-Zaq4P?yAQ!d4|C<2to1Emc6i5n| zJ=MSJTE_Y0Rg=(2P%ve;pemv=jBi|f&NWu>cCCbD+vO0!K)YpzH1Ck1>roHGBoZP{&0{_Z~mImAsLq-CvoPybo{ zHBE-P=ZZ@9MUAu6Pfz{zXk6qIla`hqTZ?4>5leSk??Y8-DCaO7 zIp;6Ve06HIGAfoeaD}!Q@N*Oe#KV&|^3JhKTof`Jk`}_xb0N6KGgxrObgFM$^i)y` z8o0v+k~Q5vM0E*)RM_ubDKs*&_sPm4_V6`BSl`#PO!;P4$F)PeI--IEqHx+u`F9P* zUmoy(jLHwJ;^}=a;76KvFPhV5kG6>K;i%o3^Zc6M%fwuEQQUYLb2NH?1dzm^fkK$R z@KB>|&Au`)Mdr>bNjTmZ2<}j6j#Eed0T;P1%Fa((jr4Q7=s~v-0DS0BROb+LV>i8N zAO!NeDb1IvD40qTMD|kZJ190z@NIGZgWlC4=ZP0m{{86uwF&7NV#-o2RyJnt)b`MN z_^!6mywlOh);9;2KXKN_<6cX>>?2bt!UkkN&hW|&W0IeOvZz3=vzo7+LD3p{n%$*C z-p5&eHEb`5J$DrH7)mn+r9Q}&__!S^b@-m6)$5!yheAqC-S&%$5ksBdoXm+tDGRgL zH(n_6F5E_(3h_3!eY^UEAZ2<6t^9rvnr4YUa-Q_kuo?1su(92v6!syz%y9|Nr-~cw z$D5S~jE!}}-_Fo2thajK^|T+>#kK!v(3UBBMM%L+tNB`|Dp}4hqQ?6UjTBkw3!k=RE{vTYG!zp-qKf z%NtdIy$$(3y2S2-INFr|@hMIZD9p1j3aNaUtJ3cIGcjU%)43P)#X>K6A+i+mTKYm= zc*>&=le=PR82#DcozQv1((&{I@E5{#Yo=qGtu#lb+c++kk)-um+_dYO&@%SqXp$Jhs3?-ps`P~Zhbby*0-~*7E$N+t))s|i5#WN z94*~$xErIvjye4}7Rs4=t#BuX$EWOO+m}dlNJ%9!YnELW;Zi}SY#&nW^ZO8Xfn%Uf zv9;`D!+c7!GfrEFhU3X6-x z$hh>c>0ehd{MNb;(#DfNDfavN9yr)(Ef&Zvx=0ys5>C1Jv3u6&3Are~%wkU>IZm`? zKKRySY_4n?UF&VTcy$`q!Lc@0qT&4ne0f@)%%-VvZJNW2AmwrqCfu_#ow_z5xzfQH zHF7J6@&)KL2U@t*kdStZ4_tgS&#S7$Y0jaY zjHsZ|^6(O?BHJ*QE{n7CKHr^U_zgmd3(WOH0a{0WJCWKHK43a&q%h(2;x&x^c^v%+ zw|b$39SgC$u3}1hOYPDb`={AA4;*-2!@?FH>aE4P8TBf3YkPNdNhv8p;-_UFt+83w z?U3DDQqCf3vv>P9gz&zLx+$JxW+3ZYKbOq1?Qb#0AUJ)Ei|v7GQlRv(n~RE)^l`3~ zLZ0K;UJnFzx4%B-h8Nu)t(0@b$_J^=NE0vgAM9DDZ+g6Fs)OOFOzwfe&!JdjvIb0Z zW6kLETmR3KK5x}zJxq#(12kMtK{v>Wav^3}%n3oI8Cx>`RDep~lCHQ2J4y(47V{R?o%qmW#_@sjb4ax3dIh1f3vceVxIB1^15FF$AJLQibNM9ylzvJRpdJJQAiFKnGvQ|f z`9isQ*7z{JDvf>PS62FUVKNS+tC9~RnOK3J8SCWU@Py^*l$ z&a3>R@`+iVusqiSRw4_0T&&8olqG43#qM4a82M5&ZQ1}$&UmZ_c53xd6c4x(w{|^8 z0Ko-P{6JyOFrMv&zYt>hC^d3Zaq(+@%DoQshR*EtK6z0ls_IGHb)Z+ypLfzW0>K%X z8IV~U_UVrTp0lZ3BEf2msx3Tv^G7RRDnFNbH_zedo6#T4lDayvh8C`q$@O)HB~&k- z9zMg_opGZ{u(oV{sZxA#Q1%8PoZ=6z3Ym)h7iz(Q?50<##Nxy~g?4-*w0JPTd)6!_ zbe=11tjaFY*7I5$y1u8|t9mVr!H6=pOssnXID=gcH>b%Ts?<%eQj(DRP4%3lXG9wp zk!dVB3pz1;SyHDX<~^w7_%57|*(wVk3~-2xJy8!b-} zl$AuP#6E`t$xX}Z27~L9J3}&#tJ?ju%@DJki1iP3ufg13xh|+L)M&qXVB7^+?psxD zL4KvZ5MJkPon-R)wHcG@DZSd9W+`S$)5I|CNifmWI{Q2V8i27MU?Qx@mLju-OY-># zLYKxm%(9*(K$~7mbwt>7MJMRfnff%}d2p}rj$em>#P2FM#0o2V*3Bj{tHA^1((?v^&~8+?bbU4#j1 zd$fiShjB&@m$&V?Q{}@}T;8h>dtpB40f>D+iFo+od(V=pV{AlH9U|mO~Cc?D(yMb>e%dnA}qr3*o8lAY!<|UR^f?#l58-k7kAzWAS_)de*6w@6ewNJ7x zTuAg0wBK@POz`)+hZzNQZI6%ZkWN+y<)Hh^H0nw2Mab&e)=U=4`IMC-7V)<-=UjR^ zipR?@WY=+Z_fUP%+qk~QG9M=F(kBbHo0GSy$9TwFy zOg&Mi!CY-5D7jAU?!2z)X4$Dc5#CmdD`R|TqU?l&^+V$rM))PPN@LuPe8Jonx;_U_ zE0A6qr|d(SFlP}x{M+=7H(!CpV5ZVxmk>PPz3u9U3J2x;>h3W||o{DpE97CwvINAR(RJ_Bb}xdf{Vm7C-a?^w=Hmw z#aMTbrEyp(k zR8EDq+kGW54j=-th*1G>A0DPrpxQ%8S`_Jy?c#BmChv3&?&;u71)${osS8BX9Vmrt zAY<~$Wz6~i3aY<4G1@@QhawTBbU&Z}Rk0rbsdDmO>*sQs|Gf8@ow6^KUt*Pwm)GTM z_w0V;I={`2SB$1mntgs2k*4-V_S-zWut5v*Q#^oUI5fPnNEwwJ1m;NBx+=}ATXn2Y zG6h>IobTVT`IaVg&d>Bi% z7M-;V0gjdY#5oQPzwET`IdMj^ra1D+tLJ8sok4Ht_irR0iV zEPRcWe7e9<{#tnFKv*wJe2J{;-ZacsLvPsQQaqh4r=8@&z}wwaLv`@6GLm%qkcaGo zuM`+(t*i54&Zcu|;Ng&w*GPS5e~h5J<#do8vA?DL?r6U9I=@RGmN<%KN{Pwo*WBnq z%Gm*id$`Aol6AEuM;&R!4}IE1>*_2II2I0r2z4EgDCsa_7R3)WH0=8)%Vv7G_!TlA z(0$s8bH}Gvw8iK957L*1SH0V_e4jdgX&B^4lKes_*T)m5&D?n3r*W3nf#o5-wAm=p zw2=pc{WPu?{HGaBa!=rlfpNw*_;e<)X_}4wS53oS4QkK?wlGzXNp})XiqgLxR=0k8 zrQiO{#7_6wKkqg2Tj|Ie<(u@kguC+PQ;E)I_B|b*7%$0OZVzGoX#*Rm1btu0 zgIeD;BG$ns*k+@P&1v0c+NJ@s7xa=~;k2{zgtXj?Hw{epHJo)KUv-?$?vaYmPKmyG zbUQts6raOmi_WbeTNCk%1Ezs>M(;06d%F1<4xHYozhKFn>nk-LM-QhxC>Zo}ski^* z?t9V78Ds5R!ZSJRcxiIDQatjv$vmg&r2w$M$(gqfndQ<3xZ4@aA)ZD(E36#;qyBsf zrsGo44(C%O)zN3jPJ0S%k6Z!=JchJy3}4V`QoK}G4R8OA7+L9JL>pPVP0{8jwc%=( z@jk*lJ`EV32fV}fsOH|7%7CU=MJKlny`)>Cd;+{E8483OF2|$a>;i zQ|>TyZ4UG+@%-0?An*a2rx{%?|JQQ>lr~ADygSHsq{=@%0@*Q1dm}6KSN?gtr)w3a z-|pFkOqyOzZThC1sV5aHE(VFz+LUBG91Kjn_whp#H))qa)?Pgnd&7TvON+YKTn z00BTmMOQ8lZlPvq74MVH=Q|Zg8f~YHYTwfmlh-C{^*56X8k_|rGB?$dUNo@$u8CSI zNdHhYrxAOgT^Z>j#(OnRv2a}1ws(gol^)X>Dy6n*q7L#C^3cFuvhI)G%TO8R3}F%P z_Ch<~tt4W!5nzRkAnsKsMBg0Lu~#1OXDo{Iog;URt7kn}%SQGwV!Xf63T>ubIL0bA zr8ttGJNTDQmb4kh!GWxATSb&ui5PZuUnR78`zFb4Kl||A% z&l>IDa5nR~=`A_?mBcKS{>Qz`hc!xx1@Dj#=P{a}(`rWxIi7j%nTks;r_af!ud|KeOdU)r^UJGS-ideRp8H!|cNSKz%hculb1XpN!R&&N;BDU8)yJ4O_k%D%I| z-Z$B`jv3XklO2F>V6;Z74c=FcRat4J519Nvy1qLauDyFd5h)TSqW2&{kO-qQNYo?* zH_;iQjhcywI%aDf1D>E^22QDf~>T7`+xp^Lk|9a2j?9~_1vMMRX z_wk5J`g22cRVJHfwN+wxMCe}|ai=p}_r5R$`JpqI!WMpAxbKu4@;Va z6E5Gkf%NZt-FGzNHBCT#Czx<)XQWeHw0R!?U3;FRsP-}mzN~wr=xD@co09E^AAO_u z9L)&(GOKr4-t^tlZ z6*5&%pz7l3vad5R=#?WF3i)1FXOr7N`-tE1UScHP+VGL2US0kM@{Pa~3hx;{su)g? zd_mK^dHN7ti4Piz{tHrfwT`K@LLH39NP^}ku5Q!@JE zpAi(-=yWM6I$BHNaYD>U+>eUCtJ>cekK%!Hmo#1A+WYsf{j34;73L-A#sWwR#61}WCq?ed=gqJBQbW@g7-LkVsK^#hF0X)q;vqA2fDD?{N zx>%IQ|J#tHm|whTxO`UljBca-n_`}%xATR3Y(|>4n`^;og1>*2v32e+L_@9bi1E|M z!82aTvnpbSH-lHeL1fss_hn4V-1SPst5k-13Z;xRcQiF~Ga5aI8pdSwR1WkjpO3XW zTyOn+%+d1%{dpCKjF7q1N*VDL>~c4)(k{lxu^3pJ zKHL0S{E6Wa+-j#=^DZOYRzOn0+f9NF4RW>S>{9b1GuCyKDdUjCUY)%m-(m9VP+4ylB|VjtEKBetg5C{ zFkF5|-yzf2@0;<2&(6fJ+E#*0$nw}btww=2e<5r6Gt$1-&H#;@g~s0$bJW8#{!B3^ z6E;tO_lfzhtYq=&+G$ZV^Xg6i=RFI+PO-iYY(Coydk~reSYY8pfu;f1+GQm&{wGyV2sriDQb3DIwc`Z*C3XQNDlH7X)H&Q6nel17x!kz%Fkxn~v4@G8@-8|ALAX)GUDfp6iw zC`XgdkAK@1(*b%I{*t`y2GLVZe;;$$z-qKjsV48r_T%GbgoCnnDX^43UM9+5Y>vlV z=)fAbO6opW?&4($Pv^xPHS8eGT*qN~GQ;aL1Xxx_R4}?~hf*I&tKOoSoqS-Y5iGje zGg3HY_c@pEE(SxrBQwS8486dTHuh9K+f4rMnwB#`S14nzv^1k{4=5WK0 z(70Ldp;nD9++)K%H+fGEmso|rf%{~%ipXt^XFqa*(c4GTWtD50_rk-uP)<^P09%%`2&Ig54ue>G|?kItO$ z1y@iX#J;^BY|9ZTbw?U>`iP+ba2ynMs*?7r=c>;H-!6xL&^@$Kg@BtHUaKUW--D39 zWXM6GMA^z}&2(TZg2!E79R*D*cpftBKje!4sq&`UOl0;oS)XRqQqPn61QPPPZhc|c z9dLhEOb;9Eri+enG0%W7V;uOhQAK_KU7mRcGJQs<1cg!%r^pJ?+xx<8ci*C^2q-Er z1C3srzM0k#f#Xe}WT|u3-hs+}>>PcTiL2>%7qfa#KxE-Y!%q6rqK`bM{mo9o9`K>s zD#D8&*{%g%pp#s+5f-ppWx2miG8(5*6AdbuOP?jOD&0hW3>DyV{WlZuZKR6hScF z*Dm(xvizru`%iJZn*kKO5YP%$qe_KP*9uT1tjFN{wgDOI}k zDq8jxwZPS?TvEh?P}B^CacD%n@Q^OK`35=+^=dq|>_TxK!&rxeBctW~d^FW0B{96W zO7Y-OC*Ie(mlo55VJfuy!2Mbk7KM?Ne{xwuvVVIm_%kFO5zto*y>b7m^intiexwHb z6dR@M?QG=AMBk)t%cAVbEyVT&u@_Jo%}0w+suO&UA{u?UNYB^hi&)@#QzT%?pth0u z!$Q?3V)KdB6|+3xkIPiIO|g*qlp0j*XgwO}wB`px>iUgj{8m=pYWl|Dt>b8jDc3+u zks@oP-D?ZSAV#& zbaaa9$f}So!JE{Ul<$zHKUSL$J3`nhM3G=&31GF zQ>d)IcxxZ#{u0D(RcX$>q!^L>LfDA#jN>QK4B|7Ku|&*-P<4_(?2$u$UvG-BhAjqi zL>8yp%w#5u+HUzMbb4XSIdH?p&=9M!Ra_jwt7oIVn^5iwfe~FMZpIewz$Gs4e9pgM zdAolznu*4!2cbpzegBa;ZZvXU15RQ!4qZMbvJ@@U^wo4439-68m4Xn|f$#L~soHg` zNOJxmnGBGbenffx2Qj@Ovm2m2ieaz)TM7BQ>;V^B07ws_z5j21JLJrn5#q`{nyXm=Pvel^^jHwP6n zcDb=A1uX`nKNBnXd9ds|8`6y^WL4%UvKfc%o1bOTK2t!pg)j=`FJ^D%71JwSvK}h8 zh@>zvpZy&5;QT5EbKxL{@`XGjgkW2x*2dSWkVsQJ73ce{@JP3(vCPxb74y-7 z6yU7UZ8=ely6HQ^`C(22!4hNUAB`0zfbogYGP7{A9$H5wT_d>z;o=8wC8)9pXU~v_ z5O&f=BMP>2Ph2aUG$erULn<#_M|}fjHG4LLawVGLj>SV>!L}$LAp67HN|HB$Y`Gq~ zRJ_?9C78=P-SYD0@efYNkeTR7($5xa>w#yk*RnQy&)_zFV+%c2h_kxxYM%}>xP5Xf zo=b#cK;vwkMMcWc=GaYk5Q-^N22n%RR9VZp0@U7gZdep zHv<)bc)5H3?;bzoTH|D_3BV8a@p&r#m52Un;7$Wa$r0%JD9`<~Y@G!F)y&L{mfu-Z zMe!wzushi56ghmVc0hpdv40 zn`fGhnRtYjS{6sWq;B%Wk5?ewxCRa@HwV|D%iuup%7YOZ^)}J3nwQ76nDw+IbB{i= zf>IxZ4k;jX3u-^^Ov#Q)L`^ebMKFe?ODi~^aS>u#m&B(gjOS`}161eP5J%f}%{M_d z=`A7%rWXx8FBMG=mMNK2lRQJXm}3btciBpq6djf=hE!I*`Bp+-zHw^><*si`9`qvE z3O2*0?_sIMLWM89pz8B^H>8GRf4@@^v;#7Sb)&Q$@iQ4^Ko?G>@L#zlP&Zok-E1hA zpY&>g$KPp0SP)1tF2HHdD7`}c+XXT&#r{=`e$}1#r#b3TyzO7|>8~mU+_@6bI6OT3 z!_(lDT10+1_%0IB1&%Lkdrj8Z^aphyEC9q&Ou>t>bl)5G6+7NMt(v8J5|85iym_9` zdwaFg3Z^~jx8ie8Pym-X&B|ym8_-gCFW=@B8cv3XrU4&oi@lVWd=|p4mtsw(BchM4 z+W2~m*&cGM=7G`=s0){gm`3J=+{SG$&RQh7dQY$wUOwEWT1s@QAoF0LcH~0r>bo+u zWXp1zUDOulNL5q0Km+O4I1#%e^m2v5b@rHGGL13pcuC``qGeVt8)Bu~tDkR1yEK!+ z>q_rDIYR(_Xe42z8)dpCi?|BO;JiCFB{Y+^g75F}CPpg@}{<|Uep3x8c zt|a#8FDQ1Vmb^?Vx2gm-od+c{@9cV&vEwj^TY=TuKRxVP=TZwj-LkzUe}}%;!iw`b zoa+e}H+NfMu#zx!g*8M^d$o_k-IPWXB?Y06#*%xWu+{DjYR3qV3uU$REjA+lp{?a+ zLiPrV%Dn*$t;~-I^&F{fQyC%1hTR@InEeNBeI3|_aprLa^@S^cRw+K9N-?XJ7AC|1064_JrE;8$0yWEe`E#YOkXl8v zFuNr({QHdXu{$wiy+a_AKDV9ehhxN4@nKQ!F-;%of{sT|o#eT#^j<8^8mewB zg)6|lLSWAgf+%4ecTaT3BbS2KXIoyk*=h80CyZS3AQ~d4G@o9~Gek`qsMNdn^s(nN zE!kz{x^WyVs7m-OIXX2{(G_o^nd&au%-~Rccbn64v#0LqTe&IVygZ>7 z93#ZUsYMx`4{)GMc>CEx=j|xmnlRJ);%t#=;h>Pyn4D|Q&nH_^(aMh9c#y*esYzFJ z;)UY3y)o7SInZ}bmgolW3mdy|#ft*zIeb+8Bl(s)k(o z+>VwNY)JtkWRCZ)f&$?N4N*PVbR);UZEglprI{BKb+sg+ae(KPWUL}&-2N+iLK?m1 zGpqZfbQQ%HQ>e5Q9{n580EwokKR-zKudMTThB|xpu!BTf1iZFpG0|K5Nztz>ZpIL!3d<>uYN2!Jk}vS>!U)t< zpUt;w?`G6l-q~%_u6k|ha^PVEf|tn8hIfH=Z!5?>J+r{VE>Ndg8fob);*N~#6C;+f zJ8%ZH-bH0aXwXG!(MLiwZ7wjzr^Z10UL&2x5Tyhi)2({#>vc*w8~r{zZ1;ejtx)2}HFlW)C<|VPW@f8(3kOpEXDW)3h5-dT)1D^f}T< zImyG_N02^wuPY1Zl=s_#^8{YzrzdeUaTJLiDa&Jt8K1$mAUagunkz@$;yD+syM$q= z$Qz!V`Q8#xePTexmdk2QB?Y*`HqLI=|NbRj!+Z=o8iv1a3^I8Vig@x$6LD4vWe#^- zsLJzt`YH8gti)bHRMXfHvg1B>;ftKI#l)KBm_le-a;8z9YYnvfi49WXPULMTtlT+$ zypSy1p2+d0q4FnAGBw=*E`jh4v395Vw5$#h&bbG{KTc|}yFUn0`G(5HsBcby-Y#;! z?N*IC%XX3o1{H8WuK$*w_gMfPOD;)s_R!7j7EV6?2*ee`3pN0FzrAl};* z_2(CVm+hCB5&P|(mE?^*;s1ejGyx&wB5&Y7^GyJ`EV->CzR7>k+Vc-Q?Ld3(DSp10%`p*W$ubgeQLBvIem~xQf71q-PVk$ z(6j_SGB@x6X>J2?b=(BxG#S%RhqGeS??Yl^EYrTXyo-=}ugw4^{oa*!+c5rieU(B) z6Tt*Gn@ZTbcu5qg0v^DYXU%a6UnS zIK0;Ze9@(;=2KJylQO1)Re|Y<1Iyt#c{PDU z#3NC8*2iW@MvRutyLH_`|E4**%x!;Zd=q!Xf-mEkboaKhuC_zk)_lR945BlcTB8q zPBQrAaB@bnQ_3e;Z<_f}X~JTOu5mK(6e{Dn=+*x#ubd*A)7G4Em+a;4-`VDmi}b8B zJ+hLDR-`}Px+_O!HuW3!-52-Bo$$toswX_+o0MnJ&+>*JuiRTVgndSIoQ?QiScBAn zEl3OAAJYuG<6F@F^y>@23+Vd;OqTR0~7CBtp?kc;|TACX08 zbm|K)D><9DYmRoz%}`ub4}V6_YI4t8JO*+Y<8Zy7?z^|M8Xt$d!bS$efUzy2rq*HL zDn(a?#JBFX;9~t65u%3AQ%a`0^95F=u2p=B7@y?&L3!TJiHI|DV%~km)2UMebx7N- z#VP}J^uEMb!yD5Rpq4e2Z+bbH9y;;M<+--0$wgMW2MCK%&{(NynLBsMg4Y422xQbf zS_T*Wjke2*>XW?HdX}_$4lR=#+ih}92WL@+-xO&>A5~XH&z%7po&=Clg{_uJnWd^*^304;gI9RXre$wA)>nhxLG8^h&ECZ*F)ntaf7J!C9x=>1Zt=nkHhW!<>G1Q8yS@sudxcfdn+btVdDfm79ZGh>-RBb%%VsbzuHGy)zN zeNlB*;{>#?C+fBT*ar{l4ie3{RIYK<-UnL+guO1`a>>`UR)EA)IdHa_R8y1H?@_hq zH94p?+<{D#IbNUX|151y^M%Ul4jci54X{FK5i&1>HC*RXLn zWGl9(^RG!sVtK4G946vXJb*n-`#pF(?~MWO$0ckD;+9F(TxZ2g7}m5;ZdK)i$;il9 z@ka^9!i+@krFmQx^7)wSbDJLYI7VDsOAzb z6E?HJlC84$+LkKys_{W(^Nw(gbFm$_AWS@75V_%o6uoKUCNR5N0f`+ zODPpc#zT1?^&G#QxWL%b4&r5EX7jC9GOaM^@ZKvdCO-j0+_+oj+yUCy;Z}6@Jitg4 zc52GL06W*~IVzU3Sq{=?%7{x=WP#DXx|R=jI7$fHy=QcLUK`eU5l5pxwD0F|ztqq7 z$|OfopG|XjGFlN|94<$Xuu%X!3lK!BLSNVar1%=~fXQdI^#oP!HWY+O_`mecuWR8u zr@lqgrH+yRq?}(joNURIdt}%ASO|bpx;t0XgMRFUI7L87tbr`Rf;hyOewc2kPr+-g zHvrZtiTL*Pl_B7Kw=*H1LfjQIG1dNR*LDMI@bwULu*{5hDLhlokJ?lz9=H=BC1ly4 zdB)iOOL~GtW}F-HQp0-=ZM)3yW5L%3FlRSSL=D8tQrOn2%(XP+66DU*epdyix=aX| z#yvN~+i4{SOC55q*22bVuCY*ja^sXe$@iyUBE3ZYjuk-Q_O>i(&)4WiuaP&AAQ_QI zb>kncezx3W6fuNK__<(0Su5=75{K0kBtvu-U?bBKLSLg*=Hg3@+XAZY4?k1{hMt({ zUo2s8-qBVhMC78!jV+X|zOk#|QeWX1Zcka+y zL19d;VVm@UfMCyLDit||uO3cBjxGCq&lGwA4$Vk#hND9CH;3eC95rpdq7L#?Ts@&p z1jDuNOcvsl!f!G(vg>}E6CTy;R`aZjJj^TQ%ynATN zmdnTRj!Tc!^I~YT*J-aAo{sR8g)t>R0UTw&-0()%nD^n&E0$wo|2wAQ8h<~PFK$-< z$hiAYG~MV#L3yOEiA4rA=qQZoec${81SK_6=EZ!|h6#~|t!f^GGC+X!L)Oa22+aC0 ztEFKeQ++MVc@qIdg{3OJo?7!gmMo5j9sLEwRapMo;GKXS+l`2O5&RGF?uC|MOauQJ zj?tt_`!`k1<8&l1`vnT7?Rt&iaXPQ*VX#Xb!vm}3uv)S-uXaywO`WIXbkE?^t(~Jn zbmD?aPf{U-4vA z?cDgqAFHD!Oy5vo6aRO2Mu)dZ8QfO=@FJrJEj3LuIqOU15 zdltP;*u)r?)&4e7;)HbQ(YwRodalcGw0S?d$ZNHm9cSWP;X&Q%+hS!rB7~{W^#zRy zVE{)51iL7;QitddE0R%m)Zb>GCf*{|34eSs+3K|&1p`Q)%CWH^AwKmQD5>{1OwFYD zeELwa^USIDhySF`L0{N2lYEZm?B1&0Mg!pQ`52UIuO4AHN+rI?+#2bT(D;8;u0K*v z)Z^1RoY5~nwSx=7dpG_WF~4$=ClGy2HK%d+e}4jLFL~N>>_@(XxQZ_^pB3Z!!)GQy zm-1}TWu?}+rQ~G)>t>@@<7rF0nxC))9YHL0 zy|S}CH8Q_QC;B;!86A#DUsZ6pU=__O+yIgPZup9Q!?%Q5WsXwWQT>L~3eE(sHDYt9 zi(Hyb6)Q|}+X`MB=?#YmkP5MSssrBs*mh{UVQGkwEx3Y1+&gZ)y@K_TLxjcro4uIV zudtUc4At=O_Bz<3XhzGtw;@;og0GRVr4tkyNnnlfc3&f@!dD7bxOFilDf*t!Xu>gc zsVw?v(a>3Q!Swc64xh5*O3~CqaZcotdNY?4y)EMKrSM41 z--I_8EZk*8r0X#k|5!TVRI;@g5LM;zd?_>DseVk<*fkGwgqp#9UdVI_>AqqmGp4wF z_yW8o-yhkwfDDFG*f=>hTJl4XYe*WGOg28&9+>iW5%KB-=bS`OM7}ywLo!u`1oB_`X`ufB0&tgyMg+Vx_yM z-1B^gmVigT=P=< z>=W9UoEJ}9%=GEQUP}rwCn$*J@LcXt(MhEUA&w5u>qbR}P~fW5^4gz=^-WwJw9eS@ zIC`4Wg^B-OzmFyr`d$H~P`%(+vd`r>g_a_`qg--{Z9ERQ&j@<&t^ zja){(vHThM8r!jN=V&cq@B-uAdUl{5%6Ts_*KX-62Xh>~~g(JxpRwo+AK^-q-Svr~Gr?JknM3fOoiw24Px zSB6?FsBNJr6PW3?%(+oab>vrnl zkpFgPyt*O-cg&+d3fV^5VlPT3I}}wq+aWJqtn(W|qgkK1UUh9MmP%{zJP127$`uoV z9VKdGHG4tX->OQzEOsR9Jv5NKu!+g#W9~9Ou1^!n^Mx}kmK(g-zIz}hFdD|o?d1e`UQ!{p;UZ-H?W& z6Xd!4?|?0r5qx=4o?WJ@jW@}pwB_f@^1~9~l;E>+_3zO<`)_3)mvErZ{=1p$J6!+~dozFvF_4mw#mJdeufpf;}vXO2>x-CqpbT{&<%$*R!(h!<^Da zCaa9bW1}Y*g_5xN$#hrin*Oedb^+BzRc#CB8H$~!9CbTB^vd8IsSQ#TyBMTDEn09F zPjc#smG-P<0<3g_hHtBDj+3&Gna&N(7osJR9lA+O5Mn=A1O$yo^&?y1j5}ACE0|^= znYbgpv1n2o^0}H@^~>j*mIL^v&b_WBbH^8w7qH*E{P?ZbyY+I1E4;!(SP6zaR)~ku zYa6fyU^Q>?-Ob37qd2MTV((RLNfuuG(_l#g2%V4)3m78;`2e z++QNrCZ^eMnp;)vyPChPmMsyc*6OJP@VTVl{?7z$>l0$tfcUAznxaTv@PXs^gH!7I z>u7(z)$!N6>i5Yu15Y#87r_^le`c<`4ZjSHY*$a` z<(IV(qFw8j{hQ91LfmLO5GuO9^mH+MDJ<8 zGdAMYHZ8K8;BIc>Zq}2#BQ*HrnLtD=j{KbzCqwN>S>ZEvRi6xsNO-d~tmu_|zpN!Y zx8k!wI%)WQN4Wou`&*XHy;|pkNzRiv=`dg{JR+s&dnr$4tBh*x&GoocOJLY)n!Q ziTL$l^ipd7)6KFrJRQLjgL8|Wes}F$r*Hui#Q6*RdQ^BWI(BY(HY6e~uWUrtUP~D* z9}(OR4)he)XS>*J*p`Zu9iN#US!OX+7D0tnP7(K5SKYB)lxa?shF{{GggKu$Jvt~( zDH`7Mbjd$NGdF|V(jE`T=}S)s~6i_hF_~^v)7< z+2+f8Ljq&f>qaaU61#oUYLfjj$m)`v?tbriVHkn=O5#xwFqB#*K87j>S*0IBbxW!c&OiMtwK^V}WuQT7{o z1*{YK=J)d{rNR#`joFi5_DLYU1qucZ<}0&q5XhgmEb=_<`dT~?!;kuXwa~ICKYApr zZLV2ZyOD&H6)Ded*bkMNJg}fV{!{mak|;k;+`Qw zbO)fDn7OjpIa+Qe-k1a5C|#rQJRwZ?JUjv0d^$?++#WKYglznt{Fcdk6|-Y+a5Sqo zd*C%Qz*9P5G+*hAm90CP9t5Tto@hgi6U=W^yw`2xG17Sa*L>q#K)_DLAi!76W!p&J z`zO}=gX{iI3#ckHum7Fp=HeQ4DyV;YzlF}dXm^VEb=!jWssM{yyS#2YlI%gPKEqhv zr+|7@ANr;wl3XQKMr83bJ?StqeaF|=omE}$>6)%;;br->`+jc6S}LOXz1ck9)bU&F zkgp}8gyD{(I%em@1Tj&@04buPJf6x-j3ioq+_x1pMEhdK*FcH)Yb0h^12FvNTjcS$Y^J@JwjUBGF+@lv zeuAWfH?Ns66@{x+CIRz&wmSH1B^HMRf-H1C55h^(94?{8dE zz9heB;E{$tY7#LeaE>~baK~;EyiOgmn-;Xvj(RfFJ0sZ#)n*N9mL9*sr`eJX`VA8tn>&Xa>-gK!nldZX>aHm;6 zpC>*y!+^tq^hL!$m=Pv)GJ6{DjddEWLg&ec3 zdNeQR8h#2}0q4BGj22Q8t#Q{q9^3m4LR@#Z>m8NMtEq_U4+Hl8&s}IAigfF4(O7mk zX!^W><4AA7N$0oLC7&p3Mi;bR#u(eanHlh~wVPp~`YsI!zAoPMB6g<0lGtvd#w#L@ z4VDSN?+-&Fnl9syk@zV;=N^eltW$$Z5c$tX z!|bdShK&`4Urm!9CRMDDmKnJA!TDsS~nTFe1MaUQFn zs;K3ru;ueMr=43p55`O6zkFKsi-yPH54o3~W2JUYx0j7&N^GEhB_T#ZtgzV4UI%H$ zt^qk3b!b%QI2>wY5(G_9SI$a2DC&=$h?W1sW$N$9_hIKsq!YW{s~@QW+l&4;Veo^7 z3W)e({$XPqT*JctYMj4%JQi32T}Q_Vyeeey%Xwsc7hp9jAJysc*+$E9diO(!O<41K zz1F73>Zc~#{*mY40Y(^iQ0ATIcp2p2nV=jr5&=-U~dh4H(Jd zs==Ki;c8jBnhK2a0*`-{Tn!66`2`j?4VVC@>J(g@hhe$UP^-mK5Wj@EtpLyS*1isF zqYKZU<)nW!EnhC$eiYw`WmS3l)o-{O8W$J-W7#u=*=LKVlkQ9BQDT{09oXeP|6ujz z0>v{B-D{17f=NgA8P?9p*%^$U88DOEs5dCEG)cSve7H2n_t=Wd(Z9~!`CuI&`Py!q z;N3^sxcSgdr#d z0i{8hdF(N@FMzX2O#bUk&eap{=@#o-AKP$4h5YfUMo+N?^SmO|vvTr^)>XU(Y)0U9 z0ZJK|_q?3Zj<8yVA1H|1#PK#8e6FP;py-yQ?VwJM-O6U;IhsnWAF09_IGT$%g7tHk zTWdTwR<5q_(gFMBv-u|ug4Z{>lN&OMUU#BMy}~iF%C-R+M@)veGAP^H?T!V_wA)R$as1B}A|SEVk^riijcPl0)_ z?tTb5&vBVu{uB&4>5~OT$;zKKSKXZt#}Ae&znRzXj&&}uc%)%Ht@Qp)vxv~}X4GYL zb&AP`JEby#tca&yH;SUtKNgO=ZZjwmnMz3N zw{5h3%&-(Mq=@qIY*GvT*5>}6#vaSs^W9{;NJFQM9^G=-E4S*zpYh`U!JJgX&uf!@ z2gETJXhuJ^-_(^cNe_Yy?MWI4;}hp75LF3&x`B`M7dPpH3-VDzV$Q#Q$P~hg5P%&Z zk#L-5i{m)kMO3WXE!r<5tnWNSr1cjHHC%eRIi+lr8Kk7wemj!=Yy~Zs-m~^^<8V7^9ewYWT{hSSUw-1C;AbG=*>r$*i;ALIBOvv;;YW2v zXBBKtOdRPgT<^*SO{d;-j6$48ds%k}DPR>(4oSy54gRCMCtF{*o{PW-5I4(3fRV&y@ai~xyBC9x@(e2QX%-dPx@cVqr`?L=W%%OM)oY0gdQy^jU_Hf4` ztM?vOCtj4hSaHs_QpyJE=*$!y%R|Rr2i5b#uaium7`s8ycV{*r*9_ChlXC5Wvxt&x zk+5Q;lYyqtO6PjN8S70Nj9m%o`@G2HAK?om)AX}~={NqU$${rpEZ$CPnJ9gm(fOBZ z`Ku~JPQ89;2q*QwCb58W&AiQ}Vn1BQte@rjOOUv&eCExgoLaieIVz2Az+#6W5;S!nIz(3HX$slmcDJ{ZA1-yy(P#jkZC)`Cer+J5EEn_=pjzN^vamvpf2n$oK$z7ah!uGK)J zjb}F6a`2;1PlM|vpzV(FYrc#2@ru1|vg?Q^PAe_aAnVp70iU8k1RL}i)#mcqtl zuRhYSY~BCLx^iC3n;M5rkM6Ndx5r~7WW`6SwZ8!Bo}68b6Dy#--@X1v%ME}Mx>L3O zLV(C?ol2XQ#DmuFX&$q2ts|J-`MGXr_8??eo}c`f_9fd`AKJz^^En*I{DFuURSaT7Le_JR=F5mOs#ffyWf@_1x-NG`)x@Coj4hhWMT z&2|1;N*;J6aP1JW>feAlu9+G2@Z0^;F_P(fxK8Zz-N{V5-D-}zdaF}JJEPc(ou$Te zxrp|@O+y)(9jt`KVfPx#`|Rtr{QIq+i`?)a+wf^D9YP%oZzWIkS?`u@%&^=sXz&h6 z4WQgR$F-|;Ffn;Fks3liTw;br$WXp)51#f^+$f*Z!~sp4tV_iKC~IT|vs%w0B-zm{ z*A`{TO4kyLngl`GdoLOey>;EfFlAwm%1S*xqnZ@tI!2dqHDvr750j|uwKGD$dP(+jmMabm+j4dtYykjJVj)+d$SSM2Kq=u zZqm@!bFP;pw3?BC%j)!F>INtv``I5!FDN=irO>){-5sX`5PoS_MZo#4wh$8eS^u1l~NS-EPRx zlvl%~rKPewPZ3bm_qpv%OTjMcx$hNE7)SY@P4hjG?V>QV;zE1nuRWl+fNgW&)S)r> zD#q+AT;KQc5oXd6wojZy)D-cwZ&`LkfI#QM5dA?>Ou^%H&sz--v5L@BmTI3k+HbxY65GU&+G(%LhkG;oA;Mok`$M01ak_uy48tSe~t?UTp$0Pbo2TGgI$cFD^+1z<3n%@sah z!1&&Ycq0&7&O`h$)1a{5PxNZoOaEf1L=8S%Js}()r#X&J9__XfW$K7{zatrJsw*kr6eAMRqh=!_ z@Y)dIw9C1LL{1fEfPv<*Tq!Y=6_6@{HVC9DM#2u>{Nc! zRs^T-|JcUt5I&l1CmSTL&PV@p^)eoNf6KR;D&dCyg>Lh%uK6WLHKWUuWvR`;mj44PA0O=B)Uu$ABXwdl>@ zW=V#S7{O7s4ZZ$q-`fS|h1ROqL-`n_ zQHV2x#o)`nh4)R;{Xg5{r3E5ww3my71>M>~Kij*Xy_<08DIeD(r?h2(ml)%Yn=a2u z75xMh*G%`cyM<*-NT0dB;q0HoXx14UYP@o-di@fPBW0%*+?-}q>I{quV;V)0_uX2R*yGBi$@u&@FUCm}@M~#UTncQl9KDwp!(G@5 z2azFkL*+NWGu?sF5UUJG(k0k)r~LiJ$VTozBe9W@PbnehHf6@dpN6hqhKN6JDxLCT zl>fE-Yxp6#n#4p{l!7%YLq%+nxS9JOGZ3XL6u`W%x1|M46EQ4B_s~jiIOG!U>6$6S z^9ShyjY+Eo(~8;w#y{EKJVRW6(i&ft$iiS~8_z6G_ZIHMqnpid-Rkz`IPO^@2HHv6 z(;s|cD$J!& zRK#fH+ee?X?%rQlS?|0sBnJn5#@r#A1R@VH{DV>CKfpoPF^nuO3+%vQtApZQnk~bU zFnBGF?pa>8-g&gH!k7Z@<9%c@mLce}7sP7t?&J}!*%b_eN*9WR@I#&6CONf8it?|YnQ>nb4=xrbMC7>mZ;=#x#sox&tdCXnaJ#s9}vyf@83_&3edv+Ys`BBx|gf5fV!02!A z6?-P%X*%HH-7gA}GqEt3%t{Rf@2#jQ{Ael^F`}>@u3dv&shUR+KWTeCJcwpOXA^f0 z=c~>CA6@Sm*5tx$fr_Al^xnII6h(Rm1q4(Ok=~IGp?3&Hl-?1P9uyQ%dhZ~;6X}pp zq}L?W1PJB6>~HUL&;9nf|0FRGcrr6<)~s1`nQ-L_PwSbAA6_^07HcQ>&DWQeLlO=7 zl~mejGRi8lHsv1#r1)r>!sJ?NMb<}C-WEnhl(3KG(~@Rx|6)DwwL0hheA9KIGM zL{kfwy_K!r5npwrn*8yzf;E0f+5_H5Q_^t3vSWE)Y&7z@rQ(X>6*4`OO#t3@ZXoBY zTygwRh^=T!Mp`#kK1;=_H3VC#1%7j6CzM-L*%*}cJW8CKNzG>Y);{p*^%V8P1u|bG z-s+;^Drde!OTu|m%q|qkB5Sc&6&S0`Ty?*1Bk5OOkXHfy>mavn9;eP9+>}!{uRVZn zm8&vw{5@V8V#hgmxj!ZnZ01OUQ^oTSdGQZ`09v{KA3FqqxMjl2tz)ukJE5l!++#Ht z@8zvfzkYq%BH)J*AN^}KlOcK~9A7>n?aooTe>eg960%9#YaYkq@A0&~jC;M5?$sCb ziVjlvr-odz$rbBq{n8j795ijz7#LwtQM4ewS^`@*q{b;ZvQ}r3IegD#P3?6WDrCStu;o>@b5~ux>X>!*TI4vn~B> zfl`mOUm%%se8|K6*4h(Yi zjk~5m7GckLg^5&=u~>%m*>P-2?@kyUI+o=2xV9+U_hvZwVNFu<8VhW}b_WkIbUa-}L(oX*F#Si#kS+lqqiO zdtNH%i5acPEX-c!`Xzv)WnI2J&#Y5u&v2GewBSFklsohL>h4@68DV4cDM#9o|Hs>G zNSxQur^9!mgh@G+uCJzuNDaR9f9@SvT^RORoChq%xltdlYqYsPjsQlSg5*xY)E z>LZHFos;mp$DVQzT8r-@ir;C&Ds7k_JSzDXlfKwVDnA#r|69?i$*In(yLUWeuxkn$ zV1qp?sjJlp-nDds-(j%>%U+!#ev7yKa01f1*6>kiYHihBMrog+TOfY+io&_oE1f;q z){5CB+68_{D)H<4=@Myheape8vMegTg{#AyI!rLS3Xv-iEcxxK@d6D($)B#)4hNt2 zrqlGKU!Tkj@(4upV}P_fkJY%E@D5%gw4H0X zX2b;ow_a*3LTeYmqNHKJWvlLr9Q~l#s6p8e1xpybAc)izo{Ra{U7x2XncWKCv$qUTlB1BVE z1K&F;+Z*-+(ROu-s0+r{Wu^r=0`Fd=qIV`?6>=9L3|Q}}l^K5WocY*VfcK! zGwI0e(mmIQcItfy%HRLyrYCsoXWHQm6P&7(QOwb<3E2oLhm{1b3+LEh|LUDP<3nW4 zExJ1|Cf=_VVqcZjDkH4cwOZx?V^2$}p_RTHWN};m8{Q(6;}=s5T1P~U&8I3Ac9??& zxi@}tLVY_IOv~w!N;|alV>4JVHo}(XJvh2H?9}S;VNpWnI`}U~!K3klVH@Bsnj2*A z?q5;#@kw{JVyNqNYicxjty`!vwvv;J`{iw$4JcxFcImZ&sh#UhfEFjTEs=QhcYR2E zh=cO!YFuxtLL@@}O{W0i>3MBlf%7l(io?ho#M0}d7OoGjd)xAN`1Q)nSd_fr2eo4R zjV*e{LRVo*jTh3o<<{Nyx>npPnpGq&0ZFb4cQ=bAL@AIj;0tM-Z|Uv^J!_uI^nl)b z@(e<4LudQIx1=;Da0H!0e_O)s^6Fx+tr{A*%86)n32vHq-6>&2Nu2FrSe3PVyBQ;9-ioD=*tg+%RpqBb$eS z`S~6G62;34vA^(hVi)%_?P^Nb#GR7KX3TI>;)_>+Z%P`aatiWIEaW)XnSetmQ3u`= zgaXa37E0dPntNs?CTW?PNc50Yw4`{HprMO^>253eaDOeg)0Y$JA^pcb=wQDoSTY~+n*3b>G6)q-8-xSu39)NJ&gXh#YUx66x@a^15Iaj z)und(O5T|qs$+Q~d!9x50VL%qKYF$x+oywoa++84~8tcK&HH)*s zSLkS2j3acxYZ`rBa<+^09y6s61)_lI%uAilJJ~pp{oD)}+t_e=dSaO(l404FMd>OV zw?qVcw?}_AW&y6-x@*0pCV$dSD(1(TkKG?pf*5FeEheb3R<>}*)HnwJ@+z-baZ?iJ zEUwajDdTHrv6P+33vR1hG%p5k@CA+tQ@5X`mzW9ERn}BtBhZ$;_ok0G?m%Z$OEE(V z%@@DoX@|NiQ>HK0)(0QuX{r#=xB#|L0J!1JXSS$+l=IO3x;+@KfW53!2NrtHYYH@b zJIsJp`@e+}ym-xvJVnhnevXMS2PXf}@sv42hVq!T#x~5$Jt2C9cZVATc420fiU$y}OW zI5w;6JwNHCTm*9wh1d_YNTcTBrMJGJYs!M>vrnUC(P@!-8f*y)DhYLWCu+Q~YTY6@ z@UBzIyw-Jq%g|I|7F`+Gki#^fmucJTMX6&IjHXS&PT)?)1v2Fne2=TmK2CToW}#`c zc;zz|<#LDF^LI5iL2^9}UFy>UvS9?MNh^j#zeQr~n>^}ON{v}2Zm)c5_BLrb^+hiX2=OrT~C)S`Vyyr{SS z%E8%^NTKw6*H#_`^`ENh@V{^KIMvjDYvT9|a(E_llu2=gpQ09^J8Nsvc>5Cm0?H|V zwry$#GKcAiF^xcnTycq}fGg~xi@;8rQ0O~CT5rSJV&=<()VgUFZ+#=KaJ{prZC=Q6 zv*ZdyUy$fs%Om4MJ6V5hbT4~MRBSfwoli_kd2E?wc9&!Hn3}~)Z*V8;g{H*7|8($v z^TH~>P7)4GEXU!Q)+`pwA2yuftUV#?V3)QbG=a4Dd<+=z%=Q=ihBv(6+~6!3ir?eS zm{a9Nim50RdzR1fIkmUl=;}eVKWHP4Pe{EoKnC}_W7#2*F2`ZQJ(v9rlZHT`dKlCm z{&m`AND}ANCo|Kz@yBzMXgQ;S>-yo*+WfNd>2;v2{a2s}-4xmC!@nkx{&&y(*S{T^ zulaYWxmmvde)1b0M-_=*`GIoitXvLBqWFJaJH&XiR1>g_q+BVA{Ie#%i~b6_InXYim*!i!uR1V(0^@x4N!#!`f%`*>qd$ zJf98I$i^ggcYH(&QGHDq)fge0tC&|Ky=?I*IY-^@ondCs7V^th4aL(l zMXzayMg1jRrfsy4bH@Ny6Nd!(V@Ci0x8ILoJxoKV>X}SJ&R4my6&q2bljWS`kZa`u zbKub^;%uiZ#}Qm-hOI#t6l9AD$G0rxiBDtbe3y63nOz!?jo!bd7bWvHR!HQVTM*Ub zT(Qq{4W+w?RKR_qjZ&{Ni|RMDw`B-I6t zve>V@0;z_HP6dLrle^3YEC9#j`AI#Ebmp{1u44QcMsA7ujq{O#nw~vT2c269cw4eI!i%hKcxL54+or|TjI28*s4BFj(3k@~^_}|n&0FoXDU#kQzONmwfb?O4xFPw63i7_TPGuFZA{150)_8#*#OiSB33+ z$An-y8k>K#0P0reAKgwDFt8vM*7=~lge!@>Ndzl)2JJ&@DslhhxNGffA|K>)4M8sY zx$EFAZL;dQm>gFHX~&$v?OZ25tGf43sa3!(;k}w)l0r@EZr^UtoQFzoWTAV3zCw7t z208Y!pmnF2`f`NP*-oY9FdI?(NBvKk|6~y?Vgb{N&gLs~Ec3_0ZI2F1Tmp+O2FlI{ zx-dyk!;N=Z2<8Qw(zssqm})xmO;?Z{sRcNmzg<8N1^VxBT?DQ$I(1sI!cE=h+z_mn zORV|x7w>^~!^B}44_=w;6_@^g6O#cdwO3*GnuzuCyA+0uK9XOFB|yrK6H@_0_E`__ z7U`AqB4FmpeA$W_rdtFj$Tc!ITwPKwVu2-8Jm~iL6dOO@_54}iFl*s@zSs|@ zM|p5LATI1izA4Vw7i7b5No9Tm4L_?;nehy$tOdqXkTnLuG;gRcvKF7TS*VgthI|$M zCG1AO=9rg^AtDxGt!zFV=^XV_fPcI~(c`iYE~JCUYdE1YRVKlM50~6o-1^uNu^!Ou zpn0U;!sS?Irf6qOtV&|p`TW#vXo^MOLfrA#RC(Wn4kM53m?6M-2=v6tF7=|>O0TQu zG19W7O2)mWW*AAkaG5xHv0(bA+ijvwTmuZ5sO7y4g6VeS@$_oOzGZ9n`c_DrtL zLu9g}ylWzctZsIH(tVIB3$i6Ac(?{(e=kk-ao{kW{no+jkh%bQE0U>iiq9@};y;H) zPv^FR^Xcg5KNpRR3xixCO>2!iu&1bi0!9!t`0^0GW})_#<4FBB0i%m|XxP>Hrb?Y+ zynIl&n*vpzo~fTTHqPvF?FV|&HIMt|(UfVgW$%{PHgNOf62$bvQDXn3wyveb#J&Ii zxS=V!i@Vk(C-`#0C+VG+*5VbqzSqf$iaxy=JCAba%Q-Hty<(3TN^7UqGZxeeb?PoU zu%)ldVEx0mDLBQ;;l~`t?HZ@FSg)Lssc6wy7nHf~1s8+YXRVI91OPGA%j}ai`{Le+ za(ujjBFgw;1LdayUP3l$9BoW;WTM1gA$-4Sh}!I}y8u&T&PA*#c5G6wPX0*}FgRP` z&Y8RHji%OmsH=X@5dtbEDw%X|n@~$Hm@;0~DhW$UlPvm5Auv_kIi(C$&F@h0Aha}E z0em1#NxrBusb_^|A{#yhQiQECXxY7??t<<2vMzoTQ^}F_ftLhs4G9;RjiAlSoH;uk zhFPrJCT(4rrucjN+furbO?3h#rn2O4264>X_*B4A9zvGZ_&2iU>ayPe0etM8fIrhB z?C`UI3l!!xUAw{pu15I4oeltn8^E-jjpx*&OucllnMN*7W{vwj-dGc#upKfJUle!| z9xrpy_v0{Ue)^{^^Xj)_w+GmTpltf(83KP>CO=bkRe!&Y^v5$EObq(I*un)%j9Qrq zYRHLhH)u5pv4BMqP9Ob;T-Y(Hb!y{5i@y|G1^jXP{-_6pD0PfGvgQ!W??T4x^?XXXG#!Wv`HV5CxLNRaVFGljQo_ z`|0qG8?Rn5`&9qO`n^OYT9cRX7Z|7wwIaa&rw6eA=`OY#u-U;U4MqFeg(nTRj|;BvcH|8`zd{YCa`5$E?yV4KCBZ?_GT$nf7V?HZQ-(!Jg? zE(wP)|2HG>%MMSdGj7N+57T_Ehy*Tm+s_EX#`gYVn`a#GUMMpQNm0Ew8Z9$s)v1?U zHF(*7w@#Tnc-d&)2~HhSvtSt(p>umI^{%Mbt}dIQ^+$qnuU8MY(V^$E6%Xzjlsxtx z_XzeL7DZGi1kc4;l4GMceGG|yuo*yTLsq$}KIS0G6&cNHe$MA}kw3R-zSzuzqpp@% zU(HQY%KIafmfIka^TPTSJMr(8d$s>;<|R0rU2N+W#Mz~0dpk@=rAGxd>%SLm1pHp% zICq$?b1mx4*-?S~e7w|lA^r4-9Cp`x^Af8;=Vm9Lgm2JVXCpF}K|OBkQi)NfX^C4H zbkW!?Ua43<`MRt%+{Wx}XLj1J0Tb+HS`lgv8qPpd0d~O}yAL(M%Qm*mR|m^6aJP^j z!~4A~@P)I*E9`{63~r>|g5A&`f_Nup70z_^=|`tibuhTxUI@KRsqZ2db_luj40il< zUKuVxormu@PI-~1Do;UTy9hE%(!@j3dPDaiTLfuUelwzPD7gm-do9!hn}Y-wtxA)~ z$%IteTc&1BvtgcyF97*fH|PFRza5a`c)+_|g28R;&qQ&0M`O4XqC{hpYQuNu>vJq* zI~ZmSUmEPT_%xj?#}p7z1M$RKznP2X4nE)(J_+3Z!0L#VUE`r#v|U*cnL;fDfw9ZL zIOq`;HVr$=x!^aZYCT}NwlFbPheFqEI;*U_q=ZFGBfxfSuAMm`S|A)W#%=+JMoXdvM3U- zE=5vv8XQr;SUO|ytCPsw%p9xWZr`OWHv?{*lo)24KMq|dy^{sF>DoplU!2O5iW3_% ztgbKe+fN8yO@(s)?LQqdOA)CiWpn7!<=ETxe_eighi3!$oZrd&*K!8UgOG;DyNPC| zaY0Mkmu`Bo(kC?Gw;0%%#DIAG$njcin~XQ9;;ByIcdHU}P~prQ_+AeON7qlkFlO8g zD^=EoESZY@YRg-j)HLYf83+Bhz3*~G1USG7uBc0hc^L;U1`oJqAcqbUsJ^;rfHpLE z^w8T@bH+so67hG9|7WA9HO+TKM9S&yDT)Mg5){z{8f!qj?c zbeWO|{(L9bNIN~#HNhUiBxD&64ek+!t=VP?zJ0&c!Y-$uhq_w4f|E$Cr+K3;Fi>-J zWok=t;2NJ<`u@c?dL=G}DQ-?>LrPx3guKmkFQxs>^ksb<5Z6=}tb)fye!p9wR#>3X z?smanD7EAe>XI;Pof_qD4q}ULInFq5>&X_cl}6fO08*}Q9?11^`+7j9=Twsh)6Zkr zeYx|2yYaJ(%4j({^mtI1Nx!u@aP=_9#Wve(Dt3R~wf&aU2BI#fPyEyzu|3Jt44l$~ z+rFNgi)m3^jdgQCB}+b=d+fZ5`jo+1eAN?lM7b7UC&pkTCg#t@p0)_H{r<@2)WK3{ zrgLrMR}(V&nL*itZB8wq*>b0G2RK6u;vdLabnI&b!oizyFH=4^Y27-vk>Mfi2M2=A zfP@}#xz7sMGCgFdQ|1SEP{LB7k_8t7KrNoUj~P)nV|c|Fv>)Srs1Xc%bUL61`=n<& z3wAij*j3-i@pmxF_j3Um|F~5ZgueVL`UJu}X?|aLrh^GH66&77>|yg}QgWw$9{}XyVyL25pO7ZhqWHjI8BtdzU^)!bWA^gYOhC?OanQHCq;^ zWQy;5Lw9RzF@!Ex?SZ|*BOQUowi$0%+L1L7dSmVJh%6WIc?o@_Q`OmFrZ@?r%%bue zxKwsU2a2^Z5o7R*P0+~X0gMHsdciU`Gdly`K7t!Z+EB(;j}r7rYAuAy4glV`d9PQ5 zRsU$VHEQ9n(H20)#IpQ_zCI)aI0Jn)!tTE=gaY;FNfOXPMSt=Aug$r?2!^5mp5y=Z z=wDUK5r1f^T=6nQ?^2wH|9yMp+lT<=z$&=odXMdYu%g3X+XKl}fr?eO1DMt|CP&6bn?pidYLS(x)zc#EjBpb%0rj$P3b19A?(mWXsBXv|c@cZ&dbEr|1G z;3BnNwT7vCRFl8^g2c(64+XKNuUCV5nbq;SdI5! z2$)Tr2R}aq?M#>Dcmpzi-*)AtW988bAOn6op4?ZE7E%YrVl`BFjVmQ!|_D5M$w6V$jVl}d;*7!tu>-`?_TA$_>=H%v} z%RJVa0z{KbA8Hd21qb^s_|5+G8lDVtT?|B6vSRg{mZSN`yuBCPdcC`W6F64~jBT25 zql)g$IXzLmk7$mH;MzMn>Qdzkn++4vFRg4J!wWL_?OKoA$%dI%O+iCdog|1UU`Ymn*hXoc>2|W1g}s zuU>-kfj>u!8k->jJ5{sIN}^b{c$0>AO@N>D{%)0GJL(y&$pm`VYkDi3Ex#p#C8$38 zY$JH#QPXZEaxW!_aoGT5A>&4ec=7%b2XGNHL7&7bx*+tN2NZohM~6*AnHWKO*}*~Z2gum2 z;(qDiNlrae%o*D z(|Awqg(b`QC9*0f`{WVDW3byFf}@(X7cH0}jom&+_3oPTT}K&fQK~YG`>(q*c~TA- zs<{0Z!`-so>ndx0(7B$$?Bei#FVO1{!N~QS3BdtPs8%dY*g8-;$A{U^N_yR^ER!O0 z$M42QxG1k7E8=Ai6w=9H+q>Oub1wt4JlfSA2#jiMTaUg6(xCkHC@tA8903Lq!3(z8 z^Fxp4EVQ1i2>=VOl%3Mbf_1>khjM59OM#YZ_yQmhpMTW6*HMe8g}*{DQg`B@h9tHd zG?WY7tcm&kSDxgBzCbGNI<~3==AKd`S28|)@_(ymrFD5V2^6(Xy^`(HNwa`IxCjFV zd7)P&SD%i@_WSD*Rkdh6^^9iN+YP5#?brw_nW=j(FOl6(wO5KZn7_wd*vDRjl|K7d791gBDk#GKtD4^4kqHPlUT(a)wy*|)Wc;O&l?{jRCXS4g69vOaQHam4qIy`yxMD4LyukG5vC;O^5D*k@*MjuF+fsxJF_dmqUHv!{s znX|*-))*pMvg^3`(!Ko8YQz`CcU;Uwr2ogrNB`O(r($m^mh)etfkyyfgTGGNT!sJo zsv@|XNUAPP7kkY(JeJ2N654)}Oqi~qBN5-xtD2}o{_B~=b1KqYe6qz3M9Wu$)NU}i zDZYXC2p#IB7zWclq9sUIA^5Ht`e97t*IFN-05A!@I7X^EpLNi}t`7fv7{7IyUjpEw z!ocMiv)z&9*(TJ2oDa;t;M({p{+zQ8B3fdn-=jj72$WHGP{`Iw(E)MI8yG< zJ6xEG{MqQ%L#3NaR5$BtHLU^%cnXH-+980HwH5n;RSg@2?Uk#&m=B|Ug>PytmxMoN zY1_R`J<1#3M>*g>B-=?XkW+4|bkm~un<8FoPW{=o_j=Cd7LhZslyL=JNdGmfe-V7H zc<&`={$piDPV0LEm&ui(R0chehw|7``lbpLmc`nvgRC<4`|jOSc$|rnvx0?1A<<AmebD;LdM?qaV5k-&|-=I!Gxk zDoxhO=UIhvhN&r3>nt~e`6^nQZM$V(Ro_2z6c^i3iyuPDHcEV?>!e{0mh3SQe2og@ zX#0eJ8Qh~TxHZOZ)*6s67^r|j7S2^#D-3-=NSqb zOV(c*&YF~=6+-cJ$9}}ZFyzeleIi5 z2}$PG;E<7Fxg0->KDEq^zOs=vQDsmdp^~R6M!C1HK2Pk!$Hk?HF zkb58_n{{~kbJCAY8QoKU2N7?{pyzX9rN&V`is5Oc1=(8pY`+#4#kKAO#-5^<+{i&$ ztzjom3@6_qzE}2-fFH8+4cgh9(mOhJHfw%)qE_6_KUcmjhCPpGuA&mIqIF|=jrr*8 zLSM61w3T=ciRrN@czAq~i@bF5L|1{8A4mJ$M%|&XoOpYT(HxKKqfP(C$+lM>d#`ZE zd+1&O^K$1ls`Kac^!~g{P_)*OL3Gm7J8{YDVL@_*f>oNm7bCE7Be_#FLe-$Ak>{Os zwlsUG$H_P$4?%oZoYcoZ-`rBuZ%~@wRC`BYhi?Ui%O8Z#&ZDC)rFzqP8q*v8fN;lw6sx)Hf?5)Z@J6o?p z4)3PppK5ct?a}1qk#a$EMQhde8hXQWZ4Lh=Z*s!tO;+cXW7A4{(NEM--6T9 z|7bqH=bmi`r?n(;(4k>w7?y5%lzD%K`fMQYpHwPsc8r5t zZX^i$`ts@1r!%!qM(-sf9ZcD#a+B=CV5Gto#jo~uUH{BGOu8bMpCx1rPzl7R6VlY1 zj=SlZ34;OK(U4G>kF)x%IN0fPVjLG zb5}riAx*2t>c@9KJSw^_AdrY#j8!Z0$ZGhh#sj(ho@8vvh z&9f}bd@lIvRLG4y5wp z`}$P1Dw-x2h1JD5%seT4U)$W2%6i_fy8O24&)Vl|f*HY0vYbwt&7?}r-Va=7M-!=) zhF&hrhBDD-I{FFPD=k|K7s$0cZ%|KH@a6FC$r2#v%$=p z9%evN$@cL;Oa56U)(V{7eZ2Dk-OpX`x)!as)4kw}o0O*lQkVM7BokSVgR2H+O5~yZ zjNXYTZf}QpZK_ErsoQWU4sd3&(hYwfor6=KZ7Z)KJ(AycmIq;+6PWxC*Pm%?Yj=R8 zp8j-TdcsAI^xPO>)-KW|22yZjaGcUYXE@Q*qa;q#7D*vnk};c+EVd_N!i{9#t&^HZ z^gq;7X#6nSC(a}YT?17gsMhgmJd-5FLEj-aBWF}= zdeqV6Oj+%9T&+#W?hFP^56zzrrUu&Lm>;eW+f5WEeSRCqAo|#0qA;{IdjHgYYP6dx z0yjq7MY-U4ay6p$0tPy<>A*u=%nf$^8cFZwC)p&r5v!Y$RPnwHs~(q`+w3p)9le|O zbTigkTk-C(Po69avI5A8+IL+9F1LUysTrcBmr=fUy~c_@`EM{;$RH4!VP8b(S*6fA znKD-{%ElJoZ(+jYgMhNEyMYPQ-u?K(K#HC2%^?n$ShR;O*a%sc4bFnePX=hR6xqQ? z{0!2PBKK!xg^QXI@lWFjxKNywmJ?4W2`Med0o&N+&d2d&2syRRm7rC6adGYt8+~%DAsXI^3F%6q7 z;aQuqx9C5)TU-~b|4-!n^#zLy5I+09P*P0%znkU17 zYpoIFU<*R!RN&+#Q~5=cCd5dve3>Igzoe(&?py1XoAUBjR{%hl0vZ0G@bAwx^O0qQ zLY}R>%2u1s4-O~NhqcMOTlFtLL|0Qd?Q|tpx~3V5QQkbWkTMtVso!ywbfB7gV3tNW z@R=AMbkeSqJUYXVeygTN^jW|x-eY4#^48w*&a_JR!5VqsdA%NKLa|vwb%VQKsb+@O ziz6yuuk06r(drN;KzJ!F^!-)bg#6VmFDFV^TEmuBb}Y&zklJwotLAKE3kZqC{krcG zF-uLvsDFR)>G2PhgizTsO7MmZfV<4@B+ zKIu=#78&YterY)!*TCn|>M1fSF|1Jl+~Z};^oV&w1#gwGCJilTC#fzDuyr`L(Idzu zS@3IZ_O)nc;G)Dy)1f1-%G{sp7(c8A9Oy^hic>L8-B8l2n++O2hHeKH7~$Rsg0(VR zZ%>KF#R8lN<(qG+OW7)|i^*FFkDrxi)ZKA|>S7X3=JiJFyVkfPaIN{(9s=VRac8CO zp12w!`4R)l3*~rE6AuY8@HQ1mSJ$61&~zf7rE+r?Px9%-uwB-T*0^}UOW~p#LBsd@ zM$&v!Q|XpP$zu@3ZF1e6h|(R4jK}unS__UNU1t?wq>#{u*Xk;Bb1Rta+14?7>cZ2> z6QyIu-P15ECJ!`S8V)WJfR)tzqXke);z(=ob?fjFRwQa5p43)4>fa1{RE8so9N?1H z5TC^xJReak2i#F6#%K4 zygbm~og4QIr10SB703f;jh^L6b;*3(kIK-PS8VO#JBIKqmpHz72I~Adf(>jtWQ)@* zEK6Ah=NXkN&;8K#sQ-lCo(Sa^l<5}B=PoX0jl-XR zDun+BBmPfpK6%6W3wV)JPkr0}JoXQw47u~zaD#TqN4cZ#1+D3_TRJIk$K&rj zW;ed|7hsel&}Z<{Ps8;UDO>*@bHs4JN#VJsatft_q#00k3Op5DHQSg&s^@Iy$2W#U z7Y@ez!}KTgNKzKJ?RQ$!fTDL}7)XwI$IhSYUganw&yH*O+>ZXV7n+tqb~2hEfzzd5 z9ja_T^Z6Wqd+z%Iua|>hwBXb82&t<|I}Ndv$A-zu9^Bi<)(=5b7EKkF6$)wY^<@R> z*$gU|s4L?Rh2?LxUnz49k5|sTw2O`DM+-E-6o@YSxuV>I3O?q zH(LhBvsCJQhSABIaBU4$`~#|o(EyiMR(rV7|AG46$JA*a0*^^6n|D@qW(>#;460YU z#0V@pU8BQStqYuzek7?J73l0!wmHRuz+rYNihNtGqLkC+JACrmRI(Mn6UPv|eC0i) zq8(CK8k?kVG@slE{?xe8eeuTTo00hYFy0<9ibkVux_JhCqOQ?A70s^-w>s(h3@Y!f z52ddJn#8v#%sf)lP(#jQMsj+?6eC>O4TQR6u>50OFT+!>&Ir?mt$qpjCu3V76BjQ< z^$8}|M)AjS<>i@Le4suwBzH)h=c5yd(Z)^NlcPK`bM)La0;aLAqPKMP5iq;aHhBmb zDOqGf8En?-fq6s%kqV348V3%m{)~j#W3~z&33L1K<7T@|i1HbVntLCvs~3EwqL=IL z?q>H`AHqM|5%|^;2(M~ueVmFZHIA7V)&lmacAOnMY%G=k`GYi0ea`axxwx>^9;AqfJ`p`92@SNKR(i2@ zx_bfF&vMfw@d2X|Dfblqt}z#DwFw<#R@Rib=+wx_4vm(LufJb3G` zHZdXg-FM9hvMfV_ais*5-k+aEDq=f{w^DRdw3qUD6}}z(JTU`K>fSQf?ukz{UguKs z$Sp48pFEe+y2(f59h#>d;c&y83vjmFk_ckz9!#A^n*)Y5`0PDMx!!vk@uaZY?l3}H zHL#30$4%546h)yjeiHV0Mz53R(Er#&pH6vAJ1&wpvfl=Y1Bx2It60 zY`C)N)jIm_OzQ2g{Lqr{xon<3(NIxtR9ds)V(v(L#{EJYG*?4N-Q+#V==FEe{QP_O z*y9QloF)sR$uBXqmM~izOzoX$4WuMe#^VFLy)3?iUo^dvDHgsVxE-h8b|95BU7q=n zpgVi#O|o~1e#lVp6(Zx#iw?YYwI?9GdjY$2%cZ8zMx?x|oy33WleYi{zxpmV>%aj9 z67jaS954y3nJ*3?Vj?XyYl{a|z{K`tCgac~t3vlb6Fl0;h4~_v51HaBvJ{()^tfBG z&sQ%jwO>_RjpJ|a&Q^U1rWG*l0^VUZ#}%zv0C_qjm_@*}Ek5;dBjSqD<=#-{C~%-b zk+X`_Ycajs)r)_EG4EWt!g%)?FUYE%y%&V>Z*$wBd1TfU-d1I;X#7*VFI!sty0*4V z4Ji$C)Xig4{GlX7U9B%uU)MP3bfyWquc}TI8;oNc>BQpda zHvG&RMtODq!My|jB|8}v6&{7zXI`DV{b2@C;bX^?+pjVuWB;4_YQy&ULioUQ-@ad6=ULCTYZ1k^474;uFa0dN8Ewmrc&j*cca-{cE7weJo z*bi^}_6tt%-yS4R>kd9<*#N2OZ5TDD3JTAE9Mn&j)igbpjTf+?7ZPa7*A?fYO9cX4!%AzTMF#aB z{`E6YrQ!kaU2JZi({bR=R7u~f@EcvKqV>P&iR~a`uWr+H$#)Aw@8Ju3aY^dlH22E3 z!>d)fI;W)O)!t(;D%CS>KU?agTz?X&H%w%)DmT#K^wWfrHNMm|?rw>peFNHM{C1qG zuYiPMw~3#lb7!1allM{;R9?l=u~>yM4_^E1SVL|QS=X)gs9qL0jRv31^EHj?PnERb5b$?L<_pa6H5tcm?(d0%Q(WY)AS$vpi_XLO`v z#=YVGR$;ilV_`hgU^+C?H*MO#7*9NGKT_s$@ zmP_LRBTkSf0;VHx>o7#ya1)n2RkCV4>%--%FFp6FY$B@dLNpWG}GoJ zvqsG~`9SS0Qh0TE^@uaDn$4QAlP!7+RjmwB{Z26zefc68hutUpD}oLa4<1Shiz!wLwXF&+o9#|$QHlxo}aGPZX-Igm8E8@HRcqDK_^Rh0u(TLSXK7+ zcDnIKQH4#@M>(CVr{4r&g44-yrm<@IyJ=qf>5MV$&2ybmdJF!umR&B zGX3iC+5YH;dI4s+SeO+r_kFSL`UK*>XF%EA<9K2# zcUEWFOR}M1;?FcBxf)cVc}KvC?-XR@uJ6kV1b0QfRTj`vsv&E0TGK3{)gM7 zFSG^3onWgXQPs)CESGHCiE%Qkm}*ax)}C-y??Mlj|FqY>AMw-ZamNCaE*ILP{?$4E zMFXoE;>{Ft+7HuWKL6`K?5X0*5RH=vr#tRzWP|16{UWAveQl9v-XBhE$ zDO_nEWn3Ze*EyRHWAvhmJ{jQF{YFv7Ht%)7rTN(3O8Mn>DJY*%jmxD;r97rQmK z^nECO|IPM9A4FDqyVfbD*r1B*$Y7nPE9W-o0JRw03%UhZqfq;KtSNkVUnBd-j_Xd# z^PXSQqYA8-IS#<bD0yRkow`THIxBLHfmX`LQgB?JC{{F?gz6R||mBE$8ap z412UXI-dq6rZda>Qvq?`j%vG#pEe6-{`&Q6pBfjRgAC4N&@jVMx?rd4W$Vc@p~eS_ z?iAbYflqW2w()wW1e9D~gi5R}zLctDCxvA7@?0Z)D{jU*U7!TRB1DQ zGr4Ne7~cIZ7G2CwDHrFYBY-L&_3cYM`Pp~Al9wQ-0Q~YM;c|_F{R%GRjD@$mtvYss zzND)NQOvks%IIF5aBg1Rx=Zy0uxAG5*HHgq^|=O*lVT(BVj#Q_&B^RNfMZ4yP@z#f zQs_6u_mji2BS=@Us5T$uXbi0Yhu8K5+hGJSUWN0-6OfRgi>H@>l_VFumj?N#b|o0J_A_l$xwvN;dSuoE?vTRQpvt z66jt`!^N!Kv-GPajc!*mh^YvJ?AFjLlbYV&8ZTJMJb)(R;Lr>GdDe)s9SoQ8JNSYI zqu3Fw!E`yo6ZK2ZgMLBPel}d=AAz|w33jTq8=w*05Iw{v7tjZsU{(CPtI!0W^NS@7 zFCI%)K?^*tS6PMGtppd?JfSu5oe%A^$fL7kd`2-M?5nIjYx}MZuMz4JkGJ{e6c`Kw zZd+eVIqrLPc`;i4Rx$PK#}Uu&0pkP)9U!mKWzeh6h$iP9=P^XSX5AD$-UaopD#*F? z?>uwv{mF9LIPVrIH@MDHnI1m!OVOiE%hKkY;WZ{bGFeKCrU-+F#Q4m{&|>s002GtP z+-nVtm|hd4MXf$4BTF3Lph?I~1BOz*vW(=at%1@IdckOQDM;o0gTh!T2jE1Dot>wA z-?P|H>aty+1f%no95;7^zjIh4L0Y;()$`u*7BSzWe<2wFM^zJ)YpRSv*!ij3brQO!S|TLU zu~fDXhuIEfC70gqR6w$P{BgJ8{dI zeI244$M99J%v3d9=zCE!{{fXozE6y4zcyQi0+xlC;mfnN0RkK5pNYC+FW4#yo5Rq0e7$m!;{HTGRLblIJ|m0qOJT6HhAzq-U?4esLaVN|Tzg8E+Bh#wequ$clWu-A{2PSV|~IXzMHPKNH!(DHcZ zBr(0x6Mn0nC$XM9kT8khNHuo$8IR@A@f4L*E)7U*!okKJ`psD5@;|M<`qaqrdO^91 zUEzBgIi@)}w%(LW!(<*o+il9Ukn5t*yqNP!6ezlJKSnEH(Lc*;R zn5DxftR5>cdFJ2Feb)Wt?IKh@Z#ytc=l@~rtb?-rw|1|9gwi10Eh6yHjWkG?ba!`` z2+}Di-AIRYmvnb`cXyw!d+&GNbI$(F_>Yd@jNH$<*Sgl{x|VR&P@VbIzzQ)$!=FM^S<1zT7o4`O7TG>wj35B?n!TOIG5+^ z@Z<#0q6B~W!3k&dYnYr!%mLYrY&Op?t}5bV2wK+3Ku~n`%EEuTrT3-sR7|fjD8dAe z9FP2KhgPqZsj+1Ue8r3=1daop4`xo~2DlEwG(DZ^&dyAFob&Adr)MI7AK{(YSrrBA zJdOd>YoO2kg8;hrx8|(P+DZJoNc~>|Osk?;EA}`tw@yU4V%ireov>dMqk<$46Vk6y z;{SfD7aT1(lCl*siiXS0G@@&^_?beHUqxkQ**pr$HGSzSbEm(}sPXWsvA#shw3QD$ zdbfQR6DPu_yzuKPvP*hRl3P}kooPS4x~RjwU_^rz@5Jca9!|f$0wG-1of8!yS*fTT zyLk_4SQ=AjVRcXd(&)i@q}|;xzkI%xM#oaU5l7}+Mh4F3T;%W3II38$`lJlcF__z7 zIAFW`+o%n zZLdq^ZGI-2{!gBUqTLZ0`h}m>EBxC0;MQ9s2!omE-znvx2K8##G<^fy2B~s^(vl_& z#@+5xKcuY8l-Zr7nvf@+hKyy4b-jyJOH)(OjZ`#8WrFv8_ZuSyS$x*l&|Q(&d);4N zLXVt0Uh}3|mTrBYCOU+h(PRZVE19)bY#0SaI{D>+1&J@gyOq%NQaToTXpsqpIi=t#tY0|d9Ru=-KSBuqU?P76r?h0A#|D+w<*tywK8w0Oy@AT*r-mhJrWr-LGpP6WK=n zj0@>rOcdthIpKWBIGs>t3mBQ1jP_GSH)oxp*xlemAHBBL3FAP=&wqi-ja4yKbc(mM z+8!89uT_8Jrc-8$HWyB4@dg^3& zAMs2~e4f3(=JGgtK<>w~&sgRC(wKaB#6j`Pt$ zpT0iL)qZIU*R@WjYAurLXF#@8Tg~sO4Wvby^sb0^NR|cIZ+DewHOPVcuvT^N47yg@ zEoYOdB%KG`(4jxrlxX72HgXCAt!O1XwQeT_?;Z-;?q-hFWZA&L0AsvTC1J8Xa6Mc_ z`!pgSSaZzeR<$MN<>gI*FFJERmqV$>TI=|1?@kE`JI~*@7b>MDUsk0=vuv^S>T-{? z(R}j8PDEIk!yw?_9lzuPnS@r++gGs;SF6x)auu?~n(z_kr_~K>OZZ>Cptz61Tt<%-Jg;S@Vgpgh^@%uI^{KC zY>8dCNOMLf1d7K7es`sTex!fqw_kPnCPNa1p+pig}bJ^EjF1zkO zmFNE|B7c`D&&YttxpAp=DEzOc$CpC3wWK#d%MXcL=jYpz4F>hoz$ppucc(8v*X8;G z1~Ht=e8LuC#3|EJtKqXT!RP6G8S>S{g;X`(OrFgeberCFvW!&Psb5@0VQj)Jj$~d9 z`n)Oc0v4So-Jk82AEa1LlGCX0~O-KJ)EW<(r^Ybib-3ZNGmxPT$^^2$}8sahm)ILwa|3QId?D~1YTr-pl6{8ox zmj6dKczBrII^8akh{J33s}Z*`ZrvD=Wz&X&(LfBW_=!*n#0!)^`6`}xsDv%wij zMKkXWfziWogMV^LY?22FE0%HVi6dcF3`1m6WQ?b(q1f6#-*y`Cg?hMtcCR@3WGg$GF<*GGR_MRfekv_xh!)!`*K4c06vi>ZQYKNAQj8*1%Ai%icI#Xv_Ujp?E*7@<>XUguff+rraPd zCUfn2cVw3O587(|9$?ZA2P=2Aqd+|60^L|f8bztZD+2zAks$c-Rki}Tm&9HV5N+>! ztL&^N?bV^w^?75%>Khg`YnG=y>=3e)%JxhF}z;~kfk>Gyw$ig(pXwF$?jORR8H(rOa z#udo_AmH#u8|LXl7P9-Y5vBogjTyR~c6`W;+e*he8|)Fvzb7J!4^W?v#b&aB&O`kV zI?rE70u2{1M?e-v*T2z3giLywBkl_LjEYpeISxeZ0GzQ&0L9K$dB(g#t!Mt9iM63zgCbLpCug4zObt zyDn}1^L4ZTFwm}&NUxU2WPW$KwQ+?&V`SjU~uMJd=EOp-%dlw7!Hp`zM) zQWZCTDeADx^5OQZ-~Wd4itN#mgurIz6Sr*bNUS57xrhw>F0IO;1PARdDG$t3;ML@0e3bLqiJtUlQwJsxhqnc(22i zbPyRT^fu%$SPF1lg3nkVk`LK&*%0?#j;{*xm1C%OK4*O0ce*{GJ$^rTZY@+e9>*}B zRCsqOUnYu_TQ&*=>r>EVlG#tu(P=bHxt8>Z3oqbP#^mtvdg%Y`Z8G0&^`#L%m54~| zyK6|{WZnWukY1Wk39sUa=iy?*baJIRp5C@sV2NnTt^Z87=(g)(-Bi=|`Y==9o!w9_ zV-CjS?|K#dl->=GQq90#+@XEmbR+{>22HN5a)jW;rf3`kOqa#E>xSrP2^l3D?6=zn zfZTn8R}EgxbH3vteV6FaV+fe7r>Wi;1+LB@c~@h9uNZC}ka898fGSyVEwgRA@Im`Q zLLh0Qt4);fn%&59&CchcmgV0S#uY##+~K&)qM=(7uKa%-hW??DfHsB#tfY~(k^ZZp zc}WE@E{ZPQ_PujU#r^q|boROF|s6+c@OLLybIcN|_^P*Q|O8oqb%U#3!<`4V5;D;MeC;}*z z^{)|T?R6d>P+cx*?Wgi%b|&wQj+bs$2HBgWxdw#&CK4aAT@x8*8A&N%LQRH*bLF$7 z?l;r;cxmJ4!wGPizjm~%oo@(CJ={h^3`7SK7+*HAZuYAB!(0Pd+?ZM#>?AuEKb*7M;QdK2L| z5QSKD5RX;r+x&zsc+5`zo|pnY^C2IKfuY!U7!kz$M^<5LN2~0Lxod~m+P05W{?(tq zYFyXvR+ysw?Fc%J#%t~1OVlTc7@UTjPGy<)gS=K>ZM;*T_TN8*UUBB9*B8zFbU5-tvT+cbbM zhe#9#TJKq+?x5d{6MG}1q%D{X$YA9RWaYn2P0l4NcyT@^u}G+LladBWg2S2)TP$>9 zWS4Ym<+T93Dt6VvxCdj+4oifb;;i5|#*_h7iqVU?DvQyt{T2dTBwA8#_L|YYxiX}T~Fyu{ru>1}KoGq(mE z7Hv8nCxtG+b%*=Zu<0{_^V$)=C8eMx{_B`<^X6Q%6@r4OtNS1IPwdR+ec$1ZwSPc z&yH-$YIaSz(XdQE9)sw}Er*#ai>9>uXg=`u2lbVO&d};D(YRhSTV@BUp@XGxg~NG^ zeY44exAoDhk>*jPQ$J=*NJ($bmw02{k9id_ShqEi@cflbhFnv>Ev_e(xwGC4CY$l7 zIEsEBHuLozx(|&;lZ%t(6G*DIx};J&yK^YLeIJNoFfdaVFWJSfP;NAl!CgA*_Dsf} zo!EsiL>77UU^9&<)M0dK`F^k3fQ)9ifrkz4S_J>kLD-$~G|8xCm}sGDyDgFB8+;P5wNB#}a+>&$klu#G2u4@`V3IW@@uXP(|&3q7xyJ_j%(Z1f+LS)Ax4 zI?k}}kkB+I`tUfDq(vb=0-BTT2(MEcm%*#+{r8lI$o`2Ql+bj()9sEw1vOmy5BP9x?mP!?K^LghI>>(|bCSqTfCo$%QT6AJ{eJ)L|P zawX2SN^K-}?ceU3(H>}U#=fjNxaFm@+~CfN2i9LZMq* zfzVBNm+i?;`;7%!{Kkm((Fp<3oRRfc9mmG7q}JB^pshGZBS+j`>xm{OafrR)z{d$X zzdl+`aalU09S2(FLcI%$VjLZ$D6-=^S9;5Pep-h-&vy0Z3^@?Glelf%0*>7#!hN%k zlF^xC#Qn@r*lNMFPR497%x0O$&>F3!2662b!u{^Tb1~V*PyAc=2^vh~*C9GBhD>t6 zB4owi<4?jzB@Rm`(~jUah6{_9D__$wM zw!bqfJm}FX$)MMrEdi`2O@TMwAhz0cx+LYD5BkycdmW-$M})YiCw#F|oo<)?Dbl+h zG!xFmBwl%WRfm-?=euk5n3y<0RtDYm^uah}M{a%1RqBz3B`iKSwlIpEchpHnb^A|1 ziup_5$k8caBq%$Av8OGsGexfxX#FWWSudj#H~TWQ^A$eMRn}fi>u}V&Ff`9Maq~@$ z@hw%DM2^#SMTu+;kT6po&YhmOFI?`U>7g=EIUWS{RhheWlJ;phoWSCB~3W9f~NpKb;dpQ%O(mr}i|F%%YtCB=C1>4 z_$_O+MoJO_7+?t}ZVqjJt{joyaQh$n@^#`qOJ_yE`D(EpMLjbN@DiyQps<~~qE-w$ z{YzO+C4d4z1e@0hY7=|e9sNX%ke}F}e`=prc1c08k{#AO7yln$*Z)e>&s;C{7WAfd zg`UuhXBC;eisL@^=%lWpv_D_|Y9j>m+6o`Sjp2lV6)rBiYH+sWqf)0v^Hj!F=cjar zI}%|VLBpRob}4y6@O+cag-&}?QeplcpULD1B@62ZW^Txw6{lL?1wtQd#2fC`&(pDB zqB-r?)At^D5;l-saeJST^wn6J)xO^ewSL^6YRvX*v%bx(2k9)VS>@vqh)P?%2<&L&_X99tudOE=~s2Q1%3lkUx*$(ce6?4*Un>`V{-VO{-HMIK% zHG4r+3|4+S4-xoBnFXoQDQs)7?m%eff4e(b&NgXq|NQiWR&Oj@0%3Z=!pKJov4F+O z++>gEES63UUB~nE6`S>fDUBz!JbQ412$#zd!IIatR@(e0Dc@fPUFjqV7q#mWaY69H z>NrmRFmMK<+e1BfYoQ3zc%3@=ieQ&-hX=ERUfga&CH3(PpFKIQ<#E~DVyd}`?YK5W zNdAZZPr8eoDp`rQbp2~P3v~kb8N$KnB;=^9RV7;ZT7a5Bd3iUHuL!YS`It89yfe~v zE22fq4aPaETbo>|w*AWEg~w*cUGyG*3O2USNoiL*wO5DomF|}?%!p<>&tCUz*4>?? z$`J(OSU5prza;&pzRRn#TH-7rB0C>?B=R$jqTc^}U1uOsVLXT^{n>QM7znZ0y(c1X zJsa+^@G<>!(!2r==P%bPhTH$hm5GMKRD>FI+wLBy#qN!~EUU5JRfB=i)hyMfxf_!# z*(W=yn-=B?jeQqQr@R^!WEdcEOj1Mp!i6)@UAvHm3 zS&?I>A_JoXTo;*tE^m#vyw766=j(zEs+Ll<(VivpX&?{>0jL5&>=KOT_ z#{9hrkavC)KXFoHxQ_h`J9OQ(dA^waI&1P;BIOMbm@aXpG@pCL$&HlP9ZE~4{%t0$ zHS&8W8uo`7gw1SaAsRl|#vlS`u1v~*To@{&R5FCU_FI*jc5uHq=x<UQ#?onCjCto)me1 zvib<@ntV~jy&DXJ7nu>m5vVeUX=?mRY_wo6GiV~e1-@@H^VF+owLMQowqrKK5yc#lz|4>RB-mj?ag_bF{6?W-`Oanc>uz-P zTp1<;Lfb+RJdaqwn_>PmMq-xZi-y=9jz3ZtvfB*k>RoSY^NVi#4vc#LF-5u$}27?5~&b;C7~K9Y-(zJZDkDv-ebviVHRY3spL-!6i*1_oqAZ ze`$p&U`0GXjAweK_U&C5G68{5!Xv}C?z^3n`Lund z(P`$fH}2bDdU%1F1R?+Fb&k}mS_!{Rug!AMzlFs3_BmVT3Ap*U$IbleGK)y~CWz9Z-D>?p_pvqU0LFylN* zcWB9DsF{%8UG5c@)R8F?;vJV6a8&y68yBnjm9pEgrM(|qHX&=dfzpCSxuO;Ga9al^ zZc&G9Dp1!-_QRMAp&_$90wsz=<0MR0d}`hXMDzrbgtm)ysSdDIW=VPee03$OpJZvAhY^=}<2^aZ}tgT_Xk z)_-mP0ntekrRsp>eg;EX$iRf47)8kK+#1KAU*-nKpalNRrUnbbOqGWrCA@d%TBkPIp+%V|U|wvUTsT4qaDkmNhTszJH77%E)1<2Dq@t zWE#7Twy=l@ZjA-SlhUhP1tcJZI9-if?=93Z&lNzqrHGFSYu1?@s{4P>Q;+$|WYQHv zDir0biV&UaTF$!}*HI=GNp#6pIx?0i0hULUg*b5yexStDD%if6HFOZl_tZft_Db)6#VTrOvGY7A+DkFF-J<3+Y(3H_3_fxu4UL*9vKc6nJ6qD4?6nk z@xHsJ$@9#xl9)?rJ~HOL%Gzwplf702lNmazW=q64K3lqDAo3g+zz*Ipp`S1|zZ@|e zqo89$YcKp!Wj5Z`Ph#+Jb1Fmrf;HT9@^w{TtX;&awiM!PHk*j|(Z8S!O-%g&cY$HhQQRMamMy)o;R z=4+YxZ-dO^u`43EE7xCShEgduNgj+B#8p$obr}v zt9cr1Rxe&49T)6kqWaBZZRGF?iP_K`h48XFJr)`RP^zs@auM(S%UVSY$3up z&!e%FMG>)$JAe6L_h&YyN>rzw=|RMolWd*G=D1l91@9_`wFgr+(Zw~n^L{GoHAE@? z@S$j9sf{{DMv>6t?+JC`32K2U*8;Q*KYrX_3pXuat`c>qfkDfa2O+3sHKw;_boqSO z=KA9H#iQpOI+*>mn2!n*U`NZCcK?GkzV7w;ea%i?aQq*!eD`abuxP(1VY6)PLmkVH ztIKnPWsOSuwloP1<~4$Um6GmnFT9?6_I!m;MK2>IbL)uzp&bG(Rjbh7(GTU-dzHR_ zm<+EaNy@qi&*I{_oDWQA!aML>K^`lD1ppTYoV?^#FbDTZ-!ezYu zZk?;*C3^p?TR}NaBV7&)dUO8cXz=_&(1zhUe2PHukGfZv?SCrU9A%1y&A)yl98JHQ zFeBy4L00|2&Ip4$-ooGYeS`kIp0GF zE{_W*j+}UYFfbbv&{tx*B#D79ocPc5SdLrlH96CkB5N`spC?C3r>6gut-x@avyiS~I;V+gkX!*r>^#~SPS9EfmwQBqg9knQRpgmV&( zek%N<`n#*c&;(`^b!q4`SU7{uawFHRZ`hh{P61F#xu_=q?`GBU`u3C8-E}hMy@^cG zaxFwpGZXPrHb_n6U4;`ZF5Cyz7NSK8A;hXtK*jvkedp}H?<5G{Ix9qzi7=WB5y-Zq z1VY{dd&|w`w9aYT^F#OV{FN^+Jc+OZqC9UkjV8PbV;|Og z&{#~zVS$y@V&77!hW^gyEo=oSbF|f5x^a^`0u|vI!JBr(3f>*<29Kv3 zAqx_rdoM6@4pjrBbPsbcKlrLW`j;ezl-t0Z;_K;BMYwrq5I~Sem!Nv8}O3+{#CNqA-xi4-P+4u3ksXdiE>5vD;$R01+?V zuL70nX){MY2M)Q*aoMutj^O(g^!792Sh`?_UY<`5Eu@M+pkBQ;?f)C&?MG8mKt&j< z0Wqxz@v?^=&yg0Yv)`oaZZI7si#q+GLMIUeJKy|-_w;9^Ylm;GIr1@qkN4ALaUpAz z{66w)#11y5tudTv)W8B>559%EyA^0piEB*hy~WH|#-&>OPMg4D*xA?{^U@A$B;~#E zyGw#n=8%z;%gt#isk!9AyltnHvJ{46rwxx-0{(->XVlx>+#@=DBTEy*J<=p6_;K7t z+~uHeJfU3Y=AynmX(1Db@!*-Okqj0v2lGz-L_iNA4CyH{_Tth=QGE>M)c%9IX(3pQWsj@+ugz=~YmAzXkXSW8hGLWZEijOmvHkdbT~!IE0;C0J zt$NogG(0@K=Z$)>QxqDHH8|fQbpjpTrd7w_WT8qHSjz=S!J!a^xf=?SefO|@>v8e% zqBrLAc*GijFY6ZTNk!zMwws}BK|)L8?N(CI+kh8GbAO4$D4NLHQq>CB!>))+b1^oC z)7re(zZWfZZ;RASmvbYHd_)2=KB%W#XNe+s{Ty&g1YSLm;5X&0)og6J2ncuc0UL0L zd^OC=yg;dug9_ms3*VvH^VYtSosoa^^2e2{mfj785e@-S1Vi(aI(H*2TlT~ep zBeaTahyG@Z=@DQ`m#hfX2LDRFxvrsoGq>jrBj4Q)xlV4Yvv-IEes`F_PxfWUAM!(X zpdbl3ND~ZS_Ptl-^x{&`saqq&VA-ZB@Y^eo`TF(iZ@YETY^j7X7LVZwLWs>QQz_5C zOt(xlnYYn{?>6o;G1u#5lY{;Oox6$5!vXn40RbY=pONIeL**ka~u0kn}8Xc&h`g{n;bu7cGIJ>mU{Q-*%+3@Jd!f&!#3sY zPQNJP2~OzHhGb*p$fVH)pE&RP;Kz^rPS^U-4o$8V>{`T1Kg;z}W@{(;`nRbKBN74gZ1NLR^2R(*z@U<8@}z3$->QtOLdUhz zWdg8>A3FAXKY#-{c0FNcXP3$pkVq^{apLwJtT1eHTJ^V7iy;3rF2!5-DW8aoEhjmub(Ay(P{8J#Cy$Z zv88M?42v|9jQGZ}iz`+GeXo15&TphE+z!~d=N(p$k0Y(-t9^00s7G}YB2gFaZZq76 zQ)ikyYE%v$X`X|LdR8nk z5jFdp0u_c>5(hN3FsVo9+CjjEMGU5YSw+V-|S0BBuB(ye1U||b}?u2itl0MJzt(zW>qs# z*$tcB-;(H68q?#kTOVK!_vd@wp2c%H6`cR#1Rk>Ko@niM;25PapDNm&WZ2e@ib*5B zdcMRiP|DY@6Wv4RXvM!cikFR#0cHsA{6*}IlaBiaXbMwCfK@xF-j-fHIe%jd^q zAFn;~Dhu_l)sOopwhRsw=7#;5B0Z}Sbwj-G7w^}QNEZxJ8tg!18ky&w1_-yTPfkP5}9?i1z4U$WdI;X>4+GK{QDvX2Hv%4CbNl-S3i`daTqE z9O0|aa3wA%%Z$Cvm%UagD-`Fifg!zw*qV*6X<531Z-opD6A#=l{v+Y)*vIcDQb(3Y zC3d=eWUo`n|B}U1hgp`BDndwmNk2f!g!yFj1KhoFQ%#H7*M@qBwJex6tJ%NX@oGli z?^0{2RNmi%1819#^ql&RwpUOJy-MQ#7M7E2)bv1*n`O9mO)uLQcFyttRFePs&5aDG zHl8M3l>hSw|Cge&0-^dy9IIGo;AO=vzZqY())Kf(hFOeEUN`9G9u z582E!>5y4aZW@i(-lxb(PS_}aHZBCd2?i=c;6*&VSBRr4iRsoSeYjMmy?(FD?5=g;v|&1u{xUNw%Zq6Ek6!yp8kZAKe?okcTvy*>gGD$E*e1d7C!(}r5 zUKjF$Zk)}v>yvj`Vx!p-7^UuPbgExe6tiucpHvEzT$}8T+N=M%pACVGv06_X_EWOn zfIPg#w*M#aNh@lUGCq?S!~-YU?s-(OFJVTN`!m`-K%}@cGv#tERIDDE0V@UcisWf; zz`MZv@vu7d)-3%=)@y%3YC091y!cJuz(5}O0RR!z>Uac}!vUWSb;*Au*LWt@4g67FZH(rx($?N4MrglCO zo(zI-zqL|n;+hf@Ti-cd*Cq#?Ave_9(fhU2^-;44n$kuy#oVqTxAPU1Jk)BCOJld* zrG_Wd^F!_yO;7JHbY2q2YJww#)gC}v%6N$O!@Mtz!ww4cQ?CY_=sDE@Qj_)PM6_?C z?GJC1(756t;j~NzOc&@hA z-!sJQMe#V0%FV(}-C0b(VEuT%PWxr|4YvN*a#(46IJq2gv@okhyuZ5LTD3HJVqUX457N%%u!Zg364`{*x8?9%;8wrZL9m4)RsDO+}n&zJ?8YWymIh!P3Ks&(Ih z&yh{qy4U>i&aX_XfvjnWPF!ERNgDqEmnN;~&DK$#*){!Fp}|8KxsEvCqoZMB z`fjj0UMuc8@Q*sm0`R)~#XM$;))0l)2>9-d7`tSCNkutc&izqwx&)^SZ-`*PerkJYW z2+W}E?+zeLJS7EHtv3{7R3;@P#t-H(IAly27;9!SBSM(S>>PN1K**PTo*(bkee74k zQ@1)8h8sS}wed8vP%F*V^QBGPkoUA=_@vC$J=b^U`E0XaM*b1#d>E`hHD}B9X43^+ zn{@@^@>Hq6D@_^JFf_M*z1(k_8}}xaEs5+ZD$!Hk*e|&R;;-v+tLrMz-@3t7tN?(( z;ziWCp0367da$K>U8#L!d!2z^9}1-uWaHsR|T=Xu~lrgv$KP@n0(#e z^2jEC{_$J${So(eZF29I(7cDZ?KH9A*KZN$@7!BoVE#;VwdtmVYQ5_bXBmah+YEq$ zg8I1s*omzp<*F8eVI6I^8$oEN&iXoG1^BMpeXQtc)lw_SbziOBrK(P>-5!9Hew!|A0py$H01N#;C#!jMZ@gGf)57r7 zYpf3mVw4MRmOO4ek2E7K<4_0%N#8(Ipq~T=1||)EYxjLsVdAGBo>*-;`;~yxUiXhb zLON@DPSZ2S!97TiS+mkbbvjz$R4P_@r2-@y?tzi(rS|v?A&5yk?6K21{}dV;n(0)L znxNYMa#MTz#6%IIogsDpXVwlRTn%yy)aydz4qRWJqbTh6Tph#JmNtPBurrM(_%LVB124D?yf%gq6jJ15DU;&Ge~ zU)2JG+RtP`pfT$Q)!#1+(G7m~6d4#_ABhVsun69Qwkk77G)Ojw<~~=py8C>ajS3BrUHb!yNg<1$e9d?wLBi<=SO)f8A z)?1EI49=$VdZ6OrnOBJ*l};C_VUUZ4+2ZO4Q5^oL2OF;j{Ra_P)%{K{op8djTg&xu z{PZ$hO7oh}#pWeobjkf*?-2KXq_% z)=OdIhBTQHfAbyGe|^djJ)Cn@PA;AB@YPS0FS9oiw%7tEsh7Lhg#xLH49aqdI&jrk zcgihpNV=+J^T%srHz-B7>?vwCI%857%w?WqqhB-BX=pPUb1CKTq~tEZ`S4 zsG!Z&(9BH3%L;dGr`O`lx)!yUqRff_%;Feux9_0#yh>+vgO3+6|{=9BCmwqZrn5#_-6uFrb<4*f_ zuTH)FukaoURw(DCnjV9-7HNtq+^wg7mU=iq{aRnO-BkZ?{zqyBLVc1R##$uBN^}ud z;HVQzHetDO8qdISM1LEqJBS1o^KT!UW6l(X+3N7*Rjs%hYMcl@J?SbcSW| z=m;*DOxOhP*B{QO2#&)2czAp)b3RlztoPzxH9N0_HOhj5u7DxWil_lH=~^mp^!G4C zn$a7IiqPZXWPx0n6x+6sQJ+M5CdfJu=S`O~{8r_J%1ye%tQVtLIcAhwtWzXK5t(yj zL|PJERYqWF+YymEkY5K^rRMu-ye#Ir6_gy|wcCYke?A|7kG>wBnI0F_ga%H@_Le%_mxY zB!6~@f!E>Hmb4WglXg7B05+uQp?sT9StR+0b<%c%#&Wi9{P$q*>gr7pBGkt1#6dXd zVb;2deL7<8)_=QoBaeR&5WM;!6hK0qkR62c{*(CE>Doe-=n5Ia@%n1%kPe<|&{X%qI)nv{hhc$)-y9?4XL>uULxS zR|>ugvr7_x0LM&GbDYh66@Z#ze19cQ4|Doi-@?V`XsID^bG|xb-5(*qE4@jpOkX5& z9}@-S?^&uE|8+fZ9a%f=BoOQ3nrRf}$t44Ux7=_=mg?*L)-nFl&w=#^(cdNT8v%@D`5B zSdNHqB;Q%zwx0Tm@r>;ZBjBdy--Z*}e>{bQ#{=lVmVf`%>)N~QR4dTgoZi@3VCndTPb4s{_6?=b zkl;7fitQOfJgU2!jxJKTX{^eh*`?DWboDgJ!V3K*;3h5XhGuJQh!a`N^y@#L0|;y@ z635q3|6y!E&W(v$UNK8FeB?Ps7Cj6*+ezrT$?Y5jX>hEvbbU%-qWtRThem)M=xAq` zDoQ)y6IziQEicv&5=>B&P%qjrQ5YosY+ZQH>x zL_i{>r794yZK%nEw{HwL@Cbv&+C<@K-p*`H_wLU(oTB*ao*xr`E^V#D7?NL9fFlU`iy|p9T_~38zY<(jF{p}(K4t!iR zT8~cK)!;Yw#KEkOM{Fj(mSYCSMAYnLEwbs;!F74#J55QJrZhP7?{e{z?Nc3%AxtY# z{TiEg*YYm>hLEsL+I2*!6&-4lCwgltrkobDaqq1bt9w2z%ZXL)ifn~H*osJyoEC;% z>A9VHhD&fCrGNBAeUA+2bjl-==z=elv#u4feHVv&+3;)H$eiC$$(Slj(lT z-8@IkeUHGJ^4}@IC#HXO?ySwh;{O(`;#&=5Rs?WeM;HgCc=8B8t!?|c(|-S9*JBWo z)&TPCTfI=_!nt-N_M$H+T%ZLCi*uAPUej(AN{H!$3kO@CtPgT&GFGG$*QlE630gfb zl-p9#$QR?4NHaz|R1pyKlas}Q@F$yTyyAi1EC3N#--43`coT+mr9(%fqjPR|clUI> zF5ix6H~iF{68Iw&jd)jWzxk=sYW}mvH88m9Ep!B*w78DCJ=*V0%-|D-<2r1A_4hE& z2kr&a`MhYw6Ko0`fc8)E!KwIfrrYfI zYg7qZt-bFk{G%xbKet$B^}N)SH`df(rbY4DM!tdVNiho+qyPa1#U1MIR*?9AtO3cgGg6|MvYvfo=|%;=c^owF)V#YwQSj6mEh6ESkVv4pJH22$_!2y5|~T4q}MhsW2nn+7dWL-*iqmjh}jSP=v?EN(a0wog^IK7 ziq&Z}3oz}s1`ra**YB?lrs8z;=iB^p45qXQ#!CnVmpvG~iM=YU@VUZ>ZB&P!D_rPx z-DOnnbp%mqGtGpA6~TKzVa^;vcam(-n<=L{PT=PD^3TJ{PGcd}z-%+EQ~UY79f)9rElc=ww;-j8!z z6pNT7+resi99H$ZEx9c^RSxw;59&YpQOoA=n~r*oFE$?WS1q{@I&C<>5eg6UUTY>* zAJ<=gR!YFE6LvNh(Dcd|2mwAHxaA2f!>b0X3<%r zCvcd~67x!rtvI@(FN)N0=ekAq@8!m%`v{ITc&@vhER7`_uI=)=snIws(Q9gR_~R7z z1zTv4wq5G{{O8*nvzwfxYDkGbboza1L&WF14Qh*9yYb8E@zw3LWwy@-T_ceZ+$Ebu zohm(vjQj1cqPQ&zIU+b7Y~4?P#Vr`8)^FjddtdBKby3b43f2DHsM209XWQU=RO@q; z+s-!lSuBBvH+YTEnwRB5-95&1dO}415;}4zB{ZHF-jN-O za#`yl`Ghy8K?G<0)l1YD|KBg)>%bRbTua15JFj3qzW9HCrQ>7lgz?*!`!sgA-ta!@ zm|RfbWO@I3wkDuLB!qG2^<@(!Bo`f-IQFg)aa1KMKci7UHJNn2cqFUc_WnO~eRoik z{kFBD0*Z)$^rj$15s@w>pwguYD7^?s4WWfzRk}0*=@6t#jYtg;P^$D!AoP}m8mXZL zzQ;4?-kI;5_s-;>1Og2CJ?mL}?X}m|*~$+IHahFxQplC#-`_jwz5W4N6kH^%5j9q3 zpcS>auloZ50vt~Cy=(fBs^KQi{lLbETPUkIP;#M7w1k0y)bXU1J#8R~G_^+K#$&1V z+^7Vw)6t(rv7N1wfRiJhtC}*vIwla>f3M$uuF1z@+VRfLM22%Mb&ZmQD88hq`czJJU7-TKZc$ZEyxf5RPL142zjLdr z>|UQp_2!S>3E2x^fw8>wcD+^naEQ~NfnLEPkEisHN?!YT|BU1v*&@ty9QJpT@>kSz zB0F0#A5jdl$Eqi5Yel10;^B1;^ zZhhBzZ?j~4X__$fgPaU~eO}#qgMVlSn_Xm!>3e`E_oQUWJUc+SK1L;@-bu4MaNF8=?636w;kv=lvnrScUX8fGp%r?rZ zZk4y3pHz1;gU6yDK20$TI&)3mA+(G_HNu`2lL8CDw9;i?4E1H%gzDDQjWY3B;KO?Cp2J8FsZ=Zv^Yd5?0^aKll0c1pS&*ffQp10^JR%Tu z;n!!08!|#py$z;xAeE1sfl`?U_KWRX7KRZwH>=jvmb>cGB{DeKev(RKYI4 zH`kZK!ugI8r9cG@En&>->iDR(0{PLHf+21Z7VUii;q3yf4l$(?jIkQ1N#qt*}^FC#1^}OA8`OGY9iNyF;Bzhoc_9>RcSK0^~Ya zXe=0sguH^7d+znSE8TsYlvBy)F<)tApH0n$%X8#>Yp$sYgRU35TfA^Iu~ziqVS6NO zQhl<%ALHyFyxp)-@ebOxT}8#_{`+3F|9o@I#_#C5oSYA`hm~;%h>Xkes^}|)n6ryV zUmtbLk9S$jACbcOWI$$5J^lN*UjM;)=;bSR(?V9CC?O@nQ=Ze1HD0;n>jJ>! zb@@7w%deNi7Auor1ODZbyYm4c&T!JQ=>$ z#il$RNZhFI6Yd=v3_Pq#zx-0UNzcrlyWt$wQV6ia`T_(G`vpucTM1uhp5ZjXB=zOo zdq8r`w!Fr!Q&+2_Quv@{;uX?{P51_;>zMCUr}Nt~uxO7u#sbPAX#TQmDa~DlwY*#T z-30YYDvSInTR3H}jt39=^q5_!;qz&!KlVm+xP$HXj2GkDU^=G~HZaZ=7@b9HrDZG3 z`^n%YQiHL)XJwH&P`PL+sa)n9%G)?r63KX$tB|a=EaOmZWo)IXk$|AYv5AqV+eu42 zu5z_c>?D>=Ibb_2ElaZvr(OLg%zkRab5YfE!}FMYV4C~0JoPJR=#lcfX?4baQ{HvO0~g)Vm|;E_^qT84p*<)6F6 zv>-P}%7=IISjvIYS{puKvWKVWpD= z5d6=UqUzGm_yH~5xRM03@AVD*feV2%G* zg%h<2lf>8-9B0EJ8eCOgwbKJ^0GzX9<`U$V3cw(+IQ~<-)*fbXu=dTt$1G2DB#^JD z?kKz5{81)DE=YDu!h(dj>Kp|2nEr0jhMO>UE|3MtB&T!wlGy7=++N8OHHvU*l=Wt2H4YDzr=jLhQ#a` z`2y}3`WTVibj%ZQ{-;g4{X4c~Z?I^v%0l%|!)mfIZb+UA z(rZ~a%WU{Wo)Zo9u-0Oww_S`88?6-fTL%;1COu3}=*BMo1^r&OPnf6N>K)&9q1%h- z14ca7GFQZr5k%bWt?n0}uD3dE=ato-o=)#~k*SL1X^R7PF}hOkwAilaXfT`*ipZRs z;ypn-?Vbhr1%Mne@Mf#G$(}7f@q&3TU1yf|bo2}Y+Qd(R_2h=4Z+~9$$}X20@(8<` zV-9m633yl4Z&BPWd^@>2aJg&iZ#URdcfN?)ae6 zjV5bom^i(8ZaB2eoh<~nRiznEzwG0b8D(dJgZ7M|w@mRqns3ylLh@~0?qw_5+)d}b zXoTTaM#7^F;fcTdI7_aeUG7^ZgIiWkLjhu9Ibwk=t3u>=iV^EX#Tm!oHoDCv6tt+| zOXSrV&jvDsrQd%opG~czd@Z*AbSH#%Eo6DxYrE&x4TkEdpErU@^&5hI@e78bQqlu| z0>o$call`Xu`N2VOAsa235+@<9<>k_u92kw9C-rNcW*D#Qmt^4zklXO{>9&G2i-+T zJABoJ{13eO`*i4(9HbjaTGEOd#t5g{Qk<-FV9&msY}&N z734U6jfPepH*pos=;2sRtpVDqrhi^WS6yHOR4!2?&7OYOc?C(6qfB%;b7?DKuqygLl40DBL~VXm5-i)u z0$=);9{+v{KulQBMMiVLoR7O&BGa3P1xh&oJGwU(+aaGGs9v1AZx9Kib?bMwZka&D z8g`}Ub2MZLh-BBjmdqpM8F0W(of?Os7w2A{TNhila(PrSjKhzblUV`@jq}42Y`PzN zpCm!MoFQil4VL4_Ey-@)cp-24bCgaGza!WE_r7W;d zsB5fG5WTYT-C}593ejjOd^uf4RbZ!vaYIuoL)4 zK;`rHlom$?GKcd1z(Ygy1)1De26I1wX}#`ikDP3k9@U!Lq**T+(}?Y5k+FY;vwN|E zj#d);3FaQ-Z^WRXdaahl!A3Ki5#S(6FnyN)Hln=q$H2lbcV74#@Si_La>Ei-k|7x6 zsZ+Om73#Z1H^AT2faobOLJzf8m}++3B{#B`Vv+!X_gu?>)G}o~apie8NQhN1?VYq6frqzRE&{67 zT6GDt%2t{(m2{^7BE2WQ#X6~z@ugd#|FtUiz5T@ykx*nDz1W6oJ`@AMW^^q9AUYgp z3uH-HUI~}kv+?V^L*D@wBNQ{#Nv+{CIW+XlFPcf7Aa~|c+Wb^}RIs_eb^>`e#-P=* zDU5dfd1<~({_&$J{EAT(>W9yr43+UyOK71v0CSWqARv=6!Bd`oqg)w?p(0|AD}{Om zgN}P^kA>a&W(PJ5ZVJ2`IJtQ@n|n8R;(G(O;~ItCAg!jI)r^kFy7y*fCpcq3ibD3x zf?3ju6-hs)W`=L_Oz)51M1-AAEIS)r^n0k7I}chG+Q8iU#VWKFRaxsMEh!pS#&qrP zq7drae+Xv$`-GFXc{JijQ-DJjSJAlrkSWZMGK0@FaQuzv`~Iqx%{D7hY)uLt5#H?m z^D-p5uzrHIiZECGc7T=H2bZ&tW^t}3Zi>3yH}&r3u6_g;hz#D=YUvW9O{8m#t+x0) z6(lR+>F8bYHvSK)D+Vn3u@WnQe+1vCVH~#`jpd64l9Boq?fK~>1K1Zm3nh`q;hl1j%@hqE_kK?kD0_W+4{3i&VlYv-X4_wnWCj2>gAFc&WJ zF^y-3!?+f1`93`fbc0ifwzBDD!=^@n%|L#0V=O=W{E}+dP_C*5Y5#y}Kv!&@Ea9lL z>0~!_v`D+{n|23rskK7wk`kM#ggD6c-J#b^*6}H1k&HGDma3XcOYVN_gKxj2x9E{G zWfm=M^^N~zLjXE~vH<1D*Ed8+?!Q$jC$*n`M$}pu;*$y7jiGASE`Tfq4$2FF$0 z7KqN_zwHtu^|-ODW=NDf{fnx>y+Ye3pM`Vygw{Sc%jn)%gSY3mv4~JKSL5IcWkpx` zawYL=qp*4*I{)6{PxcWJyY)mG>WnV6jAs$#GtnQewLiLlXl4uZ;n7Haa*zJeK$e;_ z==eUwYk%0#>MFk_Jo2tb*}F>`Jl7;-cF8r)w!QcT%JVwoik#ImRRDwc$Wt31?jz*> z>HO$iQI^tR6Qf+{mumx5=7Ev|b(=pR3O|7`{X9y;c4xDer{~^i-@Sh0xfq3Lm`c@y zWI=%znE=l|AY^XdZNjj6M{48f<4R#_%XkqLTfka@;a(KS#Zgx^EeimOiL5@S5a0cZ zM!R`h&9A6A|Ej0-jre*l;vk@Ky~0?Bw$6OA+WhV^%qeyDkpkycWy5`n0KDM#i5lYe zelf!NKBs_`Qf5y4RPpf1d4!Zz^L*-)51yG%a%vugdIg)=8oerw4*V-iORZFo;v%Q2 z?16>z<-|%`4L*S#tfwxCNu+Ax^8u%#Y)vZJU6nn%!x>+2I&g2TsQd{`MB~0J$L2K( zJL)cUIU=|NHWq^ICmEWDPK4pxq8&@kz&CW&&Abq?>Ejz zVoUuYI++06LAvnBlv`3**4*X*u!m>kRxNn8u8xefW(i1C&na*I6ROwAAtsVJ>&aCi z4^;aSp#35FDZ~^-?a>PF+DG|YIbuE)==+@_v5ex;X8u?(LhkR*&E0z$f)I_UrEZm662`(8@ocYPLMenvN8>FDfpT;9H%<2rrQt)~m`ClK4K zWtlNDiKJkfpI8x};=PGn4^(;Dz#sI(07~|!pJS(OInX})c3wfT=m$a@3%=0f)s)du zEjf+G%^zxzm^T3d9-9VIK7?6&#Q|Q(`P+WO^mSY`phd#jJYJVNH4}k94(RYYkn)cP zo#>u$2x&n{M|t#$q4(zDhe;TuQCi`6U-0VzQ`cYD+}FQ6cpL^KLMMsSC>7%==cdw) z%iKIsJRE&*um;5z3O#@1ZE!W#d-Kw$6lQI2jPFcc8AWjyO({<>8;ZQhek)vZJw~;e z;fsIQ5M|21AdC7tC4;A=j2^GcU(~Z-m4X2a(Y!A-m@%wugALV+j2>SS*4a@4|ElkH zB|Cym|GYe#uY9Q?=Vrsv5&SFE^do>Y4GN4;$Pc6)B_+N`lC=PW()5#^XX_sTiq@5R zW#0a?+&}QAOC(#|KtXs&(@y+{uYUhRKFLep{bVjq&e!Or?&I|dW-o(8D*JXbvbH$R zruuIbw02fw!2Sk}EUB8U2wf#+ZW2-Z;q+H_IXvdN50jU{F_%+4#V<>fd5^x>w(*T>xBk3=@U09msgoNUo_B1svUE~UtycIei66Uq| zk>?H0wO@A^)>_U&JtfbN#I-)648kgE=!t2Ht>Qr;HO=*^-@8brq8>LyIMpwPJ#mx| zzg58IH4}ep&X$AuX;2`S?X=Ci&m-;F3rc1M+svTA$w8{1qqcN%Shb50(@&FtgK@Dd zmwaO8Arex1@2}V)yyhVwCl)`ge0xiuwSp8Hmcri~4M1Znp2c})#3{EwEJ>e7+*0y% zvr0aG2O=@^Y(voKPMxyC1p+U7(gCzw;sWRCFvPW-ECU<-5%CB%^o&kNhO>Q;Z z;j}Zac+bMcsu@3F1JFfwqN@83r$_%*g{2h@B|%YjWIIWHq6T|__o^Z$l+ql?RnX7^ z9lS2z$_XYLWeBG#8qRQh!6lB4&@**LmUWM3o^I!DR3O66UFHL-H$U-@bO4$y5F!=0 zQJK8S;>m6{AX-xrljO++N*`!EvNUq48F$ETmh^he<~nK7v195}kc7(*AQT{Lr@~Ik zYV?-&uz2D9Vd3SE<`BO-1~(3`?e8S_NAFfvtTl+!tBgIDTw&n zXB%qyo($=h$em7>{XxSjTmdia>qQe$yq`Ch5@ir9KINt+wUnP|*9qwTYISIACX{!p z&YXe*vcIx5_0Wa@Iy3;B8$h{v(xab#Y6`R(-21*fC_hYT>x#*nJT_7JS2q)o&uc1Kr2=Kx>Q1-I2W_C(kp!R$IFvbfeG(-O79P7Lx$-$3@v0?| zXe3*Sy>ov5Uak2V0DExW?ED}J(VhCD}Zklg-l#XXVOm1k^u+b z+i=A}7majNT0tkH`xznwU#;tBzue2R{#E(I&#Rj$%?|~c5;3oc!Sy+UDqf z8kOArxUrrXtHDc_XkFidajQ><=_6IKPNg@<5QBmGiBH$?W@%zKfo_ukCBuhjMKpiT zGibA^|IF)3)A{{yW%PIiI?Nh|(&fcw{!+3xMtDkD!R)H`{#J>5lew$0+zX(55*~K8 zUdHkd&z$ex?PQff`GU!BO8LqczK#6{4E%F``@MG8X&V#rKeaGG9s8-neMvWN{Rk;1 z8mBH-?}7I{tMv8QuFnf3=}=;uQQY6R7ifAJx%Dm;hKrKxja=Cm%@aRj#uv}f$J5hS zebcyF`=;|J#E3R5Gve`hvlpCxGiio@Z>-gNu=>7)rWT`}<^EDI)dN>M z>+|*+r+MIN5KDQan&JLU=IGSaYdCc|j*9l^=M}2_L@imw0aodZEiJ7tTdsxYa{!QX zy;crQ+*y3JEZOlOJDFhDtR#tNXK^_}{E3TU>VQSnr{{CLJ z;cRz5vSuRpgJa=cg0f{$2Al6zlfE_+&lp%tyZeOee;?&b6JO6eMDdEj8qs)9l zCuOVveIIv4EqN#$&~f6*f4t0NQd6ZQw4=ERK7OCu?>$+uuW?SU$QTO*@3r56XEoW` z*?qOm4oIFb@UdQ(u+?0C^bUwqyBL+H3RNQeSEIA>5M50K7@NLQOfKEYA;eMsg>hLs&tCnY|FnGhL`7P zvMmtz0BJUFJk`}@AdR?LRSjY(zi-w=_`7HoSJ+Gco7<5cuR; zw*5Xnz;$5F^7@Cx_wPCH^2nJmtczhj<3*mZDmVZ%^1t!VUGy@ehr6hDHQ7H^zwd!| z0NH8Rt=({Knf}AGdkR!P$0{D=pML{clecCipFQ{Q7TZd{hm>ZxBmUJ zp6eCvd3ydG*-3NGiuT-ERkgG)V5^_VE^B{d|7pes$*sW+5fc$iM#Clyu}G(+E3mKz3HGk+FYi*Wnh8&2bj zV6UTFPo9BuRVW^ESJu@e1l4*BFH@(z@PQ6*t+r1HvxEC|ABrAg^7h0^%Jmq{LbI|G zyE=_&r8u_Ec_HgHd5qtODKof%Jo6SChu0zo3n_KZYSY?}*O<9a`Qs#>IQ{K28eM18 zv5lAaoC*~`KI*b>6R7~3CHaw|PVe8q z)78SM6V|T;QluBgxY%?D3Dn-%re3Cp>qcdslOM!qT;!P#J5>CG|kYtuIoHix0GfF5LkP(iJey#(B&Pk1maWkt$cB}52 zDoX{OLedX1=7)|(HNlExN^FR$?b!bne)K}r`o+8_D9n6>B#)b|y0o+XY-}#CqPxZ4 zQ#9Q9d*u|nynT|T@GRtH#NY4QDZ--lH?eJWq{ z{sTF=M56Fy{Tcm#Uzxxw?u;M4{0eu^jgdV9r3Sb&28xcUUVW*dGH%NG|L2M?>UySrAv**~Fd4G_dY>}fxyX&GI z_i^beP2;_^w|ZudE{^v~5a0UA%62|Ryw?q*FV2kaGjfL&c4mp#e(Nzl8OdGZLm_>V zB6yJX(+9a}dk3u&wyZg*8(oG=iNY$84iPIcGvJK`r_o-DMLo-&#k2ripswV*jwu;U zOg`2L`#D`C&iLl1l86k@2lu3!LTFL@nG#+^yg8w3uAy&Sc+lmU+Y?@s+KsMW)aw|* zscfAM)e_lKYNz}@iE{hJj}!|(Iz8y0NgJYq%;)K8#?BaQgx3?B8=S3iQQ z8*Kh8{Hn<|_ZJe7jbg^&nZb_@M}EY4*C5&_e@*k(Ja`%FalZ`y!wdo%B`VhlD$f*O zP@EqKo?x_<>6pZE82xQj+)2>Fm3_dsH6$Z$P<&Lv0~~8s8hC}$KY~JE5tP@W=KM?A!0i9*yoqxZegQubD-Auo?(}un7 z;8~4Uo6?N(f{rQrefPBiF`}%pp7rXpuUJ~3=R5exH9JkD7+95_pBWliiCN z5I9X#gmOMzUGeHa@fygoUs+7^{_|OS*DH{Ydco~B&|s++BGbx82FY<^SPfEUr39XM z|Cbzk`8G}*aIy?TUW9)5_}_P$XZU*y#2v+LD&exr5koz*D8Fx(g|m_G&xEqYE{1|u zY%dnzPI*70h^r6u)4p1vWkrs zd6`LRuy*-6y)FeGowQ(Jyvt`eT~5JSqbM)v-q+X`aZ#1k zPhFZ&)JGqP_s`}2NO<{Iv&rAO+fWR>*-gg`ZpmS@onoYUXSvTxG)?tyG_4YL$=s;U zz2ds9lQGLXyGjsX9rCWM7VE;i?D^JZwY{q2w57E7m@a-gJ(Z@Ta5tvwfs8IjqJeTc z#SH&L)|v>Gr#gkrbm6C))u7+q=aN$@(sb&eLRGA4Clka%2Q?=xm5!kb<7V+s1+%oA zi#1i)3GjqAJ`Do25OXq|n-RmTFXNP#I-p~IPO4{3Xj3Ejw6U`o`QX$09uqxTw)Zg_ z2fn{9PJ`TW_riw&v!A)(c@H6rSnfQ0R9@#K6tqC^4%$2H8e`%%GB5B8>wDJKRNYzN zy@51jS#>!d*KT3(-?VZEnBxdAuVj|ygOtbwWChW{qd%QQZIQ~FCfHuCS-9`~VPEOK z)SqqI3D3a_OB>AU<(3o?Y$Rs8+?2i1-|Jh^tu-S|zecudF0lue=yPV7oCh{Z3Q!v` zm?<(O`WnSb$)S=e9kD}dZsmq#!8djc-<#_q)3z)bH2Xbd~pG#2ap17TWx?yEtW= ztwA+*gF(?>pEG|c8MLSu=`tyMeU^50%9E3t!@LWwB2zRSsbE|1(O65}zRU38QmMWR zOQZ(0RVm|o{QF7=X(RIALY5!0>0UtRX<9N}UJ(Q(JGXX-k52Z?9$d~@-MsFkOG+`4 zO&8{1MK4sOdR@SS1Kp>D7NoYvSn_342%d)cG}>@K{t;FZ{YyBdRt;mNIMKk46DwSX zgLTHmu-8|uGl$V}A)^nt$c<6%O)?@h{Lco;LA{wl6X3iaVuNPVjY;I8T8vO?sHwbg zpkVD5wYjpGoKY|Awn^7OFUMkv6|7YM0T-uf9)b)z2kfOeyevb4+S4% z1W|QOt^@3}=&$qDW*~E_IK1d@Q%lsk=g!GbGTEV#xUR~4&9!(X9yP?wqImw}$u#}q zI^x~qTLt?sxLy-(-lf{*mTuD*Qdu-@*R8WtMS~kZ%yWWra z9;9c`Lr_uDMCY9c-E3>RcmZD|#vyWGxiH-@YFrF86v+#lL{u69#vUBt00E1?UQ98g zs-;gJ6NR+rUzFoLFw-)sVE3hQ@DvMzO%m#J8t|!6|5UAi*-_zLANsy}#RL0MOs*J< z@J78)%Q$w&t{>PiGG`dwmBCWVRwRp0u8>9!6bu#;ohSANM}ghl#>0R)r|5Bz=q;dE zP5D=aV7XenbX)DFxYWoutG{OFCl0SWw3l`(UjA)0i3v06d)<2*BjY`Db~-z70xA>R zNXagL9^|zc?)veW!q+Nw$$Ys~=v~5pTABal25cbt%PKyh={TFKUAIcv)`QKAW9-}JXp5mFwjf=8X4JTD}S>0Ni$_lWW=dFQUyfp9EL>~o9$Z^vRT8QRtrtdgU zMfKkS3b}3qLwEM5oVd7usCSHD8V2q;Wp|nO4D8nB_C{oF2Y2OlU#!~WwKcu1Ccv-zjNE3Rg``pwb29r+ zWh$Ff_^G7@m0mCjo_Z@G_1hY4H(r){bKr$Mo2yT1BnaBY6JF+?; zjS!(SRtU==FV|I4fZ+@&rXl@@lfZ5B&oE&ZWI0n{mC!|>G(yV zYHYvj&_Ek)RfSRVtxtR71eOd572TDr-;w$ib{t8T>*d-UzF z@W3QJV2Y;MiI#XqQ+JAQ_> zc?{33G8Wa6H1B=d@l5c5*jhYQe$((>Ah!*RS%2A&?R-J$7E1ANk;`w#v@NKYbpU&* z*1(BgxNfGu&L_o?xC~{<*YNBzdxq`=tFV^&d|<@Zvh>s{^W~mvBKLmIsj1 zib#*28fWbm=T{0byl(ctig=|kvTCqqR#Pj*nHd;4W`!LrzR)isEXF6c_E{XXXh0`_ z10*i6y-Ya9>g=>QbjW$6i1zJFIEO&jOWw>m&D(7h8lNSilZ!$X{Eeo1+= z?E5&auCfpPY=%hLf=S8|*sMTk&mmr1n$Hv}1zd5#b~1D5myt2$-qH*#(|Y&MHOJir zgUUVx+j`$4SSVE>Ys6zDGx#SrEjIckiiWd^k0|GTY}Zk6b1M)Ed)s(9#I@&3;k*0dn1eDu z$ftx7Lr*n^XTLe}lk4NlUm5YO5^}O=SPUOd{eO~3fZxjm{H$l6{;`1j z`{FZRnuwb+O@#7$Dz@oC(UW#u%!pwnQPd{-%?uwuc*FQ%^%Bk znJmQikaF}(*kON`6iGGO@2rQ3!A{%aUHy{ABbrfVB~rbnZ_{H-^5UftAt`ev1z9FD zTeQsu0$NQhHP(s==^x0MOC!h+h%|z;UiSyaNt6R(`czZDM03*wq7NR(Z4{L%A9JH# zPR~8%bgiKm_EUNYi;%J1{JN*q$%%kv1P_ef*`RT4>^EoI$ovq7L{)6u38sNY*uj8> zomyFB#bOZXv!}*6&hK^?B5M&bzlXx*GC7&NFt(Dt-JAjQS-kdF*B}D+#C&CrfG?Q- z4E>A_pkJ?E3UBu9e`*cbiwc0fxT8hDu_ZlR8>PuM>5sYDw%`1k8M^%j&m{bbrs{HB zotI72goFF7h>5szD!0O=r!SGc<)E4ybAGs-PJ{joW$5`H#|d_GY9B|dC7R(?)->|6 z`lO_E|cv9c2{xZpGnML*Pq4JO%4^+wk;x8Ga#4OZZUFE`xaA&Rx@5NP`qo*d3Utch5kuT z=|`7J&nP-6eWW=M%!Ga$oU8@qxf*{oKpP4z)%G*0=QH7i#n;a@n|Ju%##DI=lK%$u0k;ev4eX$S(FU^<@6$T?hzx3L|bc;F@`(E*f+o^kJRQ(!k+IL z=cm}+_6%iFi8X17aJ#HELTO)HuWrBUgu#7ujfcEtBJFUYLx?HMod3&-+^!zC$1#4N z8pZjj*8_A_o)&1tF9KjMLuBXnFU6q5(MJ8puwiLtJ2ROIA>@Tr>BX0i?1#FrS{m#3?wpNZZqr25+k=&t(ON- z;|j(FD8MFQnt$1ZNGmrF3!DAey*p6Ip*k7_)&p?XaHrkz*|R`~Z$Z3#N*7rWT+UkC zOdfV6mvFjD*joe=b-M-60|y0D+xIj5+tmMoEqtw`1`d>S)Kd(_KgiU757`jcb^rZ5 z4u}Z67%(?gvHO+rIFu59jRPIjGGz{k2iwKsC}8Lg;5qD`@Ta5bDIsr>5h^AB{Zi4G zlX28io5|IlMen@h*gZK$Y)@y>7SQWLF))Y-}}#%L06vDDf5znS<% z|7I^>OGy033sV?)VCBPUwOGr2o8kjzrbh+aI}RV3^nU~;rRLjB7^(LWaYV&+Mb!cQ z#aEWXuirRAz+bGhOX9VO<5<#xE}C;s$nWM(|ByUTw!+emiZ{oWb%WXOlgRbLeh;Lc z^&IzUo((pOD>uM&-WDoem#u(hF@{3e1iL>ZAb&F};$}uqL+1RX9%85m6FlHuZ>Nh< z`SM+@B5(O4TNt#ERIcnV1zs`@P9il>RWs-sti#~9y!Lk;#usPo@HUfKxZB~xLAx|q zL5#aZ%NJe+**HX|zinZ`dFxF{?>8@W2-j?LZPO4&pfxG*V&X`t)$@w+XsKI&+av&E zrFm^~1^A2;|4xu%mFn=5u47nO+Tv9frMTQs33WA+-!+S~H>+Z|vvh&;Hn-D;d;XQ3 z#I&QYjiph{gi+G%T}JP|*b}>Nu*(zQ6$8!C7?xuH1)0wVT$pjISm}Q_ahI+(JpNmW z%}&03_kV)eEA^jZ>#4$_O8zTcF|7uHkvR9s?>23*xpHAYEN68p

      X6SkPnFc;okM zcdz==TPq3j8&zv5VNcySYgO(h)1LWQ?QTc5XGvB%h&m0P;6#Mm51dq)X_XQl? z5bFWkI}fe<)i;T)lC~2ItAYgny}`%;k6TL6&pWGEyGESHiZF#SPjxUfut#o4G3&(gkYwg^f>FkdW;H;}44!PSTc4>KY&wr#2!(>I`;tm8vspi~MSV5xq7K6Mac$i5azcq|?}p4T z>sP>x9{WOYejA|C4c6+@GI7UJVa;BH0aoz~wcTH(*SM{AmaM_rs?u|FkeJPPf;)#6 zYa%83+A}M%po_0&l?!%2RDQWIF#m`6_;C>E&-qFL$6gQ62(tQ{>vu+?}6zL znw$%NHzA|##_n-SZa02f(Cu8mKCFXbcFU&Q_sjh?mRhzjNV~3TE@{DNj=k;#ec{Xo z&m0<{jYCVWbn<5#|AIw>B+U5LF^5MWX+q}IM+U#hByt^HM809CBHnwQ_V zmUGI4awp_mN-tsB69SbJRz}Ej>(ng`fBE$>rl{XI1I5i5gv5AkLNeXJaz<4JHEnaw zs5?P0T6;Bh?=t_EvZ9g9(aO&aIoFm>Kth5&k+qOOT+{ASo|S>6(_0O_`*0`^b#c}E zuICkXJMU!AMljdK^_yzmoud3mf;u~;CU6}a{OQVLw(Y&sVC3Eh2D}NS`j((?rsres z(**qs>vFr`^4CqrBF54yCnG|*!!5Hk;fe3dpA;^9%N3NExc$mx<6im~XnF@IJc0rS z1XkqWjrlLf%Br!wVkRPPwjCoi*+!!X{}F#jwpxrEdk|2_mzieuo@DAd?B`$$C==V? zwNC^T<1Oz}cg#vh8Ovw-PL$W3Z!wmv1eYr#4)EprLgPDq)VgBbhXSYB{R~rBF*&yR z5!v-4Gm^a6nbxkqohf8Ipq2W1jzN`JD1ivYR+hrh+b{cMn=+tVhmPyBexCn}fc&>{ z*9+NAKUisT1dGqTZM#DhQJ3`f-TwzO^WSmP2;k>_X8AwCZs-bm$lqD|YPxAG>`sM` zL8U;{;!zUqSLGKL{D*a&wAeG?X6O_?6lmxzfPyVoJ* zl5M2Dnt`=1S`v=eS-*E)^AFOz%FpRMn4jmKS>VHmA(fOx|NK>Ptg^`m)##T;|FBfb z+MsF6#Oa`aNj#dS~-pRWe3ZN2`5PHTh?jsKBi3nQQ173^j~9(tIm8Nca=|z+E%I{D&Mr!umaoM7wFTnRZM5gh*4xI2|jl zID&{Tot=|qOsn#)d2upsRf|$AJD6;8_4eRxh=mI*>NN}&F4+)_2bhXM+*U&n>|v@2 zvDTJEOnuMdpBq(IxW}rV>hzXW_)IHbhnvVw znk2}&VYrF|Ix}p0s-JY+6Mo4rw)XvaZPl_+>DM=|y2yVKw>g!CDDyMU{#2nn?{!|7 z3ul(gS!4%lxwSy+uI)sn%ZaE1qu4`MyX|var%9t)_D!V)fQ*stl3e|ZNuNay7Ak*> z;j#5Y`lr`_M&uE14wL+Y&*1$tJQsjpa}3Zv#{WOT;eThMzmn0A$4HUhyqmzQ?YFdd zhhotCF3r;oPfm+M`V7TR@<$^h**;n)69GQCTJ$;Hqcz9q%%@54?&}MnMYr~Wf%#9* zp-bj|Ep+(R_CR0RGkk4{8~-M)lW0zu_Pb3Vr)#M^X*1?>a9tS^|U*aV%fq95e}w3`-)GEM%h8Nzw?8w z%pN|e+l$y8o6cxL-lYqg4%;_vu=PHW6Lu&T4zjPy7vHU!YbuEK6{@lHnc{~;Bhjx~ zCux_JNdn^3O2id5#+hq>dvNoxX%DEfAed>=XaIA?-tX-?#CaZ4b2pXAn2W(#PBW{! zUO*7Ab{jp!Z=GY9dQ}2cnPj{%qzI8@o5C(_x7r^~T+q;mR^K-!MWo_Q2D0Ahdem2A zd?1qzxYgihtv;wDWu2`Ao|SnuMxTHPFcf*9%YzreN5AQK-P6kjIr|lWX>LlSF$un7TF)N+Cx7qHk7(piB6O%^*PP&CuHsy{;IX`FqJ5P>rlYK)N zmYH2(z#>C>D0^x{%JYsLo#cHJz@TP zZm|&|E9u7F^)m9&e6?rI^ewxxlwAMg1=wH~;Gu*G1=}|07-j45ym&C|Upys$olDwz zfSS6DBA90R+Td-)Lu$!jh2tWh?(_v&q&U!W$TBa6%6AisQ-xUqBV98#xmvZ=YmSJ& z-}d&#`4Xg@D-h@`;qc#C8ciNEW{FgTk8)tHe?SjJgunwiK@+Ykp6`e|Fxrk`OypS6 z7eg(ltz9Jgi`vBPt}k8SL7Vt{TB?*IgmtkV!g@`*L+^w^v1EbL^SOo^ENsN&yAQ7a zd0oMBlE~WUQ&EckzKOmWLw#(C%`2m-1CSJHD8WyO4cNiP9T8E&`r1VRklnQY;th;m zJ4jZtD(2r4LHD9pd9u}dHD!y0%S`?sVebLd^uufoTaY3s0!kO9D^&;`0w`6aNbfa3 zkSc`GLKOv(CQW)!dhfl80-*;45_*;1r4!2c<39J^=bi8U`@Az5CT23hnH+Y{?w&o{ z5J%T`C-tgqO^u*_2y`;Ia@*G&d z8VF%O&{O>1%ZvYFWh)_gMi~)9su^Sx4%jC~L%eFeZ@&F*vqg#wQMlO+J`P>od_{cs z1myXmf*LROV@GCRtVgZDO4POHZjtm|J+e#(5cz{OJ&zlnU}0W-OmDjEPLG4_+!K_X z?2MSaSd9duS^rn*4kY>JAHk<&X_P^GRR$?0Os8QkGtH0svOam}cf^wqC~O>fNIZvi z)kk<32IL7Q51hfJ1}UbCT7xeVQD)(K8{-4mOiXc+43jkXeV=-G6e{~ed-+m2-`Hf8 zXLLc9*tELJ06sVWN@q$d20;fihsc#T79WgS8GBk2kh%BqKyBxJim~M%i>>-%6Id0` ztom8uhUhG0G)jz^&~_uQ0OsN#EtfqF8Fum#I-;-My-85;3;v#NM8#W8#^QF8mx;J{ue* z>02s&hsBn~7wW&!M1$b+We3BcN)AzX*^?5iQP(tu^AiUJ?B#In=9t!e_EA=fe)JS^ z<0%t1bFBu-@(s;0eUja_lF{v{fQY|ED68NS~Q{?N8B2I3|Rl*LrQZij#S)%3Y3{EC{XwP{-a4>e;o-+w3` zaXJaAATMo?iTEZL6H)9+%41RwbmhuiPcqyA-nFG>_9of$yIuxv`JVOvwUU1o#Rd>p z@$;yI0YvZ@?!WT<3T00jk>5u1y&WaJ*+vSB7XMJzT7g*ylOl_3wNX`NV_|(Fn zGSm*muUQFPf*5rg*NPr+TgH}+CUp8BKTEJ%Qn5vO%I6JC^=o5F1zmQ=dqU1KXy+G8 ztkvP0IrG}x0iIq}`I$U%Mn5-mY;F5Hf8MSvnu{84j6Si&Fa^b5GNAYHy2=5{DU9d?8?jRbCb`J0MC2xmd)cci{PdW3zCew>+dHsgTdv$wpVcgxO=(ij- z3`*#U!!Q<69=;)Y8uE#hv7pk@1=SI0r^afx_3i6oej3o^d}~tm{)x%`{s@!ssp&FBN(L3*1Uj}V zR~_5rbd^~Cbv+RuTEQ^dI6RG!LLi$?fm(D-tg`CyeF0~sU%)5=A{0cpqR9*<_+@PXo%4CBkYK670ijixiQ% z?{i#?S!k=4Zi~5}Os7cyijYAkU-t2;opJ0wA?qV1eWOMU1^;{(t2aoYXA8qiRI{|z zh%WF}$s3HI@?(f(Z>#n(s zY5?-UnvWQLOI5}t>gbB6-x@17M>c-`K zV;C95zn=SJB3Y)=SHn)qLfuAnfbBfvTyyzN*XY6cpgc`ELk4XwwphsK2_RU28==a7 zpUtPL!&S3-3tSF(#^dd|BP^gZHMkee~@wg z+kGv-CexUQIRBgOJK zd+)P_meWNLFMOJX*#G>tD;&@a{aI&hHyuSmpQ9|8soGuJC^G7$&om$BRzT>Wd1Em; z>D!)q7*nHXT-ei3raVwrNt-VQPZ&mQeVUBZCW|R=@3qLu_8*Nk{^^a_K%LfX?tYk>Uau7`V{+a}^K^7~6x>&=dm(CN|9Rq( z#$Jp;jCh&{lzT1)RQ>tHK$HGqMRZD6k+{ATD{SyUH(5+kB?@i>`#$mL03i`HR$-_x z+LIBzms-P9@cRq&(ERA(i72#fuAN<1SP7d7$iQuXnZsV$S`y{hOubCbJ8)XB4fo97 z@|tY)k#+fuTG)J$2Mf|wfQv^B%64b=(oXEuM;REoEDf&9V)Ww#>5BZ$eY^B$#B=1A zOv?$@B_#sIob0btP{V)UzWHzH&}FFJPlOAEF~|`+C471R3%aDd*05g7^cE0f{TN>4 zd8e)rPfUc5_fM?of0Nh%@fAIy!lTi|`&;5V!CMnrGZSD~|SDULwKhIF@ z78+^F?W5-HolMS5R6UB?mzgL_m~w~ND4qu7d0U6`9Q%;%#1W1l-8GY=7R+U_qs zRqIl1AEa3S(gE&fM<2@+R|VZ5dqfv#ba&NLacfLsQ1o%Sd$SImfNbM|5qOu^k0BAi zvq|ULS_taZpZ0}mhHp)R+EMTnLn!}U&f$2yMNGcX&9X^&p@)QlL@`#OvRLzlnPcR{ zz>j$^6={!%gTi>5^Kq{vezrd(9|UldwdwV|&fI-+YG$dRkUz4mVx|&|r4NGaqAIDR z9KLFnAI0_x8{no-IrXImQ}*@tec7^Ee|)-3+tWjdv8Pub$kaGtZ&o z$W&hH3jq!5URD=K@Acvp8I+fo_v23Zzcrk2wbIk2>*xEr|b-6#V+q1I@7{%-sJJKQSbr}oV zN$8|+#!`POuwS6$ziCc^1H*2-oTBU#K5|8?`hZ&D7??&@ab&BeDghSkibeOm@voo) zPP(E8NTJzz*m2UrfeJUCFjC%L8XI_5| zU%6y9@#+ey)HHn6FO)f7^n~bDQHQoe4IVEMwZhbN)WC8^XhvFHC7Wd$4?P}?B@=n~ zglEO)a8rY`GzhzoLN7DLjcrNpKg05PqPuActTuB@HKt;2J!lGJUBRdwyl|!`_xVb~ zLwjYA-x>dygS1Dj}()uPR4>{j6__eW4Vtu+@`z zOO{VH@vYw6+xsP2cXrk+BVu%g?E&2liOGem&+>7WvgwAwTw2vmDsK2F^!hufL~OxL zG5oV_`W~&ODiUx;cnY6v*XaN@marm&HlCm1Nf$vwu%c8QSi_WRM5ZsVE%W`vFofp; ze7&Btw3mf#(5$wge&3O-wRTP3-9hNW$ z2vKL6tnC5oPPAR2{mB>R(iigMxQR3pT*|^(4i|%kJIBf7>(4vs zTQQ}vT~QG+H3VAv4RwvXVUjKfLI>wRmo!$r%5XJ~ISe;%Bi%>JY$(YMLq4R!98K!B ziOH$98xo31pHe(?a*B42qptO+`?i$^-)<7Ps2nmVx=jtw{6s2wMThQOB{I5ndUV-; zk9~@GY^cGXD5sxs6L&OVrz>aUs46yR-xVtW$;z0QjhgH)$?9LCWN9?}RbBW9(tGcp z*^u;Cs(S!?^uJ~OekiKs8i#72g)kXa+;*rmUqkM zOX!2|i|@alb`3Fk?ExKyy7k?;b-CB^6x|B)M96#C2$|(D!sa?DKDw?0hly8xV5`sH zCU)C$)$hJM+o1RkTfkU2oXhSR-Id>qJBx2Te81{u5@r5=KZ?Sx8(v+ac(z>yMg4An zB7E08FWkbz-oogoLwP_X`xyq#rVEj%M!tNmO4tjjSh{AR#fUGA8GbRtyinTlOxp!j zOwn}cI`g#RRzq`B6&2BpAhZ6q&RY`=l*BjpynA|obW7S(MrSC5;%_fD3F?=d%2tWz zPq_FLWF1XB=*xXFmn!9Qv}Sam z%#>b1a+IgyZ&{QT#-?GXT|x>*cM`JJY{3wIl_m$ro3tS=zf7Qet%i19YSq@C1~079 z66M%fU)~Um^3Zt&(hu432=EJ-27GL97t=<&cSLJ#?e3JjhsDu6&kB3kP5A3wS2NR5uD?q08Uv9AG*L zMA#Jd$cUVxSA+Fs>&m9Or?H;s&y3K7dql-qmz~c7KYX7yI3)4X`0=Je^#od?#=ET{ z_@tYd6F**JE@2OPyjFwUAt65u@esyPDLNwK5RP4Uh~!3v!9CigX!QIKY}45<8S7HY&f7BP?Lw4Y;}zm&5k#9 zw3NW`SkpSgE>4$MK*~-jUi_#2;WYBzNN((|?8xY9(?C*MEK)ALqPj+5+$R*3ESC0L zSwu<2LV`l%+?W$8ox4}LR(igQb+bYq6 ze@zOD`&5W?)G~t9Gk$dFPz#8hk+3}+u^1RB4pLXp4JGUV2HXG3m;Uc319VpmN?4W# z_E!eAi?_zV_7TrfnUnP_8m%XQ;-$Rl^}~Wwab&)AGNu{cR;j0kSI%@=$Wgnu6bB+M z8;c}xew8G6O5b4WA;!)}q95)PX0snHa^DZxZC0p)BFGR`p48zNL zue#$A>n>(f|0W$bb^IjIRcP`Uty?A$PES~?O78v{EBYeeiPP;+*;Jz9ewURY^RW*= ziGw8F91NrCQY)Z}^P2?7PVszWq^ELzht~l=dr+}Uw<1vv*u0_Ap?zG)X)`4~ZBy^n z{F6qQ=T>|7VLT{WiRc1u$afODQ*jbVNpB{ zO<0)>k-eP0U`ULT#RY*wJG~3Xp0631jg6g4)j%hba!N|r{hIU^`$-0UOFWmBqPiJH ziT4Ce&N2AJ)oGKy?C0_bqwbZC0@LZlv4!O+&axw0c@6^4yO?3+1ZSx_+Sq)^LE8Y` zE6ZXkXSI3!tTl= zNH9?M6wXWqifj-{xo6(gCiS2u)>)P%7tk{T{QoScxa>reiAQb36)nO}ZfEzrNf0{V zX<`v=c5A#`UUntbpk_Kn6E^HKRwjop%!)IsbdMcQu{mFUKOnYTX#S~)%&pjDbKnM? zn;&a&5q+SCZeAbx9DT6f&Nj}esUXh1UxRPT0DV(n*)r5|L$o3yT1=z-(hFx?ytQ7{ z53#psHP+QG z0+_wL0_hQF4Zg5-ctL@(*YmVM>IXw>qd=>A606h(0;?>-PM=&H>S!VF)}EvAj3-L7zu1!c%Gn6X6g&#X`eeZ@&E zg;Mvi#Gq4mCL{U2Z6K|WN{Bz!ZiphsAO^3m)cK)*B+(?TAkA54!$G&=AiY+x8ue(@mnbOhd}4H_?F$}u*GCd+OMOSO zf=!u@tMJG3xaebqU}miGeeqRclxd{?T4R_(@&jinuSdD%Gjbg_gcPzc@{npo3>7B> zA~t96ynYGcIotjWzO}Fp!v%Tw*X8ty9-97oRn`26e^tI52Xu%Ddmz_R?2}MMAO-*fw)BDm3^AZ4Ok{A!@b%fi! z45n1xrwjDB)Stz5o%=lya}@ce@u?VHg;8n+-#(^fde3Elgq;6s^1~I zCK4KrtORoRO2VJBVC_}Hdc|B*i!^3jd0ZyS-Si)Cgal~9g~?fTX(ESh$aG%$W#QEF z2K3QCX$5wmzf8M(ne!D)xc~GDx*+sydDIb@C)sU}r4RAING)*ph+C?r1x90gI*m@# zvE+;A@8b!+^%Fyi>q6ImZFVK550h|-QH&ZSQ`94>G?Fz}BzTH&n6E6t_;0YZ-5={B zrW7_r8hzyclzqCcUa6DrW3*;xxw4c>nqGCE#yJkNV-$|WX~w4)4vB-K0;1~6`MkAJI0r+9xvi3THIHpxHyg-_=O z^qO7*-GHhamf;fn!KvoI80G0J82o_vwW|bkKZC6aS_F~~eLNr-Y~!PIYq{ngKn95a ziSe$>{JUlOPhF-J_*Jf4e-S|hc%A-Zx&*Q=U}PMn@T(1cXmcOB_X^3;cggc$>^7I$ zu^|WMP_~($fzvze9pn!CF_crfk48{&0_hR8y{NnY-VMydY4(6m!gPV}C5`y+wxK`! z4(PXmI8Hvp(jRh(8HD~#e1?Nx`_nYOl~^x@GsfK%<*R4US0(DyoL9o_ztyM9TH+8Y zVIj5WSzKy6T7_e#bCMMJV#U6v#kXm|O6wkhADW=1DRkn8AJIX&s4o1-el4FyCd4N^ zlC|T>SJz455d2gUiC14Qv{t8A=IMxZQ;w3ZmA&t{H+H-7gWsxTI^;1zJ?=D(+$2Y6 zFXijR?Mh~6XO>aQyV@}Ij*Z3leORxy2`{yKKNF%2=n~GLGX(nhGV#FUgf7un%h(B6$9T_bAamHeOd}`@M}pKfHKHWT+on@|zYOCCAu(mnJnbb&nqT0{wmY<^Z}0IxHFA{Sv3|y7gy% zqsaUXZ@f6UQ%Otoc*?QK&5VfrF8_5Pka*n$#vZPbPvR3`JIbS9=j$OgYq2l0l&_jj zn2zC$ra&N!p;pf59y#cOdLi=(;Wllj!2eRbY10~*msEd*H2)`Fe(i>Fz*QIO#r;VJ0koh)wbqDZ`H#9MnDsOg00Y#hr8?p?EC}ZLamm}CCK)j=%Y#aO67J+G|hCIiw|7-;&pS5mVZ7D*%r=^ETJQ#pR@-nXkd zx>oo=F1y=++=63#B~UZfZ5EwvZOQxNm2;WEX{ zLU@6!^j1~Q5tEbO%Uuj)d0hi%P)V=`Hw%~3IALD+WoJs~r4=4IOzXrDRceaui%tB9 z3@%p7I^ceAbIQaLv#?mItDt1pY>dmI-xsl;s#)SNG3MfvRy1bDvDEY-vKp28VGJ?| z({3}iZUyx5kM$nRTfWYxV~@aFNhN}X9AGLrK`lD?Vn+^^7d@C{)rmGSf*mN^h7P+n z5G=2KnB;}*^w$nD%=wTGz&`e4q5@>|gFNiyP)3meeR$)*HHF*v?K$#TfBFd~(68md zHAbZY>z&-DflC1Pab$O{Brd@4_rDDM91{u>U}1WG_f+UB%yU$&KRgedsGND`b)OsQ z>KdHv`c?sv3J$!YWWl|w^$XSS^c8MJKU>SaMjZc-3(EhFSJDJm8hC;LSxp z5-#Rp7Bwd4I3W5>aOP3SxvW3i59wx|w*0JxE@SAecUki=%3Evk&{DHoQLGkrcE8Qr z2ctaw0s9r?7e2%5zDn=8HpfpwJWBEjt3T$bxdG-yw7@SzkB9B!R@ zJA~xLQ)wdZQZNSs?UGo!>5IkG7MH5lC9}NiuVdo}>0c%6>NSeSP8TfXqQ0pB!j~t( zOc>-`bRj5t4>SA&BRx^Gn9xm^%MLMFy&nWdgg}J6Oi7-6N%rR97C%YRDgD=(C9!(V>qx%^h`@S?u2@?sq#u2> zGF*0;*#WH4$E1hU{ToXElm=a?t|~>Is2?JKWnqt>KYlYj7gAk7I9B;=hQgqyw(yDi z`#)!*#5)@JL`V@X)}$e>!krbV372R9W&(?Xkc z?HVhz_Kh>UPXzek7C=R%SukaCxWy>S>Z4Xu$3=Y<|2Wsk|NkJa|Cgd|=l4^_8f2{-uH@ zV4(c!7DL56wh%>kc?sl6Zdw9RvWWH1bUOr24#3JLjsMBt7}dY9#&rk4lpgKyAp1Y_ zF5q3R&#qErBzXK&j>Ziyns~y|O7(yTx3gt+>T;YXe1AjLDibMCyG?GYIv*1ewAB&8 z!o266juKIweB^2hb&nyKdHeJC@I-e<39l`t>>=Qy7p|U^+pO<^W_Q1B(wiHVn#R)4 zt7eo+aUp-AN_;D*Oa(bA()7yK z70*Pr{W+zz`^bb(pEYmku=dzJo|(O`nbhms1j$hfL61!&WwUt)4fe(d4w9te z>sN?SCt%Kx=uJ2)&ac>U%o%ax{8!TIX3e-+-Y|=FSgYlf&Sn}eYTW!zeSX5BY>;|y zAxpfyaV(q@A{Ix@}z%q&qUL>0)arqNsk0h8WnpCsUDFMk#4^k5~CfA;gy-hB#834GK5&y!~ zF4(jpQ1`u(T3enu){7cz`}J()o>-5HWc-&@@4G-AB;ihSb#3?>G8iPlZ*fjqQ_!U0 zJK2?zwmn$f?A6l;K#M!7L*lPt=)d~y_2gIdF3ZM@>#y1HHT#vU1r2_~Lgj|X{;r>OI8Z&ke z16#a!b%#!Xv*~I^ZMS^#|Lsjag5RWC^hT3M_dx7vowR;T(%0-ZoCvWJhK z>Ri=}ML#~Y{*{sWt34K26Pdl1u4qJ0)Kd9c_5Hi2gAOd^i{HN7jy*Bp9Q}Qi|Iz2< zbU3EIfR@{1p{q}Uj%CNgMgD<_2)L|@av_B16A%sFdolfvJ&u=JKvZ@q_W%BXdPH<) zZz5ZE-`b=(mRe3P2)UzH+yxGwg9ZH>e@GMokBDz8n>JWobb0A#5`HlL2jdj-b!^Esq&4`V=GJoOWA++_F53OJ z(>%!!YCI^-15S7U0@N%^J^QD!Fvcn=10f6Q&xTa$X_Ls07g>Is7rdgJqfLEDUB5}# zB+Pv*gB}|J?jWg=mX54;10Ai?hO|;+EtcV)%Y#p#2V*~bCUWr=pb@zk{)OnhoUkcc zQvJO|@|&U2VqcGh^k+`PG&I)JMgBC^!8G?DW+JN|i^liR(KU3gyx4nUd?XfpkW>Se zban`BpgQ8Db>3~fxmHTNx!rGU=%1ZU>rZ~Em zVgY|LI`0JeLaE#nhKoc~W{lI>WrRl@wbNTE(YbWnNf`chWS1-lDf4bUrIG>TH*7F_B`=@-JxqC!fRkF`Ce8J?fKUq( z(&ayy#y`Fz`^Bp-F3D9 z9?)LzUDDX-wOj;mmgHwC@1Rp>(6?ryr)1#Ij`mtjB=}g`$S_#EZ|vn1lG~7pk%#Ma zm@lAk&-0Bhl(DJzUo~XI?2K$HH{LLPEPQ1<*^F% z%+Afm6V*#OhyLWPO;+5y3zzZKYhu3_buGib0O7K_vEB*SJL3B$5SDbwy<`qZA&ncx zze_K_?^}=F`v8s{7uu(Xb62km=+3~Lu`uNke!BhSbm*qnI zgP66l^?c{*bi#WU(F6_G>eb z&g-nsO@qnSE}GF|>th{dvb?r#a#d^u$m-rVY>}YP)4bGi5otOyXA(Jo)2xSGV6u(n zV`6r-OcNI;7Jgnn>N0rpM0wSiA)3|t4nJ>yNZ3y8$=Y>y3ZB#Zv=baDWQK-W`)pwq0XZ z`$-g|qgi%mS=;e@KkfT~tpNI34&h8aD|uyF26plee_T3D$&xL7FogQ9wFW7Yqv+TC z{9e@zBJ{u+tCGKkLsp=tTb3c)bcJvazeNvAaMICY0@9=bIxZ#cQqfJ^jNL zYu9&;wm7w?To+P)KSnWu9DXqPTAckvgt!bN$3{LceLl1*LPRib(|w=2bdOzyZr6EU;;7Di_bw1e7eF6n@{z{Xk5`eFH`vVn)5GwF z(AsG9ANXJ2wS!Ng7%qkcH%v}WZdj*=98|yKeo$TZRymw3e=PSIXK|ostNcpOg3h~2 zYACa{)^&TrcuqdZM^Md=sV(lEk;ZJ{3Edl=vjGRH7Cp)$jaJhkS$qUObYd2WaO$ca zYe8M}IeS(CU2_;^aJSKOnzAuz)^TmV;$rBh5jM1%ox~y38kD zDj|a(?NaCEYxV4VnD*B@n(wJ&`tYio$RTN;HUtSqNUj$qKy=*4#>F6U;dNxFYaKVu zd|mu{Rr+q!(|69-gydn2tnogN=vy&9mC57gc{6&F!6 zTYzam_Det4^GImtJ;&S=zkSExnL&PC*AcUJ>!_`EUr4D~b=Nof1I6|}-`?((WUH=q zUz16$xL3j|?&siXg}AZx;pKIyf;IhdLyxqnB!Lf%ib768bt}i~%>fq`t8@lO^;B=T z9m06>Pd}myK+myI{y==BLLn82c!aH2=6M3590?osXF~SviX^_$E|>(Vt->zT+|QU#EjNUaSP`qc-DP8YX>r57yIq=SVo1 zBdC|O6P#U#aII;6V*M92QkT0P;k>m*_QP{r&#qJar$@jx(m!0R8L{M>PL$yP&yPv* z0C76_dy(*46YMwmeH%Orx$q#V`awX&fN)`*-CVJn|DkGGVBz-QdBwG~_CI35%BA3t zVG9!9j4J+_k0}t6W}XZ53+c8Di;ko6JSD62sPK$*LnBQZu=4tv#j~QDGtl&QAQ|sju83)$ zQ~j|6$%`HO2QAQp^cxPpADptBKG%gMO4FdNLAgA>+D(5_*r!v1CD^BGKp9lVGV#J5 z{cL_5j^h<$j|t?p#u(zkn#&RAjuWs0`){y`1i@!Gd#NO^C`=G7-JCLC;=UzLvSXT> zGf(|9EwfFEtE9JeO8-mxGzw6_9lPKD9mPFkDlNJFPse{o0b>;47j8nW1*i5ZKzAaq z-$bs|0H${5o|-e=QDYAMHny@qU^7Tw;AU>SFsNf9Wum^m=&-YkI`5OYb_1VDuMDJ; zZ?*IA2ltFUa%#*8vGvJ0W_0RePs&vCqNnAoL$d`q=kCyf6T1R}2m4i^>AStY8E@+M zvy_=cVrS2o*0c1wg2d(0`Lbyuzb%8`|y}Qvgsb z{|QY1)&j84zso-?39L_h^Vj%vq&%xC|L_#}o7qT^EljnE`cM>S=1cy&AfYN*?MP== z$ugP7{;|+VMXauo->EfhkrGn+c30N^;Sqlh5R7d4@ci^D{*3?Tl&T=dsXF=25Dvn5 zu)h_7W;v94EES%aFiGA4`Ox|O~L?4veqDRh~K(;?lYhYb#^5J(1_WHCM>;>L~w7;Nj zI|Wlo^Y_}98sk%WJ$}lg#Ag1f)yEsof#{szOx%3HJh;KPtT>MEYCDDu) z{#$tY-`9vyLP(X9C{=s?jY?ckhqCA3p_376;m}2 zFh>zREGsB_&^MpL$Ugkx_sl(7lz4L%(ZHY~<>ZuXNN%&wKoN-!AK~dm&Eg9j8u)>yPD0CQE zkR~d#e};}LG%#o`E>)@u*31bSX%ZwE>g|0otVXH8<5QhJCr>QfZg9X@w$z3aqnF#4 z(^Z`2M+0A`2OIG}PdB8$?PQ1+j} zPY>&PtfN!u8W^(et0g>)Z8ORD7LK)-blA_6VKFdm71w-NKJB2g^TK&B`UAbVB+MT3 zEA?TGUrcEo^^w$XStxxQtnHK^#%BE!Eh9B^=8%34Mpl?-)}`@HEZLMn$8#YQX5Ohp z3EyRy00af!-*kr+sY&4u7FVHZHKG3#WLBo|c5drfWkl#+8<9!+OFH^*Tzr=X;*2WU zU$dJB0b!>6!!&P~x|1O(AM?tnbLAiX%7q`Cvk@nrr}!?Ef%Ek*B|2`b)+)4{*y|0@ zDCJ(`>v-pX2R267bzC$$OQ$@aLovY@uPWP^c(bFs%`Y|Yxpj9bA7spgV($iV!~?V@ zWSGcrW|zG@yL4%>%P8}=jmAlTueIC$(xzH(=S^ft}XfO<7{Z;zjX zEUb)gveV0T4S=7gB=*!N5{nSaR-gbseTnr zUqR;y%C`ANa=ywIcTEH}+KM`M-HXuc?RY_gOK^E-Zhrp?CqO{%r42TFbkXJAV}ZmgcCv$=;+j zUGL-kv8M;+`|Kfa{WDz}={6&NMu}sHYMdf3NPl)TZ|(+h#zOs@JI;6JJ8t+18Rd~Y z&ZpPl`AD!7O_ZcfShS~8c4!$vF0sJgYt2Ts&TcXXt>W6$3wx>QtuF_wqJ@~XwG!h+6oONDx|)}IH&h}M1J4ngfR>9E{K7_ zPE%P&sz)wr#ebfNmpE@6hiQa=vxa#;L^F0wQTu$$UKq1CLU|6+q+2$<8G2O?o3-@F z1P7;qrV5~1n^7xr&c@YLHfJS8#;JR*cP9?Nb{D>~%ba*6;`ff@_P)Np#jy%#vbuNp)fe7afoDIw+aGk8 zYB~zA9aG;wL#E;D-v7gA8vCiv0swTS-1%#b^sfqQXNsZ}02_KzP2g`_Isn1`nHGL_ zK4a>5u@ZT)!$s+y*I5e+>YP>zSFf=!HS&9uUZniQS1d9?jqOu7T9^sajKACEQ(e4d&rJ(=os7K7-+aq z?gQ3X#q!2m%JM+>LJA|j&onniiV_<56)k`$0`Jo&MKv5LkthRo$wa z;w>I)0AV`kw&;Jz6SNY`KNv^UROKOfj-se#a8Vr^kY|avolSOpn~G*O3abPsXZLPW zS9JBi+(yr%8uL?nTqx~Ilv_C}n6}XL@rA*TB+T`rs`mBBFowG_p~XSv(>XZ$BckG9 z57psT0o|#~VArEg75O?;%(#qpx1{csu4`w~8YF=MY7 z^JduF1g)#Wyr&O0<5$Xd|G#L@>{up5Hr52lXzR5nJbz6^{|ZvI#6bNv`2o2t_*WPS zzn8Wh-rX1@&ba$=!lIaqr+u65o0m}m8&Ql14*KU3A*}KE_ElfJD`=i`Axna!6#HheeAg1Sjb~#tOevDJE-U zUn=KI=~8AG)q;98e^OaEIUX!)<>-iXUNAu7s@3wwZiY5+2F7(iW&)8XVS3@YU{#oR zL!>Jk%aFH-dsNGc6nmY=xVXqs6@HZ87(;g10DbN$X-62fDdbnwBCFSH_Iat*h6mgB zYM-1dmUYjSm%r8N(g-{g4pS$Z%<-Xz8}?;^6kiWYn)cKQ)3EH&GrOnjBKNs^jgO)n zef9gGo_r%5GZ{D}Mp;-#yFg!RY1yXPxn<2lmu6{aFB@br5-7+p)*M%s--{M=92wI) zx{MK7qw!C4JPo)A=i9vXjN^LE27OkYQ6KYqhId3_KVxc_I-FtRutufm`|DfsGRd%v ztwDAK3pxa0Q+Uv3R50$OdiX6|z^8Fq&4 z5$qAAj^=rAf~rOI3vUx@d(%QxZsgt4tg{NcMQSh2{B-X((4s3ZR#4n(Utl9jr@eXPFjcwvqf8z{$zC!8<$CM!mjs%XH;5u3&=PZ z6Ds@-m2ht*|C;bZk!Z1K)S@L7qPV2rJN*-wI7hUk!KkYZX8=tutCjz}XP6Px^_wvi zhJlKigNQ5jEqk+{OiX8^<@;A!G@WLVS|I~9=gI4L?Z!V{EYRpXO-`GJ%y=pb$>(8Q zO*^P0B-ca=a(e9IsXbLr#C8E>PF%$PG!ugN4;Dq${ zQqwc>VB3~4YH(Wuu*1_F;X9e?UeJ`b+;iSAfhh2;S|qdLoux0f zbx*RhqK#g(`{EQwMxN$;obny=){!wgwYA%FD|1il;RZ-Xc7J12pUO`6XG9E0 zu|xLq%Rv1kGFfQhzNlqJ#OAj?Y@$Dr1Pjoq^JqvjwGd zy!Z2nCsOis_PKN2&0kJawiyw)%+<21O6)&8%oG3^imNtv|MgwdHa8O0QM!Q9d9bL{ zt+yi_oJ0OaJDe2O@bqb=Ym|FEZI=m0){^yKYT0E|ck7ZAwVcXkZVSrH61P5i-_1pB z{e1VR`N-ly>erZCKkvOz3*wYuAQtp=eCX5{60S}VT4+vvZ|^%@9p6&OiY}I{@w+R; z;A>q>k$(H-vzP#XqdUwTLeK7YVj8>D!;(XeL`{Xy5|Wor_k}!rW!iI(y_tQtdhrFa z6Oa8gnMWjq*hoi!DVIh>%i(+Fd*1b-JJf^1`$zT?NCGUuh15Z)eQ#H$6}hz?$-(K` z#hb~5^&pr~zx4)J^N2}?HOsSvYe%_F$3zPsdGH&N4TS4_XfCKKrdTi&Cv?cY`Cu1O zsE;tiTzd4bTyI3Tiwx%EHzyu+))L@U;U$uwEp-^hesSKmt{tjwKIy#J5NqAC)&Ela zVzoInRpGl^l8jY#R8i2jQQ~6c(sOI2k=>z+NBv~U&EI@* z#V4AZ4xL18%6A+;?~ROEjuW#%!akK9n)b#lxBAGTCH2)(NFbG{wHeY0gTYa&AlE7V zNbCm~pQ)dQrqSM1UA@7?q5c$H#UW1}wSbD`jbs3Kt8!oTUVYOHBvCJ zH7{KvrSDkV<+oL7M-HwRNKEX^eDoC`+_f}G$H`L-|k{!xLq$3@+aK3T@4a_n74azZQ4s^|wuI=Swf*al> z$%dZr>L{e8V8q$Fd;n3eKjuv=AvbUFEKbYiI{C|u8j=MFo1r(!=|{aYe>n{H&IAnT zh#8JHVl?Gpmo^{ud)hw=4-P$aOIi#V>}&9qtn6)&=Pg-ySYOI7T2cf4IlwrE@dc9E)}%PU zJG^#d!FzQndWcw{7w)7XZOml}@=L_k98Bjl@vQnAh_^E~fv?bT690wV6!33Ps}rQX zX2?Q=(`Kvyogcq#PyuvVx{OWs8CVXXN13f*eh2erY4WT}?PK*!^2N&+g(SfjINI|Z zp^}~Fbj_nC{FamEPetBj7tBA0W}+imKMI?37s14=16eJ%O?DWl zE?Pq`(X=K6D1g2{1f-zBJ1)Se%o zai*%wjdQ46S9=hauk;70w9}`hj*SUiSoD>%tS{}svJ3g!`YcZZ@=~uyX|&*Gwr2e( z)0e837RCP40#klPdl6)5ydE}jYD8;7f6Qis1wVAY$iM|py}U}Vr&+)^=lL4@>U&Fh z(9ZO|cQfDhb-K~7PxJCK1AbZ z?_3Zb?XfR)eA~8wxaZF{=l#i^qs!2}6d$Vs;N+>g6WMPZRm=UgEoc*bZTtJmx~8zUR#9y_t=S6M zgDM_s3FCwHgIVVr2MR@iR;jTTV(3eS*V%Vq#Dai!2corQZDeWU*DnxSm$dv64F=tYoh~vY1=ZM zo)@-e$-5}WtpmS}F4i;@Ltk0pm(YgfzgxZ}m(q6_cIo}@K@E50PHXp^YTQlhQ_Ztl z#gX&%-`2;t{+hRJEBh88-EL8DsF-6^kv0&86e*pU4=jpZgO8PDYt4zS^P@`Eq{sV9 zNJoVbcXDpfYJY$BV)_8l%`+iRE*slHOsByn)f3nx|EW=m_Gw75eSg9Xjm}R5@BN|_ z6Bw@AXtM$MrB|Eo-CW0ypNw!T7ZqRrYZbj<%3!0lrkfI0?!WC|?n0qIa>E|K85nIe z>Ev_{t1e`_0a?U{TRHi#(pAq#w%I~&$5@bP^pM%vN--~a)|(Crm7nmMUl>_y(>{5b zgB(}`n$irtu720htY#H2c%a1Ylm9%dxIpsj*AY)%ZBJH{;7;w`26~`5^CcL!QPgtp^yQEy#so;~m@Yn$vuk^L=|FElyo43nyikt>#A*8@aFzg&~h9RLlC(cEy6YLZ+2BF8Se~SF&)`HwySc8a8gC0Pa8>SfqKE zugTXLuSu`7{n6!kmWM3RpGnQ9cvTwNQE#KRb~Bz0q_1qHH=(On)~cS|Q_3@&kY+{Y zDb=D=H|}l=@Opc(9R8x8e+d{O8_d{T5>0ClcKgwLIP`%uFjE{SXUT7t;PTry{|`&V zK?7Mb(!Yh@ll(#_M9^Z>QS;m)MvyB+&Gv}m{CJ1}4olLH_ zR-mNr?I{ox7w{mN@gRG<8W3#F@WGvnQ5|VEjlHHYP@OwuS_8H0s6fb6u?ND;b;7Zx z=v`VY(0i`Jy0r0b|EDV`6y~;Zk zd-fi&*ggN0?4n6}BQO51W?zR>Qp*H`OWHg0N|odE{)cmyOm{}mz=t85H&SbE+_mTf z&@MEraNo{$x|?^AH$NzofTb3mKPounlw~KC4%IIvTHLX7&>2PMLccX;Nxhx%t|aP9 zU;+#(30eRHu_NXAiuzeG{Bm$Xeetleq?@TbFNrBmcTjL3w?JNc&6g%hWZY6=Ag3V$ zT6YFaQg`sxclM*(AK^a*WRI3kRm|7?NxNVss5(ETTHTq3ttDY{6Ye=pg84xra5ry( zbeyq?pl)A6x3OqJm9ZYjEH*eN$j;fR4Q!<5YSzMn^^G_AaePAi6#fq~qOpfQs)1nh zb)Uj%&-sfWx2A*rnui_BSFBQc${uzUBz&ROj3QXM`L{UcpE`MW!ZNb|mStSK%mxvh*RtnM`#T;HW|H3klFF#73seXiASc(fAu&^ES@ zmI6~_y?3wEw%}8jMn-3bzIE5j#p>d2dJb!_1yn~=$;gr~q3DUgjy?n9`e8&U`O!}M zCKvO|z(Uqfsvw>y83|zgrHicPyR-^;IjU*QL$QIn+2cNptr>0^M-CpVd08-Z+ybW* zZojLEm&JxXp1-Fs)Na+*V-BWCo@Jd2;cbm9j*Ha;`=Vt^F^y#t`6c>$Z?-OhKF@1L zXIsk82zd&#n-)Mj?1WSf%mKwXxXsH!9sOY;PQ#Up{<&-c|lI1Yp}XB&sL`Pdc#E{BRgUc?|nts&f6j zV%N``+9t^Sa|l9H96!E+ZOIu2`DVPFlEKIeyJ%^35W@DC5$uO$#KG`JXZUX1oU|(= zVf%8B_CfI;owP@^DuO?a9 zltmcY-!4!tE3?q2VAccaLP}ONyJR#pls5vaFrvubTKUm z)u|Pqm7wHzw2GzP4PFgC=|LMwsG|k5l%WsctG>s?W5O||$^57%{qaUhQ+?p> z&&ZCQJ%-;D&i`hgLwnlP_;GZ~f>?wP((lqTl3Yd!uC^jmfvg5kD)2 zmO!QqxLns%ce|>*E+3=R>=>X2HO*CPAaV?Zw7({~M=p3-=87EEv+ej^J<8ib z*RTZyW7fYwoCO>U3;ey{fP}KHjwMWo#Ejf*nG~l*rX7F|6ViQHc-Liz)82GOXeug0 z-Qud6Ipe;I)Sts1grP;G{EYRL7;Hn|uxi|s{x_!~mpsyrv=Q4{NZ_n=dbrRP`KU%oK8~3ag<|_B~(y3eJ5IL8M64}AOmM&GUksRMdG*QGt+AWA0agxUT?Ij@sDba z{~v}?K8rtse;r1hS-n$b6?izpd0HhteWxj8ujLq`7w_`Trhk1CjYpL*lKG(H?bD=+ z(|+oLE?M!u_e(9`|9h+A)uOd3cP~E$+25AZ5bN%H-Hw0sxw>>HhpHa!MdAUvhUO)7 z?JRBfwrki}{dYOG!(3VHI*^xR)r{FZ>meKwd9o_kRrQ;N$=~SqGmF68s~Bs zTWhT|?nMitjZ34rNCxgEs~2wO?LK$=89kr(zh{L{pJq4fhI`qV|X@9c7AMdh+V zjT_{VC?%%Kd#B`Bk2e}z%6~NE2~hG>XT?CoP1d4xu@LX&{krmz?IW!QO{#IFd$q#Lo3|ld_4G(S$pZ+m1k$~9QV7>5kBRt9~xwOQ`vY&zWzb)FSk6*wnpMn20gKs<@JLy_K=sZ;|>zj4a@+(B&JI^ zA<>EgDg$c3!?MTo^S)+B%U-~1#dOm(?hvyb6a4CSju=ScxTx$e&|p6-CL>^fJ|{&~ z2U&jrI>I&7)W}A^C^-?YLVe?DkY*zDO+@eGgHotEm%p8Iat$K}w2Qp=GfMCXN9JDD zp0GR7=(=nSk zn}JGx-gAw1QG*ZE+Oq8ZJ?SBjZj7AJ5b8{M8fqOr26JNjx+JCTRQ_Y@Vr*YrC{^0l zSc)38o)zi7Vh^%tEFD%KL<3+ii+9v7pNZ*{F|^$;j6fq%MVW9ma;G`#2MissCHg%Lq1w#nusu8GoSL;6JG^46)ZR`i0B;?rv*!-N9DhpSt~qS{+T3v zdZka+bqDiY5V5`4y#(f|>VdA*jAUu=U+2AQkO6|^hz09qI%;a;JNuL*XX1;OTff-K zferRJ@YDP4eA$i4*yOl)YtD$VNq8NQst+^4eKQvsy_G%Ii*~c$NZC3srJRSOv$&g% zt4UW3vW@UJsA)i`uu^l5H+51(m8ru$?ymh3*E5==X=3TQ6V09F(~?x$S8m#Ss4WTZzmB{}aQVA{Uf_Im1A zRHLZKD{Bm03Cn!jD$61X-fGLH>518lTjT95IzP7OI-lBe?Px|S&se!JJv;>Er zw~}J^x+FGNXq$-u<;o;d%J=WE>ysKa=Zp}K*G$m|1NF9gr^iOQpE?S(RisywlyD-T z8;JJQaFQOs>B8=$JG-F4*qt*e$4i`bXdcSK=DVz}YqkE3R#$ZBR~7t>Ef*qw5XuNo*C z!;_@+jqkg%KYu!fbH{?q6FM$ZQr8$>E#k((i;2@>lW7uREZ9%q-)_A<+C(a>98S6K zV6r+x8PpdwS?+*jxX+!nAP)G?^GIal)T!bf){^M7?Nf)rqdKz8z$~Oo#!9TgR+P98 zVfxfcFOUxtBtt0Ra>&*S-SM$(9&e*w`7&rUd^51~8dH23s>9oZ*t)6tZagpdQIOzXe^J?Y3l&xR7Bk_Y(c7DrPR*UXJya9$lR@-XRR#@qL9Xe^a`DY5=A zpcES$***)(3gDCu;SGXM|C2V(;2b$;q(pTzkXXrhgE~KKKwG#g`iWevguEnj7Dr--O1DSX69OP; zWY=yuR`GuDh_gI96UQYZ>ep@940ion+KQNVLe1A3q{b5hTryUv$T{daeo;+|U$c1Wu5jT1qTIvVc>{$>p}bsk7btw^)oye+7TMQW44s4?oFZeD%^_mom%<^@l7Iqg3O0wgZS5IE# zWTxne301z;=EwR85}vv07Q8z#b0y^P=V-$X{87b;UB{U@RP>+z^1IKqQzxY0X@>DX zLH`>O|9}7R+c_85lS^7B^<>>!c#-SLw%~roM+_&&%0TZAaH>b_pi=Ay}PIYf{tmjNcw(<#)S6YeUYnPgLrRa7wJS-66F5z*ePUE8kw^oJ~ z50;9ay`A;E`U8hCn*xGNzDs{{KQpc14&smU_q=1jPLS2|ekCE{b`UR7`N-9RH26W? z8AUqPJ0BhO?(CsONYBWD#j6~HBe#Idc*Z?F1XVv8p65LZC)5NP|rD@jt1M{SN)RMT&^49pUAvUDa z+bnYwRFODPL2mXD_+fw!rQZZxoQlLr~|Z=Hi?Q>7UP<0ILgm-PMj&Oxg}zy~#c{67Jj`g`!))+KS!N_Y~Y zRGR&!+bbjSxAw5;<-;hu>qE%!bc8V{m}ZdUEIXwq+u=RgDDmi7QbIKba@*jVTSdE;BKSWZD20i;Yd2 z+a@tgv%&^swXW)NjCo>RNq?)!WDq+CR&H+_U|(uo2NAgPL)!qft>)-}(NgPe;ljs< zTHfu8o$Z~TCWLOfWiICSv-~?aN8uqzgrw=BiCYInC*b-ZzM=(=d)Jc2C|+5-Kdo^{ z8lefAhgEe6J9S+wpSoA`kJ0Pe?`KV7zq+9f`~3?~6Cj6nv_tfQeoz)$MS%p{YyQJvHP?6DIJC8mGivrCCh}k3Iy~#ewibTf;*OPuGTazudS&9WBLl=Y$f)JPZ4; z6|Bdo_NPZm{`;GGadMzCiWCi`Fuy(lwKxwN*HOkstCr05z&;08TD9E1icI)_REy9_ zwXoQi^!~P5M4$0^w{`<{g<`5Vi?%s#bDV4U7XC6N{Xpm2t=J;i$V3jTK=lO+G0#Kq zF-!v4xE=&d=@u7m?+Fh=$(${S{Su8xQ?ONPC0j=NkO7qV^W6I&6T zLe{DPaDH(B@Qc;rkrr4E*$=WqijlIA_Vc_ST?Xl&)gits7lhzMIpdLoje|VO&XUe~ zgSD&zMiBy|azB+45darhS-yWf%$z>a!~T48J5%3E_L#nm@{C8tL|<*X)9n3}#K=zj zl_Wh&XQh@>yZo4^n*2Mi@Pl`^xnwSm{4ZvdaEXx%yJ0mu7rqg3zA;k5F`7O7KuA~Q7-p?zgKu>rM*&~D{o>P=sC zg{L_h;thXpl%$YS2+0k#snQ0$=>tMr0m0u6L`L!G5kk<-mB^oeLb8oBPiy0(ElTPg z5@y!#&(;RC2EB^@5GR*!O@*D9xin9PTF7R|+AOM)J%r{O#cRpCO@yo_-(K79DY1$V zqMkiG?jbfD9Gv84j*qhvf;YR#KSNhhv@%*j<*EPHE&9(X6B4)Ew`;x{EjqlV~C zVDA9Urvn=l8~Rqon|M3nv-@X<^&z!tqd0H51TImtBp!>oYL?jMcaP3Cr&@_VB6ybz zfmT|~YSx!DI%6v4fVaxr9ERS^%w79IGWIzLvb|jwDtf{? z=e4pfqbKG@L*Rbt(c^)?gQLlsg?KYT`-##F+B4PDukW}7%O9ILaBHJ{@oH@E)+ z9N?V2hAOJvyH!=F-ganSm#IrjP^%n}3cutHS6jw?S@}^_!BFL&<*zjG*q2-!5J)K4 z+VdLxqT4}c=Ch*t9@f7961Zdhxa>}>5vDo^uWULlw8ZWnQ$iT&nNvN!z97BFNDsXO zD@M;c@Cy!=lFke{*KeC<#fMOCC|wt4^>88NE~& zVEp7*D#L0vCD0n^{6m*pV)jimvyP`f8W0yEI=K?S(!qaqxH|aO z7-sNwP1z0aKE@JOSEpd#Eh3YyeA#V$LqUyJtieHnvjdFPzFn%BGIY*G^t7+Raq_Ma zkrt@?j@x*6cB2`lSB1Gx*4@I>cKF`LSb<+rWgY4x^$C=TY&E)CMlG78Aj*%r^rsj) z&6+ntD^s@Rol(6dC_ie?MJrLRc`F;TCY9X$2vS)#pU%y=BJVOCZmfB+T-Fs6UJiRp z)~{IAR=i^ykuj%w%mGVfaM?a~!cS8SV}V=!c>?)0nov|@0(4mO&$xWVI~^ zTBxq6sZoG$FL{YlwoYc{H5}^+c;WtWA{)5Rw6@%sKEbwTvOAIYpVN@eXGeI^zt7a} z=TEF)Hxt(s{_E6p^5x{^GlpvXh{;ve-M=4gvpCy-L(dm&i#<_b)ngFh@OQ9v<8WFq z%ac#MUjhqAy5H)D@TiqsjFR%AbHmbeC_bvL=WxDtS0IF zqrgvbP)Y*Px{ffRk6{Zt)WCW)mRs*k-<}r}iYF%0^PbMbJ*;tUEokPz+yBg+7>ypd zRd$;kS&u1>SuHlX#cQrv!;1etf{OL}Vn_?NXo_W}79JPK(iIOEzUt`Xi(mbs?zZs& z7LuRir%e#4Nmxr0YxYTDRJg)r@1ZldeaYePZy`~I1JgHR9&X3tGHj+&qL)Yl@kB4$ zUN$Y;^_$G3;MB`Dg@rpP1tVEi+?qhNBJj*+-yR)hR>He62jg+rPc`_899XSPfG%4f zuCIP!^oD2iu-qf8jP-Kgcut&XZ>WohUt5*$O+X7unWzM13_g8tHoW)TdGN>9xC{UI zQ4O})Z47&)i@>)vqzmK|H209pr$r$i7VZ#F1MFi`tKv#=1E*~DGC$ue=JSpA=skmq zuh8zJ;NAd(V3+oGDuf_d+u&g;jW@zZy@k@~6<5oXgJM$nWbz&5-a;LWm;B;!4YiN1 zgiwCYkPmkzx@(l*X^iHQKB|r^{`=wMmvqvmv|dNd+WuDWHe#c-nT^{m2N4j?(hi{L z`kk27O1jk|M`dJ)SmW6oZK3R|w&&UbP_Nh`+ZK$mjYu{AOy?v&F*9nYl&pC}aRwVb z^p0RPbJmmTTW$0*pplpCcjrus=wjNGWwaA(j)u^9+Y9150fK9XZLc`7gATW#HIvE= zq^rW5deHInpq+OdsF#7OKV74b@R~#*9OZ##sT&GrlEouR+wLnd$cFhaLEK zrX!mJkMe9x>11p5lf_T3eFu9$if28ri{t+ZGaqfgYu||41P^l)3t&DSzO3>IU|lECt$3H%-xJ{~o{^Z_DHrrCuvV+JuG0-7SMeBlrdcfdJasmQ7+ zl*&>JgAbGWKJ0X?>S1aX-&4oPn*Gb7PL5eY2z*|msNO+7Oc9vc{ykWFla>;90p0z- zPN3M`|3qFIl9Nu*%SncJt|)`5f}p9^0#83TX#bY1KZz@ZN}s@!iO-IHcYfPzJg$fK z=4P>FxoQf^l`USAB(H*9GaokFkxM16oNChCjexjou}SudhS%~oZ{JVZ4Q4x9kefnw z{gKuEu_#K4ow|`)>k}j*r^zqmIsQ7B#En?HGS&IoAJDBYeDno?1@p7hjTX(l@i&mT zsGVu^0B~S0d|kCZ^PCdgEQqsbw@|Y0fq8k2;Y&>pQIQA>zVd^9e^)9guQ@C}ovBk) z<()WrSA0Wl@srt~15NreO5ED_+~dGiqPd|@4SSNF_vpiFk1&2eKl|Dpa?-hjveQ!6 z>xDSgiJ4BIf-Nj%x3O9vf(c{y8nuCOvZ4emK*lHVho;!NR6_e_dYI8i8RBSBj;#W= zsi4`#YPk3@Rd55)p5b|eWV)TOzItvq^Jx9z5M z_hW~HRMOUyrEv+jsyhPA74bZAtL1wX#r5Mt@@^WJNHBi?96ktq`IQp?EIRgZ8vtm-QYgco24GY6?Hap*lfV$odq{vt78&ZC0lRanyX}vbu$+kZu=uN z8T~*KiHPPD9Ya_vuSDak9`#pSKK3{md%Bwvy=CB1&2bF*ri1w?>DZ|nki3f|_80A) zX`V8)^S=#~f;?OI<0yG*2RA}9PBVF)Z3>geD2v@mzQ?&EF}oXk_Gmq}YnOrYOjq_o zv~-Q7V&toGv#i&eZ;BV?@=Qt>_KiTu_M^?u(X=Qh}_Q z)!%U}5@8sCKJpTxpMUMGe6i>busB@;x;?x)@Y1e z7_zX*ry;mI2#4UBm2UE99ngbiVYnXFWTtxOCO)|S4!@u*@BTnldFI)B#y}m1CoQn= zJl_waS0N;33u3!wq2%z_K#+foa}6;IqSe4dgz3mGr|icT`no%p@?Ma@Oc|AtjW>>( z<{|v}1-|z)x|Kdfam6Vfcl!gsqv_$bZRbhtvFkQmuHeQyPgRW@W&5uoZrj*o#cQ43{XdBo-`18w znpp~E<<_*jA7|BxWhT}gOqD1Hl4j?bEdM>U{SQI=)%V0dPE>>`|87A3dS)e)mSkz3 z%EJhWu(Lhd6hzy7G!=cbqH60aG5kX2?c*3TR+F)oPj`<(3SC7W6{(^|Wm{pAr#G%0 zkg6XD#qPSE8PBjmBtCL*IxRe9SN)@?Apq@zfk-+wXRuc7@_)nc9!L0B7HfaNy7&bn z%DS(8AI!(8qH=F@pq-Tr0sKx%pGZ*cEC%82W! z;OoT`WNQOTI@RP|!oh4s!OchZ{b<_mg~6Ve%e##+MgU`@u!}TSeb;l0#p+T;F;mj&Vbl2h2;XEQK*YxbJ9-nu8Qow8glfymKS1yy|A6o{R z)g>EzIl*r;rM=>Yxd!~7YB`H#y}I+>M3Ck_p8!j%T{c-I<3Uu0;-&&wwfD-1z5afc z{ugxcV<`qUInk9@ykW1z84war2WijhrcwG3C32__c%$~GdUv!M>|S$+ptgcDLUInD zz@=BKhD|g(3I_8U0E{<@)cz+vkFX|`8g zu#m70z#a?ZX|4li*?r5p-;OUQoc`CvlOH8Kp1}V;E(gCxGuyckOEFV2oNsM{d7(k( zX3+&g6X!oZzb=k99(%0XuEr=)$rG>nh+NHxPw=O3tsfVq-e&i*En=BiFb9+=8Fvh^ z7}+-R6llv>MT3E%qB@`*Y5Erxqw8xphn}d+ zq7b2WQj*cNB!8&jEPd$k_YfY zw_5MB&wB=W(Od&hW0Ne*xVP5QYI5{R`ceh(uS_M*nuziVd<82$!F+OLY}y_mRI+c^ z{&LY0ehSz%RfH*?M~y3G!9m*woIyJNwX7QMGGUq9%v@QTV6TJPAR0iQb8-cbv@GUM zrEY!3@rlNSh_0wN+EbIZO+i9^8O!^B4_Y{QR@7MSz`0oFWJL>LYCJ`cv}Y>;--A`Yv-h;dFLrOw(FewjDxcFNxoi!e|W zVyx~k0I-av&ehNXybbDWO&_oPoG2dLDvS_l8qPwedQ<g>La=t%Ly2$+&`yioD4fD`kyI#7>`Vu<1S1QcBvU5iIwqGPQe`ue7`) z(TEl+?TczFYfp&#=TF#sj&EDve|r%9L;OFZr41tWT7FNmoTj5GmP^Cv9$I`@FnlM; zT6z#~+r!el&O37}DeN`x1xFjMHw~A9DeaqeTvE9nmnq9FHNOsMEP8OKz0_s z8a`POBZ!U|x<}3d2o0D^U+sxA)`LB4!%4LNvHDu}LZZK`#rQ&PIkghNFCH;HgF)G$8iJwT~{q3~;ANkPoq?mxe_j^v9rXxx& z>}Dwk*fc_~p3;L}ug-`Kg#&uW$VJ5A!0ejb>SrNFp>6=Rv3OSJB8;eCZ1tU($6I`{ zaIs3e=6eaO$E&JIVzx7jCQmG)krl!B-Nfp#WJw8S)}!{oa;di2^eUu^_$U)?Pm+Z) zFYLkn(RGNfh-$Ie5pt-YOGUb=kZ2E6DmoUQm0%dOiVl%&>~?zH5Uk7;E-huX%ruU! zHoO|lk_lVNZ-Wnor75K?awKT+2 z9%&iMXt3;wsTT>k2Sr+ozY{j9;T50j%P|@I7ck-HVg`z z71)`oNuL;4Zt9a2VNU}$#nV(w;}P}(^D&-jk0tE|OE_DqZ9Ues{LrDV&yV;}aiZ<> zbMQ;x3`2UqR<@wi0T8jMACz?IeeIt2p$fJ)zrMG&jR)9*yp1o!JIE$(9WKQpGOw@2 z7Va2P7y7HR7!rN;u&|Qm8q{j$_C?Hfn!^;CSw?~hdL~dizu-Ob7dd-dN|{||TbG7~ zibk?zCx%J@z$e|2IjG$C1$k4G3ou`^E`VG&usxOlRo1s>@$CQLu4GbZk)05*@ z#j;)(S%`H&ez*ZV5aTxuy*0CX0gek-5m55>=vDDzTJ& z)cSJN6=?*E-m<+|s2F2G7zGu2?^v&cc%?F?wat3~YA#=1q(#l@2O0!o2;Ng0POH7h zzZw37S~*O8EYm$wxWWl?9`{{u$uRjYh_XmhHlA(w``qJ77Ien>Y2{qDz-Pag12Pm- ztivWgwVyIk#*s&6bbTf|1Ab)tR6?f@R1UI^vZ>z3&tQp{5{e^z)bs}Sr9z~Not_M~ zWukhwAeQu8j^jFqLEv4(t<*vXX@@Jrz&-TMBa4e`7Sd`1w<;lSo4BqgPV$jlB|yT$ zey81!)96GO>=?b0;-O&l&M$Z-m99#*3`fs9+DPPcg@RPC1Z}#NO8VKL* z|3Kf)Cy_4Q`v+;@S#5Ctf7qNpM;g;d`8aHmu*vH?l1oB92;lnHdF(X1%!%?uZ`|1b z`vXq@#N$x$?XKnLMtgsed^xFGbE-KKDNC0YMajG@pVw;v9~YTJb|0NXMsrJO0}Snj zN9=y+#)aWa;snFtToABhvu&W(Mv<N}A%QVOvW6gVP=q-=+7dcvPt*qxn(koxNO3=7g039Ry4}9@=HNzQT zZSOewuqg^rd;PR3Y68sA5`4Q{}&Nt&hOGxbUr z7}$oK?@i=2_r7bwSi)!Y>3NgROZg3P8EC@4tJx8kjSSpMs{_GqA1O<*BGvDuFZSCx z)Db^6l|K=by+7a_h(bK_;P2r6yvWbTn|kKsc0r@r3@!g#F2Eths_f!fI-glPS&f5OA&i71yLW@6t}xCFT16>`qFc zZ^>wB=M%|KxuWkP7_`|u)2psL!aRNJWC5hd%YSjl`~unkzvi;9(y?-l`wOooJ=pj+ z#N}AcqXr+Dw|)Eiv~!SY3gljAC^?>9D}m%W>(W}Y&c~@tNzWzK`Q2&F&Ge{ZGb!gP zVH>k88s^IRrWiZGWGfFHXl8{iEUCyov*)3&4>>Xmd>dkDr1g>`_K>w@!$>uB5JfrjDPOzw_;`TG#>yS*7h zt#7qhziMIc$W6FNerRy4Yk@di8ffTNDh6kl>9~^gW+ZlAKG({HmlkYmWa_maw(_Y^ zo?POb@<*6NronG0kH=(ldB&3py(QJ`|M(8YYbJW(F}n_jj(yZ}%Bew_HjBOIaoSXE z-;h?xVC~rtLDyoSgt}W`rq~uWt#n1}Dfu{EZUZmH205@j;M(v+fEIqC{b9)jaunEC zRlHK$q;a3I4B~cV*n5O0%kWfT>|%Fx)`kzT*|UludN8Q=*cSjCS4!E+!o9o8`=^#S zKqM`DM~Q}L8Wi@1@klG$H2*wl<5w!-)k*D0|KLr!%Tmp2i*T{Z}Pi+rqRBi6`gu!TcPji4eYlP7F1?b zt^a|xdBmcAwPkb>u-7P-)t0E)Ir3N)6)iP|jGL?jE~bX>Bo7%8%|oyXwwIuSywa_K z0JT~q7vvV#@|pOIB#giFmu6_;nc^gkb1^sl>*F3Yb&CX7xKj+>Hq zCP~a80qSp=;T!em6)@5WQdrJzplxMFM_{5Qd<7ta_}N;GB7oOB>|t!u)b#(+p>2FExb~eBkNVJ?mSV7 z_+yp1oAL0tHC&xqFadXpeakdpQ(iheT3)ZN(|43Oqwc_#RK)sd@42?gaeu?#ydrhV zgP~(Z`!~|%+P0GCAHC1vQ;8>jF$|7xw}H7Ex)ZkR3yDbFX*>?cVzHWQ42cls?TZ#a zp4z(?)uJGs&6*)!ov?Un_IsR0B#+zrkSKtkdseJTx?Tp753AeGa~YclW<*M^1h@}2 zp${+0z7%MdEl_h9GAa*lkm$_C`K-&PSGnSN29!#r2B#Ebu-@9+j`OmD$HOQKH|1lR z+yq)kz5M4drQzdAO8f9}0Qd*1&INGvy~S$(;|3~xS6=e3RL@qORL_${_>T1%122c?;6cmMl{g;2dPHofvK3 zti-P6^Qc^V+E0v5uUG8C3u)U*V-5`qZf_W@*Max zxfI!VZ7(GWh4fgo#j&s;xy-G#d7Gy_k zyB>04+SAIBp+}!7-Qi;cz^xI@4;kZTieWn$&I^r_&Ttc$ zUboqO`bZ6S82k)3Oh!9B`or0tl#?{{{P;oB*h*QqN(Tw1?mgIv@vrvvg8ZS-6qCN4 zU5I+V+G}o@b*5`w z?D?YGI4cX0Z4J;u_d?#n`cO)Fz|27(a4}EVv4wh`y|WUGBi zivpvWSp`7c7r~6ByY*EzD-!Cd@|lBf4r*20B0x5kB9|3Hb;x?#45?}V7Hy2jMVw?= zCLf;YmWQw4&i#ps60!1Yw9#wlc>%}=GRj8wRIVBrLot#27Y5q4@JRj z_LtRi7_RUwYCcIrn9S81+<&d0B(Y%4dNp0krF|soDVPs*|3UCopV6u`2;FzCMV?*< zdp3`&l?LDpmbVLLGX*k^O6ARM5%RQ)m+8&W@vrBznGML2W|RvGV9C0zuZwWAQ&gpD4KRZIzsHqi>y`;fjXrdDfSoFYJJTg* zx*L}&&%_N{Cew>T3pD@ICVL7}<~*Os`cfZP&bs1VGd0vXkN~f{D0R7RWxL$V3y2I9AHj|w*rSJ>0#fUSKw{~ z0q#u)oqqsqu)zG^o`?dTw|5;4{N}~2upoiN2J~Xs%`zAR48kw zzOUHGW!hD&&4rd?Ws1Pg{pR-XV#QqnE7?bAq^!iGYytP#j?5YzhU;lz=`@y5C#tL# znT+tBlgk$P2*16;2vf)cTIb?i&5jU=fovn^#ti{1ywgFTR%g+&3sS7r0I8`*Ft!^4 zSQA{_U)Ds&k8jmGg)qW`v09dlZt?;9#8Il z!hrz%!wSuE zSncKfyoBS0k~{0t?gtA0Gnhx`o`ei&DEMS)^~&AW6OZ^_mrjUx2z|h3cUBpxIlukk zlDjrMYybdlO-W+(9U4>tGj*rX7A^f76#lAsqN^)@+kE!>CL?#QFiy_a6XCI_I+;u7 zcuohrlmwmH*Z5LQ$h{|s?~`&R_ad%O)+DpAJS;v#WZ5apy$x*pDr1%?l~CY}YK<3P zVwMoO0Fic=TE)=dZ`11<&P%HG;%@l!e?HDedSuyX#F;n=claQ)ww*Q+%>s^yxWW7+8H5p z{t~lM4TF|agy)z1+OJv92w>=@oG5t#+x@`Ww7!V}CW1fT@__goWrR-hp9Uxcku#w4 zle1!Q4gljS_DFKs!eZHHebotEmryD(_zvogeu4BQ%Yq46yriN1Ku{)!Y; zwUu{A_=IuK9aRIG%I` z;%4o#2?V|4c^pq8_fn(DPVe#jXcw!;I%DPWdvg;#2UR4OecQqm?zZDF2~QWfEIatU zS>g20USRtWi0guDz!AXeo|?BqeRiaWg~#3e!B)t%(h{=e7R;h!6~&SL4z!Sa8Gb_G zOny9htA}bzvBJpU?Y+ax`(M@T_I|=s?#-Qm8L8-dTG28mYt(z-C+mw#Rzv2l(5CK> zkQapCFMv2HkTtmz`-Jzi9pG=hT9X1yeB9oxFcVdus{$<Gl~6Ck!kF1U!2zDT_0bSA?4!L{>=N1KD*9(u5s&yn3`1dMEdG=srViSuhH$o~V$ zWi@6|KR2%l0(PIcUjiK*!>I0xll6P=)3PTW>n>3f=$d@lrw)Iden?V#P|M0{6!iD& zakb4DiMl7wHYs(}qlgBMK3i@p^lnd=*P*?l%mzHOtWjxnh_OB5>@LaTR zN+0Z`ZrXAE?cWSw7%<`-_%o0JRJ^OaNcSs9+T0Q{xg6((*a zUGdklAZ@dV;)fVPCt+aQUwXGJV4z~jS?{Ym=%Z0?X~#?Udlp;aF3N?Pa`~2Bu^{Lu zax|OfMD?vj5xOwa3VWCc|6E<<_S(V{D{4_T^3oA#4^v|}fJ9#394no#u~=0@Ejs~7 zoRqJ`bA1inYo_3~rPZqw6sTLKXVea7eA1v2+MfQC45if^ux(%gEk06;9L^TE;Jw3;jKa4k*>$>mU<8{C8*G(p`0|SBU)SZ9qXsew7TIt$7 zSyIvTvc6p(hv+!r)ATlvX3)xdbj$AQGFRXwQNSI-kBs*I7uHSIzT3o}d@b>3g(M09 z^g%8^{ySkTYWA;z-XkT_^)LPEE&CLv%(>kir%LXYS@3E*JXd%U|7PG(m^s^oNf};E zQAA}%0RQ4nD5T<3jJHA8%caK^weHKsFH(?U?>p9>JT9OV)N%}!uc{2Y>uXH$otFdQ zV~35VGmskkBA6nwGChZZ(@q7hE$Awe6;aMQG00dWq}S7H4xXlX7g(Yz-`-Dthz_=< zNQ&H*YUrIl9ynAV&jfv5>JF%cX=yjuf!Umupq!G`aTIf?WNJvtj5=MZPBUZTHG#Hj2x~Ap8kzx(@N!$AZ18J z+#-WFb#WYuC8Wh`7R9rV>vA7ozW}f+j-L$$;>PN^f1rFim7mM%cP+fR87`?2f^8zbb;O*igHTnzZp zIUy=^ZQg8SoK4;Ibb1tB{_`ikI}Z95e-YWZRfR<`wrwsFDFgnF_SC!HAEOi=d`q`Y zXlfIjbSyVE=6syWcA)0apnfagw1;+Sa@I*D6(Uwu!(1!2mg$wv!RppU4K%I8hdAO%@a5hHT^Qtzwv+OzEO|){>uBa_SNX}F;g+eSA$#zD|1P{E2n7`@=9ft)+@tb}>Jw?z`901KHS&o=>99r>OD!;`Zu?R z7M2&Er(MUm$W4K|{s@}W2j3uKpF+ex;|pR8#LLBp?qUym2EKMaF(n%tu?sFyT8`=q7ss&;{d_~EokNw@eFLsxQTjj7zDNWI?G^8OPx zH`pd~cgu0=(4L)&THj68@}VZ3K$QT`kD#4#lV1K17sYeaT9+n^kHaYo5z{%x9iv*Q zH#JqFgeVWORyvyu{WzV+y-lc$Ue-nbcQYqeKD(SwB=8)W!n$&m`klo`K{_xZaKu5;hauVtU4IPCqObCh4JrdTA=f39j_3*n^FR>%qOwq*5Q5 zIr2NXre2MAs9UQ$S3@i=5>^ilJ+EscF9>=J7XT7JW8O0q`T>yEFDZBnsM4gEWtjh5g$N_>=a{O9Z@k+j`f8+u-1 zbN}X|n3>}Il?SQ2UxCU97#sjgb)kD1dn*SA7#<(a9zLm745Hc&#IM*RWXD?3!+xR~ z4gPx`Wd3o!;bhU0RDDCBAq&<85?BDD`Aeenmek$Wk(rUC~gGz-5Z|2NhPe)mFnr$8T5=cjLJfWt6 z+rxUav$zgXNh6RmKRNM(Oq0lX|G6|^aCFIuX z5u`NL={K=q23HXMIebe^1y@%f6T-g}Rr{!8%|gSc%8ae5UsXIjC@D2^GU~e~1TCPa zQq~o!s{1(CORcV^z^ZSpfTlv1M+?Q;i8D&H#{xX7LuCV#71N5;OJ*2h76#(a$lEu6 zV#$g>x#{c5(r$gvzzg@)a_2(GKQ<}X&u^80Tx~&zts%ai<}fu(&Lot1F;zDWBQTvG zmgetCt8^1Een4)H)v#GQTwmgW|Gnj^No0wE)MwX0%>&eK1#AQ8-Vn7gq^!HeZOm1_ zz5t|Mo#76*WC}655SmLdlFFf8=Su=oF)u0joHJ>MNeb)M&(l<%z=-z&!72L>OC!0B zg$qcPmBU+_#PV&DF0a>Qo=d6x@I%q+SmAWc8TFfIh&fcV53l8jWb9i)NlNNnL7*hg zPT1G)3x9nZU~#cu{^5xS+jVj1UX&F<>Z^GBv}~oa+vGcUk+i7wNZ@I(M|1BV$5Hh$4?u7%3W{!)77dwauB`Rpsn zlnnb8ju^wV0&oxZ?4XmK{9LVBSWxWn$04Jr5-*(YiC*zDNp0Ftr&g77lMgVLkOxdD z{+#W_x3tdF@0m1mbitY;k1hj(N~%L?Z#Iou#+B=j4f;tW_dW7vK`c8lP7%|B(`~0; z8t5pp*x*Av9hBLcfuIJ~bT;(Harrj7SPwqH$&vWi!pYB-yWX2W*j5`@-e}4kYpZE= z#V_4biWJb_n2vpz12?XoG~CXDJY!EeeY#c?8mt+JfL@PK`Q&*V9+QiYRQPHe9{abr zakxt;#zF)$Mode;mi+6g=O8$PTD_gPl*)&9C1T?Q9gGB{|=R^0mX>Tq`U<6gDmW9(z0U-Mbp^@=;TDMx9{eYc~2@bx4NT8#?KR z>0(U(H3=v|s`!sGliXo3H#T>+tPP6Z2{JoFcRFSC?(I#Ct91_jW3;6D4@pMEO*>03 zoU|vfIF&+LUP^aamK_gcJ1*$wf=MvgAirLDvM)tDLE;mfzX{}|yaamvA6?=f(ZyO~ zpDbj^2&w*KM?ZEL;3Y#uX`O2F<<{aiipe+;#YhpUenutYy!$Qf^eEAq=T0FLVmGH< z{e#YbF8EX$SQs4B`MjMAP_23R$<_R`RaIQ~;_Wks?9=*&Ur-K;M!3F=NVzc1UtKs{ zIGlnzFT1O|Rj4cMwIcRC)>Yig$xNbt`TB!f(B=~ z@v9M*>A2hHS}l`*aRC<2Le|q?!WMWeBX$CdG^4NaJC>DM&=U=UeQ)nvZ62VP6H7eJ zSit?2LAK?73TmtnPq41UWfUQ*nK#J2tfNep!~53B>D{iVt?+FFVuX}v!PRC4i}=_9 zT(d^cT`LTr-(7+#cwHFCH86bs#PNMUisbFtvtFu-p66w2v4Py{C~r?IL^FXkmSBzE zFRo0Dcm?D!mzH1kZ~Sc98>Wm1t-&d^zp&4J+!wAF*0`Q|S5Phq;n#o(o7b*%_$h0- z?GdNOekI>?a~tdzXx#6wTDA6bg_6?xzb2Y3KiFQCchD#rF*mC5_2hbhbn(=Gqd&f+ zjzB*V?ivHu*ja7wl?mOxkN(f?7Em_Vz5{Tgk|Pw-pGOV`!mDTYFM!mlF<*!J3CqW2 z+#O86CIXU(Tu`*%tBup?k=%(}QTlrAJkRY_0#2B#y5K;fPO6o#Jv8hs?*)@g&($W~ z@9}{WS=`(sw~B_EOx29 zaogd1Mp}`$bbq0=iaM8-K9@TX^Cc!%v~-65B5vX{ye+*GdS#@UxCF7l+fAsI zVu#t7=VNjep#s1?ks-SqgI=9=*T^bfE7vGcspn^+MCux_a^ob zm45s_7k{mH{1NW^zmtV)`&0D4{|FrdC#k|F<#!0V8M|$H(|m1DR=co5hTm+Ek+C{8 ziBF?nK9klZolu`vITv+1J2~pDL>tWU$3Z(=iT5*U^F%POUfFP^Rl`ZNFa!I@dwU{E zNmyig8UG@}ixstXZdz*cUTf*&8$`*pvEK;H!5q*N!NS!V&O-})({!wwtrP?6avq=T z5<8u5Z&jpHg2Wi1kAJ%i68(PhN62Sr(Iv(LUt`F#>{ z%%m3&s|F0@-#y+b!d3HNVb(rs$LQ9|U|`)VFPYRXZ}{JEp^y(J+vHko<1(B*EJE#X zZRg)I{Geq;PLr}|^74Rfel3THN)4(qdZ6sn0Z~jtm(M+IkBogjbh+3=-n|9xN&!v6 zwga&(SRTaHMKhe~YqABWLYaT@E9eI*(gVLqlPHR;_whK}6agyZLb6uzfnk?xu~+iF4C1QCcA}gUwgsmqfg26jVZ3<DOfnyyzd7Z;h&pGWL;G3w%O;9%o7z1q63rd00nr%XNk*TfOi{PoILly} zzqrm?EMIBSAMUzsqR4mR;Ii(cSW1DB?%1hcUqUWOPw8cZQzACWxW%NAc5z=>-``Sr zm1a`1fvj(OY0BkqY7Hu0XI)B6r~K4XS5PlTlfrhj#Z(DnY&Fsn59~i8PdL_@Z_YLF zs5>Zh-rjRxaur;k0Kmk%M>GKa*7&~ch@pbu=Pjw<4V6!3Obn~Q1;uSkH3 zdS2Xw%l3wQ>`en&zZ{ARpEH9dDClHIU9s7~#Bv_WNo9opcV1AY5@l-U8>Bn6_W=NWZ3dXGsV>`y`qAqxrrxt7ttBnq6TqP+ZLB?FwN0vfK7$vxH# zIevGxqd*+~(q(Drq6;(u`^wg_xaXCDq)ZE1fHc zd@;>9_VoNE8&N^?HRvYUD?Jwj0tpS1r^G&(8ox`{qbV|@f?z+Z4it z(s;~YG6u#J^q$|v9zW!DK4@w*s-@2y2Li9DbT;%1^%Q9>RB;W%>b$Pdf}#wkTcU!A zG71PTid&2ez=?tsC2PfSh|D!;*x+KnzlMzUiCrzx>N7c}nrubrH9a6u_r0T0-aXt) z2j-Ws#6<|{M{(pddBB@%UC*L6;_tb|`a@so)L($(r!?}mN7)vWdNDi)hVnkCW9-Wr z)gG0;mppUehld^^pl#`cys6F?D}vpD`KlbcCt%*erK;N%YZ0rVsD8GJ#H)siiMXh9 zT8VkCGYTJ^*nJ;C`-L~8doH8a_GYalULla5&)_8%##5Ncwr>t)9asoX_I2-kRv}yZ zeFB=GES;`|;x3#>EiMF|%%4)<4Cgw0iad!7Hi5ZM)YQP5`^0ipg-{iEKN~NPK*q-0 zOn0J$Ocp@6!TtKWUj(Zi*^eixSwYO1p?}C2?QOm z+R-Bn3E>s7+t?$I_{)97{Rpt~OX{h|;}4cwU4C2&aB9L7AUU_^YRP>4zVugAgQBOg zZ_8JIzgZ%RfQ=ZdvrKOg>_cJ`Bukr9Pq0(oYvZPkfaOCt1Y^8-R znftSRW&aYQZy3euzeJm#Cjob?J(S`JT($bQz5{H1#%9j z1a=$ieBCNR96j{N5#sQ5URp45E8YMMHkt}wUAf%niakGkojTf*x1O}N!yS#13+$RZUDFj?tmVl+reG0DklyXHy^U3Jr<5vi)uknXCAZezG zux4ytZVL6ixuwKf;5TdB^{3~DntTe5re9cT;?{zqcu6LK)#;};r8B7+_BoV06Q^=O zOb55mSTx$!w5BHmm$0#y$UByUsYO&nZ}!kq0=n02-;AuZ)EISgzLx<3l;WrRdV4=_ zFCR^9-CxZ#OAbo-V>bC`89T-gG|*%G_t^h@LcBEoz^P;&GxU7tn?cRUcJgji;d=a}gs__u6r=Mp zfuD=pkaF)uOr7B&y=F}=rl$c|zHP%Zgo^u0!o;qCxp$y)(`$z018XqHn-8O|)P1X^O9j)c`b|5yO)jxIwe9XcG3|KUj%T^XtSrvs}mg@R4ihIswg(k`cc6)c|nf zdk>ic?S+A{q=Wo0{ko8M8DKNm*3;3$iuN0lE7Neryr}Zw>}aM_$KM7474T@{g((H3 zw*&JO71?KLnTRy%sf1l=i1GcHDiDZ^7#KDW^o7M3qvUGo`aa)h;6>qQq@FGjk9-E- zFZ9>@086FXVq@uxaonBY)Q@}{4&pAxLcdlTw)BGhN-VB>;=u2XeO{$781G97lQ+_Y z_;+36Dwp`%6hZF>m`aM?tUqsb2)Ri3^c-J90jo%W2``sdb?!I1O+7*zV5v!jU^ z{7eR%Aui#+Y5>=A?K=ZohUcF&(14H-0o={tFdjRpDsj;A@~-{CQRwN&Laie_dl?h! zPwa?3cEzbRvD9Q_n4jowul6MJSDP3Vwq_@u6J1xpdrM-uM7v3vF-Xz>s>&@>@x5-U zF9sRj?0ga8=sCNrop;XbtL;I2>NDBsrjy^1{vSfsWQsCMN!VMHrEH> zKSnC*FUn_?sQk9_AuE$NB<4=%3`S%a5saz=?EFVwAuYUV^OmJA!wAzjRy zJDTgku0IY|OX%{C3}5VYj_+MJbbYeG!Q$M^)HNir?_F> z_HXO{Asqnm1%5&;n6UwD$0Sm?-YA{!Onud|Y0P$5>G?AO{i9aEO8?%P@(BOj8td=c z&9SEka)c6{8Z9YR&nJ7b54+}lYwF44)W#@}fF3ROs;Rc+ykQ96cIdYjmnYVmQ==9X zZ{s7h4>Q(}Yj7gCJkXcT$5UPmwVgS%-W8(@X*k&DqeO>jOfrU*Z~L16!eo1*CSmQz!OGlsSpZfVCAR)9b^ zp)^TpRbb-}IM%?$w}<(SK{(+u0{kbUh$r4PiO%uF3cB@4U6wRd031@Bi!=Xl-?(I* zuWq8GJ8Kc!Jg14}c@^a1f*9Q+JpQsUq=vZA8Ru9%>4>>T03cUTeMpkmb{>>oP5CK$DUa`#)ORD+x5P!Os{VCiMAjyZq7G z|11h>rS@Y&(x;U_m6$+G2y5-h6D8-{Up+s8Iw9sGEjyg7{?n5CGpgBH17lYX*5?Cr zq@AP?%5EQxOuFPDHE)vOD;}AC`e;SXXLItwM5nlnjw$k-MxR&6so=3r-K(h=?Y#ss zIAvexp(NzjZqle~)(ibhNTir{GupWm*rd(Gh@<2BT5mC<-Ym&!r2BBH7LyWd3L(~# z6|r*F@NWG>_G5iTy8b4c?`G~NX(c{tVT4%G)hPO(MrYDXlv3}*?#RusKl^F2XtlYt zwUbhwvpX5vt$rmy*(P&d2by=>Y5|A3VC}y={t?nz_Fzb2F#9 z43FW-B}@7(SLGbE*!!ApJY#GcogC_jd0XB^YsMDkgXp*LP4_3`>N3)*<+W+v{miDs+^Z+pySoJe0Q%9vfXJ?4Hn_8|ToG{@{JxeF zxyde*jJ;)aA|Kw07M~R~gz?V^aa)siYdbW#E!JVTImLo46s&uf$&{U(7r65_g6aDh@8DS-*!( zWMAUiWZLJ}WO!4LPf^Oh~rSU@*RjUt15#eYK_Th7 zCA}q!f#f}9vaM9Vz`)mUk|yCxAq!)|nN>yIms1Zv(W+Ry7}E!P61|KbX*sUDP#GMn z57wn~t0Y4El$WGbOADIc*AUtEYRm`|C6`CJ82*k7rF!_brQrDMiYDMcgPSYEnM=LP z?lmaSD&QkcJrH41wRQT?xv?c{<0oxU_U_zXO&KIV8>@|D{ynxz?nG_*UtWNVBaklX=CM!$cE1 zF-#2OVVlKe+D9MluL?4cs0#x={#=o9kBD&;%2hz-y&>VD&FEbl`@j0<+A;Vy*By5S zprUIHW#y9_)=|UmZdsn!%|#E(>{yAC2|f#HJHw&~WUCRpE&A%U=~;g444ga2jF~;^ ziC+5Ba>vi3b1ta2Ud^+!k7wlKsu%x^-iK(>w!9}1v85>^!`16(Zc;qWntKoVBZCc? z#AQOTze8@6#yx6A3(AZR1-;#NHsBQYrS-gMXx6@Wtz}`x#5mqwd<6^o4@=;|zSe7Cf-6Txt+7gX!XsYqnXG7N`mz?qSfJF&<4eXoP#W8c1ipTN+lG0@3;&?E9 zJJN2@NPrP^_TY(&5`ft+<~pIxPHeE*4)()3691id{L5#;vn~N%W%Mwc{SSWh*#6K& zUe;f7I^z6K+G>jkrwX*ab8fqDt8s-`3MZ#{Yd?P?Q-uA7Xl9Sqy1Y!CG<Gb7u6y31eAxz&#?FGW@qJWZm;6n`sYvBzNvS z8l@5_hp5U>M|bgbp1KF{RziZCl1JzWkLHZCr)&n?LIu_eZ6acUb+aGwlcW!5g>zkG z1i5#QW{Y5zEDr?HNTh%g)72p+%AK_Y{3LR@KW~b+R$t zRPBjR_!Xfvm$$B1@bL*8P-0LZe}`wi2`_$43rec%?M?meJ#`K#>Oi>*4Us+9E(Cs{ zd+cQYNal(owP;FwNj2vTVMOW8-l#fkh-C3Pp0bEi1yeJFKl!d;BO%_Ck;;xNZJ58q zC`jHsq0RvYa}Z1@zDANVDkiK!m(T4si?%G(7ckwutzl$c%lo<)(adLb!MDiY4wUDp z-mXZG%Dv!N)#vKI>%4g7bDIOX4`ftTRZYcLRe;zK4!Lz8D-OZ|JFXAzOaGu(r6KMeLX~*G}u7KIr&VXxsAI~71WqH>NIVR9Nlh` zmA_W;%FAfqCrn6^)U2MHIx4ffWzZmo(!^{Z_lyxeWQystKx&{*I5J1@8moWl3S`-? z@*cVLbnJxfpx2m4Ux4s;(J&*uc&-Px{)Z#{uTSIN9MBce(|H&2r@Ruy=H}@s1F%vk zFt6pYx^_^(k+2{3VljnBniInGJN&IzM|t%!*1l>fNakq=AzKZ*otXOPa`FD~__$hd z&+B{2OGBbl9SPs@as7^RoZs~@!e(Vk^4x2uSu55zT`7V{`A4D`q^2IXhluGG)7-_( zHn)v8?I}*>F+bb)Be;GQ7;|UxR#*i4f+C_H0}*ICk|UK!2kxwSsV5BKj6gc9Z61R@Y8ZCt-)t7I(IBnZb< z7x#VaS80A#S6#lXVvH`%Pp&5Y-aUsBLa7nU^)T!l8AaB56okWp-#9yPm7@G;Ir+S0 z=T5yI=Az4P&MT3ude@a?`40-2DR%C7%NCd_Wph@)lFo-TMqF4<$#x*??s}&8Geie& zg%`}*CH!k1C6e$O*M9{1BcATftsWQTPx@KheWbprqn(IQj*sUCJqVw#=na!KMx%A9Q3-= zbwfTYvbk;Sq_1OkNtvqr!o&W_6whkfbIm6{=B*cSdW_k~v#k1SYs*`s80tWC)2!9e zmm^A+rE+m)BJEv&wMJL_czmeJ@T}b1Q5MZ&bN4)>_TnYCvo6H@MQ-!B<*z!W3sH>x zBcBq!*Pz)Khbsg*UIp|+E|Y20(&o$Kmi0^ zgT8q?V`QSlQUQa_RWnA)vo3yDmn{JG*Dne#@)hMpy;a~ysCJ1lC4d91EkP+GO%2u7 zRv!|aBh~RcEOAxPqBWtBe~mcm5qs#jdU3ANvaWxo*UA{E0)7CM0Z;La`tOY;`PH|>q6da#!x0_C}y7Hjohy6oY&TmjSdf{zVZ!dp>gNhi$iiYycrmvM#S5p2ek`wK&ons0>I457D zN)?E+zdzA{dI2P!)avWdY&X>J-=~qtRtbx_DnFg!R{AJWFe4h_ku*CDsv#d?Sk$lw zYsjDZPL;NoFBqC|`qGn?aO-YoQ5Kwxpc)g^An)wY90kMPqYB@w6Nf?_WgE03B|`zT-|(x$k#uzgaQW2g;2?@73OQV$*CVxJ?fp?jhulh z^$!*Y{6e|;i0r&_b2)70-^B+Y&hUm0=4FZUR7*FudKBNWW^81(@Rk?%Ihz$NO@pQQ zs-HV>oSeVk0Zl0L5R1g>gDY)h!ur0gnO;Qp)WeHnJX3o8^%mRv50{wdzy7t(T!k*wNcjv=dtLZ-@sgTWhiic#Y7!wFBSPSMn`M&ObnNh`w6q3L?^P9EUR2RK$WuMr}I!D7nSq^_VN{B00R2 z0)Ts>3{iMfUYY+apq_ za3Bx}-CW*`{hC2pao=`|pP`@6C5x{P=BoSnxAQN%)wJhj#lAT%{O4Zvz#(BEd)ztw z2fui%b7V(}J%ib@M-TUWw|rtr`n3f-I?_(Oa{0}0_5-QYw5g$cZtkRoouW^O8}V_; zAo+7q`eI;_o0{p|qtV~x<=@Q8;*ShvrQ!7zvvD!A1E-W;#)cHuv0hR09%QIiob}J& zLPLmhoUM$pRc1T+10O*<@Cr#{v}GrC#^M91dMPhhVs|{Kl-3+#P2j+bGI9`0QoUbX zY|Jz6?`5%EH3w9NHD;&@-zO?6R6HOv5G$|fbuaOuEK8*Z8?**=-bW%H%+d)Q;@6!E zN^-wEOY^A}{areb5*Q#lBdsLJAIu%%K>Y0m__XenrVLI$g(fCrcSIMDV{3 zwlX>A*)bnrZR)jQzY$lCEM_OJX7~J(4jFmkDjq75Rf;_)k@C8<&PD!5)ZIZFZ_hW3 zA7EQ*0bJKl3mj;6k2$}Hvp3~M#}Zxe778ADv|iwK5!T1i`(N!z^r=>ZTDG^EmB{BNw{{}lrl zj{=Sm_gp0S4-w+0yl`1lQ&X4HxrK#AoRW;y(32Wz&?L4fw&`5|$$N#HaWN9vOMGi~ z4q)4t&q88F%R?^zzQ}fXX;jJ!-kB{rRUEWalkGdD%qe2-m5A3V%a!}PJSUO`)yYW6 zBfd#?j48SMI`Zllck@QRnaMksuE5tsbfVh2vb8K2<Zt9uC#b zThO7%?|EpLK3}{uI6tj9ibtRb@RWANlIkXf;qN@ARG}pS83s)R#$9aA^eld;Ss*+z zDzkAhJGA92?v}zW-lb^Cv(-6_fK1ngl-4aZOVJ9tyW67bSK5`*&8Dn#F6IhdorT6N zx=;bdkctrh{9DFVB?);AJ<;>pi{1Y>i+v! z|IhD+6ZRj9jeGTnhiZ3D_+`Zv8|QvNx9l<5Pvn|L4x_4-KEFjIZRs=PPK!fdREok6 zNI4}c=uAeAb08h<_na0JMs8hxzx1ZqR80D`kD5&VBBo6KT=%oqlQf4XL}hzSQG4{k zr4_RnFoWifgg9iLi@LcKP%kbpD$eOXUn{Ax9I32M^%%=qTN-U(nx;p8OLryzn&#Z( zkGkelg0vl=L8xQ=&%OO%Okd0xCPm|l=5|dl79(EBKc3<|ZSGY>D`h<<%0ntDs}b7F z?v=?dw|skL`Qf!GMj!7C6XbRl&8x&zTfpMRyS${2r3VSYUrRqj zrm`Ve;?8V(=1O<1%T#~9RzkVyd-})-3TPjA-}ISg^n?0`ijCANs4^)H8-?`nylUoH zJaM#X&nM=51@lR6@#H}NZbtrIg3%CP;#o&B)y_my7o*reVutD^N^5s6mB7!6{HfX!=Xwo&VI73kV1WXJw)zT6F=#4xbQ;{g0Vp0C*@_SNHIr zazpcxubx67bN4?M$FqiC>$8IgYHrK>TZaspU%p*wUzG6Fgb~eH&pD8A0DI~otESm1 zF)ASp8*N9}&>AY3s=8O17PX-|rNAr2M7Z}p#^epV7X7y1Ug^W&q7|whD6scWs%Acr z8r~F9cXVf#Ki9%v=Zw(}YPwl2S{GvZaeAu z&jx9t!A{oBv2*Lyb?LY-TuiqWvxUehC#pHcdrCWZO2eTdbLqw&)b);o#{eoCCv0Wh z0O#R7JY0K<%LWg{ozGNHL|xT5gtUE>zEDD-^RlHB!rK9ImVxU#6I?eLg|o6N@V^0e zMT0U$gN77FsX1W&MRfh7Xt{r`83S%O>P>4Zb!9ZrhVr$N%9F_3vx|brYr_rUuz3S; zg^w@IL21`WBc36&)ZkP)%E!UGF}y3%{wm?~Xx>SDGQ@!eER_dpj9T^csunmSCXD*@ zd=F*l(RdY%sC72u?rvv?{f7|~CU?)M4d#7*<7*}b)Ay>3cTq=ik%nBy7wSujmKn9A zU^iZEij_&Ct}fpQZ@%38pl>L?0+vwB8sBpC9si`O8;bXEcfSU3dX}^!Vf(1U#Z&tj zX0$B&&WR!s0G^a|yYc)#C5<9+;2{_98-K_wtuuPIo}Qi~6B8P;<-v9X_iY$qwnHHe zK8mb_u!kg*+mC~g5(Gd1w@zCNW&Sf&=QCgT|Oj+i7qZi=i^wce;?!aHO0o}Td3I6YO)byfTf zH=L!$`Pi4F8$7F2FV!yT$r{xY@s$$$P+p8|JuM?+I1~c)V^%S15ie*zFe_FTdoi|; zJ8|0OD1X29OA7R|I*oaW)Y6QBNKc}k@7bhZVX)#?C&0N0cOSs;mJy!~1szqC6FW6% zW-%r3JX(f2eQ|gk&Mgy*;Sgu@C2jIxq;k| zC=|-WhoZ9o?l)K8hZq>=&^J%~7mNSXS^kUfw9efB2~6x?i~9m7RmAfRh0L*g!*fUN zMAvM^h$aySRhesPc>38UVINwwD)AL#u`1~8WYFBrwDT@Owb=lzj!Qc~|ZB;#&B z+RXN2)LGF)_BpF(+DAV8C=E6xA2S;)6lXVEEDqMo>~~ri_4oZ2-^}g_l||=VA)?j1 zcfVd^ykf*XZ0wchtiDgO@IqHcy=alVl76)uRMko2XR~X1TZJyUbK2EFU-DpCYzBVh z-rFZ^V^VE3t7Uj?g`p6^;xCXNpU3pXPcihX_Oiw-q;6KLI2l+LB**q05lS?27c4J) zF>1X!rsDd3AV#j|%5l~0B$TN91^;4&d8E3h9tBypygbr}oP!814Pd{E7Ich^kQAxVnbuEc^GC);ko!A)7dWznScqI6@E3C$Bvutw z6Dudrw&WQi7GA~P{fp_cY^0yc(hlvlAM76yQ#<_s8)smFN)i%x;lUrEqkq< zkaNVD{onVP;k_}fG#agEKXic4g(LgS;zuCGJG4{F<$v z-xWw7&oPtREmwpxt4vsa@VJEVu#;fD+e24$<;@ikvu_mK(;xNcZd{(yY#} z{ur+%g}}6l8XA0ou6*_engthov=;^BG98^?Chi6!;fn04p;2ENCDBJ68!_*-8S$r& zNE-GxNAmc58?7$Qi(h_RWD(ByA}{O-&)n|YP;}$QH0;p2WJs0@?CYX5U2VZFZhb68 zW9M=42)0Z5YhN$_;4BQq@}x*_+Z&`WYLu*Bntn$h$GF~z8*g8)OW`lFGiR%62ozV9 zP<{1iV)hpC*McAY^qPO}<3c}EW{8P|UirYbA3E3Qv3pY&d5L;ORdp#nN_k;SDc{Gk zhM;Zx!5;zjg_j!d6>jM<|8}ia5<(V>R(n|Kj@P?&Qa8IXL#aqpUnSQa+m%jG-7-BD zNA0>P@`m8F68r4t5Cw0pZk*)99oX<{F-8D2V8rZKLC(atfM)B1Y=|#9CmvK05qO>@ zC5Bag;^F=$e=`>cS;z+MBqfbwjKH!u5UfwSp`qb*z?c&1_tAe?l>%T(x8#9V9+UbnH=-}nhW{nUmNuEibaOv z9{nJ{osnK)(vo=OxP%bXE&2TssEm?c{%q}qAIZ>TCwHk`TM&ozpS%mZd~2q1PW03z z81~Liv%rjz_+uO+@<`*lo5b-(yE}SnrPMC{d%A%tpwCe>KZUp;kB&xer3}|Yk;5DfyH!i+#m)=E%ZF?Qas0gR_R_GvD$IEwtXx zyA`-Zv4Sc?MN1St##`v#jo6~x)aa35D}Mm10On2SULFqZ>?87~zuqq;llGA%zpq>x zp8lIFEey#oZ52TIaQ{Ho+4Y0}33q(zwU17E`GffWFao=-9Nv|ub&NafEC+b0MNvVy zi!pn$TKc8abV%8nRMgh5&_jkqOmQMC&RZrZ}4p~X|o()pu z4uoN=_6*NNYNK7}Z_<`%pt_H^p;I%g713KQrRzo|Ch!=;H<3oKYnEJNRdh{DXQ+K4 z0-K_Nx2-K3fDAk&6ze>l?L!!pf?TATn1ARVs15SY3k^#9pp%Z5iV8X z^c$CiCasaccfrI-WaHgV>^4!oL6zPvc|53T^VPL+o&ef7qB+Nb>_h-(f~QvGG(O6U zCM9@T+=!nox;Xwh_^bs&0Fu)0kD7s*HTa|_&Gr-7H|8u@u>)3cbE-_02Kf zd&+|%9ejtMW0z@u`RRL#-jwC`W5)7E7Tz)eS~H%vq<@V!3R?W>iG`^ypL}UTt*^t6 zhsEP7z527G90tq)kbE`|>aLMSU%H(@d-U=}`Q0pCNt{fplaX* z#^GD2H$o4-nz`3E0sx{4nRDbYCEa%#j@Nq}fI$c-q_PFn{#qK5rG`{*U{Nh?>ajc9 z#c;C^sNuUp#MQP5(fXw!#7m% zAtVS@uqt7gjK1+@L_Wy&N{teRf^er&sbU5;d=umR| zko^x~sKBld0Bq=-R_$wP?0@y<`i#|Np*ggopM!aKGC|gOa;RKys{CH1t4=868ci~e z3W?X3)Xm`#XnR&koK4Mc7z+yRuStxzzkn3(qJpMLDt_owuEYf17QePq7Q(~P80n)O zto`b|#_J+mIRFT9`o|BOE#xr&u{(u41(nkIdU+*5zbpQX4zq_idx+;ZK&V}frxLC-! z1EWy6oRU+HlR2cBr5WkuOp?PeE32G|IWz0vY%F1z!-^cYnNyo>#_#32-q+{*{@vd7 z?hm)!+@^n?&*$TGKRh@=-CufrPEX%u9_7rpyd`fx^Ysj|rZmKERe88aq3xRl_ zct&AwG%ifPi*~hzm@eU~{Qe!B6lFZyukiVSk0*P2G;ehfs!;VMBS)%npD9}4yEdlioQHNLv)Y~G@&;s10=9xSol$XoMCmqnM_AlSXA5!%-Z-JF(gyj=h$BVnI{oOO1uRPJ5dH$k?{h4Cf4 zZo$4VBVUUFb<qwf+`viF1m3RPM4!Oe2u<65BsdUr?xIY1AMo zY`|L*r6t;dR7X8;aK=5l?Blw}kU11iG@e)#YG_}>_4V|$XguSAyeK#1-c&wpYAZ(FXG!oe(A z!UMyY0T%9$>Vo-;GvWD+Tz+(Q&9vM@0dksz_Cg>y*UpgRbI<(41^8n=TI4xrYrJr3 zFYvel_RDRXtpiqmwhHtaKL`SUCxfoM{V?OrACg|eQfF4h&{B85*0_4Kos!qEd;7~R zar>4e5O}YPpTnM0%;~rtgS=4fU2?c2A6f8#zvui-G#Hx6wd0cFNMZDCd~%Cp)IJd-eGs8Uvt{+d9&W8WpJgh^3f5Q00#Z**EouQwJ8@leC*Q-xr zP>5V}>Fs{C+ty!mR_y_#_&l!IA=FZqpJKLX#PtuAyqEYCBmmf-cH1F%=)+V7I(~PNR`pBdX z_05EMe>g_Aec%|XD9Q-A4rlwPVI9z%UzJSN4|EX4muh?7x7LatkA@M=ERYRyx2@%z0^`VXIz@*vmY-R*MJVd6X zdz)kp+KIeCvK=*@X>WGw`8O6tegAKivZ4>e*_t#}EE4N@+$eK5&l7+N@sf z<*jG0Ydq0i7R?wXNadZB&nX-MZy|IyS-_QrECZ_fwESK^kB z(Qz7-@}&-5;vIYJ0GmhJ2fEdVcFmLGhZpiv29kUNgImeIQQB`hn%nK(75nGb+q8r? zM?6nzGQ4Y5+j9ccmk;3Ogd=C^VGv8*Fu>By7XG=-dN9E9u-|CmhQ|4-p7NA9ib6*# zO;Y~u`w*{ijl!`TGv1t1vOHAzT6skfF#BU<>sBdH5g45{!jwZW@2I#VoTaYCsi1zr z&Xwwn^i>n95U(xgtEFjKFnFywqqy9``$AvA$i}|U{2@|KyZhO~nvI<2Xid*2vSSS< z)tjk!?UQq(XgQdacpkM(cP4+DPAKxAd>ss69mA}bL!LUb_Vp#Bzs#PJLgUw9G|FWp z5kqkG6`IXiLNKq8OlQo*I6ku8oY(HEN>_9CAvD3#WA1G*fTa+$b+ke8y3i-@9&bJM z$6qg({W#hG_1oKFTLhOd_wLI-Ml2GKM*{ZAKt=fNM5`1Jza9<@7PHMfDaWtI?JkzL zNP`eAJrL^(*?|ZN$=YrDT2}3@i-*+Lo%e=oZqljW9owf64K6BF&a|j_x<#0eIF~zD z4>9RX{r!>Hg4ZuL7<&a?b^=O~NB?a%jy)#DuA7{35kDFaG?G@Umh5D7RBJEUfLf#+ zjS*FBdncm zwY1DtC7Mf?cwtbWLjGr0)3f@qb!BkDk!`_Xq1xF0e!&3ECpqn}}DrWgwTMy|`A>3F!{?5C>_`(gop?5+M7?LU| z{x}Y!OVB-`>>x>|4w8Lc{zRxNwP4QNHmd)W{EfgWh*xpxo+NGBd?wx#Qtdh!-;pTo zXvwQbygJD$vAWcv2Q6b}PA5WovK>MYh>v|?Yc=z;>stUdkQ5%h+ZFb1msCn>g#}P0 zUJ$t}pt};CA_U)$+}X|h@uUAY-`;5pnyfAS*fp_LSNwMkk*K##efl6?2<3nm;&K5Y zqE4>mev!(V&pb3ouUcr-GeqJlMf zFm?I9GE?KdAl|2+b9~g(FVt2rq;t>uIhwu9Fk>v3Wc5;2%EO-T`kYjJ95mO?&PYIS zjB>U3#YEqQ-tiSA*u$=T6&2c>uMNPPFGQwp1ckLKb{dS;NJ!3^j^lrUVyq!a#-!Am zQq&WoYt0?kk*J!jU{LT?#885- zZF*Kboo)tUFc_6V`6R_K+417{fHQxwJQYz-tLOUcocI&8p3@($KZ0%m+j)TN(#v~# zIsq7+M0o!`ld9(LHwwBkth*{8y@gcDxT?Wro*`{d%S^deMkrr=60i@IV_rHZ(eT}< zFcr#SD#8af;w>oP=_A9LD4F)7m~gp^Vwx}8hm1LI(e*ZC;uTHnf}zYE@CD~#+6|1< zqE7X#btc-2bz=wVu6=@a zJ&M4)LnV=2Wgd5q!D8>2?gx2KLIzLfo>%#$a(WLhTg;pq=)nOW7}nQ-rJWiCB?oES z%ezqRx(|bwy4+P!q-Fz=c*G7|jyZe(%76=AVC+6`wnq0D5*tly9&&c89owssgj@){ zxf3UFnR+T{#9s}I7^`zFt}oHo&7DfV--J92XO|{ql^oDDg^FCmm{auI_fjBin$>Q$ ze9B`@oqjLr@}-$lO|@ut4Hm2ITmI{z8+$eLzJX#~`va?Y;a4cw5lsCIOR4%-(w(FX z^G!^6?r3i#qxd3sSf^d5XnHqjCPQCx;6?8+i+nXA+<2v{Hkck^2L;buu-*F{9;}vK z0I>e5nFwp#uX8HYUCT|hHi^kZ^^W`;spzU$@&kuQg*vz3prmOty5dhew| z6#RNb-r>t(g^!|Lb`(LMhb(tCkA$eYy!AK9tVm2=w5NjifzIPK`LolBGa-jA@T2SA z78*a(NXPv#yhzdvSPp~;Ejv|1a;vnN5nC~VP-@_Q?gt0$pwlrI44WYiGMT% z&DRM{!JtvH1M;C|j!_jbT%ouv39mZ`^YJY9)ETZX(d?k)#S+MxC8oXy)$x#Emypfw zOoHc331uVEDi-3`Em63}mFQPLy;zk?MI>7#C0N6AATv!B)te!qCh(dD`wH%Cc;DS% z9=;A|!5A$J>tVY>TzNc?gv0$i-b<7YeGj(Om9v9dW>z#YLI>S)Fb$yZ6^?`YlRbiq zKsj+j1A1-ccUrn8D3+vE$eqh>{QXbQ&+aoPko}nm5<^%|DI)36X9E4+5x>%U<#31@G%zw&q+D4^wO>HoV>EG(XZ{T0mL6+e^$BE6QvB5#lw zUbn%cYcn2C|M+B@hcHr6cTqd{=H;Y{i1zx!+DqDDjys1xGvMp1v-RE?>3X zv)Q<4@fd11YPZL&WU>D@n(hbn&F)3_d&vZi^Nww=F7j7VYv1Q#^5uSQ#tn3Wg=@!qVeeNZ@UtuI;fft?QRoqt@KO6-reFb?rM6%nBYN|K{INWtKx{ z*pIj>WMbyp?tE{Rw^a=lvJym7`V#6&EUkpP`~7aQEJMA!U%fh@_a+lgS_IbebQ2Q` ztn6SAx|J8@8dIgjd84kkjzzC3Zs^3f=bOTCNYj`eM+1;IXH@Z5y>JXKzhz}-oW235y&1^cNJh^|5{&#DoLWRtp-vD4mA$3qdszFo$BS|TNCdx;hrF8%u-mVRqs-zq}gz89noCqXXe;#_14mcEA}d zI`r;>@6DStnZQCo-P}frk|HW{pza1{7{`VhbB4!dF_HsyMv|QPAcRn_xF`7x0l@#e zC3l(B;kZ)b*GhB(;h0q6p4k-=$!w(_8E#BshHB$k>-FI8r|bE1_of;hU35w?1GLVmcgO%sbB;%U|~(uUj1lL49O-eLQB=E$9Bfsdq@L6 zlj~F!p+Xu(jBl`~{>ljcHIF_2+}?2|d}$wWZa{|rvQzEu`;Ga)ChoL{MdWLsU=WL> zmjOXx?oAe3HU%5{KRQUDZG5W(G@4ovv!4|bwQ{?o)wgN{bK`4>G58>t*0P04_7Lmc zEp8p$Got*yv{EwWfgbbq0hJ9ltv#o^BsT2`9kFxMZqd72MjspNuV}cidpD^3h1k>B zS&+z*^45v)OGvWoYx0q|yw|N_TKv};jm3K|ZN-v~yY>^;v8KYv&_Kll@bqZcFIB;^YY=?<1`M_)2mI?@1nNgCkTwjGx$WioX1yWG@8&q2yivC^ff+yOsXTtjPq*=&zC z(i=g4#m>SAFl*9}3vx&wGxziyC7d*O#*_IAhHp6qCjpETkJ#(>8akUDx)Daf*QC&U zb>t>T^h(!9V`vXi7+{9fRWUVM2b+V0{axhKcLOD&wA)9c^^ZmtdL*jP!}%3B=(X66 zM77AyGxGfPn5$tE2X73pmQM6^{IU znQq;BIgpB6*CU+t(zw}K>Mgas%eh9OffxP=IdRJ0=lJsf$&qGxwT5rj|> zlNodOip$!JNqexsEZXo+V7>SFCl#>KXT+1=FwL+O^&g-HWamBdr7 zx{A^cvb+sa4xZj+5mB#H(GBTVigC1)M-j(gea!4`C*zz}{pC|6B^{YpS1yyb)vs3! z0rRBN;XzF=E!YB2^VX*F#=-(2-!NnZ3?XQO=f8PWvy|Tqqt&T3&rE%N+#+OxlBOXm zW4|#q^K|*GY!&9Ck@#zyCs}@33lWYDgrI9)@l)MT7+m|Q`{Pw$E|rPHb8?&fWb7a{ z1S}O*!7l#Z8VC|xPuUNQl=;y;Md%xhgl(vO+6GCckk^U873eQ0CaS8rZZhjY9`T%; zMwGLk(^Tea=(ylh}xF1n9!jsogH^`Er!RPt;7%taedS(~0rDe9(K(HnW#2@iT|^^b!^W z6|`0xQ&ZzXq75h6_=)%V2hgVW?=q@br61$KgfBWE@7|2yiS(FcflU0P4Rpp>+_MIH zYN+W2zG@(GbDiov!5U=kY_3ooj>4tSsL`)|4TY%Mtp#`liaSISl6v1f@YmZ^{x$V_ zL&|nTZ1I=+zVYd#*4HCFZ9#nmdUrBowBB3V>U4T&LBW&hS4ozpM;D-vyihiygPEFA zsplV*T5&!3zf(U}v#I2wuuk8Ayu+q&Jq?{LVP*(MVlp9^5$af#H2;yYhCw>Eggx(q z((jjEJlbx~xq(m}9`@sUwqDnLGJVqqiI>V>*pY@9Xd_`%^wf%A7hB&d2K1uGbfB6^ zgx<%_ftp9UuF$JA>G=!cIMna{{#S-Ix%D;!1-*4n-S3y97Xoka8$xpc*9yG~WJ+0z zs32PhHvNu<2?ywMJqavm*l&NA`?dFXc<(=7SU}21wm~@;U=fz5ya%J^T286|#ZCC( zIN+}<01Oxa?CHYi*MUD67;JsR$$JL?HXl}}e;ORdLym;9o;tLxV2!j_#Dk!e6~zy| zW@lx@569bi@vprE>_~h0L1dj{B^@=iB8sLJoM!eG@MaSnXbc?i^8$Pnfr}B@oUx zXG%Ra?2o_DmUWPxqkXR&1{9gD>8=f#MZuf^&(B|>S(-0m@7y_Llw~A5nJj+p(8v67 z-pY$uLRiio0RFX5<6&>fVut0d!YWH=dp^~f+Hg|mVdS@tu5z#kXaiAZ-GaldkxqR^ zT`vGvAszFuvkS*jf$q8<;yu}y2Bekdy9SYg0d`DB=t*@+r273nK>+J_>4gv%5JJfH z03jw-v$gO}@8HO(BVd8C`btpj!j5oh+p$ViXXez2_0;dPjQj;NkWL}WoHSV z@cz%>(hesH@agea9`ybKFV;c>nS5oR0x-nW%{%&+@a+fg3qOFDBfFjegB4;PJ5r`RW^b6$DZ7Ok=qF8L$oH^?`=zg>kVB z)4GVk9O-bGIOffRrkinCwH$ zx(H}cOU^g7hR~mCY99=64}0-TgyzfH96^Y5j@yjW!g!FJ=tBBf4*j^>p=$Ry+^EB4msaMk7tIX&!e-i}IqZ5RO*2g9ueq zI%QZDJ=2~EuPXz&HI8iz84AC8+~XDYy!0|SIWMr*ZJ5BUUI7gdO;xi0f{=f3sy`b0 zscHiVgHwR8o8iy$`nw%MF|TFTf1MogmPYUP2Nd-5YMPcR5%I`RS--2vmLPwV867~l z>1yn|oKuPy@#h6r)<9Fkg9W_s=ey*s**|T~VTMfbD*r-^$CxsGXy>K4W8? zoR^z#?m1ti&`yTrz$x~F3Q8J}XMZF3p9*UVfjS)`FN~Vj{UJ}084%(-q_zz=%|+9h zXK|TKy62g^LKph=@we}>)&uwabGq;$Wc%-@W1WHSkBwQC6gLg#A7w^e5s>c;#ILkqap;(rd;nE>8*r&s)yCl$~SHCuaEK4jXi zWs{bBX)VQ}f7ERCfIqU{?sio!{LmdbF=E(^J< zm>Z>f@Oc0T*uo5LpZ`Th|KFMXpE&={Ypay&QdO|n{X?AiU%wp5d4~o;(#FQdR<>gR zTbE6U@Snr!jcT+z*-dgm-llJF>Ba9D91A3{!auUj4(iF*lA5ETyu}L_#707z8+dr=NtELARD>}t^`_&DuEs?)4r0=?pCQb@&*IeiD~eGY_RZOYTh4Ws_9 z&Q{14lVvccZQouoDt(=q(@wF8dC=f3J*M9kZ&tA`ElsfM#>7N;R`ieIb=AmM$L(?0 z00v_t!F>>I?ftAzxX)fa(cOdA$5lz`9gmqWp0dRR>4I%QdNWzwc_%mU{BgYXVU6@e zhbV37Ylk%sgwExZPHSqg+u3iblbfa>;td{eH8h6vr{_3vMh!)5t?=;|?<5{3%!8j_ zr1-db3puQN-wCQDhV8csm)P$0iJ&&u$jFe#C;1TzxHoEA6XshOp;-e@#iupAIe`*c z=JEjnh4;Vw!vz>7ywPRhLhWZN*bd?e`!?7s5wHM^Zq;92D#JP9pZ=)8r~V~m{UPM{ z-|^llU<(KsmHQC(pJm=Zmjn1uZYS>^3e-+JJ~a~0VY8=6y5I+Mq(|>d%S4=#05SUm zmAn9I~Us;YPd4X%wV^SMv{is79zMAdu8PpT;L%z=!9A3CGnu^YpCP@SZo5Uu+ z9f)YScj9@%4oaG}J;Q%wjcX5Ur}6^r%8c2)waS?k^+J92gfr1pS8;dhbyt4;qn8*I zw)+yTX0-4-uZfZEJ@H21`GJOR(l~wxaXLgOlQ@>KCb&7{%$-;WlPKJuIM5aeU<2Ql?H{G zrv=8~1DLxe@Q>5+UMI-jzlsPWWmTZnsY~Vl${(M!=fCtYJ3L@k=TTfBpxo?eg(+^N zm1ky=7y_TGq|z3UFndhUUx2Z4y;0^>T!pZ94AfJU)*YN3*R)nia;NbdNb%=00H)Mt z(Ng=#C|$+homZ0bb9mei7VbNoylaH~aA{dlI@|{sIow>7`AN$C@8hx;0b2FaS3c_Y zKdH~8)FT0UQ4T8z0Q~_h%8ynItR7L<@Slwxmx~-KKBt}uZM$}C!ha`%$Bn;#jT&zp zn0ib@^w?$MSm<5ho8DmIJ$v+MP0lTOJf}MFhlr^`FCz`=?DDg6KUMk=$ereAIR;;A z?5#7s@XgP~<&>|b`6D<)mD<++!c$=%v$bhLvn$;CYgtbo)L3*th=?#=J&m+*LBAX4 zhz^v9zmbo;n}_ukpzqcKtI@0GJ-`YqPQtq3Nci)(v8LQ1C=p`_R#dXW^b`D7@?s-d zZ7g+usMob`DYf)_%_Nj_xKVt&qd&K_5D<&PDlr|-(h zZ0iBqNu_~TxR1<}3p!c{zTkEj~e6afw47VaYaO2STDb6E8U%rDmLfVQcgZZd|Exi;=+~ltH^!v%` zIn!oYsi%YvRO_tF+p9IyjNjWR=+hp3yG8c9NCcK|0nk%$ZpUA2(EmQnALZXK6ndG~$gc^jXkE^LA)Tx|qPrTRQo^^I-0}9<1DZ?|3CnMB&EQ zZ)W)!G1nR;mZ>!gL5Ct`bvvZ^oE;KGw3?nqQvBmKWnhKTD>7u z9&~PcB^sG06Oz-hY(1@<-T%e2lCzHK!CnF>N@wos-M|1m!rljf#)at${p^We)0=ER zt5m2rT(8HkfXzsZ9S6@mfy~ge$E_d}(_~?;^YS&e#o*`%_DlaNY{0r;yhac|{c>M$ zcX(j1`{6NH(^hRZNG~UI#AcL{U8z-CFl-nC9aihQGo9urgbqHXbVcee3z)zREKL19 znE&qp{DUlgdw5G}SaCpD^vCR-`|lo~xczUc*!r=5i$T)*izmg&ch3du$&bc0S2({c z={Y4b2H%OKwowCZ)7fYrYa{ydTR{7xB{~6qvwOHEj(qgsP`Fo^@RB#}1M_i$Mh~eb z2D$V?Q*my2IsQyq)^rlvde1TZdLO5O9&KUws)S`kC%10E43$)WbC3ys{2Hgf>^^YC zVR^Dtm=0ND-AS6L0f$MHM1cQ=*;CLUx7>4G-YP!)g)1YZz*^SY@#@Xy`4vurUzVQlt z-C&HlLQc|huxG`K6&A^sm$ai{+$h;%2Q!=LT#g6U|Nhycs*8ZQD=A7ZusXMNyYwrV8zF%W%=Ot_Bw0P! z#9v&k|9X}`s&spchDbsisLk7&OYpV^Lr5e?^edyOR~9 zOixnACC|7d<9yJ{>~9dk8v0UH>`cAlugTmeiDEHu<_bkYUPKTY>7`Ii-{R=&7ghbj zX@BLFaC0yY?v;~z+jek*cw)Ttb>CG*UjK%kTr0`5W%@>%sTQk96FH8*Fn2Pk|E8Da zsBWl(n4XHAYf4nOQ1T+e#uUsSi5rn$x}9gZM)h_eN0&@Fl!#@fL*peB6oEou+(Jmo zgrUupcZUoPIFJ<;RkV!cQiLVd#&+rEC5k_y=?SZifiP9n4X};_(?KvwAzZ<%3cO*n z8OJ<>7RI-q>dIe8oS3BF_WDD#L_^{ zx)0FPrD{$m&)^9ufN%sL=MI#6)1R--&EObC$cMBtO7?4*Y3zc!#+yt6X*9J0G{7qG zv&3Y0a?ip{4V68LPX_SD{kjcfiTGTqfqK%XR~}I4FyT98#o@sv)0%%R|7_Ux-(lPS zm#rN9=%l*oPjax+&A%zwJSvrXqq+E^Yt$qAg%!%G+{a&qUF~S`B^bx!MUC1jL!8|D z()(5;4!yn#H;Nv8lr9fnB^6(%8kM<&M`{*RXewWW597sQhs3+$+8T1L5(v&IM;IZz zW%n^3XZ(dxvzyp%FYs;pO1{UAui9U%nwckeVnK8c6mW#8v)N;2S~s(kZ*{tmT}2Kw z!d^kTPOc*6%$EhgEc$L<3O$@mlBn`rpAy7Y1xkt+llsHv)Ihi9D%wfO_1^n|o8}Dl zQtXX!uB5HTXHKb3cIQa;9&fs}f05sWoCfpX=Mcy5E9$l>gPtVL5=HqZlPEceCN;uj zFpQ9rQqq;F9(LW@D7Ud3UYtc9sB``fip6TPDi9LTr*i7~txI)pI7K~8vICAtA)gJz z#9#-rH{3o&Cdx+z#Gclv|9hgUT9JASwH~1@DlX;v>t7GJHcjd^d2ns& zv@IK5HkQ-X%1}&b`qlO3t4@GZNZNtT#RFKMZruNV-ALME-D7?luu-t)5@Z2q?>P)ed7i&s@rFhZ@-JUgoqcNcL-txH8^ zD2LfL*rd7=T~bOE`3iy4DAEzFmnGFJ%76W1EK^TIFqaW*3{4Gfho;$UD`^)T$A2Hm z{Fd)_G1W9R`9eChnp3!eN7}$!Z*7>3dhQz|4&1}nmXy?eE=5)++Ep`sRjK?C?<>!kBR#0oap?s4u3FtBOJ77(1D2c#`#2q`jqW8=NG2m^ z=)i({%mnHG3N0tq9UENiW+*p5iI=~Q?*Wd*$@Z2cWSjN|{bO78|tfIhlNC8v1HM(9V~m!$B~On9S!*@YenbOpxa z2jYW(r6PjB0~W6GDf?7>yB~Q1rXTi|=$HlNZMkBp_me6fO={6wy#p|JPIdLC=dh36Cnq42q<}4B;I5$tZ7+ znU*>U@)e_Ti}S`Cy!lp7g;^G(sI&uIg@6p1f+a!x78Y8F&XDTdF*4?~`hY96ax%h{ zU!lA5nix&gba-a?O?<4#ZgxBYL+;RogtV>wPCQ|-dTGW9FdiE-@+SKuS)X|TiFZv) zeA72DaUsk~dTBD4nzSH3pbOdnfrTApt@h6?IFMTx9$QIQEIrjUYXtS>7Ap=Lg87i= zG-uIt#;M+hCt>4mhs)FOO&WI#eHZ-364~7nIP6JQ!%RupSn@5*s@lpOYzHl28S!$x zsB9~Dj>0)(UE^a2qg9vhHST$a%Udw%-v@hE?w7I%0X>}z7;PJU8c{-hRyrNAq8-sE z@iomcz#*W1?#*_xA>Dy|!-ZXuUPWCw_TNFxp37zX__KLqi<{^tR4P;To92SEsj2Z5A&h5NJ+^h&4rcTo#+96pPe#W7v zN#6v%>sH+^_?g5CZz!f3-hfOruEN)E?{WyVT&XCpSbqjcJ%*jfm)UjhvX%Dr>Ye%bUqAr>24HVDUu~}@< zm=HbnUbfa|<1FG377Yw&n8gEa&eK$NZoB&oXTh(#20576+r6zpra*7GLF_lUZ!+^^ z4-t@s4WOKpwEdq}b%Z;@rfr`bH00#gnq1&$@?EGyYY2~NPWE~!SPj4O2?c2@UASiD zlv{^LI0NRlQN+5xCdW6J>pc6riy;j`F-}0+yJUet1T`e@f`J8jiR`+53}?lul#|VxZ}>^ z&r!R#4b;BmO~X_qDkr{P-XXh3z~nX7_Lh~1?DTat1ekcv9>HHs*cz?-V4>q>MH7dZ z)n*6%TNndE-8)hMPDG1=hSuUr0lS3tvCN<1^RY}tq{yC>J5&-A(NFdh8vgDR0SY~| z#L7H1#5MH@arfZ#nw?=zQs(r!G!q6ST`B{opWBTOPXsxs91Dq47B;4 zYZ}4v2d1b_i242s7HJ7K5DAEB2e?lU{GPWm zafEo6o5SMfY$qFh-E;N&LgPwL#pwe-=;{9>4&45}rQw^8`(5Q{onNx>)6JPFfRVo) zFzv8apg#L`xx179Mn?K2UQ90b{_TV>_P>tVR7#^!q*$<*jIY)UeD^zqi=swnBqEQ-=B>7O7D(F2CuO52HMZ2 z3J}zKv+Q)TrmPSwl_;}HBgqE9`#0mvtW)vO-4LOBAww{TKf{2Ztp1>fK-G3Asp1YL zV2tEbZEcZ-f;ZSQH(Cc1UGEPKteOd9oZxtqg&kKwxF?B`AAD$3f_K~EOx|X3Y!*?{ zrb@mL*q|W9>P7d96LtG2{eDdAX|=Ki>EJ_0$jV)jJ%Zbw>z@p2Ov@edfq0}h{0bNz zXen4kSUO{KHgSX(nu9stH%3|CtZWf_5GLttNQBJ{_|PsIIDC*FQjVBny$1jelZ`4h z^|_~mK!G58S2iVa0sJ3nVrQf_Va~E^QifA|@#5wPQ82Ph1hz0J=dC_VZ}jArvUsJy zHDH$~0of_7=DC097B}1M3yOuLE%zA-`ntCAhbmSuj3sPFMAc7cLf`$or2G=QLKD zS7dlc&Gp3AJIT`JA)0oIo!!k7!XQ~~hpcGL*Q^DX827SVa)4||fOPrt$&KW%7h68F z?SWeXozL{MK@#ODgP|i$fWqcT2Dv&3SilpGXW+v0gL0|NT}FfJgNUNL+;PH-dw?AT zxL8*Ga~*>Iui)2LaXZ1?n*N6GY7&B^o-T&b7_m8XOqUIF(bA{w&S*tXWAKEzLZ%P`-I*S zYY+p6Z}7;spSn7e=*fQzCy&?YLOXn2IH$@z_{ggqRjL8V6ygt8lVWa`qAh}-+10VA z#!uqW=cCrglp}_uJ!Pa}{T6Er-qmX;9H2~PeycCD z*V6W;n({-c2_>09!E;WMgMW00dU&RN&VWXil zm@ZN5TJuGmU;tQBynGAGzBP5qWi|K6X)SfT*2Yz zlFs)vN|1@tm(g9ZhX*__?g&o(WY>Fr>BVpJax42c6WE^(MI`}ZtZtLouB0Z6zQ|gF z@w(M4sJs+aEuRtJ{16k;n{9LUcdV_PIc=8;HG~LFI-D=I73~o)A!Hd4FojWh*ff|BrDD# zyCFz?s-u0?O^byC6{O&L!&91q$trkN*a$Wu!fU~E#DpUrS|$ma>uPlgPv>@-CpZ>1 zOtP6_jasPdMpGZT z2zu}@;mzOWp}FOk0~=osq~+;P{T}sZY3o?KASenXwGX+mf+aRCK33CT9TjkBuGLe( zA$~djON3xBf{?8XS2n)G8AGTIx*$*2{$E`Ak4E9&wTJN5NJ;KPv%*gpnYTlQqvOxY zi6$nyMHTx%ATyDV6D+iJy`=#IX_L6*v(!wuO|GTdTD&pM*{$G7kaZ-UJxLUIrjEGmp?=|Ej>JD za;60B(e8?DE&Hh%&Bol?4y5xqPKsK209qkGX;bxwu5}SIjZEmA&DBjWVzXTeJm~es zy~gPnqEd%J-iVeXG74@(#7X?wLYjFi-QbQHJ0e*3a`82ORz8=QD&A`h&qKD(HC1hf z^pNrsi-#MUu$o*CdRlAn8*Dx>Fkz2>FWvHPxhS7VFKq~@1AYjhw9edQZ0vvkf0g06 zFLI)51R9?(HFq_F_tJ{CKCP{W-I&9=^i0>0Q47oRM$7V1Ie?L{aohHTXU}~Z1WrG_ zj8|CpTfzEQ*^RHup;%7n!H55K?Ei#;+a?ErYB6@VXv!8CbbI5{u79`y|4~x@^ARu% zJf`vWX`uCC^}*2XiCzxhbV8Yct?i9w?#w1iX^A z6hb9FOKe^Fv4a~l?U!*MH@iy0Eu>oGDqUianf%IjXNc+Qu;%4<va))J{c2W=E5B^|7TUq%>6fh8k<@w( z<=NoMn2uO&+YHT|qb;SeSqqYGPh;X$`l15HO4u7WlJT(w2o+Klp}1LL_)UNg4~zF+ zE*I&tUn+B_56+6)PwO{bDGa^eMAQl~JzklvQ>-5Q7Fq#h?#~u7;2H0lkfh~itpBjH zClJrW>zS!&?K7Y24L(}1B7J*6NV0Ax5=&rp-dGj-leuX%Zi}Q+{G83TT&g$qJd8P_ zgp?t^T@|HR)^R()3g$J^WbsQ$3w*q{S}!UOxodd|R!8Jy4xVZNoV}0;8tTjR@d#W8 zG%%3$DZ$uKG(`yY2Ilqk{|%V+BL@i!@GeQ(TIV3$>|(eEf#kas&l7Q*nM2(f5}y1)cKMs|Ap#Uu*O24G^7DNG+|V;v@t`2UTJ|x| zfua3X73$nKo+VnMzJC$!*uA~J>eL8Qeo`TU>6dsHJjM>+gc{-bhD9S#cO?|u*KvlF zwlt-wLP>Jh#$%OXDh-Aasv-0oBYA(LEQa%(X~OUIaMid?KV*bqNRJtImm#O>YR68^ zM9HbdZX>zcugCu## z;MWDTP1JDOEuKts75darT}j7#QstX@Xl0;oE-i7kTAo)jJr`=~ zE)Y)aP}!#;FQhjpYmjrou)eQQfFH7{ycGo#<_pKnnD9!NCjM z-p`^>rhAhc^TDN>!)IgbF=NLY?4h!G!}>iv1Bo3E$h4CipNXqtQ=TA3iA|`XeC;t9$*JibXD~KT?s@oos z1?Lw1f; zQx2B|k;$Caqj7ttl9R$bbx`UiTYy#2mpbxVJo);@I(f6vuWLK-)GzKZItGLa?*H(%MLM*T7uQ()E zY7CzVF^u@eK}#(&Y#K)sP+|?2&y5Ptk7)9$;Kn#%x+Mwziii`T+t)b{T<^L%b7C{Z zWD|fw@~~Fxx=?y+gthfH?Dl7-&$i!~spoH;jgUY9*zZ2*Foj!8w&2iV4)PU^*DRw_YuUKM9Vjj-m+d_@SRVs;_5 z7gK;V?euLczV&Fzmu{z|sJpA1(CPWQ9&Zb!_3t*Df%1EY5Ui6{<7Kz#!?ql1;l?ai znVu{)JM9klPU02#RuD&wzvp|DindtZnQG#v5jIyM*hXUn0eg#aT_oA_lB_=a|9qg| zoj87MlWDWT_x-L5kZqRN+Kj^M7V7C~Tgf3V!(4y<|7-8PqoT^zeNhELL`g;{vX~{Q zB*_^R5EaQuL}E$KStJ>VB1t4i$vIPnpnw7rr3562vr>!+2E>~U-t@Tt=hurdmV#H&YCZ+xtck4 zXZ62HP85`mds~B8*M-9#mM=#fJk}f{I5u8$96a;en~dCgPf(yhj+BgyJh#t#a##t% z09Dq%wSVACzqE}pX{a2mo)KJmPw^pL@)Zpa$M;#%)gqBo*KSuTfQ+=8WFfgWohQu_ zE4DH_uASr~|M0t+#W>xI;?dqkt-WpQkpSLR z{*lZ7VzvLR2t-68iGRKW$9}&X_<(0-l&C>Xz6C~4UI0(avy0h=viSA9mZk5;pp@ zi@FNN%>Wfz+GZhtnl$)oKE|J0L-mwqo$z4;H=!I(d!<2QG-M(Z5ge?>jMMT zYu&TOB2Ai->90>LS#Y~^##KJx`$M^OPW&r<|ndq5Fq*-fs23 zTWf7C!F@RFatGs6zH(fjs4u81H!7k8SE1iJb4LW@_MlJNrem?7V!Smu{dAKHY_WJZ z<7$Y@G)jpLG2H1~J1-=Y%)!kv!5Q?G%`Ml)eX{WS+F=}arP)Y$Vb0&`Q{w?R`acws>E2`rYKDvip~ z72s)qVRloeiLREC)p4@;BI+&Lw zzv@(C9N)dM8KObkV}DNKnkxibYmGEs(G%)xI!!<7QFZ6))h5ApF`bwD`Y`3mRX4{_ z@>8t#WZj@7q{~03h@0-`y}|cVo@ZwBYG~j@AnTgki)cNo z%tq@o`XyQZTr})lo~qST%1xcIt3@n~(((eB%}lH_6x@y1QvR$t;kae+4o9UpoZCG| zU7H(Ro2_#rdwF!MbJzA-7VBvN`vD5Xv^y`${4bH7w#uCqqjx>@^d4! za_sD$qHx6C4q`wq&F}5bHz8nPr*#lMofW{tUU}78^?bj@niwNqY)eE=cWD!eHH^NA zd}Lw9Hn_w#SRnZG!(R91IDpP3i`_Acx$zprtC|mLI*ObUF&eHJ4v#mwJUgb|>fF6l z5K;Z4Q}2IsV8l=&!bcMy8dCl_5dU==h?1ml_>wkDKak|Qz_H;$8NN5`A@Hm3AIDs3 z(vBS@3_|B>m;CBuVj0$MFiu)-8QZ1(IJ?K^)77^!E~l>*m38PA1fIdJe>xm)HCv2h z2TJIGHi_GaiYo{3w*3i8ux-WiV1*`OncqjPkyw{G^#Md4$aNjP8i7zlbVZcua%6*` zVfpe`fe0Cq&GsgoTR)Bs3p1%=aQCvJ)0ZWrv^V*=Hu}l0G}KWp9QvpwEALx?D|PED zWfNIZ5+UOI_gMT_jZpOvl0bOQdF|Fd@6n1m^wCv9(pdY(VX7lKYd*SVdwbigXA8{* z1@@MJ`}mc{=-A~S`SG>gwaZ0Lvfq~)}_Z~7R^Y>=iHun zH2dY!>R>k<=Q8YZm#@rlN7d*;{uSgWNAP+QisQdNFx#1b`N<61tJB*X$4gV$r+9sh z(8&Dk_rbUUuW_sK<~Pe5>{JibrTd+vt7cpj=H7jZNrp_NmKH(G+e+QgqJi~li#>8` z`JG^==Ym()_GCo(w*@}oUL7^4r`nnIy4molMNIk+DMd@3VBTYID z=W&z-9AZ<*-%ZHJMG~<#Sr0Ue1{^EWr`r8mRYmn|Hg{^ee_p3VcvYW~^Yl{d%XIb4 z>%P){sVTlv_gvi6wqC@xuYRUJeCWGrGX7@VavRrin-pf@7dQmkKX1M@wif^0nRsir zaBqH3s7+^c8zlwYYcWNILfv=c^0N31?r*fR>%k69K3>s^H5rPwQ!OR zlr(G&2%yMJoaTqP!n$tuiHg|F<7nvwyc`b{>{)Ai@7XAEt zy#o<0zyts8+gxhh+k+N<1-rVs`n}(uaj%|8jfBNZhqL0O`m{oQV3GS8u z3TW(8;w%*eJ(ey~_%FUm6EtvYLF;s))B5zyw^1@ zgIfdtDA>N)9ut1<#!hoKah2AG-u_AT{_IEpCRt@=h?HAaMMJia97AbLNeNOcht!W3 zl!XTaWY{lxDxYn*^uycZ4Hw zUW0rd#&bUHwHxF8e14cHjQ*AQ81?owR1)#*_@z&qH zZjHRLGpoz84=qNRe9zD)5e0CdQ+>3cHw zeY}wFef^sL=8sr23v?haOVs;_-qz@WqL~*%@{f47mW;-{jDE##rOp=hwMWga-ZFJQ z8oC*vP*=EhI%X0-*|`;}VW(Q{8eS*=XT#wmAYe%fgvW>EH(Bg-Td*;1CNNfP zBQnaa2YLr&k)6?E2ysq@cnqI$*W0KE#pT;9hg2Q%W^MOXBahXZHr>Tjy7}9ixqHrN ze#X5;Ml2kSuwCbAqze%*HfV^29jlMz7m#_hl}h)BWJ$&YFO!|mJ}PnX&O5b%ojC`_ z>r=ruKW{DF(~vPt>QmUT=|{91Doq&0BE0~ze%`{ z!FPZFeMIX8F$3+3F%^_hW2teR$9rQZGJNVy$%^_f&3asEwG#=S?9O9%Ow6uy$5Ca< z(D9sVQk6ccEd1orn}<@%^!Y9)FVc8qfBvMAUus}|xk1$(jnf@9H`r#1%Z&F3Sa{U= zqjOQ9{KV+uh4QOAQe#x#+}lC8{yYy4b*i&ch<>s?S2b&+y0ng#7q*=(w3|-1nZKQ> zIe^#itSFw`EH-~2&D9h89p5~VY<@)`st;vz&ZP8Kf6w&SrkVWjyna<)DD+ za&X&^k8^Gt<@eHpoOP&JJ+R*dsy(oF6tO|XY=7R=vf3m5{`~G+r5-LDNU%oYD#BP= z-3rOCWvqwkZiJZ-Ji5DxIPM4^=ILL(&s7F+>ddu+dASzrz7}fklM(LkxnqLXp9iT8 zECxEzIj(kVT4-Y?qcM{;E=W&UE8qRmy7Oa+7PQxF3`d$D12kb)u;@InB(!m7EL=xu zBjD?;)$dZ9v~rjTJYcGnxFYk#vL3oIwoq_qQW5)1<=e3GhYqF`8m1-9vX+Zo>)15qSox}rr^smbNDl3@zaDvGb#Hg^oh_Q1)_$6fh6iZ zC6VIuf(nZ&zU1n43IWG?0@f~EpyglfEtcDoySbbCXhWfgajl~G#pj09nUo#)T7%oR zLIA4BocH4{^HfrLbXT^Wy6erZ7Qi9l3avACnS;&0^k*YW`mJ)KWj+K81F+v49MBfM zrM%}xx7Sg2g}MJ(7vrJiLn$toO-`SB^WcyqzoaOCaRwx;BM-WPb?vAAYV&n|S?0jOLzJRJdqf%~PcE}a`&%wIhOXLEmbcu6`DfoU4YB7LwxBMn=Dd@63&A~EjO)gb#6Pa5 z;O;Fki#~bjuIQtW&xdoFCF)HEs>(H=-%64Wc)Y#7=pOg#;2tdjYAUJqRa%FG;#AlB zJzA!j-5&d;z+~st14OPWFO{!e-TVyNGQ9g4A{@p ziv9Od`Te&b?{ic?#31B+4i0e(hxBP`BvzcZ<(qG z_gxb-$drhZ`y{^an)u!L=oorN^ICk#P|kb|owdMnSX@(%p5HdT$n@J66-`uiF9FKh za(zLs+;aGtaZNqk{qu&+k*}{PUl~z+dGfM9R@Y^gyN|ZN%Bk?=Rh8y!H#l2)bm)Uu zNke|8B@A0fycl6;>gboS8*@7m#iVE0Sz!j)XbiN9?6C&W@~&1qCc|B;ynw0+QU~pDIxbaQ3@RD`s3$C=Ka0h)OsddmP4YG z-}93+RmtwRRaZTK`Fwm_YGwq>Kfb;=l5#8bLUe-oq^DO_W0srMKzxgMIWa8jB%p_`$-Hma^kv6A9$rtCeD{NoGxuGu@geo}d z+Iim8%=VQUe0WB5-C{sc9d3W*lUQN5E7u-CAGzi z4Wt&VRBt}{o+l<9NH=-y>DWNE3l#N-=;lWAH&<2JZ`w`-cNYCo$aCe&twwwoF}7^g z;i#-7n4+|6O)&xv@(mfJa-s9!@^2h(z^w>5ElGZj6XA&9&<%&xf%3dcbNz8J^nrE$E{~k*ZgK6WZ4jLWNmN9Tvob} zD|X0Tvu%$eIf1%@LFjQdGwpk&G#&yPS2iL1=g#z)KeGTLdbNjWgk8&WwflRg#qcmM zeO%~>7ZgNu6MPl7x0hSP>0!K>p)7{}{PfE2a@*MM&em~i>Pwm>PZrIr&O(D^h~hGN z{i`{uv%q4cJ|JMLt@t#X#yRDT2a8U3Wc;9Qe1>~xfXmUo?y>6%&&^J5$j1vMdY&G5 zCPn8Z=&~wL&LDV+KM2E;qf>2)SBMoXUz8c7LXiyT4h3odXQ}0je~I9j|0X+_ zgfji50b}tR)&ebKrt;_4=VNO5zr4B30{1)o`1#w8F?AUqVCSrqm#`KZR_*;vm2A$S z>Fg0Wi#X*@48M5`Ji>$pAi{EYzr~;oep%lr!8sRxw1%`lH8WI2uGz#jEO8qAWJy35 zhDP7_v}no5O$Ra6qZq{){_`|947Xd$^Ynh(!6+T=h)AX!?Q-d_{u90Y&&s(xQLKmu z`_AS{C=|=@r#tQ*5Gu<#{Hsn0`g4S?`^F~{>N1=0z=j08rOmet_E}KGu;vVIzhtEnT*ErP}~Ib#?a8v0;dmtg?p%`{hpUJgNQrpkrNl*7`G>G z7_jYyi3Ku>u(w3>MnLK78?fHp-Z1DXFnG;na=*_Fzxg0nM?0KD*Ss=Fy$(7|W9gtf z+Sz05OPCy&<;P-tA>B=D$9HS(Nix=RKW@SUii#$foybPojDEozmp5lBKq#Ji@Lsbt z2j`|!<#;buG0xmO>79^EH`;})BmKeq*NYd;dW&@WG`ZUxUQKVw)+s>=ov~{k^w{g~ zx-a1YV1lYcS3-uXnHd1t{fq~fJ1r@4L(94Drg31_YqO*KgQ;}KZHjQBq;3~0yHgdS zli|Yj=)2S0Ll(CkHG_pK*)&Uv>~i_*J8A-!P)!9B_AAQH(2;!^97=^U}w9%%{q7QfpGn#ZHFZSw=Z$dGy z!_&Pb)5&`%7IjVEx3h3^wgw}t1lT14jv#1KE8KMgtp$F^-wgJ>(L^eSW+( zF&3A^K^02@Kx*DOvDN%UVOJwa`ECPLJftn`zOA=@*XZajX!digCAxmg7P66gz35`o zZ~o)lf}SX6kI8CY=s`EFetN@+W%VSTR&RWXVTt#W;5tO+tJh+UWg7vtQKfSC%_d_L zjOzqC?9_bm)*Hy^@4lhD!Do_`Rse8c9e= zU(aKzeS(ttm}_pyAOa6gGqnf>fr-S9-4DYuN_1NOJ$T-?UA z%b756Z%f9F*iZLy# zS+m^FFWbrnr(J_ni#b>xiVu#!VRF2+kuxff1w~ZQ{OEo#XGn!41HjTp0?}eL7|TG z^L^>${^lX`eM391@J{bvr^TOq3wbT%yy#t-$G2`!{GAEnNgxwQ7Xod3bClC{wzt>- zx|NP17BUQHYzt3Heuj^iI_=Do=s7U5>1rDJQkh=F{fDDSneXJfz7R^56Wz^v8ORrI7`oxP-hpB+K0sdQ zbhcV!1XS@c)D_T|FKm7dnCN)u4L7P%;Y7O_eKe2R_1Y_cwtTjrzLHICV`$nF0`+}c z=L%vO3l)A;Idy(7jTt;DfJ2s7N21hG0R@TP$bJjVvG@-+H!;gblRNRPPdW4ipitP6 zE|_*_Mfua*s5etbkiNUorTzMY@(HH&5#`E9S!e!6gRrD(ljuWbSdM?C$M*cFu#@}Q z5M;q0%g}AsUr~&#MT_MqvShsj)7{yj0BHwj(V6k&WS(jzPt;|%;8DrOkob+1fm*-dqYlv$1R#&$;drrZvD4edSJ{)pn;29dpOeecAw0f|3&e0fXS z;U16m;loErNP>a3{2=MJ%m(Z)+nuGZfXg`ic}27wJt^iyrefvoT;?tUq@H!ldXTUx=bJ8BS1L#+(rveai2I{0RDu{qc;*7y z5;%4>yr%g?cJQH8#6RC>&DFueSD$0XI|82%hEgt|kZ_P$v(!8w-OR_o513h$pi@B~ zAj_E$*Nr8q@7sXj!wHB;%K~zTSyhHPDHw&h#JzpV=}pUXfaNxUrgP32KM`(rc6y_o zDWxrCzUI2Ym*>6Cnb23hbfzhe0>EO3p1NTJfIDc*?pXZ%6*s@Fk@EI6xm=xU`MRvC z*_IU-ve}NoqP#b|DwCDsO;b8^P+OBR4YdUnoBLG=Lm+9Ey8C_gcbz9UujbRMGw0#b zbR_t>B*h6Yb)?8$bdJ}cXk*tZYqK7ACefxysthM4t=d`bLo^yqh4CC9K~!ZSKyJ%z zCto9+%CP0_sNJ#IN|VkDpJN0{mHMDb-}0xv_G8meSTxu3u!`T8>f%5u22z^tz%xK%Yr>#}o=uq~T&m_m9eBq3_=$a+>(yKt{B9(mFv8m3Akl zz-~eNqIqX(K*P`hvm}?9f$B;tU9<^_|3Ea5o|39oc+VBP>2ABeXyvE=NeMCW1ZFP3 z8mUbBO0rXYd}TmzQS1^rdj;4@qTZp+Qs3+eMz>Z+QZ>u0x+zginw?6fN{i#*<+$~vi;P_lji`7 zT188HaW$|0%3U$0RLs#T|KUq3V4vl>F&beWH_e@SH{n@!aI}~gG0j?uy42WmXP#bN z@cAFrE;vUIANRDVY{fbbM9th}uoq0*WM4QLX|p<`ehUtDYLH99W^T^$9OYQDS6Jcg zU`>{j5XU>H#0}?#D?fbIzo=1U%nTwZ=UWeayG-H-#hh>X`ZAM)SZqJDiQixQ&R^zD z6&E4{a78@x`8$&gb41V?FUzI&s5MrIK>+$?FJB`Aueny5J@m`tLQEDaXk1PhOuS0B zl!F8S336tAPP3}rh5wl@-%qxjPV!4ilaD$pT{6e4&>iwa*C+u2@a=e4<6yA4`i z^EUuDoXm_)=TTOK4yaZT7mA|H(uN69Dtf1HBVb|@mmAlI!nf@j#km<)@NjR!mW!A< zHjC&sui0#kV(ZE^e0e+C@M2@FDY=`9^{+;~LzZxa&OHDOEY1{I$XN}9k1o0>>Rr#F zUqMY&riywJnCnm3r0G$k5(_-`+=FaPBc@rNL`d z+9*pdhdHfM6Z^RzCZ=OBxv1~JIZge(okq9k8`@G@-GH1))d|gnE4Rr$78Xg6c$X!9 zj&AF7=*6H#h9YO|)J_zyg-+*eu3CPrep{%ot?~K`-+9L?Zf8t_LwwC93#S})PMniv z0;!FcY=BfT-hWpZt@f&PP)HH2QtGy~N;7=ha4mH8e0trSl7ztT`g(>IV976 zN2Pp1ipU5{pU&v2m2^uipu4lEgn})Up{9gEDXVldvnyL)7_O6c^o;2U_5*f}fcD}1 zjh(Hv0EsW4K7|N4%te;4DW&@7N3^9J4%3!6$z_|M5Y58?C6IF%1le#K6B6+Wi68vq z(47MAo-Ax5E+oReSCVmu{>HLiqieUbY1dvHtqes>Dpy@ij-nU+@EJO)98HUhC*P5$ zm=WmaHSM}&(w^wkFzQV{wQv$O!v9={wpu!NDoA8E7-(%@!3&)Ts66F9Sns*`wdcYD zh6fL+h|!~--*8%b-08Fuj}<-gF6yxy%=(ZpQJxCD7@k4XC{_)A+X+}>XR`_%6HPorVFvIK>Vqi5(aY%)^RpKe-Z~ze zY;#elW97QrkkKrjNoQ7S)^`E*_7~c1<(n#Elmzu}uFZ1*wz%%D$N~7QuhfEc%OsM(KDxo zsOL4DmVSkRhOmhjbX@vg?NT~gnB8gv8k3IL~1p1x=^ufEYc; zCMl=L-=SB;p0I>(MO`;wb(vz;5PCe5&9!hAR!X8@Dt*{Z8^aMS{DO9!bHHUnAoi+3 zF7$_xRr+25i?y2$r{a8gx;ufH9@U*NI%`?bGlW{mbKA0K8InYU(U6!xmy?n}y0toc zLpzQ^w43uOu-1(tET}hfKu{#T#A`KFroSx$R%1d1Q-)9V7H8t;`umEeL7kPSP>&|E zn<#z{AmR`a5lzX!Z7+WOKqn`SZmpuNR<+_k_`xS%GFpGpQ<)uG0>BidAIgMXQ9cJn zm04kap!zDxc?p`JqgLp^9$ch#v^Yg-v=2622K3wNkng+haC4S`FIE=ggiM|{yvoYO z?rMO*Zc^?I8X9V_0Yry!1L#fifHw5JFGhZYK{yA_$Q|7%b(461;->HKiOEonzdnb? zye;{|9ZYj6>vdi2F-n|Q!gU;o0u9z1VdN_}+YS)ntY@}>T>PU2Z__W7lp&5Rcs}pm zN_8)fL>2+=?5K8eXi7QFq4UGhD6@EX*PC36l$<;~&VBQi`}U$tr;*O#m6KPJKPXyY zcS=TosOYKZ&>MbzcNyh6@}9^1hp`q%C1}l;n}W~A$mK@&KMZhQnWCA0pqMiMq&Rh- z(&DLl>uG$T>ZEDD%I{mtYD(Rj;zhU+|6g-9&yg8Q>}Mw=oGKD|7Z==WnKzQ~vD z92D&i1U~SRb+nNKF$)Enc0mg_30ll+9y5*7YxtouV%v#>dqAHfiDlIkvJR(0eI4s+ znBuHRtNRwX2v1*gxMxJ)8PTzDl57U?yE*#(OJd0%i@2`e1qEzmIT8XSmZTXZLZU!Q zr6^uShe5f426Yv@fiUy_iXM#yz^{)%ut3al%~=WF4yH3?8=RToDr~uG<2l_$MW1F5 z9VBz&vmAPgG$IpcGOK*j(w-ud(35N$UE^*VcJN$?&|}n`HM3|kY7<;e%;4COCQ)Zb z{4}n$2$A3(4^%ms+hdxO=HG)B9fk4YdU(lU?#G=v6XSaT{6VMM1wvNLJ)0Kie&vQ? zv!!X~{NOxxZ$M>lfy?wqS-yNE=LfUr@Sp)%07kVpiW)Fr4uz@ZKPoS7rwj+)PrX)< zmUqF;;TS!CmvIvIqG8JXGL&)eLz$9e%SqE4Tl3iSpq2{LOuY>~ck5ABPv|r?@N-ATqz_x_rPs$oqGA+gn+b0z1X5hWi4r?a3fUNIrqg z9{1=}SAyPQ;Lt~JI+{5M7TSZe7gmhMRRFlAK@=?V$!Y$XHw-!5gxi3QY|%Q!rpoF- zp8Lj8#PNRlLf;CFS|7vIsQ<76xV%UGwk)ksV~$@a_k&t6sQZbEAJTw=5(4(phRfwUBcRwS;&0lp zkyYIayAn3}12wTU?8mrirY$BWmZ(RTlhP_b*^#otqo_u_bh66|6sI@>`>e5fpeB>m zHvusqCR5|~obuy$flPJ^Ej9YvY3EN`@L;5R1)*b{FP#&zAU z%hj5OGLtwM9mXN1sgj^#I5IW=Lx$1qmt7^y&DU_D6~&$C%!BHv2anQV#hiy~pTE)< z$t*#v|AcfuxDtkd->{t1XsLZV((h``uC6E{1AYOEteM-OIG_^l0A>X51IZC=8O%8h zILUmC6`JPh1}KljDhfd9LmEJKIrNN+2+A@OFl?soB%B8V9RhW?548tkyR`8az|02V z7?o`Dp-di%#tFNLlV`e22?C9H4}9Mw6hx9+u+kN6T7eUre#)^tra&0DHlan{Z zF}~Smwdsk5Y-|gt@LGq#}2ESdyixODO#`6xZSHGy#!nRFY~*xYTSyiz%DUFgzj z8i4Xms7LTa8Cj-LQ|G*A&r?+74X}Ir9;0@buzNeL_AaQi`xxUK8qgZfxDiD%?lTj9 z*BeN@o+{s!31SdRhO(ze-R^!SGd>Vx0L0ForFh)F)EL6D1Xudjee2if6EA3PqyiN0 znH9yT`+o6&ZSSP4J! z>)>0b75A4}NkFH5T%jZ6$^fSgceW8$8ai`K7M>Dvw(Vp`y-NU6kdi=&A|KAZ9nKmx zS-kAPB&*hqKGk(E|B)1AuGH0hZKskx@V&pEa=jm`Lx>x9s`Wn7QD&u|cI`eSq0eyW z6cvXx>fNpI!13Eny#c!3-fgRLP%V~UrXH*Pge_M3FDl7j zYxPqF$eTP7Z$I`oZoXsGAp$}|128+x<*op2$WkwqMjTANree}5YC*?57=Q~}%dp7` zdmqM49qV4*$A_|w_dKjYc$l0EmTd+AgeLI40v=Li7LYnn19}Jl`CPM0yJ!fkb~!N5 z&x7>8%7?N~^feTtMzID1oaoGl32_np?E>yrm^)JQp^z|hd;+t_Lr%X^j2xLUS!Io1*-IqX?*SVE5*Uu&^o}rH);Fhf1DxtKoGH~Px+rLQnr)v07-c*|w( zH(Cgm3-Apq(_TdUUm&Pf#7B{61E)DoX8FC$Drr>DL5L(*kB!j{A94t+!<7!s!$QBz zRZtm82f23Br*XM*pqDB5bv%uRva_!LmNQ5XfDdj4LRAc2HdPAzVESyVj|3@1o3wfQ zwwC!GIYx6`5{&BU?s4b{$3@&-EtK98pIJYiF=R(TK?Z_7GsFd|HIKUaLd~F$O%PfY zvY&~ZSSe82G~a?+E6DS0tKh-=`>AGh!&gF9wkOUGIE~Z85t!)sKn(VqQ&;-E#QE!m z9fNNnCUzUpdJ0h3l3LcRT28LJvV#t>roiLwr{853h^Q|L@O>i>j@HuPVdoevuY%@* zQbRA73g2EG!l16tQ-d53jXS=xM&z>(9IPTm#12x`7u@@`o!`|Xb^`r+{}5QW>J<|X zsJlbX#{G=J{3Y_Y1f}{6@YFcdP&u!(bxCbM8YaiJ7;5XntP7o*Rd>8`0aT7YX z`g8$Lrpr%|hr>PDTH{0_M0VFD+=14VTaUg>mW#Lp-pVl$lw{U^w*+jmvqS}uQ%yq465 zLv43f*r7{Xxf?Hn#jg{QD8>f5ik)ei*k#pRot0kRDl(9Q*$_vMHdQv{Y=P7A*aEpl zkM3YG{B%4&#UuTmJ*MJL{KEBZWzj!7C(lEsNZ?Vr&n%7 zHrwVO~weAi9LQbgiuki$+C}onNQ+0YM=4(~5| zh%9o!o?d(nN7zy^2(TKdK|*ut6q62<9iQH}KyK_j%p``b;wL>FByykMa%Woc=}?5D zjgii0rXAWAkP4;&bWxk>@fIG%SV4HQ!krhVu7-m~%iX=j1Z*#LTInhopA{R~2}ekm zN(j3{fJ0s(n}E)>exXCGP7i;eN*z+Yb!B6_mx}M}7NY*yeCv^kX0f*iS2lI-L|sqGHZZ1cfF;@A>%REujPwC1m_M7930TIQ z6upxWBV0xCKSCGkpqxl6ry$&E+!*3}-z5Dx@ez|t-{Qr*s&)-0ER+CCNqQK8?Q9|_ zppqiS>MR8Ly1A~8Q|`E$$XFql3bM2hVqob;fTgDXHAPGlAwf@{9!P2d9W7w3dRjny z{xX@oGsKd2J~UJRuBO0iqTZyCHcqHI7PwEf`YRy%^DUC^o)TX-KHycFp;cb6h}FqG z>mfl2JpzG`%{Bi>|i$KH+t)ODu_K$f7Ux7{x{10h3N5ic{~IZg@O1mx>r4 zilOk)CsJ(liC8gyhdEX-B1-AR2XC1wr^?g3U=+c$->De1;_WvSWxTjK7OXsz6`;RXLBj5hFgxd=2V* z@MK>x`XXxc=X({fNFmEpGLE}TLN;y5MmE(x=-ue-j+7p6>ZmYZnlN_l-qqOJUQ3va zuQ_yugZjDsKYg7~(x=2tTdmD$UslPAG%gEOo}}BvG(w0(_Jksq1N9_{S-`o4Ye6zj zkt}ra0SdDS32~a?>j_O$44rMK$t)6nM_u7H&j@Y6HW53d`NDq(+=vf>OjpHVrB+GAy%c%Wp2-tIdv6 zzAM?8s$c*q2;mk`yvc68xfG%q2X%s0?`(pc#QA3O@yuy37|uaO5pHqY+jV2HO>&qY zNEJ3@KD4*J97JU~botV!2q}3`>5*(Dl$9H^JP<}at~ zL*lrj#7r}&3F0vJvXK%KzHzQgFR1K)UdmjcCZlby34Kn=ZP{#biX>rBpq9k7oL$8X z=Ar4Kd9`mKPl(rAZC}$>t8dE-0Km~RmC?EZjd1K*UcD)kz*;M8GVmY&R1CKFS?I1y z71CtRNcK>(5?JtOa27_2ffP+t@nh9DP_n<8__@kt3H`G&z+uqw64crfUs1`L=zb7a2U@y|YYBd% zbMU~9gC_>(M^1mm*=^?@|0-lA&O?1QuZt1!7~tZQ_mB&s7dV~j5wWh{;_vzAivKTF z;QJ}2^tzmrv-`}I-?bvoh`U1|{@;hZpC$b|qP+cTickdS!-6io<;~?#oJ0X?<}PH& z)gcw{oo<}>0q*d5(4JC99A>`XgRLhMt1-maljWm2#<|t$-*yL+$h18t&qz3hqSRAH zWhIE6f~;FnwJ+#iw*u*I%2O!8w9?{v@_mS>8;p}*5TZU!W=4XZ#Mp{B*J#csAxTEt zbf{9FK{|SeDI}e0o3{V?@d$)qNK8b4a+!S3OX&N~KL=zbpl*2JHpK#(QhK6e1#SEy zX%ECqQ}2|>Z{Sd;K!Y1dJ%U?P*|I&Bfx8L$q~W&`0)J{PHv@@V*RA(_T*OWj;^zl- zp^nhqt3wn>GasTDVYgn#w5TxptCr4maDoCOR zp`w8$VSqCKWp=INOn)faq|IBH(Qy6?3xFiTEA?^Ew?re8m zCF*AgY=bC`(JHr=;*$Ld`j&ft3d1k+Zy8}$k-ZsQmn8cjuPT`Q8Meq@tae@@?bINr z<9q1+rmP#ZcO;_6%;<9TmL*pg<;(jiw{HQ*Agh#oah%4J5piF&D~aXl_-$o*EC!x< zE346?aCKi%u6inn)(ng@n!$C=H-4HH_CK8i^piXZK+u1I0`~>kW_Hv1LB&<`wV$$?`F^b({YWKtF2YiAX5@gghiQ61 z#d;w~)kfZ`6G8@Qc&E-GA5eEIxEUevXgksNVTAJxyH+ryW1~u5Ot{>xd(QQyF$)xq zbD#tkN(|XV!?rdEuLhc67pja_+}?uwSZZY#2iA#wvY-fm`L52C3oZ+f=~xR85x1-Q zph=|cIGr|#t_j~|n6&UOg?%}pi<-09bn}}vH8qwyF5(hT!dcZH(d_8{h01=w2UDl(ust1Xp>TyomI^=(L)|=CVRU(uLT^;ONY&Y zet>kyP42Jhqwrnbi-E@+$O;uEM9|4vK-<6O6^~hOBlH4)iKdD@n8*Dpx`ooFXEui_ z8NPQ1`?n2~!kQ?gr=%S7@t#UY^Vy8~Hza`J%AxDF5|&-_2DD;1$X@#&I9-UkX`MVV zyx6AvPl+q4(+E1m8&K#3+*FdJ$wr3C)J?v74qWuuUi7O1j@f~6bBU9<$`!8%19ztm zO(F}5>tYXpvGYKd!^u(M`4)r1pP|%=um|)}gOkcTr(HNM>UEDR@*`u49>fd>m*-okEjGt9pqpoyE4dl33b&MDD+0Yj3@TyeEXT`-$*miz%pjZ>xI1-x%lWZMN zUVNhaUb(da%R`B}r=%w@QKk;!!8rT^rpqtb;$B#s0xiAVM_|&z3O@45N*dKZMPeZl zQPb}&HNV=Y+x)BC`~|~GK7C4jW*amD;Ff2wf##>XDV+((F1Hb4kccD}RfyDAVEz#mt-2@3cGJRs+UCz8DZd0_$2BS9RgNZJW}B9r}!i_7tX(KEw&%%j<`GT+}vA%bpW|L6|Cu zXq(F?&6{)DZXy%av`}>{v{|t@QD(Q2T%hdT(sjp?y_i-rz<8quZk(43=UP2+&*0Rt zU#&xDGK&Em{UeOy?V8=X?n!v~9K<$D1xFXRCvVkv-4f7dnDz z;Eu-<5D@sUNmSTQZZm8qdZJL431U=%rjT5M{AO+UF0ghc00g8anIBGVMh(bOsq?3O z9l2E9y?H0de=W^76O2dg0rxGl_Mw-^el~+l{v6-KH^Gy&g64T*L8cINNNzz&^eur zjmUrWsel_>D!uv6grpOo)?);g%+JC8VH&Ja4l^sD&uh;@@H97_@#%eSD2CVYc;Jnh z!dCxjn%F$HowZtkScqC6ElRSlq&9;{YBQ?OSD1eXIre#N_+;DA3lPg`6DUxe`Bo#+ zaWZsV#=%8qecj{k_sYF|gOP14NG1Vmp+xZsNwLok6%Af|`Havs-oG^dR6_|kFBWvi z5Eq8ZnhNxxk*h-IJdk}*U^FaNv1$}Gs$6OUY4;o`rv;#wAZmBAVw?`_aXg2ZQk8hv zkrP1a1I6JsWJY#`-5=21l^|u;#XRx0Lss4|=f3`F=pjpJM(ax7ZI%DE7z)^Z0g^3^?EFIfU6A7id_F`5WRIA{slMlWDkXFCRcS#JPcJF~>BB0b+Fa8jAPl4i16Xyo*k{SL8Ir%E=ow7V+=l%Xh45cVj2hkrg z--^iqa4IE8%W$B6Bo3d#bZ1g`4No(e1npBiB<;ZSN!&bT=&?`v`Q7(DCG!Mt;0si; zQ6wSQ6v5F1j6;%wMfLrCsn-`2O!W`Fg$>VIu=DC7qSC;AH0r%Iki-=9x_Dq_yk^(t zM{>W&4s&Gxa9}(MR~-jscV1u~>{_%j?5T9fUHoyEglS=l1V7uS3Jd*qVh^v~{#ul9~ob`5(dNlX3>qZ$TKatI_k>4jV1rJRGsUe9>N`q0-Hfs3%UCmb{< znF()5ZH0nQEeG85tStkv%mL5mREYS`W!lwvUF)-M;p)^kIRwEWn}xBy`$Is_Gfc31 z>ZqvE?R_ue_g{EU=LvQAGolsgV%@D_XFpb9b1d58HcX;(kcqq{)D zC}9|K6axgu!+{`fm{9?e2?wP01vb#JBJNOM*)i|m-CN|fg`WsQv082R-|rp!RG5%+ zV8+<=vRape6IWUAy z-*jEiTXg3Gqkb`mgq)mQy#RBOtLP#~tgj)Bd|9h34~*AEgZ|^w;`>d-g-~D%?#~`m z|IOe1cP9K_U-L|71De`}!OW2FbM=3H_uqL-jUx1&^&_wK{)Q0pKl=Q{5*UhmTbl*@ z#IOI~{=x?iioo{O;fu1qecWPj&vw`ub0G{Pd||A7{Nw$p=${sS%k9E<;7LW`QcgX^~+H9pyy=Q;rY-IBT^nIr!2+5ZALrYGP4 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/checkpoint_monitoring-history-subtasks.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/checkpoint_monitoring-history-subtasks.png new file mode 100644 index 0000000000000000000000000000000000000000..a338db9ecdb7a3ebbf256b71de741c7d5c234f52 GIT binary patch literal 170772 zcmdqIWmH_vwl<1OaEIU$BuL{<;}$H0;BFxV_r?Q38V?p6f`{PlE{z3ucY@Qnb35-o z=j=W1+55eJ@AolAk1p1lRkdo?oU`Vd&(mRVRpqf>ynF!%2ZyDoAoC6m4tW3$4zV8% z1$G7Dm5V0@5Y3a9$($X||huMw0>e zP6~&nHR6o|0{W}IAR04dU4sB3SsA1$BExXzh9-n6Vi`h=h|ht`SXDoMs8yOsZ$Ea> zA0CUGw4U#z5BQ`EXNb6r`n&9)em?v~h&EyB3CDet(&Z?w_Q}>jl<1mNLV}1H&h7iK zMHte>*ci>{{`AM<7HX*+yhc;5jDuDG=klEX9k33Z1Ued1rECdikO;2tFg_X&yu>uU z-{W*jp!Kn&jv|;4Tw#%_@!@uvm+2ShOK}1Mvg6aQ zN*WV>9?$GUmi&49D|$oG`cS+ahO#knipr|~+htXKGGXgi;ieQv>d}B?S~2Gh-*sjIQ5%j+`8@9*Id`6#98$?kyoJf>&0=}wR$<-syw`)chD>155MllCaFm!kensZZ{^2O z?mCckXxxP0NY>zeB;e5CV`;G2P}w?=$t2LUzPQnl5PqhspgcrKG!Yj>+wW9&q^Jp; zToY|TDhmK|q8-DdlOm6H3W9JLY0A%=0s_A~GGo69dPf6f z2$_*g%clE5_%jGw0&8q^jODkeEAoEmu|)S6=>^IiX08c-36f2x+$mi$%H+C)Aifbw zVMp;Ag&_3`igpLzDP99I5Xblnsww&dMWzIN0_Gd!{D>-v@wY}laNmo*CVM4gCqa}m zsLuC;@x91TayP_RlC2WHQs(1?yhL^Z?vnSpxnpb>uP;zsxtj0|W9%h8vK_vMj&J^U zAmx@LFpqZXCB5^sf9h zhixnFjwBM$y*}l%BdBeF79BR!L$r3a7QHsIj^dcx5CbA>!CpZ*iHL!mLp^VtA50%~ zAK;$Q{o-iUI2i;9q|j`;UP_^_<7Q_#W^leqW_XPo6;{@xVD_3TNk`I_&WK@!sVy-! zc0ArG4n2N-z&5Vvs}Q4$?gyP60>xMhxih*~UwiWG@`T@^?qTn-rzrE&d4Js;NZGR6 zLODY|lRlHz4KK-Olm5outX}t~PS#!GDxX4fz6d2MV;0ryVR^}N$6Q5pPK5K?=q*L|cjh}nz>A2g z6lbt=EceeKO>QkS=KR-0ER>r2nlH7r-q&eFYmI1?y#wa5sY@1-7ID6x(0ubgx)`&_ zM{D|BezEiW`k(wo*gyQ<>VMzK?o(%dZ=*&2{=P(2^IBa^V>Nr_=aU3tsc8XTL8YE~ zR-bGX2}S~O1Dgpuvw@>er=g0@q%x0zOX-f}FREYY>~`$p>;W@Wr5JjE8qFH98jaea zn$b@&dv9HrTtUK?!n>}S>WM|JpM+}z7i1}Cr>8Dwi4Lg7j;1rFcxECdh9+%hyuYhW z749(|B<*?Z$B_RfW#iJZ8vE!t`_)8&XY1uL{~`Z_iG?kpc}?$+>EA__sa)NHkcRP_ zHuU+PWs5|`r~ws?5|&@Qzg)|$v&d{eIP~!qb2r;#r->w=4_~KV1A5q8_454M9e0XmU-$YWmMf*h`ZX>0Twz|JNTHX4jKW$XD>*FV zQr5jSL#m`qYK2j%g&9C05kV_qY7f6{v-P+g=eg}Yv(4W0-caJ0hW$ovL!5>;OF!(x z?8TcDm)w?9j`@ybL@&hVsA{RyeivN4zr(vzQe#!i-c=mMF&Z&CXccZzU-4TBUTFn} z1J!|vKpfvo9}ORJU&!O(7i^L$>6SjcsFO`^wO!Np$=r=4()ylki&ghS_ou6D%43mzC89Sj zZ{*!M=I7tZv9q#y*W2`#v2MF>-+K^vM0hB@4t%|EMS3{jwyX52-nnA1`e&nFwU^Ch zXzO7s;=Sk<-_h)Xan192ObfLG^{L3%JM;G%ez@M82Wy+sKpL}w({U})C*^q~4Pbd&gsVF$%F?Ih@KVEh+;6vh0j`>>h&0KXBfj^Zn6{zJL z=}2Z05YqlWD=(eRHb8HtCbHgH|CODS-D4qws#{%iP1eKo*=*BfQ(^H4a~6w`l4zR0 z`@X?QV;Ht1B;uT9S6*1#b)^~Cd;Np9j`m4!M?@0f)~IPrGICT4M*LVzSEgdZ?Z}oS2o`=5qZ+_dX z9~24nI!v$5v{rl13gHPuPksCD2XKO^r^RRdbA0!oD9&#Ow)O!H03u_R_Bwxz2Y3J1 z=AYhl5OQ0|!4u{e{3^#nKW*NC0n37SFFDnV2N{k7?{j&phDJP*hKBKw z)hfma^61F<=maX$NOZ8h#E=4o*7Y3mO_4QD=(} z!tZ3{{;NCeKQX$GuC9*40Dy;w2Zsj_hl8^ffJ;b72*Ak=;O1tBNwB+k+Pj*9*zH~D z|1rpak0WF0V(x71=xXg?PxEJ7Q!@uQS1~%eKNJ1W-#?zy5@h|)ne1KuYgw=b0{+|q zaB*+~{%34hSJ6LLh2L6(ENyjVtnFaxfz1IF;ujSCOa6b|`R9!P(o^T3J-K)Vx&F25 zzufwNchz*Ube49ogU#s*{O8vESLc7d`ClDH0e_bMFRJ*5p8vWEQ#9~}DByoK4fq0g zdmUDrsrYGaF|MO$wb<1w1_r&F_v9V{#zS7`v zZVXV*t^Xt>MM9&^iFIKJLhavjRu<2sruuw0+x}^htjv2WR{nK)>3v-AXehN`!|^Gm z?fU70Q+DZqHBrpPMrkUKut^6d{z)9P8ho;tfK`$oN1ip=$>9l1@eJ@h^R1DxwZ$+=E z4YPAi9W+|3rAq2hpdXvMuIo*N9z+Um)&?Iqp+^!T*lku9Kt0+gI>gGGkDDYraop_t zefB-t@0Y)GKGFn?D}2{vD6Cq;8(B?v8JVA?p5+8n6uTeQ&KA_SKOXJzm3yjq^|Bhp zVvC-3$j}So+tzAFD>0pLab;932DvU_6HMsdD=VLP4y_#;S?4?L^{A;R7y6n>-R)ly zOptl2DQDev1O@yqub#Pay=o&kU-iMjBp#2sYe&Dm&-F-g_JbOf-D>Tg6lZt?DlZZa zHg0G$8WQZH8zERrOvjJ|5^-T#mE`I>ouw}os=xEHL55JBC4uPf0LNJ)7tRRh^bgy+ ziw^B#c9p%vdnyiIldj`}8({j01EcGvn&YN}olXDe`&HBom%F{fLM4tyi?wFf@)qvW z)~kiR&xqKhH_-ymk0<_kK0m4LB7bj085s0|DjY|TBmo{C(x!&Uf6unCSq4Lz_y6sVzMeQynwXS11C^-0?r%gKe-!Do+?O(Us z*`grxUP|ky?A8>xSG*oB06B`Ed2A#V#onkaNP*j+idVs&3 z_pxR+l5*vyPhjT`G;f(igoER?`W}GG)9dL0>4$bQ4BFVt2oDQSHM7RZ~8Q4pRG?CDu+ON6X)| z^tJ4UF=hUEF>=Dd$$(&dzc7oEC@^TSd{b)YGsFKB4X&FGAlt(Mo~Mj5q#txG?1VT` zou`}m=05Sr_3mVPA75OS`-^e)O)zk#qF-pTSeOm2>eRv%$%LT|I-VdkaGTcLV-mj| zU{gCbQ;Jsdw&SJ!#UuX)2M82ez9}ED_K`1M+2GrpL~%%t+Xf|3H_Qj|t{Sk2-ED2T zo!yck;%o>Y&S?_mf+Kb@)^gelcNOFRV4gYVd!Du$+1-Xs55Evf zpeD-ocu^>>qHXN!Mm&fk{%{%~^00;~t|0z=JL*Dyc3}K0%!Mm_KFo9OM9eZE%}5FM zGQOKgnt5E48Hn9*Nr3nBqpD29F6G&INuYPs9IY4c0tdCUQNfcl`VJZH%Wks< zNjf0Gy}Wo76gO~B*25MPDlsc=@2sv>hRh=!^{C)xHlBI;f(O^QQR++4vr9iD!XTOgy`;pQBkVu(Q{R#E6z1 zj{V`#e$xxU$9`Lbc%e2I3cSdEZ-B{abl{C1NsKUAoSpk>r&=(RSIbuO6n16;Rg%Y` z$wp>j7gb5EwlbhizaR8)^x^bACs)(E6SRfEB-Q)N1UmW5aARf4{! zd~*-iAn&%-D~!>C_9Z@9gVow@lRPqgahj^}&IGJj3h2Hpz)PLDcJF&QRU(3Eg%un%EA=TA(orDvTl+As3_aDILrBzaHuBzPKf4{sabD&G!7fHIlHK(WB z-IKvwabr$`C%$bQHU-x0$ijBqub#4wy$kYkgfJ^yw> zpF;pmpS7eyD)R@R?PlYPJwztc>(?ax6h`KoZYcu^A2dJ%EVCJt#<9e?z1>LzctbhW zH=5_?Q|Iby6;&>SAF_QTne0HLG6ZU`2WcWGm&m1gH;OnHFcxBBYaIDSlA^K;T1%ry z-^>?BERw(suqBZyV)$aw6ob9yEfs{z9W>j#l6aq~3!U@pfy~ZFNT4mSp-k3qxCY3x zY#KyX_Hqxg#b?3}$HHB)6VF?|K>*Cso4_`arYX&1Mo))U;x<7#+9aeNT!oUZ5Y`MX zT|@Cz5F*Z-@4;}};Z>@-n=xm<8X5;v`>gCXPkx@>l$Bv$Oi&y6-EQCR>F~0Z*x4tm zrQXHZC62y2+$o!r zwX>?KF|DuiZ$(zNhuB0^=AMk@BQmQeLpHLXBp}V+-HDL=8~ZoRlh0X8PV;_En^3Vz zf*FK!75|p~G<3>eO$}IszdY1buUC0w$I67;AC{Ynqm{1P9pAK2-<@o!`n@;yc9mBa zcezHl4j3^y%&LVZ8y%h>UgO2Dpv|dic%2N$w+rl!9)roM{ zvSbmPHj_17OHsXElQ=uwXyN@I51IF87D4+-aRd6S|R( z;@@uF;g4}j94Snc>7!TFrlR30bG;QCI#8l)6m`K!cBGh8AD2ws*p+(8N&`znmGH&Bft&D{b{H;g~Oy> z{qnbijbfwApAX9s#t$2ZEWLCDEAPP{(Y7l|IUU#8d?pjbw?ycSFi*)I-eLCxN=hI2 zD~ptzHBy+U(-u0LIMX&dyHNZu#`7YWxcB!5{xW?|b_X{iXra+qbZ9WDs_5O|w-=^% zmw~}i%kGQp5mp3Ds>5n{>)8S#`sDWoeuVdJ4ZR*l==^v{T`u9rQG7hES*&f|l?8?@iq>zAAn z)nBn?Hk4EsGzyv(YNYL8g& zxi1hNQ}XXXAQ@3IwR7nfjM4Th4_+p4#I_v1gC)8EfFo_H`IN&BNVKtW>YdW|$pcfs zxST|+51No1L{)kqoGRXs(GcYP>yz|D?{S-ho-%IW^F@;Nr_(@<(Z=&k>)q36cBP#x z8v`$0d0W3vKc#o5g{hVtbWD8~Hn27)JjxWTvppgCL-wf&Gx|ljpOS2QNBB9&%LRa? zwGRy~i5Q{f?JW7TYU7!2tZ<&d*CLDGgAAcDihDOM1-(LT<=)HAQP#6@(aTT3xk=G7 z?_>XX0;2;L*Pv%)60#(!Jy1%FYp%2zTz_uOq0oRko9$l56~1yNCrW6^N*x)ZMweO6 zC_mi8S4t_i6&Km;B6fi+6Ku8+G=!t zBUwTnM=Q@|Jm70pF-?rA#FbovcER2vj?RO^U2tEfVyaWF`!*ldku#jpmsPI?p$*C4 zp1f;_mSNq+@ty0!P&3rccxeiROup#lO&D^3>JX*{$mxbmh@4Y z&CiV>Y5p?Zej(4_|Z^X|d^0{43) z-BVSlfNGrgQLQ$WOYcK*a$Esa7qIEjWm}BcdVxA6zVm4M5Uo=0BAtHKK;lBR^K;6j zuToauYcKyio}u@|2M{nW^IbQeRD~sFz$xFU^GkYk<}|NXsc!2B8qVUQ%ECf$n-Uwp zAmiaH;O={;hjuj;A*&+mBXb}bJVo>Nigg#1UMrux>$C36G>pw zZY86;0LHDr4aYkS*Qe*4>u7Fd{z|`c49itLN6q}wi)|03qe*ZOywF0x0U*9UYSu?m z#OCDmQ=y={4U@2w;z;CFv>5FxFAhDJaR{z_t4ZuZ%ubQi%J`+=b^H>Zz}}i z2jw->N2&w-JhP8ESnnam0A*K)nviC0J}@&7$WR&gEK;lCv?$A^YHs7)j!3kAMBB1F ziTQa(K08%e^ct}I?G~;1;SJ^MZ=D-CDu$wTm@S}rf<$yNeZAmocM3q;h=NzdY<(-F zI-~W-69Ifh;sgf}L7EdTE+*mZ^iFuB-SoiOZ%N(n%A{;}u!;YIg zZeEz8yEVyNw{JWeobmIlotJ=b8w)VG^TsLdD8p0XP3% z%^%}PIkOFCH>AWd+jModu!4YR+=7ixx9^j9p}lO=_CcZL$$mLSlIpyjl<)@#%jz=l zlH~+P^^sO>qvGMk<^+Pqu7}4GP*98H+0Hq&c+<0WMF~>qFV5e%sUmy>^4UP;j%jSD z#<}wx&9{s8;xL@YD{JQSJ3Rj>uKe=mz_OU=*lF4=9<{6$+UI5J2k}tByL-NWU(c0x zxx0=P0K(T7#m$|Y#VMyH{)p>8x0C%#>fyDQ!&u5?Rjb^hm?$tAj`ae(gh~Zk`)(&H z7y(hzjlz*uU$I?L$iAPxhes^z*YDbJ(q{YxCFweL9%4^o!gUg_!|5EP z`sCDdGniu%1Y!NsUC{(uyM5{H>%w7~(3}2~Jy;N!yO&Q@QGV{xc1{^+&3nc0VG&72 zW8)W&Pn>*3=MDUhK(}A6D%(lCGnGZG06WXJF!C>_W@J4!!X+G3c^R@Ym=65=a-hzh+TK|*6$i~m zmJofPQ6%KP!Yn^8jxP^rgxe8?Vx0S~P<3dNmt?bwUnMmc2JQtBh-Y09;#aY7fz)Ij zzrR0pkv;8S{d9M*dz7TpA){PKIopR4P{*Ick+_`DdR8l%FEKBPv?X7D5V^_r4GL~% z3`mir>5Hl?=eoAA?;^$6M4!#kKv{tI1z;@D#*z5vhE{T7^t;X28PWhjb=-B?C$RLu z0Lih;Wc87GGf1hi3_oZQW{8pPL(t>6GpT){}B|`j_F83Rib^=GG zpyTUIDB>)2&0|6hE=Qj)&Z@PPuxCsd;$ly_Q1vr+jO!#G zH%m@;1iXz#oFIJIC|JEy@#MzmsNhblWN0=*X317i zG0#3srEUZW_T+-uHoks95|bH8)HZ57bLUAlVw4pn8s7>g6bPcGQhOlfaFVKk+^0Q$ z;UXr-B{$P27tLxrODxYbj}~I24s`*ESGFP`u*G*OP_Jx35u0uUbsx%E{B?XJKlbsU za9CxyX%SzTNtXR1Q+3x*DQ_;@yDtEZeK#TjBC>Qp%F9@) zV;hh!NYYRhBw8x)^4WQK!N(kiA2eF@Hq`^bjHNz1v~HbRA+L?tv# zFlBE)i0WC|xxuSPH+4`iR#g&NdWzKjbbM4rXvO`$RAtHN!Z#?osg&+wb{O(jIi+vm zueUPimW6!@z$1Lbd)p+!nW3exm6*{~Q+&jcqhi-f4}|pQiAH{%ZRPP7t;4qLxGKroXRVIKJE6a=nlp&0k*Ore$wkYZx)!g|LXkHk~dg?gbjF zLJ!;O&vTPQGoB3=s_8QUPc2Rv?IU#<8S!O%2Hs281tc+9N4D$EL$DIyiwrS{|J5SP z#!6(KOLVk*>qE|D5R^z*9OdphtuP2u-)nw8oH9WcoYrHwS?kE${;Je;+oj>enKC<_ zx5{l8=ZRD3v66l#hS7WVekBRH?`B-yFOSFX$Jb6vpH-Dq5HEiDR3D!Kryvw#$i^gryfEQLod#B-> zLj2}jq|VaJhV3l&Dp>qXoBR=JVF6q~2s`)-Y>H+tv@b*|YpF0=XcZ>ht50`y%EEFW zA@Okfl2ySpYQjAl*u!7Ufn9ZZqIg8U1e+?EOBm)jPJ^tEv~~=ebBFq9Ik_D7TrNwZT+VtJpw=TQ2f`q zFGxgGKYVCu-%m$4q7RkciZtF!GBb82#&+z$h{|R}_hS#hQ!z1soR_MzY^mc>t>)8R ze=<-y?G=mh!ZPPgk3mHZM4+PMSpms}m9D*%)RAkOkqjtn6nC=O7Q;dbwo_ zP&cQ}TV_7&WYQ0Gzbt=0s3M>SU%Sn;0@7h~tjRcmCd^+~t8`69`bhH5GQ4HroVe>Y zTY}};MrdR8+C5rkwySjrK2{qlCvsd^yqlq2E3=%0;BU)50>(+*u8Z}X) zAFgiFs3~NZtPYZMy89i6v2JO-h~q=|KlAaNPk1YSSC)uvQMO3%wI>*m*L9Xtc9pME zFba{eiqz&w!+onQgL)U#k3B}PY|SaC-Qfvw4z=3 zs)dc;&D|bqVk#efWa_{i1~4MI@X;i4otq4{`x5(G9CHj0mN4LPWn|pv<7P z+32)Uq}X8iJ;kMS>F%5V2t~u7+B+2(;s+r}haDTSkw!AkWzsaW@hz*mA8-EMy#F;4 zXD*W7HAB|;=T_DKh`|0qfhW)-srT@B{r+YsngBn%w+tsFy4{KY7A^k^5f8vqgOvw* ztYYyJ{zmBkj{_y&{LYQJVzvFd)xo1j>LC27!N0s7<{4F{s^sD|`&*3x@sb6d%$%DO zft`Fh{{f}{gZh)+A`uSA`%#fe|BcXZ0(jXoV8H)*12NXgLScBUgGle@-)aj_-lXkz z$0kanSV0x-d{o@uMk4d?m>2k@EJ#$|hm~bH8}6)zre_R)qdd|Gn1&%{WvPsRBlJ7` zGo7b)vZ&(U_(~rgY!@n%iwpivsEO{o!U_?iTHN0_L8SuBQ`mNXTK}C;8H}Cb5j+t8 zZ`dh+`5TP0V#0?1mVr8=@l)#G2z_UTF*yEjRndhZLzbwgbJHL4#~&0$63^=a3Ymmq zHJcZq;t!@{*V2!U5K50NM+aDaEgOYz`4U3mqI|b7+s40qox2_Zb7`6*7%0C7gP;{2 zEqGaV9-uM)!=6vDkT5cKMD26h>3X}JROtV7p4FKCv_aCY0K;&Uy{_i1Cd>UUiyzu% zKeMd1-~Mv&YAxA4FVdei_V?X%n+n#qZ_GbSFEBTFi}^gjl%=d~=%#rG!-GVbOJE2Q z+b`yON%+f~^}WI@)t;ikj~-7j5PMs+L|AnN_;A{-8G7cC7v^$js#20VF-1%=xI`Z# zI^nLj8g1;kofsx~vDkom=^}Xabbq3zCU!Z6iLB_yzillv7E0yCJpRGWX4c5tt_j9- zK+RCS3v)>%In3h+x;kZ zxdo%fc&l|@Ej+msfHb@peqCzrj%9K4PydT$e&7gb3)_lkRCyCh>8|5-WMiDR@~|UN zIIV3gc2qUQfk=n_rTq_X?+e2psAdT1*YJ){{H~i#v62Ql)cnrkA7)iLW({0ah^HJ{ zpnZdr_46OnF3ytLebyX?*z1*HFaSw2@u!6@z$#(q)x-Jd=)-;Q51-MdQ8r$}$+k`u z^1M;lBGCgXejUTMeE1TE;cj;n?WVh^j=0E-8%2dJH?KY(yTbOi@DBs!55GehW;Xi7 zp<$!)z_Zwk)#s=BSreDr`-!Bsg!216HUHG7tNQj!3ONCibf-SDu+y}|KW29aGeu+1 zH(mo-K?Jhya*Q zwVU+XR36#9(ICg^S^lG8>tB4c!+U}9fIy;_8Q7y;q>TD7=A;@&9B%q6d)`txB$vX5&(JBK!Cl*1M`53ioZI`9IFw?Ze!+=NVM5DFiUFZoTnEV4R!;Gk7e467Jg|yoyU|VQ6>88N|Kr`C=1*OsoL2KJi8>ijn;^JM-ejwCEH`WV zu}EsvEIJ_RbNV13nXgLsV5VaIX%qT|`hf<3|5f{ewv7<^#^+0_KU&49o_*tXfsIG% z*$>8snexSn2|~CQ9Rmr8mcyzcl&91(Yhk&Ro3T(}0*QmQKVB%FL0Q8_xR0*B!8PnB zZNpLAXsj|WNQZteEqAx#)l^9gUDb`i-iwPX-*7*MYn4>@p4?xyr3v#GTyx9_5ToPb zTPI-)ZGDxY4rFhCI7`90fki~U6`hVEf&mgE(0a>_?0kWL=gLKa z!1jqFFWCUUfq7wUdS41rJ#-VY`xF)Y#Ba{lMXC}1PxB$ALh!{Kz@x@B<_vNU%GFR4 z<14Kie636K`UM1pz@1`Q72)AFPJq`5Up{i`%;vRc=sfVs`k=IaWOf}Z{H;2F5O0ey zI-+{;{TY?IDXcg$95@64URCzcrzxUnksDI|k`M3I(R}M zBZ^9-w|$KuQS$`Dk+2)ug^*4_OPgZ%pL1J~_jJ&mXGvV9%j~8n@dA%};|-;iSx6xR z9bbegS}SPX{QQ@Iu@_8I{M2Jc*J`_|+jA}ugG0~}a~;F4Gt?z~cEb}g$qew_Q#!A; zjj!r^X|v@$3@b%>av?YXBL(9M*P#@2ZfFXebnKwiQ5efZsLheFvu#!W0*3RXJ%hy~#7%phm2R26aI5W5eHsgFV^Pj>(&f{Yo-|e!G|&?8_WdrAc1d?rbOq|Jew!7N-j-_a#t}0tVUsF8Y&n&vXMh=9SsD7NXqG{ zaq|`L8=N*LgbzyM?P1wF8h28)S=w7j5Jm%z+BhyMaV=W2?KXe5PfMDxM%xELVD%5_ z^gWRrW6VBV66wr|9kVIl1@;L=BOe&QiCl*__kmMx&HL$h@#bDo!Bc{0Zfm$Z`8NZ3 z3bfD%dMerCgHfTvBR%3t&`W0G`|$O5s9b%FqQ3&iArvqUSR)WuzKg6uhWo=R=?IY`DqK|o`P6;Z#wXF_^gNnKIp!}3uU;OwC(kzj-0audEAio`r9QBaZ zNa1ko>Ub7c-5QTt%vkh!rYvuRI(DORf>qbgO*yfhJ2UU0Thoj3$ku+0g*jLN6~M-7 zneZTns79LSQs%bjW%iP$oGI!LfWDlPQU^`f&Qk1B;o)?d*c1sDAAH&HxkHzW3u`$ z^l)DM-w_H_@$M+?0H1XX7bM(>XtAp~wvgaZOjOb(E<9scgs8T%Qnn?#fUzu$N6gR= z9qOheY$2qAbT7EVWRp!ehx0zeyN)M|@_;R>IyNsvUwYji)$gJsOCfCFU*oJNN{TJY z(DdPR#8`4WZUaeYLw1?Wc(Uf{EK@AM4LC`FtfZ=72R~L`+JI=LoShu0GIIrKMZD3( zBaGt872yDW?O608OA%WNt_#ZyziS(oSqsvLZVtThi6Hm|CLkaYg@z-jQKXTTgVanx zuJrrysuNLP2i#wh82dOqVdFW%sH`KqbW4e1#kgpv2}~4>iLu&SEZeUbzX^RFY5&6+ z5?c0QJX(qgCL+cd48)TXj3*o9!2p8I{?H(u-zJC$k+i6D_29118$poJ#!)&c%+F*} znMMAjN#{Zd%PNzsRM|htZ)~ zlW^utRjdb@lpnQwzg)!*0OiQ#Gv8`)%CHR5*ijlyuyK4HO<+I#!)^wC6X$(k-Nid% z%7mf`Ol54)yF zgnaDHf#1OM#lQSkx8^zOZ3HwTq{34O$H2S|8fkYn*c)Yjt`zL{9Ji-G8$*3Z4NH}; zcN?lWC6_UBfp3Wi2qt73bJeoTH2hd{!8vR_Z~eQBpyAqEZ7jg}SzB^K*x{?9ESSX) zzn#@Mb^i~dFRx6ki_k%V%xM|5TL87aw~P^%L~^v8b&1m(ts!@9u)uMd*sDbh^c* zJ~HVb<&S=Muy_dJn+~qKN;vQoj`C6!?@P|L#+0Y9K1kC0R5Y9@x@29LfwIriE-mRF zJq^uyHb0bjR@2K-8zQ+?`)4J?Bm_-4J-b+s!|W_37>=lr-lWf+dVL_gA0m;2PL6=m z#Y-Ho5=4qpjfSj@mdptarwyX>bD=cq;M&_HMRZ}T0_&o^lchJpoP$!y2+E9Eto@3? zHpNUvC&&&PfXk#+R{e>&o`o(PZ^yV16zoAQC2|d6?FV{9W{qE@~&!cfK%)VC#rJ{FoBvSW%`le~ZZohF$y42XFV2;S5sx9oOL&V_oF~-aa zrrMCA`bsS0xleZ)-6ZFW-RTBRvq8hL9{e< z03C%CYapVh>T=g9m3+r{wn|*-Oc(!>MP5y4KfLmz{Jd!Z+wnjv>^X08Fe@8+w$jd4 zqcpe2VYK}NwmW(R^Mr8~{&|M>ldDXa!?T&zcz|gx5ZTgYpVW^PFx+C#`nt$Mm`8SX z)=aa({0TPJ2YBhwi9E_eEZO1js&Cbbh7z7nH(0$2cRNI$2c2H6f~hl8dqSIVw*#+h zbf+LDQn&KB3+6Fmbp0(7^7A>AYb-A}dIGUMyEovhclOX1zP&alr`xCALuZ|t7iM(g zII?9cs1U=sM8+d4{zR}Q?M+#?|Fd|9d-}cA#*$xqTnu<-s`+E+W+oR^_K}%-oGktz zmi1N`b&x1=jRLuvpR8`Kul0f2o)sF2IW6&*SC^U2lJniEcP@8Lq~7lhhb}?QbUoFd z-`D8`?PgYH5bcg>j~CYI3Ru>%N7~288xh`1m9a--(X$cXoA%#)t|K%hAh$N}{)w!y zq07f_ksZ>OD&F&c-y0y87X0>CAh8{WxyBvfj=0Z@^id>-r|UJwj|BFG3`YAY>ILV8 zoY&p(NV(BpJy1P*KJ^u?*Su|G?1SdT7FG$y)cyPz8Sp#erW+k@J&WmXf5C=0g=}_W z1sX&$>x8x8GoN}#Gh)dmEg~VJ`q-^OZu1kv?uf&@ioVUtZWI zrWe^~bITKrwwQr;pdfC=u&_b#CuochB?4baKW_ZWgX}If>~Kb;gEysX;;iU1(+>q< zofP_1TmXWONWX`Ke%Pxi1NPMclyx@L?Sh(c_0Skv7KY%t=dADp1%ng_w64I>ClTe! zXAxbtP#y7A9@;e=`d`W|oFMwMIDn}pLAD@qc~C$vt0lTQKJR;s@(bKVI#I?)v}JYd z8>;M|pYFpOLbOH4xq_6~f^-HgZDYHy(ST0amqFZQQiudHEnrThvJI&bDGYYrHOnFB zhDG`{Et)fCjC2XJ%y}azFe@Si!rzC46N_#>5fprPx|261hLo)hWFw+Rs zz1|y!Hf)9>B~>GW7@+Day_`!)kAr71KLW>v)R{UUuyO;Ep3r(bt1Gc77D2L{R}?MZ zgBTEXif5BM{MKF<965?cEKB8NH)*3ur7oaouA`oOxRdE-ZCZ&m(p#DpZbvGv`UY4{ z6Hp;37D$hvUYQyHfvu$rPd-A9!ZhHTQca?RIvYJZL$)kux|i=Nd6$KvN3l*)a%xj7 z@CAmqrFuLNH#c1-W0%J*Nzf_1K|YH}N@$lbsv?Z1B!k73Ro%ZhA4lUfb`X zX@_tvJ%uivSNOy?og!GjS{^$}t3z@k|Df}g&Vh5`B{ACuIzkW2g4WZIT~oV9mZOhL zIMA=c0VZU(lE!#+o*Mou?(9%g$#3mOY?))9nhT0b05|2D_ zSIiB5U8|UL6DR0$MuWyx3GdO$0K)8!k$2#0Uae^(Nafm~XGQ7DtolpI8s8ptX$$Os zIEx(@@hzq4Ow{`69Y@_|Pbe_arxbsn7}P7~^DlrEDG*^Z6!62EHW(Zv(PturNsAGi z`@8ogB)nX9oFX>NvFjcMCoVat?|X_Qcf7i&)nK1_3auZ6OXdpK$n0zoju!<=JJw^f z1SkWv^+`0}7(&Ge*6cB9L*(nZkm64ptPW5ns4+q8dO>QbxYxMq(A2HiX(P6tG@~-I z{YJEC&@cVGCGvAV>UL?H}kGv zc;<_Ie$c*kT`8@VT4=CcGKAt6Aga<0hjY$Y6`P-L3Um@=xNDwJ$wZ;?q~SjfWLM;# zDe~4~{K7EGk%2Lr;h6O(^S+#O0aMg2#x3>;tO+BDaGG|?_~NL-d9*b$xqX{dBrB1 zJ41lgm}#gjSm@v`VSB0_1r>ed0^<*&8jN9-xJ~IyS)if;@1wGHs#6jl32GIc5X_9O z+k(#u{Y;JHa&VQlN!EJ)FGY!5Kc1PmeO9SgoUQOIex?4LbnYenQd4)cDy(jMoCrp; zZ+}6`s4c{O@~rlPj8&exX>2$-I{V2E)acRjVEuV%RoKI+D!_QW8?c zp$aoS%0745g|f|OU7HHGLvBR@evQ_N#4W-{_vez+sLF$hBUHX@sF0zrA{z}6(U*@O z(5GS8uca7~NWrJ6ds@fR+f>?aoraE_x~99)kK0~zMq5yQ^|}xw6{8#T?yzo*X{*|< zLKJS(v?1A_h^d~kuE-uP=W?%@ZJli@_)__*V$5r+LoD8NnC#m;2>vKMaZ`61>6&_C;Z%F>5&^AQ0t9(_Jq3JyZ@*&EYk{L0T z)_C}s(Pg%*ZKdM7rcawWwURY<;4TWEsyk^)aG+((G`Loos)+Q$mIsxT8R2sL%ukZ1(ET0niegKt>y20i%epOqE72F&#buf zG)aKv=rSM5e>1Oe5?O|j-Pn?76@JS_8sBGs%pmuzJ-1FXkV_n)%HBeyA0mdQ91{JF z&SLIsPfMBBeE3dOafp@_)h?wgr3z%g*QHA@%qLrx) z$1)sE0S$kv4;4Xzt24gKA-bL zh2AM%CIEvJmn4zVoHyK!tZlO2p`7Fvk9X_Dox4tQ4mvCdx0Ej_d{Djm_QuN@$`v*0 zLu#Awr`ENxc52lt=#}koxRu4lpjW=(%%oSlD}6@ZTwCGH$mR8KsT&CC*n{N-$zVF@ zJL#R|;X#3c)J)&=0+!M}dzZ@lgKV#vp0rR0(dq4S#}kJ*u84RwdGbChSsF_w(A zy%x|y$cl$a@V{WS;km+Nx*Tp1KYFkp<*T^FZ1WtI%}`RJVi?G1&(1XLHK*dkM8G~F zjLI~;%JT7q>eu7+k5(5gL~Fs_On2$?*W^NnP^|O{htc$lzS7o987I2+5$8BN;Z6$C zGMi=j{1$LUW{u+>Jsw_~L9rM72K=Y?j=Ie`%nQ$CJ~dAq?c_TU2d|rd&+XEdym>(6 z9R4akII*ka!(Uzi*K;$o)-mcLop&4^TB@hxzZMGE&P%t2+Gq7G0&ldO+6TOIY_Hh$ z!j=#r&1V=1^*iDv2R+a71w|*j2Rsue3y9*p8%{Ab<2#8`l#G|{G%~2Bgl7vU1*M`n4vi?r9+DPLM2^<>iU+iRgQu&Wdjk#d1jt4 zB%x#mujVs#T#Sc99(Af=@LK)}C$r@*jho2rYTkz|j;vQp5AND5IB*4rGld%pKhnH8 z=5SFbNu4yKAl{{pLXbI|H>418GwqnQ2193bX|^VYFykGNJf@;g!BFuMQ z<{6_$q*x-HHPe2>UZZ(suG6hkD|~WSIGk&%IiltHbBnpwnQ8cCYOkhGyVAQF!;FJ^JxOh8>99p~U^$#z_LT0X@Q-?!v$l>V;fA=j)j z162mskZTt2E0?Y2(S=QWLi=EYb0NY`DizYZrNjCY9vP3GT{{$GKjp@ejNb#av{UKO%>LO~T6|)h@+fLQdOt5&D3Vra;oTU==Y+3BR#m1xjPxoQZKtZoV=wk% zPO^%*oPB7PdFo#eY5O$g5TP@M1x=$@?Sf(VE{b_+9X}Iwvub=H)tY0Tpd-6HTO}Ul z%}Xgxr`;`J%I5F*yrs#sl<|Fy*q|X$`;)DUl92Ek519(97L-+jLriNbaU*3#QGA_y zo%Qq*NtWPZ0e!RE4J-{!fc>gEeP;7kHmt31pUP!IuJhb&^6uv%|Asb#vZ-LBbPW1m zn6$`fkuh^2g`<-gup_o@SdyVQQLCh z>)Nv$bG(@*QeW_im&58lpt;($RN;rledSs>_kcW8T}06xJ~B)5ltGon)>7~;sX{7B zHT*-_X3{FhEOFT8N>ky5`ekWUAy-$c@0+w{#>R^-a{l)#9b2;D*?gJzPNHq8Fsw6o z^r+ad8AS1j8#)+eHfYjX6eMLp0#bI$&enk2i(*|I9t|=J- zYv@xvJ59h!v`{l5%lLKV(&*Roz4kosTU`rh_EQ34$$XgUUj;;4jXOtFCMY)>3|lEs z*iz9Ld(}j^zdxdyz%AlKzFVEhX?K`VUs3uhA<`S6-(*SJf-Gjc$R3ZKddAX?VrK$^(KeTsKU&i$iy;q_@2y?s=+xS}&ip=QxJ`0l z^L-g5#kHIVA10n}@!|See{xgfB`zujwHf*< zf6lLPhPLa@*DbokqL0o!?J|=SB^<1uxMgk5iwtVi@;V=0n)TedLD&p%8VTxu!{XCnl>-gMuT zyMI!5P{(~zS@wX`R$@k4Z0%IxHkx+p65eS5(fq=Rxv{}$Wk1vXnAFu?WQjfJ7pFL) zsIvr4S{Uew$1SkYntX2qvNJBFKw4lOcD0XJ%{2C(4o=%W6lvyb)8Wk5$x^^=TY{o4V=t;VAQ*$&qdX7lxaGn5wqMMomw`@6R)rv;yRoTpyv<8Ma#65!fs z%@2O}RuB2XXD-+GPkr$3&Hq2XAs6MHtNaa{zk920Q#_8RaZ`Y|`j9#VSN+YGy9(so z(LRgGwj9Vp7`@?d-u6X<0Og-#r^PvY%Ktt`{sAotND%{nrCwj*k>9R-Y{+{i*ao|CJ?-sY>`^}4*!AkIj zv0MrH%|nd74W3Z0@ND#_-@J$v7fc6>K)e5^21%flZvrWIhM44LO8omP{|{pyErA0S z{2kIBC_h$A3OGA7!rjLE?ITxX-5j$*dH~qpX!$}Mr}|%u^glT8hae1$lL*Ovr{4yYWCTo``z0zj~Si;!WPj554A5EAClMN=@W+b=nO9XuU zG;#}2mp*4d0G8OjQGEai8#N=x(pGs1qQg|@8vDqd4Pam6y9U%v9b1yy?d(H0zO21M zVi^Nwyp|Sr%+9tG-vYk_q0=9p^<7KuYi)98Zby@|XJV9|;t*HSq;SW~) z(tUQ*Vt|7ko|+tQnv$4-;xTUFuKwo&cw!Je${s9Q8o^(>w&=keA4f+Wnbj zpmtJs@x{_ZFk31l=(8?O4|ZFJ?yj#*y_YK*a7gB{X}=(**r&+McdPaA3uFNE5w+bZ z*}VjemF$0h)$iNTmOI@=UF(L(I{+D_w;TcwG(T_Pn^jH-cT<|G-=_L${w)G~jAj6nBf$Ulj26WbP`8|) zO5lgoKX{nx^!f&Bv#A2m(hpzye)_7{bsWYI5r2S?)`YNT%lGjR*8<^UxT6p{)t52t zTe_IyuxJ&|rI&@=CAX!&8eyDI#sfcgL&#(Py2ahun?N97^2G327w3GFVjBiA&{ks-b9NK$h|YdAO4dy zX#?<9%MdPcMb6(>3V2QJh@Sle*!zztZ9pN5pY8A#&&T+C0f1ZK*gGH}U_d(Z;(b5K z-l6E*0iz{bzde0dgD>Ip9gK|8TL$peQCe%qJ(YaQN@-}y=J5*)5CpzmPqSlPb#GpWhcdFGnWXg zts3eU*F0hYi)^|K2+Q8tKDl2fpN#U2pA0B_xjgol--R4A9#}?P+x`hT{&;`S4jY_s zD3;m8%{~nL^}BU^6I#c&BdN+g!A8XUfZd>+SziWARTOxSw)ab1v@rHD~n#WBzAfyW^8RMaDSs_LE7Otss!ew0#E!$qhaK+l_tyI#EBPu*5Qas- z65f$|vH9t>l-~p-YUX-W-zOfzj;@IMWS6B_D%ITju^>1 z9BRM02Rvj9+QJ@VccwG%*fMf%{O*fi{GI}CdZPvK|J~p;5D$F?1ZbtNbMt?9`KaLl zk|ELS+P~Ayl|tS-9~KIhU;Z7B{l^y$Dgq&+XG|yc@0R}u5TG$TNIvrZ%~H#903oAE z?)J^!3_Q#MjCCGmvsxr7SPvpyPZJRgqB!|i)?!bSmfK|jrxF^0@63@={HK6 zXiqQfL*WMC9L>DvA~-L2uJ##q=>=4&6vV4fB)%`MlM3*FWI}zPr=J?7-90=4au(@R za6YB~Fp%p61(TuaCq!o5-DU6)JMU#8j(#m>ty=Rmc=krHHV#g1c zGBfczfRW@|UoNsYnx{a+@-Z3GSaS?eaOuoFuoFrK6d7$=YVLr~BLe~SZJQaqLXMxoxY!sS!ysIgymT-SLt>`xnq{+M!{5|C;{| z25hdtSe5Lmt)6!%#Z61G0Aa7~c{+|j^M)4Mjil@e*HF>nDuiDEv1W=0_w2WXVqcpH z>o_ffvm`kATksH+YeDYURDe=$t_Els$Zap#9+xB;2dr5!Yd~VI2LY!)hr}iwM7sV`$GnDYTKe?#YL3~gs(Im zeE^CbF%K>0OB1kAoB+-*?sEHJiPmq=)DC~?JSSy_o||F~V|ev^d)o`rx;U2ui^R_f z6hBYABo23dcVx}Lu4vmp!2jprBY$P=sdc)?NC&P6-b(@0b=lKpZChwl|q!4*jkcyBQ_pN~8BFz2|U(!0H_ySgXtwbqSpFs{2 zLj4$C#kAqYbU{A5f62=0Mlxu7UEl)=Zx1fm}3uPM<5~7P9#N3f&{ha7+R9S ziU%Ja+D;hD3@%i}l^zyuF=9`NeZMO)--2{LglDe-+PM~_Q7yS%F;wO$xDfI@2?Xpn zf~$ZUjBrKhXSuURRE?1Gq4l6zRlMy)EHJ65q1mokEs_RvC(@HkMOv{|0^Gvfz0f?Q z+2U&-%6mH(GAkg9;;m1`*Kmj~?Ve{oXdoQa2Qnhw*qz9&cCP{GzSn&vKfK?fqPC*3 zqP5}+d-Lv`4`gww zI%$8rxu{Ztj&lJvb$tr2^nTEd3={4?s4TFBZf`p|&ES%n>M=% zMK_#1mQ+!yM-J6JUf|A=*~TzG08ql$x!M^!)uWtwb|eDZM_Iw-o$Cbq&6Zz+L1bNA z#48@dLs?tH-G!*74nPYV7#SAPGrO(3#@jS;Ke*@hf!-y$bz3cU9}=+n+~qHAxp+uS zvBII9{URVgHN?%vJPJ$()KuuUAgqfK-czb=}vhV%^S<4!-?1ZKl7MGS`U0 zFLpYL1$&a%TftfDmHN@zA0CG$)h6NYH6K0Q{0|fM(ksj>PNUsB(L)a)p@wM|W(+dm zH6Bruw&Foc&-)Avr9HtU-!{{zF_y@n;TDh4?(deaMG*n`IaB zl8*HlC}%kT6H*^~1MWWqgu&G(YIEX1VyE!Vs-Y|QT{tP^q?Q8AXv$!X@~H}}Gd@Jr zjxJ4su^qUUG#OHe>t|pK?!Z+E*@75*ugxRy&sb-%b5Z=gnLwqqKzRe)1dHzPiCJxL z0+B4d%>taK+Qvv)N(*Ya5Q@RW9NJp0bi`alFq4a|Q8?0fp8fEklczY1vulcfkJDO% z0Pa8oZv4BD-iPD>q1$_*`F%tM3%MU;ND~LU8L%g646e`aN8z+ypH8;BmbfnJj|oXO zkkB^rR3@_oR6T{HPnPU)H)wcQno=&??_~%h&pm60M4~6OB=^>^K0a@6F#GKe5T7SG zfBEGJSQK4UN-eoJ?#Ti4m+QWvNpg;_P>}DkhFQvM!-vVnXOw)a(W(vXv8d1&Dt>QF z`&B5(Ga6WLVb1q7Q7yADA)ecR#bYzuswDFnN}Y+H^q&|Jz$2cxFhN&^-qWqv;Jwq2 z6tE9Cz=ZAG%xs(FmIh)!chkHv$f41a$g7dfW}xsWzJ2!IZOLn&T`HFTa5di3bh)m& ztJOj^@EjjDlj*(kPEvN46XAZ(c@bO=ZZZe*RrD*`L)Y_-)T6-c_>W*G72Y?Rx=QoJ zF4Ml4O2u?1Wpw_pm9rF%d=@ZINaAo(`V|&RslwE%$Z#C7Q*FQXA?-d*EH^up>%8v$ zaB#HzlVZ4~vA+K6RP^QP*ETdr17B0~ApGs5kYmw=h5Rk5QSwi`)GFx-T~qT`^T;+0#6X!*pQRTfg=U5qUu8HS+x6Ch zJ3m4l#8ck>3ugDc77#-);i6WHP%pfl;2M#OG&lnF!hxlHr0X(n7SXpd6$5nsSTF{G z66Wy^XaQyRdV8L4blL`&ke)^xk)xMpg@1$#N@Cp?rEH=eqmx1<2J1^3x;^Hl7xJgb zh4vIRZFR!d*VJqoGq+DwR+l1jMd#2)f5!Jg0}6rXX+nv%L&Z_7^|oD5j{uo*AF}M| zASXmDo5+@$HH?~w;ISr*u7OpNmWD&Q-F>z&FDQznpwdIO}0kPhgTO%=M!7V9UAG^AbqUFcbG4 z(L=sqSCnW{7A@yj!IFpPVzZuQHJhiqCKq%&TIB#gfdB#JJ1x0nXv7Nv?nZ~l*ckEN zIL^5XN;G7Y5tI~LE3NDijiRB5ST3|8F0Lb9&SL{%>GT1wayD>$sVuEENva=0*YtMQ z+9T7x?E+hUg81816j_lmKvTT*~=QO+rDv0o}r3X~ATMu-h$;^8&ny@g26X}~< z|K?YW{})7GVHq4Gi^OA*k)v(Lrk#(C@9kef<-k=?UD%iyu?f5G1l1!knzsi!1{0T~ zTa9QAN?=OcOh{~v8*e?fU*xItBkdKmH5{YrmCIW@e~rJx$^g8+D8j`gby9;_tHxls z1rhndQFlRfVx~s$VMwE*^>9>;2)cUfO1;CghVFw*zBqIYeIh>Qu@THFq6X1|d@)!j zFtb2x$PgGA3hPMiG@Ux9Y>t+4!6Dm5an~%Z3Og~}EJ$S`S zqH0(UnOCSQY+V}XrcQ{tbYMF@#~IA4t;uFQhd*41wyk>UF=G$|J3p0E`y}P=YNt+J z)-Hdi9+7zKUIpTj5wpN>+2vF17D9ix9d%uJ*U;5t=Ptu3(2^8MBb2TE1R*2VXVKx#)=bpRwSyX$qH~`sh9U3R1x2xjt zJu6u(&PT~Lje~n6HHnB#)Lz+jCiu%Hw-pu~NdlzhuNKdN&jYki*k6r)l}3MW{pbor z$^o^oQ*Lw;oFOjB<)#!n@5Jq+1vOf~O#^Y|?d168C-DoR<{Z-#lYeHNjuf*}>tYlP z!l@84mt>fm;$qudZ&I+nKF{N~2*TjSAOyPv#v|C4*(ZSJBX5NpFFavv1)ro;iXbig z2}tg0e=`iVi&bL^?!C@m0t+lJ=3=`Qjz+|v0oTpG{fGz>CQdhP@{o449@&QPlC*Fb zrKaU3F)lnzO*_Qo(u-}fnuAfTIuZP4YoZNbf&)Ztob3WJe6-DKg;lB{eqBglj`{GJ zd{7(pVH-6be6g>k3oe57#E2bky82IDtg=)msr;vU4NrhYPF6ht-phY23`W zDi7vqW>se(gI}-g{4ujO52xbrR3c4;^IT7lOObgr)!2`q^`OWoCtZwa49^_#v%)gt zOUGQy(Vxcd7FMTZ#m>b4!+=+v0rI;b;d`oidP^B}ht&L*;5L4{R8 z=d#`=V4iqZVz*Sd%{`G@Q!2dwj{Y87$Vg0wtKGLS<#D~>EkW&zLMpLU_diuTVuCy- zRd@36X{^4TSgb7NSAKSmrW>U-`$FgWSe9gaS83P+Rqj<7#K&mJKonI*+x;H6ao-zO zLba${89H*H)ZBR&(K0Kht1FD!9W+=N$rlgIeLv zvrBqLr~WIIXdX{3Pt zI67gIY63(Lr2})e5PZ0Hs1^NUL&ZF0bO_7^T|X zu6^UU0n94Aq?4FI#z|M<*d*-SWZtHl)Vh=tD-P*8v!3sjYXLf2{4LHm89E7lrUpmW zK#+|4jDKB7@sg9=x`}1xv+ExXEWPpOpVpzsD@V zZWHB=HeC#7+Py}^#>-XG9@4HU_gvMbI_)(!6gKq4-{gww(LP*3UixPJ&HA=6OO@y? zT)8r}$re32L_nLRk4{S=$(K|&WnDw^muZ#6m{;r52+L=sD!iRQAyB4S+W@BIp?Tu3 z&V}b?#x@=cf@nn#Y|LY5MhX$UL3vndVGSm7^Gn_D`7|NJ^kO;;qA{Mq2Aa8=J9xOg zWV|qyNyD#;gJJuFs^tPAX5*A-5Ij9ig?B1-7!r85qgH2Ivcst=%TWyt*3gv`XsV(--EhHT5 z;3kIKlIkUAy}H3crFl15Bt)=-#ZUT2m6dke*3!6mD;pD_|GH!p4o?@5Ps-60%w z`?c+Wcq_oNn6cFs60RziytV_A@X4d)oC;~K3)7sTyne_k6U&?gzVi``N7#D;+i(Ea|(4uLgZJKr}xYP38 z@S{|x<84-kAWICFX4&?ws}D>Jo#~lHTg8*a5*Fruy7`-UIf5)h!ZwT{(s5$u^8)p{ z&h%LIJLr&ZOc4u`c}tf&KvqkFYZ%LWc z@pP=(e(pxR%=07B5XEUvCMaCTNpV>BXe}Jp`IHFLN;A6o;IiIU)rRVgAk1}_>AV+S z>b$(4(#>@95qU74(Z0$OZ%HO3)9BY!)21ssQCHpJ1-5vOc64njR)3wH3Y zm)A%PBnqqY?=N^PiTfdKW&IeUsA?Y!RXr;?(tT|q$$-s$?6oX<>ICgLR-fP_ZVsJq z|JZ93lMAnaXX_LW!|HuX*K%_H>?bla!fRgkl{gAMU0DkUx`0 zuemwhg9GTYV9K*Zp!;4}o0JDpP#R)RzGdpUFDnJ)zT7GMsXL}m*SD~DwawE8 z?X++1|5YqBuUp%Zq#Y}al)of^(l3xlxL9nRc1J~c{^=Xn?Te#3x9r-I^Qd1s-jzyZ zg`<<nembiyPc+^Y%Oi_RD5})B(!}GG!K|Ybf^9`se$F1aGyx+C$#=7B&@1k8aqS z;W)*mvLzP^>FSC{jgqze~;ppJs8Ex zLa;t~?v=#y=@ic0^JH9lI;kqWr+l@Is^X-%Q)}5?YQ(`fYH28c^wa7>S(}H}R>6fZ zzfN@?Z9}Ld6aS3&>)N;ETlbkv_t#{Y(Unhhqk7#?Cl6ZR`u5r#8_x>IR>nt{9?XgM zT2ZX==!r1}EDBOGF}6(xwUr;f#J?bQ{AnwQx6tV~3P9)f89_}^^uSgpD5!U~~R0d^J%5%ZU zk#pHCjSkl~e}Tr8RlK6(f$o?=e#UqXW$r;A9q$b`mV&@=Hk?}tHwJw+2JsY9!Y{ok zc#@}oe)(=f?5^pJBDUw>d%e_?D|ZX^Ip;>lh0c_zF*OPkNv7P z^Qh@GJku{`&@_|#3aWGb*CiKsQ{%=i1H>}fvDZgn6y-+2-Bfm!5GlGBKX;AKaj5Cc z4uvA15Q|=c&21UjZd{GtT9a;lRbC-(TJtvZuXB!+^gL>WV9?#k{H_h}AHVEp);3>O z&x#{5NR)mjqNzc3-jpHPx?ohFv>#=aMU78rX!@YgSD}H{)4wsts?MSxol74q>S9@Z z*_7865CdrODl0Bk$kvgmu@_|MSD4-+Qn9-Jy926i0?P|ZuvV%ctz8-dIasG&Q<=#B zwR#+ck(bxrCg-z>o)w>RRq_>fmW|*YU9=HGWv6NVjGN!*YvbxQRYR2!SxAa^Ne!Cn zIS%$oq6VnoJv&X0qpFuOqyySF{`#K(Sym6ddOl{J@xce!Cs0~xC;=t9T{R5wZn_LP zu~aP`0CPDerht>J5|CE;Ni@_=Vfrv+=>WJ|GCPBs-+|bnwVgJ4_u-si^#A@9oH@Qd z0&2aNQ8bjbaq=9~3}*zW2^*0vQ-!UM>SuBq^S(F2)o5~)$p=wuVn2Q9TNwkh`c zE7cSH%3D+fpj|5{jgmSP*FF`g*Z|QY2B0=XZ_^&%N&>lFtk0g8RsG=ZI>1+W)t}9xLNemSh-GNJW5`v)k*_{>Zy1 z!Z&@3Me>(es{d1Ll3h;3tzaaBq|kSD)5rpM?O3_90}B??`S}+Jkr$rMhN(hOu}~@E zTyO-0b^)+VQiPqG!d7#W9ALOV>3Y9s!qO)NwJ1W~C^L}j#w7;oMp?sw61e@@DGsO@3sYzID zgUY@TZ}t2u447slwrOrE-3Ne1WF}icWvaiFHv=E;=vMElR7}I2Q3|P;&&zZ(cp|ST zHnzq0|Fv9$UQC{?#U^b04|58D1{w70ZmpI9~Ei1y1*k#^Smh<;+6=a5-Y5T9YV?e6!1tt*w=%~3$GlDN*ldBE;m zB>gQA*RqP0#=2>!I z`=QyGH$MN!le3a$@~C|b{xWdyyno_9tfSg0fKb?|7`lzYgfYh+E?gPR+=jYt@_?3x zVgm<+aV?3*PQa+Z0#s(hHv%wH(apdz@Q!9Du-~GV1GZs@h7lsaRI{N)e&j)nqz z$s1F?J3SV_&MziT1g0qm*v=#XAH~gslChzh<6Xl$vJ7DsZ|)KgQ;*#h3uzUIy@LeY z;OhfF@RlTw7=>i0!5{@^2Er2FD@R&xtm+%j=K)jS&s~%3^@7g|{@^)fcLm0ktweyp z`&3vGa?BGUFk!~!ejU!KGJ zU^^0go9#|MNv`TOMvkA#_SdvlaCeRr3=eL1mfOFS7r)rAHWTyaValIKi;3bnrLpWV zrYmF?P*+MErf0X7C>wmzP7ehF*`c16qHd5EYmOT@MB2zT_&(hiQE>)L$JZU-#**r; z8(;89p<*oQCk4d|TLItC9P|<0duzEq+yK(})P!eB| z!RQ7&)vMYHPg&sG&N0;E>m>b`Y8oI^)J0zP2{e;}-QjkDw~f`etK@q!XfCacICJZ1 zi!jCBkfOMd*U9A=vyC?6!#+p&J#M)- z0eA(!h(NziK%)kAT;nJ)N8XVg#^2-3gpLB?Mt4E58uYu6>um2x0Q2VSTRCo(@y}18M-gOwyvHQ(%QZUARo4j8h}{jdP_SZe@hfiUQlnA8meIHh)RM3oM4&yhOBm?FnI*OLBKm~82y!33ebL;t z?M^JKYjN#flZ&I`p=(O*;I-+nCPzEWyi=N1pR6?huEKRMdJ#5A z#qs#Hfn~&j%rAaAlznOe7CNAPImSY{h>st-7XwCJhN%YZa)aD*$7+E~@w#{j6(%=O zbo`RJ$F7yPkh?HGpZtvuW2gXyEfOzVb|TGTvw%pME7vl5v6qG9JQa5Y8*Zr6fTuB< zez|E__lSL>1_DL%{18_Ce9gV++zho#!E5Z5IaR6 z2HrY4slUXzB>0$v^|XV>EBiT3Vclk$RMK{eDLis3U3`Ka!T462I(%MmOV~W zNd2URU53t^|I1oggV{xtbzB2=pFv0j4yt<#&0LUJ`u3P~(YE~OM=mYIGxicYJuZzE z9vn~7F~s!3=W(^;%*56X8cEG8f$CSCab-(hS-f#UGH=#_Iq$9L8v&VR2VvOihpRY! zR14@zMa_&YREbpJX~!?%{(3pb$|&KF2;*K$-esG#j3FqwEmLGM7+H2dl;iyBct7u` zbC0g@&(fDSG&7s^EZ=DbD;vs4%Yxi=F3p z^5>)zw7hxr;8c@o(7vHP;y!17^1~DXu)(2A5FYHXS||ot8R0&(X%m#+<8O@!O6N1B zjD9=C+zj6az%N=mww&MNmGuewj;aB{8`$?_whrLsxous+UiYvo*oQ-Vyl$%AWOSD! z;84Ue@Ah#IXRyeUpzGb+B^r}C^Tk${*E)3dO)hilj@hQk>B*E>*9th6;Q!?Xpf{MY zb%<>V)|XNt?qpI@O}Ve`5xJ@SQ-T>eThG$(Xp)s?CvM=>R4K7q^u>9~@?0~8jTz5} zs&B0xnlDP;Pzix1AG(tTKyJYO`d*Eg*2Ax8>_)aigSVJSr(3~%PV$jKKFQE1hO&Xi zY&XJ@Y!5;!Na5;oc48+mDg+38>>>kIw(|w({-Q{O>L|@!l%w@vI$no&W&|b^4^R)y ziN4vyoM$M0wD(ha><1qEC7TwKJV+|gFv8+!BQHVP4@(%oeU*5$Y~g%>Dzo1ckI5HL z>h`FSCUAGJ>?@N-=7DVjXg|S{oi>@x67(0J;Q}3tj%$h0Gqo)VdROPMT5gu#(RPul z2Ov8I!?wWMzQrGmr81}=o9(1KBvYD45y-x*^S;38;!S!2GlYPMhKx}NFv0q-3vRy( zu)q7PR%QJ9)%CX~kJa@va*9gic;cvr6}Efwa28&}N>B_G$0{}o>BhfW5lYAZEC&Sd%L`M2)7g;DyrF&olh!PCI7$#oaMz{2f95T z&!qs&9MQ9uZ_yT!vxsC)T*=V&2kbOHfx>~!2OUp|K0RyWd{rd95PvU6;nh(f6%&Q4 zX+{!Wu0#{b2ggSn*2@IEjh@dTq4MI<=tBv z?>mX6Q}~?J?<;BvRNVHQ!^Q!yPUl_%0yVUDJ@F}5@rl^ znxf;pwY_xEdus5pVW;6LMaF{TsvwX#d?ole681pp7qvosr)xVeHF(( z*-m_;Hf>vIN|swXS{t|axx&PSH_4~)RGZsU14>a(^q)*TV3(fhgtZIiyOg%l3`j?< zGY|n$%YGGv`@4TI)JyXOCLR!S=Oc|kzr%xkmHI2AnE5CT7Tj3d{lwz&ni*N zPMLSo_Q=DBKkSpvr~?nX`0E+;eTHRIYa`OdA`?CG_c?bLA2gHu;f$1Lo1v#)H4B*} z`G@b=Na@t^J8<}Zb`?y&_?;xkTgt4qs-ODG@)ZeJyQ{^j{r~Jtp_FlMQXJfAWM#G^ z+I+VQH{6<-|4Sl3-UwWtU3mOm{&mLv@i&3{#NQ&iN!YheGQf!y%)=D=NIjMR-&k~z zGiU#G_GKCWI(Cr7 zc<^fHHLL@=?Y_P`qV|sm`rny)bHqiMjQ0}_p&w6q_!=MO{o_CV_aFPoz)|k0#GS12 zUnatTl~C9bgM0Y@r~i>kCC|%TRU~<)-=1NyiyXjKLrT$um9q}6rvI@#{xQT>ly*^{ z=x`6gVS6cNnq4KDsP^!9vzek92-Rd&R6-|$phN`)7WoMu4kqRF}rg!GubNtX?`SlN=~jweTHpnmnS?Nf|D3V_BQ*u@O! z7H4sZIpD+%vvs1dn+n)K;eY~cJqS`)&z}~mbs{J;%7SfQ-oKu(2@+AmHYQKV+B!Sm z&WPHy(Z31y{{@mAHR-9_LojA@K$#Jc=^GN-0VkQ!^ zwHp}72|&DPerJ!V=i&j7FHAc2=Qm$oZaV$AKV8r#M%Ep8k&CsThFy|^mkDzc1L}sg zN7$6?4-&%tzktw_4#JL0Hk(MoAc-frfJSGD)osCoX^(57t%4%LuDN#PUTL3K!Y4= z>EY8d9U%vVQ_oAlCl1xsGrY!I(tv{jaPb9A7$-Nj@tZx+K9WGnH0w&B&j>xaN@P>g z&K4B{+QYHmiG>s#k(EDebGI=~-Zn=3gJn<5qoJF=Wqtj(XC$G3Eh{MM741KKQwI_d zOcvlWO$5TOfWi*dHb_Zp1)kwuSx9R6(|0`)qC!_pAX@Un4kTO`C5mH~JM7SI0%;ps zY;JT?h27_2o@R*t{s3;1rMI&+wog{7`hN^Qnm+BCNIWs2qu5HztB16>rHt)iTVL>2 zs7f-vvSYUzIb468?yH_Yg~G;~t2Hh!Ax7luUhGqfNy<};aclpNH>MnG<-m2_XNrVz zKU%n#XiT+6J(F+lyCCsyvV4BDe&<=)d5vqrkM>vieF|<}qqn;+gllb|lQBKz;gX?1Xmr{84guxPgrC zpgL!3UA(M4pwWw&mPk&yPai#G_y}_LxH1!G-Bqvv+u+x?#3&;yAdXtE_!sam*XiJJ z=1CT;e+*QtE!gkctHt&H3qYUvF%;Vco}a!UV5ofXmf_A^42@ZJ{PHt+V4HUeWYLHh1-8jrH|FV=S>-ZJ-m} z6_3R3Dl>)pJQM|x{pJ*0Kka}>)5%UC@-~Rs2e;v3-tumt1V|k4lEho};$Q`E%GDFH zdWt_Cxt$%WqcVQ0vlG;hFRs#JptM_iav63CWKGRFNHN@nx5SpIewKB!Fniek&_C9^ zk4rI_SSEBc3=V{mx#a3+UW-ung#2zNs>l4I-EiIfkYa6?6GpO!BYsZ!)^XF$N42cz zz@HzYW0wfx&K*(gJ$JN}-dq3va@VcG;nOoM9KUGyvAKQ?`yHXS|HIyUhDEidUArJi zkOCwqMHa~dl5-ZxSrkEn?e2cNzwUlH zf6uwD^N(w9Etae1TJxFD7~_7L#xl2Ez`9Ct%MiXHSLf@VH4h-O-pE6ggr8HG1SQE@ zgRy+(XJ?yqW~bnC$vf5GV{4;jFQIm1&K;Q@F=hKL_GY}lT6Z~e;h}~GP%ebOL09Hh z-T2n~#?uw>YI?i0HPY+e4O-it=#g}ogx~C{N16VBhu<-u>jmb#3d3v&c4S>Q8d1y6 zCg?N}P9OKW%oYCzuk5G$6K!EP)N2+vp4Qzt-2pj`K&XE}<+>0(3!T;B!mrG9z;q?k z^ zOSveYHr(S@nG&lHG#mUqd_xb_*wyywDMxA!P=KJCE2{;foUOIhYy|)x|oHUqu>75eIVicP}my=AqbK8P5=3;?tlk*~`Oo@P+ zaRqBdC#uL+aeA_re1W>xG6FjINpxaZIUDAv-6Qd#`J8mw(NAya(wCOi6#I$5 z0G9!-frlirkzA|hgOpl{O&9EJEfDIuMB!mD!=>1-72PhT-n*Q&=S823e`emFF>ad% z8RkblH$*t590Fv#Wg@ji!lZQF^-><89>%JPSbL~h0~)}(ecLC6i=_8;sA*y!pZ#vP zelTzjtduWtv^7Mc^d4X@iOg93B4!Csi*vz`2# z&o=rm?WC+V&GamP)&k8hI>{Gz2V?>RyOmjk!mwXm!ry&Lbpw6hc4~abtsWn&CV0;b z3$Fj~+?n#qM0EUc5&t@9PBF!Y(F4qm+Oll)Jfw6d^LRd3rrvM;l|-_;U))0_xcUZm zuicxhKM64os%r-ubXSxs1`3_r+Q=cOF|@LRD&z>#0vw>0_G@$3?6VOQpEcd7z8H@Y zlS=0MI)6YgoSO#A<(>6nleVgTveaAjGl*!Asf_CZvNqwd`L;bx4D>-yV84s(TYLhE zqD08x6eZix+2Hnd@mTui?zmkkHIPXqm4(dc%PWd`vha8 z^vDeidLAsn_);ELaWLNtpwOASdjtkX24TlA`UXMa>`FOEu-8+kF;6LVafODy3uDi8JDdQqB&YM_u{HaJKW8 z?+B_B=~Q7{+{M{1GA~ILy?4Sk;$3~7=KQXwv@Me^-u&TzdFnTnfK@=HB?npO>WFNnl>ty;j z_h)1Xgc(-zmsQ35g<_3hmLKjoDLajUp_37%_K!dBEA2MXO*tCrO;HQ zh-z%u$Ul}cZN%G&wIIruZ=Lb>_+h&WpO-7RKV8H&q7lJ`rMBTcXhe-x;mv($KB(w( zF-Q&<8p%g*>Eghj+1f#^iQ#8<#`qK%@Dx@spFta#1IY5CP{R1cKvbHuQn8dO|M~32 z_vJ08ISE7-FTAT-BLuZJ=FW`(9l8}8Lc#NtBuyj=X2ng)OgD)mcRQB|qxo_MTk@^( zwi$CN0#27p+rsZ2u@u;WiyU^a)QIfIxR4Bpsi9`|Q>!DvYZLCI4 z2iG&9yz=QRjODGd;Nso_JW2 zm>9xXzgY`Xc`i_IRWp2isW$CkmWbxtxUnrPyFCg#VsqT7nEnha(jy;)?H!msb3^#< zs1Rp3k6F`=GtqBIr`w_5Waj7YJO?D0w;AW58>@D}a+SxhsP2)|H%GZL&Fs2o3~>tH%NRrKHYj>~J@E77BllR$W1&uM zYmC&4Q@rL>%qJYvP<`NTzpLl&Icg7^d3XH;7Ar1gdr1~Yb7(LukF7x~1l z9lRGNA72dauOGE$q4^79UDu}Kky3wft`v(ltfeiTIo=fNLa|kxAIG-*PDqSvG~bEF zcdC+8s9obyw<`2C{_j+$wcf5I)>Q_upw#Y#&!*kp8+&H?z$@WnSxd;xEXh_)`jwt4 zh8J&Ur-t38;r$|kM00lAks8%XQi<$l9|$mISUD+j#amxTB~T3zGRYXj-kk zRo@BGj0t(g)Edh-7u~!~tj!e5?Z-@5yLe%<|)i0rj61#<|z?AopRHz zzfZZZD-Q#2S7HGAyHYdVm zz(ynAXN@&B!RA``6ytJJncRP@tmmTFke%X4W*M_Q;iK#(L;Nz2weNx`n2d%0plgSe zv-`Yd2v_}p0r9n9)YKw)JGqYPh#of z&py4vjdZDvi;XIIIYQur3quxH$!%C5jLQ5l#@`JS>?P_KaJGpyI2*MYA$@gDa!TzU zON6`_U>;x01#3YsmBD<)sGMGP%hbVqjhN~6$l5oBljP^{Ph*+n(t<4-5gP$CjiS0S zQ@;1xzd}QMeiY*LJ;X#AX$vzLzY3}8=oE5#NJW)6+(iG8@v7B?k-3ykIhj(b2kTVoW z#<%Bhdc5)y!ZGCjquY`B*7s&Oi=LB_RGy<*vk!pj7Mot$z4NMVB3J0&;0STVngW&PO>9P_qOua`#u{C_({;@go1S2Xw)TV+9`!(?er?qj5@ER{4Ghx19iAGo^owW%XE-6#*^Q@aTAc zSoU@zkTZNs?lvk>Ys(L`8F`p#NKtlmlE3rR9_*jG3zxV##n$xj?mF-EN=hi^ybaH* z+i88Y^X5Lag|YRq2dR7g)3eh4M$7fwyKZeal|+$b_)9ULZ9>BilBz}RxPuZnLRaA)3y6vbsXX#_J>t(YF=8h?>M{3 zpjP*wRP#93q3)br8T*aT?l3m8!dCaerD3H>iR7{D=c37rJVObth}}7RD%v9uhHo(u zf0*!q?4h?Z$^N@r#vzpcJjmmDVN>D^%Sf`4c}(A_ruqd~yKcu%>*nfvmgepSyKNPH z@f3d|pM8@x%yG)W+r5CV-d)GqBrFtM8DP2J_c=l;=YY}M5``&-_l{LW#FWm-diC?a zRxd)Z%!>TOm;u^(c;Xa>R?6)^_bXk`KFLy#HxJU@7QX-Uub0aN**L+Ww6~E7MI9xz zxT4^41?xvIB_k;uGmLnp(dP$ck9o|io_Z(oJiC{aZP+2EzZ%qmjCs@NXeYo@=w3gr zZswn(w5a*C1MBfN=dMkOS-SNPTXRm<_hwy#^IvTn3Y7$`!j{UL3>bHZ-y0^7e_%_R z6;1D|&Ayl9g5W2pH$#mmStOPO^&NzMG*hyI#k%f19+IL;+7_YdgSB#dSmZ`FytveS z%445zle$1hnfips-xZczhsdU|ulD!?jTwBbKp8{+cK*$R_-?>^Ga9LFp-UdqhhhnJ zQTJ#*v}Fopj5dCLP#@EHXOfe8p2T_dVn%$hN4!GX6Bf_9@I*V8(>>aHf2UlmDPq^P zL!{N_*xWrsn}-vHNaiyO3b3Il_Y`1DI&`~u)Bt%dmX)sUaU^|yRdrrq-;ONTJZvYu z9AR4IjVDK$NJgDB{V^blQ}OM?F7oPQax$$gx_qDBUT-G1p>!~}c$RhJoYWKDaVUK# zV}Ws3swDsQ`Tnm+PxA|pRm+{ju4@0cu9=&i)#q=UR!qOjTa0u(pK_dDDIi|jQ=Cr9 zMaoy7H^F~pSKJv5Xo4@jl>}<0CYhJOy$mN!&C&eV$9-}unHfEXhR;;l%+pnC7IF%` z>}93BG89iSQ^KE?f82O){orZW0p-W^KJO0X$q$F3vm=#0yc2p?r6rb(T$hD; z>$byeo^I`g$Nk}t+$z>h?j`w3GI5UkF-Z)ueq0EwzNSbu@argoK!r+dUTkyaE7JG; z7dK|-Sc1o0*0#sYN4XJPFBRs)JaihAGvh&5B>ru~xGWl_(%?6nb?Re2qWjZ}16U@e zMFkn8E6qJWPrF%5+v*g^P6%pn2Ehq1J!3kU(06LAh}`nsXuWVSaz;6+ih48otiIRW zct}{xpfi9m6R|QQY`}oct@yrnkMAx`)~}4$-CaLHh^w=Snxph zV_b1MijQftga@nYm%wR?`wVWoGoP&cL=2}`g-&&9Lb=%Qmb}e?whteEibn6|q7%N) zJ}vlr4HS-Pb-GNsWum&ya(cgF{RZ0$8%8ZN6H(ot83XoZ{CJ!Xda;PAG>6Z7-cz6a zx+@;w3@=TtDGiIWwxH5>%Wd-FcimN*apw(NG|vX?To z`c2jmf0%pqPM^wpSp5mHWX-Kdxh{K*_1e$W9 z6alD0oiz^A_i>DtjBlBgzZk~*^=I3yW=m@LG%L4HplirsQU-0ZPO$k#xLggF^FRb& zOa-=U9fx+1e{`+jbvvEEX)aQqiAMk=Z|=5U(r3$zipHND?D)V9s&z`$CS}-OP5tM* z+nM7}+{b@a*)pGCKFh-U&{zGw66unO17o9iCFcuYFn1FBJE(7&<{Bcb;@*2nnw*9B zx~_s`oA(jzLN_SgMV;sB`>4^|AO6e7!B~8Ke+K@r#I8gji?0Jk=#tPT=i{?yjZ-XF z$V{0%n9s}^Hop?>-!YFlmmO^&F>d%i_4btiFeu*ms=tSn(nBuPj@W>i>5-hQf5M|v z*AUU_Zrs|ssDE7kzZ38NCXQJ>%;b=JMI4yr4t!-V-bwWR-Lm!X0@&Z=t7mdX#qr;5 zTmMCn`Hvrc37CPH*@D-;@W0!Mqomewfci?UXZkNAnE(9(|6ln*T;;b`+yN~^kLaW0 zRYYelMlPG-!uW++M!6kD=Cn%25dKG3=z;KbOu0y!$r$9fNhRenilYDVNc{UvLC1B% z0Ma3_TjXDbD|_}pir)4A|DgYedt_v$Y+`qJkZh_X0Q-4yK%|7--K|SMWJaFz1-V?n z3KRLjnQYJ%FgJAEVIah*zk)50P3*fS8m0EHHSBJliPi3@<*7LbZOjTzA?P8HmW-f3 z{wo2uz2&W)F3w=!yvmht2Ur(IXiTSg#I19mN#Oa@cCu-Vh^_`d33!}Xb6R*|{n-$w zcLSP73#53ZW2YSLpY8SqLlo?gsOOl+kn9OOXZzpL8=yz6PzEyM-xu2jl0h(Vkkqw( z1u`h72WTxqzaz7!4wwi-)qaS}r99z$8PRaOm=gHQ3%oG`+jg-ffpgyh9HkZs-<-Le zXpZgNWu2Esbe0&>Y7{+g25RA>gf|1{LfYCA`z5TRQ!Lbb(|`pgtxNjj^~ZL5{+$W3 z&K%vkK+72*5A_&K?y~Jy8y+9PM#&qD^wkq=-u>!r_bsxDm-6o11rBRM;8|0v_{=lG z#nfL&$RaaTNXY)}=WF6+PG25_F9vURft+%Kvv%u2WO0^hzsup1eq$(yQE#~Mh8V7K z&5k=p^5_90J5Sf!vWa5Tz#ogY7xrIAQ8&W6#qLgI7)U-ghxEECZvcgTiU3TP7J$Y0 z!8Hym>;=d5n%R2bEDqxTrC+giX7l0W?RY@DY<|5i^7av!UVp~a4gL?e9q^ypk3hs$ z42gI1kXLd72&nk6E?Z65!Pqp2Oye&eJ|KNiCa>GL2>Oo%@IPS-z?h|e1Y|H8LtymJ z9ukiL-Dht+&@Gd6r&5-rh%>(NbWf22cx3ASQX)b7nun)Lz2x0w!cob$jhgmvEE{8{ z>SEguM?h!21yCYyLJoJG(h>lm9W#34t)-sAk2W0MNqnhB{#}sec45F|(pcF%_)CxFt{;3kV6pvQ)!3FT z1V||aLhRGx%0VUZvHN2WPO!0f+CfW`7GjMB169*!u{%!Pzq`(_NX;Xy$i;_X-z? zpXxfE`#((6`UrWF&W<|Ny@(eEzgi#0*KF=!C7k2%<0p$uOI3fj;Aqi zPP<I9Uo|DKs>Rj-awDDS4{+1mMfm} z5LPNJe%C(F@hER{270f6ip|eV?YBAUW*ChKiQU^9L=3#mZS$jO@7uPu&)SjQs`4r1 zF{U*e4VM=PRRN#(#b4$^Ar0*qZ z=27GyC|9hEf~R8*HB<2m+nr}kmt$E7)s zg_P#SxbC+h4yU~@l>}x=%rKHVp}36eBv6X-e&xrOF_wZ@7*{2(*%aL;yG&Xc9#grq zbKvLF=JjeNl79>uI%^TY`B3Q6|FeH_sGd}D@hkCsC-Uu&q0!9GF|4b%PQC$UvCb`( zv(moOKgqE&Ml}`nLhCYyoNRux+lY2tquJty`@Y-Qpsh)14EZh_5Es8?@&k6iK~$?4l|sIHFR3X%MzvV} z{<46g!VpMvQJntlmZItk7Lg-2??sOT?!@+_y8K>MVwukohU<8<>r98aA63qUK5 z=(}EXzfUFFEJ7d-Y#tKnCbqId>0u1R!ebm`KHWN6v&YOExq<}_Ue3(E!;#zqS>d&S z3}uoMX6W|^#4h5x8x5#mYsp)=L_67ur`^FinU-ABO%Ix0Ba+I@J<<9ZOWrk^&iAyg z;p6W8R0}tVyr}yVnF2Is4~^nW1CLlxPw_!vupbT4^lcL&>ADVL>xt3hART}){^<-r zE^h!fnQXVPpV4x~8>yBGQ62X7pj!Ic#3o3Vyln#+koGTP z9)Y*C-zry+t_JYs4_SW8$0qC=h6a4VNA(y8-0=NE{MVVoVp6gu^MSQ2q}c%YVy znPwgFiEh{+U<={MLLAlD&*<|D*N;q?2Y?^3$9~ox!qOL#jPIJcDNd|1@J7gT#(~-O zU9;d8x=pT&Sv`dq=>d=sjp}Jo*~SGRpC@-BY8|NT7rTv!m)EvR&U;4D;eQ~B zKIs*0UADChZm>N0N&)_NwuJg5zP!qzzHX4Lp`$!nu)nqSun&uS7Nq%B9czcY`VWTg zM*=^NF|ekGihPImk{~f0wBYVSYTag%sg-9Zk5AsW272MLjuYo<9y@r}mH*a{=2OCc zLws%#qsWAd;GWe~YtUgh@8Hz?y72k6@vR?#tVz;OZF8L(!=Cr#C$MocAIukvb-`x2 zS>ABEKHaVD?|bA7d}D8c@Qkk5kH1}+7ikfQu^(f){;r&70>maB2so1LlkN1qPD`&U zuVqTZJC70xj|e8C5xMKe^M?Do%t*%hAIoi_8U?}PBP&bSs?{m;-yPyz>Cup~6mcB{ z7~@BrXPXp`c#N;#`v9!4kdSJlQA;>EKL)+-<@2`;YQTTG6gw7F%A|5PJ;1Kfd!Z@V?Nm+UU1DSw-EMgpMqGDTWJ{pm zKtNqP`EuE`pg$FszY`Y@77i(KfK#+ud>CZH>Vs|}Fm?iBkdvnYHsKf~*!G5uN6 z(i18b$NFfahbW^62g&%Ybzm2AlnNsXtQ`E5w+VLukj_f0g{5>HC6TT(XyLTEw*i6n z>Eobx>H=L9`9OBGIouqDD8x> zw-2n&C-d~N?Kab+ z6})K6at?8>KNagZk5K-*u$P0a*ymg*oa(32%kwL1gRk_Nf|7Jp4?}%zF8J@!`u$=X zPcD2fNtuT+Php*1D(5Zbl5Kw)gdTv5N*h0&^kodozjNofbtH35U z7UeqK<}0RlM`en^GAh^G$gENL&TcX1a21J+q@GM>vDjDBQQl@DpWQutB!0j9ir^|P zN)t98APKt8@7IZESG*Ag*X__y>D^)66i>M4SSHndykPze>ohgj*3GN0(W$jYiy{|P z-WS;2v!;*bTgA}&vS%%>%y7njw-G_pRMIR^Zu=(qmwK+h zSg|}y|416@>AG*Ufp>li!g3@d&e3+E%fYeT;5?wYq~@kBvQdDEmC~4aztcbM+NK~D zI_W6`V%&J72d}~jW=4dS-(54i9z;f3F^eM?;?_Jv(?>yJL19IzVzJ9bV8kZ}FU;iQ z(PC!i{11{ zex(NwgBnW!ei!aQ4{~s7e6N^G4?PfA*{w~q_?(7|aBm_CTQNXkK^;XnOaqIdh#FTb z2LI*O1JgT!bNx}AU z(Tl%T_nk{g%L^7DtS0>Xda1-Jwm$cLP1-FUQ(6P0-0>+7x{Mh7g!Or1Q&^J)eb(y| zeicW2bG?IWUr0Go^Z|R(hR}mBhH3S@eEo5Dv|q==lc)u8f8SIA^i3&DdI5~5V_ah- zW5Qy_VFAtw&IRO(LGojx3~&ZTLO(&}dIS2uDe8qhD;@Ruhyk;%J=4p!hxF%UcA~La ztX^ZDV;e7Tp1eRNKz=cHyupdL@)FO3sa`4U!J7R)YjY`-t-h@nR4wL7>NH}*jco|; zWDevwDK4;IXt|x|zq_ot%%DPC7L!^1ytgg7{!pM{rN%;n+ehp6UgX|*8`W!-+Ty9u z&!o&)5@NN>L$O>%>a<<&1NzqhW#T3tb^&yn}tu)!(T@D-~3>)f}z8xsXk~!*E^)0E08>m zEI?FqW%5pC)miH2!SJsYH#U8Us}~;cq*J>+=rRurGL*jT(}~4(U>Zv7&Lzy`TvMgR z;Q&FgLf(&^vV+(w;}v$T*}HVkc&NGMx+M*w3?g^D<)>@G%@ILF{* z(r^H}SeY_cNE|~rhEPHzZ`z^OHGf%vPH4J28Fsprt%RI9-z1vxmgF*vmB;$Qu8R7Sf#p#(tCiE^SX-_c6gxxhb5~!~ zH!7aBz@aK*R3aN)HWeL&jiVS8!(C4dPCI-^6{nUu+#3Y+N9{?)EVN@C(Od8v$hUso z@Ru+g{N9R9%@xGT#2FLKOe^8K9(~Dh{7SVgR06jRty-JqH5QxglT4`N>VxJNSX*b2bm_aSfT5DSb)!!_Ye3AHjqP6RNF@p*$wqA`467YR*kbG05*jAX`!fM9x zB@*M%QeWay*tyH3?f+?z6_=z&QE-N&G^_Qi;c@N@;E-IQ1j@ZWY;Kve=>JykrOEsW zVo$`dN9#8~Jy+eY6cB6B)kN1ykM$6W`1Fdy_2s7+!pV}@z;)w(V=fcptv*RDbJySc z6scN-yXlQ$VIAmb!Fa0}K^n-FeX(PaqyYe`tE(ATp&_T_@-oWCux~vR$~$mdC7lnc zoD)d1M7D%#^bRe}2>JK++2@_&<~qXP1au&n9ulom`^nt;fVVuI6lIzBa?G!2RDw%x zufh9gg&2Or5-yi5x&UA7(1GQi!e&yJM+ zQ63kX%MdN#%XeHo8!q~`JB5ON@_InX}e(Mv0Lu%MVhIP_~1!NX5K?ElQ4FP#v?wpH0SU^M0zj?uVXwOC5e3iYcQ*PdM&mu&Ta$j(x6|%wpD@ zAi(#0|*+-QlJQ}OAPM#9{Br8*N=KekQyM9)@ZAtrV;YgTC| z9?8kxXT;Jxc^VlE7c9Hji$cxgh0@YKOmJB(wdE@v?rhTA#-oIcR!hhquqt|D$~2NREKRFGW5*hycJ$CCi-xhz z@1(IJ!@EJPuJ(ZGd|6J#DYJB|o5wopm%kfHg7K+*X6f&a*NHG?p{TNJg0B8#d2bKM z`(&$y8(;7WjG{mBarm}sv2&tUjWrm45qLi>;D?c{bC#3DWfGN{PF4iN zsY;EyMDa8uS0biJvx3(VmT54Fx5j1UZYLS$axM21k5MyD5-mF;XL2QV7unPOGt|x# zLpA>dqE7RV%ru9YvnZp!cvfr49a21Z6CNwm%*8%wk>jNH2(G%_$RM znBC58rFMF_SZYq}i_WB=G2kR*+rO6>LxM!q)p#+Bf>3-!7XTb31utEREd zkoD$GB)rmi58b2fTcrY52@}2L!0{P3&=TWKVEH}t;+4^rmN*#tOcC6T{T2i9uS$MW zV)-utH-lbUVJ=6Gq*>qc+3MYDmafa*f;_GIp^SdoLBhKTCuYL*zZ?dmB2&G^6LVvL zgkO-%519-Ou33cKUkW>^b&}56QJi%<)r}`uMO!K&+$*&`^%(-eK^E`Qi-iP<#P!8w zBIG>cd-n8pa02~)R9+h#F{15vaVx7BC*?peklS0#67GjMNEDMVkm1aXY_kfu-*4?_ z=%!u$28`M{MdW0;=Z74YgT=>LD@0>({rtAUV{1a+i+9W@CNaN(C0)8G?a7~d_(|F( zCSRoOzY8;yX91jzaL)RJjPI;LjyL)u2G21Q;bz(R^9<{E3G^yCD%pndv_F)EQ|!NW z`<^HwoX)?B@nS`%&nDzZR2|JAhC^e3muQut_w9-}*YbBnGX39*%0KD~&%%3M(z%_b zXW+fgV(P~q{c&T-#-3gD7sZG(jj}$fwZ86kAp%}{T)0T8?iANjUpgN_zA1G$zE$y7 zfb=IBxKP&KkP&yI`V>OeB_G|i@$qmJ_p*sAAB_=9T&)HNXLslPtiR*C%^o)eBJ9xZ z`q4)wur*nXfiq79Nvs`^Tlnmjc=pNOhW8PrB$idRN|T14B|QqC;Fv}hkTXgC z=+`lXR?Kdyo+govy@1_2PN0qX4CRzw^7{qfYc@*p6(T`TW3Oz&(Q{)k&L(Mbrl+X| z;S^(=7T0N9IYTTF>%ljGY!Wtmgstur?*<|mf7c^anuo{=yux@&KUb-H?~JE-mX6Aw z(q10P@8=b=x5?QnK4?e@Ko`JJy@)XmeqG$yQrNqI?fxfUTIjd`MSqANfsBNreR3sm zncT%r5={W8XdH=SdHQ=nJzKKLz0imV$AIf+B@PvJZ_-{%Zm3YcX_!GLN5RbD&@ANDaVu0~9tt}5pYYc%qAp^1d>-Ng$<6W;!k88{2@HniN zj=Wv##AcprmD|Yhk@r0NL&sYxh2?%Vh2ggHw2A5|)4{jV$2e9gP4{ZOl)WQ|BffWm zrvI=i6&;B6lz++at~av^>yYuTCSRx%f_FIne9X`fLw7oyQYd$gGw=u6?`!fb0?9U` zl62oVxzE+n>&nhn%d6j8>cZZI)K3%)@Gk^=Uc(-f>h&DA&iwe)EB8YoEai9jQ~cAG z_&DlAg$D-rZj8eyOE`F$G%Z{i?@Y$3Z;W<}@BBJ~BH(J*#5OBckJ0aqm(AcCGQdBr zRZ+8wIc^Us*7mv6SL*J=940D?RoQ9?Nq(|ESJ9aJF_LmiTh`IJ^HcrSvi(?BXF-D_ znSr|G4@In3gKuZ;@4yyX12_7MFQkr~@eIxP)BAjt^pdWj z!F={xQKI~fVhQc;(Wt@kT3G9v+}>}cQgW7S1ZUWxx!cfL13rbh6-AD-LW&{mYWxOy zz99APz2m>wKAebs-9FquiIvkfyGOiCdd-vX$syIhc*k|u^_h|9vUZM66)Az@(Yb$gsZ$56u;C$TNmhnS*EU~Sv3%P)2hIL@g^fS&w zMTF*mA121-gbOQL*JjDLW+o@P40;)z#QV>?@gD){Wv6Q!dV?#BW+(sM0P;8PoH^7( zgncBs!Yd+GKmveI|6lwdx$3)%fV}+e)&NDp6<-WY>l$eC+bH(xnJ{DkT->u59r`Oq zbED(J7|=iKkt-yl{yrG&Kb;+59T~_1-^SBh2_aW}l&>$K1$f-CrbxKr@|cLC`kz2?_9u=>f45fsw=g|1T;Uo;2X;)p<5iZQr+_XL zF$|@x*S~x`yuw!DOTY+4!KxU|fzTg2q?sdE#n>|S_zE_HNLV9~Y=2c_ew|br?{_!7 zVuf1?f*Z4bAAR#5pop^hbvRS&2vuhIlN*-wQn^McnSpergW~j z_%T)RoqtnqJiY318i)K+ly3G{4KiG30#`lBICFlLQ_K!}cJb>3uQFaaL(lF8@z%vv z>p}p0<>Nt9lviCf1gNxk!9{pq^@UJ^FT_q}GyJNno`kOY8)j|8RbNQB)KiKM2)(wo z`l0mT#u%8AKnAfCw}6hnw^~z==>?M1^K~0wIB~uH2LBcRIH-mq81%$i5C9Gu`wH+jeGLNGFmUL!6f-P(JSV`hAp5^> zbD(Ja>z+Y^t@R9lJ)l4YLd|k^Ky;ghM3qDkS#N=;5$WIuA4e)<%3EHhqdT{{}TN z47jtNM+!lc&=62cJsfomu^TCtwjWIcGI2A|%t*cVa%ckq%#}+F&?g*IZdcoAUbVUI zW7b%FyM^oqW*D8uZ>L~rSA_RW6JnAwX9t+aXd$1;7Ra#&f<-By|K`UNW58|vsGM8| z<$upwxEjDnya4d;VZd>u4LyxPwVjZt7I+(_d1w!d@cY|O$d|u(S zh$P~?bDL$U22x93Zu^mmj@JRT_`kgv|N5`}Zr2P<8L+Qo{f^Uq##*mJGQSIMAlW5k zR`MaLBUj`cOa<#H2U{e428`K#3DITSzNDUa0x;g#vUsoIvf>sOv}exItT1PIycz)) zGhYNT9R7ffmsJOGalAC|Qj01`GzmTOgzQ6lbx?$OKqk%q&!>9uAO(Um7+bv#AJ$&R zRWF28sb99A!Tj;aR%^h2$EU2 z96%_r#9D}ueNarT1=6g%e_gcbgiEK@HEk0tnWrL<;GDhxaEq_)%U?Zje`BMz4`~VZ z<@Um-*%h>4t*IO0SAbw-AY|+)26*U!7LO3dcpUiYXqX3>4Riy{U^7%<(<^hB1e6Y< zFBV04@i)NAapI?G_?MFQ-+z>Pf#Vd;d<=4pv=P9~pn$%)0R>v!tTusp$dS+W3!v(@C_8PduHTv=dIqgmpPiQNMXD~Om>HB2qlC;OzOG@ilQFkK)Vm7`>iE~P~ zdkh@82FVsBj1$1Rk$9i&{0vxBngMNq`gw^iWz?4uwLmh*2&&OqQ;G5SP%huGbjR!+ z^UIML#=djX|NSaje9?MaYO%{zcASUH9ai*t9ZiPTMd)2SGyez4{qOf2Jc?X=IDDr4 zFXN(XUsUUnQ^?xgzv?9t3(AGf+(9mmD;{I6La?b4-g{~PzZ0gVuBAN$#R+p<@djkykvxFiKEk1gmPM{L2CE zpOxv~F+qh8Y!t*duKJxLH~SSNY(vM8S6H^CI6xgz_3(`2KYqhM--Inepyo5y8n(Eq z7_|XKPbni$x3@Roowua=jBbK(ZJXLX{Gma$YZ7?txtn$0l6?Bq1QQH52Vd$QM*Pws zwDW|oGTL5m7{G6{mX7(Uu_ z#+i!#=YEEg`nX4bU>Px=i-Xm51!J|=WO(1f=zV?ZW_AU*sKNp0wK({Ee(A@hx)l9a ze(5U;(IDs!v{Wzme7WK~v10g4P#M<mjv4As~3fYw#8L_osB4oTGJr~iQ29P^{;JFTmTV--2O!-^8-?a!oPtH z?i6nA3Q&qU19Omna9%$L17Hc;;4Ab!z>G#qVIYD!27gL4e;d3$$>;UgJ)r@YR+SJD zw+%;B13UCf6ai@{Po7KW#&prPMgkc2ycr1GlVZhX%{P3@Az=824slR$HG{g*;9Oke z`LkR0_MMf5YJk;g{-*Z8^*IEK2URd7+>DR@Ty-4Kvf+?jwha8$V0*9K6k{-+7Z>uN z)$Y$nO42Y(#29~0+JpV(sx8hhw9AhWh=EGq>etphWyOZ33J{T~g;<%L2r>Yd^*_K1 zgd9>o4hjTHu+1j+|LF<< z@SS?NtM&AX^`Yiww29AG-(1pt@_E0RWBgreI$wr^_c+jvVt6~BM9;C@I`=EN?| zo@Hf{5V=z*Y1KUN)SGAQ{hC@>f&9F{&ThCLGqrr@M_A34^7?D}@vp!Ek5WY4?^bJJ zfOtg_`)RlvcfSHeSPO8I!l_9@dq1!0*q_W^>_z8FN*;jO9wDZ6)4(Ktt;Qj`QjhSy z(vXHwP%b!7s47Ry)aHPh5_ZG_pRW?O+A3u(a#>pnik4Xb^T31r zUl!SJ8re|&RR&Po0g&0%@ka;b4NR;9B7`91g&%+mX&kRFA|AOvq}j3C zg;+G1qb7||@fHktkep{*!ICh@%Y6jgE;L>M%tWy4iN#hPO)oD;e+<^^#4rapUjliD zmT>=V&w0w}x9fvYF@6Aw+o<${a!PJ;sWVSq<2U`%*w7J)+={A^!cZRzEbQstnI2Y#zm~f1!gdq!*H&k^{4d>`k zzZupX$f1aY)v9q9)3JB8Ee9rWPQA4DdE_o?A?Upw_Pn*lDVv20r%MSC>lzT^p*N6Z6lh5jR){pQ>_ptCgtDM#nPhA8M_8H3F{GKgpbWF$hoCB)R0wds?aK7r)f*0QXse?;aC zvpoukDs6O|MHw4UIXAie?wDA?3Xc=!6{UY!8K74O`2aWY}@1 zrV)#WawEyX)fia>+^&vbr;!6a?r_(Z`{SM(&XvXhGiB+)G+5WU$~1o0k-BeAj?#`4 zB4g!tz0ZI>b7krn0%DfVy1UhqBFSMZC0D~-R^rECPDc8 zY4Vrf$zun3$&G*T_V+t30m&Xe9{DB;ogx1fVZd zy_QI~S<}XehT*=1O+(cy7U;uPYXw zclWosq<7qif6M48x(;~nhz%SJ{M3k0Y?!HmM$&DQ=X=|=<|I{z3q=n{SZ#=L_k~n_7T`m$h}|FDB6waTBhCVO0BLbo>o0aTW8#s00-)qlib4lQhB;+mw_^!|1^`m3l2o-2?r1E6Cy4-01=YNs8Nk#O`uJ z(es6x1@o@2Jm-s0?xt!YQHWzaK6;0_ub3dkX;3kw=2=09WZyOGc1bq#L$y8`FAuHw zNRP|Mk+O9X<;WK~pO`<)ML_^*ub-#C2*ke_KC3xDf~Fj_F%+GHlj@5#i=t`{6Dn2l zT!{++R>VcVE_$KhW@vq?^IaZ(@?3j}u1IY#KffA@tA(=Li$=~`v&9Y%`FD=h`u3Bo zm?eL2X{vbmp)GFhn`-CYZ?_XoUCHFt zhxOAObB3)&#SP42UB2}WVJ2KH4P|b{$rXt*lziZCXySQG17^MP>yggXvP)k=;r|q*bbc%-JjzWiD;rW z?(dmN#0vWYH=@h+own9}u7sz*1#hNILx;;biEKgRlqqzDv?K9iZ?<;K;9h^{3&TGH z35WIN0kxdO>K4UAwoZ2yBeDx+!&3<7_P*%3^Qieeu@Y99q6Qj#q=%=AJ^B}jLQ$w| z-hM}kVZ=lRci_3H6h!jw9oUE1;|ZzBgEsiAR2_%Vo6^b~7376^tfjxsT=z|ooaQcL z-aIHjN#z_WUwUtt)<_SF%mjyWk(W;%c+Pn#c&{D-m5>seO$S#`)>*al{75Dtmweh* z5t4~Hl(gGza?~=TG!78}EAV%r0<i6s(u*B1ZlJ6ftbZRMXi4p(d?s0&on}wD9OoG^AXD5rxH@e*_ zO+6Xi7G~9CE%oDbP(#-}J6v56uYA5NQmHxo6#1eE-wv~^;RJIkNripTnP)a&Hf*nK z$)T>Ee)^El?PZN_hEc!zX=}w>BP!MA3PSo5%2YNkg>2CoMWwGp2Y2ANitxlfw?0AT zj8=2x(Oq`5D!$su6mMQjbISa1XW8paZQb3b1?nq|1Pu_z?|-YrP&|@o!X$ZU5F`|m zB0(x1Q`2>We2UOd`i`)B?&+v75 zok!^EPYVX>hN`SRFvi`EXS|LtifF>JC1PH5eG#B=e>5f5u)wkJ+p4RMrA1hh5cE<~ z?ezbU_m*K*wQKt?B_$2QM0%3aC7seCVIV0WE!~2oG)%g?1*IF2PH84BNQuY<1f)cI z-}CvewcfS&dY140u)l2&d=3tu0b|TD?lH!F|E}{quSk=m{)hl?y;$FN%@>*UmHFk> zq`aSF@!zzB7HuXsBA?Y!jo9Zaf*R`X307z{;VUqjcf%5~kk#}j@5%DJ=ibC&zO5V|#y1 z|Ddr?t^p6xjou_{H0*Cdm%XE9mvMqQ z@4E2tJ|*vq(!QtzaEC+@dUzA#1>uV^N}x`eF!uGZE*9j)t)x+MfjPpU$fpwY_0zfb zUk!*sWvd~(do&yZxe}KR4_;j-4enpNJY|-jtsj&PZ_>CbAT2dGoRr{&tp5 z%DsP!?NFEa#+X^6p-ewV9jq}+U1yq-G}nVMg6tn_0Ko@X3_V*Vt|>@D~azR zI3JL@%$iZv#JTdz&ehIf*XIbj-|N>%@aNO`$oQF#BE&j0L-3_-LLh%bwt5R=>kYKR zvTRQ32Hg75rsG2ri)=y2Bw(rA$5i!J7L&%lXjT}Q4t(wcXG;EU`q64nu0-Ao0rH2lfRFqWh#roCf zYs5ku8CA412VLR#57Ob9kKRI3iKtx?>M+iVd=;!UmSxCK~z-}AWF zXdZqx!H|2oL!%*tNHr8j@^JfC%_-h;-aA{ugV@STYqYS;pLRui6e<~RrWQ^WTGHWR z5=S(O5*#Oc*>e?>ze9zpo=d&le-?Yoy`wSKuD2n*hLv;@aT}78;YEw3Mp(JQPX;pX zgA2(|DCLB`Y2CNMxwl$t=E}8_m>}-QgxyMml-v7@#xi8%Q0CaOB)wueaz6xAB4)h< ziP0;m^ZnmGU$fD;2#sSYvZlVHX?_#=*G%H^Wi;|(FwS!YF*97in;fqjYp&9!%FW4F zG@Dd3q$+Mo9829AKKa2l3YHpI+`C*GT2+n}1B@#xM~3v(R%oPHrN^yIAGIS(IDKD5 zPkF+vs-c;UnOx5&@RRv)d`veekFu1Qv{X?DOU%QOT|;841#6YPUunaBTK3Awlqj18 zt{&)_GsUvnUORJjqmVjcTq6vCf!ixIcRPZ=jq zw_%N~UH zq>FSm*SdTo)I(BX*mn82S`lX%lrF9&MIe-El?48<2FUfn!YLfePW@T>e=&MX)+rX#ueC%$r+EMH~hn0d$2JRTRCI<)(qVuBScl@var#j04st5QZ`BgP2&R=vXsiH>6U$FZ})<>+Q97bl1Qc3K#q_`UiOF?lVb`k^|Fe!(g6RMQ+*_Lk#;!!2;iig*)`z9=tSdJe zH+2Zh!E`KuV^8~7ynTKzvDYdHr3JF@rGlIQ-~=* zZMC#?#U;ovAF`3_c-%{ENV4KkGhJS@;vj|iyjnIC(4MBYQY23597}Re;u>N`J1y9_ z@92G~WNLZN(|=E5^^y;}8xbbQ#Lj8Z#V^V>!iHlh^YC>$Tw>@ajZ)kU`bWAXzu;PB zy;V`&Ommw%t~F`To!Q&>)^)_r+4aytZ~W!hYAy;W{SHd=_cQnzq-h)D4sOm-x=kOr4V zn>U<;he7R}y*@PgQDKvYA6(Uc(6&jDniqqeQkco0NSRrnU(cgY37^`zR zcBsih&Nc2yt&YIE)aF4 z(*-dT5QZPcF|{QehJD-hOdJX{q64?fnbHW~w~i;*eKV+wbK3?vgv0?Okkxi||LPr~ z6d%npinoh>TQKrLRsa4ZpWbs0sgh*zEFsSPfWY|A?nzG%>#RsiI%?(7UdZxi4WBD? zFdoJ_Qb_Cwd;E9 z5z}lhaI)Hb)O6vPWRzHaRcu5sx7sMUHv48N?PvqBC} zjn-TC6ZB!}?>emBw=wuZVe0P~>U4Z@?}oy<=ivD1nh5vKd8s&TF&*Ew5nty~TrX){ zyhKiv->58#nKGiHzI3!+jxt!_letVx6TeS{>9I)EZ;J@*lxh=pXe0+NR8g5-1YHJ` z4bt~DSc9yv7>8KKxu1M-oV*YX9h!n=lA$h=hC3CKD@v%X2syU<`38<%(;t1m+-4;0 zJeh^92J}2S=5taSryF1%A?z1wgB8o!**AD|HG!AOQzAt6l+W|EYEIl<6qjp6yNExkX-qhWpOnUkZaXn*Q1PC)f}iW zd-ZtMChDHS$Yj2swy_qXl$Fgyh+cDuqrz@d>hWwEt=d_^bJ8^ug7?jSn6>gUM^>2D zV@gkWY8p;N5OxT!QvWgYss;f?=nJ$N?8biWd z{EnYhCex!}C}EAgwheNcH!CkP)<`V(%P_0(H}RrH!+XHD&aH7D%Dd~GVCfbz&^T-b zUQO{zliRmij55Z;m&2qYq?D!ILG;W;ZP=I!y^P_9o_GC|y2tl1&{pk={R=tYZVAJN z+2|0eoDAXf5avB8XG~~*{>UVsgPdmr->sAG{^u$!bu_k%^%vEGjSpXRU0c(s_$T)# zZzc=<>TNdfzp1~j;5ar}*G}PTVYOvo6oFOXX}3=#V@HN(a+#ct>!$FJa?)?di{je* zOE%efYt4F$uDX+IH67q-L#t-_d4Dv^?PD|d3V_}>EjlO zX-56^GFZLS&(rFPMYTNW%xDPtmRypnWGsti@g?x#$hPq-uJcR6K>r7L@^TosM7%uE zY*O5yt3HKVUH#02Fyg2#FdWI97A5))cgQ`6sU^6Nj!niogwu_Zx*5;x#t(@? z2s+2in}lGi2VbT+jJozYDTFnElKqA6uh{Ur1*yc;ET%;Q{X|0~7AONDluCywR^nmy z73#WeksPG*W+t6QU(A$?sGX!RQ5uq*x5eGb`=Vb5Pz>YT;i}Tx+NJ$`_(Rti-wJ}R zg7=+mKmt8e%x=qu6(R_ge`{j68r#USbEZ$b+uq<3??y+S-{V^3E8|?IU~TNAABQz2 zO`TrP!)E6^M|8ziVRty<%u8}M61Ar8T+c@F@j02~8tpHLMfnZRLKksojb9+H|^=z6~F1O1MM)@s1d3g-WYL5aP zzhAl_qFL%(;JV;Vf_-;`wy_<#`hq~M*-(9mvyOorm8Mv5?VT|bY&~7CreBonWCL*S z^g!ydl(CqGeFWFZBx|$cF(+ORWQz-Te2j9=b6%?PA9vVi>Qab|I5A>Ym6 z@kjaJFRY7liSB)&Uw0B#xt&H+u3-Iv zIcrYiKKs6HILI2`)=G&Esqg=E_`S)4q*7eXIA&YP>l`kEY4au8SRaXOXa`@%&lh|o z#UpFj`SwBT%_nuviSc&Z#rRIAWVVv&5F9US1>?zJR>p@f2T6IViViT+{9f3)+&{D{ z$L|lAy8F4_RnMz7G`PO8F|C3j)x)Ra!tSN|G~}7BU-gFf`g7j9sjSN$l?+?7>&?Mu z)w!+(TJ>eqk7%7|C;BHogjlffpvQ!Lc*uw!+cTk_qcYB8G;^>zudcTu%c)A1rg$f=Vvj%m=KcZS%QaY!$sHpH z<|tIJsjz9efv-7i;$9~>&=H6Xb8<*?J7a6E_iQHbinl-aVfj&3rWl6V+xwXl6bF;S zmLGVpyMX-+6k4FFWeA zDNj*;#@cDNqIk9^BD{G43aLIFkb&IwgyU!Du@YDH3ddhD`rj!AU@N+~=~bqGR&@5& zS@^KtT)d=|CKYkZ=IhYQyK_Y~tT7%?KO8re+c4ZJjy+eMtFLDaZ>P+QW5ty?Ze|hZ z{cYP|vQ@&!KlJOn{bIVB8_RTmcyc#!;Vh$#OP`<) z6cA<>IrsFknhDUlAr$%J%qovQJ}^*?L#cCieruAMC+YaM_k_k~vxL*?#9%h(?FQ-I za3brK?T)R;n$ec44fePD{4PDtAswF%8XlxKJw+B(1-pTqZ+UnOZsGi|~zpPhLB zvQ5;}qC>6Y1?|a#_;(72${d+V8)Jr6S!;o|WmX?`pL~w09ei|IaAs9Zcob%2^;_?` z&G@IuB%93XS(8@`{CEssx$MaeNeB57aVTAo$sa{7LFej*h0n%tn0Q8Q2X(Z_B))CI9~w3Pah$=h|6B|k3is67A0YWVXy(t8Eh zvg#XDD_rJds**(et8WqA`BMquG8GsLZQRq|D_*gTlVQv;TGx}9GhRoJTMmgzm6jOz z!@}>+mYP?<5qa+<>K86Fwhzkduz77m5H8fg5KX2EwLHp=eE0qKUmS%z%-h6fO#6!6 zp!do~eM4@|PRzG9d}IHs2G47AN{Wwr?2GFoqvejUDz;3->xMCE2g3I)Lj@QadIV9= z4W7N-R45-|QuG|$5v(``JellU#5wo-*v-Ky@w?l;M?;zz@-i6mt8{42Jex#R#6*GT zq@B7JJgV>g_l9adlFFFIRMM<0rMt~tXnJcqdT!)jaMJ}&(9`-7dviHsYm1SFV9X?$Kx87fS06Y2HuRJNRoUS4qnEW=mG@H*p z8pN>`T`G}s+_l|^OR8nW22;lh4`9t}nexSbljDBsHS4waWP^c_xtKWcoJh;Jj8JJ~ zQWT9nEPK7A@P>h&@q_9EEVS}EHsU}XhP=ahcFhNK@EMFJ6fNgCh|6778TS&ST*HH% z**fmA@)4y{iKMfw?PdMwfNg^^$@o1lN%pX%qYrZzl@?4@!rB|GXi)n9{xu;4j@>a+ zsrdFCRAQq4{#7cBN;$am@X1+30CxC)|Ek9an=O9V-g=<$x2Ni#|G527M)A;_^r_#U zj*TA8UJh*5==G#U>rcl<6zENg(B8+RN?zgVxST94Pp~;2i(>To?1!_X3RLBl$R4?6)2eh`?Um@c>)iOKS4OCc8uZAnQyJ=J3-gLAJ-tpG)8(m6I(D2 zWn&jOE6Rr{K{_1kDF~FFVdk-bqE*NTxLg5^&^PWC(3~s!;$+U)V2j+13p&A!as7Lp z@D%)z+u#TH$?qf(F*oackJ=BWW?J z91jB>MX^UYn_zL3IO<7>yV83+TY4?cMOplkIR-A(JVC`nAkP~e1#xB9JnyxTlBOA7 zi7*g*G7I7>AMOFA;fN0>Fja(t-hzu?fLtvTr1Z0N2BMrILs`8qKqP9Q0r;sU@S_`+ z+3q?>xu2-mg9vQFgiauX*ce{KsKCdlA z${68jc~q2#7z|Ypb_Sw@7Y<0WM8+v@CTyc_Ig?biY`<@p8U;##z0$%M7yp~9CX1&_ zpwV7bd;2`$bSh|g_{qNG>X+0N&|$8uwUjb~E11($O%I-X(4XKKJ_3!{A{Quh(Qw~h zhn$ZAQOo!9KWA_5iJ|IuJ6 zle-ZTSl=Gw7b%q_73VyP3}x-=x&*?}kL|t}Eq9(7!c*_`r#e57ru;RGcn6D}%yJ0A z#NVD!GIs8})4q}eTBjk~Sf)bA5@t4$Dv~{9Lgz>ID;6|-*1YTbwQ%=0Ss0MHi5zZF zynnv`8fbf5Y2TS!uxj09^b(nmVP^xrkP818$=AP*7vH_j7|N&5%QDC5e-7J%j55*S zd;u;ppq9#U1$loOSY`tZVboufMUuvVW#SV^e)PSrT8E#rpq(%x9Y!r6FLYp>Am71^ zO&XXC%2$5+;F+%BAZ|6q0Fd+^ikucD;0_X?uNI zX*o(CR!W1p1R?C%dCxM5MO=y=2qbVd`t23vnn3#+vl0(#XtSx(L-!(AfjMQZ@v|k> zDbNApyLw>m0d+uKl^Wh2(qiFxGV{z6QoSgvgY`Rxn5Ksm?!cPm^o9y_jPkt+AW@Y> zW}kfluA$8s$zQsT!A%nNcLFO}0@{&nn?3-!o@1Jx6DaORRIw(RZudaUU~7x#mFFf4VcG5l%3t-8#h zgf_(7e-1Q`J(9pG-~dYIuE?m&A2nGV72|!Kejv*D8M@dD0eCEQf*x&KlL|xUGdWa- zh|8YWqZ?e>QLJ0=Q#E-Mv|_%5u-L?Iha4|5QyE&Z&p77QKmTF*TQHa{jlLh zU`msWrIqw<22D&?9*S-x#V|V%6_9^0j#^$jy3Nz7)n)|~B7dj6uF%WCshle~TGKOl zQfxqCA`>uW3wphQn@Kp!0GO?ofMMmSMIdn;szTNCw#yZerMsa>?cBTVcOfICZf0Z> zc>QSq+ZN^tH5CB1cVO!;BH<;3nS%D0?wu4vl*K24k#5 zZ7f@pU!aTnH_z_ZAsWvwYq`YsG>Q{Q{V0`;O;4nR?IeF@sMSM*sQ9Oo-Wav#V*P=jnMu$b)>a2( zUyql3U3^>kiv2|wh05?X7@eVcQT&Nf9?WAZ51)%PY5wl(wB)E-49cSO+X$B*}{%-Z1jqck#$5XhJ67m+HJ?wFOki1$aF~M6Imwk|Yoto+OJ$(tp6>en{ zv!%(Okvv5ehaB1c0**DuS4=dRH6c>24*3?^FM-JbS~r&Ek_^*h zhUsRRh?t|uyAi=Wz@pyZvi|FL@ z2rx7xPo6Y^_oGhq4Bm*I>kI}u(IJSMUC^B28YWZ+7OHl8%g>y!fa&~r#k=n^SkAjF zADM8P2higBG6wlslGA|v==PglW#pTOA>g{t1=qRz(QQOnk}0N+h`jK;33@Ksz%}Ui zya)O>+e>$504G^1w^BBP^*T7Q#|9*%#lc5(K@{i$$FwR(I6ozDQu4=Rabs><;u6`b zErvOCB}9oZL`m2hjJ>gX*b$pkc6uyt<9T&j-X3!>0F+mc+q9$vG#ELF32}HD81qWUf5dP&bzH5KfQ`t|{F~|CNVbes@C0gnl3|e98*w z(qdW4{F=5Zr!SaObPLWu`icPYaw~dt`134Nymn^jgf*JHuUUPeCSQKGWHbMIAVOt! zrOswqB&hI#ILsMKj-J5qZNhqUuS_QjD?8rxAXWagd^E-nrTC(UB~p{mGSENM75PIa zzG7fICw`S7fD-93Ic9|QeHJzs7CW;ihubm9)i zh&F)(g&lE@A3JHedvrmxtd@_dz4HYHj#oxaHc5rAVnBsox83E z>{qh2W7a1;<$@RStDI%`k~)v_{1T2x62Y=OW=KpJ1-<*Q9e)af7)m)|*et6=Vf zVgz>mZB)D$BN|L3NzBi5!$ioY7AI_VA9Z20^{2%5$c9YihPzwRhw%6{9xr%sp<)hW zdvn)mIm75pZ*n&ErmhQ`zQ*-BKS-)6I>9bFap%0>pGFwefTL*B&y^pTwTW!lejzrh z9?!HUaxQzVn*#F)uz36xgVCqioHCFWQOZ((&oC;QYS(ko^5ny}=y+u)nvpDhh4iQy z>uSbKu18-Y$AGB4oqQ*{z_?z-n*)Nzz@OP9UgzG)JuXQR2&1G)WsN|kSEzKj5f#J~ zZD)%9O$d~u!^KAUroG!WG9b%emH{8vUtW&LqQ`~g)py>c>LUC%hyBCxPP$}wQgB`Y zAJ^3+1OnlcSCyxaK+zV$d*oEg31aYO%tb1~-#73N5+n({c9gwux`7V&$F>b)#DlVR zaHc0Pji8un)dN%HxgM#F?PlGqXo0(RqV@KvH&Hd4OUiBRkU$B{gFtjFZ&sqI8@@|* z(Cprn|K>Ba%l@vY^dq`%Z!*bnWp~*A@kcP7wRZD75x>xT8A7q4ET2b%UEPaY7e{nX z2+J9%X1n{T(~dC(o#_#!nb?WKq&|i*^*mqa$xhaUE( z7d2p;pSWXS)cnZ7iK%b`{ey3vq_OQY1vSx7Iox}eB=F|L*CFzl#)N+-#dwfz7B`+p9mcZpWfqflGQPNjC~LTW7vdGJyWS@_ z5~=+DHeNSzfu-L2@M{L2Sd3wOZNk!2k%#Y|IotleP(Rkh(1)SG*E!n)Ny(ImhXE|( zgh0nP4&O+on0SFz%H=}$bWK4}jl`yQ#u%{)rO*PKWkSAOwK`Rcowj6(&0=dwV5#!$ z<*hHtTL?v>Pun|egq%@lz#(&{7l24;?dS~l2&B}(u+$#dTK@`LE*jvq7h~A3Gu4;g zV0XwGf?>LjICBboaH{4TCeBXv%dtMNme$l~DjT=JVS6smP8?`Ss^!06_f^c88a3s* z*nfaK;JL6#C&~)#;wzTDG=$^iPJ#T;k)#*AUg+ZhLMDVFzdDBU4&?@zf)Wy)-vAh;yDn_$~B(6;JdD& zzO6y|_98=Z7^W}F0*9fIXW34zQ0svQTOjb?AQAO^fdA>XnVIWUXSz<9*~ ziMPSqF6^G(1G7$e)8LOnE~|8UD)TAh{GV{g$;!GBqTNb`kjI4G6kipOn)d7GIJE=d z0r#H~Ti!|!%--68?C5!e=gY24fOi1n^z$Do204v*&%Q|3$kWxu==P(xg-UPdf}1DL zf70)>I&}`|?!7CI_~OY40p089ck|d@3y-)f+y|B}ULq&8ys&j$eL_>8!4X-E9#Pv_ z#>U+7)}%y}3XvMWIG33w$CX}>;P%8$+^lIBM+DQRBqmruCJHxBVSXMqT^~!F*(8D0 z0~*ms^L|A>FCSw%SBR{Ne_SO%Dkqh9u{m@jr3RIZc*|YEy5ZTv8vF9zv_e%xMm{4C zRFAl$;TxsK8Mg~pMO;e553S0@GuoGHpQioZR>A})l^dw{Vj1oRDJx~7J$6+T)52iK zfK2d)%)s7YhFG^-UoqVccXK9|^2ln0PQ+k@KPWVg)R<~)7k8VSynf`UD^@08?-d z<_fDfj&5xKFQ`{O&ku>m>C`ru{^qy;&(Tg;BAH|z*4yF#g+$XKN%X)Y(Lb}sdP*dT z{t>w4^1qO1C4iXEYo2KQ&yoBcF1J?%n+d%(X)*jWi9RO=*w>FUIq}bQI1{6S8N#&aYWzh5u*D=(Klt7;5<|3~y6 zxTkLNDUQM3^+MmF*q%0-`^le=co})I|9?|<09SEYJ~{W#7kiNiZ1(^8shjbp5HtTo zbPARxEYIgPrLVqW64GWh8;i^5En@)Qt$7NXD6v?U@u!C?imNIH)LtW5_eQ1lKo|jy zerjtHm>Tn=4xLdL`)D_{ObwmBw^d_}E7mNct7uyS)1w9aq1D1zsRdN#t8)M%=hm7E zj6rD6JtPPxlT_bQ$gBJdj3@WBkUX_)Aks26zTI$0_gIzj3`vwN(@Zg&03b`g!SU## zDab?mOdY_A!o&CCmpiht!)q|>DSP&2%?Cgn7Z9A(lnExYOXLQ2YskQ^Z9iY4Kt&=E z5K#Z6yl!k;N7wXL6G$j&0k+~o?~GQ9L-6T-*<N{WQmiVo1^c#d|yx%&J#n^rrz^_iHd?5}%x5&^QNDouP+=uf$P|_RP&e; z39a$=G1V7S=zjvReBsjmHIn1J#TE4Vc5miJLq}3D;}l2W=Owr}E^fol0(A$P1*Y>& zfcZMSNo~&M@5f6ez3Bu9J?!4?q%iBN8^FWc{07>=yR*W3UvD>*xhVIW?uDG@5~vG2 ziVj8+YAtTyqQ(^lrHT(odR_q#N`x$fp6}k2ru6QZ#F6?(L3@R~cNnE=AOy$(L$ZLNF>1;ap$7}*w$Mp0@TL(!pN!T~VSmwSYg$8iQARp1K%otxn~ z$_b#HJR>enDEf^^sUs4On2;lkrb#lOq33~py}CF(jc@)Jo?iJ6JbgOEQN@(~ydz5H zG0y5Gpsm?g0b5}8>&%+%maqt1%hV?*vawxf{=molVUQ)W{QbV*3Dmu|h4`NY{_*bS z);@*4HAAj{ij;O!l$IiwtO+Ov>}wH#s3s48BXC`O!*;~Ms&xrTQ2pzqc*>Tr1***u zSO9mctn@oomQ}W}tLt2i{5}CA_7|qS9}_)#$tl3|YoHF{wL(DS)QLKCyoNQFJiGbA zL>jc+%D&|mJg{AdEtr9VN@47Kl01jwOdHMD5ApjkjW}ZWC&tD>mXu2ol(usscUQ_5c9+wRLv77Dne?$QSPe$YBYcn=`k- za-#6&s$td<7(bP?iv)DxxC2fifF*jPe*&H6R@=|MQ#q2E%0%!HF%AM!u%$@rW$p=< z87( XCf!ibVvN?!R1beEdM-7d1X^lzvCY!2bSRot0)-K9Wq!1Z+g4ZPfwK<=HS_&^xY`kRv3v zJHV^}Y`&w1`Eg8oa@P;+<2%Qc=LBww$JB=n#hbj?yiq|b7htg>1+p#(ytdR_u}$*2 zKEunN@!Tl+YdV4OLrpTT*&+)e{P7Gv6b&_%9b%KL7h-gaAX^E3`dUJml$sS%S6GKz zw}3Cc+<5#N5tJkj?M^RN(1Vd$A|5G%(u8V9rt>Cnlx)TpI`+K_+e@EjEjjss;WD8E zqEyVBOJ*IR6Iy6VuhBwBG-TvIY`s(XfdPsj=V_|Bq8@KSF4pKG0d}y7N6=^**W_Fo zbxK`n(wK5gAUSjJn;2br6m%chv^<;vGJZDZ<_3g{wRoY8pfb0=HV^=wmD0aIX-*n} z8pB^Dku5Ny1suQi^N!GOg0L4)Q0)9D?3Zt7_Pb9vwg$#bCz2u&4z8oa3?Bxz8;{4?}9WL;4p>`i-@XYp37pm zkaVU+qo*KhK``6`Re1#3fQ;D#vNLc`S{Rcy;eEB)k-q+IHsIz$ogBr}-n>7TKv0`b z2Dwd{7xrLfI4?I{8H>g(?k`?{L-ZDIMT}ragA7dcIdL;!XlN<2rkj)8(4e!&=;FQ9 z?h1&Z{SDqu7pSxHqz}JY*_w#3K8WbVX~Z5z`Fg2@)%z}LnbAfH6)zEq+l`-Ng<-=# zuz=T3yp{d1-p}%_Q@HDt<^KFx!>$;bNcz3o$A$Ss@KMKloP6^JDpP$avnM5Vy&@0K z=nJSzoU>M~1vUjQoJ&LCyBO>YDr&ziPCmfkkU3TPyw4clL2J8}rbiUjjMXs8y2*G@ z#h83A9rC6~2W=j~8(MdK$DXTE#uAG`0m-@ab7gGl<_a5ls^(|3=e5bTM8e(r(PQCR z_r8|j59Xa8u~1~J)3jeH{mAq?Bwe= zBI~gA!cU$HHmv@iMToS}_<{wvD3#FBHKBKJj&g5)3reISX|_L$DzT-Y7Ish~Al|*8 zd~JO#j*g3&0P$gh#)YBOiLv=l`!Z>!!-i9Z&6;^%Bf zcGsVZoFCO#>AI#kDqAs{(Jt#LA{ZY_b3ql{o3P#}jX(6A~WieRqjIji_ z$kNF87dghc;iE3lXHk|Pz(|Aiav?P7w}@1qe)*mJJ?><1HbaOBm3QIsR}yRr$(gLxi=AbV9NAm@!b2Q+Dt$p?-X% z@&4cxcv#%6aIsgrqv9}}VnS`En&mZjFmJ|WlKo;<#-h4ABhhEm6SjaI1jQ79e2Uld zl8NtOfd^RA&-(Pp#H7tdj0-N~Rcgh^755)9y$^RarB~`sy*Z=P|4pY&4xu$w zmY0gY4L@?aKhOo_1p9@3YjWs82{t4ePeTbqMdi;9%{X1I+U<)sV$z-!6?88 zZ#C7yf_uGn+PIep_xY_sCQ3l-I^SEoCR||4_jUsAa$U;np4u33{s4mut}GNHouRX$ zkSBl<9&YlMWLO6-B%l@m+0jL*z#Hfq%CsrO2(!N?n zHKs^glch-vm*F6m88@+{t_zwth{UerAv(VMHa;_NE{_h%c@;L5`vZj#^YBA(>d}L@_IVUV!9>y$&<#fxH;m0BLC3LpxmVl?YDLxH=Gg^cvj@ViS2vy2dQ? zgkR!mBY{{)<{AMy@8;K7iuC;QrYIu(*h7nc`@0kcl)O24y$owf=45Jk_Z31cxIs@| zXXz7S^g_v|f-^SXQ+k;s{C<3D_9*Ta>yx*(_Y%n!Xd6*y@EGaR(%F(=bS>F8->1 z4OP1RvT|F0>-|+*#&#Uy3B*?kEgO5zQ5UW2dIF-P4*T84SZ%mb@p6w3T0Ofkg76hp z5}ACve{Fu|M^ARX736?-Frvf!Redr8BOI#aiI+NRX2}s>iSUAG+v)Jv1?Uaoy=z!9 zF;wxx(2Q4l;#41^XlEH=sw=$K5{D?~TBLJWKQ}y#;f^AwEgXkVu%U6*bmP$mH&U=d zQNu78$cax%6s+&rD+i>D%>U4p?e5J=P_X*&p89SVxUtNbSnm$Jtu!s~`($rhVJb18 zY&#TPfp4>7|1S2ihG-uUEVcw*_PcO6zlgj~>jOQpTXLQJdL>W!U7we>(21{N3ZoBK zmQqe-bN6d`*ekFIGDt{Snat~qyC`7jvO&2$OF5uBcI9f!yXjeQdzV?)%aNc^BwkXU zaSY_#`1u|cWLqt}-ZQ!&?PTc7$W4-ky=g@Wl^#;pNjGo1C`Ip|g(`UsbJSB|^AzHQ z@Q=`;-(7$(89w7zF9V;I5{)w`gy*fYuY;N zuxAa=NqOeYaLCc{5$F?9kp%rOC7WU{Qc>z8(w!r7R0X+1ZBvwbox_T@NN>I;Tty(v zmZv(K1FY|>?2v>|;l4_U^ZDXaau;+ic}G?>zRutEA9z zhoN(_gY!cufs80MuStMoti41cD^GHdMACx5!|OhBgUf5XyNsCnq|eDC&;mUZ0;Mo* zFKF4xL6hR7T-5=7D7j>?a{#9J@q$4+byTfmdl#p!?)i}QRr1;5aO`gkZqew zJ!1mqC^qr2z-mk==9&jFw3RRJ=n*>xBa`inelPdb(^YFsBC?MNHK*FWWM%HdK0R48 z1&lCgWo@bZ$`$8k5OIby72-jV_qYp(jRlDnn-}L!58ZBQrVLu}aUD#EcQXiA{{KEH9yBIxMj6i1T{EOcZz*r#kpco@~%1oHO5| zTio_R+oWHW#qb4NntSiU^fc@B1O@XbYk)xJh68UBV=H6VY@Fs4Yoy^lui8th$2G(Y z@*u`=mQiMDz~+wcyZik+yro@kO(526jHly~+DFqX`jZaFmG+3?e7GlxZXlx-uveWb zdM%rEisvB`;jFva`p)uo=Z0C!tuS3E+9*rstDH_~nPbvM!7v)A!vA{Av2vEi6%!qk zrxSci$sfuNWe@uY@7Csj=W6it5FSqvUmm`qe~6bVe$RzVVbW!(mF-rce$J6I9b`u9 zxqA3=4}F&-B+*Ol2rMf$?%ahrukka<-*fi)ow+eZU;r&oc)Qb;u=T$DjI450@3>tA zOWht<9loCY90Mk!pU#xQqnX{*3e;CRtTH!m)>*h8aBVGA?8hh`>2^SVxvO{~Jb`A~ z4Gq#ZL3cv3>83F{9~DnZ)9|UhreIFt8}GQ-LM8dA($+@`)CHUr4S>7*<*|9Wp{f#w zO&{1OQyw$s1yPRPHNI0^a(n{qgkkp^E9|PToz3Ki3LbwTM}wt?GwrKqzIREB^#L;V zdCI!ux`LLcIpB6)FG=kmnDj3NZIwK6iAtGbLWz+p-1o~mKr8zt>jy6;pl~0B@#5ou(a(Q6j!*vQ|3?+habt9^_|He2N(|vQS5D3Th=^$TqN@EumFJJ=U+XS7 zuc_a)w-x_~@&8lJ{O|t~i4>2s@fB12$H@ElBW6R2?bVLh+!_BhtpD~K1)v>&o|HBj z_1|^-V00ygf)QElmDED~=d)S>t|gBu;J<%&;Q#s%sz~Mjq=f2H_@7UpHaH;^%y0kZ z4*mC!QVd)XUN*GF4)T8<7;NB#+$N%ucvX#rc3-kRm-SXl_cD)xbjt%sd964D^5w!> zfKC)Rb?1z0E=~%urym3y{sJs{)}=ItIPekiZ-5OREj~LJSXkrKHpo^`z8F@duW+~o zfRtcK@eu(U;x^DM8~hl}%k=|M>@^q1W@p$xTWx7l1>fh&mjF?zl6YsO(gz>V;`0mA zj=*5-i>zCNSQ_ak1ey3on^d6zc33_1aKbxC(P-hfvHzR+JKJj;$$dY9m(Ulm&itHi zr|S9xppqX*2{%&Q?cQ8MDD}Iu0%Y1cRDSsGQa2`6wxX$I-R+Dd>!2rdK_)aS0c(W*+4{EX2A3V;e z@t(+&<>=VcYqp#pLL(D5rT(H`qxZ?l|?`T`yFO81c+GNzX4^ATml=8 z=&GBx7xlY8Uvn7)iF8Sju)9k_)p*m{M)DfrwQ?NpPBdKfIT>z(eY4LxJ3I z_xjrFn%Wbz+x@){MD0=jLK4yu$O{W(mIoW#AmjM-0BC%N)r6pwW_aq zl=^%GEIBb|vw(c&15Q3WfiE;q&qrggQ7m_I!P9p9$R8i6!J*NRpZuveAB5#+ zRPlhgI(zpHsc@M#9IZyAv~R@tNuXXLq*KU;|uk+w@cA_N)CW%B0}ICv1=&e*Wh+sX7wEA6zP zF~w**UUpR`sR`#JXBa`BN9TO;*GuY_A@(viSN7<(s{$d5bJg?D^&N`ea6%^K6T+lo zLH1aqnC~XIcb)$)#@;$A%C6xT2Bby0ySqWUyQC!qq)Vi`r5mJCx>Y)*JET*PZfTH_ z&a>xvg});F@dp-q+s0^cpR60Yx3RHr8#oEEV-ht>7p2PXf7uqw*WC zW4j)OSG@`#4ow?~x*FQcAj4`4bR^?;CfL|gODihRlCPaf07%P2?7##RwY#N89L`*V zz0Ynhu2<+6JP6(51z6wCaubA41io*@gDTClYHX~ggKuTGZJ_(XN%w2l<1d|SU8eFA zkObc|8g07k_$=le~88L!d!AMd1|vu%HGlonPX58VDNR1JV5YvdV6Gg&Y^O5~6wUXV?=sdypN1QRtmggbyP!rli+E$Hd6BPmc=4 zHn66U{^qV-JTOJSUc-Q5@s6dR7Pt<+6zU|$0rySAvZ0j@&9jb}}A25b`P)NEM60tcppYp+eFK;<9~>QaoyZ)X}rm*%Dgko$_c& z0)==6*=RavG{4`FORrv#R_Bx?1 z41hUW1fcbI)VmOgrk&dQI{{vR@Te$iB)2sZdu6nlnT zV}PoE6WxxaiT}fc+br>)8uF}EdH`MXM;ybDkTmjV8X5eDCRl?hJ249pk87s{yZfj! zBD@SVdW>Q?F_XIuQWj$qZ8IlQuX?>cqij~HZ{iO%?-Bz`@qNl0Tq{Hk!r`$zgQ&Oe z2_lflR)DiYn+x`?$JcYaF{8(%F+{h)YD^s%5PLyt7w*7Jv8GQS{&nB4((CcTGv$jC zZAh6+O$&%2nWccTtimPt4bhx7T`)KdfO{>f4GN39eBM)B!*N)%aUOJ zl%*>tU+_$BQR4MPdiJ)7LiUC98bAe&*-AGa_pDR+hneKM!BQ>SvZR6GG^-Kr+_HfT zT(C6A>y$lSo?Wm=O-iAm9D@Q5_LCD`S(i6p26?DpS!31$Lleff5i5-tHkERQoqyf+M z|I(E|hHqr$#eo?-=_r_C`34!TYrVhgs3=+W+wZR!V^kmVaZLTaadINq!os)UdYhy6 zH;yHwsieqK3--tL##V3_D^sZAu=BI{PZ6)?&VhUl5*G$-(`)u~|cDEUM`LYJVJ< z*ztC!dH`v1GvdPL?2yIJTV`xyg8`EHvWh6r zeX}^$P}m2?!q`pm!H3?%P*YL_RC7T@dkD2=L+!cLGF43{*6X``(Oc+6u!rpPe#)?s5-EQKUsm=webzI00_sLNXYh z`ZG_G9o3{$@+-cx^!`zKwDei6&^$)C1HB2uhp+MlfiE#ljQkdVfGznP(q$}AaGZRU zMn5~|=@z4h3Z113Usu}S{%Cr2if7K8xa~Ei0i*NQdk5V!cse@hgAMuHVGt!`N*QWm zC?8Y;nItUar9(kQlk?{BGbN-{T0~U;ov0)vcE2KoEIdcua4bB(!p;p!5SjpixTMC> zujdV5VD~0BM2ysz!kd>0+K7a5Af$ ze=Ir8FPLa0a0PjYHnaGO%qOMl`FZFdsEJJiK6^HTf!Y~8uZ+49Lwe&Bj5Fh~UzMFg zF-sYl>QRCa&$lLVXAwaPyl+y!kY3^xOY0AQ&7`%pcRwdA2;u`m_1sb-Le>`?v&H1v za}O8pu!AozG@uSt%1VcI8L9a?0&drK!f_{`&jCG-C1v@o)kVQ5EiwBHgqs;V-P7o! z5~#-a@An(*Sz+DWZoS<+;mk6aUvWl^3D7ZyG|u*A(Uv6t_06%4tCYMOB7_@6;i06f zkpDnLCR2+2I!-cFUmaf8^o#W50&;l4IJdz^31&FP9xjk~mBbek@KHSNfog)*9|tLm z?Xbsvn<>4BXXwCnDAT!82EHX2#-HIL(#9pta04Nlp7rNDsX(Q6E|d`bpSi?%_A%I; z!Z7H>X*QO0sHAzUG5F)$BT%9Tzj%#&U)h$nw2KA^IlepU*{|67(#m}1R_M-%4-z4M zJq$r%4GiWijyTQ6i@`N93H#Mpchk6d_AfO%?~9(i;D$bDy$np*FlwbD)g%`i+vG-G zhw_baZhWr-+dN(v#e$!r0Sv>4gCjvmuNW{@TpIsS0)Z@>Lh2}fO=z;DrY|h2fs;czbWg#0H83qpPxyW8%e&5< z1@5ItO7uA%>pyD#iM;7Bw@g~M`rk0Dsd(_GC^EuAsvF!#ZXAGN<%saI7-c8Ka*oiA zr!Yc4NhVA2O$-d81KK7?DmmW?vLr+H%J72>J%pL{Jj~e#1n}li<4;;Skmn1+hukM)*HU2JHFzaML)8AS7yw;_k6@0f7%mCIlzM#Ffe%oRE=8G>|LYR4O>7Jt}T+1>;|`aar(g21Fam^GtJxVRg{N@oM&}%3B~|5jtxSz7YP(%}xee zaML24&AJjY^MVeW(kxRlKyyjKIP@9Q(Np=U1H<5ZYM5gv?b7%2#Rw_`_64n>>+HhqG;4=~=daraNY#YW&5f zea3%=s{fyFTo8E!+1!005Ku4QCm+QA`{n=7cf$Ua9`lFG$X#NIa7doSzP{3|=x_A@ zym|ljIwj>pngP>($WQx!b}d*yQ$#ZpHUIhRgTEHLP3K?T!uHSK|2+WLt$8$2|M^o% zP_m&2L+ATHN6UPKM)tgyQ~Kv*8Ffu4!vT#x`sYzt3pk0KF$Jc7{*;mu_@7B)dMj!F zJXZO@Agm)|3j60zDMERWX{53xW~3mfTKGQ_Rvi3w75*y=fFbw&K8X*&vF_cW&An{? z*%e;S0Nt#+1CsWL=6a}*jE79b>ksFh(BEDFa_*b}x#zeAmhTZz_#Jqx`FmagHSEAi z)o#MECdYyIM8!v4JrAJZd}b^LRMqXX%33$7{e{^m@K6o;6~G&sDFUvksKNs9!AcVJ z9u8ofHK%0&8qe;MIZ#Ma*{w210fByyx{A+51p956GX@r6(3pKNSXYhWh{v z`(cMyOU*v)ZvLf!Jkd4K-z|J$(rU$~im%nV`d*zO_4$ zK)NtE?Wtq!i39;SZ8=?b*!0&aq-3V@O2+$1Tp?}Mr&4Nm}GYU8{GGUS|n zsMp}3f}8?f_=poK+n=K;L-z(%NhGV+IhMQ)AwpUnGY7KukKK4%4>)QfrlsS{9={j4 zc}_Rs!I5@=^CG7uL6Td!r}i(%^SOh3+U`0W-?@W|MV&eTm~EE$Eq1`{>!TOw0&D{J zc7+X~g$~^a1a8ov2;aNo?`3xv`R>Sht}_97r3#B7AL7!b9&r8t03Wkm0tus6>bY#n zMA2REhejasZUS*9GjT!(Ul3eR4P9>GVw?0~>Gl1U=3dLVmjExt3KX10-<2SOU+Ej> zLDxwJ~BM{64M3^7;u{$lPKqP8N^^D|%?*k*&;HBXdCv?L02;t?xvKS6}s5c9xIeh7Clt zfig==U>Gj(wOE64WhW?Vrz`iD@<3(b)yUfL%Zmhv$Yvys&(;IzU#5Vn;)}@MtjvG0 zE6d4yF8&J!o5tdo*v}b($(0fQc(&MWD9(J8tV0ynzZppTCNf9^Ae1XPz9-NjDk=}Z z2cY^)zmJ!Iokp{V8ZE^6!^L8&lI9ppjxlswx)DjdB5)B4D;gJvxun0e4uLwKJxAzi z-TtdVm%{!&kp%PyLAL~Z0$$>;5W6hOTzuBl?<`{eF{(e zCu`bDiubS2;*Oy_sdH$dk@Un008vTJ;m{UK(tpO-eHwhOKe2d=y_B>nw#3Jh&~~>} zbMm{Lzu4{-bWVJye{1yGlfL=~bxf0aykN=gcTfcAMasB|CI|NTtfw2R06A^1nbmti z%==>^l2YmrP+s-95ewKx5uB+h^!l! zqaV^mh(s`tRe0HzK6|f=-|o?T2L)p{hw?Wz=h_;rbG7fIum>5(VZhl+?P9>WQQ^zc z-|Rhx9E1sz^{$}%?H%CpfP{M#Saa5{fzwUn$i~eo;J6;RNIhpY^@uvZS}s&v^)cQ58VES2tle4whhOJDf##uGwi)&b!Clh8gaM2Y3~Wz(+x-Vg z`)fy_jjx#VUHl3fl1H$*mA$L?oU+LMIzgS)^A*#D5&n?ZO<5puBp?kgQK>!~>QbOt zv;viA$gZqb$6 z14_+JtlbL7FM(hx|5oKEGk;8bEPqVP>|!bvn^Hii`cixy=y+qbr~h`vUw9`)s|AGg zP8fm6D+gd;xe@?DLZ9A=78rkcIe`&^Bvi8RUvEcho%H%U2qby`v}`+JGw(I0pebld z;qD4(uZ!JXf!Mc=q^N@icMAUGoF*_4UXt_OQv+OkJ->g{JWR3KBSXQd>x$HFUQx#V zWe?UC9!rXLx??eDPEwV*IlP)#;#?`M^N!rKTM0BV>J?`1=-5>t7@F&K)<&Xwx z04jTK8p&K}yOG`!4-bR?{oL%A=-gxky)5pbr}b*}?t7_w*>l#el8O_|$0Fd` zj2hlU^U2<(tF;Lp?@4r>>n+9sO88eo1EFjF1cS3>vw|C14#iFsR|wmT*+gbQ-0rK7 zT3PIK8`S4?b!vlFum%~uJXsoKn=x}MP=NXv2(%vcO(OC-3ph;pT+`SUIutJw{cY_T z?&wyaS#K*_f`{y_A7DlZs{2A7Ky-=PBXeYmIcF5CWs)NO*<1S(R;qHWNa|2cFnqU! z3sjPdfZXb*ksm!aZXrVmVahv1QJL}SjnWF7xM16PoA4g+e4H}wg%Z?p)`V_c?0U59 zh$KRd;1M#xg39QfbPpf91-vdsqBdX_{#D*J;Y3C_!Fk1-$p6j!oNoF#jleh_8af(W z=iP~L6>CONRL~Ak0-Ra_V0=)5P?|s5j}dK|k%jOw@)zyq=OALs1Q&S=oLsKzrNva4 z7GuzU?hsC7^~W+@vQ|*1d~}-$I(42NPoM5Dy4FGYmy1kb9ldj+57ug-e>ff-1>#>q zPgOvo8V5vaTMnN=lj&yqj11oR1geuf_O3^|#)!wjSV|1?@Jqu_B%P*wr`EiLybx|O zbm~UvcnhtiIsx3RTJ#Ba(DE-EfbNAqb#QmEkdpBb41qa$7AT1KnaH_>&41z731KBPDHbJ|PGUIThx%Ob-+Gy@ zMNIuxct<1A309@TyMRzqy~>_Yx?g~)_Y?2W-W_x;Fg8VeJ`wYX(mga$C$mPyaTvlW zGl|6YYXz{KZOH_p7NFqgvin(Cd2Mvy>ULDVG6=N9Q5DFa_`tVzfECYxCb@@o4!f7M z=RX=yXgomq9(}gHMOF+Nj&( zJL@{;sZ!@!O2VtJe-$C%_qyXA6piN|W)uU)ng5Mlh!@lQzW4JrtzjSu<)3V79~Ax^+9Ejg{8NkD_F; z%Aam@RnacbnUp^(C#>V*D^xYv%8Q>$n-0{cz%MOKc6{SMI^!W%^=fTI5p0BA#B^6) zKgTptk4&6+ESS;LxaJbz=CrsaP^y)UH_x#6pvbF|glQU)7i;>Z@8i2fBT`SGG|)w& zhPDiA@~uj|o3|P8QI3UE>=Q;rg`W@^c%6cP%hz@$6X)^*%p4~(N#3tX@E&_Pf8M!4XZ@K{2|oU#`<#C1yCMnw!O0<4@WVNIfz9d--E*JwJ&m#C zFC@5VvFIYkTJ&tNyG8d{JE7|A8uv^yfbWVE5BH0bF_XY1D0cAfPyJh8tNdx}Z4~nQ zEwoBh>Y`_y#6iP~scMURX+Z6@q}^-=O< zS_AmyS-8sq0|YQFLTNvHGD2DFC#8Ni&OCjDaJ-STO?EDtZZxNr8NI>Om7AF^&1&57 z=te6fbjPc#rrpfx0EssXmJf5QToRd~rtn*3ip#F1YtVqWWXzo*)(?O{?j>0fOfMpY zJrJ;`Kb^?{09QM@gs}-azppsT0P>dI12s+ZN$>O`UzDt7ZqmTZ3B8_kwFB)|RE>`FWDyLs zm3o2_eybH6SSe)7q{hBEwghqlest6s&fXa-?mV2CP7_R-BQjCjiD5{zFvbHF&pS-3 z3kbuI{D;Tz-EObQPZX@JQfrdgYMaq@aUJg>$UOeolUdiUa|yG+V%0~%Kkwf+OH*Bn zhu^$)v#KuO?@Fm5Y=KR}|1}t}s5Fhq8$=U*D;tQHN^1nr+uydrg}YN%F4))e?=;8n z%JwVOA#4SCC1`HywkWtvBv4;e0^}uOK2=hDCNd6uAq`d~qi>1Il0+C{M`VU`^f(Du zh~1kaDML zYxW~19ST0el?zJ+$2dEfQ9PJ#zZYATZ%U5kj{I@5xfLF}F!B z?*#VDu_H|TCwa;V9_p_|)I{skYx=o;x5F{IZWNh(arqy`o3>KcMX?C`nAU({-0dQ4 zSOz}AOlS~nIgbN>w~Vg$brDLI_Fz)qq)V*sTFYbhTL)7STjNM#A`9j0bJuf3jwNxm zcUxK0{bW1HZZO>#6@*CXDzY%he5eH?bNEqw(`Nn?to#$6!md!eJd9nxdMy05ZvRl9 zq+)A3WrWM41)lqbK-HO$&2cg-%!3oqn{@nHVzsuLFp|-U)Nk*WH8)fXKLSxmP8&F&1t@I7XE<4($*^VqX?wL+ zw>I1~HuCF4)WX8O`*LCKKl;Cj>VF5k5Zm)I)EP{vCn)&^}1QNLUmVA_~+orI;CQ^2uJrIX3a)MKz9 zi)0Go)2nvdz_(njoQXYbq$s9hmI$c;y_q@+3jm^*`uWS#e9-KyH^-rQ} zq=i*RSiksS3mErUOvcLX$P>_%*Sji4jTBoel{{XaFVt}Cl>wpiX$<~e9hD#f|7fAX z)Ub=OheCOrQfKWGTzu%E?Jb4N<01Rbk{6vR%80rq@tr7Z;jF&Z=^xU_5=-KMEWJlG zMo3Y@Na3f>9j4rREg4tSNJ=;&hRO=OSS>6Xp~QT`udI-Xxq=4YC|`C`dC^iwUK*0A zJ(=9!&FmwsM@wz2K=>5SjGEa!fWh~Y%qa9RuDFAw`HxhDRMQ9e_9+)t z=ewqB@g1LZRGk_p)s|vm(*jIU=x1an4PHIfv^86uBUC2PnPT!pUUMV{;+LD*_O5`s zlI7-Jb5pY+Z~Q6yl89x74tkUsxTqWn zy}GfvXceUim@w5)Wa>G3dBQdY2u|0JUTu@Pvp%mbn3Qg2Y&ki)9UCle5#yXeu>bo# zuVoPpjb+^}DOVAe_J&%#+a!^t5yB@y(&?i0(BI$7ZLA0wVsa%QaJR-ssDvK-rv--! zO#vzm;xjX9PfkNq8>z2-U8XywpC#il6fFrtc;};Cll~gO?nLG08>T(TQ1- zsL|r_<4pmStc5Dw<4P2sfUb&b3D-a~7vxwdAYDu%1qaydPK%P+8FqCsTVIAKJyWj5=}(C+~G3g?*IJ5 z+29+^k?*t~_RH`;M^V9+W5&10X$W;)*X*@#UB;>zcUhzyV zTM7(VRjgr@k_8QH~pUaG>XCrAWAuo zVz!PgCmtkjbhR02@00rHtUn3j<}e&%ow3Y`ta|^5ur)ENXjM(vPB0&m;)hBJ(*GLH ziq#)<-+d4WuzBWH0`G=Z>{zEbx=+6IZa~9IXOjF^5~)6^v{9_HfW5XSmbqeX?8(3= zjTp^y_9hd0M1yZQrS#(t1BKq9VlcZjLVO0a`3Eqq28LSstC@A-b2M&KT zWB%edK-e@s?PqL>7>_lbjAv0)Wi#Pjbm-t(LmIzkUyQFoq?^rbQ5!GtSz=_WCWjV{ zv#`lU`B$~kLfr^eB+Xtl@R~9+hbzaTqS5P7q%e(@HxINVs?fR=qdbRJEtQ-z2hl{9P^$i(1dCj&I=Eu<&6p4H{yhT^%t$i>{x1v$+yw zG$lG7VmMkK-`yGnb~{2c$h}vioQM}BI$Wluu9;O=|3Hl$B8+{-Nv!dx>BkffeOP zq+OTLO}c|B?Z<|#>*3sjnb?&kiEbC?5>I{6BcSn=;iEiDrDAG|2s?)@o7hjOMw|3% z?Ynv5d|GNp+MSwkNuuS%3D?Pen+Se!`ffZ|s5?nwz4<0&-(hFzT5p?f>g8O#)#cUd!Kz9+PetZl@k;9MU?GYkCoq*I4wLeSucsJ zyW%MN$@Z2u{x5}S@)b*7dVw-#f3}V)y3ih_u|r0XL>I~fKURL^KJ{L6Jr3a+8)Mk{ zxjDRx388X~s)at73WR z9Kr(-8ID3foTF~93fAc#wXnjsc7_i z#aoFeusDp`5lpioG@ynJAs`=NH64O0u(qKjEP81mZ`%C@~FruVP-VBAs)jmkyN3b=~&>lt=Sj!iv~!*v&5y(Jx6 zZ;brNZ*PtzTBYz<@v)I;f^oth(iVyTT>r13_X1hSJ1d0QO`Bq){9{#@kK4_SI;u0w z6CCka56^szN$=`g`o(T1oAy?lT+DWV>(3A@5HDT|8ys-in;43#=In9gJCKswIv_B!ERUqYZ5Bxa_MJkoSWK6cFAA%9+OvYg=SOu4aPd|j*<+oc|udYtzqF! zQL+78hZWcF0OWcpqy{>@=w+G~QXdE4=JR$TN82tM)EH4wvikTDgJV06MG?r@v63$sWzj_8kst9e;l7}>D>zOI z!@)`_QNn+OIeWY!%igi5yI5_%a`5ilT@baN_{3v9E?j4A1;Sl2HDAC9rS9OqXNos7 z7&iXu6;)SvhQC^-w;;=;^M!J^<;@d8*cJ1=QMq~c3$2VsrH#6I506fUk+)xY3nsrs z1ZQ0@HT(U|Jvn+J_TZgnewe7WtpEI{ZwYVzD`Y5;Y)4i{Qnlrie}qfKIA*2DyVoCa zkesbk8%gcx_P>gJ3n$w7{_$nD0q*uT2fk4@)+$SXN_}=z*;J)%(LKvR_&sZX$8und zYd`Mxa-p7};uU>W4AT$6i8IX$ZfC2t1g4H2HivBWJk!@%xHL>?w(LLAnLSV4{5=d# z1Zy{jYZ!G(Q?`Z$#lnKIGcJWg;#TP5-o|@BakAii4;yupkQ}e!WhoPGN*O6rx+*JS z)r~IH7{9VuSkDeQGpl)m6+vy@lEh{DS4Rn2PoDR<&7l{`Cnf5lTilpTMx~Wa%CJ71)Qc0eZ$c=?X3(H^) zViy@Kjct=QD5jk8cjEZ~=k=@F^sJ+4uxkxR>$@!`y?F3~+xjEylLMSxx2^}3{ zTx5MUA=zv>XK;j?!do3)&T7uVmKY7u5~;tNX7!K1vrZY8r6&m$wdiX&Hw{}nKob6t zoY9rPA&13^lGC$*-p>SW)IlEXGx> zTQA)zoZV@*HRzI>KA2<1?;m+cSr`9AyWzKs&U_+J;xg=>%#}u2LBD{1lFVssUCkd; zj~_bRa%D$%{@JxifoA6BA!q&f>ABAd%IEy|=cI8b&04~oP+v{qcx(fndhFJ1|E0`T zN-3_La7x*E`TN&N*0yeDPtZXEZgOZwvT`9BWelLVL`O9<8lGvz!OP8O*QeW z=S9wEfCI+Ut48QlQS6TIAyEX+>gj^%WzZ#cy87MF0rW3?p=U_c)BBOfeP_dLFzVbR5i(zg~_$21< zgBD&(h9GF`4>;Gt7<^Vv!W)`^%Kluu8D#xmrLCb%+GwP>218^NZKS1BU+Ou6JUD3X zF;Z8lCQmG@7`_;E2&*+~3^PQdi47mY6VoMFsV$Vt|ADO-Rn60fI9{gkKVhL+UQ-^c zt4&rNvD-Gfs04hM&hgX@n8$XkOO1PzQ=!q*h+}?k(#YUbvfZH7tR(X~i2*x%d9#(E zJ32F-M9Ial(xf|%n?#l;riQaUWs1Gz@5TfN(~y1KXt8_fpQ4PjACC-fu z)C_!@?m+TkZ+!VK=5^rviBfV%*dIRVxZfpf0kWSgTZ(b>+(hpKw>t zmHzuf_S%7^WO&v=QN`Bg6U&9Y$NMDbD;|eI;d=AtX)xQAgyt6XoM~94`U;YN^hM!p zrO8TutalO6Qm*N->K>;B_BfY%$ZyXL1-`Te);D2}Zw(tLS#|yZl%@Y+JBEi%Bx=X& z&8v*C3cWTHK(fW26;ux|-v0q7annbT+7`24OZ%PV6!&Z+_~%Ia9(hH#V0^32Wm-=_}7 z@EA8KcTZ>9skdzlkqz5;jM>t4KCT?$w z;9f$JBnEFOi2p~cL}kc$d=sT9jmiNLL639`fV-o)#vpKMOU&Vos?dH>liKp~vzD4~X>uDFUYJv4^QuKi~#E(`e5Hs(7Y-L;hf_5SSBthDc28{=Ng5a?hjQ-he9Ebe7k z7R7P5c9Z|Coi`yebFrc_xyeG1mwx0-Am%psrkZl2`;z$?A1CaH^D}Ez4u-MXsRts; ztrP3zcu$)_m*Un`)wFY90HsvP5fOZw?Jgc1O#F%0R(AOSLJPRHFI&Tfgh6V7&10$_mnCOZ2HK+o? zI&;&M!@;mMIa9;kE`kj_KsME9#eLdm97z^V^Qs*;zSc6M^Sm!22vGm+`qtJFE-mqL z{AtzCXXLHp^wnxAn1U==2M>RLYn<*%>S~%ix`*Y=^Pz%UP`E9#Ron}1fj9Mk--4eE z-#?nkH;)dSdUOUn3?qCxHl(9-wBGD<^iV@Vovk1CZa;}C!xJ!5G~x*2v9;`;^g(%U zSF2}T>ozekB4`1OtK;P8oe#&3g?8z{LMO-ys(Unu0)SI590iBIPgY9H1ow2?yasZ9 z8u%p4PAD`;x+3ln^6~~~hI5iYPy`1sV&1Inc(fG0PiZyQ9XA5v7UxU2m~!r9$*(yQ zYN=R{TBGOJn*^PNBY{dAz$rvdaMLfU`DTVP+jVIXp4GP^yL0T(P4u;xZzc{8+&=D6 z=hI_*cFO0|HFodWZx+C!hnFhvF~L36;s5;KNcgrp~A5 zik>YCL7g-x7xG=pv5Iv_PN!>>P7ZHv`sh2P??L$^nD z>Vi4bH5=h1;o^^H2^$}tp({l6RGap0Qs>`1YtD9wUIcn2ev*2FD~-rHn>`@A$735Q zd;L*2lkvVA+}72bbyk(=29d|#$;w&xLqf;vp5*BQz^ZK4<&G{TR0$Tiz4Q*!J0ZIK z;)4AXZf^5|Sb>Z_A$UbE-I{8SXKt+HBPyvth@e8}4L3_~F!DCj!Yx_M$}@dp^@})# z*LskiF$CR8(hZ*$n%6~Ylg?Y1)Ykg<%lE~l(NT+IVk-;kiZ^O}(vMS(7XH*$N%%Z| zw$t;OLzOf^C+vf?sgS`tVHBP(`__Q}30UD1ZmK>~%NBY$M^L}=I;k{^vC}@hwma7I z_TAVw+EbK)ZSEWxLRvF25%}yuP)U<|B{XS7+RC!}W)ZL7jRQ;SdEkY1^E>!ZwnEy8 zH&t6}nXPK(L@gDdk+&K#a7{wAHb)KPjFBynIKGqKe{6x zdRPqXcD(O_umbkG(nZeHv?A`?xn3~VMA4R)e#+ztk61g-<^t#3+LjM%y^(s7xnyus zo6cd%XG#y(^_(9xEk_5O}(uyK}r;{$?BE;o1BSmviNGsURXZ zMHL;7CT&bFn=+PP|1v73%M4Gl5F=gmI%m)&*BlU6dLjc~ z7@}Xp6R;S@R7)IPbzHT7k=VTgiZY6d;pQ|>P*8;uQCM`hA)fPB{C)%DZ($FUN~?Wq zT`cVg0WmhnR#g@Nf}JfsrhePdLgXgk+TXl|Nr~&30Oeg|b=#nX2Msj08Dup}numQ= zS8~OvK6>f-*x+-}TIWaaaZxD_q;vavrQDoR{Q~Y>_*=S$&a%+Z+f93Y{v9BGioNqa zs*$gkQ`z4ziv3AF&!dH2tdJyEF78X6Fx>Fnwocu;CjLbavCw^qXuOH{z$t*=D1Re8 zpG>CM1tEODK&maZ^+5y3l;aM!p7IJs*lCWxRB6oFCe><}k0&KsQrPCY>cYlUyed4F zz3;n9@4TnWvFoc^9q)AUmLxBrl)uiMJdwFh_+z{-4rdIJ2dlTJMm$xlf#NC|A&u$K zEqDOG6gi)iKFjX83wHmW#3{_!u2zk@E8-4gd9@yb5#Ozpx?^TZQ7Ez;x9P(>1eOYd z)5P0%Rc#O@k<^nx--Jh^2f>v0qYe)I`^UAgjt8-`RjNCA+F~ipIOji``I?=J)dn0> z`rFx7#;$=5GXd3Q_z9#mkf(0FPt@k1MT9ZU!5SM$c@t&`w&<;s5! zK80c&*MWFF#_Vy&n<^ZRv%FK~3(*#d1K;rtI041xMyW=lGhpoKw*yx9?JuqyPktaj zI0CGNG~*U5CBBMRHaqs7>1A@J^DasdRy`qo5Vq$aA|t6wJ81x~E<~fE+%rgo>d1O} z)*ru45w)g`j4W2O7dn$~z8)lr%uJrL*H$zSSjb<5+k`rIP5P2fsdbC*%KH7^ZuTjL zNw#TkYeQl$fl#is23t|-#!Z~L0Yp5pdVHvT(LtDR>`nt?ZgKr*!}R|8Ors|5=M&g- z;U%c$ii!Rv+X*JZXiD1TG3ol?mE~utVML4w;v&_{t3k=rMAsXbKFLRBY(}}RTHV;_ zLE=vmx`%4S=WbSCw1`qHIH7Kq7uWS5)R4yuGo{1UgD5%y$TPXfcSY|CDIOR-q8mZH z=O3uq_!hTc*zAF*pkJ@CJ?=Hb1@5qOs0VUqaxL<6)_`2MX~m5Z<42eBi9J#tVE!Q+ zjrRT1jnoU|X4q8|Em{^7x&jM6NfKy6TRpK0Fm1b>@OjcPU`pBL+)XiY!+wNg&#)XU%hi7%jtg~MmD8J6VQ@o47mZqe8lf8f{#GT!)3joZeSB9~UM z9p9#K2N4<3sJ6F7it0*Q6f+&cc!Tb;^^kB^1tokSM9ZxHWpsxFV0 z_&IS%jTD(yl$Fw#eltu?E4kE0h~ovHipavX%fis{TP8*a6WBNOjAW$qWQ!5&w`J^~ z7o&!wzKfzuizM^d?8A79#GUe$tbDpsA8;OONv}6#Z&!qWcfH^ywt$Z?pHcG zi|vm`D!!TVEJ`S>!t-zJDw^U^kCS;hrN|Vm@su?}g+8`8P013B6(nTa3oU|RZWR+b z(zGm4Q0ElOQHE@d*bU;XbFat@%g3|nHu-K7$H2N2gq?MX6s&0mP=zQ|@p22>x+G6V zq(t72xPjMnLFAOx+eyj;t$EAebZo>6S&JRRAT8SZ>^hgvPZL>fyeCY8T7K#n^fv}* zF5=|y37>wQHHKQ(Q47yJ`7wW_y}+%XB^njUGs-Ft{$!-XxHAlICr@ST;TVmhTVKIG zY9WuWZ;^OkxVedJ-d^Q@YPU)~+Is_o&Dpbk-d&d_tJI655}KVJ`JZ zF#8gXR-=^1D}G8E+l?8m2#4aS)Z)*_@foYFz#pSN+dwZ{(Ngho)1(zZk}^hUb+11{ zT|~?)%4kQrGuYSlMK6zOQq;UO;|mqab$nc~hThFsG!0NrF4P8v!M)&1nJ~WYA^7G) zDCD)^60?I-8Cv^7ynMPWkvExIKa0VxnfHw{=`zNlMQA+zt^z+QYOxR8&lD_HBBFbf zVSzXpx#3Sa0i#@lp;0&t`0Ye9K$&`6|HV-4#8(rEmkDz^Y@2_W0)&_AT`sb8W**y? zG3EB>d@Z|R=;+sQo9WKiN~OK2d)29IuO`59@Azjw{LP3=&Tf9KrcB`FC9IKP$R?Pah(##4gTGxt+YP3;eKj+qzRN24! zTuiAhTCg;QLA|Or@c<3C9L<7O@!{KfrhPNFWVUobA!(_aydPSLI?~HB?GeJY+1VOB zR|d~~5gvD#VF)fp3x-uBRa46jF=EwBV6YfxE^U*EBF~MDBNjn!PY3%(GC9pdsV3=r zU>?7?<{7JIn9%CbeaB+4CCLy>o2}9qKN9ZR*??SMPba8lk-XM6OIb;O2kp{&7fe+0 z0>!QIJ*Q9fq+UU94nDl{%wIw-HU>A|Y5_`-!UA0*_t#VF#jKs5tim}F@bj_dr(?t) z|CF)Ae`BJ#rm?`{CjOzvzUhLH->t6VEZxC{5EqX4jLw!;^z|>5Q;Sg25jnBgA1uf6 zUA6N8-m5BVhOcf|FGY+h#J#J`Bx93EC{h`pq7|7e-)mY-)<|dg#1(xKJ(G{|bLUGy zU0QOpN$e}vCSeGpVP#)c|HvcjGhL)8dz1EVi+04tkcn1t_^dqD;GJ(VXGLR$?&IrZ zf@8Th`<=*aHY7U0nGR6QSNWFV45vWrXZp5t!s(wVAKs^K6Z{yo5-2yOiTFIco=Ec zmwLcDmw3%^Z8Fd4{8%lVernKer6q~D84AJ7Nu}017y~V_e*(a5Hk$2)3!Br%PnASW4Ke!+0p?xUh4? z5*o61>5bec&uhs0^Jv@otEGxXjfPNpo2{8&r#&AD$nWQTU|!U3a*zy?&yDrOl+#qS zo!2sKQ$yYvMs!DfR>#=Q#L}cyINtS#ktKB)ekU$~;DHPBSzu9;khE)Fz^;vxay&2u z5^quQSGh(3f3^Or`rAT|pa&qA$6Sf#UYB`X-^vw=;v@E70U0Q?awv8uV+F0Vg6E0Q zQ~R#tU_tqUY)cJmm9;^B(g|HIrswCRF49+@4ydD43q`a8+&-woZ904q|8dp%^gyla zr2w}gO_?f7tjJ1z0X?z22WZh$t`6~;h3WH>D~+#tO{$e#(+y!vg;*XyF*(?)mqHM3 zZi>o`PVC-~zgz?{{U(Tq{(KjK6QA|U` z|Ha;0Mpd~k?80;--Kl_dE8Pgl1Zi=CASEp+QqtWBOc0bt1O#c2l5P+X1f;u_?)>g& z?fstj-TQ3!H@-h-oNtW%XZ=`%HP@W?b3b=n$%oY)%5D*eNf5(nzx3p&wCz?T9opq) z0pre@gk6%j{_#7Oyh*{d0{cSCt8jGDq|d537A+s;ni79>;W01mbDSl78(x!Cxr$7@ zW9#V!PEloc6U93d2&I+DDYj0{^gC<>@r7Tw##_BFTLtXWgzzFi-BlTKX@ZixI9dlm zgi!Wa<|%V-bnp}bf^3-}&DLBOCET+1vJgQ5@E-5y-x#s%h$d?;Lp5}oC8}OoeywO? z9+E_JchEj|tc5~SyA;lcqg2Rm4jtwOnZimtK@R*{N>Cct6SqHW66J%VoB5l$cudQGDPHSm> zi_9K-omBU>t6t3*ydSt#MQ=4WW0qf6;kG74v@QKs0DH=_vzZ9>r?!U2DT7`gxk6yk zZnG5}-Fv0~4i;(Y z6(7X4BJY1_UD<|=D)zV0qRX1ghW1c`mQTFwi6Z|$KGGm2}{+R#M5Pumd_lX zH#LD3mYo;BdXxQ5BuCT6#PpK;e*?+?4xIyfq>%haMb~;i-jKm$`s2Y9Ehnl7I6@%{N&2%wO3437r3jkC`F{A5*Qj6ZmgF#t-J)_b*}o zzkgCCH*(^#-TNb{nt#Ow{QZx>7s30#;*0c;a$!ML7Z%E&HV}XOSwmxJpulhZ-r(Fl z*gl95E49vir}57;$A6FHZ&l;f{**@tr*%)Q-xUdD{ec&O{sRpM6po&)M?X&g^}PD) z-)35a2j}pg5Bh%{qY`;tZ-ApL0Ivow)d-ODlK^xjA$yI}@B*ysn8hUu*WgBqSzMT> z1M8}^9+6X0Qj&DAHu&@NsY~;N?epA`I_MW{<0vd51~npmr62_`I4|9kgMq2{H^g{A zacz5oK9{GLqOQG$7od?QB=D9P8%XtId?xO)KIC_&E;aLm6i~#q0#^-vi8!d#wA3DG z0vn$R1hRmbOe{cHu~QgIJYlB;A%nJF+f`%woOrY%e5%QOS*EEN|2?<<`MVCWq3OiC z-DU+D^?QyW1S|+>BZa1cRZCM-3b-S@q5w^V1F7TOA$X+5X$fVG9+({68;7d^jA4BW zfHbm_6;H$z03DU3woS#~f;7CE>+>zth%K;rOG*INW&yZWdNI8L$J7egt=7Yhk)<*l zuBN_R)W{=n<6@1X6%~*GBdxjN+ltMKsQNU>+DZ54*8^WhhQ)(I0GZ_gfCjBk8ut^2 z^3*d*fiW`(M1|e$45LW|zGg96a1=3t_5{#SWb+f0uC%@yH>4PRTc3mkUDQQDC-Y*J zK@*6641oZdXHPe`KsyjNCat4PLp9Kynm`+w^}~fZklx`4Y`-B1YGUkkpK7?@#z4?q z6rH%x3((uXcUG3K6i?h}y8Q1(xesl1bLB$#=r;(DTxn+QS?)>5$xfXsX*{=)XaF5M z9o$C1acod2V%aoZl{yDB@joA=@S7j#c?)3Z0beNe?)6>I04>0niD6t_<=6e+7i?8k1t>B z$;V8QlWqVznVZC`DZ4KUH>>Q^fUc#zC4IRI8DoPAwDUf;Xk zP)^t%kD(I0s~?_++A$nAfKMaDwc9dx4j=^5;=0XGHGEcLJGy^nb#tJtt|QgN>6`b- zj4&oZjG157G|31Eq%BDO#%u0ZD%sKQEm#NBH%ojy-M_m&G7 zlwCE=JtrRGs3oajCjAD^}{d$#Px)oBc2uO@BC^R^h(AqK1xJ z9+Z!xnG$`+<9F5gz#L94e`5>qcz4&lS@lCkHT6`Cq9kfOt}5wpOf068_VL2&y6^Rd zOc#E;P9R)5!pjZ_FPuy?$Oj)(1FOsWHS_iQ@{?K@uCjJwWa$+Us_yF!*t|u-#ba|2 zD7(2lqrX#bJ@RqIyY0*D$k_u!mR(@X3`KHWA>DGbrt32C7Ws$QUII_;Q!6z0aO?01V2DLcG5~a_?dgHi?Cb{lFKmE|7#fYTmna+&rLe>Co?5u@)mh+~adcqu5e z$lq&wobdkJ@ZPWr?@C+!$R(Q3X&b{pd>4Du%ShR-(45BJ+QZ&ZQLuQ3vpwyAM!T2( zl`9N)YC6*abLU~MB=&L?fvM>7%-cl$UV~$?lloAkBjTGUle5asIE69OTy2*(XH#?6 zu1W)vMqcmOX+Nx-=8VYZPWnhu-pNl;H23Yo#@T(ESVwDr)_mg9_DGmHXfEJ#Ldn(xD z)0@|=ZyH;TZHGXd)ZVWXgH$C+?m`UNTWB`Fm0n%N{3b)KJL4^{z*8ZY0(-sg>vCPJfP~zR#gAzfAquWV*)5$w`Gg;g?+t%msM3H_D!S z9am&TlaolgwoOM%-JA`EimqA}EdH?e;u`AZ3ILTTEE0gq^G7#?c(G?-P4-G{x|rFK zD>k9C08ZsH0|W3SosKc>lbxe$FWZcK&Q_dm&q!ZwsNc>D3iG*P z*Iro$Zs%A*I99^zFBJGNJ(V@M`aGuesjLVnJw&N6j5h%Phuh;>cqFW9c(oWqTpKWx zX(3*JXltzNHti(I013jPUJJNJ-A^~@0Pky3ked{N%MW?~zrPv>aH8I!zb*FDf?s7H zJ0Py5Zx)2`5?V6v#k7m5TJxfCsSQkW!f98iyDB8SxXta8aIZAoP%edE23{^~8Qpj` zK3lk3*#y!@7G$Kf248?p|NH$6yR**vDpi9sK7$CQV3e!&c5%a&=cbfb8`bv}B9 zHU;*3KHrj@7%`4z3@Mb~$za>MUuyq7i)-1=Gdx$B9nUREFQRzc_MA?DZ-V&%D&)l&h3u$7Y1c$N;0hQ<1R&W%Tb;q1`xkg|&4TqtTpFNTTfiH|}8x7eK z9|SdE>qWy<&yg2wm@V(qj%zb|4cvK=q>>!pk2$UZN;mbo_3Q_nXBoadG6tJ?Y#@6L z+R{rvcXXUQoNM$()lLG1p2w%@Ce)I}x{U{EJMBl@I2mX35F8@b$LM;TzWw&@Zm9I-ZEB@bA2nU~(9HFA88n4o5T%%B)&%1F_$klc zy;mj!xx)!HLlS45^K>5V=Tbi`!|YPSvbK%bBB{{Ud|NM)+@!at=v;6@w0n;FOV@}s&IaOpEtD$=_hv-ck5aM7gYo^QNr z$Z!M45_e-@*oi(D!Wi(q+A`|$^agc*kvHE=LwHsujqt|yixo>+kP7ek&)yjlfH{*F zJ}C85DF?TO;2&)2UDL44dhGePkvM_3lm>poprb2F#QtMlJr-YEF6IpbQXIO|X`Pl` z5Zt1eyAF;S)ZCjIsb-H;UwRXLF;HS9X=##7Q{(!D^Lg78sfjwK8Dyl;CdRUxB(SQ- zMC>2Fr@~^xvU;vzPl?AqHuP}8cQ4YY{fOYx-WSnVn?K&XucoD8WD~$ak4y#IJn!;! zc@WSV_zEdJHgWeA=Jkj0%{{}wQJ#Q1C<>_N%*q4!8q?f5y5qz`9gmBDl{Mcd4`-7B zr;c~KU}xjt;NQ69_^sf#5AWwF&`Rm64G*KcPp*Bl#Bf7NR>Af&pQenfgE|SllgMIq zh$!d!IbZw%>%Xv3dQa$bVRUmoE;6-UKX+xH z`aMVhAr|U-0n!bq+a@}d(YESY7~BU(w&6}4WI1vkAl~kbs$}CT<7E~JU~SZlf3pP1 zUV#)1-erTLAP|Scm4v*BMN0kGYzRU5z*%Khs`Z>O@Cfo{K9h@tckgfGWnCEalzj@7F zcguqL_T>)9GfJ~6Qu-=GH^BAyIzjYH`ps9y1H(^O!RdijyJ5mrQ=gzKKc48KBA0#W=C(;(o|B z3(ytPixJ+#(QYf9d`(%*kCjRIyc0Lc5UW>C>@^#9il$#5^7PSAKATW9DGNcdSL4mq z4nl5Y;l&cO{?Ux7&Vy$|+w)g37xAdKL>O1mb#K^u_r))>MXPvZ z5|+uQl#k2jn&qG@o=(;CgN5>zMXt_8mn9{h_*q` z#n>~ceZmG&bKbo_=)V7J~fQ&dw_1o1$*0ZB&oApVW)GZutt}RD?4q zM48|!ToJUS#BDOtk=sk3)Ha<^uEa#+dvz+e_T`TMgG{Xyd7>m{z4W}f2{-0#$x`?1 z=`x`(+ z5sIW@<_cn2uc;CKfj?R|PDY*BSWMVwf4G~2PF{@ebpBHwbnn*e2`?wfB`4E13C&JF z5iC9@ZXQ1Yw=kNkFsM!Yth>z1)WrwHSama~&?#LeSfUp`6e~XERZ2R+s|0s4hh)1EI}+q}rim(u%|YdmxsE_-u+R zI8K~&YRmrm$IQLTidUYnoT+0L4cin>1sqN*j7q&e35g%AStUuaR%KNDUl-VG6+M%;A%sl5E6 zqB9dSa+JS{jsh9HcaNxz9BSmxfFPN6?q$Q)={&_P+1eh&x@r7Jh7tFLzyPLKa@d1E zY^B)dQtYGW`?Rg;PkwihZ2>v3d&o{@EcFSh+mUWBj zY)k3d)nLEEequGI?h2m4ozu&_nC9}-D+jM78kPhd@ZLP(s1%Q1xxMY~C>(nKc7#2- zle|gBea7xt#5wy+B>HP768U;|_A@HuoNph%(>vFzvCN)Re@f+s&o%jOl6q94;7Im9 z;vKJQt&dKXy4gg2(huJ@ewwfrwL@Y689mVSU0zAn!d$RGP{|*vq;#}r4@aPR2FmoE zGGo$5i*YR_FUkhmw}%m(VHmo#=I({Ga)`fTeJxF$sd}Y6mO1gV>eq-PGY@qQ9u9&8 zVb|~+7Tj?ssMF!ehz<|_Z_itW^AL9EV^ztdx>o$<&TDYJi!2r^#ZuhuQ3{o__Ckp& zAE8>750vVGRE>FA70FbAoc%H~psr&gpUsjXQbOzte6Ni0g|2_v^x;s-w>t9qI@8w0 zr>Cd6cxAEkQxe#8s$zrevo6ziFG-p)`6?*$S#g6s6Fl#+=Q$xtpUo&*YVq|WurEYxVreiu z-$E;TW%5um7;I`5KW>BEY&_^7wdBKXRPAJw_4p~=;4`00dX(@kRo`odteF-hv99hX zTmzaSIWO+cQB`)`Cuaz945f8`IL&+f*zSFR9`-)Aqnpva_n6QAMEYv;UX&o{{`1aaWW2s>j~PsNofh~>s-;% z`pkOrNp&9uoytFXR0TLwMv_s(XSh|lv*gt9l}Tau4haoz?}jRsfz*xlp{z0i-Xku! zz(QM`rW19*#N!FQ-)@KN-}#?y2eVig1ml$wtNNVQ@Z#`MH(+iJqsmf5}Q7R?Vm#F$gE=N+BWEP{P31tx{+&Id5p~h7? z;>B>=A}aj(d|@mfTXV{6WGNwX5{0$??O=>8Gu?MxbM(>U%&M$R&kStmm%`nt3RJH1 z^bGKY1+n(PCoK?rJ;-jhWfOY5@b9W``Ob`}C# zZ`mJf*wMN5*nB^q0Q)1LKIXMv2qRbIA2aM<>9tQbM}r`y&ZQ zUM15OO}LQHbXoV^e^iTZ9zkU1qe5b4JUPtHSEEWPzeS7@53;|PM$ zdB8qBfiOM&(uv7D(o;i#d*?QnSnxb*1c6+eGq_OmOEe|K$`)-A70D&Xbm!x3QzrMe z^8b^?2TXn$YiXy&_VvZw^piY-VE?!~|BIF7AY$A3SvNdt=R=?IxyON%?XG%y>dEc? z9YFO&J|gw{Ri@>j?-*k3>lM#u6Vm!Mc^;Kem|Z#kNHlVY5rQnm$Y%PVUkR5PGe@fx z$r5@wnijNR(P?AvKI)T$NBko*3hs=4E0$k7sF;rK+Ianq;PD@00SLO9N1sk30G@dC zWk?mn3rP$zK@1zc)9t|4gG3)egj(}Vr6{v>e82jBh@JHoL9n_4aK%Kw&!+VQhE_TcPxcgHccWrfneYf69YulcbeK%D%Sa?Q9|tR(qgP%}0#VwG8h&y*a_N5gZn&~wBVrhhV=HI4n#esr9sgJh} zuOp2v?@R;p=K|POa7OafW33nK8oe(R88%7os)pfuL)C2HfWsSu<6bmE_Gi}Tw_>Kc zn#Z|Ow0$_MFGvcO2TmkdDz}dadw669ewLsU;Qvx{v$v|2;DMIhHQI99;NV*0ch_F6 zbaWT{?4imP! zQJRrDB&>@496h#KlYJ&-av5T;Y@lwtx>RBejyEv$l0IPpls}C@=5Nrg49<_$eQ9@n z<^huSg$Ku(FrU_8F~E5Ek8D&O(GaW0j0SbwKfZVW>uraA!95xvTlkh__VLLd=dB;w zrXrx-{_{cqr^d**xUS>JH=;-Hg+9OEF+TkB(!~r~kgm>;3YG3j5-W|c^-us3g&ukF z(VwU0L$L2y45Us}VR`P_%zQp%eBr?I5A)$4)hW>m;K7@%gcij8aRp)Yr)LIBHm1pI z&OgmwAmFj$0(uV`PHENXe=`>k8NpnX#F~2gXHV@L88b-o13xpd`FB4O6FZ-INA~9| z?B7gk>PVn*=Jg>BS;k6(IO z0)!eSASv0TH3*w*>E0i|AfvzHZ7@4|?VtLm)FK+5rMeljZKX1u)W4up1~Qwdk?8 z6#w==PW1ozc&(7YqpE*{&6c7BF}VAQB|%c&=Rr`Trvwnh@GDN|8vv`!0^kR3LCNfJ zkQD|@E58Vf0p0E1I9-JX2YrBTK)FF)vT_c@8^R6Xm+Vby83Ak3BawMO)Wxc+ixUEn zkZzY8HZ(fyYF)fQudNASFijxreK1x*G9Ix@#w7f2M)}!Yz9+nM zO$823`@Ni)Qp(`mMoYGG=>t-gmlkr9^f~O$^zCa`vO7dCR*F|HfycV@Zf_KoAc|AY zArC9tdWA`no<#~|KvFPTVB2$QYYjd9l<2(C4D;EYZ&?64`YqmuUAg9XZvX?DY~;Qz z7V>QS^J}{P7@F-t44n2w?W}64lxBdr6ke(Zal>cgP;lr1P*3*VsEG%Ue_XPD4QgXU zAZ_Xkz%A?%Mh0aks|AdLa~m%`I9Gq(tsV5{p^GVgcDe@`&LD8sllE=+IRN%m={}S! zwmM}8U7V4)fd%_jfF4JmNQ0T*M*C7k_I8;Q=bzr}dqQBPdDWev#Nd6ePO(K=wAThr zm<7wHehrYPHL>*xE>iT}-emPcX=afO&ePSI521F|r2@!EbS|vE;0j(|M{s%xRUp@6 zRt4r&x)|7&&(-;3oi{IAAfgMPzlbJR0M>tpNzkz+#kkmm^SM94^_Uyz*>MC_a2>(2 zBOYJ==d}ArtrK-3q`+Fmi?jtQhQHqlMjI>tha0lf#PEGTsK-UoJFiCy-)xx@ZPqOj}d z3yr`V7kq!*KyVt&QjVTT|HI@zEzd)aXor#CgSlE@UZ9AX02HihL1o-7PxgUk zF`Dms&2v$J0!4C+^eJjNmK%tbzK{mN_^lBBkwJB{1{OzOi2lGe-OKya_hSSkWhRhU z?XQ5>DXRAcX%7&Dxk}wdbe>{J18u^R+&feZJggc?Dt^-kr<3E2Ve*mk99Ft{1?%7&#a=^%g9F zg$Zs40)JlhB~>y9p{57T?IgHt&irQuF|l4nD(*ttXs{Bl6|rFBlaUwNBUZtxq8|If z_WW?;d-?kgsn@XG-~qupE6byHX&^6?c}B+b=AEhWya`ZOo}t4>oW6XPXFR#a2YjaM z>G0deA1k5nfadBHsO=MsP_z1osAW}1=4V>72QsdHSw>2+R;011Cz5-wn)W@@45CNK zJSE9LjIp*A;ojW^6|%a?Kt;7VyK21p8yG&NRp!9EI)?{VsGWQFr`3m9O@H6bJ;H+4 zB!4Ng70o!ABpJyIy$Sc38(!p1f6$4_>~&OVgzqlJ#?mm(UwjFi_;8M}t|>pH->v#+ zhVWb;s|+b`n{WZ@W8DanSVrzOd-P=Jx~4l0?xIrooIl!169hS~W0ET$HI#*dk7>q`q&vgd@=@TOcjmeF5!zpnfrYyIJc4 zo^wSfhIzQ2Mt|7iJ5#N}@-9#EiT%)vQV0|4u-_^ZV$}F_U9>Xu6C>Gkg-9>*?uF5# ziw5C0u$|;*%9X<3Nb(WBXMVM5^0!2pSD}*O6zNMqb3UL*fGF)ER`_7PPM2_EbmiwC zAKk%{)0VH9(`qSokmBA1(JUqao4^852|h)I@PqthN5eYIr;|GUt>%oBEru&#!KB6X zi8S0PTV?jU&0u~AX%Rq2_5D_mjjmOGHI=Gv^{HOYubSuTmjj@>{2N8Ks@ad2n&(NM z;oVhDCZz1$-bm={|;XBU<@qO=ODtom&;_Bpf&g84xKq- zH80IoUnk@vAMnyT)c{>-8xRHN1Um{DvPVs=vN(e6v*U;#iay;1?(QHceu5UPOlQtc z6XtJ&!Kg`1gAHZ@6tobqF+fe`*#rjQ3APTpxU#?g5REK{pw1;dR3KSyNn+F}{_80c z^HJ~KcBDz(%&i1?togw<8s}Ft>fN|W4&}ss-j-(WW#}B!xEsv7yoEWCMLK63h`(6> zPN*Nq94Xs!w{X5xRkB;f{sQMRWKJFn!G%^Q0084wJ^=$2>$si;az5M^B(>^8&X+*7 z2R3A?g+za?Al#EWP=Ha@3#4JFEaYC>o1(C-feeXgIJtG9F|*>7eq1xyrRnF8ea}ZS zOrKJ;;YLXz&WW1CB7~^6c1(0~9%d$DRU+b3BjN{q*dzONdqpyG%dqE7Lqn)>>&bEG zV!>N4p&vg8^uwA&K?=Nq;`bIc)dXlz%q@X@V#%}!vO7nuf?S>G6&hP?E?&*bn-yN) za9u(8z!2p_IaFj$%~m1?atfFkVWxa&;s~U0ZN7I~EEO~`s7g8280&%SZZ<9x^;L~P zb`dFw0iE{}&0N?>TxnygO@P_TGWh;6<_Rs-y{5WsZeHsT&DeE&IV4|4Uwo7I(_?3= zBnR3}kqb*8c!`dl{t(p&6o!j_{h(%?Ht)+05W;B|Jo|;crFZODP(2@j+gYwO3Aw68 zSa|eM$m^TB^b=%i&nu?u<2(^U#_t|=OQD9k#1vp^Zu4t|Wok@-^IG;1dxZleMj8v6 zz*i4ONf2ZA)-aCeWfP2p?NClCn03WO|0bE_UZ(l&x^D#vW&CBIqxIbzYiJRmv^B-p zMifJD^R)lTwz|)L0-y78_k;objrygfW649BkV_A6Zi3w{f@B)T7~Q4RN^F=2&Z0qn zS`(SKcJIz~-Q}m6xNe8L(A@a(ROV|58e=I?61=5-oGTd9)WOUaM5y>2Xt#V7 zrK5o2;ufTje6b!2)TcYep`8rd0_!3I(RjI{k%ifBx_U8IkfMjH_(?MA|zA#lY^A23_1om_OQH#BpglaGU^NWukGnkAote}<}8EDofxU} z^-`c4Ra`+;Z!!J`@X`cd#$yJcJl5mF9_jH+lEm19`lIrZWHoR8hnyqR;7V_M1}v-$ zoiWrqs+*FR&$EqlqRjLnIylX#TC$^JCE6?6a07Zm z+rHKg<-4qN+&r1==KhX^L+4N-A^V3mpY>(1KA~lj%2mX&1Xm=aaAU9@!A(B<9%Lg{nQt# zi=kmy?)OqUoioYokP)iT{Qf8vRGjWf_w<*`u>;O{h#(DP@1rXXDm6rOH5+_&(%xbB zcfIRp&O_!(%oUz8(2HO>wq6&!^BQzLwOi-bOf-K=h`|ncY@n47fzsczLkIBepXMXN zSZiss<-<|@U@F$CC~Rmw4i<#Ww+X46lV*$G?R?Z5RF3GdOnwr zt4Uh~L#xz^*I+lG6Wbg+7oBcQ3s;;xa>^`P-H!^ppI8PvepX!i{ClivYgu{jFO%Ev z&Px{J*Vx=gS=!l0XfO=0CHP-F4Y1C z_N{`Pw_jr~Mv07P{J^0>=$2J3XYW@tElN`I%0R0rzS79i8%Ts?`35?}gtb6d@U8sk z3jLX>T1F@x`?1x?uv?RNSAtSvYua070;{bc1%^Kn7EoYZDixTOw>k_zfyIlXlm3D? zSWXES5d;sbT63dt(Z*;Jq6+m#^;i3X*O#-T(3?7n$4cGwg%h|(bodZen7T_dyGbNP z+gOtmboHB4-eKz}!szDznBjWLrKAfl?Z-A%9wuKPB%u3+#nkld0ZX&~GMoFBFW^0YlG|2sRS2+@SpXP)f zk6wGb^tr`n!dzDrygGq5(sN!1E4(}>G0Og0Pg(V_^GJD3Z;2;A_ZvJXOmnpmg`ScY%a(48kkgnU&PgV>98F#Fu2Ej&d>|*%^%UD) zf<|@zAY$xJbA6x?_t^4bU^2_wzw2Z^#ZumD$&aF9KP=oh;j0OS3vC+RbvGKhc_2D) z@m8;=Kx%OIGZ4$Xdc%j2GuIb7(Vt6hJJ7k-h*lQ}u5jC#HmSsH^2R_<+Iun0OgTn? zMSibfFXOTMNNJZ=wyw#&q~73LoH+dCV;a<Rvy4wBsnj>&jc1>HaO3$X3QP;x}l|`y;aWg-Xdj0U659uQ$w*dGb{^o4OXKcRT z>#M1;+(j(bmu5YI!x0$t`0iqXkQYEF%Iqkhq@?nVX4Sk##8TuDxNk;BiY?AHj47H` zod_8GM(cL-DiV0=7u=g{(a*Z_9kk>T86p|$9f3r?j_)qL=ycMEm6}mm>;YpO9<)+1 zR<39ga1WCFDB)#YeDk;3#RM2X|A5%ndl^y3+ zGaPN}NO1x3CdHJ#`snkjv6z>+uPfU;CVB8WM_hYgxo)aLFKWF%cIFgyES5O>Z#a*8 z6pNwc?#CXZ1uAXG*CUXR9DFaP_)QVymH1YnjFpDvc46*(2Y`?N9|8-8zXTSOLTt+w z5heY0rdNpVj)1Q#Gnaj2M?8>VQ$`=)qtDlZ9mUZKyLYwaW5i8@=2c*zS3fLOGL(^b3l3YTh)S6 zII?Ntplceb?7U*R)#99j&u$;a$aJ?5V?oAg9L=ah^F|rA38kDcoMmUOng8U+l zOAntt5x1QOhxKTfW>N2i5+pn9;x?LoEBkKwt(b3X+D8^Ytw;fIWu@t`oR{>!nYb4j z>`#Ii=>Y;6(p}sz7t8;aT+HHf4g+~NtZ6we#OQxL*hs9|j2c9y&vM|77KjeviU~`! zFT3>uJk%<;3H$or+2R75tY}SbY>yI)N!Z@{1`CF+H*o=2#ly{yU0Um_j16D!ytkq1 zCp+yS*M+8s^pqV>Wyi}F|5{^`J*=OYYkG4pj zmpC< zqnZ+VOj*ft04IHOl@uTR7<<|zB+;!?Ky z!$&NqaidlSI@yN$BjeBlC7 zIKb1TGt-gB_0%r*#D!ZSr`kxx zTQeWfu)>BfOS;Tu5YUHDddl@F;iH_uOc(`;lWA=z3bWpJN6^`*8Co)m{-@zp0`<^L z4cW)Gi#4G*`}zg<)!a#wgK?a-AYlb%;=|mCAjb$a#^(AgL_q2RTW-WxU#{0fnx5}v z3f#=NH%#d;$(E}K3%UQAz-4>%pQ=c7{x)F!rR@ZgKu5thQow1BBuopBD`XthvP<(m z7d6y;Bj0+S`Knj`SQ*Y-r(+~eQ#{>ho9@~zpYZBy5e^cx-!R^LpkAf!vc z#izK0AMEZL$4}1K?3qe|Y*NanjvK}IDV#+|DTWU>lujDc^eo? zq6A-VdH9|`prriozsMx^-;epsX*jdGrl-MGpZ^mWCl4A+WGVpEv4Xu|7Mtjq5Nq=B zYEQW}p9<&1G(g&T`U7)|KCKB6^>c8bihbHh$^Y%#p$_CgW-IK2U^()(`9KmCe8rg* zm|w<{W{%=YiQ7ME4P?|wG8wcToIo#IICaL~kkZUofC?+oo)t)hsGvx4!eGtCO*9HN zDCo|hJ9F17cTSZE0%CF;oX7>6Cuv*Ov_&BkWZUjSTMlPZsHWr?-Ig1Q)t{9VW#19zb4F$RBgAq`oi>`(reL6~$j1(nyWw1kif4!8kvY zLevz*`JF09I;p+8#||9^7J!{D#%)1joU!OfjDnkUATdGGsVB%xZT$jtlW40I)ERzCxI91Iz@z;E&WcOM9{9Zh^% z{jCb(qU3V@W2^nYeqj;ZISr;8zwXKj{& zWIDpuNQY#Mk2oI_xz9k7&DDXHj*iZsDj~ybU-@LV`{(Q6W*`Qj(DKbo$8OFKtEq?| z^XR27G|xz>0zBX+mRoH?Jjh#to!{Qis-<}6Y=1*`u@|26dg%_>yx!Tc6EXWxGVFW! zf6uwFq?(DdogX`7r(jqUBe5(lGt_jB)N}Q^n3~(}X3rH*0_g=Y;P%$x@}!~kJ?gR; zHl3hF|N3gZ7qz$A-ejC@jgMz0=CLzxPfY4?F6Q4@&aviu;4U!8Dv?LKb2_u* zLlLX3_R`){3?juYJ=`V(n}3#>nNiJqSAkd{*JT5iZ~OCrwjXHfV`GA1aJbegq(9-m z2O$@-DXPv8Eq-s=!mnK((e;Z}_$2_c zu1$(OGCQ`mWq&!bHC-a@pGx$w3&5syH#fmY?SKoU+NwJA9r#?ReS=16mWKs$Q<%=& zTzeU+8&2YYMA5PO0Eh|hU6~SUaqV((ii_L_Tq?iAIR`rp^VAB zXFndh&3af0onK0go9`&R-%K`(+33bU?BoN=00*T3t8O1yI^6ab99Z3@T*tL_ri3?( zFTQv-tXI2_YcV{(AnHhee+)^Xp8P^|O@_&Q7BhG0Og!wJA!5}V)=FAF{V@yJe7RzN z(qiC;a0JHt5Chc0aZIkp!(85AU|-7X8r}n84nkZYc1Y4y+WrvCSE+t}B^9zfo`bgE z-;`O?NOeJwlHaI1aWAvI6P)dS-YFheLvIvvuJ>PA7yiLM9_lL?4^jTgx{&be<86{i ziiM&s%A)3B;VJA;&IqDy`n9xTni1vr;Dez(nOq0yYKs1C_lkj zs`kKM#!$@ODgnx|+c(#uyUnNvD~kZW!-dkMAt7fQ2=$={l;#2y3r*%d4;nch^bu|v zv}&Gc6h|uOmUq1)hn!kmKn+2qEQ;Qso8MtveadgKR(9MACCR+8r5)U_tU}c4@f<1{ zVuKC^&hOtCD_G)mN=jo#d5%V*j++{_0bmy=y9G2PpS%4=kXoa3r?FO z;#;%oN^ITwe+eZ_X(H~Yz41rI>^ucp$0V(uLw6G5QYoue)vYY%@nPZ3Eas|dyN{|V z#HGw#ReaA#NUnGG<|LgBeFJpveh?eg5S=?O*^B zEhs4$Bu7Egmq<<5ohD_g=`5Xj2|5i#s0&7fGH@p!)#?d5Nura}4(M6uwmJ17(J+V& zL)xEI9%&UM-+T9wNg3H5K-vfRlhq;_Qzqfs2iiZDh z5TsxMkNxFt@*J{RPLRGEu{8>bJNeoE%C(q5EeELdRIG74Zqg9uM9b$LpRGfJowe?0d5r$kWv=O@d>A7&+UMa5~Khe z1`aaPr(w;~h(2M{JZskGgI9pWwPl7dZ*p_BB+oOFTDn=YAGK`UVYVQoM>8%CKVgRV z$?>kQ55UUWYczhC1xFB*>CRqpF!A-mwMVpV_ScNU_;wQqu|cTnU}?OT0ID7J4vry z6CLyzc5k_~+zFHzug3IPp+qTu`Y)54XR|o+dFEEl$}IkT0I1GpF4s? zR(~kezTwi8RwS4M*xo_tYHJzdUg;Wvb8ayfoZ-jllFev;k}XXEXXT}45eN@KzCaJy zZUQ|M4=g1WbF@M;K$BLPVe|T^9DA;>$(%8d#PB0e8ag1pm<(uMv~&8d;QdH~j`HKT zp@wH)J-IjgQX0|T5jGEbgSR+np)Is4#55r-g#ijIJbafWE3*=x2xRTI2|tE*u0ZA` z8qQyNmiO-yCi7vpJ>D;E=~FC_MxjL8@b41GZGk^DBVfACbPG$A4|u+6aK3D)FQ0wn z!ZSP(HY6JEpF~99o=n^uT%Jzt89css(n{ohbu_-To*+0N!n7&p9=Kwv*@fZa8Onzf zbIVwJixc74z`Km=lbdyEFrbSl8*-=_>R1TnQE5 zoZJ%w$%NCytQqP|v5q<%cM~^Bqnu$eY9HEs5gJfhMx6?~vdIQtsi45^!nsWCC9lkT zfPsy>hWg@tm#Z$r#f0I}O=rLe7*^;WEcFytL z-<+~=>ke+CXEz~)Bxb@&_rV-otP3ut0fkH|y zQ%EVcOC!;BxLW(Z>pAsxLS@F~1S45~oiX+S6Ek$@{;QT~vRw_}9dqUn?h+4oC;al_~uN=MD3v}{i z9MooieNe!J2^ru_4JoXvvfsTNFW@~T#PPckx}~km_VkA(%&R6TGwq|k_5>? z86X)XN*LsvC1+3+5s3ndNR%ufS;<*45>+IlfG9A?AW=kAL~>9vs2~XU?XEq~o@=hF zr*74$^XKk=Ryn8%bSj#BN|pS% zON2MD2<(b~a7r4)&e8Xlv{gA<6wbccQ)AR$wJck%{!U3#3M|Mr^{KTLE}Bi(+Pzw|dYmho9IKo6UfA^Rc?dl|h!{aOY@TOH0<>-r8U)&@2UBo{kkw)|FJwN?kk)e# zHG8VylR$lvi!ZcObNbwP&U+*86;52SK@p-(`)}vi<~@!xD7zeJ+?mC)i0qLH+ES_K z=rwg$zF$D%cAMABJI&}VpqpEM4h zRGaMyri*uZ3IZb?_53&jT*^SNKZ!@*t1-YFH67fpq>O4zOMd(`=!=*PC$_AqgLerz zJ`X)>TR7=GMIqbSi(!kL7SqTbW9FL^!<1Gxx@-q5FOgOkm-tOTM_{d=RwJK^>QJ?7 zdc4aq%QlVU45+^G02D|90RaL2`&v|%3{Qt z5&^H;)hyvsTA}D|^C;k4+3cH=8P^C>kY^%jeGW#pd^umvzHrXR$4Q%`t?SwlOcjVMKSw)b)~ z?A6e!rqja^JC74;1fa&Mks5lK>0gJ6ms2Vyfe;;?D8wgSm4AW=691SJg;5dpSHG`) zv@!3XxsCv2VkU)jaw3P8@{C$8rS}#Xy#vdpO=+yj8oH^(Y9OU}+VzG?ki{+f5PurEyFhwuu z!i8@_*MvqBg-i%73P(R)j2o?Qt02?OJ8dTr%+A2Pu8#?}JeX}`pn$8yazd3^lc{&3 zDF4z8xd*YvnTv*JPH;YJ?`urQ#(5Fq8zfmRE7R18mgKS#$v?kjZm%JWadq6fD&ODnf6jVVcq4hMxIA9Ni|1uh3*5?%& zo=toj#0B_P(!E53K{@qCT{GBsA~dwOO$5l-`1FGoqYN2T?6vU*>3Fl}iryvG%03IA zY-fq&YA_@=|EZELE>|XN&)CZATtQ{?zm_UoGv) zg@QxE(Fd>caq|(>L>*U)h_Zc$U)mno$@8PGCsWP)24?rr3daU344Mm_E!)X&WMZTr zoHmt%7?#Rlj$A?h=}a>AOrb>D1$IB32G5-vnD^6`LD=e!XK=;{V!p~=_mQY`fX$N0 zcU9hpHGSx!0F2C~Q|CMznEvoF({waQFU&7qjH z8l;)4a~VX8b2xot4Q)Uq9aE*Z!{IzeTbJt z{G?SV5NxibEThUERHtt72&)LTokR@K-mf6zMQvm0hE?-bIDuWxju3yVHH0AIl5@jp zJ*}j#om>m_%T$3GQ4|_!(`OtLJ7E_otw*9MUDzemjTBJ81c}7S?r61~DO(+3E4R@UWDYUcZ?i++saKm%DOV;YM_u5qI z>qxi`0*V%vomZOrDLK`+xsZoM#C4Q^VMH|-0dd3IsMiBYapoaRj{KwPE($Py|9wo; z!2~D#(!6e+RPx<>0EpVG zY#A;0!$g51PbM1nr&A&1@%q%z#XgC7m1gARJ@=&5S$>ifFPtDSWtdzYWC4=E9$)b9_ z@~*~S)_%0qa2n!0uX2yT^>o%ox~6`%H7E4C96T#qK~^!&btLNMOBex!A>5ALrDe&q ze;9fGb}XciW5>+#dyj+~N-gDuV36iwd=eA|&^Ra+9EAs96vbx_;oj+k?ZG%xD}_ZC zBnY{)q-SL?(@Av&!J;p{t+?V1P7PfA4VZWn%p}fI#+Jd~p~;tbko+@Z9zkX-uQJ5= z7NWM(Yr|2W5fc%0z(BT)gt^y8&ta@jFZ{lHG;04oSjZTzSdz8ral7Gpwursn()+sW z4Js$c)1upnS9aIK!!+H2smhmPo_V9|lenBVmzVYeypMcQ2D?61@hV`OS#zhW*FC&M>k2m)+QP_@!cpK7%FD( zSdRUchmk=k#Oc#-kP-iHV()*GV36}*^c0)`U%$1#s`%53qJz}7GV(=PfA{%dy(yKz zq3`*!8g>85pI$@;sozU;6N zOy;ET{hm<65}ZMKq3J3Q|4b-=D;gpzEVusjI}qbsetzr8S>Y#%Y9CQO&pp44@hUb& z{^3~GMz=KiY@4Mj^5xv9PdOjDTJ+D~?QgCLxJ$qt>}5_w!(aD}zy0T5{|r9Q0pIF` zl^@R^j=aMa_?@3sELu7L>FrJsIdI%OJ8tnOj>UUgIEPlh_@tlu)7zbe^Wzfb)3iSw zjc;dRE3xm(w4n9}Y=&6^ocsUhekS}%1&DrcKn;3Dt52HhRX>yt0qphuv^W?gDolQdwD`#%-&Rj?>Sc{PWib8ws`B@>G4BNh zoI`iGHX*ne3XwXt378^IgP0i?z6HkRIb#N()h~@bx#SIZzP8=XS={{`s4cEOz*8dxB|0=0)jP^rR)NU_MR#`$sk z_H%RjjSI{$HAVq*V;7>p^f8CfJaYA7#Sj0h=D$CqhsfYkzS*kM((V8GVM+KK(<^cw zpGSERT*EXdc~Lx6YV$~j7DmeE7Z92f0YFDT7%DNx`9c9T0v|jOwkag= z45TLeZ1gSoH7R4IKbLwt$1%xa5X#bad~6`p9$*lu5Xb^ewGZFfHTe4jzyP7lTftjI ze~(j_ON`#==Ul`7{YvmY9Y(4CKO0{j;Gc4$*yXq{>S!dIyzK{v*max!dKqP!{}afO z56@86bEZ@K0G>%=cwQbI?z@6#Uwffq z2yUYr@MY%$OHwfIufNsG!X?p$5Z~ao`>Z}nQyHzp)q33@uo0xp{u@dcAb9+G0rfeI znK|&lb!ntX`?gAtj3*kyV5bByMKWyBCC=gH3H(FVCQ9| zgCqK8PApXB{KWi{H_cr;HXuw{+&}E_vF`fYtE6*T2=5{c**XQTflOVH=KbE_471}W zAehnbQrrxHvT!1Vl9+`OUG?Jm{XPE$610QzS&L5 z`HbT>5HS{ue;zO%&u+xr^k+x+}(h@vex7KbOeiPRpIH6Q*!kUD7d z9^k52lWcU*UXj%s^+6)5Qtk_(5S0-eX7`7weU#}LF98Cis%AP~g)A!kOOBjhXRQMG zkgFRyF6fR^&*W1l&ck?0@S2Wbrbg?4+g?je0PsVm0n8KiUFTq%E61Cy46g2D4|k1! zyHqRga}33+xkZn_@B5C8>+lzgzJ@5U8rHr=1rhs!=(k8sR9o4_8f2%3nQ9GbL#X2~ zf4r&a41fg~*goD2i-omtj9aR|i@Hp;KtfQay+@=O5Z4`Fs16HH8woHYyK)x2J@{yJ z>(>DegI?)_rEn}!+jlLHXlNMiwPx-kDug|0EW2nlGI40M*l-!ApE6zIrbSt9hD>?l zPv~TK%-VmDXd|bFlR~AKmZ1=Md$`S2RPn9xdqM0RMs`jG*vWs(>>us;dIq%T9CBHet`ImY>^L!_r4JcF;BwsN{b}r zt7&Im=#5*Q#qk7s##Z{3wIt&^3yaBlkL2e^Y(8NM?;AhOOhJ_KLlBgu0+E<`F7wYN z^=$;gadq(0D8U>((UXk0tggKS^I12;Pv?_%+y#5~qzbf?ZZQ5-A0Q6AEg(8@V2;Xp z!0ndJw+&JKzP;qvWwLN+UHy2wkm9LZ&%TfTiE$_pg>=-(ctMdC0 z2hRtRK~>M;4m<56_tAj19j8Qz;`@)b_lZfx6eOAzZ+}2i)!xp|cp4L_1Nwav4yk)m z9bAD;d|&m0q|>WYayDThR#1>T^6(nngLdzV8T8Tm(a+{upJ2Ozc=Qnjp!-TvGMQ~| z4^ULKxH>tMuM}11Y^8kJ$sUw3TK2|+9H+wk7scKVutB6nYygkS?-0hWIfCvD!%l_~ z{mfInY+MMie_^lWh)+95n`-jZl7wTRvqg4opPdX#i^##ZXU*Ax^0Lpfj5~3pwAa6z zk9QorM8JmIU^74R7C0J2<_pagE~n*=YMt~-y8Ca?d~h8#?uO;}{vem<8>HU?quu@~ zE3Ld$czLWLwzmEVZqJ#y?)25!M#&aGoy*h+9)5Z*x=5^@H*#wfQ~YUou&uN6`8TNi zXvJBLR2Cms0RobF{ku<$(Ta8Ac21rB?^0;bX2_<7Z*efg-EXInbA@2?qZF>eL9J(8 z2ZAc7gc)D0nfNa+hh6H$;y2}8VZg6x0PSDY#Pu4bLN9aGY#P{i9u&~Kg|Ek7_is<% zY)}h4H1qC4TTda3L#YA>l3^9XY-C3e7!Ai{3MseJ+3|zN&E`+CFo+Y=rBI0^^T}s#WkVXqP%SN_IOgWKF}v)d9uut2gFL9 z2S`Y@3;e-?`oshfoTlNri1M6C<}tQzcn-_gsbsbNCVn;5;zmw>$hE85Pkax`9qk3CMle%k&c#z!NU?kF0}zCGqJ(3N@ATPSzXjECe+973qf zB;Q)`(h`Z%1cz5&vo@CnE7&))chZu?x9mkpW zO4soOHyoL+zUBMYM102l}xV$014R%BzFi zcBdD~I!p3|ZFFb)3)sdSizXf|n8#BQ&jxo0(v9W6di~11ZTn=I5|ekA5Gu!XSZ((Q zsA;2&n1DjPuK7gwD++2D@(?y67nbt*oLp>d59ecS7bWi+vV`cgRV)x$wiM=QGE193j1aoY z2K@+kogTjRu5f}|zw674wU5ucelDGrwb`NGIhk|Hmvk_4FCdXgHIfvww|NHfb!8u8 zdyyKLAGp!v{e63KzF9Ru|6_dU_ccTom3f>B+xuE6>41&E0^Fq4Q#Wc2{l}!pTMy7- ze4wlxbh!%cEl4!eCFL>152}#Mi3$-=F*cr|dwNBu@zyH+7dDdJ(1(S$d&$;InjAR< z)yi)_OL?t?%v~GC*cRvC8>YsjQZM{GfF4B!&d`FBo>*Uh4uV~&N%zh-rE8aLD=z3A zJuuGp67!U2ECV6`uA&NE4-k}?UodWPw49oT*wFEYX`dS%V_$Ar;&>;deK}VYtkhO$ zu+gr%k*9e_g8W4tVN5z=)DmZiB8VH#s^3J-ZC1BR1{t1U4-kmfhAfi^263)91oe{! z>S;DQ+2EXVs#N-ABpE!Wc8uh>Ch~|A6pV$7qsB&Ztb>L$cCj8Rd&Ha&%MX>6Y(+2K zNw840?OLc}x7s{5@D7~$m_yu0<;E(Zb!&$vXL0=pi=tf5e(-mKm`H>OKBt_oI*aQS zM0tmdA$qbO+?q~QG7ds!@h2_9P{LZ84b|%Yj!kr$#46ktr6@GIlAAFC3tym@JDZ1G ze%G?>wNUl9&u{pY^F$K#vN*~_d@0(P-MxnlY=k{OzcAN`asbfA>_y*xvA69uQndQ^ z^wy&l9%|KBE=uAcNI%4|v7KGxA(_{=0!GKG!w2P=Kuz_qoIbv^fWVL11b*7~%NFm3^Uf%p(n;+%3R-VV*7-^Kg# z0lo)O)J9Y}7X>Elel)aH8)e@)5)WU4+$6dG%{W&4=%FzRi|QRq%-WHZnssFE*ztbTDDj34S3#m!><;5(5h?DgndD=!_w6?S%| zyI8MP+!ca#UAM3cOo@w<55o$W#Yz8Z{qWHZ_p#a^;)HGTaU{XKBx;S#J?B|i=+&-+DXo}iIuvVe={;()}woV!7{RdY6C~`^n5Q9DSfY8C%v>! zH!ZaRA7c_$ekq}tnp9rvSxJeemc1APcdF>m8lO>fA5*?MKX6YNPz&{?!sTaY+7K$b z&GBS(=pF%5oFXlJ2>XT(vL_yF_{L2!%Y_PyS?&)cP(MhlK z2!jwzaV*O;1BdQJMpI;;XG;h5!J~96HRnTIRQI0#FghiNv=RGA$|FfxliJ~`j3FJX zfwR#oEzjdwil`H5p$C+`Xubo^z#=?3qI}Zs?$EN^ZQZ5GM@Ea!UmOx9|GMzkea6_j zQfi?M>EB!+rRpxX@^3`xsa?YTiKV|?mORp2>b~TnuNU6n!-1o=g&BD7WLBHkQ&&fH7QAnp$`*NJ3c|0MSMNYGOjLQdIE}bny>1xIX>KiK? z6oSd3IKq$bS#D_j(4O8rC^so-DNQ6j(N(7*!?(odQA|U9_+^yVpYT{qV5XNkRqx4l zVqls4I)Lr+&p^b!dMPB$2)t4c1^U5Uir^{O=U9){BS=qCdxW#Gd>n2J=vQQV&tV6nF5cb>}$mn zR^4pTI2Ifog&EadgeaO-r?Nw9$HpgBgcPal{jIIp=zKn={D=9W)3L+&nnKR@G6}E0 zEu;b+_#Ts%x2#QRk@3Zcv5H%4TxDA4OPi|;6qgQ4aG$(4;E^?zc3U3mP>JdE${!Z> zyZn}qUC~=SR7#y|%wx_a06R>%Ee`FPav<|3M-6nrFO`mu-N-U7Y~uil>a{ z)&%%Z%`b_5jIbdVCs8gm6humY!+Wmznh+MHKlp-fjQXbWW9kAx%lHXJqZ6_$RKnx7 z^WPr7i9N%DE`t`E7Zi&3kAO0FYS!Dz0VAYn6Q;THh%$P{dj{J3)LlI~0PJQu;&M*6 zk?GPY9MK<My?IrO$7p7+gw zNWpJuE`auKgp@#IeX-l^D*$MT8?5~$(wU|lz zJW7#80$q5@{p^%St6#2k(Cm^o|LK1?Pozm3(Mk22x4We&Y%4T>CYN`Fz6dQTcvUof z9C6@ZfT7aAyeE&|DBb;7{#6Oh`?dLrxy+-H=QVJleOBliCM&8zxE_vG-oW*Ea1Y&h z;$!SBo_w|8@8GyT791X>sW-cifZt>INHhpZU7MI`h(?LA=`#^6X+E8`7j^a_$Hu}# zH3##dO4gHnjxH~A+}==_*LIEc>Dk1St95Xo!cpO|_Dr4mG=8@czHzu)Y+HSlRix%( ze9pAY3P! z8BC?ezgrlgo#R`uJ$uD{_630BOg7v>UgZd;8-urrA?(sI&BAqLKjHCG+K$PHilB8n;WQo$?@R=H%ia2=qXm3@Y$)gA@KH&XwznENGk@aZ z3cYs$+MnXsV=+8(f+p#j%_r4wJ3{bJi-hTPuRXph=hJO7YI*0S8%89Lt9-Pootc9{ zh>LSzL?k}3(U@sl1 zmHFYigd(NfSmL|aQ9fN95+p|*2J#@XAI#s+PjeyC z32#FqtnBFE9_;pvs}Agwr2ns$mCOXNtk4SH6Ad1I(;&IxXrwm6-YL|U5;fa1-UW#Z7=DgoYbtPe%L3E5JHdY$KxcKO4RQX#-sv*!({wXOK`NiSW z$j%(OX8*&jLM`qli{T(7Y-6Jg7o7nL_9svJtP$B^xO$LYaO%~nRhZCfKc7(*4Yk%x`a&rlrcV`~YUv`# z85+l`qm?)pp!e!pC{Z~;)~^1`s$#>SV10>Q$E3qWExed=B!uWtEsiN$?@wcRm0w1c zuD=;oh&+`ElL33`9eiS_ z@hHrDVNibG30x}FGq%@(XcS%5r(Gn0NQPjvLp}G01WPxwn*9CUB*|r-Sl=;fxxhTW zzkbiZ>vR8-aiy_95aIXl21}K*s2V7N)tbA>$t(4)vp8xVoMW#BSOWjYq4Iy&p@PRb z#PN#A;{8RbogtNiP3h-PToAY*IjCX!U{I0UdLGGod$WVK>(dM0ug9T>g$R0rd)D>& zpj(CP&azcMpC*&KKz8N39wFV;&ZZ+%<)C=(3@I|ADTe{x?P!&~UjG<^g5!XVk@ zMh_l8vIO*_1g(+wsEh1AVkRGUFEkAyqZ4E*S1ZZoVZzdXqUq$7?!Q)w|2UC?$<*_5 zqUY{;)t4A|Nr1q-n1Sk6%5!asCBAbp?Qay8fBWY(wU+e%-}^y`9u{{0#RB->dqcp9 zB9hF>@mtwY5$CHK*cx8Qp|^iq2LPA97~({^%B`#MTRjbGaAI0!z9vrJ7qK7HKHvWb z6g^mxufHw#bU*U6;aCSH|D&Q^f9?=@`o2)(}j0?5|xqj z+e-mVRiv<(dTy7#D*e-oGT`0JsKw;|aTomi4_A5$F;y)X-Z%Wyi^SmF>Nv0Y{{1ii zymQ?We;{>GhI(YjUPyf6Pu=lMF z5BBPs3LSrMjk++0xOylyVQzm?e8Q!z5;Cr#h~SJgfb7wrota%)u&scvAD0Uoyt(gH zMu_)hF&q{*aj@BO<;Z~lIEb||%d(J3%FT8hF*EycK~f#3a}U<02ugOfP#CpGfRiyr zTVUPCf~V=z|(VYRi$L! zd?^L=PdV(iub>NILs@me~-mCgQ7-JcK*<~Vv67~Q+P72bY?-Fp?#^L$C4QYHa=7{b^LNT#2YxR}z zD@P#VpA~>OT5ZksxCS=3OIaeVUC6d!(d`f^q(HP_Z|8><@CWP+o}-4F6HIcxq3p4e zYzwT6M_^kCLyFe2Kyz?3uIY2z+lA77g5YH}%ja^_D*fX=w_I%JK^<{p*4th{l+u3^IFjHxq zD)5qORm_#U`lN2`_@Kg(fAoo1%3Pr{{u&iVJZa=jbBLxe30m9+SW^jP-#r6g>(FD< z3azB?lE1$sZfLPo)f_qA!*qV^o=@r5lGuR8j` zB@qDvDCf%456-UCNukzI$MSaf(c^M@c~IRdp?{~iA3aK}AP>~Ip(5?HgKyS+pk&<; zR^kqK-pWkD_!kC*W;Q?+eD*7X8Uz7VQW4r%erxThy@{9?H~c|fY(UY6?arYdQ(}@? z+E^67BGQ z-RQULoT4A%-wD$X?-7dtj+pkzZGo151T0rz&)te@9!=m9l00a>OOAtr#6;vnu_{cCuP$R^?EoxhSdia zy}^>u;q>t-IMLm*G-~v7<%27^N^*R!?S3|GrjNV6Hi10!&Pqm*k39EUNByc}5d+tW zB$_+Po(%*bKaxPIz%z5E4gO_74+)l>M|}6`a(=g()G;re;CK*28En$OZK(c!jSe;9 zNh9n1cHdPXg*?t})17mD!p7gK*K9zQ_7xKyZK) zTwdgu5pBWjT(5^qO2kzO3C}mxaq9Fj_YxNC z8jDTm0@P6a>e*@wE|Pg%EQ`Lzo03v1yWX^IWTczgpCY$Vqv2S|;Xe+KqT?K-fgU6{ zvR=Il(8@(GK5iL;dL;1CN6bzjd)953A}=3p-5$By+&@N*V11L0+SxXL`{8UsiQ(u({D2=c zx6CQr7LiJ{9Jkar;l2m`ldq7jopM+}UBIl;MWKo6HDpOdoJ)IdDtibz#0HmTbx`2` zIEX_RXF{fD;083vlz62iIK7uXY0bL6X#+!C5G=>&0miAYE+)S-zq(bx)8m*};St^U z~NAvVdrOB{!BnGoP`Q5V);DjR&b+uulEzA&p>snfOn9Zre(|Mlb4PV)7!_*z7 zRJf{ckGh-95a~2(tr8y0aVZeJ;%nc|Phw#IE(nXEH+ECGI;`PHzB(v@e!UqQOyvSM zK|$t2T&aFTW_CUo}Eivh1KM3_Wy3SaJHbv8gGNHF!Rw3Z8%%Cqwv8=0RDIA7x4 z1|pE!W!f%c%oJ}OyW*mxG~M?4bZL=BUIL+LRszF1f%0vk5lpa&>}guIX|b#m=-c_XjyuE9et8_r3z_cVu~RqSad*$t?I3JVDE4}zb!8$^|ci58y=0L zU;?H{i`lslx0a*q@7VYO(jFI!6K*mjoxpgz@qeb=(o^S~7bPkv7ER=$kExoYo3%{r zMKjGis^Zy={m4V-Y-1*lskuqN@m3!uiloNR=8d$ov5FLzi}=3%9h?mHl=k-__SJQ( z9Wj+iCPp-7A6*4o$)G?<;PA!1=k(9vA2 zHc_SU1he)AH>JuZ!2GGxUARp+3FWUhKeTF5Su4r zAA_Ac{U`lo{iefSy1Kp;xo}+t81oEcxxsZGiu6YeM{3TRBI4L1VS3hOm(lEY%3=|G zI2i~}sH((NBZhUTKW=XU97#>G*E00LFu!gshz>`6<%)6m#+O3P@~YDdLgk#hi6JX4K z;b6cdbIVEo3~Jp&kVQ5t@e&);ru$O(n0z8L9)ps<0}GRE~p zUqTvvlf?$lu$~#F6&j;@PgPQFTUqB+%OHX7rb`IjXl5@X9q>g+V`Xhs!5g`@JJ<5T zZBThrCtZlOICDo_TJ&FPS_Wy63=0UmBg@!!AQwK>NQL2bA~uc z`e4)@tsWw-EDA37_V~M$cP0G@!}~`STM8HiXXmU!(qfDP@wf?4w>DOIX?m$tatuXz zDM@lsCpNRroJPDj6KACg4{X&XD{+e=s{863D^oMZt4w}4-~5EPL1LSX#ptPXHeenmqStlHBa#@JZFktktX?xNgj7@G_>gAb9O(d@4S# z=d$}#q$V$4T<4_k5KW|%=0Xq6+=z^D%vh2X+mRMnKI^r3u;|lF6Dy>|g>h-^KJi&n zR^IivC0-8{KU0eo#@0CG5U)E;)-5q2^;jfVl)GpKad0IIy1d0ThbWCMCp@icXT=kA z$K>sRAL;qy9^WGAW3hSE?y?mhzo|v;( zixil$5vw-B$kB&wm(7?($7;%VoLw5H7$a(qvDe%`lbGFkozQ!C0JsN`~<;%Co!XV`<;;H=*+@F)f&oDnwrgSUbCDpunBZ%7 zj@t>GV~JR<60oHi*nLnk9mI7i=Y@A2izp%5VHSN(3ng&Z5^K=c$cU$CdPek`gQ~R# zCf#D`i3RVCXU~{%LPuClj$1@u?2wt# zz6bX@1e#Z$e!zfWNIi=UmSP1m9nAT1!;BG+i_Cc5N0uC{lOnxmNC`z){vIE3nl(-s?9up4V*@#YjmMg8NYP4O^ zo};HLTeFGPjt((AL708wqK&XLSGF-~IH@wmh2P_W$eq$GxcV$jdTtfFyl|?gDGwG< zKDDOR*(W(_HMtZ`MzYj8-9B@QJKW^;zt*s301 zmw7MeSo_Sb;G5l@?_!Kos4kx)VP2^>i|-z%j5H&Ruxuv>{f*Xg(`(fWPQHkvhr$8H zF4eY2L2ukdVi+xiGo_iQktarEpegju@Vt=6>}rtZft8VdFAuY{cVzyRSC;p63tSj+ zWfu*WU;Alyn2hIf%XWQQjlh~TLj^|?zeI*M@l|tYV+~8CT1z9-@Kh$X#8`~!&Eu*= z65YUd+nGP&cx5`#`%UDgFG!;6H#xoR9*o<{7GejdvE)o>^78A8YSnbo&_DKGFN|$U zwlF(y{EaX;_(68eo7@rCcjI^)8=Vg7iE9hXfnEodcJkFvITFJEXk==+-do~q?u z)(<|qmdCo68B5HpQy6nSLTT7@YJ^z07*i4m7B%`#4ZY~g^fJpK_PF>OoLeRwLm7Amqv^>B_>y;8vyQW$8}qg( zcI%}oVzik@HvMrGJNK+?P#=~_1+)CCMnq-HZx9YGvB%0B#esR1Mx>EKR^n7{ zt)}&1sSHk2MC5+g1cB&D)MTU=qa7uWY0mM!NqGK7wQWyf?}B$;t%6*7#YlEpSrXiG zKCOz%?+S3EWM{&ZBT0&EC|M@(na(0@m`TUBoOQy>ul1Ru-E=542t}l$y`Q0_9O_6H zt(O~D1Mw1R_r8Pa_~=HNn=WICmgNbi*nu9u9|*@J+;#HEqq0QPjgCI1(Y8e9B~Y)NcRvX1xc_T1#-H!vE+IzW4;(D6kSJK z30j>Xe3Bf!+is1d@`?4gy%?$UUMK&!Lq_32u|vmsL$^`w^s}o021G?aWWp=2j|aIh z6*y5OrhQj(qj)$Md*)w?co`+DfEDt&@_B!OS$2R>T$$BLQO`AD8vondc=zqGPVlks6Cn1)MIe9 z?kQ;Avd~W|b#oe#ilWU7t-RAz0nA3}W4^@m$6iFbgd!+a$?p^H1v&Q&bBN^jaye}R zs;(oJL|2hziUlDVR-_#}GdDMHyutag0!}m*^iU|vuq3}(5viDio+93SlfW*-dh0qz zRB)PdMG4jWn6ghztXA3M5c5rmfJ^!S+IE{1alqvxBGf3q{4*4XeI)~)Sy+a6?5Fq{ zWku82eeEBgbE}z*?{L0`ct!4?Kl5{JqH_r8eL;IrhEUyQ!r(#g3;Il+0~x(2_xXjQ zwPX66P3p@h)G1#L^u|tGLya=}a(Nh~_=liqnZx}s0)6@iuRIMMHc@AVstda1;o)>s z0$=)S-C!quw9LUZG`6&u!$!e4mwxI=E;9~nS^H9tN1IvPiP4##BeB-+8m%WtH9cVQDtBuB|mMQOctF?X4F-saw9#d@MBcJNs>* zjJ#Tu9&A_w-ic{$wqaU7LGEuXVLMRglwHhs4>4t#I`b|`$8*16Cj;H2)Xqip zQ+3j*=>a5*?46=*7`oVWl6U4I8w5L+RD#(yLO~);v{r0)V16&JP!_K&0^@bb*Rrm) z)r_PhTv4u8^O%cuX`@M&!@~hp(b6Y zI(V9-3j(=C!Lr14!DHy+Itl6le)?^}szPL>uwD0=CdvdZB^<(pFLm)eqaxge2;FoYTsoVEv} zeEP_S0TH$-Db*1hm0W! zo)n!XX*=BI$bWmLgIe#yNTPR3sHWzfXOb8W(QYI*m*ziglM(aD0|p~%`U#?Za%joT z(Yy#Ueft@;SSC5?z?c8@DVSwqt)`pZV@4PvRIQ zVl^OxnZ0C!()8iNZ4uQCAc#a(b=-f2b<;gGU4P#8^XNl_(Tt|i&R9x%H|w{vL}~wc z(VWxQ96Ig$PS48!ZvL!@L#o~4!YAfkizG&~0!K>tIsZYeVep>1F@l%owjo6u`_O1ih-9)5b>CNHGd(80iz| zq?SagRjj}SK#FjkrEBM?LL~4vVW1ou!Q{&gwZS}S-K#a;^u`^8;+A^*?b| z)Y)Mfmd(DhR=oJ?J~aeg%+}(MeLKy*i6e9P+{$Xi944Atr{7-vu!%qD>rxVp7*btv zbTL^|McFBcLPB3r^WpegB(_%x?wU|&b7pV;0=}K&Na6CqM-ZNbKsFr$xb4AVtuVMT zDq+%Fl)HDDX%$k^anzAyU~>(3wz5YO^N{_W6%MTY=gIuH5xtqBX-E&t~KQ?0CdVMoY=eL6Ak_+DF9*vjY z5q{F1{%zSZUgc5240!ZiU+P)XJ%4v{0|O&k^Z%l&0L~*{KkEh4->Ofb86X~p|3R1h z@6aVDD=uo?lgtN)hQJJ8defiW26q(SmS4V}p!2xUP(p1B#F)C{6oS8(p%roX*@bRS zHl>~(x@!+cjsL-W{NJUZRA8^b3UmgtEJxh;M<8L3ff>Jr2lx^O?u&4_+!ot{Y#g79 z1%FY4Khgp4Dp#R7HBFTNch|Tg&b=&T{bqX!dNv01ClqK`X_arzUkXEXt6a9uRRpm8 z3oI`JN%#Z#^S!WqtWCJARCR^R2mEMvCm^M#5mHT~jvhr=1es3NnzHNQny_e*RDktM z2=t{P&@qKT#d7QE2@<&~;wkAh{Lf`@@!aj|OQ&Z1ntcG6xu0LXetmd0CQsTYBy0+5~KVc#xFJrujfu zeE{5=7GDcIbpr`D0wi`W%laH)J1uXRFkpa;kg2ePq!W>QSIyP_yB~Nr4+PT4Wyc7} znHZ4U<>8OF&1wZlAZuM5dn}(z_fO#4-@ZUCF2Y{fpb!$y_5J+N3IK29+0vfJVXErs z4WoAJyP<{+p!&4Ab}`Z-fz3)N$qYmKHg$4jrPc(2ickim5G!bXfDiuZza$9CLH@6v z$cz|@GI?`8fVAe)PO085TR8g(dK+9oxsE{4>^H96x+uVmzywgh5gUiMnEg3x7{|fx zUlZl`TTPfHSe&!c`_z@@a>3?S{8M=Hnw+{BG^FzfIWWGQXad=#{q&VDWi@&`lJ+kq z8dg3*+Vp}d@Bs929Bp<4xKK+qU1fD+X444ux(b7yp$-otUp~hRs(G3}f@TMQ44nn2 z^FdtukP)W!{rdTw;}+BJ?OS=efc&YTyxmm4NrecKHh0&Xk8Ya_z$Ryp4`gCF7!Ato zeR3??n5dQGhf89m5`f>KaNFf~woWu4d9ArYe+4#gZ2*p=NWFaPU9D8O?{pW0&lw;_ zm-xxT$qqJsx}1a^L%O$XjMC$3<6J~cEqv^n4w3ls%Z+0M+#KQpzavBie?#nEq&b6N z`Jq)QpvYyg5mQd53a$SL7f0-uP#Mkr+3*R|D)-m&sqL&>PQ>9mNJi83Erc&9W;07> z0Z-8EGNQ>9hJ-Br!RO%UE^6kr`o8WWt`rxt>zszsO>eHut(Ebm(>dZaed_PMfcJ(k z-+t*6(rF2svy)u-y8D-ZkDhX<4oa2VP+=0_HYAwzzxA;`ZMBU6ot}C?2ej=1C!M z8boh#k$*?3>HTL$Izt!RhR9Xk~BJKFAA!C!LDs+lVoW z#8em@S14RSuF=L(ds=`U#|ms`)1Xgk?+|_h3)#$xzU)41&}_^k=i*f!OgAEPsa2Ru zl{z`2&r^R3^k{h|u-1GGvP<1W%VP2mlD_ba1`m zzNiO61XGt3&ng_;N+)Lm00YBQPlTk;0@7aSeHoKmtJ*tnQ}^aPaXXI!>@8;lNZ|=T zJm8&EO~092Pq@@~5OK((Pg-8oIfP_&#DcW^-R#Auova%$BZn-gk=N*d5kvS-E6SyQ zu3i|a&JtJe{#nP_f;T65iu;DPB|zm)f1>^@d1K#+w?Udr3~mTd=7x^NS}_%#-DGZm z80nUhB06Y%>yp@)=rVqBE)y|FZ>Rm!8!(vb+=?(!W4Wb}QP9!CrAky7ZlTrauK#=ZIKtMw2 zE~Sy?y>6eq*7K}Aw*T?}@P2y7{=#6XYsP)wSDokYI1VCik`hX^YtSkw^P_8N$(c=C$8=nz5=!u<1XZ0Ale}}u|{vA28tWdp^=n@>|wld3UY8d z?>yXF)#!Khu#eTVxZ93nA0M*p=j6FDj$3T`PhO}z*A6AZj^jYa4mR4{KvxeZ0@Iz~Cl-|lGn3*y(1|V2RH_X2FQDQGh6F6X%8CShw(LMmiT`b)rD9X`04C|o_Gh=(n75;@^ z36`cdQ-Pa^@vS(I&&w%hI-R7%5_uoU&?WlB1s4PbY#23UO2GKKM-bV5yA-6i*tS?l zBoxDG(fAv76s_}T!NVveIwNqqNQa!`9GDkGkggAN6h;wi;D*`=`hazBe_LwCFG~Cb zEYOx)LqQL$HahOfqV}=UMc(aP0Y^!EkzBxOjZK&ith{!}cYec-2HL@viF=CJ^jHYM zJa(Mr5?RX7 znETZD4oo=M4K42!A28WR_@?x>xnZf%4NNYYTUB8M>?P)O^e&ZpcvFb@dy77o9LTqy z9Uk4iRMCaPO7jEUKr2c$Tg20wSd?TnW;Dju2VfRU{E<4EtE}4(;Z#SvpET7W-l7K; zo0gbxX=xvssDlTI@Z|Da!k#Y5|$?Se4KG;Y-%wquOxqFAt9fY0G*XtWpTE(0TPjwI9l;70Vs@@*echx-PtL zThnTHX-T_7rkdwN_hUL26MvX9-*R10K7Tk4po=epZMU0}u2W|Lj}MGxB7T`8JO37M zHM`ujou);LROw@fIH7b`2?&o{^vBZ@KLiG4^K$Dkz^9$6qcMFKL6|>Nm&|XdD^7CdgNa6GYANnCA zoXAgI02_cT9nVPxoFT!`yif@%JcPazR3baz$%|UJ(JJN}CPWxp$k6-L~Y!WZ|rF`zgQV2gnGu(r^!qvdHmMkji_5Dq7_t=u5o) zlLP+2h%16qZb`hWJa?rZ++R7VvgNVwx;x*@S;A~gt)V4iDE3vH~7XI!M^e3__pYP7%yO zc4^W+?(l~;w{ILuGF)bM(pvPbTfuAg%e%@I{D837?7D+E^T~F=ulhB?paf44@>N4_ zQt>?)H#b+9MOyt8;|w5GY9mxI!`vD^Z9}$us>akS8t5;X-UI@qCtn|4m~8xl-jixz zm#c^=A*^Az=Qd8Fd;#gTA;pS%?;xumF(^u@g0gIoBVFQK68^>m0}(gWU8J&bx= zuI(1F5Dq$xC-cj~28MKNTAO=Bgug>y0Ag`nK6PG3p|SJ!56&fRJWf-c?s>dtn~c(E z%qZSZq8!?&G_nChwfPKW4Kd!oO;`MGRsV-RRa5*UX}3#kZ_JzP7tMf3>b@fJMVr@2 zfUpWd5z6LC9T`0kztjDjs*ta1laGyKAlT`hjEFJ1{0^_8APv*AlfME6KeSRduBK3+ z@_zX64ZXkIH{{J;2)Id`MCCu@)tbjBySA~RKPLTt`2fgT``s!)0LzyE1&#AyIc^HB!NawmXAa|@0Ht=_jT#N`zAYQx{xhV&qIg2E*-lJAw1t$9ZY;KapL>=pq0 zny*T2dHV$hED|IS*m5o0=)`#CSo>g}SYMH9?v_$$u0YoU#8%*pgff9kYDFkwOiYkj z8r*{P^=4temWiaQ8GGuR{f)SV@wpB5pfrXZQ?yLfvnX-Kr51p35 zDUl1vyQ+;9Qhzkgf^P`jd}Z;ykLX73l}{-p)CKE#A3kC z4A-&E?fg!B%}_FVH(@!u*ht43I3(^iYczJAp-zVl(RHA91!W+!V%j;xC4(fweeo*- z54=?3vVfJbAg*WD?RPc(;P}HGbB`8~W6(C$cFLtZj_qNWT1){}p6-y$kQ9z*NF_7P zAPut=Q{}`ss2L`tzcDIi{DhA+$a1cF()<81yT(_@f+%cniRtz;ZdNG14-Wz9ThH^Z z3K%Xpo=gY`4WH&#zT9q`(q=s^{n*wPcQGpwwG_$}(n`BN+Z+go969I*pWpkBT$ZW5 zC6-EXJnFd4)FETSGV~3kQzX(>bVDZoZ-78yH}(Z~JQD}3Ud>0Qp=8IjJ(AGQ3q&8Z z1BdJVWc8cRD9J171)UdhUmlWm(a^P`ygWoA4K&(29g~JjTH~GxX8+D=VN4<-ywgCH zXq!V2&vEw52AFe%vTnb7vuZM*ZLp%pA|Nfo-m$^*+mM54JjxXigY0d<>~vAQFVDQ7 zYR5lt96NYP`DON2R2Hcss&eXs*bmMb{7juZx?UgNE>{^%Dsir4B*{qkMR-QWM}trn z64aSgZ$9>IqZN;qxeN>?(*zNrn-@iwT8iwV^twq>(&@mWY#Dg531h68%9o_>Y!xOl zSYW#>y-6BzGobF(6xaULA*}H+PEsf3{*STUo}$Lvs~k4dRHr&>AMUNGY|q`z|DD!C z%Z_{_CgIEBiJ86DLCDVzYyzn67=-h#e+fct2e$BLB0~(M&`Z2bh4g{kX8v<`6XKn0nebrdEdgjN%LrfavD*Z@$AL>3F6(V*+L8oOjLsLgL0vJICj^R!UV#= z=3m${<}m@fa+*wP-g0lKkn@|JNWdBCL}gP|m!#p*kdd=3+dP(^gM=%U?GeAJizSbs zRnld4DSwGuw3?ssvKx8~Ctmv7DjV0*+hemXe-a*yQYwt3r4V}zMwOdm!;HbR4KJye zwMnV>vNXIDIWP$gLmuC@b3fGWq;Bc(J$k&W@zTF3y$nnO?4R zQT#RU=slu0DyN5=lrl+aRm4iX_Likl@ zHJbi}!7=9K+fFzMm}Uw)`c0_AQJC2crTyVIXv1)ppRqgLvVE#YSauY+)wWh#__Obw z{=b4<@|^P$llAk&arXH5k_@zNTqtRIsH7BKqqBx0sUSZTK~a+OIS4d(Swn>n3TMc=*hwP;AOoiJ;#= z{KWRwy7!|^(%diIYp4x7X(drMUH%ESy0s{6D*DC_LqhSmOlUhv`AP^=m%9bTmP;Zy z5rY$1xzaZ|`2BaPG@|utgyAo*S=teNPV-}y6PhA zE6gQ$wNhsRZb2tcrN-e$_2{|wb4olWAu?QflmedRVD%`spj^SO4jrn`RF!v11r!+V zPazt5TXf|u=U3Fbju~|8;snv~PX#V^;+;~IGVI#5B{3sSn&`fq%+Cb`@;&BC!}-)p zXr6fqoU(2Q!j0h9;ny1auKyQr@ln$_{5g%V)xWJZqFyj(k<2_>O23lCI(dVxup*={ zrZH~YX4%$}E5^z!OVuAFM~p5tc^ zkk+KTOntyKub33X6#(RW1{LG=h9?pH+0vpw95cQ~*uE5jL#V@r*?=gO<|6MCQySdG zQ4weu$<_NNhw3}&NU*1#&AybeBSoO}KuB1+i0UYcSP@T2qV4U288U9rhM>lC{mzD_&N`h2q!w#a(XlSYfZ}fv7$_8 zYOzEeAVA2}>e0mM09G269@EBw7*wbo4i?`ma3r0o69vF|a@M1QrJMJQuHk7)^zHn2 ztOHgP^J{ar!)L)>f>OMa*6>=T;zu_deMLg(`ffOUI>X%tfrM!?JakF-&)@iy4u&GLH7 z81L&mn)P!0`Hcp3TUTHM z{X$gAOG87QZuRPe7yFKZ@!?|kwTE1a`C+9I_;qobeD^Z$zoWj z2@i^Huh#8nt)6A=%c$o+m3>gGDbXlqSzH~UAdpjd zfj{)BK|p@Mn6f_i@l{S2?;{G-CS9B;ve&s3Qb!te#h>?lrUzC?b7M?gWMUguuzDy1 zXZK|k=H8hGWfj7@@A`iqOG~^^lK(j2CAq2VIWhLe>Lu_Qb9+O_XZs$p`O&u~RD{;N z;0EL1t0!5jw66~_dg6EdhumVqx7Zz_i^a?hJMp>Y7U5oW0ONXcq3`oqCr^G6L-Jl+ zgPZhnYRB3-T%iGo9PEA?p)Di*)e*Vk(-aIq*&$Bo6WSZB&VL0S>O6&JUub2Q8l9anjWK zbqlS}3{c+Pn4xFiNkzls$PN1I$IuJ3DR^GI-V?nEl{YUB!px=alIh8#VuByR8faL( z+_=d;Ir5?w=;L=EWKXmO2OVqjHzwtT>iasDex4R8%0qZlBQnYGkPhmtu4v-~Q{F)B z#53%E0O$-{qn(~Q70nl5{~%M&T+Q|%7bAe9s)c%i!iA9^!yuX{!Kp3@ncf zJ-<4fD4_TK{pQe*+%gP@lk~kDv4=Ekg2QVqeg!q3m;dXt%b$^OR%sSVPR zt)C}qAz3i?x5aHhs3nA(Qaukd;&FV-Ib5`ph)1r7_OlZ&bS0jRHF@+j%RD3rk_|VK zV_)7OLgxLqY(CAgq$~=>ozDNNQ0gbs&c!9s{GbcfmUO7}oAplMNB3mX# zoxAqQcN9U$Q#+)0zOn?>dnJ$^vfvTO`QP5@um8-}Zh>bdA#9+DJL|L$15}$Jb35@}YIsfr={Po_ypO|d<7Dk_JsS{Awe9^iF!Zi7lWLbHi zfOc9i<0)uKRO-bvLUp$-V3a%$q*uC$DgG>TzkwpR#V!QUXoLfavoK$x{yHb9mEb_~ z#ehpFy98_GMrSs~R_WVvV5!dr43cF9btO+egJH2P0KkYe1{v+uf}}<|Necuk5G*ro<9kRU4YkqrSwYkmUlQfJbAK?g#KQHVRiKFtD# z+mOMlElL1r(&@ZEEBSk5{o~rgF6Z>vW|+CmeFEcTeQmv&19e-|On;=NS+J zozr!LNT>JXj)X5Cw`Pmp19=2)DB8nWLmE5OuU@ z7hwCxTkVO%WBN9?s178)(ts|h@e_I>yHSHzciE!T2ouKhzjcia<2{8`{h$K*#!s_W zU!fE}_qFjRIaiyEa5z!w|Fhd%XX9BkrT#ptZLbG8K z%OQ~Hk-9@|-tdQknJdqX@zHC0=6PpTpHK#y8f1HHjOxbvpR3P59+yL0=%H)EtXh~V zGILaZWK#HtvBWXd1jhV_ii`QNzn`%G{;E~^;MN;=Jh}7V|Kjie75@fN#dD{8ga|NV&n^S5u2(C0hqfKu2Xai1&TM8kk` zyG#BA)DHJFX@OA)6#E8|ArNDSzAXdd`gSig_GbV=0Xa{<*cXy({Q*F2!R9SnU1#vV zu0X|%i8c;QxbdJ4RPE*I=6+%m@Wg}?ETAs*ZuLW0Kc_RG2r! zpf~>>xT0V`13*mIul~>d7*M1Hv^&U3J%N7#i8I$HH@5IVB@19+i`Ly?K1 z$aUcT@WT~+^>CnH7HBlut(z4PNAp1=SDoMYRlHtq}YkWx$Ttd#@F1+9lgFu?5 z4#qr;BE!6|-cX2}+||k8+hGmv!c@Cgl$YM%!BPC+JiS*z=_drJBm@=?>(J+;`k`X5J3@DZb%WR%pk$@sjml4 z>N$x&SsEFvdM>y`U3_Cz_u>+35b~$&vbzz-V-RhzpIN+4B)OLB3T*2UP?8V?Yc?FMHExMc&zH{0+L078TQH;faS+LXeB>K4=wuj^nd<< zsF>egLqp~lL8ucwIEC1NG6D!GeA2_ui`lCltSIY_7iyN!ES+qyU-XL|z|z>A_jW!( zwk(abM*wv_P7Iu1WJh|~&_SMJ{>xxj^+xir7x2ZD$DGC*;HD1Z?iEw~1b4v)1nV>r z(+3>egf4Y5KGpum%Ki5+{+1o&Be*faT16ta^|rkY00y*BsNhKQ?+ic`tNZEp3-Z~J z8xLe3aSSy&%${w7&YD~F@``;3pmSJh-8!xdx`Mj3Q@4FH*f!aJNU^@ zL_|Wix{?e&-RB-cR4sVTt^y zhC_czYVyjhN9v2x_ExW#C1ji;qjB%_3uV<@;EftsIX36C@tTpKSw7&!{d&clXxa&} zOCa8&`l``%>Bj6PlCFkhNCd5pKQ_f(t`2`T#a#JAQI326Vq@ps`#(cUZF@0k&7f4P z6hg5nfy*VB?T00aQ_pSNE2*J;3U+Vk#u0-@8CuM4T%PuCp>gohetID1Y2TSWz z8YOStoZ^>j18z^+aM_oNFCa^b!R=1Aa`;x^c0HV6#rd>m$yuJ#EUCiN74=A}TS=15 zeKBiE&W+D>%pIUfeDF`nlCt~4C&+oUSm`}AX_vw)lMRu{y}+wqfhQNKPH(;_%1jIg zVDwq=-;q>J%KWfHTC~V4WBkf+sY~?XB(Nb~@Z{|G1+%`{@3f*TuYo&(Aaw-KSrJKPPasFL%~YCwBeS;WXn$9D4j|xM1F+B+bPY0K03crbtOCmlMV0D zyDT?-R9iFuaT33NOI038d(EhDV69O(Vb(Mjf$=b!lT^5z%*2wU|6VG*BxMQ2%v@WT zOF>@E#gLaV0?B?(^$B65F8B;KfFgZd{o@!|PTVjCSy;wq4BRkqOl~vK^ekyF<&*QJ zjFSq0lDz5yeJiN1nHEEs)#*8Y4b@*XHzd}_AZ>V=Nt#vu0U*!X9C;{;3a z4F0-I7wIuD*QbNrE+#fvKDz>omJYjZD6*|;?31%eMpyR=dCxReL}~L+(4Rg3H3-t^ za^(0RrMj8l&c!~!9kB-3Xz8WGcrvOwK!~5CwLY32+9N@oE84i1ROD8+VE<%z%6k#}C2aL8euR_JXuv=TX z>PY^qU0m?O3GGkZ*lxvUh{_2uu8$f?#Lo(cQnP|B!vXpCg!E}!OmpdSc`8>nbKhq( zt@0E}%)@vkA_^l7D*?LLvGq_Z@ml@hA@z%=M} zZ3E;;)G7L9Ji{nA(A29zzb}^e9(M>cy@f<#6TiP$_OR3y zxGZ%k+^MUCASIqr%!pQ9b&GaCf`e~>rt=po0!|eyztdzPuq71fe*F}(1|b^j3o0|h zGRO;&Kc!3QzQW|Z&htYl2w&1U>TO^>h(d6AA<*N(CF6VW9MvUcMyp?$4ILwu3uC&D zibX(QyZ6xec{kh0D$J2r1YdzYK#i~(%n6;0Yl@`Ds)qP+6!wr(1sG_)m=&xuW~@xm zbWKon&7fhXMbA=&HVmm+g>5Kbp@=oe5aitEZt_R-6v-oq*KKhd^qs5${dX~}X$Zb8 zI(o9`L^`L^pQ{k6SpL-^YZt<={Fkyc(j`3ZA1_cgQ$c)k3OL{I6!JgiZ0hoZpDc%6 zTa_BflxB|`=oDO4aF!>vE_Ov<3qmJ)KH}DN0X#-1tbUo;qYj8AxdiNI?n<$YyNd<5 zEWysZ$JN<;d=N&#AmTGxOlCq1$$-@3?>j`kI3(TRZBT6td)jd<>v)2kV9aJXa|b>) zgV^(nis#i6XvyWNec8-hpUoZ>yeczv^&e_sxft}IjVlk|M(>i(SHO*(V&pakM?LyM zhAdU_Bk@%G`d@|&qH2~}b1PE_Y-11#!z+hpUyJL@#exi9DOD&+w21o<5l0G;-T9dI62n zjLoG+%BJ-kUDZH=8iXIcu9i!28l~n>^&WO zB|$^Buip-lM2`Ka!hwvhEDuz4=cO;FrO_ab&+Ik-K-cSTkX)Tc0E2(XdbR7ix7 zm^Q}&*PI`3HYskB1;o36U4JB{-W11Wmsr#FVpcIx%7^|Y?lXP>M70fyw#!N>^iLIP z5l2win?Jukw4C+nEM)^@%|emMIs-4JimSl2J-Ee(Yw92$`3q{o?11M>^a#az_Dey8 zHOFcew67m?N_#0UiLfU%eIQ41^eK|`_%&Ow#<=G4!1d%6-WsN_W1J;Gz@EAxNXtLP zsJ_b9M0z~CqT*E9MkqR9R+q%HJCUb+0#NM4oFuzCHC zJa;6fR6i7Tb}%J#zz??nW?QIfn+-|K$_8rXKLP{|Bzy5yIjpM;eaV;iBdrCddv>pH z@ZpHpkx}r;uRqb*0TP6nlMjyMsc5enbhyPs%HfVH$lK$^qqdRw8fYk4RXjfogMt`m z%?_S%@?a3*pk05%a|ef!I4SqckI%%Rw0Ex}F$K{UO$I9+Nsuus$#UhXbmlWPlxLBO z9?UceO36MTVeC<>plITE_vxfOmB(nP>+RB(ggq-_99JYIKUWya4icT!{eA24sY7U| zq>+c>-#bx8QwF#pafJ6`YIy_FDmU(18Q`0l=roZc{q#BhUiM~UZu*~;g%JBJs4!pE zp|+%bW(A+7eV(uQQ1)S^jv{}8dCYp60itds_Yi4)lOrB)HSNG6(Xs#`KBA|(N~`+% z3fk2lG0qq9BbMD_G|HaWDcF`;doW({N@p(fBhOpu5W`mJbzz_Pg?{-yo zKZ&o%p@}$uyzY;dRiP%R%xwqrFK+wAx>_{~AR}6w+9E|p9`YDv=Yzt8ix`$hY_V90 zpQhM8RlHwuhw@V`RWJSADJaurB@ukMA_r=?hMzJ;wg&&Ct zEyRgx@2U)P`;wb(P^ujj{yo8Of(f43Xg*B?vB;XqhZrx7boD3aKCkd-nhQ5VId>Tu zVj+*=jZxCc^aNjPVN6gCpx{6noAs>#LP^@7ns;7o{F7*ja^@%8` zj30vy9OzUZri_LNJ!YSd!p$`%g=_4S$-0ZBB;r#_V|19N6kgyAeT!iv787tUDCd7? zQ%de5O0GM98{GS9{$M<=;L_#C8morlGJ)@9Lv9!u3;^4fw1MIYc!NrerS7FA1576^JC0tVrDuL!?Tsy;^14uN%DZHRtq!_RgZkYFLbnZ;RfS zft3yZYICb?Q?#m`6*hh`?DqnDn}A)^*xA+6ypo(25(3yuB!j8?Aandd4Su?1uWWNLRg90t_77y z*;|}hzMFAP#hMoU$x^CD86SE!D11xJvqN60O?J_W(B_ic;yv7LY`F@2*jPIhf`2)u zLUhQq@<0}^O!IxT7T*2cK{m3I>L!B?W?xwj){(3VL$NtOTWHrC7z9U;NkUeW#YS&b zJ4`4}2hk54;7x-Hsrn4oJ3>Ii8&SVZiazRhLz!A(_7A!knw?KO{lrBCX zRAE5$n?pf@Lixj#Rgq|N>%d{Zpz{m@qb}JMcADo%atTTwjE46J8ZP8lCXR#jL4UTI zIo6@}YqLINFtwr}1qN^QW zHAO|rGkOD_REj_Zx|T^jL9v<9-xDmQh1v7j^{=CxYK3hQtku#>Y&RT99JG4`yRRh0 zs@?@xY@LeBRniAgy-lscLmF!|lJ|gfZtt)gM&DPHR~Jbyuf0lsSiY`S{{-;S-88R9 z?Qak#Y=@4@BQtatEA^v?uZr?4yFTj-OhPhrj+EGjEKW~B-w(rf&p)CQx|Rd@^Qx@n z0bW(Ue0Z;V%nb`ioZ}(oP(|fNbM8;#@y%oo_Q7B<3D<<#?A*qqq{|{3ws3|W8}xK0 zYGS$|GL3@vR>Oi(8(x=!Dz^IK{TC?OED}Ni=tq|RNMPm z-hdd=3*uwGg@p9%4|d)^dKlbhk8FlyS`#FRb}_4D?wQ4kh=dkDn6vLRICbTQ<8}{0 zzQS@Cv!nzlSumVEnbsf)crwzJ0I5LuzMEb_9cv;Fs<`ZiulkP!P>Lu)O%#7xj0`^d08qUk^MAc)jK8SlE4pWOO)cguNX37-D~YR4vprIfDWFXGd~I zigSOJZMj#bM%nB*H~w3>x&oLcer=zog*RS|_Sj;8e-0Tmv^TxS(#hB z&{5wQ%*s_Q%!zYLU-@nEreDk(IFaVkt8-Ecyso;vA%LPNrO|t>g*S9;(;AY`gYs{Fg`v9d@$1j$b93^N8CaUsg!{K54JLq)&bANZBQjM= z+&k7dv#O{TROm-kU&K~T6l4k}OT%6cCs!Mt0KF}hnck3Xx{q3W9G=1vB*LtzVULD5 z*+p%{K~roQdyAZPDaxIKnh@tAtR9-e$$%6?9>y;LH3PhkSg1`jeNV4;8uh6dS3P{T zZne++IR36ySZB~#mw@K;nJ~`8fa^lmlL{@P5bUaBO4$YC$F!a{jYZ~)RKylya{aYj z9KiaUmA29MhCHrk@*contx_>QrW$v^>UsZ)Zq=vsRD(BXXkURB_F0&>(PLBQn$M%r zn&#IM4^oA1J<-qnV90+Xn23_aN=U0SwXfy&QbYY;yh?#ebf2!4UY4^g5Y{!Lt^3NBejzfseq2VQ)IxkyoKF z9a3}=jj5TmgXNMp$0IGve5%nd9qzIk$>ZswyScCbkLs`kFDkXf7c~y45(nlnZC|DG zY@WZ`eDbWQp-ain`Wkc@Gs}Y^_35K|dHM6gRLj1>R*uH1(G}+#BSYX^J8l`dJ#rXM z#>+{iGnWxe15x10KfPsv)Gip0Uc%NZ({`b58Zdl5j;STL3raX)0IiPZc&o#z>Uw!vqSWE5&4!k5-$}BV4 zi0xm!l5-S*5`wy3(T4?j5u~kp!Mj9JM2=AjF^Ma`F_lQH2xsXKEK8A3tq)R*Fq|ka z3rY2~0=j}>WH3%Rp-uh{`~lL5+QR+@t?N=x{I1OZ7eB)26M7_m@{>&`t5%KAjbbq9 zCQ@DkBy%h}G*_1=Nc#d{MW+@hrWc<#SBVx22q@M;y$*p#6K~UAAcDj}Db5%d_)8E# zy1R2fs1OI$LIyAX)To7vqK=e_SjR#JQ#L(uOu^@~2-F9|CB!8~uNmrEDU0J?A^cZ# zWe$zRmy-+^teqec=_d=zhdxm65ej4eWYGj-0G8P8q;4E1O0NT}^l0>$`)2v8><51& zb8sObq%IM%ouP^XZN+27b;DuXZea3@lW*nLNshlfQ4elyJG6An0L7h%W3MXOj8j>) zF{FNmBrf9npM)2Mpjc+{!RObR+d#B?M$Q%3tVXrMRosd({kYz2LRQ64xXPzQoq+w{ zI_w--nH5#tk&{0nAQa%y^xSeBE`h`wHb5Xj^p|j4&QV|hQkO&9rQg{;PV-M!^4>456oPP3DZHpgjPOLguQBb0CJ}8 zobx=YZd(N+YFBY!FVzkNGMRyqvW@i~W%)Hb6f`guTkcJgZ^5%dY40sqZHx&7EG zhixE8pp}OM4ZENxzRM8^b%+8hE!Yws{pn`wg|~!Y;y=vr19ta$W*?mGSDA!WcndS^ zho4U}`tZJbEX`Xs^k5+C#+bRs=Z_zr(U2!;U|UXeyQBX53;*YDL+cnBoN7&4#owj0 z&x1iz?pqv+1O`@j=CQo%2>IV#see#vHfSp>E>wm1e`=DGIf#NA|GB1n%JPpaOBgO7 z4gU8v{<^Ng|39uruziMoD5!c&^JH#3`QyV3%LUeS+WRM&{{m2o6g-vw=i4YBJiznT zx;3QeUFQ$eC|(ui@991Q^j}2HpB{dl2Q(ugw15Tg0a9-{RUBAx4FM)<=XfzT z9GcB%{6H^c;VgNj5h$M9fbj?Us(nDncaULg7cklR^0y^Cm`m|7%GrU>24q~~3b~W8 zKLsmE8wjvy%aZU|aQa$x&IdZ#rVP^Fu2h1yx4nUfmfg@5)T#UaO14>0R{wt_6F}oX z7;5}KK~q?PtlXhQ@I1{_J{zFty$0zl&2Ksy{R9RY|1zL3Hf4fBBj!Qp0S6KdZ>MIa z;)07(4JF2j^Y6QWSde*WkF}q23Q8Pah{YHwzJc#xS$F*;*xm71ZKN4&ndiY*zXo}N z`j#%()&2M#;gBY1*9JL|%E|F=aB|{Uxy}Fi3RkuWJ#M%xvb=yoAdl7U+n(VH>J7|7 zHD3M-AeY^Z|1FV!^2KdJ?0tpB(LA(h;()-T|tm4)b}8UG?_1 zYyGclk^sBY`wKX6HvqP09?~RM-P;Db_%*;;R7ZMA{$Uo+)y4!!@betwsVWla+bxpJ z0`B3scFMhfdHgM?E{8*A@K;{G2I4NhGBS{HnoTFU2?1yZdHF{uDB@}rE56>fZ1hC{ z=Ykv?AcfW7?>6E;-vtj20+=i$?Re*v45#vl&E};9 zP0%NuRA8Ds{=)|3hZ&f*(2oWIl0auz5SNo>eG4JNO1Ad|Scu$`WuI2}m1~MSyYpxU z6giie*dOBuAL#loknWa2E^sSix+8SJt8E_Y6+%HmX`oBxD}hx87IaHN`2hk*U0n*=Ra>{N+G?95 zWrz`if(IK4OU z5)k|dkj7ME&3Re+2Mn@G7u5`q&sIQgJ=n7_DE->%(5<}77Augv73 z(*^BkSsdC5bCVQ{Q>F8*2tW>^J!-vrV&@#WN0P5zdcs$HBvunUtZWA)ApXhjDtIVVO&^*V6$rj*p-|G1Vv0V{uLEacXD z`H(vJKQ|E3oC)H_CGyBUC^P~>{pLZD+0z@4r3-|V!(!yza%Yo+3kZf_XD-Gm#(F@C zM1tgYO_`^)dTg#fKnKw!ubKf4jLtYv#Re){p_h%k|DIg5Tj_%t0MZ`1a~GlFLTp(m zkt7E#@SCapbv@@raDH9y9$2iq8GhD6onU3T9U!uKH{>;$7fw&jZxpHhNxRLxn;u=x zqq4uUyC%LSa$7&-G-ZIfBhg&}tyWB}H@co*i>P8s!?@s!R!mbiyK<^Jr-H~`_ zNa-48lgV+Okz{lA<=DrrH}Zv>tT2m_mbGHJ3r9vZ?agLD@Jcn~giJu_#wxU3>XKBf zn^WcW#M@xTm~-sxypp~;1A7&#+$MKGIERDmpj%`#Ih@&M;(@M~3x1)JWV4iH&L1m( zGb|^A^NA~zjN-ol$J6bLK5iyLQsDepKR0EQqnbiDNX1BG)b?D4dd!q;O%08XO-%>! z!g+o@9FN?Qpv3$xnS*vPyb`i>n)vti)$EyYELpxFtTO{17*7i@|nX@4Eelc_H18uH7(sgy5fAt_%{HZW_MR4T+bW)4yy zq1wKT4p;Up)lL8ajVtX1cmpO~MCh#<2&n`Zu_h-;HpRbg$HI!gJiWa^Za_H&-eC9Ez%{6ccrKSA4$Qo@OKNml_v+>oE?4U&+dOG!v61`pVSKTr zt7h|f>KAPv#be+8m~vnZ%D`(ZB1mA5j@j*zEn>nx%XZ>`B^h?-xqkAMU86elag$Z9 z+vJFi?!0c{P2ZH$d|2kvNeO9uTO@)upEH1|tI|tk^~t5!2!K<5x*zgW2kcsCn}<5# zMWQ`s9kK~oLInLbr5f;RvJ03|q`IM31`0+uMjVL`E`g<#p~x~(>prxs z*#PZX?003SerRvlq@o9pzBTrIyOID#PWomczD=18fltjRd6$7wAmxZ(9uM{7t6~VT zKG!C*WIn`0MdP{ww_m-lR`~*J#C^UeIt73se8wCO7s|iZ3=k-is2p63RTEr1RUH|U zo(~_%oC82sn%)dCa_qH_2ZhaG(K?p)D`U*kVAL4u*f>WaTT|E7TPoGL8#I@88D)cuW~_|Gql5Bp%$uDegUbt6QG2pjQwm>a-$ROOj_H+!jR zt==LPm~aDc?B64?UZg6v=j`U-&EY)wG_M%7&o0L7Gi%U@GRut7E@3rvnhJMY?@}q; zrx^i>JI&EDFU3uMQzkOTB~liEt9Da*vuM;Tw~_vC$2WpTWeGzUk(olaQ3LJu2kST! z@XoftMqN9QZ_o5`c1Rc+#z(^$*D+P*IJoly2$*^7Tt&e>Wz8IP31$?|u~Gbrz&5YG~F_b+(3br86=LTmBV&3@hG9apbpS z#nuWDI$mLk06gr5TzAE7><|j`w<**UmDZJ)lPqXe1jtkru{%^`@#P8r*yP+8!?*?` zBp=M@P3Kn#$QRrTK{phWu=@gv3fk{^XR);S0d5%#!c#kq=i$qRhfX$}PyF|$lOIoa zd{%ZAHNcs14=Ac;;?=4v)zXu-SR?0~ZSKd9O9?eT!w_EoVf1LIp>5v`c2g{%4pnOq zrAf+RSG@qmH_jZCE!~7R#nqj9Whm+`Z%$L3v5{}!teY#iMkv0XHlmtH?4m2w0IUfN zr5La#kyrW~Vv}vNr6%e}hX>Myg^Ov(2Z}WTO0nS@J{}|P{*UNUomV`f55Z=xgPwTu zv|=@MWimE;YWQJ!fwbrbY4NI17#Z3Mne)q$351iTZ?Jorf|5Ke;{iks9b!_~7LD>k zp#@vlhbyd_x3*+Wa_<1WuBJO*hhv1%o3#!`CQ~Yce%I!MEa-9?tp78_d)U?0m;js#;BLQ0mRzLHrrhNS? zE^vwC@^3MorS|cN+2ZIGX1X{=N{Fr*>jnu`FKKR*%ML51YCIBsYap(DT0dH;T9~#N zEl4}Tw|6Lf%n)cMnS-(_>MK6fmqsy*C>-5Yhqw|mh%3QDUve4t8(N51jPWGisUBsM z&)~n5lSp+<`~}WcW)7&G11^qe&N_O-Uigk{99spWPJNO(EO+ypUs&+Fy}wDLRW#76 zt?BHrqtrX5jN^?w)vSyztI<)=LuA&LzU52F<{ed@XtG6IdB&&)sA0BTi7wJ0+bTO( z-@VrM2q`gzkm9CjGnVE1dpQz&zC!pVb-TdWrp0(1dmdV6}xn`(JH^IAu8wS z-)g1J-SU}o@A8Yx%S=AF3)0h~k98nG4m!9y#ek!+6n!c!Y0_3@DvE=$jzGv%ZZZU7 zK4y|{`-t9c z340!QkWX+OnMTu+67Gq3VqLPFfBs*D#u`j9Y+uSHijZ3-D`E^y?=D*+XuitO}I_EWW z<~iqkp7T8C`F_6N_xCXO2xUC5rbz4DZ7|PC7@f;Xi=AG@^W{vX=MMW|N;4a6v&{21 z^qPD`k;k2LAg>`O@fWN1#B!cWi&@$Fvg?h?{gz(3uKPxNnB}#sI}gUPnY&vYiKsDm zQXma&7+~Hon%Y;)+nqegz1tF#-G?9iK$(qydWYVEjvj;luTg&j&)hXn&CthgCn z$~du62wO{A<}E>IeLpmqt9cG&arg>j?map&s-6LKTstbIOynVnNoKND9Z?;7O-SV%S>WHo3rLFpIk)-c9p<~HKurMWUQH|8Fvh7OE&V#U-tybcMXC#FSJKi z;rjuRJfbfzlI$W0igT*7!LuNQza>Y37E?l6%5#&!DID%={K$LlaCQD!S>t&My4xhc z4oh!HkcnZKwhrQ#-DuJMDdMI{3O5ePs|T$%V{;bu`_-a$pB7Sa{pl?Fy#UnRZM@ zsL}>32Ae$j_2h9^J}$HXRzOLDnfRWke+GClIg}IXU@*2*+8XK@_lyQuFn<(B zOz$5j3a7gLPULCkujY#R3K+k;2V$~K=Bm0oDN)Nl!z0-{t5&DdrJsa&=GLE?Vak~M z%(??LqyXM^g-M6FMMaAeLmXhchSn6<1+*={0sa&62r;c#Si$ zwXn16b?m12yGp>}?k6X`V&>GW@jzw%rqQ0Zn3neNQLtnl^EBh;K0L_slG4~z<&=Xh z7cI7eTU%BeFb=T$X}z{#BK;1$if_Xg$InnmFd8=(_Zt0A81Xfo`y|hZTY8d`h~%E9 zgyBYJQdmNtY&dd+@I(Qv5cw&(ol@mCSp(lTUSPoRFk zAwjQzqd%|I%77^696dbRRnP-Jzfb~57{>)^qdO(B^ATck#3#IgA<1~wye4_N>e)9s z7lh#E+)IblM)ADBv(Hbtfngfl$L5W&Ei360X<~29f!|)#@19gB<+t~zd&x_<7COi{ zl6h&!dDWXa+jS%=(!R?x)=f5l(rS2jumUh_uj6SN^rfebP51N{2=@zgJ$uF3qydOOaTt2#DUwTOrnWj z70r2+C3g!GI9GHw-suk0{Yag}qm#q7A%Vh%8&0&49~jU6GPjqOw$H#{Lah+p2x0E???UeLJHz+ey91+qxR8hOotdlD>#-;gK&_Gf|#3tvhB5nTH(PbqqS zwmjWg{qpXn4;Hg`#z3iA;>v(#RY!=;pnAh$bZT6Oa9eo{C;+l9-dxnARq0&KTmJdO zEoNULBY|6@q(@j6C7!>cbd_Wf0}`Z{P5$^ZmH5^hf{sYtj7I4TGaw`t2&aG1aa?SU z;a8G7S*5?lzZ%-dg`afHic;0&lbl=p`qc~p%5yEbINO5s7^@$E-*u0lK#6?Ck1)uI z23B>w_1>cb;yG;GJa!z#mnwx@_Dr^b$}F{j$Y>*;EKfpYHGKjExEPa;=d8;EfGp@4 z^+A=IvQfvJ>Xt)O@exW}%__XLQ6V#e=HrC-me>d(DPtUlrEwkD_(9(0bd@Kp_1Z7U zhT%1dJo1TuLq?teY!%%O@13^(jbqNd>6-wpd*5wg98=LjTMWU}*9{clKGO zAM;Ngh-sX`UDjAoAxxjJ+@(_okth3W*V;~B$i8ImeNSHWL&M%G2!Q?=L^Oqi`9gL# zOc2ntgrt?0j1+-OnZV}!5kh^ULAhA0M{gW<&>zm!m#`!Md;ftBP zo9Xo}TCU6h9W@-}N2jFQ#eXu4^NUBG2)f{rTn5fB#oRKzQt#LIzp>1h2pi?1BWaxy zrmmR`4azx@@I24kUzkwVp*Iz1;WGf$EJn69mH4{|}l{Ha29Iti{>RA8N$5S?Xi^cY1 TAK$>hz(@Oxp2nk7)?t4E$UG=_ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-backlog-metrics.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-backlog-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..b2dcf2bf43a4050a76e09a87250cbeb4e00ac26b GIT binary patch literal 580860 zcmeFZbzD{3x;{=QAPAxeNFyQLCEYC{NT+l!=`K;akxmH#L6GjXK#}h5CEXp1MSdsy zoO?ER-+j+M+kbw)KQ4TLx#pZ>j5)@5<9VO=orEYWN@JoEq9Y(6V9Lr!s30Ikw3urfuT1pya5%pm4 zyS=d9^y@3f6=Czf^g0+8LM4uF+4BdZNaO~uXhR-g7=${1i{aDkG_UD9$);Zg!Cscg)ipE~>!HV~x$QnoVofB3Af3ihud~Mf)`~ z+`Sx;jQgvViTt<4Wye7GhGZRVx^_ZrSTK9GG5MJPX+U)8*PthPCf(oSKAaS=DRl+% zBA;P5?A5izU=xWG(|3hMs_YXae^bV4ee*sm?*02TiC|3f43_9XssbBsIrIfJ@W9!N zd#QN~{r1%85C;R8X`L)%{MDunqI6!AC7wCWCy^Ka6Q5<0h%w8awe}6oypxIV?O3pJ zIr58;z+;JmqA$4l(=y5wM&X*ui7l#~n8Jtr;YLIi@6#-B=QNVipOo}G82h$JzYyfu z!i*tNoQp`}@8mSntb6+QC-(A8nK{x+mA7)M0ky2O`n-7g!G}g_C$;a%Szh$%h-@)4 z$tg$th{KqF`)Xq`dEwiHglU?aJr)mzI>7_*9SKBkG?e?~0bIzB{fNnJHZ9luUAr=r zo9{k~zI&%x_Q(BjnU_bwKe-Se2M9lXgs}dzmRzzF;j!q|CV5dKpZ^O

      q6H2QJHNLSnZ+=*1IFn-U-!b%1euwNX5B_XUJ;}>EeW`#hSMn0OPXz`1MFPhFj zc@XD>Ud#4QK3nlX2=F#DNU%4nsycfdIqpNqaugh;DHgLkQP>gQsSS3PTI}`B!iaQK z4p26p);X=K0zc-wOi#*+W}v2P=PeYeg$Q}Ar!-Ejb8(5G#l^>mIryD@%gD`DP}v5y zPHFx2D4un7caPxk-BB`;iEfB`DG)yI(7YI%?M~sfO?@)I{GfaE)lOD6%@}2p)_<{Q8~d~e>~ay_HDvr;=c#@UsGnzMl?a_b&84enytL)7+MG2FJ@|4 zLFBG3)R-np)$~8*_`lxx@2@=Zk`VV$Tp_GFlyU6*>9_-&U?qY6?Ok-O4<3aV8)n~& z<)Tt`Di+kR5BZqDZqu6t0Tdah9Tm2Cu29MFhQ;A9r!b}r(_UGebTT8VqVCc8xB?K9 zI>%VOpP?@+V+*kf;o*!MgpKdr>6P^XYy6>YQ7q1#x6EONtDh>GKHGPhXIG0Mut)O2 zNu;)G(b*xw+3XF%`h1bg`gtED>$hD>s8}3(JH$?AXdNgAC0@wjGp{EfCsMAhH;{_O zW2Z;8?R04zr(2&rt2~_>GA_}Ty_nV*3tgD3k?2HO(LcRfi-|y`^x<4j<`obK)fQkM z@{aEVyk!%x6AjDzn~RLfN*yvWFYSto3K+oWm0oZj7Rx9p;r!{O|2O!z=_L*tV;7XN z^Q!#F_6+vlHsN2n6ZQ6^SXt=|;L|NVH=uBafi-CB|Ch?YBs z;TtmaGp@eK0J1wBT{BxITiKu2y;Tos>?EuAaisiQd`h&I>%qKF1c|L8FjwTU@e*_L z(IVFf*Rw5#c{XB|XFkGFJe5rd!%6{AGoDPI3nnbS?a_qE%fTi`z1_81AS_{*+qLQ9 zid;nplgrQR0Tj!a*$uyNVSXb58i2#y`h?$svOOy8H>-pF4SEPrTPmc7q+-I(5|h5} zKp;~G_HdasIdbqEyrPv_1LCAUh`510N@E740;nbUgM7cT1f?mkJg_j&ndC48>01nJ z`lF~M${ll=i87We%*}SuB=wde~6o#n`;dP3=_=^O|&G%0j~1L&$f3 zDxuypUUPhU(W8T)$^Bg0$;a%g{Lnc$7I%p-QVPO~?ML}*SVN2so@~b1H0o}`^%r@B zHV?M0=HKBg9hK82EcSIfSsIq#Y1S>@dKC9y7X)@wcn{z&`QQMzM5GWpCvth7f~GOM*F

      {cqs+kFCB!1*aj__qh=$avx0?}iKb%^3E94Oi2lps;NCEm0wn5S`705(-Y+ z9{(kw#^vosf4|qpMFyKxGcrL->!@R>?`30wY93oTHPn&MPYfYG|2@;3)sd zbmo@@RURbRhsXG09~qy!HANp^B!35-4|pNXQ(ivaB6eH}q)^VC^%bCYApd>EK>&%YznU}8evnh$ ze~xc?C_=}8_v4i`HP9)LO7*>F(2kp&bq;bqlBbd^5WOq2$`b{VvP3hM-ZaD;c7BbJ z_-RV@^&~dY&%DL9Y4Z$3&)&K}{Ieq;>;9XE7R7TAGQ?+1EW`0SA({8SWXqrlE~MdW zn{Az*qC+)Qcb@qM&PVj}v`Ii<$jj$*H8$x-H!x13K=&uf6|Y4Pp*G_C9gG(7y$YbAfIZ-#ry}4(>jAanepVztcZo+y`O`Ce*w&}4fx>&jk`Mf#qSGzsYTM%3MeN8-5~p#7iu!b2 zNwzCJLx-{K`arOIxezNq=ZD>m)W7kX+>2R6?T-YX%r6;4y$x{X|SDrx8pwN)~Iwl=sFC5h%`wz^MN zpIK68!W4IJPTtLnSUSR97l@<8m4jRtGGWnal-QQ*mu>W^=M8NO+w*TVz#qTFdgF=3 zpyN2eA5Ku2h0BUc?(S@44%^Dl?ZH*p(c)+AhB60Xi$f1Tv9+I{;tXD`oM9!MM(HgJ zGkv^URP!4KheXGw48=FRDjmOtU=yH1xOJwD?%u>|l*sccje8qZ;Pb%R6lBz@1jt;itkgUk?6D5J(vpMEn)k7^~7hav-6YhJ^ z&ECWO%VBvrn7LQtJc(=z%PU{TP#}mhFfc?E`ySYs9W6CRTPP*4;#4nr2m0R@c*{P6<756k{@U=v_yEn{ZttYPKToVchSbo1Z}=FG zl5}tn#es)#3&6_QuRGjPg0#zLsg|ZPw$x+rH2k+dQY{Z!+!Lf@Fy72x&s2A`T0-Ao z1KlF^w?xIO|JnD1(C@eCX44t7g z^MjYVDQxi=MyAIlu%t4UPs})uSQRn58p%naTCPTdMi%xBU00JW)40xtdU~+g*x0O_ zD=M@X{B95a%145OgR@itdzpf?3uTYG8vtE~$@Boy&cq{R%Q%(DDES8s3e3TRBN3ke z1=1b;1*E$|1+Q)eALXsw2f#xekFc;Bao8wkMjZr*gt^6+AwJMqqNl~ECct~I;4l1^ zj~kzRNUYyGf(}F7(%rsJ&o4AAm4MizP(U*NAMfO2-%6AxAS9$GzDW|3AsWB_j(T&8 z0@8{mBybd_ME=)~*#I=Eo1LShTB(Nngrl&}^js&Z! zN-sO^R^t(6XMm+2{n)#XyBB&(qB3$_f5`tp0*qGpWz3nz3+6k9&YAQc@p@#%Ki5Kp z+Lc!)Vj$yJ76v+6d;J`#vdMR=)2vs-VGn7l-C{yEOCMrC^MQcobUzp0&Ve-?zt%?X ziObXOC(Hagb|(;rEm(PAZyY*o>(;wYo3P`|!;L&{pi&xztntG}pM2)r$fsPhnD|d* zK9l}QHL(&!7V8|iVh1W+I~hOjN_rr5QsIemx-Q%H4!mnExg2!kOeN6?W|phZL+p3C zxJ1x+`J+>AZHgOpjocSq6&IGq$7xJFR-c@#Q9kO+w1qN0W=q`X3R#iArX<Ihww1&gv&aZhm*_g)&LdIqiu#o7SGHz52DDBtKvSz@UTQD96;1X<0_rD%-gq}|&HO!{WoTfoJN4=2A zE$i^(M}1KG;9?AJcf$f0(G?43PP5s~9KYAMoPWN6zZdy^n)m!cm+I!wrL>J;EANQw zstH}B^bD>Ee$4$ojw~^skm6j~$TDSgM}6_(?T64m%Dyb>fb{{jZ2-N)A+%m&XQn_0;-6Fqx_S z3=O~KrJwEoAW6q z=C51f()Z&Z0;dTUKO*pYu|)Y7q!9fIf(3xg04lu3-r=5Xki*@>&RW-P*yF`oKQ-ky zwC4GVa@5Pkh$JpDAePem|=! zHTO9U*1?gJL2F1JA=GTiHr!frQ4*YKd` zcZ%R#xLCLau4(T5wMGIC8u~V-H=remiFq{3u;*#lB*DusEy~oNnzwCB9q)hnjf?Fa zP~tg=o}`;SYDsXtb7(sM%8$-= znNh#?g6o3Io$O}CyLs?3;MG%Ur$?`z^`Ed}1zdym?Pg7`@3em+p`FWjKU{_=@+f;P zA$^er(LjG#

      s>0#P9cb_EN<;u74};s(eEUxk?G;& z?U{qOMVT`+D@eL}#1XB@2*X8QLw2V#xSC~gct|%2an92jx0`vIu?dij6g=$p@3p&E z`XOeEB(f_kkZZ*e63B;MI(qJhhx^+os%65J>8Yuw`_hT=abRF$3$XVasaTzj&($^n zTVF!(^(k=q{ejJfK)?FjCC>=0n~xu>eTk9469Ir_{|77E`U_S@&YlhoN)pY0;Es9Z zmwqN25cCk}sUa?f6k2nFGu-z^E^pby)ypuAHtYCCq(U`4TjWkXwn-QKL+H zBqx2U_CBg^6Hjg5v?>L;91oIJLZs3smJnt_yM_TAoyq(dOwVEQ+O79YM&d<>hF4^L zvGAWMPx5unodrXl@PPe+6+r28X9cxzpvI>rl(!57K1^2dzN60sdN+@hZdXEQSQIzp zr_{M~&k>$SBl6Z|PE6A_fx2kv;rp@&whiU-Neyn6lp_%VIm_6YB6#Wdh1s>|G%U{c zh%udy!3~{lU6%G{aN5TNy;+UYLqnUw>-FMotK&%m3z;gBpq`w`O%&OY3X_Ko@n*dS z>yl4rQtDBO*I@d?Enc*F-5rla!kK$Gz=>GL6vtD9JT@9=WGM`xPAj{eC{~tXh({={ z3k8-eR{hDL3+G0Je8hIkeAOO1V2{re?Fb|Ijf)~=7po>x@IxVY@wuT#Brl5^I`?UC zhKyQ`o}Z4Qf>uz0fy1uT;E8~`3dv9)c`$KueR{*RAaz^7(`c9BeXv)m$HY;jsZXzG}& z7`R3)ak<~NeOdgbZeGTFjU=9YTYt3qzBan~O^X6Kf5MjYr0JE!`7B*J&V{8{rVAsZ zev^#QEW@v@+~&-c*SK4hsw7pN3|Yy|STonk!+uDxxb)aqY6AlUuxBmDF;X;XY~0w` z7z$`hPcKIBpeD>p>fz&)*5>TGd8hRMmFk;XV9$7asf(7h8%8!a(Rx{sp;-Q{^8Yh4 z#RWRU(rvt4@lS8O<2e=Jm>MO~%2G`U%&z*o+u(spu2U za--=`DM6BA9PYE=eRv<`W96^DZRlZ+cU^&=EXu@FL?6bf z$*FC@E$p>!hG;g@*WaD?dm47GpZ~03<58bv8t&$^ia4QOkyq@GZwpf9&Pz}Z{AR;agnHFjx@gpGITck2Cn;9K!i zvggZcCf@53aWyl>=-nnx$8?>p{gu^Wu8DpVM>Evw<7OWgmZ_1TcQ2u9+lo9!uB*#o zXUovgna6Xr&b^q$wVXg5U~{-QeJT=rV(fu5fnb}|3p1Ysqfpglzy8KCMElfCC|tGB zMfK{8<3YpJXx?!!CE=r;es5iMShe52Dxw^+-J~2uYfI~B@1S>e5APQHB$T-)6yXHE z+|&wQ?R0{iT&M5ZPx3F+zCytotrC|Nq(rHEs&)iz<)Tj!T5vnAg%A%px;OzX6~yZe z$bI>Tj*@vZAGXVGpOOy&Y?Q93m{A3Uj2uVY2Yc)f>T?4U6JS>>M=(MqE;jW_uhmqU54=Z z#jfq=nv;0nnUdj!{+eCe2g%jdH_q@U8qn@A zzRu@I8T>KxiC9yjLmK}@sRZ~%E10D3eXGF(xkPjzI_NBFa)hWWN=5#&`O|1YPFk9! z_Yq2HWQ&|}ly|NYGq1KGQ0wMi&AU|$oQ{y6RQRg+j!BXgx26TQ8YrL_szGls1{e%v z6)*+zBv1%0pCMvvQGG1n@?`5|$nM{BN{iIJ4Z*ROjDs*f!8YfEUD-*A&(WN|F=}I4 zI|s%Dz%>Kglo*+(=_I~CZH%WXe!X=X8?ecf;GBIc`7NM{BEZJ6Vo$qzH34OlIl#}R z6Tg^%m&(IE;$*RaiqU&Bv^w0quV?45XITz%4>>svgt!Wv%!k&Hr*nK;z88`sS?Y&g z4Ryy^4QiMkP#O5R8-9uBIq7X2LzgOMT@m`)fYo^X<$(D^on>wO_06N7(z$D8gv!xP zWc;48o043G<>7rC&-k1)_=IuC)amNUBienPL8=krD#xjjkLbF8LGt%8kJnBoMI7S1*2eWn(Omd}Nm%!RFO|bi zB;Y(GKcJQG#g}T?B_12irjH(!E>JBA&$lWym6JEr{f{Ed?=fWK&KZuH#IG*sEfJj_ z@E-kAYc8W1Ky{?ih=jC=a@Y%~NUtYjo5PWZ@04vDZl)x4v{5XpiI8?9+ zsWLcz%|aqrP!1#Z*oLDhI0ePSX!(W*nq-pEhbhBAH;sxmbGoSSZ1mZjkfI0N@Kq8< z_f>-npWIgw*$A<|8~2|n_}Xwj4$D*xfqV5GGz|dvH{73B#O3v48eiQ zbbCw-8~2z;Xe5ZtVkHm2!h$}^;9%2O57i#h2 zpqZmRME)b{U_b<$773O1zqP<;=Xy_zeSgn^<Gkl{m@8YEw zHDXr__t9l`lvjC?mrXjfilM#CU?J#3BJ-RO^JKd4ce4Hp8amx)Gmewl6Yy+?=8HGu z!*l}o**IfsNwPdXmybL30wk75Mm$G%`)c5!k9^TtYsoCa1`XG#vY%q^wY<{_ESuMk zVtIjVjEYKjx(b-H$PpOr)_Fz1lbe=4Ro~c6M{F1_XEvSj#ZlrjxQR1GH{HSVqK7>i z95lp}pK^%{I(QD-?LPQV3Y2(-xaqeTVgI@>=ohCrzAbc1lOKlA>Kb2FpZZqHq)MM_ z0Y&=2>WXi<#~A@;c3_xCb24IgELx({&qMFE#?Y9xk=ZzuSM&%{-Z`o})b7H8%w2tw z2*BguSQ_Vd{cZ3H<#1Hy>lh3xgm(AfFmVCL1Bmy+<(TFf4<@LM*A}$GaGS|ghi@xH6f$B z|1D_9o9Y6~)1lAQvIz~>P|b_c?DZ|k-(hcGx>XBVJR2p+2!+FM5~`}6<2-3|kwsV8I#d3Am= z@aoa(0}L%ap<4274*~K=nab(2l4{u4Yb|=2&Cj9pElo{#{QsjQvFDBKASkPLi zCNL@%#QJck2{Sy!u)wGj<{1li*b)4?6i$};zDB5(@4SDi`xd&;_mCIic`Q}c%$G|i zhr!^L3~*9vK@ioP*Q0;1Asb*28fgJB zO#xGpM#WJ{Fw@VG;R&v$>=K}rd`1SBVbj*VP(^2q2>-15oZH}+owlO<>nZsznsyNA{=5a{=oyd@!v}oX%B_-T zs36Fv8JxE-J6>uo#cn3GQ&fHC;j|;#$3<~{z}{)o94#x_2AEfSNr!<_sh*sGN#xUj@=%S zShhX_b7tYEEd8`NFBYnl93G0+W>&%RUv+5H9((Pw=2nTI>mXa9sp)CciCXGdIZvfKOlNf^J~ zax^0MXu@sNnLJ|ro;SGcE=6@P&2kP! zsYPM!|FLz}QB}2D+XqBKkxuE5l&(#8OKch`2?goyO-rYANOvP8-Q6YKoty6Xme2FP z=e*}R-x!SHVz`EXxYm8IdCz%WzpK#Q8bdx#O1@@L=$vv7u+`4Z1fLG!Kt3(+;zhh< z(J8hP5mbk(usqFko^K@2U4t)55-n-D)lJ7agDnKG)b<|^R$OBqMt&Fj-|%8x|2Vv5xF}osy~BTMR5h;vOlMpVp;6!%qwe4tPm}$0 zo`;Z)W^N29H~TCIYt1Hj3S!-KePT<(n*tbWhJ@i<(k8w1+s}D`0g=pVGZa!j+v?Ry z%fNu8C~z6;MsDXRCl^Y9aKCcX!#&iZ0H8*LnLXBCuLH^>xV{1@WqDW#x##LwcTyZHLXnn;c{_@v;;$ul|ltSNFg{w^qbwiwdOaTnzv zA!#w4Ak+JKbjB4faT)-~rYT60WdlNVsQHa7{yqReuTj95NAsT5%;KXW5GcE#WpZ|z z1hpaiu=^_n8SV&j2Yo-Taq|{nF6R(Gsrh^#hD1^fD=ZxJZmV%2S#`>z+Psi1rWkos zmeuuVh-QdU021N3kH~C6_nuR1vtYj{ArTE+GPd%J1`)^3t0rGYqewM0^f+BY_ygoZ zE!kM4h>t_dAGNLElWdIJF5y`VoG|;+T9waY!LM~0sCGRsac5*Kldw0?_!y1lOu{#Z zJakLo%Rh!i+9Q%Ph?_i~E!J?W7n-A3E^J-m54d*J__EeymXD*rJt%;)i!QGn!qf#e z&z2+9Rk#5e1`shk=K*LXHz?mymc2-QPK&^GWlUE(S{;LhO^^7EKTHOT1)s>fE zs>>cpKVLrD7PO-8@9HVG=JfU06!T66pMa^<9z(4k;$;!<(BM96zmIVD!`<9CpGw;a zV6e6gI(1ApXMU1@x1yfBY%9a!Q%lkJq#O(t6wo9B2>3K z|N7+XQHM`z=_KeBIqSBR5RGEC)A}5dv4xLkap_tET@WD*PzpNO>Q0A%r}{^YEH$jX zSTbV7sJ34%15f}Xvr-0bRD?vaIl>nEP>!w(&0N!x+Pf??q$W55b%&2Yt9#fC`Rbla zDMfk*53o~^_sl#p^X!-fl93kkzr@phRs6Jthq2BgYO*#?hs3I=Bbs2-!SGI7Yv`9&PPgzZiIe~`9Peq; zsE1^@59A-&m2E>bFsx56`0j`if3|>$|MT`w1fIA|gbDN3YBWsw6uXAa}U@5f~#J~JwwaN6M@GJLNX1Nlg7{!+U-0Q(FR_bR4j)a z!6_-#k4FU=3k%voN$A4_2vV=LiO|wCzrlE5a1tvZ1~nB)9I@I8j`Pk_=NQw%@=W-k zJhpumdA&)#K4Wyig?=Y?_)1P;l%0WRb+MZ!H?+v8ypS!`nt)nA*Y7cKV=~}i3KG7@ z=3X3$=K2Ccd33~UB$FKpG`ijiP8|U~~d$Wb8#zV1g zF5(H=LH$w99o|$9xJ-z%dSeeo>BbhNz%5zB&)aWdgZyi5c*ZL6O8FIaY#v(&M%!Fe zUV^7A%?o+x6pmbbM$SHLu+v7ZvA^ zd7!?F5_Ty_gKXeY6Q%bB24{=knmv(r@8xbtfKYtPl4JVwsZbwIfK&2ev6Z{<>b-IS z?uZWFz%-2B|nbI-li@8xq*w?=cfTFV(%~uODIeg56p( zp{E}~I*YwnLBSqD$ZjhsLGCv$t`!16L-B$Jo#w}{OEa2U%Eyaz9urU;yt0Z)cPE+< z(FC@ku}v6IK~}B**0#pUSNMSmByeUG6iC0mKOi?E`A{S!C%k=QboQk)`kzImH#~L9U#rEU^xM!{8wsYEauhcv zWVjg3+1Rz5_k}V_seCBpvZZ-i)O~Z#a2uQ8NFeGfd6St@@ZgJSDM9!L-UeRyv9I;- zJg1K~Mrd``0gZ+3Pi*b>i^x@URtikP_jHUiK?!OB4``R)bH*0%PQ5&KB(H;kger3} z;PLL7HW$|x)}M#tFXGzq(!ZSBBNkLp9rv{;->=0d-2$YCvYBmPT5or2HMd~SFWHc8 z$wOugZISkX*vH-zk^;W^6%ZrYesRJH#t4!gDl$Gh7r6}b@6$A2KVe!fs&Yh2^x8nf zT2oR~I3rGw1cw?a_wZ`aC&D|$m>EB=`+1pRIxa&G-3T0<%#^myyr9gs6|`C}ft6!9FvHy`5oinBh=!$aqXW`_vb*R${nChaJw4n8HtHOpXZ zi)8{Pn)g97&G5H33hNObP<&&T-o}XYMYkIJozmGWy~oGJ&2-h6n#8MZ>_S-iXiszE9CaZy?}11rR_U_}BfSdPUOtVI9%Z}V5^yu1lEX_O ze37TU7tFi?dex_Py-%7>t6e{#g`|VFdJZKmEb)TWKg(vENyALMuJJ8BbT(}KQH-_D z?16!cx#A(Njc@m*AO^Hf*dE{96Qo1oxDvW4;8D^$dp?XfLzcmyoT}M9}egbUT}Jb9z)574|`t6AhCanmV>Z;PNtfo z-$Fv}L>v}`DK^IQIH!<3)FO(Ig7`N<7vM8s^%}a)A&p`e~cd zAsd{tE6hLM+^JsZH7=We6%T=0l&MXV35%+JsPiUXCueMstX34~b|&;cvdSdYG#67n z13YDJGZyae!RIa`;d@Hd7VS;M2D&rg^((S0ab4yy+~|FSQ?`!Ql(&#w(FtuOL>yX6uWKMs#gw=BCnp@oRjq&Lj2|up(V=dai z==?(^2XcMc!X9#Q7yQp({>D<#dY0YMJpFuoc8=-iE6-|Ykjo!M#}jr_K?!>Del>()o2URogzB z5D=)K9EFBqiyl6glWIjQLgIIM9gY;SK z2ZLsV4DW~PfuM_f)#->l`sl=D0wtwjq?_7Lp4Csi+NiKNdU_GB#l*zmPzjIFTO`{S zMWus|j;?)Y&}-Wm=C@+k)mDBW(qIrXV!++R&&TA+CWm-Ko8Ii>5If|Km(9WMS!6~U zlHsxUy%cf{CS@? zy>|#NHI}aA)<)RhYzioZ-W}b$Ze=EfClWfcUJcag&`g};(FY1Hs+=S}2t7L#?hPnp zv{;s&wZy3W>=|L(i%({YN!$0=)?EDPOuyQ6`q*CU9d;y(mD+L<^Zc2pgsb)5KI?8* z!M=Ga?g*zb?V)VeZW!<3ix2HFu*0VZnls-90aJ83jqCpCbn?QpIHt6Q>o4P~dpeJ8 zCu+nTvO~Cw#d2X}@AQVI<%5 zu-Xdhx+!LQ;QJ#4mX2nC>FU2m|3%{Vl(4>Pf9>vc^rQ#JBv-4mr=?-yBum3v@;hUg zF<|C4deN{3F6eM+#r6W9b_O9IgBapPv21OV@y7V3g)JJl@ILKc4|7DogCa5p@{ipN z(DDDzkGK3^NP&xaE&4~m9$*Lwwtwl|D*I3K2H@pLRD(~IozikNT}JfVgB z3wAZ58$XpgXAZi9I%iX;k!J5Nu?5;ShMmddw;`v|<};IQ(3d%gtZRd$uJ?qcr8zN{_el<#t(U;bO` zkKhR7+hA?bdJSGOKh+4)W3kcm5x18HHj_|qG)=2caM^D+6FvW{KYELgh1$QNogU!U+Pq{x##%%tc z88>y8M}AVUJ|11!H!8ocaW+}Kue2=GlDInd&rQiSlFyiqToo>6v>xaE5xXK!aqd~A zal2;VlLPpL5m@R?l{|NIyy z62ePjFWYap#Id;XV@#6MD#(t$IU7Ju%E3(+!(N)6U!6e(3)XbM3kw=TC)NiB?YoRR zl@i9dOkj$xoN5aU!jz1Zmy21@JOFi|w^o4u;Shy92bWTGj32bxdbqqQoDS$yRikAb zH?1LIiAds)=y{F$Hq>LT8$8@s?L9R4DNA|;k4u9M3n^1NHK4`f2#qvG5;(Iu`=_m_Ikmr<1&ZQrmSQewRP(B2yP{`c7D z>Xz&ybfEn3IH`xbLkj{7Qi>wWtA{vc4$WPA7=o?YM_$5_if`(GV?tGRPT0}*hf_^> z6Zgq7VL5q64HpuECF1MNg&kV;SG<=sU#*YK)%gbe?T@pvvR`Jcbiy^4FBX3&7=vQac%^qVnr6gDf`zW)K3CX08j@YkU!+&dt_nC1@_69~ zuOK3Lsvp5!j%my5o|aOs%jfs*-;Q*04n)Gx)-D|?7r&YC%pSS#OYJWM41++2UH9`* zw4}qP?t|$8SZbqWDz;a+!LJ1$ID0MhZuXqU=X4JGsw5Uc)}DqQT8)|O@=}|QQivOi zGSOMKI#mfC>kAQSdJl7Wxs=P&4vyH8Ze_Krq>znNa|W$zu1kV~>)xN=HZmw}Bkbf9M{=KgX zll`#;MF8D#Pcp-y|3YKW$$uI_J|jneyL;{hN(-4xP5k~j;G5DH>8D99s%}3e5k(#N%;+d8=HZ-*izt&m1urf|+kz`A*9(C2 zh5po!d1fHhTl^$wJ{akVh53~IY+;{a(*w0wBWEO>G0i_wUdp%ErJ^?A70(RBtc;kX zUT2lpk7qdF_<6q`7BdKsZ3g73xpEFW9xJ zc?25`H2?{L@aAm<=D`cuxuB(5Cb<&poyd5FN{S9tz~yc}-f{)NHf3jLzAAr=Xo|;l zth9TzXIg}L=aJ|O#@wZ3%%7byD{~4V-Y7vD{H4GmwQ#cRZ-X~eax?{mT!kvWwI!*% z?@YuW90Yfyq4jjrBZOsHs~EU(Va-xxZ!XlHdqJI$4Ay3AE>er-am1gOA2+Xw98Z#L zR+G|0caHB_rO>fcb=bh}XRSbJOSE{QXCeAflfhpSZ-H!f4ww5jY zh~}?Ae&-bv`5${xFpYrA!YI#obX3Fh@p-d|`iL4$L3vaT+;}up>v14WsA8K;6>klR z)U+E>`@)rY9V+zveZyTO!ohnoU{nj_Hv9X%#tWBy_w?#2YN{Wc*7d)3DW8vb@3maR z=SwIBW?YL$oIwV&s4#;$?6SY&IlaDRf1M-rvH|t9RbK45TiX()WVfG;47^@}_Vrx4 zoZKQ3p)zUKAdE12-Hb-^5aE-#)U`VE@P?cy0L9PCV~LxHH+~!RW!XDkb}2?V58Z@A zo;l0b$9)gw0(SYG(FoVILeCGR!@sbSg1R8gnAa!4{E}Dle`)^EVcHm2+NOqsDF3@i z82Sw4GD1sj5cKX(ert-bKIyd3yBz*&AO8Eo{zqG?HbZG$vDz$nF#ilr8n7JzM(#LT zl#U%>T0$XEF8(Us-`fQNqsm~s(w2?hH9E;mXI;A5L)_fDP~aUp0jZFJSQs?o!R+k- zJiHQm*1~UqbZ}xvm<6SL+rTxHnO*pCxXxZPH8C|Up9q<3``GMULPd%XsikHuvAW=w zUPQQb8ZsS$EK%@nmLk6YL^LlqG8?8)*05+7Lha=Q7^c^^UZl9xzJJ(3TMmiBSiFI)(21<85Dc-+X8E7BBW!ZIRFAA^w4+OcM=Bdz?3^0?8vpAWOc5`|{V-J>{Z(;Jvp~R(lAHm>=TiJ>pDf zlpFIw<&?4R^O}IZ;{|67pY_%)hC5o4LK2zJS4kL*a=cVXFzw29YO+=g_IiBFQ;sl| zyJgDqC4sHo=;ZcO%?_H762-DBLtThwJ-vkGo>kxr%wzH(gt9jjTip70;q%=BCtgo` z$~c_F=_bIaj5tewsi%3#W{VMS1ZG}=^rouqwSvqGO)#}6s(Qn-v?aIRp)L!((B!E44qLn1-{ya$R(sB--Rcf* ztsP)Cn;#qZB-?0yDnn~R&mUt;nx#wgR3Q)HV898g@$*V_O9|X?E^|oleiZ$T?y^Q? zFiq@HUh@Ojp}9`y`PsovxfU_KQGfN@#<5xHc?#CCwvBHMBPO)2#C7nHY5KlyeN4`* z+6}qxFDO-HFn~F9o6geb(dbr`zJxy4-?T?xL6F40zErJWo-OURRpHM zzUks`zpZ_3>Pw0;wl-7TnW;fUloQ0`@3Jl>XKNb&`a#1JFO;r zi+u3et<^&ajBR>pLTBe=^R$Lwc^fU~A+hm%rUqRZD#9z-Me8aUnUh>%EgYPyk~#moaBRIe%;rYjkk zZEUz7jW>36yh)(zWjdsh-pL!Zs@LFud{j(w$6p=<8oHr|fopm;fM*4Yo`55VaRSUf zgJT-gxg5!FCbF1~MWtUy+mgwRYLhXwoN9Yw{H&qXscT#nd!~Lm8|1qIbtN#5`Ov46 zU{T+p7!?`!a-F~y*eLhA{-yNP{PWYNXf() zxdI@sWCHK>3Q9Ggr;l1c&g#OgNYMbPbS~{>vMea*O*_p#$Bj@UHte z5e?8^mx8BL&em*?zIa(z=AAb;BU$S7V@X&Q;eQWRkwF|z1Qf6(ZlAr>-8ehxjgaZ~ zD8xLUI>62!Q*=sY3DwJfmS(OGM``RQx8C`Jzb!hgOfoJ3&HA*3Rh@_(tvV40J!Xx8 z$$(ZuGSfDfH(u^-0>hI!J6%;HAa-pth2)*9i-&Lf4R5C2l|3hKl&9=$JiH^!&311{ zZPoco+gWG!DWmqkDm&nu%mYu>Zo-iuE+Qosgjr!5Qg%jykjgn?6+_Kbi#i`0 zBvI#}%cXeQ>N(xJx@vjNLn z;I+Quh?s4BC#y&#a&$9phFD2@o#~I%Vin;iz?*qKlX8xX805W8G@$yzT}LkHJ!zk^ z{dQ}Hz4iE96o*OGMLZl!Y!9%3< z4IXD3CE*->0a5|#Wy1RJUJfO7P;7XTyQVeIwaev62+%j1`~kB(llk^I6CK6eZ|u6u zTya;o=9~0YGP4lHZ#_Fdy9`_G0pcK^QTG+0B3>CYwUFqy=twO`8!MQ9l*|&#2|g0hwE!lOTKvTt<>WiN5NzBr6trE(ws(V050v)1l!+#a3h53`mw`hO&k@XPv{nii>ydj~U8!?+V_TThOZ?NO4l{HSueKt|vwO_sO1x@2?|0E}SP}V86gK$coeOmPDIUX`Fr4*^Pn;jsk zs$m!jh$;C@ogU-3aMg)7@KPKQU788)2NuEEne2w}Xp1GoUrN2j098|>Adu|SQ;}sR z*Xv@Ekv(DSNK7L{Ov!K4wu#{gh<9oLu^)Y@;0IQ52by#yQ>&N$rkP>R zpMIF{+MX3ofb=?}y&nn3ocjvPul9uw3-p$p4dUz`6QYz7uX4Dvi*-2G`0-%+6}gSF zY&=6!A~Q$(+xmfu4%tR*GRZ-xRH2=R$H*wW%Hc;diAdu>ckQ3pR(>L}{1J0zH-f)^ zE7ARU7b7lv)dUs_&HLW@A&Eh-yz(cOpj@x}V<#&t!{I6<*c?+K%cjNOsNKOu-++^R z=*`N8N>#{1s)?U2bF@qA+P$ire4$5VkL*r2X1}oq12KFG)bukjlLw@=KxA&dn5)r3 zR6)VDK)leStMyLo_;!7^gOZwz^WAvF@Ap|-H%!{LtD{AV9+7n#!kzau`eh;0^w5`} z{EB{;JmnYPZb^xr_5-q8I(@YG;IsKk>}Dp%OWXoB>Jzf#NpP%#)N+8ySTjka4Kj-%dWu%|6kSA z-xo$ytM7WG@D#lazC4tAg#`}o|N90Ob7TvzUmAy{Cy!+Teogm~Q$tAq;k3}uJXZgU z6dO4EEoV|lYJY$9ZRQf+;hBFr6Y8o2jL;#%o$X9<%ZQ(gD?JLusg3`7BGh=4z&Fa2 z?t~=r4JryPtKeN?nDi!kUau_csspRIUmazKl$~A>nW@y=n;K|va-={DFPahdNPzn> z*%SbgxhFl&tHNLT($9hX0S6Z8slpdUZw=qyL9e(pESI_DB3b|tn;Px8QykDc+0~$z z(6ecvtaPIUw4IpJBLqkc)EJRTKk3@oAqlpEZY9CRQMyH7x@U6Ww8gkrnii z-ahCw>lvRgEkZ9jsAt`c@WA;%e6Ip;#MwoFnNy#Ki^#Mo0s~MUv9A*FrtVl*c#B~Z zr7cAafvjSCWu|w;F?_0KCGn<`$*+srFiB;HhDP&w6h5i45Gkx|X{xz?W5owa3{9ky{|6v}*r6Z^vTj?@K5pcDp_Ixt9svbo~)vGfGJ5rZdz7{SKk_N+Ka19v(|(}jQHLB;W_1l z^k$Gv_&7kc^r+1_$sJm-%2fWV!br=P zTZo^9T5>DC=Tu3kqnmJ*B>EtoHS1b_;ip>1nw+W!Fb^!Zn&4cmk%d|SiUh~3SwVZ* z@FsQdV3Qu9d8y@e!cK0JM>i|9TXQ=mHQE~fmUy@(O#!i)xOBUr^U^|pQs4(8n|~`o z@;{(PfAZxB{>C9++zsD#Y8w&AOh>+A|Mz#}(;XJ55j2lMPC9-@K<;&y&OH(Z|K-&m z?$ZiuwV%+`NJF-o^dwMD=Q(Ppm#t%z|$@hvC82>^K}E1g!iYk|Y5 zpCGoY2YGlyi3;@QDPDO2zBZiTl_doU7oi%(+JoAZ0_i{F!*9*=-?G`w5y4Y6m)BQ8 z4QDT?8=6LD7S#7D#*|g^oQG=_B?o$oi%Y?Phk4ab0e_E_!lwKR7{eB}d^A(M3x8t^ zvD`b3f`k|HTwH)|!;H)JMxCXy)Qizw5My|OP z2-U0vovwL*3O()RCFQ^GyoF_;^#P)xeRFX!Y*Ron;PM39<>jb5O&d4GlcUOzb@fkwStkMVbzF5yJi{{DW~$PYdVAgZ?QXMQLiGp=@> z6%H#aD|wB{yZ`LM`xKI+ONV&@>_M#$!vt#u|K9cf9(=MukWXZ6@6rfIERr##qvfV) z@;(=9&0#?E<~VqW|MTAhYU)~eH)ASiOQ2c9OLvz$MkH!xxyRGhNm^ z&Fj;ukklZ=-M^=(pNc=c z)EZIg)}a=K>_+0enUsSYw`{EM@zAmVlxvY>K<=7o`Q{T zb!j%)e72&SmFl{`Xu^CqZ*5(|Q4C`W_LNnR#%%AT%c{?tLb~A7J`fKGJ0t01sT_O7AN9Z!75cAF=7-UfX3rI-$aehthBGOrk>GZCG%7)7bjNW_5{_Opsi zd3(6epFgjstUTnInw>30S~VVVy^BEOunf8ez-(5PCZi;+^n>w?Sbvi2rRC0>10U)8 z!U9R$q#v&8S}z#}0HTWi(^*3IM6TT9K;r+>QUB-Rv0TE}3LT9G({Tv8$${t6XM&fZxt~0J z__V8k|KM^3fC{;7eELU}w0zs@_Hf0p4h#l@UAI8yL|%)+>M!!&GhuDFL(#;N4(Z9d zRNwX)931JE6_q`r4a!kW=uv1sZdRI^z!@7@*?d7VexZDb<|BYIg4v(cV$?2}im_@8 z8yE_Z;^`)!8WG5M_1b)L8B7!t?!+_$)oX9fV-;}knnc6WfgTBy`hv~2UjQibV-y+< z+>AU59%}#ounFJ5qsO(J4QqTr{8(YZ zX8fnS8nRq=j^3^JCw^Q{)uB4%Q+f9#x(!YDEkz#Xgge-XY?(K-P=(AB(aX|M@iXBL1TH?mB<)HuOM5J^qh2!(UIt1sw@GqYxR9 zDDh9#$=~amZ1r7-l&6J9R-dj-JSND9l#7F-AZz9R%D;edDLdwi`Rl9uhWXFc?&)Nt zIOxR0uIL4SNtP(fu4P~y8^2o9ScrHwVSjWdfIrPdI>QLpE3$8HP>O)pdx;E$_)(E= zX&8^+`jRBRCstH1YqJ^i(1SeAb*%=!J>d#IpzGz*>o9Wg@|AtrAGcYW% z{0#&CSROMh(nbyba|lBj`i!1?>94)K6v-baVrT+6`(0gK1$V%^F?3Mgs>lEQaGnJm z3i;3aC_Q~2{;(AjEaAofpt9Ce&` z0zS~`ss6R*F+fj!DQ?5^?>#Y*3hrK~m_mM{f+zr9`cpjSYRtxaflB`O|Gwp46=Nbj zT+%nWp)p;{aM`HvFc&&WscPE=1Re04Gccc)g>iQ>9F6Db=|e)b%TdJ zeYD4V0jB$zn(b$d^k;6gj*i64tbzk~#OHP)+l~tWV(DSIEL!@x!6U2gHuU%FptIu0 z7HR}UMAfdF?R_i?faf%ti2>WX7sgCcZ}JVf?=ZF)7ycs;`Qj~A{!S#)nYLLRS!*2@ zDgtC53H?ZD5$PGr+glfb3>^=TrsX1+Ir#EmCbX8zIuN&UZ*?`Jpy-UpL%1zHe%m2d z#EbN{Ppa#>^%*k0v{XuQOnKH)ODPpmunB+_rxJn6)w=0a()OUVu1FX3S}Cf$A84Nb z-$Fq}j4TTO`S@Tx|BEP){`~LSj%*!%)#6sT%l&iUpZixSg+&C*;r~DXsboGu7~ZD* zjCEZX2|&-PD55#<3d2#5u}-s#CNKFc@dhKBhYg_CTMkHKvY|e$Mul{dVDX)axjYsE z#Tu|EeoD)ZVw}np{?sc8YZA)nx}${LDBGUOP@89zYv)Z4w#rTek_30xz{*MbZ}Pi={*@;)P{ z3I)xi6w69jU+-p_O69{ktx{!lpZ+QNQgsa4T6)GT_|JobL-B&o@Ebqpc$HMZ>;#Y0 zBCw|bu7#K{hHjX%wU#L%rs-+N&z@);8}suYRs2n_^lA+{5n^<~U~&tQrzCb0)w%8A z6nSDku#!@S_>&DP2y$1g+x3t5w>B!Q6U+^xSJ)ehXThsB|Myw^xn}VLP$oitiX{1h zTyTCbf^_gWDYJi*pOH`qQRjUQEhyNnBnFxPbBx%AjTs~_jafK%4)4EE5x=h-xJxn$ zIt~6RnffD<7t?@uw)!+|YGDh@p%Xd7rW^OR3-5FCD$l^+A<|Je0D{3$R{n=*&@J5uPWLfjTgh*)i zK9cm^I_CGsDibAC{}XiCHp6wy>BwolNhtJ49Rmf7ON^DDwe31hC8k_gdvPU~CE2Gl~PEPB*8s09tIm1na?hQ76b=0rj3`IKmq zhH05;Xdkd25bhw+1V)BfM1}zwh?qGQ*@d!XhUD+1B-I?Gz$gt|-|F0?7N0)yMnxyCp}wNBMHnzto@-WOb% zJ*Tm9jQCX>xXtS$SwlUsJh9Q&%9_R_@nf;saN6g`<{ty&8CCV_!}5{AJ5^uyhB7RF zzPxqQX&$E4d7oaBqT1!M3AizFZ{@i0Y3#(KXK)#asV&5`>O6jj`>*j3#A{WnUO3ej z9w5y~8g-{o0CwFkL8l`$bOIVd1C&LrKxD|7HE)Z_Bh}uRNUy7(tNJ^#ikbMNq{5g8 zOI6j_U^{(1J*xqF1_t@JS@*&LKY`;AyQn9b@P9V@eUPaEcgK~KN2}3-7{tZAL{q?;@6KVb&IuH}oBK_&!BKaE3ulXNAf{x<2&S>RgQTAx{-vUk_RG+r3}ChC~ON8>tf z^$A~6teF1L7TJhw3VObdCw`V#@f5DCer zEbnI2U~;higX4L&C!mI-Am`B?Xt-9%3_~PMd8dZ4v za7xZ%>1ys9_!3&esE8#QHf7IP>_RcJ@^m^dP>WTX;U<7E!7ce!64lbt9BGu~I=xjW z@JD(9w8t`Ed9sR2MQ}Q!Nj9FJ4;6>7qnKFHkW>y3mxA<`D}X z!n0`iR}|ZyP1+Eo!Qje_YHu>NkF?~%($b=R)N>aWwbxE{k1IQi0y}9*eAma8DHg6P z4_4blU+qWOO}0l4ADZ3`T4)9y(?*U};h_e%( zWCTn%R30r2R8T`BuQu&nyOm5>?=Q^n>>PV7l|Myuw&SE|BJk~4f*~zg6-jnudUt-uUqaUc6*5LxTGxtz@Ui?QQy# zUOQCEF7w>YNgvuoviLXHP||zGFZjg@sU+0R)U5YD%$|y>g|uPxUQ_#7Mn;CJX*0OZ za``oA7zsYq`#QzALcj(hN%jF==fbE-*;`6OOJb5zvgeF8;ml^@DIJEwyv(bo`%ciOxZ$B zg1!z4GE8>fwqE6#y}fpm`C>tV$WibH_?RezISxQP7WVQ~pSMHn^F3WbiaSB!Z@_L` z{O1giroCSM6684E7%WeW*~hi>tS@zZytqi2D)Fh%x+f)E1ozV7%GG)z|H(wqbC;M} zdR>@s1yaryb|@Fcl9*OgW0iah!@JMF;<7~oMzf(8`%5Xtb!5j3XDzO&C0PV+;j*92 zD75_?01Zy1onVh3(m~nP7pms>#V&aT^Ow=8*-{)e+ptowY8XYgEl7_=BT5!A$dQ5} zEcvk~Ddtyh#0d0vK@VI+s&huPS)?N}kKT43a!arnisBCdy8mu-;?{*c!4vICX zvyQ3ks0uG9g3tHM>vP;tRx7JTbTf>UIHpY`-NPwW?#^rUv$!?uH@hj84irjCN@O$v zqy~3)cf0iyTEr^RP&Eh#o7$r%x@pyJQ2(3CU_J#)fJ}n~}pqd<$ZLo8-G7I;a z7{OC~gTXqZ{AP^*XpF0X1Fjt5c5~okn!gM5Wy)iz) z_wd7!3b;r1kj13_dcZgX8Xk$W*QdolpB~s-=<9;`&)SpqcLh^1$*evrCQ_oyP;r{` zLkJ>Kt!F<~_)$O)(7far7SbStq>M^F9v&{eh-_#%-9M|!s$x_4l(?2}perGscY><5 zH@@XkI&*l&ZFpYx;UrbO>dUd;lB4vsTJ$U+M;KbM>;}geG@H5%jF|57g5JRyQ@z<#{Py~eF&V;-b>;`}GIKxr6%jY;G?JQJ zXw1CXFHMNSbg|}Mh}^_iUffKO%;X0cB(bURKD$88zH2HM&+N=TD$3@PHO!kCCm2v` z7k)3FIwq7x*2g%5#S0p)$J&wO^l*3hle=CRMw))89;1Hp1^WZ;xOj*awX$H)<$!YV zL;80$D*C|{dy~1Afi;ise70q#WVu%(xOK0%=sS3LvpWzW}d{5BVg1Rhy%;&OjMQo1lH`XZI6^r`4wC z?4*Z%%vrQ(hnQ5$@9D7v@ym;)(=UmXN;*0vQb;u(G!y(&?{HvS07CE#H}cK&^s%`Z z-dkifb!BD4&49G9;9!jY{$(#`JBp1BS|#3&$Y`gHJ1cQ~Qf|~g)XKzbq(7`*-~#Ey zjb(flK*9z_*OELvSVhh{&fXC|NQB;XB}CXded&Io`&_z?n#OlnjQ zfOiS7WX1n%yFE4yLhsWpB0dDkBd+G`aXooDZ?L3};7M{!ZB#y^?n?SRo4|{TwLQJI zGHnj~sSasg`p!>GEVq`r7x*Y#QeGz&5E01x(IM*9A#x58o>=dz=}ni$#@*g6aha#u z>&cZBX15?!L_==RDYW0hRER#P1I|~3=-c6EiN9_#?j;wJ1yVS_!U?VCZ^@tV@$=wm z1fp}$J?pEyrZkNoQC&iiH`y3`qk>vvBK*g6*zX1i1hT^20@bMZGUUwE zf~;hp2W*Zp9v>lDGt&|g!~zap!3p>;Ii$xrQW0!0B%{za;JL zwE9oIw;kYY#lBU+y)ERt3la1@C6`x9t=@eP7WU_z6zn6M}Ff7mWsdTY*+>b1=YEI%|n4;LaS zR9ODNR=el$J6`U{Ma_D-Rlk~$gNLaDPcisaimF%n1hCBTC-3#xoWE(b2)q#g>fu3v zv~#m6fH!)#%blz``dHZR*A}5@T^Fi{b5B*M*W7qVh_#%V$+05k{}DtTPI+m14q4XF z_8ReS+2)Lht$|$bOd&vfZ&rzm6S7)?ZTe|>$0w=v2_qPlRDZkp=+lneWrBRs#m(%S zkV_jD4JZd#99R9^+x}(C>mMl5a3`mq$u95Tvd`!2D1Ik(zgOzbzH8@wpA7MxCw6!( zg-Omoa_z^p`l|C4Y)q8Cn%b6hU*KIcTa{qYDUj5Wm76O;BzSo19$8de8xm#x{Z>tj zR{P--c9<>X8JOOOMF#)b+224pHa5b8lR9q_O!t(>^0_cf**!iJGgr+MJ%sc+Z!_8ZOBlX(M= zCbCLn7C~H4lM_BEv-Ep{Y`k=ZLH_BH-5=%2nT~_!HS6u}VZv1CC(r5x!IUUff+yK6 zcc(K%Z#{{uvXu5M#Zv>4>)9T5e~26WKBcV0g3So@#J4V3B%fYYB`ht#GdRNw`i3Z| z#en8Vx{yh|Vq==G9y>^%_Vu>Cp)~V5(yF7nguvZ6FZsLBzE|Dfb256u7_pDcu6|AL zpKfpl^~hr{6N~eHPTI(i9$0l5$abBqQ^jA?&ZKIm(NuT0Ut#ZDwVItC4_+kdG0kKX4*qx>7Pd$dUueo(4_FF() zDE|HdVq2ng(nT${R?1ZZA$IgDi%wB*KU3<8ZS%hPN^bO^`iK6mKK-?%73k^mX}`UA zD@p}NYX)ac%b5~zP;J-imh|$a7!AcM=|=n)cdW>`@@nqVTDIjDDcr7^yJqG&Mxvz`=JJJ#E^OpQy;jq>CDZK4hVU zG-l$f+CM|CMn!wlUDtEPms`0C zJmCr#B_k~JI!-r@MsWbr$8vNSp0t-vAO`+05?M$0f!B??u!=lylJA+#4qw8(W{&770TnFffGZd0Aug#Fi0Ik!>VN_X1k;SFKSxy8{+n0$wxm zqcXs0X=(9p40Eqw+iwqz-_!eee5NKRbyey|B=&$aIcbKidLnRG*2A6>WL@ z+L{3NuSx;`Uw^a_z|ydJYS~U@7<=q2gA#x;;@^Myr>OYrw-Yj!ZKT^8_c~?GyW5=D zePj{DpR;Q}NX-n(5lP5p>PjyIsWL~G73iCwkF%mwk_&pu#pMgnGF|;Ym3mZui{qQ0`3XlsCxT++47F*8>?476+cxfuo*=U_d&m-vndtpbk+T0Xqp{d^}lo0ZaZwm{hZ zvhE3JuZ{e6&E~A!`-XTqbwd0`s#)pqk<&R?fKYErUT!CyBHr_ftC_Ul|53|ec~2kN zbupR<@46cag&V$MKDnvy#wIq&^I>O`(=SNE#&jMynXIqUSP!ms)_*;sT*1Rzda5=O zFI(f1B0>9qY<*=^oZGT379_YkbQ%cm1R8e@E&+lDm*DOYNN|_n7F>e`clY4#?(X)! z?0wF?``q(>_2|JGE%d5dHD}G5i=qwqYyg76K!U?CH@v^x`(u^VbiZG`o*l}B_3Bl6 z>zwcB7vK`BK~ZB<)6lp$oWW1{OxiRH)6>%-02wWpYCk}maE?HJ$>%VdyhZw{LpjOi zzxEU8Gn{AinYtcUvcJzP>^j1XG^y#S_-JZZ}O&Atm7W73u@Owpa2&2S5W)Wfst%2b=Pa?r2r zF*>vI(BiGAvnwo!kMX*!ql~eXtAZE=zT&i+N*VCpzmi^{obV4wjyX+ih3Z-G?pMq*mvb4}KS@??)Ig zf?j7b#GTbWoJDnF*0fn$$yxJ7WyGahGCMD#J26D0eN75>h`5NBJ0L&6w3O6j=eD{1 zXu3}%Jo2nQgLI!iT5{g7+YP=`_t!2U781NkuyQM3p$|reXRU`b9Cxn8Ebci`nHcT0& z%ss%7$8WF)O*gmY8zY@YTD4?m9gvf_Xha0Jq@A#2ANy66)phRpoEZ{zCTta zt_Y20ZCUz@J}ws^*bSEA#Knew`+mj%gx8T9=E)`fBeI)j~reYExSW-!R!-r zZW^51FaNpsfM>wwOzpefjqAVtBsxg?*S7mtQ2fsW3XlzP&RK1u0@)jM#2%eI8b!}! zqH9$gWTc}Pt2iBh=JQW-L*+!a-57oaJ^J@@jo*s(QtKDlGLXh<~h|* zaFtj`(r-Yf#ok9&SsyH8qSV~j3tj040qG0e312<3l5<$81%5WP<7;(~s>j3}gDJEN zcsHF-aZ$;%XjXdE(&GiNS7$?K6KYP<_JPX$C{Op5Z2}d`jr!q{sy&@#f zv78VgM4#W_`_cvH&;G@2xdD+miu@D74{**Pir~vv9y~#vZ`yD>D6zMRB5)4_=QlYa z{FD10=FQz*9#Z$)ewj@rX&7c+>m=$4<=@qQCN%JB)l^9Xa-6VJ-wxn5hER5dGH(6+ zY)JJa*>5H6p^dMxJ**CIM<$GvOQLz~PnFQ#ZfkVbOX~B3*broSN>>xd(jSyLxGN{4 zt6)&jAa&{$tS~>2pfUDb%Ctn$G#j^EWK7mLq6Mug61^99nzBu9Q(U(44;<|PaLPk! zX0TXCQ?c8WBHn#(D!$-LK?rudWrGkNx!y>J;8jI~r|McmC+y#gwO1Q8Z;0V&wH=XA#*XrEki^mN`0JsqzzP;M(i4 zNaH3lW1rI&jqmB3#}>bwDG#Mc`)a4DHx&gTw9CZw(*;^H^5dB-PQlU2`CD55*4*Nq ziTkAL+t1CBVp?9gvsGut_ses%Y5bUei1G$=xmA=&gd=H^1GXd1KDS@^v>Cism|9MW zFBdD+NoE^9c;~P4QZ(yV_Aw+gnUvpVx7|2qevlsZ!JXtH`FM61Ess{v?I4d88_8|@ z+Z2y4^l0fe1PYo&$%LohjWM`|_l$(Y{7f_3>>5d6=Ca4$+~`<3LUj4G ze8p_1I>s#nQ@ko-;tCx8{{4~n+1|X548zC0$NZgmnZS-DYA?TKOqfUwSZ&}FHqEx) zIgQDBUH_hM=k8}pr^+L(9mh(NR_IAU-1*o$&+;cf4C|!z@c%p&Dtx%ytflQR{0>Y(Aa~P0M>GF8kDInaLw1J*rJ1{Uqj{Q9=IpXfC zfg-gfq)v2{nY%M3EYe$)6nz_kjcXbe(F26s-PIU>&)X*~mSiuW`0)HIE23L3BP*d@ zxRV1?-*yLU#GrwmMocXtP{u6RL%s_&P(mn1^qKE4A`EH>XKhi{KM<>xzfweI+7-Hz zB*;3*-uEJ;LTunC(`t#qM}v>QT1HqEjMo+^v^N7X8K@dHV$alcu|6#bMTmlqi_OrA z!fM0Xq$Hp$?@L>s+pNO$fu^QM>-O4x^AIbm!DBz5Q-?%YJt)^^XOFHzBsO>|1|eVaDVd%TY5y!x~~BtmF` zexFgky5*k?Ng34o3=+Ge)G2H%6NRTqPOkiJEXhlu2-1kveejNxEq~UNIQNLWv#s=Eu$RUFGHhD zV}>7a?T*x~eH1?5jgm{3J|h@x2WO~@Cs+7|$S4b+~0MToDT?_DhnyOru#e{p%s$DEv;p!{@qnV8dq{e4HmzWYBS zF0bdKPkZCJFk+7n$BhcrwY8z&zQHd!FE_cZZ>QMc_4nsww)gy9&;BFoM881w-4>I% zxLtX5R}v_Fz}ku*CH(i>{{8Mh$o=lRl?*2Q1o)M5?&cIb+mV3Wfe7pOdv@>>iESOn z(Xm+#jiv}76#+f2Ps~h+h8!#@r!ANdy`AHX2%46M$Fc(jcuJp!+k3CoVnJ0>4mZTNzrOJqjEM> z+vx7Wc@pQ3N#D%F&93dLic+W>EMxfy6~$^7@J2DG#Y0DV86Lj( z4HEOY9kw`(>v7qj+Pe&`0`nh{7p+eP>Scxcd`DV3{d$rym+RIx4u%OV$u9q z1O=?TRGufJ`%x;dbG#tf^(3(E1SPnnEH8&mQF`tg7qTK7&w9G-fY$ z@3xnIz+mM00KF>f+lSgBHCuoVQ)9ncUE3>ACK>fCc;jy>t{rprKoZ~)hic-=WziSmw_iPH=TuR9wzshwtMBV#KSF%mnBIbSt{2Y~$W?=$)^hmd8T`_C)#yR19@Wspw{lK$KBw=VD<2)7hg*uo z7!PBfRU4QRR77Glh_UDRTGGsFW(4n6DUhX<@?=Nr#HZDwnXThdII-h1v zem?paJ=aW*z&%hQI@82q) z>{o;h$|S2>{V?ReD?^JMZ>1yOw_5IEgI0cCJHN^^07|s7c>K6S9*^vmSit5f#^*6^7z;EL37KBiS&2n>6UeGYbdq zR6Ux{OvzP&L!Vexy;`?vDXOv^)qMWCAij2)>lk{3ANZ>r+cNm@wLdKCzV2ySuj3{< zV2E)wI%~#zTuC0(K{#k+lsrIs5~`s45DJmi(l}aXgrOT7C`5LP?m|No0DYt%OoAv- zAuM&+p&5Bjyg_=s??*Ec=->Wz#Vs|R+P!NzygcYi;jx@-hw9}bM!u8Qnx%N5Rj$dA znRXtXU~*yfe*T1U;JIyQ1D{+K@TcLIuY05hn|6?lNsjnKRzQ{Xro06=a)hr(%^qa05wJxhb0~MSy zgYxlXUu;M3^@nvDNZ?+OTP_qDyiE?uz)qz*vWdkHVlZepk=~|X6B`aiSnb;Pz~8|k zw+_=R#3!yN=E#KP^6!mt2n`;GjK_t~SAW%R&8t{pXl5BLzmDd$YZ)MlDQ$qyXW-W3 z$pA2KP=rZj@>vu~9Ix@;^cBGsqHmX=kGI*%CqT??usxJ?&c5%rj`UZz_8%e93FoI4 zS>Gf9W9<2M!=t_9KhA|PTDSqJH!qlmJ>0F`YsUh6X3p%zIoL% z@LE~XhBYN1^;@hjxL5QDk`On&fs*U&NHXG@ZMZ>MUm1%S3d-%ezU<86=#KWaq@<9xI zy<1e)7Zmg{Qn)cNcp4$QAv^-j%18YM;AW!b5n`e0odM#h3S)?x+@mZv)T;btD%;ts zC;TRqPgKnt00@*7S6dH@U8a}yJ3qd^LL3;tcn7~}uCe#1(j7;&QwV1D>uUh5l;7=% z^ORfc1mcN!Q&v>*LiyQ}1UdcU3{oenn-kfVY!pg+)6p*Egq^_(ra zs0YdY*73XiAYAowy!}REja0h{*!7dz^n$?OLYNxx?ArcyKt#p(OY|?|CS27WJee{E zJi=x)8EYcM;$<*Kyp)p{Gns{FfJ0136_nnT{YHa%sBNN6F1p5O_sTBg!U%|E$Kbyl?{dT6&g2@>7Z$T^%1{SaKA@1=Q!RYo1{u&;Vn z{_^ytCI7S66{g+z_v1+?Afh_$5J%Xq&G2|<_!&?Aa<_3}MJJ`l&xGjqA8dsd&j&i) zJ*86JpZF6jVbE%6lfD91Ci$q$EJ(|dyi7dg};zIRjNX-*0Q=R zin4xWR@LZ!H^m)|dtD8Rr^?%sg7H!PT*8FTrNnUo`4n@rS^)h{Wuk*r0juyE-GNyd z?op0DGVCWby2f^m0z=mR1zWUJQxF-Qzo>A|@YLAhY_6MJH<3K1T~18tkajN}x|YLI zFC}GtXwYHN(A08HjQhgM3bm{xt`3F-IMpM-O{XX70twjvl1qb8=wNzc3YKxq0^Xl; z3U6!%L|9KQi?3Q_eB;Q2<{>zx#PlwOCjMB0TrmdhecAgL`~G_Pmd6^jWG`!)FqyCK`FFAq7kiW$#(c ztQUL(zzKpuv-z2{chj%15DkJmZ`%L**81Ej*kp%$ay)DDsHUD3;c1 zcU&H7?e5yndpC{(sc1YX2%Y@oZeZ-p@iV>0=(1ugS3K^sfs7)Z`lXvQABG3f3hdWR zw&NVAaBd!5B#`*By89iIcjcG5u=tDbW!YMa!ghYw=j-G;cb8qgjE`jZg>BZLnG!~w zd-3~pxFIS@r#BwhpWQ}|;57<*>PEh&`uT0hgYkG&ex|ES-1PNDLt7L0X)(7a=hv%= z2xGPu7=Hi|Vt{|oEF4{p@C5{Jl>JwS94S-!QoQe*K_HXL!I#AXF#Xn8gV%!Ca@?zA{m)pJ zhCxlnL<2%Fp#ijD0USG>BSvK>a?2mjZTXvUEb8Ro#k00(A(BO=2Y&-=;Hnc@>KIt5 zpE>)Ay)fg6Y*8P)Xm3Qg)pf`<)a`=F*qSWJkMVZOn+toe2($8w{Z;MxDTOaSvd_>X z>ffobG2}qRKJMDRG5CrBiY_|f<{7DSR*cxity_+-c?Z$kDdLMOFs!+%7^fKo( zXPGu7%#|1U1VQ5ps>ZY5^3jma@Fc=H#F2N~H(7HuRklU-mTm$yIAXTcINJMGBO==l zaoijbl)I{r_Izsam5t>duUb~*{HtJ>wP@lL#7<%%@KSerbsJ7fH3y>7&&)nM(igvZ z!p~Zs18stt+$GZ8)_0r)vWXJ66|Mw)_Q4e%oyZM5#|}Zytg3isjuBZr0yhW5&?PC` z$<388;DIh;wAvzEspTL#=uQtJ62Da;%a)gdhS~KbL7s@^Ey<`@3lK!#`nK_ldzo;x zAY`k<7qjb?{@A7N#({UZ3%S^nzqoks$8CJX1um~z zlI63OL$$O<3^Yu|SRqy+QvwGYh3|)4$zu2zR?OO#mIXKi=JYAqFI2fZ-xjE9P5*yQ zz#u3X(1u{;2wmNW4wk(Mc%*lzxe6#2E{t3{{0nXH+!SCaY7;_Or$oo~;kg4_%1FRn* z9@Pwt<*HcycAhlm>ymaDq^GcsmR`AUHT{k7nZ}qFxxa8kiwf$hqdl6P?0?3T3v@xo z*!YAbMy77m3zSIF$?76#2*KGY*;unk@7E$;9kHe#j?d);J^E1&ZajXGO-oh*H=?`n z-L8=fPWlIw0mMHfiG=4%4G?K;wA=n}?a4(qQLrM#y!K+AF_hlU7?_i)tAQVX>Y#GY zp4}Hk_95yViG$M-UqEx!L^EhoSKIH11TmIiHdRY3wOv4k;$x{ozN!u#6_ebeM3kf@ zP8e@LsWoy2O_qT7qF}(mbMha8+I#@CZ5t88{(6%ikurIoKgEax2|{@p^+W3Yut3L9 z=_RwYJ0=0MOFJTo9GvYEeJ?{8#gDcp(p8Q;#}EVdM75LUoMNG*WqfHlbt1}&)5RA& zfLMao!%9fr_QHClIKhQ`#TXw>8l`mvPyK_joR8t!#!JQLWbOcRBuUg*#%YNY4WT%X zm^eFfQ$+9876814DiL?Ct^`Z4cAhgNcXf~K7%FQ!y$c8+SGcn2ZUUKjWKycWPg=`3 zTQ-7BfOM*=WJy1QSJjC%kN}*=lSD|>TevthNFX@1Sd%kx@mAydAUF{zAD4gxC(amm zSW)_fQAYt(F@vIA-jZJ5O#CeYn~v#SBR`seEz_BC-KcL=@D33*yPfdeA8w&AS)DB1 z|4fH`&r*jJanwaIw%7E|STJYj^tJTk3P3nlxWvA*Xm5;s)-Y1q3~1^{HgXX2U_*`g z)gs&MQD{G8NrQAJ8>S|O-dnn+x^3uc`>dEpVt*E))9mqMzgFk2r%Nn&!;t*sYM>qc<}D)JEb56PP?aAXl7NRdY)IFLM*QyOP2pD}71(QWnrJ(^%d+wci}uBf6T zVaR5K>hS&xq66X8rXiv~gn4dfFN~%vmTM;l&(`06E3E!h5ZdrI82=&97aD%oSb}t;Y7gI0~E%6$CnF{VJHAbLlDF zr@jFvJobIQsA!`a6(I!$Ty1bv{0>TmZh4SK@}>{!hX6O3m*RVQsb&ZSwo)xdq|eZF9yf#SD7o z#hVH`B@-=Jy}H#hIKu+E_#0#w$3I?v+-M20a2|^n35jniK4e+Dl=^&Vy30QanB*Iu z1#+iP;_i`i@d-Rzo%oKAoUEpOi|QbMwO86R+1J-kfuoNyB#rj*qt+emZKnBeaC|8r z+b3Ev-V5AV^-kZW-|7!9865aLZ)%WzeoKM^%L*|hHCLh3$lpi?`m5F>o!YS`mo)j_ zvc{jq$_XRj^TmyykL3!mzsD67xm(F;H@$u`qIvb%!^6iAX;Wa&QHsx0IG7~Ee2_V2 z)a>N(ND>B(W+2nE6!Fego|TbPU=9c>!#%eZ4&1N(LmM0vnHaXHkaPn<0t9==%7B6- zd$Z&a(!v{s@EtR&vxtB`7nVkqL48B}v71oooyvOgg(uY#l2v2}%*BNJHNU*78+6>^&>F5hNgxM$oVd23BII*b))uE?NO;xvKD;u5AUSmiYc$o;1u^ZFGR?(96}fKl4M6P$Wt@UZ}O|`n*3Zi zUx;`B!p0lXAX|2Yzn7O9j@!f#f0obdd+j~kH6#`I6_petCt`ONj&QhrYAyVYDuGI6 zZ)|pJ?R+*#Ir*cawdRF{4uy2i9f|Un(R@oYglIWFF+K>5Q4ED|+G(J7bl^c0uP`q3r$8%#<+xBg|`9V z;}H+YgnPe^8yc#86}KYrGAyg>QK<|iDKm6(TxNrzK!9$Pu~(TQ1NUXDLZF@WoEdUt zqxj_F-Ni|+)X47?rC&j-5oTA%wLS%WXjra4HfMcc|32su1ccc!eI6swIW$=BU>{;o zAYvlrJ&Q^lnNEI-hy-+kym zVe`blW9JJ@+>Q*hnntEZrCLKW9=TxoF}F&Pmn@K;p*9H(d3Rqm@1^BbF}j0VG-eb3 zX4)!>z`}s5B~(}>kk-k#e&aObi7!b_R=a0Zn<>F(&ADnene!OmRsM_k{l{9e-)`tT zM%`1z-GbJs6qF%-u> zo(@^MY_|n%J}~ej)5nf1TzVB{>#mVzt^G!emP5Own6f1;@Su5a(SITS*6Oy_Tx3#C z-5|*$&P#lmHlY_Nk+hc+fapgO-T@UD`&WehXYs@WBZ&%~Ve}X}=l58Uh9ki8&(olj z#IJIlree?c;v}ZibC-|rPifVII7>L!=~@s?m|RX?DQw-1AoKI~j9Ac}Uc?UIA_UOU zUdhl5-&|eGsNLI*jp1Qcl>Y+xyt3TL&JOC^_F%Yv7P9Zn&v?Fyi6KR_2kkwuVGuuF zp{wzGdL{uYG3KPjqXGmMcW-5;Yh!bzB1+8eETR~&v3#k zVGS2$Fp3kO1xk?2YN%P!exeq}hF3WIVuf!&2>Q>j<8S(iL0BSds-->-3?@82=OPX> z^rdCA9?*RtXA!AS`^tMLu(@gl^zOW&8XoOUS&QiY*m8O{ktDS>2{&Xx#<#1Gx@o(# zVnbYc!RbcmE#I~;k<9)U`Q;b>oLS2hzwu`HNU&EqC{N^ahQbbJ`<0-t<_P?vL!ipU z#l9*AT-?KE+^5P=U>-hdE3Sa#-*9Ky`jV$1J>~%v!~1-c zXx1JJl|IK@FW?`$iAm1u6`J21fd=V&Jd%!765JsfkT5+g;Yx;ka}oaWG%Z``{)D`2 zphzVfB$QUfJ59Jweaiy*%47@9kB_e^8Bct4a)2k9dj>puw^4)|O>7zFu|N!tb|Z9Hz6B5l=z6PlXCW zF5nE0#}z=jv(@JuraY4n*VL1)V2atho3ysjnY(cFxdr$+4dMMu44-^F`*#Lr=E zW)e_cZsNC#71rU2_3LcWDm=Yx#!)tck25w5Zr7x-EXX@tkkbGIKzI?`YnrHH~j8T4QR+T%F*VEv@80JXeti; zGE=t7#f!(z9KvwEItLk4B_r&OQG1bkl{|Gg!#vB%2$VkJkLB|UPm8E4#JOVXPz6MLcddcT@!(zs~;Rk&RvM%A+8VH$1`sV2hkNHaC_;P>PjI|m&xpG}8)aCW1Z~3PV|l2uk|2+) zwe%38p_vsA9eZY;A8A2|K<7anq)IERr@MozrwdXS7PZ{`(CCoqBs8NSflQnSH6JIO zDKv6|7JAtdHBtv;OjthjLWeNJwlfXQ0!Z<7mUS_m02Uytj<`S#9B|4JouVFbu=q7~ zrx;&Y`rx?$ZY^{YJbxkik=OtC)X;w_;J)DkAivywaqC?keL^=2;R-VD|53QGU?$qA z^aaM6YD(*aY@VV;#<*tbm@nJBKj|=(qFSlG$@s`9w9Rfc5QBWl)N?(5Vr^Hvv#Bp_}&IILht{v1^lkKiJ!cg za+=>cf8J>^JMu7Qt^uXj?fJW20-TkDmmf*p)s$eU3;7ed4t{VFwfAzRU>tAxj+#A2K1ofAk2O;RP_#3j9q%wBZV3Lpk2sy_h+O0Q8=oau!uz35JqYE5QR)DIzn02%L{{n^v28BC_N$6_*QHuLLYz^%lP}mIOBmcR#-zhe*I-~cS3|0Ub9m4il+cO!n>IwQ4zlW9%rMDV|p8w&|h|>1TE4IN@Y#9`mSrAQZUyj zPXEQ<2o>ux40~|{;f^Hailrd;<-#L>z#&oM_1RR=7omV=p~Ksk`__Pdss_4wey?7c zLZ(!ps|tlVQOI0yF4}w}MHo~yy5v+(f{&A&))&;P?HXzf>BoUyPaIta%XZ)H95;yQ z4r+?CQ?ic9_wlctXDDtG9atjSvK&28tG|zTimC<1Ko2~=YMRoF@7-bVJ?epf0n>@c z&ozOI4)TL31g8)o0UbcU(ftQ&U;AHO+@g;r>`&F_a722fHbfS{G77vLbcDUeD75r? z2D&4>wDwRi3}F z-sI=JYdCM00UR+q0vX3hAj{5E{?pL~1^hlyM<|TkDGf$cH`CcSz7D#gIhMw^Xi_6I z7(pU4bK^n>k3R_*&SliKVl4y;)*H$oQBcq;l$VmdJqL8*fzJDe{v~Lp6^V6X;GbNW zXNz7ZWx2a3x6f)yotY4HaWCUN$8b=Ltey?mQodzS-BAR`n0B)QW!_eSh5b>WKkPO% z!zWfdX}o-DFAS~hLW^25^wb0U{-i$jO=J$M!Z~WvoGAH$z8@EH3Nu>OyPqvHEq@*gq|CrFbKAeE7vG;M(O=frvZQ4=^?&kG?Dx6Lsd*Sjo)f}Yy-2>1cql%y?+HNkZx8<@yOC#BUZgf(Uu! z&gMO5k1a45F8Afn^pNfi%6osrBa(|b8yWk$X5KBK08cRunxq>-58T1$#fkDfM1=1j z&sTT-uMDX-n1AeI7XTpTq=B4jo-R6R}rnE$iK9g818?U&63uTfnUiHDwY^lq`=E4F{84azJW@;?8WvBgx~*JC=Dl^%SzxGt2ZmQK z)my`jndBg$6RsvoN^y#igNp^_;C2FBk(lrJrI z)^*f1N`g$FrSAoje}(W%nws+xmQ5hFN){Q{v?@|6Hw@9=o3aeNWN%j z{lT_;Cq_8(xMra>F_IC|pZ~p~|3elf>fe^Ja`cJq`M1zu^FIY5EG8H<^hk#;1+1`$ z3UR>J27iHC(FDU#xODt)KXDBVg*EYKXh&AiI#*qt@T*D}As0VQ9#eB$v8?gr;hhFm z?l?P6a~gx<7xHYI+xs)xKkah2PY z(_K~LiZ&mC19CdPIw#5wmV9PuqK5l7@ zzcnTo86M3OOXyH5=wS9EODeX5yw>Qlzt{W3N`YCuGgw%OiZf z^T@kNe+97e*<)4NyOfy9pD91h`*YG_>goT$Swl^|Yo1wzV8RHReBB#MIi{K!V$&f~ zOe#-J$FQ4uC8whuY?;f9r)3p`@$4#9+~%+McL1Cg+q8GtgwkP&f?9K*+p)pU*M6Xv z1oIZPx34bRJa=QUb}Wx}H~_c2!)Gp9a}^9T*WZ2s2fm#Pz4WHE6*jdCrX5c*S##qu zr+(#xLF3EZdik&p32|phgvN*PtR`RMw!_wG1M;k{lF`TShAsf8#q3`t0nd&KUJx;Z zo80slJ?xki?#I&vX2?_71UwdE4UWy#-PZhDY$LnwcxSI$t6Hu-LZIxL z2FA7YU&#s)p-p<~xcL3ranE}5dugtS*aS!Z$3waMhP1)SiV)@xcZ`9!%pPfx3)y#f%proW(n)DTD}8KyA(TIFL5Mq>YPSEj4;|OEI=njfcXMC0-KML#gy8PpF2Lj7?#%N&Qi5BJ%hg8&{mpd_xT9$v2ps z^_BNF>22Cf^U03BsN&@E@w0G4*(kP*HzHc6&c8xz7Q>8e6_zpOvsYA&+Q!lwiPcNJ z1|+d=?hUaIMe(lhx&qM(AIOwJuhz=Gt6B*=ezQ2xzuwMhd2e**({G`e1%GxWrRC$3 zv1?WoU-S%Gd7HOWhvc#U#w-e2n(Dm4UyMy2r(biUd9nb(oGDbaePsEBUGSmlLv1>s zn1igHlP?sb%z0C?5LC9$@f@3##fx=zRUc-KRFx7FIzgjR?T${3fz`5vq2Oo*_p#;S9jTcPT)HqN&W*Ddt{_QFw4HMY_g0rQtiP+S^wZOcCImdh{BKGrVk+(8;_ zr7>3dsc(Gfulp5$J4ah^ASU+bh?VNuf8pcVo?M30<|!=y+^~)8->^+#S6ckWrRJTI z(YG|~c&i3Izaf;$GGzkwEBFPGxyJDANpXkHCk&Yd5MxRNi~qGS0T*c*sN1gLCHAA! z&jYKU&w+++Uhi8WiU;0|K6k`tZ&2-YzCNG_n74STitKYGN=)Gre+5@ZW>m=1ho*P| zW0Nv;w>50^Hs>ORkn#bo1A6h!Ec^OU8m{g7UZT?FW+k<&`fjHrxe?;G(WG!!c|}VN zsK*q}DjO04BojmL78Z`M>LQ(-I8@-E@Jokv>jmsuKYU=hA~=e|VnPFu08Uaj=_dqv zgIlJbDPHonG`_&gqxe=SKgV?^xXnJaJ)zV!^hIX{2HiYyFhgKhpYu>|zTmi%DSdm; zdul(j$KQ0PSx)(Oa@xr&$WrClkeHn{+yBxKh1I^!Kmn_01gItVjsjdYSx)r;5d*2$ zTZc!-LGwR+jjq26|4e>xVUrn)S}3srr5$(mrtK@7vRD2=6!c z*cj=CHN|(fAV+z)%c2QpwLkX%U-gv)BMiuo2lMx%+$|t~bgE#!GS~SOgazsp7N050 zA>Lif{FNpivb&~1GdmZFuys-Ua!rxcys<=`E8_gKOWPyE?^qJXUnmmSnm=knZfjBT z0ruC|iSGgPo_z~%*r4L(MBmZqJg79H<>+BJ$k|(itiENVkyrsLil8Uy3>L;kZt?Yx z>MFhyD$kSM0u)3A?*3N7ui*u-ezem)&>-~WHPlC1r1AqcEAeY-;3)mR=a%;k^Tbyr z>sS+x^XY9jaH&^u?_xVKQib(xp;&>$RaM-bvgr}GP+FTG+LJ_6rFDQF9ovTHf!q23 zq(o+vD_r+7#IEzV3DWT|om&USV)#S#XQxaVGgv{lH4n8Gi!j_0bx!laL+I4R)X1+) zEd~8|Wl@w%?%7CLRdqpHdIwAD-XJggv^9SrQ`7vqt7$C+(%&6lk-M#Gm$o%d#w5vc z75L5q)HU7R8*YzXTIus-lh)OG{zLTn7aOQA?%RL+fzLJae}9fYgMyxL+p!Kid9dj# z#!)6B-nVu3=s>4d!WZez@b2(p@!yQ2pVAj0;=+CBUy}!On{sp$=JIfnZIfI0!2m<1 zQoMPxeEKEv*I`w=OBdt6sF)rUm>}P*zGFoT7ho!>(x2Hs8N$HyCSK}ltfdGk>IxV% zbmT_7m8og2DEbmNe!(zsS^>N1&LKtBeFtZcRjCE=u+~3%mV`%f|ENae!2vlYA-jnX zB0Ut^GId1td`T+3D?%o;u`x0=L9q~mAwYUi&Etbd+dXr<#E{~ZA5k*h0G&=oi_iF9|EvK-pEtmuA427Cv56n`^8_h)4vvgun;}jB- z<=WKf1HI zZU1*JK;h{b z3`Ktx@@jt2g<^YJ0mgqa;UjXBQbS+BrFO8TnkFXF8F)1shd24h zs$rxioDhqz2s!b;Ukw8d1~`PAF3#A$A8=5MNP++-qtX!cnneZO7t7r=yZ3*CN4t_|XyrFe+eOXEj^*@dl(HfY+ED?Ib_S_^G>JKlpNrnjd5J%& zquxI}P?B^WZieV>nww@t8r zZH4{xlx|zx%oLgIA*=l*x45N6cWhe2xF6Yxn#!{2b24B(B*EW?2Bw}QznvU6E`W$W z=_M66YjUWcs5K^7iEhceX&ZH{VPN8ctu&C2TB?ZU+FbX8dA`~QqU{SPWxm4GsOREunETJ;5-8jvPP zntaz#{)xB={4z0q3aVHee0|J3-53C1&1$%>r2BB=XVq;-7bO~plr@A$v->yhi*?Z# z>&vP&fvQkTf{-0KU7GdSR3Fvj&sG>R?;!b(Wq@owp$Y-BrMCA8Aq?n%-*r}_;?*PP zc051pCPjoq;Ev_c5S01(qG!XZ9^w#!T{`%n#W#-2ob6nDAI>pup#>ctt8LMDs=P7@ zGGW8&YkTFigCc@!Y3}d%+CZGcRVg zM0@5xNUqhC&i&tB0El59;7-d3bx4`}M_nUJfbxazr=)mqpG&K=px8+nXQaKiCt|=} zS2YRvWBxe1M`!^4$p8S>jEPQuDt@uC+Qx+$?@(2PgD_1GdKTwqGa1!^{5)#*P zXQy95d7X_&k+hX%ceVZr^G`4#XP`oVf)(xN$Yrml7n60TC=eUVA9Y%`#SQ{Yz)M5| zkKaa-nMBrl*hPrIZ~PL#8N+tWgBT3o)kEFy=7^!Rth%Z&m_pMdjyk;Jg%O@v+$^R4 z1T*--eZ`cR6p@3Y?uU$#vGFVZtfPh6K~9x?d0fCdUH7QNfITCw%Lki{c=g&W0 z`?22qatCa~1Qhg1qUW(=*u_8U?jeG9ZXQwt>7d-|)}m5qFf!uG`fq7$O3NoLtme9_ z#FQn{E%It_&l;RhOqw%9bMC~z!-19{liSDWj6QhDfsgxWm}~lxe2e-4u2Eb>^e+7k zfP+nFTuH~zK=ZWy=oFV(Sn?ROh@QtMa%_15MH{$JUHw7>fw7@FYU71WI{=&8_M1YV zt{v@ZtY-u>+rU8FGfdLY`EV%Jbl~rAg~0%0yE>A@@;zvVd?3$t$CB^RJ5U6EvOmk> z_|kJ}Zk}dAEEbC&aXm{zqTR?IW|lJL*CZ(p?VR|0FlUt+54WbvprN7m6~Ypw<^{wx zp^wHkb~-vEF@aufMi`7x*lfK*ZQ@jTAK}^f)gSi$i$2uy=mK%E@eBs8uZk@To@l`3 zAD6xKAQY&>H3p27xS}+C7OpMIvisf@@k|K0bSJOMhkl0phYq|stBtkS-0(MvBAtY! zYpI=@SxKpM`oNtM=M*lVU)qH!;-|Y5HI#YM%JVG7WDd&IG}K1U7`iztF$>Dezhn0S zm#r)S2D%k4C%Tw`!Li9mDy@N`SH^~DAX+~sFkI#g_eM}qFDEZABq0H>!l!kizN-TK zx2xg*z(&A=0Rxzm;ilhUC-gyKf=~ay2gIM16mWi7(eT7?Vr$V=zHw7i@jpt1k>^P^ zto5z}a_LkF_vf6t9 z$X2p4iil#GRh3Ncoq}4=|Lt4pGk~GiU5~r&l(QC71xdObsF^dRRF=E!F}9{W=qHCs zhcSX~a`DjZYv;%;P6mF%pQMBK7EmK#_Ht6wxy=2ju!y8K?^GhQ%pCU;-YV)zF5Mbg zwrJAgkPKZ17tXo6{JuejC0+``QhY#A($;+Pvl^#`4)6&hSAu16F_!AQ<73QMHFmR+ z%k^e2wcy!8jr$~TUZ38wwW8zhk?ZeJ0Qq70SPHu#(FIH0c{ljh6e`2WQ_xVtC*jIu z@?Fg4ZWIN^)#!0a_`4*oF)iFB^VC!>PfW$vcYtT*IEx)(%miHrF3^mcXxN! z^Yxsw&wuapdtR*KWr*=vs$VSuPaI_3~E)p3{ zNJkyvIvJQ%;N&_4heQC()!dbr<$^47&Ic1rZ?Y9H}K;4gj;WY93t8XXfD4w+KGYD z?w7Z|D9b@NYaxMrj0X_&#n7kg8IywhkClHV+51T&$B3rmHNg*$XuYRIjC?*Ucm?K#b6+%nS3WfH%`CehC zhlklDriYZ?0x-~;NGh4^KgSe)wNP{Q_juVlAGW}QHM}=E@jG!U@tiuLv^h@72Gp&p~_`5K> zd8xgye$o6f(G|@0;J@CY|2f-$C+WC|6V4ry{s%h;D=Bl#02#d_z(j(8i2Xeq*Z+VO_~XMa2-B`6YP52X%2};%`JycD31$Ir zNZFk~^rS>BJ78i7Fy@@XDASmNIMu8YJ$`$uVHSnbEvTlT2OFQIFU z&YgE-N;@1)z!8APyrpFLb7n?md~&nwuKRjff*EW@2yjZ>m`;iQz#9Qd>8aw2mxC^} z2R~uTBU%5QljuZSq2JlP-7h+SUIjuVEi32F4+5^uZt^yMXlK4Yx4ID? zN`#2>aMZ+!QFPvCf#%@2X?+adJX=&Pf)Y9ri)BGX!n-#-eBRZvb4cSMFLMqpz`q%6k0C#VS#`zz-nugx4;;OVt;aJ*By5KRK-|a)Uh?2 za^KLV+0>Ggo?G*<4g=cjNmgnAdq89@_$Qi%z*gZ>m5viMTN{1QrKb8z2B9z}7ls~q z5NO{E26{N`a3~!Ko4&H z+@~&5xLOUfaeD38<^T;pcG7yhHvP6Gx+%Fpl=p9!EU;&pcv1Jv?J+LSU*3FkWaogC zi|KQJ_rgA|RTG)H@A?Hu3&Q&XC7fg{|7r7Ocf)Q*i8vYw3F9d5rBtKXBPCkf9~>g5 zjQ}Q2)9Z~v@y6`*bV#SrZhvfDos}ReAvHRTUCYq+#R0;m155d-*tc`rf5SapGyh;c zeP^M(GcRs)AL-T}*Q1+AxTDVtFt0*LG8bgVe(h*W_V3zQAZIdSptO3q{#lG@Yu9=? zzqlXA+qie1Vuz-s$FhjW*Q+8THQwi5KTC{60t{kGR6U{Ax1s@A zsxUpyMsmCWGA?A#v`A6i4H1%Uus1yQJ^Ey|BN*LMB%z%D*K$mOQi!AQr_Dt$1uUH9elmsJjGl_hEZXJzEvS>-IKTZ{N8L$ZKh&+aI2 z$(v+0y|6l?*!IJiUmI~no^k=7E@b!w9mrK_a3h%T1-Tu6bCbdg@#_~LZG{+$*x{}? z#EIH=pV8ZaAZfot?{9p@>bqK5Df%v@&@Wl1t%ov zFo|+|Y(680-iwp_waX2Edo%=zu|TfAzT7?|zs4!bj1S(Wa#+s zSwE~0??c9}v+MLMUt;(8rFr1Ow5E==70sRN^uJ-=4?60z+iTXLLDy!r_!+r`U%KcV4+)W9dw?Vc##12XY?F z$i9J56TV2|xqfW*u#*S|MrgdvX;>)BK~{bOZ1)`J{}j4>^@soZey zH!}cI;(Pr)V&j#*r|{GvT;Hk)0S>Md&#NWRE#7IyiqvnrtAxLAch!2|uF4ao0#7qImz>4+it@{I_%_ftcCy| zbT%5|Y?`$E#?>@}v)WC1lrHO*?x(+}!`IG!eBQ_eG+YWe0a}vvuc;|1-L0OFSEU9L zXB2E~$^u-kXX{xx-#9{~ny z&^hz?nG7p1&g%NqTMBF2}K;#w1kQ&%ZG>_7|GbaE0rD^t;%`AhJfi7Jh=r-Z%_{~f`eP4 z2AA^OqWpjd5!=4@wTYFo=2p);s#MX4eJb+bx4#w~@gmT$HZvW7E)6%-gWQ}w*xqJ^ zvOJU?o2YoqhPINHWQkk#m?b!Urop(LUTTbvdCh5ocmqU7hoU>_U04`M7nSIBKQY(r z4SJriGw*@qZy#ItftiftbE*k#yHqoi+)$GIRA9-(RCp0eh;vPRw1@RQIRNv0(kTxz zl&`DL5~)D?6q3Bj2aJUBP+00)fZ^h2u=Xn^j%I*c<@$vTlg4MGbAaqoJl4cDV>4r4 zIIF|i2NFmYofL5vZEF3n#+02#WrG4SPqa(@p+h=?S&(mG7e?m8{143K&W`ggLTUg| zEY=eXW=n>A?`8**kI9I!=GT09`g}gXVczu%p z`60#gsgi2T`svp5LZ|#B@=~TPg>XCc>JIjEZFKB%ALFx(ia#qU;Uu>1U3f=|tt%Yd z8rOzDTib62lQfs2?fXwSNR&=MkC2k8M_$b(6(c5q(l5BBi1xuad5^M0S?uf@LGk#b zP7nPe=#nklrstu@(dRb#UFKuGF0~eqQ~cs=2oNSb%HY0+DlGuaf1U7KJV2$BJLME) z!>5Sv8&755WEGQPm$g?ThlqKW^?p#~sy{4ml95Vg>9I%y-iR?_k$y#r1sa3dF(^VW z3w0JfMv2<_qS#75ju&YM^_w0~ZKDZ3TNZ`_2!@h2hELH< zy2KR!cwDrfi9n2H81=|Qig*~gsBD`ke1rKzeI%6w^-_|N1k6VyXoN>g>WNeJ?*U%+ zcL~8kh3b$1E5&{p@(Ty3IzfKzW9eeLB1P(8Bn0MD!@DnhArdX2dngL27Gl)&54+I@$-Gf7b9hoz2aW1MXSQwd^51NgpMp1)L%ADNkQ+0ysUo) z`JAfF%TmRi6Cj`b7BLlpuLN+{yh>@$gxp)?QJd1j5*Vmk>ql=6voDWC`h)xfsWwp@ zeEwx}(?a_53_`}uV&OqWAP-Iu%>Y`BQn;pSP(isl2GAcZ1Ypv1L(DQRBV6ea|9$`3 zON9{}q`Afp8<<{xq zRj~_@9vz2mcctZdVBYpolOmsO^3ZGB`RVLSd1i6y;9}9r38jU{PJm9Ia_MM zCPXFkhrfT*)ra*T|49rNQm(ZE7=w`az-BQMZ0&WKs>AB?d_C*9@^pXch#&F|5hwKG z!fw9LqN0ULhHD%1^SwU`xbW6wB<06s+TU>wf6SI%NJvu7vOoFY=|B14^MU`95B|?@ z;KKhYEWt8ij4V9Ctz9b{czU`GNn=Bx5j?>|9Z#*BYfz_bm6&e!}^g3&5Exb8%I;(zg*?i>IwRn7y1X< z2Y?Z-Y1N{o+3m=7+-^a|MH)PE7C$jI$K@TqrVaOT*^P6B)(VoXT8)p7ULu+BmAG!G zb!fS-6Dp1%?!JxS0t912P`Hd%*Hs5wIg&+-`K(x@3Th9zu}fO2i1oA=n-0*j73Sj& zLS6sUTmc=oXwoEO~4_>@ztgzW<$V5OqeoLVvf4vRg|9d@j);b za$W5XynM5h8_h|7izWy%zww}dxm=rdh;#vSZRW#L)QdUv#T5ea=9Q5=_a0l_U)sP} zgv-a$QUat+5g;N5lIpiET{0ruMAy}?_tk9@J=~Otg82_QEX=y!I z*;5f(cRP7;g;+t5+R+|2?uY>SnCUyLbn3y%c-48twP5#+(Q_vT_?wzgYq9#pWw0FR zUzDmChd?fXQDqc@smBh1_J_b5o~q8Sv=7VxAm^Ga7O058LKrPBb3c@dn{|YtRk@ZD zM~cVkpP6-KzjW6ksxHw<6!+9AV=XWM913_L!SIhHw$ z;@6U^a_Ym8=1l%!8`?FqS~cr(PpGp{R6@ZvCoDiU79>-9WRUOEPd!p_Rhe{s&w1X^ zgtF70J3mpRa4qI_`pZZnr39&zYtgb>uh2W4t}y7h?Z3PCwewPp6@P+wbY8LzQhf~x z3E4={v_dNZAUNsi>H7ZxVvdvDj+656Xlgl5E9+Qt{jU)SP|tz9gF0U3z+LWr>E+U2 zEAh|Lz*UF)%B^?CC_eMswH8nt<3KX&JbC(Kkw|67Upu@m}Z^ zN{`!gCS6~B-?Z6@#zejXSKJ+J4L|``(06{HVG=)#{fD(gMM~5+sU`T4y$rxsDV?qq z)&R&pGadB__&1jad9FXFTqQm?>chcv!qb?905v5!JF{{(Uh(-Ji&9!{7!8cVHSh8* zgtW2)wW!-F1JwI{ykIH4isjGIuyI9>xJ0|Q6LOTE%6&_IG{{{p>HT+dhW0GF_cD%9 z1HfHW!nMiyuXDw|l571zdMuU3t=|lKaD)l<+9o}$)}D1vt04^y_d8rzW`mX&b@3w@ zZqZ(*!EQ=8Ix2=XpTzsTFtfnZZ*kR&p7ami@x{z~@KGG?4QFQy0L)7$WCId+C1ex> zsc7y@$4eoXksC)8-h0}IukUu!U$EMUHq$|~i2wCvnPyNWWMr_NS6mBz-LAOz@SHYN zOrJ;t@UZ0w6W)Xg9vM` z|A;uJ@_AXxTPMjE zm}K7Y;M>8CU{E=IVE-@DGHCtuvg<|-m2;`@bo>Z8`jS;kT|Z|WZFvE=KHAM9NBZ5L z5g&m3gq~={85JCQo;D)@cfaF-9-;<}ye);lc6@y0cVB&DqN-fh#?o?|GLR?j#)yh; zzDEQS6rhhQ$bn?f$N~4XovObC^S&oPAVCw1k`sxRAw`9xj9PIwj-gFnd3>7vN&Pqs z-p$b5mC=3QmRd7*vdi-r$8)Q>x**7WK1AQ+jWoV6b-PK{|AdEOypXR~9J`1hG)J4{ zSZrtRV-Hb!amGHF>J(#V`g{3XXwClK?**mL;JdqegNQI>Z_bpP1UFC4dQ11uxpgQp zTu7nEY80eF9n85+O`oGr%0km1%nFxjG(Uz7@-*LH3%=VTnZJOs^gRBO;<@woquF{xxs2*f z!LchoLf@ct5-tU_shdeet|nvEp?S#q3vaI zJ6%oY2mP118V$VEfsp=gYUE_Iz4cDJ z#>#6_fP1Wy5`rktnJQ+&pM%|A`xWRL@Ct)q_*gU;_H`h3S zrAQW}Y5^;6QeVtHgHFJur*+i0hrOoAqRD}C2Zp+ z5@;Rcl_f1B<|J4cCO?27j?Dvt8r8~h`qZ=r&Bf^ny4?s{Z|PDXB-EzgHPKj1@Rk(F z*#T=|&8MHb<->MuKLcbL5wc5`qDH_)ys&=M1SN>>vsqxWs;pQRJtmk^QK;6WnvyP9YrhvX0?tc3bOj&4~I%=Uq5SR^N zsEt~2-qG@PCV6%;Op)Kp9uujH$$thq)O1i4F{N72VUrvZnRTrT=p-scXBQmSeom5uRp~*%#7Tk zOWfxBI+R8dj0fdsc+j?7m1y$Ku+G3rJbbRWLtyxIW^arGl>q3iyUss^YhRKdrwUBx zx1~ng64H&ZyEJXud)auzM2gwG?;{$)4ljO)FTz}vd!9cn3lA&5j=|1&_8r~+7H8CK za^XOVi;BD*;2?fLAX=;&HTQlfc zmLPb&CeY;S^@|vwy{iB1bQ`yqJlW(-JlxHs3>d7bp%?hr0DAQv%6du`%^}zjhlz zz4s+DP0{D+mr!Ib%NVc6!#ftW(K`1dyL|maqoIcxci9Sh&O3+C!=38fMfIc?DBsrk z3=ZFW=M;F`z_&adj-mDM*}M?Y1~#y#8(d_uIEmYiiEY|y#8FW~LJ8yyUWIGbR!wl!Y7IsfYQeCvFtHAVt{{rr6k$Q;4( zJo4XBKR>>V{hs7+PIF>bCm0QfvLwi8%9LW^O@myETN{FLBZ~NrG~9+~v0oO{W=64@ z>9mPr_HYrYK1s6IpaGRGku=+JQdg4JGXq{hyLQIerg7+XR7w5`A}mZZ^f1xpbZNeM zTx94|ts%5n1ZXgq5oW){)+!a0n6Izsj4i`fx%l9YoVd4lH=6FdvU0 zHVB|i@Dr9)c@P^DJP&P+Pmcb^w3k~4(#v_1Km48Y0qns+Zj*BBJqaA0nX9OZipQsV zaut6cClQ;RHfiiprGxC zUQ+v}I@=x5D*g9Lv)8udED*h*aV~B$buLDqD<6Xr#jm1FY6`NhiK#y^K0EGL}_yy`-` zQ&%4~AxGwrsFX3mYY_@@#fpO7P3#HofL41(SdE3JW4=Nh*7Q$d0c-^lc3t;Q7= zK6%@z)ZF}RJI8q+-d)7ukvZLzh>-@H{m^D`t&8;q%}DOP_fh=QjmS|Wej1j|=x)Zd z*)&S<_jijc3I}(M9hxYw)Y|7|ZB1hLaZp*Q+xnQ~B zOxp#vxSp|I8v;TK)O8`vj)$z_+qzGuwvSBD&GkYK#N0C&N`DciE~Zwmm?OvCy^X zegC1X`5IgTCMRnj5GEMC#>>2C{_=KXxipRfr}?2B$FZt0vh>%B`%EikFEXSg9`pe? z^z@@kWIw}!Cpe4<)ajlV{9C2o+W+7q8ABoq^&AnVKVDf#+id>DCGqHWYiUvAlxkHR zJaE(epIsaV32Kf$KW$<+#w1K9^!vxZx0?z>MC9hPfOB{m=;NGc$DJ~^T|mCRLV8l@ z7t;DQ@bvx5wvzh^7ml7iXoE>iipqinQ8(s8D07&T@^A_t;${TbBAJv}1j)zJozP1X zPjl*bxGiW3tqUSEvx8Sj51R1cuI6KLm~3rGFb-17=;StJ2>8P_4w5)ezC}e2l9rkR zQ|A=Zt%bZu#uo8mL%Oeczd;9YR35~nOTqAnav^OojT7VrVXNrRZdtBH4I;zKu~qO^ zOV^#>jgiJTp%)I@i5n1NNLZxsqL6F83}Z=gq7a!NMW^t5qd@Pb{o&1FB&wGY$CUy9 zi7NsxeeRmsWU8`^Bey)-P&|k}qBCl!`?Q4)!Jin;f+am9yM4;6Dx@fm)rgF(THoEm zpzg;B0UlAwj4gV}p78Oq8R?C$h>EILQs055rrM)gVlcvQ%Pr005%`+Pph;f)2lFdH zrnWo&vh@-cQt#?J$p4HTo@9_sqHe(GKaLpZ>i+%?W^B> z0d1X2I_x*ADBdsP&aD=T8Q}ZkrldKqSh_CnZfsR{$QYxa#SEQh#T;8tt~h7b(`NY-ww`}mEgtKJ-lYli8Ttjc z8o=Rw3489oXyJXM&S3o@Ir!zb$-7+CQ!r4gRM`3KP7Iya_D{M_%I=uVk#kpXzdiII zMfauJ>S~bXy0e{vnvAJIpaEXy$yj!7H|iX}dAyI`op-+J43DxdADTDuoH$h@B|b37 z__d}*r{R=54fAY3#DAMxQp(Gz`#`>xyYbEWDLS3fM9s1?!FMDf@ipsWP0r>lH_?=g zhZ!kv{aJsdfn&Vc9$Z4GmhDqcM0fS89g(D&6>4?h^M~4;(LL_$uj|deb@~evLv1-} z*dx3@9lU4AGOk6Qn923Ngn_)Q5sitwDi3rZbruAOIZmy)+xYFLJwY`+JY*90zY;qu z)4g99>OTnh^j&9LK7^LcCGY6E%{p3etFAxaG&8=acyb6Daq=>3$l_oWUgP&ZXnV{b zKgodAuY*p})MgieVlZf$_|T?viFNNt(_1D;wiKRII5b=ubjJ54`Uth|?ds;)Gwv#$ zc=dXdu1M{!Gryf&b+CYuPSFMEVg=P)@S-#K?sRCOnqjZ_G5iu12UtW&mz zK}UtNAT7PB9L?kt3RQO2SI4 zduJkD8GY^uhO3p!Ts=D)Q^xKukehQ-bQcrBmF}NK*l*12nA=!}1X+2csHHzbVCV@t zz}V^#zzyE5HE_FcQ=0+HYT8pa5w=PD^=qb3ou;M;rO$+87R)G_a?A=Hyvp zHQ6pO-NDP0I~{A}ohyqCi@dQQd3~e(5H44R{haf zpJs=9vo=yQ;3?f2oS*hV3YJ8#V&I&s)FzVcXlRYoLI-fq*2JQE@quj0RgeV(+bz{J z8=;Z3aB^s)9%oSJJQPXQdL)FztXyTE^Luae{Rh+EFst&F*weR|PER++cQQ~yU?h;u z&+DBsOQ!qaB?wq7!ae~F!ZvTvh3AJPE(fRDi`!NiEUvdQS1YAKZeirCmWHs=UyH#z zu~CtG6dh_P-Z+O}t>tADg7-}9Jy5bJu%5$gnc5yt; zreU->ttDwW495k}+3w2Dy?z@`=TYsOf9^#wen=Np$KtKcWtEjL;;#*f6quk+-}WSm ztstn6PlX5hp~ofG(^TCgrxeMRy={Y!2P?TqhWe<(1j$k`SY82{Bs3QSm1-_h>|uD% z9e9%hk4bli*YKOQeg_C3Q4YJpHW?u<6trkwY;TxJA@wmguWT)M)5T2YapJi^KgNYl zAu}G#-F)Zzi#AfGJ2=vq?^GW8S4(D}TVx>3NAEbxCP?l5YrR}1Ek^0gNuo`p{8K)V zcXEzUKK`6i<^KMB;{WTm+gatI44Bix5${z)26uZo)~3vb5qJ6;d{gE=nqU966O383 z-u`oMV{tgYi!S^!tih?C&322lIrXEu;Zt@JjZ&Rp*vj=ytofa1sUC7nWqnIvY=4+y zg-Y>NBmHlN2pnW0wN*LmUX|EAWfJ6VN0YYTnGIu7rHIUir2|BHf4{0}?+2LpW9`$Q z18@1YH*1iY(wqjd4ir_rB)Ddd)(esIF%bLr-tnwtN3L#aDlhK}eE3G{=?#?ozpE`; zw-pZfESP53gT`J5#bo%mQ<^-vi#SdAaP3FlYnY3II@}nRFTowOpBBr1K2DvRq@SEf zWF6{jsyekVZ3jyQHLpaDgr>-a^KuFpSnlt3=Hrm$L6NkIh*IkcHHdzuvESgGrI9oC zjv>|noh{d77mJ9VE68+F=lh?k)6U0oawj!~dRiE~t;qI2cr0-R$#uvMWga}VK%;fo z$SQ>%1s@rUU@|}MS>ubGReKV|vFeeF_d=cXs|mSy@1jb%bADV)FG!|XNDe%GN7tB% zW|{Aw*=OX=kuxw?HhA;YCCfm2WXwcnF(0Amr3tT0R9eP=G9 z?L)sU*3U7ZJwkf-O^=FpUi0#1hud0|_}=Cdq#!}2k<`(0(+S?g+P-_4Tc1-IVsaG9 z#bM`nu=VwJLO*@lX2LNm3S}ch#glwd!)(IIt-3P}LN;o=Zk|5K`bSddS;^+ zsiugPhFg!S>G==ce01>5thaZ;w3K*iC5w;*X?43g$Fr{6`fg@*@WP@h|0h?%d6Nx4 zdygAVrNiG!p?#p(8a@gtE^6?ez0vgjyorwNX~jcP${d&!7IvZFJ{rjcXXG{&{%A_sHB-6yc}$7B4+bD9yJ-k0_E09 zOE`FmO%5i83v2@^u~A58gN;k0;9S`Edu2hQk1Gf9%m$y35;YxKQ{+W)M-^$+8hVi_ zSOI1iK}}IQ*&=`^&Y-M)BZjpZ9Q(WVVb4l=?4Be5mJZoU0qx_4Vs+R?>pgq5frNef zGVL3zDxEdM=q?#>YYq%iBy-BahSVGdc zMZ3y74FA%^{#}Ux?YDaNk{ltQUet8n*d6%^tikpFTxqYuw`I$&Oixkm=Kpp2MRbAX zM-qS3Rsb4ol$6?kglsiZ2qWi7@?3+FWuFNRNv^Cxu^;85P6ic>W}~#+ABrkSmuRWx zVU0=)kvlhZRP0xuM-}Vy^NZWcNPUQ+cC$K3WT5cwuVGK+itpBsIMO^j1|Ud=u|U@` zmKK;Dw|fzF*j!8uZNb$A^hl<2&m>RZXL#+L1stJ@w%+T@0P*{>@5oeoFrWH=vLS6E zD~;ZIpUEyX?1V7gEG$k#>v~0@E7)`9bjv0mc?V6uriiynlRTc(JZ12#4648B@8+vGrXoSaJ6DRb)L*=OV6HC2l%)c}u8gMix31+5=%i z2;^AWC6qJ-=}FEFuEeMqOFsJ%ZaxoPpXwHN&Y~WDZ3jo17|%<1@r;L13PQ&Y%B@4} zyaQjG$U>N^W|=iiKdv_Vk5cDVm~mWR=SHey*V`xI4egQr&!LwXtMB?92cr=XnnxEa?k^`m+z8yR_2@P zu{+G!F9`yT__f>?B#*w~Vf=?-hcwO$3Zh71#tMWhvQDMOoyP>;c(q&&Z$EyY?wAGN zcq-Q(rc+~)d83pBSfqN+gmdrPBPtsMxm3~fhFuR8QJPNtP9QE>@Eb_Q^@7@UI&;U6 z8&HeaS7uBt)k!4Lo+*vD9uG(9bZwV!)$o4JFF1b|ft6?v7`%vPu>PIloS8qdeCqB1 zy}cO=zHy9v(qqBaxq|*~!)Mzu|M9U@Rs#)bE$njV^-;EQ%Oa9Szo=$4TPKJ#0jDMx z7Rg@wxvzqC{7K%2$be zFxN;E*0V0G;Ctw+1a>4;MEHNzqwDjF?n`1*x);3|A~N&zLL7)0^z!-}7~ z-c80_VZ+J1?PISVU#USF2)iZQVLGOAMYFSo8J|mDNUI9ObEz`i&O@_A%Tc+rr&4;F;r zZxv>illp5v;g-Ysd^OP+8E@GsXIP$5>ig8d2AWoiDyb!>>vvCyTAd#+R?@Y6>%Zh1 zQ^Ah9CRFu8SMI0f1ZbKIk0-1SE|-VpiVy=tgVR=_9p!IfXfX^$3q36~0qx9e{GIMp z(~L?iURfBK1VDB773~k&aGGDa4%I6!U~vTINA26Ye~nBJ^IEZN%WmGMUR-2)gEt2< zuj{*Ud$%P_t3g0W)QpDjNMxUy!ZW}0mR8%Ovr9v3+3l!I~W**_k@A?XjldfV#Wi_lM z!5itNG?5Yx;iV}k`kY|7S5-8KyJa%ZC!`8WTkA>lV%kd;k>D_NI5J%xAd3WIX{6e6 zK7X7Lz$@1jnTF_L-~$-M2W3b$a9vXn4Ey)=+Q{Uo{6I>?&;U3?$l1mx>&UOuUS zy%eR*P8&^3N9QvYKGp6J3Ocx-Cvs>WYybwk$b0J+S!LgxR+ZiIxwhyKcX!@}T=IU^ z!r88FfG|{$;Susa8V|%{X^RZ)oz5?(iFz;K5V+xDA)KkuMvQXMEN7YNHqp~HvtM59 zQjk%;AfxV9Wl6$ADoDI?1zhG8VGGs%rAaMws_|JvImz^}eSttEp=hD?X6;yogjJ>` zMPLcBTL57OW?Kh^1&pJ@9I|qpcHlI6{IGknTL?)_tB|YyN75aUq$sh8Q0LV%* z{CIv>O*KYc3MIX>`nX~rH?}J({*HOEYaC8SrShv)f#fSzsn%P7XOG!pIg2jN;R`jG z^S!M<8tp{E@l5ymTz~K+DCOsLH08rLfya=`5SdEkw0wyvWNHr1Il*Tg%^qKAZZk{i zPA~D#xeA;=5xBmfS9zixjj&5_s1W@KEMD2;9%VTyxL}KpVI3%w*cSC^ujT%W1<=#0 zNLTW-xzO=k6%um4u#Yf!I53YvONDND)Gp1%#6^hbU{T=jKkB^b~O{n|oYK^$Ue`AM2N|l}Mm#ju$=E`xyfq zhNiTT%}h8RGVNQEQOic*aYD)yQJhudXgk52E?e`7@}E_kgdoi;Rv_&Yns--a2 zk_CUilNLo%Xe5(lqzt|PGFR^S(Q5|beMP+kNz26|quQe{rTJWy;qm6}Z%LBJMg@bD z{gZs)b!m1@(q?6m!Dp0h=}qcnqs@DtZm-oe3Rw1b=1H8dU;X??^l-~pC&Bkn1hyW) z4~waKzh;(pFO^IXmpjg_Zu%BqH|XAB@jb43f}Vc9UJwngvE@#`26>cAk*7QEI9_!W zzm_1Y>cWyL%W632w4=6<7wv5gl^&YC!oKDiM(9=V#W1Ry?~@{&x@^j8ynbEfdrTX9 zogm*LF({xF;r$JBMD<}$)BP7R>Og00-thTOd0r5lZ^VtUg`wh`)j=eU z20=ywv+d2v*lzpA{H8L`;T3R{X)QQ4qzYj=+ftLGQP${5ss6T7{_EPHasw=9|0F~F z;Ln-rWFJ8o7?OzpI6%XALEskNQ1%M*pHv^{?y)gr{?uEHaQz%#7~3W4|DU^qL<|{= zlP)XQz&QU>k6)Wz9gy+x@(mPY&ij7Kf9Wg!l&K)(OhO!UMuN^c*2CGoZZSP-j_R^~ zm3bzg8L{B`Jy8ENHZFWVK)O(>3XH~4Lg->nf}T# zWfpZ-yTHCv4fqWcqio)}LMoA#mZkaCo+JF z9!ld3Vo^70Ep`c-CW2e-KwY+^TUi-5p5mn=~h4eq5@I@n(O4lFvO=+fb^sjK=)}a?$Hl2Ibn} zmZkuiBZq%#sIDRYkY$&hnQyGXA4lS*6={@=;cfhsl6JAu+DIdiwRl6+Hcv%pwp8*S z`9>74o!2dXZ&R7_O*6@TC<1B(*B*E+D?Tbgc%%-C4*yNgThdSnB%`E6MvL`^6vL;x zEqbfia&kg4Iw8ybH|EyI-^Em-J#?g z1lOqjk0AQ5!w>fdoD0R~s&iRkV8n#thoEe-VRdHnTsguo7lO+{dM6zjw-6p!2GG zHBs@>bpS)z4g|AIj-fO;^^sxxo7dy19T|UQq^TxD>eAQ{Vh@GR%GHrtvs; zTV`dP>6jajOo^5m^CRwvgV(uDnP?o>K?KIIb3d=8j+xXBe5UR&fV1C1?Vb0)3P0=y zfjTO90dwzBi6nziGuA9vln?uPia3}n!>NnU;i;W!+rj@OJFRv5M-Zmd97-Rz2>*H| z?Ptb(JE!5f`NY0iT~$g!3Cuj9`I^ zmy^%fv?QbGK^$!-AXZSni>3g5Hyq6SM!R0Z!J zdyH=89viw3qeqAysb}{e*c^3uf~h{tldA-6c+41~UQlp;63z-dKZl<|-G+|v8Yz8< z2u8_&XDbq%cl8wY!1f5wwhUxnXX26)`l30F7a9-NOyu;i{hHf8_q78fuGgmeBM?U% z`;-|t6&@%myL@E{$V1nXO*BM4iW^ovGN<&=h1)~-Ar{pAyjn-?$8Xmvx(Q~D37&%jd>hTu)q*`I5+_OPstmd%m+4PN&MyuD1p4^^*kSgD(Q`LGHXp@(e+ zadFwbU(p{Ow?hy|mJ<@T0Dp6mTaD$K+wV66hebBm#oxR@(Chdk%)QYt66K%l~0p#QgeTQsL2 zr1VouL`O|scB27kcF41-*bg_#>EzloYaUJrZw72GhfruuP;2~q61|dl(w#BNfGkMa zQLyQ1w>;`2JiAjjx4arY4!&*ZZ(E zIVpS^&5$s&RaAOT>c#}-Hkj~}`z67kdt=0|ej!!mk$luH^($wHiJyh-%nC$tyzMQq?N&p#fvNa4(IQ?$Xsk#1>f9>S-igEZ>bN&BE*;z)#v98-1mjrirCkYlD8VK(0 z5Zr@Xa19pR-3idRyC+z1cXxuj-^$vu*4k%}d+zlpV-|d#y;EJme9m=hB+CA7G4ayJJCoqNhjv5!{vHN}Rg7vnp~uZGiEn z%Q^Oz2$G;Al4Ygs(8cf@lk=0HM}j+$Ow(q5_RsXbw}=v!kd-a4^?6uSQBm1aq{07d zllHf)7Z`TNi2p*c>T^Og(nsa^ci|Ko%n}+}f+wd6l{~N@TgjQ<@pzaDPMD1fT$724 ziYp?OM2l)Kt~kAqv&@^KRift=TBeVcRNAEIRfNG*c25|sJjw_Xpnh)~cO>ua=y(#f zqq+=CFS=!EcADiWn1b7g7pRu>=_T2#*Cn?0B+7t9VvEc*!Yp+cPq9NFI)!$74p%OH z%#pgpAaqZn?B@qIAD3EoO=SQYw(E>bBs6Nt2ZzDWzbq59N6jH3BBq>=3p-Fgf^u+d zsUG6nx-I63j8jf>kkfoFJf)d5yI8fCX&8hNz&57Q&hrsyD3Ik`^B#ocl%5F zH-_-8mjijb#6g?Xo07z4DN&-YA>p}LG{>;g$xwb3 zDJR@y2S2FIlnlk%SIwk_O<2mB3!_53o|C)_RfdDlmm-3yS~T16=B`gY#4+M)8adCxiU6gxb>>7Mt%#=8O#MBQv5JZBK$CJi)M8 z%@WLK+I58Jx!JFMq{!3@yGPN`v;XekDaic{=YMC_A#6PlnIMMsWh?zd zohgsTGRc;ADW|oa`P50M2GdpF!wi94ry~~)C^V`gH1&4AbdjbK6PC!c+gkMpZ>c%N zp2xWV#}D2%Ha3mH1ZqzSNjE&6;n$l;IG2FGEeNo}bQGJ8KXOh39!DiFFaD$X$wTw7 z)`ydhtn>3nUN8)A{S@xsnpVJG4&^ueW7%mIys|nm^n7B(3hI|K{Lf3#T)6XySYyRU zd{HxT)1HD^&HHA&FLbYWoq1I$XC2QV`Pz@vL}&-V9Zrp(8oB%UNRWbs%NXWOo_tSi zQDAH2)kW}pu0GJK8=r8~SupU^gb-uQ^v5!hOlE0gY)p7AlCY&Ys{ zco2o`SHjUKZ*JCx`=LUF>oJgrhi5M*m?ZZEU}pB1ZLnhgEmVLM>U<4lBSNL}ba_rkzUk2= zQ~e*Ee-Kn|tibtb4v`-jvj#%A1?N?MI24o+jGE)86MX}aW#7IrD<~fMte9N|Ntj!| zXvS~tWHdEd@GTAqCOho&$Xm!haCI`W zy*KLWgTTk?f2Q;`$8DM6Gmv$-b6!OrY#Ooz#OVKQTIh!Z!@6|HW?Q%wd<8Su?K2&2 zoxAhWBa??tcBm?dJ{G=FlDiynv7>>zBMoAKXV4pj820mY>+DxUH;7!g=xXbb-Q^;MJ6ClC!-L6@zj&U^%;LqM zIcVOk>>EQ;@JF(6jCWkX_P+qGA#IVpwf+|qli=ZtS}xfH>)Re_6|^P|1lUa|1k>o`@qiq=Et7}WLz+75Q=Wb zyw3HUB|!=aL9a}>kal0~V!_4ADwzs#H4~qLfL8on2BQk)@6XeBBCEU@?knC8yci2? zG8+R#b>WfFae4p#l5psdMXP4A=_M;KE%MRZlqot9dz|eu;`RH-2V|Z;5a>&KEjXJG z*=d-*6=(kR$_HnVXq^H3%Cp0!@IeZt`Xl@0!)x&n6kOvn&o;Azjck#>6tc_gRy29` ztG)<}YOSbT?e8ByQ96*xp>V0U-^0?AkaAvNqa)^fijelWPM2SE(9DQyq7Mo7%l~9@ zIIL!hx_fH&xyp#2RWo*4@ip0>E79+AbG-a2&|>o2c=wmg+557F6{&_*?-p4AJ0l%wM#%-UvR`No>O1mP}Dtw2lL zzA<5bv6vK&>HrB2RniO1MHQVJ%`*`Ld3jXlT6RpV;TV*e?ADNag&0!v9zN($G_(;n z71Cu{YuP&R&%H*GL1PT$e!})QYjxw((!Jt(dmh5EjI{*)z5iz9B;bB(WsQ-Gr)p7jYp80V%v8 zw|>!Q0sCxI%^CuVRcAQ(0`cB!pM0I!fkt!AkmN$RH%uh);Zw`ZM1iy}DhDTLQfyqD zs;0etl@;*-)?Zfr@2evS8mI^fLr%RX-%K-?`0Jqu7Udf&71Wckz6|3Q zbaV18)m1>Ll5s@|8AI~8fJj^I@h96HK%7GNjwJ#^lbA^A@wtEGV>3bgukuJ2@Z06x z9Kk5kOZ09KpzLIiimaz$SZwK95YKI@|KJrJgprA&81*DrmN2m1iTvuT&5Q!J2G_~; zjyrhWNC*pN(r_a-Avcq4%1if-94z3ip*oz>oJ>rgT3r7K;q2<7F>L=l!9@Uhiic2A z!6j0LJS~;M50p6=;nsmM){oiF?SgatKv=`5NS<5=`Rga@r&?Lee0Y*gB9=JunCK%FyVc1CXe)`5ml=`0NP&Tt*aYIv2H*m}-=*|nd*i=;r zi|k7PDjlhJei!gFHo)+4YogMYsU>WfKq;_30q(7$-pype5RZst zGUhpiA-*TA-^x`24*$@Ae}xb$=z@W~T&z747lj^A&l_#<^ndv2^w-eLt*xm&^|`rM zI$8qaSI5Q#fnG(UBRT)yOLmv2ejTNUSghVJWbre9URA&S6?rg_iUXw7v`Q$#Xn2A! zy?o@PbG@vRBmz4D8n2=3k>bwb1_fFB;K83HH8Fvzg&&1;usf1MkASy+Tm(ev!-xI{ z;OL~#t%AGfN<2FZFT5G|j;A@^(Ps#L4DxG4-l&>XK>6XWq2mxlQNeqtPkNN|cx*~0 zrcSgZqFD-KtFHVj&DKa2k$sl<#ys*0%FHi4ml|5^vD3X&z+1#?BbF7*p8p+n4@Itr z48!+u8k%$?BD@(Bl(@4Tl0DO|@(2*svis{d74^S-Bf?S5Ac^8|ln zVhFo?$1obSuZv&>o2xzjIyk80woWz@a(e(|ET)MGHsH6)R+7ZHB?^{3_d#p{A9s03 zwRL=eKrv-$Rm9@mY_O*K#P5?u21SGpa{_G$P_if>3L6y?!h;=FSpQM&42Xcp4ejzH zf-AfUoch(3=^sw}$L=K$6G?J}83VDul9uynwcX2ra;t?t$(>O9{O>n=7(WFxG?PaK zg*cA?I61J8t*Evir)`K)C?qppI@&hgsC}anf~{zMIhV}UBjg!p4EFA3?Bv_rs?;I0r`gV}Pi6PWCJ*0hb<{4R zjK%#4*!5&ea;lS+s%cpzxuuF5c4ajC=jxa>eWFs?B_jS%Ooa7x!yCmYR!J$6GCV|- z`e$`Nk%_cf5cGuN#MHhGl$m z%g(1%)6W4XB^mYyB8q*ba=&lnFAbOU9jT}uG9I}!3m2%7c7lh z$!S+!j2}VGZ@5q2z?S}?`Uw5anHir1g^xMjq7beIWg?_f!R>QNLPqx?njCFN<5+*7 zcls>g8UVkW7fr!@m{G#4keBV1U$a6x#04)~{~A+PZ56%>-jp}!%M9r^FY%H5>#gaJ zi-^?Ee&(ifeP<1HUq!kip8S0^sjkP%Dca{N-ikO)4WR|X)YRoGS^X7y`jsyNO3KR9 zIn&eAU$)Uo2+|r!ZzSRj@CAAP>X8LBZ<7dHG}GhX>oYEQ(u@e;xIh0ZBKt?Ni6v6d zX5tfr?XRcBW4vI)DU#|n`p1P2?94Rp2J~>T z@G<2F$nL~M6dguHeoB%76a(}U_L#Paf&Q~nIe|#kHla>V$u2v=y{HL0jJO;jf4H-@ zf_4CkJU2aWQ#?akMHvuRT7imowI%}rokBPQyATfsVde;h1u1#Esuw!E8{6K&p}LyG zRKz^(0TIz<>!3gz2)xajjmI4t_V$-7vH$#ru9I^%o>3e{-|ucll}L$ns_QIZi8!oo zN>u}eqPYzu*LJc5*TS(jNtcfn_1<>JXa$gwB*Ry&iQx~J>D09a z+p={tc;%ZR{jg!lQ4UL!Lss$)F=c;tJN{S`%v$JpeKFQ+8cZ|@Tn6tMVN1Ei}NGj z32jnV+44{Z175K;4tqPz@@G+u>V;6Pnz0oPSwJAtg!TO`J_(F*4O2s{B!)|(c+I&R z&>E~=q96G-1M1%cv=q}UcI#?_8$lD~%r=IG9H(R7`ue;LwK5Bo%70#QgoUmG-Z}T# zdg4ACIE5MbeL^^D2S?()L9$j}$%Bw=U(aP6sr?v97h2TikG38~!MpR>uDT5)_6mr* zj5GqFN*CYg`e4%a=)gT|bAG5ZcnFJRU*L^Wsch;Y>lKV{OatmTn4^H3%=G)(!S=;F z{JlFWlSuWz$B}E(I#?mnnaO2qZnjRpUTYgL#S8qEhqYXS7gvq`+xh+n9gxH zMcCR+_>jjO9>cw+ezpHN;C}|taF77zP*HK+8BJGBLu0Q_ju|#ZGE_xrkyr+^`0o)Z z83IgTr=CFYDr(zH5S{|9lH~9I>Sqwt`As&hsJ=cC=dG5Ag@r}WCl^-Ye_XkMxT{kt zbFRE_!NM{0@vTc^X5RCNmFKsRy-VDls)hEbP>>kfp1Fm}tKFUTRAVhY7>^p{i3nBk z;vDez6zikH9T_z$EykXhkg$kC6`GDhySABL0?||0E&{mitF2$5SeW~I<14>#0(}ze zIcrMa_jmXa^em`Ou@c7-GMtrqly(F$J&CknUTSEY19NyH>NLb2pQ`UWdA=G~(9(im4`y7S|zF+B0 z2YS)h&IW@rzW3&p#Y?-D*K4uBsp9t-e}=xOFe{!HvIz|i+fq@i z|NSHIo)G$14TER7acEoCok6|lpL&VNNsaYT*(AqgvWY)IBWjn-T776w6_!6`8_4j+ z5W;U4fTQp>IddtPTg~#mU&K*_5$q2Z!Y3JdZKnePNt_rD`e^O-1dmaprgft_s*R8E zVq+=>@~!FA*D_XlQ;{U%@6g&yF?9x{yN05VL_C0(x?n5PF-yBPFaw&8m@CN8dP6I2W=O)SaS+|{;R)H>j zh(2BA`q5^bbuK@7VG~4A5m9B+t~cAshSz`k%d-U#*5(O_v+{{1?kA&av0?m01!q21 zhDiQX#pt&V$cE2_`h6ab8uo(-ZBEe)=Gr_s**Q5&dJvL6Jlpk&?EE7%`j36{52x?H zThRaT{hEvF522sO)zExSmo7@O`WpU9N0NbsKgtkO*;zU2ZGUrnX<9@ zXPUDSrqTeFSDFcQ%5hNaLl+3!b|_Ch6!Z~2Mw<}qB?gPIyCld3r>RriRm0#dHA00g z3SpFuba8I-mQOt{oHQ%_hxG*FE}kN~r8KI>;OWXeq)=+p3CYY4hO32IhmIB3#mKKA zxuv0w%VBa`5f`@xGn)8PkFs^!tsf&b4}KLv!`@5*G?N3!qn>@pL`|-=!RvDMC&ToG ziKpL+tg@&s9YU^-k6gL`4vMiOAle`KIG!J(s@;XhlgJv^eF=AD^uSdn-8wMV}b zDmZZ6us=8Asw(us{oOF{4UKjIamfpq!h)R5XNO*e%cr2sv-a1mw)~EL;<2}$s1tan zE*L?_j*suCmsCsD_()Sh;sdY&!a@3lBdz*KBivwtOG2MA%q4<4<*6ca#Cs zbBs}+lr`~~IOroVl&YPvX##_-tTlAqWdSZzdQ^)NdV>ODAQKT2XaUX&WLN4Z3=wh) z#3lVIRq_%?J>sK3mp&K>0F6_J#}H{L&acpNJ%|vbgXAO~U%*kvid0@s5VZK@fL5&l z(5eCU3P7#{u}WFu&y%QW_VHxh99som$?ZF3eMtUIpdNZ2FF63`s4X$sB)KxKq$l_ef*+#KH`bfu<*Ub@cBp93&$C1q332 zF|oBB&Z!yMJFBmT-AxvB(DOrC0E)G{1s^ddA`1C#0g8OsNL51+*R2+MI4ih z63`|GX9N;w#TFwx-_Kb}xhN0S8h;U>K`*acw-<~=O1DLu_iO1+&7rxp+u z@~ce@-U@`rOYhi9vE^FN$wuFCbN?j|j^gcY%^zLHRRzrtBXq=#wCq)mC;$!o8RqJx z`8A$0B0w3v_VYRs7>(hT^5@w*t5FNgi0*jED4a7W z=Nwc3l*cfJ=jJSQ*_^f;(2fq1kOyYX zQ@#B~*b1sYP=xGIa@2MeezYa_Z1~1Z(}Upce55gqd&lO357Ta=Io399 z?21IM&>tFJ7nqIB&3!Aq2ZCkSRj;Xq1u55M=Ol`ihT7BB{LiF`DJhbitDZG&Iluni zN&ml{^l&bi#~MlTCGxvW=KfMSwuC;)ydx?nvt@-FA&3eVgzY`}E$!$M+36F72UPWz zxj?LvlAuAuu<*$By8=4ZV?d~18og?qz>BG%ISU8HA0lEufRFVPyknYDj=9G{pXY1& z^ICg)yl_&pli@vvz0A&R{sRCyiBDJVKiVh`m_Cwq=otUbuIcuf5p&HxGd5XcSwxl= zVWSA?%Vj@VE<+>AS}fu(=l2j;!Ug=H861L2Q6WTJg%P!_6rz`KIWQ=AKm~!$?7$E^ zB3r{>tUXZt6LspO;d+FlPy{8W*m>WYzLzMxS}D5D`qTY)+;b3L`me&Y{}Lkr zw}VGCynsMY`ED<5#|&pvLrkTzOF^n!Lktb_mNl3v_uDQ~pc_O4Bj^ivhnMd@X_#Dj zBKt%_Ww=&&CL#ji$x1Vn3sgdC(y}MatuW5xCi>u~Kp^V0?6uHMb6e|dFr^DCLA+kf z$xn<4hoYu>|AdgQ{9wP^=#R{klmhAaAutfJv63KEvzL+f?ASC-uHeCIYXwmac(QO{ zPE%IhIuY%d7u~j*T)ObK2WQa*0{qweO>sJEkB#gR_zPd4{xd15WQHhcgBhA?u4RL$tCfag&cQZWlsArCDpJe*2ePxLDe z^8PJr<6qwi>GmxcUVOjc1p!Gw?DI<$eGy6a7Rt6Yo=T;V1PMaG2hWjsQLHpP@9vJ` zgn7Mfnc6-+tH`5m(!nV0u(jJ9frRPF9JmU}ZW}H?zGYw+m6aizvD^?E9pED)p`u_& zea}K-LAT}6(B4bNxhuyT3f5HBxkl(SpvQfYv zbcMnW-i~-c$Ddq$exgVfW-QrG3t~@CS1ro@lzbP917ws{!4R3)Y;rB8zla-&>=h7d z2}cS6J_1SQO;R=ub%OXCsIJn&>u;dG-Spz~p574tebzHy^v;`|OT?9o`haH}qSJmQ zMc_mP@j@voD!#Xxd2@w-?3i%(|Gpi=M0F<9ZUs1%BL_bEu@;aBKmZab&a~+UVHR5{ zcRQ(m1uFj(m|)nzm|$VQdH}MwpWlZQ=v7p_IXSspHVAF+0Kw9w-T^f4FhMqO)ga)H z5~v>-ZMQr%z0$k&O!svQ1z$E$r_&Y7_FBX5sfHn<^)1tj5Wtr2{@T$wx~awhU~nRjHGW1)1>df? z)4YASPd3A$gJ3_D}jp0xQW+&xF#>RgL8 z4{W>o4@o*A9zw)O$^-=L1i##VaiHUC-XtzT*ga|Mc$r(!a~W>n>+OANa=Llo+r=#| z-ymaGC8kV2!E(t=Wxhd5t-hoPyT%wd1{@N_fQ8|JG%{i`QCA4CTB|()NW#oPJ-44; z2g`%0A`pm0_2ML3#ZEtRK24SA zMp*&E?2Yz@BJb034#39t$;;3Tct&NDG9#5Y#Wd?mlFB;B{^!jNLoMaT*yDvweZtL+ zx542X8Q#T@FKq1R_w>sb?PcQWY)_AURU63$IhnX?9-n4;z}(GA;5j$!-S;(iI3`#l z-_F?7HJ(a@Zaeg6l%LLeYF(Ed!4}Hat8H*M*Fl)G6hx@FfMIQ!|4z$8jam-ILpRNu zn*SDItqu1Ef@`k3iU%z~nKKg-uyfV#DLyZ3JT(U|f^V$6M^UESi^RG2dhQ5D#%@=> zo^<3aa77X%b;c|N<*jSjqxQB@G0?0Sh4J(2jLcmmp{SM1BfJV8;jrneY`QvR84E=k zqD517+!#6Ir_)fjiq1MoPU^S`(tk+2|L;Nf-*fW6JW56gcl!XiUsAfE0a_2$R$6vc zz7w5*`{Cy6z>tbVluNZ36hnSV_^$BRX3(DB-O=|(7w`~}f)_noi77)sIU*oD`o7(x zMru7Y1Q@^ju%63%VOIy^kf-{CkX8rdpM(syaG0NybR8Qgq81of+&-OnjE?jAJY~Km zC8yn8giYJ#< zP@&+v-vo}b);ruKutXS__|Dnkt4^8CL|*G+z!kQcr|EOP@!I?l=|oxd)7 zkAV%(@jKU2Lw6m5T2pVeju>P}?}!~~zWX@pY?s^hTb_QC`4}^=Q&5FdDW^}X=Y{(_ z=cY>VJ8kf~%5$GxFf89;$tGUyFLOW(RDangRI&HJEqi@=7gW;?Kl&Ixa<{}}4){8o z=N3rjs~5CZMSs34GzE=BJ>OZC5dD$hdXw>n^c#+7H*~cQdFr61Mt3|#h<`5qDE^u& zDmm~GOvzj7Vi(Vt>`0);#Cd2lJ>zsK2-oZWTsXTh!g+2lYZT$<*$&u7+18&i`|FX= zEiQ>z5(&wD0AjTIEC+$XGh0gS36-{u-jh0yw_VMF>-8XE*|QMG(aBS8Ma}8AYXEH8 zEYIaJ9m(9ERlCRcm6`OHTbI{yp3hTieaCZ%BZ3Rkyuo(HOTj9=8Ab;Bkh?!yEXb#s zzrIoI(iuwpbzJm`)zLoaLsPl4_B_fvVC=;EnRu<3J!YPETiSwMWvLNk^<2*Oq@EOL ztEu-((5$_qTV~O_H)tnLbN4 zrFy?4juy)B<-`x#x=}3^j~4G4Cx+d=+*)Eyqi^)B#s{r(2j;+@*G}R13_=)!%&;?=ilQ79nM+2ys@j?|Ex8zy*kaXXbgCWnUB%9oGR}SMrYyj3 z^JeZDtmHbGf>zn=B6`Nf36Gf;?B#a>8bw8{oxdFqXAgI2%&Z8&d2;riTK&Z3PE1sq zj`9l)+Lw(GNp7sTb!08Qm~x{lg(-#w>p#Vte=T?Zml~*kHrnaCVLk;)lss>30@C@NW#t|86#ku|tuC@jv;zs{wfv=Az#MRwbFRUg%E3lL{MAoi zFMDylrAePJ)JHyH`)wSFcUZ*?@-x`M^{;)aZqz2;^4mM4+T=a-$9rBlv2BRK&Txja z^yw`oZ9wmFZNEz{7z3ts$avm_zSIZ7yE&Xu>s4Pr#A#gY+PX#83_k-Z!->}dFY)mB ztEB<&r)Ko-x8II@s*6H^0JPR-+gGDY;)mgzPKof$o7oK@f&9vS{BZc!x`;ER; zXenLqMVI)6lte+Nuq?~(qul`q`5kN!n~2W|?haQEJ#oLA+l}o@Uqbi^U!*$eEk(ko z6Y2(@pPom+>{+O?!ajnbL#-(K#W6I&Eu`pvbm#By`LqwX7bJgl$(DY#Q5UOtH(!=J zz%_7d9_6fC?*q-)(vZAd3<68LfI(NT?z)hp7|~U4YfuZT*KeUQnt=fqkO+s#KGrGA z++zof;i0vy^fRkuS_h2ScRH@9Z{FjRQn`v`zaF_qAvht5nC^Qs(K?TMT(}L-;U0aa z2yg6?&NMV^LAFUbA?9fPrA%)P9)rLCx(G9bd@vk%TD_7j!GUbeIS+aF6$%9Qy9@SN z6Uw~WuCb~a+&-=6ocN&D%mj;$j_`txL9oD?P9jt=?3?2G%DgT5D z&<&-RiI1E<-FSqNq1M^V{dRj7sC$YQT={5!e1;~z-!o||QBm2=e9#a} z5|XT>c(&ktCUrmXz}4s}^Emg}o3P~@1ph#X!M2RPsm-{XCuMZ@{1rgwb>!2L8#Z)Z zGkQ-o%zHwUaKe}x4ubFGFgu5eM2qgVzLyACT)q8zApGKbfV^s!aJ_w?TNXdy#jA0q z!+7u^hIDHr{#R%-4<_@?!t_fM%x*KFwvrMh-bI(%6Ci}DC<=jPTfflO#tSZ!pPJQsg&FPZ?o3m2dn3E;`>{Erc;f9 z=1}VrfVto&`FgYfiKHd3E+v+YcJ2VS8o+Jq6ORU&oiyZ%WS_7JJYY_XiWsZ(D-sCC z%8sN8PRcdKvT9Dd5mhkKarHE}@cg#UnXZTPr3Z(4%6#5^w_De--jt=iUBUmESJ(b# zmA=Lf1$!sr@7My`%2Kq4r?U zS3yEvdcSbwfM%ZKtdR3SI#RNZ9yhp18Ea(r#2(^v!RjFT4Y%j_&K<|bKOV%qXByaN z=_foXXC;lcJ*=Hg;1N--7qTr;sa?@-a(Gb11#IC8t{1AI!FWdBcCgR9xFxv{H}BwC z`6%2(etfT)c>Zj=e923ySOa|VZ&EZ-J(V+OL|ru+FJ(}7`>;ghYa;peZ@gX3zeZ7% z<{>4rndB^piTwbVU393?Gz#m2SlDO7Li~9SfazpyT>7No^QzK!tIN|lQ@?~lf~VDq zszfnA?Oq_Vw$}q-LePH5b=}QewiqL+j5o+AO{Z z+^}IQsiQI7D&Z)TWe-D3*W}VZVro3Scnm+0r8EMVWkn@5%SyuV26gQll@!6naF44W z)#f;l{1<)X0jK;_BIgs8TH)JV_Nb+@8l46^qOrvOt*wjpTkVN0&C7-m0o~HmR}KP5 zfwMo7mgS0@;-6HG%sUq$TU zjiaHZ@P`a>WR5O%ehvkf6V^7}GNNAG9Z-MdW%P^Z1QIm{4}iZ@Y(8(IhK)8Qlu$i+)$u@4b~CNOyB9<6aQdwo3sG9jV0;IPR|4C%O>2P$WUiL z-``po$JPPZ)1qNTh(idCF2S{|>Py3fUY|afikb-?#p);GRG;TE_gS$$7z<2eZ4Itt z+!-?HXkaRaVq3StTrE}mYD&Ip&5+Ck50jno#H_lp6Vcm_>qV@%=-kwvJLoJa7vr%J z$pv~?Y=2GpanGkUXf|RE=ERS2VCK#j;F^Tz3RHQG3H6P_-9awLdAaM3;VZH0S~48l z^pMbV?-OPsz*{Mk74;{{QdxF$ViU`#sRwV%aM%pCIU z%sBDZwbj8Laq650oWHmiaP~c02!QedX zkR#?L=~~FF%d^R=@>DDN8xD4#+UXJ80}u9x*XDgt?*aEj0iAaP8}bO&g59b#setgw zb*2&8#e5FbF8r3~+P-OVtC~zOe*;hK|7*YVpShJF7dl=I(BJxeK)TX{SaivLjvf*o zX;A%Ps9yk$8|n`wffyPT4dgu+W0r7=8s?=P9+_(<0Cd)k0*2Gq*giE`rHrIkZ&3i= zrPhV&C!EldvSPW2!oznj-(5c$duE_Id>)8w*zdf@bxO2;U3U%MAXZx$P+v8PIV+Qq z^YsO+#r@(=zEewS(L`v_>m$RarH!e1D6Pimg{wXXv{EVRv$6)=Yq`?xMXo*7ziE~q z2X65oM4a^CCP=Z8tRjyUr?Y8Fk)=8wR$hWvRKcw@_%1xuz9wNjKgylxW z{6f11uQ7I8FKoUck6wRlu?y$QsIV{M>RIE;LA)#QSW$O75TG&ojYp8`ZsK62Nkp@T zioxVr)P7TE9Bp4`k>L6ZNa=GPh|0C~c3~q$Z*QwDe!go!xYMX1)9~OOZNEoUpV!x$ zU3Unv2>cfdAoWJ$sEmToz9yBY& z>_RIB_vQ;uSOL)XBbkE(bL)IS>uhNSt{ZOXBm1sOx@^rDpjFQK)2P7OHt?~z8oJ)p zc{}*cMrB!N6*TOM-}^W+3&P+GOmw%k;uGEM>Kqr;T?lKkvKI7PCpS86x?XrApxF+i z#Jn@JGLc=M8eVbFGD1njSZgRSm>qUJ9G2`9F7H!1W8dz)MQxVV$?SSy$kTe~tf~TV z$~Av1i~`<&iYzUBu?;}`dAShKK*)VO@FQtOu4d#m(8B=Fv4(c|W%>OX-`EL;P2V>`P)=*F4*R-RUz@Ciy=+mSn3?!>b4SstpZEiL2Lm-n>Kan`~vc6+=b- z>Qjc-=S6j|EWAmp)n8l{vlsbGwe_u>mNGSs+P;Zqnz1@kJK#_-zcD4lOsfY70I4k>C&@B`Eo(+vGu3o%^l~`V8Meu`E4+S@ zXp_Ev_ef+Se=XNouZm&_Z!F^jcYgJ|5w>w;g@wPW=cT)H&`BEB18o5~+0||o{NUZj z(hQc(KY)mk1l61=S!nWiJ|FrI#AHCOkI%1^JpQW7W5!x@zuYc`491&Wx$RLO)~&G6 z&qKUZ;DjqDyX-;ZhBo$~TepH9C%L%&19PDBP!xcGcTWQ0d>r0A-yB+peC2K6%#4Kj zJo5S*lGzskT1RVGWebZxRz`m@Bzx_qg|oDeqe< z>ol#(IeM`OA<>rIFwY5L^OEz=Lk;ckT&@Eb_8&uDfUzgd#S5uh9@<@@#6tr0>RYR2 zbCk9^#_?~vqr_pE@XzQ|7zm=(PWsp=u#{S?Zr*J<>7vSUo7Un3hTuiO#)>U!4-;sY z+II4}Lf1&Al!;HeWzMZ@zjjTm_huml>whGNe<>6Gm)Fo}0GJ~w4Os!5HF`OkK`#6F$RwPZl`Bg%LY{R$) zOQpD$1Pt8^kHxg&Ec8pYBs8;exYb3POz}?HF;pM+dB-?Xkx0$GbSr*#V!3^}P@4e2 zwBWEP(xv?V>)#<(NzG(GR4gdm4;rH@w# ztNXG6$b<7g_$5LWaA8>^D(OIpV)%<7pwYe~oG&$A_Ig=uvOQ_#z1V5ez@l3 zbuAyoZ(_3@akZJL6y9bJJ;MTTDdeiFskR2#_^eNmBiT<=-@yUpgUjT6B*-t0MZJ`I zh`JAIUqd1}bD3X9X4Ex?qDDDIp2UnJQqX%;O51al2adJD20cj2=%zl^)5+r zP?=r>`>LSzf$?KzhI_k%vVhRW5VBGhr0de$VlKdGCBliasfJK2kLOaS%lgKKRP^$P-+HO zM=$o`dv@vV-4(%X<);nyKSq>3Kto2K=L~_?Zy81Q%d0U+9>L&K;lS1>3eiFNFphqlP1+R|b}nwU z?T5J!zIJ|n>76s{oc(3?t>?T*?bJRb8IS4?ywBZU#>c3btQ8@RJ=$BskxmYK;} z=Zg8WfNaXfS%a=82sTfZq+eL_O?>(m){5L#!MoZGh{{#r=S!B-rEGgDq=8Rti&xHuY5ZYjz_G)S+$|ANT|ms&sTa~HP9 zxg!w==3t#tKf)yJaO;UzUk#BRoZ^I)Fbc(oma%WMHrnK@_-aVO!w7AU@?GqWwBAD; z*D;t@RjHxk#^!lss@Gl}^sYLX)HIB8rUfEqa-=kWFwFn0%e*lWR$9&yQneWtdW^F} z+v11Bz{dDb$}+lcIoi~l=lm$S)UR$!T!@5VrfOSRzHxXQ!d+K=n-XfZwzjx@;OQse znV@n&s{sN*&ra!uQpn ziG(j>jvwC8?;Gg^9p$-#w|?cEd!xjDws!OLd{Mf)V!y{A!Dn3j@p(dIL>~{{spm-= zV~`%>%H@Vv(qx)#YzbKX5U>C$npNm_ zHv9G-_3q7fPWRl*#2X_au`2L;t?COJV4u8Xg%OOL>FXNG36>CbnjWT863lQqS2a9z zJt3r81#cWNq)0R~Mx2fn7dAzo#b}&Pa*B~ykx{*mN4RCnp8!o<+vqSPc0BneJMhh@UBpB2~; z-3n6buMV^m+bMg8nnU_MDOKtmkgNA-Us|UW=1D#GLN}6slD^*_=Nx>^fzBPl)ba z1v_|mXb6|y2H-co$Y|!{^0pZ+K!G0-E&Zu(CrC?)Zj>_&HO(ameFnNkjcJPW#-p*N z{P3yW2;KtzG&AUs0FHKvDz%t~g6AVfZ%LRdo(Hk8u{&P+uziR6&DE>s)oue^seQXt zsIJuvK<4el6|KnIn0fi|W>7>dF3f|nxOPzk0KzUO?XAeErFI0FPT`BN9Z(^1y_!Z$_4^i2LCz)*x~Uy|f-?T>8dN`K#sxj)x?*dD$(A1g zn@+@R1&MTGGLA_9&9a*!H^Mp{6EAMuY{(pD@-VS+?36GgnWk?&=)fyC699B}>aX*Bng6#(V}xk>z_4$wR}VR;qS`*uWgEgAD!!;#0Th`^y~FsR1k=|c`*pgu z)|Sluw`7ilg{#dQW zSsEOq`#zx$IyYq@K8#Qv1Z=iW*}_VlDLHZ0vXjpF8NXofFrv`xEobisz2z<1-^UHvb5({pFEl9PHdSy=mc?g^ zU=9bLEO>&e2Zec;_Oo>Tc3wMIK(F+{5&Z4EA1L? z@FqAVT~m#!h5lzPC$@bK(z+TK&cXMP+pV9_Cf~;mtTj&;_gQnvwoEu{t6_Fg>3%Il zSMLfop20F9iNH_1n{M#6=G8YfhT>qIxb;QP1ta5&MTH|r?A*Ybn2brz|_cP zA5n8CvJuPATZWPT{cPB+ktwOGQw|Oe79vgN2?uU9m9+x2r0D=MN&7o?;V|8Q=qTW^DFIa7|++uKVRueH1k%0c^lAfG9dC}V5!yIuxU+aDCmS!t7 z(6cI^msq{e*h#g>$d6UbPct}Yo6_Peb*jiJ&Pbml(?zMi$s4f$dG}-HSbiQSX(mjs z5Aj;bSm)|ZYTY2Q)UwkLh2?J(8fwoox#aFpGf&i95iE4S2J$@tbV1hHnPx3TNk8Ju zvb3WbdgJk2Q;>W|&?%l)Tys(aCJZ8L2n%_MoI0ou993&R^DjGgZZY5-J znYqx;w@rY1zQ_^AH{~CkR9frbv~^4*58ofF!t|JZBx?8W2J?{H^acAS1#t&q2)-zs zue6%Kr9tkNHGGLod&-pDu6t&tbPcLv2Jf_AhbB_c62|zbj&dfBT|1m~X^1~6f*{G| z%$9rk2|vvF-?&_(X1tR;p+*{I;kIvazeOesR=Y5HJt1dmT^Oi@5ei#I`4Np%+{gv4 zggbD}y~}x2`lG=E8-v;N26K_4**wv01ZdSa$c#S?T>5DS>g!Ym9g*NJTYEOM8VO<- zq0C64V*5&(Fw`vN2nKOzl>a6L6Z|1wrpw0*i`h(KQZ=#)+YhCSe&IFO@&tWDurrn< zHTYVL__wbtv)gPo+w!us?b>0HeFl;e<)*t~RyXBg2HN_atukU9gTvR&yJ*%h340xAS*OVso3>k4x zSZ8F&1ml;{*d4n*zRU7ZGUtfz&i`CV6n5Nz&`sVvd#{_Y{%aj7(<#H$7ZX&~H3>jqy>V?X1LNLV}ph8@NaZ{X8>1U?Wsnx2y-Em+jC5RFj)*`x z2MpK)7^^dK;4jP0v-$~Xw(^(GM4%bl_9vHuIb}%_%8J$-9TQy)zn?E_ffy0$WyCRM z9)wwNJ};0}&nIOA=E$?d%66s|ABeT(PbnFhgeM|irxMwWh`X&s#hzEpEqR>shwLNV zqC?s5XX$qlRfSjn!M#ghJC>;Zxmuw<1Fo^OFYDmjLHf$DSb+zrzT)EI$l@H2!6ZrR zCE&f&&21B!KK7G(mjCw3^8ZWT_E4Y=`<;e^A)Ri*6f~P zR-Zm$axK{fezLMkYFvBwa#SCb9uNRSrdOh#d)4+Ebb3j_`Z+ZU#vzE z0x8hR*m3&5BWzElb#cOxr_12uNKx`3Sk|j|LK;(?#h3 z3SoC_^MySg8Cb>t*MVX#N6gy|Aq}Q4;xVUAFQMyM!k(*-kXr|$7rPecWS3+=UVG9T zTq4O`Ns@Tl+0*QUyaDwYklK#;6cA@hAh)0z~`(1bN zG(h8j%pCr%%V6zRCBW1e*WA_%Xz750^NBNkdb<^y?Oioj^Q>#4fjDK+$|W2OLWgY~ z3~8gKXWjD?YXm;ZTG{j9#qJL?Ff9`f{fWQ_N7`m-Ko%e%2;uRDW z^oC(kJc@0(x8Ru&dp_(Ji(6QH<9>O(gk6aK6CMTLbX8Y&jowddMJI!1iJEkK2^Bsk zg**U(Ub;8uEteyS=MU{U5Ave6iQW9Mevrxy{?$}c+p27)fQ$+KgsCr6^JB;*U;2u7 z!UE7_O>^_YGK5vDb-&k@3#r9d6G_1i3K2_5jgpJE`qPyz&TNZ)DlMZ)x(@6~5%^I4 zjgjU=J$dVQ!`v2w$`>L^l(3)zdR(#=&t*+EMLiUGd&SAY?$U_Mub)Y}(%+GId8>zK zg^=D~({9J3bBM@sR3?&Q#?qdgLxVsU$K_RZ0XQns*D(V)rHUQQY%}NUhMH6Ki>)~i zKStG}$#eqNptp^dGY;tVajwTs)9&73N%nKn&BzTxVSN~Vurp+R)^08UddoPZtDE5f1)g+APWlzZ*#=cej9NI3oJ9cnSj&onu7S zH+Iy`LdXaQ-uW{D>|{`yr}VXzhVTV>?+5%gv}NH z`zWH`HCpO)r27afv(hlmvA$niZ47{#!P6HK_t`qfKD!iBr>vlQuWqmOgMag;K5Xpy zH4ko!HQ8%&jGK=e$n9^)&9V-0?d7M-X5Qxe^S3ejE}lF1J+P4a!`aqKQ$I;qJ)L+5 z#_Q1Y?*7&cL>Z2yBps zO(rZuf!!n9lIXr*8;I6-`dYTJ>v`RZoz6j|9|BuMSj9QH=M&NWm3_qECV5ns*IqCm(~FFPC+Q zXA>4?$LwlgwZb^zJ+!E`Cv^-~``kS4;9Y+aAhE~>oq+Q%*0VAId5H>i+*{+-bj3K(i5Q4H^c{c2C0ow^%L0n-nn_}Ja6DB2 zT*vqlr*>?}v5b9vh9j?c3eAa>O!*{u!adbO=JAr8-c45jR8U7=x9D| z)2-CVD<|8;Am!6~eT)72wJE9ouH=-! z2#ColYo{&? z0^J|UphuGIphr})E$nc(9a%#!ehg=7H12dioYHdR(>KbxZJIU#S5R!tY^{pmpKEg+ zk37%d2VCnX;S@wx21?fk)XiL9YGuC8KPOQPg<0QAL^U=J!;d=r8scO(4Uv7&7MweMxWUp7F3ZWPld4E_s(eXU9! z5&@SAg}t;x71^+?3M2Spqg3n4RaZlJch@oJUqZ4H$il$*lAFOkxmRyMG|!xB$%7_P z3FgLqQ2Rk^*6!)CY9l(l=1^U7GC2?a+U_izUGcAiT)2bwOuj1~lbWX6qjhxFMCT~y zo1q~}D{9?Loj@GhktA00sLIxQtX+!R`T`TikGCo0ax|+_BM6|VBEJqUsry7JX`Ok%wNqw7UQDn?*0>a8~#=_ zWZ+Cu@QLF2u3vDFWx$rX}$Ov7VM)8?wsSgfERsDqfdj%O6&m-?o zlMe1A$l%w;emH#HY(<*8KHPoe$KHCuGxSn6Lp;Y2gI?Cr)$J9>)2Bf8r_S}NZn&C| zy*5SvmqOD4U{nM0u}KqN;@>{^zYPfsZ-AGmn3|f--OSFi@&h1nGA(8Vh}H_1Woi_= z!R~2Yv~Ab-0&3U>P@!<_T2x1w*#a?MgpRt)x!lm*chb#KRq`;@?KJ*=f@h%3 ziD6+x1lY{Um1VLQlmu->PO)m+KTGeL$|iH&FdYD5^X6;w%&!eFmFPSkMT3k=xn_z; z>8(sQJBiq5vf}-qYoPD-aps4e=LI4lmZSAkeoU9L0(T2_>Js3r6oi$dNkV^6Q-l-K z0@m?izin@CQ1@-uUlM<;xh6&Fw8Ns z5Lqy5O~SkoYE&(gkvU)Zu6uJYG5aOQcu*`qmvF|zIq%|dJAF|89K@I%_iIP878G}{vboWzl^1kE-+tf=>G^5uZ=H%8Tl2`Fs9(2?CZEoK8oK#F zxk+t~k$=v`yl3qg{8vkb`pbDRiQ)3tT>D8}r9#_!Bk^3d_hSX0KW#NW9F#@>Ace~J zo+;O6lfgrrMBe}P>z4;s>Y3ony&?gk!_vHYE+g_iQ#vpC!hW-ckx~8u|B{e6RLh%o#-MJ86k2NkckuEeq~KFammAhO+f5KcNvv1X#@ zn5>oHm8ffK{!WKc8kMHb?lFB7g5JLP@Bg%GmX!p~nWLGQc(mNqODH2doAFO%(-#|; zG3w97ZY9hnyn}&b=*{aPL_I)xdE0&R49`|n{FSHI{`=|vV!bWb3S=GsLv6ljD50|F z%@Hfs1?;_icS#Z~A{>l&N}S>BH@KJrQx!W!C}BQXoJ2ZKq+c!w8{ex@$!h4UIcLDe zc(zZ>cbl@e@2W9~tio_)-Lmh4a@w zR@C!8LTCVu3e1fun8*x0XG^3yu)3J+wYUUz+JFtX2FPTqws$m%RwD^vV=Rvu=pQUc z+26C2gz(P$CRvw$iz_@2qfr5NltFK6`CTYiNf&kz6_h4ea;wrrQg2TuEn$706dr+N z%^g?-TSBp~S(t+-g~ewhVm^5z3oJ(-wrVib65nnT%X9o}N0iBPtm z+3SQVK?-p%Ye)HJsT~|&Psd>XM8sc!yd#dUrwg^e!R%4QJQGQMM1K#gA2ZU`RT?QFA)R@rcC~3Z zvB`0m?_PT)@c6?0>o6$0nwl2gEuZ>OqYFOTSt0Zq_&z!Fs6yowkMQ3v`rGG{pn$tjPOJrdl)weVE8?71dI&w-y(5zIHXqia$$W<}ZDw z<>?kPaZX<30*+7esdSJVk-)T_3@3$v!1A^u>G1FfOvh4q3N=tX=S zTNAyq#7cuT0ypfesI`~yINYZSu=`mZ{ihRZLJ3eFmBL)jU1TD-Qx4`9lVMlDfryJ% zkd0mUuuimHPf=%DtPA>IkaB&xCVB@SH`DU4DEYV=BEZZuF#qa@Af5iIq;IV6kMsRs z4{KZbnO!x2HX{)TXDT_+e}ohfRXNygA)bEH;!i-4)SI}_PPSRrpvCKh>l7^;n(R@m z8Z5K>$9r}cDl`)I&G?Gdu{T8g(iA?;9zNo}>2sUGC2J}_4FZWr0BP^Pk;XkExU-dT zR|&&aBF~1*CE~xu@DH~6*AtqQ<3KEaFnl`DWQy4sn3_(D;RA$!8V#7iYy8Wm zYN387?k0DKt=p;R7I*YEtTTK*w;!~YdF>Wybequz(L%Yv{@tKiq)6iyUY&T>bGlDD z^`A`VLo!s9tG$zHNr>bLrQh6pD<2}P^C+u!)N7zu|C|<@lRp;JNjlM?P86C7{(40u zeXx$jMm*47fzz<4lR?=?1b6GAGHgrT8-2s6(r0V;`cxeP_M_*pEAch){f^MyR!`)7r+ zioqcQi`cqoW0=j@{dy^iP=;KT47C}u`7mdNeMg*jtzJn5-a9;!P@*rrI20ks-d<}H z4Rm)V9Lz=iXw0SQCrB5iFE->#4%QWi9p3xE;T*kr=NyNkghUJd18`6PtE%M$PhD6) zP1qaHXWm9e1Pn9bcI~5--UA{}yx9kWlY|khzjfvxVDQhQ_OB;yj2~%ghPTJ+59|g# zpCJ}tVXjP1U8e~=*Dcy$1M3S`xBETF6Y>004Lqdj*4Df4Kc?#Mh$Fuh`k|SUl~o>4 z#L>n@OX*27>1lEx%6}2ybUG4;FwuH~@7MC_vJG;A4f4KVZ<9E!VE; zVfHzH#Z%@NvH8!F0!Xx1eXRUrRrS<%lZzd9=-rNyTw7wT450-7?WFSm@K#Cx7*iBM z`#D9|CLVpWgy^*6`uf^|3Xf)|Tlo=S7dSKN>rDs!K&*N1&M^AP$n~t@>NbN*iLGz+ zKFIg6s-xVC<*~?*Rj2;M=Xrc_>PexCLvXe9*}f0nQmv`y%6@Y+aA!k*xC7_vjl9J-MIy#h05z}?&4DiB z^{sgUm9lD+X0puCv*4e4TGJg z8k8gr9;wTmkaY;m6oz4<#p_ku=&t!k<46^IU(aiHE|I2=HZp!#% zu_9i9WNe*T)$$4;ZbjhI_e3(c(Le&{Z_Gw>jwiT+)mA3W4Eco+_ZjH3ng%Ly)y&h0_zj*`2;f$hZJ6XbIyk5$SfG6s*7ERZ zqV?Gw$==BEJc%zvEV;<`JZXd}^gBDV`&RO@lWB{R1kc(9N)}NyY3clX5#EIf*%D8} zl6ZOtf%A@-+aIeLFvy;1i?d!U>FcHa?FA#jL2pyU@FJk%gV%^bZ{ARq^+3*n%`0># z4V&oafD)nI_g%Y2g4lUVnQdvhoomHSwDhQ;jlQDcOWBcxhhpqLplZaO&g?<044=nX zQ+F(7j|%pUIOb`BKQLpg(l^a`XCuWr(A98FCCO}5tq_Jaa(r#RrXerKJ`#os{nFs< zuhSX<%hTHYz4g0#60TL!updrE*g9^oZn)ENP8#W_c0x;%K~M>G?F&sjknk9$IE>30 zzW$2H>)Xi&+>`Xq_}N0@oBs6?^*kQ zSy%l}B!pf{d>+sH7deajD|PEfIAb{+#=nRg1NDOqAeWVORGv#GtE)VlwJN{s-+Z#P zOr(J63CvrI-iPzz#%o|tMv#PACiT8jmhOvZLK<}fD>d@h?Te2&ja+}Bg8y@1wj z6L$ylxie~|`JeJ{Bbmn)v+p0%x{R)44!c-qjZLpvk)&&<5VT41Pp9ukJLbUfFh@oX zhh>Uz9~T)di_fE*31{kbndYIiO*deB+UUj*Oxg#y?r5;WqpNNSdp>pll{e%@Iuy3$ zwp$s!2CAp0XU?UnImg$EXmj8xLMT^lAQVr=N+qPEa)G*Ywchc&)Cj4}!CH6%g@LXC znDCI!8Urprw~V$=2CNARFa<2|}c67lbTa0Xsq|a<+Dx zZBNcu+%g>#h%tDqI?GLn%7FbiY%GgD&akpM;XvV->-Bhu`WqYeE@jq};`?`O*Aqf{ zyKe`U(}bb>+nXluDR>2@=X47JR~sb>xm`sEj<02lDdW{+pBK=H5S^ zl)NmA1ZbQiBQ0tQiZGyet*o~73<)ONIuv_JNqN)V`2jGvk+HL@&U||jT{h3M4{>|G z-~CzxB?U)7K;ZG_f3(Ks5TQI1-o7nGfqSu`G*(e!&!Z{{L0E!GSOU*+XjhY_@kTJb zf!jK~Hg=!GG8U`Uv?IsgBI~D!w72E@Ie596 zRP5t~0j^R9qMC7kIE?0l(lRj}9UY<5$r42N(^lXleF3mNV4u;P33n@7_Xen&EuI@< z|0(jHCih|s4bdL3!`*x3q!o$VXMVr+>98&ba0w?3-Z9EwC&U@lEuNF-d}d+e6@JIX zD|Jn$@Ex1|Ld?CBs!=3}XWi0n2*QUW5^hse`b%5BK8-9%*` zP&ni-!6$WE+WJZ5E22A|hOV;I;BKKd>Ac)gH+H~%Qf?>t+>yu*q3H_`3$e|xM2osp9@@DYi(Ffov9vs)U> z+c{K@J}ET?gfnQevS%!d?jk=N`k_o2|^PdIhLnc%q@`9ChkKw96)*TT;az zCV{~Eo)-5@Q_duBK$lj-8ecd_`$Z7pB4a-zH3b9UYnabIcViU{9MiN;R{)@n1c+xT zg-NvAM-7o@RYiVEX=EwHZ_j%Lu^QU&VQ6<9FT-4`Qq^rtTETL2)K|zw^jottGlSfZ zmxW{{#lp+E$oB|V#Q$vu+cWzkSqOW0o^>K_%dvp>WP!R>%hXOj9v;;B3X2>Mqa z@CU*&xolI^^_|AWi_8-~OpQAK;{a(R%fr4{AM%+q3Z2);N4k3OY$|4LZETz`LELPm zRxO3N`69dpy%_+6d%sZu$k3;Yro#&MF1oR)DGH#8!K4iMY5PsM;rC)L=yKc8x9%k? zL%Ul+{sOWCq1ErEW`&7uOK90izNPgzeM>WJxPPzOvH*o#_q{ef>{1M=7~@f?q@GZ~ zpcu{WX?-|j)TY^ERiXJ4xdPUOunH6X+@~JuU39K4z&0#4o&wr(>0MzID^j4m2uKuy zg7H$PehUgRWdK7L-?%;tS*Yz$VTw8fQ&vg%9q##7sjdoW;4xTW-)ouaFj$G+xNQYz z>R7mNhLP&+G*~gP95vWGt`ZW>D`HkN!$?9gAl>zuAdvcAZs*5LReL=w4~G4-WG@v# zkIjdY_a=Tl=;B^a3uDp`bFRLz?Sc)R_7YrKwyan&!Z_z?1kkg@6_O0x%tk!qMFhLv zDt$u!^DH0iL`vv>r5ZhUm*X9Om#;3zY6QX3K1syE_(b-AOu^ewg|5i`uh{MWKk%}Z zz<=RFM7e7AJYVX%=4CE@!d}Dr(2_g1%YyjSQzKIGkUGIJq^X5k=hBF%X1?>Bw8svA z#TK~S5{+D6G7d_`Xj|FWvb#RJg(?gaZP%@O_~b57y>9vAjjL-Gv0Ldsg?CAb%b`xF zM)d4y`qsIA7IhJy&F-B%UKPmg_rAMX)5)t^Y6yQVIWVZ%!z;SqJuM#h(2{lJe>t5Z zG5r>v=qp>ukljFDXa4yrm9EP27O@TXo4~gAG+qPkx{9ZXzgYlkYDp-A97!9_xSpzH z){lT02piDCRV<#v$YWUK1u+sH!?4IHcx(W6v{7C&lJNPq6de<0#p95%5!qPC%d15u z;c!lCm%Y?k2O^5jBQ~R}qeIvBcp;kz!-66RIjWJ+2bcdNaUg$AT}gbniW05(fMCo)O}Di#hN|J(OwjQ@ZE#I}w+VAZ$f7m6N1=j%lSi zw2+hZU^%+q%}t&DP@ckwwt9rAW>9LD|mT#J?Zh8OOE;VnoK2F}?l?sV@{>Vd0L14e8_@;SJf z$ZRL*_a`qt1n7n)r}}e$!)?SWepW}Q>CH@!T)NX^orLvpEmToovXaw5GO2JT|BbvN zw7^Y&1pz5=^2>jDY!mnLPN|>5_TKRyMZx=McNa-j2dNxM&^^Q_yca25ZqED&#g1Z% zngPxx%pm%lm8^m!bM%X+TMkmdCWOoS{pAyMF}(N$g1a8qR5p(r+3;&63PQ`@Xz*^| z8Wf%T6Ghs|UX?gdZM0jgs-p|ec<_5@sekP*4ffub1@9WYCVA=;N8G79 zLC))5%c+VzM2ssmmIsMgmT+>V?}}PRo{7?1QS%gNAR2i~3OyK-k)Th7sMp+7 zZjV+t0EDK!@3ykEexI@JX@%r{Rm1MPSXWloym>|>9KEeCYKvVHCd?xkn6%n*tK|ydlUmpRV%UfVm3dLLKPQ4d90lPIJd%&4dY~ z^z{8N6mEmG#RJ~K4ljR*zBycnkDuHNAP-4hKpzB_3WT{gb=pIXAcMu19DXxyELc`1 zfA+q+`2p-7gcey{sExVJ1>7pK#svmB-T$03S2Ez7p>L3=ePV?jPNG%aa`Zc}%=_z$ zxn_>L{xdt_a=OA;cDK#!E4_sFd`ySQ*U0c+GKbI^W&>bS-u7h4>N>ZHay>0CXW$`P z-sp||!*-a6BE!O1ct$)4CxuJ;H%5da`cd7%$D@FEno<{R@_2xj)~_-nB!+EilM1pOvOV0{VTPMksG;g4yakDZQc0 zdPB+u?lHEov`G0p18xWvmvC2$n_N^f!u)aB$5WiIcCPP9VFRMNd!3=Wjza0)N^<<%p14O`3h;r#rk0Z&v#Tb z?!Ky_=59*Qzc=yT@p1i!QT^&C#F0Pm0tl~4{9#v|+ML78MkSnZfki#~$^*H!Ev%1f zyiqigZ+EXLI5C8*b~oxrE+@5R7zCQs+MhVK_cM3>@^SItAuY+SFbYo*g*it2Z5q1D zevcf&$qmT)hRCt7G-LLkFE>{V>R<+|a01nV<`0uR@`V$6(D;1Wz@$E>KT3y^CG$`v zO)z2qX7tbwhiSP{;Y~NgCpS>fW!?1IiDsR7pi@JAk#ntn#i)Sf=$ic%Q;_+E*|&^# zg&Pv|;b>=E73)`8 zpW40HPu*2&?U^0-^CK#s-Uy0MNj_`*l>GvYWU>P@l#$AATpD>Yb* z#FRxy$CPpKpew3)9k^)pAK>(d>)>DTMMk= zL}uvZESU_1LZW;kdG8YTN1G?(;n7yp&;Z;dZ^;wXY&Anh;gU(3)+?)06f z)jucx&qp5n=iU-99l-RboO=U|3|Yuw{LHoM^8|{6DTm!g;JY}!s7$jM8Cjcq1^_6&G$yK2%?1IG(=mi# z2N;brzQDRav#R$HwG#fpcKwo%AqOh%b{kTx{Ir&F6iopoi&Av9$%Q|^51oj%%XITH zkf|_vEcxRg`~oY%b_(+SaRcVr#PotLi%wVK9-l0l>yzhmca6aPt9Zox@#EJXO}xx7 zJiAve9%Kwv*ImkAsUNzmSzr*!U@QfFh2N)c$;T8>G}m2_*YiA{b@$V*iTUuN#OXx072V0Mp>Zd#_y6C z;+`$otQhKfL3e}8<2GF7LFgXG+axt(aWt1wWiOXb;2t*qnq|&vqG`M%kANCz_+^xG(YbwD$2wiU4*oh1Iu5|yrK0yWby`0DTh5^F0E| zGlaha`5HCML&Ju}(b0F4rgoD`K|9)VzV#rFRAxga8CjAgD&Jh%$9+)hf1mrkMWhmY z6dETEjm~kR(-NiFeg<1!0^hMb;l6U~PLZEw&w1G=_>%zgY*~RP5mf)=%lA@J=f3QS z6eMOE4a4*SML96{N zN$x_ehc}FvRnBx(edDB!2TYvW8d{X%fu4l@#sQ`;3b@&hXA$*UN&ib%8PminIdSsH z66kpfO=^pfS3J4RlFTr5e3}#$`z*5Sx0FaSr)b^kx;iiB=2~*Bz`d?%e6c@^xP%?o zU&r8?8`fS)8c&RzX=}kcu|gHqy=YT1+R94Mf!l@edADw2b)D-n$e`fX=&Q% z{%SiUPV2TevRdzrQJvTnE;CmARjun~n!VL8E7X|*5bmt*7EM{nJ7Y;~XGX~tLF?TAAa zYH#K48#a;w#ypKzSMWR>#w8$P`oJu2{5H<)~Tlf-NATd@7D zjO@8UgOz9VP!|57clyat_~jS7-wyj)0opkZA&SZ9L=kbTzDIJYmgN~qlfdx$fQ?ZK z%^`!@vGsSDTdM1kmE1nLA=&u!J&Eq zGfoif6Eg0j8Q+bNGy9XKOUiByGkcOTfgX`&s)zQFv{EN1KdN7EDF!J*9D)DV4B~qyPEJCQHd}%q> zPrf#<`kn!?Bm!8o@e1=IKLMjwP)-(9Qd?IS@m;qbh0oF7O7PPdI6gc3*twRb-Emjp z!u1#v7?P<*uYBbw@s|nYWn%3`n|S1z*m8;XP5U@=d z`?1gxdA4LwY_l(HNtjDf z7qpcS^c}?YnF$!SDEjt>54(|kOEmjU>E<*3K)cfXF?zQEgZr@}llo>Ox|jol>PU-n zRIX#AEU9G>MGO#N5;j8Y5r)d8s{hc|-6+cAhM!(5gur7;?7F>KS30ayXM>>wiAE6C z3oZz#uQPteE^@>f+*uYLxGfUoo;g1L&Su%yab6d+5Jpo&NF0{gG;j2!Shw3=%!fq2 zMm;L~`ju$@Vn*@%I@ZA1SkU8SiN``?J%JHiIV2tcUAiLA71OVJXPMvsHp!*Rxgt{a z?wp%zdW6+w6UZ`|dtic@{@RQx;v_R-S6M+$(@bBZnuqv}IO0&oxjht^3ItQkkT0QB9I;d z1B{>0~5{#Nzdcr$}uLM0vawJ z7He+wzcDx98@>fY8S@I9nCvqmBzu)yAufU=-VC&Tc^u0Z4P0idi2eI&BovrddpQiJ zBQz8iJ>Iz}T3yff_tlKoJ-t9J<)$v=l`~`@4UY9f z3bL7Zq4~y|o%kJAD(gTyMlnPR_bySA35(j`B|yzsX>mUGE5wHey$M1?lCJe%(Q9YmGW{&M6RCf~5BYIfMt(Zw#o zwwEwqB{=%~Ufa+&CF0C(H@pSfaUWybcjj&8VZJcjsR7UqVm5pV{#ktPw4| z27QlcEYDA%6$^dTFN}G}Tj)8Dj8SwMR&<}XLLAsEL$R=|5p5;KrDqc~Gx^4_V)HXu z3<6UQn7azG5wc!?n5JwFN{_R}VI8(u^xyt? z{NA2Ea7P})ohjAcPAR@@H7<8h^IqdIR%*kSv-EzwHMHk!-(1Q#8%r8`>pQJ;+##l5 z#K4A9!YNZXst}kqaPKhIKMb|k7jvVj8W_~;P5F9?KYt|UCGjQKKc!FS{J5oA{;Rox zS1y!0!KgD>$@en%J*o=6!$_lL8}Dv?`YO_*1Hq?Yj=&cGE z4a-}@@{4Fd0>0tW*{Rv2O*t-mR6!6?~@Bg0*tHTmb zrmiwU%vL=Qkfe?K^r-JiO?gUw^uLZjiY%^3nHSPnd{{=r@y(&5nvIw$CUaK#5%Frr z72Z^dCtVFhF|wG#h6?=koN1Cq&mEq@@bz$}ha zc{jfO?p6GjB3G%Tp=dXG)TUpMb8b5g`g@2Oc`QdBHkPBLLJbwsB6u^<&Z;(?`cdbD zM$i|=)!&5&3104PsOb*X9EU#3D}XDO)n{az6f!2kE@7?nSk%vlC~Dk3&TbXE{qxcW zPd6N=-E9RFwHySCW~Q6=T(e4Kzl=3kb3$LBLnCoZ97kjwvUImzL0RQWRh zT|rRkOe2NIGy&zJDVA(ZN#BgSmck4bbN$_A7T1h0oaM)c(B_&kmR&`BYsTS|YIsJ{ z7~TjzbZUP!!o{Y^HJqY>yVC5rLR6JzSIh~l*_kOCdwAN`WHU(MI*(ZV`A4{?S7@)W z)89eRr!Gi#i2RASbu2pN*fjzj^Q-?4WoH={XSk$k++71S?i$?PAy}~B8rI&wj!0m8z8CQx zih~&mAPw@eQt5^CS=H|nP)x^BV z0lRnlk$Y>3K!dZRBoYdvlGpyh0&GH3ttxTHN-jY z`jcA~dGeA`vToJ9_gj1JT9$xyEuu^_O4(|;ATG97(}-XY$&Wd|BxwR>tpsZzU$88j zXa(=hs%^g<3?m5*4pa2!)I>6g$+?0Lb zdR7LfMsy3ogK^$9c6$WW_FsQ9t^}a)ooll$f~J3asUq1~Q|CvEG4+JEd+a2ebPy1&^!hz@|FywoNdTD*$>80PCEUVj zE4(Y-(C%$>EyM}>Irrp`N_D)raf3Y9$F6fiNkY&8Clqs|s%zPHka?!?epeF?{vO39 zeL{<_W@fM*W!3U7Dn(Upt7^>WvY4BI?Ql5aa?UoBN+GTTWD+8T%b%uX4Skc~Yy&ir zQLb@IvyEDPEwGW)eS$Lk!mXVf>>BzViZPJpsGkOME`c_J7w>|zh7zvcke}5{^viW3 z(7JQnQnewVBUaAs5BYCJA(qPhc2pcMoIIamXLBbDMI*u@9L_-oKh}RVccFv7C+`v} zQxfmyr1@>mD1l@9aQ-7@_$Th;zb{FNo7n$%8|nrRijyIY`Y&M9!Y|jf^RGTN5=(^Z zYcwVE1;MBu?{)zirE7!f(AnRc1P^9&UQ#RbbRGuy6eX|0o17H|?pUM?eR-r1}kzu#g*t+bF%S3&tPSHKRB{pF7?6 zw8XG=D0BFynAyNVG%;;g@BvQCu162pzjk_pxOhEiT_4^Qh|T(c|y>$LwM-`94= z0{7zHlcD%}S2N+pO14|jg=ch_!Q7AYFM&Ej)xE#+LPGvGF* zIgd0Xp42lM5KcqxZqYA+M0fxW!UyF{9y{C#(Y?`5JD#EYTo~H-*X*+pG2wVuSA)nO{g(lL@ADiDo z3F0?)?@q-OH|JU%ajH6*5jS&BZA&QdPEWG7dQVl1c?8pVV2ciG?y=KMFQ*crk{OA5X6bTPZvn>6C~uOAW^91!C7bPGHlE}GElZB%NKSzWNOW>U}luvButHH^!`W~oCdUwv9>0( z391Cf2SAu&ct&Mm)iY;6$kyZOD?dkL)r8}Tl&`ButerLm^`j(voA5HxF~zx9D;Ft^ zgCD36(9Y8LJ%k+N;9&o#$g`OTJS^gKe}(~vj}VT5CFM33+OR(L_8>*UCs_8;EJ*6Y zp#!L8PETHpf|o7~a6fOMQ1YFnKokf1(%tG@j(@ZJ{g<)de|!??i^-^{?u-OKfAkg4 zsceu44i0&$C~G2a^Zvty+>0q~5?=!^o?xHAkMcx_dWCY;Likm(y=z($nNdt`I|zQHNw*+#n40*Z!K`XpJAS7fUabK8l}6V>z3Z-G!b!kw+B)Uq zE!!B6s6ePB^93dfh3B#93RFS6{0RkN9RaN@+3*gsEfrrC7@@-*TNg?!N!RDWek_tI z>#9k^*Yu1{%>Tl6zcd2 z04d^^4L9Q@z1^Sy@*{MWfJ9@cWtS~UsqgJnrDxIyhyFz)fcGVX?w8_U2RpvMn#pFp zk)d~WMEuL)i+}}P5OM~2YoU^dolW0-b+`+n9&7DA0h<^|Cit$(bPR6C6XZN}IpFfP z35omRL(ik}&S>^cYvbD#PWnZN2L z+u4>hP_~q(LMrL;Fo~PbYdHBt+su+oG;#8P`h~Lw+qi~q?8E#6c4EvTnW9ZkKNmF> z^5vB}`!KH4l8@3oc(xMF5pr%MVVU6#ijkYkIV!~E;GSMyK#^_#m1Hn3MTQ0)m)i)p zwynI=8FL2t#*tb7|JLn)cOL_;DB&j3Kl^Zxv(K)>sIMgR!Oq*xdnDVOT&mN`VqdTA z$+)Pe2G87sR_1M_*t;XN!~99aL*a6}E=m@_tm;0ZIKm{I16w@`=~a@LRRhI!a&8RDhDUMmE zhcC%Hh!Rnd7K(;d#E2Pv+&@k$3WsGd5b~Xl0xud`O#T%_(yb5^IsNqLO*(L?3abLmMf>A zl_;C?fOfw3+{k{5rgqYVS{S_H$xB+SA_?*aQWj1Y-Hpy1ZDZYU(&Q2v5;NKMz17jo zSI9yy45jLAH@9A2S?a^R$ihwqZ0uo_pd9K29_x=+T){O^c)!NTDUO16hL#`sVT?TX zHe`(E)4sv9Gy#P+Dqvn_=Jg_YvOAf~fCk+DTy0sR7Z;m-rhl5C zea2@A0J~hX{=w-lt;(BI9EMxYfR-{8s?dHdSyXU@&|^~IieI>yzR&5S^y3IE*zu(k z6B~BlL(AdvJH7+=V584_e9h`KIRAU;F~p9_H^wY0Rd2-`TH`-ZHD0G=i&#}`XqvNZ zeMF80-PR~0_8$Xk+iC&K4EC$!L>vnDh$iIDOWBt9bO-n&_93xhTwI4sh(9J`72YaU zwrj_iUC0(9@&i0XL9++9nj$RLyYu9-me#Kqgq3Eh9lV@jhZ?euC7Pd;W`0 z9nVK@oaku!5ft^NierNb#Nkwf2Tx@flN#EFs?`}q4j%#Ae4$&t=i?g;HH?o=aK6i5 z=6j}Zy8kVK50uE6eaK)NTU9yClG&&-n#6pSt-MR}K#e%TQ6`#)k^&yweGluaE*n4D zh$`dM&2s^*gOIH-CGl5p6QyBT&Htc+{&&Ig6A^?@Oq{-2yBLv4oHqc!<{VP$;k_YY z{DEstcDszROvf_o`DMCw8Rukgh*No)alKB2eJa$=#A7=~y(gzqMED~(t*{DEn*;5D zOSDHsxuCJ}4t*9zPdgnUK|%2OEbbnxY^|8k_xsKoi-fT|hc$(x!b+6*uEze_trqEx z5@_K3ATG-`F)f24Opu_uUK$6-TBcB;bQlp8YZEu8MnpsU9Zlp{BB`B@EDEFMlt{12 zn($Q4Rmso;Kn_P{6+D`Wg~kdK;jF~JZ{Z48^Zn4W?YGmiw?S)J z%rqdSa7Wos_G8b5j8s#( z-af>OOz@sr-+UIr)mXth8}2L0vMuqU$EX{rriUVDN}KUr#S4pQZEd;b&I4`HAvWQu z`Fo#68tPP5)8Hu4=s!Oq7Uhw2)Bh$C{FZc5Tq3`G%REvIjQb`G^gguTvL8d)heq4; zsX=EOXMuYpz!}5bHq|$B{0dd7E#TGT0A2RS+CJrj1;AK=9mjLg$yusejzWAdu^VB| ziZ<2ru694y+rM0DFa_rk*RdTbFZV4^WtGEQvDO%V7;5~BZBns)ioHbIFDhxr0rt!4 z?6J)YkUxaoM(VNDDhj=$uvqRr4m1+M-K^!#|Jpng13PESstC^7#7^ZaT(|#m-vDUT z0hfxps3vW5kvl7fHxOUjS!x`1l-r)?ZRs6EC}>HNz%sK|3Do`IC` zJbhKHohet->p}rrq%N5wKGCm|Gf0wbY${(H!!#xL-u;7bD3}9b7gu)+m^&@2Ac(|y z-)@vFKL&bjB7hoX`BRbPpnP3ksd~0Yn|5)V3wswU+B*P$z*(XdkinTSSg#9=MG?pF zfgUH0KSx1YME%dpYc^a*P%M6Ds>)+#tepYyu>W3B`S+Ucf3cGSIzxoB$#=>jk*-cU zUpUB|hDZmJ&gsV1U4T@FX7x4k5`D3X$XiE|8mKM}0Zc_`av+~h8hRjTqZo$GXN;S} z#v|VGpn(v z&lF(M2gd_qOetbFJs@J1EH$YlfbzGL4DOLX2W}YKiz_l9F(wg@*BWFg93}{$`H{XIycwHCung)@;0X97F0^6cxGRU(|KJj)=zIV(I<;~lP zD{rLyeq$O=^UNR`a5A@f)SDD#qvc3cft=%G*s(p9L-6z%McOVmLCusyAqd*}fIPa1 zRWe_9Nl?uDRzf~;-|!T~DJ0$9N~`AHZ#Y10VsGT{6q~uZDBCoWcluO>wC$mz>D=xm zg_%4!jUU5cpOor(R+&}XQt;?0(5kv2)*nmEPv;z(Zn3|%H>o+O<)U~uyTobd_7TD! z#&S)M9)8!o8bWi$QmM+#twZzLvn|>QLH4@!Fyh6NzgOvekioF0abZX9aHZPxbn-Yf zQ8SP88f7Z3tFj?H;|g?-{g^d<=w2}*y191r6PD74U(tXeux0tnOf?_40hMr+A@gnq z1y|t3M=BOEUg3>Y*g+x`yw;veT{oinX;0Rdkt>wTBS&hy_ukUnEi$t0e>pqNnLxi{ zk@xcGF$JGO6ukM4xyckM{n*3IlguA~Zt!rBDQFNyfgo&C9XRvlIQ#{wZ}4nQcxdoV z%tGM{eiGryfFJLqAd8d#@*+f%7I7;EUZVLjx$ZBV5x-VX5;#YoKKm%YRB!VqRN4UF zf>d8K@>n)>uKu+J*%2N`5ymXy;bxrwlu#V0GItT1POf0IR#zzePr!B|Z%uKlw|tOM zzOqS@Y;1x5NhQZ05AcLa)(o-u$xBPNoQ9@-SK(-TL&*&iNwA2jv!SskTVq#rKSlw& zsOP)_xO7t{$L*I?AI!`zfD}Xdl>-r7x@rXPSGgzZmE7}H=04od&up-UO}Az| zG>2^*y+mwMC3x&7rc+b!kCv`9O}CmD8`6$9lzx~1sA8!3{^e%ZPy8;duiqbjP|Go$ z3pm!tI361d6E8nc5y1X_o&Bu+s%}X43%W^=bpJm4JMUO5Q zb=9`z&Rd7a+x5(x=T_B*mgzWjfavo#-aPQ7Kew{B-y4`_p*_{5iB$@zs{T0p9)mDs}_;ZUP6Uv2BZ|F6oDP;-mvhfO^swOuvxYJc;DzC$Lf z1uR;eoOdGs&VsmYinGMKVDAnItID2LQ&wO5%blfv<#-N)4-f>92<)e&;##x^s=58J zCTNY!m2G@MPjqnpN(Q&$GY~BDnt(nTa@1&D_bx3%2IBwux!#x~Pmwg=gM3k- z9BIT`3Z{T~R#DohOa*SocPoa*QPIy^axe0zQA*P*LY8dbq%tfeB>aYijLa32*w(4B zEmC4ddMmCt2m{3!N4t{wktDf^Jr(;y<*DT=B`;aQaFlJ|;%)!}TssD~_qSM0~h zs#xiSax3O2NBO`w{L1w3vTdV3mv8SK$-bUTf8s$UwpY$na0>R!jFT4Y*eo?JgTcG? zOwY@zdAHqEZB>YyaPzg2@64;EndQ}xqu|);*|Mi{Sl6Q8ae&Qk`R1ER_wt3!tM{?Z zfBCpd2Y8t*<&P#$NkcP@(5|kBS#q*ehegf?0S$d(LbB|nr!fScR=f`orwgg&T9U^YY{)Me?*TMODp}B9@uMW483GfG7 zd0qEbZLY@p;F*e?kwrS_FsNm>xg?Kvr$zp94sXea%j9*#okTMv7YnQ(`=)LeBXip7v}9^qJqCO zEII(a41b_CwGwxPTb(ejyxQ85JiWD58j#^%gprcDxr`98dJRd^@-r-yLL$wk{Iv6n zJBX-^+g^oZ7`K;G?XAnIJ(Eu(bi>CUF57b*yW|U}*<#$Y_K_s&SqHa}g2JGGwe(0& zkMoB0f#){l2J)s^GD2{|9gzIq40WVgv?v#Bo>k_Ux7hVsQx%z^P4SCRP#5a7lHlL> zs6lvy_df}KG|y;T(jKA^?g7s&eL_p*I}xWrJ7lGvxO6ZoS?LO|N7&eDX>aR|*PY|f zQHJa760+KNI;F5ZtgXD=}N0PRJayy6OsvH2+A;VsB5l~)m34aLl$mRS&m?jAl9{v*!IcKdR zU#caBug?x%O96*X%QO9wu)d7nL4gp*8px7yeiWo#k@xUMU9_+1UBb}7T>s?az5Qk@i~!f(Ts0MGBN^h+)N z8NxRUqzU*o^b_|>-=()+{ZUJG?tFvq*pfLWanGgKdg&}4MU_m_6HmV1kkG&&7lCoE z14sA!%MlXSKKK)sL5q#JXLa$H#Z3IsMT|V@%C%J2PAX-}G*!E|_gloB>>6x+<9z~+ z?z0lAS94uz+T7zp+hX;{yItBg6Iwpu*hI29>q?HC#{nQ-b@_AroWaIKTh$cZWL!V+`DbwfT25t5>`u+CG)r;6kjPT@sZOg@R%kAxP@e$LR(e@3+^r zougju5PG7_nTs!bjlwqyX6BFAd(l>OcDw`(B%?z+zU-^O1x>yezc#G9m7Yd8OZW<}Dud0c0ycm6z zJoWNCJ}tU@8{M-bmd8tSTS8T{RF|A+kFB*|^KA#Jm+I8I6?3fjJ+;T7xmM4`lB~*@ z8j^Xw$B(_gHdOM(1w!J1>g`|WVI*D)_w|eO=RWi8YQjsBko$dK;*sLx1+86?jqx77 zO2XS+T+F$w!G$_ToCJHJZOH%5YWx$`8nVd^Qyh@Hlsh(|t+wHBf9GZ(mQVQo(wZH8RAvBI?IqlP&Lrkv7~d{z4kxFf`LQI%YPx`l(9Se4ooY&y14Qwl zL**cntgh`eW++iHcPT6tm&Cq_^Yx`9MME3Aq&e;D%goB*%07lf#FVTH2^p6<;m=ts z?$!yi?R=fN8IPnd?HTYLYn3ZjvQ3&o8R^Nu>|%1LRFjq75yKuO0x5#UZ-_1DkDMHD zT2w##48cM!4Fne`g$XjiC=z!MX+r0=~D(&fK_C`1hzKrL=E$D=SgwXy3Zy4 zsQ-3deX;L^_5-9R`c1DcB?&X=sa9Yv?Uw?^;Uk*|=+i9zB*hn+dn>Y3csZ_fhFr#y zhRJzFrGi-s-?)QZ*3*fnXMr~PgBNyhR1Y5`pY(_)c{M$Z6Q#LE+b_hM8A?v>ZH}VC z_?@4q0EE=B`aPu-aC%(OFx&a;#a1a!}S> z5$+xjlF{=g`7gp?jg1|b*%n(r)APE^#o9M7T`!TVXJ2a=687&1?@@H)R&q}iBh1X8 z+W3xB>rxB%xGinfIVYiXJh$%}SWr5G-N&z1zfH@g)o!R}uSRYd6f%aMe;Z$oc3!iO zB{Am0L;1m`*uKy(JmscuFOTZBRj`G9qzQA{3vcoo)TFIDDzGa{52Eom<9*HeR;woW z;#&85{xZnOv%`LRf$In%u1fYgoGF$23Mp(7Kd%kXY0K;b(Py zm+}UpNvO`I=QZ!%xh3G3w_(OWhoWMJSbgy=XT%!x&~>W-eTx!OA5V%>`4^^De;@?b zw`gbSQEB+qQ<6A~2cB#*$+ju0Az#+;h=Ys;oo#}n{8n?-vFpj zE8P|~!fi+%LOv9VqFtI=j9%ub>be%aZz2o5-g!n_YM^&;l>!TBuguBCrIH{q^<3!3 zw#u5NVnJp|st4f~;(oJ@;Op>9tLg1ohwCtpfc{vgpr^}WJkv&Og$<{iF=V$&EJP`= z#G#_&sD5-PN$NQg>r(a%T=hHD;?(P2-aj#T9iX`w^n;JZ_AG_W;h0t;*#rVHdrRa{ zx9-z_zBi^cixb{>1*=G#Sp_Adn!Lch?N10p$!~Gwb_%KN(Ncpd6f>SjX`L$0NvrUb z)vLFdL40{qvaTl3H*+bApcNPE>k<6GpgxGhCmB!}Np<@_6rgAO`omSoh=m zDvaX5S3Xj|JK8TL$3aXtUHW>*bt*_II>=6j<+ zx9VsO_t@TnG$t_iseL}Cv^g7_NnjcOOE*|kBA;=o8j|I6a>$6#+2(_-#jR7h0UY>s zDk66ei)%9swb1D5^(G;!dnl-oYU23$l34HTI+O1-jri$gl+CA9m*su;xl=*yd6Cr2 zK)0_4m%?Wn9jsvD+9=1e_+{k@dke4qUXYycT zUD3tEX~B`WI%Kt3UIW=clZVWi>3VK(+fw_8yG5eu13UPX+!k0V(a4Ed!?-$XIC1c8 z7@S)+dIr}tDTLFBr5)LH4LAx!YbRA!TQnsvlSt{-7Kl5OZS*Q)^wf;sl!xYGn_p}d zYv-9=Pf$C9+3T4maNdCnbFl$j(ZEiNA(5ZmRT}p5oN6zupZQOt=E#4ti&P++=5U0+yBTd*dK_*?%B=u-oZ6>|zpip>FY(&h2*JFTy6 z1>h~zpAX7}M`kvdIAB$RqWSH4Z}F_aF~@l33#qEJQ-UvVt8 zENN)f6&=3y(h#dcusIpU)_cj)@UkWQ#>2LTl@OepKQ$$2q3e3LYjOs<__Nd}LN(@8 z?ZiGpHJi^Tdd$x|pBR&UPuj08PWR*0vxI(L`f&#OZYDGcwq5!&=+)fYl&`LAnW(!{ zK&aqmiSW&>R;Mv7*JQn2eZvlRkf;&coz$&f9^k`xY`~}g>MaOROp+!uBBTiTw~Xl z^RDCz_<`~EJB4{d*96w+43|aej)2rT)}`Eg}L9@jKGsZ3$Rb!5}n`h(d5O+VDzqV}$*70LW-xe1+r$J^N2sFGGiENY;tM@Ewwbdiq&)#uGO`H*?%$!4ZpN}YAf7_EymFJ z(&ko~PWRLbic!*=0Vb%C$XA;MXBPMsv}ih8=QSv7A_X`R4|%q=EnF6j)G0jhp+Z=i z3`<4=GYx%nUextqAIyKJuxi8V#(r=sIi70}k->tZqJco1(-Wh*T#Am;Z$BbS^Lm=C zg2QIwlZv++ljFxd@jetGW)*zm4G(`+B1ipI#K4_g$uhwV<(Stoert z8I9wjF(7ZnL#AsFbCdGF4zqYr@Gq#I=&MDvW3CfJ_{h7DDR_G!(Cu_5jSrG$dK>66 ze*#cgr>4M9oot^LGXFievT43avE8?5cGNp)HBTb&`~}d6Tf_KSnf8*KV~kXRf7f>F+d_E`TD}*T>G`; zUs;%CqKu$6_Ao*J%H5t;p>5SM6t5^afq2?DxQQpt zu!(%;1paboKYwLRET@Pg>RkO)D^qM|261Mnv7KTu9y0C9@~#b}_`W>>)=tAO%7P<< zKT>h+DjFUFUQ43s2(8K#W@MaU(hU{d2)j>>MV3eAHa|f~5H5MMcaX_qg5z%1-Vt;N zh29@rMa?U#l{O2ayq6zTx456GBg5mDeqw&(bQ+Va!S6^<-StWdv(*yvNJ5CmL8lnz zsCMTuk=HihfF$$QHcrNNgR$?7IrG)!*(!1AMZs3Gx7a0wx>BhbJdckQjQ&9sr# zK}#`*OOd^|_vE{OTA!M_uD%WcM;I-yZjDxWJ-BD2CNd*kfyK#mV${8v1v)9qT{Krh z72Mx{@RzOnR;i#vwiO-l+DhplPCWEyLM3nxnBs30>pH3+u%gEXL249(0U6jHEK0J3mp~{YD8SP3)c%74~lfn8=Xl6e_ zBkv`F)pC!!$z_~hBP9K92yi~NBNA)5KfYUuS$N&etLB#`A><>R+Msakk-x{xV3wOk zNij`BfpwHcG+j~ZpATXADwkE5UnSM_%$dPF@ie0wv4U!bFpVa3ohX#QSpJbC(WiH> zierz>UlXlOsr`OHi0-N@JliSD^i`TD-D14fGz(?B4VqyC3-|=T!r!*fP0FqFkG4|W znprmic7HUp-@YBUN@fBQjPkzwTPm@um1v=%_w*0XO#7G<_t^$(=IcM$HNXGh&A+Ki z&7$+E%v@Gs{IvHWCijr4^51JF;+;S}s&2peouu3R#}eV_?Ro8~DI(OWLVX*M!hjIO z01YZQN(hcSffBem|M`_&1t;FWGkVFYqZ{dtk?p?0g+oiy056ySRZfS|Xe{D%HmX2qrL8(0?fTB=FnneD` zsg7L(@sE_E89I>zNEU>MA7c$ts)9#3Rp|jY$P!i`!o*r6J~5hjnRx)Io-Hcd;10k=+qNh=yVPYwRV^!^%1(Aj&5g+qq#Eo8YUUp7x~N#hsL1 zp8~A>Y++fMa`Ap9jJ>UhxmacyB+<*~ZlK{JjLIne&=cuN!(~I)b;q0pEdgOKt0F<< zWiFT+4Yh44t@N!V6pX`S58hI@8xL}v_$NCRR1u&fGQUml_gYVn>lrbg4Ea*>}iG=E~<~# zy4|@@H!@SQe!q^YjBSH9bhi5&VWtFJaeh^?&ujO zTHpb+ho7*FBKD)^{!5`Sfftp zjB$i|z-H^oj;FCw6pK5hGz-mM+DeX{Si|;t3^Ak_s}OMgCV#o|;Zhj}o5bBYp8c|V z#K7w4;JReJFVl4;XC>2d5vvXZLOo|=(WZ+-RnPFPq42*Poe30KJcKT3fT|6J23Xillnq60 z8h1^`9_;suJIU>>VoV$Mdz;a9Nwo)wc#3wBlm^M6h(;wsp#~&*o zzc?sloTj-!JD{!;W6W`9k0j7zNxFX+=iGmW6j4c8@y21o9Rhg3dn|9$!_M z+N6~hElpT`jFb4~E>mOg-p%s|1*f5qPZQd$GI(bq+L`A-1*K=Hw>bt+%_$uRA3t-?_=?dAeILLdc`1k?0UXEw!ZG?k58NmLD5?S- zmZ4KdNV+m^lN!05sk!Cd&uEe$go`3t4&9(3yS+nZtag{>l5D=9;fIP~fs_C&xGlko z^87_*fq55U&uXAhQ>NKMkp1?imyB(UHkG?%S7wthEJXh^@Iu?my|^%?Y8f>CnLZ{T znVXoq@@$BxOz;5D3Rikw1Ydgpp?fQVEwBQ+`9spJS1SxfYfH1tk>sgsPz7;w!^S7^ zD_e9=Sm=uIGlhu{&RRL&w=mixT9n>7DkBNSe%AHcSCm&;aL@W2_11uYeMy_LQ6~bS zZmF~%1+iS}QDc#U#1qssnCbjMRVK8w57!5As==^XUOn-N(OD9Z-q$u*wi@I39(IjZ%6vhibEY ztp}j5s;@_xP~NFhIX9`OsL|5zJX5CxNv~+QC~?WxR0N|;{YV!Xn^m2V`^{LeM9!x+ z*;>0=CbSb&fau~P4_K!Cz9xlk5dSKB2y5b0rD=F73p!Vpox>_LSns<{OTDO&S(d>| zTnRx})>4-2G?S}Xt8#;omK!r%@)E+VwJPaY%$Gz}{_HJ#(f_0nM1q3-Xs-z+L8IJE zwLZ(%Ng^(G+7*O7J?4Z{CFsJ}y-JbN_eiGk&6r!Ko@>6W=qIN=)3!=&l9PzauE^@I zxR|0e2Dnp~iVf6WZ_v4?!5*|_edOR2H!}>{rN|jD3bZ1f!-QxXWJnpHR6Wmtv1yQy zp_$`~)P062|B^Wk^G){z%1|q5lOjL)Z0tqY@FdEgtQx?!H6J{-Nh_zy-E!bXA0+DuFi6S6JEm1Hqxm*=-ul$Pfrl%tRdTll9>KhXW>RwweI2%m192bkgd1FL zLk>M6_w?>NuhrOGO`X_U1tv`zycqn6MxZX!Pn?__eeKr1Ip~>_cp?Qz_0)d@|EXGdZse%P^4*u8ps(F}gK)3n$}Ifp6n@x#R-ILViX5P|7F}M~ zKw2W}!S=nTTi8;`# zJ6g#rl$Ge}C|x465L^5hEptkvuX02>o^~5TbXOK~+$xKmu6B@vfpW`hLqbALVRq?m zb``te`c0%H)=WNA5jpvJ_z=yXC4X$EB5=d+C4|fat>2mEWnWHzlF_8cCDZ2*CYFOIUhqP$cK%jj1G0Va`>s_?0_g8GmOl>u1<{LT+#sjD%$l#jFS~6 z1X{wCrmB&(N1u<>VAbk6va%_R$&#Chgq6#aZ@Dx4R6mxjv)R17& zD|P|#X%+>o2mNa7Gt!Nv5Q`_xQA#%-akq>V&z3#H(Bq?%I{ofj++1a)OZfl3I8)#l z+Ad(U9M|4UDN48^;UV&o1M1rkZ*sFc6<&{0`9+5w@=u|L-2)|alHKxq73|;E5S_)| zT$86C9YN`E-SB$n3UFpV*>0OPJdAj^G~xzQ9>fA|`9C;`&+@MZe2-2Ez6F8y#XL8} z8ltbH+(y zBd1zN1xrF+st)3o4KqwH(*Qvj36Y$(813}(bfIcDHqYuN`$l!<+nZpaFznU1($S;U z#|7s?@4@ZpurLJtFJ{X=9RjM2P93reUkmRT)UAnW#q^alVxz)uDK}zOi~Fpbx~gM` zw@(d)s?Bk6DlFfbi#N4&$12-0=M^BTNnPRK4Nk{gq2Bfg+F9x2JbnUZ!N&AhPv!rD8UvHZrokKpjP2rIbt~BB!Mv+LCYprR0Af zs3dSuM`_Q8($V`3FN+`hp+EOE4G}yGOx%_Rc<1?)hcH4TJW4foSX-fvZ--m(G_tb= z-mm4i>2W14i0ENm+4u+3Dg(*XCsHBE@wXSjlZmZF)wA^KX$Xihe^ z>4_34s+OuM1&)j#xGz=CgD1F7i?pU;L`R854t_jBtD) z?W*h%+dy_LTC!-Yt`1%QvAy81CRDm&jj7PR`tpW`H*5qU!n@yh2|{^VXhA|b?@6f& zBGkwWtO#=5OaU8wq9ID}((PP%3dMwSomWBGf6+rB4KRR0+wy?r8MP9xT%Yk}?hCp{ z7!8nc_Rgcn$u+8Q4aI!l=;@q_1sk?P+Y=+|tq#9GgM{2KDqv^k;EV zW^8OsRbO8*I=cOni6SaC78acF|NTe&*N?iJB@C=<5(ZB#;=)}TJ>VZ};7cLGNH%c6 z(d1YWP0_K4MEET#t8)sXWPE>r3i(g5Hi#ICSBqdqL4N`H_gI?M@0A3fDd7^kk`+b9 zm@AU`;@z=}PfvDsWrdPe^Q$~RO{Rr^+hYt3W$((?+m-Q(GoCyk=CyI6T_v(z(Q>%8 zqHYo@Kt$^lCl3}$uR|s)QL}J_PF+XpxnOFUU zhCM{9QxIxI7$9MMOM0tNfJsiCPebWN7h9)9jeHE}=>3?*RmlnHQaK9`U+GbR$7fuX z9*SuV32F}nns|P(YA^w^A zjSVFYa=7waGv6vJ=|A@A|2zH(0EZGMmiyjYBAioKC=v|`6&SA+NQ|ABBod4q&|Kgg z={&8}87pjxj`<&>YoJ~Q=Grg?1FOXaNvfS#4ZA#yLU+*+>h=(KIY|Y#Jp_5VW&vm^ zsw%BxlCJY6OZerb-s{U{oeF(5PaEn~^ThQ7h1VN!QFmo1)_arghh6qqB@1lmXsyLI-0J5)#svN0)shBqZGS zXl3R4C;qSSQT{YLiW`YSX+>E;9I4z98HPq2Kk2L|d9A@pHSzf<_5*oRQ~!TTSqz*> z(69-m!Vg$2g47OC%N+tnQ7B)W)y+Z;=Q(Lg`;4|# zXz~pjl^xH|%s=nkpT(#jZLwyK-4+nSX%Wz+%$$E-m Da9hhtX>ioB1v&;WOsGnzuEg=Tx)-=R)3CAqoJVRM&#``wGV90ze$5pWvq{q}9qEz;azslzZ z&oCqtV`G-b?D=Izp(62Rd$^7sH@SXQym=C9?gyJMW<8WAnjB?!AEY3cjP_vNl0S}y6H z<~1GR(H4dYYSvClb*!uS z!xebh@JM{B;4waP2rpG_u?teJ3>EVi67>a6AQw!*!RCJUk<6MI*+CFn@o4T}$&vj; z35H+3;bpY+y29xLHsNpX{v}tc?q@-J(5RyaldXwpk4EK3SD4A~D$OZ*+Zfzh`H4H+ zA?OR<3!U5jAF?8lbkS0j;0g)~XheOzyq<&8n&i~~`&8Uui-aWjCETgXr073g6U>t6 z{-W-p5mJVpQ3@g<0zGJ8E%+3kG>t+2wis8-_;ify2~*UxyDs!SaLzTn9gSbrxd9a{ z^ZStaohMaKP%|o}r;RC3$=G%q=8p^o^oRdJ*;hwJ{lEFr-H3p6NH+pQ4hTqhmo!L& zbazS$D2J$vr={?9+mnR(vNE1#kEZzVgb^Rm?p z^4m*>_9HAzf+>v<(6ifVw6sf4a>~?&y`FqKlE7^kAKDV29Fmw*pZN?91E<)Z^(;)E z+o&IHQnkf-9$G@9j#ZQBgI_B)%MQ+?@{C8!P@b2Kc&nSb;9Ckdg73||5la!#N4rzJ zsd~tiqY=%>DNfwpy3Sz>;#I8--;~i{4VoOSd8TFo)`LNOuUDGRjST8*3*e<_7RcZE zN{X_zwWV$E?cJhO!}yPQLra>_0yv-OH590x`5yxs5+Ny+>L4WUi7%BY!p4;N$>dTv zEUqyFs7>Yw1Wqw)0A{C=ERp>2+e2|dq?44R+TWA8wdH6;1_u5B=)QGPevcAr887+X z1`$y7)X~|0gva3ZfvUbYD^p4RK(kxdZqy@3egd8+BMp{@7W}e(qgYA=a+I z;4?z<@|a876b9VeS@f1N2bBOm;_l20e4-A89y$2UHjWNjgnROR0Y*}NC`)3857Sk< zN$r=6^|5uWc#{0CJZh#sZ};`LnxLh58xis!b%mYTcAi1Dd5bpw?zlQf>xUc`?#hzB zLWTbJkLA0$Jfjqkiz zFnNH85%1}Lf4*N=$#q{rKZ`1we!lARyH{nRV(T5V_aI-E@78@-HP=oe9nxI@ev?ua zW^(JmAQuv^n?luHM=INOS2YtN(rw3d4Rt#)mc=%@;KOI=y(+=WPFb7W09I=?BLIau z-XzfjmBIsufi4nKAI5JZbXo%@j2hts-4#Awn|*Gw!$S2+$`d7k9S7{NMnpz|!B_%i zeJ$~3+0e8=dk(O)mbbKPsn7~ro_`^pv_EZ2E)T1n9{sC6%H9Et+iOp`Yl4^q9)hFg ztcuyv$3Jg%4<|}S!-VG=(1u~akSXtSOrp;%!kdBnx3SGE5JoGC4IeJ~`L~q;T6h9V z?ZpH2qMf7>Blx1#Ac97&1R$i1&Ln-z9s(Ax>WfDM;u2$F3N##uG9xo9uPazrZAilV zBboFKV+&BwwiHiAe1)!Uaxuf zr#h0nZ>O)=GQ7WWI@JvimL50DSF+4c{1Bzv#*bP!`{gdW-WzYThnim!!9#SgpW$I( z@U3^>Z>?O9kRQAHMj!t$=qg`X@lp!jW0Hg3Z`YB=1YCBpJ|xp*CtI*n6X!dAKqn?occ6<2&0Z+6QX_mFM39ltOrJ8z zi((ts9K~jDt`k7eW_0jtmL~;0@}ZZ*MpA>u`pXaM3HvM6>53b9q1@(;mCq)vA(DTN z-_?|i&oSA%ls7%QT7FU4`lfry_jL=V^<78iuk>#cl;|Qx6r8WYBd>I+%}RUUDUM%) zh-N64zG|PPkWyt2kK7wX2-4FRE&i<2OSww1&AO!;#3OS?8weDd7R90E7Lg0i^`R1=;DMeQYZ3;$J8Fpdeu1`4&-9#&{HZdw3)lTp?eQF%bs~4Duj2_vOe$Z76>?;EQr~;p{cI z*wb#AbDnaNK)TQ`kz}VdH`sADxAcw48 z=2FgH+B|44fSWmLag~J19re~?OWaR+ahZX)nobjqK zjvDrK=hvQI!Hh;kLadgej6pz*!i0r{E+nDi^f_3CwH7VwVr`xVS6k*>ju#1VK&8F^ z%8du=%V;eQu`cdeko@8^8yT0JGRH4*OR2O<^hh4=- zOPn8Y)iN`rm<9}oD1sPE9siEhOAk+V>01ehZH2wms=97ToXRl=WI736&UwFMIg#Om zcf`fMWsSDVdDU^!Dcud#8j-1dO!fAiwxyUL1jesW4P^FJ2>f}JV4X158iOQHbjgm2 z9;0zYI(?8=sO)ewlDP7`94IS)Iv3BR8kc1FoL2xXNNzF_b2hqJ^RCz;PDxpZtZ61) zL(U;wO`Oq;Lw)OH&z=E<=E1i|bEWK1BWxjz^<@>Xc=L-EyQ`QkhYkaMh&21p3Mw=0o^Z_y5;TlOZx zT2mXutO%18BvpTM8H1jiU65Pq8S(2-EgYsEe@gzypD=Y1r>zSMQG7|iO2s+)^^D*= z(+;9Ls~FB#!e;}KJacO#)0Uyg>vK( zm}Q3HEO-tM793?Bt&H$$;hpNIu$fSP7ei+{)6$PR25_;& zYa+BYZsjSV!QaDdcbH<_Aat`1SFBT8axrS-r{DN_miG83btDN1Y6UfjO-@E=a+>a6 zHGwcky0N<}wkW>}^C8aH-9dQsoIfSv2Fhkq{JL@A!h4WwbCS<{8;SV@Ed%(BT9b7i z6>4&biRuPpjseey_4zd&>(iDng$7+ieR_>~xz_3eO<_&5w`Z*6Qlcl(ac<8<9r_8S zEfGu6tFo(B>i4Sfa0cFBg$C_AS2GZtGc58#t0Gxn*}fN38M_8O=)Jo9P2q2}r}F3_ zpr@{^`CNA3P9$c{%^IybXVZ=E?hx*zt;w$mp$ckWt!_FD*mjeTQ_2>5K2kwoPvVFm6RkcTuVpqGt?z|a&q$7z_@{h7fK?vxZ?W5u26YVQ8?H4ZKAfVIQBzS zMy7uzxtdFvIr8N6^Xb@GRnZ{~^xPHs&(;CzN*T?}P#-1m4U|fh|HWx>c&+a5&KbP_ zjMs|8j5Lw=E1z(8X=N%VlSYTldARd8i*?Zoj#KuGV8!28#a{ppDxL#LbY5H@dp;0?FG|2t{8Rrf8u}Q4@~Kkk^LLa@HOiX-UBi>GurMMei`~0Lp&gY zF^k#zO9g&Pfj;7a3&g=%&RQRGl4_8eT(zaNFXoZo4vj7)cX>SC@gTLHOjl zD(8t<5NUq!#JC{8DkvY|^L$M@WOh2n*atHy>G>gvFwa(9mq^ZRi=u`!(21)$;nULV zsggRWvi&6pPr%3NtUR-dGXLxFovZd>=bT+@UJ1{46N+Qg>V$N~mo)3`706v|SF~x; zPE=?m_i2aaGovuDJ~8_tqzD&p-=xay#KuPtn54y&WxU+et~7(jIWQIzG?@!P8NYg% znxA***ZJJL@-04B(4$Sleh8V$n|f3K3+5H4fc2bMT|2_?6OUMXj{#XCx+`1A^y@c| z%9X4vFiT1@1__nJIs}RlF6Vv#{!>W6dPpH!2i)`R_b~$Mn9A=^+6-^*=a=7@wu6$^ z8W__p3NzDB>VByyme%A2-R%*lU#^ceDawLHdb=28-Uk%ZUl!RL$GcD@G9@`Q=65H- z1I33taUMmwfg72B188jbmNfTdf6om+#u@GbM+Cp-KWsCzJ?^OmXN`j(IP7Xj)WG)* zf-5^HcQ`3WQ}TJ+4ywQ-*)p4zv*qWck1p3@6eq~ktxJc zoN`MC+m@wI*KW_(|6C@)V`BZZJeozqw%e{2{q6c8;Fc1hc}k7#AVz1Kts`=pC(NA* z;YnRILYB^&;8P#&(&k_Ap6Cgi-dll% z;yxAhFTd5b_7hn@Q5+r$$Zv&UF*(gIF7@e;kfSHB>6v;`jEP?%H$S+_@C-2=5`=b{ ziJJN)OKPf(zrx5GxE<))-1qH|Joxh&n$F7=9u94~XX*2f~SWcUCf7P zwg=7$EUny)(zvOA;`M#*ZcZ;7Wk7P;>5iH- z8z5L?<-6;jpKsx75pzmD_MVNeevf;;wT&{UN+S_d^6L5! zb1H(_cZOqs=&IQ7Ek|*zwLsECKf7F790y!IwTZGtCV&;VCc)knOIxq=eQtA2>l6E3 zD=ajlutkNc>0at4im6MC(fZ2rSBor}dxoqPl;_S}WU1Fp@&qNsJ-o}3_@mD1a2c7W zi?fwnJ>Goobr^+48JQ)n+3H4M>ten048l$~W|E|fx9Ce5uiyD#J(<)gOSA!oPqBCF ztsX?2d3lmxzj*I}7=;ZJ8nJeH97DHfHjhW{sjNb?MfAawwIb!EF3<2jMqdJlyEC72 z=u4uaK0ZC)rWBqz9|$1K5OT>&Z?zb}qJtQ{lmACB5Q7_uI59birrqB_ygh05OX1BB zf3Q)jsQF+LWYxs!fx)BkE_5aH6x%dOe#Lucz8yEfHl^S$`@y)T}fe|b*= z1d$-oP&3@+35c9mGWuz_a10u)7|OSs84iJ3$(^WlmwP(S8=Bb=;kuYaHID&$?Rcr- zkaPAJT=iF%4s`JpP<&|5dItP@fZWe1Ot}#ACFnCItHGx@E43C|D!e(>GGE%p&{q~j zTCbgcnTiu=5ox(USkqiws&@T`jrs>Dr0rb$i1~knVVN)(Pgx(?|Xfu?zWMmik2<_h{Gb-h=ogor4~t zKTpnjcQ~?+(g*evch?bqPT8%(4!eH}^5q0#7Eju2e^`)_g-^=!gJL9;sYiM62zI!? zUt!W*d6cz@{=Bg&wq*1t(=>4D%{PGy&l^vsAI$ZB$1}3Fe~RlU z-MA&L7}iUlR(5oJgAX=oCJmHT=h3Fx-%`jM1?~uukZVJu{eIl~o+Sz48U{`rBz>D6 zbPE3@#Uy{=gQkDhHL*Oh<$D#m9xNrSu)s%2m{a=%KTkcBIkOC73oMJex?`-ZmPGr= zHg{Fh0yiSmf~N<=y1Ox3RW*bh_Kl5*UAIGsp6UfRFrtwHHXe}QKLU^F@^I&npYl+3 zH;*yHSbg^cv=+5PHDl+cSTkXdKTDVBBTjqi2E>N;00Nn))56Q?lNvuhy6(6STc}+R zUfDMnqpg92dkY`VZTLd3KAz@YN$yYN@I|LWPp`gbuTQqk1gdFfana3UWg~*5-EBkZp-w8^7X2-UW%K8 zYH(+HNQ2!aje3IX9u^{=$&SbwbReH86eZxc#!uW z`B?wox)wmvJfr2=44MqNtF)sf0_0XR%@ol^QgE8I25zzCw7LY4@@Ai;wol`{+Cc{2 z_}PbY0949^qNn=IG&sQbT(Zr6Tco;b$)@6zl|+h(PC8*G;c#Vs`PX*0rtV24ptW+= z#?0QFmpeioDv-;bcAjzZMc4MJ*Ev*4SZSS4V}+N0kCfPvRzMJ5(?s1y`!f83;zVOq z+W#h$aGPEH1OPdjwhl!l$2Ip=mLx_2W_}<1%gS_VRtl$hY(YE-a@7Q7zX~LW$@txh zzTfe%H_xRg#KqXiUKhuUWyjeonfR(hbdh(T039G*(^Sq~JX-t!`zAad@k&N&JJZMhN?jAc>V~olV=IhnylRkT(wt3s~ zo8sxAU>27WiAT=~`u2%J0H0;Y1t^IZ$>e$(pPqAnLAm@ue(djBoZ-7QkS5?1T9G=8 zN6C`|q@{3!ynO$w#PThe2j!MxR#!}aFk>{@J-3n#hm*r;nSEXJiPJrFsZy%!E9j3KUQJ- z>zKr?rtB$tEa8NqzMkI@%H{1+c+2$S{HmpHedN_jh;ZH!fZ)$BDxjG^IAX3<9Lm}G zix^%+VO{q%fK}T~1Bd$vskZ#X@yC||!e4j2pqS7v>cu4Hdx*3NHra%0ZOXXtUQ?af zkwDt732W-9dYeaKB_(45jCh2#4`+B_?S6kqqb4swH9zN@DQisNN_xj8w0K2h2-k?$-XgOfN z3WtLH36p8-shhMz)WocY`_Q5u{cYR2e{dB&l)>pS%||3y14rB&0mgs74ok*AIC3n= z!#fNxKlUOVXg6cj@CaK)9MbB)2&SpL_HLP7KA>Yq!V`*pmKxJ`EitJlBud(2p>=-a z5>Vy0-m5AO&ohxI+|q2Fc(v~cDjZP{u7?gkc?*8h@J@nuQD*bSuG>9`au}p1ns4-k zB3}ac%cn^6jNNejsq#(CftH9vK^_T>WP^Vs+c13SiD1|k-9#~`WNRTL#AXfESy$nY z?X(f;1Duxj%U2!bznmfvSSG?cj$W1J7l}L0<^fUU5<~PU-GMIL=XA?e(IT9lawG~E zKP)cL9n1EBAyEG8_m~JqczhM6A=^ac#nhh$a~NAw048eG-g*x(d@#7fHVN4GSo>5NUt+1mK_? zTU2sly+t5|I6voSpvu>q8D)pu=-=CARYX%T8w{wUwyGi8OAJb86v-&m%$Duy;7Y=c zRsi2Fya{(;8E#y%sqTtU*i+jtn;?W?vV3$vR=7_~3P|$qJ}cANddv}U>Uz&t0noT? z&3omhlQ{yPjbIxL%cirlvu?MtLWWnC#9DTQ!~q38AN*Ny_S=^161$|=c;KQi8zyX&WJL(vQwEZs01`my$Oyw*JfM!uhzBgD zhXb+gaF5Pi>2)b}a_#6+?qGazPiS;`P!FamG*b@r5L#E|wDZ!mIc7~hyY2MVNQPMI zBOFp$_1N5fswZ;vf?MkC{H$CFh(^+rdK{nG_(9f0mg=+VrLchQk>u$exJ&MHz=fM1 zEV#Sj4lR+{G%&lGnsVLj3{Tc##d|e})+t^1qe*Krpq18|?g5WFd_6SMt%TMeODh0xw=YssO*HWI*~A5HRzP*}M09*@`@T8h zZ7q08w);_?V{G&a^eNyx$Jp=KJGU+7XklmzJBtBf-c|)mT;! z@8YrjLv$R@Bsd zeRv2M8Qty<6pqj{G%_mh$cZ#;4Kg?1{1TX*y8q|Tmx>BD`*-iYY;RkQjE(8YHN=w- zuJFjY2mQI&=naqH9nIkFuUKDOtEsRE$|wap*x6~gJ2*RkRY+yhP*qKwczBpi_xMK} zS0Mw>CdTaoA64Af$$Cu1*v^7NnQFx3?en?}rhfUahF=vZGHDpjiVs)I<^1 z?^>)POmCYagCqq}5bsKmP0V*&ljne)HHgmeiXmnR>=6G@MT?EdAhh)<-v-WP>3EbNFf#zgvUse%B2iSJvho5^_DKE!P89<29HRkV z)!h6(tbrx-J{1zR)U$$j)Y-`+)t5$WKMO33^_rJO@H@Go+&ffX~rR-U*vrjCtsEwoYL5iPy{@Tts9 zyODI-HNXWamw&aJ5)hGl!h3)myg|f6?g90eNC&VDAw&ad|02sV$EYJ-VWUBe1pZ*! z=9$ZBdzHT4ID9FBjIv40^@t>XJB?q?nEd`M9$^SXc8vZt^ztpwi4}kBr@RxL??U`Q z!OF?5JzqN2!>hp|LM>2}$67Pw(9d80(HlT4;ZP9guo5(pX_oz3buBHAZ&@ zOF)RhC34;O61m&%Vm}n&OQ*^uI`f_{5`BU~?poJ45%5S#d2YeK@z882#(5|YbE0h1 zwZP|rcM~S_WgKGNRl%oT4y&|oSLrPkxwbi;(Zr^{d!J2y-n=6oxpS=|8~tKai+Rfu z4b@N?9^%5u`jRMC1{L*okh;TR8W&3YtUnPBN5)GWrie3#qE62N1sePneXU9Pcata6 zmg_L0L!}n6301!NyEIuON*Sb-Gf!Eu^}X9HJO+{A;-s5#((_}yo1S`eu;|raEnScq z)!5;*R~7YQ{DyYoNV6LX&N%CW|4~EdL=*4@1yVX-4+%^O+kiBCCl!=|`E!WT6N@j} zX*R?U=YRm=Zw88!uMUQ>;G4!X$-Gfz0P3`o3#z_~%Rn+8ZsDFkuwP#{rty}2w%8RuR-ehY% z8Lc{2oWXK(kGNlSjL5WO3cK&089S0|*zatv&B<4Gfcm4n33~8(nakG_qMDUkJPc#e z(p*(q_x%=qe1C&uG9V&xDLb*wpK_X7+Wse#3P8-do0gQVms3i6DRR^roontB;vPd( z-Y6=L^U{HT3Znn~WERI;_FfMZvbM7N!?ApJk^dwnARsWg^4)!}dD*$){B6LQY#TtI z>O5R8yOb8mB~A6?gOxo!J#{m;wk+C?+Ya8HZ~xRbGE%}9J4NU`8>G;cKw@$4#}!h4 z`!kszVn6>aFn@Ys z*L=Uu>}ep3h4c>;XJV>QOu6BS%&r*vB8wSc^jnFzSkI;s>aZ6vTUp8YTdmD9OiB)X zK91Vo=w(9@2CD)o>PNNN(d*x`r-i%X_E9ZGORc`8(2ke12?tyiS}2YYym-kwQdrTU zWllurM(8nz!_v#o?XrVYl}3h(Mm{FQ=_du0+D#Epxo7xPLE|kciILe1>S?SHKqxm1 z+ucAVxNA6|lJ2bI^Pqmz>I#x~eoG}_VJyMbsG2uOKJjk(2 z)ygbn(%+w}k}kc+BE7)R3m9MgjZVoH!dq3+Y)p6_PUX$a4&DE5{#H2h14o!$!22lr zIR{5cj>hro{jqx*j9^-j-AadW9>J6pWn?(2mk}9j|IO*1#ZYSVo#GBb*j2bh0YL)( zdHpAzZgm`00$gM~zVW-8yy9JDcP4bIp)4WXqwm#4|2DYXDS`S86hx-0=e2b+4JxRv z31c;S)KgMf{&C#pJtLGDuP9e-l`4cA6HeV;)YyJ)n)zf95wIep;=ikxuN%=;XbmNz z4EUQy7N^Lt=0CSxdiG1`2`1DS-2YVOvjZG7;Bi=beX!VD1?7%+U~t{rbF_wb+IX&9 zKjgznO82FWs>KMt!iAN3CkbH8Dg=#exG45Gh2Uc?K(>K$6mGSyXGg7?Zh%NDZnc=u z3)}!bw&aNWK~|vJTSpS5o%UI68_x_Ee6GJ!RKH)G&;euOlj{`FOQSL2X#)?k6iWZf zmSmYXJ|W=se9CTrctTIzo2F*Z7OxR`=y{I6Q2(8Pil}#%od{0JquJO>iDacLx)JHl zM%fw6C5NyMOZyRhxdah6#1)?z8I-yq?6Q84G`S{3-LNzWJIWn8))Y165I^D9z4O70 zfV|lIVKFsIoTETQ3J3v+C>D^^S$-6+xjpltJ8G}Nr1-?I3$2WCTiz2GkIa9iN`~+r zR3+l-D5VOY(mvWpD^1N+xfZcG+48fzldmvte_buM)1OU?iX`iKigC2vu7*hPU9IUq z4Z+qV{^7~%za}nVCZ_WI=N*iJ4H5z9al$8?*>Qg$-PMImMehAApSi%StECATagD$a zHhbK)crls(0`z0P#-uWBO~`qrOdE32{S9=+%x;>;f-0+7XLC-asT4 zU%q^KnR^l-M|c~28cwq?eAmV^5&&yof0=xjTkOTM^7H@&mmV#5j!(ByC%SaqGEsqC zL%Lv{CkFN~k@XxJNYee;_M?~VmSY9;S86j_?%M{^bCDcnA5wB%brSVcy3Np>kV&x z?DKL=l*dNImw_f|LqNR+zdg1CNg(5voS&0;2fRsl0y*}l!ouKR|K<$~q)=ktj?d@c zjGIoDIqJr+vJw82c7Wy}i)vP`YY&;0Jnnm-$ka~Y&A>zMA+n0tB&&PRpiwlY&Vy#4 zYGkik09ZclT;GNm@}2UgCwM-Z$!S$f&?ewj!}yReCT&YVVFPyv3{ z&qXwbEQ$^~kaF?`MFiOTJD<4*x`YYaI>VK+XnD*DHyqXhmY`6Grb}9?&L7@i9NX0j zR%Of49_uLCxkMvFf@TpG&cvc_QFIbn(FHbx_zPAeJ&u-3S zHW_8S$7Th5JIxMaHL%j)2j9v{x~DWiyphld3PQ!Q4a5XMg71qGXW<&FJL)wu|;U$4mKy4 zE-=GGM|9FB{fnm(qmE*1@q)*3Y?z~4I+gu>#y*|+PS?7J8@!EP&h9+GwQH?SEY5^Z ztHiGFQyhd*Y1aM1AN4S99Kk^$ntq?75CmD+R9H3>72D%;9x_Y2YLABov8?O+$d=`8fr1cQz;e zy2y9uGJYm|an7JK>8rK17pNrq2i)Evu-+h+9ecUP+ZkAwv?j5jWQ_}NbhKye47PhE zD@RzuBr^#eS1CBT$11NJd{yuy9|(35U6R{LfP17Z_dpY$d^_;EJOo7_W&9rpFB@#z z%PZ&k6I;SFvb}#W(DbeMO7=WWeeRN>jqE<{!)|n3ho{mH5sYaaU(Ql^<*@B;-$- z5NQ%#!zuQcXiviAabL;Lc9ovVtnvj)%Iw7v@s z793aqa_|hInwMi~ut(nfg^NGUi@38Ib4>d@TNS%ag|f(^BN5ZaPw`sY-lb~utRe0M zL4lI26i0g1xP|7knV#_c-W^CgzWrA0T|e0>hgFleB`h)nRLhpVh>r2}qKJBpwZ{pE z7c(@MtPssBTyj}|s28Mn*^nwV`>3+(8qfzYAiMjFAz`%jp)t{FYTnWGhfSx6c%4-@ zF=|JfxzbGAMJ5C%s~jRCYn;MWcAIx=DDLBZBCdAI1Z3zKqx!yp!wlSdW@m5to}-5Q ztNqnsDz_GFS#mn^&XHwq0_Hib+i`MnXsoT={uYj|lYaA-8`gOS+SF>?5)(Np0@s;z z13i%mT&;URQ}LE5=7pQ#V&YYVnqLraC)B;W#f00zAoKBdJwUa5OY{JhcqmaUtIqj& zAEN8cW6=oS325;G&cI0UYogIJ3~%`b{gD%nL-mm$i#^*AKFu8pdb6D0&0V~k-jJ{1 zUaXL{dyBJ8YbUb3Nlk-yumdG_;_dhNJ#*AbeS2#qKZwFA_5?9(xStJPs>rH%Y>ua; z_X&!?dV1IBYQi7~S+P?7*ZvqQ`MN4PIkBW>wPs_q|JC$A`677{?QiCnYo{ z&y{|a`sT^$j>j|+p4aMc-Wt*EOWfGt!*2zi2b1xD5IY)6KR>R|QeQpDYMluYxO8!q zCc$b>3c5RYvY#GQCq&1tY-IqX2L)9xKUNfFlamZxUV5}W-W}|Xi=JphzTY3W@71&) z7dF1Cm4?>8coz^Lw(56Kt*x&9r47Y5Xgo*wa&f{hw;DR0t7LjRqQ}BD?p`BlpU-EF z6$qhNSNFU=TDGO&!hjvjRcqP3gMBh`F8Cocv5qNKVcKAW3%?lx>lQzlw0li{;O>%# z=K0rD<5K+u2nP5D=vn(o*L_(>hp~=9<2ynrR505}1NDawA9RA%PlFGtTeQjJZx0)g zMf>rsaWwxy3`3HDj2lB4^twwy&|5UNQ#|*RhzS>QJ5bC=SFMc|Mv}&GRt|2{OWFRy zSV1x_)iuFtsCZ@V|7?%)C|rAWTBl&94={drl<(rcH+0&=%N7BT;Mxy9A4v%smL=r+ z5!|@PxP;P&3`ug|EMm6a)$$nq9rIhgMRAO?7-*1~y9B&;4@B+NuX|Bioq_?WY_$VB zwE3N`Cw{5th3@v!1M4C1(yLQ=$+$9XqiYW7UEvzvv37v@4$g4B;#EPW8oA7~-b z_6s|-4Q5_tv(YBp#_QQ%#uw}4*l{tCAlQh>dLQ&mWL~ zfQ6K;0$isa3zB#|OVaZ97rnZ^*7R%& zoEL7$iTKaF!2fPr0GfaPst7<@|ImHBiA{n}IqcQ54dA{30mdrTSkuhnlFSijlcF7^ zwfKHLJWXI$=_3+3m&@+^k}CYG#qqf}AATo^B&4IKDf+(OwUQ~{{ViA$m*ncNXJs%_p-@IXY&nlMre4)-Co+{M|@hB?|5(Swc5X%CRXNX9k?k!D9xCbz&pRHE7xh_1hD30G{jOVd7Jk6=0;tK z9Zl=+w3rjaA<^Im^QC4d3LPDt$-xAQ^ooZ}+dN-oTS13U_oB9{iSv8J6}QfUp8MKM zU8@HRaeTBbE{j`kAJNRg+!K#@UzQaV6*UtPtgWpZ{bA0zhwc%EPwGbiU#*+|Qt6Yr zVjMhI*49EUEZe2aK<@uGt^Vun{U1^6f0ZBs0T$PY0G1=X*}_6gOsv(Lkkgqpec2{C zv5GfeK_sQtRKGd_1|~v8nWWDcM2^aEYHM@De&m)OTLa~Y2i#1WMf*< z#tZ2llC6K9Sa1Q7=Au`LVg#Fp2B4q}Zble;Mr5)3LK6#En=1{PW5+ z4f;P(a#Fp2e-x;7>P3of>o&)M2!3B6nApZ_;cCDGt>@zPWE@O$V6DXtNGhpY zO9srNk-&i>Es_K!#UjAku+LF4h~IQy0yVK^xD?W%`AXR}G()|%$N8(EkOaE86|<4& z{qDjY=z5`>-%PuCIB_n%=sIQoRZoDYjqbAsGyrW3f+40gFe6(% z6`hif>`4#X^W?1_8453No_tmhzr|}kkKfP?ai;>mnrT($?5BI9tmYez@r;&FHJq_!1((K{pT&DJ-G0vuKy z|NgxtCJyD=t-gSie8TYL2%v~yv%QOd<217j8zHowHe-06_n<3iSp*E zX6L`=XCBaa?T8}lZeR(KE{gI$eTTfJwQ}!O%PXkagq8$1sY%4G5pyV6cp8L zk&%(Q`u~I&1^y3$7f8Ll9?}l}+H**VVth-L0jMAKf$WNGRg|Lt;P#^ZiW9C*v5rjv zLd8#QT6SsiZ=slCw1>UF(Sm3lszlzEL&HZ=BLnrVv|qinZOA%AprgUMTfL2rn>l{&cRrnDmbSZit#{GyDSWE(hZ~(79yW@LE?z41sLYI; zOTHDX9?ZIBrXAKP==?J!HvRZ4et zD}=BfNvRO(Z>gV+%XP)Cu*K<_*dSeESu+#TO_Z83wie+1kcE6n7%XM_v%bnk-t}hc zlLPBd%b^%MW}|zd?u#;}6$WIu>3L^q(*+f2Phhv>$(ywq_}QR2eQ9mrnOaBr_{|87 zz+@KxX%GTmx7|;JyciL3)I1pRS_?E)FQ!ASIDsn)ZxDX{v}z%XxAA^g#CDWJ0CLD( zkx&aRYY2s0zyb`}D@&*Rv7g)`(RipcO5AE1%81&!2 ziTqqveIi~1Gk?gaWDdMWpSx!%Z+343weznTkmojZQ#GHD_!8|{^A))CV+Kc!!Enn#qKQJ63L(2^(# z5rjo9!3sD}4o-Lh5uKUqRd#YCErBk|i$9w3hj&`@@nP~;bE-sIfBb5%*ELG*%Q=g> z5u3GAFi1Yf!9-2ZMct0uZ7+oesv>kfpKvX8oU_EbA3}uyIjIStc&;v|)eh)ktKRu- z0Pcj7(45}V&Sei4`}0q5niv@DDGqzHQ*hMQ*H5y2Oi#Bic}e%rC|7Z5vtjs@R-1RH z;?EZa!7|Pauf>Pwn9lOdf8axC-}JwbzY=x7VT42YgCc5bIRIA$ChKC>Vf_2tVuEqc zFOPX|M!wT}g|A?vsy3t0=lB0jG^wj>3lOAyX~P35ACMjl&=Sxio?V7t|H5~yCqY%{ zDEV30Y;~+WMy_DroX0aH2Xp>yCB<^JZ&?&{z@GGH>xOWJhP?aJGk;__M!qtmBU3{t z+1sFDVhsdbX<5>3=x7*bg~x>7Dk(jlBrDmiqcP1gu8a+`6z&j7{zs|q)Vm5LH*xn> zt@xrcZjc4fb63C9(QH8VQ#T44-bWjM{uPzNn=%C=xwU&h@TajJ&BuB zEZmT9W(Sh0lhq0iv?k*gjsAi!1D5~lfMb8-`ktKQx$H$}cnzyQ>X@9&j~?GXBgZm7)CmTRt70E30&I`z$1d z7ZJ{>W@1;DmES9}e5OhrBOY2F?^M#kb_Fj4gSxzqMO%HpGYz;$PBx7H-hS^G{@Y1} zxGHc@qRI0XJUeWsRJM-gX*JK>?B-4}e%0~8Sy$kt|7_Qli$S|2*-)RL&GC0bCmIxI zq40*;jt?%)X_syt>h{so2pFC34t{@nTB;~qmt-t`lJIKjBO0xFoJGHs*mYxcd`a|l zkH$)MLP#dc5j;gtC5w=VU58r>&-YsE@#W(b@J{<){wowCKM$O@JEAtc=~m?w%tE0~ z4#mLi*)?4UOLs#ygu`IeUYZy)b2->uU$3itXS0+*cpRWXL3*n`-@eA5rN(xf7Q6kE z&k{fJjOyhugXGM{^G?f`JtZiiyQY4XuZt0zKL=g-s$7q9WXRw#Q3UMH{mj2#cP5tl z?&NL~)N>yc^Wga2<`1`n!xfRmBy*vmitrU%9GnrZOE{&PpNGPm+O(^E#m=Jkow|{yd$V zM4W+BlV8EelD?odNEhiaEz8D9W+c?IY`sE!PxN=IRX!~N9txhWSH+eosih78K*aXpJdf5`a18a)YsiUv8fG&Pm~sVV!sxVW11Zm)K;nDJyCOG&X~ zJ^XP*sq24V_5UjxKzU;}41}8B@CC=%_(--5BuzO2i6+BbN*UiWUg}jBW{CpIAQUya z_asK|bwev^xh8j{s<>9Q2HL9xFm}|9qNVnz!4~|86j){Y3z8rQI#holS2)8t7I!iw;jgNeKRI$lr*v!J;g9k%-Ekc*La1?!2BaLey#xOKX0&KVA zQ0_r!6fq^=mtz;hIyXqhdo0_EkA(tlCZoJwSiH-KZftZ-P%IsFKC5s;S5qU+JD%#O zlla!(^YExjIB=^%XH|CCrSO0hJCW7>2ji+Ns3v%b$)9dJJ4XJ_=QYCq zIPJ*uYQJ8QDop0LHYL(3+Zqwi398qhhD>xxlEu8M1ICwjWj0sbXC8iajb;0Zzu(;} z&Iv4$Z5{LBZR!(bB zV>R%B-%+E@HR?yY=cQ6jK{Hh)BvEvbq;6d2oI8G2*&9+MI-gZsWM3&iIBs`c4c z^*p0`($&~T82Th&fh8*mlBHVd2y1wC+Yujg5yLn`{QCbe_SSKAgxRtv79_X@w+%r9 z3GNO7g1b8ecXti$1b24}?iw6+aCg~AfZz_VGxxpo&Yd%J@BB$``O&?*zi(Bos#*fe z48Y*{<@OuY3Ry9MPX%O9W?ZUwtQ`zlJ~Pip*Mb}7@fV8o)*k%<*sO@6-i;RpvHqXiLjS z)D7R6A0wl)S8QAUx@eEWxzauwn`1P?#h}Px?XRClgO7JLa(CA{%}~ofu*>6Ib54A? zY3AQB2N*pR|EtE9Trs?85yd|iED~8%S)l*-@oHjT+hy`uzNN6 zK|WL%q+(N(dflF)Ivr1E{wELcrz;G6$6#*=-F%@Z=x30OU5Ixqc~M1XUA?66tG`+& z0!0R@ix6QTC`;;S+Hj60Ggoz&P)e0dFrhvm6Ah%NrI#1|8#W(rcF{`Glz7*Uqjc`Q z**n@um9uBm@JMqdPY${%k*T)>KD4V6i9Fnc&`CaHI!DF(QPU=FS5~A%NB+SmKQ(CT z7K#WekaNTqV)dxwNm{tWZR`U#C81mjd13)86Q7{JNPr+v*s973-UJb!Smi~Rmf#zs znM7xPi9v2pDhrEsfHI~ng_S6IL)JWfN$oanttwF-q@eP3&dC7ojZa+vR^_JgU~IU+ zBQb&Ob&{H#KGsEENDi~NNKY8jyii->C%Kw?N=vu&+9cFKl1wYUcxJTt2m6KCxkAwiRrq9>!YEkUU%UPd$Mm^z8c(UeA`-& zar#^j?pBOWcW8^=jCO31&$&x$pB*MBu@GrgE+Z~NT6y;!c1W96o&3Qk?hTJCX%1O?rJOqo)w}o)H?&cAMgyvsF?xi;RfbDYS`(@Zy{@ zf5uoA?igf6K>O-Vde<5pL4M8~?cm?{y6;A+S|o0jB|cl4CXF;y)0R-#N|y75;w$lc zQ1cP4U#T@wj!!Fb0Yd%+CWU zw4Q=UChid4lZTB`myfTSCr^I_IZB|#cv9U)sw~H!F0F&Tn@dGC3yRC7yKcWPU(NSh z4*BI=D3%TJZr**vtDut>IA&FM_g2sqxXnN>N3D6CD0V&@AcHtfm7L9D5^;1thk>6& zyts16T@aKU_kGu5LT}TKgS6M1IglZlHWPvj_2}4PG7fv-^(OJ2gNoBXzEec35YRGn zw_vT!sadGTZpOPK9igoCRm!ZPh_lS~)cQ_^<+m;`-=>qlV*-Lzs27f&FSjj_bUz0B zV_38+9n!~{i$RTutFQ}%K0cH}Z!U-S39m3X;vz}U{S5-=BJx)XtVpRrFa-sacfm9n z>99v6&Z86#6ucp{m>-hahHwaV`m*(hj>-ZdR1(kOe zWlP@E(SsshbiO!sC1z?ZYZIv=FuLic)hJ={ft_i8hXzH(!tyw!-)8x^N!2gAm8yvU zq@KRy*Sg*KJo;q(1Ek_DbNAX5hpSu?ajc~Z$6r&Bf4D+7VE)~Z7n9$d7RaRYIgiv= zK^M5ot$tp0>WZ0LwU(1Zqfxc6D5IyPt!?6Vs0=%>XG-O;Nnh^ps4oYMKA~tNjwED* zn!37GfTs0On@!2|>(`54UY(@;o(&PPu?uKO+D+Dq06#c75XD;x3_`fw@aqm#dM%Vn zoCSpUz3nE9{~5%9hY^j5L!X}$&Mz0$=5n2 z;#?o0&^I>!hN=pGL0GUb2;Vs2IqMm4W<`?X#UCqXBMnoQl=TSOg?-0zUj|X(g3>HYb8l zfXk!>GSA$bk9q8V6EAq1QD?FdR*X4w>~kaL`P*+2RpXtL;QW`fs;kwgaQS(#_-alN!8%Rtrf)zgvn)<8C?JL+Ewk}Y z+%{-ZSqv!UTmCSya;r@)jSkuRXok%1Vc`~1K8M$~34$HQ5$uYX})^FQb z@jN|afyy7pzH;qnZu6o`vmhOrFOY3E)GX)B7cc$nZPs=j;rVnjm*si)+1AQ-R<;j1 zx|?L};Q-u02m+c1C}X+Dg_&U&0VtD4)_3zmKK=l3Y4AOD{NC7h(Z8!8nC_-+6?dH$ zm6OS><7d^!&6vtQ=aIEl|5;|<2(AsSb7NDxG;KYNa5xVge#qn5dPaNw)pNPI`HkJ% zjVAll!Pek<(^3eIUw^Mo51&x5u!e_3Q}b4}%HgSt+CbY@qWqPEXCO9t_TDEh&Mn?K z3wr|t8YC|7tw$oK^UdPDwu9ekGnuL;kNA{AI$qZYqZ}@4QrUu$O!z6KzV;qn59i81 zo!8voaJ#2A>%>*a{d338OO2%v?`mJ!q?^6w!!VT?i6GEly(<%-Yp3?oQ~{ zXs>i`&_lpdYJNWbtlgAK7YEPNk1s37F%Pu?Mbatd>{j!ojaEKcffKgETxMQCV|g}{ z`=+~0nB=u?h;#?^+$FI0U73x7jN}#2dMOD$OvS2br9YF_`@QV~Tmjz>LgS-5poN_+4Pr(wQ6@Aw=ZdfQn6w!kN zV*!@n;Y4mDVEJgw$9DJN^k!jZXCU^4KaCS#pLPenj7h~vmv(4XMcbl`sd&%b&mX0z z`a9A|CTjBW=gAVFl!E&VHOVu1z!+e)c)Lvsp5(jX&$D@7_!V1mC58B1B?s|yZt@CA z1;3|4=N*HR^4X7Y8lNwJ3-pe%1@ao7r|}va%irYt@T?tHa9$gT^MgW*=n^6ywi>dt zjh#^3B@QO|cA7-^1>yv=CbL&Rx6FG9oPTx_+Vy`5rwOjGu%Lm16UjpE?34)kghS^@ z41)URB8K13DWxqPBp~7~ztaUHLAfAlpi*AYnU(2sb9|=5s)U^FHnMx*Vt$O!#wb=1 zWZ7;(tMB1n;!4YM3D|R7#IiIT+j!OHmsRBBZ*PuInO(VxlE@JG2*rAggRqllrbD6? z$rJk1qkl{~_V7Cm;GE#VBlIXlg}=}q-+b^F<-opJNk0IVu~)01j?){P7<9D2x%Xby zlfhi1Tr2I|}uoR}X-IWZD>?(>61Z70SLJ-%w#@ z=xzXLv;X_H^MAe8LlQtCG3*v<_-C(pI`&uPbV%*F#-ANFUw;l%OBd#FJ=~8@Ifb$M zaJjz_U_HpgXEA>Me_tGs*+)Qc_r$0A;}{!tFa|wav8Yf1?Y(*&$=0uCvk9y6V&|GS zYMw7#5pcBw%gMoixrjzJP}O#P+8U_YyhCo5l;VlK zfozC3#lo^3+nZ{g8{-m$h1PM)jFhHUGk=~~DBDvXg{+61K6+g_CkbKd#C+>j^k<~> zgQ0<~PLqkT^X9_kLZv@>dfu4cjgB!(md@>d(~~w9ZDO13RGb$P`!lD}h3?t7yE}}5 z|0LsafS9|@F0n|*ZoeL@Gx0n6`{#>iZ8n0Ou}i`jid2an_DEtJnUedctrsxT{(EBS zMI+sObP;~Sco+lwmIFC41Y7m_A+&&x{L_^$-l~~|S$hm6qym`oYZfxIk7ms08-I`{=5pxru;dMQ!1rwW&?+=SGbVJD*jRcU*#5Zf{KidPDzVIt55JPg2YM z$jCS+2V;+8zYHM6{JGy*_nm7ok&(*HYpxj&z39HVSujFR}nD?92Fe6#| zpINs5;${mWgvr)72w7GnLl}xA+Y>GiN+-%lj6(mpQK_iBhhPYA9#)y>JzN+SwS|rA zyNUf@q0s(4ePi7fxt}N1RCNlQs@#=)u4efypDX+tjmPHL%X2V8G?t>wk?q&#Oi=a+BMV$zbu=+agJJ;zsX$ z7$yZ;RrCksQzhs`C0S8K<#reer4T7|~GOnBKh_UT= z5E00yD1Sf^N!~{{;sOgeF5tc~@!rgg7u4H}Nh_Lmq)RtYzak1FWK;3V7Hk?Fgh#yT zA&W;Cy|roTs+vLr;c!)+o#BaCGSLF`Aw{-zxIC8rH;|2o|j$pI&!!#WOd_OgHW3_4x zmT~f^obdMZ^NWnQ;IU41iT|9#x#3+8vqk#$N-jw2CChESth(dk3xr8~Lx49L#VP7* zdZfSgtKjRUpyi>`hMVK>pqW1Y+tq@>STeBce+mGY{*I56d3%VL3L$IP7atO>et3<^j2eJ>1S1x9>&7TRLzsx@V2Q=oPEaS`*o zwviiyFwjWbOFGfg%2q|8JhJc-gD9mipLVy6wjMQ@b@tWRyx<-R3xTO+@ox8p%>RlO zkqe{reARa@ziypufkfP-o(zNQ_;#W#1#=0Lo>iRK^G)v-G!O-L-TS{h%lmX0dGO# zqh9@O%?dENI9r)~i>traa>;5$SX?af01c}MA zx%M(2dwhok>;-H|aYxU#L(Q^MQ-t5nh&8+&spFr_4xDM2713=q^Jb>%!o>#kb7!9P zb;X&>uDy+6p>laqaZWVv*s_4YA+-qDhDN-1QXSIycJ^$Car*yVcA+ z3X6aVxm8QDUZJgG(o;BXvm;~?pH2{lY~tn(?RES;ece#rUIBfWhS^PF1@!9|63dN! zzOZO{#nzp#+Np+>se-8zpTzk5XW&s{U_zUHN@wmq`t>0duICuoh!T)FA%?hK4uTtR z(!!Su5LD;lL9CH4gIERA=BHFX&6RXVisyYdw-($Pyh*ukQAmD6n z*_z9XimeEP@A8)FP7l%V7Olf)A`zNXl^8%B|l&Nx+8{Dvz;2<-$P1jW8r z;d|djDlBG#gDI&Tvt?dr1$+>Z2F5+PHsBA5AuS1|Ah%OXU^;);wSr(M7I)_LiWVUw zi!WcU%%DWad3U$}k z@15+I93t12OV{i2NC zcDQ`}t0GgJJEP=m!Aa0AC+EDzS~Huc>d}_FVZym0BA8Ij(mxR{3$a2^63OvF=&@&! zaa#zPJH2j%H98)@v%eDM)>sMj=e?)U`Vq%EA_zQ}&2>tgR?*Sees+_HXHplxy6Dw7} z=9|@ve4|c9tYz(IUWC-gP1;{6@N;(|QDrNSuP`zFkC?5=xEmZ|8$P2MAd?b)~ zw&D~OL-m87yZTPtzU`o_TU5@p2C(Q_RbO1I>N2688Zl43F&3AmFNYji3fV!Mb&ZX# zh64nHZH-Iq;4(xzoMRWPc<04ds3}B8WV2TKpSNw41p$&uueTC+oUx6aNxPjRS{_V_ zZ8cfg&Y{3Y`fsyp{~0U(@6RnKE$onLCtk=c+)3f&A4 zP*$|ly^rkS$@HDAy?^a&^Q`GEsNz%{CGfWq4kod>HnAX`ui1Q@y-?^rqmn=c{=JN? zFbrJLh#QlA4+|?BrMgl6X_(ubw6u{CUrlutcZg`WGi}!Y>Bs&buL*Ti0!jS7p*lP` ze|5h96M)2ggad}2sVaAd#-TTQbBg#%8&|pfnmP~XAt8Zi8W+^vlv>)9t23WVskKf{ z$|;!(8rVEa`*&oerdQ#W?sIa^mKHp6(#5sc`Tr)9Fh(&sSoqDWp!xd9W7EEgfib+? zCV87xt}O11JAf}Ve0E^29?iZ-Pr*JmFDB}e)<}X4aWSuD>5mg4OA2+eL!xV%=jG)+;Pd0TI5F4$3|s@lzzg?LX%r9R_W;=`~cc} z>3wnmFP5vkD5z}xPkguk1rGmD7byS;yjL711GaTBINIaJ<|gwZ;W(1Ovd8P2U(1aV z2GK9)JS;3)uTbdXe9g7zohJ-B*LmcNaui2~`1Kj|##|3DO9eLv!aj1p^nl6KqeOxW z^Ue}p6+8{suvsYpxn_!zlNY3jGL%l)5#^Q7sqhU4LHgM-)iAGngj{@EUOExZo};20iw1au9vL zU`7+zus@On^f=wz4t$M!p)IAAv9NF#wDx^lM?0HByBKf*6d^)RPW1xY2j=(OMqyc= z+&cf)TRQ{=<^t~zv~Rhv9+P0LDmH$mXtewn^)$*7g;dwX*^>`*&o&rv^k_Efa^Hk) z>DWm8j?}Z5{^j~F{iR~&T5HkF#h;o`e_*7s_nI`)MQ}|*TKW1 z*bDqGy5oQmMgd{{zL!;v^^&Re{ckyWk`tTqZ`qCKbZKgWdDNe}67n0%6V^W-F+{35 z>m;Ph8_P`6BQ_$@nUO>7Z67ScX>CWZ{z~AAt;l(1jOXEho1~oLtCvl^pJ|DlYlv5Rct++DW zQ2~}}>-)u?W_!y?T3Mwbj%-fbWR9q=D)@NN$J&8r>%;8p2YbTuF9(ku1CU=oB;=4+ zI4L<#Z`}MX9ntJry&*SmD9B>qnXjs$B+xL9vXYZI2O&OG1qS&?Il(8b{_t@v**gM)+f-0vL6|K-m1zaJEZf;J39+e}31 z1GweuLf?f=o>A%t2f|U8R^zXs(X%T{Q+Y9@isPNBMXBMy+$_sUG}V4%Xl*FrP!dI>lXdh#5<_ zS{*aXb_z09v}mQpzLL;kqUME0VO77*OD;;_DF2QD)+O49sx;;r;rq zaVMx`t_fHE=%fyriCo1-8M#?1AWsYWA@ff!t^Eyq$+YSo|*8 zf*q{JOU$llMyEga!jmzwlB#p>+pAt$tclCEORx#Lc{hx8n$PBAVE@cuW^M+&KV6BU z#QRx~AA{;c!XEp0*?vk%gK!cjaBi+43Jhl<=hFn*5C6%4r_|-KO~8 ztAc>uU?5Bv;m^`VaQ-a@zEN0MoVcK*@!GOZP0Z;^K9W$C?6p`NZ?2FCVko;xik8XK z!J-&wU4CwWH?E8!GV+6}K->lb3oVs6b$6$eYMA*qaQ4O7yQcDF8a6=LPkw6p+QC z=1gs^?eq+~UH@>}bFw@{?Vf?N$qD?fsN4olib4k&ikTE@C)MuZo@o@V`y6l_cfz9v z^^k_ep|c2JPq(vTG_0rxdt)eV`2A*TjI4Fl;xBvOZwNEi$|+cQL6g1tVY`09a%>Tr z-7scbS-A=9`nz4wcci2`Ek0elbh7ObTwN$xuOmN0esQXKS+44HRmrMil+GKzKtCHB zt`u=sySL`2!Edx4mF-hZC~z;vAdhv5NW9u1GKZx}l>9xpO;3@}!t|`%%MR%L|7v@7FC9R)=6%{JfCGWeP&_&56k^ z(Ak2NUWX4$LfAfdGxQ>%f%Uu}`}{v<7p$>!#eUmH$;=jvncQeO>`K|Ekt>Pd-rTb)T@d!x9_sA3`NAdC#xCjQGj*d2Em06mfRsnpQ6?R!l0db+ z{CJKqvok><`6nu%o%sqxvpp;-0~3?Iy>AP=QKdF_bN>S`6+8)y$go=`OB_@#HCkkh zEZ0}V6Jaf=sch6EH<_c)2RBh$JB>U7H$g2q3H%XZTThg%3m28Vyoae;2dU<$-+F}r z=I&`AS(!`pbusn?$h=MBHo__+Ry+9gha9e;9#11A1ej5bJGCydm*(aY5_#f}JwDPiubh zJGl!ZQDaGDi+xg^*?xyrmVNnSVb*CDW9Sh0(=;pnHhCZmpx2(mYkc|Ppv=pA_>SCY zTX(!c{nfs6opJ{>bVM2$cpTM_OqxS>w#e->e|>sOH{{2frf(@;GLJBRs(mV@cIDV&G?aH4ZD zg^}bSiY4s)RF^+gAUh3LVkZ?$+PM_kD}@UXj)HobTwYzRw6wBfG<^yVhD*xIBKdPR z9ds9Gdhow*jQ{2@M>A})p&sJJt8D6ky{K$M-Y=zz>ZI#s>Bkk;_lP#=s!fY`FQ_F} z36(-xPUT@i!T#Tnu584phcski6+3^pr?AYy)d_M4h$rS|*%;~n101<{L(mt=VTNdm zwBMfE1g9l-pd~ojEJ4ezaz2WC0U}{vRh6wz3aTC2?DNaG$GK66eXgIL@>FsXSd|(% z-Od&k+amgB&01yo@q*)ncR#Gvl=;TC8WR#DaD8NE94dB2ck$xj%=E}v;SpGpR)Q;w zL#6KzlYx?+AgGd}_w2u`seXzRfDC`3+*=NAuy!$lSQZ3Nmp`Ja7V&(v8o)_PJyp$k zv(#B`(k#MTY{*o&7yTZj6O%^bNC-(Gp}|se8onpjjK#hzDr@SqYxtJ1W>w|cPm@zf zk7hgLgiJzHggzq&!oTcgWVq6>m)&e(K%eP2<3UIh+YBmz1(Jd>9gUXE=wuuPs_VzM z5GZn&`lF)dp)Nz?BLQC^&P8ZQIi znyk=L=cYZ(%c~0OI1X6szxJ?0i{yK(U&-D7Bj`Qg6NUdHI6|V^&dR3Ow7D!i^5ep- z9iJ#i$9eAZ!>O@anoG@U&0pI@m5m(nTKBiabV;t0C@Ys;dZ~#{yb$0uu5U+GR zyOu=KmFUWT8MqWJzc>|b*51SNyz_1#{F6w#Qm<#@bLL?6G4m?fdyDkBqt|PAkn^>A zzUz+0>J>r$NkiB9QLtCC@xmGjH28-V`#vCB`vBq0*4mxASJC(^eSu4w_l>pp4n^bD zm|yKhhzQJLkoOsz7_-LJ;hDBv!{qaizZ#ksOvwpS3(SjShDh9Z9gja)*%o99?ZQ(B zaMr&!0_+8fFkHT5RDRG4+Mi+elcSFepgAk;=Z;$xVe6PZij?GJAnM_9LnU_4z3?pH z8`Sr6;la>1SP=vo1<6zY1Lphx3*uxAFacm*u=AlQTE;Dp-PqcTqfr@Q*w#zPs{^su z)us=iM5GpTUgo*Wpl|g#aHD2G?E7085dp~|peMpxabSayBz;?{yi`zpkCBHQc11kA zjMB#|3A6!KJ2S)F6sFt=V-TpgDR! zJu4^W>TF6!)QMLuC*0~w%U%HBRylfOH*IavDo?MI zvUrDt2*JLbJ@lVe+4)J&CA=QlxKyLPyaC*rSd-}Q3W2$>Yvt0}|G2)LA_mBqCl2x^ z8GcL6b62c2d$K1O4F?fthhE#%py2)+JNFFYW7l}r7555aEJocaqc~CyL*bl+^Qj@^ z?x!eOZr@cTJPK{xDo^2>pyw0;>qPSs^4T2YGoWP+$51Q`vbG$JPUQ@uS^?_%TWJR;}*a#``z(?Tdsu-LU+!#-208S~`-`~~0$WDN3 z^TKCTYH;Y8`&-zf5wNI!YQTg7#uHaXzv%obOS(vri&C&V3sWFMmng-W{%n4RzkEj= zS%Am#QYYkNa9c*1BqqU7x=lkvQ*q~RH;V<#TKnxpQ}AE?!+&3R9&p2mCdQrhR3r~e zSs(TfTCktQ#Bfx>p<{7Z7MtjIV_gW>n!#?3PK-aPDb<_8GLzCntQAmVDfj-w+^G4; zpo;m2%zZz-l@Hz@9c+sq_Ne4RWARmysjO4CK!5_$1Qq$;TLn|=aW1G=@F;P?ZFSY{ z(?;h7`LiozMPO7^%#QeR{2cn=L;@($G4i8gWI5g zic>ci$k7|($irE0qM5;Sp_$VJPV+AZK$-FtHp;#EK{N>W^D~!aPbL)`F)Pt9BoCh$ zI(-;A?}vK{Q_PKQAJTF!OyJwW5X3iA4tgbsvv#K-fk6doQ9B)>-HUv)xBG;>|aa=zz5kYjOMdZ%DQ9Q@fLZu}`{ zkD1>YXh@h&({00mM5v~owjV{Y<2OCl9sN+}vL#^`Z;n-dsp3C6=xt{D52FAn?Y}pB z7L+A;C*1ml-#{3+jqxYyG)Q_2fX+rWa^o|tn3Mm`w^VZbOkfG`tB%d432m9-MSndN9%&fJ3RR5>1_86THEv%|_f zQ@l5twO;kvUEPTkt36>oh6Fj#JO*=#N@F9=aZ7Hw$Gmqh!<7y5D6s7(nfgA-Wfg4X`$hUh==RO{X(I(;G7A5b0-4B!U6fk0n&h%)j zD1gbv>j^|XdJZGpZCHZ%h%-}=Jw^!lO5IcL?%el<95DB0jkGklMLV$o>fN$EbU*Ou zfS1|VUaHP$D?qpRse>#_XbShBx28Gk^+e~s(sh5<;DX{@V2$UL+kR?x3?vY7Lg zEczC=Q}dHc9U=F#%DQP)z|tgoj$+a7xTYv(hC$0sBWrJX%He^M+Lj)by>4Bv6h*|k zS2LrZh*|GFB|e28B=nU+VyLRkd%m>H@(Bs=c=*lF6~XBR2G>m2=Nd*x+#L8hW?h|c z0dIMbkE`~aUsYzo6^Vo)pIO&EgVX5Ek(=9kFA_pL)8c0^kLaGf)^J8sqx{m6nNnd6 zwzX*{QA_MjTd&U{1~*%6Z@^;5$4vWsH=k+*9jBhJw@2y+&pxr`9|bEnxsb%KF=yna zZnsl?W4;dVq`0_a3aPMh88s)ee%kjYn#hm@UH+WrlvdFR^c6(6i=F=4D?<4U2aG@~ z_7b(SsvvzS?2!8={(F()*ZjMZi;D{trG#5MnIT)Y=}V!{D3_h$tQ+0*GUkEa(Z`VP>_GxT0{`is@9V>&u0g3iB$i3u!)AIni-sB#>ok z=O}gE(&*tqYlQ{)k+<)Na*Qao14>b*`ND?C7u*_}mxi5JL!8z=F9GCsR8u znF|h~|H#d4b|q8A65~UTQ?LPeR2HcceCoul2Q^Z6(f6D&h?j(z0_T#*KuU^<0`j!j zXvRvc%cGzKeIu=E+jqvYGGod=c?GBOw^CWcT2Oy*#fEB#MEJv2Xp?0)@hMoF8F}$F zB0yj`VWqOdk0zhVHT&aXVo-|%;@-Y7!hevYP$JeYz)T|#M_dKmSj34^t_blY9>o?Q zPdq7NpTNvf7a?itg zB}mqz2a$=gPP-iJVNT7N80gwbF8S|^}b|Iu~t6H@o~>@KBA7^W+20i3jR?dX_Di?!*~okyGdxRzt> zuL>o4w+!@`EOg-?7v3+Yf4rvHZRVJp;_@y1>rylSi9H89ps+Z<_o`B&_lSJFZp7do z#c8kaa~xo&MZeESj8Pb9VOfE+ei|>RKmxq224)V z^IGTRO7jgW33~`py4MuvGuEUqiay|uz52jC{PUVniSunRt5bxA06xRrkvh`glIKBXPu@m7`2Z zVsMUfKz#e}QC?Kt3SCmfwZ<;JT0u1>!GJ&6cCIdmyzo?S`2|)^hG3?FW-d0u# zYa$1mmYVkGR-@tqSzSvBte1`HA0)f-;?9L>t!zur+l5G2aDwQ`Gn9SSMrqzFHUUa_ zxdcylK03>onMmb(95{ry_$3hnbje3kFAmEHt>};*>UC>pMO!%ir(O~zjgCCaOAE-Fdc_;={|r& ztQMU<%YHT>;M3P^blUy;I#?9D+pjj0PEiXU8@<;<#y?x3kc4G0xb!ffG-slt&iDl4 zcJ&`;QW~d!IO#8~M+=yE*Q4Z{cHF0A&|VDLvaZlPkDk@sQtZ?#qvM4v16p|fui_qC zK)k!}ZQhE{@ZElk5WL@QuU(oZ&k0rM_7ZAPIw#-=9(m}FeO0LPyROmtrgQdt%MTO& z@FvLbWjHD7%=#f(&`*_DjqB3G>s(b(39@$a`VbWr3xS#)UCQ0n`Lsv1-cW^UdRVmK zJsBEh#~?UJ-&faAoZf=fnt(@W+oX8`S8Uvsa)*})W>OAiz0y(31efyh<+0Ag5i#Ask?$0`O30E;hE4H%TgS zG*EPb{H@+iF$Tx^Wo8?2fB3)ljNqW7{Zpp+gP{Xh+-zDUS{p9@peg07=jC<~q@)(l zm1uUdIB%#_KtmtS2)^gjN5MxvcjJUmclma|EF+DeOW8tg zZ&RBXQXjiJZurgtZ;EY^uFp9vj58BC^SZs=W-s?Qu+NQx9fQxP6(r@KQnSZXQ3qNpPb@n7G%e#ZDMk0pharsiC zxgsg#(`G-p^ss~s^BjYoGqpP@RH(CxAgg*qUD)YCW<~<%pA>F)KL~bDJ#S_s zZXcz3K$af(=ZWwIPf?ZSA2X?=cJ$l;Ex6?l5xS-*EFGCK8M{+ilDht0tC`bhlro&x z+d8sR5;DGi&A|t~H%hl@oa++7)@S^LqgU!r1EP|8&p8`fF{rfdyU2nI*OVh=ag<=8 zB(m~nYjQ3y_bc^Z;kgB>m)L7E zRr70-zeU`ijN^(!4~@d6j^|QY3g7cN!KOL!@1&WcF;oGU?2k`1%cBPUR7b|u+ghO$ z0=Eoob~S2qyKI*aKAcB)1Y8<^FRm*v4`$3#_v1wbXLL1|inR@0ADJl(C z6W{_K9PzgjquhS%3zdHwaI735XL(KId>JVx5?3r+RO5e^)*e73KKrgLSOM^5dB>(X z%iX7p8AmUUJm9fY-?AY{@?+9po(SN2dNzfiHeSg0f_7KrHm)@Oa3VTrkr6}WKYKh; z`diKF#Jo}_Q~n!$$A5?3_(zC~OJGqPR9apvZH+lB%m~y0?GJNePmzEs5HR?`Qz1Kb z+E)n=*!;GXsaR~s7?O2)Ol_gkUU0z;9jP-YVXSEV!#|( z%9k=^Rk57zM7gmY}lSOa{W%;|37CFh4>pR{+DtB#9z&qb8mxmKayl>Uu{SMbt4+K{>6W1hhm*MQ@v zC4i=FI)3?GvsQz&vU&&DL#-sV#nk7Xx3Z$nSDPNrZv8eIjt%@JY*Wrl|ySz~w? zJ$7Dbn7VX$F@qlj94;Wf`A?WzUaV)pW1;2_HklSAcFhca>R(a2sJ-??;eH=%WY4sB zOcpa_ZGW@pJ=srq*3GLfU}Z`nG6F1X(ziIVm+0$5A<;QC8y}myZd;f_AN}OPRdexF z|L?N^^rgsmjWPcNcIqG5h<~AB%zL1fbtWR1g=lHPU7%`ALKlloP!*3zQu%gcTBYKP zYmH)m`0%?Z925~8hOV|O4eE;46V^+az)q|L6Sfhar1Icme z6C-g83PN9{DFVr7y|M-&(DyRwPRS0ooX%xkJeO-t06%RbCf3Z-ITBRPk$5ib4m(8} z@ep4)5OJIj2w4{PF>T0W{s=W7mErGDf4^w*hp=q@nO38Q5&`WNxR{iEl&?V zgy{op+-JA|V}GdG=18?}lWP3>JQ>pTSw>%EUGS&pi`InKgpB2jbxs!5sh% zD?sN1M%Mt;vjMCWWDpXtR`W0~G|}PSqzyxMEoL`F?=IS(QKX1J?yp!W7j1HqV!xf} zsFBTmV{R33?CZ*YJM_hbIfG8MbMq}X-Y75Q!ez7E{2BeO`@(v0PQlmKcvd-1&ba{D zGrDo9ru1nawFczf=rljlv!nI&%oCaEn_cDVhWPVVW_~{lw0hQ|E*E~v-Lv~$Ra>PE zWBVGTpg#Wfl%-y0FilM3D(Tjs5}2lO%*4j&_jbl41lXb<>{y35bu~L8hlk#KN;q5R zk1}SOQtohB3GwGtefxeI4*6}YVD-iVnJZ`$B?yK-Y_hg=kqcDkrSj3`GuiXdXuA12 zReRc+uuDhPQo%!PozZw6KH^I%L#kN zI2c#gEtw>7@cgj3XYx(Y=IYfzR>_+x%}=IvS{L7P5Evmf{!%!u$s;)(!=T-MO{81> z66`=tvq8;;iu)Abu~9;)fhQ?3Z{PdooCV9BMF;nl(A~v`bsjvmEmEpaq#ch9)17E% zI2vb~k8Pdrhw7)gSf8|03y$?(BIiDRpTguXqgUtOik@15)79BAJ%0xx&n8&;+IkHR zQBx_4`A4b%w661Yr}K*pkU`uOBR`_zVf-vXxcRa+-}AG>ZSxtI%8^s_`gVkR0>x<6tU_l-eQ6>>Q^S$Ar&htSl9o5%;i zBm8Z|%@rJQWaap;uH#TH9ml(Wq5=L@Kkfg2%P&~CKxQ|uU%Q*-Fd`~IXO!Q()HLJE zF8Kycp^4_BD#o-eS1j@9Ir$AB8nD_{7Ih(rUv;WvBCHryv2qGp|ID2s(pi=*ONBcl z#Ksvey)Pgq+W0+bKp10`MB3zud_XjDS5Lc1|9#QkGBLKl)5vLK8#3(>B>EPkMrhGU zw41zH{Ffj{>50`e&UvzfKtlanclFqX9%MQAFn@nZXs*{V>z)v@V`eq#c~M+$k#9Hs zKGMdhoCQXSIvrS*oiEs2kVaF4N*LR>J*g!?omBqC_1GiO*+`324H~sjq~s?)B(Ld- zyfM*=7m_6=0*i#cD0D4od?#jxEwEB-Dfs8w?erH%1&nCGx-;4zi-u4FdxNboJaf=A z`7gr^q#Mchv=|;COiF{E&boQgQwirl#2JsiDo205IHD|X-^;s-h)9)nu69vG$#U|F zK6@}rP0Ig6*;@z2)puE=xVyVF9tZ??3ymarAh<&y!QEXOhmhcsKmr7JcWVe1++7-X zYZ|$o_nn#hP1T({&sTMd>gqqbsB?a2@3q%ndu;^cA#PbS8KysaT8`AF_f;+p4sJ{@ zEB`Fl$zve8e8D}~3XHEX1O^~RFOSwJsA=+#H_JMBA3v9^d|a}p2TsGwa&0Iv`66gg zCjuebY}{Pi;Zrf*pVw!`pLz%GU!tIMOqC=aH4N@Jf8l=#pH!r#%)8ylvD#02{+0_{ z;2WEKv3Epc{vk3wxB!V{@3zcmG|~+;18bY#ZKSUT?d{o*p_*dI!!Oh|pScWa06!ecN5&&zn&nY>wZYWSl7MehNGG>t%ZV_~)t~1bJf05>>(G;H537Dc z1$l79dXj{5l}L?UeMiDsPP`X}2-+5P``7GlUZ=mN`BHai@iDQ|)~7Jg?99YOVYi$% z?-V8nLH?c2{?XMkz9zbdpB({{nWCe}Ubrid01R#nDjNh#9Lct}R{p=tI&k-mSKF9S zjrS+Mw&S;2Yp@BH)-@`D~?p!2hRc|Eii+WZGNEW`U7wBcsw*R=l`=N5Ig76s?WUvEk{3st3M-Gmz z4#q$Ta|1SUz4WSkA-$*hr;Nc~Hngky8xIXKeD4`>jTQ-UdTL>l=;(*S7(`GB)w&Ng zsY%8}MhXP9kK={sLiq9%%u5P3%LGxXyv>SP#}Z%xeiG$3C2GZF+$ww^rp^lZX|nyW zb<8xzmPuvZYGPR)PcVmkgjaNm#vBnEfhZNf^mJA#AFm4&Y5P?QgSFUM(~R3N|2XQa&|2&YyKM0|wuBSuvgP>0 zeEU|3J!^|J14@Ra7OT>N_MTT`S5fI(+kS$-CnAN8mRQzwq7)XhM3+*7W{4}MZU{j)P;U&mi?h1icD=!Ozq zAkUl*b*zqwUqxT`xfOoAR<`Y~S-KYg4c8zV$i4W1fj9<>sOQJQyj(rG;kNfAJO-)F zzQOx(h2%t*e8r%EkWdeAM9`dJtlQJ3eJPpTgDtED@R=b$))9}5(tHSiZU_cB{Pdee zr9tcn+xSCr;HB3!&{AFdur9>;Hho{4z8I6n5-v_I1BkHwx;kDlWTJQMwWp>limcf4 zymNFLLllwhBK#M#{eM>W|5d2`e}5U$0hGaj0$o){DSz66oN+9FIvuB0;nuWF$yi6eZTy7!OBBoYlEGsu%Em@g_2Bk0On`IC&5!k0=A}38vP!6m zLN`#=3hXUQWod)NohUe9_qq0aLoUB{st$oXZPJD62agKu8gKQ%aig_n1;b+piY>=> z<-F$#(XJ1&M|<2p-+ooR)kVkb(iYwp%CQqczT+o2s(sVFP))!M zcJ`ML8dXMc&LP*?r)`Czz3G%PMsM}5)Er;DR?Dr6My-~QC6QVic z?$|PY@-&2cr``?_7!O2qHbxD;DEvundp|aav=4nMAnl4~7mT7=#`9iPPq+HmK>OBr zY6*X7@Q$g7N3*~>=S_lnuiysx?IK*FI3pee(_tTtXpf`hK#U)kdM=fP$p}!Tz?pAI z-A~hRkQ3%(GmNNyJ=bNw$8q73x2~VvYE@#U&8SG`&QEr%J$C3I?^e-^)TZ8g`3>dv zvT|YQ@66lxs%$MM!du$%gJ8iBFC~5|yj4k7czNy+cxZ0yvDP9iI01t<(sZ=EZ=J_P z?QAeGAvAejLoZ9QBJub~qL|Gopf-`6t(JN}`FP#5t1MZPn-3kl%{dnye-^tk(!jsfXe~+j?Ck4RoI#bQwaTe=sN1g~N8J1Wxc63nsx2Wg;g?0bi>{+|K*+-%a+Rdd; zK9&L;C7Bqu8b5?^f#w7=_wNuVkH%FL2|i(L%$F+P>Y>QF1z{!Ee7j0;Eex5?5)SMr z3C=f8D-pc`_puP|f68ZVi&A79Tjf#O^lb|-fZLpP@UjvNu7IpGY{prLSO(A4;AVCq zM``IBjPV8K!PfNey7Dz1`x7q(74biA@{>ne_`G&a)#;4rl_{Q;welp8H3QYG+rl+s z9a}_`?pBEG3}^t7?&@xZLG@ve4=|2&X6YX zP+HB06&%Qy=&Sit_i&fet<)$GHUqBil)p_8s*UKlAaB-KudFK9CXDDl zDl`y4=+V3XZS%;q0X5`Ffq`-1B-~OV#3cb0=r%|-!E5S7c zAIawSU!Fenvhir3j@Ch%u$N%7$Zic)&6>ulu zaJA|pgmD$B!pU*5nI=8Ki&twVJV}_~^~2ir^(##1N89BGZ1_TjkoZ&E4|@TJ?GV47 z9_5ya0w+xfj7)oGWmPc1&XWgn)`0)=z1?)5poH&DMnyc$a!pT62_hS*2*Vxc;i3!= zUoY?JAITGJxB{%udCUwh<0_gvibI+7>;97BZescfY0G1`>w2t=&y~N+4Rd>$t*?Aj_8v6_<{4;!yk{NS>7{|Irmt2@OE1Z^F0-nq zC2RFi(88V%X0+9DEnHz-TTnA~VB#~hPr~3GJ00wxQubQvZ|`;sIx|~XxqoN!f5#+* zm|>vU%%a(u6PzTvLC>kJjJfVKJKKL|kO+K@0q`}&ANk=VkiKovU{)WclkaQvz1Qsc zs;z2C0(!-8bI(t(;0BkLCb02|Exw>vwDo@?aS9l;coI8HsRS!aPZmw2%YycpQJvQ@ zTxA*Hf^&@pU`4Qn$N|U98!#m~5p>C({5UQAU<+jo>T_;e?+R>IOz1FaBg>RC+3J zO>2aGXzJHB0{DFCZzrCXubh4}i@c@{?{#{`A|%>*j~}GXu#G==bM2;}ZP1+}zha-r z#9D1iOUUr55UaVgtas>7X^Df6VYFXTJT~6!NS98)nPl6eT>^ptv#mpX1AS3EB&qzu zxXfnl`r*YXf6*h3whwHmzHhq+*{P#Rp)Qr+JmkEg<9(rQS-RiSWvI=EsrIV(L^@a2d z6s`F3$(^Aaz&qj7nM2dZQUm|%M4!g`fo5UaFoI@Tdo#waHGTMuZ2MC~0t48rhzT!3 zi&1gL`~XAr^@eDo7ph3_y1}&#=t~%WpugO8k)W5nR_|@p(jC#B{-J~vEonRro=A-$ zlPFK?ijxbCNFa@&m6dyABJLV<`L;;)H@&cU&+QZ{E7c!X3liBm+aQIm5eko*b zq)u4(--B=eNCf<^>~B@{vF<;iIhja&jagC+R7m)3Q>Csx+~7w&mWiL-9jY z{?)i8g6z){Z(*krX29c>RiGg*MYI_NN#5o0FTEM5yy7X%Mbzu4|ALq5FtNblZx*T1aMkEqAOPeEy#F^xdh% zox_rb8+f$d!P4^SXCvISyY6zaUAXwt@u^kV7xOD(_5EK3fsWV6vYY+Vcl$n8W;^=~ zjSjD4J6m{jo~wq3hDk>nCT=J3y^gYij{z%8WmM=f&aNCf=<_tI&%|oj8q7#CtBNEt z)_-^#(Enm9wX4?qBx0?~8BKxn`I_Hj(&nTl7ZXHL4KG~E4$>=A$V|;0m^R!gToh!h zNbzbA0a6uABL3;jAB!)*EX|!oYVh5M0eBNvfBQrSh!dQ5_YHjUz_x;R$-K6FxTLNZ zjOenA38Xm%Y=_+?`hh7;p-hrR#AUO=(!0D%byLz#;`G_44FlJXVtHn{=)SC5#-0!P z$}_WG^i!jZ02#M{ucrRX_eY;hM0RO!B`p?-)NG?-Jlw|?1||g;so}2k6^V~O`lPWE zm=%5$*vGs!6Lag*NhXzDd*`bN#b*PJf~m ztB|W9K$sy@7y9&)%1XqmH)Emhv#}kCVnMG*pIW1LZuRSqPT0?dgbalrd?M2!&THDb zzjDyd%qp%9_!NxPWZKnfvioW~J=Z{~R#>`{g70Yv`q}GO7tT@47JIIZrtV4UT|6 zih8v$d!i{tHPpR=%|Cdt^PzaMIO<1$GJ~*WpdP{AM<-qQJ-TC;EZj}NQie@};JsjP zLiTL81}b<7;^BOWm7@=LV|u!Nz!a;CM##Kwqt7*0>FI<}WxgCpH zui?IgC?WzE7$_oCjU@rO^!@8aAMT%Y|DRs;v_q)CroY-OEulQ$`xzq-ney+O&sT}u zRiG<|j!d-t)h{f0g0=2c2aPFd{t6C;PvNTr&36!5Wq}Y;L7OjPZI4XK=h>WjaFytL z^_&R>HEC72$t-;f2Ya)$>#&4sn?g7(!LOXHT`8Dv`vBn^^Lzcdh~=MJg!ERzM=Q&!x^P3ntmlgIXV!S^hGrbuGW=0IQyxci#wOH-`4|m zJY+jHvK^&8LZ_sk3!A#C_ug;JUYHfC5Egka;y@$1fl>pXlUZ^ci2!%j>UDoNJJ`=- z0>@)t2NG<1RXik;q9W|jwR?mhZAeE9)_6Pdecj?UnZ}>Nv-bz<)FmabrIU_H4Fk3d zj^GX3;x}xjgk=Vmw^4Sh<_zAQ*-dw@^;@0l3j?TZSip^g_zgPCMZ=)S$Xq)Rmgku8 z^RDI^tLqa--m~fTq*EKjb|*#peO{8K4xYT+FR7X(Pg6xd{;`%-vWdPUxyi>Ugxt!X zz`z@z|sfXTNZp$i)L`Q@P^NB_=aAX&|-4cwX~P<^(~kL@UU9Dx$Y?$ z=(uFG1@-yyvHmwTKbTuAxR}HkF3TxpgJA{_yGM*d zAeqjnBUIdu?cgJzT{NI7+m)pnqUB#wxT*4d3&6o}pNG%06$FP{mz+d?C;s#rjyZ!7 z2GiyqRCW@wEPIg;R@E-}FzWvfO_a6><*RWu-_Vix-w6Dt7l7bpHnsn>i{@$g2l$E( z{|y_&(<~;N+zQq_*PnuTNlCXAKJiNZo$s?&A6nzBgdm|=ppURmrAIPvvVB*0%D)Gb zK$r(U;dHuI;wu<^dSYp0FTxT6jV*(yEJBC~)SLAH=^*{YO za4OceO!Rk_xxR2pFHIJ-B0<#h^SLZ*l%@Jj@uyT@u>9HMh}tL9xW=FfBxC z(0SpD06TJXI@nf9J9D7DTzy_S8=k$x=&F-U!gPdXm>ePw9%cq=O5GOut3+j$DX6np z@!^XCewydM+sHYW$2HeXuR^!W{Edm#5aGT)5G2Qn?~4U*6OXs!HnLZW?6M(7F$rII z{qtw=cWExF`b5V<-YXIP-v)L>Bsn$4PW)G2SWyvCk=V6K%{3>g10pd7i70#7iU*57 zmVc%oP9sUuUbI1`1iV(Zl`ZaxGsL%Ni@9|&t zJSz~nz5aF)yKU{2rz*_L(GoBhEhVYtq2Kh#&x24W=@Kd519GV?dj3-h$(y>i*M_ui zOsrD+Iljj{)1Y5P#rSy-(WZPie~tc_sM9@0>)edseJM5@JYDjYr>6V$=sb<^@STPZ z8G94w{l@EB#}&?7H_}I%I-yWzm6>e@`SKNN)90+O1JGR0dr0}Q2C>HBKy`W0F(N9M zXzG=4QJzn^FT`oBriRIKd5!+hK>wQ0yMT)x`uFfAVcA7CLbVSZJ241ee#qW2PQki> z&moo_%itk+&qi4IY(YK&IHzwE$Hk@;lh?}QUD%91%I9kWOzvOEXsGpDi8 zQHbRrg4j4?j{|F0?Uut!iT6qM6RUMU=LmM^_F)K(^!B(P>Ne`!Nr#q3&&y&*;jgo43#OjBB z?OCQNsfWp$bG08zcwS09+K=t&x+-Xf8cH*|X3}0(6FzK({DQ=S& zaIe|xPgC1Lh((prVU=F3jnsv~V{UG8wL+_%Qs1<~5g1^s64 zu*cl+g>^|ON&Y2)yk#4NvILYRHt&p_8OXpVVH_Pu%ca(b?I<<;$hVyC7<6fYTSqpva zrbrNK?tecJ*Otae6quSuiD2oGEoB}R&V(@bnDMHOTqgKbn%pH`l^MA^Bc10~7KvHr zA6(9X#OW#e&0;@~R-{{Q?O2)kd#yfLXR0*DedXr|(%75svl{o6+_1nWAl#E0&j|c2kK?8ni zdQh)rticQZLOS-K!rR*`C*;fB=h6#E4GZS){IM7VCE`{5q6?ax{>^M*`^B4VLE!{8 z|6c7beh)GP;AFza8aPfaZfXV`@UVYnufG4T;3X862C~LJr;!TUiER?+7k4tM?LTi6 zq#>2PqC*}KA&(&1Z+e|gRG1x>eb_KysdEOpEZ^*LQT0X>id=6p?u*G-XuhFQ>3f3^ zh1zXd3miNpQY1E70*T;42;x&|q^=t7nRa2%hT*@NZVJbrUYK85I`GpD)8Ys8O57H7 zy2-6wgaoM^3~o@}qc#hn-TNI~GAsz%=h2qO>wLi_m=>K{#`#p-8iYfj1M6N>N5HN_6pj{|1qznVIC z*Wa{bUPt5P!?wif-Lf8UJ%B;@hS1RZ5C{!l?s@=$PVLEo=G?qlP@ZTr= zS-hgDr{+WQ1WE>@^P}L9pSglgZksgeq75(2Qo5bOW4C!(u2oq?x8bv)3+@D4#{XVKio#t1 zFO#N;xCALqaV!LlG?)8-&$bl7h@;5$sE0K!)JynqGswxzU+D=swRJhZ;Ka6mh0&OI za9vo)DsJWBajlInSujvq8f#n%fw_#a?`7j)CL#PUmkM&o!bNK$osG%r^uwPq4Cs_I zud5Of1LD7~kRih0Gh7Q!Q)j_h5U2r`(is?Q*KqJ#IJ0#1hO#jM2{7_>gKp}Kq&5AX zD<-;DBY_WSCXZqe|C)Nnnsu%putTQi9U~G5oC4b|s%E1x@JX-XOoR?I=Ra7Le`{m? zv!;SK4dI3sfq>-GOoiU&%*kY;)9BnBbkj8BmN_w>5svXC(>Jz~Ri_I>NUEK_s zcrDRC_^#_WLmO^Y9~lRIRCmobP6;5LBC=ncI%$V(5`*a(r6zp({U&8h6LYqMtx74K z1g_Z$rO?7G-}UEThn%l}E3ojlPjtOgN;_gM21Ag6OS;3PE`eT#1;!EU)R&R-bg72g z)-|pcp%1wYQw?|t$mhqKDBJo$PDQcKdBz%>4wz(w0iA)sdAemH!(N!>7Ycg)p!@#) zZ@MOOn`Xg-P`R+M1h;8DgEY~EkB`uwZaI1#hBa7bw6naQBf7+7-^_+}he*-pD*xx5 z{`dXD~&JLaGgiR580VAU)>2JG5$E;GdUX4n zFJI8H1;dH8N~~!g`Unxr2>R(l{jF$U?;_}RSP_I0)5rgf$p7TBW@#9uJiAQ2SU+oJ|sSBswGi?0C%;}xZ+amWL6!H_!&fJHL9p{SZj_Am@)iJjPj_N4t&?ya#QkeHk)N)R9hq>?gidRh9ZR zO)m)_9DEUBSO06-?K35_xbQ|hI(l!y6q~&tbxudqyImdo$w~g5uZ(KZODlR>@7dY& zY%x;NQ1*=KimI zoC473aL#GoN`Y@{ujusC{$WH*k(|rpyAm}zdire3fK=fK`HV7#js)6XUiDt4eY-QM zTH!B*MZWrk=(gq8@kK+=S7^5{AVKw8v2AofAz%t49%TTq9y|K@5gORpanckl&crsp zCUC;ncYylg(ztVWF{~mEDZi7cL>#Bbmvnk5CRy5<=FffEgGErv5C>P|p(K2l^qT!J zD4|kw-EHN@dTMFLwo&d!UT!ieMUl-9Cu`4K(%QHRLLw8^VRDo0p#5?o7}EH>%k!yxPb6;1!IZJ+8w%o|43mb z0*ZOsmEsLI{~xq7aQ`P-%! zjYQ3qHcz7bXI%9mdJh1UaQ=5=&WWj(K~o z^ElYUdR9Sk$z-lbdpCXe#4C#0opo34hbV}+l)ygyU$>}gHvZsE%@N18i3DBmsCoZ-C_u~NXY-W$EYA>kAe_-M zd*54F{sf}6c*^Ab%y)Wv%qJK2*zhu(+r~9FMO5LczYQG-;$r|OdtlBrsczXX3xnOPAuIZOnI(j8C;x+(J?YR%yh*803K zYiyvRdHfAgv0mHix;#@zG(qq>jhf4&WX?rSC73sFDSv*KivdRQpUdVB3o8(l zHKR96JFqEN8AA9L`Y;9d(nplyBkXGaR(^68UU(#Kun8l7;z6{p+3~CGba)+BOZ>lp zn*TyQKa_$)A4p{58!7YaHee0PFG9`HBv6NPzVY zN#+45d3>`U&DO8>@fJ^zn2~n%9P{?Q?qQQB_PNr|Chpcj@4$aM z$j%LzGU>+6Rt3Mr&)VX#634qW^CFZXm7h=B05N$0&JsN{F-5?_mbkaBat`RQ^mxyi zpg3uK_Fl5B_UG(6X~XZIS&AQY!G;0(}EL@dvXVWH`pIY-*sfbnN{z2halEDZ~buao|~f$+%N zU@f5ZL87kvR^nNryPLkU2GvgGY0x9-K#%y-?Qwbab8VB~Fm&Haijeh4?jO`^GDx+_ zT^YK?V?@875(h=0{T%dyfg@mS!o#B1%efZ`?U4j^@Bw?43Dp#RUAusq(Wx5T*Pd2% z!LnAG+|o5LG)>@s8);$v252a`xM~6V^J$lNczxBHX^viKCzkt_&lJh~X6SUFa9Ypr z?Fq(KgV1coSE5sR)Q<3YS2efH(R|V(4iu^Oeaj~=lQik?nwN9lyxaFmHC>eJznwu9 zPDCr!QX&^W@cx$2J8{5HFT<=N%41vpZ_VUAtsIT~GK0a%NOtG}$}*Sc#rveh>e?Nc z_spbfp&VfQ)?GmVyV$`-T`cGZ6~I3JdGJ^h%|@Se8$uT63Fz`9-ImLyAG4Ze%0M(D zxKvcIyA@cDPT;XMDGt+;+xd)pkpl^ie^q(xip{Aq)V5Q2KB#Q1bQc!T^=?FMSM}u| zTF!_aLGIF(;?u%6VmsE;x|FsuWMuMekq?kJbrEjN7UrbJ=acKN~2*mwo%$}WNTk1(uK1_COS zqkzg(?s0;yy~b5hv8Ok2lfd6FCFv{-xxKKp{7vA9Q{8(d-aJ-|$j};}Y9X6Dry>w( z4RAhpM2s10L!}Oh5PkjY2!U+nH;W&mf2tlQI*V!-J_yRJO(~iReIimN1jPQTGa;mt z!9W`3$q+YS*f_SI$4QJ?$IwKlK?k7Hnyh8$aj|EeW@AR~w45F~rW`)i4UZw%AcMpS zD9`asBlky$6NYY6)LhK!}Hdbt)`S)NAKw@=8gXF@-*TMHvZOB#>3Zf-I)O30 z8wDK|kz(jcdTr>#8afUH6Ys`cxk0)vf=Ov;`NK_wi+SL5pp_#?|LECJL5Hzmyz&CH0A!nPh0`85q-9*Zdchl{Ia5E;?A8``C$-_&PM8{>FQ=y9B?sF zp`d%~pf?;AiE4<6{bZkpF>dGijt<;xb6lf-K;e7X5*FOa0~59c>NBwER6 zz*mI&dPg5(^k7i{4>ElaC)Thz$-=YL7sN#Hl?3F0-XnuCWaBt^R}0M_wg+AOCeTw< zkwS@0?elVUnUNk)g+2#F0r|tP-38DWnW=g9g$N(OqdZ`{Mt@6g`R6d#`VfT$~mR29qd^u(net#-wB2o3H$CAyLUuGO9L0^$<&-_4npvM zy7oOmFH^oxI&wDEB2}Wnxp$tCmMuXxgyc)+>*%rKxiH7654v^|FHAiHt92B)ukWg= z?YpV8g4TWf9D;-Nl1}pu4|d4qV;12U&rKg;w|vo}|EAm{>R2!$K>Ov2eTzYN{n!cm zW(HNWQ;e5Dnpk-i=QCIbbwsl#T3}2I)S*LBBMfQ${!1t_*1!G?#< zFRyRi7vBQ`af%&@#C9poaGR_(b2;FHuZyS54T>aMovh)}X`PWNt*WKCZ8*5UWej_V zYutQ*{7#Wp%YKa)9bE*?Dq1f5yaF{YZR9GwWob)|Xt6^#Ta2pD938@Cj~iuL16d&7 zD!|Rggi`@nMaM#uflX`Pv%GlO7(L*4x3$3grX$NJS@%Gyf5C*V1-3s+3slfmpBtM2 z%74St?3YOQ&zN_)e6-2yZnTiG;^^|_YN z#Sl1HW%7pO6ni9qk*jaS`;#y1DY#gg{VhJQ=d!Qgla1qnlAyahb14tw*Zb^v`YYw# z@+#?rf|tNQ?EsW0iY7UfWtJJ`dOx#h5R3q6o9uliPP9QDdyA(mRDuwx(GZLZ>gXlx z2?+6(-SKD3g((O9;fXiP|jb_jICslYgjgB}On@8WC|inyW+wDf;Fg@&QE= zA2JPCcTg}&3zp;BVw7?SSWEG+){LnIFeUyml#_N zsKQ5yf|`Q0-iT~=3IVMVMLukk>!Jgh;N03xhbE%fJo%TguMPTWJsu#iIY=ds`;m9N z8YcnwFPt#sLxZF2n$cu)hQ{J$kI@u6(hL%_rjs817uwsQ$0DpE17>T^CJ6K%FF3xq z{n#qBvHspHn;JIID{dU__iBu%o8lAN0`IyvLU&N_25q4)w4d7#?c_8!fmIxhhUKlD z&I06h=Q8JL5B(Ub^Yq?VE~KZ+K2N1{d>WH4m@N#OfeBVxUHllDP+qBQ09C?K-K{(y zIDy1#$QqtKY@)k369bo|oT^|E)D^;)I;Nhf)_kp=eh)j~(II2?g|IQsa-l;fUr0Ej z#A8n9t5*8wih$j6Q)L-w4>G0KWUD@!FEDd8OgSsGfKC6GN?IVLoh$9gz-f~OkzD=&(?o5 zc3_$})f4n=oQ*QAMRivNDPX`U_%j6qk*9e-k~kXh_lwm@?+MYO{W0ej8Vo|ab*({quYYf_vvJ!Ac=W+CnGCx5`(NCL}UtQnL0F)TZ z2yd{STx89pxt@0iJ7qaaEm2x@xmVdKab1s<769pE-0Zx6bHgFX!h{QgyGGl2ry!G5sY0ZNmtb`P z#eopzQNBoXq`7o6=&5FHHcvEi5)V8d_aqUC`~et{Zianwvx4!AhO9mH-LyX8cbJhG zIgc{i@YS}^`<}?UZGL4N?XaP%A~tjz%~DV$fK=95*K_1_zNR6Au*ez7ZwexnZfpsbS&3F&yW=tn=G&BZ>A|4G`M>2cAzO{T zys|X75CqUEBY_+(&Za4;SivaNw&RQ*EF!_a*>mL{9k_aeM=24|EiS}h(O%Ol7Vg~< z?C<;ye`fYC91i@5ZcdM*IWMS%R591~+1AgI;Y2aqdySFVwBeJql{foHf~ZT^cjP(Y z5v)D=D!LEpe%iYzz-L{~j8^|IO+=52)QJmr1PfFIk92OdiPwYGR#r1HTSGlE!a(jw z2Kq?Lce(S5jli0yki^-g7?P(uur@n{&;D=0Xe+o_@`}(Rs6JtUm{lv6@I`|xEBZ1| z3a=GYupBDR=2q1uN=4S@!H3*Y$yK&=GoG`puXLjz3l+US{W#dEGVd@N0|XHD&F6=-nZ zE!0BG*lwc zzB`dW)wx=+!Yfu4(8tc21z(zU4laSAT7(22Tf0i1%QzQ=F0ZKSyS=u8)TSqn!9+l> z7!gBAt!%L7`8b-#Ny*bH8xL6#Sbc@M>)Gpb!In+S zsM>c|)p3I775<{flawh!g>a|zIsye682(VTX9_dJ+_m*#`CJW8UcJ}L3{zn|gX78U zt+q{tf~BwV`Pkz00^f}M0?s|MR6Me~jWN~mYFY(dMQCBz%T!FUONo$Ci|Q=OXh<_B zpf?q6+#wSBDj&-ZsZ7|s-F?}(wXW_co6){%Zyh_Uz~+5QydiyHHk2?V1iC|p5Y-oM zwSJ)Q27jmDSarf)y{8Gels$fOm}NxB_qOj9)j%hs4U$k6aVcgIqkj@Tqs6YHPh);X zjnM_&vkAJybGw9fT98$Ao@7gG--q$E_-r*7Ex=dG#6(2O@noWRUa4JXO5c-HJ4p6B z+Wer*|4{T|&`H#Tfnh+JiCCp3Wo`8t)ursP?j?8NBfUJ<#pO;dRo|g_5VW#TsbSMpAn!aV?t7@|G$1~*dA20UYX zaETKAH-71V3SuKHboL23?Fos9&cX-Ylaqqa4$$~NKfO5wa_1KBf{ z$ZtP9wONq+h>KVDVT}_WQ|GU&CZq-v&}vo(93IChu8jRw<{FH$l!5ltS8(<%P%bpH zCV%WGf2tzhvOH-fVv#i5jb=amv`<<1^v6?pR+|*-HzcEw_!w~aHBAI?O_%}uMn%Z)yLL>Yv9!v5k#(+keOi3H z(QZkf#d7;WcEP{=tAkX0W`C9=m50iqQ(&6UiDHnA$$h`@O2bNG5ZLItVZ7OeC~xc$C;S9k3WM2xgmiv%d;Y29wnbQrhp4=I-xqIUH0n&en-Zh#Cuzg^t* zpVzCsDH(M0SvY-)UU(is-W~FLUZ%D7+j6Nr4(Sw3dtS|JY1wC|S%Xpwb4i`fmNoa3 z!?l}!tcxK?38YOzJ8}*+Cy;F+E4s|qi^;BK2%JyhZaSwegtz_No}Zz#jekWmBVw}f z0*KboW5zfk#Pkdm`d(R>#U65Np7A$ZjtPV;b5Lz)8reH}e@UIL4kJ(SW?!C-f_*ZFSXo`g#mGitutgBff;LEzX`(deK)$ z?Tn_#XAV1GQW4&cXfIVGE=nefZlG&8X<%hTfD7jEF)nJc-)>Wl)-4lK(toJ;Cp=W| zTQ{1Vq9opCG6?*_K%YQuODVdQ=zL&`LATpa@MrsI7y=K*Rg!GJDnN%RR2b)o4KSz}i6@x%Yu0?<2EL>nDIUd| z7*S-<=xPx^Bg`-eFs8^0&c2u9VFmtyH#IPFf9 zK|_H`g52b8Uw6g(kma3pjptqDIMPzkO=87CDD@imcu0gJE$VXT`12jT(MqP>sueuR zdmR!O+6mEV>u|5^HT*Q_sy)y2@}GT}V>8`pp&@p;W|!>%o=X8^HP0EFj_JC2@r-d{b!n+)7CT>9^#OwpMGq_#tcM0X$uE zg@BE+ihC+d*KB6juDEYAUXoA^E@yz{ZLQO^DK%rTLzO ztzi(RP%Awwq5J~kw%xCI1j83Hqp@<<=)KNAWpf5(G>W-2ARiN(^eni}eDzwqZMJ#3 zZ7SQnpipZCsN zo8m=Jc!R|J$`7+Ie_Ja_+OHCUPV^rEo47<7i~hU20a1v74FruTOHOH$@ii;P%@S4O z!qyN6yBj#-*a0vZX-1#VK>@t)kmZUa#|ho|Rb$T2H34f~cJ3&q{<@N@w#!#O z|KC#rg2X_&hltW49Rn#91rd}kK{|$XjuJ+TK}abEDBZo$AqdjV=-^5|bN+#iYZqMHyZ3(8^Z9r^i?R34S^TwJs_qd5aR9Dkix3%zdPl+Ke}WB01r z2i}%@qo2kJD6=WlW5)yF-8!+%bmM}|G}Om(Yj5T368oHs9aFcMrwzWn*^*X(y&V58 zdLBhk65uc78QD+PRoF!wP1R?uD9vq5Sg#pbwa(P97$#4?$KzKM&!+qNw~3qHcSd>< zp7|SG^?PPv9=%LOlzo9J65QNFO*0{jkWWUHvm^=BKX*SPj3Cxg)kpwNkb1m2 zFcV_A2OU^4Dr$S}>V98wAmAJMzwii6N z9dglNvXTP71-oiLQ9{nj>y{I*B1b@8+$z-?7yR=mx4gN32S>eAYE@tj>6FSy!){aq zu`4&-X%~==prfkypj&=94^Ns#y;<*m=29>F!%OW&tA>q2f8gK4dJ29Sqi4RL|(JD(b_pg;^P^1B5h^|wF~nbzYj6%p z@E=7#^iFX*6JuM~+j$;k3q;gTGQCmz8}Xo!R!FrnA|E;YYGHlJD%I2=d1&|I0V<6* z*UTqwpAyv0M(w8S7!T!zNGOoZ!!eV5gVv)U@Qm_cfTdwU!gky`f>vg{H;B9cxC66p zim)mopIU+-)Sl9mHJG#|rxxD1p6tgH+Evd=lK9D5b!$lJ;9C2bb%d2#5?{Iz@i<8? zTLI7gVt4$>EMNE|=3EtQ*J2GQF*s53<-Vg3rb&+VQkQyx1^J)Ig7@B)piC z7y`d2jfB%5wMh|*D@rituXDOalaUn-U(2Lr{Z6y10bd?0Sy>LbSC(R`Sp7A$no2b8 z7h|2kj6$sr&!=q{wUW#!d!eBnVTo!yY%mQ}CFgGag3Xpp|(p&FAvQw5c>U%p?a z6Nh_CDQ1M6zo|)}TiWt5Uo^h>I)E$gmHm9Ouwrw0dZ3rP3T1-@FBRZ3OlOal(PnaH z%~-fR*?be}B#B)`QNyllSR14fx6myA!Xo6@H2LKPv`uZ`az^oZ1znGx*gu6L z>oS){4H_aITzK_eLI|6Nftu{e#eT_d<=&azwEoYwwZpIP9;#CH6Cs3Ahw?SC12^!J z&91A7h8{zHKdoTydRf8 zY{RlucW*e!qkfHypHxxsfHh^UHC=*y4MYS1LteyJ5uwO8-KJEas=OhJM{5e=AaMNR zchUoiOmc(2g(v!Dk$2eH_tj*DkOu0rjz7;lRQhNJ%jIQ1$EUJ(CenT1I?d zxqQ>A8#}Q$m`?4FUcZ+n*NRF!n!QEM7bB!CL2Dp{;C?NcNH&cPSWLWuSW>N}-&O5n zLtnFiT7>;E^-gEFSjsN>@SvHteIbpzlSV=Mvl6@51@7jD@wEOi`9Nx(&JxH-xE*1e z;*k_boV>IPP6St@5 zzx~~E+Vz}FuY?DoJd* z(NM8Po}SmFe@wkhy!-CXZe9|?h5F(&&kk6d#JP7QYv-bC3;#u>=7kaP*M+~5d?@rZ zg22B^6hCTtpYABRN}Xe=kl=cZmd=9irlo89+PahhW}g}J9x^$crb@&^8g=E>22`r zizY8cdh!&*5=@fhAl|6FD>?8l6rUrDj%b=1V6~7QYMkSR*CG#|?@=F{*DjnlpOsKr}pC2zI<_53uHbhUE=lFNanVnBIKp~KWBFM1a;h9rY-To=f zqm-|dMN2%{G@=*xdGyd(ZIQ{&yeD^vYgvg}+^o?$osA9am^IR}r} z7RecqX@0nOa4k8HY>}UV1(1a?GT1aw9P;>uJwVlPg4SBOYfry=Hb`Th__lYl{bUEP z0oH(9bZ3MhZm$h7aO_W7y}{SRcOdIDJxQ0e++=-myPWwnMSl8}t{}pX)IG2O(nzw?WbEKW@vpgP$J!lS)mk_E{8#dHI+(%`(s@C)`j8hV!8DK#icAUXA9 z?nX@l$M9{zAj6jEYC>Ql(?Nr)LOM_f8@A-d|5P7_@01O^Z0#-R7OUyu9HqWjG~V$9zRkN#8| z*!-c=9&E74^6RBz;gYeG-ZX9kMq;GmRS<>F<74W~b!K`lEY5`7ZQB>C%mCCgBg(@I zwVxNA`sC8Nxdw7fFV`(KFpWCC$zCx7PiR-TCeCf9se7qvE>u6$ki-RMV{{E`4r3MC z_2&8AaF3bx<3;aexchD?+*-VF#-*pxXU+p8^fj^^-($(S?5?Fou~b3S;hNpaD01t^ z8&9LA3-A$67qzS8VmGQL>GiJ>&!i}fV$49P&7`WwGEg!901Y3od^X9{zBf(S zUnAPBmGcpt8I56!+M-GiD_=Q^XDyCa8&JwGARSkeXv4doHAAM*9ngj)x+iStlBuCM zcPYcB9m#s9d6m%Zn-JXMbe2A4`&sv+2=u|7HowdB7K{?(s2%W8yt#lSYyT;UKcn7x z8%#WXs)Qq+-f6DR`6TQc(l`4)zdaCf*iz5++>!$zP*nVIza~xmY+vDewR_;aySbz+ z-|Pjx#7JC8VBu}4u|>pT;ncxj85adx{o4I1*m@#|@z)U^CMv-G5^V+xnlpyBNUcWs z!Uow{JVIJ=5)9IlYiNVMFq_O^A`JVs{ns|ElW*Wx&qrN>;=4cfP)`O9M?gJIdhBxs z08mXD@qPhEizaVB?Yi)4XtHS|sf z+}0tBK@&`_*2u{aBQ=Q^BZKFIp3sWMQyzUiRcmK_^?qe9DF(;WDrRvyao$jL=TLv_fi?3OO;AB-mpm(E9?N$jrt*peU-KUTxATe z!4W(nZEQ&{8b0m>=cJBiky58dcVJAA%ejUb-+@w!`~DkM#h_7Bi0j_U(3v37r1iYu zOS91b!2&oYJrKDu*fKh14y*BB8AnwYMY2;Ea#i2)Iu(|7t%vetvo|>t~(bQ}0 z7?ULQoZ#HW-J@gQmqcFVdkgoTRk>71muY+7>v)tqP!d8GlroL(BQADIhtCAJHRXo~ zk#Cg*_2w0}WmOwDp4lBO2j`4Ek8okY<4hYa#_`s3&(M3~li7?zhZGnK0ap}HKf zvxDZpl;Cn_64qNrVMde4oR*%X?2=jTX5g}ILjN!rB8S~{QU@>yYc^%kpZYY_3v#jJ z29pMg$UG+zcKc0o?uZ)*8QbOMYnl1)cL-5QL8{7AtsDH3Shyoc4?gU{nI1YlQ6U-U zgo+VSxU~hB7q+=!yStx6jIuf4U?wEGNB^R0V}0!ngiJATWwl~}390;10zqu8vvCL` zo4E91M$j(?-b7T`PVA{KVG%@Kht_8zDeJ5)wKgpMXuuFmz9D9amr@DeVRK$h z%fVUKpJ8ycOElBtN@$vlf!XBSFN0$b>8)&}{!u;L_lT$h2nOc%btMgdsIrzc&b>bB z8-=dRFf8od+bzaz)W<%LG`V{U!P*Q<*ri(n1#POiJ9;Ocsg^?h z&e|@T1l!n=IRJP->=QURH*SXekB`3Xlv>B5@a9Sr=6C`ARC$dU$8b3nh_^i73U?!Xu8wWka{_5EM&18!3yDOtnw!f?-?V$_Vi zrD5aQd%HhUEzb1;D}2(=s$pB3Ec9LQrL``zUw$c*C=T?9_y{kdDI){^p_H9j^Y*Sv- zU53?|Y+D2gw_SC8p&+-jrjkLpc6ay?d@z{td21 zd;`_=wnx-b!p3%uVNxDPyns6}e5s=}A<91UDRIw(g zS@!WTuVJTB9fxu>Ji0Zlc@{2J|8Ny0tn-%4^_0l)?zK#NTk`{sfwEZ?SM7p=fin)2 z3b2HJ_1@C%QBA$B+EqAdfN8skm z(=lr{J}#RXULstL+e1lst}gHYswmT}?qfvJzO>on- zia%Mbi^+blENvqvXpf6Fh?uC+e-xYcU4w2{59-Dp(d6uKM-~HSW2vlZrt@mi{Cw3FN ziy7zLzrSDk=QTjAosuy@B-&>+zl ziTCw^qk`LJ78Hx3b`TLyfd-&k`J!hb_wNQS4&(!MbWfpKxogY9gvo>99R*5^&BR`! zaMhxjp)-fgTdK!@lE6u4GVF7}M#9Rt(LHEz1>akOZ0!KI()N1y*&UCrqQ=}uHFrVo z_nI$&t|L=dSDYGZaxr$dO;EKKY;<99&f|b+fF1(GhWS4ancn zz^{H%IGWT(&$bFI`H=KI^w`l-5#D{`L_2lgVf&vAY z&(OzKI~hhelpw4#&xsGn&lcb-A=72QX>W}eHvOiUwooRQ zTH?OAlaTZ-RR1j09X(R`7+m(}n3kE9y|Q2UT(QMBo9JviJ0ao#c?5%#5@zh^x+#QxE4{#2dbYX2_ zG$o}Ali!oEWmJ2FDeq;uysD&_y;;NQ2OU}hUG90`xbw?IC>c%prpdP3QQk2LB>eUj z0gLAT_V4!xqhURqdA>T_hkKko6XBO@z^PQF5K8>~D>?o|k#~uZ{f9W`D*T|C{^gld z&jNpb+a4I-?YJ*s;eu#-=J5H%}!vQ4)rj{=1F4>G(%J_;6yJr(4J z8b)A>D(yZxDwmDF$WlHp2VgbnpJHFFl_bWw>+bq@Qs_T7h(CC=Dl#nM#*PFl!eALG zS9$WSbDS$M=VAe?Kkn_mG4jSRyAeH1D2~4xZg&Kp@r~ZrCkZgFySDmKOTz_QaLKeQ z|9wlA9~esYJ9v2ujhkOeS76KbYH@{)i`0gky4FS{Fg-@bVy&w5pWdIORMztPxs1>y zdWAKkT;iQ^&580-9qSiL0KvWT*aPk@NPpy^Ta4S zNH>~FN6qg9arU0HpPrV2>%He2hchRrkl*qaS2gu@6Pe@{10^JXx%+84D^XL{ACX6j z?AnFvrjdJMnU*t`fB4ap-ZMErDGtf7*jwcxU%jo$vQ}tF=AGzg-E~yc8J2A_@w?P9 zK9(-gkz%tlkS!9#k=i;{+r>#Bd*r+mK=&|VN0$S3>MeS#Q04C;1#1L+na3|$mW$~0 zj-X}fn93cR-j6qA0FP>Bn^n#l^sS}Y3NWW@{Cxy`4Q&pwqTSuP^j6uP1KiiUG-L|f z+D};Sf1n|&NRw1kEJswgl|Rb7zEHVf%F2GXBj*MRH&V_Qp&b=Ob6ljB+2rN+9hjbn zAF|f4jGF+x$gx2pY9hYZxNr8IU7l3@Mxl3LOB|kkH5wnd;@&}Ad>(weNI1Cr7;5YD z4|5mU)ay^*LNx4?^bYdKmxWkKK?hQ~i$e0-9$W&sW9P}vT}lKBfa))kiG-V|0Yk+l zj>H@W&K|e`9Ss))J42o~il7Ya*y~NJ`>G)ows#JwawSkeLl%hrJljL$V}1jorlnKT z!o?N>xy??3K@h}|3>{x!1lvo%3vOmi$*fi6boS;&X8u-83T>|Hx&miJzD^d?-@Vpp zh=Vlh6tAb2pAQ|0x?61IJ({KS7P&Q!esnzTFEULai|J?T@!?1;pPrq&IcfG4r|Fj2 z^09~xgc-j)9BjUkYUUSbI>2y{?+%y9>K03J*Smi-llyx7mXL}!n^PmDj-EXwYqFw5h&Rm(s3f;v^=Y1D3m)ElYeoVzLpeg=gk*&|&Pyh&&sEsAV+ z5B2mGL1&Fk2WbpWOE`NY_WK5Zi*coj!9l5x#R*6)>`Pyl=u%`UxHu$xJtUZ)woj{L zd6pxuj7{IHs$KJa7ao+%k=WvAm#_M_MEc&n&l>5LiGoQre--E$IB~IWtS26=7MB{2 zJw;8=GR|#}d1{B}IC48gzdus3Ol2tN{jHRX;e7x!SH5+hul176C(0E|tW1MDkYiyB@z%V6aM^^)J zRacS#lY2*o=BvN;&1HjK46VAqgNQ8ElOg4})G(jnDhc1S4!REC7!FSG3Lqq?oF4sD zfLi`-!$8|I4MD=kI9}2E>wZsQunrbD9UV1s=RC(=uCggVuA*6mG5-H{L2uWPr$f%jP0b%w2m5elB%KHg-V0o3ZcRkpWq$S1aW} zaC#tHlCO=Sh8<6PJRI7+Gg1B)tJh`z%2Gn}_1dI-t+sT%q)OW*TWf$zb5^h)QGyZuGhne>nlTQE8ROeko}&yG-bZQ*inOipGP-ts&OeR*tfk%#vg?n}N(F zNJmw%zF|)+WxIBp`#PTc91%Oq^KQKE>K{!zaimPvX~U=gbawgJ zLefMc?py+b0&#h~7vXB{KN9vqsX?E)FWKddGX?(LTM0+B&Hz0UKxql(W(r3}BhT>!5h~F?EkJ~OpPZLk17FdI-(qKf zCe4!PG-)@^K6r}rjBsA~sD<3gJ!puczk^AB4{Bo9%i?#&ft5STx-2+;RMd^#gX^~d zB~a|yg5~ThNns4O_7r_7Fn4OC6R@@V^)`u?*dQVTIS7YPCjRn0u#^yYlC>JNK;m z*%8L$CXgr_ja7liT{!Y&yfzZ^V|Pl^CA-Ec&Q!N56nh<=meWEU+@kcG*0J8s+zfL$ zfxL{lf_KR>8E!DNg1)06ShWap3}Qj9_F`zf{ui++%%w#9(l}-bb=C`$M4snCBy{Jn zcd_AWVN*Ry1D`B|mv~Q4=2Pbw;PXk+%z{EA(SH6C(jN!*dio-Q`E@=cW$*|Jkne}@ z_AB*HiY2)>K%?5LA{CC^)rch1OPv9i3dIPIdY(fAg~yt~u_S|db0F&HCkFrQQV+LvTiqIYnh z{CyxL9Mpn1-8t%)oC-_7Jk-G8=#n8rbbWl~uK7c2kx%f5s%>A@JU9=2VKZ3wU@EzKOnIyl2(~u7#4I;?_ZBgQFt&=JGSmHlHEikLL zn}yuxvfv@-5f5^NuhJ=(-NLo;Pk_@VOE#}8na;ZB@5rNch>jNtp&(# z-B^i4OtFlapmTAs=jS41xP1zYW2>6a;y9@2C{^Qm+{N>Xr7!Mj=EyqT;*Sign+=z( z6lsI=x#Pjz>~r|i4AdzZ((X!rVu!Hh>VDh30uL=2mzm;ai^vJKzS7-`UM%-c*zP2k zz>FlA%Q2{en7ZT41gH80kQ?(>XE6Fl=_GtF9)x=u~EFA6rEG>+78PW@f}z{$$W3=B(IYr zHtU*j>OV0(>4Gl0ADl(LD=5V_2`fcWoL7l7vajz*Lx;i_7hqvjrK8WITs7yZl3ksW%rw zrAAjhTPFr{^4gnTI!yGp_@<<7)Cqf}xp;|N1e(eSK6le!gVIm?0_TQ;We_~gJ!l}6=)N^F ziK~P4#DR-ZNUi-v)x~{ehH)*v9#98u(jeA1xyL#b^M;Fqj&iO=MMM37h&XWvN1Mhg ze-4ydHytFi1mUuK3OZJnf+e*!GYvEQzkD;!oQuhi+x*|u8g~pbAx>$Cr}@jv7g8Y1 zqg0E&mXV6TT5cLNCofXL&V%1SO%v980}L@N6YE#!GwJv`0egm}IP@>P{7_?y2>lR) zSIIe#m?j2c0MAa~$kVTzDdxcP;x_-Evkjv~U!R&awad3*KFXe!{vAUp&mCTqX&NqU z+ARumOQV!9j^ubI0(i>m+#_CMRl1THj?tESuG~7X9j!gdkd!yj1`Rehazo?1FS)-z z&nwU52ep`Nyx%0+xElL-tL&V zi7F8%V}Mumd`W(9$H{Z+&SN_y;>gV1K8`-uzBiDu4_314^s_>5LyrrAx}k3*Ko z$zF&}AW=pyfL?KChB9OL8C&K5E;qBL(fZk!q0@4SZ@H0G;u9?i9~I4)r9l zR=#o6)17E9Rq8$GA0^8v)LH_h_oiz{6~sF8G!jf0JGqs1*)PhCdlLcawleTPgw!v=62N~khH$=i$6XESYuDLXL;CS=p08*w2mF;l%jJ;2+1=NcPE9IX>r6;|lKQ>Ck{;27n7^&ArHKwh z2#xDIWgN+C!5PTg6ExGC)QW4L`Y02&q#sx-3SERONSq#!T0aFW1^c%J<6*cigfoQz zW3B53%RzTzekO0SZ!nu6g}cC!rh1T+toXdy?TX{R!NDV*fW`wQm~#^0+MrA!Y9EE~ zEUaHl2sz0w=lUZf5H2ZiAUy42akSuWa)1Qn-4jMC4(lpfgV< z&&x8~{0AM3Q6y_xmggt=ZusE%1i=zGYWyp!c1PwhMk+m9*~h>^JqdAv#*fTk=8$oX zG^I4QGqt!Rr0du$)OSQgYnDte0XnPNhddiYr)SJz(W|6&$~foSzi6tXv)Eo(;G^ z?$`HImCw_*r*Y_;O$bxA#99yUzRKep{FPm3e<{nL5^o?Ou5!4x1Gva$>Mb1#Ad8vd zrF$;A^Sh~S>E2?bwZC4ib#ic5y5;sZE>es$P%CU!-aKKsD_GN9)ln9C?T8qsbQ-vP zCEfQS9QR{?BX?M+zqPmVH`ml|r7LOHS8RwB17Y6g68+%#A1T9s@cCm|vE_=Vk%9_IZ56&+lka~kD0#22lFWTPqDwh4f)m?^0-67M;4z{FdQ5z zA7W@I#Bs6l1w-?v-nBjj{yCa!Cs@1!DXKmp%+SKne!tRzNPwid<9<2}#F#a;Ifn zcPIefso?4ix0d3f47sP!FS z)Yz#8eCmA9ygoWdhY&Z-`fw~VSbke)Efd=!uk7GNt{%QRCRE7azf2MBV0UIT=o5eS z#v^%nGIKb6N+!3-=h3%u3y|%C+(&+*b)e0DmigMQr024B&4j51cMidX8wVUX_unLn zDCFNs=u&5wbD&TkA@^81dkHLFa)};z4uo6%)xWf>Hec!D-kU}RaX!>=;M#?lZ`J+) ziL)BJ&a04H-&4o`M34axo`n(w>d9X5CGIE^3>#l(7v?_q&V#ak*oFW+yv2BZ*Hrs0 zDCc=Jdcpx$e_0cR_h8#F9@jVC^iID`P0`a|7fk)zmwraj@i3Sw#~GKu_;Z_b+}Pf; zND=b-NwTf1EIrEF?)Q-bgYtXF;Spt*M43)R3gjaBv!}tYC1p>sRU=a`JEX+5A7pt) zfEQI(Tt|lJTs4dVG4o7dQ+0bM7A&xwnisKmE9`um7H-uD(9t(`mEyKg`2BwLcdZ_xtqpW_;qBlI&ClHd3ZjsROf(o+ro+lXQf~l6!FV(}O|x{SwJzD_Xek31!u^azDC7 z!U3@0?s7=CV*>iQ{e&$`Z+{PQQNIcj!w(7Wm$x3WVn6lcZm~_rcJjSLnyd^Zm?-jU z%RW@S@KI(3xAVu+K|Xd-mU&9F92x97E7~k5vE}Fmk<}d>=K%E=!!w`Q2R8v6}tFPk(y#F|@$TbAUEd4{jB?Ije0J-BsMDqx$XTY1R;qP+vjYO!Q(UDla z!*(>SQ673yM4-XdSfV|s52L~t0IvC2=E}{g03yCeMrIkGBoFUDxFkM_uC{re33uA* zE%t)}1P7m&K_vmtG~k;HeqW>dSgggBt6P4Og~Mb^mK47xp>(lr>D=2!U#2@vRv(el zc+ZFOsvZ5r`G`PiL&Nkx_I#c0ws-FRX#mqfCVpJ6iZ{PC}=*ji--+1}^ zU0v^O$rehi}jrtU0X=N2epjSTYR9DZPRn>O?__7H`Q z^k2?VcFC()i@B88a`VF$dfZohzxEk-eBIM&pn}dgpusz6AKe7B-p>n!-Z#r>6SSTQ zfZ_nS>nqX0JDaqB?0eDH>N#Bc-?HYN#WT0$Z~a!zZE>#jXQ#u-(*3}*nvc{=c*0jt>lzZ~U z&SS9r?0&Xwr7YEh2T&o$jdba7pUHJvQRc1~yut0t{ zC&#<$qx@tM)0>oTkHAjqMOUsQTspV1P-{|kQqZLll5ls6B)gc31wq(Z*-KGs@1GRq zzfOkNGVwWsQIeAH?4D$Fhm@%WK`ekm<{Oq%w^};h(5UQbW(HFlp+_^<`W~q3f(L7^ ztcDlm{{{S7aMtnc^jyF3*tpx zJIJD5yZd{8HlR@M95-M4rJoIRN%UU&{E|uRwgtvQR`##`C5q3y7`cIg(7@b;{>`rb zFVV~@X)4E}-xpr24F;5`=MD4=zSfE)|M-BHCPJnE@}Rb$s3jAi#LY%MSoqGHOJvFe z+dPH1p9j25o&b0)ML6rGyswiQ{Sj{)f(GkD0qGjJ-rVCv!t>n7<)Z(`d%zrp8LTTh z$XD-huGTxL5>+1so2(&`=ZNUZN|+)I}`!x!fj&Tx;v{6x-7A?Q?iM|)u4!;V!Z zu@u0hJMKOqHZ9%#^evJ5zjN!Vn>p)EHUZ0-x{QJNwJ{HBlTGR=XR<9P+QE55w;;7V zRs34!hLPF3D;CpZ@Y9C98aH`QL$mGQLcnt#=%hL9WypT|_|(P)0L~XHD5=+0K6+dN zz}w_bE2L|h|tu}-FQhrBGI7^> z93Ov@eLCFf2dD?zp9AZp446@~WBIdB!pMi-YXkzz-^9L02x?>rCAX^$E=PZ-eu;D2 zma$C}wSeED`QrI^QLks3E%D(cCaBUJP_cmq(CM)qTdvOm&KLc%4PFoUwL_z?DSNB# zfN?g`Pnrst?}pMu;^9Hm$*^W7mbD;VQP*6udai82J2>f_S6keAb$ za$8T_^e*Jt_T;p!J6-Ny9gX%Mns+XbOr_;Td>$nqv1klreSh#7(AfA-Gi5X8)v0a} zg6?);?&fOOv9SBO`{};v>FZ}AYv&v5`Jk7n-`N%QUnKVbhl5?r=K7U!y`KaOGt~An z`DQf0DBc8GkS(lHxYLd9*r%mf`aZaE-e8k_FesYGEwUK>z(UwF{*H&I`Zh_6_Xhx- zyw#4G;2)Z`j}iN&dm5L22G>klU)tC({$XqJU|(5+yB!1nQ8Ku#V7lqj9^}$lW?fh~ zb6>#?pc%0P=kGc1Xv5xvzvIK!U)#QNwPRQpXu&vOaGWm{nO$ zL)vwc^2TjNNo(ay;_K=lau{i>Q(Juuee;@Jqr?B-`=Ua$#uR3Jw_EGj2R<36F0@UV z#7+JJfbaW&y^6MWTtEx^W(`wD`^-SuQpzZ;7ep>4O9Z)vyOK3RA?3#GNJpaVo&pKg2n6+0B51sSSt zC<+ecbd$_~7#yEwnPi~m0tn^F)1ANCPoj_Br(|z@YOMnY_0K;T8uS{f4*U9$uXrF? zxz=sy`|j~z>W+`tU+3Rc^3RyJ00fa|)<3ha{3km09}!oaIpSNn3vH`?>AZuJf{%gK z8ws2cA-zc+YC{j@_6cMEH!s@fh9|cpH|sg>dyu0>qsYiA*K?a1VsBCcch@?W*0F%u zvO!IjDy)Gbqo`E*mX+4@#k)G4_>y4HxZhdaux3xrZU&xe%?2{cgkMP^=0zt{vzpyq z6p^R!NSa>ZLT0{9&2~@qrf3^N3CbJSQ!9aglPytHsIw;G$G-iq)u%>fVjFW8{#a_J zhYCHK>=hERPHYIDzof?WZ#@W0hR(`_$YlrQb-Fg_Nk!F7M6!@Wb~12Q1wC_Jt@i-y z7uxjL;4PPwg#kyoa6ZpgMC?J8i0*F3YW(92!rU6cUGA^b-g0Ad!#@8C7XM`!|Ic2~ z>_x|1^ftS{=}%}GPk7YZhJm=K98)8^YiDVkxyif&HL)qc^-w=n1b4eDxF%Um+z6bG z0Cfy=-`-`b+~HOq_$^$za&WP+l^B;2T!w^b{(v8ja=(sJQE<@<9v-e({1X*zF2^g` zr75Y!7`l5poB}dhU~VQMO-5)gM$?Cy0A7fnkXm!Y*&!x_ufeh=@~n0`p-H<|0;_Mc z3|+D$y!!ILb4pfIQGes);nHCP+Kif1j$5UBzg}g~yG0w#YPP55V}fUQ7H6K?*WYq# zYzpC1BaS?KB~?Iws#{4o?(pF(kxXwoFPFOO%NQVr+i-LJ`@BZ|W-xPc!vc@T+vIgD z;`{6Vi_?Fc5&d5;WwRIPUp$+}z4zqDv{E#&`!eS3J&31`hLQcWG3d&!TsXj-eMEAR zhMA@KqQoPw^GL_qaen5Tz3nCJw4WGJT=4tc)I4Cb$@sPV>$oL4g~Z{^M-+ESh zN6tyyuqBPMMtn()>?crrNPa8q_^ri*Dh_n*bor~^@_;>@!dCa%`kbLbj_pH>4AhBI z)5YFsAf0g_m+(t1YeR*09In;hu#HgSHmm%n$h%(}A(V0O@(%Mn@aS~7{&jbr3XvE5 zL?q=C-yR1)BG{b%mvtIm{T#f7hmVHhgUE%+2O+%Us58u}{lfmh`s$ z>UQ6~f24l@&!|--$;~YM-Fur3gBV)e)pipSNc5WVhRU*M?+QvUXb4J5CPUgk}1U8k`I|l`xL4?4+Z8E%ltAAFyA7c@R z2)l*E)9NL1I`K!LqI_gt17lGF!rd~UJ{}U^-O;=6I2v|!R^YQe{Y+r(HU2@|*yn=q+ zd!D)u-=}@n#=R=8w`yacL~;ga=VG0UrUD<8?mrlprLB_rei9NVz(dL*Tkbt?iUaKap^oK$zRciLp+V`34YGu~g4H>W#h3 zEZYNYpQ4`nn^B(fyo*r%w&zNF9<3E%P>Dl4xEOuJ{y{GAdt}tfiTF;#ljfkK8 zqVLvPpX48{;I0YRgwb$%n)wM@I=W3(de3TUgH3wL%skaJO}{RM>5-pI*Efo#G6Lp+ z0$Njx0Agv%d3F#Q(fpI&`yBl6kKogsIJw-Aw`495V}XN7xq=fLPrltW+gE;=Q`bjLm|ew3=B@lVD_%ntq0vV2|9QSN-?^T7 zg$HR=dgvDB|AD_JcY9-V#L@oP&HjT4yv@2l!2C1*y56OBy0oQ0qRY63Ifv`BfcYme zdaX;!gs>9EpK1>7H^4{hks$}LG&-xU+pq5G>7~0p^WSzAo`|NU3D;N>3;m6$bd-)o z0_((lw|n68pzj<}{@*3kitK<%bE0e0GOM@+ip z{qfonvxoiJORR{^2{#3qc0_i&mno`PCpKTUUVn|P;L)wXIRVoy;QFdS$-VLi&T%Dk zkmZ*B$=3(2zIXF#DxzoXbEP69{wBgMo>_*Cz0j*Qhwfn4f?xOSK*mwnu&MGOTN3KV z)6eJl9R(`>;Bz|R5U$AwEnC;RNxI_s%_P9+s7oP|F)|SCZ zFY8I&m16PVNc+mDIJazDG(bYo;K40GfCP6B7F>c$kO0A53lHv2umpEWkl-!>0>Qls zcPQKniucid`<#A#@4J0^ynj_d4aTT%@3rTgYtFSV*x$v)Ld3H+T|T&`pvkX2E6-cV zuAiDnwf)T~Gl#_R=brt4pvQmx11f|7PPMokk<*uy@40gwWjygJsEAaXvuIqTCPKcn zX}+1|*?hK>!Lf+TM4_rs7J0d25>}pJ{q}srCX}mBR(;jTzSGs|P>6uOMST=)(|@3A_4OJ1G1l_BykOfr`=wnLVuc7;|~x)f{FKA)jB&uy+oJZGgxQBFwRE zA!~Pzv zXFz=)>2-gs2vw@j>Q+L-RiI-w=xk!eJxJ@`r6^`yIB9HG{>jFeJ>&(M2wZ*RS*_O# zgV_&ZC}6KEG#2eQ9RqYh?Kl=5SJllJ$y$2v6sn;%gfl zRSkKu6s_4C7?$9rve*2MXmapx2-5(iY*bQQb`|X| z?MJ_c2!}(ieRgVENdI+btF#3+#HA&oi!xVgSCN|@sUOAGFoXDzRDT4U&9Ln>FTbKh zT1)ID z4FitqA1MfZ9#tzz2#)QqE&Qrb(-?xlYidsCqj;oV_xt|*U|Oj$C~R)nB|0<*@m68^ zEb2$n^N3>umo(~t8tQ7xZZv~2sCRSo+35SRi!BsJU1D!|YEFXn{Q=^}t)_7VLoW#O z_VOm|*rKXUnlRh%dr#RYEgzP+c9c3Y4mIT?#=;^fJWT&|qg{!3>?+@1SdJ(h+QYxq zqV6Vo@7;>bhc#>ns=Fb@yoGJiEtQUbBbir@k}2K3_kt!=h7GN1;6zel8PKVIM}_{u%*WnoII&|EXFw%K94K5z zBtFOVlqnUlORV;yFhRYB`>FQ9CcQSRQ+#2>qJm%8AV}SCq#dkgm~T%_?@_;?58E#x z*yFjAfmpNANWX8c8F08k<8I{TtPs9>YDYPNmae!6&BoeUL9J@05SNZDb<bdV|sHrdKfhbF`7>e&oCYoAS5?TVf;gjfYLd zLMM9sPJ`(66G5;EK?$Xjky7_hZbA4MWw`fPZ^KPy%h=fbiv^1Nl7o8VJ8wEB@A?r+ zuhaK>F+GEOZOeFTf-J|4wKsk2dzRN5p$)stQp2Ielvvi_>4gQ_0yLhV@vO>npk7t0 zN)e{`n(%}#&tnQ%b=f(!^uLlLx5B7nUB!={c84Q7&*(D}$kY%0yd&FK4~QPm^oa zCJ`-)eBnAUo9x=-e9=)Ys9~iiJ5MhsVYR`CFcW8T3Y-gxpR{^ne%Q!sX~78<88B6D z&c2SW&~@ABFVKtDY}q1Unu9e>rV=rQcAlP}@%dDBb#@ zv6dcwMy~3Ze?=6`lzhTo3yA};&++i|E(yJb+;@pyj zsBZ}=Bz9;p{GM1UQRaUV8#X=Cpzc|;0eBt@axtD?&vgTt8l{`b0 zS7{ZZ-!|kjVq%oz1D`B`Yzt^>_9J44zKiZrB1^d1i;GAdv)>)QMe1yuSpK1+=cGc# zBVz`Sv~PZ!bF*R-bcVsW!|{(<0LKRGC3|lSFvW%W+D}5Oetho4MX;s%(7r#>{!ZzU zvx~6OjH46sy;$IFS3|Z-oiJ=yxp|YX@m|q$v#6nQFz)r}Q^rU7r}mdHfChpjw_b*=W!x3LJZ*4s}fgT>Q^tsomr?X<;tJE{SZ>u>P zAWqdL$6al$(7-s!a&x*T8!1=P63D$^+!{$t3+6Ha1H4V9ftMWTm|O`VNarxH&S{Mo z{=TY(p6ExDOD)#`KE%5!mKZW6YmW1j!sIfcwaQbg@np84ffzD}&~*kxR*YyOxG@)T zR3XGP1D9tm8RdI10;ZfJZTd9A7z`{pnlSveg-1nr%8)x;A{nUH`U>9)Yv1LJU9mZN zh2lI@mtGWf>Hb%{ue!|WCEMi)ut5Br!;f3{6h&D5;^jW}N2D}8&Y&EXO2@10URh(m zuQzo5gx{6^vC~I+R3l{Za*`e)wAO3mYIht+dsRVk8Wlw(%4;;`X4i@la>v?7wU8dV;EiWbUO-KXckBYy$- zqSW%#NPR6Q@6$aawoQ`ORMUG<4C+12jPFLLzxOL4oE_hrZ%-3)pVJB~=uW!Bxa3CC z8F}w6zk8LVwY#Oat>t_oYEgc`z*+7hqkstFxzk{Tvt@!B9X%IvC_HuPA|{#1CwH1W4+7nAP6w+a}D9ci(&`!!kRF zt_#FB`A-@luQbttM4Dzc-6gII5)RV%-|I93A!1mgQR^0@u6VTEX1cUYK@H*#KuIm% z?gLTvuI*3tm^4>E)@{fS7M>l#wv5WDMD^a}@rGP4BOr`%V~&BM`PGN@d?+1}xLn7xa2P(#-!@xLM;d|%yt6;XS?<6+Zr_-%(BX91)djX7o z#-|MpEY>^qcUv3y%1oi=o=%x2D z;i5P1gpVVOm1*NFBXyXP%)VC_^Ix@tCq3{7|WVGliN@(NN)TJ;= zC~fA&n7&?NLqGa;;x&`=|9u7BKWUHuy(q(h#|n*oJoE=5RALDrXGXdlI{@Jh*TfkT zd~Nw!s{QdTR?{@<@pXAJwA%c9L&i4A=^0H#qk5D*sf3x3I>vacowqI#0dUYvw?fJA z2yqKOO)15@P1Q8e`|b2L831SS2Xt03FzlqYei8k|TB&I!?X`3I!NjzN1)zYGO=wG7 zEO1UXP-Y=8X(I(!=|tEc#SmGnLmOj{>z9 zt&}>;zWcnKaCofkJH?L#GZV;)vA{zI|iNQ zSghmLc=l5d``)<24ufQ3YINfbay^FE261bVxDg{7UBHA?hvzq0bPkal7pS^EA1xKjHosH-4-sCWFjt79%-$g19i#CrvJ3Q zf>YJiC-zqtB5mBt9eQjOmWFWoN$F~9y)exwnyy}!Zo`BwSs6Vbb?TT+^aXSiqR8?= z?76JeO|fP!+FYvkeK!5j4&3I7d=F*Ai-q>||-sqlY`=tW#Sxxm6>dxP}Jc*=F z3HQAJ&U42I1HA-K{2U41V?<;`%*+iO#mF2O2dlB0hc;}FzLy%1F>vwu*a0Qn;(Jh6 z{$Cx7fBkuf8LgN&+|1iVpZy=7@sF7XO(*Bz6lA-PS=tsmZQy@Ll36>CnT&h^!CCKi zdNQpe?z#n$QBDnb}_MCD+D4ZmqG{gX`VX(S;x@s>19Yh=Sccc;* z2`-W3J_-y%wxPp4X%^5mz`_pLJWyp+#hjw5H1OzzouSG^^1@FPOWgR_FkPSf9gG*U z6H3if4z&3BqJ1PQZ>ld`ZZGQ8CF`dS2p2mE|dXwjOe! z5N+OUcB3gzxBgQ(__sd60Apavizx9k{zw1#ro~h+%A4R=tP#9HDMWUOy!ST{29{_X zS?};;)wzi!s|6}tT!OtpK~8Yj3DBX?%K`=yDN5R4ioZ9|`RY>7y?0`T3XHIQ#t}t6y~PbkW2+EG>&?mI+~`K44tffV}>(Y8cwLBTL>`L-gu9^b<5v z)9NNqYx_8W)f@3j=ARX$LJB8Ficf&EWa#|1!<{C>^RU~%v|()yv!UucNJcO5SBOmC ztV=aM_1CZ6q$6ux-;Fr&&Czka)PLl3OKO8P{hzE=sfV>H=)RfYjskpD{7nDKSMHbIcJ2nNhkdb7;rz27^Jxl_8;*&S?iOb^0NR#&HwKP2 zZo9dnD0wk&zK~IT!GAh4`z;ZTXV;a6Spaj5-u&0Mb6JnAvX}$MDYU^3o#lb)A{*C> z7GQo`SRHB5shGWe?1|Awf=luZNct25)iK3MUyxOiDman9@+&-u}zc4{-9ts|3lm!^mqUuDBzROs} zT{=_JXTS>fOyXAs0IRNIIzNpnXSR*0I4bZ@oM(LuoUnN@kb{JsuJ_j$Z8ge^frb}p ze|#XV7=`Lps4Q$Q2QU2nvn@@>DmTCYzWu+aq2YRJ(o>UTtN+c82|naM#^k>!bJP0lciQcS-oE znHkv(C?EO9H{?=%LK0SgNAt4#_9-hi73 zn)B-=ceSS?TL33H3vrRU15bi+i(1s^ODsXIH0vhea(S(~5IKHk)OB>Xo=`GNpIt*2}&4h${w6jcZ ziImrrN`2d^7`Hkd6$2@ff#o_f#1s`FR?^^&;lEnfI%F||V`^z(fBGFeJn*$ka@Pe9 zDnk9JMr50Dh9xlV$}4MCz*j?lI|EIH<#jc>hBxhk!0k~R&NFA1uMaYjOlkXM3=X?o z3&Yz4$&nN^JT%N$wh;XuIl9BgBW($SZ||?hD&BS2F?)G?uLE;0Hon>$uIcV242kUG zl@>7_n@IFwPAFov9mnl^Z?Nlrv3?Bnmwt-+-B&S`K>@{aWe|-svHeU!@OfGqKB5Yr z>9u+NNP1f9jc$gmifW3{%>hw99%p#EoHb+Vq!w;CnR+~2yoGK=TA>XxzcFO$+Rv-KuZ4h_>nnp6#M zmQsCst~?;qEik3V4Vb;<7WUTYpmwFj%l?=cMW?5JE_>?Z;6+9pw|6LO!o3!R;AFIR z-^SbXGNRI)SjJ5C?K746FVxQ+4q+#rUYUy6zWo-@3FA6E@H(t$7y?^y$5R270O0+& zK9^lId2963jwWbrdr`!c=j0X9KXfErd`u0LQLPaj7&oTgkQA2U0+mCQNw=2)U*8KcSBLx z1OYuo7LKMLpt#P%83IO60SRdreWei7U24Z&cRDi}P@YVOU$6&MeIJ{@ybX=GYVNMY zh0u@Qp7De$XE4akIoI;x#?Q90nqD~RxLw;Gnw%T_fKT5uD#MTN>V4Mk?rHTXTi?pC zR%xk1GWG^-$?<@rKMZKsyC$P@289iHFep)r^r4Mzd)x<}q+1xVh4P5l^g?hTkke0^ z3Bvc!!?kFXr3z%|%uM+(zAop&g)>f8z=_VAOpbRLOHUbDZVyDOG)9f<*9kKj%j~zE zO~T&(j3=1!9m$sRcYkt_pqX^*a$ep!BN%uaVJ#ABq!H5mJm*3)M*V5^R{n6hh;3i# zmxUo}6M*KM!J9JZtYZYeQlop5mAS^6EvEHwY4NuBU4lHb;65GR51LDp2r+c$CoobP7~yX;JTVYKNqif~#pQ;H1sb0lPMN?-}MZtP8$v+b*!U&PyEAnQse8X}Fqy z5?E3*D(SfzlTQ*Is?UnRkjmZvXg@Xf8Ml#m6alF-tXLZZgM9ug*>EH|sqMmt1vOClrk1Ttye`deel=&Ir zFqY zsA-_`i|Kakvhbxp61hi)q!9U@@)He>BWL7K7S#-x=iIoe1ot~Ay~3jSKq`c99~#kH zYT#^}zVCPLJE0&L+`-hdFui!D`O|3H?{T2}ox=%%jN9y{gY@gMBHI;WMyeQtVw4@{ zk;Zo>tLpt8^%J~8!L2M+us|En_5R8?9T@pF3BP5#n|mfrC80cf)suk<%55ksrLv11}G9K1e4jX zq%_T=2F_U#h48S$K+BC!8I*~Dl(2vm|JjIYX6ZU^-*+X}X}J!~rwto$mLR!HwPD3Z z)!enQq4OPcBaHx1197dEe|{N zTKF*n(5b?yW5Il8?@npe`{ThcqX}HtC7|u@d1lm5rHSs5=fq96d@39OQ**h_tMIwi z^2lGWRMhLW3A{HJI#`Wes<=>lWfs~zR?bT2;^LRWmxXi;_S7pjy#?S8^kbG$?TX}$ zyQz>-QQjwK8?NUE-Vy4QCxkidz0F2rqO=x9r#f0LKEtn*cpM%%WiR^YJoTykEo+>M z)3LRvBC=Nv4nToX=fb!62!46{A)lYy4FN+8{xZ>zLJf!uJG*X`B)O^)y_fHz?V{(P zk)YXcWhcj_U|YF)oLF70o*I!vjz>!KmAT6xhwt%aTY9ER$=6}>C>~Z;km{=P4DLqf zY_~)I3&ED&2;eA;90nm{OunflF z6PNs}_z`YkR#K)4tel0+B^;-uV1wN)oMx;9fHINCXk^B(RPT-NbVQHQxz0ehgP(OI ziS(6Q3>4r523}gt;GWuA9Jz3JnVhWKW-<>nVP*+?XTb8uJN5W^i4gN@10yv^A8_bb zmA4bd$k$zolrT-?r1GsW2vBP#OkO&xU1Y*TBo$Ur@O^k)Yq^TUcH;wIY+d zGO!Ei1#EDQuSYyK&~lT?nSZ6y^O2JX`XJT5dWQ6sS%SR{!f@iADMByxhDL_##F$t_ za>a%>{i%tEvRdDdoYf#m>M&FR`MCQs^R(ISOFeA`%i`^i1A(_$He#>adS72)Y{&^E zLEZo@JTJWDpr2GcNjyI6j{*ih{+YD=gNyzzFNjSa&d1lw;aol=x7=R*VP7yhxoEa% zzLnK%&a1VX^a8#%@%$y=&=l_hhiz+QP z>OP@|!F*l_|G#%9{_RijL&dgxbRKIt37~sCDHGI%^;~Pe-5AS4?7b=uwQO1)z@lG- ziceQfrINdbA7@4H4#mm?Ds}4bMICIkfRk=L4U`)~i{s*PMQ+A|%G%%1Oq}hRWCG^* zS3gyMy%&&-GBW(Z{e_tFV7Wy_%dN-dxKVP}(9@@3)T>!wKK}+Nih`cnqG-?60i|R_3UiV3KRAy5zETw)BLVUl6UD?VD_*W+5nyTB zeMvLl&@j1BdL$d$1Jvo(QqWt2MuITfCSRGic^*$-iZb{h%IoT|7v$;WR>P44*L^ry z;5?f4kmBvzigV5JsWHZ`P+5wx4O~y(+Ua?fis0?g{a0aa6d6ecw6~_pIE~%E8D0v> zo8YP0F2vK#fBv|iwjF!tiN~3*hw+Q7G=eo!I$23J=MeuF1!(v)KnUaOib$xQN{8Un z>`1oQ&*xE$RyUVP9uz>+bBA|q5b6&UUGd0s0!tBDdHB^o?+s3VKa6z2E_yUfKtf)0 z_)&DkqU1+kQIj%BSX8)H?p>QY-kBWcM?0a@08G_u+FsUu%3|TY|LSQOm*Na$wn~66 zTW8MPj$nzW%|503@v#4;(*4iB3Mi2PgrkxZI+^O&k&6$k-*^nDr#~Mo9*Q zrY5B|N-0z95UTdZ`pKB^)(<24Y%8j-n_jZf%M5C8!Luusvg}LPDeI`(>uUN&LZ##n zg~&)Kki3fc+M3G@j?aJ0XDG-Y`h+Wvj02cFT%o-)rv^z^-Wm_(xMy4bb z77lfPDb`e1Gq0hySx+T~gnPhg!J4``&p|MC?{7B+X#NwV61P`pLa@rTN3&J9>v`pc z9_{&??6pF-S$NAXY_Hb-K+6_Lh*H`;51#J?% zA=}SYx3X}LRB;JXZO~5X%!|VgNqC?NQoW6+CShh21FLGzs3z|Akx}7^lPyS4ldRAq z(eXs%yc+lHh!kkU@vFxxe8w+z-?U>#htW40-2Xi)X1I@SW%}x;z)n%B0|jbnv^_Y9U?*; zx;bvkyIjGT+3LMxnK(v_ zAsbOg9uCK>#2WsKIO~7@vknv*pgDo!MOr#@b zZ%E;#-rnB%mHT1QvZAGOA-@~F{a?TXnrI_#Em5vjYr=P*`G1PYtD;4b+Rq~B&lfiU z05;$=nz-kXkho2ab+&RnHk^t2$$zR*dk-Lhc@T^b7|?ZSm%$-Z)nz&ct%t}GqVAN2`X>mAEbFNql#$F@#(Nc zgk3-=wype1s`2dZg>tS`f6#wH&;cwD z+LwQZAF0bkI;BmQnH(F9N+$1%U@NZ|3@y_ z5lr{H%0D}9Xf0O>EdYE-v9q&NuYP%PaaNQZ*DN0XQ^tD9;0|{N*0)>`Pov*vFoctEK04R~sR0!va&ud5-`&#E zzS23X*k3&B&db|NANn*C4}gi*6v9hc9~EWvp_z>Q)&p3`L3aL6|FDew@qz&bka zK9*W4ONj_jV`sQkvTV7AQQ*`4kj%cL23|OrcN&N{k(McMPb@v?v1tMXaep}m*G&%; z1BdV|!~@B04VRNNj~z?f$>!eHvt-z6L zllc6P#5CM!2V@-0$Jov(m+Z-KSyU6e3!LNto6641l$ANBDgN)17NtZ1i zeoo;e++^sN@oS7ypXcS3<-es%Lrqb0FJz=VA7zphJuwDV#Hiz!MqbK08$`r7VLP^d z#|m1x>of=Mes<&@dj6Zfkog+rsKK$yDT$Xc z`i<1({SqM_ULiatIy>p*k^7J949eBZ%xmj?kw>N9qeIex$D`>7Yq#|-P|_>TI#vBW zH~sR8j3nf~is~heF7TD^2;Gy21e~0})e%A=8Nz=&oB!nn18KnU@Njs2K1M4Az)sHA z_(73D9<4V$BFzY4h*3DAGy?DLL#_4r15tvP0lLJ)EI@NYR^wx7nHF4J?7EZ%Ry~t> z6`%Ly`Qvemms1ix>X@&8G*Ag~2p9rKY8nG=TYnk)Sw-}e8|78(-H#Sl5n%{{BDh&3 z?fS41YsO(jHlXrx6=v?-OzHNgO_Hxo6*baEhKhBGlJa0At=E}k<0p^vcV`db7L`5M z2L^2>K0@G!O5bB&&LRg(Ua}lPcBfaxmpv1)ImTC}({P8@R>snMtLrQ@vL3Lnp=K zD156gUm0B7dYN(g{Ei!3f)m^>Y7Ug@hjyfihmF?{3vq%qEQ^nc6O=o|zvj?C8M$BL-@%oZ8m zRF(6y$kZt*OMG75-8mfB()P$FDBS`YI?&mx*ts7eCb+7=sqdOX#Y~>N5*qTRxD1-C zkkdxDjFbL7CAN6PCGZ8MRzt*Z@X{2~Cs839`BD3MKBy&`WjoGSw9;;3oNK_fKSjhR z@d6PIIqdw7AM=ayVW6|)sr7qir(3a8fVg65Q>G^5Wm`45xVwMs41mZzdJ*;Z=R2$? zb2sU(bq%|B%dO~Iqj=ur!(GoT_@vG*&SWi`1zvw?ebY7Rj2#<6(x8s?NRM{!zksmu zH{5MY+oY~vv2Ou84~KIv0S9P~UdZ;tH({yD)Q5xd5w(ANfiYYuA*kCk?8f`02)BEj zs`h~*LQyBZ#c18VQK-A{>A(Lb|8DC8B%){Q*q_Epz>jD=X_F|QJd^$+LTHN3#n5x` zqi?9=6~{}vmcl1|%Ej}Xm1D%&Bue}p+<4s>C}GkAmRGc2`$hi#pVq{Uw!{_RmFMqhTkKiJU-&#n+Y;X zZIYOToTIbh4E&@hKaMhX(A~NW;#J)IOWeiyBObt4K8k1^M*InDq!qsG9})65Z7`L* z^wI-PfnCQBnV?3geulZ%)U}avgK^JX@f7OuFk%W$v$I^DzYzzm!LXaK&o7?lioK8CeoaKh>p9&VTjsG;#soE|Q*$4PXr z<=!nhN=I4waV-+^qNV2K2M`gxHi(e%3mtV2pVcE{&IkMewy_hxNGFLM5MFHx)jY*Q ztSbe@;@x-yV??a3BkRN<&30VZKuTj=#c-XIu?LDl!bvA$J0BMcJ+w^yCTTx+0r(r~ zRiocy3Rx>tBmpo*eZ8q@eMFrRq>48$-m~;!0`cA zH(Cyp4Ba2(DL_u~%f&kN-2LudrSLOb@4jUB4^&MHM>RPw-oyDvObYu05NOT{6^&j# zNPKftU-Frd$X%LYucC1{0bi;U7_5(opLzKQV zX$`BNK>w;y=fO4jY0sV_p##GIXyR{6SRd(Xm?O~Vtha<5XRjN0M3buPu6?7TtgZy( zQ`5{O-$%#ep?6&IH{bFqkF#t6wdrF`+xOx0zAc+Zr=y`vP<>1qw4BVUsbn*MGMVt^ z0ISHq>6^rXF-~g2TEj1g$JBl7?2DyJM3zrxpN>4q8f2njvQB~UEM?iM0jiT3(eSC> zsKj5R8v!MfR4%(Vp9KH;h~y(jp=$EJVWdn3Oomx8JEK?pT1yV0bY(AEs}}=ZzX1`$ zQ4aH51+vvJYCFxQ<+{d^z}VY z8zPoE3CJDF;E`-?;L#|vJP)-w{e6T^@KjX!O>fR&L{}`2rJ%00b*W?cv9MhFAfjD3 z5Y5A7jO;5lapeerVj!kbngBY<$QawFyT7!h0UZ_*QF#s_BFheyV=DiSj4vn9@eKW46#qeO1 z34nJ!7z{qhG_z#&HPqTCi z|Ke9$_yVABMcY9e^?yQ6sx*{J*kveS+!ijgvQuQ;46iD7taqa@*HbwG%_aOrZ0Cn|(jye8(w@4I~ zj42ZZSy;%+Ba~rL{mMBGm7RXy7dXKAn^P=f`i7a+gg5th{4n>VNWXjA_QYAjf**tD zTejMb2Wvn4I1gJEfJb^Eve!!nqe%0p;HD@OaX1#THGlsG>}cvD>egWQJ*a|Z&||7*X}lQ4@mRiRt9J9VnkVcU^7cF&qz2^Q-Z37#O^jAOVm%3`wtwPM_Ua)WP?8K12K<$z&h3|#n^sh1bs+M+6tnd zU$$Jy0Lh}l`TfLt^)Wl@{2>6<SFMNiaIHj6Ub$bgf*z7@kI5CiLn?1An$X7Bm+L!@(%>_1}80s>44QP3F zcVmxWSwM`We%J^}Vn__h&+j11JbLPm#yhFMND+u^sXjX1%yh0yK2U}u=~&}5eZC~J zK8mTuR&56)l9;*EcSB8pzt6B8O6Hb1m?v1EeIRImd|e-f@ldXbnGk(qQ8~i;6Tf|c zX*BS>dn;p|N7(P~TDyyWBHeR1F{>=@V$(hVQLb~VwAR8sT6z9^YOE6FB#TGBwf%yY zwfnNsc?mdXcJmYeChB!rAr_$oGmXJzSsB>gkdn81*Yap!uxSImVUtM=lny0HYVe>Lc*Ic0J*w$D&q<>u$acfX^; zZqN+<9=vnB2DS=?K(_aDgDB&h-M}O7yNY||E?QHCx;CwxS!hI0jzOKSVe_T6ZWPo8 z9WlG+TeseNoj1Ij;BIkJT*&a-r7H-Cy0!q>p3yqvg-Ntiu)ZDx`|Zv5X{!!z8}v2D zufuLzD;!SoPX;O5R}IQf`8E({;v2FjM-a>GZ@;m86zNF; z$DpJ^=W_fOA0Pkf0Ek*i78lraOER!k^nOVBGqbl0nSNMy%T)sQoKvFzwoAXLZ^2N4^;xRX7H>0+}Kt5Ka;qtAX`mf>-k6m&xl!QxI1Wjni zV?QQPyBtzbMfiO=TrxPw!&{f>WhhlgAp96A0_7K2(98e6ns#$2R7m~|pVPvNcW3A{ z_}RU_w)|hhOzLqD!&;BNI=%g;?QV!-lK34|_)#6n&J+l}VZP|&wH>6?Yr(l30~ zC%KCfh@#`#O%B$~8=-`ps{sw?%jGZ+N<2a~YmFpW!2Y$7--iy1qnn&|HXt{IMk?_z zXX3^s7zpS!&)eQCxaR}SM}rW2930QQIbB?(O$85Pmw$H9GxkI3%X;)n2{1i04d=aPrf0w<3|E^GWP?b5@cCz0R(ORSa;E?e6VrpKyKxwX14#9Po&Ywri-e&}u zD?%?I(yKZ0_vq@skm&#Zwj=!wAop|bc{(@z`J8Yxkbcz_h^)QoKaU*0Xj^B=Hw2tD zC}U>S=r4WtFjA{wK+Gn;8DG^l&!>+Y6>t^+g-@W%Th=no&T0EssG_=ePU{URfw)I{ zZE@I^O!5&W?n?CgmeRFmdeanpGtY?+be9Lpx5F|~Ie@x=_sQt%Ss?X|27*)sZ#@tu zmetp-d?|F_TKM9{r7%1ECpT74#vb_csa$%!$}T`U4vFrI^UgA)aZ0M14kM98f`C~g ztVqAo+oefMBMfkFUdM0>fPgVfwdl2gBl^Q*@6aPK(?BR^DDGN@G@tkvz?TwEqpmpi z1@JTLv-A!EcPb^;q~1e~P_1Q0(#wLAxt%;tO>gXWO_OKOpR>!LSk(GVRPO&KOs$3_ zX+58_OSp*H{`t4mEBd1VAOIi2r2r5@J;`L-Cp8{@<=<_}+q!{bk!wkoo(S_<5_5Ot(PjYh?+q>JUnYOq`8dg!dwB+FCMYlVzh9gd5xDm@%f$ z*2zf!n!ekS{$M5NJZ*HI`ST&Z`iK-pamaen_r;NE&dVn3Sa-LKj=QR3q;B_yUA`XA zk!(oEqq60yJ%>{VR3&s#4@4h#m`_Kf0;)#+@Z_0RZ7SHKb3 zgH^M9dpU6DZfJ8Vn?$krTmu+U6$`d;0vv5`MX%qo=w^B!MxW$Ha6zAyLV;-aH;21l zZtRtBVqU2mw9HI71MaN)z>}U~k_3K>Pc3fHnf#*WiF(N%BlH<$l1!?~>lql`c;c4~ z8{chMV4e3uve8u#C}pY_wq65WT>rSrY7!gsc6xNAaWj8^t{ZR7G!W>FfW{?`5B&xU z(b{dvX&>_fHv&M+QjG4bbXry<@hC`aBhJWSK`NBE7tt%v5cx8waya;QQGuXgUv!M(xRUDt z?@V$yBa`G&7UG^}JMwZfW&Hm8E3AR+23?42Cpw%-OX(e79a@Mb44BN~C6vFW?gUsO zi#WT`3r&drB5`681D-0=5l$oOpC2H$Mlz81i7l>vB}G)l#80J1*}U*L>(}iL^J5Si zd8E@nW&r?TZ-i#>uFlA_L*u%2i?#80QOnnsfruX*anRLBI(G8s3h#PZ>0JB zK0O~In7gP1*z9$C#Jsb` zb?}n%Tq*`}RlP_@TqFW8pq9a4gsalAC+jBuAT!NxiGVP zeDbnz*2$m*9xkV{@fV};0i2(KHplN#!z8sfeqf!xeAPVretJN@nM-wBmB+4LEKxV-H2qGtDJ&NtzES({K# zF@LBn(1!uWIRWDXaHp=C1St~$@C{Ws8MEE7yqm={7ZlHpWmhl0KLu} zfzgD25+wpA7I&<*a=VS!*^-uvNa*kP*5pc`hChvo9hm8kUP|ezy+iMYH1P znPPnk>XO>pmH~T>r$0@zQ%CABCPmqJvyd~zNXFOSD~AR+XJ?28@i{&W1fJR@1V-8= z$Ng5ZP|tA*j*t~fTM0(7f1(f{tjz}!qdiqhZ$BHwk`N!Puvo0vr)FmE8fXE^=a!Ew%P8%|qD zFy`<$&xgQ=r5`BOk(h*N-ZGg=t|@U#D9Ahr;C^oq??uk5yzcgeY37x3r=+#-Ko(%` z#(Rk=EYwm&Cwyh^|8Vx*VNHG8wt^r*lqyYnN9mySUPPpe2uKG3k={E<2dRqmCLKZP zJyhvkdanUOP3Q@Pz}ud4-~F9?j_}08Tv|`SFlxM>?kVYTa+W=!R_loKmZOqtPzri+cmYF@VAsg!Q?=I1Vyr;%}aoH+Vj}}o-IEhWbq$9 z&M?9pdlBtErWlYPV#DD_R&~_jFxZ-Wv#%uyB!&4nRX>N-`#1uw2BcMcn#=0J>d``< zBd-*O>H38W339KD{Ccx361>TliI!BU7KuaxerwtvyXCfD>fb2df$Z;#eHxFLHvoFL zXFl^=@+1#Rp{Yo_h2JCxD-{wa^e`9pPbfrWPW%8YTV;J@kP&M14S~S69L2Tn}MeH*=x#N8%ET5>OT4 z4=;Vdbcl{m5^Lx}nP-}@^*4v#(>18&;nxS=i#YiW&JrST!3|Pq~6KTB%731Ta zK|Hzih6Bg9+p>;s# z3~h4pti1m6^edZ*j_0qkXQ50rZR^!-iTf8e2Axcpj=xt>&Xm;I4cqKB-XcX2z6`a_ zE5Kz((O4IUV&Ct=NaS!nxXdDvZdueC`ywW>yR3zt!sdH8N*`#;vdwg0(aN_${HEA z73F=4Xy%a-0%E(acKtHhd>Nn6ntEQY(3Foye2Qgs4Cf<7{wx4h^RVPc2ZJf(_R8nw zEN=a}-AgfM?x!AAi14{#?Hyo7x65Tvr;PI35zd-@btF;H?5Q7K|&V#7*34<^oXb(&FC@ zgxhsRh8FP4@}e#gf?2z36vDtRzCWvez9{;4z5KsAw4#1jwtHUALVumcYYNIAHFylH z`Sp9&WgN*d1pdmFKYYWv#p)nlb0_QBwtl?CfD@x{m=~ng;dzGwd}eHiBX<86rf2pu z8SV(c{%L1Wu&*+cXZ91K&AaW?)#JUXFyr=2#w-tz;we8YRhtzg8mn}^#+k77uZc`S zzM2DUM+`c@&@3}Tm#^HEhIuq%Z!L+6{uavg5bkDkFPyH{7XVhM-j}=UEol>BaW`-K z2K&>%<3$JkXorE5w#zRWZkq@19b5_&TB3l!^&3%IEIO!BfPx3ej4w@rWG!#SKl!(F z8ncLEjJEM3u`ksM`sOjED}ck&^G z7vIpah-ny2r9=vc`ba7hVXYYU7NI%hp@SH7$zuR$-!ilj0Sf&fj`&QM2^|(r#>X0G z6@z(ZBo^0RLe^s~JJDKLSyyW))oCccKd>wuXi3sy102`uXooGa);$Glhq44f;jZ;$ zRRTbNM_RW8eh18^`NVxcL&y7X0YsD^u}Ed0I{qa=(Y`}e$GF)z{ra3nN6d;9uuFXa z>=s@fVf_z_;eQfA|ILfQM=$7IKfHkUapZSPJAt-Im# z=fwtxYw-KC1Y;}JTV)R`RCynpC-4J3R_GUmTmkHLSG(%U6<#Jr^3J3B`>~g_?<5F6 z8(&;VJ-WR#^29g5{_9K? z3QPtDQ%Re^Uw*S>#EgWg0B9z}EaH(>DwbPO-n53VhgvR2MHI@(E}evjHJ$|jU~xFY zexd7E{&qxfEsZ4gx2sVZa`8q<#kZt%#AoPN$Mk9~>%FZO`!5zlZ@A%CyDM+Uy6Fz* z*mg|`2is0Y+E8vrAj+TOIXN)7^5z+Q85B$Xje!(SL^ql;1Q{jhtMTJZ=C@_Qdt%}( zT4NpjHNQgyZK!w5=83JG+~&TG6f`YhQ>=}C*~j@QC_dHceDl3jKr&#WLi6hpBhV(< zxLrp#2sBeFN)t&e&)`X(8tVC+%-~Sl>}=NeYa2H!ibw8XyzVyp-;R!N&*+&M{ruq? zf@ibY0W51oYs8d93`FP4$*~}UWJkYHz=87qj&h{6tgIaKE8I%i$pYpVyURja#{TTp zcgj_jm7{3CH?eVf20JtV?Y9GEaQ~yjhKcY5UsROpUSfNJN9|tW#JDUo%k?Drv4&VR z4K7xsG{7P~nfkK3y+_FD6SW%rB*rEM5cpbum&}2F1iCQ++DixRY1srA$$q=c2N2!e zGtXDd$k{LZM3KXa)%LvB#p0}7XNkn5>FklI)y(4j-cRK2v^ZGlR|C}X)CfjI8r?mB zDN6)MMeDLhT5ywQW{cs^4U>yRE#X{eq%U38KLEVr`!EWe*n6Kkqc`ZH{hXrv)xQ>2 z>ZrXsL+f2HLi3*0sX#`CN!o76v0c0N)c{I$8cY1JjE zj%T3u--42z^cD3ridhQ_Tw))6@@rakno!pQEYw!`<#-(J~n1vs`SprCgJkZ+uF`9ab!eo=YZxjr}xj#)DHg3^dnzxVCLsLM_5p) z8WgU_PE3;5e;EK25*VOTKyDrD|J5}57KXps){yBb1-KB%_TF;|oIZ80z9-55Tc|xY zv-(3_9Zx=gG!6W?I6<+OMHwLbWGE*RQgU)vt@>&d_Ja9oasG#;4SnDkaRVw;&C5&& z@h&{|C0c-=|J0v-fNrJtu8gmuVt6b4mvHug??L1;uVO?BKgY1Oed{`Kg@^u4^eZg& z*G&GyjQ`Jd{7)~UGa zQ)|*%TB_{q;u6f?N|~RZFQ1m4?j|TAqPGr#usnYJg3#2Tf5G!8eJGlBeg08jc7%GU z%|EW7|5(ZW|6d2nfav)pyOF`tW@in=ALu1TMc)_`_9rJLymvJ|I1W?nO6ySpX=Zgo4cF(_>{{WpjZc; zFrbJ*EZ>%{Dp9$S%ydC)blnr^jF*YeXa9=T$@uu=_HM4tRXGO6MAMc9DGBSn z?`3AlY3*08_c_DUjhxitwX)_J|W3w=bPFgQAMSIe*D^0k`|G|S(p_?tc=%LOLV z=2P0VyA1VHi`s+1vQ8_kgXte}E|67MiD+xus%(bsn2wvfhf`WqocR9HPbw*sf5J7UxF5YS0^HAQc3&Cf&$It~L ztpJ68?^pietSu}I;Jdm^HQ4^mD+a#H#Gix$*n18`Wf|MgUrPfHTou{F!0}K)lNgn{BL`sfLHG0?x_(NY^9KN9<9MSYa*Xdd;;9Jc>jG~Ad=tfSdBPD-krn?{}@2n-}@ zUM#ZHqNP>oHY)1v6C43la9cw|*EI$6m1%0I+{)7OhPSA`p_j#v(h_4qJc(c2MG3c{ zCJU_x{^Vza({o-rFArhDQo`zwOZE&c4n7C%jTYPuM)%}dgE<~O@~BnT(Vyu-SlBa~ zuj0K8P%pZnre~|p&mT&B{fZ^vTeLgxlPBAkK46+`5*7>6v80V|nZZk7L%!4ON0XZW z&V*_=VE8p-Dtozn-n{gPUqZ*pI&`O?#K1{!bGxdrTQ)kNEA2&nL1pi$wqaRKmA$LT z4sW<{d(HLAKXw)jw#zOzHqwS?Lr9xLUliVVGq{+;92So6C{h!zuI8wif%>KR=>{95lw*@J}KL+HSdq!RkrMuwR#=h@z@@4 z4&OMcvJqRn4OV>m6t43Xu~2Ol=d&Kkwg_7SFAXGeEM1=MEgf$T4uk46&Q=!NozQ19 zU@H5D4v#Ge-A#jjv!_#&dG<9hemV)H_FsS3U6Z_8BPg1oi@*y&?-@^Yie7H#O$>%| zswhx}?yFls@dL5B`bLHC9pRV~DyDxR)8ti~tyO-VzFNnb_t7Y#-GKhr((*o7Q?t5n zWb8JCD`rp%Esut&YN^h(+|ZtvZ4qNJ~dAPOs$a6e%qVkAkTtngi;$DlfSRVW7P zB1PaqVmuXI!;1*oS*qSTwyB z*AD+tUOt+blw`U1-PX>|t+uw--O$l~$V^PzTw9C@>vJrs?=|#}U61Cn1Is6(f%f(f zd%L^G&}Rm(2miQKCa@h&32}PtC2h=PxgqcyX@*;1J>WM>2zxt0rU^5hmy%}QZ;bUe zx{;@avo*Rr0q+aIS5|%r@I`9tKpT=7BcC6rsdE3#CNNm>tfZ=HJkw*@3)ao!f7A>v zo0^;3X<$`6T{s&Re)Hzdm0;N#&9>7G8Udz5Z!7wz@ObqrQ^&~$H94UpKVIU^6%CBg zVB%bhy`}y>Sn|Gw$(?;bVL7nXo&@UdUCA{v-Czd!Qcf7yC~xBUb@a@w_JVuU*)>}k zP2*tvoRhuMI>wXk+`Ud1>r#u-@Q2UD@D!JRx>_A&4H`=bH8TB7r_DDrk*fdAK3r+I zSW~ZhHj!z)UeQ=R9L^>{`j!oU^=F*P3XHX>B+O#R@4d{QM!Cr`mbbU}qM^sqru+Wv zT4=h>^mnITx`5`=va+mKkd~Gfd(`=X{a_N;^3uDr-6{Ldu=}vD!T1A&VQTN$O2ip3 zL7q+-jKbWN8bbCJjg50|t|>=B-N5ck)IvX5sTvIZ;65uZ1mcGu+UDHP0rFa?dF_7d z9D(1qoc%(b9jvz8?}r#9T7VT?`l!6kVMUV1tnAyje#g2O^J7eL;_G$xOS{FLZqCRb zhTv%$@9i9=ZnoxF$jO02z`~Lbyyf>lbLmxe`RynhA0MEPliOTZE~o38A=5>YXWwp1 zUaXxjIC$t7`_)PbW?u|hjjOo)P$y3b$GeF2p%|gNK`MGcVuYH8(}hslyL5FGD5nbahtBH>Zjq=;wB7vrVT^S9E=if)VmWj zo@X)W=zkj5N4uY_-aE6s^a$2PmjGq#435je{ZEo+`{A_SeJy7{91_1tb8EPGN?`f| zIO$^V&&wlM5&dZ6;*Uix)uzmtC}p5=ExmFm9CYyFqVgPtGpk94kPm*S_C>;SuM^n?pMwis`eOwN)7DJ&s;B5hn6*7zR4=p?eQj7an5wO7JlQ+coQR3 zxR>%pKUal3=&J1e5$T2i1X~RuZ*j0*LyE2|5Ll}IFJ>H1PyP=PRMoEk{Z~ho}n2DvkqCAC!&qHKEcmSP^Ov{;rnp?*vA3P%bl)0Q{t2Xpa%yuIJ@ za_e}I2eED_zf5=0r^w3Hw*vNXXd13)6Q=xd5}$6z968htJl)ncSo4V*BHBv4vt>us z;l=*C1Y!}=iduqSTGu7qjgNO8Qy_0-le*|vO2>QC>$axuyH0|C6cF+8 z?CFruEVJm;OGCV2r7@02L5=nl_dY(b+p*ZFw``pdmZCls1_#@z!b}h%zOF%Y@n2ilgQ*MRuvQoeuKCWoHbS$ zW^9Wlr0}qx9!rLuvr6H&ja{1>B)Ub}W#=KL=8+*hua_!*h73w z3`;_P3MDPusff*6m@l-|Q>ZO1p*wgUO98I7h$|?RvRw;qAW4*9NJ{hfy0E4L z@twU|zN64?iuGe7&56~_GRGWv9p8oBG_Ap$(feNhN5TO<;04u?EaKt`S}5^>6MhIq zjkWiD`b%T^ZS!+Z65q=AEA^cL9Rtg>^J^0PCV<2G__p%QA$~(-r`MC^w^$|JP#}xh zm;Ny7r3!BS-h$sDqW*w1j=DN`?NLIfkI0uzzkbn?sC)>?z>eOZ*8Q)%m=T^*=aq2< z9GU)?JMOCAX3;z90mr$oyE5SZ2rwQQzOl!W8?0CA+No(NhDyjg+&*KpaV*H>JdX}t z>;t46@DUMchN}qI5DjKbOw9G-s;-k(G!>f`gJ{qFNWsWKZtzg1=Xxa07aP>D(8_-6 z#jrym0dwR%Diy4_oIdk7i7ka+%{XPv$1e~iIxPd4KCs4}DiS^EcfcO1(CmQL>>hBP z`^LjXJflO*-Y2t*zYN;^(emy&q9Pz(jt;x}r7bmyvsWHF1<70NPbErwMPd7-9=<^? zhjtD==?gX1T9=Zy2Yr`axYf?7PsPTh{+KR^;QH?9NWiWQ^Fj|nCvC`&cS*L2>N4Hs zV{nfJ+!BYAGy;Zs=KU5Dqo%zw-!wCwnxPE~TlSO^l^g_u7ph7W%JP5K#Bvo1RXcYV zB$hSXD>U`la?w{1m|c-_yhiTD4slYNT^w)1it}Qf)x8F^7J(_|AHYQ<7l=k7zzR_> zy+<@&kDdg#W0M=KVqct+RZ8@d=lH$svcI$~Dl;=BrqE2}9y*E>*~M_#W_>ZJsP4A3 zG`*p4HWx#`z8Prlu8khwA7=f!vZnw!x)TBhwR_Qnc*;wngxI*X>RbX|7{uO^j5vCv z;v?>e3Ag6&sOIYJXb??vZ<;RSkfFS{MPWyln?zlO<-kx#o+ zW;r$x{XSy+TX3Hg{`rj6x_#GXt@8G0#G&^C4lr)WJ8Dxua7f6NIe{-Von`_^=R^dPoA zGn}O$b=@nrR-Lxe@5Eo}Mf+i9-z6J!&=q##c zJ<3VQOdYak*$jm>?Qu)FU@75(ng}G3t4hNU0w=s8PDaEWCMTDAX??y+CFMkC5!}Nx zr}213N&Hkz?BI#Vc22nEy$roaNZZ$c49r8cfwU9L#_o2>s2%;%mhU!te&0Aac0Bw( zQI{7kx|$R&(mffSja_xlh8@T6*{s;im)h4?a|P$J>G7UJN-{>rjbxrY<&t|IfLluj%$iYby?{Kck*QAH(PIZVrj$- ztgXN4gWV7Et%z0Udjp$dQCJ<1via6iM(^b*b<`0GL!iX4aDHz|N8IP}*^t@w{iTJe z&+du6{o?rYEud*!`5w~RC>39WtwHzPnW;9pj&5ge!6!--IQnMch?%< z$}0Fgi#C4w#?;*AaCD3n&QG^E<`j=m+nuIG$!)7Hal53(p+Z-I`CF>`9ggbsS9ukS zOus&Lg?UU()5^%onP{8ZA0$o_P<=37nVa9%tB%u+hjXZ{XY-O^_NJS!7IjiuCOFSH zoc@^bz}kqYAbJZaRzH?orKA(Pd~O(Iz8Tj@qd)y=vcrr269ol@dr|!Wi$$ZuEgy^~ zTws5{6bkd?;jitL((LNL*cK0*{Gqd}xX|rs;M=Dw)>R(`Ybe{FbRIhVfSj#H>K`bVr+`+^H82>-}=sM^PNq zZTo2N5ySoR&TJ1~wUB(8nm{^SCM7tRvBsZM#0M&?*C;qbh^;u{mQ=)%R}9Ie z0=eNv>(wTYkD^0Hl9Lv#`l{oD_4Ozo+Rc(y`z2Atbbad--Zc@+iWfA~D|Bx0m9>L| z$Jtq<)vtS~^b)Ic#V$37(}JxPYvAtgSPzp^qh}*fnYy{VH-&|%Qlc#mVY_|FP@pa>nrQtci_kHPvqd?;%CXWu0XPVEN`}N)USV1;Nv@|(`4YXJD$cf zHZ!;XdJ?~3^a_2)`)Y?9QH`L5E!Z7Abmh4ic(!jsUS`itkaBG4*Q;ahD%kP^p_np|6oBRfe)gak$s$#Jvj2F6TCB-MDtz za9_+d2}ZPVRX6gB4yOTTBk2lWgp#oi*ye!pVuJwxh+aA#aL?C2j7E79?j-BP8ALp5 zj(#o%T0{BxsO*b#PS#R!$lQvFka8b|xPK&8Dp5}4ws31a zc6(pUAfLePh3XJ(7jiOCZ+Kn%Q9a_Fu^(#}zNR%U0mEw4FUopv%%anM$*?LCuQGlo z;pnP7FcXUrR9Ks#hW1)e=BD(=%>k;;C+ny(&p{YU!i?XAdzZJ}qGfpZQ?$lANBl65 z+2Nrmsx`f6&di+v%2NR5vqrcO4h~De&hG+JUD2N4AMcBbt z0nPL$1Y3g;BO?f8FM2h+ay4Asq#kfNu{7#`E&zY4K6iBYIM_wtU@J}56McF1x5`^? zHA>FY-7yamaDxa~YC$aT5h7ixZ`U*0mu2`K_g{;Y1%F69k09_r4E~Kpd=x@$a+EiZ zO8^KM>?a!|m1_4tAk{x%$rV6qyxO31$ILZ8>SouP9uE?qZ1 zDP9V?h=_a|y4BmBE~@T-oUk;JbzXBiobPI7>4gCsGCDl$BtT_dL8Pwp=#0zkJJ3L- z87TG4=Y0RDJBd?o9bf9o5%E!{`6pNE40C|48#!nD6H;+o$LGAr1b_3 zM~R-MTBqSf-dw|P+@%1>Z$jK@+j}B?c_V$5b#r0m+P&=xyn-HCdG|Ifc#x+}yd~=9 zBKyW^#?T{EY$IN7vD6s--gpZ@K@bbpP|8uyXHEN_qoyO-zw`69Y)4}#`XP>h^XgkN z1dlBgeble+Ki+nAS|pXJ#?8J#jc2EdIH_+v^O3A(KKcm zf&_|&-4umDlj%>196Fa44NIPom{@bt0RLl2@9UH7Y^SU^XhZnulM&qY-&ZoP6e~|O z&*B*%K-ELFf_#?EzOI9RBbVJU=R2V)bdk^b`XtqJ)-?PA@jeG?vsjn(2i(OZhlpa^ z+68UMDyeyVndYBM`?$x9hs%dGbt0=jr@* z;QLXroMq}0XjcByto}F~zoFdJxA+)XWqdd?*Pt&$1_wOw1)F+9ra=QvI1{vr9ne2>g9Tv+04iSjtCE+_G^|Fig494e))Qhj?)Uq%tL-e z*viM_cnz~dGb=gbsyf)aVQPXLYgB2>A@Vj&rRPo8)76XYD^Un(dt!KFe@6e+iV4=Wk`geX z4g~h(jyd#KasDyQn&1~%6_>_(@jnSpmzzqO2 zo)Tb<%TT9lz9TbooQR})KBOy6U;Iosi^Kh3eiM5+(1#solwQMs1+|>4)*y{cHxz8f zS6j}B$%opMw@jmF0?^4TAt-Im9%#>-S+W@g=F#92gj>0fVYBr>a&Ss!*eFn6^DRiB z_N!ayvx6qNUaby!Q6ob$LuWGf7-U84je0O#1_Xu$F_8xFBiRvCNddxZ7l9b6ua)%K zlvsor6WLNX7o*E8ttF2wyk*-6OD5+%=NTv?aqu0OlL*&5!<@yf=n#jgNw30~iwOZR_~fA5&r%=Bi3El|&Bsft z^UlVBV1!o2>=#W9oEwD5p znhY`s(^`{5?^o1;-0iZe?|qq0?<&PyEh@#FAQ5-CN+e68n6JWDn{abIR9-bm3VzaK z4#vP_#Z4LblAMg%t4+C2m|_;v&I|!iL8l00<7?8g025y)pn=W8fUUP-^OGX&kv5-R zhcLzjb{Bd`QWn^K<=na^wxibd&Fl@bJPqvNK3J_f07lB180ATzQkE zvtgGlc6bFe2c6P?S9?z|6VP>3`9YjJT+Hib$T9C-Zm80xZf{C3JsqEiL$b&s#Y*f} zW|ohhMJ@Sya*B@_Y|knF$%hKco_(B8QtL;=F8MqHnkTNHv{cgempJS8$jy%HPf^zr zG(QC8+9P}d*j2`F1)BRrM4Q{2D}6M`NEqnZ3({@%6dq$(CgUmUc~Gvzm0&A#Zl?Kj za(oUKXRelL(S?>*GGGM7*+0zU^Pf9#UWo(cuo3G zN$B7Hq~Hm(Td2Fpozp`1oCWWpAaPQ(6Afk~3%sBB~9gQKnzD909^rCAe7maB;+*5q4Ui0Fl5GG{I^ZOIcM$+{MlK6dwC zDh}Xb2VP-aZ(7(amL+|O1ifgwEHSGuF6?^xDsFql-_JN5HS)u9{!P-4+!yw!MRVn) zj=^WA5jW0$+2FHJMCj?-0B2Y?+n0v=485eqcFr;jGD*vQ08~KBeC_1!n8JDE|> zI{s3G-0L$?wj`T$jB)=CGovP?at^T>n=uaXYkFLb-7}NdkQI4^xkc0kz%1z6bPa}~ zNu>Zj+Q{-?-aQV1Q0L4vlY7J)IL_u$0jsy^9CZwwy4f|a#}w(ftgb4LxKfDw2jJKO zSCi7p1H%WcPfmiVE4@=oj~M6A{T5ojl6AQ455($933tTd=n+~$N+jkEgWh$)MC4C$ z&}QKNB#xk9uWg+&=ea%e83euTC^v~wudF-uk<$~ny^R_RZ%W@o#S2<1m|{}ChUv&< zVl9&Z4t)a~ZZ1G}H^zE+^Hr~8)8K3zT97_1tMdQ0#7li4M|UnZ1|55O^-B5nC;E)H zQTZWJr;=Q2w-^M25cJHW9z&f@&u4gCJ7zbduJli)s9R#3+-jd(LioJ*W^Eqtk%7Y=o1&}ltR^iK^S46t8p7;8baz}KgqO9JpJ#prTYj@#oDh{<};oQ{bqvgPR zeE!8Kh#1Zy?%s>V2|?0uZI`W!)KTBUwyi@hfpZWmzPR}$@z5K>pV@BHdN7euuh`#8 zh2>U=Z#IoVF#ZfF5hC11GlzOcJf-_7=$NmfLxlm$^6>>@YiwvGHu_a`_J081PWNFP z?DF)4eduqLFE|gay1)@(M2E+qAm5M;nP_#&#qt*6W%*f9zgU`w7lv2#s6##JZ9(*a zd@$f!nD$`_JOpi%!s>s9$ieRiExN9UX}puN zO^<=%o!6E(D)Y2F;p$S|CDD2Uy=(uLp-=I7bh{8J#3gaGmR3esOOoW~RQ+Zn zR_Yq5sX!pILDpK&gVTC7{VDa(I~RDZjP2{Wf*gk4}ILTW9I-%}a^Mczrk6pH0?6Db?uTcT?N;PM0RK1NoN)6N#Ww-He2Xf*{KH-4)x^C0 zXfCPT$Tqg3(rRBIw!}@m^`Z^6c;jJ%0(>DA%P5HsZmxKkMv!x8)I|%LwT~{_k^>BL zZR?p0wqXM3v*ga0EU0ipU0-cSx_wi>q5s7uzFcCyv~9L)4At!r!bX)RywaeKvn5%^ z6bg-So39`H0V+WhLH2<079e`-b)xmGtW2chC-MdHFxHc^SR`((OU|>feu>Kdoj6Ja zt>|vh3|#nC(4p|i0&ZO%)Rk4aSCY$mJb|AE-NYs>mAN~?&

      32X}qIpfcZ9 zbQI_X+J}RFiCXKox>Y(h6~DWJj0kTfy^7!1isSWRB$B@oVCn0n$u^gFqz&8n@(WQp zd+z3egeZH-BO{Igtf*;RT#02rft?@v{oBWQ<__KGr+t#F%{UWqxD#lLQV2p{vYTv` z0p6h`&UN>d{g6l#mF)2UfPeFG>9-X?qpC3ghgLLW#nZL0PDL@d`Prx(P>HncYL_45 z4+vv1jC5m*0Y*yO{u@kmNA{QZ@8565&6vW?Xmx3%9pjc+ahm6@F>ZDKDYZBKLs@rk z@web)#^H&_wLu_AB-Rk@USPcJ*m<8)?a1^tY`p6>fjHnw8QbJfd3WU3$w?MMJp0uL z;hVUCt32{ah;TL!p&cY7b~V!lThTRSXENK?7ecl2Tqw{{jEt>FNFFbPfLX{9e=NDI zj63{SX6vy0p}$#D2#uI~w2w!Qt?FGa?iDcjU1%Ya zVF{E_!QL2`Ajqw}1a--|s0k0);?68iHCc#x*FN!FKh=a+Ea^{cev4`;x~hfshmWE7 zP;P-_diQWW_uOahEjym7bRfxKxt|fnL})qZm*NUOHQ&B=r%KM??PlElLzrlPK-_RT z&+T!;t+l2&rlGwNtEaVb4f|FAi%5J_JwKCGJNdNkX;ZpZGE?4o9(l4*{;BbECiu#j z&7F$=@u>wmIk>P8_C@qZ8<~h>FO%5{i$SsgRRA%%OlX5Wr<`W$yEzP(+W%Q=Ivq|i zdbVcU`9-6Cb}07*Th_gMxajj$xbCn?xuQd^c8GJ3y$x-U(i#r&B&cCpTBCN6W;`u362!m?efy@H!u*o5Fh zaDwpq$v{&<+3y^y#{Y{ z=IeN*K1#i6YpA#@*HV-F_-T5(BR1C9=G2*WrBK;6sKSQ`x^SPr>rl8deDw~lFX09W z-yXetk6)($+m#3M4On>MmK*+OV&kKSJ#e_D-6No5QmM+Q*}a66+D?=SzKuZsEt&A z7MA)hi}g*Uuc|0(i@R^9i1vG2A#;TymIOl5r}y)g@p6p$z#VHtT@`lU&ub0;Yu;w^ z1f$hFd&u2Z0Uibr4Qply(V z)0>3uKBMO1E&`^?(#ux+ppKzch4_GVs^3Ua!eGN|)qZ7halh39|3dh!!%FKC8s^dF z($Gf|;fiYzZE~R=|o7)HU$YI zmOW=CFUmijsKnP4w4ZH|{v;gOcMSlabpXeo;E|#}&t~8#8rPfl%Kq{6Z}brsEimFZ zHN`aZR1R!Ve+2-V>y8uI!ysH{W^86d>rfrD!}gWWef&<%#F&rUtUK?xD2Xf#6{JSTj{(O3jxcWuSlyW@4cH#h*-qySFKr? z7+cT$308j5qCsoQy?#zhvV+6m=;d}T^_Y~^M3H)VW(wwguq9b`wz#T(7rtyKbpK`0 zHKWUydMe9KMl*1_N!iI$r77n!j>?c*AzP=7<~Q6th0ivk&BcWBfUr=|eFiGm(%k4( z$;=%g)~WW?e)Bz6mx>kZfQ(>8gK}Shl{J>NgnaJ)ed$#!jfDc zilSK_$2Z*akiTY@;XRMM6SSu$RtJKwh0BTeW1iwz-K>6=6z%c6j^ zvE?I=z&-?}xHirDi4S+ip4L;^+CNcQBrJ}@4q(;z?D69t@7Sv?ds<@~!EV23Y8ogS zbH$wXl6}7Ry9O1Gel)KrACx#-ZS|aL29yXyGmnYhk#tO&vfde+nSm}NS`h{1E2c*6 z-{TG1dURO~eDb4nc`*&(n=+1QA<%K7dOP1%$9Y9-#kBUNG$XxCD~&2vEy(9YLA%qL zr-{IVaE7suO*Cx?+eWu|$j1JhoRmC<_@eqpm3yRd9Eei0n*6Go;p1HwLUm$#MTILf z?Ir&3cQkzN$+%t0F9uXovM&_xeiCFfuPb`np8A76dK4;|r6+p!k&1olb@`IP>T~+- z#Wt~TDY#|HRyD&7?8-E|-fEqU*S~IA$kN-wcldSuz>P7E>8o*vUt@%@a*?wa@m*NN z1SwOuJ_*MmXDc!(@39gybHBppJo@Q^buM0Y8$%HP&}lVDcy4qoAR@JaqSO|P^nf~J8HsfA<=@J zcwL2{2BKYO7pAVT`&3tqj7Ej%ALRshNOMA5Ix1-yJPGe7C&e11**VbpWk$}ciyw78 zXTs)OBfbH&cVlb5SsbFGqE{)6l)(_sp(r;+TCcZdB`#)uLviUZk`3KcQCCnYcpQeT zXBVO};WC&&G6jYJFQ1iZPJrOEl}s&uxp-FDesyg-vY*?5VJG1M&GZm0>E zmO9NLu-NC;aenqJ=aIJzV>>{q-d>qUwB7h2lIojTjvkU@Z0Ws=4-lbF1aGI@fSAgwDB1qvLmQ4`|5fbREtz!(Le~3aRdeqCW z1Dm`neF?VIkGRW!SM?|gDUg!?DgQ0wwIE;{AzJ8d_2we7$3SF;Z9ZU%*JFSXEGB( z_fOp$$TO=1E3a*TY&0DKGJd=Ozvw-Db)Wf(V|VP{D?%-&VZNHD(*gqecLeU2i2k&_n0brE&Y0ua?es~-9KbF!XG02H44-)ex6E{+C84$%X1_SLYiWKSJJbq6GerWzv1;Z(ynBsLI&Gp2>cYHgiSk)&#_wRJL zN#tcNhC5Jqf9B|17!@_?UFqf3AGvMG*N2a_ZOB^3Hc+~&nCSq5`00wtEM~n~^JR=Q z7E}YxRx=GvwtL_B4{*mrpF+u?4uqF3v|%olU9B4zi0I*4@jRmX=a<903}Z}>gvsBN z>5WIRsPU#_N8)DOWaRx5jvUxe!d|_Sd7nctpYE(qWnH9A3gqv>U!5q6CBNFv;D}rc zFTyzi2z}-jYBKw#ox+-G&(B{o(%F0vh5i%aq6T1@;z%vwAjm2vU0t}9{cEflTZ0@q zYJd$e9b-BBR$4kJW}~$YC^KIqE3C#zXM65f_Kxp${sa^URAhMm%kM^@SlkD{N&9D0 zip6n|oAl+=!kPv$B9WlC>i`g0K)z?C(+m*)E!C*s;_iLb>g&sI>V73YBQ9AGREN{w zfb+2;&$eoTI4=V?KfpjyYD>gO4 zZEw1OgvY++?Dg;Ngsnxc-VA;$c<0&P43tw$pNiE7Km0AVeTDPAhK?{73YmqqlrR?# zwHIH9R5Hx>tsa~0O<&znXgm*Gm5?77Ata`hO-=RO62~VZR{3d@rNn4b_))0!C}t=1 z(fYv~vVWay7(^7RChgH5QDA)mTA-b+tiEOJtY+U){f+*5C~|Ut|7rCSUl;NVY68Mk zjhlUvFX{nHuvwChOI>&0KOyKsmP^W)df#)~l8yfwa55|e{;eurp2f~f@$EQk zAyld^2(!~dU$x&R$WQl$xnzsIh#u`pb=_C@u%zVy=3Q9eEh#ax&btFqPJ%h1XWm_5 z)}}EFadmh2P|TR09G9wU-8)$~H)yC=tcfwC%h>tD9T%Iu?%|*6S2F0l#WbB}!_M8T zaTF=N_0y~i=K}M{`v>mE4jU>j8?@`9&^a)9A8fGD=`G*WYV0;*|7yNQVA@}^GN#ks zZ0qg{If*REx5}c=x1AG<$~(S^?+5ciwSM_vG_yzu*NZ#RDM93ZNf>DgRWlvZH%BbU z&s=!Ej6DENcQ(r2XhBR$U6lM@*!AD=L^2FjeW0h4bdu_MsD(cMl-@n(le1*tSHgQF z5$7nr?ji)nuO@$69);K(U7?2Rrv%<&L+7ueWrJ{pR8BHol-%@sF9uPQQAc)nZs+z* zCq^@LA+QhVu+R|P&t}Ms59EHaR<0E4eu@92`l67+@TZFR95jFTZkA<**qrOOxC7H% zThTGu|KaSrqv34ZeG`!&q6`s4A1!)IbVd!L1xXN{Ac<}my^9hxNOThS`M`yzJCN67YkbN%T^- z8wucF8jwquYWF?!YG8%{dAU81F5%;3AKV@w!$9eZ+Z2}OqzPv7k2p3t8%|uzx(%4@ zkR<(f=nR;o(h{&Z-OxpBAlnusRt^Cue-QPe^=7Idey`Jw&=vr8J9m-l>F=0NTEtZh zyd`%6ppB2Eq}_vS019n`4yb4?jihD@Zvyk&N^sCygYiny9pQ5h!`2S^3>bMVf41MA zPN=6Opu_sQSlH^2r(gmq7%+L1L8fBv zJ#Mi9O56AF>7dX(nJ`;XALn>L98ilT5GI*APc3CdBM9h71Rhv+ZRT1e`+~#Z(aN*! zylTPE*Cwsa89tQ#%AhkXIExq4jhnU#xivF|cjnn}?x}oVnNwnHlg(MO6NyJgc~+;R zv&48cNQrLLPsk^VHp}c1&^0(6&vjqs%C!5JL9mXB|J2VDBT!sRIdVrR(^pd|;xgl5 z?bkjGmz7~JRw7tjDxGJ}2QI9_!dL0J{ADfualP)bZ`-CcG;#gCdTrlX`|85fh`7N+ zpW!E)YpQNuI%imrE48lW?4p)XY-wTT9zlk;`MHcAprW)rQ0YDt(CB%pH#gFRI)X)s z@}Dc*Q5~>?1$;y4hfR7-rxx<1|6FoG#xT^gLQF+t4^wli^ZSp7R{YA(qNfyX^aO&8 zj@2w?$AvAEjbf&j%CZsFNY|>Zy?k`(^Uyj&%vbvIVkg`1yt8HDjD)WV6TnQelH@SA zz_W(8<(@lB;Ho$bA6qX`;4q{{*ipr2rOI_!cgN?s8d1PR({ZKoi|C-EKNMKLJ00Kg z12_Wo4*xK0U3r-mNV3!vfZfClTAXsmxWfq+;wCsPomAd9q)&i(pS6-;GXj9~ zt=zg0E1e+oOH#G!zI!-f@6FnoC4}u~){Fu0&bl~*yqFsMJ<)h$?&`Su{0jJ|CcAPc zmc3uZ3@ha0aB!gXvU+M}_OWW;y+C%%E`9(hXSJjEFFCCeVu!!ZCAw*lX)?o4P3#ttQRC~FTp(Q6ME zIV5cHrQM69EP$^=xp#Wq6Ke^)+lWL?7U9PPv-lRip`CERTaq zrs^lDyfGxw(8>g=iRI|jp6Xdxs^)!0eMX5S-n0WFroVYi!TiDj*e3VS$e}br(eEy$ z9=CalI9z)-RJT2rJi*!*46^}DPDS01sq2=ejj`z>FKRl*sBVi@`+C&+wO2Zz1lKe~ zC+|56<##Gtd7oB{Xs8nI*IqTdDyW(8&o|%qx9O(3#UI2u;Ym=G9EnZoeR1#6gKLY1 zmLtV`cKkxkm2exmy~?JaldAzUcS87>YKjsxO$UX0Oz+7)7cn{gU6es0i9@dQiNT%r zF&LCojoZ3KfKRH;`{4lgXgwM)^A^FKDR7hv6VJaY3}onWWZ#;+miBxCRzixFI#_kmDWLM3r0qk@Zrmq^nuStsLo^GW+wVdho|Z{Mod2Y$2@np?e+3b(+m} zO7vaxi<*1AeXSv10SdalQrR5@L=1MRWqt=K=TfybK#DH+{T)rTNboY-u-`hlHMbxglaw}Zr9vB~W^&r2>&fIM8Rf{`*~LMhOEa@u9(3m5 z(Q7dCfzmi>A+BFNQAb0p!w4A0>;M?773>Kn>^kQ9Bxc?BL+TU!t|?u|C(oWGWR@Ck zPKC6c0>TMnw~e2<`-03hVh6w&R8Xp3rU8`u&TyoBwuPEdgktXc3d>#x>9A9q5Dcee zT3jxPUIkd3b$_U##ewWu-Z-!D2SIH@4+6Kjg_v7=jgJKpyzB6iV#5Y`ZTUx((|VQf zKSPciR_;tbnJ`3)e7e!PN3=NaIHNK-KVx-Iq3H573K(fm|JpHmi<2q#3#WvjrN8#% z4juAQJkK+dWkL_)%U>_q4}~0wmRrwu#U~CF2F;Ests)tjnEKw59?J(3mphOG9q+gN zqJfL(mKrwa=;epHMMs$sat|x-;?i&6PBL&)j^uc0=76fnnKnH5ew=XYX|FXifdEt4 zIV7VBI!3hF2GvG7_#ULr9stkc;kiV@G88_DD~-7SFR}jL-qsS_r-;JeCNX}sqajtm zcG=u`RWio*EvEg%2PSrHG;vf1fn=oo6se%CM!&?G1x zG{%_Lx72)|V`yl%%Y~7_2%l)YA{Wud>SGyno zy;=50&A(5>Y5j3irbA|`8h^rh@@ zV=IC5cnA*9`o502w5{-IRy_SIOt-BQel$b71deV2w9D^ z>ZdEb#i8^d3yau(jgM&b5`R6icvWY)IXtkAfBvq#pscfgTt|ep6V;zpyD?E8`(_(R zOAA_5r^eQ=k?qm94AQ&ONJ>~_`1YUi6H9_t7J6oS-*`hCHh$Nyu>U=9HmJ< zOI|UESy>zw+*B&oY~BU?)cf8ZYz{BW)Z}h!XMjS-avQgTLk$roi4Tx_W%!G)uNQAz zSYvlaOOvCVz@4bd-T`?+!{4Ca(XQL2xw&2SILi|Ejf{+pmtr=hwu|)3t@^~Byk^|r zy?eLkTCUguh+UF>;%=0@9m?XMY!I^QlrvF&+x?)t^D-ZhA&%lPwZd2rh zbvQrWPikVc`01DP^K&{Mm6QQ76R9GpcvJqD0XLG9Z{XX%h{95-k6oc*HTL~4#%Zy( z0d9{b=H_-xb>B88>)*Od5Y2(%7@(X}J&b4%l#9`4I!_xYI~`b9pTW4KS&(}S3UnG` z7^Z_rc;8w0*sqv(cnl+vn<%9Z88)@kXj1Li~M?mM}L{vK!4@cC}nN`_+0emPv=Xzs2yF5iS-0mn$0G?Go`o3sF-36+qwjO?Y=>*%(_l;V~qgL-3~` zM(6n^+w>$6lO}02EK(Hj_U=LYUWaiI?S+Umw50nSPwiP@fg*a^$nWyVcV)CBz}HOK zd0gN6yLGhG`nULttOFmUc_m8x~E(7sB&rOML4(>842r%RNx+hh}UUZK6=4RK5xlvL(uA zwiyXs%S3Ks_A>11G!V^@O#A5g8zSzcpBW#Ab=*Q1Nauc}Q?^LfF|{rgB3l{+U%Z_F zRx!DX2(F^n63DD1{;2V+@%LeX&I+;@X|fbJ*w(_ z5$O1t;IdbJS94e5yXzp@mm1`Elttkyc7fhZR+R)Kb8}&>;kL5A+?+AJA7Jj$w@*J; zs(NSnG^97_g#5@eSy5ax^CDfMO#EU1Zhe1EBVdabdyoTIDLiM>5dUFP%EU9!MwFot zs7+oz@g{uP=?epYKph>vu346AR#)&*-(#8fMGs?WBMzX?`6Pj`2xs(#M=~PvTB7xr z-?rG-hwrUhsOW#s*KgpI;yvyywLw+S)^Bxn1>0~IdR0AVVa{wVF7fj=4;@1%kpqX3 z!H8nGaVf-w`_~NTXiS%~0@K;~^iT8Zw*QNr@&(Dz#UE~G;0CSohd+iDXK$k9y4nkf za^b$7nPY(84OvpRS9B)#Ay)HvVf36MHO%<`1nB$RaCBI^;r7HtFBR>L)bou@FiufC zm-Yq+{#?BI!ne{B^qc2R5j1P|(5^x*sEOyLIEpUqknb|X@}?R%M?03m?H2TNp+U80 zIT7t)vOWkJQA@@`;wbeXn_TTR+xu?*F?2Gg?CFcjc&VoO$icJh7-?~@gBoK^MVeaJ zVu*|ENjbIMP6!-6>N<42vum`V1mE74z>qwafQ%))n?jTo#?Nofg#F(aTz2@da8 zf4d()m~JwT%!W3_xd)6it%}Xu*g0k5-24*KaVT_;i#s!j<<5t{0ekN&P}u ztJ%czWOMJU$85v8jb_bxb!BZpMwLs){xBH(DX~}+adosZ%00AB0eyRu9G4#YBx%pN zX~0t~RgImqHRfEB!mfS$oyQevETi8QE!WYDe+DZ4-G>V7|L9>Z)+Nb6Oz!L^c|3E% z&L#l2tyQXyN0Rk0XDGQuDbpn$X12&)%S6KIDpuRtXG&`zysHzblad@~brO9PO>GVj ziy7%cuMM@dUrB?zH6JB#nMvRen2nCriDa&Hb9MqUFiSQU4B+)+ze~3wjJbXJdCWM> zL|be5;}$|xnTj#<#$RN0f*Y*D2D17dm3FGNi(Rg4^?e05tVSSt$pB%!{bNS-Da3W zpAP%3Z2To2JFd>#O^x~BoRN>#MQG!)?Pp}K$oXBrfkQkYmgGY=f5S&kqQ-HP6j)0U z51EO3SU+K&c!0PFaori3ZQRLV2egk~2nk!)?-JYH4|RK<60-ehTyMHcUQas)|A}Tg zGK{Ahd-op~u>GyRLBUjGMv%lb194}Mv^$(}MelXhDw#1jCH9cqF z*+|tM)U9slG#zDsKlwR)Ye)@`#ejv?@Efu+|6E=^$QKsCFE7_*K_4w&*YrTaGywF7 zgTqPZNz^8eyzZ_e7nhJVn4{+AOHoGrH7SuNp<0;Sp9?=;{mpkb1>;KlTEOcU6_<-z z?E?~$c7bA<;_B1vt7?z!hD*H0BIYN14#!rFf2Pa4CM8eRSIrP7sC2Bee;vNf3WfryWdP%oteUOvVcENR+rygMQ3`{*dNq1CliI-9A; zkjVKDgF4}2or_okETEqA&3Y~?`il_1->JCRUhh-3M|qk+?&((Y4O9L4ik=A!X8Ty< zz(;E3%mE#+{(;F+s>7N&duUem{wU4cjml$U!96ct4j(J* z+L!{E9`BNibHIiv9W zYrTeLJBkGN4HuF-?#WfX4Y!pyrB z=9wD8W{d%5UrNOJMu|8EO1i|%rgV=XqCs8}L2_(?bD#x=8tCfRPqO2?{3|oC%7@@T zn1NlMY3ojnaAn39e8r1etE5ObfH)o;+Oz``d4aGUot)BjvMgkk9<2lXe%R(v7zUO* z_a>?2A(1JVf%$n;^J+OVxy4RNE;RN1j~_cBJAfDkt>%2dumyJye44>)6aU|YOxhA$ z&%VtSABcvLiAcP++ewePh-tMh{hUy@9o1j)w$A|Pr^LuX)%nIhx{H4bx&HW(Z2tcB z#knI6k9dkTDDlmYwaIg>@o@H-$W5%3O2yVg@psM^o(uv{L6IHi)x>m7$5DkySL%hc zzT*4qutBcPFK4c|Rs>OFRSqwvN&J%|)=EF1E`^ETX78vp(ttj$+kq$wiwm^%ma#7TNt9wL8}jDjP@DOUxUwMf8Qy8#J}S1^*YvJSq0op90xM}{;xXje?e*j zrdXZ}#!!3>Vr%v@o%rhYwz~bK+=IA(w5jb9uGRU9d`;#pbA?y{+ltZWKfa6}BL263 zFL+!yHQb1f4*5r!_HSM9zyFPmZ#MNqxD_aQGth@^cmZ~VUL(W_`%U)mI>KnYr%FT_ z_MR*PckW7Fa?XC|NM+%}(i09QF^%1@>L3dVu&d!>P%NUMj`wHyVclKwlEjx|0J(Tv zvwgERE8n#k6+I7+u*L5JEI(8zM4SMzlX_8J_@*j16C-{y`L#2bFds@<2F2PuU7nJyI`Y48ATJ_gTq-hzBr^(rwn7pG7M4pT ze`nD&M)1dN)8Wzt|1$j|xsmkIyU71IW&Rla$9AYKMT;;zU3$+l9oUwnxI6S~eqofR zyBXL>ziB}C?i?7xo~~ZW~*1*@=~NAsgh}gvb{B4 zD!N?w#z5R=Pyy-Uq0)OeOFwhQ)g?gMkLDWMd3GguP8-*n9AJl znP&Bs(%23w_jseExNT1UJvVKxyt<<}f8&do!lrDWiMIC6v!17hCma!abPFQp*F`n^ zyUu`|ojCMNE)atYFhqdl2A7#vTqYoTF~&FeHq}Y%%ePwqLqxH|xSb(ShYh#rRLZC@ z!$z6W^h<#3x#V3SoIgT4$c6XoHLKzDz4uq18zd&Bqp6Cw75K#Hl<9054iHsp{gq&` zq|d3N0=8cs!P3+kVr(EPRFaD_CGgQ#hEM_ zki728C@Nv~ir$Jp#GAL}WzI{|^9smRlB1kBls6T|dn+6F^kDDD&;KCS>u8u>4}{JR z^!pu~m#47W7*-OLEXpVt>Q|2jSN2{&r#qhTK34jIm2!gH%$9dxt(J4h)q#~gdJfU| z zI+ET|34Zi!; zo#N$HL#GeD?g?Y3(hU=U)K*{0PO!huv3Rw(I9xF)d6?nD$j`8>2hWBZQz71dw?S6- zV;HK>fLV%Q+#2bc?l_N=(aHnJZ;3@>6GQC{L`1vFolFMiK{1%^ZPGnJ8M$o@TEndO zkd7r%i72d3sYv24+iaIJi;gk(F*tKnyASp^G4F|y2TG44-Pa^=yr}R5nyf9)zxHqa z00;bvj&K|6^6{Mv$aoV*X2Z~RsrAY02eqQpuhoO(D}EX|@h$J4KaAgW75k1_yK)AY zecu!gy_#HyvqW>!2sN;DYsQZT{+B7HnVVGfe3!{>$Lc;pIBZ_->`B<)ed%MMUbXWzTN9d3EWz3+{q82{`uN;; zRfNvva9u^QLIXD^SSBibik%#PVDn4ilSb@UHy@H{Ohx#CchMtfMPrtj=hCdx3}s94 z+6JlKnKVPfG2dh==1R=vhjR|a+czt}+?R(;F{>;dzgUc;Fe_$}<=L!St~`w2)1FU*y`l?v_piWQNAz zSkivSi%w_8IZvz|KaJUuhRad@n=Pps%V%qv%jHiIkj?Y{b*HZgk)H&Ge<(=-`mFxF zHW%GHec66ZE!(~)$6I5bJV2qQ=8wrjUZs#(@u3{k+@g(@9UXg&s}@vzyB+JPLgP_H4zD=^5YEExF&6nVWo$6 zOTQE9V61*FT|YZIsRTTJ$a0fVA4V$%hx>IgdsQ?&0xNRvAyC!_=+LUezk`7P_w5E= zlq-InvncK10 zS1BG~jxwm(Z3$#VY7VyKE>3i|a z8QJ6#UKksh{8m4%?Fn=`gjLoSRrwTEWWARQ{N!!H>=Ez?uoDDY*(I(Gz#y5{TeXL? z(6)z3pkB_;q`HJn@mCv|q=6F(KPH3r(}9Z1 z_{gHfu^1;L!%3qJ!&To548>!A%G8BiO>5FtU)_6-Jcw`J3B@?pl#xeF&`U$`1b!Lw ziLK#dqDew$V`dhJ#WyNcs*(_%XbbGgAjh-MFURVbHR}19LFO28Y%P!5+!l?jq2~2T zG_?}4Y1B<_yby9li@}K^_ZX?dGxXC;1QG^ouGa~<+cl?8pGext499iTL#)n!m)idE zoa7vFatM@<%3=oxXZc$d%TlrC11MbgK(CfMd1I!reqP4P7x;<>2+2RBCoM z>DMvH1MBkbD(uxbcg8OxO=Od!Ja>K`lvbjBcZA}WkJLL@{8X2}7z)r`ybSN3{|I>Y z-Cx^GreD2oPq+F0CkCte-?0BtsX9^9rs4A`zrY~QJ~P`x*HfPmxLeG)DKOh(K4q2y zBj?}UA-5~qt?!`OQ>t>^)h2dNjVsYD$l-nm@#Thl0@5^J=)5B}!FNBvqy#;`#MrBD zvF517?R}Tt5`cCW$_*@Po_lW7gdbL6tSFB&o#?zTAvf!PrHn=EEdR9(y z+#!v&0}jD6T&(Jr2dWGz6&q6&MP%K#akjj!6-V*dbYR(MHhraPE`xhDA7BP2xt;=>}+SZ`;C=j46YQii`k#Z-x3mF zQ8dx@>U?i3B>it4kSh-Ui^L*{X}jx9jJGamX@bWnb#mKRRpY&#cdILMi!Dqq`N=oW zp}W`BwVhT}OjGX*-U1}`$|Po2J|Tpj_LpK*Nh@uhuK3k}`eV{Tz+m`v1K*mmaBeBF z9+^6Gl^%1pte8){_Y1Eta=2n!77`uMWK0e?b>73VF>1?G7xU}hnsqMjA0OGaUed3+ z(u1po90;S$F7w(n1njdFr)wOlh3)s2Pv^AFc4qMdC`*e4u5yIp)qqw4Lpl&CNb&OM zkC-{%sG%n!;-HkXrs#;M3yM-gm%8_xh4;rRSDz0_*o{-IA{?F+OegV%+MH!{VUCe` zG+&Um0-J@H`lumcnl2byiW7~7`|*#ZD&KRO`oWnVv0~3{jP*Of*0Vr==}UpsFhlAg zu`@S6@|OK7T@_!ak+D8H&-Pk%f_~Y$=a*LD*RfeU^{ce+WS7s*T2g(AUXJoQ)2Y{A zh_#;QHPj6ZPM1#_&d6V8X39=9Try1zHH^3-u}qyt@8Pu+b@K;t5U~O zNq`2lk|mY@`M_B8%Ip>Es&|ovS|C^X73qTL8RfC2mR2dlb%xT;q2xRn`(d!N z&fZMb1esl^HOmhJ-;&k1(VaYbO;}XSw1;&mCLB@c+Le*K(p$%Sevk1aGCX@)+U*$Q zJ~=pkzt}oi0mb8XS#4ru0|)q#v~VMZV!$cqF5~l;h`y^k_s%dxw_oUJ{mmTgpL)uG z+sZd&eF2BO&CSh&Y>4@CEILyeDBy}6xpa$Mo;)F+&<$H21w9g;RjRp$B)_&o*|V$-RjMD5#QVj%1h7FJjG?tY0|)nm__ zOA1#vz=zo}3cRw?a-vx?!CxA`+#g0EEu&8_sWl3k{IC8plo!+Z+KStvGE}xHfITrn zxLisZQKzPc8kK<<7_JTzYn)RrSvZnP-yC4BM+hs>+dTE!<{sm8=ERe=3aw7*Tgyfy zms4bS?T_-EFw%<^i8&E2DW+m9G`nsUmV;<}WFtb|frQYDzOk1H=tT|_**?}dZLm#! zT$1UL+C^eM6-wG~Bl`semp~`=!QH65NToL`5}ifAf1~E8arqGWssu z&zem;5krjJsqettW%4OP^PA$VZ)3m5i98}F+&@K9R`D44Maq9|I^sk8T5F=zQ!4M@ zLV6;+E}~Nfdx9w?=ud(Spra!*8*T81fU~X@1NM4ly24R{w^^)ouYI*MpRJ+KgB+*D zTl)>%roO+>2lM4z@T0IaEXKzBf^X-y&R{iG8^}K~kaC6K1k<#@pAO`19+YRygYF*Y zayYBwHi9sxiXC6cOXqC?>zdD@8414qIN{+7jm?|cgALv6yd|`OOL3tN=PD_y zp9W8mB^;NA76eR8^Kq)Nnq+clH13*?_YKajL z-GtWknt%YTsF?{?AUNd;dODR`U_yth=tMn@mK8gT$^YqwuDq&Bg}$c}S}cS?CF}%B zU;PNGcPb)r+*o+rTjO+WXL&L1N9$R>$rwUtM@;;!j2tb!s32J;b#Q_Udg1srX7hvP zS==*0e4EZ|T7|2_2wx2r)72md4y$E$%5GrMf<>@|9ckJdLd4UTqknOwlWyUj)K4{7 zKDZ`!Xez$&o@dFGqQsuqp(x6Br8JhW73|fn; z9GE6V=tk8xY-V1oM(hQ3OC`!sVYl=mP3(CWoOa#z1js|JuhS!X8GAw%Ve_=~ff)Lj zE7YJ+r-9+?6~*z)NQc!bcwNBNTnXpp%*o^|wYj$_-)ra(%DqrI(FBO^Dqmag;lZ1Y z9Gc|753kM#VtG2T{nMynQg64O2WV@CcslHL%glk$ht*7JKRAP6$U?n_9>W0FIAlNf zx(s6{(V9a5C>zL>9HSVxV35b)PHQ#VYf|qR@5PMkep0kf$eTD)mL^H}O;_n!zJi0n zV;^)2$|#YkGj|(s#S_|yH!MA9fmprh09^sSlxnfpMNz7f4`TXi!eCfZmtk*ILDzGd z00br2bYFp59$@^I=r&a%8414sej5>OaO?q0v0rY-9db4yI<+jzpON+ngYZxh;Tvnv z$O$ia&7dF3+M}_5O{VNSKuG`3bKd{kYtt2c#$g^lU^9g!GgjZ)9y1+}AF1vSO_-tu zz_>x!4@r_^#v`6Vm)eT7%!+!kUX_67{71^_r=Xt0 zl(nM#N`VHoT8k9YyL*O-Vnrl}$4OKItouw%H$dR5laIIy#DS-buLhUPq|35Dv8JI3 zg6B&o`4aF4wzotKaK%i|ZEdD^NHtuo!sCA^(c$Wpj&N)*cE|MEzOHh!)R; zP>e+s6v75e$}NlYjRY5IH$UvQ4HVb#;+1;7tsk9veeR9wudE0_3xP+o)ik?&$%jSG z7hP9;Q&_^VG|&qjtD@Y#_cGcy297LYzU_h4v}1y=&cERs$Q!7D;)HrQZ6wLlUayx! zD9hS2RE{b&{+y;&&<#I4(jPngL-r?qCT=tO!Un(Aru1O;^q}J+@vZyS8zu20_7f}& zQm`&Q?ss6P zX841=D9@+djc;bP?7!EJ>W=Z0J4FK4$_+rYKK|vz&76aBT!e0U)Lt_^b)NiE@3%bL z;Dh5|JMZARL3k(5_ZIZxMb7S8Z#A|Eg+(xuQPf!7py83k_Ry0&MA#BOAaE<` zJ2P1La2}c26&w#JjTjchZc@%DI-`X3kbK;|W^Y0FN z7yK}=Rk@Syo0k5?RIA#h5Cg4S5O`p!z|#NRpHPrHY`!xli2~^HZWcV_UYGf62ve?N zdQb@2mpl25XLTIbQ6~D4xyh($q2@UYFc6~JK}z;)VTcLf&_1b@1~WT@ zikdXO427HHERl~WI#QZcypKn^57+K|G7AtLSd5HEWNX-N+F^s$@G&2ffPETjV=N{n)6N42$yhy18;MH74d@~ zkj2H$@zokTpALF8VYm9-0E0}7vzrIKajKnAxe#VHJ)IBhkulZen_PbfCE_I4&0fq^ z(0tt|wL!ajLVABwO2j6kf!+O%lkYBuz#_)<3qT|?mPPxFeLrk;Cs6+?XX&bxga%Ps(nKq(H+ChK9RNg;l~z!BAamv5tOox z5RD(@=1yr;T6Shw>W{O>n0p*jO5qga5KQpVX!2edM7T5A zxyY4N-+490klNwL8`Z)Oe`lAWWa|$x;n_fDpm4(?c{&)A4P*NOkqr&dj zL1jo4Tgd1#>=e-WrO$cnaQ{^12M5H`TMtUIoTk2|AaP@Lyb*6KuGEn6(n zK8>H)J%SuFV{T!~%6xVTo_>&7N2^)gdGiJrhsrE3_z>8=l0h}GUcTs_WcK8hC~M$8 z*Z{lHsUmx5-2Md|tYpgTs$j|s+Ey*#|M{(A=yRwHFHmP&4 zB9a<=#Bb?Ue#wuxjxkKMs2D@7QqfAl^t9?-iFXz4(t1Y=98$;)K@^|QgR*gO3e+OS zhm89u`&5jbDrCagyx-k2m&;(vTGjo@XyUm3n&44@l(GysT{D?{`L|n2*9X)>CyoH` z(~&BJqXmf-bhmJ1pqp73-5ej^e&U?Q@Bk2O{)?*u_e#bspr4(gA z8Wh2Z@ZC@AF8u&`+Cnf|y+R#vPnA=i+u@beLLC1J0YX!hi&5C*Xf2(WNP|RSAyGPT z>h7C6KWa8lXsIv?QJ+@mj?72wP&a8wyz%;K1mmNTSITK2-3+U=j2=CCcjy@0o8oNe z#q?m;42IY3?xfk6B)3u3EwyLi@YvJ>X7$u`)*C@F4Q`@!^Kax$o-$0mp(9G!=BDr4 z&&>IMxVZm(zXSG^Wp1CM_liQMS-_{i&A5DbsuovYu`^PFJVe!n&zp9E{pptP$|*sW zZ^Py@%0Okz)3U+ag!`ak)kejc_;*GM_cZRGy?FI1nfoh?-A{`32u73kMeREv?FEGw zo>tVPXmR|?#e1K8-MqkOUZO;;Y*{-xY>RTVIQ*-R%x=4BtyB|Q`6ajkqLix;4|@l+rGy1i$syrlOohnR_FG0k!$<2%>FKi`2x?OipUOpV`qk}H8KHmG5S9(KrzE4>Rq8F6o-SO&YCAwj~{^jzqkIXF= zw}R9nD3XNa+XmG0aoE*q@30L@NP?C2Gs!UNL7EASaM~+((c0k0UUNy$tk{)5%UdOY zi7;;`YaGrI^OXZF-`m8RAKw;Ijs230`wDWbnP}Hq;S-chH(z@QK@g@JIxk`&oerjT8|0^BikAt&3WF6xy&xR$id+Feu&xZ# zrVjAYa`6kF^dzk(E*%U#kX9R_UzFAgb;dC?E}gw6}#M;Y%$Rn0juGkrP_ zPfHsnHJXBb9Fgm(@JRbcuOO`WJGAWvG2wceh;qc|DQhO%ovCtfeiKRda|PiXqsCXy z*OlU2sSg?cB!^~vGDdCJ+4kg5x=ZI(iwlT{Sx*n>j66{eeK&I(&AQ!k+wS&P7y5MO z7>Kak^?lQF3y;G@EvvcLmv{5YEpzXEvQq=x>Ewi$FDZHazce)cZnEfN89 zC{4I2FA(m}ckX0k(yp0us)T5lgbwm(m_}u{g?uNe`v5)CS*h^^4ViRO(^B@b7bj?i zJrUY$r$#9XxbxH=D3zr;&%RSU*{7{`U~utP7H#UVw*0~venk)Gyi%J_vyM+l&~0Zw z9JrrowOig!p6;bTVt0V)AZukP%1@f9C8pR&_^w=#& z?*)QlXY}+ceZyFPReK5mNBqC0*#Edg=qQ;xJ&AkcOO5$bVP9KuisQ#+C)i8({WcYzeS)m!i<;~75rF3xy(g4itDipt zyO(C3>nDm9^8V_ws$Wm%FptBwNGF)j*k8X#R=MU>dDM|FsCEJoVaAd<8|zXj224=t zKn|Z@XL`b{n7dRD-NBsoIQ+<)4_K?@?o(2MuG~mIY6PKS zx9AgYL!SKgd%tDzmmdoLN!S-+&EPy{gKAeceR`LhFtSr*Iq)SKB$j%w0Vws|$_ZzmjrR znpPD+e+b)C>9f7P32#D4xvdKdjXyXz?Jkuo$fseK7?Q9A-GpC?ZthUg4!#@{>0+gE zrd`dh6uv7xZhKB)l|c$6uDT^;Jo;GeQ(F~++8ec|la#Fb{}UI&fr`6pWVSDp2kd_Q z;f-SF$7wvZ;Jq?F8<0`2uX_|fU?oVn=#4uL4%sM2irxzQY;e;nkzz^5>YBHGos14K z_kI&^6TCiB7!jdrB)!`hO zghh+};1J`pSY`bW!fE$<>TJ@Wx8?F`l=JY9J1)HX1W-dbOmBWtdO9!2c?>dwjC&`f7;QeVRtS)|E`zl1`%Po48$_n0tK_Pr15d|^2r z&SZeTvaN1~7K}vKYZL4{wf1&-bBmql;WZf_{ol zlTtH;D$-gx_Pi^}I&=X;{n)~WKW>4CbMsTvifuB?s-&;KA!lnXE8kUU$ZLgU7oapP z&jtjljQ4g;j6QuI_i$N}&1Ip$NEp8wSH~YXy}TElzOHwh+ExC4o*pcM{X^jN%p&ha z^TMf=z~$$k#M71iWZqQei-5Lg5WZ<8cR_UE)Qo{wkXj^fdF#buoM81Vf=i6EE`k$6 zO2AM`PPh^=I1_O1nuC+WiQ^qgnaZ5+o*A2*G>UXk)$0OXHf#C8aGQ!B&!5?q=^EF_5L8ZyplD|BXS^WjWLi!F!jl*8_ zmIofW-1CY4#%;v)rye?feDe%TN27aZ>{0Q<`&HF;>Y@*v;_!#H#_OJdDy23IfmXF; zL7nRXUU?@j9-c7iDDTx~^=#SkPp`JzAfMnisv-0yTfYm^yPkA8_xu&u>gjKKOHm5j zN>E#evn%ZnxTL{Hex6M@QB@nwC`B&M14x~Sx%)=v|Bf?7hmCXu#V&!KzP(Q$@?k z%qF8X{K$Q0byO)78tV#Fi_9G-EN}G@lFF3TrTaq?IK1T~UZ0YolgYBf6xa2$VnKKC zz{0`uPCx0c^;U-#_-W-kgdx>{#nC8Fy~^hHW(Lu;9Gn!ErS@!VyTIg3<~Kv1)IRO% z8af>oT~mgU$T}%!SrTtbbOy@);|Dn22Pd=rtq0?7~NOugG)Z*z-8x}Ess|4-T= zs(#$Kv9|2moY&Q9_=ngdu~Is}--)-KG~Bud6j=6W>y2s`l+(Xk5)A3;p2H{$U-3pW zezoZOTB8zOUwIiM2Ig0Ihu`A^($my6V0^(zLz-~v(R&A+S?_%uvmI7-xt8ZU?Sf>z z)p{)o(yqIh%HZSBR1j0tijj|(s z$^_*`#6f`H%^e<`p7d1RTCW{=uQknrnQPlpg5seoJ9sG+L#L(?wi59w`)>RXMBn|` zVn+TvZ>hfypgZx#;_z6^W$)8chWm39B>1x&^>t?Ej^RHXtaLZ47>8l8zmad#1P6Ty z$uw*YE&AiJ?$+d~-Z8q8-l-8~n%^$ZRD!aOnSgx>zhed^HabGq=BQAcFC#FtqBMxt z$IbANyIyvPru|Py^%&RoHqx9`&I-GVp^op32Ct;dgn2a;_OLeXN#ht7~Ur;r>gU3Qn z8jBgb^aVbGa~vVZZ8ueqGPWj5S|y>F;{Jn}OXeg2&XVL$51WDBsVtO$gW2w zw(#3ZzpQ$ml&zj=gCOIwe}9uC}e`-+L9=1^XeXn8SC{B(dp+n*j7C^0fN zyG3B5zxRvpof0ArNPiyKpI&;$dZgLTZw=n_JWO?pUPFKTIeqU>iSgf#p#CbOI&TIn z5DbbEH%BJ2k^Yi2dNa;RliXOr&BU1q)73LGGKrz^)X&sDhjDSTn>WpNlJ=GdU!Rz~ zXHHE_c%?V=nc~7AQ;ZPw+o)nXpaH*L$SH^ODdu*}W3VPL3Pa8Ft9ouc0gh-^v z3OtvH-NMy-8A=zgk8*~_@mhd&43)fDB68jZ({~FmyC-){`yk%X2ejwT2crnz9c^2K z%L-^!ir{rv5^WVjIX6I$ndh(_>-|C`z=kvlVM*&RkQ4Tinn9ed56S+o z$N9?p7zp!ZCEE9YsZv*}kMvbN3D?KTFu5RMZ+64B`UWBGSk8%584^^N8#C%qPC2vx zi?a8QhVyUReiM-xQHN;Jjg}BSjOdIOEh15Zh@SlPI@%C@bb{z4NRWt5w2T^^i4vVL zdLO-wvMwg0h}f6T&leZS{R0PwT(yRRx;`J{*oxHep986<&*Xm zCHGN?_dCm_-ns~w{DscQ_8k#3T=~HJ*=HrT#fW>3${Kl;a&b(t{URdt)4N(MvJ{Km^M z>9zruidOq-k5sW0N<8Vu`%L$RPH0?B*6;oe0AyA9_n2;rTTS0jx4Jd$p~R!&T3YKk zt~JBgkoJ^`HxKPb5^VV~8b2ETzl7TVZZ^h^L4|AqjkE7?PDe`VZ?8VZ-C%8DoqtOx zXRg$|*#1UY$G)g)UO;7bM9syS?LVhJS~q8IphMWHD}I%l zQin4gsnd-6YqWbY4MErFUQy?o!B+UqmHsqbb(gwW(o2S0P6Jwi1ZZ9uHp;K7(qPwE zcyzk>#g8NZ=d0Z+*k-49&~J(MKd!773lolXG(0v?!+#)!km3$@%YC<~87cw0YhCNb zqFd&ZuiF8cMpp+H0we<*{HVVZT`S2R)EvEs!(XNx$8AwG7IN4R;NRXlK*Z#UCf}!k zqf3kw_({Ufn2d+n^S|C-0Xd9+$eVzuhi0@$#s0eZjA^15M9Us1JJ#GCD)w7+T5cYm zO+D~)osOf7dn?!De6mYc&~6g#GVHf8S8?7H;6AZ1%Vi_fRfRJ=K0pMz{ho#l+_kAf zeq$vk0RdIBp&uS@LggZF5k`L9BUV6~i^cyQCI)F9Xq?wS1f6 zpJ^6mUXT#P0R5!UEKqqIPto6P&{9|L_OhFNJ;AbVos;#ss+eE6cj7g%0wbVvHY2I& zza!8krs9RVwgp8$elWY(_$UA${Q|-(CuB)2+9++Mf2AV6#CcIfF zp1#XsIA1Kz{q4d0aXPV{L(PnS;cTZE!PoHtj`zALWe@_3q8Ey8gfzOjcL}v%1D!n& zzfC_m*QukaT=?@oQAzw`WRw{&vM*HYn~#?FeRM-EDoQ)UovmNHI;*;+tnk0?0G0Un zehbe{D=JF5pA>Ide!EXuhK{|LD;m`Y{Hbelm$@swmlnSMbbDqPHB7Hto$+IiMZy0e zWY0gw<7BdIlU=FZ9z!`iS$pKs{O%IPc{h{mGM7?FyPpm@bc;;C z`NC}KV+-%TN!*dv)Rh-wU4Z||o+B%HYVnMAWB}4+;5eYIl45egTbxuqg1XI$v6~f) z7)Q3p%=z|H!zubi5}iSu%t0Gx>Y@%+yNQsdU#E+qq8tI+c2O|{tf(>3&Bx$3QEUOo z!avJ&d=H-rcC|*hwz!l!M=#E*`;Wqcb8K!O7opaMX<6*D79QF&P>{NSuxXYxAs%Q> zLVS|Zi_CMa6+`ajI2un!6Kig?jQ!i1{3D8r9qO~b`E0wZ+9oZ@=o#9qy(g7DPwYc5 zdBnp<9=fY(X}m&$RF7lDwr;KP5~-$@Vf~cqvoiGS@aSLJE_2=L-B6yeef1>a%gl;7 zwUryeH{d;uZFwK8-HQih-AHt#(Z&AfCHZOmoTB86!?etDRw@ulTq$_8H=0#%Qz~_! zJZN_-Xkp;7T~4D!t@sx9LhI1vY;429~War^PPka4%=KH*ARv^X`7wx-cCkaT9|1|&zP zebGp%cLzu}p_04p)OyR^M^;Wwr4zL-dA;KAhB)1SVvCAtK{xs@t5zGzP=3;C`5zX zom3I7!z+^4D8D)*AlYo9-s8I)2tZoEmeZf zfQUNAGru$V+1s-o062pvk@KILnzhlETuN$myaE8rCTU1~k>0~Z+>HTey@EY>=J_Tz zoK+li(jh^Lm?V>!jyE$;HztlIAdvFh6f!XNNdDekJ@%ThktfvONj10V(W6H;6BX7r zzawe3r7CeKeEqHscN2~tBdlK#@EMaX;WS$ooh*B~8G|4Ja`Oi&xGyGr% zs+#X#jxJuQmILb5MD%9IW34N|=e6r+GL9Sqw7_qKt+Z2H!=<%k9tUZT3Bqn(mYA^GS3R&j>bicd5?}&s3d8 zw@Tc$0d3Z5PZtaM--MlFj!z8gm1IVVb4v@_mNX1>L>3c&*4B>nJy~@BDfAI*95ej> z`+oB;*}Z8XW16f6Y~GngLKP~`n}H7-XI1ID6QZZ${qmP__y>xaY7d}Jb!Mcwe8$7Z zBS(O+8BdVY?1#>A8egg_?c2BzuG}OXhiTZh=!#&gBn;Jt4?;b|B0__SIBF=%3M@lF z(l66{Cb=sy5i3%GT$L2NCZ{;vP-b6ow29c_I%p>kc7hGrn+}bu)MxB)`ZGOrwjh?GRZG_#b2fHg4|N>PFFD$B47*{&W?=)Y%4){IAv9BkuC+M+1EYo? zsS=y1_mfrQfb%?F<|fm`={7XnSy0MiZ}f-Vbi%6*F_y^+^ZkP`cewCIIqx)Fc?E+SFNmsa=NqZaf9nzJr;N89o*4>*4n&- z)yBax?1sxL!E(`*7h_uP4$D`!iE?uy6x zQIpK=a;%p-;celT$7m^>blY=~afMaH>DgG*HsOH3cXD#_9hc?uPL>T;mWmkpNMA!U zGgjm}7;fPdfS-cg)|$ovaCPpvy}>wtu`a1a0tOcO{KM;{t2N0TTWQ|*%8%+VuSw`` z09#a&MPk;=(&{4sKkqvSGLtDLS+&IwXYOv;P`dOs@+jv|WIO}TIq(o$*#XKn7O%O$ zZN!88O4aGZJ;_-59NUrt9xxkMPDNLPM49@Sia9jk(r+JNS-kS>oc3|r@OAz)@4u@J zM7jzJ0kOuKL!`Y8CPs;ZHE zH`1iNFgLAVO&42E8NRq*@Gk4+fvk-)&W?Cpm7GK1qcG=q%oU648Nd(_0)jTZix*bn zNg0^pcW-NJk%Z)j>}_nmn3`mMH4! z=P7k#DI+I@n8RXKVP=lGSMf9L4GDV|2I#1k zx}CUQsv)M=Yp2#pjq_WCZ2zK<*oY0a+tji@dSNC;g>LcM^j$60 zk=hd550^V{A=ZyrXL?K$wWG`{c)=+;{KgNzieH1Pd}~TLTfH{QOoQ@$oNx^|*_FBH zys9bF|D<*)H}x7)*|J9aKw51Z+vt=2D~xhKLdUOFQiR);>(ZLPpPsdo*?zRx$foJy z$juvBrIU8ecbnnYg0fyeeen!|-%C-GF}I!5+q|{^w`t1eBf^7}g<01E)pt^-kfRb5 zjPS@ZOvbc)c3BT9f9CKb z59mwsGUc0MOvZ=nAkP2r^dLS*cj4#7=qu#@r%tPi&~z4kHkzKzv`N^M-V zHl>cc4{X&uoFi?9tN{O_GB{)d1?c&KwfWMAxTozWedoGHQ>(d{g+)xATNJ*3v;9Qh z{*`6uDVJv$N63Db)X(|}x70dL6EpCi;jnLE5su6BA^57#QWQ#Ac4Q0cM?4L z&N_u#DF~W9geEm-Mmi`00csJz?P`|+&NIB0rR6*e^&##$5&IN?&dn1_p{4husdrC) zsDeVWyyhcpT7p_*h+i2S_g_;t_W`1I!ycG2;$)f`d1AZVmnLB0(_S=3nadS2G!D?0 z8JW-?!7d4`2`6EUjzTc?&e!|tWAyF!Vac36!UShf7pcM7@{3~(=7-*el9+A#OC(B| zV}})SFj!P!<>&bK+e?8*XvO-%O9r}Eb0IoFa_|TGE11jgEm~&OC^rg z{h~(?KhL~vpm_CPmW*q$G%u#PFId~-CCA=Q&0n3EnsqWe#M@Gs2`zb>kHRAp;JtHE z{WSKZ3^#r5SGGA`+gj!5J?s7x=bXC>bhKnmzwesET2fZ`lOI!O(&xAW6TxXxhvaH4 z{Wr2LXGosh?UYxaeG_ED4?VeZCmXT?1FUGJ!R7ds=MD=u-X|yJJJNyAzB6)5s@}-N z)oG`BP*Ju`uyzuJ6u^o~9Wyx#GMis}dK$L~c)w%R-O+HUTYT`~k!Z3&E25Pl8+nD& z^#rOtDH!FA^%=`7LdtW?mq%A_(cxDV*!!XbSJ*VickhH}W`YCnEmg=osBWi~WJZ#srq`Ze`~5K* zz>J#0;N;+Hv8);QVXeU0b%2um3ur3zhOAw{OjA=+!(S8el9nCORqx!i;)%y!I5UiKKE1lGr`q--ADPyVfY5~- zRJ-EC$1Ug_61i;z_c5kP`*TgQmY6ydl(Wa@#ocW>w3@6%heO^hYly> zhTFc28T4ZmbALi1RGzqB$6*;PoBP_m-nr{57IlqnbtXIl6qdnIau;uCww+v%UV3z; zUb0!2TV0LtH*yir?iw=4jN^K@A*2M;-2K6yr^E?qC&gdcP5n}vyK^@2GIGrE3P5luG`U0YIt@q|XT!^quJKTkJls z5ncY-bn(NOVoXjtM>=5QKxXZ1tG3=%(Dh>AcXHa+t_cb|G;5Dfx8GPUKzs17r(m7- zWBQh4zD6s?ntBbmZZ&W^vEy!851|x4uDOj}cijIrBM|yPD5iL+HZ*ES#@@_Y9zNWv zdlt6;s?GH1ts@Bm(mt1kT!QbB*}sOXd+b81?fPQmOp|-;pS|~a09DeS8!b0&|LPKH zw7)p>^gTh^n){3>^Mpam~ycO zAH{uJ`#Cn~XN4B2Kobp94hMdGuwyjjb&5~Q%Tdzql=T~f8^XDN3MwjkK3@sm6qeB@ zr6<3I+RdQRhM{af-V`oL>{-j=J)haqPZE75*xgw_G|&S?Kw*0}r^_b}Cf=U9GAT4o zu5vfC;;(tWsZzK`(L&zAC2$mEkvId{4n9EA59%l#{J91eXv@DZ)8Ayqd4(%Xht82g z(q#q@gkeJ@bOP>EAWFxEEpa!r%rPN&8+_a45@_s;Bv(HLo75KutZ!`ezcesig00s@ zN}aoc(HfjSUa*PDYPAkyIEBO?RaUl0HZLI?zxll5>5J_3A`b zsV@II!Z0jfp*BXhLl(a&y4V=-zTy(7{p*zgk)W4Z>8i0QhHY&*!^zWO7C?PTg}!Av zg1*vB7hCLQRP|PpAMpMW#FLnv1f2p!Ixf?L za>)WD;d1v|Nr`9YlIa}NfKrmWTxLYfUsQPE(1_`o(PX4ovXSY)la1p1uM5-dzTE&@ zL{CrDz4#b7F9^N8Y!Q}W3^s7{(GNSv;wkWPL8+9jrmG8_z}aD=_rA#i(4HbcKDv-* z0>0{A=9L5*?gJfEhQbKn>8h%i*H7zuFIP|UMGsCwFz&9!EXHX1tw0QG+bt%Guzfxw+yL+J^9*p;wmb!i7X`h`JWB<)IDO*(1#w~zVViZ~ElfOi;3Q$wF> zXF_#9d^Ih&kN(<|xg}xJ^TCMJ%izm)pL?^6AuPxmJ_zXBPj?N5}jA)_f?m8imL0Xv?j0G`jz0npKF1|#G0w7(s$!L;N4Ipt-JAJ-Q0h?2m&*^!}p8<`sO!(F}(||W!JU9I{BQPn0_iQ`x>{RVk zb~m-oJNW5j%nIw^aSc#De6|u=Tfe?5qjZCP4#<+W94$4=d2ur&3%MR4f+Lv&Q$^Go zh4jOcOny3MMOj41n-TJ(=yO_?ch=0*VDwa9?zPAsf`v2km7CuSg)c?&l_2lI*`Fvv znYfTit`$ZQZ57H8_JhbsSlo>jASrq%-U1v^{wEfo9R;g

      mu<+bj@wTR#W7J@1Hb z-MR_Z7+e;zm6PFsxcJD!=vH3HXOl6Cr|S6cB?bzq^k7jl0aq6T*!L?pZONi-Rkw5U zlP_NA&T?%5^oV<9O0W>589!ag2Z=`77WPgs&lgc>cQI&@nOH%VKtkA?&WiOj1|YDI zO?cSNDiD_PPgj=33mN#s8k7we8IKQaahfw%d&b-9W>GG7Yx2u#fkwDZWbqLZ=`?2lUx_tXJ6Ay+noH0@flrwGGauEEk&zm zgU?2<>#35`g3G0)r4L*yj|t{pa9f^ERNC@z?RT!bEU*%8SVS|EJm^|ERV-!_KQkD> z7B7=zsi!zJX|bFIUdQ+cLKr)RYRlN(Ui}G|6|?e7R!cs2`nGHD+Ozlzs85n8j}grZ z#HtodKyK$P=U9rY{SN`;L~OKYx+q~XvfwLL^V$x9x&zapf;ssp zYPA#mqoy>}`I4YiC+8$DA`J6-4dSd#*}Ejg;r%Zg|Nm^}dXdmw?B7hSQ8WMZe1X7cPxP?>r0yb3c3j+8mFz!g{M4)jbe>9M-k(U_4g8 zx(QmxQDplLzgM>$%L7ly6!#x8f>HU!9K5mhgJW2>KU06fux{Me zQ_x=T_YaVfI}DZ*o)p=;9MKdkOxcenCk8(tBeS+>kUeHLwwKgce`_tv8s)cVGV(WE zXpVXeP(#WGN{fmwj6%|puBemWf=_mPlO7O<(}Q>7iOkstFDQ4iO*B6 z>q?~x$S3*2Xz)xDs1x}YT*=r zeV$Z`Kr5f3dG@1d+fd~6ykR__fq`MW9LE#b32kg9W5V>v-@lSB3|q=cPpG($6YWlQ z6>_|n)8wOd^07dPHKv&%-oInAv&p`#^ejwXYj)7&lzoRvSeBd@`jb*2r7*wGu!R)l z+j`_F-pJSYK_SHXjDg9s?2&Oz>PAV!fWv+tCBqpY-Z;-&%_7II;e#~u-y9HE>aeq;^3mq{)5iXmE;B=VjmH|;D1;f9|Q3XnD%%5UVFXY+Z7A6qpP@mTVo`M^r& zyVL^sj#H5*er~mNes0$iFP(a(=g?T!rc7T&y9Cl#hgI3?ZD#vi)D4z`TayD)3mEu% z{14%p4}}+f#qABMFnCCLAGad2b78{jf&Xb^6MZc6;zHWjU(x1W;cPj$cV7D7rs!IS z4lk(jpMXe6BC==6KQUz`)o##MFJSg|2o}boitMFN*>aYng3p-54HxX5n@it9v6s=2 zNKtmXn7!9&V(`41F!VM{{o zw~dttde92jN3*v}=ngO38Mfq#Ix0VK|C6;LkqOsj3iQUD94)W~4wK-B5XI5Hc_kUW zVn_cst?a)Y zBhLuf2|!q`Tp8W1O7Nm@#W662YCo>G(KDK|g5coks96ENDkpgEm8oZg!_*T%w2tiM?K3iUCw+y40zP3jaXHQ zU`O_%Q|fU1wMD(h@`R(|%;%P{>`MJ)O~xy!y!VrLFM-BiasEY+VhaiMW zk@9$Q3^kf{1SGdXF zai;+09tyghKuW*_W+RHYTN~I+a+4vfjMCh7{0%#S1`}bvEX>ea;XJgKATUC(i|*#L zhYG07ydTrIa`ZGbG=%aLGN}p1U}8LB&4YBI-7Wz!e;ALFA}DpD#W`*$8eVg(Bsbfi zfvgw(E4Ic&)#xYXn}hz`QL5raf8^{<{=P_23VvW-lu6g>eKZ3%n2yWw=|lYru)F6kz{d6L#8s zbegCwc|js+0m<2&@}G@MkF-yC(K403NNBPx5cDBH3@b$v|8`N^t7O?gP~gA_E{s6L z^L{D$D}kWbOw6!&os2aI_8chCrF8RIe=&d?=rQG*{9|xvVnwAob#~01K~7B7^^K@q zb>9H^f8@4A#ecvNBN#33b`UKdhfero_5on5s22rq_)4r3%V0V9iJMuwH{bTz1QWVC z25aYx`|Z3)c~fwJzbZmD9sU7Vhr|GfgkaP8p8PYxLz_qBA3+^1*-7<#7gqMo7m(cZ zt)Cc{MO8zM9EGdHYYm*9WN_sVd{BqIP$gso1HY7P>b5mh$jK{YEQ6?HgT~B7b^oNvxxLt(xj?12-c1uM37iYW1 zh1curm=gCY7@|5%i3aP>Q5NS|^mJ$&6%JCVHuonQ@?K6OBgv?+T!Z~gz(y_s6EK}P z_G{Z?pq#mr-DUmj>>gvPm>*HD|E*q8^)t;TNd8N`|C}>h23~Iuw3-e*dHBB;)c&Ue z`IoZN40->CNokW!->Uc%q8Uanh_BfG3Et9U2pCb(L!XYMf>@>NYxi_iR^@{WIl*F^>SrrDZ`8Rac z8PuR9a6jY**9V!_172a}XKHV639Dp+zZlrp-adu#CW8_=+&|j&-XHAx{`&CY1ak$q zg*3PmQT`k-MNnJB8-_S4iA>(&2wGwSGa=0B65a^MvL8i>fH%7&bFIvLx2hkMo(`p; zUVNZ1g7_=N$CUwA2`S2tr0S{5H*y=VRTcpnlf09q*2jOX!FJ{@39s|^BLCF>cr;7= z;Gx+!OBf=#3ze_r9GluN5kPy3=~e0v0;MWWMLBhyssJV#=V*3klK)r+*<_SUnjm9O>AH8ZOQz3J(bm$D!>p1k z14LSHmzu@euk4X)%qo`TwGV?T!6TMGe*6&l=5mq_HHx4&8(Ly+(Qbhi*ZG9r{B!k^1~3J5o(H#oxF?bD9G9(PQ&)l1f=2aVjDs zNuCW&;2swL=Mhe;2IG86CNB#P!Y?hXJ|ytD@+z5<7Ojg`UMS(I3~^yGM~t6z%r*Z` zO*$4{m$3TWA>c{@2$EjxZ`aUGl;Nxp@4>e)@!Ew{ngUY5j2>yh`htLx^wj_yClWc7ghf*(CML zl#&|S733Sy^z|@x&yVl*H!{ommAgCBTHq$gp3=@Erkf!SRMmneB<%ZO z(_wdD^!#B$fxNXa_2j}rU5mTwtkuh}K#r6aZ1oBU;f3|_^Q(m$J;H=gpe!$)-UbGb zo0)ePO-Amw@F|KC%8n$2o>2))+9U6QLtE}B*kzuswC3`{-j27pTb%d&cNPE==MO-dLg3)_R}? zYHeFq^-gi)$Unk+NYWkuh~KZcz8x<_`%dl*Qc&rp`G~YPK8IEKE2{*7>(Z2nagFeP<=7ZKM!U0hvinUHEn8 zVmAiEqXuE)u zn+kC6U;i)N{@?We|0Vvj6I|%CG17jemPDcNu0*uPuCnm`DAepRdZoXVTMe&vk^q<^ z`bjy^IQk~b&gJ~7mROn`c(sFDgGpuX*Ot4Sku&}}1{u@~m3sQROR=X{Nk%v2g3e27 zwzh>h5OrU31QNHX_|7rbR0B8D^3f7n5!OmC0yxeA;p~-a&?_F&3F~9#?>9TJ8*`0w z+krB=rgDRlA}?M@jEnGp4{Y@4M;6a3H)BOS0v18-#B+s#NLR|+94fF7u1;uNv)I#< z_q;KMmB`Q(ej~-9XfiCM)g_k{_W+VB9K_0>^)}N-1R^gUhFsq5WR>8siR|H9p?nh2 z1sftBelh=gX#QT{t=p~7Lvo=Yw$0e65S@{ylT@*j0%MOBUlG1^vG{0G+woV2L%Wro z|5aR%()|c|K&%+D_Kn6XFFr4S`Cb3NCg_fIE0JXjp#iLvhmmCldw9E9-Ao?NT$h^j zb0p?cYTFF5ZhDMfsr)5Trr_>9VV$xR{Gcl;k?NQ!s=+K%$vK)06Q|a@i>)c2Jl@#` zH2Nm=gL&=j3-hfMF6FJBa_T$;>CQ*yTaWgSS(qI~`OACIM8;~E{$gP^Ix@lY*r!ewrh zz}@l9ZqIN?u7vyZB@U%{D81;C`sq)xd&~0g z%I{6CvVv>FzD~l}#{XM+#unCGGAl4#Z|%`H(T@Gs-iuLu93U7Mv=JXo{Ve^Nz5{Fb zb7lKzt1yl;=E(I8o%owVnGO+$>+a&WSX-1&oPKY70SEcP`x;b~58rU=UfoZ#Px)vd zX)YafoFJ5YHLUODzCArpn`01R-HX^E1BXDxpP9kVEs*=b>XsX@FbqV8Ysc%=vnUga zJH@(amV3bXIk^*d5em$$g;44i8t}HB@;K?r70|8Oupka};V1D*t4Wsp)J|SW+Smi~0qaXTnD)cBz8=z@lX8 zZz#GD#H#l@gaHF~YbeRgll2@Dfm*iI{<&5XL3aYT5F3~c7so$_OivrYgBBof=R$eT zSjwob8DwX~Sw?A=?{?iXFfE2#%wM#fp2AIjwL1+BIgdUE2J@4$Z4kjvHHFB zf<%WQj^wO(=#L zqC;~69?jx;RkO;x9u&fEiYIA#Qm$Eg>3C~j_&+g;JP_7B7R{`AhPSN4wV1D6n-h3f zC1R!b)aVk@E;Em#E+1~ou$_rx9;Q;uq@lFW0&*>EIjmBA<9nqvUxSH;%nC%)5)}9H zHFEmBQFA<1NlQvw_sMLli@0qrXyKU| zJEswV6k@&99ncwKRY^v}S@&&SkWUSemVKWB7jDD`z=V08O5mgq@uNRcOH4#*E_v~u zfKBN(Vk*k)++x2XD)8wJ(2iE>Y{(x^seq;D*DITEj<}( z-Ld01Ymw~3Vf++ur<~Hd1QbeK5L+IW?M^j58{B;`#mZ_UN=r7TafImn9v5)F;0k^B z=t7yDndF1O8~70nLsOzO;xCR#C_MMk{LgLLXnlf*U93OgDjQ$^eR%voljKbV;q%)s zf40I2Phcp)@5RDgUzU_kD6>#sT_U3lb@Jih5r{x(MbNG`BUgZdTD%T>b-$iNHnTou z(=hP}9)4I~5vB1|t1c$UOm`T%_=<_Q;wsr-w9ao+uhgWU?)=z@-49rhDVye1)9t+D zMriz6WKw@htpDz=gfUOpP3*6WJlO%z%fo*CXp9V77i(Y0d$aNQ4vB4zIuEtgwehu( zOYLI01KLv~cV7xh`9FhGQ&OCFrCx%u5`*8PoW;t+pg|yiBo!FZ&M?>|3|mOM3y#=% zWC1uIv`^SWzL2DkO!DN5^&<+ zM6s)Qf0%}P%2vAV%HvY+7|DY_N~WCV?I~s#>jwRQ4a~Cc^@|&nxEw0H@V&;z55xZc zI^XLJUXjcUw@nwbSY#GhIAwdf1#qUXZYwIDAWx68yN=Cy?yA$hUTH-)(Q-WYR=KE9 zZr)&JEOD?-Oy~EzmQq&6AqoC@uEi#GL(BsCmg*xO=a#^$mEfNOf5d-J^YB;R#WXcx zH$dYRB&#g`1MiYJ2%*;A0e_hpSYrPA4E%&jlF7yOp>SSb)qIrGus(kqp>(3|Me{Ia55j2 zwGbS@9B{tKYHOTn=v>S|oY(ZmudlUF@_=T{EZ~>h)6=o0OY6H2x@?P20j+xK@y3Xr z&SUAsqP`p~fvrj=wC(7R62WhKO&^skBj>_`BvutS)~1I)>%eJEw2gW|2Q6qc*ho{ zmBmFhvT$|K*~1!39kLzQdC=k>{i4(LK-v(^PRhi>u*n2$NT-Iw*fFmtM~v@*8IU(q zZsFJ6iED#4fY1OOH3YBXVMliuC*l>~l>4M^2DxX) zty#UYoLV0ts+rY~PWPD$k!Kl?03W_y2mc-YR=+$a-S}69=VrTw`Ba&C@aYuha+y5* zcP1eFe3Xjq3U@b$)N^sLMDxz2r3+?f09owu_8j#LuDS9=l$i6QpG_WL*j;x+zv!k= zCOiqEf;~iGO(+e_OR-{60l?-3u6<@XT#kHDuVyoFx>`O z2V88J!4)Cu^CT>7iUA>oSEuLz&UXVF*XD$!FIse_?g>-e^;>}TA}U|RZ7KZ)fa58A z2IY8Tu^a@&_X3tYn0Drhtg=kFi?^F(tWctF#08cjK#1xh8Nu`b#?;k-6ymoB!6R0+ ze*JdEswUaj;iMB~WJUw~MN2L+Fy|doBIg-B+>b56>X(3UFQz;26#3)}E5r5*^rcqk za||SCcSj=bh&`V++hvOeE7NLc18MOJM|^fOd41(2fD;0o4v%7Qx@Z$hGtR$pYX_^I zqzH70Ba~~Ka&;2Q?;yx77*$5xy+2CE1-}`mtc*GxthK(w(B22FI8~Xt_JCeJQ>b~D zg3;0V+!rSAegr^)VdQm>2Y1ktE17etP>SddPso#xyWN0^V}m{Sb*f0z;MDAq_{s~O zm=Cyny5ko{l#9~1TbKv59V&JA2W|-2fNkj>MDcE46UeQ{gtAE(ae`?7aiq3Rk`Mp- z3`K!Xb=;r(N|emof~y5Zc);~x+s03Y+SO))PrGbC`(~tbE}dpS?8JW187p&Sxc~8I zBiAUqZciSWDl4LD>I(|HlI5#*6(pRf_WAbTJ})+MrEQK|S8ShFj9UG4;Q|-phaL3z zz3DKr&zHuSGkuBSAn0BEI}v8|nKwY~#aZLYQVLJX-tY^eNf6YQt=Iottj1Dx&W54b zb}lN9Li>-d=Smj5{*quN2wRn~-jF*28 z9D?(wJ?tD;_9f|Rd_kw#1EZUwE6I6XlGOL;QWjrE+ORMZQli;UxL!#G@xU+=)6PoI zP7`%xCC6=UxiIMNftkQFjM%%1k7zlJUjrgpV7vy5PyKfLCflndwK!y=vR7~!5Vd1i z%dDq45Du71n(iF>uW6X2*JLV3(bkE9+9TecBxt)G7Sbu3D+&Iwmw`R8wyI14X{X-lj`T zf`~P>&RKq7o$19@}c?>F2iNZy$cb&A_$?l#ew1dEHPoC2#6I3_Cd*g*sQfF^3V{jI&k;G{4Z zPnD4A8<@wnMQN*X`a1rjlcc3Y2bo{u+<+m~9chxiEJYLJm&wjz+-5al)NOz{7YE&! z?Qv6-gc3{xw#pK<_E9Y@t%TqEoBPtjC02b$mey-T~eXR>IhOLU_Oe zK^5%;Y>+gvnX*ezCJ*EG2Cz5Y(_gzKBO3_8Q(xj1<$)LOBL5({kA)3Qh~u+lwsC6p zifvtRhPUE*{(6mSh+Bz{IY9@gO8u7jOueBW8W)&x&b^G76P4qVT_(KlR=5c?{ML}I zttf3w11ULE-#>waO$(nmEDdMrI##9Kq8*lIBKY_)3y{C?)59dAcdNZtnd~cWr?G0U z|CMF(e_wflgW8|ms|=oP6#SpdOJKAnR1Z3E-xNdqG7Y3}s;vUn9OTR(XEI9%9&mct z>C#J&Q`(}_WO=Hagb!d%=fM8U;>Y|%&z0*vIPhRbrqX6$GsB=PM`WD;L7w&}v^Utb z67lz?!EG>VLdO;c@Pb1@mcVw}GEj>-u|c^66>B{-t|fX zD?Fw!#AI|=)NOGPQfRz`m_QJ9Vlk$&$1#`j#N*oo0Ri${TD<`p1CxY%-S>2X3GBi10eJG&Nuv zN}2viRlrQ!=R|@>63Fc(#91BufFQ(GF0KGYtd=Viw^@lM&n#bLiUzzd5>R%V7Z{}{ zwPC-Tpy|$!LS~2bo3db^Z1{C~^Z3EBLv;I^QS3ro-$XHVCzc0h$d2Eu1$K6J)?>>2 ztPWcpM)6Lfq#%d#zJUKZ<=NF6g#m zEgdT63%GA*Njx9-P4h&&Jz zy|0&Z7ezl_7uGYr$o(K`1CuLYH#6?xlMT!04QuJA%hxue8q(A~EpQ&Qvz19Sh;dM2 z1U=_+eRUx-m|9Dl_BbDeaE#mzODEz)O7As z>-_P|;u318iJ~CJ3d6Pai8VK8Fgx6Xb6k~|CBOT&^wX%$f3|f0 zGp&C5#vQ!6X3uzUAtfm-z2m{{?{8k3a`af*Zs;>UC~11L^pAqC#Lae$p_^MJb$*C4 z1t^?ShuN)9IN_68oZ%f|p)lxESpWVC8m>r3*jS{wK38zP@J6c3QYXvZ%F6MKDp{ZZ zVQFTfhpujd{a2bmc17C6!s-gvpJB|RglaIWAzd@@gxbCH!uk(%>|wB8nE?r?clg zjUg{+i&(Z~#ivs9Y2>Kihj~w{5~jlP4*Lbsn2%zv{Oj3Wl0#=xA0+#A8? zBgh(jS2QZ_Kx=5=db+6|`}DFpzvNOlV5%v$=@-Tyf7>#92YQnz{JIm3{V}54eiEJN zEM(dM$v-;)akM8b{a8jXBsuRi1=4& z8*QqEI#{M4?klp${0>it!Vp?5+hh!MAzJM$)fK*fYr!Skc9t)bXX#>IQa)r`wCCWJ z%Np!t;3P^|2qTQ4XIu~iFDKeHX<_;#sEFQ|374$A4q+w_(W{+tSupB}=><&tDD2|G z*#XaiB=iQBvaJ%`5D1}4xFm6yh*hpbP3xJndCZ?A8@ygSqxQxvPRt_~jo(`aq9q-T zdy@9Cb}p|6LVr%6zu@+I^PTzuNTP$eb@ihePMc8R! zK4jT#AxGe;7#lhwcEpKf?+113;hYP$J0~y5<#lt{f%oxdT>xy^*S5T8i{yrLUfUn4 ze|srM(9-BcyAYGLZS~{o!+)N2zZjNnJ63bhj8l0iSLm_ONya@>SNAG>{NwQ%29BFM zublqu@EP`sa6)p7qcqgjjahAv z^7lR2$E>5ysTh0Tq%ra!HXdcyz-zU>?TYD}lW~&+CO82Tg-QdyN6{V)O=sZJI;y%^ z!0G0nI+j>*?Mh*bH2(CifirMz?U+y3dO5HMJ~82nhw{xMo<6C!?*>sUY;4_{=2U&K z$06G=Q-Q@}w}h812>1(!1~9|z6+K;nO??hY`xbNc%IY2qzGsM;e^>MJL}p8UQo4iLQSR4onNW=F zZSWc~zdy4C`}_CFv#h&3Im%#yW2-AqfPLP$Q@^)R>c!TQ2)$l_CiY2{*kWPNT8}>Z zRoSVt*A4_Pd#+ftYhZ$ECz_Jr&Ee>#%?UP&AZcWw@#ci ztVvZ4x>m){cHh#{u^BnYsb9&Wv7v*vTx3A=Z1@pRg(ZkD_G8&1HaLbvVo`flG-4lr zsIm9ypfeJU6(d&`MkIx7$l}s&f`F3vlMM+83Ei0W)s5SW+t>DN=zbQTDs9PkS$tQ7 znke>=#biWGYW%D{t}7WN5f@(|IjUm*=#QEq?YXWIGrc4>V>jdaE9jm;^L%Gsy1td~ zxVE0Ad9>*n*@PQ6M1!212I@4J!==WxIzfT_8>>jk^IAPa!-fSJy0iW5qK?QnY36K9 z3+>mwu+nb1ggiqRDH(suPMFY{0~|6RXDtGEj(~JRvwOlcCZMCjQ^|-~_E)swe=t+M z%QoD(@64!Kfg^`LhbzKmbeD#MXo-12k9-8%T`jfoaR=;^F%LjRU3A%!ZMOyaPv-cG zO1)Tp{$yhXcp}e-9;F|AYK%#*tG1hvMc!md2@MPxUYToK9iF^lbTQuxD#|z;98`>9 z*C$iR3uI=9py&1CsPvgEff?Bk#>N9J&)}yKaSnzavo9Jpjq~!ElXu?J%*j^UpxRyn zl~)p;@SNb*mpdL)`P8?)_$b;*l*;}5L{k0bEtRzsAk#J*wepAE4fWca&22mU4CBvj z*#yuX7&RR;xTdShAN&@IzNi@G-mX$E;#qFK+b_uFo7e1+VDP1(K;;E5GZcY!)&GeduLzg%; z-XEij6L90!8i_vAr{mSP7M0PnN@ZEr2^M8x%l>;-B{Li=bNZxyv778Q zoNmZopw^y^SDvi>d*(3kM0y#>QB%&;IiVlo6nGsscCqs?@k^@@mSM6(g@#N~0O!2Q zxueZYg@dh@vq#dl&|KG5!8=M)Wc=fWpaF#r6D?Z_5D4wIG-;@>ZAvQk*>C<7gna1Z z8y4meLUi#&4pl9^sP$;=9!X=je&-^G85(Scb~nw0x=4*IyY|Ys)TO97ge&N-zFlbO zdt%WANg5>R$ziN}q0OnQhRtiDA%ZR}Uc%YxT?TWN(g?Y7W~NZ^uCaO473JoF#DW8AVBFpF3Da8CV)4Qg*tJ$V5X4S$ zb-b{yt}f^>@SJqpw(TlYlwgab;LLM3_;Z`-pucbRshePxr}pab{p7WIWnW1sx~N&% ziRy__+uGM6eUF@v=qg)rulH0Ur>@K4g*+=wOW)8S1$~D{T<6XoyPD5TOjGN;80}}j z-^tK}W|kqDFOLfQEXu7gtF%M1ViUcHS2 z1WG1MO9r&}N_PI`O@2P{PQ>rU6Q7Y@F309rvB{FTOu?PD>pNa<+%FLf3!914l~skw z=A5rLIc+wV|K)c^h|251x(7L(9ULN`JbA)TYn>!Eu}8L+8hfZw60V>SSPaT{8UZP} z<=(9Vd5HuJ#!k?@n}stFxzvIsDH(OhL`N zA7p(0#|7}0kx2vVq|(LbHL=((9^cevva7fsn~6z z8tPE~`}j%qVCGzRXEnyJJu1KR#a{uazgG6kPJjL9g5s|Gj|sMJM){gQeG1zDUpbA+ zY2O`^k#DIF9k&DNTym#I5umf*iK&Tg&uZLD`{N>En8C{vHtokhOZ+ht6t0k*>-hFf zYMMycT2r#q2kGPz+Z!tVpdD*;bDZ{I_O1@bNdM@Xq!nTiqQDIIhpL_~8lhNPZr_o{ z-t^e4?V#KBu~(T)#{^xQUz!wdWmz_8<=Qu_B$sAX?=FcUPZ>$hbiH<~J9F5Ib8~U9 zxG!hson@@W0}Jgri;wH0&pR(~I=S=p^QL6RX)8ESDbC68pLkYr*Z)nr`R>dU%-RTe z`>K<3(NKw;pn`(0X^2%|xl-JMRiQ1yRnGXxks~pncV&!%)8ym$b=Ks!j@s?6Tt=cj z-05fGpZ%s{UY*<{PH#Sx!%??y2^B?AkF>lu^B1}r=Jf1$^K177tz+whY|o+=D<=V$JCLxW|Gkea7iy%G_8 zXxv&GAE3G1YMIR&B}uXogths3-6}6FuTF8Wki=*c|+a^v&Ykr@LjA zc+KvdiqyB6FTSKL#_`0LgsmkR7iiMap`7|LA5B zpWkW=Ob5HPa9--!5@(Z`=$pMx9?Q5O1#Tl+?81OZWD_^UK8T9k+51NQxhJ7vGhWzQ zV9rtv_ngO@&t*w9eK0idCw{K`bUnN13jy24khk& z*%{romuRtn{Js3`mu$HI@Xt{ajejkiA^*>xDE}5!)g^<0=oD|mIgb6;oay&78c-`& z@YQ6cA?_cy>~w!TW_ZO|A6tjI*8|GvtlyP)y*0A2F-SD3O$G%Tk8wHm(KgRR@mr<1 z*vR-RzMT!{=;jVbv@wnOBzAQKIVzEAc}hteUPqy^$&M?tNd-H4`pO5$(+K1YJy=*G zr5={}Ww?%75aw8QPtH(FSdz&C z&wN1}%t_d*jO~$j&FWhnOeD^v7s@wqHl`8JgwH?u8vM zC6$}`Pd*k|6s%yM@Y&OMFY6S!$zgQOn?3h>96;ZcctTIhAaOlkULiQ}vA&+Jb^W;q zH><#On)}DS#18gtO)sd(G?W$Higr$oLXvj*UA%44f)$cqw7c5TJtP}%>F>V-X6~29_lE<`CZ3IS zLsb|0srSlbn8<)d{1JN8B*t26IW?*Tl`Yp5Yrh#~@(BeW|2WNT8H(K{xGpGkyD`|( zWBqQ&Q1@s7XNyE*r&9DpK@~~OCHJ&LzSTa26TVz`!D}RrT4k#4<1x1M7}1Vv)zA+X zI_uDgv0@*hA?PcItJ9=uKhjrV2@6RoaCLjPU~Qpac(K{JFQB4}qxpd{^M)f93`h&>v$5R76oR8P8oRfexl45s^`JhzCvekI(IoXzA_hNs5e(TyiQBP}-j!hl6uY0PfJO?&QK;FI+?zVF4 zRik<>VxW1Yz#BplD+?{5ZvA?fkG@Xm&)Ziv} z&+-KW}8s5wB6(uH%*gKap>!%%xp7qwRg4(SCLt z>%=C5Hq^S__px&bY?kXN1e)E}qrtRVa`?$+eCqts9xcoUBr0EN$eiqPmExU?$81UL ztyb}l4<&{M`X(`(X4>6+^iDCR^AHQu+|ROge7srpXU+G$+T6*_%X&Cdqx#AK$k17{ z>z`iXvM4+fv9ab5wHU;zC^KB(m|@npq~p)xr<%K8Y-ra+pEiEH{E0_z=T=Z)tZ`-B zDv@iMW>y-)=QnU##Z%7f`U;xFY(j#f_P(-+ zaa=rL?(bq{Q*>ZpvZ<22*5S+SrT^A~BksHh+3i5!Z)N-?8~n{3#aJ<`+Dx;upx><9XY-Y*{Bk<&DJHZ17(QD05% zbhSzTA{s5IeOuZpu`EmA%KesV-}}&>Yaih2NYsu#KSrqIXa4t|Y5Smtl8*>~KTdJqk`EPw+KIfL;k1(|o z+j;)v!)-<(#;aiF^h-K7bCX49w@!uNi-O1HPBqOrr)j}(u^lO;@S&A`jT7s$PD`Su8j(fih*l=GF#Y{g_zp~J0`4AMFJd+n+}C+*3+rG zY?~uUx)yqCPKL}aH1~` zEmR`#eaF0BoW3x-dhDus=CBk-=ErD+Fuu9RhI*-Wm1e`V7zfK=;}16ZY|Axs)lV# z`vAsb!&^4M&n}rn7%xoUQrBC+xS3CQvr}P88;#k2A`MDg1GXame)f%ux~{GTWU!)eh<)n5nI8;d$z9`k`%PL}Gu5o>j*TapOdsP} zV~mW?R7$q&uE?;hKDZrg#DNH-9ZzvB%+QZtK;%rM2v{^Gr9zyvZbReY_1m#CIJdMU z`qT4+c_pF*O?_PA4R+!?Ly4^U0Y`_ag)nQDx{76k%_6R9AGX`b<(skYZ57rewn@6w zboy}V$XnXbqE7n!3~;(*+;X$ zd#Xm!uY;qxeq8~wqfQXb^y1CGVlVFP+p$}-P9;abaaZM=gV+0NRM*k_8p03Lv2vFklUAZB4nV6Zu3V%0pC8#Y7Mn?}(-7Ehd+oVAEbRk{E zRvW(vtsaik9w>kmgi$T7@GPsg;F**~T<*krPMW^*beSvMX`zev^w=EGhZOYG#wgHj zFrl16))!JOQ14C-!)HF#m%~pePSPEFK({aGSQz~ghu8CiIR!_BJyt)yiq{a%A{y%j zwuM>iifSDboR~MP3$M5Tw?ZghLAwvGt9$GlPbReNNr$7u?u+phNzl5ZQMsq2`%92J zhZUKkb6Te~->>X}2;)}EjzcoF85_9!KGfcnl6df7&Im25ZB&&Nkr$PC)Fbsd)lAP8 z29cAoz${G`yk@aZxG4VByj@{3i$h@n=mw(>|3d$v8ixaCb)~a(Aq6{{gX1M79?Z@@ zg$1WPf7CqpWN(W=7J9h*=*;Ea>Rm#$f?c~7UQo68#CIQ zP(9SWz`FS@|EoBjl{bR43w$IJ5p6Lj8`E<;{?AMN*jTaB@lCU~3v-X|6Po7Y z6yr>@l_1ihgen^RIaKOWTexE%k!y^&Eu|4A;PMmWTOUUug1DxLR%+_`TbS1;PSsX-is=Ul`Q(gPTk9M& zZa{^Sal%YZP(jX`-FrO4E&Po@Zo80ZP@A^&%&euo`@@^CmDNy+5%m#~oi& zc05rLp$YgTnHt{^lMP49S{5kP)Fdf22QM&d6yO71k1e3ePvLJ}O-tEEtO66uG9YT# z7iPS7tDlLfs+}C@5^{Vdee1bttrEoLsK{1X*+BHq#>9Wv&jiS>yuj#LS*A&qANEJk z<%B)&Yq+>Qb|)bm*2^|O5}*NZ5IDOtf*yF1rr6z@lw4wH7ja3zc74jIhBo4=qe3m> zAS=8kR&+|ASPF!-TT%kXPHrGYA@W1Z zq?rJljdkf}GOY8>h6GEf#Q@pQNSwKRa6lw)%i!wq0;91s!-|7r$zl5ds^L}$iK;_BW#oBC|F%c0+fMbQ0&e;G2e@ZIi{u$aZ zrO2-630O-|Q?~O|H$xoe@rKedEM4$Xh8b!qA5(AW=T8!m1_LuG8;SRx2CH$Csv>qf|~V= zP{7Rnd5*Q$6m(%XrO^6nR3yYxpyS!bnm5$|^OZ;>FEZF$=&ro&Ew?ms$^^>T4k0>4 zI1TMDkGh>U!UJ*%YQ&DfO*^AafuPtTC- zj%L4?DGGJMaQxe9SNjTpe~HxgJHwypPCeF~sttK&gzpqW<7_V5BCBd|^347##X(bY zj{UNy#_NDn-%Z46K`QSj0Rc*lq^i_6_o^Wm9n?Cz=%wCF>8Y!;nv$9;0w*WEpUqOg zR#Vrgb=^ZeT{<`$joheUd;t}Ma>TFwIZrSL)y2DRJ#j<=@W;by4%Bmh#c%XsGw7EowG#!4z2nzJK>Oq9_5rYj5mo+}0ej z`1p6n=--^im+ws91y-Mw2Q^Flzqr${i@+s6d`LjLqGE;q_1(Uc>*_o3Q#R!G{Yt-H z@&{e~=NB+lU;&`YtegLhiu?8IZg;>>=~3H!IKR`t_x1cw=jVmw0(@*eEMltg>m~p5 zI)8AW6c6~RkM+ZaRShm|yv)Agr6H79+t1bo525`Pu7#>F8gN>+AKW-%<_dqu)9=yZ-l5Etm{c zDSOiWFQo7}AWE}_#io9@KfgZO_r*(*e=3V*js4yJ{7nLWrG8u*S+%V4%4$~Sj85<*() z!Gj0>3wFln;qq*k<~=D_@}<$m+5YRp7~QEd<_ zzgc@y5vcPp#EgGh2DmtMlihx7P>dTWvWETSRwP2Itu?Sl?{YmJ<$m~X~s zovvRYZ3V`h^cL9~6^k*R*Eu><)_(@S@Zk59s3D5qDk`kP+nMx~MTu%?o>v&<@2A|6)%(;oB2Q%RI9uqj-5RM4` z)H6_M9jt9#F66X07$wvc$$zRQ7y?0;ttie9mrIa~_IQp$X(zaULy+b2z=}ebK$^b; zhCeUfi{Ze1aCBgT6Hh#tm0=NHQc`l?%BuO%Xt1y)TL^r}v8P%Ci3v;G$A{C_(i$b= z(R>AG|a7_+^i$HXeK!(hkl3T=7{%lUyG-3yI5EQxi`c~u&yC}yH95{H;^#= zs00eD=mDD7FN1XE0_j8zDLWdm+Kwi3;8`JF!zncz$pY= z`t1~HzCWK^&*k%<7WVylW>+tGK1j{V3TC!0JF$O0aJ;Iy$-tH~TiYccWdNkGFXY)OT8U zR6OWPGsKgVM3q~(xvs;B1S|;1Pc*EUm{X$sue;-=cM@p!h{2sHmne{5rQe>UZR_Nj z1(`*o{zlR1#l>P|;$IMSn(q~LSu$X}PXe1%3&szM$C6Tl#wPjFN%BOOH046-L3Ys+fK0RM@b`A&!eIRFz*FvvWBnsG#*S+GVCDAhr^9&4^ zIGPun-(aGqSFeXntVnw3XcPkeJxMxn$raJx-+xrl_59MNxHBRkW2XNwUkQm)yr8v zDa17!KrwB@s8tCZ-W50C>ei$}hW;*l$_n&TdeWm8sJgnko>48gh>iu22Xs9(Akc&e zpx_&&s6ge!YsPF@WtbbERls;Ut{0x?vC%k;0+w_FE4x+O*N&a%I4g_D!s9Y$L?JipvdwQ{lDQ%x9en3>b(RHrQ4Rvj=V{U z_a|ZWiSm-5p$%(%r~eV|9Aqf3aqqjk)kSYqQ%u%6D}bzjC6zVbm&-W}CjWeq=&tq! z&D5BVng+g#!DK#^3y>wbv%T3y^e|?#VWZ#pPputP`BE^dpl`@rthg7ig!W0^U-8|% zBS)p&9FiX5@(7gNCwXZWMNj}Tx)hW7QRXh4LuRR7&c`v5a(*cK8?`{k%b-%yb-FN8Q9JzxCH z!U_OeNe5?A2F7F*KRMlB$@+=m2UnOv9eZ*G8)mLZUXY6zSa5d1SE=P2y9kKbBR2`Q5KkO{T8a3WtjRqIrrqqd=w8l#FGiZXuR$j**-8F>_M*Q zAC+*j$jzRTCdU2 zXr8)LJFc02oa3fww{9SXJJ~c+0V*=VC&?@jb&fML<0l-GAL2HK-T8o3Gn$^;>`yjz zu)>*I`RBmXya+^y+8Q>-iscS9ZZ0Skxvos!$u_)7w!b*kh~ZI>Hs|K%2AIa^P}(+FCxzc!;ik3gHzJZIuW8U+$w027#gYo z{TmDaV$aK<4x}zJr_AtBQdYmm9c9n}ghQ1s`sDt`V5EErFKM#r7Cd-1MUH~$M&a3h zqPE}K^*?>@XNmA_6*ylRONf{f<(qnq9lBF&XM9maq;|S3F?Zq)2bkocM0Sh@UZfo6D+@CXc)jUYH0xyb1@JvbH9WxuGfB=JSW^ zQ*rC)dGBqOPU7`7?LtksU4qN7$~gGC>%Zx#6)8>x%zpmHaHke-{N#a53_=aVmgcg zh)0;xw@D*9e{S>gIuqi#xPsVPeHkk$1I?IHwA|MTxpOizoFYjLodQJY&eB9l0?P3H z;vmjj%+Aif0xrZJe`t6IemXJuByC&DbAi#61Z>a^a&BX(JL^fz9i3l7k%Z>_@rf5S6)~s?nFy`KLJC=-vICz>N z8F;xwfpczPuqz3H$Fz9XKSbQ9_lM^_50;xNR!uWf1S`jQ$i`fTQj#GwxVJ#W5Zz35 zng02)*Y>j)TruMeC9@S<0~LW_8q7U=LJDaWQF2j(?H;PNwe|BRaKWVJvHd&OlFR+^nGDKgH+Efwn@g1rgBD{AwU!jRy^IRW#x3<+@Fq@aN+KOJzH ze(95ArdojG&MM21_v!uSavI479Bun7eM&Zn2UlP@aGPeF+_u%6@0-gBjLXlThidc7 z;Uwaz^|xth{=j@Y)lYUoVId<+h8}NsB5u>QTqe1(5f@CgvZhd;$>Mt29W3%a`5 zn;Qa=+a0u@2qT1fYbqh7C1p@8^4zhA2=F8Csh8@NE4Q40KMGvYP13;%h&Ft88JnBb zs@wEh>tD{R%{hdRL;5<=Q~G{u0>&(rLG%x>;|EV4rzk=0Q^`l156nvfvlPeR0sZ}% zTQhk`r1Ty`rR{TERi3hE=*yJ0fD_04}l|^pT1}NFR?qI$%b9uW@ zghLY;MJiS%hOhKmekR4>@nOYgn)4p}R#~hjHodKHN?i})S-7@#_j7}#_hPV1+gYJ; zFRWSIqm>PyaUmWb=2~RjJln@c+_d3m+vzq5(rc3Ae9z*2Ztu#X*MndXetGM_|CXdD z!Uxf{-3Y2IA3gf=FAwkXK#$&M@3Ho~kD}L!B&jV+(ei8DG~$Pr92`z3#!;fO?I%Kp zM}X~4mt(%wnIe6J>`MmElz1py<^sR#vJyj&#N;oLpi($M)vb*xDQ70?tT zGNmA!&dkjGw17)B`S2RUo4kmol;q^(Aud&uKgLdN=1H@GMf-RWBt0DU=CtL_wx&m^ zvkNg^Nfog2lA)epJ0rGO(#N-_#sYTHjm19kUVif=d2_z=I8@&!f#89~o+CAqpEKU) zRIbwgaa2V-`RFSE=75&1fP8)8>9uzRaWWOe)~&Q^+Q8i5^n4KcI2re^7O+{?*iyk? zen0oqIe$m<@=VN|;8mOM?gC=l#q!oJH4RyHM|Cxg;dD9DC6;I18My^k5YNXiO3MLO zIKOQl14ry2brgl|lewC(e9Okk`QAlzQ-bR|7i(m*wZ|B<<~r`XQ%^JX{EYl4z&pwkXF2fhg81XG%z zGwz<)y{hX3HH-nt4k7+eS+|`kMKOV|4Bf$c9P5~T<;IXQ&_esK+W%q~f1{dKQ`-G( zl6PY9C603^d0E5Aw*K6WsMVE+gwP4;up`{J?|G2`sNYnu0$-o+W3Vn;zE&V+*`HS? zrX3+T4Cco)@B=hcc*|_8nn}O0zr9TA+5EUKLpy*|y!O*zfFCU#K0=Um_ta{MQ(?6~ zClCI*tv2FWWBpAi3D2Fe0zmzw1oV4RV&>^UDqwyDb7vO4s#3*+maQ?yg-X=tfo8nk zr_wC7F=$8t9c4#>#;z-10wDhvY$f8{-#m1O`PINgV2~{XGp7bL9sJWm0+Q!2(+*~z zYL?a>O4vb7jn~O^l2dpb}X~uzG@8W;ZnJR2#+7izS8Z(;k=p!W=`Se)GlT*jYU7m$G z7E158y-Jq`@tH^wYrPm4@Tqd?sU^XERc)aSxLZ-%0=Ac;}Xslt=-PKEtc^DFnkBHL$F>!wZ7>Um+@ zj*gDD1D1u=RrF96G6tEHK$JWRU3+anc*=(?Dtk)iBwXObdF>a~TEP}sbZdY8Jg?x)R89EUm?a!eG-D6wO!l`z>@ z9O}yiqK5^2c%6dv)oSboJeS;Tu3M$hh_C6RwRZFx8CpjGUi`@^>L+)*srYy9HbC`& z9ugdNUVAj9_$Ej!>`aDgWd+5Z?4j9FkC(=qk#>}nnEF=R5%KbWEkpo%f?-nF2C|K-(ui`*}rBt;; zhl2b1o#Q#LveP#z&1LSGN?~v%gar#OO!cj-6f4&IMTb(rrT zV|I_T9i<0a^#!7a?XL4J3UvDI`(7`2$&40}X&IC!={Zef+yVtqB7wryaGBs@NOyGX z&!E?j@97@+1va=}ng=idFcW#e1z>nafS_&3Fk`?m!B*ivRfzo$p>k4suvt{7QQT<94iGyyv^NO zrRb25z9*?S==>&0y+oXH3}7ZOR}H{n(@43dD}SB*_mfVb{=$v%K(y=!V>Kqp0Ka=2 zSLzms&-#H8N+J@q-zSV9s%u+WdRaA;%lpqt=AY;B91HV-zs16T_od_gCIOD zJ0C+daobw!sl5t1QI9M7Xo2aykV2HFyM^THU?^SkiXwjFBTBtt>ZKy!Eh5i52L)=<++-r6g+wc23>qCHm&$Ss_wvVr5XxdJ7v7~a&AyF{9CaRqbk z6vN9-({2LXnHUr^yFr$N5dqXgeFM=Q5s>HVu1y4iYbq7$whJqp*PP=mGJ_iC1Smt zM8vN&kwa8QxYe0Hr{nIR#j!DDuM3D~kC8j@PcUDf^Ur>R8H13D03vW{BSObcuDik- zh1>+w_81@@OP3_^^SPStySp{WqHAP)&lijMzFC|pVC67xSt%Q74gk`8mA?(|iL!AY zw^2G2TxezO8);mpnS>^#k9V?gEzQ2oVtl`uYueH*(1CF3H9RI=R)n-a%20q7H%7JnJDoQ%d4l zhKVM#crqG76U7tmiBI7?eFNau*r?F=pTPBQSrv{v@K*%+WAUlLc49@Z9g`nqk?gE(fP#tpOa{ub`Ifkz zbWK$Y1c1h_mg|U-2rD2Ua<~;l-$R=;7B$Eay;nR#9rbL7+f4N2bzkDiBWH)8O7xp4L4#h3EP|VeMUceUKo*C^6nW;0$_T7@ z{L=cqefuURh1bZa$bcRh6=-u7R4jV$@!LNaw+D=u=7xHMPVf&hH zQ#|6)Mh9l|<>iKWIANnQ{)Za*>xCQe);*O{B{=?R^lhgqzij}(5L+@UPC1WMdRfHf z4C(`@YZ*eUg8QbWr3p{P*Q5y8j(#Mgy^`$Erie!bAApoNPW7#E-i*A(Hk_=tzfS4c z8tr)D=WLKU;ibn7P8`)WLU7>EncC>rp8fdU(D-XjJM*}~LF+OG{`A%le`&9p7YN;6 zWAjBYlcRd|VcbR_a?fvEA6h21zA(V_hA-^kscVN9d^D~Fv1nej1YO;B)}cV8zFTjR zZwFgiC|;VdR8>_P>{|Nz^_c=|l;=m!nagwwx|d6kaT3nQTR7c!NYZCOcYEq-u}aeG z!r6!;FEWfe#HFBcd~=A)+$|GAcl*JxZP#8Eac}EJ_`h2m`I* z7%R!a+WlLYO8i9wV0c#-g^IyJ28Z$@pZJ$#1q-ZCutfO67JdW5Ush1C1!dx-`vNBj z065_b`a?MBCfQP@F$E)>$Q)~mC&4|C;!cZxMJ9@%-_e*D0<$*m2t{WclGFyf6z@aG zKan9t572|l1E}&(Rp_VPDIex8za?)->$wrvvl1?_FBuIRm#y(zkd`hw0Yqmk6#=;*Jl>fZLP}?DMi2Uq>S&qqf`9S+ACla)ZF7Eg7WF z)_7o@l_bIu5IoX>b8ulJpL)CxW^TeynqEJy2j?`D&(y7!Gg)}0-X9eEBsbBY8Md~HEsv6UC6jPEgt)E3;Qgab5I<() zzMWo|^JvogLNYt{a0r5KD8h+5jcbqRxfdJ?=lWI7(7!_!+ndISHF$#(dJxFqb7pmh zwI*EB9Tw3|S!T6gp6TUSTl%cZxFi`W6qL@1C|2#jEcYOsh1`ktf%(}vnBsR8FsY`2 z!##SvxD#(Bd!hLlrQNJd(yRkXY|pv~l36Igg-=T!l+q<}8|1_DOQZe=2E?)wx-($eFtE!0DkF|Gd3w0ArwDYc92- zbo{%$$RV0=K7BS29e?^Zvn1_4p9a9ORV;*sgsu{glyWt|&T-v>1BJv7a;*$?(!S>> zqyu2Y%o2xrNa)-o=UxFweqc2v5R6FYHY7U4WhzTZ*NFTRMgkSKIUl34$KFINU| znAR35JuA+jF*h^&L^Y{e z8)X`=2?8<;AO@ABej6k{Mru5Dj9#hqqV8KCEVh5Tt`-F{{*toZDNeM>LIMv@>z^mX zrGS-<6@1XzY$;#l`<6aCZD$JT`m&>TNMm0;)xwmb7)~3;b@s1lh^7Gvgx9tpjYlYy z?ulNdk|l8uafu)Fp%_b*QQD>B&~~-&rK$`;L1ZYD_PvNm;SO%A!6JSlhL0owtYk!> znB>zfu+zEhTb;!q8Px^o&>Y(t1hAeZ7`GRA{qp7K)2uI@Zw~bKM#f$L zhDDNd9PWJ2hyAbx20Otd!I%h3Tn9?LL30ki;S3kCDqF5e%_MB)>O8OFEgV$2_W{Ch zctx0lwlC%RBzDocf;rcu=^Q$|3t)*PoIc4s{3TSVQ=Pa&0wkQx$F98B$M^S)<}{pd z5XtyLlQKS3t`9*33C?rIcmScQslw>_=Ra&F1B2bGe{R4^#+t`_#DGLxjLgJHx4yH5 z_k~3fH|rY`OOkQ7Dq%>)?VDk46HO_rJJez(Zn1mrMxys^$6_`9poa*jb4Qr(rLZLo zhkEnPH3~?8)|KwZ4D z$VnUIa7)p8zd8L@dG5l+8EN>UJ>n@D?^HQJByNFvVO)j>N-z5rNU-+b zc#k6wGcW{tEE^<8U5~l}k+uJ-A7vQM@{9&xH67X*zv zq2We_A}PXOq+Q4!7VC9_IiE#SG=gq(YDM#(P|qr#m!!RiHPK{z@jy#fZmGaHzWLK) z5%(koOFTm%Hx>TbB&bs6;DHe6t;M{#)BiL~e!OFIot?HLf_kwPMKJ~yiymB ze)jb^ezxL{P6Nb=4#zNpqaazo+dezWWW?M(HN00Cw8Y4G>2}i5?f-b#z1PWV15;3X zpOV`6ey|IBLcIa%V%Fpf2ms{8pxgSj_BSKGgF2?8XbU*(=LLSd5AGEig>mVLrGV4^ zUh>?@ecf#Y`+WOQ1gj zm}iu1McFrtab62JYMZBP^q-3f5*G*B+Ygbu4m-^|41(zf)wE1H2VQV0NzWp8d^sww z_ycF@rT~n`!o^M0_7NC!c9I6!1Ida1^xhJ(cByp*Lf^jT<}9C ztzUHXWqC(OPz8O}QC(e4J={0XN=jwUNh7EqW(4vt;uN|;cy>eK0SATiX5D$wYA|vQ zBnUcOcm=;GA}qY#KX>!e!p5MzQW*dh`ozxPdB<*5^!bV6=ZN#V$Kh)l@G#mAi1ZS9 zECpc>zLiC9eZBz9sXYXQkTrmf67I`~t%`K~oXxT}*HgPth+<==mlBykIt-hU;x|hl zU?n@hkTL)kS>)FVm@;53MJw^puk&rZ`6n~S>Ua$kkV?*auAXt1efBSen;F-k8Lz$g zPWs>JmQ0eZF*L(qP;${+W@*sQAoM(ostl~3Gwkpcc( zQr`q{(YDgh=Og|>v=3$Aj6c^v%nQt&Lv8jA4h~w?;Hy~4Ba!YY0Fn&?GZx*+Q5Xve z+D7!dZC z#TAKUxrOSQ2azUWSuHlG!+YSO=1ZVG{voTVFOQ5l2kxj@E4e{|8fkFZm0$pk7R^`- zNCs6g!F4D~ETn40QcbCjz|M)FJnRn!APQ6sh+yF1npzVTK7v;}ylVxn2Wn(`bByH+ zT(J`oVAXF<7?)H4C7Fu^QPLVPfCd4+1T^0fzOto1z6Ab2{f7|Qr3@4qj-(I#KkWT^ zAe8(2KaQ8uqD7q)m93nLC`$|3M#m{m3rdn?DIrNn$Tri|DJO(VB+CdjvLt(!%(S4a z*|!*xeap_6<$FCxbz1Ju>-Ty8{X70RQw{F9@8|P+Uf1J#JRTPSMUgRaGg~uP%0Ib- zlO$^JB$v(U_j5WZVU0hI03!F^P19n$eYgfZRuB$7Fnjf|_M<16KYeIelU>V+XsQ_! zIcybCGJn8HVpuRT=k@aGlKVU2&KPDp*@OaslIJ8G@tG~-SRSs=X$l69=~CJ7=tP2y zoD7;*#vM)A+~e|&P_FYhT9i38(pk`&=Z!*D`&LKJO?@W~UUc})R?ok}u7iDC-hESd zXEBcqN3h&;OUJ7*ke~=9B=jXe%KUaOeL1PR4iWjNc+gwq7YA?i>sE$-qx7H06*t4! zs6G0MdovM#wei62Y4z)iB>>i%p~~a|y2U4Pk8%QNmb&KxuQI&z_q796sf=q_SB;>n zs~bJVsU<|;iEtyPPwK}FWtU0Gts)5@%_@%DvC30Ne~ttf$#B99RBgxL$IRwZ*^CG> zcP=nQm|Yt6InXVW1-`@1f;SwAMnh_lycS3_&N(z?IktDKS0jM_#ipTXZM%#7xw>VW zde;;c9GPL1KHn``=465}wq|E1{M5Z~`x(yZ6s&Bz0N9e)(6*?WDRm*^6^@~f5Qx$z z)4)vj<_j>{8nV--a3}B?j-5ZRTQ+ByS5RSDwo|_sVAkeL7boVz6&q7mn>35h|48ziGfxk{bt@|jlUVQL`q2XG%-xBiXV+d=)hqr6B8Sa=&RYX(?W04$u zcu(!CB8xOt#~_m4TAUkYY~oz_Y6?SH*to19Af=Nm%GUK@O}V%*&)sdI^&l|dW8xC4 zewR-xb{!r3;st;N9us$IehE14FXR%$tT{R57?B&y-2z(woL#IzW24zL}gtY%__jNn*q#Bz$|+w=apUk$+r~ z<(T}6mb#hr?rkip#X_{)ftE=lx96{6WR!Jdb z1Wx$1S)($K77rsrWVG5eq(23yTMo1d8)up=ek25uY470(`1df^rr+0vF;rky9=S&r z%A-k`Eqcty08UiKC>d8GH>6DOirV9rISjM33v4o)0(Yo5O@e2)`CO(UUX&vL?sE zKoeEQ>MKh&WdvOQ2;6z_ZSIeWCX`(Ds{!1&yO5Qot-dpUaD&h`kCo@$ z-X0jn@sRBS?9rC|Tn%hCq`EBZcvGI(+Mn5Kuq)?&KbCf*MU&Xk(J_e#X0L%rlU$0Fou!~q6`L29P}3GEWntsjP?A_x;jC@IO6SA|81I;hraL_5L|O znng5QF=@i?CFDU`6ViVXh92hqvs~;HWb+yaZZ{Mee1iKK#yueYBPS)D2J{snkSi7T zp!^AN&a~h}pwyu=n!`|yXGnb3kc@t%rLa)j=8L0|+62!>gON%_P*C_x+6oo=(CTlU zhH6mbr)tpD6q@HE0u%LmSVhdc8)`hE>^y1wAsD`_o5{Hs*fTk%TZS@up8wh`jCT>82C z!#>Q0+T5BB^;vs7KSOWt`nez-)i@6Etc860^NFii_KY-tTExbL`SpbO`r@ONd-zgYLR~zsz+B5iNm?tGiA;y*VV(>*UEqY`W)cJB)fikt!Q@PvfCRU*tX2&H+?Oo#rGBmbLH)Lka~a`oiD)k z@Ty(wt&OW>SNx{z__&jwpIF5rYtB7n`r_fLDPjtRD9LSNCi> z%JXsegm%uPz0-0}dcWb8k+WvyFEm&LzjC+gQe=G+r3;l{mwVIy@a&4x5jB4*WtkM9 zLg35&M=SW>>t47_;r=WJ(G$wE?ps$Tf}|hw-@`4_LL#uB)Q)&UVZh2Fa5KLF{0g;V}Qx6 z^Qpr$o!;6WWa10d+X4_%EOY26zJX-G9i?Gzb(u4`2Yfz#(K1sBH&&O&lCXbyIz+SO z3gNRD>O#_GF(kH0n~vEST1wPUC~698xM88~rhhMtju=Nh4pi&+6>kL==OlzVr#~%& zuXM9}5?Py34Qts(2CkPMjy1Wi69MtUK>hCO?tUzgaiP8N*--x4Ho%7Cpfv&wO=0B1 zaPi{BfDkiKn%^o_DoK3&_;Hu+=U<;6PPH>H(_SrGxp9_dU&3Fe_YXzLm%f#V(_OHE4{vZx7hagqa2Pt<`7s>X4= z-m(DDiB>X>X$3-c`L7-fjWb0ZacBdIr#9mZ<;jhN(mA8LEq>n!+lY6>WR5&#lnbhG zH%lu{zM1o>GU!sAO`N>D;1q>D!aAZoa{c}tzYz`_zW`X5?8znlm;2^#+3hAQ zbH>L_xVNP-4<4*T!hY^Nk+%{M7k9GI0VjZiYHhs^K4!|p!vhgnS2ZCZ0@PiWV@S&a zi2rR5SyLFh%@Nxxl847X-qBuFpt*ZO|AhUX8bshn+?O_hmwb0v*PI6L2HNM`%txE} z;Y~PBwf)4u{&&Q3F`cDU+va^-elCIFnT48GdT_UVfBRqSH*qIldoGe~F zfrbsj8=?9S${{o?k z9;P02jnC}Yy(QPuY4EB1n}8$ZI`{B0=GEGRZ-l#PNilK?m8tn4 zU(HK4FPc}936N5oCysIQqN8ITB8=%9IqJz;^wuQonYzCAym+C@I__>HHBQ1a>2-kO zH}`!BP&(^S4j70_Z2n|_RJ&fr=9{P=dYlLj-nQOL14rmdbDKJ+SNH7d-{xVE!63{w z81}5FfVfNiumXaxd*Jxjt@j0=K;c4SmFDa|8f!eG4Mqv>>e&_yxF;ioFk*YEyW*k7 zj*NtaEo)J&18s)Zh)f9`_RB@mK8Ui8BqHUs=4Rt6X|Q~WJh4^EI7ea88|~Rl2{%Fm z>B|PO6DKOm;8e)x?ny{mGz?Qo>LuUeC=jkf|G7LZwku_HkhN*woDr!ha*z8yawM>l ziL$`VAQb1OZpCnnq)x852g*5FlZ)!=uMUBb_GNpSPV_ zVTx&y&re7UIs@;}Ub*&IbE4c#43|-zd-&Z=YRJ&g@$7GawZBBCanu(P42E~+(3?T4r_i5TnM_BCy-LuuH<83w=z-`Q)B&?t!f3h1)sE__ zsz2#lX3xoh{7Td0cY$ZCY0@vrv=^>Vg$pMZ^rHVWJbDHTSR}3q^{-#AhCQh8FY)sS zrNfE|B1`XXV9W2Ld>lC)?l)%3^a~sdtT~%DQHF>(#`%SF=PiU|=+8*3I5(S1W8*vM zX>uo#FOQX$maf~>PMol!j&@gi+eH0M?Dp$Z+I{A9;sKbv6PU zpbfEKo3fod4_*z263`$O&iq7i5$y-ZcalIz6`Oq^*c*{TD9Z7k`1$Y<#Az}Koc}L6 z#%sQ0;#V(ED%7f{Nlj@#Zx-NZmlo8YjF<1VQK&V{O)MW~eRy!CH}1JexCzOUOMPM0 z_F@H!ZA|Hf>20{l^HuUy1%F?qBtLs_X+TDB$wIje4kPfygDNDne!2Lb6fMCf!)aquRVsqc?#7+|&mZYTV2p(=u_OElyxgr!t( zCyqP{IbWDuYXS<)p)?pq9iyNhe{4AfzM|v?$S~UUIuS(*g*^qJRJIZ43&$vSAa9`@ z=(5^_-abDgcP!ylaYLH57;TWoD51Fz8=ox?TZxWo4ofv&x$0iw>a*WIjjuWIKWA)k z%S@!r-)e8SM!ugj08G?mf z9U!e-aqd`R2%6V{5kXpe)-VmU<0#RRs=I(K;xVSiP%@In`&Zt-Vp(r6$3UAEM(82sNkr0J{|dnxSM5cJZWem>B4i1|~L~Q`YX(+|3mF z#Z$f~6ERnpLOWOWl@Gkmho?gMP7q>j0J2B+78;P7>^ukUm_0Hqh-z(!G!As}BTakM z)zuLhJn8h?+wl}KnGDoj4}=|_aIDtT>pY9K7A{jiphs8S#CYv3B68k~1NA8I5bs}K z-8mujcG436sVT-b#&6+#+^n{;u`anpPqK^{2*y2E=jrR~dP6@PF@CHW%*u9ceKK;!_!U^h z?R9f=+i4D>F#FP7okvQ1Ol(Nt`0fkR+CQD0or{$xjV*XuZO@&H-+5#$=IP~oti{W7 zLeIzJQ`W+AIrh67nwvMqK=jx;H@~P81Swkgw_6=Et}RSdSyHsM|6h}D&Pn+>lWWtD z00MYL+p%=;flxVz`FsSl(9>ZjTbROF)N-Lhh~iET39H!B@;g?{^s%#N8(-7aL5 zKnXe|kb67`Qf1;D$V}KDU6nBCJD4As^Q)tIQi~H8^B=4qHf%I6#yyt4a&@_=j0N}R z!KY^9az+Bb{LX)O&X?;lWx&r`A*YBp@z4ix@(n&aXO&x7<5(J$FW>Ux4}o9jJ#X$u z+)!(QSWA2;7yS0U6zCF_v)5#CVXOws8;kBw%@}1ol$Bn zK@^5L^25?djcQk@IH-%icqRatXSVX28Z9dNE(* zHB$IRdT;g0bm$ZwfA*z5{{G5Z?6|g!NIEEFoL91YCc$Yswk!a+L}d2ub>F{p=8pKN z68WlF@6>toKphbr#(;WU81bK$x~y9(f8GbgoeHk5uAB1{PGrgx{+c*ve)46yy3tt^6BE?W$n!@Y z5%)tvQs_{>wgwuWf6Fz7&VL|A+D9`x8i$GxPG1^+yU7fqW&Ob6jSLWYu9qC7QE&r* z-ChDi4>M@>BB4bFCQG&D7&in*zZx=b^CL=bzuL@+1Ms&0I(FUr;-5J|aJbIpL8%4A z=9kwNiJ7P2+I+Sxb$j%s^lbZ5dV*+I45*;tV_?)F9S-hz@u6&cWrytSY%`$F*FS;P z3Ii0XAd?k9U3UCE1b}-bqmA_aV@JE;x;l8(eb7neMUNYH*-$Rm2@Ed-qOAC*yNT`_ zSfkqP?T@s=T>sS1fPTlh6M+0eci`N+P^8(ftA&$LubiOlka520cwg^#1*Cx^>=u_4 z>^ep8j#?3;;Y_gpYQ=_zt@~NBC+|a5NYtLYc`)|a&sPn9Bjbg9ACTUYQ{?t+hP|3R zv}l#oKY-pYy&u9g2K<1R@g88dw~|?~EdwyFx?14w=mZp9%k$>9tbX$32`rQ?&YY|V zGcdsXj8jPYJIQT#3G?1*o5I{`O*Vq5}RumW7bWLlY7}Z~cW$C7Q zSy>bg>rGkN+1~aA+vLF=5P-T$I;dS`jj- zYRq>=uGn{ZoO^tbCIUCF{+d>}vdf!A;d18t$|t~_v%rcRwhDf0e;@Kf9XDupw)fZO zY0pK}A=8ONwSXZ;#>H8`%YiTuc=!*fsg_*TxRb;gpT8d-4blF08K~z`8tLonyDssYny$4IB3Bsn zbx~+e0#^Zd=kLlo>j%(jLcU77D|Ow{ws^>#be~INRi9KDRYK+J6y6GzsOkn(S8?lS zxL>{8n+JQ6W|mY*b7PexXDE1L?GsgB1TQlYYx9?X z_tYoG_DhG|8LB=_Z=76gbH{b^lLV?lZ@w^By?U^4i+kakt3~efLO4wPc|lW)cXCMg zi?@!fK6m*12i+Qh6CK;?IP84cXT_FJIYfLjyI}3}a+4Wn9to>vYL8i}RvQi1ln>l_ z(srhqw=A)78}ayDjKB$``9UrTU5^In$^8}<7Iswum5Z0gs`XRG!J5L0)fQ}6dWiz; z4=-lNT_9SR=Rv^KTT)k6;DXh#y-dBuBXwVYwd(QGKtlAi3}-3;9dfW3Y!54$rQAR< zKXL-V?KtlX#+M*BiGo-Vm!6vXv7~-vsMrR)b@AL68_GXdW2a~d2?TD_LB2r zzdng?U$ltNJr2nioLJOD#XC8I$3m#aiq;fj-2$Ke#A}QA^MN!MVl)lj9c|b6jl~6# zrtuO8aFgl4^j{54g)Q{MA1-z5{9KRqCqhxQUYrsYB_{0Fm(+R;L<^FI*j3L=4lJe~ zfj=F3Jq0r(raoSYxO^NVO1TfX;WuJQ&`Xu!J3m zg-SgN1$I-IH^+#0mN?oT7q-LtD6*=84okp}!TJ=atq@llG9HyAz%PdbBHfi8z;d<~ zbWArl><>IW-P_$Q0VG5k$PV}r9BVIyeLa2$8eyD2@s{i!sK87`3XdzWi3nU&ID8mqy5P z@8y(!uD6_Q5Y-4ETJl*7#@WRFq2i|JZqDcGlQ+PzWY+OwLavg378v)%3GMww&9%R! zmoR-OU|SpRQq$i8_ja+LEj0xwG)l`f9zU?{)3rB=4xq}OlkNN1m~bHF09NXBnJTaf zyrg|c6$1M&bv!!2=st;IW$*irlQ1yOy0)cs9=k1kWrf{1SSnrq;Z#sPED|A(tEJ+~lr@q#y~ZJK0% z@(JQsZ!&;Z)L-+Q|KBHi2&{#jU69|~wfTqh<$qsL%kPV-NFUpF31Yr=Zz>g8&)}Hd z+HID&X#>Q~gnWpfG8IBu4d|tEpUWVmu1IGN`>?#RaV@#b2cH8%5oo%r)PNY-w6MuD?2FL5ak zD)(5G1iH3?8dJ?*+oMN!Xe^PS0@{puPAkWdb18D=P*G9YW4Jl~hY|}p=^MLju>R}CC=#?+)lN`6vcOMG zN{I!Q3n|pehx>!}7TzP-er^bMR3jyO6u={KUluaS z1$FGau(u#I6H}=er?6_-v0XGU4LGVj61c_iXnD|PcnanR!W4H{BV+%~bX0v+*#ZkN zGL-+YKkH$T@qImc+vZStH|)&gDa0lH)q5KNk9yiys|6P>6AqPEbX#|!5q6trDGv+< z$hLz{*ZD)j$B^E=W)#jZy^CpSY35_df&$FTCFzGZaQB#i@4dk^bmcbF8yg$F=|9w9 znkK?us3utJ(nke+iE-zVTPEF$uJ(b&&RzK> zuByH9Daq-uI?0jJq!7z@iCv87!_9-x%ZXi5{xk6p;k{S886+ZE$swHZf1Mm^j&%0?c-AO@=yy+dJ7QxeZe&d={`1G<`#?4uKJ=M1~Kx<*2Y z-c^{=OP1zdoOXB4aM9{9&ozk{7{-?n*Ya^;6)8u37bf3#?%qdKTV{=I9s<9itHZ&W zkpN(_&l0y5>Vq2)5MG-wt4;JmFRckm2bHr1>_ya-vn;D--9vR8<8vJsSPj0$O?{a_ z1!hF+3;+^5G0#cF@p)omVo%W0p5ETJO7LroV0r*g^cKl7vvLx@M3uR;=IU4rtA*8# z@4_F;Y_qpLqBgf}Z`DB35zGU_Py@34 z+|--`428@Qy+gyp!>Fp-V>l+^=KY5IF)EY2 z;gyN*zM5rBjh;l2w}(7!i%~QTD^Yx%h!@U{s$cQbkwm@3UpMvFya!3bYR|#5r0=1y zWNjvB94GnqAM%kHO`2VX*d6sbXrBple#eGS>60xC_}O1DsRVRMZ2 zJ=a$kAn|E}*qPjr9T^^*rCm3wnT9U(Nh|ZX?m7icDpKR(f{p4dJMjigFL$#)T7Z+| zJ*r9BNM|!C>>zFm@DP|Hs)!X)zvEH#@9k0(01Opz4QJTWRHJtnULmD}GWdwkUk;V& zuzfTsJQ>I3?l2C1*uItho7(2j+?StjA=)!q&(-=LiN0&Vn+lAd;)m!ZfJ6CKnRP#wo_d+U zA1x0&g}spXjV~IZu~bliakA16KN{85(^F_c81goWuPch&L5 z?K#TaKH;l05TiH&!-H-@61W%o=EzYe3C4h+h#4(C|6}9lR$WnCI96ixdC~3t2ZnEm zeY|G32J_%jB=Vgm07b0;ESmL0|13O4tE{|JT?RVg$?*(|5LiO&A!N#c#o9CQk%@Zz zSi-s)IAJh{20X@}fr-(n7gxC3CH7Q!-D~J2>J!*Ro??+%1l69f13AnW>qAWn}hIKGw z&P?V+Tn_Vm33HzdXkm4ZiI7Z#K!r3_Y?)k;6nd~Q5_e=H^HsmJd8IZlcBQEms8;l1 z@sxJJ(($atQyZ4-fp{lq95FoMBZ$vgj`f(n9s}4fsQ*!6{73e;V41=^$Z85MTVQ)9 zV1KKr)B@C|+hEh=WGZzc&>Gem^{*8%Io{!^uN0SIJ-7e{tpv9Dj+*+rs&O`c(tgMh0>s9z^Xt1CJ8(EFO zoF?8=fa;-%S9##%dvrrA`F5%vS! z$AxSK_>qP#jgd83#-76YCAzmu9@5nv>CHcE1s1x9l8z_}k9Qyu0kY4|HxDv83e!!_ z(Q;M5<2tvit4mzI0Agwz_+ylH@H)QtTDbW?Zrvumi1BFQk{6gyN#+l3VFF_a`Y#~m zvkXS#eUi7a64_&ve0j=LiDq!${PW0}9k`)$pwR>2{1iCb-vY>D%8CZ(ih#nFIA>tu z896uU&22*_O!g?O0beJXV_gO+8uOu!P<@kD|vSr&} zvxnFpa176T$}Sg<(kKK1XRsEUnuan;9DAzF3?w3X2IN^xnD~+!Lr#LIWZ(hq2m7Ke z8K;SKq_qN#nF8clU`qB0$ej7ml}Jtx_>1i&`Q%Mu_cL&$bIxIoHzHbnyh>Qe7xLY= zr^{@vS6iN*F}6uPucXtr^R8h>(bS2{;;`XbmT?UPdQ!`On4ihH@6^daG7NfGH%@KPjSLh!^U{qOZ0MhBbtyID6T`+<`&=8w( z_j9Uefaa0+9_%vOOY%mL(nFW5RnlYoeC%j>I1iY8Z|Q+S1L}uL7fEQcVPIEDEGQ64 zLsT%CJEq6Dk1w@&BW;@)l-wTFco!W=rcc${g)Z7rFvkWg@Nq$ytwn7ez%Hn?jp((3 zJ_0hGu*QMemn2=+q+t~dI{=&}zH9$ozK?_t*Z3{^p^X;T1y!kZ5B~|f-G{dFgC%UJ zUUpX2`mnE*%I}{lXEmCBM6B;!iPyGa(Y{;w7BZp;qkc$^U*gN+q!y!5BIfnIG{B3F zrC1yUlVjQ&Iyi#~vPVg*O0Se+PGzSD`#03F#=u}le_~F%!^mQKdv4ANPFXi>QHHsB z6;5&ia7&UgRDuMyZ()_+^Zgh0V82t52uRMPAA1Ow?+4@M5Ws^hkKYKlKX->T%m>zx z23RO%?t%>cZZ#_L;Br2~j2smeM6oA_IUTPFv#cfUacl#TsaCs$&yoY;EH7gRfto+j2EmCxZ6AqVR$&WbdwREV@abw>NHK zmB^A#62A1kZIi8ECqAU(s2kh;<=+e2n>McyKnMD=8}qR0jQkKP@S)m5U<}o&6QnB@ ziJ;;<2E1&vb2>^w+HBe5w#eFZ2z+y1Q`i%1Re&G&lnL4&klM9B^m{*?U*T-^?C*B#*2A#*N!b2 zll@?tPQKonNqKMg$xy_~zNWU;Tx4JLT3aI_=@>fo5uec|5>FY6aFuuGuaFXo%a*BnP;%Uc8KXSwcmt(*d1z=gng_# zBnHavgU1rUeFlb^+`5bA1t8`i66hPuW1hXE>(55gc1WtvyZ0K>HeZhiMFTp;<5}{A z5qVNh(D93w*N|~2i&4gA_?vCEjKSu7ba5YV&bqDs~QWoSRc%kyRagsv?!20v-c}g7*NppW6V{0?`E{ zRiwX`rPWZw=InC&m+)x-Nb%9ooi@j-FEiakvH2F9qm6h6i9SAtC}A=U#57xqx0U(u z^@XSNF`642mAd-qAPbu6B$YK(+AdIme9%i46qlO|!M}fNq05JEGRZgK8qNl0uevyl zgQ-O*v!?urLE72s2(ygz^n4imn4C1H)8Z!UySn^K`bm{pi1J$rl;7SPKPL~dS)h^Y zu5($EN5T{A8R>f5$WkGm3DZArAU}LG=|c{LDa$05;nF<7H^YZdkd#iCt=3p5%`vThzJ_C-L(>cTi}b7Y5r_B2con?h4Xp6oKP9&Q?zG>L zes!|gF3+~rci8*7Dq$w-B`343U!LO^@Dj>OvERA|21@h8VBD+5c6{p6HIlUkFl7;0 z`I@!9zF}aB#~V@4&XWJ)1u)es2X=MHcr`gQet^zB#kRR82 z6;z{v60s0PjsJ&P&l$|4uT^S3;sTBfQC`oH4vKMMQ0?j`Kj}JyWf4=*(xzra$ z`%Fm)u#Tl;=sLeoeOiv7;xa+uoVA1@{n(EqyaXv_5HVhMPf{I7IGNWKnux? zKqKM)9L3K9cdX0A_)+FEt*4O{Id~lD31x%7Gdi(S zp|>&(cEeuKR?CNIYwYNrne-pc()N>>tc_&~AApH^HVXP*H~4YBfm18AChdjL2aY-f z%jt}JHMZZ^tIeOFL(2STBIwEx8+ALb=G%TF!t64HQEu-cnj+U@0GK7w3?-2BNh>NU z7M38tiD!Tdq)hGDv16qNc7d&Mh^r{5N~5<@BrK+>5t&}(2T)sRl5&r>mgQkqV=bS! zfZp#Za`B(#7-jW%h}#XM-l1xDUF~;uQ78 zFc+5A--!3|sUlztu$`zk6aqY84VhNK(!lkY{K1d4c&I}>44C;fw$6FzgQrM70EQ&) z_XOm~B+yt}%a6c=Wa$WFw+0|84t&DGJ(lIBZegpU{fm9O#P{7 zGOSdLs(B83IEZqhoLwlHh>nyRCm=QbRHb9-YHZdv=D>rDFyO1-`8*I-N8mDp7 zmTW2DpdPo6J{YnS?1|zjKsWKqJ%Y51-n&pCy@w6k$eRw+Zgm1i{@|&g0IxQs2g-<% zUsZ=_OGv68&`k#pw25Ug?!%^*SttY9070 zzv*yo#C!j;EzqjOKO zCH8X^N^>`Zuav$kb^;u+4Asg<&x2nCpTCJsBP{G1Whht9HW?TS`vbzgqN%AV@0oHo z3lL!MbmBD_BDO1vMYZq2I0N9ro+k!QU43;x-`0Nv;aOO}A1ASzs(x7Rv86N|^Lf15 z=$~8<&M>NtG?2Hp>XPVH5gv%OjSAy&BuMNjTl)z7WBAb&bZNPM3PdOaw@iw_98%04Kn( zJw-4XP2`z?KXbm+CqC;E_cppt_LuEt5NsKAX89&#hC*&uDvnX*njWsv=&IETCaMB} zwMda%j|%rNM@CgWZ2DWI$%`9^*Xyn}(af3yqB8Q^-;SrXZ3kww5^A3V^k>zAh zpY?Kbfhu!*&m&olK4Q9KBZmARWp=5d}LqRdh@nwh?7^>US3C?_KxY7ihXcLOAx@3FQF9)jgexmFCNw=SD`TM$<4!qz$5|aLC}@SkTbj+`%EcF4R!o0aRCYhoRLCGY-}0 zg8xtS)-uyO7Z#C{^IPOdFE+LZK5);?+P;}wD!tmBy?XfiN;b&1I$mN%= zp~LaFs(w7={{78#7~oa)4Dn%ebCRK*7Ow!g{<@UZPBUz1v*pi)%9o7HBmFMqGgder zm_`kOGs}R~y~cP$CzePAx*=#Rj8i-~=4{J_^#oEJRG;R9on4lA=QT13uLz2nLNU2;;?a^Jru<5DMbZ`cPz|3CDttKrWb83_+QFdW^4yaP)diu`8{vHiLKx##O z(KRgu*mQg8>(zefi^9WRRgeqAJkYwd;ui6PO>U$>*PmW z&Zb>*uQvMPknpvbKrgP&HHM3;14?NPOPbwIiC?3PJ5#Y~2`K70V?F_?O%7CX8|Qm5 ziN>(O-Q5J}wT`3f|4GsKc9p=r4LBhjcRQjP$QW~|=8XHq2J~L}O|G7F{$mnQ6a8;cwGB3RTCLXA2wdtiAiVp#qAQf#${r39Ik{f_dNIn5kD-C z5eaXq!_>IU&A$!}-1(_ zk&?cy^I_?K$sC=YaE4?3lrt(+bHRf&H@0zyXRujogLxrhD<8Y~RC)}2FVoA#5F_Tz z9UGO&Ea9@in!Gc06}Fe3p4WIK>DW_xNR9P4#rZVWo!xtniugnFXd)$4a%(PHNQkmw z-lf-LUGmbN{YxbNVZ&MD8y+6+#kwo0pO2s)WU&g)g~{LOA>Rw4X`@Bb6y*@YGwC@; z4jjQwNF%WSzw1|hK{pBiYK>jzdn7$}0k&u({#-$|@5dic(rNer?hm|7J8(~VCy@A!j%ZljQ?;zqaS{}*zu@W|@VfM=z$?)#(Ei3J=!XwjF8ZluQ0t%SwW_|5c~8b@$@40dkw(;*XO$%2 zL$Z|rVQc0eUAygwA%1H$-Lp`U2{eVLw++!`T9K!AV4CXKbEryh$#s$Yk=r0NnN`fQOknm?{PDESi#kBP z+1VBw_hg?@eGRk@fjeDNC?Q4=_91`A|rWE^IUky$er>I&7v z@7#q0!mreh4BLgWsUAz>gZ%7csH|ay_I*U6iL3a8aL=M;?e^+-dD8|Tbe!2F^Ba_N zey{cOc42ii><)7uR>L=-_G_)Nn@zPD4H#4QKrvoLoSce4d0!snC}>7$^9f}Wm-f3u zuS8ff@K>#nNyN?!9FiUQP6b-eKYQnYFp*=Hp9wZu+bwHnZNql7W7*B`;Z?3VmZ-nz zT;eJz6PpvskH_ggj@xQAOcmh5dC8AFu&!HXGdFUP?#sbVWR~p#@;Pdaq7xO3KQRO5 zlz}Mz;K7oJUdZ~28Y=FghFy;~kq-7jCy}89f|0#g7k%8bp;jM0l{W&O%rjH4plNPJ zz=m9WoQPrztF&wHT^Sqc*IE5(L^kS!OAK|AxO=oZe_^=DlEkMDg0_eP*eE&Rij|bb zK+J(&&p=6o>PT?e>y$FpwLontyh0lxN-R+2?qwr~b|A{Hvn>suk((lz4Y>*(^#C*H z!je6gSztdZ2_kS#O}`v6iY?BPSk(!rw-+v68DrAFUN?|Ayz>YZzbl5K%+f5HGpD_1 zB$m5Ge^h^}e^xslQ0V_w-}|b!=u_x=vf1GE!J1&RXa*|WAK?7xzGeA{02B>WXZzu% z?%*E3IZT1|eU1S1uM%~=Tz84$U_M0iWd`76;v2!Yr_TgCp^VZq zHBFx+j+MI_mAai?c*(@$^mAdcp>zMWq$}paARreR7O-Z+&0oVHKnYO-SdtAviN@)r zOS!QCHu4XnfsK*(0XjJS3lE2abF|g%K7tqYZGxjE*rlO zKwr;s;*hurN#p5hem6eBCmi|9*n~}+RZl)L(CN6Vj?qQ4R zfQ1x*wv^(){O`=W`iCqUJ%Q$h#@iI#BPG8l!LY6RX8|k8kUjv)JFlc@EjaImuiSL- zKA*%tI9f*;Nd%2f0Ih(M?V3?eOWKChl0LR*#l{2w5Qeo`Z# zdR&HvTJySv0eva&9&GKE?aG>}yjjwU`>qKoIh{@ex^cj9vEfX_nTM1I>eItZ-;iX@ z_BOj0mr00XAD-yZLTh58+OOD3+3z=hT)2rx?azH`7<;@?27HzJcA38v*(}3CWbOxP zp!~ z=2!HV8Q4@z4WO0IKUwqqOo1UTY67K++HRPwsW7_Mn(ScBZ;?$0aT;-`xVAI@Yz3OgX|Y2VC_`o%)q zn$;DFT6xLa=x7KKqPHyn69tROI}kbQVAxv?u+a#)t;J}9Hfq$@q=PRL^l$mSSU`#w z3kM2vJRS{w1(LCM?vXR+&iNpVerw?0#V#lu?CScbK-eu_00G63@jvCP|62*r+WS(u+)2LmtdPnk@<s-G-RTC0S}Hw zom;6%D;i|odAY+cft|b-nRo!WbqS0x%)r4O&F<^!>nrwr{h40;D<`1OPa63){8~Yj zM1_+sMU(*`!&8MjFjShmgV4R-#0#=ZTH*cWQpk&aXJ3pX{T&9%v%Z$ zKrYJJvKTEl>hguCZq9$G)}NHq56sd7#+E!WG|&qob}_hTm~|Y=NqqL1jC=x|2ZSJW zYE_G}&h{e25M|nZ&${n{m2Xl~QiBerK%EoGi6>VI#F3^K|5=#0ZNup`z6p2*r&A)B zaG1b;E;uZF!COM4a>EXJc@0;=qCl8Q?TJl#oT*nYE~xVLt-KhB`4j(LmKZD6^fWTE zGf4DI!7N}fN{@dJqeC#FOO?0cU!f_#$1l5lf81pifi5P0l!CUu_+Fo z)58{W+k#JT44$ya?HqA&fnD1(v9Y+lWH1~nO8<1d5jo(E6I44Uvlgow{3*75djn&Z z0XcK!g$D<)dp6&`^~YOWh-Q?!81eoeZYmE{^d{^03%6sXc0c}}@2>y<|55+*_I${P_JmM3D~db{ikLhkFt$=bAqD&!3t)CmmF^wm zu>BLNkSW+PV7{_9VCi|_Je53FQ(c{q=3&zCB+X$)b*gp^f@qKXvZ>^#>8S)Wn3VbDtiuE50#qSEC4 z!Fw|6kk;gUHkxCHjATEo=`6UpmLFL!fpf=d>rlg>E2#rO!B#x~e5@wE9?={j*X~RA zMpLDbYu@>?&P`jh`;yaC&-mAleNLf96RBA&ckL)?z&%t^6~}`g46T?$vVhnsih*vMs!&AEMDt)}f%mtgbnKC6Q{@S8 zpAw2~wtKate$F*iIrV`(giz^S;n9aSR=8o#-7wjX5R5f?0rOP>%URg^~NLXd3zOPO>H_w2&vR@gu;XcrM^WGs}DVmX9wG z>|w)>n@nWY<}Fp~^WU*wsgF#32F!H4P18j;)qXj*LJ2b8)uHT!V;3Ujv)jl$)zx?* zlb0K3##ZmVPo*>5(SQa1|~0xfKwDPwe*8U1<$5XMfIR9v0YZ46Db$K>-_$`fIO?qmTUlJHJV>w~mM0f$}h zeE_43Qm2n)Jc`Wnp!Be;3p1GPyqlMWhUb?&OkMV$0s4BD8c*>)qM<~L)WcM7cb*d_ zhFNeHw3ZQA4o7W(*jRR@s<*c{r7hL457Bb($=e!pSR^gP`A&6K2;wW`YM@Q#%jW|- zwp7^D6(rZRvkPaAo}AXXRY~J|We((_Pq~L}%6#n6;&3q0gz!9Py{Q zI9g>!d$e1401JqvD$PT_&V2eqZ^ZmgPy8YANyVe>9i;P9t(gHZt$K=d35JZd(pdep zQ-BEd+dUcCO`jJQaM;d6)wok}YRDFZ{hAY65)akj=AUFyP3{*Uc6lGGj}6nsTO+P6 zEetUqRB0kTbPHyL5G&kqLpQYfrVsYM8I0ph2ID;w==44){UP%W=L(50Hn!Yiyl~o+ z;Q7G32}WUZhT5(&0Ih6&<3uxw zI>paDHD_tax#Yk2EmwYqbLMP7H(%G(J1_lP$8YWvp}+I=GAF=K z@}3HiSQ02*;v92CK{LAqKUHMnyLy|&Mb+y5AvA&`Y(MX2&6#MWHnlLmw`_u8c%@=r z#hjZ)4@X5$X7TPz1hH`Xv4^90Qy71jH(VMFS+P0q;zz^s&$|!GLWNMOrznVsyQq{% zR$sd7_vwwK3ZCwDg&;!mFhIUSUK7V2i-z@fv!qmx_~)+f0v}zQs+yWWr;zr_=lc7) zPZq*TPQLO|#KS6N;=_z0Y4Kj-4npM8e7)m8>!80|$k#7ZVWM}l)Enjg=!V8o7!#Lv z%n?eG(qS|XCB4@Jgl9P&I}&feF>cYWr>B>fC829%B!o~rYn%ZNUJbH7MghKrGheoW*t*LPG4#?dj)IB|U z%jr=Q@ap%4B+*b4vDgL7SdhElADg`gZvdC^~$hYg|akX91~%E@-%}^aIa4I z(u#p8NL;$sv_IrFVYKq6yyAOpob?KC6;|Qi(7ogN;U0-g&@Xjf=}!_~tk80K&c5tE ze{E}{S`cOJ&Q-z}B3s8ynyk1+k#_LJ-0Cy}O6hfr^0i+OtTGw{DqhV-HoHjXBwlmp zb0t@3XbP?V`ni>$MMF~rO&JxJCt$4MdKdw${U`lssN-LNDcr){dj=<20dT)7<2od$ zs@gE|sw_pQxzd-}jo4_Y2XYsh}-K<{$28VsT(#paR&-v{{=>AHT>i zP~#E{$Hx$|i14MyVNK(?l!JZH?^%0VSP@ttC`iz?MJfDQ=%mr&-SyGG zuaQtO$GO|MqX5UUM`X?-;T$CZv|2&ZIXKqxxF0Zq6#Ax6yFU^ep92fkpo#H-GO*^+ zbo8iNOO~qmJ3RvDl!K`$`<;58zFO+tT}gJ<11s>{)c$1kYn<&J@p5jTzsX7d;*0;i zT;7qmTy)%e96%)L!*?cdaDgENB?#e$?l6F1^HLX(W33#qE<)@rKWKBE&Td6rFzYyD zWzwKKiN}>kX1~$|Ip9j0{N8=SR`IBU*_u1q3Mu9qH(*x`)tBLSUMNG?yVh)9{^!z( zF0MoWL~DeB2?}6==1s%vngHjtlQI%ESiJYI7h?~3yT-O(q_2uZnRwgJ&?i^2z$o1td$B3cocy(UVef#FNgkIADx z5%X1u6o_dyZ`Zme@k~W~>dF7d-g^gCm1XbWiX;^U1OdrNBLb3<97RDgl5>e1zVp{xZ`D+FS6K@8p0m%{d#}CLvp!Gh zq0O?ZhgA?`Uy+g47(G=Egw_DJjJb5oxz}gM@f`JCvFYOedB7DD_;g5l6mrLCh zX{@SWx73y`E}iMT-(7!ko{Xr=+F2$K18~%xZ-;JGCk7w$;&#^` zEq`o_>=%t%O#szXp%wrWiMl@n&L`hMsIg|R>TWi%Dx&l|2>T|byyit)4yb?p7y14l zr@v8Ql#Gw?jEiPTeo`WEXw#kps_l*E^k@dBz(*wjP$>BJ!TIZ3HNd7;#LCsK2!XcX zmbpl0Q0*(7D=%Ax$a)K5;1C8OF2Tmy8p~=3k%)f?ZFa$c9)ePeW>urR zEW#_+h8ZB>y+cC3ZdwbZy5H%adm47NaR0GKjidE91wh6uDkZxjE&b6Mh{xv#Z9;+A zUo&{2wP+Y(4u=4isZm$Idm|8huin!IGRPRM7ZTC%l`!DL%LKJ2wE#U(?0Rd;N+kMx zmLF;D5oc<@H7X2 zp^(58{jQ*~S^oQx#4`5o_d0=)>3~%d#uSr)(c1&M@Srv2qv??{E>vxMp0 z%yV}Gy%dL{9>tmz5SX34p(xx+CKr@00?Vd>@Y$UN3x#|~ za7t(a$9g6HZuT60xeU-hI+e1pH*gHXEy(laOCTtH;ec3HC*AsZS!P9QVtTje`i~bY zn*$`cq8wT}k>o~y4xvu(hNBECQjXU-?|g$mY=DV;(n0)D&CG8Ee-ecIH9d+zJ)f>h zZpR5q4JS%#;C*9tSR4Ijv(yQ(Vb&{C9Q57W_@*P0n63#<0C_X@dfgH5Eo>!oob1y_ zoGKlH+BLlc4+xtL1dS9}+>%{L)S^e=%K~V58D7V`lC@u>H=+~9Ax1TH26(Ey2D)|x z>=*t3$I0J8fe!9kpzbkt+6fGTZe%6CXeVYr2CI(JJ=ZRC9mP&;3c!Ifvd#(wd&;o6SIE{oT4EMqr(K<59pU-}}7YjK} zjoIWf{BdSkU}gcvS@*sQpnZTeC}XeO!F+h?pj!sk2`EWH0dd?a!=cng)2kple?c&J zleU2k()l+7P%N*lYoJrP`*hpuq^fR#F@pTc_rz&%0yjn&<(%xKoVz&`MZa8BEzr0L zMT$Y#4>Mgda%0*{71P_GFC_Z4Qcz$zJyMe*p&^PS=|Z z`-*qgEm9n^>j7u3tg!@tEgA(rz_d$81TqrTYtVW4zL@l0DEBx2`|ZaX;SL`Kwwm$E zC_g-X>kuu&w5TahEqab53LR14r z3Cf4LhO=1Pfuz#>*!}4l07P#AtUtATaH$CcTDBkGt%q|(3H#H1b>v8}zqTWM)LfAL zygoD}aPD#dg5Mwzm(ukY=F&x>$AS)tVVjR=gb&BjDjWL87vHu@e6*y)@tpr0F;;Ab zIOHet1k{ePODONfLRMnv7F5g=RL4K=L-3Duut9(-B)vvPDWc*0G~(qE0Hi_Rs`nMm z<#2cX^ zfynYiaKEc0o!a@JqTKrPh7o0oq>g?r$~{i|WKG@ieFy2iAGak;1AHN6lE;P<0%l%x zslKZ$__mP zr*3o|xEH+jRO6`FjCze^1ih~@7VFJ^vhW8LcdPGd=;h~tWQP@CfBkOh$3J5}67^J| zbbtzu5OTLnTWw(DknXW1rQ(${7mx?JJ>?1do5fv(^#B*2>@%H_VU}<`#Ol8qoB))z zm9Ngds*)M8B|lZU0%*ILsjxImqR&R|1RNyIj<5{BnbkV-q}2 zcASz)fGAvNArkCicmTvJ&Li_h>-V4b7QB+~ciz4ZGu-{&R|13x2pgmLKhk0(lBrAZ z-zMeXul$G2=dU&LuS-z+00PD<4PKdYKV_(X+|hrm09>L5l_c)|bMkpA=qSi#`Ic-wH%|JT3yzeoCW#P!pV{`8Rl_eg&l3}n*<4B`J~ z=`RN6r&kIqSa`h=%UlWRX(_*<@zsvz+!A(W^Q4J};n|x?Q#0*zSh<#vw|S@w4dC_E zd4yCDN1}jCDdas0vSJvTdM$`Qupq9t_6;|uZxj)8vW)M65=_z^I|fOYOQn7M}n~!(~;O|_!SjMRYB7SgdziAe0(562b|$d z!2EpvmJP`#{rK$BMwB6v51vy35?G{s*{M-%qt?G+t_$J1OWoTMZF(X4j z%`62JmpN0)70*DPz1`UA4(MV3*YN;eG(S0ifyyZkhV#!u4Bxg0N3s<-(PTPw1BVxV zEd=-GI2jV0GXj9eu(*`e@vCbNp9VOin&Kc;FgSSN@Uok559Bt2K&$q7 zpmLr{3&;=LfO{Zn`BZI?yq3R@?CvWN6L|;^q9L1hQPjO3)-wE~Z*HYv$b@)K8(9nv z6;@w5*LMvKYhA4Ta1f2Q^|qSXsP_9vDAA05#CtzglZ&T0#W1TbL0%>DudgbHqd)nc zbzR~QD)jr&Dgm9m9kv0i*RyWj(t5SVKw=pv+vb3+Si58>&tj|-$e(DvP~p6M0yL5x zppoDV;J<>KuJh%qV)LaRAvw@VDIgO=d0!zkHPg67ovYeXB~&A0!3zV= zVk4R(A1B+hH7%jFTQki6sL73VH?CU(X0S=!X;y z-(r8q`^`{e5SHqri@$`)C4Zd<4f4ha*o3$)!s(48c2^0;03EH`s|6Xd~+v3Zo{ zZ%ZTZ#Ye?d^75K7Rcd_1hBQ-r#MC=JNklR2_`px^MDvRGfo_ce+Q%ZD*kBQf2-wP zciix&!c!Rr0(*-X`kUj!SR}XXsTC(e4KqvPuV53Vb69i^np!fgxima8qw0mxFy!S> zC=%IfaoqFZD!wIck}J%Zi|2m`gKJH4-_> zd7%n){5TC6X*iE{A14GC28?tODUW6}+&SAl2y`;g7jA>X zpz3(2JNe=E)z%`W=CuZ^+9#~$F799}rr*#dVzrUBOy7>*ZOb!LNXxIz5TI<>JP_8= zQh|eR0Z^U{AafYiPg)w9oFtWz@v**}of|mYw!XSET?X`y_{2{uHa@<|WfS#GPecwpPp z`+DmE?{nY5HUplC?!+@{5XZFfoL8#XnYHzeZ5%m=3_qwle$|s7JlC)#T|OLgcTG$; zB_(h;y7k`IGtpz&;P?AuzpXq|!RX48%vpcylX@}-+n46Jc~r!>=8}>X*5GcpRyEhW zwrR`7?9bI5Pwu+GLVpz|=M&b@dn2W~f(N$aZMk!&1+gkSmKodd-6X_j;uTsjn*^v* zn@&Q?E{j5dkOJP{S6#?PlbL4$^En7|Xyt>rw1xmhc$jk?PMc(>0IEq0kj?>0+6FFd zfGpVo=vj*+O95^+TaY6mHqhMwpT~U+rv3n+t!TDsBO^?(1h)z~0C(LVX@<5C0}AIP zJXvlkKqMJY;}?2-@g!#^<#f%&`N`HPP*>13IzT2wvGZ#hL_vGwOP7X3p(dWE;#tK*;oJYv6F^{0w0O?DrmxSHo=SwRK%^QaOkk%B{@m z`?vX1Q1%B-c!BC}$86*bBHTijo^8PBC6Q|jS?|V=2-%KG&+mju9v_+PhvfjZGFDAh zYtZnjD@FOKA8jr6XgA*^Se_!jHNu_W(QG*9%QxWodwxxfj}<6zmx~-rzz%bSIVhj# zN?7a&!8z%GqRvIu(=GS@*@2Q&r{*LJtZ{xRlif$Gu`EX9NwQ}T@a#QvCapW3`;V7qi=4uQMi^=?#GiX z_2hoKgaWE@c8M-4IqLEUkY51cAepZLAq9yEN|IRh;bMZ}+GvT*sJ@?3?YAsHR0uQt z$VIMpHN|1vOg#uxdsz2D4ZwoyM-k37oH7X%D`!AGD8k{aNAXIe9N#8Xx>WB22~0d| zmw^NWur_s}oOptmr5QAUrG#ujJ|=>|oIy-`YNdhJS!nqbJpB%s2&KU1;wguTM<7xg zmNQE#+@|Y=bQL|r;oVt>a8T7O0q$NV*d*zcryLFVu_@&IKne)Mg|r{J3`WRy77&Z; zeFo__ifvD{3rGTUB4QM3-wKTMtf6Tk>vfU+)pu_`9rx5#vwG;>`06oniP2`Z*0;q< zvl3$gUb0?%{o)}oB*wASeVns?eg@-#x%aGSZw`oFXmBZ~f!RHDEj~?T^d)>&Aeb^9 zm1%vJ$pb_9-S0WyPTT){JU64Ii1qpWQCj};kk?7SU^|VF$H@UVU{XQ+sV|v!z(uwV z__Gj$HtmV;ill;WPKhA33tFFF+0Mr|$VA8R(^=ow59zZY^3NiH5fP_qWB70#mOLzYgq1*BXv7Q<#S`@qfJ+{c>o`)L*N?#PRQLWDDm$Fzxy9_9LY zO4g<8+^Im|8k3QFQeNgENtX@dI6WF^U?R^X+XoekIu^9n6kwlms9$O5zLl!D=uaK$ zZ-1>-E*VH{GIkq3UZzdN0iS1vpg3s6xQ(`&HMIjt#`_h2R}UjOl?ew3BN|%ha(??= zw);<~MsG<%(~kXj)-yB>xs2;~bVsbEj&_7&Kr<@SUYOaAt^#Kh zi-yyE?NBb~?Rf*|@%s&VKg3p-u=#sul|YO#2`c33>Q_cnc5ZIOxw;O^A3B9F)=6~^4T<~TBM^Q)bcm7C{ z4c;GVd)53ki`?zs9!*u{2kHtAdz!m_t;P?Zu^)7kGS?iOkJzITMx1%-G%T{WZrM*31wLXPb zV(nkCP&LWk=)uY1sJ`>F{gzn)o-AL%Si6noxVetfpG(r5ldQS zkCwHr#OscfgY1>T+k(pR+}N03Hrfo!>2=Lu=IaV+AloB^Q}fYOI0}=e6PJC$#3oeW zIt@qCOIn&7Oxi@(iaMF=l9?0NYGkMuJ*E`&*8-^Hza1%7g}lX%y-{n)4s8LUHSzXW zTXm~M7>f2m12!StWu!I`uK`3e)az`zIdo{lF`4*wGgEox(*P(hiG3Ui1%)N2fDjPm zeCrp%xv<_TkTW%SSRuV;d3}@~re{GC?ZRbX6F}15E zA;UXim6hf1Jr>r6UH~0;8+5{jyvsm`!9A<$h3HY2)q)Y*y}X3`oYDMgB{fd37R5qI z$I{LR**Y?eg;FmG_U((|c8& zo0~_!N5pO{U4j4_*H)mf8~?V|+8*h`546oB=4x|Db~h+_<}Jd|+tbi~t6Y`<8NCG* zr%qP-qD^HJng|tS?3GUtvk8v|0%P9WEWdpG5mCSx;xn~La(g{+=EItgsm7VX$%uGQZms{oL8tk)O0$<_ANfdP9kI98=~ zFhQL-kd76CEJcvN4F9K<_8;$ucOifWIIh3@;cxm+KLj$I@}qWwu;bdrZ@K%itZd!9 z78(SoDKFt9y3T@EyX8cV3S=1O<}YP7YdUDuzdsz=u%fSAH(!Kb)p2J_?d>~wQZeQv z<3qMJ&~(x4q-2kA>kwf*J)$4Do}i(EvESf6($`n_OoD^awPX|L!FfRr_b7ThByTJM z(yGD=1pf2nEZwv180X4^WQVo}pePD#BjE#a?=ik#9k4;XjW+ElD<^bnM+!i|Hb8{b z3Zy+_-C4(WS|F=`NZMb8ss*&-z;1L4E}}V0(E-_Ffvr(DJ@pjyd+eG?hx}r(Zqy^l zChH6;L(VsdymA7EC(P+O%s5N03D_!~%;@R_U?i*GqF&82Tq+Q2CTACG zx%1vUP)1j6oB+WOR+tj(YLS=hiH*8ULHvTD^&MYxOZiaKi#nhRf&04vx za5J@Wx=akQx^cjK&pb#*tfFV@Gnds3+e%Vggit-3m3u0$j>cHi!;bGtAiw)VO93{d z`wEoKj%GVuszRTigILG-;*vE`8W{pD10<3YAJ+zF>%*!0GhY|zhy>B9e-BzWAp)o& zXiX0UI=4rlQo#>IlaQ+g{*nz?mI!xFqj?^~2~o^%(fbSsG2ZhxjpJrV3%rM8;;Xhzq0I7`~8u>Qbe)1Glgb z&|2_52$vlW-Kc9M3}J2MCqP3k7|*>m>&O5RnY1e>|b1az+aYO$zxvZh)(Y zG81Fhcj@j53H%nWlQbZLL8dg>1>32Hk&XafKcwV^nydEZg|B>)E9y1_hYR(3(5_}L zxDbmZ5cn}Mos?5gxmdXWX*&KsXIUPi5+_hA$e&gCcpW_8cd4RFs@+!Y+S;Pv5K|;% zUG?ee49dy)P4}MLeMSGA9tHDc=e+|>1V;y5+ii@IOFW1%(?igr(*vg zRake4BUjMC&Rsd(4^u~LI4{hvmUMe5EC1HxX`_~lOcr39+eL7!UxD`WwVlm zI>V3;;WpIMEI9H>`v3krf80^(OAz;~_O`GoBN=N^<SUc>=$hdC!oUWuwOl1NY~0&}KGfZ5_e@FAPl)`0Ya#Dbv;;8Sn*`o3?jFz027>uC&TY z2z%Vpp;Y|Uj0a;j!yL?u?4YJ6dL`ZK1k^Dt{6S3}#;4S>9;VWA+N0d^G6R>ffaHap zyCV%HQ3FF%ns)Td4G^l+2$>2T|_K0}!i=`PI0@RB*h%h4Q!)(;Ue|+%} zjAKjcNu1)u?fP_YJX5Q}FbL@b4%z5dH^d4ty;oMA@y)9?N3lsDA3-$=@0C5X^lM08 zKh8?wac91Ipu?mWr(6%Vc06gX)-gslQ^3lx3bb&ZQ(fiA77M)#sQpY6kQDk2CeB?? z?_KNRje`Zxq|R{UOU}_9J-!^+p_vSl)wfX$|MACqs2U5FZ&Nd~FFIgOlGvd8-WE=! z^>!0KIjPU8K{{WcQ z>B>_Tz;CLqM6p{Vf6ssV(SN>pB6;v&F11@f|F3`Ze~%RDvhlZP`@cu}Z(hAW{)t{W zwNioBY8u(w9vRjMuu1JEY{tJ*X+guSM}2ve%9C1z!Bj zG~&^|W{Yf3{`)=t)9rf)LJ<&aO#8$CdWA|&1i|P3ZuLJr=l>r*)n^@MY>U5~Virz6 zbZR9Ak3d_+Lru*IUyA3{p^#>CZ*PzF#r)^&yu1m2YLZ__U47qyI4;d;r~tVe2dZzr z@Re5Ga?`%ltIa*krY?V_61QP@^uZmZGHFqMjgvW0Qp%pCE zeg<|EF-s3eY*j$8)G~EGpaDoaRmlQo$db5Fd@{_$@-_j9viQa;^)xOuv|R!qr#Or?wP{gFhZrJHsm73Y@DQ zfAOaE&VY|P)>QKR_sa{Ci)Z-_QZNQ|SVa8Szxw~xmsXrk=~pAFK>yH-ly`^!x)NjE z_J+EN-VLciuFv#xjh=oC@c1|&d_{7L^b&fX(dJeATUb?*{WDl9yuW3*c-wU^8D+rfnEx7dB1x2~b${Jw z;V(CLA|1ZyYD#(^B2`G{UE0_YKJG_!AD(bmM=kQ)pe&w6d-hCBhBu3=iFB**()*fB zUeNJ&rRd~z$j#nf$SBo-z%};nF-flG0?XrtB42p01bwB-hmozcNpulAijm z`>^gFcDdXVUrbc#{E}41PZeIQJKh^F*Q7>9b?qgOBatk7EJUc=GZKzEQaZ^ESv$_K z9oOaR?o)1HyxVkD$@+jQ_|KRGdfU9)@gGwd*qu>Q3&}Fjcd0I3>H3wP^3RB*T|0Hx zalje2E6|f=pYFD7Q(e?`f{6v0s=u10*SPf4>wZcf?kODM`MB>;;97KyeK%tDq1|80 zsoo{b`I-H}@$s6AF3e#}I~5D&jA80=NEE-`_ITen5dF^1%dYK9ZL6Gt;~;|qo!fNL zDt;OPz23`_9*-lIb`a&G6$?^5*u11y|Boz zjPQh;wjh>@pVaTzK8HC)uhOoNB#7c{<-{u$J(!Vys`bttbX`G>J12X zaar{X=}+4L5sr61HgiAfumeBkTaLT`7-nyKe6Y0!fyRPn&`lDAomOZ_XrwKG_ARR- z<9=p^Kfab;ThFo~k4Ye9JEJWM=CDI6e;LK$bJ6R(r=dUnw1txQ?(9R=+PN6n zcjdBX)qkx~-jXz{Nhty6lCeEl6`0yF0VX=Qa3($M;5|!6e zzVGnE%V>k#pT6_Ym+T*x3jViq)$`^u}} z!uIWIgdJ1S95VOye~cISgKxC3T!Ylcik7O8s8r{(%aK_i_}H#vGDgzd@NdK9izr4= ze`6zu3AXlPV1k`!UNttc?G%H2pjfKM(Vt%fel2 za1jgc?wvmk@PGQ9Ki=YYkcIJx(;*u8=}G^G@%!VZytfI!d1HE{Z4KEa{W(dHy&b~; zyVZX?Jb#Qo@~65=vkCZutq}VG@R4+i_KB4<8-Xl@8p>$85I~kCkOKn5WO)QkhkBxt zrYKR)Tq{J>?17&qI?m=y-ZigWDHIZk#)g>?RPDv4^ac#b#0coFqYtHJij=_LD};PZ zLBQ=cb>}|TJvgt=t;<_=q>Xay>ipn|dVp2OTCq5bX5<+BIm15A5Qc$QOSL*+EA$>ym>?G%Mg-uGP1p%l@3oPsK)IuHRLF zsF~{6Kz(cvWKu!_D=XWs`92*Z1b*N@ym2dZf%9*Rr_gCg#2I-#@GH=E&;+)+3Dz)Y^)s%F zj?0dUPQZh+Wdycgv9<=O3nBJkZ4OUy3(0D{^_sHEwr5fQk{h5jl?}&c?8>K6qE6sddB-8Ue64`0)CRac!xK*p zx#}SJ9U#|q`mVe6)B|dzQ3KzUK@j6Nz|(w`k~KV~26e=ow0WKY`ZumPDK3O?PHF4^ z=u4PA=-E)+DPKq70kn_9Ji->hEQJ+tEjHmC=LtLKV|>^wwE=n8!bZ#!j2nqaq%ji6 zT`p4ze}CQUdQG7GLYgCHeUhfBt-5Mvn6dn4)_ri0t`c>;iO53|aQGkyW+>DJ8?PvqlCGT`Z{K~E@g z@4gf+i&Fg}1)r*|noFo5P_f|q1meOez$ERn?0?WEqSXTcD63(!ZlH}2u?mnjm71VZ zUK5f>e6c+Tu1qcPg&Sh?KyaADt}vmq5ARQX^1bJcl|McG=p*JwW8NI3v*j#d4KoU2s`t8c&E({7c9KQ## zr$ClFl9A1Wf7$1njLo+S1(u5&WdU0}{xfRzLECR~I`Ycxlw7xw_cKs69 zBRu3t;9WrGGDv&LkAe8Y@PK3bS@%rmLez?CQjaF%Ohx1ZB>FjDl@?hdw$I<`5t?CD zt0vEnm===I=}Pp(EiD3*1;YC|Vnr92%*Ao#Eh4TiedZkjv(ZT6Ipj-FR~U6f)0=%b z;vcO@v9bIjM*TOjV4_)Ow6v3OD$%vlZ6-i`qZJouP~!s-yD5MoU5S(nqRHCadyx91 zgJC*us^ZlXW^G_KG6kee8w9#;TP`9x^1L<2!+?A7_9c^w_rp|SlrAkDAIwj@k3Q7* zueMZDp8(;yt}&l5th^q$zj;(K9*XFS$zg7H6u=j1yCv7J!lg#fwQ>$_hGF7Mrk-j2 zMi9Tvd&RL=1f!ZoapXyVBt9;QPRc{I79nrGk4T;MGgyRep`vH5S>NlX!DQHW_{2WI_Nf`rRw^q4I=D68*LlAA($k)$KJm6~BDXGz&Ff4`BN{Pr^M-s!ji`}ZTbW1_?BOEb^jc(KX$PgTK=WZS*+L`mNKxXgMg z=F%^CUBSA;F@4$B-^8B~EN?%g^z-7MU;27?y$v)ktN;bgjhO>w4J=+s)Q7@9{g;}I z$-=m*s_VW0Fn#RI{!Tz$L^g1VGq9qfq8jva6g+v8XBRD!S}UfwZ^$t`R6FZ@SwCP5 zK=Q~2Xdy?P>k$k(!BrDhe115ICR!>aC;%jvaJ<0mc`GG2N14KT9v4V=5r>wbDJjkq zMqEGz`c+~aAu6hywnmPiw=sQDsy#rFwDf)^^tT3{>9Rt4KB4CPl)4^=3pTMTYCHgO zHP_Kxy8}ow-?IYGfEsfuh#B@h6j%=GCePj91q86Xhc>K?1%o8erzth$rev)Grv3uL?p&Zcy-M5F=@;Nnpm=oyEUks^g**tRFDt`jWhpxi|iR`h_ zKMyN#kkDp&s~E=_5gXoEB;;G}NC+?mvr6c$S-UqSVb{KxiYarBpjyli#D8O6e04%I7SoP;=_miFs@G=v)BB*uV``2 zT$a|I%92ZAdA!0)P6fQN4S_NJ#$v2t=u4QC0(F0D?1cgNHkQ4~>sg(lV@I-?PYi`0 z47wNcE61Pd1eXKGw`&B$VT5n{;ZlkaZQ|+I>iNU+6YegKfziUc4ux%MTm`Yl80+<% ztYqFDQnSOry4MIEVm1&hZ6q3AmaArS*)>gR3%qH#WzlfzDEvifEsIYk zm9%FyfCI<=bit8HASNLus!5?}0Bl;?nV zl>L_Ybi$}oz()-)UkGkp(5UtsTVU93&TZ3 zy{2Mh^oa(^FLnT@P8z);KhQ{B{Z=J>v8aICbo9DaCfhP9K5x zLMiu>UbV&&=^`@Xu1+d=nkSPj?weF31kh?=iyQQ}hzUf|QczRg7Zd5BF7E18Fa zt6RNZLA0WEg4c*xD5uE_zfK*4fEzK5TaYZYeTmqEUOGsWw^RLLc!L_7b_Cb~=mg>Hi3I)}eq<%*&VdnCad;=BNI{ zF@LeFZP3_7;u^om$35mG0sF*04=|!BiBgkK-^xsa1aUGCQQF|iV_HD_5{R6*!=&oJ?&4fL^xYY7Jb>oyK!-;mgrXw7K!_Fy1{a*YsTzY*WpD|y(y zn?LJp;aReNtC$xA#;3EY+V)Wtc8wRjjH10T519;<*&#iYp5W`MTdVBh20=krI z4q`I22w}t!rCJxW&x~WkMb5 z3R1TRr?P59>YAtYp=!HpylgfkxNSQo>x+=(A7=)J0kd>$BTXkY%{Qdd7Zo3x4w?{g z`^@Syk8LSiIFm;g`Ja5Lg1q-+wS6jkAqMxx$(>0G0PIXO2TTA%Qct3HW9f+Q#Kxn( za#W3+I)&5R{k=SB#BHWatSg+FB5$rl+!-~}GwKg4(8W#OCVihQIo8HSyJyD!Fwx2G zy3b4t%=2DhvR0+SS;o2-r`IvOVTO8%I5`!M5iynxD>$rRbCsKq{$K!rW7ZHo7x(XX-9nB!}@n@~8q zvBz8c8>Z1GB!U<%!+t?@j<|VjwO=8y;Z8y0j!$@yNvUpw2kX~)TPboqoY;z#)T@-( zMSzjM#hO0}8V%3F_P~}pz)G2|5zU;UqDc58=S8N57QXc3BAP@hAqYltI_ z7=@_X(pPu!<2(Dn;oQaj%{l7=h5gN=3oPSg7Q$4{VP~jo0Jgt%_DtA{J`T)4&1U7GH_j)vvZ;eX~iW@IPAqX}F?H!Z(ycIB0e0dqZ^hj1+C#+soSNuMfvPJla3&pP+h-2o@a1~ z`vmQpI|O(SB}%bk{ex%f1LNtF4w$V>W@`7sJsvt9ANhUm<%4bb~B@rd7!rQFk% zGV8(Y5Yue?__AW_xeNr0C*R-{W0F;vc{?&7Om!L2^g^kzbNtU|fH?SRy z?bM7asH}=VqEB~_C-1*WUX`NjX)5*msDM>2F|3^JM~TlORa&-QSTP*ClmZo<44bPRlnJ#PK zSFaCP)hypR9;5D1fGKYI%kh=+R9?aB=JUF!&Fw+g-22hjzzsGg9jyCh2mcL$kJ&Yp z8}q7P1))|_8w`?=yI=a^)U$-Y8-y^~wLj^d%h(QP@7_!<<_--rA+9bBZe^r*b9353 zVWqa>P2IG5a5PF0>h!Hn`jSDbA5Tb$N?(S2?YBUNlb|+!rzfoG`q2?0Y(M1yF5;Rx zp!D$XNV^4e3L3R~siPu{^9N>qgbA_fh^7fX9q~+<3b(oj-b%K_{i0xvEigG?M#72T z_ubNf`yr{*#jLH@=veIh`KdQm5Q!uA~X~1I9##n zW9AL&zPG8zWs4Oil-jG8&YG`n7AdzgjJJQbo!4Jg%Wz|9y2_8jS+?nJKH)SHUjUXO zk0`|5Lv9X_9X~>&vKP|>1Q>e+56wD5Mg4ql>8xB~Z6_wNYb=wg_L;lUe3nLem!VLh zV~0hMS2b4Kt29c;_=JO6NK-q&n~Nn~$3QI3!s2aNlm-!FLV|E!r0T3T)+$=AuFq56 zVFvAnBarbRPkFsfW}Q5!x%`-^S+YP^eG>DfQ}>}qMq#ktK)dCq%_L)$@5gPQ5WA%} zci;dwN@Un}i9sbmur1&ECVDp;N!6`aPjOj$GjWGgoGaTaBnK~Fk-5}Uqh4Z9wT*um zGxz;fOHSM^i)>eULMm(4UaCZa)GRdC=x!G^PQ^3EEI}>P>L5IvSbewSIm%Z{0oK^Z z#$uZRL>yh*(KSa3ETurM6rb zMcXUApv-DVpkbPp_!X0o!EBMz3x` zbW@<=Z{zOOt;Opv;l=5fistRvG}HE$2hcxB?+%-YjY=lbI_sQTS)!~LxF1GUyU;5J z-z^Pg;q8nI>=tw@>wb`?_mcEAtBSYi(=q@WZhJ?{+f<4LzY2KHUNc>7?8zCQ!i;ze zEqSv*TjN;YW0Ozs0D+#J;7pOOyl*odO+Q5DmSvJ^^P{;N`dgE9cz;Xr8PhxmsqRIm zp!@pFBU0iq*Q_4#2NHhJsdsV4cVe9cV3K-A5n`v!SmSqv#Z5Oas;OCLd4v#J^IDii zW<2pe5?J*=Wo^#kV1cp@XIwK_qP|!GS(;fTv6hDxn_oW)>KM=Ga$1 zQwLqYZkK4M(^-?<`IkZUV~~Jf=S|G==gSyJ#dswG&g9fsWYQqfj%e7@lR02vJ|a^* zi}O!PvBT_R4Wl}FaPsl1Cl>bk0`UTI*^94??VW`QGa1pEPFUFWgRkApVp2(`K~dp^ z4)?>TOU4Ad6U;Prq58qRG|$ZN@N5stlakBZUtYCV4{}7>_Jp|CGbdEkyb|%UfB#!D8BW?mRSSe4LPF@L3MK+)8TD;XM|G2O7+pr z8883eEF0Ngi5)iGts2&{mC==jpf`o}0#&PpgGb8SXLBNh#s}&EtX_{tOHv-fB~bQY zMu-5G6;trWGlbpUdG_o6BLekic<#d#!{MqcGY|IHo(pO&H4kcMWA7xs;e2Q_i$xl} z@gOED8xQ|Qq2!>?a|`43N6)>yrxHLX-=X!LJW(ps%DP9>CgGwpF!!qx&lLU4GEX#R z-A9wJ)sCs}d35XVIT$O(frgY*szIB$mTq)3b~Wxp0Rx)>6%mrB{RiE#`o6cmb$uu8 zhY5VUAW*D8r~R;?J&s&BT~Q@Mm_16hMemzHYZd3rz$kd@ZdUmb;K{*?%irHI@%yyO zzIUUYf8}PmW4uRzh}HLKD7Q($2v=^elXP2U$85yXhjr;acK$G|VSX5g%EJ{ZcW@Q0 z$bnR1Uo6wT<55TADP4E z%-3cKk~?LXV;15e53_nuMQVJ#UMm_**80s;G?}J${1LEYT_kE-u-b?c#Cs5;J71YPM9QPr429zKTW*HMhBW1br zI>$YB-t@rTmPDDSX$7k zIM+h*?q(*V$x|WEeMTJ4?(KSg12;aaB>Ulj`WFeYP5VZSqHueTdvSet4am*D-_7hp zpF3_vHFt_*WrdOXwW0NU_Yql=o0+1fu@j4y*MRKMvHh_hQ4jAG%Y^6x8Lw73;>+t{ zzO2L@4{-Sv<$Vt+-8Mw!+QRdVJ<{<;n-3gDoVuGh)WoCKt7x35=vqxEUM+TQVp$uw zP52^cUzeI0heWKs$6HNuoDO&uA-?6%Me!-mAjC{|J-Imj^woCYu}_}_MdhtX4J)4o znvf+47$z1Dv7%W2WfPU5XUa-Ok8NOLVnb?kzpT3`fvaO?w=# zCpJG3>!tE#W$B)dIk_&d=m$63w>QavW!bFgf3inkX6;4OkH57L7ezK7^uQv{D0=G> zVGi?eAxC)ANpzw_kwo3-4@beazR8cdxgKxf8cRT;r*~YA@2x_adZc_hScEf1y>i*) zti&X|H1VS-xBI(9tW|>7bQK;(TN7_YlGC2dlse*>Va{CA=Pp2CUEF3aQ`l;Ay%FZ+ z#EXT|#1;+%Ct^YGsPDF&M=G<&El2F4tb(+eLVe??2ik>8lVB5nEbgdcrS1R!*n7*c zD!X=F7!VOeNu5a|Z#?nb)1yC<>7eAZgWesrz$wuPw=x z!sGjnbth4qO6mmrjM92pd~QNN#f*rL{YW-`Kk%js@i~%Dneki0{BLjVDcdjG!Yczg zUia`#BSl?j9jcL#nNdQ^(@$hW0+6YHj0+f9u`XA;vS5g1y2XSR zj$EG4%XJ_i(ygZNyf@vEb!TVM;cMm8Ow;pPss_x~1GE{<9sS&e+ze410|F%WLRWv4 zT6S5*kq~<<1BABAXRTHt+{%wz<6Kgi7(g}?qsy*LN3DWB5GIE4ge~x*0ou};I4hO0 z1#mnyHh5tdxjFhkKnRO4rrwpL_n8(=wIKn@D+6Pq8y)`!r=#btG%s3OJxWhr5UQj5 zRoEDV5t9DzzkU$7G62tT?NEq@X0QbFfk1_1ykLu-Q&jsW@QALqr1rSl&bzNUmX7>P z=8^U;i{Hi>f9oGj+%q)Jbb$Btbu31CSmU_KKBx-iX(u$BY{upgAcH}C*l8~``9+-K z5wV_hn_N8o;B7ap#Y=&gYH^&FWW#1Xp6;TpugDr0KQSwrDT4G4FQXESK~s-B?`Vv^m8Xs{|Sb{Z|}_=@&d=|1ur ztQlFxqrO9)Alj<@kmsRfolO1eQX?%KL2oJynPN^s4tP_}*B^^#sW`%o%SJob7*Pe= zWzg;OX^|un(`9>zanF&pTrDZF?vZ<+3$~7cw{G8IP`HC`2p;oZy~TXwl&t&9jyG;{ z3Jzra_Cx`zS9+8sFT#Wsm2=me zScr`EZMY(6wK~;V7;c|=8IVVu5i8yP5PhZIdySpx3NQX-|Jy)E?dL3%koB4scjiM7 z;><7O%4fw=XndTeSI0>0E4^!Ixds_-qNnFp?y(qaB&LiYoFu05b2tZC27L45owZYL z1Gl`%*YPidH?aL5JZZ^cj1uTSRO@buEzN(Qk{)3br-t(e)7=N|Mt)Lf3n_(vlH+Ak zxDf5xk}i$B;)&*BxcwtS(+ITIW;K}iv}X>J4Ur$HA1~aioTd5VXjV%-HApnhc1+ry zWqw=`AoLb~^L`R^K4S?l;kYS>=@w@3L3+gLXXdwb;XqL_MNLF#6}ohR)aWt-LO|^q zo@arhv3(?F&$%)UA~4;wjLit`LzhD2PYLv6cGglm2Xe9yec+#Wp&3Wl$O`-AaGZR! zn>Nn53OmR*>IBIz&;`U+kWxZ1Ueye(ena2ssI0sWIB}IfvVIHmu)T!!!P9v_hm&}k zj~{d}Me_LMir;$m)iZA5Wf7g!g@la-fYz`+_gh|0CQ@)`YO1Qe#8@*g?WWRv{>s%6 zTbrp#6pbHC5-;dnCRvBKR?6GyTZzd=^^{@S9O|G-HVXLwQaZ^?$;Aj!)W9OV@L?+1 z%uz1dCl!YMsGIv!<-!LUogMqf!R`GleaI?<`ylDQELz%-QpVHRYGIX>K^|t>^SGsk zO%DcR)Qs4skNGH_tT3(-gjAo92LT9gpV+7QO>8l{%a4fiXjo#FIH$ z>@ej3Eq79D0bbz-CIL_#^^qmB>5=*)9XlOFHaZS$s&^PENE4JBcILcnEe*V#i}BT& zasr({orO_H@sfc3st_hP&9$|PXt-a(Ug+{_Vq#9@e1H6Ic-b`4LDJl_chiv zbePays|lyMG$j7)EmK$^gW2ZCS3aKz4-CBOgR}RInWmWnI`iAO@2{F3pj{@2_3aqZ zbTL-AV+$O)LC#S;I(DvjidTeGbLrq*ama+U5enqB-~NZk2Se(qeW=M^@XqeD$f$Dv zPx%aE`Ra8_d1<$5`NvOubh5D_?lh-d$Z=;3#kU2;`F7l+kEQKG`(-@wG+HbU<(pIk z=?t4b#l9UUYv{+`|7EKtzkU3OuAgC#=b0B>q7hwG(f z=dLwKPHXIkZ7NX?j#65i8L7ZcRV2lVznk}EN>?Gnm0}n29`h&s!hlM2hUP` zk9m_PYy&!Cj_ZZjkIV33ZADKFK%STzBgQjd6FY?LfgSbjyZCQucu6E^bOPFK*d{|V z>f0+oa=~~;?rO5B&Fysj(Vz1nzGJ=@2i7Kp#YqNB^QQ5k^byImV z)sRoA(6>^7)0)HxCk#)DtJ_rkmK+O)qHl$O+E@p_kJrpmy|k=1Ob*qucu%#G3ieou z%Z_J0q2J{c;4+p7)NoWje!-QcaW`DSR#L|}D8oLcKyi=h3eWJzNS(`yRqc_*WE3s& zz!gA^F!U|$;Q5889z}1##kLD0B_Vz%R_)70Xpz^3?QCn=ItM0VT$cRxj?+2cx6>EYKf#2dIz+HJyzo14#=6vPxE*=Wj5g4>UxMWxeBuO1%!v~`YF zO~tWPeTd-|OAU-7xACG#G^$(CF~-Y6mS)y6rlc6v#?qAvQ*}7SgG#!^dXfQI{TUU- zvcaR0PU7G+nSu5yeBx`o*>EO5SpTIEB_G=MPa4$U^)(ZnIo63DJB7YHCG00I9AA?zI~hA;7xc1JeAO0iycZ+?>f}1LvOr$9??!kb z1aE-VNUo{ou|BaGEyqflFRcR}k9qfY6{6w2)Jd|2IlgTjkncfE`6M^}17Yrk+*AFP z=F|oT1MRiJ>mehp?1kip{o)0tmuKa6&?bKSfHib)SvmYLhDUC?rlNoy=~aZ5FC>hU z9zVy32}qy=>);_sB=gV5CATDQjeIheeyE@NC|AruN%&o`EHBcZssXrs1?WZ&D%q)RbVauuh-J>zA0w%3-hoNqLZvtsCb zN@JzhBp>jgJadinQOKrh_ta@+5or+_Wdg0fxI&BSj^ES~7vgqWCXI_XA(9$N@`X`L z?B<^ zyjruz$?9@g1y0M$QiSFk_6hYBhn0k%%EArpCQ?4NR8PnS_*6z|4HQ4sU=vGDMRdgN z4p2%tSi-6Vv$O1N`3TZ5XM5!E+4^~I>BAm+t zMNbki8(ZewQ|f{z$O*8F^1o}vyi0Z97RDo>l1Y5uG0gKG9^c<{TdAh(aO{V>IMESc zgF8ceY^PX9kb?*5%MV#~P?r0#>X4Bl)qDNCseb)d`TK{mL1sdsuD%5Vi#jXWQX{SJ zK4-wvcxspNM!j*_$P^Cb*O6Q6HX$xa`o+So78-Q7A?FZwMzLcxO)-l*ap;4MCfMki(H(K(|}= z%dxxj9*_d9-lw`>Z#lYb%0RwF$p%%;Jci?{^JdLlz>mhbPg*)^=j)l~@Nw@X@u(}x zNos06UsF5o|4fQ5YV@h4nvI$C!J|G3isc!Ao-C0X`xa{P@D2*2YD}{rgbWBXnPqAI(aAy6hi^F7za?lUtB}VQe z*SZ2#GWuGqtH-Wm+COZwEI@L`xbp1O^Hvw=-GA-dfS$l;D+;Xgf^6^@edP+Yc^^{J zM}VAbc+ho>(qC20UNcm>wJIM&=hYMX$23kI~=bcbuN^bvU^fr2|p|VhpE?`p6+vC)K^pj=nsT*FL(hNCqL}Xd=8HOACA|r;BA+2-yfywLlsN4PG^??gzq-qH$RfLhZ-o zEccvu^QyIH0H_tgS{*bcZ6wL57yF%Ur4~B}%W;$QD;ib)isCZgv#MBf^ruE>9!B-n z@VE&BH2wLxOP8-YD5BZlF+2N*>DMulvLGV+^klu@rQ$0Znvxb}A}8Q0H!UBQ2-{-J z{Wel#G0Mn{9sT=;NKAl%Q%pLJrrnJNC_ClnRUnB{13E$$-5ncyijxLUTpDL5bOc}I z@+SEbe>5Vk2CctbDJqd(9!l`)w7@fVK*5!_I<8Xr05|8JvbAsoBbtLxtZxtS(2~=* zdvdpgZ<*$EOMsT2WkkO9zt{i>2go7-{dsCZNfeLl{Ek4hm>7yjIZQGN*x-FsLX1Kb zfMfw+pN-)M{t1Su+yc?XC3AC+qnM(qs6zI|OX)vPyMtQ@jm z3(1Yza&7qvfs)U4%4x1a8gNBz+s`7ax({rvmz*LY;yJM~1jbe)82a9@mxY~W>?oaW zb_3iSZT|T{G>)9P>C|hT=5k=8?0vw2I3ua@spg?p4>O%y^#S5NM*AqzGoDA<1|nsj zDh(|g@#L{MMn)7P%ioR45Ae%nXJ~qGnmCEiml|T9KZ#&1aiS@`0WGM1t&NkWcjnD% zILxix(+f@KqgRFx7$$kG016OWha%`30{e*Ix@N9p$Z?Bt9vPw=e)3lp3$sU7vUyCYBK0n7djv|Hrm zIk1U>C}^y*YA@shhZS#%6?+<04u8yT1aOM%4D#)jj}qFU?Fg1%cqmQ>&1vTG2xskx zv}@+HIG`JW_^Y`aF8Ni@+|UVi*$$a(=3uHb-=3eVRY) zh`@vEm!*WkK2{$KR@x_bgXjtm?F@Rz5#W~Lo2FmUOIJVX&C_i>yO=g%bePofzuH9Q zUPTAa=fM>ZuWSIcCRLeLoPgJZ1>?IUT!B1Dzgn1>*Z`5Sm@dq$tPn|Zdn?p~sJ`v+ zVsE35-ehlA%k3=2LLsA2n&1R@*0?#J&VDI{PmC&V`lCT^psC3N*FgKlH>uQnv(w{0 zfmd9#Tu-gwpK{i%U-l3s{ek>fN%5cSPjOGZepCil{^cZuV(JLwZiL$%v z9cs>{984_3)arlRyzSgP2$H9SDaCpTXw zpyU>(@1&U2)3%B~2fdBkIm485NIwf=qS%BWJynUrmTlrY8!A=}u~Mf9JldO%=l;A! z>2%)p!0`p=Ted-g)FYYnP|Jlj2uK}>k;n9eJH`PcNV3lIG4q=d$3isz)1lRMa}P9J zrY4v1$|<^P#k#?yFP-Qumiym8TTJ_glaw%q>}@t;9WI}}EBHp8$^CJRL5l0nJo*kH zB>W#{KepfX764xR<~Rp+3Ny^5GquEVWj&OGta*2(GJbD+qQdqNmnP5fZ$P72?>n)e zoL$!@^yYycrY{)n4f2bk}s{DD6E*RIVs+16duI!3(WL|ciDDufw2or)ir?|(A94>Yl8=Srl#2k7zj zyA3*r+^CZ~jtzX*@EK0-PVy%xkD|&7l#@W-;znB?{vH=j?e2T+3RAjDz9R574oB>4 z!6pQG8ab!V4ORr0tO^SzM-N|Y#X~bJ!thoy8=f|cb2X=y1Kkjgr*}qb>37XA8AHc0 zPHV0^3gb(MYoH+JG)GFWwMJhG=)x(*NSa3X$@9Uyg#xMc0z1Zf7Nr5iJXeVg z^~o)g$F`ed>tCld{hp!C9CZXbAcROP4T*S3Ri~U;YB@3>+yto~Rr; zcC!z4cb*b;uPh6xV*Hkuq#_e~gr#fE>a*z>z${AW6&F`-{Q7AA^M+{M1Qm)!@e>~z z%(*34jG}-8Y_)cC8FQ^v3u@-ww5IOO?-Z zBVmsuSb09T-UPf#5oQM1<1iAwJS7&vWvf)zPE)juFO5engH<*x3u};PXe-+3PbONl z6Nb2OuJ}&*u-EZ?zCIx&m!qCR?<%vMX2msFZ1U16PuS79Zaq7MshgOoIZVb03YL0V zwFtjZ4*RfNO5U85>y^D@%t$JRrcyPrCIvZ>0VjwRy#m>=J;Q<0Xf|}=+zUQXYklWk zniGGVgVB@6wz~R=C$rn4@0+NhXzbBI)r~0bv3Yz>R420@{PKBmDD%~9>~U&#D=da{ zY2VUKG_RwFl#4(0s9_^sgqQ}EiompcA)ZBaTW@*uYq~7HgF;snZr`2cBvg)V{n@Ia z$fXuHRSR>ter3&E_Yc>#l_O_#bw$HVy}NBHb}T)5J(Tm5-4`N%frOBIdzq;HH;`H} zvm-9#aFyf=tKQx9FROzX(kiR=2Du;;ucq}lVX8_)RSWF%C01+!hb8-9P7@XL6jDAV zN0}x20S!|Pnh?4=j@mFwk7IQoqznHYd)GV$fc=LSUlh*jH$8*PTy^nLxo(*{YT00P z9Un=ZS+$WQpfSr_x_E;t$>Q3uuwds90vvJlwY>qE^%{t(5^|d%w5kpL)B7` z!-bJc`rKzc3#*$iR2>_R(_Kb6Qrp2ag6Km@dgLvBPHbB9&*>#_w{cH-ebz8`xP;Q{ z*re7EbQ{j1I-gS8StV0Z)-$li`yW5sU^*r7RqSH-U8-P!`$aYMV(o)Km`;N`aHp%ud8F|Mc zVrXn=b}lX!TFm647hyiz_0!HZm74Y0d^hyGnaHf7uAg5NF}d+_^wO8Tzmm=pzXvTF zuHSOmE~>)Ht4hak2ffX9{3^N%Q$}wkr$3Ob{CmlX03y>F&9m|DJ&3Qqb;^QcuYo;X zgExAd3~-4AP?2{$#O@;-CBupL>I+VHCI;nR6LFwUGIy_4TdfcpuR{C=IPW5lsKRqk z4+1f%uWfDSK!h#@XA@`SJF7t-Zkl}y%Vl+h-iwrTgAlZE)Y)KHzv_w)*+Uhs`d7@f zvjfW?JF%lk0(XGFWKAI%c1k`aG8hx%X9-#Dj&dV_HGAkdMR%S=C@1T9Ow{xdRQAWD;jAVba|&37ZR!daQkH7|MTtP*s=4iE5zl@5&;->oUs2hjvD$3 z_n45J_=UN8kD$Q3M4cE9-c1mnvI3O0(7q(+B5VkW?`QJqbM60fi$2*HcU&gD{7?DM za*NoA<*{eqE^_@k66>#`X2>ewpsoOnE`zm+pow3mYab~0$;>cF+%I+7r?FbtFfORm zT$fiQYTui>h;uJoCnwpXZ1a_u^opeqBYv-YaS-=D$sisi#<&5umdv1w>TE)bm=dV8B1Zls2{`V9Zc-=45 zBv2iMu}a~`Dy(kJ|4~<`#Y`0kl-D}~7vfmL8&``6!3o1S2gJcb!3ef-yKGvJsH;H{ z-7kesUIA8z3SCv4^^9rsl!nZ$_HL(?j^N@9l57M~3=Q+~wbRyJb|`0YS2=oge-7I`_r3aY=aa`rj@j{T@KBI2IqOm8YQ4CGWsta z8tJ)3it(KSuBXw?mK9Xd(L{P4TYPE4Qim5y?}R=Z^C?dWc+M~I?=`WJq?@53^y0sFR@sGkzutRp{3>$ZrA>qb0x`v6+Liv28Cf82;IZS5x?<)C%6RZVcnMn4ui+(KfGpOFpySo;t>tEpiSg(-h$c1)1pYw`)ql8l>+ofY}e zdoUgBzxKx31B4VX<(+BcDawH%9_tlyo*&Z|emDn9Z0VNx?u;1pO zzxl`C5DolT#59e=n=LF@40)$4L+J9p1?&DPnm5ZfNJt4hvPObYFe2My*tG?T8ihAN z=skPr_#c383Kap%@O73pO99;U1*x3fdO61aHvr~uHJN`mlK%1aCM}S!3oNG;rQc#| z|D(+RKm0R214Nq_hN|QOasKfq|CNFMe|T8m*}VX)?$>oRPug!D;r}4ze}mjO!ug)6 z;5?P!0y6tT5>#vVl)RVzLjq+an(%2&r#{Bz^!{c=|Kn;IX~T`jne`$xU{Pwo<0pE; zU!k}`q8f;3`XgVZdEDApp)clh1EES~8V zh(mG7S^P3Hag{Gaajgv6lb&hLh@}%Fu|E%Z+a*H+arL-9J`G|!8@=0 zB5;X+`HW`kkIDOow|MZzIk^Z+Bhar!#{<hYxR-ZF{cuy`y%`JFWQQKc&I-XP;jg_&2cxLOaF@yA5HJSFUd?D6pZo zQEm10m6}mnQ2H_U$A$kO;A0f9*BJ$jJknZ|asAidUPmp@>0Da!qo!kH7+J#2ZL#du z95AS7F~5o+L9a+cHw{uc*$aMPEdIx&(<1ti804SuYfCW{^Ln{Ex+=g7i~qyV{_)ZK z6ro8*Giv0jk}#MBwu0^k)3Xl*>@SmIJozE^ZYqOyYbeM{ds*;T3W(Ge5nhb;wL}0X z@!OXO*`O~bHNjt=lf~UwmhGuqS!>zI4}JLUn%}4IkB`Pa3w>A1V&u6o+o?>^-y};r zRl!voC-3L+M;r?h`aZare4b2QHPcYO_v=+2{8e-RE4a#}k26t?X0*_x$|; zHhKbgt|Ot5W*18}FzE2`nSd7V-fz&~-!K3BtP$g7)9_Q6>Tb$w*pxmzZ&AG3RY3-j^{Bgr? zZ{;7u?s!+;nFLeZq*TaGTBZ@@Wg)V?@RBcUW{;PpP)BQ>bS@~YT7$nT>tm5BnB3<3 z?RHW)6p1E-F?>+qjdO7Bnm_d@slZ=~;g|c2KDEb;{mcKsCj2jB`A@?EaK$lq|5H^4^s0>J<^qpOY-}28N8^00kF%Cu54G@k}d| z>c3%Zc!?klJ(AI~9ckICG+(;xPXl?A^h&51`O+dq<_`gAMpS-)(2Y9L+KWiQ*5Vh& z6f*vAJR)hZ=NYeB8talNfPzEJT&juvN^X}ppd2ah!+{e0rI#1~$JgN}3zdE$&tvxP zeL^7K(mOCs>^J;-aqb^y!EbMcOrCaaD-2A_3ffz1sV>-?jEsE<78*p4AZ;|P-pT)S zDTO>Gp4{ahkr(KI9$xgW(*kDy&aP(z)&_gdnT|i)ttI~+;WXF{@=nl#qtsm8KMxpk zxD3Pve1^zS(#n)x*?kjote<}ni2TD5&rS=!UOUY0ZeE!jADMSJ$aFo;tI^{0itqR;e`-dO-<0CINIAgc|C64ieg~SyV zx%~}?PwPJ#|BDNt`sE(qU(c%ow-*yCTk@w86I#J!vVg5Dmq00`SFfT&A>jM(`}mK= z`0cIy3$Nlvo)8MG8UNdfC1+674?g}yO^SB)%P^nyw#j{D6zZ@k^;Lg1xV(WdijSxwt_E`PblUR&#P6GMqiRA8vn9T zo|1Z%-)$Gm@v0v*f7~7;AF_s@R@)Dr62GZApsTv5BE}uRRQBt3RSmWn{SL+bb3A`PzlT)e zkIyJ`_M~QFCFrrXbM_{naEG3!U}Kf?VJH2r-~Ul}|FL-g`Bo?xx<|#$%Nc-c_xBSs z-4SkGqbat%Go9kM|Ns4(|Na_mi+skJ*k9biy4bK7n_)sO4zT4~zkf5hD?JJZ1ut>F zD9V5QU;pw?lIiG2Ze%(?rH+K<^2aqI688mN)+9}w1pp)?hz!VR`_Mb(KPUcAo8gb` z_CLLi7ZLjG%K!MEmgVnD^bfD);ebOf<(Vz|uRrDg`ARS*s4xD1`}hBk>0g_||M%xf z9^T6eGdW3J%<*Broc23-o0hg&CQNE>&TGlH16i3`U}dwOrn6=2nm-?{&Hl0SFVu+TpEm3T@f5rP4xA%wztkP1odHPh^{wh zA1_*~yUqP_U22+dN2g>1+GY>jFm8iQf1GQY4%L{1 zugQgQIcx|hT&)nh+hp>dcpaHfH`LF*bUEAMC{%qUGACEWI5OigMcbMia0k z@wh305-BAaraH}@j>>7F`$AdhJk&z**?Um%SlaqQ4%1y_sQEF7m&kK?0gj64dVpTo zufLc-vAYJG7z+~@E@#^f8MVm{7l5a17C`7-?+#|Fiqy2d9?n7$qSvQ^Jk_-&(DaH$ z6kwNG@XzT3^+hV`YHBe&)tRTRAdZWwXu0I_9bI6pcn?8%-MnY%brjVVOLc^Gd@-P| zOWe2uIia6WpnY?cKG{TkLWH&?e^#K$s>()6EBgR|V48!10F-w#x7#4^!q14@U1izm z?t4};$xIK3rt$8iHK@%VAUt_B*VzGJK^+@p$K*@Z#M3l+2{SPr?Y?QnX_P0yi~J-1i##wpzKeU5{Rmf(OJG1Aw5 zbcDpvQ21*^ehiE8ul3kRJckxvdiYD}>~}XFhIZSMe-^yRINC3%dRn;c?MWx8yZm6n z)M504&?3ahHr0kWr0JmnxzsV1H*s<2LMj&0G4FWNncyI*$K+)opJ>Y9GdP{|+xKl{ zXJ)of$nWMchMW-WJJ#fe;O^h(C>ql%3Gz2<+!qz@PckMNeCBmp5g67k=;2TmxNm;$ zFt)d{yb-WdbDI@bCw(Q!t!!cNAYpf^(d@t-o202jJMGq@Z*8RWaUx9Dy@Z_o6VxU@ z)a`1{f}29x)s&;=xTsqpww7#_h4Z7knwc$}x3@U~x-@(as&%GG7VpPi}AHTNH*LPiS?qaCgrz@(np0}L-%UmOaUZFD! z_4vLs!1{zj`4?RP7^gNFQRkNA?(h-g&_OBdwCCh*`KYW-+m~{!`|Anwyn@OXuNUJ( zq#D6UMv$cAnVL16x3cy@@y_;B&l?wj5zQ zmF{Nv6GJh{`f+oFAi)Fq383;ndK;`c`9PoehI~DA*p4f1t=O1W4!VK;N>-+z^B-XA zc08Ih`WouWop`{7Af^tg1Gty1ksAS~?V6(ykZ%oeVAtp8u!RDIX9~a;yC1$j7Kv3e z=Z13atkvf%F{26ISPrl{LP;0F#-w^OvUd$g`R2e_rp>B9D5|vY)oi3{v@i>;Cff{H zL4kE~Zov#czY9Wf0~+V-%YN1>zpfLV#$`cI%`w*sP)Ga|z|Po56JYH!p=Bk1@wP_x zdv-#ba0f$bcgOx(26WK&Wb)&t`+i)R8DZ`+kNEQ<_bZ>;>oYw%dsOk~aDaKTw5Q3r z<_G?_l`mba*X|yBSSu@mxPzCDSe&&CPxZ+s<9_)yU7>}Cw_lowX@>P;uS9L5k)t(H zb==E5hq;*{zTFZcP+ zc8${;3O6x6bP-z%4m@%ikDt;jnVOlE9?pM(t2xw->@RUfGEQeqAt7VG zA^xDN(Dg?Ph7!w{4%A$>em~(0<2(D9uC>U4MN5u=YWLnFQR%+JGLN8I4&L2!=kzV@_sw z#cE652ZeGxTi=w~>LzuAx!V?=bTnx?Z&7ObQT$+R-=>vu+!S06pjf`ZIG}|;<@1;X zIm`_(60Qr7__lY+1>jcwn|rMBmZ5!{fcYcMD0o zzv(0b-3;bMU5+OuZZuHyRJ9mcbKlvs)RIaxWQLR(?g6l>%|@C>!&gEX{6P{Ccg&S2 z9PU`yXOrNc`zIe2ui``GpvMPr4T30`EPs&g$FEt~()Cxq4#RD%R>o3=#}D?zDx#b( z3cepEgBN=5I_u|FA26ATPlUu6UTah5!hd`oI(e}6C2IERmsjr;`QZYz55n5IaEVXR zAa+Vi`m}}~ya48-z&6wDY0M+OozqL1h!Vh!%c|MF;H~=0W=0y+!h1S#;=( zcv;0aF8_Oz33xb3Ac36sn^@Ij|W5=DcuoI^0jfr3b7X=TpL>(AWdCh=| z<66Is?XDRfLn7^SUm|kG%Q5f1>Oqf?ZNWMv&Fh8d4p*2fa|cSuaLP^QvE6tYxP3?4 zl2Nlx!@Do);_i-j;RPr4eXkzZTFfOE?%P+joKY8*?3;gn3bl7^;A|ySH>gBzKuHsG zD%>^5;@WV|*+#hl>fd^851U=HdjaB!KMKLR1BLuo0jpLNPDkN;fMwnmZ?VsZMrLEknjjNm zX+Ue#;<%~yh4UQ;5D2(d>kaio_XH(c?8*KeC?8&Kt_^79c2HB3gmBMKL&>buLV!g# z__DYO4gl>50rw^&$z}7R*TUU-RiRv3n7td{JB4*pmK?WrrN-+I10`8Eq=kaFTXLxn zIw|ZqzL+&!_UI%r&X>DI=K#vj+FAhczS#$ifb*dQoD2+FqhUG(`g@>gfY_fG(68!| z2|(@v%gaw>hU%ucqrO+3RktDq)FVDZDIDAgGNuhz>*ho0!ra|ZJM(t&=_R4-Ve=8M zzYt)GPA9Q#zBQS90?sN-65a7A&V7Qxo5RM< zsm8BK8#T4IAw&yS15S@mB~lm#_-;G|IZ}w!x|Q^J*m|!rb9)*@(n@|L*CBklz9C-A zu)Ur_qFJjh`4Ukdag*pazI{-0RD;SaV8~-xAW(vvwjh{R)sKdyo6`Jp^{cDjAs(zX zKr_t@Tnr8GoCUUD5{jyv`LpwA3LUyr;1k9U(v)Fvx7Nthad<4@kuVE*o1?1agwMGA zqGjw7k>jEyJeWVo4!Axcx?ZV9?Xa;^7^q__UAe8>ot;ziSH0CZ$?>OdD?^>6XUo7% zgEaGA&c1|nl(Eg;jfh>`#a*RcxLypOTM;A7XcWsB&h~wd&~Fz26|=N!=RPzpM+~|u zFX_&&Uw2--6`-v&7fy*N^|byfpVxkdllaURgfc-D6XI(oo)49vmZW+#-i*MsSa^4D zb6m|rKkBc72j>u>f?m3|t0(VU=)U5K!2HD_XDgXLzV^XkYuFv=wH8Fubywhqxh*c- ztaEbfx^gg7FMig<>gGxHKVmWSo9yG00eq+#+$~)o=UxK}w*;TzkoPgR>6Nq>fYco* zEj>&ReOWQ~4%}P0q}_zjU3YWbYuZy{JwN9t!wH!?aE}`;WP$pDG8|KF2kUj+cr1Ow zaB%A>_S3Y_R%}3}=!4m3YNku!;!hyJUAj86z9UmMl!r4Ddh%i#KunQ?3JEJ8rjR1~ zc+R$61~d%v?9g+5?1~^2yME!CF!y9e$Dc@dxY+luozZ=1956B7V}7wreJ2uOe`x$U|keF&w{YK&YRQ;G)^40VSh zf}=2lkto__w}3XWKuZRdxcF8@>ezQpYMLA``RQ~EGUG0wT-Rz0Z3n@F z{8y+%V_PzOsOSZ!!CmB*cb7Ze=IgJo5O9Rrf_cq9_JS?iNMa;~z zbTs&{_pSSt{Sj@dVy>%YVz+5|HAKAL$iOnDs07e1Xx7#0dRKmF)jLfCTq zNGlB&&6Bi1T~O2AgS(uyz;U6T;g~*D(j;Wf87Jgurxl^1M$l}od%{Bcij(|lEs3=P zo2q-u>L#Jtev8>xFF1u3vKRb}PdOV-8w2gjKPQ01IN%^AGSW+aFi`(h&1!{M zVa?Hts>sy68<$gnr(NeuHLk@y?hLY92Yr2v@;n|&WHkuDFst=ox@g7)`gBl>l{sq1 z(1ExpIP6K{SpIgx&JtX>O_Q&D!9rMHL)7i}=kMIDG|oQWUwxwpMV4^l zdTHi}>acq}De_KFZ4=_aaW!y)r&r;rSD3)%(x}r7f!*X zNiJ;nrBw==bl*0zz1fI5kPY!|;X@8aH9nc2Fia{bt1CAg=ts`pojuxzBtT8=vK(YIrLw<+ror@aQQ%}F*o{S!|B`H3CTH>OWvY3{Wf7otH zg3F)QXW)5pyD@*@YW_0pUFzVwUNeKCCH^~Erl^79X0s!0Tqy<0xH%0$yI#7l&a*_r zF4M;_5BB310flpg0c7B@Xd*aTJ>ohauIDV_!r7MuECm&LW z-~3(@rpM!y1CFg^WK#^deP)nt+Jb_GvFPCZ0YH?_65IbuQ1OR|^Pa}UC_DkU&^A6d&$RvU42nN+y1R1De_M{YQUWw@ zWEX%=cpin2YsijzyfXfczDZ|X=`5%~xe*Ey31@%HeE}!zoXEWCl|Y6^^C9RLi_B7= z=@QSDM1+Mmdn~Y0#-xO?26rC?Y?eLw!jZaF0(gcPB~4T8pX~XRBTvJWgHk#w>UcLs z|3ODt&k&NO^h#h?39yfW0)HZlPALy-bdkC;VZ}H4XMlg98gLH((xUYK5HIst_%R0; z+%N>FfQ@QSQ$q<@gj4NR6a#kc2Mo#PY0!nGaI{Q zBz^Oiz}|(V8DB+uQTJKudNXk?3}uMU0oAw{_bmKBK><_nmS@fz0l{;&ZU&YneKi{P zLVFlqRAfWB(k79}Kic(umOGPD(>B9yWYp-(Q3o5vp1;x34YQQ$vJc6igDu5#WR4M(ekzNszkK!rWI?jaYcT> z%~!R22?Wh*3?)_{&vK!btiP=ax2e5hc`8Dm2I(Rt7at>ZR?sJ69J*QhmAE3}!trz% zlce%s+l@sWvczn>^9?x+mwZIf>=)+;p&oai(SR=)+I|wMk|Bw^VYN1{(a&&l9v)=` zPSMxh-u;BJICo54xL_oLvv0G^F(09JmpFWO10mo!SDB^f-k_tPuHi2t*ck|eKU}hT z=9kyT^8l`oS3u-UC42CG7;PcuE9^5g$xQw_mlY@XY~vG>DCr!OW5g-~1+`MNgNa$3 zq!drHl}w1CPFSspYnW&JtwA40@t{^m`vV43-V4XJ?@0R#Z&US{9zI}ZLKv%K8L7!x zNP$N3DyQx}>gLEUH!hG#Wc%=nj||v^nh@U669#@R(M@CakoUd2#p>tLcBOvWxg?Z8qu6ec>yJK$L~F z@KUBiGwkNa*Ilv8M1z26jG+}uVlP)Ck8lK}lq;eAiqbUKAWI_H`!tHdMONv#gbm!o zXb#KX<=f{pJL?HGCtNe(@f<4lWoSnbw<4L4d7P-(nzspWuJjd6?@Cvc_baVy)%5S? zt>%$0*3>rYD+f_DPS^(F4u&?qTBkHyaHMyp)-cKTA`3n@7B})%cr;qxHrpn$6XU6V8n0j~0IYwZYcx`AANo z!__U3t;lClWM21tE0?kkI?U$UV#8QWtW`yH#2ZneV`tSTKh1WE8SW2 zm-%>%vDtn6LhEO_95~xt;1G?SCRTK=7Vht=a6VJDSAf2U`QeDd&t-}A zbVQg1mlc7TDWs1?2qFqj6l`x_EME-uxH&+n2*VobkxyKd&Cvyn$7YhVE**G=)!gf^Fq~Jp4*@6Rk;~B!Iqp6fgdT(G zQW_6DqtHc8nbRa{)y6#Fi%ELy^Gn(9MdvR5-~ig#j1n>jOWe3^g}xv%p9X|4zJ%O0!>PA>yZ6j8oYLR1Us|!CI zE#FR37ZY8fZY2f0q~m4nE8K2vFZaFrWT$5ho5M#XxFo~Ri7$`1!j%?xDCCYDuRC{M zz{#I{3ZJ}ddKKa4h@Y|LZc@%$_k?0{9k*^`*_hbkz(ibAvH$K&Gv{z;>lIh*n)pIX z&dYlor%O&|+SEwKfVLmEIgY`y$R2}vm|$etQdPBQI6Hd=0SMAj9lhjodx(8UOyB%;;#s@%^X+l=HN3QiEA!m?2|oigB(gm*zlgv9tx!yB5qj+b}(tNkenAV}Ee z$0;zu9Q?p(=g^L~&6f^!Z_ga0#c;?xFj2~xnprjdv_ji93=q#-@vu_E5t6i-aM{F* z)puD9tdw+XoKB@#A9RG?r!JZY9247j9`L44?E93pCzHqx&E~aE9PGq~@KuEeBET`_ zA*Aj)Wp`ghGrp!f=W1J|>M+#qDWt1_G-KXhIhKCC5n~%jPwjP|8SMYCPEXKX9($bB zOvSPI@zv@Fp594+P-DQzllHRCM(b4BA2-eVv}@ti>d7%Kw{s*;H;cl9mP!Bl_)U)B zeVYspz86wV22I+E;X?sv%*;-Hw|%HbCq@WZKD7Zk7GVY!Iw>wqyV%OKS9E+i{q`8< zJLW0YM@>deacjms0Bt`-gZ_NwWf~rbXup{6*UkucL`SR_N6 z)SB?|jhlcJZB?h3ar(aAft7)Vb}UwpYO0ih$7|H;zPfaBhmRV9!}3bZ%y-UjiTr=A zc^b*X8*?9I95sZHFR-#k_6M{sl47o~o;2;}l|%5{v~MOhQ@XY35CR+S^*bT4C#S1t zHtx`)%H}B|Z}DMBtU-pPYqq2o$N`RgHvS8?@;*jLE?Nn+k3J--x3$)DtPmF?g+hqD zT;ZKF>w4mdp-51gFc?jJIj$x6i|xFkpCI_sX$F|WHO0<>nd5wL4s8(@ZV4!>1~4N2 z%1rW-v$pBJ{zA@s^vR^7Dh7(Mvxf)O*Y(QjA*B4}pBidu(e^=QiU&^vb*WE|jFyX{hh`TCX}|rs4=66AbsUwto=K$>YR0bL ziZEq4{Wf$x{y_jI9~k=1{dESQ5;+9gqejq=o^NM;nf4nV_y=535b@IR2y6g;Y&w_x z-&Y~vcq=}+Ft=NPM7WL3-9f-5`np#wwO zWdbi!ecgayR)Mx{A%=>D#|lVE9KOzJf9y3qOoXAy!^mcOXB6!r(U0!-UNOlpW`(`QQpXKKgQw5tlIFgpspSK1H9H)i{Gmes#(j8)?atMPT7{g%XZ11h}&ik8lernA87W);w9^OT@o=~84~G=+SWYRV%A z-^%gbOzS65KD#!rU$6Qh8Vr&O-o3q_We5wI`)++CA(FwUaMG>l;T60jbaG(Z9f%g{ z7g6(Uib6nY?Ndd$u!N{qPPA8Jw*-nlxdZ1Rtn2h^8Qh)UAvBb#Mm zb3>b%@$lgqL%2dP3q(gE?^fPADjIYZ6%gRfUsOLyiK&SKXO3B(h@G|U4-2w|y7<+- zoV#IWXbI1WlImOol_%8~8gj1Omb!%L8B|z&_H<6)HXM^*@3+Q@dWnSgwDNJj)z8pv z3@=l1GyX803ys5FoMgNh%v`h9@y+T)7kHX=9YUXAR2o^`lbzQ>DPz(|wncc_vnPh4 zl-?*=Cgb%$QKH)5#$A&vi@|i5oEByEd*7lV22Q@iIB9FDXB0}ycO-pQB6>d9cM*r? zB&z499pj*rlo``@pM+-q)vr%05^4#X4*w3zf=7X0-tcSez{`8LKQvrhp{gi&8gUY@$OnFryrTh*3BI^iSz2GbN;4(MvM;EJTTV+Yx4 zR$t(w(T^2NQJa~*KM;~1ItIflY0vel7ck!NPL1>~Z5C8Oe%zfuDE&z2v;tnU8-< z@50da#q)M{C^j0hMqQ2>4#^4wSn9X*%tcDIR74J-p#VxP454ziSLfiR$883Ydgsfe zzI?J{6w{(*2K^uvOWO@2&IhB-)*IB-3$8E9FE)#iew+!53(J6gn4gtR7D`!~9u!3p z3%w9q%gT6!gF)NNfou~EeQqX$0C0gB6>q*wC1qV7$y6S|sDNRTNGzL#%+2%bUnrm7 zMDq#tYa3S7`*NkbKdL;AS*?2iMX!gT=@yRlDGPh3+8_cGtK)r-tK+j8tCzvLLOMEq zsh=oWnx;gz<&cN1>AUMs9a;?>*?Kp5hlNNNy=lE2j6QT-dbRS3i*WK9V-TABM$ebF zj1*H7X7IULj6_K{hDHeOWlXHRtad#gkKeV*l^;!xX({0x*;{vClT8%DL6$(g@~fk;QMfP<_g$94B1ca1!_C=iX3 zoDwacoSZQ}E;A+*o9}16FY(=BS$2By_I_GcRvM-&+^=Fvcwu5fOI1h5Q>XPjsikbU z4SaLHcly2}iCGXA)7DaLoeQm*6KRd74gSW&G1VfkqM(8&Ww4I|o)ei6QamI&AT*v* zrJQ1OF`VF~{ZZYd_Lekbg4%0-3IVb|py4q~IykX_+k)~s;MGQdM%|T<;T+eNg!bU= zI(F%Dj+4q@Fz&}=$wb^2Ijx?}IVR@MG9oR=3tt+!f%>_&nESmZpBMSX`dV(iwf?S3 z!^3bsfVK{eH-lcou`i6?G4abMWo(bq^ab*j&s~y_=O}{4#6q8}!TmVXeY##j zqRnrT;2U+fX{;c<1hK6fbPf5GzJfx{6dfvM?oZ+fh&Y5;h>}{MABbUmIS;f*1TJ#p zfe+u)%NiH$l(^U!lOdC<2|tSLUi-z1_G9$5yr$QKfaZe7+~7lt4ytb|;hmp^I{g5e z@%@$*s2wcn9g{ru*1H6gHb?~J5gLZbqlGadV?5;ca8e&@GKvdh;butlfdGN!M6Avc zP%VSnh0!o8gJb*v=@X#sneBQ)NixO%L5{SNMvleegjP5;cwJ$d(O5HR#ulQ%H>Wfte! z`?YT!=w|MMzu8UpuXuDaXcRWs*UfbMa!+v{0&$-^z<63t@G%DvRXqS;IsG}>>t|y* zcQx?Socld#I1wRT%+^UJ@4AyMP(+bFl%>&VA zHo_L%c|OtD0V;e*$mHzlXo=b=F|<##Hng&Z zu@`G5NX)?}82i$#8{TJ?Q@EAoEfL_;K9$2|y%o^RYNJ1HO%kpIo0+LYn$*lm#9PKp zY_34CM9a413YWZJlEue#S%cdj_$_fu??`(@(`I01f)VfB2VBQ|J_uD3^gQo*JJWPY zQmveUeuqbKTMyZGOYyT&`V|=DrlgQJ`em0&{KA(~lXKleAKjiDcX+wFv-KTl~teYxqNBx(j9QZ@8GyB!683t_gUG!9=6$L&a z@gKbRd=h71S#5g*pJ~ek_zdJe2G^!TR)uXJ=|Q=vw+-_in_U|!jy*N)z>-q zHuYBRl96Nju|>w4!Ux6J)@5IaqKtm#@#RYr0xB#zUL2)rQ z{$eZ*|1Z~cwrb+7C(x)t1P)ma9!`}HAP|ez{Tcdq6f`m8WGZ&&6{#FLZeUeZuX1a+T_kdEc@O=Qg{VW#@Agti`hC%_Ro))h?Kr+kn zj#$A|B2x4rsbkxK0y=+LccuwCQx9?s3jjx`m+PnY3alV;>Rexe0k^yvmSg~wBVt7y zWd#Gc@V8Mnx6E^{2CQ1x8;2t`hyj^D5>Fr=xV(6YuJ=%=2F1ullnP}6d6iKw5tbkT zY=uWubc(Qm&pwk~1xXy&`IPH)Xd`6~iQeDDf1MBu0Vtckt=%bjei1hBCx$&?DvcF2 zFzQ$rHdyj9Wq5OE{PZN8L-~Xc@(7n`oZ?Y)%g{mZ{d$9pF9xgkx9WIleNm6;Rb+!?!Qos=|8OjB!p{$ zB`5lEbtnSzj-+u>_UgGjOC>PZIC5y?sx*QhA{9mnTcQr?+c=$@II&@z3$@s^rln8s zCe{N)3KsC<5j63UOYdZNavpCoM^L|L!>x2<{4kI+iJxMV^o7Ac7lpVBAGzwd&g)j| zw4%vw*_U=vYvP*0x5jsaGwOVsclc}|=6+1c^FV}FY_NAToYvvCZXY|p^_$AzMfmsqL4Ku<3F&LpPK2VIXE>uIX>Vzo)8dHquFtyw+XU6M zQI0Ei+dYHE_*}Yr_~P1Een!u4aRt6#*wcJk?Pg9kbFYoLcs=_fE1;s6&vuh4iCh(WK%r{MV}H;Qax5%ltnRttmIIxF}0yp;$J#6_!+O>n)GPSA@uRWl`dFZLNF=L4tX2!7>!{yq^s|~rCH#jC?XOA; z7boFSr3Q@#Zh@RUU-U5e?tcx4-wNMLT)V3X=M3LlrNBS8R%^anzI=NMF2($&G+eG} zLxhI<2yMUzc_o_IU7>q1T*6@@k52rXk`dYKht1z94s2>B?~flJKl;d;?q}SG_dv|2 z$TskRAAJB5yrB+#iA9=;2gDhT$YJ?yM2(Gf7B~Q20+$Wt^#eueD)r4%+SjcugMFua zfb{e#mv40F`2&6eZAJLk1FlcSsFLO~u}3GZ>z%$sW5JwRjPN`lxMc>#q)6-eE{a2_ zi8uwQy8IF@d&HxseP~s0*{LQ>q3=4#y#UaRrnt|F%uy#}E>XP$$C0}yX=yv->X(Q- zs0@EZF7u}(?7&Q_V%cnmX8%mOG;}Wu?Wod?A{(Moy~2H85-=HsP6hyoazFS0`?~V0 zt9Me$rW<-8@SV8l1DF4jcvRU-{Vw@x3kQHG?vbitGVXl}=RyeXTL0Mq=DH!2rb2Wp zHzYH=9)PYxmmvF9wUclAe=>R~!)oI3+H2UI%&?3=0K z5{?SYD0y30A=qyQkaJvx)K0p8T7P{HsLY>+a8eIIf$rXI&P56HKJ@<10bqk1SH%E~ zOpPdc{)+6&8XgM}CMcjM#_B|?6S?=*b;X7|{@OSQXzOV;Sy4jIrU^{6n)+>Vmz9_+ zEOM^D@mg57u&AU$5c#yfJouu_+7~REoXY#GFG*2DAiA-epv#?@d1BAQfV-J~aL>c2 zrezQD{n2&<%C&xW4mW~C-F_d-K>(ii;+K_z-x|9rZ{4LXp&Ugt z9utEEf-=&J<`dG1v15x{BeP0N(a%B1T?C82)>~P>wq7dK+$Z(NyA})eKHWo6to4eS zaZROiz$Cm(S~sb%kL+D_OWn?ex{933X10q<7T&?6*4J$(x7aM%&I~q6+Gjckz+40a-LPN+_<@mPkpwGhwvjG4}L* zx_)ixlTTBs=+ocr#^%etyLmP|>fDuV&q|v6JYU=YTM#dD=SiW&xW9gC^IlTQ7>%9azC6d*?kdM0Dcw_EZoEO<>UcNt9x2!F4*908We zt8*{uj@OZ$T>aB{szPt5dv*BGdmm6d_jK4(>5)oEIqTw};N)VBuh&~0-=TaClN1O+ z*UuV&j`9Lf%vh&N+~OlOP7qNP&RJk{R!phm2=lKl_k81^p#jy>j_odl5USt zc{*>%7Vwf0?%$SZfzF->RTzSA&ldnpvF3^9NxUIjLag9;Wxtj`K2LWKS*ot^meFv=T9ji z0?5rMm^tZ%_4mlWK&g`gl*Z7C?Sk_9RBN#=TS#m2;{M@_g_3KHrU}~Oc9dmOAD}#Y z^E!b}^}uznv@TCnyE6{d+S98S)R&}Gjqxpj(c(DpqdMp7t`K=mTFvs{Vuwk7O)OIX zvo|n6;ji&QLb=V(T5U~{t##aLBi4yo5lt*t1Uc-4-zeuf-cw|9gh!hnUMS-fM|}6L z_iUt7=3`+iL$!*4N>bzcgWSQC<;f#jlycq^$?^sl(cX;FtwUj}!4b;KX3l`hRn<1F z+CY^{_Lu68aC;^x8}(Pm{E~#|^1E?lTa!%? z#3K%i%I4-*A?IF@4KVv6ZX-mLf4P*MELE4AeZk@!-!m06Iby%Sv)a4cgi-JQr^CJq zC#v6aJ{DWl?V@|uK?hqZ-6OmYsLl7%8g83=sDgC_1sWM&WsDpo8`^buh~BAnQb8{G zyvHWQ0@Zk`hrZr=Dk+baJ5}bYnqD+^Taj?r%xTx%2-%RPS&?xboiqd`r4WRO5!waB>WO=YBR1h!&m35h z!kX`k^^8eODkQ_o4}I!Ae)6D?)hh+_>=R188QAql8b~E^}48i94{)-mv`R4Lp>SuSF%2%az zi$($J3BtB}-nZHpx1TeGvxpJ+Dh!gb7C?8an=qGTkp=9eR20rWO|W4Nl;MbqDXDq6 z7o5nf4QxNXM6o?-ISp|u<%NJHf7rS!ndg@^-YUdLK!$*ZhT-YY?O@Qp8YG252rRuI zb>y(-HpV2@I!Oe?TRgOoxOI!hva$v?CXu(4I_M`~RdceJ0Jdr}K}?rqG?zGO{NuT? z6?t^xkk<`tvBu~@*LbnIX9Up8_c30lgS?(O@cGTY>@P;c2R37c{E=%G>HfP#pQ9NC ztCLaoG6BAKBTXofcPBru3v-M+^XxT7Kj?2KqSudX3$eOzIi57j>AJvOE5S-}(mi*c z+e0z}W(KfFRUq+Mp(?aX2%Lk4_`) zX`voLhZ3uaeoG>%@9lPKS>0^kp}G(NK}si(QY5?u3Iy}-trh?^$t^Hbu4UR!BCo<# zAlqW8?7|Hw2Cumbqn*)oXwSNnoD(R)QY^p5l>riTCa#jKwkF(}FWZ39>1p(n2ln6k zA%WdQWtF{Y=@5phzAj=%-LIgbWCNF<6>n=aNm#Jq*BbuiX%TLYMK?Ryj%O!oiFyHI z#-Y#D{9lQx%IKcRVLeQM5@JD;b(*d2(9i$i0%)J1M_et#R6?|f+z;tl<$4M4cJUNR zLCZ>hJzQnxB1vYddJMg^qMVG0xgF_RO<78KMc_-JlS+=|Hy)a0lh9D1uDZExmXe`K zk7Biy?uYTiWX<3E@Tp}(E9;h%)sp%&S?Eo@$i%)Wb3I`_t86J-!ct8Ry$(!{ueGn zUuVQpeC<3w)Yy0g=PV>A%A&)}=0+?TVi!&&Vw#tPlKXLoF9RklHRTSTp}(P#OWOR+ zKKEV%L;9tndi+g^kCn`__Q2tjeF25fC^uj1D|TYfyx(0dIQ&q|{%HY)*3VgZ@85ro zoYsOE*iD&z(Iw=|Z90OygjJ3b+gaeHqx$|r;r>bOmosL$7omJ^P9Jl8QL$(iP6}53 zO2A+{C%^I1_o_$)dwH}!314BzwWoUFz$T-e#y%wt)nx*9{dB!38O}rkE`TD@VVXBw z2enPTHN6!cQXTd7<*LYoUUK8fR2;o}3hif+J*}1*ZXnT7-jQ#`dj%d;vQ-)?JO`CX zb}F$6ILsw|cziV>au2;LmO%Z2BGaw?F9(2i_-rLN$iN0(zo2tmEz$~df6I+T-@h&z zP>*pKyM1aDpqc{V<+i4uu9=Tg(mU^#!D$-9Th!BF69uC-MiO9Az4x5s#tml<$l;HK& z=LLYQFK3BV`}w9cf#DjsPw+?$O}w*FPr{h%SuEe_T7v3@UNYQa_q%BF#2itw0uX+SA2!>>dd%;hvTnR=a(l#tX3rL`Jgyp?ja_MJDTYiYEZZ4+Z3;^8uY+vhk)b8JlH4)SSMcSh= z-{IkVq1>Up>ox<27(6w@S^Y*a=9rg(a`Z%?e=voyPtnW;! ztnOI-@xaLt~5RZD<>*+ z&lZ)&yyY?crd)+-We%ODMCbSKI44??Wf$L)am7tJY|>CFRF;IxTi*<-(wPcp-CN_u zdamaiy)#{z(b3?S%Ck1;fm8F1pG;dc`Zk7y$8B3zhYh?>0CgtRsGnh%1{TxO z0Nzy1I(VtSBk&F<*s}fd*E#2{J4W@(@!gi%uIjmTdH*85A$X4xh|Sh;yKV14qV;t9 z#YbUZNUhh2-$)YmdCj*VIJW9cn~uvF5+99C{nBYYdkj3B3?XqA>itY8NY4!R9!d%qBUj z-G2(!%D9yo)=3j;?%|}<)DS5ttDn-Z;vZf<%donB#OE`)C44kV*Sw$R__!vNVZcG# z)0Q?X;a*$7U~+l=L^QnB%s$aqrI@twD7yT{j_sR%z&rQ%b=!AXhtUGEI4k!HQrAA= zW2dSiX!jRsUXk^fB<=d)#E@;!zd9m3h#`icpQ}bUu#kt_lux zA)1=|i+X4lb~RAA+?+}Q4guaonLRWa9#kt%;nd@;Kqt3FO^5h_^FQEm$5n`45CSqI zExBs{0|DvjvgeEi0UJ*8=KfiZkV}ttf3^NT9T754%O~*NX^v;p;n?Q*p{mFgwMZH4 z5~$dKh6E3VMB!lp{Y;IalcJaH*k@z<2AiSwlC`Dp_tgsCxHlkw=kV`*D4iOGLYC4e zq~_L+4eNk(2fl86uc4Su9dQqgbFb~JfUOZYw^m7C{E*o^9Pq!X1CA|+8`ZpO`Mujn zLvaeUk@sQ1@%q{F0-A08DHlNgmHvt#M>gs?52;__D)cn#ut69$KepYnEly{G&;fW? zPqxy-J*8`bL>}++sna)guG-Do630p>eX%n*3@nE+D0Hp(i zn;p0D6bWwtGwAssdjq}58`4gDVxhm>VXRPQ_8;C*Obi)!f{GbMO3Y{^X1Z!H%WQFt zMojRBl-6~EmDO#uGUCR<6l1sFe4V|Dkvxq@ijxA^Mz9Ncfq1h^%VY_Q2j}-JYIA1G z<9>MQ-I~8PB4b)tgPxH@LhJgy@MzcG2|knaCT5Hq9D4Rl5VE+EoC*2e?tRil3+v39 zQ?AJI(n+b{v{zVydD^+Y|BOpvh(>ojYo(O?7JJTP9iJ|*Ak>Fw%_$6%neW{$&#)Yv zS}GwB_It_WxwA&L=^GQ_QA+27RS4|5#@Z=ZXzZZcp=8Vs88?;D|Ky|$dP&r`>%%|X zbS|g=s^@r)W=b`?P|JflAlV|x0-XX{Eonj<%$FTkA+i|nF}b*ZfIv)5^~RN<@-vEJ zSBHCnVKyX%^H~7SJZj7(1&HW~0&>{|Gbi2<+Mt8Z$0k=o*HJ3B+MxZG%hOcfK-#a{ z-&H`T{Yym+>&HYh4C>oUB|E%#E9V1}@WC+#HX#;ns!R8#OHI-tY*-5|4Gv{JjU}HX z*AE5f>KpIjt{j2e{o2FUzkBGiTCX0lD4goJgt&Kj*Xfr3xEuHig^H1>>#pqUnS#)5~C%Ci9K z4$5xSX%;-<%?ij(w`{navGuK36gR!(~)P^ZaljwYxs|x`s{p#)23U zlWMiVd90$~4SdRmcjl~!LZh*u>(rz|sYS8lg#EiRb99|&Vvug}4HMRcDCpK7;@fT6 zIwBZ;x6&kK!iDBo^5V&qId8KF5eDO!pusDn!$)Q;-ph@fNHe0fYocARM=TzkoJfaY z1soLCMgJ7{ezrqavDn*nfZ zdgG6HYZ%t-9@anZgPW80(izfzYeNH)O@7)b@=(*IWS>5NkqEfAb_#Mx2`=Zv^$g&51Q1G;jK|HTYW5`0?~oDNcjJRH(wMRTrIUF=!+HgfKz zs=2G$$)t+r%cpPM_Ckewve=;t+B@F>9vb4ik!Iyb!uqJq4tO9>Anwr?nFb&ohjl>T z>E^KAsF+9jL<9=R2wBhq6XCZ?ymVx0G1y1`tIDn;6#c3lbltl0fl0ud6Y|w2EJeM# zCeNJpBnP;wI$O)=*tgk@iz=P;5%arvX;p=1_XUd61K=f{w-DmX5)2G zun*^ApAQt-<89fGEL(|+->o{R$j330>y@xm;+iJ1!`Xu$ZL|1F!C!Os77sjL2l@K7 z)Il1ItKsSBLms^1p0m^&LcWKu7bP91=_uldXyU*)1hX*Jxc5uNgZ>?v);39eM%xj} zn@PWqk68Ak7{&ra?&pdZa`acqrY9@?q0^pG!x}AyhIfU1K(Dz0w(OECsINP%pwEVoy zXz~>uE7d4ptMjy%u>Z6aaiYtGh*_r8kb7n+mC9`{$8+yawOMbumm< z?QPi>J;B@-rJAJfL&@_)ZUL|X^dp*772n5Gt>55p`i=8ScVWg86x&>3uHiQ?)bTE; z`_+4F4#hKSu3f$c;a1X|%X}0BLp=_kwONDEEuB-_4W`H$F~vq}<9;f-;w<&e!TI4e z1cUhb>^>+rF3`5~ras8ZEWHl?SSc5=d_l5^c8^VNShnghr`+5Ro#FV8FtSyA&gB5R z+If(((A$`?#B$Ho3fCCNlX+?*Z z$*fk>wygx249J{sL7*cJ!v`JB!8@s!X?fxf-@>tKeEICxv+r-S^qsfU!+lhdim08i zAJV?UMXYVUp9eV&nKX^0_@>xeeaP;m_8Vy(L+JL>IkfI_nvy4;@UzY4k& zxXC{|>f@xUaL00h(vLUWfb$Jd1+5`iVT^V%z@gaDjieNXH4gs>uK;S-<8fj-I(4gG zN;S;=Q8nRv+MJFm+6;ngTl9)QBvdY%atg1RlFb?+@$yfK`| z&C=W#R=fywjURn&$A1kW?@nO{0};>3xl?0H_pw%$h=EOfinlOi?Wc$kgnB4N{)pKT z2BfS?WNa5nYw{-7{W{u0tKFYmi7IH>E_@DtcDSnT?v_?Z!vVe23{^oYP zNCU%_jr>Rs!XMx$B!D{RmDgzykz2uAZReR!Q%GSIEW6k5{hts+!p*9aTU&o{wp+bt zs8>c(Y|ejZIJ@yUOxvA9cBv`A~RHf|%lSbddAOKRY8N6VYS!c-eG$X|&Z{4sq* z*|b+dY17Z!2jb1h9ak0^%Ot9QU+JkguP}^Z%c(ovm4EUW=R^oNEO_6G?eym+tX=SJHo%BUpGAIqeru+MAlnA zR}e{&F+13&YC>!CRb_!niAe!$a{lPX)AqOnNbmw*oA48es zVc^1xH7n$X<%r5F@wZ8@1V`9I46!xSvlX(rle%AoYY3m+%n1iy7o!2gc_*u|ldr!Y%#B~j{po8n<`Uc6!l}->Gmk7myQkZ7 z_gHg!5(tQo5wATSpyKm@OuXn4KW&VT7ZKx;2__Il@FN#NWQJbC?yb;HB)~_lE z5UH{PyFQH-D!V>tG?2wyu8-aU2Y4ER7yU2)qjppWfPdk1F?2oX;RPlm6142+F`UUi z<{O>7D4D;*{ARzGj*mmV$mbHknR_&4D>7Gh1@0WR_q7`rZZCK)xsQES^-Q2u{`NK$ zH0%dEJG-o6KCaCUd0-2gdadOWJv5{O-seh{7zSY`=9ppaI;BaM;cI^im73B*;sR79 z?LpA(Chb|P&s7V*041OJ8s~r2qua@2Nkg9p*rpj_&yCI{l~=HQc>}fek0!Z-ut>O3 znR?IC)_;m{i;d6=5gQ4QA`@+6eJK$OB8ZYg1cD%^1<8ejOnv1yJYoqWhfSRcZXR?yOz#M#&infbGPm6O#aSo8xZpF0nJ-Z41KT*LancqPv@eF|I zlpkTCl?|8f7lQx4yZnD1gOT#PfxrtBN*v^{@b4ZaH4QAu#Fq_e$s2{s^6XacfqSt_ zp!=C7d}`rno%-r9_yffV5p-z|HliXa|vABFbl-|a0Hf;ZAd z!Nd}Z18rxc`WgOz4EO&t75{#5_7${TC1U54P~?Ytz<)n`fByq9Djc!cB{ZE#E?}qq zkFWAQf+~<4++~Q1i~LU()8F0clNewjDg&BPwExXjsG<>qZ=QL|%i;WAPtjkdlM69u zQbvy21_%=#Dw2Qsmw);(EbO~Y#yk%f-3l{K>SMIu6kz;@2=hn(b1Nethh+mp%d;AK z`NyUWLIGZr9+sb436|7Gj3OxQ!7z%lJDaX+92bs#ndoK`1~=@pl6cgTz(9FFFdS~%fj2u8{ow)zZv@!&b&S_Fcvn}`gXOpFsoRdHBG6~I*v*SJ} zS|EzwAt>M^jUgC6d;ZtUVJe9o-ff3P9sQf21#Ya{A5ZdkD;kGhoP0~K7Gw7b5hR5# zaS+Fv5b-u?Gw#b0uQGw~IIbXScE@N8Ev6 z`A<^J>@;J|t(;fn&>&lCA?D39fji5Vt+TZ#rqUs$3MG?#{6Gf&7l&cZjJSvdjRqhZN#YIF|n}=j)X= zyE5P%yyn@nyn613piMsV8~J+7BGhFG)Zk+#7EAUFlYnA25yVo@Mi%knk8v3%ljDF> zZ48%?Oo}ls=tg2S#)WSdAQ1LrZ1;ZTzD2)B0;8>d1l=M*-G82- z3^A`^Y%;s+tXfi+Vt?n?Zz2CxSc3S8xEQmuK4V{m1BnP^g@?6JdTIup3;k{9E~N2?ii@JdOdy?(i`m z2VWN{)+j_7IDq3t`ilYoyN_~s>~zkBxx*Yk<%Kx#e=PUVn{}zmmk|K}@jVw$ELLGg|${>i}#pZ2D{V{g2&Sz>=VRt8_k3SIeq$|{#tO*ft83f3)|M7T# zJAMqZAYA`JY5y@j|LHg}=+xxroX!-2A&j6mQo!Blo2JcQ{4wc8QGsq3%fla!Ps?6J zB0>*q0^>NKQb_U^RxEnkir1Tf z#iv0;Z0d{1H*Mo}-TgcaVgI!s{7>Ho#YvE#y&*t9NNNa15aml%jL-a)BCLKQAwbUl z5%0U?9dkg?Ptrfkr}i{RLNY9dW@6+Q6J~%la`8IXtEc~L5##hy8vBp7ISMSM>@-J9 zUk7xiK@&J+h%oms6>){%ZHXHKsKeVjDz2 z*w=UbtKs<6f^`~ID-Ix!SoSHyO&utz{q-8gv1rKc{<`UCOR5Jc_rh|UH2*Nd2^0e( zmmW6htiT(v1@AtwLq&u4!o>{He^Wrq`62-*^Is;4@C7mlF&iZW$p$TLc`WD>^Ur7a zyK#K}AU4ys7A3M(wM^wwsU|u*zzF2Dd_Y8wjiqDykl_C9=K1Ro|BK59>16`Rg5C{c zg76>K$VEZ)K16((jruMkS@NH{rdqt-bBj%Qoo!4UA<&-=!m}p1q|8Q7dzUX++hI3_2-VjRpc;4-3*u*z_}?TD z2*@XZDL3C#JfHyo3nTr*Z;KvG*{ApHA7&VTGiR}Mf#jkw!GDIK|F;hXm<4A|XXr!h zUl-qB*5N<<7$gK_XWq|C{}(m^Zo~)-23aCb-ad7-aGvHFX*5D$wR3D zZ^WylYD((=`Gx`30{KIUhV(Jh1K#V>V>3Pd8e2A^53S}cCRbrCpPl|k*QbMz*Gw5d z!AhXM(y+LWuI}0x8_l+CcQUO2E(%$qpi*D6F9mUD|JOqh3ldX!qCegK)&iM2)Brej zP(Osr;xO|#eg2`!ifXju>?ty}x_DbuUnPg8^(<(ej=QWR)Z43aex9jCnRu(eBN) z22qbDIEXoMY8Q)ax4Vwtv$J#Ipj4kIM6%JB#-RLDii0y~>Dz=d;nW(}-pcxR>wzMH zC)Zl7s8xn zC;Iz!Go@bTRZi;RLdQN3-@}A9_n>lk{Hc~m#K!TI&jOtDx?97Zh%UjHBuHJM8drafoJ8Vwq&to+I@VK7A0MTiwTd`Vfrt`_-!ZjSvnzqSc z;PSgs^_|5LXGtwG&67;btIL58EO%E`xd03x;`jOed6O(?*O^!39d;KZbEe?#2*KjE zyW{v3f1-P~kA3s8t=X;X-e_~vJdJ0v_mK?UIHwo7FRl4ZGZ{bZEMt8)q;xgRc1{`| z78YIN+tD6%a8H3M2FA75=cVQmW1{KP2?Y zUb^**Y5v)U$^PK>s6L6c?l9FZls21x)OT*TyjNM6(YkC?6C5(T$X<9eW4kUVDFsx2 zf}Tm_8l6v|{lxki_`QfSw{8kE<(tuJnAVM0s$q<)P(?wF>hHP?jWAc0tq<+szx1Ad}{ zT~Y!;I{R~4c^)3~UcArm{=x)ncRK`Q(lje*Iu4y{&)ceor2v+D{6lN|;_0WszmyMWCL1tQS zgSNqYA`Rz}2jZ;+3I*=$^;01e#)WJo_19EWx-~QQO?K*&WTIbVBT9@)4XUi}qJS9a!5+zK;_U?MR{1tq>U(r%y;fJs1s3S#BzNx^DnEdsX zs8Q~I9_-Vis5Jbz(p}ZX4>>u78|M#%J{1-|i&aJA_ZX{g7H}8Dd>ny4jL?t`FWCcqONCAoA^IZ(Z_R z<%O?cjUc95yrGwDO7l&28)GccS$EG!cPdvGyuV=eJV)|hEk3*CnQgi6ojOEdghjO7&tv2a9R}+`e66yCJC> z1-rMxRxgH)c59l30ZbVi0mXq9zQ+M4Q*8rPsS?P;Gv)p%oKY{#HRZ{ig}}g+QsVvX zuD}8lPplPn2GP2EYq#_v)@=cdDWkR#v(&XVaeF~>_&4abxqsn$trx%Gn~Ks21pIUo zMEaDXEYonL{re;#&>JD7LFU3aY8n3Iyc2<07hJX_y!!o<{=zbJ*UeE6l@K5v5HoH- zA8&5^0G0GGq=L^WghbpsWZi$_Fa|e3o^1j_GcMwJIpp^ffTDop5&EOyOS|Tb&dj(j zk3fH7o80VIH;FECG?i)BtP@uF%r$8uyk%1)7WPBuZmGCK?LfF9hgM{$JJi6HOFP}y zc;UJL4L6#6qdsD|TMY+HpOz4^;`pRi5PZsMf3OA}I$L^9p}f6*JXhPuQhFDmyhQgd z@~VtK8~Br3T>rSoGe{G>-%(pp#ow7nkUN-q6f{)U-8^Yvh9{Ol!4MwNQjmE zl?m>xjMp7hT+v{kvothn!qGlkFtkR_(hT#q$~S?Hrq0xMv4s!P({2ScKdNx%>)2Xw zkEbP6bMg0UGeD0=KFU>2A_$$Z2!YQn=p| z%sHGw!~#KK@HaO@qRQ>QBs&~EP^o)K5;n`&s8grcNG=5J>ly_RyJ3P`@#ap0)-4yH zdTc-Lw(T*{po33X(M`Dclar=gIA?#~f~?*$YLzkxchB4KL3HipU_$8pvCkc5cw~D@ z*)fwFb7U6LNSC@2cd8s`YGO6{r7yi%h5DrPyXiL}9u1Z~pt0?)Ln!{Jq^ zJ?06nRG3R%)wh{PnjVskT9TAp{f9y-a%Xb;G?Uvj^u8_R@#hmx7WO_lsy9qeYKT(9JU`fFqiag0dN9iK+ zqU-WRsqCUYtChU{vb%oC7_G+j`k`GJGsW>O$|GNd}48y}oCa zNQ+_{A1zz=;XT?WBuE$a*AZ&By1$OeBmp&@n-Y+aQ zN9-_1`+D;Xz=dhXrmq9Q%ZmFTukyR$0y9aXRWJN1GcDE-LeTB=^j3Z$uvNXdi&X}> z&_E3d*<8f`Q_{7^GuglY=gHGi=vU9`qpS*mDh`4qmY zs+*mW7yX=eI{8hX`(xqDvC@s`PW`7MOXXZr1;AuraHVESgTMw?AYLztSjj(aMTL4T zX@f*l`Gz*uP5Gn3a1=?_>c)(#%4Y)C{>k_Jif@f9Xt+L0GDm1}_6jT5l+ z{mGcmD!&!v(I`rbg2b@+pkqS#fU^GgY!oPqF*z z#8ru&fVKeqa?#p!CFWA>WKs`En-PYeYWRHN(w*Mi_zvr`ho=AWO-!734xWhyjNh^* z+*OwPI|k89@H+aHSiERN`Ic>uT((^dlDL&V7wnWb@t^{y^FjIj6|?7*%x*i0CkIp< z=tyoTsB__RJn>dCY~!9*JD(yLQ`Ie+5@GC1u)SC0o)y%uYg}He4y?#Nd}^3Xo3?BYzSJpZ+XbCTVn^Pook7`6AW0(8*4Ux7gk^+KRsYyrw0-?!wLpS$@7 z)nz>~xEoh`e*e1ErBn;lwdH%cgMrE2IdiK%IkO;>@s%x)g#Ss!wf<-*i~aQohP9mv zdY&cS1{!&xyj$R8s`VcZ!&Lvz`wT-po(KU>@B6N7ald(okM*Dvb$z_gHA)92SU2L} z4~r)LGd{*#kXJvC(5jU``P}$KBc|7<*6(_T(su4jYb#E7F3es;QHmN=R+Efh+%Kn; zGW$lJ$Qm^ruJchh#He(<=MY$=tYqpxM;J6>0Kaq1+FxCP0RqXHYk_bzOv%+57tfX{ z&Gd-xC7;DCU6HpBE%jq&reNwrx$mOJfsHwS%9)#hhi}S}=Msa`HK&9VUAl(fM#1CS zQKL1x7a9SCd(8V~5F3T^HrB6dab_)b>hZUkFHQF&+FcEC)NM3aPGoCC6B1PHsJYuH zBg1d9sfDhvSs%1C`(-)0i=911rpO=*DIu{9!^D!nD&tedu;xqPJU_gazsaornWBs> z8^lPSS#YS>1Vdy~&);ZXyl^Hzr78!pkC~eAcVM&8xW}W^zME-ybayo`pbsqz!8?ir zrJ(zS9BQdMJU7wZ%)YM8iD+38m6pVE*wa!L3j5NlNlz(%9$t2ibO{6hMa%S~y(?2p z@|Pavf!Yv+_5bIw3HEm}`kB!{eEZF{!-vKGk&QRCe>I_0uAtmYC?o#%+AiJO&h2o><#C5i*Km_Io=dH+0eE8)8pZPs@s1bhE?IinEO5I1;j z!>?ZrvL+h|9}eXme@*kLNZYa6QXXrR*EruP>sO($8zSxx*V!Wl+!#;_!JL3Rtc(LG z4qW?^^tcSud`#Sz-fi$mu~PZ;z87Nwx5S?O(;!=IEbkH^BbU<9O4E3x=wt3xIo$HhO^Khb8HO)a)V7@Hozl=F`uvq_==si}Qx3I|_Wv$29(n-+KiSQg z(u=Ios3BBBSD1s*c6h4c*An~cakXisJc)=u)i!navaN(Ee|;u0c_OxGlCRDB*}eS= zq(~E{1#USNtvr?UC%Qd|&n!nU99+>C*b$orhh!6xeq!r_w}GqcZ{KZ~j&iY{Y>Suo zw*9evzK^5NNY@aejEGW={uZ_U3`+R%DIJukG_1Mf2qjh6ujbs(k~)XsjpTz5e{rvD z`lS|HgO@GU7^+^5*87fYSHc11A6L7{BW-)CBv==BA~!#6T+Fpn3#pZ?zxQ#hh%~-} z8{LpCbYfTZWm1j*mwuZ6=+r0MhVCbEocujsqk~Dfn%{;-k!`I182}Q}J~(wRl_B4T1`#uO5ccxMhpcE-0Ba)#3nhlYl<)^A?s6&jT&E4SfB0dBl-p> z_li-4Z>#;>;~K^{R#W^3E>Ch{ap_|DOPT6zn`Tt#TBlSOAxv^W z2;qN2*22aXmonsVGUDKH#_o*?eDFx zOUwut`3tXm$0}6HEEVGQ$?ju$yAnD(1&iZ?g&XO64wdmCa3x6n-~V9BLrwX=FBpY- zpYLp<80tRGBz-a;UA8s*b?(iw4(+0*Kb1}D+0*cg!_JllZ^T)3ok!u+25uZ`VmFIQ z2=5x}r<{=}P;mPSZcgAp3qN~=6fr%K=%QJ93!1?yd1rrvp^-80&hY{rnr5)61>1YT zJNEf)50NtZZlNcpB_v~yjpX#bwk**BHI<{;Qc|jV&3zNx^rTeK`@1MyzM$4L{?C2f z)4u(yxW&|sr|q!zJC^Jo4ynrYu8XRvl)W_Q_B`jA(T?@{jUTmqm9cXL4*&;`l=<&( zWpIoh?a+LFM}wXRsw7H5Tf0hdaFeImJr;GcQ4wQUWe|YMjVKz4bkE zUA_&vaLo!Z9O@>kbJneX@4alL!-I+zAbl+gG!MFq5IIc{4?2IFRWI8?0JL-^MVN=eksE*>Q<}32kLEimMeXX(rpHL z;fJC^9*Ajm`lwKRWiNlAwUohs0=4EK7(L&)h@M~~U2fmb->lRUmp>D*qDzv8h)i)- zyI9ZTo8&LwcwmsU@k9eeLVUwa{Vl~GIr6q};wH}-QDYQT6^Pk{1T8v2UrEjYHGKq6 z4ERM?j~M2tKl$Xsv^tGO-+waP-P^5j!^1JaSOmjA@k z5tKI}4)jQ|ZY3Y|ZgSOWu}+Qvxnhs>JI6Xx9q+%RREY$6Jg1Evus%F4depHU|xGkX8{^btho!{u1*rk>(_~(xJRlm67T^AuU&ZQ)_Su<~eIfeIs zZk0=={jQ4nJ`dCCDEi%D`y%5s(l-}lG5`3{fTW9FMhx->V1^WWX$Rd|hQ0FM>P<~_ z`eX$ZAaj3ej6=Agg+51{epH3>bdni`;jQ$Jnb<6DjkgIb1OL5pum0YiU6&3B1oMk0 zDtnQQ3>6-DN>l-NjG1W?4ZIPFxGdq~?1uQRL+>;8{em7YxGg*{Wx*{-w!>=;J}yeJ zgo7zqVB(F%u&O`o%ZZ>}U)&&kb@dJ>MRGp8Ho|&G1Wk&Lo*3d*j7+^n>^rFUG%65( zaLuk*KKBYcDplXqYT9xfQ7KTL@|%c0s

      7e;Nz!rPjT7o_Sce#0&0pvwiK1a~a> zG`o~8Krnm#r{q>*eXVFcd50uQZnn4=&ZSqAu>D-;68I#oF8;ei$NZB2%&pMg#Z6t9 z`ekJ}9-RN*riq`Iw=!~4SW&F+dLOqRSLtEBg?!EA-`MUbgs!2#R``OgW{bP-pNqTl z1@z!SQ5MDh9@8VBaF1$}0)%f;o6*%??LU*jev?nLWe@AVH&fq(|__g-1qil)>5 zPBoP%C+_X^{Rgm~Tj48B)BQNIT>$+63GeKx`iaiby;^TlmJE_SIqq7eGAFPl1**W47spKtudirzh*Ng0wmd)~MwH%awErmdpl40s&i1!yr`8E* zcD9xO$))GG$XM^k#n>oL7Y-T=R0~VZns*;#F9b%|)BMt2- z8t2NoDm(h2pgXghv!%JKKqt!c!?)m*w_Y3RUGNDDL5uC%jY|x(O zr?iGo|9Cmk%4%=-KbDWw-L$W{GiWqqK=P}Pi_Ot6nPWY>Pstl=0xm2~!-8=W(rrD- zfTg17e(m9V&-n4BqpG<+wnv(#w)5IxmB$l#3M_~3MHW?5UnsMCdUgt#RSVj0KZILZ ztfqgj&api)@)K=!*#$b&TQYR#vHj7z^Yi-2 zo2~glYRY*VxcQ&DHlQ>BP@jH6?#{WAVq)@PIb@fru&>ghs4(WAL|fa!B|gP&1}a~bo-Y;_Y61NVvE4vSqJ*Fp*~)kgFVc>0BsbJlLLhz!iBRi7=8 zQ0HxAjo_xS*?awESEF}JjQ;X**b&3C$Ue)De z9A^;)6i5fDw(MbiT@E#)@A$aK%yKeQqrfW_9!Z9+YKxD5eA5iUv<7pV4cB7aWYx9< zz5LnCa;cp7<}E5Bmq<;W3Xgmf6;IP{axXkrqu1L!jze9wC_s}jGys*PX7fE8*4Vzg z%5wx76ZFpfme8;rl%5KOR=gpGqPDtYqd82c!4xV3HXqHAWr#~IkdDEPNm}=)1NQB#)LEec zs6{YnhZ<1TbN*4jSyAIMr_)s3TeIN0sf6BOTrB9v)^FUc_bltI_aA&Nbn$sntQ8~R zll6f;6Hk7xz~)>++y_*yqa<%$3%rFt#X6{c}fy>@QM z*E0d*r%{<_mn#R)CifoAo%P+D{Ls>4+*b@sC_`;+wRLOo>nMO}Ui{q0PJ0%(0m91( zdICMldkCj-M>)IvEQP#q5A} zO3qfbP}I=&68SLz>pruzAx9idA2wtX1LxQ5q&Cw~fpXv$y_F53liL{`Zw^-}Mj#9; zRcn4NK_6}v2;!m7ACHdu^_&r^uZ~hP3rwKw{p&!;(sa(8T7N@}xCvwSCP^bZbm&aY?4p44nmXhySm3_cI9{;s zElmDfH`$p4JcvarK<odLqFn9Ekx6sLIf_~dAy#zFfjy%uq zTwp}b8CvmSrEoc`H`-wTmuhSjys%8*FhF|LRKe2(5$})Be4-=!NWXwc6$zJixAFHUcYv}8sE|090?y*poQEtBr{Jt08MF(-aB~Tkaxle@M;D?0H`{(4;>J1a^-&zw43qjk!;5= z2_y>+AmvYxGNfGnjzV}g(`k8LIPQXuF>Vm2@|@xVBZcsLr2T??<7lDg#q$*2Nb@(o z_6m0}oG(Q2mxjn~v-H3|eNei75~k^`Ow>j4w28d{oVDAAYm#RCY>|g7=cMaO&k3oZ zo5Ivo;t+B$E{b085jlul`bZb9RV70xqo6_O+}5H)$jCJ$(P)w)*h!4IXy;LhNSylS z?mqA+rL`LG!0A6Ld~rwif;>0=72bsXc)7K^Qpy0Zg6f zko8%p-VQE&b-jEukK6oFXmy@y{8my$zqIKz+#oughoc$T9wJc%{Ho!oPW}YIn>xzxAx6+6twibRwyF%R&?y7A~00k&d_GbXo=n z4zPNjuUONuQnmt>q0zpoh8vZ{rX3zdQm?3eo3HX4#F6r4Y~h<=gz8-AzG1jvdaZz(o8Fhru=dp$h1Lw_kl z==cwIPIULWyXi}pbCz&_@a8n;m4<%v1gy=j-8V)7+jGVPfkwDt7e z5x#~;f1}Yc(Tp%djmTOl(MF*)SM5pAhK8{MdC6&CHw7<)Rwt(x3JdJAS#0b7<>uG^c}nB9H$e^@I%cK`qY literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-partition-assignment.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/flink-partition-assignment.png new file mode 100644 index 0000000000000000000000000000000000000000..e607e216331b04122f6fd197b0bbd85e2cb4a45b GIT binary patch literal 66575 zcmeFYXE+<~`#wy4ny#o$Vpl0@2en6AMU_&eRi%QMwW+rTWYZFTyyEN5wGXy`TW z-+N3$L)S(_bK>i10QEl*Pv#mLnjjjDdw2A`Ojfc^Ci0+?%JlLsN>odlTb&cKsu8lX zvYNkWXbYhh zZpxz2Y>FlYMZF~~4t@Q+s#iN415j5?%W{3n`zLmIF9fdt*`jEEtCx$PlY5iN#rg2( zKK1#5c22(4`Lgo)>YSi5xY2Xmy0g$@eF9c9a7skp_i*1~m9SBqjAW&Du7HA2@e(|gJVvbpVa=_Pb4K}PXH<~%=>BQ_b8R}p86yDRdH_*4-iK@fT8wnqK zYEtyIBHbTUzS7NF_p0}Q8MeNk5|H=j*tkQPqCVyZ7-+hwV;8#`8(ZMwc}Z@OOKb;r zL5L0cmqik7TLRsR?Z=k`R8Zx8^8(|SKU=*?D`(i*!VqQRL?pnETh=yAW(H%aVP=9e+sP<)2w%W6H&%pTA&8gJ0cJ#o2vXExVC`KG|-tw*{o7^kR@ zo}0vE4x4r8=yq#O%|kC$T_IzAJf)?`L`8HyB??J8i@N&57kp~xQ!=@>sF(a9Uytvq z;F`^1`Ko`Q{bIfif>6!!v);%@%7=VgH;XW=ddnhpGjrJ>YKbjIj;>i-Q`u=TDR)U@ zrTa~8J?*@NC1rb13id`BqIKXm7^eCIYA*&!$Lyx&3TUQ5t~^=sGf<$YXku#npdvRh z%O0X1G3nxdp7xN6{D)Zh{s3_PBIKDqD))$T^CQt{WJe7M87(nRl|I!ck$WEY9wYN5qL6MB=3(@qUs({XZ$9H z!uxjJ&tPTNeU1aOAN>a?zMM#M5!g?K2RobUJ~5x|RVw+83vjH7nbo^!HGt1ky>;p` z>9*>=r|>Z*_hTZoEty|Hf`m;bKh;N>-3P0MxHfk)8k*l1T~{@XzOKeH8>0c0=LJd$ zew?k2$u5;qY83XwypDZ zPc>%qXYR}9q71s|K?_z==&t=@XL7&`eSlrHMpvX0W73E2+h8FCRLM}keDz0>S;c)O zOSqG=HHQSoX+5#1ZHw4QY9XdM80iDD8oy83t%qqw_RhU{qEB$Yr;F#}O`y^(R(rc0{dDqzjZ;uPwZwC%*LnzEnd>_=6Oiu!)~l&V%R}Kht=LX;w%nK338(k*z8M0B z%a#^tH3d5TX@CW%OIEbHwjApHU4h?e0xu{^^D-vMW(+zM^7XNvGd(TX5-8O<8XdX* znNI8BgrH0&07P2vAG@l${oDxH3*0!VYWZ|=Ix6P5DDRaYc1?@Y!v09TPv+-${XQk{ zh<@BeGXo^roSx-`M@7AyYOukYQXzRp0g|g{eks*_#}IQJkh(s_I3{i%CnZtfZd&B9 z=Y-avE)Mf%^^@A#`BvfTJb2t7dL@a7EJ?LcQ^e6gFL@(luZHgx6-l=tiNPHDbxxxJ zjsUObt2QBH7V;|Zp&;#t4Dx6_{qDoy0i4E0>m|++p!1R8=lIoCNMxNioz}YY6no2H z*wqBiAJ=|g@y;tDy-JD>bY|}Wqw^8nT0$Ssx@b)-@7czPC4f$f(J3I06#$oeGS#5( z0ABoIw}XykvD`W~54%ZYiKO*YV{>agQYu8(ksE`vvI$_T4kTaD>ku`Oi*-u~lz zQjj0Ws6#9UDnxb}tosZyMRE_+p7+VU*=I*X7kzWv*hnAGxFvrb?IkYhd6&m0bWGdH z5y%@uId`b_rIDyQJ|~X2Gco#;zJfU*x{yH3#nBuBa=EcsFwzu|I=uo0 zVy&7m+&<`j8ffRujteU~S$id8ySJN4m`#@_rB(uA)H+}_m3fycS;#_6lVC9$YaxU>rOFPaPT3ba!T2{U%}u z*vs7^rN&tVG57x7jAeSUjyofQ0;C|_4-)0sXH=)44`WLsdtZIy-Zj@1uB-9Xr5B!; z!O=)?euom_QFZjSv;~E6Rh6{lSKrpoAA{SJZ+`SHyKHBnhS`mWJ_ff7eeX0j?%j|k zXEG;MIOyVOT6PleNJTei(3V9d;TGlS6#Q%GKQWuup|(&o33iFoEevb953!d0G*}8^ z5JCq8b|}{rkz4mT@$RWkWuku=jm2kJaQ3R_J!6BdQVXoJAtPhtSXGOE1povJV8iG} zo0(K>w?B*lq{_Cn$r2T$;FGD3wiJiw-bp8L%^(T=KrxayR~Rj_<=8LtYIsFXKk>qX z4&UmUm@Hj%r6r{_K%JG5Xx`e-Oz^eLhmu*)rL`s&KEhyRXXxh`Go%ng7{=O6Yc0{12;fzh%M#O;crK)yy=lkzvIEl}?h)C3;&!X( z$Lt37Nu_t~eMcJe@#D_u(kSFr-wuIQL>JAJo<8cGwJ=O(G(huk|2D89jY-Q;R4tM= zyR67DqtjDdc1S5_!MCJ`2v^;H;bU)UZFhRnGAOMWt%RhbRtE4oX4ZIvdL>f{(jE`= zP0@Sj0}Z!c{Ifiy9igE$Kx3pdh&4PlpFG_M^zL3URB1DG$ua-tY_=J*ek!l;?NUQZ zHmdWB=P4Y`W$vj(tSSU*NB^LaUaT3SpGVuARzHGuIb|NiDe`fDaA$0E`!%$?WEd9^ ziWT{L3qyNb=t+L{MT|AfQ{U`X9njDheP@||05vXGtQ~%)*wvYc53`!{}f-T1un|w3;5)Gq+y(O_%-Dbt2 zC4K0CQZ*T~;HuZ&qs>gRQ5U7DOmRD^H<8rZweQ9uef398I75`iy7C1&`>CFh$fd*g zalyLm#-e)@ds=Xjk4D#s$6L~#p}#LENSsmhqikd#|Hodz_GYt0#;9Iq;I3a&YK`+{ zwIH)v{j%AU-k|f(rH{6VxreJMUdk{_*VgAVZAa#b&zgHc2KumYv&3w8RPQThkjK2M ztJ0Q$heYDJD?h-cDUD29nR<)~7R@iTp69c0@be0fU+s9%*X$9o;}$K7sb8UT0OP8x zlW#r+kA=f;$%?H3ck_dTF+1x~FS{&Zh30kO)ga<@gL{ixraYmC5D`Jk!pP6bDh0f{ zf4lJ^HWxjhn^vj%Bku^C8yVIh7HW6TGHAV31k&*0A}SNJ&M%PHh3uifQ3wuob*e!O zohIGaWi~N4ml35Gn3(@^EzHZwXXs|g!oH%6KnLXts@D4lfSwjZ?csui&@=hcX@Dyo zN;*JqG;(!S@-0oE_rMN_EIP_T!L&(G!SfMaj8W>7Xc-awcm>GCCS^w1_cTa;^ zatf1hIxr>F$`H;(Iur7*q68&1>+4(XhUBsoV4N+oCH_fIO8Ik!RO%tsqW;JCzZznz9(b-4IC*uN*rm??+_7#a{nyv!c^;ZKV(N@|v4wAt-s`2GkB-;! zt=mbyA~MB(OCL2mW~$C*PVpE+E`v8Slw3njm?llXq?Y-1$l=ZhZSuN{7~i+oADCSD$1!dS2J zEGgW(3G=MlyFl`-1=igO7Tj#VyyIyUdFQyE&(frsxa7QYC?eN%wC@J@sB8V^gFz@9 zJBxjCn%Kc__$Vb5mVJ7EgeSqnIB@F<&ek$IT6;dlCHeuY1$OOrh?bqruY&fdT+hrA zBR053-Df4JO|HKZf`Ld;SJ z;%_U~7B!#@k#(F7?VYR}eQ#{K)+ndg4$_J}Y)pE}&tzmu-hA)Ba~j~B*2Q=R>oH|M z$8{38AdmF70il93_kQ9hH(X~gF@ho_+;etw~fsx0e zD?UF(h+*6X`&G}s%S9dLzcuIBaPSfTKr`kX%EZ6XvUh(zT>|ucr(DJ{!gcj(iYQa6 z^_uJKEdvI{=FS1Ive&|zYf8VAhz`yZzL}pKY+|{X-!9A9I^sB8I~2>T7jV48+CEES zkgv#em{02yb*wULR9jXI-)$-#sdg$al@?WQUFtPsJ+!_5&H#%Dix_#>us5-hI`UW2 zpXTcP`Zg8exT*5a8eus*ZxB0dL-dFy~!QZodA+5x;~UP!y|49-Rk zDjWy-%cWZuFn2gx??73YXQILRVM&_?2>tB~C_t(>kXy=30Nhr{S>cIzw(z0$WU*LF zInMQk_n~ZI5^A+lQwE+fAair2g9hTe>5?@<-%SJYZXe7Ukq)5nm9ugByz!2vwE_!c zQIA%^Lt2+PB=jc+y0pB7m`(qJ4M^@1Y3L!9=BidKoXnRV0g9=FJ8&vifj*DYa04Y; zy0?Cc)P89r~q= zkTSsA3Tu`@e0=|7C!)~G5K!;o&ZIQMD2)Wdvuw=zT=CtZKA)fGvB^=Q4YyP4`(8gm z_gQ*L01{0G7rSFuCjwKZ`-N_Ax9z<&;}->r!FCVy;0zs6^(KK?t_ihI84ee+@cs+7 zR=4K;>n8=+GO|r~eCdvQpu1tCqKfkyn_)UiqbQo{7?YQ4m(gw#ZD>&|c}wpBvNs_! zPom~I;Mu>351=AGuOqRe8G;t7SFBPk(@NFU5-mv1K~I?Qa*QnTA!AzXBf9Sc0Ux#y?Tx=y2rxZHhMFzY3+)mS*Mb@?A$qQ zN}mYQrg4y^IJB{KiyW~3{bjJQ55jO-RtXp3yUsC?V20kBbB*GugT|(z&<{ox3Juby zv32V~>~&{g>9AmVZax3Wo2VquY)>_UY_Sn(e~(mjnUvEsNw$3u`9GRTw3Jy89P2h> z(In{=$E2kDFYMP)VSkX9`DYH(e>6MF3weeXWmkt&<1y#26~*H_xo;WGz0tY!r`Q21kH<3O4^j|;)ug?^y62 z4i{s#EBkb}=nF^Px(L41Alpb=dGgit{`Rbn=#=EhkuG9V*TC zCyqzLP_2IJp_}<(PUqo)#8swVmndPj&F`GE)k`Dmt@>*<{RMahw|-rbPhU^qHPly( z$ndG=l*Ej0Grlgee^^K7yZZtelaeEB)u$2`>UY&sBV?@+rAK`_Q|B=TFI>hpH3?ZD z4$4B7b=ai+HGw@*Wgk1Qqaja(D8!yM+9_b0?7A~%wV3F;O~m$bYWc}ga}DR{5gbqi zPw+hhG}Lo6?^n6TJW1B6+cm(}1M~g6ofO?X#X15h9nu{n>$MhRm@}-oF&Js59l8cZAE(Nmi!p7^sYuD|tcpA+76nC%hC}Y<2_oXSCNv4#w({ zkI75&qJY`s&KCwL>;9`bu1N2Bf$~p3=bRk__vUJy2PpeZyL}nuGEJG5^J#Y{I(K?T z5d49M(|uXcn z>3<}f`o+#CS(X#qO>wvHb&JT{(D+-8rqpV5|Fjnk?PV@NC;>a(Z$_IvvkbND?P4;l zFb-I}@!ITtVK}Og)6KMT`Ml8^u7Pqz7hkci42O-zuTef5Ch{kVsb&ShP$XWHkeJ~Y zu)3k7UuR(K(c{|TKEkh~xD@0Xygz?ltqm0-CZ{QayzixhzMxW4KjGD)tpl33`f!8s z49YD07SXh1%yTpDFsZn_*>AM<+oom;Q>N=Zu|^|@Dig7;R61Cv`J;uKnT{)?7j~7& zUgnM418E8=`s1mP!E?tCn^9QD zr`fJ^6mNVRkg+Zb@nnWgol#=6Fb3o_9`+xZbqvF-_?pTZ3>~ zDX-{Gp>^GwA5iOnkoUZ*7iUa@0V!jygM^*01i2PV*Q+w@K8 z(VD#D)o5P$hZ@oP@0YUt^^f|ay8(M8vkYfi5W^X&d9Dje@}B5RF2a=ycXQN7c2K{F9BGsxSc*)e2{&cb4~|JJ7X_Y4zR zpjZqb@Mvzk0-miBE!h%9^!}~^kcsK(V+ii1L3Ow-(&iI8vJ*1cGWM$#3U%iT`Y>nV zwp*reUMJn6iyBwr)2?qRn_qImEGSF$#9%tXZ@+}M0vc&wnB+#oAf%ay_L8p_71_1&%sSRl90 z(t>Tl2rFIl=2^{MZw8p*P)Mw81SSmdrXdsP%#vV`Mazqs5s&6K1#SFb%D#kjigoCS zKZFJ`19b9KU(lghbtcF5$sDbuj}`QE=tp*fDk`VfZRx?1uf<;e9FNkWZw#m9RUZqZ zou{BoUgc`!i6RtNIrAj-Qz>?8PG%h(CnPYEY8;JSH9#=IvZ={HZI+%s>kc<)gN74cm8 z>lCvw38A&RZ`TZ;t_GQEJ@Qiu zxL^7=;{bdmRu0o6ij>~mI!b__R!fC4bkZcO`yt3J7Xjaqt&<)T;`SXhsQEa+Mb|ft zYeL!G3p8pnnf~ouT&F->8(BNyG{A~&fYA}%tv29s9YywXqn5G4eBhN3yF*Z=DPZrN zV(V8~M8m~1)dMyRPLXbrB)9);$h(=7^kP@t=Aqprqcnqv{W+IFdNDWm+1}l$VdXF*J4+ zs(U>e7Y;w~@qGv!D^P&@GXIo|BcStQXIaj> z1o-RS`%TxvKga1yHAU>#>K6oV4C_VEcJr76(n7xUqV)$1^}qNSAKC@U#+FLO=voX4 z&q42iOnpWc8qKd71G#wt&<8*$N}F}P+$v61Q=fIQJDr^q zyEfloV-khfB9@#fwvtvVD9YwrplzpSBxz zP&A}ewr+jbsTVLHDcd7Th)U_{Ra3QQ@`x~;^-g6tW7K2tTpHUosrd?t`s8CAjUJ4` zrKTswBrDZ6hdQt;XB?20YLjd>eFh&59NAF*EL1Rk6-a(Zm?>R^!F zJ%gffqicVk=Wij4{VL@4imG;@bBCrheyXXBbz29HP5Xe)-(7uKU_MoFZ22g()7$`P zHxG8Z`Jo7_!dlC7GtTw@Tk zqx5A?hLDc^!Rq@fbqA8?T+QB%p-K^{Ixki7!6xX+Ie!@^4(wYS*d55Lab1602+mc`vL;+$3pO5aN! z4?tP8W=vkV{&4&pTKQh6m~*}1NB1WLt-_UvrUO0d zv34lvnqy{bvnYjcj$LuEM1Nqyb;r!dn_tHmnui$8*h8bk9Cuks%PW`qTqaXAlD6B?DcNJ*$jz66$s-dx`5SxrPD&7*A0?$@c4Uh3d~YIfBa9&d z>i4u|f4Ug&!&=Jf2$jY(yQL6}#cyU0URPrMBfmiGJjyF7rI~=#$nR&IKH)5Ima(x(B7-Bgp&$&b>%S>Ri)W1_6@vzesccakw(SwG$)=#23Xw*IkaCo05b@+xNor%)Ii4oh=7 z-dlaYa{ubz6H@t*3D}DbbGpf7WIm{bnc7?Hn7O{IyDmAC18A-IkksGE(TdT;aGQ32&Vm1|0 zWy_G;ksBNk-;`BG&pOMG712^F0Js0mI zW>n(PL**A@%m}}O0P~=I<)4&b>fEAobt_vp8CnU)H9Jw}vgQ=DP}x5E2o7aQg0Y)N z^q$RyzlSy18GhMywz+TM0SogNVdObi@qywVW>c0&xPyh-w}OS)o{w!5B6i*UKOObW zcDRT(Z#FBfYp#ow-QlnaEFd93~OZ0Gq(jj=ePf5A!ZR1br%^q_o2 z=VQ$`k%vqkSgtL5GvhIJ(aS1h3yP)<_+}Euh~w29*=%YKxWV>Ipes36KDsG4pUB;3 zXq|COe`8EtXUV=A^PIzZ51C_Ld-R|$shCRqN2Q6gBX=2CP$mSDUDz(prkcL@MzV3U z#QOo*?J4nTnCs_ICjR+vs>B3bjG^TpbH1Jx|MOIC2C`aG6nG z6?Hv%$}c^!MdEVe5Ba<|F5P-*XIH#U^5a)u%peuc9>gNb-=aI&?R1rR$Wu%DYy|Awyrw`H4I>`N`1> zQ>R+@o+7xuf;@d}cjaFpOrL$3)hsWbxv^`5=CX#%OjkIIY;Cw|1&X-OxUouDX(0f? zaUy>ac&XE~rxv~pQp}VLcs6@-gssdnjLq#4A7p=EOj?!>y&b+RJaUaivBBya=!;TW z2x$DpE>5J7h>vz zKn3OKUU_ifUh(eTE;wrI!DxW;KXTc0Yz)D%^kQm&%b$%kvZ1fSCw= z^=1cxw{#B;Lpo2koI^`T^5=WwZa(jdR`gs@6An#7QvNU}^KjduY2qAEtRQ)RPJsXd znVIu z=%RDz9-svJ0*m2j*dSH)$Sc9+h=CoHVaLOj%oWz2>wJFk7Q7KXQOkohL$PNRkL&2U z?RirM!5+tVq95eYzClsBXEHQ=nQ7$VX7q&I)@X>_p;*aKY*wAbdL|{p%dwI31UrT{ zHtaUlH8-B~u`OV6sKTq|1jA6EpOd?j`PFco6AS&Sw=_{%0}51Zi(Vies#opV`^Eoa zqUNvO9#xD)4Ey;=6v<>PMC6-WSr&bjU`sLq`(V0-Tkef=P)GNt6&bz=gy_$K0<2M= z=c;5Pw;RB73AkA9SY=6}%)4lPaz719Wf1@kTD%oYFEZ02wx>W9jZuIs4{{Hl5$njv zA@R@f0H4&%eSp_j9^&^i344v(BTESh5*Tqel+5ECtNXlDO!SO%CpuKdtN|cyyD3id z)8c^CizkV`x`T!4a1=cbmBKM7pM?t@WH8&-w=v4F%|%3*PCS|i>rr(r zc_6u%o|K^*%(h)wxr z(rJTd-usV&;a$d}JL@&5ShH}jVmq1vx?n@qCFezw*zf5GV2iGGlL8==0nHrq`(05u zQ`nI1z8ode?Jhnh6-vj(8cKcaqjiheP3RL_8>K;!G?`!908)3WND&5GRALwM3#p>- zx82{%w9uA8D0bvA*J;esyK(N5TWAZWyO7GA_fJLtQyw)Hm6PYoS);x{n`a$GImx0S zrkMJ3l*Ttt>=9k`8G^*SdRe9O^Em2w-OB9jXPt$M0Ru6o~v~U ztdg6VisWeL(%j0i`vSsJ?+#YqX>P^PYw4)(bjQ|g~RwH+T2I{S_A%}AUfHc zSQlriiNCF}-(1p!PvAg*ZRUY&RD-FO-(Cg*h7{*_0TRfq2u4^04)i3pm zqY_7xch5^pg=0h)i@NT|meQ4bAo0(5nfgLRxgwjEsCNUAk=1?J=(bbfrkdDr;&h{Q z+uU_oTWTg~wpnf+1tA40=+Vcaxnr5?%taLF5x*uqEhWK2Pj9FbAOSm9T7@ScMPa(} z;lJd9+8Js`&YDpTUD)oj5$B70AU%Jgqs{m}<6|?i6C&!j1J~hQlT83AHF{&g)|_ovq29dKLEsF@NU;)j z6;0^*VqCmuDx1K{oZLBaEXPd`UByT
    1. M-Ud^e&96-IPVA|=9Ed*xDssXyXHwgYLcB)TzYskXKNqC zT!8ld)`z|opNSuT&;cxG4X0HDb`U5*I2)FH8EM>53Tw|y^`BWnxbEG}4%|ci*9-k^ zAhD8|$jsHe>u+4%OGjt(&U1W&ezpS@poGnih+1{IH#=59ubD9Qcn^+LAevqf_pRyk z2|>05D2apUr#A2ZI2H=ldb^JhinRcXN5iuf1U!kQW@4Y4F;Y~Q5T=ZBkWO#Airdkan;$^yX7842RTku&IV-^jQgL%^Duj8wYGLC+Vvp$`N z@t$+HVq&c<8j<2tDMvqt9~{-5oaFCggOI%y?Q3@4v14W}!updiK8q77+W9r|$7v&By9PS`zf$V4-Mc2{qZ9+k-XG zLiRepf9Cy}Zn&j%M=JvbUz*3K7aLk-ZMR06SrnvMlQFrxA|D#l9X~Bn zN7|d~OYeKLVc`M8e-7s6$F>X~y4Mx{2UP z`+u4RzheJyDe`RI)wsT$_^Xru*VzACikG}|c;b9|f3>Lp8vB1qktd@0QT%;H{4at3 zUt|BXf&Y(H zg-jjQHZK#I`i4XrauwFL&h^@@c@<^uwM_BiVS4LF8w~gneCr-+?im~9O|^&!V*kK1!9VW~)f4>HFCJ>o;pi}dEegx=c6YT+ z0%jHQf>Zmpj~(AHba@7XHZN{}N^Yg}1)e3SHjW?f1Tzy1L#_+U7HmCS4%F_Ak$mBfrNd_+j(Epni^AA^ z$M^C53<`R_ErHv|h)_zaXW%BmV(L53UjyZUJD_QxX$H2z{=(~rFx~)1=D8{Ht-1MUi}E36^MVBL)mxGo~=IB#OGpp}$_nmncjkPs*gT zf$aLxx$$7R^ne5A-dRj@(T=)}%&V`1e0*RZ|KmJ}doPs3D%&zyi4gK39Ijz$wx47m zNPsv`yN2}Fi4?NS*EN&ZPzQ-zY4d4L!KU&4T6wCg7}JLPGmAG^1Ux1jrd-iAL%dV# z@qI!NC30_5j~z_!h`;R9u$=#lvHiqjyk2?BNa2J(ra3tizW5~pvzC}w8LD9H+0oyt z%4n2p>=Bw|l+@C?>?Y*uo9#{DyvnpGSBvqL(J)`i$8zpJXzYC4$1{aZ1w(rk85GR# zrJXMeUUq6$Mac9NwlF9OfW{hD*x>=SnZDWk2bm0brGwShotyPzQu4`eXjy~C#i!U5 z>Hy3j+haQAV1*!U-t}Zeh(Vsb#gjdlyJIb8%tPF|6vKFm!~L=BF;Ie6w#SKiFs+!{cK~swb8A@|wD{g4RwTfk&M%s&A#fAyfq0;zOy^QfGf^&T5cTIT9 zQNQbtTgoYO+~ej=S@CMPxa;JUaNn*c!}oXmNvJ>prWbz3U*%Zs9}kGlYUSjJiI9f* z5HKHJdMWbupLM9#vE~s>cM9RKBOC8$!RpTcE0m^eubW4VkORDAx9FrFM7CdY=%3d1 z+JJs{CB%1S8K)&d@@~LO{X=}ZWnZ_W)>l?EVPB=8|OXZy0%y&BGPZj7LGOe=HUU{D8XNc5BV5vTa4?%uC-ck^FUAe=)s7YU<$Dx2|CL%(LP zyy>Egb2^*FK$qfm@j7Y3Z1Y66D$r;Tdiz^IZU_`VMZ~J!bDFijtNz{Yi%o zsCbP$s+S?Ni@X`1#Z%xo)0F8_kk)Z4XdP~M9`FYDGiz7wTIM<4>azNmGO7cHP&2Bj zqu)gxTQT+${rcEF&c{r;q|wF7ICzV^GW%BV_#5+^vtnapJP;8Xj`qDjG&hIiQZSGL)F(5mY5U zn9&QrC|nA<*NU*~B`CGJ$Z$S)I3`cDgDz;k$h$!=s6SugnF0f`q-P`cToA6t@{9h} zN7qyKvnl|$foImuy5GzS*RGs3TZ#_E4Dt+oaEi3J5mA@{I#BaQDz`Ey?$kQHvHS9^ zuQYXqRRnDBjhfs!<3cj(hd2dMf4iX$@!?Y-$Y}(KXAbw;xAvKE&yOqTcUccoOCo`) zRLd?$>!;h!r-e*!AI2S*NhFT0vx-bQuV3bJWO>ohJ;MCc0a;83f$xozgF4dpNA zJTpROn~~qr_$3S-KkW>Jsgy+Xd{}On@y)A*ee$1pE1#K5q`KHTz!AqLH6J>3&BZ#@ zd@@_X#qMlQ|E0zpl_B^{`tvRVxQHj`@x%Rw-JU4iH_;U&WFcZ!R=Er^;YQiS?*NN?Z_%=Dmmu`f>>^9n<_h9nnO+`q<6vqZ#2tE=P`PI)QMvA6UR zJz_g{8VZA#1+l5c^HWn%Tw9W4m(l{41k$QikV9k?v#A~MhLjJyDXrwgLqA3lb=O~2 zR4l>nhwkZ7x;)bPupXB)6n7hSva!SFChWXZb5|V~->&k}p7tcVbAiuwnDa7SA9$v5 z?)$k>sm5$8InD62WzJTjrfNGSwQGPZLg(G{$#W-wA&5}@uwoqimtT0cNgZF z>`WQ(b*_AOeSkr@)EC4QeS0GP?ul>a%kaux7Q0<=mIs(+A@t4Myn0e2Gtq;Qd&Vo( zfWkuc+Tc-qm^652CigNiKmPVQ3;aXLH*p(F_3FRHSMyTW;~nXAL!ic{hro0-?rCHURZL#3$q! z=$~$JdG|TAo0~(TGo3V(s$hMeks#N|xfds*gq67i3;u;Zrgv^kNLy`Py80>>b1QyX zP+42D20Y;v9Ljhm>5gFE_p=4*d(f{kJ%`=cFPu$M^!mUqLd#;$c}(DTinD(mEa{ol z+!$j`RZfwWmd$q_Mm5yiu(??qlcU~F*`0@RCDhqQ1Wvd2+s~QZ?Qf2U?3Z9&VE5Nk zkTN}~==6yiS4p3Wn3K4f5xq)t--%@{ZhGtiX>y3WBNwJ%WusJ10;EovE?s1TxEXml z(hFO4H(Cwjm%j@P_&@dmp3~5B&kcD~B_y@9>wY;#x*6t9R}^T*1O6(x_J|P4nhoYU z0vte}%~~`+`YPU6exGSNrqz`_00palX zUWip3)%|eS+PdT}gbP_uiOY@7HIF=l1zojlY~LNIlMB297q?uM&5BCwvN^0WG4@3u z_H@0ptbx>+L6s&6?$^w_d+rFTx73y?fp0}&Y#(XdHT0=CLYj!vX{ic1dROOsNM*nd zL+G?Dqlf4CAjck$>ry<%2=6wsabKT>3a_b3RI;!giXjzH6$vXqChSg>?;DFM&Oo$S zJ5GMVlv&DL}}TaMlR z3JsZ_5A(}WWn{p5O*JINPFTqoz$onI-A5HMM^TiNg`^x*%tfGaG#GT$YM!`g8>KEE z{nQ1QwF2UZPhGml&87S@z*jVlwtJ(yxK}TSaBvB7nfgPD52gnbtL65Nn-PRJgyftP%TvoJ6j0vXmOKgCQhc$Oh)FFN-K zbe%Pp_<)@-_ zXeSZHE=fRq{dYy%cR9gv=kKsK{jP;V7E@4Zmbu@) zn!9;8zvU@8KRSsXWhCcoC#H)TDpD8dCO9jvohzR;zt{DRk!Hpmwei>|tduWx`c}=v zAS(#oP$AnHaPfwZ;urND?7}~=Uf0jFD0aQ}uV9h!EfESi_9I_x%eYWQ(XK7;r$a9)ZGG84yZrsSQr^Ez z^xOkO))^dU8f~l-#kwl={;Ds#Bb_my&;P~Vdqy?2b#23NIHFQSqy?m?Ac)ceNGGw; zq)8R&5W4ixyN#C61SQlcy(oy3&_fR$=^cbfml}G2d|S^s_j5n*JI43(8{dzU9}EUz z@4fb1bFI1NwXS)sSL+(I;?6pWIh;*Yb?kigBO2=Sgue%IJnK;P-4MSH8j-YDL0sI{ zKy5yEBbk4{LCf8Noa^Qztkr#3?4LVPvo?MUKE35>r0c+xbw5B6n6hMns5g&^_!|E_++6;Wk=+15|0t|a zRnIM5vjvte97uOS1mK}#mDkq>`ab*ElII*GJ~jFzL)!Cb@4sW)A#Db3WVUw*u^H3< zLVr_~!g$oD@qwM%-kcjS+0DNOT99vR$jw{-W^}*}2WMun8G0s+mP~G~x+JA?89}QB zNiSTs&S`XZv!Ikp$5=G+SUz5tCV)P_6JTCE0C>_@-h&*K6IiB6AYwHYDh*zY;4RE+ zdkD81j#x5wyU!WP)A!dFM?M3KkkB<6={+Vd2Fh6J(i5_03q{GAHOAWSso6K32uw#> z^Hp~pQJ*aKZ3X>RZ%i>q%dWu0~u&)O<4-a*__Z?Z5lL@Z75z)@Y~9 z%b)lrW+Ux+5I&iIg$b)WH(UUf1 zMA1#{c|s!@fsI(NhLo8zbih0-<$v(wo^dq)J;y$$kdFIBffdosoafnRmJD4+oBE{$ z?cL{V=l^7-1frM~(}rnO-48mfn-`slSi`nBb-eoc7b>~BXS&}5rA<|#UyWwyk6#&|<930m_x6{Ql6UJpWJJAodJNlF-xZ0# zcHxoEJVV7Snm3Z5a6k-0plHtPv+XU2+t}lU6D~J_t3}ZA2spw$i=@&Ndn1UEwW2ZO zRuI%oyl|&K!y0ay63RMlAYjGrN z4!;I(i*E~)3iD!KN{#5-zcz}dTbyWanH@BT+ktJMQH&J;`#&bOygeC2e(%}ZcKH!J z$Olc!zyZ{zdO~L={dU!CZaC%#=rjJP!meMat2n8|VGOq{FL(zKc?x&?DSF^F*I-{y zpDPuFkG_SWe$AUUZ>uJ0B#Vu*Vm3p{>GGzIIL-BsqPr$i$HzQ&Q0YTHrc#369It*N zUp#Ih!ikhvVP@WU&(lbaOXpKle~w&hHumwWJOlo~>ry53m7JmtUC3 z?oaLXP%#%f`caU37{6>q&mtL-|KYxogM7UNN6UfpH$vw=Kt`ov&*4eWjV6!Yba8C9 zeik={4Jtzrk`p+a9{^LzVY-1-xSxqhCTrBVF|~C1G$k-$dX`9qNi$ae-U%xtH=8k~|6)X2`!72culc_?PdPx~d6ufjv;97f0&PY{5s!;;>spgdy zV*}IXb75Ta5FdTP%*j+I9<7s`z%*xxyTLf6^Y;36fY+34bwI6%dScpl+d+4Lb-ptd zcKc}wxee+rFK>MkTsX-hVzvvx^>pTHX6LypuVU=yJ%fCn3K&MuTRotmn{+1%hvHCn zi*@5v`I!p8_df^KQGW*A^Pz=*0sH^phGU-Hid@(IgqI$xpb}@|4i$@!*FCGc;7ZcF zgNtESd@~Av@k5$L-tW*i&@m)G%WbMdP~?s8HMBJE!Heu0ZQlfG{(ycPiSNlf7rG!a z6$@{aqQ7H(cz^F;TKtV-!HDmeyiMHaTTTbkHYF6jg`<~O<9O#|7_30bma93xz$4>H zy9-$sq4X31`!r}v6caAvMRR8gxuWh&Regzh@?K{vW~AxX*C`5iS>ac5dW4xWrk32H)8_JYPNM!nW#PY(ae) zA)91%Yuc2E@*O%?d8w5WB-V5H?aD1ko9SvXd0YG77UNYezX^kw<_+r}5(^f0R<85v z5P6)V?DZB^9ii}X`PQ`q)IhTfyL87UhIQEIqmo#|1@6Z3a2YntLrWvM)25v_=l<-l zFDm~17mMB_s`LIfZUdl(X&I%ftOlcpu8idKHEL4Lz=pdl3lSTwf5T|jEn!7u+UdcI8M*XOM%a4t_N>P-9R!m)^4M;+s?v#vr zWXSA|lHp%$S#BrjUTuYWe=rA|%kZtS{gFfIR`g2a%Io~>n?->iqJaU-);b#>a|6pV&ZP;gqj@9^!S|#6` zG_qfPd_(A#4R(q-Qr0Jmfqu0*f=cG0LjLlWVLWYqjPRpyYMQozohHWOZb!?TE<(oo zp}rhth6JAv2jmhKR!JlfRdR)S&@aY&OFSL-5$A1SqVKLUz86- znr_QSRaNv)vTyR`R!?6AMBid+Nmz3x-EJyc4ed3&ccV?kg1STsYbBUR zsiC&O`iNFa{Qa|Mi=)T>?_qLy{D3R2v)kB((x2s4r9SBLp23VY2IPj+Dx@~c$RFh)2 z_dONGC0&Id95BJ$qL}9j(bF1-#tvUniu?;Yfm$B&KitW4JgDXqrUYd+!rW0`>vkw~ z)L3tupH#lhOo%aRztvn(eU5(B%2y-ZlQ?_-hfLL@O)8S4=R=Tst)xwhSpYrB1vz8R z^G9?jcFYpSBSLH(1)0|3Ej_ zVu}NN-JFAmlSUm4r%2Ua* zq+^PA>P3M;@?rTZ0cO7^*&!}1Xz6nv?5yXgyja7Jn^=<*=AC5Isz+f|){Z~8q#tv( zVm}V`jM)&lh<8j`Bxyz>+TbjYwruJ;fK9|EqNkxp*OI3@%>|n0Q?6a zRt=6Dn7uGwyBmg@4MMsaOJ>7)>(5=oe*S^K2X8VRbz-|RFTbMp!+UfSAN(rR*L^@z zU(;z`me{Q_VAe`Gwi&Nj^t+2}Z!ujpxAzGDn}_yM+cD%6V*ev@vzt!^V9|tXmyquR z(oFpI!oQaHmTb5GzWDD}{k|Nl$yPNl+x4DDv?22^a*bF5z&{H7oBLy*$3r0})9StW zd$aQ)KkN5?zW>~=gDgg4I}O<80V@2TJf`&Vh};w~dMondLNoX-oqRIeUu%|!{EOwo z)6S}FRsXqXOOu^3H(u{;cAW69-Ot}`?H=Ulp1(J+^{a~L7pv~$6<=k@C6BX8@!H?l z0hLC#T6IO;2lNl$D~7cr993>aY5dDN`j>U|FDdL_QrN$oz5nkhl?tgm!S*h(Sq48^ zzlZ?Df)jXL8~0rUSR3-_IT%?s3;k8KIWd}Vaeq{~Lv>WyKFeOJIPOhtly0#FLE^8v z`oD#D{+6cs_iz6i;{V+v!YM_SR9(u&OR~>R)uR?73#Oo$-O2awom?>3Dub--HN#^I z)!^qVXhXl;Y?x6p0aUweF@MsRv|AXD)4W8M0BLBW20`I9L++uEwzh*)MH~Khaqt6!@;sUe zX@y~iZafDu0o4**wwpCM(nZ!cG0-OHWS}-fRx~rv8I(zp8BK25zZniP(_L~+pM*SN zzH>W%8hw4Zw5&`6kVAKLHggQIb+FHTKO*SIAGLnr{lMLvF9GmY^)Rl}fzI#?4 zh*~P0Bt~6S7&h`(0OzyAZPpC3^Y)<|-cmNg1@J@9GABzd+PHl{{TFBaW;LEoqwLK!xE~&eo)f0ShhGe?-;FzcnAk2-a~q z`B&$ueFIdPn-qrc9HJGhd3|di5I50A$EHN^!uS0Q(I=?pg-L4{XBXL06G{;`lBurZ z^8i&zrfGj~aBMB77({wTsNgGgIT}zvr`f_X4^J&p8%-3~6-OII7RUullX2bhJ`2(Ikl?$tPn{yIxiRdp7%mdiT={J zm4t<6Z_WzWjSJ{|J=02Z=maEHS}h0Ix2z)zB-7hM4~_rP!+{`CrVywct@;70~x{eyys(&rHqW*HKns!&D;ulUhDgW7i`~gGB>N6zgaO~ zBGfH_Ca^9mjr)MW*?I5Bx=0rnTPg9MLjy60g6r2qnTq+9C|Fw*4(!z?2hfq7El1KHfQ(q8_C4pM zo5m96z=`v*hG@fx&LjeXDEP^863xa5m~!_+Q;Q`avhhtLV&n38Yr0)ob{mP$-cwSQ zb%A#<)8_4B&Gk-)(m1e412#PU>Zefkar?;5+MQ|0XzrsotfpcOBmUjeQSj>Wx)@W7 ze>(i#w^C){8*me-XVbE!-~L8I(Q*tqrx~F3e)Fon+{aV)L3wGB?1Xhdtj;^K;Ag=4 z5(DA|>vv~FNGqy;wrSH9MZS*+u}B@S_biLD=$x`#9o&bs7+ahHOqrMFoPKHih$G+} z1Jv9yr{4-t8;CMe87F!&G=Kd^G*7qbb_XbhC9*TTE#v$i7SSoy252^$J<-LC?znG;KL^lJi+YbuM1d@1D zH|DbAKrO**ZMBobkp*dB2xSj(`QUOK(oApU5}S9pS7>&3;KDf#{ipdRbX$AN%Z za1hh-;6&eAXd-MYDRUkFmx%L&a4mAmE6wnfS^CC;<^&6Co6KG^*jGqII(&0B*m(sb zD0tZ^_dLB=+iEKugP|QFD;*HMbCaa1T2mP)08!}U$H9DD117KOd)UZgSVb#cgfIeZ zpU7>-_Hz}6$;_81#DNN?lNbAhGd6Q)%xm)KB`SDIwrymA@l^bz8AX6QQMzu25~E@% z<>MGqwFCy2e%A}&f>n@$_h@jPuJpL~qSl?IPZ(7Xz-(Vi^o&$~emB>lP-S8c@(=2@=x3adk`Du)NY>6N7)U(52mS z4vYa~Y$ZKSb<8C9)H6LFz8HN6(xX7CXhJhvdd@5E9YtB~XA7JEURSksA0=G~nD-%L{bfoaqTn4nA0JMKKHoBoqBb#}Y;d;K@Ny0(a9nSy@5 zXmKiki_E>!C(>CG@NT(})UR-Nb>d}D{y4%ZNxw}zR)apxm||MNKC+M1)?qQH6UuJ*3dUPX9>oU%iX$h`uL zg4&P!znU|B->pPC781PBxD}>}7VcD10Oa}Tnt0cWn0Zq?-nw?u6GMqbP-wRV0cjDs zl5;T;?ddocSkn_3xH=-~0$~Hh5eMCfMTLDAWgScI*K7b1dd@C78iDmB)R5j1Cs(ih z_P2~$A$k_y4@1iRXpLjD;1eD>@7Z#NqrBDZFy{*dniN4bvEhWdHA&8kM$TpW$UYd| zg#4%KChfU*bCOKNp?uP5u4Yr|f$3mS?S4TQ#%y!;^yl8zjF@+;)4bP=MJWq0=HhA3 zXmKco_pFdW^|xA!H`y%IPg4Vpe?FaYc0{1sUuD@)Q4FbY&B9fE@Qa1`efgzS<`Dy% zj*OnMZwGx4HcmNGcSeBYN>ZWFB`8;r<`l#eJ>5(`=}3%*NaUB_Efb(JfAPJ=292S`>9SXdd>tbJaZ#P;hdus<;ycq zaP@SV8Iwg)Vo2!OH@lXd>gq?vKVE2aD+J3#9N4#7GXs65w^m%GJ!ZS~xyc0*z*~Fx zowonPJ${L+K9HNO&SloOb}Uyz-e*w3qU6oub7=i>Eknk*wSS8GD6K8p)phEuE3x>s zSd47mSg>07LE=>kv)bb)0WqOHkS?hz@I$~;w8h?#uV%F_rtwX=4uIW=i-0gJb+@!q zzm4Pa)T@KS9!JvI%G@XV%QIL)+u`9HbsrT5?PfVuzyCV3=?bB5(%HZFJ~7nW1zRuG zhWcifY(vbd^NK2QGbQXT?ZJ%XTsV0EXB^;QWJyWe#r2udFldTf_{{g-X~LeZrrBAN z&X!zSsNHGigGon&V;dMQ`KwB*RR%e<)@8Pj;Eldv3Z*Y>((FU+cu4yxqfOA@1Fzcf zV#usg-|SVTntIaDHyOez?1}WZqZ^f37iCP@!vmf)78~ z+mLo!RV}LRbEpnxUGEP(t9fgr2RE-{5JihFX@n|R!Rp7 z-0veA4BM;~(=R5&M(pIo8z>OSm(&{4asuUJFSmKAtg)mYM_XLjKxmukZDR2zG#b4L z!c=xFATk<`Qjx%Qrp&YY~%pFyTx$Ms-$X zpoPPx+OTg)6S_=yS`eMHGetT}IcOprn6Z#z3P@r@1nfO|+?4s}cg_9he=_3{X9M-e zawzY!rz=V`Skg9;uBmS}NbU@s1M|Lu`%#*AHf!wPsA3VJzAudtbp9;7q3}p&1S_I)4WE!Kw4!)F1}{>eCck-q23xS)=Z+9yWKQ=l zI^=vF0xUJGJZ=d|$l7nFRnR;PrMcK8P~s~72U676x7&3QdghnrsmNqZec8^k#}Rt!&wN$Ue9U+dvH<$^Zb&wf9XZX$4h>MnQ zN$-ZA8uGR^lTOYu57Qa=c7yBg>N}G9p0u6q?!_4i(~g>XxCeZO_emR_KwExX;*jB~ zm@^=5R;UMjHgc@gN5LzSU|4Y7kXS~y`ujIaXxTH;l~KCs0Jm)!$}3rWwUA*Icn&&BGR=x_{N&q@Y>0C)%%O-?mHb#1*$@v zHp$KtUlx)J6?8BY+K8|!n>6lj`Aa;%DmGwrse^qHt!5A~7 zSM6kojD9M8ozIJwbF7}|RKR|VlXYQvZ)jfc^K*T~I+A$#q#t|w1`_-5vq%3(h z`K4|#1)Uy=O&nvq2HVK9pro#mJZ>tJ0n@wI?D>kW( z&N7qVzE}DE3&lL`xfwCqFF+tdUTSS!yw`LsYsgHf7JL~Q>b`w8T5M_`^C5eGt*G8o z>L@&LULrGZ*rQk4CM{ncUE&a;f@m{$SL1{?#zd@Gv4I;vE9kmCg8*bBpmX3Iuk}=o z1}>}PPt##R{K<#`zP6$9_7hTj`1pkeCCaI@fd3n?R!_!(QO&8VLXgwE!->3RvO%nP zG3p?6sIk~gSg8PRAJPR`s9yLwNX))R7>^(5mO!s4SZ zVto!GD##&w?WdL;F$7YQRmeE+dmSo|VKQhySL?Q-s$Z1V?X8n3yLm8a~?-wNuiUE{hllX7-lTNVg)Jc>Mp{NElXlQxTugB1e0Z>_v$P`kX;w*#W6 zCGuWvb3(H7x~`mCa*4SjHX+;bh@n-w6`5rTVr^9KYYpGr_W#L_>4UhU>2vPELcn!q z@;9|0y4Z+Js9vEjK!=3hIh~GuM9V9Z`HIn>r?voL4<*h@_FfCXn$Cf}O({U+EHQ z$db&J$f6^pW{nX*Fye|TUCU7&A7{l=3L%<=!$I>>*CGt0b zr%y``3syA*SdkAF&ie&IAP zQ^%RZm?iPar}qz^v@_xV-9ZF}FCw44(VmZ|+Q`I;C5C(CCk5PeSuzcW(`?h(2Qx>F zz4GmDsN9EmVbUxoH7F0O8KR?sLt?~XG>(S((h+zIFYpI2lmCQ$Q(NIDMx}QuE=6qL z2luBl>tyac5GoaJir-LlDh2K{Tr{qG zs0%u;q^)^kkK*mG3$@gKOSm!FK^89)*>B^oOCyr$2Lfpu=y0rzA*te{bBi@}376ZY0wKCogb{bPcm8w2F5u(=6#;F-Z$3EJ9u7$rBp;l&2#M(0MV%`S zf5_ZsXj{%SuSu0HGaq(lZvJaPKJH=nt|+8L@t;4T(<46mv{}2S=;r^3vN9fY-G&oF zJ-y)R8F`Qs96qFB5h7zB=etNXiuUoC;xwT^USHq~ELazY9)FW!NtABwCVdvV)tWlq z>^&vn9P^k;Tv0;a=G?+FTNEUDkU%#MW$Rka&;@4xAt@r?rVw%tNp)_?M%4cN>u6gy8m6vopSS@$YTJDKWFmYf6B| zZt%ThNkU%X%5gp&V0m7;x5@l8Pxwki!PJkz#a?$21pSAJk3_#Lg> zSUivQYkuc;X(PgagKd&mEd8o^(3 zvZCMQD?e0sdiE~y=$)e&*G}cD1G~|Z@88^21=(mU3Nd;Rug2|4BM!Ki3^;V z+Is>xJ(z#^lTX+7vuWvUFH^4(aBF~K_vPZvP=sa#K`P#@G1Xi-zxl(H^hIA$5=&+$ z!8}O0yp2aE>_w-#L{9c#-`xev*Cm6-%m0<&UcHKZy;JpNnKl35)1|ak*6jsMPz0@- z0DnJ9I>;v650YZ#{w&`&(MiI2v+MCOK-v zW`i6ZVK=ETb@h!HpstV|Pd7_a_kx);`vvp~~yf+_Pl}q1>-?=wOgh8 zNJnk2EO2s;v3pCS8&@Z|s2i3O?PMxksH=m!VLrwF-512qcPi7fR5!oeJVEVuJt#C{ z_(LHzoR&wsrV#Ob_@?Czx+{QNtXzsj^f=Q4#Iv}>LbTBQ4Y~w2{^LQxCNemlPxvWR zz2y-n&s`nR2j?pQp))zLkW2V~T^UE|^E0Bdog!N?^2`Gv-T+hj&joKOeY$@^FuyV0 zSENb1z_>fZBR=>jB>?bh7I%={)vw%_4#vz!Omkjan*#DRn#iAEelx7p7=Bg zlg_SYAOCaryqRCEV-obp$J)lXqeGeU6x^EBXeWN0uaTz9iP? zW;f=0t%9#)_}{OHw5N}0u{MQB>@KU)g|f^B^E#V(;ae@5dnCgrv4@kzP1cgeL(Z6^ zij>#X#nQz9va&ox(4?XkkPAjRML%m!Ui&-)sKYb=h1&J!g86$@1AzQ`>{}mcN1<|l zQFS8bsnPRyu7iQ2t?L-9pWqbXaNUi#K!=Xn-kF{aG-1c*uGXQktoSs%aQ&|3+=Ohg zzyyuguN(e_MFo_^*qeB)`&yJ_GYfX&LZhYk+Q0NFKF4rz`3_jjO}m)JOi~M1JehQ@ z-I^JkE2Gu`a4nstt?s*lFs>V}r5}bzREDw5{VzgKinS z?p3)F&vG{}>1fDX*KnM&p3R@Oo&9JEEXPw6ceS%Y8T zqqdj=x-E+nIHE+PH6fm50rqfoD##b*{la%5rVj{)`?PUU6=T=fy-ph4h_B3QIW9rZ zVu9ZHXsvG`<5<1dHn-PjN`-+2=%74+ER9b`fR{( zadbb0B6D8w$?)9zP8=Yz-oSz--J0_0yF1ruBt*+a+L;d88#3`8)!UW=T~zG7MOuwP zhlZusNU|AsQv^;1VVv--dZJejT*y9>$n(LXH;7 zt5gya%-=T9b700ryE!zd#-Sm%R_RmgT^}SUw>`17RSFH}bZJnNII`M4lnYvoQo@s- z$czwY>zPPg;e3kS$9u)BeIQ2XWf8wNV(bCfeliV1rIZ%nLj;#VI47d`ruW7+5q3iP+~x2B`k);qUsPf7L27*Z z(&=uoFbwG4f5s|;WB@qVb!Rm0b&1bW=91;?-3Dl61~lz;e=594dR{@6*QI(TH!Abl zP#|$sxH+=$^JRQwUxLrgB9V_nG3{h#uEWw`&vg0|`Tg0uIVJGMbnW{GvW{pFH+!NZ zh2_uvIZ0gg^!~K8KuPy}iEztxN#jGaYdr}Pwq|~c241b!O}W^kYoAIs94wkH@*#1J zQ=f*A#R=~kXnBDc7yt;P26RzKmK}TVRe??3pKW6$a9_spYaK4iv^>s?NXZ!=x5xKP zCI2&)Wma~nH6xaIAcMoB&jj1}GB<|YxTD~gm)?K&8oSZPE%`I=Y7#lG|2Ydr?-_U) zhz?}BT1l=<^XY5b<85E*7{SW+Kfs*f&yF?J23Q6*kAhVKf3V|@2n9yMh$RK>x4xXc zi74~^1yC9OL=N9A)k?CO}EHsoW~Ii_&dq^Ap#zceX6WnekGZJEpFsqQzIc z=j%Ewe@C9z^N70mN}uKnIS6T?ND+`ys;gZbUW$(Ijx-LVzL^L5P7yOzdBP3IHDf)?CTrlsBT1MGRjys!&FZ z1}?juFOP{(!MSGRC{!p;GJy!HVr=J9GHtVttDOZsR#asjzQJ%UFsJ1X@AbeOASYl@ z7uWK)O4PNOpKJ3kj*?WZ9OK*9_Mf-NmB++Pg3J|1+clK;h8dRuJfm0VLp*s~7iPmLWh73-*Ju;M)JRP&K@+PcHIE}`w#5%k3P;^TR4KMx1AxzX?6tS82 zmb6%P_Rf*0O8wix=i6{6nyzBJ!0`pvkp%h(*cN+=V|o9r*#=gDjM>wNzm5Iox3OoFOh8 zv=~Bt^SHbHp=iT!!U`8G#FrL_?q~a~G2vUxfxiHhv43A({*%Sq_}14nQ|eBgTwvrR z{~9^IEsVObw+4?9AKdOou{H|)4BX-@IrV*-%Cz>DocCtcG3%9Pkhx{eu`$xm!h8VM zO`h|@z??^7`f?-LQ*~)J;G~xlut?U292uJPC65GPAuNU-jX`~lZc?U?do<>Q`N}O4 zlKP=vuMM=DV8VZIDn&vTn!>LH5Bu;yq+N0(ve?Al26tBRwY(XtjJxkUOcsCtEd;84 zz>lA5trFu16Sq8m#QbMW)!c*Xoo4{puM{4^H4qz5UZinw2&P0`m9sCCiAYJEva-BN zVa&0;Q1_wF(OYJT&keqkgyD<{91Fyn91qiaZO`n|{n}|=VU85u*J!#IOJ!JB`4A9w zvH8R&pIj!R_>vzd03qvGkZheYOOgj|V;y8%t}XTJeKCCq(O#2b z>B&^WGQ-k6eJ9>GsLBaK!!%x6$ofwn3S*N!BA}!#h?$k+tU30RKFF17A%pqpaOhYL zs3&VIofv=OXnfw@b-vWc zH|#@##evZrlxsphX}d5VPUBiG5TU#B%ocrf zCZSW^^S`|GgL_e4bWy1?*LI79Tk}hs{AhmesUdTJUF{ErOZP>u`Up9IVHo13b1T~v2s=wR?xv(<3p_$~R+6hA7oGl3OO@EsQT~ys z?7+d|Zu|vMv!F0WmMk-IyG-@imIyHGTFm~!^)$e5U*xxmWO>MF15yTW-uR1PjqUnv znS)lWLaJgdOA?=2SMqy6 zP@!7iVpiI=${eQm5l@{v?YDsF&lAjfbVr!0oWYxomc(k z0ajmSVtl_K`nZ3k_{Rx|6oJ$wgDE7OMsiO%Sc4E0p{@oySwRC~7tWdAIaT=!9&2*Q zVu+U(Aw3|pYFVv@Q(H!&O}QZv++VrF!vJho+)6Pc0?v#m1g6-{pD{_rZ05`{cpKWj z4(Yd23#m!kg#f|;mzz(FbJgGA>q$9U*LyWf``-~Q@S4Sir1OfMTMo~QC(_ZOm3cSr zCdeQuScsr2iVs06^DkwgrdKFYh)2;Ktt>2>`Oyd&q3_Z{nY7(ZHf0oq21_M5CC_a} z=07bNVe7ObNEX9g$?T`_eibfXAvJK}QO;>4*@Cag6!OB;9&(n?#Q`S)MDU`tX3_=^ zb8BP!kA>-Lf9Hx~v6Zf7@Va4GKx}BLzHgguY=x$D=xYRAlzQNT$acGJIv1Sk|dd<`#EkqEHfh>d@L|feWoFMKukMLdQ*JFb{ zU_O`WEJv3=lAskv4#<<=j7yh~{WxvCnKn;SDO$ZiFlYHYDO}}Qe3n~@7qit{)>V2e zN(j>VF+H=n~ zUirn6FRQc-<9mG0o0{;8XXtR%eU-Iq*PMCWE-kWm(4tq9&z}^FOr&wbo3~*x3K~ zRczlI`Tn-7POf;vE6u<1>D_z{`;hbKUqlWd&}PXVVxvY_sv(ZqVDs|_j-mb#AADyxAKk;et5hg&Qq>q^+OuneJgi-7lceg zJ#+_X+yC(1ScvVfk5Yo?S_jVEGaBp>(W%^;kd55O8}gyO6;OT?D-)-wHq|RBEH)VC zb(y-H!|EJ#$uuRQW~#F=tb-SQqg#yh0KjW18Jvi?k>Sx`&~coDSee zdLtKv15dhw)Eb}{mGyP5~|ux&**vERRr>y zC2eKY8VwzA{z4B~v;p)`IoH%x4+Ibto4QMT=%W_45+UK41-52EIw=*uUtv|;S!Q!H~UVG5E?uGnnVVoXB zEIjm?j>24^7OwZeF3D|lBbprHG*^eX=eZd+pE2`!Y6H+1*r9w(yQ}TY@^*36TEA3h zhi%>9%=y;7Bm)kH%ug`jShjsMfXoQ8@g{i;xK1dHzUG@bluPxS8}4tgWG>)#!J$3f z0!-Fisy*rB0ntJh5yt&bs0GjrZZuuf`HnmNjD=Q+HgPG5Fw^uU9wxMInoejFM&r zOApU9t1R}8S+K&to_=>9arswH#|GqdFLvASaJ5q3R3w`@StVwiz0(Zl$Vqb!y$`n_ zY+!;JcvAv)q^kArG8YH!r`UOcTl0|!-5F{%m7vsuirGNy;qf4>8zT9qwgY!4^5BER zy=tx6zZsY>%9=TexOz_apY{qImL8GEPF~}Mc~@DQo_s@hIae2(9lcOOvv;`!|7i7M zRbtv-l2sK=2mo>RoEVYOAr$Er7n}5$~h^4Ef6=6Q2Z>&}H>tCd+5jPBkoBJm2Vz zMzZM3Em}szom-R@_3KoSXIbj*|CFpPv}5l*yiUi0vl?Q}#<>KY;>O~giUpB8~K!E z&y#LXf~-WvG~|c_mb1=Riow5=4d6gNc5lC*Ybr~orb9Quew|C5w{FEmJhTkEOBJ5^ z*?UIJv8vg5JN1#xV^6cMs=d`e+vZU$v##D!yN6=O@7%iM0yiY(w>(k@+?MdgjGj~1 zew1fH%)<+p>a!B^8)PoaLFRwJ=c7_|`%3aPm-H2zaslA;FfEp=Fs0@BD$Dg>+|0X- zW3GcyS zlBD3-1HW0d(r#wV?AbUP7HKXi{PS%KaZ>p4YBm5rSXkO4B6`*ovu?gOx!%T*nSp#{ z!OXgH!}D;*h|>-O<3Zq|hfOu7`#=0GtL zW@$*M$h@H#;#(yCM|Tvn>1fX6Pvf%IM+9U09tvm69Vx*FIMT%>3+F3WYNoDr_xMRh zj*baG__e}+eye^=Hev`eAyhu~SYbLm0ocA)0n-V+E4@s!V4r2>0ibKz$XfTgcbLx( zpJ^p%#d_tQvva`iGe@Ief6Oa-U$eV&2e<1iPW)0eNk~%pSOlSitEjlyB>2Zr#2{JH zR+?fGAd1Q|pPMF~R{ha!On}R;gk)+&p0ZD0y#zAfhh+%{>Z52BZNM@1;=vv?Tk@K9 z^e|DZg*H>++}GLqFb)=c*gB78feaN6uB|a8njC8Xy^Y!$Ii@$5q*B;s0DB468tjvT zvvcIq@?t0Yob_gj;zIn#LncalrkR^pN6ktq8XU9&kZpyV~P* zZWGz^g$ySAJqM55FbFS8S@%j|&YH!KXCTBook3#_Bp8k^u7>#pbhFzJ)Vc&MkCP9= z0RSZUSV5(Z@pOe}QE`{AFCxxELnFRcHB6w zB+>IRuIo(Cgs~3_;Y0ZcslruqioXT-Vg)Xt^b;2;d){H>8Fdi$xCWw35%w%A%MkrX)Mt*vlQH`xG3^f<8{HNXiTw%jZ z9cmrp71eL=0w=37(TcNt;y$kHZoFj*zY7!{0K>MV0PLi6kc$bQ({M|5-iaOQ#e~J znP@S3L=8a%LDU$GFpLSJw^2qPj6O5wyCnDXmi4~BzJKppj~~`Dt~u*I_ul8;`#7%1 zmCog&6+T-@PyR^kZGa`SC8%Wv=%c~%^{E9OIkmt7;{8qrW*!!49jJ3?p+4ICz`Kxr z@L$cviqOouk{jCTy%Sz}yFhVVR}{pTBV?)Wp^mBT#p^49NWHZhMH92xH;J=N;!YD! z_Y?5agODYLY7W;xTV7vC8uNR}l*=JQkwZ{sAf%2NFuD-YW`6xD_gmji_n-^j>Uk3Y%TtSK>?pF&u6nEpSu0BA z+<=4|4(e~iCvhN`@P?!A4^MVqoE^W_$z`+q&ik=Nj>!NB{eZeZcX&p#vf@F4e>PF` z*`KB2xOQqQRZnf@N*?QUEWeI<$#b&~>Q~>JEaC)L3Q`2mWYK#u=ewDImdZ%&8TBX} z9$6W=2x;p)0gBh0Y+q_bmM)raB?cm{--Hg9{)}V8BCH;~{PrT0CpuT0x3K10I8)Il zXBfNN0+q~+pJeRF0fJcAA$cF;Jm0rl1-!vHQ3v z1fRRa#~HqZmTQYT!m9XjknZ0Dk7?{v|L#cMbRzNHy<>b7(9@e37K`AOeJV?$i5%h{ zUd*^@#-D85+4Ab)uEJS_;)i8p;lf%9wvF|gYa9wU>yO&Ejk5ks%$^C}>yX~VHeCE^ zJOR&+e;xLTa{75@HU^Ks>>lv+;iothagV4lo8EILpcqGew}|qBwHLoos+VyLXXJ|1 zwgo9<&;1haG5ORLPIS)$sx}fIQNIa+Rd=e~5iiOO!8u43LY0X_4~^%ND3_>iu+zoS zI=yF`SS>C7B-tl|6kCRO66PzzIP^p!@__O!)FYK?H4oLI>5Ht~#=OxF-aA)K>p@S{#2X{=V{(`j?k7Qn$IJ zDl>OVDfL^u`hvTgpbx$H->6Ahs+7m*2g=8Owx;;`L7gkI9;=WrDD`+UM)!=(v-ejA z0-9)6+6TXZtLRi!H} zKPd1GU1#${$(3F;YNX9uc)0Nsn>cCr&(?1n65F3&v=LNjtI@-c(K0)Ya6zxw9#84q z5SnKguwQ@HGEYtjeU`9LTZDY0ZzJf`B2miUJ9*CLMddCG_lhaofi~ZNrj!y8Z@ww@ z?M^jmYn4HRFAqA2vnqQM>{jCLl66Sc1*sbuuxRHeNXZKa51>D~xmo?eOMc-@c!7q2 z52~zD=AXQ(V8yb72@O6q`8-v91~(S*b4l8Qi!UWX+)E!4Nq)dJ)ISdq^>k18{kPj8 zICA}mWmHs$HfR3&dRlC5UEuLX<#8=yA&Ih$fpxq9){$ZoFWr9E6sk43bM!^^TkDyu zj_VR+(Hd;mrLf388Oxv?hg$9`V>bkuTOo4?A|w^2ME)qfL@DuY;fvQ0gG{^mNpOwV z=Q11#ly)VABN67&7tgw17UO+sOmaANE}3#ExJ!>s&86Siwqv0+EjDo}!e9@URriRb ztk5=<0oVSyeP!jc^<@7$7K^f|hmiSuVn>m@%{%%Yn=5QiKVTx7D2LCncxso{1 zDJ?W9iYaAPMK=fgD>4_f;`rpDn0;rI+!C*X(nBfJbR{>HFb*kBZLREQ7KDSF-c^5H z=2wi{UFa&}>~MJZPq@Y_`Nah{OyW^W_R@2Jwz_o>8FdAGyTf71DIt(KwgesV$bKP~ zo)QB++i%3FD{r^r5T_}G$yF2EkT{>0ONd#JB4u}^RO6XnkjeDY6-U*``{^VAhn z{&y33pwN6f6K-H?->bmt!e2_b#`gm?jjU-DcS((F!JQ`fbei5hxGI1R@p9I zbp(iP$;o|O7-%}b!*%1-%KiK2-z)HcYz0(1BJ?)KhF%9~Br9>0f6KHQ%!m|hwDOITUm@~TisV* zHT`{RsP5i9u6_)$nm@KMZgr|zW$am{nAcEs3`cPP#uNaax}jA}(TSfi)M2qemYiFp z@3@N6Qyp&oCgqZE=kKu`^of$k&+!x_0{mzu`ehzBesV=<_72#%;v84O93(6qZpgIt z;mo{kIMiizwn)uw^+2zAa_ClwTPbpQs85gd1??R9-J*Q_{f}e4sWHd&?faR>8u`Qf zEk#RZHNEVWXz3-`fyi#+{KmUfk7-jipPBhQJ?>wQ>GyJQ)kO&&&vS&cWgZ%*$?iSO zuf-wsvWJEQhBf1V#YZM*9+#k<5sriwl9?>WWd$>ZE7Q(3Fbse#9Mrjg&RB&+>)J`Pk?0SrGm5?UtkvXTAr$ zVOk83@Tl1jDM8<2A?aI%0((2%+Q?DvTPbI@(ZA*bud?u{$w_PYRWk>hECEUlPI5Ke z!yVjL{q#ss%{6ZuOL|EJxbSDiW!Qu95VT{6T5v+0oEd3WW>)q6#mhu}%ZVrRIX786 zR%bGgTRz5elw(8d(Z|ST>|w~pJC_;@umZ);pl?q3d*hoML1O=`%;S8tv~&{0dF@g|wWqIPUfNOpU1c0b+SFgY`OhJE*%eMm>$`L3)T^)EXjFz`56a;l6? zUp%FzDep~j6FCB#hwdubxn|l!Gr-B)tdPMIAHK6DV%I>~(u$T&6MfJ_!5_=?R8h1t ziC!}IOj)?Fzgo|XBhktx`Z9(hKg+5BhRrE6qM4+(CfkYH74N4e%7r@#`?5V;qIm&0 z{!vlgk6sK}7-bCtMB3GjHBV}ctXnrHa$3+FAImbTYf!bR(rp6PlAw(si3dg#` z7d6*R1E+1n>)h%bt9&h$KK?o*Wt?xf-@g&0qCNwO6DRDLhet1YnVsdloB`z$J{el} zIQ9nkS*FttI7#sq7x_zC`z~aP4Gq!6n|zb~IMge9PbSK`xb@&cPbrQ62hBB7=u+gW z`~`V4LQcBr18sn-e?o+?n~F&&pwpE&l9LVcN`_>4d2 zxl074um>m@Zxg9lSQoV>bG|xzt8#4m#{@~CGsqSt*qZS1*E+|VaqIEv?p+;A?i=X0 z^qwb|^+fkmWeoi8THR%Lqxo4MpvMNGcbA@xi%HUtH>CY4Wg9Rv&??{4@Kb3-)+|Kb z=?eE5OSz#Y(9s8*>}zuV`M|jS-=jIc+rNfEOb1^BusHP7)yMT>IIvLiEzKxeKK5B6 zO=m^O;t>&Yd^_N(&1)uq%@7?8a>#t-fj};!nqU52z~5=mtk~cC5_eS+#AeQt)h=eO zS@!l`oX%JUIwSV}c^Q8HXr^+ir*dr*3T7LU5WkDpSY}_^hL_Y{1i^~@R zq)=>Z+;&%324i*Xz86vB)1(>#a+BPT^H##GMy0}#>4mw=y_%x6nb-mYr@O(|mSrN+ zLRJ_RuEWkW1z3*#MdQDsS(Y=BcAy%4;BzgBvPDwVfW{yGNkyRzIlqV(de_Z%O)%s6 zPAp-3k;Xr<=)}R2+dKQ89k_L0-b~0AX#T3)&8`%yX<`2NZ0vEiIef~%2G?qqJuU8p zY0n6f6t;~SB$r}*m1XxLHC=q{6~CZ47^Vvt9eOB(s{Is2dIzULTyp+2@K*`az`=o^Zat5T2-9oA78R zVl&6Y8lugB~Ey$zUQ;% zS{H@)YucAbRqMUi{jP=@MLi2WrrmU|(R?BG?V+|zG53Y1&7*aFSu-L|UJ@p0`U|mE zQsYDky}m~`UfJ3hgKupc({FtWnTR_!;r$xLxvIUQUtl;lm61_{4&)Xq>3sR<_kj5z zsWtDJiarnW{Y@)pwg-E+dj7j)DGi%sMNLRG+Ux~~dZ10d-6tuJl?KFX%9c-P=h%Qw zTfvi}%B+UkhxSf2?$lP-7+q_0Me;FVj~d-IvyFFx1viuL^i}E1=aiuXLtw@~g9kk( zcG34h<5j#g}d2u6uZTaql^W{4uVu@0u69BY870loN{Epi*&um zieiwIG=*WyH&N$*eUeq>3yqrUWifv$pQ8MWQT1uc@7ogi(4Duj5nt9~24K8c&iMzCB%G%9lDa!bg<^amo@ z;~4WoxxF&i|MPY7yf-ugy4j*cTzrFw{U-G>m!VZTn7p2^aWu;E9}6$)(IAU#9S8GZgo$9YF{`2 z=rCR<8^3874@BIA3nc3^Lqb=biY}T?2J4{RCI2HrrlWe&13!fti2)EowZqN{Ycl-6 z2_Qd+eghXAvAk3V0-%6wm9DKswD)!h89>glAPzqU0+2PhIsW1W8KkUw93q5q%;giu zMiuXLCNSE9`qA}*u%kugxhRsaI6#ACFyrmRcI&w|%mqMLo{}QHGk^YFa%0Fi!cG3E zg6p`72~CYGRnFkMCRpU@k*;J_7RHFrX9 zbM69CMoh&UC#Uh08?(3g8?|_ArCivZvCcBU#89nt+&}vvJjGa-#pH45kF6cJLc}%# z!~sxe-LGGroSzTc@@NgM1L(IxwkB2QPNC2fVMJo*s4!9qx&o2jJmn~RdzN4WFowV%&$-0y72goPTNB}uo zI7%*FeO;ZvfESlEINWNio1f>zH3&_^c$NA78x(X^woPF=$PES=UEe^c90Y;GWZX}Y zQ*+_``|%UDKPCws+z=HAMZWSuu=gAF1SLr|A-r{0x&yuGjBfzi_m8M)0x)L%KwNrb z`$DpwSqilD;+RY1=e4hy9UC(tlizsTHB_Ok!5edr-r-du#Cd64r@g zP;=O5D1L9kCyqIx9R2x}0OyfS;#nI@**5-P{ZBlu=&RY}eh7Zh5w!}yms$0Mpim{x zI)9mn4Ki_xc3WmscHFFZf5t7-^RPv^(@k1v`Eq`hN={(U4GaUtSMt^hehzQSBeqZe zh=H9FoDpZRQtVtZD8;9ztSqr#A%7ykfIa{UoYKC=MeWqVF@XOjCw1RM%3(B^H~`_2i^q21*9Iu%gnxzt(Qb>VuC7C<2ko?qS~HW~mc ztSY(g)#g5TQv4*}PRBsDb)o*zH~^1?CkpfY(&r`ZDu7-?43tF@YBs;2`sbQN`_Gq^ z*WP23KgdF#8(e}JL*q!cyM9ZL9nP=0*7(+w<}xwMT|cnE%^Dhdq_4QSn?LWJ^=y-F zdvy3D@w~QbC21#9T--waVR25F5zJ@*ar+8u4)csF!?EYu~iW`GPRT-)Bm&U<1z8vVE0_t$A zjO*Sbkh;#0WmYon{KZ88^OrRd#jSR~-$!B6Td(yv<*40@QRP|d7KVVTSxZn^ z`K*Y^=ui9&S$(#812XC5ehd3W2BK>3_kBr&_(fFwG9&|5%QDxcBL28NZR>~1i-IX-&u%l zqBK-dLC!KQY@Oe5$M^|W{5j+H6IhaSK*cG=&vd|Qg5o{jzwm@zfEIGa!A$!?^DZ(U z4iHB$X;r9&&b9bX$zgNZ)SNT*Y#I8iN~$v_^CrN2#6J;^%3+(u^S_g-@8ChH zi|qtd)FUxcnkbhXXp~2fQHJtHbmE!JnQB|A0>e>ru1r!(5blTP`t8QZdB!oBY3&~b zJ###`oy@S(n?eRyb}-&5J_iryK8N_8uQT@zB!n&Xm!T2>9zUJf){g=19k9(Q6>#V- zu^IiI8M2~O@8vyiNO%*HF@oZv#b+J+{o1BQkd0Rv$03bd?Po?*{Cl@LdeCHm@dU-p^Bynjk@yiP&0Zs76Y3@vhvlmA zkd3Md*dFh-Q3_+6w2%EfV$Z3S|5)_|aWKiOKjm=XD9tz)y0+ETXNuUNMS*3I$^=*I z+QN6|>as&^&=Su((UEV`DqB^uzRu)qkJja)Te(T9PL)sRs~9J|GPm~2M{3vfC%s3u zRvApYLM=jf^Q<#{ew{Pcmv_SNZC*LqLT)KydaFRKyG@^$o4g|zSsq_E_iIPbHEQ8v z-CDx-6aBMhR7sxo%UiE7-dwGS{<&C!2X3TQ!K0z*Y1o`KZGvR^(F;84oUxu-K>f}r zd_MkD1wcJ^0l8?@E6yY_+pwJDe(VG<%Wb-})^N?{!HWwZ@ndY3OM*Y|-KlQ+tp|g; zR~h(O{a~8n{S^((avW`M$XAT@dDiN?&8&wP=a(s2Gmz<-jvt~i8l3#Bu&GIk;w#IO z6sE@S8{R^KUg?d*vc4*-tTxj{0pLE^o1+ZN0`vNvTf%NM=&`gM!=?lQu_FHFyi{m9vD+UgQwit4F5uItmSdb zk#A-ftivDA)xn1iivusc8?rxg%x*7AxsEj1mgsS^4xYmls1aC!i*h>^ML(yDFv zh-=0jU5=0KFm3Tt9VfMCv7nY_zgDReV+IDIf6Lm9!WB%of&J;WsSLT5uKqzxDE!#);(dQ#_)1J`Ul}dg( zV^Xn%Ma)D&mUB3*iFyMw0@AclD|4|j4slS2_4mFRH_|m2VFk__mc?+z-VGmYU0K<-A8w5_= z^1|o{Ky)Ybs~KQ&wJu^Wim!BvyeS{G1ke!yyWT)qQS=v{+(BqtmZ}VDSdwcuD-&bY zN;t}8leX1X*&Gw*;?le2G*hYe%zNAH79C!dASTm^MXhyx_1_?=7nq#K59(yoZ=F&M z%0G2m^!#p3`}&mIHRHj!4O_8mV9cAP3>nwveGcnG4Ts1l`l@R0vOiT`D` zZ`)FLU(>y!^ooM_uLXg>!n>8|67?jlq`y)Me~zynX|(XTh&pL(#YY>+xyogtTAY(ZEo_Q z0WPKfl8}~rv`1V#GqQSlq_u09RhY*Q+bILuo=u`aTw2bes8fHx!q76dS6Qcc+|AEY zskBE2Z@Ackv||DLx6HIDCbsjQ%&_#_N%|=r(BZL4xQi`HwZq0tMjQ>a{Zvg2SAu4 zh@AmyJ4cMJ{vgPQsz@3>43gkmy>(Y!$KjstW7%LsLnlra$CP}j*JLMI$YH>GK5ech z`^k(hl`f;ZiqOX*-h{I!Y+jcbLZf(|H0`8Uh6MW~$pQ=iZHhESE*98~wdU{376+}^ zKz?C5N&c%gpw?h~DRGwfA`9sZz#}di?~y>O*WB00A$2`W#iH?G&f{v3rr`e>QyE2U)G*SIw{_lvJwm4Ti~@`3S*v?S4)y+{vcmu zCrw%5Wml{I{-`B@M=g$2494lK#WO|SO&5TXw|Nlr8a#th>gwmob#zwQR!;0tK z81f0cvbpCV?5Ein3*O^NA_cx_OC@kK8B|?AqYC9dDL!CBb*QyiCRv;aQ!X{o%OsFC zJcLk`?vZ~1?Sw#E@ew>`(`PEbJ5n=S^}k?*fKO}}?x?C5Glj}LzA6DxQ-J2RCNpef)lPSDicZna$K!JswYdQOCN7c(^9wAe2gOmQs zIfyWIW5XcZjRf*?L;F_X^UVw|gc7u-)(-a&mcfJRvG;lWX9q-kl`MAZwJLFt?j1^( zkzYb1)gKwT8!p77g(}PXKk5DA%j%PA@8!P_>9WeoP-?^e~~N6hxgUr zRvn&C4NEc%L~DzWB7(cL!y#%DX=e&CCXcEXXJxV z5R==>c?$9gd0cte)P=-|bK+F49+zha`-A6>@%c=VU)DY`HU*GWQtAXI4Be&0atx(~ z)i9K?oS=aGxhBKSM2fmKZdYN-B6OrT!uwJ_=du(ardELw5D>qAh4*1?BpJ~|G?6`& zEdAj2+WhSHm~Ha&f(P;l*>@B&P4%h~(+@?CcE9q}n#d={eL1{JkWZ#CZg2?7q<+18 z)_&A6>h)`CMZ{lJA=9+evP9J1e+OEyuLkU9R4!B6*UJ&K!@1dS~prZM$N|f_QBGKW?W1Z?rmNd`pl$Ils}Wf4w#=tJ3&WLM`$dx`+FMa zej){eoVOv+wmTq=`Jl>3r9Gqd2dIisEQiK(>IE`WWP^d^#*Y=TP4lGK=%#A3EH+geCSu3f$N_)L|vjK5t3r* zclQ#7Cqujj;gyY}9vk2EP*Q|(YS{FWq89SD5M@!MlFdctb@?^ExEf9&|1E}v>zB#^yi`6*d5k}*|MgdlM4iB9-L!!rTFU`2rhuCVUZi;q?2 zR*LE2FddGl2EFhJixTgZ)tC|h!%@NoZ%zGah zhkJ83Enq>b{NagwCdGX8el6ShJ?6IVGt6cpYjNk_kE90R+^CQN_13ZWzUtI z-Yo}?;Okhn=r&0g=8^BBdQ`!i3gFt4Nno7D)7)OwoQ&jTisVZOwWZq<_rq6em;6R@ zVwp6JRc9LwgXPBrxNoHPNyQ7c0VU`N0gg}1y?@&*@jnaVXTJEgI!uPnbN78Mou}Aw zwHWXQ=jWWl>Iom4lA|o1^)(92W1F@M;Y70tDXpLPsdOtJsd!ilw(V{`J^7Nx4yac5 zpb_101-)n|T_wsQ-pZ*kXl+5fbyuaisOT22dURa|g^a1ZaNFDuJ37yGEo=*p0p5M) z#KBRfENDv z(8r{7tcMVw{k4+)lm7}^Td$+XDlapqJyO{WJ>W4(vVSBEouF1s@~o+3w#tFObpoaq zy7AG#eCD8KFo&F(PGNk+$K;xlUm ztLg@jnK6)(i~miNi`@6aqay>_w(Ir6JwnSe%Y{F3M*IanTHt*F#lCfYTX5G%C6&V9 z2eto1ihtMLr16U{>@SMx2*7i4?AZ*;-WhJb*XXrk6#LmvxG0 z#S;)7=tt7;D3V>bR8BlDTPF^y4-MzP6Dp_Tbo+rWU?ku@|nz7Fd0j=24 z{y?C9TT!@8fPYu06C%iVwK&QgpNQWfa!c~%Gewk;tVj~8^B7+gpuz>a;XS(8q;|azHNA? zCnKeHi?AeBjmda$pm9>Jl$3BP1maZ;oYiHKNONoWCC8-6#_8jCv8}4O`Pj?ue!ByB zPC`A2vAIT!rgJ`s%RAGWsf^1wiW=5gi`g#jR{;`}$hQlPTt2b9NAV%x;2C9_O6lS> zVD=}=?8P@PHez_!pb7$QeqeXlnLc0*kbI^{r_;e5C?#P&Qu}+DeWZq+2YZjPS!gky zjjbAhcMaSU@qDn8&U*iB?s-ruyNIJeGTHb1i5+cSU4o zJ{AHTeKtPTSGy5gfVi^TBN`No^uixUy~)XJ%=Vmo>pHVt|Lxp6I!{wT31Wg433h_^ zA~OQDVn1Jjba+BdDUta*zoRd)49mSR&9clQy&^T5{bgm;cSLg#4a%Tq29WzBaQoi1B(yi$~izn78-iM4W* zr_^kcA34{r0hNIhCXUc$*VHl>5_3{|mSAl9y6kZ0Z}fRcz^~1CQ1jS7@@yzfv>Os= zIo3Kx4Le(A15DtN&L!Yfm46SL?D%Bz!-c%>b;#OFOEjoA-1fHa-r4 zrbqhTR7gK~uqDUstt;y20OSGOuO8@|cbGoc!Qw!9q+;%n3Kus;Bg$-LV`EDddd<*I zmYU!Zy1+o&dWkXC(~E~AS~~#&xHP%jl&$eu6nH>Tx8>o%)AE$|cHYzcxHmI=@mgXp zHK-bgsbM3FGIkYV6UxE2H=CQDGWof@?N3>7&_!+tKDI@Rx---O&rF*J1{j1aN5h@F zM;cbG*K}~PAVNuJ9<62QqrYN!EFx}HM?IJ%q&+OE^xG@Di0bv?1-kdq(Kk?Dlv2le zaS_&&dE^CZ1m0H$|GWXWd79R^dwWe(0}EU(dg*iR>t#h%0e3waL=3l<61Cu$c&UM` zYxof#Il}#f+J%Kql8^Elli=JnN_W!@+Y-UL8LJYm3s#9K(j)Ob-1s4z?L4VO3S*sx zuTs0ug6EWTpUxlxw9l7(Hg14e^D%q6)3uYEXuLYN1u_K~y$A{Q3`QN=oiY)0U+Zr3 z>6=K{Y0KysG=31LJNM(BialjU^VE4dDTQ=J6So$@9-$T8XovJeo};*iVW!j-m_h=Z zG}Lrk&|AQ%BPvEseJ)KfT%b*C7VbPZxq1%Vaeg4mnmlPjCl!~<0}k#|OBx7{<=}9M zN-R)ue|3eRePtA()aCwa7bdLY{7COW9oym;qKqr#zWRdg81zG%g`eF&UJTQN*fg1o z{_Xm9U8%cmGp(dP&;Esn@Te_rqCQaoPfC1X^!2kp3w$4viUTGu6Zis`g#XI$EX7;w zvj%%KqY~?)@uw$mJs!W-Zr`Thcf!bPyFw?v8+N^$xK>W0xsK4WxFI_6W)`@?3S>@u z_9xD0MWN_gy~T-B?~cm&POV6y(U`eVglp23q-%diLxW2AlFo~CO$G`kSRXpjzgY4R zyLeYH2Vzs=*uubkfwd|_magNVb?by#3pg1a5!9&@wF`NaoNgEp#seU zp9HElwma^dw;pabtN{~KE}Kaj`25TrXoJq^lXPEb5>^pZxfgGc-FyPOnxj+{(r1U+ z8-F6QcO|F$bT6EAcXyHswn>9)xvbWZ>#FfA!3y+Y_UAl4?Zh7(f|>6%F?l2KZf_4y zoQAB2?JO!cwmq%_$GIBiPvno}!TKtKXTTN$N+Bptea<9DV%rC|Lm`xdgO?n~i>*6B zoC5n*=UAaSSwIt^dvFPQ0xY4;LwZ_kvgjjsPaIZBD^%ioDDLrg=H?Bx5!a*y;IO1G zKeqF+P3mCGKEZTGT5eL$n1^zj zL>JM*tkJt&UvlwM)EST+B__J9UY@X>x7vdfy#uSGS{VHu9o)_4ZZKh4DYn{j>(k!c z@F$k3=`ZJ+_mm7cr!Q1;o-SbX>||;5-CHv>D>yZBH1Cn1m%X_6q%s>?eNy~fS1?v4 z<4s*rhl^z7EsxgEZzSYl0+ypevDE%DqNAmuzs0+_n2)l5Vz{5q0J!QDbqpDA9G`Xe zJ9qjQaPcwK04@y$iyNyi|M`E-`50=LtqRVVRX4ce)am?zDvhSD=e8&puixsVNnW@R zpThc3G$PHqzHK%oSDhI>9K?N&)g!BTlF$g?dWHIcP>IrLqkvo5Df!{HH_3@Vg&gc|zB zFheuy{)x&sPyZRzc1gg6KWk91bo2KSV1^nU-bpUe`8}8XKOd$uC;$1T);*O!-vn&d z$3NdJ=*Yj)G9ch=bZW@{x90w5pV!|=$VO*8I1LZ|Z_WMBKJU$4(EA{idU~Ju_Y_VA z=6_qZfA7K{U+O>Q^525=Gx6WY`F|?7|05cyzD?-p6_|VwD>F^p?8K=YJ&ujcA-?Xc z)(_glc+Z?XR&rNZW+ZR6-kd%{qa^gh=Eq`9oA;Vyfl@lwMc9C8=x%JM=*+Wz&vn)o zLa`nAn3PKfhgM zA(L_y8h{yOl?a(qW|=vv4iiUg(9BhjG|j(yg;FEFS#0I3z(RT!uUUlbUe5XI-!5ZO ze~39CH)YXor#Ee!$-OaQdix(U-@F1PpM;4`pUk0d=IpMg(T@}hgRwPpfoaf*o9&T@ zi8~Sm(Or_|2?Eo3X=F|@;o#)wNK43de#`x1zl%_OgW6s#=;kK*vzA2Y?g-grzx?H? zp#~IvCVGUtY#N)Eb@a6}KlH!}zPEBz=?}-agdL77)@QR=^qiazA*YeQ&5bH#yF^onhCp$D5YKUQ8W`*xiXG&l>c!%*f8gx@EqO`S}qQc7$P0YqHI9^lE#N#&%+{ zz{koi5(oh=WdEcE9}r?iLB6$nkI)(tf#^{9>3MeuN$L<>onTt5fHHWPhf;T68|c~aMbJM?KUNz8*Oj$BQX-%Z)}7P4`nm3c`i zKrbKipgUq1;KepP2RE)ta9m<+>Nv6s;~F`RdA7$=r)KY!T4n($8M4?hN;AtMb~v~y zq72A$Vd51&Q)`TN?gpY&oTF5wgs2m=8#X6EZQtd^526$j>%OZ$k6I3IZVQ2_oZzjJ_nC zTCKQW-AdMi}vmmIb>}Q*WG_+b=9|l7wr0H0%BnTgIuPq9$%-*?w8_nSeyj(ul z-`hZ}nQ`~LC3cIm!b5o#rhHLVzM)MpvH7^RqWT0jS36R_?=u3DLB1P_;!$8`L)MhS zOYZ!>euxb|HK?DtB&xo+qqbWC4f7gBY23g~(4l)G4z{csxuMM1SPRm-x$WZC?c_PH zE9Si7F|NfAp@^)=J8ug_6B@>DdQ|5fyt4QTB%1^CTgM(U;Tq(+FaB-AF_;#*#TpxH zNes`6W8-vwy$o#MxiJ+FVb@hrzg zfU~I;W{;?SiP+GNd%_b!$ZJW@ZM=t3b*s?^LZZEQ?{se;C~Y&@fiR6lmtTA%x!>S- z4?@gGNGp4IXq@50#rD4+^{sWW2!eN;6g`{XeOD)rBI#YtMSRt5zr>JG{6-#kLF&_+ zchN9LWR|+PBQ^w!g*WkwMC`_Gon4sIt?AaJ8i<*Wl4qKw>Q+iOPJ_B>?5N*9?#m@T z9n<1_nRk?1kVaLZ$8o2X5XVZ7Q!;zt&**jsX>PfXfW9ONTsu-JD>{uE0jUO!teBsP z3at;vM$Y4E==i*=b5Gi3A`B_(TBfoidlchu(c@fmBg3}X@k`?yD)!0*29zAAh87wFTHCf z*jB#O0jPqX&-8722#$2v<_^a*5wak!>L>5!AIZF!Uyh%bf>%aR`y22x@^g)X@QRk6 zU7V4c?|s!KMPd$WzVn8h)^c>wFe@`BfR71ei~UG#+4`KMtSuN%U`4h$Vnwwut{|&$ z-I){TSOWJ8Fr&d*2R$kJ+1n|v`DAg#LjxkcXg$rtUpHO%tLmEPwhPLR@2e%G@%mqwodVvOpNZ;JLi>~Anhj!QE2 z)K&fxtr9u^2rihLOq1ZtrkYPwe7BV?PCqq&A1kNL(a`<3(yc`&6sMVyBs~~trTz5Q~Vfq|bfQgAIr zVfF2us!XP4S^6K!OZ|SFJYlPAV4i*jld8BL?%1&{7>E*lB@RA+8MLjurrK^@WOo5l zo&=Pf7-QHkmltx~f02Bk7j2uCXKHSBy}th1{Mr7_vr-B0GFO+#;wvpZbtH#Cma*>!BXhv+$+9N*`s32@9@x_VS4ka9HBq1vAYor=>AOjo-j zn2ew|P4k| zb4UN8xhM-Rj>(DnsBG&v0i1v`@fVNp&gIAuDNC>f7oexA-59={Nr83U)jEUF_bIS7 zbl>7?Q3%#~8sEjXYC*r^D!N5)w+IWqxJs9m|8x>~k%#M(ujdw}(QR;#*V0wo8;Ak- zeyzvaPv0HAJ4}-PtoKEyJ#r7SPp?rp-PXXzyOg1y&DXkF8jboOrwv4zAbhtMtFbaD zLeX8}^OY}zALoYe#7n=yr7EpgiPK!z^x}%OUh*DL@*>!qY3YKTi+SRs1+wNHm#tO# zHA{b9??^$KI|$6hCsN1NTU{}Yg9(Ip%;ioLZ{IamR{#o)V zfq%x83aJs7+FxYm@-BKVc=vdFJh7yU7^}@;*7^6LH3UEv`xL0T6rLLrIcm$&mk%%Wwcc;oRAqpqU zdeIKLL#>LwpS=9;B~m2+^)=t-L!N|YwYwmY@23HO<$eX>!_H2nRcaW zdYahOm4*H~^7i4&F%}(n(TES9B}9Onl?Y4CYUkfLS0OWn$Mmt*c0rlBHJty*fQ|q0 z=s?nqM2hNV1D==cco&E}#)NrxBZhZSK)2-S%bekR4T?+iOsxLrYkWCTcB8Ua(f*e7 z)^WCUD5GohIuY>oh=$N7Ot}X%k-NnmvO-B}nyd>O#xd0XpE`5rd}Mv^!h22|6w#Ea zX^t>Y8T%#Fz{?Rgiuk-a8qvzetjBNs;E_)7mXaz^pm!sW%YJ#qz4^^_%75&-O-%oE z(FHpVDx_nXuX23k6Ic1RI+T zT5Ml~{mcJr@65xY?7s$%P(<0r9u2}+GJ_|?kY$9dDaKNk$-ZStV@(NTCzIt7$}T2N zVp4V3EncM8> z`f{F%*h&h~Cq3vn%vWEZFn9 zis(7(W0KgKCc3tdwPg48Wq?%|?)#z2zhiC`?9>V6+=cr5{my9KEr_7tUGux(Qs^58 zI8bsrU~T(QDvf!7k$%2c?JN7ng7nF&W+buCe>2>ru1yK1Oi_?Mz~6{EMJNJ^SH;1$ z6m)LaImtrhE+x{rLyFmc5>s;%DoHRNf)Lb?c8#hVS-9(G;P~XDD6~ zdl^=5P244C&_B1v!p39Y@PdqxixDeg@DeqKKJB zz{-FHh#~GU) zpsNwjZ5miRzi$wNuY7}++)Ng)=Ctk`aJ$015I8D_k(M!E9^#W1fJc|g_yOFK<1)YW ziQ6q7@y-D6*0^{BX#UvR!sZT2yIe}2q-ma|4KfsWY@TSA+Ybe>N7ILad8GQo- zw7%c_SIak-@q~O`VIlW>*G2VDv-NVPZ)(T(ZPsox(N1iVtDldXvGp$GEc0-&wgp5P zq;fXGPVo36St`YJVDbX4vE4gm+ilW^7-9 z7*2y|oagldvxLoyD|ow4K6tPcKt#<-HAVqyM%13;&OS3$}v8jtfguQE>(PrUrKx*FMp9>5vyWhzQ#;!70Y(m(MgT1)4|yYo)+(G{L{wLm~zl<1E&T|gVl~Ahz1rV) z7y2dyr}+EWQPgX6gkRI!R`knj?nL)H=-!QVjy>^P#J8_eF`J16`B%ZFU*1FsXZq)| zWLw3cPC7Sq@B@g~X99g|qJRrp?pW}?M^@4WZxN+(7gXUo8}|lJs%<*&Cg@RwZge@p zmF9BjL-4u-&->}ZLVH2LbAzteej{1*V$x|N_r1@p5dfsrAoIPuHR4;c*BdsYo~4n1 zuCVQ%{4b}^jgK8$sr%R*{y?;aOzhhxPBi@9a*)biL$PW!s|3GG zV6h+(FEK&mEy_jR#h{Xu8_ngcIUse}nJT-cl5m8M?ppD9A07VNdwNVl*m>VaX@}P% zMZn_DW(g4Euw!ZW!sLeVbo9AWq}fCG@%f|>1y+i(%GX;2d%I)uN(FlIt#j+J{MS&) zviCJLvP;Iq!@?Wf9j7b28-U@ZIQ-OyU=WkQIm)^#Gp#&xm|&!N%C;0?d`9j$iolU} z(r5ho^&i4*$W3iMawsakNB|E7c{Tj%9@6d?OXto#n)&@~_(XLL+CNkD!RA8S8^6H7 zrOSp4t7lJtDPabx_ab+XkgRD~T>bfmbbemdTowk$I^U}ZtUWAFF*~PI&^4Tp$1)3| zw&UvpHFGpT${~mG{wEtHDyZGVieEgh^`>5I=65OvEd{yA-}|fRLl)Q;B_b?vAUR?X zCw+UtEJ18>T#m0vxr=C#&jvg zf%|EHb20VDjBlzml)a>AZ)R5B3!nxjRSQwaK2_bCTtkQQ;3&AOiUFf7CG3S%ArXIK3xPa{ ze4NCMlrzVmoo*(oUmLZ!-`iGcG?rC96Ze?w^0d{2m9DrX z!&>c#E77&wM3m&-lx+|A{I_J!_NaE&QW2Jz*?{d8bW6vt=(rBGm0O5_NH_KH{3kP^PCI_Jwy7q>DhD-)z=DFo>!GnNCl z<2}=+CWAk<-lw1Folp&v8exS)x$P|BydrAZ6gwC3D5!m0-G~(>S1VGz# zh~O{37ZrRJ)4nk)tqEtLua$b@BA9aeHC;sv0vEK6g1#k{D~URw1{J@~Q1?H&#ozqW zU5wJ3RB+FQ)V*{)A3n)=^r5ISZOW&3{4ItQkeog1a51Ks@VMAyJJTTY22fA5i?nTF zioJbkswP-^>41G9DJ21GaSa{bW_`b6Z_q;W@S_ z#--W7bbk)?l3H^4dXWcC?rBE%JszKKq19ti@iA&IRm3yGB)@&gSw0J!>pX)937Ojr zep|W}Sfj|aOdTj@3s*^Nu;Tu5|m#wpb2!AI}!y}%4q1ewS=RjkyPvfYp zMZFR0qCYutdn^Q-GNej7-=CuH*CztJ&mFR1qDuGmnh9-d z|8;}MPU`JwcRODlSkT*oX>yp> zf?Siwo??r{%1D6Vg7R}n$Uyv3B*4)?BO=Vu^Sg!c;%I*ykX3~An9K}H+1xmVfUco7 z>NxVD{qEIEJDbT1u9{-iR8rV+`eT!&Ag9%dYIxZTYdd<8z-^fGH$UI$R&e{i==>E| zPCZjZ4yRbVK~_~K89uy{UCO3<{0>G(dzVO@*MA=R3F$@jTNDiBrmr#>T@VvFj5J@TF@C z5SSI$0L&S(4eB&#&6NX8TaWA&vYQ#`T4|%}68m7-bXO4+ykKD*NP6@0tfM~LR#O7< z6ldo2s0pTD%@+APB$8toQ&t#?2(^=_&nqihw9*-p{`9!lf>NLP=@|V)pzG75^~TpQ zQGI}SM*6eWrZq<{%iFu6ytD(?=IuB$3WFLKByAYr5=~m`VQBfihj0ZMMxpaY_WK?N zr2~79WTJw-P7|AW0(9)0aQ25r8a|4-&nO6t;%~pU1*qmKLYA6>8|l%-dl)8Qj~b?>&4V181yaL^xXu_wR(bsnex*_O3!ipqZu&YnIC}RFA{PTJ zYMC_Rc^+<(AvX~d6Q{%_L_vk0<%7#Se%LiWeO}-n+)YEqqsk{uYQhT+99`+c`9DFI zyW;&3Q9Y~l$m4B@Saev3mqZ8I>*cS)V52kI##>`s!Vl6m`gad}(MKE1#QjJVgVq;n zK+#}J3~nseN-%Y$9#Klw!4r>`*RYkT*1O#tv6+nt&X5u`?|D>uA-)Ei%VTrr$Ik^* zX51e4^jIv{b#UWKe@**SkCCu?-~`cii>z63gDVE1RCzu9mYfffh!X5ma7T*Q_jJCF z{o*|9ksYW9hc(@^+C(m@2{Xqt2v>{$6-n{S#^Oij1qZDR_QE$65f0N_)OWG2%E^?~ zI5i2bheu#C`K1%P-?WsPn}zCtg=vV-7ul56aQ>=J1q#-Mmo)qZr-?QT7x#o~QX7_k z?*G?Kh23(a=a?gv>!C9*vjn(SN!q&>RW?>a-`X>UpBqF0SWCK^573PQ1MTOESr~Et z>uF03#pad?j`n+9ldorrSQ<P%pB7s~bg*Y{YnT`9>> z553kvJiK}9-`fD%l^>Ir}vSWTwI%HMWw2aWos~!GBj9LES_I97d z|3$q1KyQDbxBm+|{r@qnVnXwxEG+xl%}or?{kQY1zE&%4>w&iSLG2@uZ*%uv_hAc_3cu$-?v3{Q^YTXn|HT?`DBU@19}MHR7>x%! PnT5q1VQumN7$xyPR;<#3 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/kafkaio-wait-reader.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part2/kafkaio-wait-reader.png new file mode 100644 index 0000000000000000000000000000000000000000..57f49a1f0c6543708d399f24940807f8aaca63fb GIT binary patch literal 399857 zcmZ^L1z1$w8ZI%&AT@|e4UK>zjpWeXC`u^OLr8~KQn0D^Rfs3?tqAl)4zDK(^Y z*8q3p|NrNld+xdL@XVgQ*IIk6z1RA__kF)_5vHY~NPLs}CKeVJv9glf6D%w|4=gO4 z=inQ_nRlJsNZnU5Pt7AO?j=@-X*tf87uZsY`SlHB9cz=$u zuvD;V{&oBWoBi)HI9OOA)>xpw%RC1@uYZ()KcM#CpSW4r|86l0=bzGe9$C2m9OL|{ zxwejk1GNYomGoV(un5Vo|FE%A(kOxMr&~YObJbH<6E$?cnw<*JBan0V@8_EB~{m-oI<|3h?m!XVw3>^mkPl_w@_>j~DvaZ25B*m@X1G zVch?kdJ;FkGEC@UVM$>r%Sk`=z}`s5_k+v7yDUV@B{3?+Fn)|MHhyM)=k{YK;&rB? zEI;;uoY1V#1oD?S7Ps*pt0n2;p-uIk-o}=fCwqJwFE(w2M{OgSWcy-ZyUyKEch>jU zg<;6r)|gRWUH!)e_rm@!XTN`Z{90DS!UjV6K_ECXQeZGPBg6mrQcmK3q@QR8*Zcn! z_@^oqo2>2=_@8h8yH$Vva6j;j{@Fsj-T%DVziLYrFeLo~cOv&Td=cH>|LUMXu+)ne zFU;tcRR5o@@N_^RO=2B})!`ifHH5|kMxMubQDGV$_bYv|H`>$gD77(dg#~kY5 zzC5;ue~rWsl0*cW{EW+A`Uod3HsWp=F8lfU`FDX$yUmYJx$>WUjQ{rS+Y_&IhQ$8w zJ01KVKUOK>Y4QH`_>2v}t;glLxn`Zmd`#=>>-!`A{{BBkt!ICr+rX*3 zik`D+zG7oe#V2X4yMYmx622!7 zxbVrL30_mZ8=puePO`4Hz8}y07#WHC`B@*`u~oE1*0SdR_gwPVk>&yZhpV^ev01UwoC)>T-IAPLu2sF#U`z7$G;Q($mu)EhIph5GQ!a zJ>Ns<-mrhsD`ooSG9W(FP+%mPlDaA-Xg+6U9eJd-rWrX0xnvqwurV?d*FM)irzu25 z9(jLT8nE(2Q*~3P+JCctGu}eC+@T0tAgg3pOgFl`u>A@nx09^wtym2M&)k1g8dL2` zw6=P&<=&v0X!yqZ+v#%Z$Wo%wmK@i>$F!$gc!YQJ+9D_lf(R){K5l5oKa&4vb{S8S z`Hkn}daQq`VyzFutS*+81#K^6cx|n<_c{V zKcRx)a0W>c3A#*_y&8RxsC)d&-RP{x=<+RB&Ei|vUzKC|vzQ|x0(jIk85jNF^1uQt zT5Ri)OC(H?ob|ZyK%QsmFnhvfRFZ7E?`QIK!{We3g5+uKXZL>hx1+x?TE2^lS3YAe z-PNPLrqRQOUYkn|9g>&3Gb_=;)bBXRzevU!7ShS@XzZoFtoN=V|2BJdd3@o~_h>Ay zPjKqtCh$59V~@wpqASj~Y70UnG4n;0a&T_Z(0ONTQ+kwX+PAxZ8wwt6$cL(Kcow2j zw_%XC?(BEr2>sQGV&-kQ>A~gswx-xdiR_!p{mjgxDTfBoT2EDFZS4z|Z8gu2-64=9 zIG861YMs>ydaB0$%XLIO%=coz_o(~;xFh4?(a{;`Ho0c)=GWBYUHZchiZwrPkw10* z*Lsq|!)BGjTh>U&%}tA1Va*K{*^=1HsTBAMd*i;VKjfs+wsg9hL4HmhzG zsZmY6zQz(SVl#!_R>q3Hk3znq|HovFL^bNomY#3{@4H5$%)nzlNMob^q>WpH+h1;3_qh-lI^$4%rURb%c6OsNSx{JnrjE1@p7yuyUm+7aCTmAWXvrnh|a`44*CwFUR5{ zqGnRpHxUoha5Akdozg?l=jVk+7v1NF2irqOK9eIE35G{yiTRci`^jtbQxenfBtmh$ zzGqW5zy5x*SG{sp1D+xHfh7>`dyq??zds;3Ysr$-Cb^tsHmqv6_crC5$UB}NBg)}@ zCOHAlw0AOH40gXcyDv6;nqxN0W&7)G42lBg38ld}`%Ba3w|4#0lqJJr-^0R}<%y%b z^Tn@KirxY~g(n%zAggzO?B~p~&w(C+6t&e|-f>JDw{Pt7KxU@lP5?z-ZnVT3xi~^k zgBtaw(!D8LL<8ll+)~;wMbop-+I!C9Al(OP>Fpa%Z zL8k9Fo5^ew_mlaM^Np*Vpon0#_x)r^Pt}0AK`n*#(uiU68)af#!XOTwrMqQZ`oQUx4au3?Q>=`eVrS}Vb72J>I<>g#Xs^ixtVQoM2 zsxGcRrtb64Ddz~z{Km(iI!DfcDlBlIyr>TmR%6vwmd}TwmlnY*dq9Q;8yY=JrS24&Z@jq3uiCX$Mai%XDz8`_w0N4&YC7F=Eu{_FWNGehbqCBiYfrIMFTq`o{Rq=*R4 zuSyPx^C_Peczw(Io1ZnNaHrrRP_n_G39~<>0T;oMv;=n>vu=gp3N-j2sIn%C5*{9YV6AiAy6sR=R&*f4NrH!?mBE$(G@fbG++` za^vFIVCp2J|xP9Ky724h9e!h_goj>fO^Dbucz%Sk5 zzu_Z%!`F3hymiUO==}MlP4V|S{_0J^sWGS(z!CBd$d;Na;+Qfjb z+pmCN=*M#{Api^gg=77drHvn65(NlVxbjXpE+<>QXM@cX*cZA_l)*2>b?JR~o`ufQ zwwiWMbOTP2R#G&;!L#SZu#Mq9HG-scYG~+jp}8#1M4R57r|*hGlZyUnSPkmW;227v zt?KPC)M|F8e`@CF>mN@U?XzKKeZE=cp2Jo@25p&HZgt6~1csIr24(SE(iLWl+#EX>>gkrQZeOT+2mz4j4lg|nKSEHk!vI;(57K;j8 z{r>w?VOoLNNM)R+#6-hdYqoEv zJfg`neYLjN!>zfExH=z+Fg#$oD$1REEYOq?l1{k7)1ukcjo-`m^6D!SuB9KC_lb=TEngQsv+hne9maekBS+kL*+n z;e}_4)f0Dh7Q6m2HV~Y0LhQ5O2NP>!X2HZkQP-|2Z07pemKXA9$4`l{_TGOR?-+L>xVL`dFqyWsx~>y zxafuIQ&;1*t654|Tg{EuKmZslWHG%STe7(a=mw7%MNOd9S^dTv+l2Pm zr57Ev?grlg$U9sB0IG^9`&!TMbNa40RM4O6QYvW6A|b|iMt{v~3u+vV9cty&sUg>e zpGrooZWBrKH3L5O&r6J@{}@xjDckZ2t0AVVep#X__ovGZhf$f<4mzh~I4xwMm!30* z4g2ZaCnQt0@!sPmAtI$Tj4lYLVh`I^P2R|Vtx>T5M|rG$N0Ee*F%Rivz zwf)__52}4Y#-!U;;?(dA<)RTNWV>5x!-d<>kJ2!zZ@cmEX)uGkXVbOqx^u{p@9FzE&1=6M9y zH+G*nTzVydzgUSb&dCPj5qa`#_}l-RVDnS^n_fZ7RS&QwyeC6O(DYXxMwKoA85mgQ zQ-d!r-)>UvApK&HU{|U}cszP_`bzY8+GXS#${!%vU?y^(N2#D>)Z+ipcAi3r-|9so z;xzFfpWEmDA3l?LKE3j)&{d#Dtk5u@j{^k7zAZdh->}|^vzK{(N!_Pmd>z3L<|j^PZ&4dqQEO2LUf{J2^8cxT#b;Fu zpE|9&0)XnP$=72qM@Ymr=2B|C#SVYjZXOp8_t}HACL5PF<0UWvFg~jAKHWQLxH@mR z!+H5hWOLex`JTg11hZ3j^xuL2%J0lH%Is01PxIz(pF*7bexH!0*k*;csV3*vX3a`Q zy)*YtGU4ozh-ILVA=j&E+v>((4x{tQv4x`;wAG>&ECS0(TqmDu-Lq9lUvaOE+vt-@ zL6XguP--T-kde*)i&^aW|a!koK;MlmuD|GXWyop@q0~rBC2A2 z*LyiiomsBEAZ56elTmiIrecE(iAKmnIi2!`$IRBuRjRo8)QFfKr+yb99q&^;A&(eX zU4H>S_6(tyVdUYZLeEP>?q>S!q&sh}q3JRH5y0M_BaE4>URn^YCkG zd3(f1mtMU)`FLJhSTUnnSoiFmEbuBzA)yXt2LvTeh$*|O{QZ70htFAWxBzP;z%12p z>&3)8gS-B_V*qXo(|YeT)z{oyHy}<@%uz~lbeWJ}B#ScVI7(xTsdnxL7@QhNrmS0sf+*t4|-R$tW)@}fgK#bjP`|H=QqKi&FdstB? z_Tm^i%;7VS(5$!1+4a2kc2( z-R}%5jLOWz$SXT9rbm0Yp`DYJHWyxvLMGR!@ulCf|3=@-?>nb1>6%QuK%_Ek&-#scP^1cm8kS7k!X~dikAhycs!CjUxG?k&DKzjJ)9tI*r zjIVAjSX+HC{703Dn>qVg^&WtsLdT!>t>v&dD*;j}z7{~pwbO;Nlsun^GrW3V>Y?)d z&l&~!FC0IrKXuA<5VA-mk;Lq{0tkvFlnFX(S0Q-~vyw!>Qj~)saIC8?P2b@@KG=Tp zzyXmAXpn-K!VXq+jU}bPy>1}N4tk%1p&26rd!}wAQQj<0a@R?w%T|-wG~(()f-!iQ zqKHhwIEU}HL_h`O0mF+9=OM|KwB>c9t20-0l#uA@aq@t7d*ZLS%%SSg&@0+)55+Tp zLd{Wn0}n?70J7#$KS(5@5l-qAb)M6rRiYM?-#&k;0VFJPua{YJ(&FX;*y?t_R{;rf zqYe~)H%_^+tmB4C`e)M)1~O_0I$`K9lHX;4|1j) zcmoHmDkSz|qoKF@0`Le8m~T19H30jcheKm)t9 zC##OTuU1e5x?U21jdTLV1h$dD_F|)w$DNJ=T=Pgx`?Q#ebT@}+NjbRY7X^z2t)P++dP}(X;`BB=Xy$mzENGH&Bbe-cO4|;)TvI2 zGcL8IKZ5qLVKu+j=1#l!Ub_DpZJ2W?pPqjsSAzo^sDB;BYcCt}rVF*8o>z zC$~#n=OVeHwcvhE0-Hzj9dKXjEC5v2gc?qxbfH_)3WDbWREt468yrTb-L2g2u7VB6 z-?u&S^7)o}U;xn5I72qI@n+aXn(_8A(fN}{1?me}oDWI`x7w|Lur-WbA@>Fc@>~Eq zz~bbcx-=)2>e6xTIer#4t^9!jH@|7bufvv-U1XPP{!m}lVp zVX&mjAK0C0X z?P2Dd-yh@dZmUIos3g{+{4K4=8`q)=ziV@KaeN$1OZ+`Kk$iDW$pmuSl}n`k7B9#e zJOYDKf0^!Z04HHQeNl5=7XnP7&R4Mtn^qvZOPD5yq~#h73Dy_ zC~S1$^G#)p*^4EB{ngqCOP&oy7xkq*db6J>Hq7-5SWe57PtwU1Ub^vQey0lJ#?Xx;Gm+conM5kgMf z=rlUP-&mDg5gAav=3c+@n#}`RS(*SsoO1vyjN#*u@ah+@TPlGyfX17-YP=vIo&k7Z ziGnGw4bNkWC_xlh74%@t3WH=oQ~EA|$s(nq=smM@mP{ay252T!HEV|OO48L%bONU@ zrtZOD^?Nej7qOE;zvsKoCNxzkyX zR*YL*9J90c8JT`{W@bLUieViHE2LYa^z0DxT@b~%WbV|oXrh#N^?hkq=z<+bGtDed zf^r|Wxh_yC4tO2WRa1D<78;!Ny`00g$P=Rr;q^s6s7;dCa*B3I6>25-bl#^93eDY^ zFWAvb{Db-vq_C(uj6)puuGJC%Xk6SkNdfB<>$6ae-zwG*vSI`cf=z>{0uhte!e_?^ zLQ8;Wf#7^WN=0_Dle2KeJ&!Tpv=Dte8$3fi6K#?f<-*mRp?Gx<)!DId6aP#i5?AER z<6F0~9A$LL8UDRW%B$rviUue8VuKEsz3%7}Gt5Bj$~>DU?hMwG1ZGy=dB*DcxekiF zI~+RIk;f;TnZi2uNRdte_zkNIK(ECF0!{A#$q@JK>AYw(tO%WU?#LU@*xkq5?};jO zFE*swQN0W+H>Au!hbPSoMRSv;x?oLrw8b6=N1QD9PDco%S2GF`)rkdhlIXM*GDNGY zsL1gXBJ#0=Y~F?)$*X`hzv>d$v ztm-@8^QqZ0uj69_zNyB95FHmz%^d}K0JAi#-B|^Ap7j?K{+LvId=qMtrh{XftF4q` zh*LGglaH9EQkXd_bH=RdkUk^Z5xKy2?xhHoh*`*I0luq=NIDt(4n-XQ$F@6*+`~Ry zs)h@-x@AzLfcPOQKdz^4wX8ZP?qf5alF zSv*%#1e%>_J=;jf;;(+baCm)-8oIYf-p^Yiz`Q-pj7oK=7r6k4?aqY67G7}L3Q8a} zkRDYsUiAGmA@!v(vN6AZ7yaFlh5qz)d|))YNr_W0sfhx0wJ;m!LRg*l&qk#84GZs1 zX2WP>q+})#P8$ZIJcj^l75}J%nzqrvtQ@}v7>6Ik>NRLj*+?{b<7cn=J!cVfmqN3w zJp%uU04XDc(b-;N!$#rTJ=lm);7+KoHPl5?l;RIAu>i3Svq`hwRW4^R7(4vD&wJmi zg28O}#ekb#q-s-4=|c-BRiNZfBQ6($s-xXQw3#{el#I=bdVUT&lmpYMH%V`S;JN6$ zX{)1~o$LjK&^%yB_y#q~BkT%@O}P~L6W7i`!#(6hPkpK>PT~;;SESJ>%Xoig$%Um5 zwCD-4I{G~?>;~m++C{JG&_J8^lR3AZFI6nVq%%P$fntsH-o(c{q>rP}P=k^xwfdT1 zX0YOkV;U)CGJ#>W)+!EubF-`#zEYr(xZ-#yy<~jgonJc=pWk=&JR{P~80z-{q~7A% zL~Y|x8HSY~aUUcdpT^CX=w1MV--Nom)=x~jngWLZJ}RNANmV{C$H1`(uimM9_?{{) z;(>K3qgqFV^lDbwCGY};FD{Q8+`s5{irVJzy$I@q2)>GFClXO%&NtYQ=(K-(4J%%f z$DJc(+i01*;?xDH%9UF2C9Xq@BfykN9sJ=lk~*_U(|H=~{48*>^?L`M*U4Mo&b_PJ zxvODpe5jU?FA-ceDZ_iOOM*NOAtzG;i3#>P7Izn$4;7U#zkl>v59Qxy*o_kPiBI=t zn(OT0?SP_LnJ>>9uBL=sWMi0(L+PbU08D-_qm;~WNV;0)_w~}zpfK_-jr-`W*BG!z z@7&3_Mm69OY_uYSI;gQIP~12x;b4HLK|9IR+M8$jeCcTu!#v2IpLGBTgei5Sune>tY|5 zPY;y{5uI##c=Y*@VLa{@u8wdQJ*;1Ul$u^XkU>4o{oD3U7`!lLb=%)&KZCcQak$^e zoff?bOjxW7`Wr13BguoAnVi-4Gye#;?cFgY(itYdmnj-ff>NdY%+035h@Z07k^+Fd zVqZh7w)qJI1&Gtk`e?f8A+s}O9QgC?YjT5LF4`Fwr)}dy@uEVE1E=9?sKB1AhBitn zxwtOk=ABuedE-ISn&6MqUKI+t!-pQ*oq<( z)ln5xBP%da>J}ZqqV4gMFe@ZpMeJ&&CX}9+(<>5*gVe!1ZQK@|6YNAXDRn~~y|k?l zr+)xw3tvcCipqn+7G!ehvA;)(^JP*w!kP()XjbG5I#fg3XZ`NCS;RaocN&np3gon8UABL_2TE3uJuALC<1W^#GdHvV2Pmsp7z|-B zE0UTI={>GBz&u(OMx2Q5>tH4?t=C3ry;^)H83SXh6)YwM5PZVj9eLH%g+=R<zeXe5%BW|p#H+KQ`6^308uOu|itKM^F$qgPVAF+s1LJC+q~u+@Q5JC}i~z#to6T|2 z+VQD7y%oFm$(1b8b2o-iXtT!iS8~F;B&S*YXa&P-ig+!QL9oqPP^f#8Or7uqSNQ_h z!GeC^N|FX>+`w)?{P;k6S8a7blkKj`i)+?0NQ4tf6G(1s>j@yT^LpG!GAeS}g;JGx z-ZPq+onWK|_hoK0vVp%}xP@R))lT+#@1oQP84`X8$+Oj5@?$E%j>a=LFp#h5lCinH zo|sN>;x}{fz}i|NaW1v=iRMU@p3GSUbVcVC=TLs(8_gGhJKeSEgmqJS_M-`}*wwSu zZagL+#Y&XXpTnQCxDM|9T*B2-3#7cCzJnlLMy|59n zdhKC$|MCQL{0ZI$hvP6Vq|-Lw;JM_7Q#ohsXMeR!c9oIJry!4fCOwp;*SuKYxB zq12XN5pJUe*WTvN1j5;H@&I_bh_;~rnv;IZLGC7#{#O+vLB1+mna<2q3+ssrr(;CP?}PjE3>vpRHsZ(4;-7NNZLXFN(aIsaPq9BM9i2f%m|65T4HTj?6Xkp zVNL2e!rm8EpTMQiw2hZ+4VgWeP;5mDq}CZql!Uqqs0h6Iqxf2hfMTK<3-3c`K~s)j zFo2V7SnB?S10WZq)WJ)pfdi9O#R)7id(WvT!L%xL3Vu4UJOG9R@;CUow1?sq*a8^e z&TcxHPm8(PR%WTNFQt1kClA+h2!EdjePT0iQv3Vah+T=a@cZz0iJUKVXbSj!k#Rf= zgPF-Xwwy$KsZO6`vt|mKxt%87>%y6Xd*1Ur6C5rVEJE$Fo7q%DszD3wRBONC{P(yU zP);xkNA|p?hBYtm9q;emViiw2~Q}h*%k`6k^)lq zg^lfRy3j>hE@I=8HVuUCP)AD}C6^d?i?t!*`BBKeff^K_6N#>3-B-z&2Ewg77cLG= zDCf75+B57^3Jw{T3(T{zI#9N^{c2!Bj@dmD=Uc<&hCvY&RKWU>ZnF-aYm>x@2#(U4 zOB5?NSq~uI;|5Xs+9tHn;K+y=2-E}0XB&;}jN=Y@z|AMgdy_$0Z^k+q}$?9%8=Yxiqd5&-;>bC5?}-95|hu1ZwsxQd3*s48M2)+ zDJfK-t5wDwr$th1tCUfV_xrK)h8Yh1UF3_x>8~s;nY(PwqxO-G zx7NsXF>*U7tOR=?QGsWLOv+*}{W?A?v{DMek}>{92A0R)!>(y;Oz{`+VKF#xT38i2 zRQzM61Hj0%2(#OzQ~?wXkq%+Zw|fJuSR^udi{wkKW_^xnWsuTpDerA2D#U*o<*6FH zKV4rmj@*`YOQi}dOD1DaP59UmltUS}>Iiy&e?y)DhVqV)0HU0D-e6~X%akT~r)qrq z92pbQfYwF;d&OvV8Urv+$!Q7Xl*|>BeDj;cM2fzL7yBcVCstJC>AJ}IO%q2mV~~TaFgrtcKH!r z`SKDVJO+53u-5=_%u7u1&SS+lPLb#3hvR#!)LIUf)?$KMGL^p`tVB9}s5L8Yr+#9$ zqGG#~vxj<7VHL|wY}^(2i2XRkC^dO+guZKgL1MkvCX=@%-Rinys7-H23mdFhP@oJRkJ~QG@FK`cG ziqMDmut?Jg^g9N@dR5;8lZl4g{KQc1ay6yew)CP^Vu3!3Z&*n}DQab5LefvP&++M* zXK3AR&!D@Xm|v;Vhq5g&X^zje9kjq3>U&34NatHjQbY1@3wO8kH_MB(%zB_aMbQMs zJby$19ce!YT#LYyubtOH>F?n2RxeK60~71Sht~NG!MRs&uU=)mW&-^|5RX8npDB8d zYTDJ9%=DOV_^5R!d7K9J{Q7K?^pIz70Nt4I;8I^STEBhPX!ktjzD)bn#$mt&7G};g zOx(=7y)ZA3SYJ&gg-MSOk5!05YP^$Xe*7W7x#)oDv_?aulZ3UA&*5msETv-Vjlz(o zJQlP&)LVLpOxF?DkyChhq1LoDcj_G+s!3Jyh2|8DaJ-MSRI_KP3<-${yJR}wHs10l zo|{(*NAZd|wWvZ!Y0B|PrKfjbd79r!lWABL=N}oKG*f&F!)u-gW}9`n;w=m1ru5^m zKsIdVY9c5rFOyFZfof)=C^l&+ps0NlH#lM2!%Ap6ZhUO45a;@l9Jxfr*PPDH3>{~ANgeep>6f`kKL7?JbJQXaWl9au6lNA{K}3-pB$Bh>lou)hm8kLT+fHPxRqk~9&FP_m!ns5aIC z4YAEdoI*pW)4+1qlYtaEinQ?_X+I_d$lcbCT0#>PgJaW({)94_e;`(?1u+UGxg^Uj zMPz1`=B{o4euNZ*o8xzpoZ}M^*(av>Fm$>KdZ6i|t}Lefh_NXGZ!ECT*&Z)2SUK%- zUF4jFctKjhuO&o9iUjvp>7!mmu3mJm(K|`2Qa@I}$hgHx&^IY@FRX|x^^QcB1=0T% zX_GN_hRk_24(Lo&8_Z1Drd_9H@Q8yKFWfgkWdy_*Y-iqoGCurkBBdgg(ZbYsBiR@i z1rO|ZobHvq<6T*5_zrvN-PKlKOze$K@+4+gB=rlg#S%4FfRn`6=bDulwhXr!U(T}% zn}#Gf!O*UPDrU2Z4Ek%@FEJ4j=@ATE!URfB%tfHMd0Z>IX14vLBJ|S)rwr;x75)*% zX0938NpMzr?<>#-b}rEkL;!B0x7wCaA%a0;LJ`0Ao6tnC*-6l1J`rRPP*yxSH^CIb zxr2Z*7dd3cbZhUexbGZ9r`>R$;)_} zw4+gsAp_;+WeN>z7>V3z3%1)BCC89?Q%-|CE8ex*bBj2>hs_aPHPxDG-cz!yCglIV zX5ExMw}Xof<#LQVCO_V1-lfRgrxj8+&4lo1TWM@Px%|N4c@mZuCn))wX(`Bvd0>ar zu=vYNwvIrx2hADlgsRX@mZzb^Po$a^TIQ7K zma7Idtv>p;30k&<3YD9Ku`GsdyV3~V7nr%3MfR{`)hD`yPW>q58J-B1uc6nNT*)ki zK{vTT1eB#sNraEiwT2vfHs2kq8g|LXRL~7x?vygVv>+amQt(PiR(4eM+l^(O^$YV$LxR1Ty+7c$(px*SsFx5t zbY|4Mofcalyq6~I3`8(eoccSSY5a-JjFCwhmL=ow&2Ge>ThO@Bk^@;XY7{sJv8#o1 z0PMTGJ#!UNugYu0{EPc+r1B4s`pT#k@4%}Q>D=mqa>|elRQ0oMo1lZ$$5N*ebQ@XC zm5BzdbQ)BLQbxp)z;2+Q3EoiDey2XEWhi5Pvp(QmorTd)xa(M5_XE9QlSrbqZ?6s? zuxU}X$YiwZQ8kO~-9DyF+-JV14#{jG5qnrV#F|=EPL;BGFTNhgDLEDLcfd9CY#-O4 zrnyo5MmWvqXku+KPV>l@4j$sg8%GAX)A4%w%59Y5lM8YK5DO0p39$4>7iq>O9w}8x ze=gqCUw__X`}T%VTQwaW4hfz;GuSADcLv`En#X_Ak}Rh`XU%}{)Wmr7m#0Viu|b9X zIChaaRK8j}frD+)`|NI{h5-k=suNwLbwnth(*Xya?Ks0lwk?Wi7#nw9NHZPHX!%=c zf5>r28N=>=zcA{K5$|Pb5pC+%n~HLDKx|fM)+&nlD9La2Rk1Pb2`nx>Gj=%n^LgMS z`!drs>0K=olNBRsdD*ib z)IiS?t05Xzd_(L?+qb3|-v;BFoSls|NC4?)hIFAfK#W~pp+rQuqld?s$5B z>8&G|csOk)NUB?0%6$8@nha&;{b5HlIW#a?NP^vau@Qg6(O51mmIUOmh0}sD^fly` z1cKUsWvWV{!3^5&GuD=4Bm{BdQ6UMM0OMdK_xIlEmKqYA`tp#llI!+Ga~(n?dF=rg za1Gh0rs(IyhAK5>=F+J!L|hzRr=7Viv6Vkd1+vH!g&VL+71JW0>01+J2ILxn7?<;s zqV|n|%!AXnzPHP4bNR-y87 zxwx4-s_8z%cGoNd(}7@_((D`&@T%*wE&O)GQz#aWIyhY-iF!Fz(nnxd4Vjwm?Xt!Q zhnUo1IW?ExemN(;QNQ#hgW0ZGs?!(9#&Nv7ArEtGT*LuAf)e}AvFnkk*EG>Lr*G2( zIY6KIcKn_$Q=2B7%ri7nbcit%9r~ZcLezq^LX0oM@2|}>M9tGGQFsT&vuf}Jd$jMS z8Ktc#HcT{TGC0_|{BZ5%?Wh(}>2=9U5`M>p*nWKDMDV1Np_0ZvqaV)GI!>+@$+qcU(!uD z*s;RxI=gp_r-QUQK%8C-#B!RYW*pgw4n%61DhE+{71tuMt{~Va*|u|yq>P(~gGlyf zYP+Q&gSsgXdjBpf6IQgo1g;3x?tPL!lf+iR&w+4Ocgod8kV4jz0*5mY%SW}>bx1Nt za?-Huj?#0_AAH-octE0<@ozG}-j<0Q3--6jIDf{?`jlzL^IAkXv75QvZl@$%LG6ml z+0078mBpc;l-dw_-)fstXY*d7rigZKaz}`Fq~$z>XB&&c?=BmNpQM~nSV?HuOie|i zi?l;n@rd6-@na_$eE3V8-kb0&x% zqp_(mvam%$oWzQm^O@uKE0^akAv>ar@3vovcf4|Kmc0y+AH#$nabn1hJ$2)0xzLJ| zXR+uTo~mhen(?O$VT=g_a~5sk15_b@P-f zMOu3;@u1m*xyC&nZBG;x8BdNzz>6XV;dg8HpQ}T@{)~F}TR{as;5Mhe0OLp4)A8iF zoa|U3YRDpcRAl$S=bN-T4^lV3t(r4l=90E*WmA}=Bs(cR2#@zrV^OumQZyW7GI$JRt-^tGDo zQw}HOgREYTk)L?CN&wMnwyCw@o@jdC)(%^;?d5QeB1h*5r!sd9x|7|@Qn1%3*dt3{$;_z@+N1M}ewVw|OJM6j99;8BwRoVd;;T!#38kv5^byc|rF zxaS@}+INPoOdJ?0TwFS;@A4g1C0_lkc}FREi{5|`{W3mY~wKSw|q5BzLV$SDii zu418PO=ptoW>Oz>FwdWsR1qeI+QEgME`=LME=?f7=att9@HQWEL?W5#xTnBzh{*GK zCQ6&~2NPY3zms5l;Z$mllu-kr6|kWVQkc?KKp>M*(nSt5FM?G-*q<%QkC=Ke>2==p zT$>NHx=bbIPQN2}yuamQ;%GNC>}q}Xa!=w$-ap};H|{OI-+~vjPp! zx1|ZYdero*q5RZrTxxX21O@*-aGdjMnB@AED2mkW(Uu!%t=*d_TuBm7%tdY?ue!&r~OFPcB;%))DN5&2`03>5vgDQB@4H`_BD zQ`g4}F)~@9ScRjjx}{VkBz7#@1Ccy_+`(giK`$=GMvqwDS1}%*6S3)$V~kYuN?IB+ zH8(pRnYkSun@rQ7W!irs{XNX@aH+2IzOG)#z9XTry|L|~y`0jl(qhaE=V{l(V4R{<^r&c?L-L&n!*(2!2|Y0nVyFEIy>TJhK;RK^O^!ZZ456ll9;YS=;t2owiw*|Otz>VMk6 zX`RE^`2LAD+1l+VyjF5Rt|^Ov0QG(A6%Ui=^m1|51;u-3l6)`OJFZ4Wbu?%($1D`bRq3{G@wNP z#n3{Vft`#CNdB%mXB0<1NFi^PzI1tV>n(LQa&Q7j@_`W{*o~^KV0r$8&_TL(Hy#|? zasn_6W9pt5-F&@!svowm#o7FA^zrpf~{zm0-iwnY21kU*QpW@a+$I z0OLf`=FT>cm#&(6W6~}%&~EMx7n=?R*SiDjyQn!L=E*7Jwn{tbiVnkS@T$QX^OUt^ zGSuYJN%zb#5oU)x;Wj`zXJM+iltP>YT5k^|+3cBg%TEtp?$%!I;JcsdLw=xD%_O15 z!AD}5rY;QiDGLpRnYqQjD^Jn~+Kg4C2u7#{80ryYP^WT(!A?04jJuU2tdn zSH~eTp(X!g0(rk_P4C1Ve94waW_AM3jKWQKH{m4N6f{L&q)(VQYIJ-x4LOU z$}~{KWIz~A#J0?P4tDA$#aG1k><%~SCh*;1UZl|2;CO#Qc)Yh8jhH5Q-5XPyL!B?v zO?(>3da$2y42vec47$i{mcAJh@#gjzGwnys9hN#V4-PdOBs0SQ-swHxUJEH5aztJE zjwF>L*rJ+#hjkOkz);@V%gHer`T*=z;AcCw+MsF4W(*HzD`(3ULCz?%r7y}rDqn}H zBSFpNUIM+J2gmQrf-SqHP6q|xB6#z@faUgLRDL2S4mvleCwyjfy#TruV&7wq+Erm&%$-+Rg$2A$2dr?wB#w$L|L}VFBiCLd( zB4BjRtBh+ywGxv-G}~$UcZC=H4lGVvuQMShA2NI9_f9iMi)~cP!WF(eAJ`X25$`DC z*-nCU{rKKkKH;(WXB)`iD{%Es887M_yWv;xr=G}0TWP;b0?CRN1E*}2t+!XJMtuoo z?0-Vv{vV|nZo3Z-o5T9Xy;vxa%yRT-S7E%3w|q0fCGbj~)DPMBZeBqQKzYn)M11+U z@rBpyd4@xEe>H{tBf-G{e3R*L+hvI|8nN#oQ0GvWJJNbN=>-1x^PMVVgxqz5k3<>6 z6YZ*cYp@n3ft*+yv!@HPFDN_4@AUh@nqS6{SnuURtHqC}M=~F?dVtQ{AYqPV@f6Lm z&#ED{i`X;%QFHgjptnHIL%hqDk{Q5WZ(_8>4x!=oPFNrQ^Ko~4U#zNukz=Q~33tZg zDe(ADAo0s{?3Af3YsdgP6Lt!3JWq=T=8{$emcn5A>MQ>W$Dw9ql%efpfY?oQ;f2Pf zS*1|VYW(hA6-PDM0Q}b>KG|A+ftcXpCa27m&#%xIcwGbKFvB86=a}&w<+-pP-L}Vq z26?!;cyx6yep7Eh+*_byo7emS=dr6Gii_D$h{)-avJ;t1375(@z7e3{cP44hbjR{@ zqVUgsg&8852r4#eAr`oB7FV;ngi|k69?q5tK`jPVv*Y)iIX!h%;&kwgp!-+Q;aA1sdT() z1EmPBv{}F6BojYy9SsSsvKt+m2Nqk7VZ8qhA^}k+yX_i&XXxv^#hK`oSrJqo%1K^q z=gF8rH@Na~73~U!as#b>Gl;h%K@g+QpMm7kkxu290(BK*E5kFV%5hIQ-4}p^GGYmT zkp6s`zxOIawnyFx4rvzWb8X{g1=1n9?PztAl8P)7?^)M6a-{!YXblN&N~An9^bd|O zGwVe2W?#3aCO!Yl)1P4eYGsZ>=G$ZJn>u#ncdm67zrPDW56g8pi3mx>mQ2v1JN(c&K%7$0eR6m z8RYdSLx5InIQfl@1EAX5+5I-C9U)}A*;pQtFPNGZ09+4r?cSvH&MK354u;c0RSHF- z5gz=QxC&#-7QU0R^SH`_7j(hc-!GkBA?$i8SpKbUHKiQwryvc87f-sMjN-lAeR0+( zu^aFaMbi3ZmYT=`&fpvApq%bz$YZpA3}Uejn?SiQ;Ryl>w3IizSVIQ_b4J;UZMCa^ zbq}eglJld4=5e+tyj=vJw7Qd+1)!IFna10;S>u#~rMsg^tI@d70eTU&`Bct(^>Z+c zcZfabeJYjrptxo^s_=JLyKXVeQGmUv?Q{cSafdD%!m6W;jR^kfv)BC=tKm*4Z%KmlNK#^li z7Q1snn$9lLzWVfhCEHwvW=NxI09NGocUYfe)VL!6Iqosz2sTALXNQ zh8>`O7z_r840bDrAC$ETvjO0=!eDk5(Rp0O=;7SPE~Ghd3zVq)Jz!aw2iI^hZ^if$ z2D;E`n9o;n?c;gx*djFI zUJrFF`e=CvFCyqNO0xk&cJ4Q?VYEgWn z40orunWrDedRUd`leRt^L9rttxj@&ieumGR*q^w7ocn6P}dH_EZ-^*4W+P#+0Hj3C}VA+G$Qpy|L#wY3g?KS{YeEE;SMaNOpEd z*s--u+-qE=%{0Zw``zM4IHVj_?(A*u(KVr7ipte^eElplqH6~o(KnEHgVI{pVbA=jb)-DG_0G*3OCHq<3hv`3OzAh2?e zph?3h89w==FrvxkN$>E5xU7iIIbRezeevI0=f4yJe=Q9^WyGT4$z&G94)Wp-K~K27 zsyc7C0e@8Hq+m^yAnpnW^gLVVz1;Y`KQ2@^IPM=}A-usW4Z%9uCV?;gK4F2$h0ofP z;$8A@b?%b%;9=KBtxRlu${-HH&$k;(toN_Nz11Ns*4=PDl72n-uZPb@1th=Xkxz^Q z!+!N4#Y<0v)lg$}{_M5>k^%k8g#6b9At6*fjJ$p8f!>Mit82}j|6-V9>W`W|xz7*- zBe&|+<`}V8=JQk(-AMYl3g=bDS?zK6@p5p47*_n$deeo;-iCH{WPPIeC9^QhOO3@l zSnC*I4V5|A>vhQK{)Y-uRN6RbI#hmC^=eIfV9xmC7dol`Q7!?)8^Iw$D!>Cs?lI$T zm7I+EAEd{haSHW!X%p_?Uu8|Jjyw6_%j7GAVsuPogV3^NF3;{iX0z)RpO?!Yik=jD zZdkD5-nNlLukI%1?#meG<`3W1KVV&__w_tMI8*?-s$DNg?#+ zl1py2IJ%E-GVvr2Ci<5=%3A*5LZaujFm;*^-wkX#;qgW2iJjpZQm9}6eFMV(qga3C zM*c2iyCytr#G92c^`)- zCxUe(kLslg6ADzxEwGYoFN6$V&SB0>cCjR~A?$2@4|l(0V4AxsuTDgX$`$I9UH*H~#xmtPr$>D}v?*A=P%%-mF&? zRPA}jlj?Cr@>e@o*e`ctV{}FvGhDuPv*x!3e|4YU6nPB?tJjQ zO1H|~e-$p9l6B%az7w^l4*Xnnis{Vjto4Fmss&~5n~=_<9V#gA zU3wZu9tns`=fUC6{jW>+>x~sNzODa)lx6sdE}IUI5%$oaMyE4)0^m0jwsyk4r#(I0 zG!Nf346fio^8xSN0Gau1&zwn$cgg7R(*7=z&4Go(;)Mmd)#P(s)40a$PKM6zZ?i=J59(2DtW&=TvpJlqQA&)g0-HZWwB`d$A z%U%wTUH;Cg~&W3mlYX6KO$&e^KF z{ZB9TwWCCP3JNA`q_bPPjCxk!Q)ArEWSg^@>p278K8Qwhcw07pLHP6d#Zv8?6KHp{ zX@kYAv@SWd3%A9}{RtUKR}$7|(7~%q^DOoAhGb98O4#Yi3(Ym`7h*Ku406TBx4S`( zD6%+j#}4=IuL_0MPg$SDvRo87juby%Zn|AO?(j(RrkmZOXMt|7#iw5X3gqDT>-&)> zGI2_2T(kzkRPf1KBOKj@C!tAt7VnCPED5yXN%EdCXc*x~ndyh%S{a!lnf-37^!jz^ zcEs{@Y=Nh?sTu@KM*5^SClL(Y0`|fngCVJbFf}K zC{&%ettM6hoELVc>9h#6i<>R0dNrup%~CKd?trD45iyAwnrF`imFxGPqVDAoMBdqU zj-h{fDl||8>O2gb>yX=y#E@*TMazIEo%rW3v2W6=EseJ>&2@ zfs=}m#^l-I#BIc zE^01wLjKSq+nenChCR({4qbcD$BV2AwKoH9O(a<2$_iyV-4n=bA!=&svVWKs&o6+B zhj#G_5A>U!ZTXN}s&G^{ZoEX%)HWU{rYCvf+{pXEYM@t!E3sD`*FUeF)#?tKsq*K8 z3VBtq!CyiIGT1z%KPU=?tX&9hx`Gxp*4d&&#%xfSKH`^Z0jdxj>!2$p`VVC{@`w1F zOaShtT5lP8@NMx$^8Ud|m9PTUv-3gQX%QN| zR#H^N?oMP_5}`14VdzvZiou=~a$p3xeQp~N*6}hmNxtnDS8|eRbBHCYCzB8=r}Og? z1WmcwKL1=}&C-lB*k`0I9lni##NmA#Ir}^CcpCq39MX!8{I^>C%%O^uMwN&4BhYRw zXRdXx#S-<&m+^}1usL1g`i#2+*I8pfgS{~$sUhv2qrJCInK!rM5 z)@d0N`l`n9sgKh(rIe4t#m_kSu!GNk|4MoOXI^6p6U}VXVk*LnpJ|myzTvSdj+@nK z>xN%^u9+>a_4QkjY4z$Ysh1m7u>2+xK3yn47BU`tIv-Z0NAU?E78+VLfZ&x0KEgCl2f`USq$f=juimq#u(EDrVmbvDj(AgrYn$=ELX>%R}W;Ri9dBWKROU)5gv%nT0^APDW%xg4@U6S z`9agatR$mn6nvYySW{nghSr+3B=E^SiXTRsXl(!yL0n6i2@9J#c8e015PxU|KZ5Wfk zPKA|`Rv$=Cur!70BI@l3uY9s7nR5M zPR?38+mFDCfjMmjnhvoqA(d&q56UaQbyRd#gn*6aCi|?A#wB3>fOcq)LQ$pvSb#eJ zyBZW<@TbBC&k_5hyw8$SWL;&YeQ35$q@&-G2VX8*Vb@kTpM?{kcz{lbM@8J@Ypv#W zj!teg?Zv|2UtKlaDxOs(X7!XcH;mOLqNsA_1I*htn8^$Wy)Fdhl#&(+(FDc3IpnZ z>&Gtfbx>Zm8SF_`38Jpp=3p*L@Ifrv71 z=X=Kr5IM3V`48ROfxNRzsqS-2hE?r1xuvy1@>t;E@~C#^gw!{M2sDaZ0#c%G01ExT zUsPLt%Al>QAKlc!`i}D3iCOb#97GJ&OKUQfum?H)LSLcMc14%_daH z!Uyw97HonO{*Z|Ny*+CWh9CDA-KZV8<}t4d(}gr0jS4~pL1883x;{1w;|}?B#X?_m zj`{0KLMrHbU9cTIz}v>o5!mr2l>AE(PSW(X7JE~gX#BK9OKK0opZo3ndF||qb&;>5 z*1g$uXAE!@dzwFyH{TT1YtS^iq%c13S11SauQjMq4KDnQzN)#UO#gmbV?QH9TBh|( zX}qO*YCySp^q|h*rbqGsWmya33%xut%<} ztHj8(U)jR8ORzTzVxTQ$vmUw*+t*7SEQ-s&3mLZRS}z#^o^&xLGDBP^uZ?_pQQ~;c zvo`|%(2EV>DgDs0W+D_9N$k5(hay0G6;aqnxm)H>Pa1jQ$Tzvq&&aKU4PDx@I^gQ@ z!(BP|;0Qn36<=3S zht3cJiDP<>nrq!Eaz4gCw(hTae*lC3+Il#^`pUlew@I?h zvkq~W19!JK!xKj>O}i{9DXofU=N!@#Em%vIP&i;~+OLJ!2xD@gRJsj4alA0$qjoKA zy93gQEtd$6XYNUIWQ}GhZzzjbhMpGO>a|^))?ks=y3k2W-C1sQ+#+*rc=9McH%Bfo zw<3l|{2q4_KErv-iqH>TPOqqUwoxbit|XHNp_LB&Hj7>y9&12>^R>*P>I@RB=0ahf zJTwdfe=sm^J8%Cz-L$~zK36OUN}gA@yr^SyYwV^-GM#{oNpDx$8GQE95)@X%83-sw z;!E6V+m@p7eQgL)qm;f5MN7|uVrUJKuh@D^gZjRRx(+UC+8!sdK+_d@_5||%+R6KB zS6{6NMwV)nZ|LYSwI66LIKpz1Xp$FMH9(r`6?!2hvyoXH=-(Dq9k9At@<{A=NOK+~ z*90+`7Ss6Rzn{Z0Qv%DiUV4I|SJt1-^j{Hcz$e^`+(%T294*1*D-q5b?IxyWk{hS0 zJM&o|HbAtlkHvW|flYstg+Qs}J9@VQw-Vrc8Ey2RhtdiSho;xZ^BFY|PV>g0yorZ# z1C_qtO)5KC#3~DpaQ;4W$%fnltrO&AcrwYwn+8o|nu_-LMRzkF|H<`K^o37Rt$HTrJMjAGqC;x;l+{JdqOhJbVw`o7V^j ze3;5|bh-^gTfvd#2rQ4PK4x$ItwXOdkNCqUo;Rc)wzNX0DtCF0xbg9I zS2`D5OImo4Z?|J^>k6xn?+UA`M{%Sc!8jkS4B2PZ4V;t1mU|$O3ucwkii{k>CS%No zo()0#_mbXxq|`9=bZ~!G3pFIF7CzHFaT_tik}CO$#?mIq$h9b9QY%Y9Q2% zq3_PTj|98_qs1tEDZ=L#1gC)+1F8+3oVJ*3|9MsDGa^=K30Yd4jRu*6D#wLgw0`*u z)Z+kV{?b+bH93tB!mIASyY-W`$$-EqSb7!DzKlE7#@woV+(yz}W|acO=z_!XR>4ko zZ%ullUrHVeAwM!PF*d4)Pjm%G%2PsmG!nnRLjJ!HBi+px(MPQQ^kYE)W6wFUYsV*d zrq@{NAR77%e=gHgN8*i?q)pLEh+U%xyJDl+;X17F202xS= zD-wqhJg_}(6DV8{32pvbum} z_fB0e_1?pp?NeP}S|&utsV2IfZ8`huqixUGY)+5C5AWeNmDFX`J`o_)*6SegZG>k% z{#g-kxto*+qJ3|_vT*$Mg1Cza}Mr!Kw%$kB*H?ee8)zPGF0$(&?* zwdkSFBiY*QuauKt74It|a@XuILdgQ-=Wt8sxhmP>-hd1Lc_YG#i-z|Bb@yF0#A2xW zd35?_d}T^>3{+U;Rf3$l`(aqH4-bv!0n48LZuU>ADi7V4e4})5{yqC_H3kTiGrYEP z5epvjd)w3kAru*iuUu!8GQIAMuT5^V(Ttonh-It$O{;OMn)?dG%0Ay~iHHmN>4N3x zSrb6FCA!Ve5XvZd3Pz_`=<|$i=1C~w+fjf$sW;%m4ngP}BSNNbk_Q^_TrJ1CAGZvK zUa#0_H(PqPr7t7v6*>&WnfL6&O`QkL<^+-KRyA5~S{bTTQ$XodjabS|xYBQybA|eH zbKI}QZI2wty@HEAFzK6x`G?E4F>z!?fsx>yb1k@4`->VhwN<&*?iMFJsW<&(tU0qz zMU1k6)2V%gDs8bBPoyA>W6UbD(%sC7lOV|O^j)FV`uYKv+XGKp^W zDs)&&;`Dj`9Je_s`|XTMPg?#jme60*a=Vq-n;wGd-i4tAlX?#1;--a5qhn5QLw`He ziH`|iN$ohl^b)j*zm)H@1TPuIP%fo&z}iCPQwaYdOOZ*vNd%sUIjcl70fQDpSFZ6h z7gl&gO1LduZkFo!qy61(*LFgCR%88i{3cCu`Mc*k?~cqdH!SMFo=W?NnwAiS68R&u z!v%!l%EU3tDmI+~EUc&oFK+zO>?r`Gp4g(US{#=GUY+p>*hm zG&Vr2M)skA8)qG#)vxA8_<(gWt+ApmSv^q-NW`yllNX|*sAzEC$Hbs)a8u2f8kMyxU6r)m#(p}E`CCp|M=Vn?=~*S$YQg0Seocm+XD%%z z*Zp7*Uiz2K=oOs%aq#qGsLa>&ff{I4(K^X#mw7~kRv_1lJPhG+HJ=-R`}Z|vUrI)LObRW= zpQTcJujDs743Ww22#mSj9LZm5XI5R7rlQparxRywG_8o@(aYI<*=2D;U{Ai;X?qC~ zOYSDApPtW{KkYY7MW@%REVy5|)izcN#LB;~>`ME{jwuqV+LUN1{p4JGTq?WFFQ5#r zRWz@Pjo-|!G{6>)g4hH8qz@PalDv+Snk0{{>u30p>$Ja{G?Yg*0c~&}sEa(}@;x-1 z@|rY8D5oyO}rDU+O=BR4ZX~DEobAmzd1g@R3kW8YLtAI!v+0fzU{b~I^ucF z+3tXTIWKz~YFmA<395mi&V(E=a?q=t)Pxn>?M@s-pv_CelU3VccN{r;9MT`(XY>2W z&fE*cwef#ssFS?6RRe+~Ur+cT!IIUtI#AIIZ!b&wT*$4Y<`(gNkqb#j=m`JBSarn( zzvEp4#5&r%9wX0U-9hsAEL(e|k4HYKz=MYscnhYM;~*?=$npmF&?y6KLqU@9Gl@*6njqOx&daJt6BBHS74fM}fETG$|HWXH#sq%IT)&o<+8t$h?=JevN z8@)^#uX~XT%WUix>_bKr?(Ce{#n58OyC+mJ2ZY6yS9Vb8kS*iHXUuJ1$+OAQA#*Iu2pvjS`Jb-Gh_ zDJSIU+Naqec5d#JhYyS=|B47xWN8yo87+Es=5VqWL(o(G1e=CALZwB7>*M5eRV;4? zf)a!A>q5>0Qhvt{85O5u`aGxgBaRjAX+K$%UH^uFZR}wX+4tlb%d6@$>bX7B|0 z+9vpki#A%(Vz*YWYYGB66-28#Wrc3M!^!Z)f-2 z#5AoPi+Q_Gj3=FJeP3Q}`yd`&0EIdPx{ZBXgr()HRUq<>=b@G_iNp&Z z{T$X$z#FxSRs-;`C$LDGFnb>D-Y=|{NIbc}z<7IyoJ|y}>P6ogqAWOsep!eN$>y4C zwOlE3A7v{$FmO;{H&Jia8*kBR@#p^H`rzQ@MRgFv$jWtFTkYAiUu{-ZQoh1f%-a()&A8 zjQ~aZJogoZvzdbzQfbr{+F6>tpRTj#{rlJiv4#Zj%H7qG44O%BG8zaK@(qGp_fDe| zeTpDQY6^jsBU`a(Qr%;DzILR9Lh28mS$^96V~?&ogQ4vYeeKpg9%1L*HGQUIQuZHV zB~!Rwa%{}iHdy8!u_@@M#|*D{&M`YI?tU-R3ZweeJYY9xKOtGvCQ0>?d5{L2c{n@j z!d)12%#h5jmr$$r`&?N&_pg^Nm&8s~ta+`{pN!IkS$#5Zr>P zya8^dn;T+c#Rw);5xl3!=f_phtGc*Q;u~8`9eFjdghay)dy}+nS$JdAD!Zm~X3M<< zh6HyGXQ6RceW=!4dq7>`Jo~8hLP@jjy=VmF;`LP}PqCvgQUkKYIQN)f9mjTv?psq)>&>6=!|r}{ zni*kEdgLKtoIw0?$Q&1xK_kzVb$wTY21Ip4_>E_drCeNUOaUj!IUe%BSW#lQB<2cb zY0HzUjVeFsth2@3?9*=5+dg23Z*G6;60vzmNu8`Ecd^{uihDlj7%v=7~Ba_H> zWX2|08m{F`Fa}q`c51-YOk?C3S=L{|$xSL9Tl|UfA|;)V?Za5*@nS&AL%M}DBX!}o z&67{Sojq=jz3S;S&~XRjj@z!R-Us5ba?!T0?X4%Xq_h)t)n^0@Oe5RY z^d+YiNj}>}Px-~TY&oGd-l#4uK1x@6zD&pUq+SsNhaOrfA5_SFOFnoj>8+{DxSiT8 zqY^a>h>iVz;_qXu)a8RJV1DzxEkbu>DsMn|VbdvjTWQv(= zjZGy=nlQWj$K2U7#O6Dxq~^b`>6YB)j?C=}8HDV?dE6e8areW*w#hZzjw`acC;|)d z^sfYU;nFz77hdjo!aE#j1@bp%eO4~Kl!y!j8M6ge(TpzosCTvrh{{u4=(FCA<&UjA z-9Oua&)dYDljrze0!G^HB91t(F$ys%-mF&a=e|irhn$leIaA9y`6ln%@XLhLdyy6TE~K+egn*et5XWnV+1>mll_0Kr(QSE!(^~oV>n(gCx{95 zhzo2c+#Rh#SEco%jkrRy26Xa=;cB75m_FXtUNGsadFdSr0)gAsN75qHfsl#(_7iF5 z6C122NYyTkj%^yhogO7d$g>P(M8tu){h2<+-7dt9bkcfO{c<9nXxQP><2|=@<@Q$b zeeT7807+irkb^f4bx8be^T|HV)$`_S5PFzz_q)q&vFuxVJUcE~(Qz9)D?LhcW3*8A z`0|7f%>nCcrb6T&xy8#S1<7nt^Kq_mjqd4{NW>{^2$HY5+1BCC@uu{Qx1NaIX4+*( zUM3+70S`K=XcmWpzZSWxxV7JMRSgK(EYAXW95FqvU%&#W?YuuPj@t643Ww-x7n?eI zxqB~x{0S)JzG&`bmpJf|qQ6ekzhJ^=ib0cS=0N$rftvcUmkD-+Va9XOsihj)pWHic zUE%O>;2-Fm|3Dr?{7~IgRmD+t^7$2*=&~GzQ?`%i)aa3%)i1X5A<`OK%-oaueSyOy z#;UQ<6rx;7ixXWQu4=(l>JQ2^VW#b1D*nj@>vWuIh*zTR?yi{=2Wb-2kDe@bUDf?+ zw;&XJa}yI7t63?h>xoXFB>*#PG0UEx07OQHtbb11Uh&!OFwWK$Uv^c53JT6$%|O)# zHq>bvHD2MFU$p2v>A8U)2X-jnCjP)(wXc0xdN5J&Oo80o(mt>;wtXck$6%g+-6S?H z-Z)+^jn5()4ZqK2imy~bTO+erNEg;^KwSQvZIca_XWQ$^SmpNQ2=L+X@MZaj&3E{? zZT)&J(OGjWOHpuN`JP5C$EHXOa#2(6*iQCTwj7k&I=wGbp=eIaNE{D`@relY`S0W z;O}6WyB{9%z7dRt2X@SIjzoIrFjQWqt+B$7;LrmC*#?NB;X zqHp5$J5WdXT6Wtgx-OM!K*x%E()?aBp#~+6?ha*0;x7Ge11ge!$EWM=)@$*l#L@F{ z?W~Q~m(zBN=gR4=zuVb$-&HT zaWae^raaiMkL%lti5tRVut_|k4<`)2xRxRl=&r>D_vdg=1wT%_Uz_Cp0=t9TNhfx! z7gIr|Na2v{m;&oLJM*EWzJA$u##oFp809stw^U&WK#nlPN1{`L3sjKhBE3Et0E~ zT~N)wmn)(iF<{p6Njq5sRUA`2MvegK9N|K`tX!m<#W{^VsacqL%WLUk1u&UuO6&Ut zIurKq?sz8P4Rf=SE@Iq+rR@e#d_qQZ_rn> zILQo~XBWn;-{zX7G!X0KH|5$TU_#%HM8%TmUen@FV*W_*Mh$y0YWTVm?x~|y^;0dG zjsY6i{3X*>D#a4I@irj3qh^wER6B49fRUbUD#_NK`@R$>1jcM;yN;{KHQf*8I*QJ) z3@^;`YKC)p>uJii7}F=e&A-pv-4wk!G(-miX+LGNw0s+}UWTpn$r&m>r5a^D?&wQc z<`tr_#;gmK#LnDPQ>1#9Sn_oipx;$*5fHWyfl)DkK?$oQ+DId(dv$wFZv?bi|sR`?&zhzIt;%}(z z-E~g~c0~8+l*cn?x3F=#I^c~Jj+!>ew@!L5^)+dIY{SX7coMrwq@|n=m1g#u zKNwk_OjC(~HOC`I@rM9N7-i^C z#DT_J3$LHh1-k{TMOrJ1hCGYsE<*q^+k3i|%3sj!1)@x;Yxgd$B)9L5v~(;PK;fLy z(n(sHfW8^{z;nWi!W=4Yw*Jjf2T-t&@CM{coNOWb*eXEA#d-wo=FY9!B)8>mRJ$}9 z@ZE$8Z@&!q-LLvXa3``ZRvl({k{y5gAvbR-DtGdxAv=o$*|#fkCI;^KGZYKVJBg6` zqMgf!nC2$VPLY2Pioaqut^#zb^HuI}r)vqZNy>H)8sgEa0J2XNTj%|jJ|6AF0@4pC z$$>6z^MD=5%^{*V;fJXJWgHaI13jg|^;xjaYmcs-%A zV+8_cZn8!l9jkGszL3y$aY?<^aKR#bLcS8Zl|Ymw4{_JAw%h4q?^@&bH5yK4v4PJQ zZ+`1m!4fuAt*+~fc()*xS*@#T5o+9*es{h^5Uu&xwCjKHDof zn8*!%%@)-61=@i5{pOeL9_0a)8h!UU{8c;s`f2oE&n~ehWh+ zcZtvb^f~=w7xPig*ILcD91n$`eSef+B{|SXCLI+N7FOVM05w4%5cO9@SQ`>&=PeQY zX^QVZ8ti% z$R~Lx;ZhjcotbDRXlw)Vor|ptnPp<|rof%zP}S@4L!IAmM5Nw-$=1csc@{#y|ljLD|u9&@I!E+=O{C9ch;e zRTQmGx`t&FCmDPJ$`#=*bVVe^-Xwxn_>t@zYGyGeQ=|n*lZWzdHu1jkEFB>wJ2hQQ z$Il(FHOBW@zc=2giP>r+y2fmK()sfL+gQ0DWP3Il-L|i|9PTSkm zZQg@IRnfBU>W2?}O>mh1?vR|qRBYYg6u5ppu?h0km@1kfBA^31Rw$mxDJbPPUXnu#-*j#musg+q2TrS+N`We~z{{AbodlU)sAf{JRwtJo;J zcf8svL8}a5&8_Q}l{l`nKvWwdIqhm}iSLR)k3sn+YLtSkvzvBE1itI=VL>yQdSxJF0Ai5==C)Y|D}hGu}pi1=k@5tQ?JA%umk zjl^ee^wvC)N+UY2Hx|WVpgJM#1&O&I9yKpLC_t-dC_pjjLUZ^b-!8o`LYUbl*@O7= zLH+P@VnzS6vw=S{am}JLxC9Ft8@c_Ox;!oKQQ%cGd^$n=G9}B^3NMR1WXVq!LRy1e zb1o+k&(go-hqtj!PblEbxv}c4PE-gb4E%IAemp;8Z5E94REnPxE$eYvYyy&Kq+8=k>9uHE*|cabXUbe(|$${?^(?%-v0#`CSrT$AFU z??P!Xp^KL?d=ln*@OD-SN?A8!tQV(C?MQlCyU=PZ3gs)s`tHEb4_c?ioog^i&AO|4 zaCyrrCXY?%Rf)+f?Z7+UmspwV?qPctibi zuRxfjjV z?~}M;RSHn9_h8opM(oK=8GkM`a4_lMi|XE7@o6i&*)~tmUfA_@eW8&wr3P%N3qpQ6 zyxvK*xKxfU-H`WV3$L7VATp`7!}f4Rpd^qNqjE%!MiE)zDfT$gzg^2*U6f4E4`pAT{vxxeNiWH+J(kJj)k-;I;FuZwxCH|!kk z0&c4PQRheQv46jxl27;M)i_br_F%3^{l;HUI>fMw?|c@Dr2>Z?QhEXPQ;Awen`xyG z4NcvNTqQp?lIZz&^3R_h`)~59-u#ONu+M37GrMH>f#ycn%}IZ6rwXB1fJ(lXj#y)l zUHG}1YV3QFRI5HhD@M@f?tunMY61Tiw@HX?*hu6`bMhMxv>Ns%lF^-&7J;Q^b;4a8 zHjn#mhnMOD8#l<{&y`vO1ES|qIs!b(OSqa%vxnci-(WN=fo!O9-wVsfu@7Xr)K;`q-+=Pe<%+U7 z_^zEFYGz5wA~3;)R2|l#G#_M$iX$Jph;JaobwVK0VNTBRVM!#?Ro33i8{t|h(wBJ> z*Ht@#gA|(u;RkEfUR5Gl)%wm293Le1jf#{edrhiST;PB|?GJTzt?Je&84u0{M+Q<) zIP=Mc#a?!^b(3M-2~DhcSsUdd-X&wNcLI%&uVh;=`Pd4_Y0NRC=``#IojF~5wGMj^ zBpYrv5oM1xo8RR~zXc?Ur?fSV4oKH90Ot*KZQ{h6RA)RvkZVcnK1^V@p5 zKJdn;ofeY4{FR^gK~Fl>sn*00i~&`(A8_zVoEWk zzs#0(AQ-(C@}&|9i$*ieI}TUalM24So)!c|em*TA8X-eDn(qpnSR@8X?5eX}E-Kxy z5x=Vy+a=1Lu7o#M8z(Jr&-hiEMJ=M&uz;6BGfoi0M3TRK49|Rol3+tA8Csqx7Swop zKc(kCSCk&I8C&?SuyJU>*2#wd>Cw*Yq^93l#uSjPTMc+e@v|g**+gPCpfE6R!KEYz zSMKL8GelE<*PfhPL8ab8OHQo#7rTWelc+I0YqvY6Gu75KMV>rnKZJ9PLxbDq`+ z>e}>o=;#{#N_)4aSS^Y?7tMlKp03rXG;xNS`+9lCgZ5l>nLGKQppHR+s)t#{_ z=7cj7&-aTe#b_%_G^0DLgLzzrzHN?Jv#``S6!51Cp zk!7UT2JXLEgXy{?9{!2(c8+cHH6cEb4sEeLj}1quUEusq2VJ8x=9>j!)}%FR+-m5hObEcA+Mrr zbdcuQAsWkMeMtJ+zSHb zw~-w3ZPt$jU3;;U^#y%(Qx4F?w8Ql#U9>QQ_u~xCYh&rP2-|@^?$52Y5BfSf6{|3W z4W-&zrV%2y%}d@x0@wXhsPo(cp_R`GBn}=}w0e9@PmUW2$Mq6C{vP}eclH3TPDAP< z;^Ik3BEIjsb-o7eRd?Uog5hz4K%(EflRH(vQ9(Mm-{_)G=|jTG*R?m^^bn0Y)Ks+u zN7r{oroot{2t?*K-=(uhkH&Wu1VlN&eV^cyYRRE=nDQ2Gt@h+t@NdK+4|5A~?(waZ zfc_c>o4EV#! zIzAp*?J|MmZ9C}U3?ZBj*w))`!rVldVcOD&V+`@2YPta^lijqsy=SogeLzDjh zs5G82{AaA=TwQK7O?Xx1KT~satfmeh%7Xbjxz+I5t}&+7&Qokm&`jYK zm2RsSo&waa)N(}2${l=J+_E(DtXUG3g(I-snE75YI?YLnUQ2(xcqPa6O!E2l#OIRo z`sDfjU3o5vdzb!)S7ARU#uSTt+pH#K3Oy}4FG4n5 z#D}+#I$2W7Z_3e)6on=Qag1i5bWYcN`LD(X?IezuF>O(O!}e^O@U-j-xGv`_bIi!9QBM0pTII`nzlG3es zj6L6vNOi+;>tdOB`r-vFm}7yA; zTB^KK`{mLT*Tau<`+D~#-XkbWnPQs#Hx3l=?-U=J?7beS^u47L!l=Z!U|V`i1mgO8 z2a-31vg+32WCn&Ul1s{xRG@^&@Y+|)0xd}jXA z{Y)n_G0Ok(nsq-;e_q=x$O5+QR$%Sfw5jR7Cxv0o;kkA;xvf0GvsZcsx9z?1NTovi zTN*OWfs*`GfeBl@j zvrH=6!;+zd!x9)^K>dr!TzWR4Ie?H{&yYvrrB7Q1y?tcFQ2hqYN}Zb@W_v^F*l;eP zYsVd}q2RM-s7sMct^&Xv7NIYxI-8L4L}E?9D6?6jhiO3NbV@LguG}lzwXuFs1j!^9 zPSi_qa3@_*q|U(6XKf< zQdbvn&MaV6;b5AyWEnf?XyK$&eT%WMf(KRP3SISx<}Y?`V*MILk}s=Sx-=D!`QZjC z+u;!8j6)WB2KMlI+MOY_qip*1t1%IHU=JqE$@PL&s?xr&;f>Juc;$}@(~B9dv^o+4 zv#?Tn@9U2;q1Q$+qH09J*69zYhuHe2%^&*$S=`%(9(~rn+VzOnGFM+L+OL9~0Ic31 zI!UuRRB++u(-b#;`zSZx^`rmO_pKqVc4XXz&7p*i89aR5vyczpnfIsLagTo+m{nd* zD#GU;q_Lr|;ep+B7|Ksd%a9MUT`*uy@%E2F9ZJnegNS;O2_?{y~ieK4V z+0_2J-bhPOmG6-7<)tOzN=X|^q?>oSPE$)-K66l{a8aW-qbpK%mUEWp{P#ydWlbyP zGItAk=<0`h3hJOr^f!;`{L5q`PVXMaeIugqMdYIg>3psJg;-mB7gDbqWuT3%R=B_o z#%qcr)m+C78~Mi{(^&|>3`0*hmP$=gDo_MS3?wW#p$oqIx85fBnA&|@&~uWLnU$ZL zPYT})-e{YMS(A@*_|ybqNg9^5A&w>CtBaI+|w)9(FN+xp!vZh2Nuc6Cc-Vfi* z3^fDDrQ(6JMN)jfG_mMNccRzIRa?F0QCm<_kg$TcQlH>GCj~SMBvSsdE{{CI(d9hc z-c-;2V(=^4qW|4}@%DDt?W;!~a4qpnmI37=0P!6%X+xkUFW2iKLjEe^Jdhj`{deVn z6JeBe|L-9ST}19<*lvLJi$Y~SHogO+&F~MXEtGaso>p%SokZK0bHR8;5?R9Hq_S6Q zv4|gZBoOS5WKc_ne}39P4ML6vIeq(wYN6dKr7vHK6c%2Nn1QsVZ5{AuLJk^0Lk?At zSbMzo!(NYCqSG`p`V`z`iq^%bV%6RK2L0JQo#Q`+zL%~a_mtjm_g5ZWdL95VCZbzV zkE5LR3$mf|6|>|-+Ex>Fgxq`17p5iWWwj&|h}l}qcmuWS{Nnw8v2gSYnRwrZv`Oh zGR4|tMZW)Z{3hJv!PMB!FA$OWA=5$4qX>4t|M^ir-JqtBXW7V%R6fZ?;W?Nof617Y zb~Bk-Vv(-#1mAZ?nma@PU!%*8bWbRTz$)C91&gyj%tY+{@J_pC{igatQDd4B6nP8E za6)~-RP1eyb+wLsp;u@9zRXylwq#D`&B}B&1 zoDeLB7-cc(YA9NZ(T!cf_2hEl?l}dm6=NBH7=apuMDv)U(1qaUeqWJFBblLP&=RT+o<06+A$#_h$qeWIX{&`qNO=1fCg+Cyt*J5qu=R#;H2w z9$(oZq^|<{%C1B&gprp zoD{hf5%01c&INSkjwrdi=H&Fzp}g&{Y2qA*RKVcDd}nbfS?hYf|BBCP^HV+5)TJ8P zm5C8~p*(CbgD{4>A@s zb(jB;qC_mI_3h=M*|#sf`P9QO!hDW98gUP(>*;rC=LKa-93K|sZ7?IA`Em5TgPcc}&$fl*#L=33( zoQh`n=SEYwVob2WVfCpaP1dTC7AMhbYDU6ouO4*;kU4d#aVK?5POf|mZffF$&!0xe z&cnZlnF=F_8Nk|0SkkE5;KB1OTbEuSxGBp)PXi4P&=xKx_PYZ`pO3f+lMcwfzQX); zS({*moCCz^t+?fWyaD@h1T_&a1<29b>m#YqRv;ejiXF-oDM=MUI`axz+$)lyxiBPU zEYYfoN{=WO-)TS4IV2i6sFGO6g|7;FBuP#*&z`KD$3$;DZixY`3~->H{3g> z2@<5OoxqoFAWgN|&GCk;|4XPe3gsE4hll)%!?YFLSg2~9M57W`c^^(0`^VmCJ(>LM zYm0=&LIZ(a;$(nj!Q^xP=wd`eKLBL^E0%|4?wZ7KspVAzBkJ>T|*JD@uWXd>)F$+(6uSkHX0R@(-ScFQV>E7 zS<3-3%oi+FOHi`y6|S3b3db~80<~Cw6J)@!roD$T{nzWIUPky$U?n2X5Ui6{p9kez z0x<()9cCVOx3+T7(D8l6d_P8A-XkmZ?$|VSza*YpK3Ll|6NVWEyRTGBj{d$vx)xUr zLV|``njR(UKo4W>I7;bNbxVD=ew7<%1}PUyCAt;rvJhc|rnDF16mYTiQ(i9UAIxs? zV1*~TK`LC<{0%zM5K{?s=qmS86&s~NtrL^QvkR%1i4`eOn^N}M=MAf8FfvXHYa%>l z_>u&ZAIF$$F)?ICOqd5Z_;4-h@7gl@+7W`8B~v1o>z1M;!Vv8IK6ji>C-q+4B~|&c zCC`?knRFCMo}fc@cDa~{uAV79fCm?CZ(oB1I@hwiWu8)Uln$zNiR)GoN%(o^xm0mz zjoz%%2$n{NVg%I`IZ4m{R9j6MC*14o#4Z+KyGXz1FdTzoi6MW*gKSJ(Cv=^&ijy}m z>RTh28Y~JC^R;R+E~95l&RCRTW6o;O`w8IIgO>F)^|hjA_vL1TNeX4U%y42Dm#BLU zBfq*tPIt-I%0`4qzeyk0os#XWO!ZkiJ3(yPrP@p1UoCW~b4L~7C~nh<<_vweOK*$_AKL=KGOl~Ag+pH)3sx^B`#w_D-#)ro$M%c8rZX#rW2~C^ z!rRyB6a%cSn%LNH)v3{`aAiR9aIvbeyN8mAe>sLKzlC{Z`9dlXWt0<&pSqb8@4Dlz zJ_WU?yK@E^vV$76m|~6NH!ph|#-E>~J|Lo}*77gT@hg$)x z{hrAA!sf}Y`xBm`WWd+2Fc0E3cYsyk`&nyI7txo~oqJswNfstlde2qE@MQJO>qOJJxn#zK~pbuYVu z<;XRVB=@88OHgc^%_PUD;5XdHcn!b~hAm~?U&_l-*(QUov8ab-Z&Zi4iv_WanUkBe zARA2>^7DO&8UhZUNsY=s!8m8Ovpa0I)w(vV-dd`rtx_EuR-f4_vvkDuV zq|Q2V*$GMcjd$Znq9B7Z6qNmy%aXn%W0z%j-`~sA)6+IbdlHA6W)-^{^iFR7nvn!>kk1^vMvRh_EOzEBb^2nr^jW_-v7qI@ zE6ZVaBo=+V7&tj(*GD%F$#;LIZwDRuU5kE|5Z2qg$wnf}|8~~oGyW=P)Nbp0zPqq} zdr4i<(2X;v-Y_{xu=&?@Gs1oZAdg9)(nta9zVtLzhDI)Ak-d_#W-P*nS*!=>(|=2l(SXE!?5$aLf-q=AXdP^Xm=F%) z+b2o9y31+Kdk$pUMw*7c>=uR-U7izM)`@AWrMrywJFl+}_s%YQ8{al<R+F?oCv# zaj}jILQ8BHcZge58nXK6UT~|dkvCG;>-!pu-y;IpPPD}T@_hLIJ@UrxfpED0K7`OF z?H5x95^@97j_tgv*|+a_wa&V$?aiDfp$tQGWfHMHKPz zyd?mh0XdA?wF6C?J_Nh$v|^VgLF^8;T4CBO(uNwALwvHQRk#hOokxi=E1i{iEGqzz zTSpv$szoQ<1G$pb#u22`ehikv{kfEDZ9Ct}U(Lc=VZCX7E@*op#hr?}2JI*~10u}Z z#DP)S3iI8f2g}LkghS>b0!>a?R95`XSrYvAS!I0Q7S!w{!1?!h@&Id@+BF^eEgKtM zyYt~#mb$w%d^wa)WdtTwyz%ZH2ECbFxK{_dO9n?nHgWoQlfSi{or6enu{}Xg_uD^h zDs=0zIVvg(zph33xvp`iec}WvkF);Ql`VU-a}i4>{_GiT(fs)okh$Z`Mjx3!m@D-6 z(@;en9!%snVVbadPj1LN=noTmQvqU{ty7+aL$52?6yeMDSC~+I=JKy?O%h>L$>ghi za4##)KpoRdPDG1PpmqL`%7(SXzvtGRL8^u|$8N{_*){F^2~5p` zsD=_k=SMUAPGiRt^_Y-EILDvvQvUUP!D~<4($7n%8N)3Qc4sIvVzrfmbklLs#sWlD z+m@lpx5Igc$ifDTpn<55rJ0qMJRQ=!f=b(Nd~M5+Q0XN>+B{pkat8}bZ;kBPXu== z)fa19;(f74Ni?}HPn`S`LmEJ?178ynin~7J?@gJ15&OIBBtHhpwCQO8nN;1dadXWp z>+ak=&7RvLD)p97OiCNN#rz3p&xT-!8BTee)>H$dz4MQ3?E<-RB(ZCQeVuvk+y0bp zRKg4Jx_U}k&J}j--+)@*a?#eCCR&6hwJRGg*Rv1_8%sW9AV5zeS_AXnn9sdZ@fJEWZ?*%q?yEH^w zInx1qJa`D^ay+v|SMO%_K)afb?AlP2V^d8B55X8@<-hB^UM5J{51!8ZtG|vl0ByRH z`7qk2tyNWek8O8HLKF8Jw!ZC$@5?MI&Zrl4(Z~$UwNNbfW7ZyzBTM{ncuUU_$7zcc zuoMjz&ZbaIuM35UU)}CkZHI0yo)SagLd{4b4rJ@SVcGlagtdF%kzMXCB!~qG|Nkt^ zF92i4tEwms(T8f{)2i=8!N@{QT}xV4G|W9^dKQS0@#~j1WZW(s8*BFoitfr?xBr+! z#}jj!p;w1JARq#e&z!!}r!UQ%w#r#@@z)CwV!qI@#ieb9C<(Mk#*{$Mj z`+<6O3jEX6Tc(kVw~unlivjg|HzCk}#-ZK3ZQb`FdgYoBx&s=FSW#@4-%^aRAA-X7 z_VZ>Ir%KBATXm=*VGNwywi_ysRZvdA7;Dqjit(0`^!YXa$lCBJ(3Q9$Aaufl5#iQz z>bdu$z~5#GzWQb%*|WAkehsX_tZ%I(@D{S}`;koZokOQJ1){e?$1KLAN@1fY!T;lB zcxAwu29cr!7eP{O$tCt|x0Tc5EUvrxF!yCdph$hYUT`J$>3vseZSwwQC~c{o3s)f! zCD&!DYGJ3c+@PMmwlsp^<;0M1e*)nr3r8W{pN9=R2+eyEGBb8`Isv(owSY|QqY3b}#MGcU+RoYB+r2RA|%N|tE?HhG!RYGHDg}F(<6`?LZ zu>MxEPuRv>4pWcBVjrEwjWL;_X2uztsaM!GmLesnpu&%Gg!oao7_-4+?;cpkG<=GP zz6yz+Nq4eprr_KR7_z^LX)z92#-RfDVOcb93QWE~?QrlJbu-%K2{hNrfF0z`3j}MC z0KGzRqrJ@ZZ(kk7cWL(fQM^h&NfsJocS*K$JnF0&^mk{g%^Po!sKIa6R!voxJ5OW>hbonTeTO-{NC$fir;6NpN)@?3 zj9nR{AoCPLJMFf>lO?_O%v^c?zeA-fS;=L#3L3n^Dh0H3{}*L(HI06o zKcJh{s#Zo5bE~D0TH1L!l2@8x=cXfl`N(C464kQVcnjEudvB{}n`92>Bv(sj%l@Bg z_WQqD2m@~q02ZNf{}z0TI%{VP{NCN$9mh<3&7ajh&K#$KIrnQw;gL3AhYE#RmNHvf z0td~_+j{7KQiOH%m1h|A5&LUKx56CKI@!n!rOL$e>5GnUA?I zjv&m0DWKeHz&J{$P+dWhY-JOX=~L&}Syi(o2rIF@^1Y!b0SpO+Q#Z;E@7DC(as4Si-;Rtm&|{=?U=34YXR=MK7=PjNE>8)nr{6>dt1kd+d3l-qQe_g^jn0F`vn`ugpglgP20)ObD z;5&T#{&mm#`iT=BF^{V6GT?tOfX@_9RJ!NksP9{qoc2|?zW);syTS4FU9ub|BNWX$ z?%@YwU&j_hZ4p(&Xx=Gy9}$QS}GhhjQY4Jv{mQMz{* z%8-rAj(Wo1gUZmzwb|KM-F~eH zwy;GoIdzUF7V7KrT_B-nB6bLDzcANrleqK@%*X!!l?X^F>$p#hMdDbHrGM22stwlfiN6QPeDJ{);+8{33V zgu;<1>$r*f<}7Vfz#;v02N9ojHO)9h<4L{G~!jF62z~{ytA5hv;&2s~`OB?#z))u}Z_Z>H@RQ$GcE#cJq zwSwc(fo*?iPIyY)!-d5<$x@BjfApZ*u3^v$M#&>o@SNs+OsC3ZJsLkY3YK|w8_Ym^1lWeizt}-WmJF-H3H0(gZ*|)^az76N;L-Y2uqFZ zgNcd;{95vv`kKS8=o^}pvByOQQ2{aZ8h^&I+%t5~wWscwEHbWv8s%aYXhp5gD9tN?IM z?}BNEn2eS=G8J}QW$J%@{uM@!>xtc;G)UA)TWMEo=&syZl7-yNGEI> zzp;Us=Yj^SGH5U})mgqj)h$-}fKfi~4FDG6xJWTo9Eyp%WIkdKC8Iln`McP5m|}Vw1cZVwq;%h8@n3{iXuThV}H(AtZlP8Ud(Lm zDr&PF{6RjcgQrr5*^*_tlYEW!w#**w%JYmz>eie5dn9~EDUeNzOPG-Tv=xp9XQ!{{ zJl=u4)3ImokrrS<=5Fgv+`8mcR%CFW{J(C4EC=_B$*#cXE01eh-+8J9?s?c3g4(P* zd>>*!bdiRtR!mw22iYH?j{FUYwK5A2Ms2ejY^FNb0595Nr!hqV3%>urL^>COR$`t; z>55Rju&Wcj-Ul`8dHM0b6gQAQg?xr}8}d6uzH8eY7>!m{-&o|@D z;_r5}1Sl<9yBZH8u;*z@Z{{%-3G?tmfNDC%BrZ<=3d7lyRuPTH$7X^Fb`@Vz9?Rq$}S*bHRklXuDt9y6NpE(F1JtLI~3GR^Sz5Q>U9AuoVTNB?b{CJ$Bqwq{6!1Y-76%`*A z#7qN~*(qVPM11Eg6Zd>h^~gj^sD#P9&A*gAP}ei~395Nz0@ObSO1ATK6}=T##S$if z>M3PcnDB*z@dw|ZQnDl!VQ_8z2Nx4JQzPcjQ47#xxC`iOW3-#GB ztStO|N^v7|#l|hwN&_9YQ8nio0YQVL&8|OM6kI25w)h2Q*X-B*5#ly{etGh1kWjkPANi|zbl40 zZT&I()E1G8FG8q*O{roEuBMdAg>;e{HUu3U$ zI*aWFioYN;Vlf{LWtq}`7x5j5lNHMVzTIPw3EyrCwDx^>>*vXU@BPG0&B_GN;(Zz4 zW^?9VRwqP{`>AuCgC@0DCW(C8mWX_>4KoZM zBRGv+s}d=-Xw$a--QmrAj~ZmD>8T`hebvw-U{B$wtgm12ShrN8x}8;~(TnTc07R*5 z1S|gC%;;*m5NTf(P_qC}z{X~mO1Y|qHmXkL{DPiN7tR=}lOP;dD~tEn9s4M!_Dg=S z(3U;1dFBSjA>P?Y4SxzGrY*BtbrH}6&lqrRLpB%3 zWRju@2XeAd?;%?xdbPc2D6Ep%%~{9%?b#x@6rYf<*J5F_ggUoAQ^Wec-*udNF&H-R z09x}LvIvE-FU*g*o%ek|vhEev-u*u2J^DO72i#U%VwW|O1y|Dexw}#*!L%B65=DCo zcz+W)zS}mRdE$IE9xS!9>w){2c!QD`PSFXx_CAgt2IZZ<1Pi=kZqTwVt*ibTwneUUJw_|K#Q0&Fkq0 z3h>vf1t7*IWQ(!3?ODS%{%{3M%v9U`cEss>ahK_F^pRuFXUDzEtht44sZB)cIkVKHhT27QiIRIadoBpH!3?L zQd6ghA%K#hz2cq1*@&5fmundk986`wSFa&|$zzsH?#Szg4i8TEK|65OGiaLWxqy?4 z$=ojFd(DtdM|IRFk$w}Dd;(+$Te&V|@AxBHR*(!_xoUOXfhLFy@Wt;w)pk8B;*s9o zR$9aEX11hhb8piALZGQ8okOl5#ie!9G=X2l&Ybf)O#fjU2J5WKQcsQ=_X<7dmbcel2t zP8K2Ag(3rLNyQ?}=#sd46ZOm8&Od!Wf00^0qA%SW#3vF*Xzjveix2B}ucm!&zKC?3 zO{}JAWyd$iO&QuQl{ho-byS}p>!Dh3iaIn@$eaCWaSPl(X`}XdYwW%tmRWWT%=9)O zqUD`WyXP)eUtqkFaT`d)WB;LZ9WpPE?)Y!2U&v4q5oSNwF+)Vfigo5`pFLK{!Ok$N z_DZm6v#WizotVKGx`xN&AukU|UlX`ulOo#CoQhRgAfjCsRL;Rytcv(X)aGPl3SX$DB$1jlp%}i#R76>jb zHiFi)Z|tn^kaQc{dM7&+ec@fdn1#S{@ng%Eev?Jn1=O2!)_@l^4P%=I&A^s<_McDr zMVIM-IA-97nkG47z8Fka-5;9qrEHGWA&_{&-1H#Ctl57`Zosur8+lbAUXQb}OOJWAo|r$$2y1 zyGwVD|8X;JuJGTAs~chI=p)~&T#>B4#+Unb_P$12%eI#q&gPTAc}}aN4jOtO3!bz# z-)}M~WN)0@FZ%y)jYgonq)ZB`#a@U2A z>SW|PcLduD^io{_LnnW0+kW^tw7nf|l%2t8xTmH8!652DFrD@~mhtN^()({E+r+BUxD?V*dzS#WTbV!PPx>e=;F34ixv_o_`FU&X%RGmZTe6x1Vt! zV%n11KVJ5RnKray#m){)CaVZP>Sta;x%WcYD#bL#^S8e)H?IGp5BRGQuP+d)IN5YyWcOj z6E)yh`wlR*?oqX$9T0;)b$>p#7OKSBF#Tb-k3E!R#M^6I@gFqaPdrV_27GmO>O8wa zIQS&XWQUPwLqhzH*l(}0C%%K)0EZujU0Byl;@ZM5re@hYI@4?!-T?k56KM!VadT$< zcaGM40j2CCv@uh2@1(nOnpzKpfI4i}p#a5EX&A>+26txY^_uo>YbT*mZZ24OQCm@v zSZb8d>R55usiT}NA$`Eo7p&VjN%fVfWPmZ|$fSL8+;iP1x5Ym#p5G}his-2hRZJ|* z1p!p5Ir^Zu}ScsRI2zRSx{|kPVa+e<%;^kFxYct?^lgH>G z$)X}NHn-HEs0jdt8^8*+U!JVl9z7SUJAaD|Xe%?lXBW(lKel3Xrc)yef&BKnaglyM zhJ(`ZH=%5?jEVImtRr5c70mfhHlu%R%kf6LbT?f``6sfM&xcqA31S!XF^n51X(9Bj z4l)*VC`w&07?%>m&}VL=E{qLP+NY(X2HG&djQoCCVI~dXopT`Lz)A6>-AbZ?$1IlT zN|oniodC+f{fR|tiZ-d7hg_oCYsxwWV<(tAas9GO|o5(*ayFA4N%#4XTJRl z3HKOhoL(*I4j8NMzN5uwukHg72=t-6Oa%jMtWwn>i3l4e!lE{T3a%VnA5S8_S5*MM z^DxHK5ZA6q_ssA#JG;Un2?KMMPUO>!q$jv+3m%19VCipAhG~ny`$#*$k(Eb`0#)!j zdvTfJtGrph1VbX7wv|22~FQ)~qpXox!K z!FfZbEtHx)St+nZLDQirC-%m<)%SV!3gtYK^9|q~#-H9Le~6E_&<{Q;7fS>hEncoi zuA=b$G?6e1*P!Kv}?L+wZUQH2{5zNdP_(AI5D4|V6P zeC4NMehSSJha5LOl@v2}u0u|=1De>O?omWVwurDGAye2j#tkO``I&TCgz+uqmm#;@ z=-JtEvh%t3rP|oF5G3(ghL7=+H99{RXupTh`_u87hF&HDq?$^9q-W^ha7|q-M~AgR z`t<>tqjaqO6{SEg2GDEHy365-$4Q=HbKLNrNq~L+S>wrl(}w|{Nms1j0m3U%d+5^> zeevD3EAdM5N46<+g!pNs5brpZ5plEgBL6r4YtkIZPuZH8z{1vUOICFpj*i`7_6=dy3%W<}h5skdc#g&QbJ3~Xf(kaJjG*s?2GjCaGBI?E zX%M0-P84q=nxKZ1YtFnD0dM8n6yFI=7JK_Dk&jhr=i5O$!1y&F%}!JBHm%@gOtxpS zEPv{6nAPi?WUJf^oVBQwt2`YunHOjI_9_Kpe=<34@1N3|N!H$e`{8kEaGDn9PPnLf zA^P%{8ttP1%w6+#To_<_MgZHH{s#o zAc{Hv3`dT7gJyQ+C2i08C7~LU81gCAmuz6CF_-U|WwwmebbyXpM(Pi*XNg}B6nW6) zdosz0&Oyf2O{ye@*_PDD7q zoeE}wWFJG*A#kvqr)W4t8^Zl5VOhzWI_JW17*elaX&+U0m@L*2XD6nE0XzCPplrM` z&0AgFSXXXw)BRu2s@{ldrgoCC_v>PC1$peB`5k`~JC`)Gke$>c;uWO;pCbRuTu>V3 z*xA6vnR(GLs@XwfcpU54Y3lMuL+od3@CXVaz-bLnWEOS>Tkz^J^glFdW3%N%v!;Rf zuz{C1T$8sZNFTMs*fOUrIVWH*SR*l(ajl*Vpf1*{!N>;ya&4NJ`o7(v)*1_}=LVSk zJ*(w}X&N!QPSXk=iuAh5r9+1ht|5?OZ$`t!!8Y*VeA?@L%()9aspM+tIbGfsZe?R| zmi{%TBwV+W#=z;Df{c&4?x{uQrpbYZAOQng<)Y)k#&hZ2pvZQ_C&{(EFcL-j8 zM4I^(hP>!4n~zX-BQ7W4;U3EpT-y^P`vNj(cXTMryARgC)4g22saZ8x?IjY`q{8Ni zpCG!RK@pmo=fUn3DT6Y7P1*@!iTUPE00;Z&8e=U-QJlTL>`3A8^0tl)<|*t@i{@dJ zSY)^fX(_(0m~3coxn-|pqo-x=_tQdZfR@vy^lY5Wv;{cFe&#I)=l@{7>2CfFUTU`s znlw;0mCEzSqDl2jD%B7|gq8wrc@@ouo@#bK*7?80d4GZ~Urxg<4Vt7%8EOMBeV=D7 zPhay0zFcu21dSeZZpi6G;c1`}3nGCBpYHeA+3m)}P{9MUky_pt8!O+3_{D7TD~{1Z zug|(X)3Kf2gK0(0$i2}G;qgYF(6$iIN=O;v=R_S;1~e*PSN=vgT4dStE}?hCVnb_X z84Z48-X>sLz@cOpBw>pJex0o-zM6YnkJD}`N$pf$^I%Z|g=2TQ6D0rEZ6DEkvs@zui!94i8#Uq&QUzC2_Fp0qztDo06N)qKQDZh#I zrn@qOw9O6Hq9)p`_KP(*s0TsHaJrzJIztJ;1CgfYZSv2~^HclbmAyKen$Z{;A+Fyh z{Mh{yNaopfjTLg4Fx;e$I(Kx}TGin>osCKJtC_B!W*FK76?9-yPx`@1M&v!j*j0@o?AT%mir8}ZH(!n$w>sc!hWE%WE zvMb+2#(T{+SKY`#jzT)IZ2fm)0RUgQbJCQzJqJK5X^^^C)|v}O-si*7b9YP^ zWEFg<;=8c;b$=EjZ;NJ6S1tnS0i;m|?PwRRPm9msTK@NOv|l`i6Eq{Q3zwvAgZdW> zHj(L)*L_CbBCr>%aPv7XOsqSL;&|#Cy;InPPw?|2dPA+U9DNIw=CZ>OJn3 zlL-T0eO0HI&OuK?&`S`tQO6l?a#9r~%4nzn%5SU{d?|uE?fL-HfD8|1_?ecv`x~p# zXo}284_&8?|4VV~0ajr-UUMPRgk0Ux%rJZUuNosx55Xt9lr&~EW~SuO*BJq$yeI>- zJVFK?n|dmb2pfss!)%?#t)6-q0o4isj*}T zebTH^t|6H#`Wy~Sx9WE!ac=C( z$x&T`ZR{c=5f}$YWbug@J^2x=?^gmi?IgbFLDdugq;b(`iQ1TJU^whXh@h-KcMM2F zIEm9V6CA$uYms1A-$tuSDA)dEr1UoqgKA{mEdG#d473g(gI(fdXq&sQWeh_tT#=Iv zs60-;8m@B2AM2${UzN$+VA%Jyk_~uvROWa*ca045<}?<~m;T2Ea3{qtZ%ZGe9%gO6 zQ-oHMznWUXGG5;kx0%wfMCROZaillp%t_EF+R23pXj__YCfxD3o%h<^9?DZK?%d)-vyn`OtfuzUD_{Us}{u^Xe@1YcYi$q z>0eiMGIajAowMbdO*|Fz$Ylk6-w@5^dh0ng3FVg4TEvVh3ZXWx!;76yD}hRHCuxMD zP^V!Vqzk;hr>C~F*`};e+SPGDPHS(prX@ImqV?(JRrr+>KF%}~F_UwFb^+z~4mPoa9 zS+f$9gG*mXz6v%a95M`^D>MJ4UPWFx8ykgm!_rvYO%OZ|a!|HBGNE-mCJKEfNU89> z@ZMD9zV`>4B^L~+EBG5nE4X6z0AN9S78R^U9|NY$Bd1mN{Ki1J8F$r%ewzC)KT41w zs~D~1Qoh+}e=*~L@eniR%h@&Y47})kXzImBsa-8$pD8uRkAvEXZF`!{_FjN}S3?$e zlItWNMy4W3G*&H*`R?mpjM3q@4}J&OAIP@fc~0{dy`$KdZAyjx=`mAbaM%>| za@<2}Xb8Bq&ju$5pSd~)do`$XB{>1Y?Ne|r6voaZ-; zB4Ace3D(eJy1*8>XSFM1{jrTtX1dzPjmHo-snUKFA$6AHiTQf4Qq7kjiSGm3j+Kty z6_OrXuWAf?O>1OCj(pPdWcpk`ECyi*ldx2?e#Tm_bU86#b?3|x#}#c&s5AE^SQ8&-!-`jRVkH^^m?Sg_3%I0YjEZxodV}9ruB@xUleN7 zN_A{ah0^*&YZ>U_z>qg0mj4b}7LpvoU<5e+#_ zYA);{+ycl0#Mr!M^l{+p|GDQ^or>Q9?)6{CO{dQU`kbPR^a2 zp9aZj<7mUMbT?ZrX>U7Io~j+9P6*p9NfxCDtbRH?VzR!mdv#apCz^qZ!PuxABn)dP zXAOr^ETzDQ<%Q*DNV=EfUv*=5c;OvL*!U=c*LqHrIy%WBqSN9_#+;EeVbxt;k13hT z#Ui(@I-zJ1=NjVS{N9$uWu}8x+1`CKX%_UL0^`F>P0uMRVw%Wfp(?j|LKiZzuEc-D z3Y~|;+i^vlCVb#pF0ZF5!*9aWVrfH?%(=)_ zS<6(t9VG;!B5b(9mrp|LOG&(?S%kDfK1#j}Qa0CxUMN*fLmZp1jtkcAs>9SOnK+#u zQI~=U1BIc3Q$6@D+EN>;_96WRe`9taQ6Vn$hG>v-klX8VG((%Q)+zCdJ63=)CLwlG z{ox8=Ay$;Iz!l6Lhu4HrSr(Ztc|RPpH7%MyjskQ$=^Q?ZCNBHS5s{S_6W1?{;YBz0 z$wLS&Y>~j-#9yquj|hY=Ggn2UYvKYT4O_L)RLWIU;8ARTbL#AO-`KSb+s!pwT{)n} zJQ1yC4wFbON8$+csVV)zBA8u?AyS@;?|R0+%6yTHcNVT0NfjE_p5+jD#9B~5>13B} zH<3;|fPFoL`N~jWt*0NW|J9V>w z6k=ivFS&`>kb`Ln$ZmbOJ=3@R<1XYXcctg#?@wS~X#26kK{@r)Hy~o`E>mS_ZmL<~ zob9cn8=L6A_=}Io;v279}Jj9lPVCW81bn zZ*1GP?d0FTz0cYEd^g`SZr0s;)?78KYL2Qo$^;Txu7xvba_O*r%g{?O8`OQEF`p^j z!oHh%%-3y&$aZ@g<#8`3)AZvlIlqcJ9~C(fl%Vcp5lwZg(d#nHqF{Q1+i2(_(6P`? za?K`X!@Y6M?hE+oyS3*w>gc6&05sNs4mkC9v?A&_*fm_v5205`%gve?W82C39?mYr z0_5*JJx>e835mQ|#Pf@y4%*>^Ba!}$C?tpqDf8od4VCvXZ;QtM?sn?l9fAWY=X(?3 z$hPSu<6PO8zns2lFEj%Rnm92eGn(da(XhG8!)B(@q^W(oI;z|Ah=hdunG zj>kbX_hXA!$#-Y0)Cq}h<{qxw1Tfv#opEuuA)lvJyN(NUFB`$x?-;|f$Vk0i5OfnX zPF!jl;fDptBZjE9>j<#WsW9qKq7O{AdS(y6233X+e`c(QY>(7TP!!Cr-EN zZFY{qNffVOx`VGV`rTRHs3>Kh-$7&O`ui;33p$o@ck%+UQZQq9pfFF;;%OQ6U% z(w^vL=kPFxFw}L9;eYD2P7S7NtZLbWhxjdI?PYfHt>eIbYVAFiJZ7hPibS`*8}ccx zMECDNQ$%P7eP_Q!uPQ5_J_isEjuqSUF*)W9P; z?^Jn<>Tp9!D!AHjc=YX0wWGbLF03}mM6xthCUt&6tH!szuSdvezz_F}G>*wY97xxd zhaN9AZ63$qVU@(THBze3j31V?$F|{jCsNowbW(bQWm9_y-e^Tg`Vyp9e)rCh=@vE8 z332m(ryNlIOCILzp3g{YRJJF=^IGaon?binO@Fl;%$JDtn|xz|j=b#8(ib z(@I8E^FwNei$My}wP`Cz!h=I7$#Si(+*46q2UMXA9MVQndb(bGbV1f2?b%%qp`d*e znl=7t5|n{cVeQtrEN~MaTYaE8+t7=k}`TW=@a0C_ZI&5ojmE9`Sbb)U!ltFnmd&`8|KnWm)}5ahJ;xF+3V$2;M$0H@(eklV z(`+Nn>xhho$0L#YJcC#U=lMB9nNwG$Sad$%*JW3d^)omcn;jjnHJ07^#o+avT~Y;m z5Yz+qvrqok06AfotYDQZ{w$HAcv>tviSi=#iqNuR4fX;{Q^0hH$i0DGzimB+&jS_c z&t%EZB^g9)i`de^Jhouhc?wxYS8tEcwjNr)U!CX6{2V}TXBUCpfDo+dMYg>QPc%Zv zZ1i*;+_rrQr6#_#|tEx zogkeFex3}irQQE=EPOQpmCq-8$@_ZiQQ0>zt9s#(l{VAPFU|(}sl7Su%T1vd`u{_$ zHOUBp`5=(%ZNV2Cm*lZ)P*t%9Dt63xw({SN(q(WT@$r7WC)mtI zOX_+$meB)!FH}{Dnq=)geoKF+7(&{d9gH!(g8a2w$Z;Opxg4T@(X!mx()H;St$#gd zxN1d7Xy-#qv)tM;^XW3R@VMRQTm|i`A_Fx+UrF&j$s+S8W^j$I?q=@-hfi zmX+X}-_UaL9tTLv{_3}LGXbsxYB?u)(Kg`uZKOh-3}!^pDjvZLDij&!zGP)N%a#?^ z*Jq3SDoWU;eTBxN_SWY|<<~Yz^A;z?8eENHU4?z?mY3ThQGfkRxD!+cDWVZZCL?;VWGt3Y`X ziZ~0c<{&}1{JT!nN8}1|`9%7e7RexICR4Ludgsw9PKmye`_=<9heu8AoPyx`U@So} zMJ-!8NtyGsodc2nvJ#OX&|IWnHU zBK~!yh1bxdRG}TCzZn7257)-~fbn(-^Q2{8`=GvrZ?$&CwRrm+hejFmw{?7T#MklW z&WkHZ`=l=Ozsso^DTJmzJ2@%Dv0`{?OT~a zk@x45qzhVPk!CaYn(~YAfr~Al;0@`s?Vage<1wn4?~dBx#3Z%cRamo zci6!huN1z2KKdzBIH$NCKfKCiB4TwtN*8{V(z$E=*tBQrF^JaoN#!?CFv2)M{)aY< zxiFSg$>lyG?AyhB)chU29Khm`%l(^T|Wx& zY?k?I+%^FSNr>sND}yqv{1_B#3~%+!G*$~Ig!s$@?}HnI>TQb&n`3Qtk*mM0xqAv9 z&MH1wdj%2u9P$PEXW1P!MResj23;CJ@ulD!ZUvNXJ&KU=@VjFA(@%wf?u?>cHL`TQ zpZOFU+u**hXLF8!CGcr{4rq@{H&j3-yKFJBCKKqb+1%D25X@V=A^Vjy1oXsqa;9WK z)2cEpn&!+1XtxvJ}~2)(TRmyDj(*!CLNg5EL++-E1< z=9QspcO&YDeskKeF+J7KazN?j1UbHJt-mt9eZD*~W5m29P;$JEYkSHe;* z(>?g2<49vsqhVO)SS?>AZ-IsV7w}Gjd;D;sbH=XQCvW3z$-2(g+->|OV>x(4ciCN$ ze>}PJ9nQ%05C2{706h43IbfrSR$HLuON#e>`r_jAuf|52om4$->FFTwkGk8Xc1Z)- z1xLoMSVPzItt~?i3rav4qiEg|1J@PR+V^*da&DA5vMK_EtpI@EtMWO)Oy2juLo#23 z5o%Rq`GAc9H?B*3rkWq`zIbXVE&Pwm5kF!>*=*x%q6cboiWPsC*EBqB`i>p!ovR{B z>K0(~D)|54QK~c{#ad9mfXf)_m_u~T+V&z_Olg;m@_lTX&AW{nNVO7NPgH z(^)j=MLCZVYU3{mmEMfJ^ePC_Mr4~(7&HnV3_ z%=ng|U0Gtxf!G*vv4J|y!`4*kj~=@cFL^16TlAQ*CGcIa+a^N)J(3!E_E$=4-D-dc ztXrhw>iAz_E|x2XwOlHj>zJO@WTD>)?ytFj%c{~s6T!aU+!s{D`^%JsGXryIga3$~ zE_}4XHVCHpxetA?y_|_Ws=3Vrh9_HkPkbMj#T+eCx-Mt@dR=)X+9M-nY*rhWs`3J4 zk`A=|vfFMm6ZrTj4`KzB)Ya7oA!~ZwTY4(4KPf${5>zva{q4w02Y=uesD!ab_wfwe zCc&elc3OW_mLuQKD+EP~6k@KcqpB0C*SWe>tDI^Z24K`wPmGR@RsE@3QOA$()XD#S zQBNEF?s~!QA$}?2WAGT+dAB2e42dCi0{-Zpg`<^H$iC!bgC$9%Hz*q{EY?nSgy^@{gu;o1(-f7o#gl1s#dqxXdu(A z-=`SsT3d$1B3?Gyj+YdX_3R8rmjacR@+XTG3KW$k-WC$6$!pyS?bx#$6K^)tPG0C} zzWTiI*-G=e^!w`g(zvK5vzBQ0y2=tb>i(-9%V{y`0w|ud| z2zSIOA|?K3S|K-Cq-)6I!NY_DU9~=MH?Es==tk;ui z*U&Kd1svY4mW0Dqrr`aCs1w1lmY)(1fZn*pZWC%pSi-UH9(z zG&Uq~3u37(NS{zz27(Z3CW|R__X1Xbcw=!;Ppgn}ILLz^DjCpo(<{~UDpj@Z4m`X% zDPc%bfB_&qGHvA0HN3?;&dMe$CqEF$`hR2CnNHffK&j$T3pn^c-9MoX* z^mdEEy^pAGI22+RGbm=Iw z*svlmxT3QyB#b_bzJZ%c;`dQhD5g^yzKw2;Wwi*9JZ*kp%h{NjU3QKDG&RRYDl~>+ zhN==gSsgd~B8rrl{8d2Z(jInz)n(JgZ~KOS6k-I-|Iit7sQNzlo$X)p68#P%`Wb5O zW_Ec#;B?N+y8n8&bOX6+AQ8b_D9 zf`A#fzX2gva$@Y)!PRI|O4ZP1n=3S{Jhd4L=fmdFGKM=BR|$6|2;xmw{3 zj%VW=-o2+`(d^9m%=lQJ`;FPfm^M2^tix44O3;v9;tumPUW;NnLAP+ZEJU_uJQfE~ z7AklGPT*r5@YBIi^6x(5zID%Wa&xcwd3S)UY@81NAOlPUzkuw1^3fbCUnm)!vTN1M z_Ob(;OgxTTq(*R}b9>Kn@1e1LuLeJcb7_TgV^Lxh@gi|!Ld?mZx}7cr98r~L1zTbK z6g)&W(VW_jET4?45IjA%>mfY~Du)kh!)Lb&m%qtvBH(q?3jIk0zrD~*n!J%_v`A-Z z*@JU!Z5%2cjXx>eA*5O?i>lh^o#v6EI!uDF>*BwJvTyVmYPfAZhLN`^!3y5tcJbS~ zx=ljhxd&-f*LRQp6wAKM$I{X7DJB2}AoUwe=nC>=^`>!t1hLaZ_wRhsZ7@)#GBd!et(rTX0-<$6hyn2EC;XFTZ>!SgiKS`!t0y)CZ^*rZzzhEoM^l$rW ze6%nbZF)4w{gtPs*berX;NH?rGV}fT^&;|4i4M<*22(-LDII!5p!}M@A++!L2ioDI z#t+i`MC7m;J03fqvVtvM-vy6;LY3ntGbXD1)rX+6PC8iSDW#h$uk${A(k`*QCw+F( zn`V>XavqGZGQtG_qx~Fl3W8npfv_Y1s)11gs4Z`bRk*^ji7g>zZ@SbL=*`HNKwV+? z)q7#hVS1U)rZ<|2+}d(|B0^H(p;Uj=CWPewWP6$pD)2um83#Uv{{)=V8J}reAZdQS z6(W^fnI}j}H{&$BKV78I1)BxbP6n-`a)Go-HmTKFf|0aEk>aNM5Y0!!DG}MF@qu-M z{MBj%ScW7E6E(ulK0mj8TVJZf^W!yoBq80nAgH~Ac-U}>OeAikn5*P%5(FkpPpQ_5 zLaM$`A7%x+fgp#53V)NCbcFp8rE3_$^q|xakfC2K{IRV0 zoV|j6)&JOAOwAPxMv39liT16+OFT`s3XB!M* z7ix=fD1hoK>L$xK%`3Jq0Zm>lYDZOPED%lo0iFbRs#Be(oBXbeeR`6IH#zXS-P z;=hM>umZ=>3efS8-M2P;TLn2hP9UmA--eFr3fJ+$ZrPKn^h*f*xY3W5Sz;Jp0`O6^ zQQ5{giO-+V*d2xf-Ci#{1@IP+YDo>H!9B!bMc6WlHpAV5 zn-UMlvPo{68faeeVaA*}#evBJu#A^d4(n6^9A0;CMTn2)5G)FWZq9epe~QYO>})5^ zahbL%CHXSz{F}{;81j*zV$B-H!S34lf|1jw1HBj4+8|iyk?=y(82T&ZQkBrod1wnJ zwU;yHOX4w|?&HG0FW#Y}?pXx45=(Q#!mF@j2B$dK47 zwpGa6g#;{kD~2tSEU}^0yL0XaF5=8r)FR0ToY&y~F6B8yN!)@PuT_KM_QFgD;~fV4 z-U*1j2VOz6F;v~stu7;^e?!V9jSqVfzx-h(a#+kmmZ~Aq6vK!#sOeGIE<(Acv}UlP!=Lj z`Wmaadx}YbC-gUiO2Q9^sB7K^c>L}<&JNxN3c9(Z$W7X}4!i?O_lvlKlx_tD{ujBG zF1$!|2d@A{*Fkh-Zcf&rI_^`hum}Tq>;7a7(1Fn*Bg#&sFtk}$w<9wrRk-0BzFs;rQPTxq}|Q^YFJPP1>JEB2F{Jgq*aK#I9G>3 zx&F-u4=4lMT6>ce$YG^|L>AOc4c-Y3KDv3?MU$8UGJTkR0G+>t%8tEULq$dQ01I0CP{s?7LJdTZAk(vT4$lt?W-aB3r0 z_MqnjK8|9e+EY+VJ(^3fX*l6TN>US)fg2R)1}4t9USXS;Y7TcQJWB9VJa$RT`D~H; zHZsI)S}-oVp^*W3rv2nVpnfm-AU7KigAFnL*il&z-?1nR1lXVQqB^1qV5xO8Oo%aR zKp=3a)eNay5UKF&2zeL=7V_QTJ%$>3iA+ z15KnL85X(+<}*V?F}o=fhpbh5wXf#AzyR~1I?8k7lz?6+^@cdj6wXEXlXQSuu$4+g zSQzralFHao`9#v(V6g2Zogw~>4gfr8oIl?@T$}2Vr2W zs>`lgeK`LV7Ihe-dYCy+CgOExclE;|q<+WlM`RpEpQi+^Yq&JOMsW%O4Z zov0+;KX)Pm5(DNY^v=V+i-Q1Swt>ckulj-Y%7_$59#o9%Pq4MI8KE3@5m0d;K#84U zC?{Jibllv9HH46u>_=N!l*G%KRE+8j5|NWF?5GJ z25-pqkLW1?0g!t$2?zY@r%ABr7)Gtb-?C=Ey5Uytvsz;)nAJ54;#-OS=+w1>5ht#O z>6gFBm$BaMJPJX5aT%$fn()jo{2M_LB7Pw6K2oegNfZvzclgYOLNFHFV$mqyERSu_ zDm1OA(tsa#k%PuzOpv_}UgkWZGr8=Va1y{LVLSvVFh@a zzZX4Da|AG{WmIQXphV^=Pl;q%(w9iinA7Ea(v?;HSCQUNW#<1N=;4TA6giaSd3cG}UPeSr7xpiLflHcshU;s#Su23$jZL zX+(wqX;;|;HHL=*WEO~(5CV)z(2C3QU?;SXHHi^tMFc`zpz#_#LNx?GoR;&=eGu~; z*kBbwFhObai)JK#{k&8Sb&#!6g>9VIuoYq!igd&Yf(Ex&P>9or&|4XB=A@qx=?HWI zz0N2A9~oEHoTUwdOeU_a7pTUg+%QI5^#e@LqKP}~8oAPAfZX;TI?%|n%i<7Qn}D&Ywn88oG3x}???`dZ_prIYjMIXV=$%by&d1&_$|smvMj+o`^w<|O zAOP2Vz{rBkDqgUqm8xtAZVR(a0SDrTLr#_q0ScZWUM>$X*+cgdt8tsHiGn-IYYKw% z1E)O2X0PLX5dQ^afq#N#=3{ka7)V+@JOJ#6Wci@2i!j$W`ybu}Yoz23T14ZHW5h## zqo5854_qN|3|WCTb1MO0mLNctW)tqXwDb&6ZSrZCJrK+CCg~N#7dE96fa^gu(@`!w zu(Na}j2rLKK{^J~peznxT+r{tyIMrZLVtjy&Nm%wNsCA)0neY<3_r22#(}J$WjO4I z0I)nh9@{-ZpBjj1YRA7T5H%hCAw8@m?VrEC9zvRs`*6CNA;j-Qdid?fJGZu zw~E>4!$5qn{1NO8p8)e$*H2H4;Nt7Ikk56<`B-})$=w~X5&uL={Ipv{n>Jv19VXe# zGfLDm648uBQGUGRPmko9w0|j{OP=5%+N#ogjIjgtth{>zcL;3y_Uu(~JrBzFapu+_ z1wd;_?n|avn^`(!pSS<&z-xw3aZxx|QrR8AXD8?I3c*2pDg?-xq$Gp`F-zn+tyud* zp@M#z=S8r_7Pj;ofhTIWu&>Gc@kO;a*)2V|H+iAd0w>yQDAJ>T7@??u*l7JHn-s2@ zFq7c$3oC~IFVYphSvkve6=dcTzQmH_R8$2)Ez0Eol!{4@mXdvzT6SX)ohrpUgVR0oB z><tFOXgZ@mHsX5#O6T7(7TAEl4{q~Cxz0xJkwLlixOb4R zu2mmJr;~f$`#=zC$YJ60`WhBfynzVCF#d&;h}d&LMYIl?at#{{``RdEg=lD=#sI$l z1=d}~Yj(?wz%CM>AeZ!1KRxXo+AU=$eh-m1A9Au!F_J7eP~;q))k}}j z8(+#SB=eWO4S}5Qzv=Y90nC#NIL4OqgvSE!4`$GcRqdi~PzwF#pn9-f!x9G>N`k)P z0bPaWy6XgXmrsZ08WrRYIxUQCM$W&l3n7v+C|qbGXi6p<^Q;!bgtJm)fv;&6S^OE` zUBW^O2CB^tGqIs$YT_x}sZtsT9FFtSrW%!<`D928zc(U_o^r58>lNV+XQ7|FR;-Ow z+oDd;@g9w#^3$r%DyriKjn}7901B4shA>50V-Y^A%#c?S?K|;nWjXQOO-sw@#gkfc zIz#WL{GifaA>R)DmdDBB;Jy_`5ta~FIj zIaT_eHw1B+QN`%S9i4rPA}1hQiAP3v7$D6ngQ|#rpFIZ~;Kx5k)bP=3gN?)g=E~ig zcwIz<51Y#Kr-E%AF!$3dcY}Tti%W`^`QR_qys%EY!(5nxIw%(6%1;y z$88>C^Q**~o>Yp5GV7s+E(;{n#3d=kC|mHCnjbe3G}BzD*Aj$H9}as6Bx)FB2~T(l z+(otYkElfcfi~@(`xxZR5E#QxXE?SG+l>Z3Has3|LXeKFyeGt)dQcusv1}TIDGYL9 zriYdDv&LanVP2UMt&;6*j5jkhZ&D}O_OGtWYMeR`DPa=|TXY4bK{>%)?K^h9rw;C7 zY%%k-!fGonv&OD=2T6S*Tki%l8wP{t}W6Hw+<@|@xF zFu!0YAB&u2ye-i59n-=wHaMK(buwKwP9`$nA^LCJ{pV>g5LBH7nw)^^wX70-m}Zjm zajEdzx;h2yERQrbYD;qW3rYH@o431}`aTu7xLK7{%m-n`KF_}B3e7myR6IvA`ZW7N zh%fiU6ZDDEgQKhFfx%?7C>BUKv!Mo{t=s@J{$k&ZAzgC2#ubS;x|U#utvZ}6c1BjZ zs5|2TZ@rI4t8L~Md+43?B#r{SkWH}==h7Z!WKi%YEIu2o(M;go9R9iLk&$(JYP>3X zUBE9DGI3e)#ojTMKo8Ma3^6?|C%65e{7{{mn;prIGIum91<41EP+ zVzIfyNrV~F*z;7m^NXpfP5Y27+3YD4%k=hrfU6FDaL>3f=K;=E?}vCA=Mq;MbAS$C z9$j_WEK25gU00>Sj`B9^lKgCvIb$&1b54|J+uq4JRyNE-v_hxS%L+mP8v;0z)jhud7E#4Dbn;$-bRNKufC8EYtK9Z~A zFC9qN+9=!Oxk%~6NrhD`pP^vyG2??UoiPh_ToXE)TPyQI)S@y5L~`O7$QApdooYVGOsT--g5ua0ghw3= zLqF$aG;C|o5w_H_Z=4$cSeEICResg7r`?(Q6vYb>q9fJsdbBOk z*-bJ&fG@v3b`&=80Qync;MXW~K$K(=VO0s7b3J4qUG+v#zKGu_en8IVb+r?Q*^~@! z)|U=%6Jn)F97AGOD%M;JvA-GA)z20dt|c+5TZ(6qmoRgR%!t*3O>2%#YRUH zbqRwRd_CX$9*pLIgIxko9yn|@8zawnmWM2>%s^E3pfWyV1@@4piSZ0Xs~d%OUR20C z5G-iL6wJ2~6s0$;?D;9(^I0yI#x5mLMd-1Rr;)oAK6u~c# zXBDAt2w%;ft8y(g0T%ZmP1~u=*4sUkuWk40^d}q6?j2mBhU7R#2d%5PDJ!z!Dq%M1B5fc zBw6AZqErX-tTT~XdaK>?t-EcY1QsIp9MdyAo=H3qk@g?$=F$Jogep+*+b6GF=sC6^ zs5A69=%c9f`2FK-9y6`3L^j*!##sBVH;C&EhKQHv4`}E6YhM6-P%;eu@Djk_&iP4Z z$gCA12N$w9Fi2?u1-rwT`m|4R?D>iqXIB>IHz zd`&0(S4r+WO?RDdnex>skx3=o{fp=SWfX)#VkYz0MB4(Kl4GX4&5xgX6QHb<`kA0^ zS=cJAlHQ9?MAN>ZmS}|b-OnXMv6Sdq5)tRkQh|a5Ka|EyW#v!a7|1UL+__ton+)B# z3Z~te$>|AqRhaDBVmld$H(Pk8h(27ITpXu((S7AIU{;y0EdG(omYI@?+_R!`}XA$}PSv=@Ze!$-XrZakt z*AVwZ^GO^^$*NIpT+mWj-XlgzEmI-Uc3x;;{D6i$l1F0m!*v~V9kVkb^`KqSy5)p!P!Y|jZu?U!(R&Qk z!!;s)|C6+2*$NZ9I7L(ULr{EfF@~ANaLoSOSPP4!#@tOf0SaP1L~!q;<=KxH3D$u- zhY@_)Lqq=mb6AWagDoAjf^ew-)sYkqQ>tceV}xX?o1tYWVJF`{PW z{m;Pm@hM~V0H<$(z$Xr(%(eLPFHyb}yHU^LZS*yAeVSE-qM*=Uhoho+i9PR=T0L1ArnrRPC{bn!IWrMz~%DJ6GZrg#vHIzW7!A&mCiZP0on@huTjHl3tCF6G)KuX;2;m?pQ2hJ zVvlnqVpfPqe0PoI4r;Wrs$xA8?aBq31?2(tF|z{lD{AoDNmzeu#7-+pA~f6Nv(5#m zLo&!X9PQ`{-G|33&1M?b`)|IahS3`)GF=cy4CuSX=Sjt;-Lj7g+J$RT-*EJEH^_K2 zotj+Vc%I}kmRPAqXy$zN!CEwkB!t__^4DJm_&zk!%M5Cpr}{8lwIy*LbvC=7F_m>v zpRVszKwlTY*Sp2Kx?O(}6Hu~MqAoUCXJ`rD{x8>c=Yk#;~ zoc}V79N#%!hK?kF~pCHY?R|t+NhCJ>&N1;?b;i?QH+Qy%FH&g`q|3LtAkw0h(2^DLxHWDMNk<7C6B@_8P zSSyNqIGTV=@3U?Ap@stm{0*58B5U-5rxt(?73$^}yA>i|YMjL+K+J8M0bqbHFyvhL z%fxYrdm<9?Je4pHB_uask1&no?|1xFEB-Xb6^gV^C88IwS+HwC!X~ovK&27Xah008 z0&Dj6AN{eqoi(t-UsORZ{6jnd=JH!iHY%(xrI{rpsX=~!PIfr9xo5je&?kZPx-hF# z0Hl+y+7EQ=yM@rmmzW-Xs{n72RY=eDFKqOkhJUOd+YlB%F@)-t50)ebM-#X7(^0?> zhA^W9<@!(=gKP7FNWIAW+y5*{09Y{)JeG*UP%&3;0`t^`M8nP7&JkExnI@e12aP{F zruLmVyL*9bMEIVM>UKxi<|+V_pV?<$$lBAgn9~idih!Bm6)G8y#$BSk2TfA$3)fR@ zjA(|{Ls;Z58=DjuG!JO!fe`brFxKub)@$_rpar+l^H|u5Tk7{j|BiEWuHX&3ZHJ;M z3}S%a4jDyLA!M&!9iM{xA|diYidcImhD`wh|EsAZu%J&4FZ>XwPYF)e7Kq~dzU&!7 z5ESr;vc~^A^rrU}{3Ms85=hTO31Y&6yF{%&q|V#f?$qn;pq>O#1LGe5m}{ecMM>Eo zq{}|qx3r2g|93i|Sx__Sd2WkwZ}=hH{XgsG4jY$p8_}wC9R4g1G^xZ?C!gI^zrO~B z!>5vGA8K0uyrka#7p`@Rgzb`9XMsZL(A>asebut-yaVRryyJWSFH(Y3!Mz)i z`1$pN0j$l0X<`P=1FVTLt7o2{q|viURww6|n=$*{f6efA$(<-O(ws=a;b;iUcUaVX z|C`bU|Er^9LYm;meb7LbEZ-|1&?-=a9G2ZB%A5f0WLkbZASbQ-dQnizJ{k<^@NZM1 zdwV(JLFc-mEIuYS0{)}(1O_cFT3g~u8HkFrZJ9ty{AINrU!Be3O4TXyy8L%nN^KDn zRjyhF8*EmfcnMqN`a;8C$IDyCzeph=dDu9`-%vNO&(~5fz7yV1H&NGrr$A|Lp5&jW zucz`gsX0A1-1~!7mxtYmi1MFW_=zwOQimC9gbp-#VQ=xhZ>R$Qh3(or|7cPT<3E}= zuKd$rd*O5S4ys5|hA>uxb7s3wbh!OC(GNYddP@rEH>2w}81dz0w?S zHkwUP8m7`TM{dI`|19=wjOzmH?`z~wh9LDq zB1LSvTc+Mdfu5`JPRAHkWp|qIF$w;*dOW)l@jrQ|V3Bsc3swhS)c874bsL(RDRXO~ zBn2mH@$JNOQ+PUu6N<@PuzrRtMis$EO8A)rw}c6<9`QX00$QXP1weKEm@;g#lDfQ^ zuv{3{nHXeGO{;ES7Ydb9muD@#!mj4~upf(# zvR>PCg7qYlBBVAY4{c5OwLa;b?9q`{UOBKvTQIG=kVZ+~mPyNB%Rvx>>D0;pG)`&U zWGDP&3K(?!&DS@aqxTl&PBLi};~5p+u$9(ry%3fAn>Szsnn4u?`;_`t9sxx?!$eCx z*jIhTMh@1aeQwzdKC!?mRJx;FE;bVj=~`jgo?l2Zz4kUJFcBnQH8`qvb|p+R$s-Kc ztfSGH$7FTkK*S4+)`X@o0L+vHN4i*5C51tTyU#r_MP3>+*PcZb6QxGc)$WmF8g}Iu z6!O^g)<3CW3;@=@TM1fOIH;3idYgb^i`xh^ejq@zVLr}mh^1&)>O&t3L-j^%0F`&H zRIuEDeVNrL7anCl>wlS5V}$_h*p?fhPLT;%qnJuu9j%G3ndNqCuQF183z$oqKSOfO zor`0IkO3Sv@gIoT$H^{VmvMp2@a=(-HwHPD#PU3a_4=R1oEd-a&)+0X599wRiY7VLZ#Q3PAKk`qNxdlQr%UHV#=9UR@`7*4o_# z`jf)~v2;r5dV9e_g~F@NFbRBETkhq?`vb4Pr=BQY2oYcQNpi;w8*eYjh{9CSihVan zolOvF3%@+DLz{NfrE2}_Jf>>SG;ZtU#(8!r>v{ujI!+PhU*E8=GF$OXqb0BYmsvs- zg`IBf6>45?gSNyPLKIN_;kG9!{pfy?!VpDdn}sdU#8mo7d)dI`%T|O}1Thwba; z#~c)sFm(LPyxo`$)AbJz!1as{EnH_bS?R;-!BMMe4I>z;{zCA6N~FV1JZy=Kn=R_V zHhp@bsqG}?v{{N5XDvZ0Kkn?`p$3a5zmZDuoK1rh{V9E<`;NJ{MP(r(`~jW0*TDc% z#n3k%M`TB{-(1tMyO)DV4+~C9#(wx}qBb?-vdc`h%43Jh;qsyvI)RnBzq|x>CI}!U z^fZmudm8xq9Sx-YiJEXQ@L;QQPCIJf`J_4?!nTxUb+7wZd-)#A&glhCgjLVNIqeKH!K+)lKyM$o6ap^=Hz`^a<0SQiplikj5T71Ah%CUnUpr_x9xXITp|Jmm+A zOu&a|lL^R{Ebzp(`3)v8tS% zH`&PH@G9HegF?QkA~VrMw>&b8DLcxu;Js?xonxIF=`P#%Mb zbSBJ>$hl;Th4lS@vo$Lud3#A0D`p^2q$ZRYo+oYY8!T_h%ks>T#qxEDpCY?%(#0oH zW(O)Dh^~cc6nNYI+wLXg14K|n;DI{n3*;8-(UD*es%wTxZ;Jxd*3GuRQ+VBF?HM3^ z01iUXha+rl#1!rFZAy`{X6(Tjun;FFtE6Vj41UI23M1>fiEUL0W-FNxPMf!TW-n3V z08dqwt!ZNY;Tl3vRmc_f<~S1T|86#I*kwEI0b8XpbVTaQ-YndXv$xfQ8LXTn!x~RL zx0Ih|ntx>5VSQ8rW$b7&;?;sOV2h2*N%**+sKn}Oh+qB?dYTeQ++@L$4PK1HyBAS+-+#O=msrve6%V)PFo!(iwZEw66YUoM-4p8AL+bb}R- zsIPHzg)jgSG3kKTwT(sc6Bn>VXk3Tk?et{s7m!*F8G+w z^L*w?1;&>4VJ6~@D3Fs99}*AvM8Yz^LG)$09O&2PseJTpm;18@Jdzeeq(eNIpI->V zF}{2Gw}`-fxiC@Xmax^6@{&|pe;J$vp|0MaBphVkO~$Pew#l})w*Q(KIdCdjtObO= z4pwN4G+H5Se@;t7OOhfy`u+PKBkIB;uNQ?tV&}!nw`N;zQMvQEex5|%j%~G zAkSPG-kd`EQ_AW1YMzd+-n_YstN-$_UvCV@1OTeCwv zgG5svqVdOKq)s)L3go<A8bjM{DU>N>wno37<32{*I84GWWjh)#E_9vY=av?)k7!Q zG=)@JxMceMpG;#4mub}KOjWDWHas7y?W&ey91-3OHDmqG(@4{J_G1rw3`wN-*5ngR zBkG3c;UveeNfASkm{=G)U(eXds*?C%%=6^rBrZXLPeVcPv6b)9C1ZakSb(@go6dti zGb7$z&xaiQ2j>rVt+nzofx@kFXx6}VR2bax5O@}mk~;I=prot5fBq6=q5p%7lahe5 zE*87Hdtzk0+Y{!?Vvzf2g05k*lZgVX)^vG=J^0j6Z5E>pwP746xNJKM1U?sRlvxxr zjNI@U#3GPh z!mx@RXLs6}|Jcc3Bq)a+li%J{KcZ!B7-?^N7w^GkN_LXHuyB3@3F!(ZpxregPaiVd zptm)kt^hh(m$+>lv=|$pf_xpoHNq{U{RR!Tto*T%>SU%IQe&U2+r%daUMAYpOSQ}ujCZY*z ziRrJOj)l)_KOO=scuX}ShX>-)jhD-dPiAIao9Ywd%nuLMNB5VkmaUntl^Y6J>d%a#ElB9$ykKuUXxR zN6q%hWpfrXV5uD1?s-dA(d>&0#!*tTUYZ@ImRC`^jDuL5V%sr}VkH=fc+5{DU@y{r zK?&!mm{@RBwYkltXtigEK=)g{6v5cuMDFthNedfbx}60r+^(QeTqSJ?#O4BWZEwDCKCJ2u^ml<(Mfgbg*_ zdwls71HSc&)#ZY-{3L#RrE@zj#Y~gYfwKhtZufpN5k$M3ms%W8bu?yD?q2pGQBH$+*eRq5wzy`*k!W8%Cq)OML%*c+t)Rh#{nlhAltK`-p34MJIaQ zIvKJ@?eE|PO5+afWnwYGjy47m>9ektvIWn;^Zz`fg-rY!X;%i3q6FKhI`kbCj3URg zd<6OwmyuDC*#yU@c0jAgnT7W`26{4Q3I)+2G+bU2v)4EuSAe$5yEAd)^1>4B5Hh0P zfC|7AeO8)qWBfGDb>xxR=&FD6D|RTYk`o#N?bOpZ5@_)?;4CcB#0l;klcKz?zrmO4 zonP>G82gmx&?{Ze3t|~28n)40y;)v+o61&|m#JFxR21CGxnG;7c;47L#SJ>AJr#(* z(mvuUL_XS&0Nu^e*-=cyA4$^w-d-gps;9^!}C73Cesjo%>t(&b-h@AD`WlwODH9Oq&S zMvmmi#|rs&+8Y_>xq05@x-axJaMl$)F^o7O2Ri1TRaMKT1pnqJ{$Mk(#H=L+EtfZE06Xk0%@^6EKpG(fYQG0WW zld+PK>SSTet-Oay%gw3c7~OjF!?juwmyB?#37Lk>p}8VxSvVF8l*|v}yg(g)biFKN zEd3;z(iWfkTTB*a?<1|JqwxoWZm3OCPPoL6FYp>~k4aOmbA57|fGS9k0k^x0r%cCc zme2FtS)S^jp;86B`dq( z!!cBmh2!BCvtp8B`Rnni=qpqp;8xYr)(6T4)!;xLKCbroui@_20l+mI)}~|MNWGeo zqGHiG=q4ar4>XQKJ=Z@1_bLo!WM^>4_B*hI&LjI)zYmO%))Qrw;p5n-^o!1(@{3F9 zALekZ_n;A6dH#Oqw|qf(3(*_@oym!d39nx-P{A<`QQMzDB_k9%aaY#gseEuLylZ#Y z!O?473)h?OA4{fTaN2K)g~pzy8ny{=5ZWcODXEo|z&m;5Qjwm&fCgX|jntcVn& zg?|8nTcS6JykbQiRkHWJ`_@>j-ipZqruLu07+~HHd<=G`oZ!G8KwpZcdI?yk)|)2^ zqqQ=;T?kbjx`PRR856L@ex@YO_`$i}X8e`}zMW~F|3T0BfkDHS!y#cQ!piKjN0_rVzvsoXBl_xDYX&JW`~kex^L zu|!din{fbW5qEbpJ`hQq^hS)EYIZ!fQQd|Kt4rAmoY zqW6zx^Wky3HhMW588)bFv^q5~DJaCzOj3QIJ{bdcu2jkDsE6%;u6gk=2y7MCJ(4aVh5+zz=2O-D%;)o{g;E{06zVt zM9MOqI#(yF9F_l*=qLfL4sW>_5Jsxpt(Xa`*R^OARF3!t3mC*`Ee^M%Y*DCoE63l|6;uy))juRS%z#JwVT_Een2y zNAP7Z*6R%}c(D3LbQQ(N9FhORFeh8W-Ek)|WY~T}_uYT3HvXq@zLFc>YWsq)YZ;&n8>) zO@*o9%@L9;TAx+4q^ejdPDs;%d@4rG*>bcO1X3+TmSR|8={&Eq`^5B8kwYw%jtAjP5VC^K^~v(LgbZ zIMVCe_$SV-2TnEh7r{Z>E&1551Oq*7;0Yn;Plt;Cs!!~?Aidtt?;A*h?B_}D=wg(D^xx@Pnhaf!dvErGXR`Vv z88N)9id6V^*94SVCy;3=$(Y9J;wj#0asoebW%ZdL8nNd$UVm+fE;iAm+?L6bnUTs4 zJW`j1ksTc$$7@;t;eDEM2B48W9Wbl|5Oo)kAnp|YBy$4lQm6Y?2#5#`Za>MZuWQ>! z!4`9jH&Lc&iCD1acD2Hwv!Qs1x97=Z0!Q1 zO&Zgl-7rsM+L7D6g3c;hiU~~;Q(86hQPYV8E>2QIi`01}ng)0R&M5%%@lj?3WC$_`@@5pA9r{z>$J!f z?lKCBsZ3ynp3Vq(^>v}7RLkT6Ohc@wfAzPS+U#n}lbgg;CbUT%@HC9QZj{V(LkauH z9yJE!42Wq9))jgksmACfCu@L5LPLsG?W8A-HVL9S8Z?^R4-(+P-t$e3Vy31=E13xD zv&wY{A1W=zXK1x;b<*oD1IG4RmCB=A*kZ3OJ|U}~sOMVW-SXtkX63oKv4N8fd&`g5 zk;0#Qol}7daHf6Vm03woG5LkHm-laqS+MYsc(O~JO^9fdO!U7xX)cS+lZfhR_&ie zY;LtSO}_APFu5Bex?+4+8drdq~j6Rh&xgGJ&>SGqd~WP%xg105c=*^ z)n>HegHI#sJqO}R%?%z0~li zfY-BZ=&PVpHS)@SHks8Kz9$oZK@lkC2iWymbQ98=Xz&cpKy!%sIVhg)1BeyIxC#zviP{}ji6K8&BjNjnIfp9QCc?Y0Yp zgk!|-JNsFqENx8^z2Qe{`H~WyyAzMq+^l-cTBqkRZgh&qbAJ8y_qAwy#jPqSfq5*$ z2X03oLH-G0zbn70k!3K1lRlirS-mkKOggo%E138cWtemc(7LV~gv-?`_vedrh;$Lf zX!~r@EXrQc#nbSYpKio4(7Ikag(i4p?#qF-kUTiEX7S@}oL-luRGoQ9|6bz58Gx%o zx)?|zV=>q%^k43Ju?CvxgB8zNtHfBiAO&HmXWr=(GT@sW_u+HsX z^Kt^OkcMRvr^6I5yZt7=+6&Qc_a}V1%`-$DpW25F|4D%8(dy~tDl8+@C ziAo2zS{s$+R-DLk3b%!D21k1!v8+SurxJpPD~9wa%aACqcPQ4B8EP*XJy>-8hDk7) z%C4&-ll6^)L5r4y-@dBcW`hoheMM`Oz33N$vpWnSP`4EBX!3hEF=<|Gc6a#t+}L!~ z(J0etpUPk%SlnskU3qA^RyZ8;-xU2=prv-2X1`YkzAFN#9=IRc`Lw zH=WFY1IXAEg{`m>RHe!5nHtykto0`HLqP%q?z zMVP^+m7l!*!NzC%3U9_i3VlXVoSXZwBjXbb9%pf-;Al_GP^N&mkEGP{{y8sH7`POl z_I3I|V(d||o^ZN4=VW)JUiryvIfge$Ir;kmnQKlZDN7LpB-;3BzRFaD@YjH!d}d-~ z-0jeSI$v`7j5o3l8aFQ8JXsV|IT?AHjy=99PtvEsn~=|rXx4J9kfjF=0!V1C`ag)05YVIP9^;uktB;i zqs#itg6Vua(aeAPQ)VpFC~mteBQWS(8(yv8QCW(`*ZzO9T!t;-rrsMMKZ#A3azbeH7~i@SGNFm+HJPY*2{p|-C8u+B*V`J@fZBP6BW9j zcu@;w$RLR;uP90+;y_`7Hw!9XHbyy;8T91ri8SMtNC6ln2B|#MxJbQJ25sB@Y2r$zyOfNsEv<9Bw0|g3I9l|y8GAWu z22x)q(0L78x7PsZmw*dE01nLsfBA9zSJH-^YP<})UL*!!0i=)kYEP6(*40RLjotJl zf9la@_@`QP^@rkMZ|+5J1eUbo9fCXm2ue(hAqfh#JP|(5R&I@Wu~$N)HO!{Up3;u7 zk28_kGcK938)A&Nw}~4_2_bBO0iYGTNl##&!jND6gBMIQYjGmDm&=|vzjDBa&}92l z>mTU~=i_oHtPmay872u4OK~`Isx#kDiPt?8{`x>8JHkQKrHN}I9aw;lW{iS-kB1Ak ziuq7K4C{b7^uhmzs=m%lvFT$@z=FM^#YJL?b70%hm-bt2SS$wux&7NXv8i6=0sCYE z*+G|hr|Y2is0im}#p-f-s@ivK=3(0T`9HB~>(otFUNZabm(A_&do7%Y*|c%Rm}77; z-<#D5E)W05$rn?~ZC$S(LB~ihSuz;yHmvdNb`iCT{-$7)C`ICKNxrhDp+nxKH3m>S zx{QP?XjF2&7_=B|QO$&3*NKh$$M?WO4I;i4g`1K62m-3^P_hIkA0|SFQnUfy3+^6v z6vk&(vs{Ek#*cJy|PRrQmWPhJ1Z!OH7NmzOyIN8se>NW$b$t?_6)`>r+tXRpDy;;;Oc z7InBEr^!H~u^rlopaG~)UysyXK^jHyv2CBHt;;sogn^egN!{mIEtfT?>mZJHp+ecL zN~=0%irmhx-c7=G$U^HOS>C6zV91yEMZ?K2wmRB<_bbfw%j*SoR&~lfs z)zZ1A>OLP@yg&;@pA7O-&IHO+RS}k-5jH*F@@B0`;wmrJ7&ruOlF{o=1j^ zVg9gAXzVZ@n^;K_0-ssL6CDzsZkOvgmxwM*!-${nREh253iS*vRO2A%rjjbKlqfg@n2g^jUdKgI-@_UX1QyjcaqQTN6RC$ruC|Bi4(0 zH%31H!>M}<0qSP)VGgf{-A>LGBJxaGkO>bt|3dojwAx_Key;N_9v;q1@Oc+ zHZ{om{$?jw#9!*b0B=KuclIAN3`6Na3UN=l0AI_&tard&)}GawjK_{LbZM<$HM)#YmC1HT z?v-a|#f<_-)FDuZp3i)~46tp2B&){xDrN*eo+kISFMd;;E*J>G6EmDGqaGGitpBFe z(Y}9{+;#DOaM|lalkLmJW;L-Y3o_ok_~iQp!J^Lb0as9xr3k2BECYK_YY`F$g5BVx z(1DMhgw1+<@(p);p9~K^oE$*u3w<(5Fq&Z+vFJe;Tj`MAsQ={O^^TXNLF3*+)dK4Q zBl;S>*a@J34%v|(K|h(I$#9x1m+C~i8v<)p@(5h-P6)om#IEBOrpkjwg1t(w96Po! z63aGPs>$$2%dN%JYlT2UW2g0zZ-;AfkbZ0=1eu!X>CdH=6*7U=44xOholBre z)aak>jJT79XCyw>+5g}PU%NcO6ZKcrYaPF4*>iL=GbFuVYUDH8#lJXzem87Q&ey-SDHu++SqTs4%Y;pwBixnfMK-nIE+~ zbDx~U{hdPQDhvqAKB-(cVkb8s$O9gIWs0Hvw9qy(#^UQj z@A;%EiO2eLg6}0|a;QH^2Yg(KaLjCiPkBAVd(o6`nR9JtFH(`|t#+-+CF@z+X-?U` ze@|7+RJF)JShLA8p<%KrA?y9N$bOWr38ulQs(BmkvN|+ZtgDnTmhisq#}kiR+s;L> zQ37&_gqx2GxK|{Jiq+xRD+m((DV4s*vRCaJZ0;<>xy{zJJ1^4g3@7(lV2WY7!I#8| z>bE|?Z}^>zEiK=jwe%l$shwquO+FD+kIf|O{DIS^P<3o z@7V_Jh^)}i}@D^8qNX~1^J^<$zF=!ENemd{ekAj%j-|t<9 z01tpr{po z%hIhj8cOt3MY3`q2>N`|qRGEAs?wf*;oT0p9LQom0?EcUSTdvi84JGLeo)F-oiNut zZS^piESt(1U@KghNey9|xXDC_o;E9u(pgS4BKi1XJX34OhEfB`jBn%W_?U zBAb(@iL$2VoXtNno{$d+r4O?HQTXX-qE~|@YM@m2(7W)z1hfZ0Af4zng`*PtuCam? zm(34uT5h5eHq?G6TJ$TtP{VIUdt_lH*rsggEmW6aDTUv==s!tPKZB83g_eHtf!*U?b@QKB?892$lv;l>%cUpCcxc#6bdl> zPh#^^3{0vNUQiJBoWIT{?e%a%(|E)$574Z)4$zE@DB&)V>^9-4KPsH#7_~`v4Z^3B zo_Z!*6kI{*CPYLag!QA_d}$x(^X+Q-3$fb&_(NJJHF{HRD!C$#B`Jfr&VD?(OYW&h zTkgX@Vl%wnaKhmuY;uH9de1_bHy9+EAkvzH&$e#Ki?EZcvzk-*t-yj_6JZqtA!@bv z`+K+ICQQE1ebkT}-zmrO!FfHhCV=Scyx>%f!!lo+^AQi2K$b`Zd_?8j=QPY7APRfyn+?Oc)C#z{5zFreq zZY=FSZk!DKJkWcvK(gU)s8g9|Q#`%N-l0;^l`ttuOr<9x2!L0XE#))mS$b}LvW;6OznQT4jK z_dr;yS+YKb^F?=0sy>mksRsYxg%YhkH}}3c_wt36G2_@Y)(B1a#ZQ6HCt7_ydB$=% z6ZAJtuJlfORc|BhY2R#IUe`QcJ&EDIAF<2fUwHeVZ;Fn|$VW;t9tyn5m0%!z2irF-)3W5L~@IP=py&ecKygXc}wnqOnQrj03KoYbQ z^&VhJ{n;DQ5GuaOl@2k(eJEPfNjYkbMqBy>He z5|_>-C3w;qn(j8h4(vbYPf09~<4E@?9Lw>MNZ1f;4nN>Zy!bl!J%;iNbZLR{=saDm z$?T2&ddb1?(O?Gc5us%_Fhnp(%8x-fuURWA=5B>!R7dLQ!O$a5O(t>Z>jb=>1og+@ zf3j{LO2M-#$T6Z*a2GFw8z^x$Ha}Pg4_i1wxYk=Ck;XD&Q$o*l+ThR(OvdZ)MwK}r z`T?SmhMg^RY@vbl7|YcL>GQWNXd?;3u=MI(&S5({lzvP`qq7a#*_HJHPNzpcKH3L2EQN5!*cV6CyM@&AY!uvLw-cBMPprUGJ@w7LSm>2f17G` znI!Chd5rDav*$g=8iC-%XHKwlFo)1JNA)ANAcSr|TWf=s(9!&o6ibco3YW0+$IY+& zhq$4BiJ%swALYS)oAX6{R8l{6#_s@6Jul3Q-8ysiVW))sS*+Z!F01#{CT+g~3JN(*-$aHS4*ML6^VIRNX( z!NK;6JGE8MBT*<-LxZ0s9L~BNujcFiU>T-tk-*OBN|&tj`D#fs>YEt{*RkWNpn5yn z%ckdOBAtb1ig6vHlqhXLq*WMq;yCMm$oQw%s`$x7HTnUZ5a&ik=TE%#qb~Oc>!Z`t zddpW&1S+|=un7GDei~8V@6}qF_y$Yq#QRa+uZAruzdaHOUYW5jiuyl*u3Rwger^xHnH{WUS-!xUUv`MTr&vxK8Sdr&25&3Kn?PH`e*= z_mfs_dl9c2n>wT?ICsLqz|@x;-L^w1jGnIdt_Qte2fxl5anYvHC_7H_-h_Kxh3Up~ zbY4YR&;)1mXxDprkguH_AEUpisY+3C3^=InL#b>}!3$v?Mr#z@= zg&i=&M`?WkTTZnXM*id_jeIt;w0!2t6~44Z6z>YzPz<#C?!#xYe8d-=TOM;W1wG1W zGi5W_c=}j`|Njj>7a0{0WZEsp0Ri`RuH6+oFgzYhWy)xN5#vZs=XTEOk2+M_N@d@F zJqP}%iUlp%J%(fpu@#;w2on2+8dU|N+keOBICP!N_WCv71;c_$re1FuDz*0f$MJN9 zimw$BnEDI*W(MHu;(Qt(JTE|@O2;Rz0k%CQtN|G}|C;Mvj&M^3g4^HuY?U*?zZC*@ zIVajMd_)ZXSblSI;*_fhInz`^aNP<#zz|F4aj7&BdH+sUk(o)~ayGBI$3K?RWV2DJ zpU$G|u@%VeJz;W;FK}|EqUEn=fpY|Nyl&eYas9Xxb)wbQ(jp$oeN4%9GsfanDdWTD z2=G{0S=sA)Kk3>XF3O5MTWKsy=XRg4R1fsusO-2un~DA<=tv*w%K+Y9aDPt0e+SD2 z|DsfuwvaNo$3AV=JU6Tl2WpL z+Yg>z?l)TxYdV)-b6<+uKn4#X9Gvc<;W&!XPR|#|gkY(#NyQ?m z1c8sRs3B{?mMUyjonwKLU0+p>eKo!M#q#j?R^?H7rpH-|`fH!jgXv zk^qaL5pl1E6Wa$p)J^hhrbPMHtZL}pKE??=p&1K9lD}UfzssfkuD6yFk-K-_c**p5 zJJH3$6`|NE66n@Vw0dIJ7V-3Frmc+jJXXw#<+*5A?0i|f7)e{dB_F3qZTj?_yF_p#C?>PPT~@PhevK+1cI|=4>9=ADpKu3rcz=H{Bu)Y!?H2qJb@6#R%1g<2 znag2+Xr@3C1Iz8Q8+uP*`n~Jpr7KC`IZt4w{S3Lm{^N@1L%uh}35Ap5z8(y8GwJ>A zFgLW;?pAkgY4K&Y1|P!xDee7k@I#{}7K28GQcx0S<$^l9Ref|ee zStmIjEGA>oX*Qkem<&4Qw_iDV<9C|(*Dk4S7a`i^FC)h1#aVtU2%qsQs}wDo2f_1d zWUgA2LMg(iZ5#Y%tMhqnshl7j%`Tq31od8r25BA?5XCXeAW1RG(lJ<)skPV_fCp)< z8MrA8+x+^a2sJYC^78pzn7!XUT5e|~*?2=a-C$&lNbeS@$Z@H}dF_ zaKE1|S@6O{s@RA!pnRPXBMkAWbN=hp;36)~f6Z(~@ z^Nk|i*6(?VsB9yGW4%~MRmaX42D2{Hk#^-+&&gmBh^3XS&l+xhl}x1C-K@N|8z-w` zVqRv13+tVGw!`AhGt+zB=sL}V1xATI8}kKqmyp&gIdK=RQ=b3YU7z5LaNy>ji3<|o z6qazXS7oHjrL$kv>C5iXw3)t#aGw74K}P1usqu2 zO5tfs#6=R1)$yd{EpU65@iH?XQHN4GzIv?{xDxwtaUT88Y{rdBdSp#5sXBtk68=8# zVrzO|vSu(-wDFWMg^3Qpm&SN0xxZFKmpzU_-{EDUr7%azgjvh6YblxQEMx2hHGV0lx%`s7Jn9AwmP>Z|CnUze4b z^?EBh6MBO^Q?Y-P(=`=m z8xg!_f}b3qur9jpG9Qj)R%N}PXRjw*Iu*N(CY;O*eE?^s{5LpKooLq)HXb6QqP;a? z^c3optnJOt8N5%6aR7X8vZBo2rKx`T>{KULXExrN@23p>g~{*k_eMrL+ zyO)cP_lwe+(bnp4-Pdib-;67|ul>41E}b_7>OMbMxvBHVQ`scVf8Wii(%cJrzthwD zpLu`QO9n`TBQ@mxagXA~`(arxg-C(-M-AVO-fOgSE~UYE8BA((VLjiNtEZQq5Z5hhdFoR;M(EqTZyq0ZJ2&vYoT;+k8uSu z3YADAhDX*b(p}nLt+%XhN;yMFeA-rIykBsA0v3KTT0%LErA|R)l8MDZlsw;f1aOdq z_??;mU_m;-p3i(?92fc({=o0ZT4$W+TO&Z524G;*d6K$}-(;Mnj3dd*H5`Pezx+GJ zrdO80H9)aB^!y_Otk$F}P|8n`b|?1BcLE%AFxj9hW@1*{;7Ln7)tL$I6Ix4HH=*Kt zn@##&bxbg|nNX=18r9ppMC(;1!X{z|9tdeY6w3C;BKZ-8|>{xjmunn%9Fz@ zBU0`v;jdq`Pm91Xy);^~wZ)!+u;AxQl48&mNrbP@YY`scuC=2o2>R64yfR$5kJZ%! z>7Jym7ItYP_{RwUA=9@E%mHw9HOBVM;s0sOuz`OzYqU2jn_i!t%~SY+IRFF7svYq` z%%SK(T8&+#J=ULA&;wApB;ESeC~%fKPT@S4)44h^)#9dI67^MsLlhvE=0hU zBHX#%`B<3XhSS353_er|odj{>BiKP}x~d{X{BkBF_~%NoTc0#dRU`#S6i`eAe%$GP zC>NJV*gmXXcr~Tg7~$;3Kw6|svMp%k?c-rb>F<|Udc*0Zw)v_<&B}EaY-4mAfk$sV z&o>D(COk2JUNy`$e28fd-bdoPvta#Zn2_F*DYCeNy&t!NF<3h;d*Ct3$zUw>n0KoA zKOMRkZr|ssyp*NYY$t56p}W-@<5FuOC5{kiaz5PGZ!%Fr+cRy!;=E2sBIXsZ5d@6j$S>z|RUTrEv{sT@sekG&!uJ!)QZwA?TMO0b43HJ|Sp z`BwNN@CP+>CzcQY1#On`wE@>YXQ7f8*-gaimpO=zFTek~k|K8S454p!vpK$khDlw& zd^al6Gt~<=rD)sqgJ&gFzJHpSPyvWpWK&aYuv%1RGn}h0de^kG>7OFc)`(miEIrFy zqF|{uJ#+Qt7^$E3I+yXfCxvzYj_$5@_{l>ll0xyecr=_Jp294ZLI^PN8*AsZt)}14 zo!mDK_c3oN9h{>ww4dhmAQvGQc{{lu$)t%x75XYpg-6 z$1*%K#3tAn=2e%qK)iX_hQoV<#Z9xbue*`NMWJM$MIQ4%V_H`4yp7@+&|k+4zW5pD zs`c@Y34*PeoR25^s#4?m5}8{y`*KYIdz8gdPZwC@fc;`3yZ61;oKor`Sjt|^EM5~d z-t0la4VE};BS9P_+NOF7{*)x)LU77&9!om@=5w41cLrx^qZn0=O2h(Zw{n*v51TUx z`?t5Ip$P@#1PaI3DJIA$E&3g`z5nA^1=)Z@crA*4Jc7<=HvZ!HmpYeIB)dkDM2=)* zXfguk2^JUJL=Q>CrC=rUA57jReBhlEc*^`EQy{DEZgJk^Ita~X^#$~0{YXP-g2C=J ztCu-`z5*|6lYyTmi+J=yc~1oP8;MxErY>ti{ws}=reG8xmM;XbBCn9DWU5gAih>zQ zK7AS$wnFSJrs$xhGkdJdKU+=Yl5yo!iR^ibgMq*Jd-v~!g-{tE_FfY${7t)xiV1D% zB~$%wcDLL2-plR<5lcxrzo}@`9DWyie-dM*H#QT!PZK-`Alf+N%iNu(AGbyO^mb6! z*nZJ{7{v&pfi1|Mha$@|)W6krq()u;juh8e%hnDGq6L z{BF?lxav<$d{ZKYj>Y*=5+m+_apH8FWZa4c> z8RkL0i_o)myzAzou1B#2%R0U4J$Cw>M7i@rA2@%&gF8Zla&TSM0??!o0Ww~@_c?Xv zr_}>Wi#bavi!BX`*@_e>G7DfYlx+{E^UAciFcc76_lt&lv76r!n*<_bpusUmn1h6W zN)61w*g71|7%D|%-IH{GS<6qyA%368XxlS(+)dJ5Cnm@ssCKFo!(ko&DMu3_MSa{% zEow5NjJv+{l;v+lFdqdqqM7PeYuq10Wd{#Mfb;PKOIryxoG+$jDh3Bti%rx-m|Rp5 zmi6ZZ`)odhD(p_=8OLT4jmmyG5Z}xPgkmFUeEw<`x-Yb?7r6Y3 ziEd8i0o;(eZYv|-lG_bxU?$nQOs!hZ>VY1OAy#ELpV(>>B0igUFq1DCO%-Wv-?!Gb zFObGxt9S>$8mS>$6%VICEg?#4TeC>&hby4*PWMDtv%qSaYwCRY6QRw>Rf9;;+SP33 z&wD+iq2Bd`LFIFx>p6X5{Je`J777WLi?{kyNtS5oKX6to7ck)iz0qgQ4oVxZh|8Z6 zYEFU=1fUXsB-!D<{4tkQ@=x-ql(=V{6sb^my5#{~={Mi*bN2jbqe7HsHa&J6>eeLl z)%FTy{fQp#5BVJx5Mt@oYgZlOa;~}JUxr7q)YWbi%zJkncK!FWc}64dPN_>8x;9J{ z3MGM!Zc@dq$dZKa4IU|m_DP-^oJ^xYDx5-kv}exOZ(lH zOH2>x7t!NK4@1;Z(DQ>18SE&3tGbIWehi~=+U&0;{!wGi8+xM0D0IG9uedSZhJu7N z|44w7ab%|;C(2>Q>@_Fx(<8mf`8Kb&Y4ZhM=F<+Gy9NwA#)M?XH3qCE{Q>QjG!w*g zaL~tUb~2CugneTNUCgDI2w*x<7_C7VC^F=-@w!RFz<(Q2?aWmJcR@-sRELxdr9-(b zv@$3p%eLS|xyXk;8f-+9B=!;%MR~&8cnq5Hx6;$1&iP`P{q1KrYac_g_UJVX^K8vK zq}$OWUUZNp&168XA$PBf7)7G2PYN;bjd5mpuRR^ZduN~8A@&5p>e#{J$Z zFwxcuaDF$dx>_$Wokfya&l1EAWr&yVAXj}YhITQw`r#?#vbNH6AoDkzwYD{%XH8rC z%5kp5S1|1G|GWScZ8Es;620q0fcMPeRj7u~vkeO))emGE)2$mgqIKBk^Jkj)^}D3v zjZmB5XnHgZMY^?C0LG+=$x&bMO;Nq79%)bYR*L|pz>218MKk)LCv(lqEAR2?tkyjLl4iKxn{rL`BCaq5ictP zz9pv+E55HO#n$-+q_O{#QWXXxg4LMm)wey)+Dp2uBDHObaM7d5ptTTKuDQ)p!E_v)@#@k1l7I$ zeSv8BBd?ag_|hI*U?e_oRdUKGN!0iR+hV9Fq1D1_3EEPF-XWSxLX$+HyBa^O*?JG> zOtITYp?2G;>BEK@Un<(F*Q9o1OVKR1yYosSG*PWL=TXhQ9|iBMj}s(8HEwAaE?$Y3 z7PMLP*ha@rEhLC1AR^Bw>7UmG2ma+c>;;?JOFd^0LcjAOhnl6am3MCKvkbD_8%ePK zl_1i?{|Pur7%H{FvKATBkG6A+EktQK1Z22whw?ulFIND%0G?)D;OpR|egc3^)FjrB z$=767e-TZL4?yuX|E>tOEws-#0NXwa0v!ke=4hfb&dZk%k0PF#ZslJVmm*p3?MAe{ zDVy>~uVMytmC4N6xy@f7kb^XQowGE8Pt?9^Nc55nF`JwLvs}|h&U-7Ef^dmB7Eqs! zc1J8w4FlTBW`)YUd|v2U#{e)9fU)~+6PV|zeX=hHT-qUF{`b3~8M^pxL9Oyw&bT*H z*jkIBW1b)bkQB%sunTcG$}Us!g%&+xl{-#rkRmP7Nlc8frK56>3hN{Rn-8=g zSQKd1CxbwI9H#p$i}d;hYt9uR9YU_QR-_2-sXEsmQC;aV>n+^$H-&bGNdxz>^Md2s z)8xm3&?ra8(YQ*Ut3Vvn=|abSgNARZ@;k#{=b$)zwf3X^&%vKvWX{iB)wys3##?xE z5epb}2CLTqb5!rh?XBa}pBquuAv`_MqrghzWu!zB_BXGoeWpvk|ADXXANqVNwQ{~@ zbA=PJAOnWXcA>sqc`(#F_LhFtRC0x`$N@rNA>h*K%Ix>$G)TIs-$9Of&ctQzrnXsX z>Zid1SP{~%84ouX&q0j{P!NJ^v$6tf=8*m-fZXzUXb=dADdO55U;jmgHEsHD-*OI7f%Mb0`8?+tIzv_H8z;9?Af6Q~y7TkQ69*!P zY`Rvy1vLE<1d}nJnA0fRN917lln|?AP@8<6hM?aNX#}&;zkLe?Xz}?beZ5fmX`8DuU^}Cu!#<5B`3uNe$KpL zT8V_+^2xGnEd9AwlEdvK8iTNn=X(9Th3+^QBNYz zz!g>MedUed1rK4w6Wsg}1t~1g{N&qDjTvaRSx$F>ran-VPV%Z|A_UET$dHJOM!9+z zoAi1_Md_$ph+BM)eLz5x`zfM7Q}*M)gRjn3^#cGDOVQ|M2i6>z1Z_*uzSA=4Sy?2a zF-2jf53S`#V+EETa0}CmvHYaASt1H>FoUR3bzYK$>zi%qklNFW%7g@Cg{9{@A8hg? z#JxXrSvyKCXGKue{8n#M0`HH$;R6`c-O&j z-Y9=?`qT^dsEwmjPGSY|3lKbM<4#S_InqMO;LrL;2b7qku$)HQNOC2JB%mUn<9Qz@ zsBl;#8&5zfIqZ+WNVtqSvWMM+kqY!7lE#)2yC{mAUcRSc29H{}+8KlUtzpGfiW9>4$YMFrw}0%)#hE1rGLl_6bzEH{$N%dM+vIg{yYl zx0u!dQPYq!+;}hkvN%}Hi2O$iS)TS3)&=vo)x$mF{NlU?-OY+s(LV1WVe4|H%Rr0s zcjfvvd{=b2U_^&9gDt}&+hlJh!7HfhhLW1$ zgNkHsNC>@A0RR~}LiMjO1ymlrn!S-#o-x&w2XN~5ZLBU^_lmJeyRocEfe(+D3gOb^fjKf)ekCF!YBfy8`0(G=J)ku7ZW0MKfjvqED( zvdekPL8}37n?p*PBEvBN2?aDf>h0N*X8WBb#UuG7&sgdW;0cQb(gn$L5*rV;lW-#N zcy|AoS|p=&@T`tv@H~=yCnV+!_4N73W*w|FM*7+7MGv4ZB$z@9iu4`$3+4fSnei-7 z-V;lh4AG1Ly6nf&Jtb#?fv7}5svUeKa|Jr(t!D-`e)Ws;dC3lk3F@`BI2L>Mx!d*L ziQs5ml4XpRdlzgHj=}K%czVajI=e2~d&jn&v|(c#jlF}$cGB3k zoyN8rTRTn~+qP}v&3&HtoWJ1uvgW+j8uK^i@YLg29UrRgq7f<(6fWnybQ@efFEC6r zKY%WN=t1$BYBcJzh_BMEj(HOE_vey@n>*gqAPlYCpI`nxPXh*doXMolgZH3qUqQId z&~c~K)$3Lhs-&QO2A1P^fpOll3%qiz@OnR3R2wqx>uNioHVJ^63O~L|44%a`@G)Pj zi@+qz%D#&A9y7|)2uUK@z=*vvKU9*(MIK9csA~%hG_Y@@7=sexI#&-4&~Q-terZtk z6~g70L@@$)B$I9xsZY=YG)90G59$j+jPbJbXBK9KPIo?CXg${&figmj3O74}_9IM9 zp~@9py7_{y zh~kXZydS`_O`^e&yfy2`m;jffFr_Q;9BjADp1;^54s(gvkk>+lmD4Kq`a1gey@1X| zCe}i-81p5y&{CL-yeEyMs!6r|my0Ehls{aXc51%_@FfGYA>r{KK-u$q@*c)-j(;k% z&3pL{#F-Juf+OsaW|36bE{{tzv}`|p$67he!p8enz>BSI30ojYMzzKL z|5tBOIpFKHI{ivfGcEI!X=wCBAT7lz z3nqkD{b*`iE@~(oqL6M4ggMlJe*iuy_ZBZMgh@Y`&24r=9-2PA^KSNyV=lqgdB*13 zfg#_G!xhf}?Go6;wg>g#^*ht75MCZXB_~)7GgVr@b*{`46i<{>kr<^0-*zwVj=g#E zuzwGKy`U7lXeHBe!uq8aeCe#11Dq+C14oHb3>)|2Y8}>S2jS!TXYTu9(Wiyr`!b_3 zifi=z{NmHzDwwvjXezl2szF&rYF;>kNFFN$VLmp{?BGs}Q2)@|b@NN&vu~~xwCrHF z+$U6%8!TTuD07OP0ez9?z=gS$z4Q*1M$II{*`y~G09C8u7U33pcVZ=$UlmxQ3Do)3 zd?*gy4@7b_JP;=|(Npl7z?iYv(J! zxFFXrMf;&$cJTy-(;IR)hkd)nZ?Y)5pqy-)JVj0|q!nJ^4hp|?WWJozLAL#oWDGKWWHL<6;9mD$T6@ z*F*gR`H|c|LapTX;I)sXI`kLq@@YJ0-azJnv8Kz}wzlcOePK%dvK>)wmj|mvwGR*( z=APtyD8KBrSytI7qqZu@|Dvr`aqNx}PQB*mGe_{D+OSQaGB}7DJo5hvwU8QsKy+rN z z`ovPPeQ(dY+nZ1m!3dK1qCvrV+!)AX!qD?U>hz4cJ_WSCo(^_iiovAK5gvOxWkMJ; zZT;E8LTU&f$)s~EZ-7=`D4ypR>0HpBV^Qs;R&BgUMiU|iFHcQ~6b6OG#);)dqt856 zy{r-O4KKpNJU2h8f9>wSVWN(W@x-XM=NJ8>D>sJDQZ0sdY60|2pLBEwyNU=o;@y0f z&)GC;Lj6abSd$tU=nkkdocPn~k{VdRA7vqOBk_uL3*TYUNF#4ct>d21wjPad|Dm zo1)V}z2eW_ek8`|uu^A)JFe?uZPbb0R+o^HHKtFwhJme|G8gRiG5Pr{XagSa7}_ng zQzU!f}n%lb+V>8=B zK^&kBaCS*9_=m8CAlI#`TBUt_h#R~V1&=V3D9Nubm+Je801og?Sl84boVJu}r zf(nu}=(@%Pycm;E)-Y7nO|=TM0E#@)>=DZU?}Q(wU<;W)y3V@J!uq@pB!hd3agbk) z6^374_ubU5U=3&hOQFi~$Ow)}&|kR_mD^bp8(|g2@cr$YYuUSRg zBce^Y30|l84M?)~1x_`rt;*4gu7gtgTbo5Hsf_7_WiwB4`;KJweyYB{`Sd>x(Hdsnim=5erB5t#fER5<_=Pe~7 zoS6%Yq;g-ljU3r56zkJLv26@)q<7h1E?GGqU0CANxUMwij6696iSl8^{;Du^xVelh z{>QBI7lmNApk;aUzkepBq<_n$!$U7OPtE)0=exJhhrHZZ1pK#5e|LJvC6#hj?izRi zL1Ly`=&-+Q`!119)pj!rs5_GchSI-s=?g#NjT;3sGlvZ2CcIq!l1ak8VHX3YP{mdg z-31OXx{>X)he=owGfg0{{Vb#~yQi3-qV|Mtq+|Nld!XV|#aRePH2j9CYu>j}eC;Uc zMtahOQ{xH=7j;~-0tu`g|tRKSZ-m^bpcPxII3cmITB3X6xsl*i}ry#Qnenbr>ag+*{*}!ak0k z6Q#7vXn_mZB57V@|I#O_qtz-x<}s5GO&yY`{Hsy&t!|}y0<&A;B7v5p5D6l$zJPv2 z8aXRVUMAEQl~sm^&UaiZH-;YXX2SEb0gF~GnYlN>^rd3#@@UOKt4Z)|j1Wl$chGGu z0*f}2+b|$E9^wjeE@&IfL20ArMU-e5hPR!|>Z)7eavs|s$HIB^w4kM@=V~gzWFC%w zSC*ab2 zk$U~e7L#BLuvlpE5>Yft=es+CU_LuHyh&NVI3~q^+p41Hqyj# z8{XFLh%iDbvbHNs`#)F6|3p;W4De}KeGc0RZO+|3WYp%_s?uwZCTAJb%UYjz%c&ky zqY`dwvYmDfM~UIo=eLT7We$ae`i~zp=}g5}v$&(`7?m9-QIQim9K32~K*0~@hbIwZ z45cjwseKhiVYxrwWvBFPXFQ%WYtlWVE&EC$_x)OygkR#GrGD}EoB%Qkup zx2s(c3d7#@Wc7;~BgwN8Od@W4jDw=8pNh}f5Czf5PyBv}Zn0^)LcARaWAGPnArTnN zg?NL~$(Q@n_rcdV&TXlb7bAS|&>h12pEPuPqe*NnN{`_G4z9W|gcU=X4nt$FVF|K~Gm)==%Prvlo z@uqms@C^s|XEhs-lauKmJQHFjMDQ!hY9g@LMv}^nSqvKAZx*|*+>ps5{vyGOVsRq& zylEDxu8A-T<$o$8F ztl9jfLg+g!IU2Adi_2+rv`a&pBKGw^FqWHJ{RD6Y57$tTjq|yg-lI5Tzo}yd`B*u2 zgXMy&!#3Tw1OZF7!FC0=y?JDDMX$^h;F0OgZj3y^@BGn1RBBN}S`gI{!+N%He7t}@ zFp)+)3JR}3FpZOidq3jQt5GP`Dc~g{Fl~c0`Gc60hSe;)r zLg8zlZhCZC&I~aPPAUMa%|%04nPv~;9PTf9A-0DH`z5BpPeVzShtuX!Dc)YDSnL<# zM6GRyS>FsAJW%0W5;M!Ol|Wdto*o6?!>WpWhCa%<4;89COIIZQPjb9g_S>)Jj;>4= zVYpvr=%zAC<7)I-cV}~hS(k-i)kpc(8FgSed>&a_kVxPfg)RcXU+D>;$q5*lbg^MW z0Q71Av0jSZzQ@%1-QFZO>bh-!P#^>;QVdheo;EL_f(+55FOLrTLWmp8g+`BB zP;JIgKF8@PFBjTdShxLf!*}fVo}o}=!#{*9iyRiAM&?-&9$}x9=%|O4VJf{&!DR-a zVK2L7isEZn;ioYo)bSMTu?5p1vb04qGJsW)-`ur$Tf&gY=>?qmmv;rTMK~huPE9qL zmZ1xusQG$#a62SNWPm_J-V)xltjSCX_tR?LdJ3|wT1~c=a;B6*ibuS@iMiu&{qi~i z`wJU?Y>U=qvl~8fW|?_k{(PzEkf!1@EZM&jL8fP6!=2&m`alw#I{k8lCK z+(+Bil?WcfwE>x^0`E1{9sU$V+!{@is zracp@kUC;qJ^mr0)?LUplWLr*KIMNcjcth%kkG>iF$RZ&i@C4S>J>acta^Wwqcj8_#6iW|w*E$cFasIA?{(Nm!Fd>d>nl6%+O1BfJsua3Y@cj-QTY^c6;Pw13quPwA%ngAh{4)*7MK(6m(bWy@rx z&Rb6R4v1;ONZ-18EK|3m$U3$&=(CdKMK>$>_Dy`U+_tH;Q4SmxeqnL50Aog_#>WDG z2-_>qwws4dm0v7$htK{PZDt$mbB-~2vod{mF0f!CZIn9he^m#kn5)}urWHB1K|0ot z$OG=l?u;6_UgI5!ML-Ka)ijQDy z+1a|nbIZTL2O58ny$ahZlYJ!}6CY~n8aZUhqt5R)jA8FBbN+z}w1N#0N6nD=6m zjYW)W%x{DHjBcbFyp^UV?%$Cw)NWL-#gl=#_I3uH* z@CMFjLY}f0O)TL~RRY!y1kXsskkC0PEjdxv!Ts@ z@c1<-o^i>#O*M+cd&|P7<0}?SDUg~<@w_4Ie>?Z9g=~Y?M2E2os`0Pd8;)(buCBJc zbAr>-q3F&cfUl~h3ipQ`gMb3PoS@gNlbXbCb^jFR5_(6?;o24mB09!T7#d7K4Kc9O zn>-cSfLUSxx+LD>?Cz=K4&`=(o(R`ZDZZR1Nm z^;05?&w-`c{vo@t_1Zl}PX34#8a5L=T;6a8lRobE%dEf$YpRR>##ArlVpW&>5o_-x zpV;q$z@cO32O^XX(G;YW!8H!$CMP0A>L98A+HeEtSqe`3axX!Q`S%2!-XzhU9oHe< znplYT5dyikA@#OT66oszvJhoPq0kcP{reJ60By%E``rP6>J|j>)|Fy|fOGGAP|Ml*A`Xosvs`4WR5 zxyjSu<~$mWvivpfF(LW95-it;#bv;jtBQl=!?1%$ZN{(jkXxXNHHGtAU(Z6myI8IppAdXT^jt6~ zu?VaQ5`iZ{=w4GogWMW3`FijDSDfAb)C%!VF#WDYZ+W-uzn$&ycUOeo>NH`#>v{Iw zyKFDhMpi89ol|v2e$xk_;0kI&`LH2SPlXsPK3FE|Jc79dIX$k@gc6znc}|SAW|;S=$OU%%eve6c>lZxo){AR}e>(!RMKoAbXVVWnB2u|x=0lh&!=f~Rq8F3O(l7@b$yn(4(&rJRD?1W3PSD^*gF(vJ z{d=bN9AYX}WH|eI%naV@Bcx-Gg7G@(M-wi9Eromk@t1#v-s{Waxdpfqm>S*?z+sw+ zlEh~ZNkJx?>!wII2m(Nf2))C6fV>V4h)TZxx)6@D}CkW~1y1bBOYU{mN zex9RyjemNwX(UnsuXgKQScE>hZsFhB-Y6^BGxG2Ac#?6{z!?7O{I%_Ii6!gor7WeZ z|HYFSb#<0w>Zb*CDWhkG)OEri<%~B1Yo&x@%Sigvd7Jg$qz%(d^mOKyTFox9Yd)9! zE(3M*2ZOk!ThndrE3VAOiN4Ib+L^~zn^AnXzaETKE1PJmT32t7|Ot zJ}k^u9D!b$70gFd(5BfGq5<%3!y;XoeD`&!|C?+tabgO^ATZEdLVtR@i)|PScYcwC zXA!(M$stL|+R9*Fh|_Y`%7O3i6RXyv%bH#Zf|n67Wr^;Xn`iYI*uzyrBmCS^9Ck;= zP2{$l)>KY(jfdHG$wU-(%}LL&`-Sa@c5Nh6TM3Kv?C$&QmGVPM4g8vEbs};0R-&z> z+(XRH2;XnWWz>J$>`LdlRb90RoB5mKLxt?D|DFEnmsr)#Ki+T?qmd2Qw8=c10R|?1 z_QINL^$LRZUp+rGF!4>;_8S%q_90^0_vy!MAX#M?k6vIY9~bzzjvd7w$!#3Bp|A4V zH|g|02RN1PN8&8oRzFn+`Xlfh@F2ZFAprq>mXb^iv1(*9=4Q6P2#|zGmYrg-vH5N9 zDrvvM_G3=s=~^}d%M(+ z1u~Ij?{-Mf@%lFcSv~rnHd<{CjqBd0;`b&a`BPQ*a?*UEn6u0hZp60EJBhXBw5J+P z4@_CTple-=tC4>H7uUw7nsIc!#tD9x*8>>qz>#I$#S@sBFcXQQn{)g9{XprSP!C`( z7d*d@#EN%aGlyD}zvqLK*87?&!P!ak`Krs3ks687B-gc-7DI;)M^tTXKB?nwYnTbv zDCfsoUEKeUekwO6Y~;ghy(knCBrYS@>z>5w{-UmBQ|@?(&~ss_(DQGClgjG))sK&E z9S+;&yv~GPioIie%z>YX14urpPigeaof@WbF8ePk#z`8cjjH3>wm?CXtjes8F-_bd zfLowCNhIL4?MWmZ1&iFsgg;F^aCp}zV}S`syC&=Wq2ciw172iX9!sy= zDTpmp|G-vz>D>*{o`kQFl=|c24K?5`ROr{wtWu>n7QqK3Y9t?qE4cPRnnSOtAm&>i za%lc|-{bG|U^~yHI&m~X%hL!+^UV@Ra3HO+kO0wdlL4(&yI5k=ubK~%3QbQw=PQi^ z)kki5cQw~Cxthxbd&OX}2iJA?*Gcpg=pq1%=HZki9pu=c|kfQK``;e%Xg$4i34Z-g=v&O_yN~7B@+V z;vk2m{%YRC(<}^kB9hWDEalnvwg8WTNq(CTG29nuiVXQ8=8Qeg z41i|ym!00n{12`wbNX8Z;MY z%#OVIKca(~H8xDXuMp0{RNRH6^~jd@2>bxfiOHlD<;d7ly5h`4wzaNA#KS#i8y1qm0vkHVuWLRf2CA!geh;lCc_R5uix$UsJx?Mw4i zB|j$xxIKjkq|LP+^YCis(yZD>R;GLPJ_+j{OF>p&FW8n5Su~CKXhgCdS!$RclS~O4 z0Sot%%kpZ&odruzgQ@1)KH^$c5xV*PY>yvUOCq){PRb6?hb}yb!`9M>*b_xMEq@_Ak}pPt3%P?8Z+ zt5}TPhh67{`ypc$=MEQ!te4C(_|rk|19l7)f4D{EL>%7~-B!oB9bi$%#g9W%ydg?a zudL5nhgPT?0@Z!n`QO7OmUrA&ZrO%Ji`yB4jf?N`CyxhnV5}vg=?po22q5LVb)>45 zq9V>r9;KFLS4@=#!tqVaoi`e<@*|-pr))&p?EObpU1?UYQ<{Z`;>?28{+@+0XVz|% zB*=yW1xxyXN3FvAGsh|XRJ|~tUrjHBbQ^TF@#^)~AzkNn*OUK@6B)BJ2~Wt>xc7uR zbHEet%Q7bUxCeW&$33VMT+0n3Aif(pc=%_>?4zhvLp*X=0boG3{6d)kZSRFgcOrC5 z4z8bik$P;m>!EsxnX?&os!4^uEx?D# zt>$JiGM_r?;lt}aw50I3>zzzStGE%T)|;6+L?_8XVP2w0ZA@8Ys*_7p(o z>+UY*Iv~r)B=5v2fgh&HR#;dN#@YXWM`p&91P7gU8XjUVdVfzic5-CivWdp3<20>j zJIZ#xgL{aCv7IM_v{tPaF@-5aTAj)7AVWX7T<@6m@mXzrfLw0OnrDPUj<%hrmS=)9 z_4xzIL94(uR4qG{Aeczh8*ASAph4!d5^0YV=lWNFqwkQRuAFn-M862A?irEAb*TX3 zYDBlVAI^5sufWpD9HZdL+Y}_?dvadw>)Y+~Sz_(dGs`%&-N|#8D4f!|?H&ykK>4%0 zwl`iyxILmfx$aE{MX{~u=2zzGF$l=^85igq7=)Vu?JLr9mR=}y?1?BHUIOI(S5~-p zSFZO~y>5T#dQJo6Qf;GAl0Hdk8u33uKlg7<%jK}G?w*HG`epZ?A=>eKE^G^LV-W+1 zcddBUD%@lD{4;#4ZWT_M)GHKqTCkdLUC$RB$7~TI4&2E88KOoy zfuSz8y`Z1?^qpF9R2M3}WL8Fy>N}c57b#{-B5Ej5o+VyUydAhNG6UmWgm0_Wlvp zxk|lhdjBI=Oi7^twtZw;XBWMRcxGh!4pFqBM;8QC>lAK(u|GYWIYgsnp9$;DTi;Kz zGb({AzT@L^H6bqMbHCF!}WuP5Tr66)qR5dV|n%${yeb&y8)Q2=Kwf+0E-691Kzy zRIO9~Af81-G*0d7mR((GEI*U^yfxYHV-OWzAwQ2tojVePBkNP=_c%JL^u4JC4}r|` zBTM=7jbc<6o>i4jIQbv>WSlKc+L;Y56nf(QdZwgL;6r>$XLr^6dh{pt$BQ%dkvC5+U zY?R8H?NKYtd}yfImm2idtw=mFg;oFdKLnz2GJO+U(`5V+x3XF{O+%9CnAfOM)_>R^ zL{L!q7+#g*QtqdXQTgmsIhqEDM<~RONp!bZ#gV+P%GXFlXRGg24&2Wm%5EA+2D&qB zl;PjV+67S`z-kJUy=}Ox8v6vAQ_H!iO8$%w#^XehK{a1%HZDXI4Wd-p+QTY8_x*HB zLOKGBXMrQe#iC}yr|UmR9(}GZAU%X4H81VrRM%2Y^XkYq^ZSImLpO z?1cbp(CZ{KOE!+g&ShC0Z<6JsI2C%`gK?B^Nn@Enwpco^{~eSEDky1;)f=UtddjJd z;6-qUVyb`p%*7`(`0U)bd2P*;WiSp+h3-v6o%(;)SL*w4mH@sWlg-ldgduG`lP5cL?3FmR?8a776uLa!xvkYd1@yjK#fESzMaBT`@bD`67Bu!9M zR{;PQ>x7_4Nx%w5u&RK2kDxI4s>qF@t*{-PPp1JkFVKv}XHPjy zw7}9Vf?%+Y3=RIVliJG~3o#`(u^*7H3Py|t+I!tf(J4s*-{uX{j^Bh3+(7v&2XVDv zS$TDA#!*-Dj>r93pi(B0`Fnp!1_eNKbMCTq_)Oj=~B& zA>?KFz-X`F4NbSp^iR3HRIeV6mwUcKtUI{Iv<4ykTRP}Fd~5o?3RpVI>TuNZD_Pec z9PT^+tZMX{G%Xu+5b#`fZ1yCt@U2i^_GVDne@V@d+O`Q8VdsJa15#_TF`Tv(KdH72 zBm1oqu+XG#6rFF++KuEJCv{v~8wu!lrV1$MS=xEkS&dz-HjM>4FMen}KXE5p zrWc5qmsO}j#Xa}tm&0#L`ib~kw_qw@T!^AB_QS7+i+IAWXqf1W|BlMU|}-%V-c;&?&W+oy$!I7U!+_b!C&0UT68s%j~wOt7jR@{bmk& zZWe6(c!55p-*XlSX%dq;ZHNhP!G`?oG%KfH^Qg{h+(IosPP8J`r=dzC<~6N zmfrf15=KNgl}WISG7FApkBXg}5x!1D*WDnWA-vKt45x8zef}Ka)g|Z7}Qj ztv3$FnglB2_*pec+JW?iLd*A;&x*L7Z3F6M=Vpuoltz)@8Dx1|?WL3_oBlQXncxqF`bg^y~vT3hN!;3uZ zf*7K=U?jZjH7CsBTX2~RWKyvh3c6+UgetAv=a0IvikYIkRnh^-IE++knjtnb`-9$j zsu)t^EKenJ>G%)!Kl^7Z`BKf1&CwMKtTQ`qA`U4bGyP;FV86?&+qckn?lL}1Ef6ittey43!*C+2pp1lQOc0-~Sm<@_q_}L; zm@P>;CZB$nG*ZoVSJrV-D8>-p3=knWCHEqeETCa0^`~R3)NewEK!gPSk)5zNxJk#{ znQ6jH=w&}Xcn*_p)v!)DclcTo!7c(P8M^d-(08tp!%q~)|VVQPVUhKy)6*XHP^3xA?q!gp~3Tr^X6WF zJrhWxi>;j9zvZ#tN6Je;gH4E5e<&|)GG>{pqkQ)N;ur@y4e^|4eXG>bOPrUK_d!I1 z*KQTv{~Pt5nC#&u{~8?!Z{2;29!>K-2>*a+U|7>1#D%S|wVK{8Fo$qx+bgoip4TP% zuXi99;BoUeet_x@-knu|)gOkf+M?8Tt6kGUlOly-zT@nw95`QB5`v89BF@KMEY5zIRa*dkw8Pz}gBXq)R78CXkNgRb*mq2oqOs4wrjj zt|M)4h{YKSm#kk{>G;JIy=Xek349I6YCCY;3# zJQUII44>$E5EHflyzR)Ud1E(+7tbyy7-6>T@`!RwI(Q}~sq!qbja>t7t@S3M`$`vjGUUpzks^YN`_aSKB9m21cmKApAjDVIoh$2Z zmu1Z;5jPQY1n{O5`a4S$#Sb$fwjnhsOGzjFNG5!$`Dmo79_30W=*F^Ge#!0)Qe1FM z0N-V~lnkI+_mbM7fMyq!w8+{7CQbGKdT;1vRl?0 zl%@*{Rs?}1V=vw%6Cc>3`PirlX8bXK7yn3C2Q&CMfpbUHU{offQ0>&wSN@bPTYFd5 zj+OYgbDvO!| zM`9?!$;;VNOOj`TBs^qKM~Py!!?wbOnz;kcu$I_PHU~DF+m_NkKN*TKV{v^2Twv=N zbFv13A5ZItV4ebKLReMspYOP!4KysGpdZoda4jVy!W+fmFbAX2W->}$dES|KOTJ+a z^lV8K*!|$1?6f-iiuX_)nuhKbtwFOLl=SI$YtPm7LYc`OLyUGBs=GOL?`lH>%e_(= z&{$#1`%#pEuJmBpff6?6*wj%y+v)i{dA34O#kk6%%z+Yq%7UGnQc(x&+?%HJ^dz`w z8D)4qqN=R=VWk#b%%HeR@2!4j?|lQaS)=#)tZA9SIY{NVloGAKX?#+!X>?1|qJVPw zeWXTurJ{tWc_G{NL)p3s+43t_nRPyj9#bE_3Nt#KOtbwyy|(fAc^Uzwo{GpjLg#Ud zNR_voBK<4^CqF?u4*G8EOdRyGp;cMaF`MmmlIF74ZjvrLQkDO^;#v2PJ0|qa0&R5J zI*7N?yJ@u1g~)im$9f)pEHx3d49*x7kt@923Zc!HvelYQh}T`U8lbYO`?tnhOg0MnjrBzpOC-VUKZz z00bc}7e2rR_)Q%bB#H?PF1-ZR|SXSqd<@l1|W zwC>O>ohPR(z@o5np!ID2rmaou#)&G@3bUh{dZY{6uTeyW#(v7P*EA3BgUIYN+9NX5@TZbrw$tb3FLF z)Q(B!`%csRMkJ(%-dePRxr8ktutM(8UJLqBb{JxK(RYmfg; zr5&@1{}%P6s>^>&#i?US0CIg+t!t-TY-Ilho5wJI{h zxYeCFR;CbmRmh)JceE?Z+Y#_985(88HdQ|!7L#pITDQU?c3YU}*8uZvb>gpYA7S^^ zn2h^}AxA}Ni&TE%1dZow%JR)g_w_rhAtGB>?*UAxZ;w6mr&677)DqN@Rok!FgNrve zWJ@7EjzSnk_^K|7RC5dDml$Yew?o9$h_ZeFu%HM^cL_3xe)bo z<=w?Spj-|*QbbD*QqZtXztn2Gwal+N0{LbyR^M{h>-P;ZHU+45ioSUO1r=JYUjAym z)j9_9+sy;d=0sFicVqegsqSf3C`#4ej=vy0Tn*48n?ImPM@oDk8%v`Spnu~fHAp|t zA{19E=#&j-+U@?OB2!dX#b;iRM3=>^*#luWHWbEz+&wEqqgcJSBx z<;I?*>wg$WnLA!Amr>EtvF@8jdHG9;M~1eb`+Fd`UyC1z4&J~OTOVgIhy#HAnqPdy zaC0jbp>Ur3e|G_JA=X4}zZ7bs^Y-b>cmKAj*)%;oJrBQC(;P2jcs;3HRKG0~Q?0b` z0)lYL2#g&$OZ4vjtP}VbF4rH-Kn0E&{2udYO=a#e?6mJEy7X8Nqu9kt;}&EJ3GfD@ z*IWvFvvlOWQ;lPT`J_$Y)F=gEBN_#Kzt!0iNL)D0#?vYOFqxeAKjD|+;^00$A*C$d zJFKip8fmfg%}5^9DaM_9Ym@sx`ia5JqXDhI#Z0IV5uPW*T+{>KjtKHyW6iNUDr4B` z{%8g8VMt_5L1BCwQ*7dIUzYb-^U#8y5~fM))Jxi&QmAxU**inw+%zQlTu8C%F`)G> zK|RO!qgUX5FmL(|vMgv1 z!-+njKpY`O_E0F6oTSk1T$1C% zJzn!hTl6O?QQCIwEy2DO_>Xbax1v44f`$mPtDSAHrl6^%Xk3+E!&C-^TNS{_;8lNw zKv{6H`r0w7hnjG2F^4EBKo?Vl+NM~bd0 zCFF=F=dbN48J+e(Xk~A2BCozqSzeEEj3dpT*h7W;146|hKJ0YCn59|@x+ecsVk6-T z6cQH9Fh<`E!#?Q?y`~3_D7DGDj8#BA%sA%!&UwCWu5diGDYwA7E1$%m4DN_%@yrDt z;6Pi4Hq_d*pUQh4v^kzD644fRST!lHta|g94<1gl@8Cnpt=&g6Wp&ex1EBxZ(0wPU zJqW^4JN6*h&Qs^|WjzC$OL%GN#YL>Dz~UB>WSedzC2WD&pc=2PtgtvDv0h@BP>fL5 zY*KgU6?(Qkv}lB|J+(BifUfjr5dD}VZF{Pfg5eOlzHR333P2;_NmwrbnNDeb#+M@Q zxKjXCx@PE5ISQChHBJ1URi7^4YxJY4q*XHD9AYQ9Xo6`**yF#W)|%~{;wcT|z_HE2GV(R)ZzdQP8?kHvw_3!KQp~W?h<|FW*16c6QTJXV`=rc)_#sABgPK!FTb21Q8(>q=D*8Jq(4DwQ zP3pWM{*v02=0!B{7qw&Xn(}jza`wa9EMD;l#y-hN4l>_}qb$sTV9?owJY7mM;U+O> zcw#}A=W8=HX=r?qWv1Qr`w*VHRzirR_c`HeTkEjTQ^Mx5if?-_hd}!@HlvQAnZSeb zYiPZEEffNz+J1vE-y9ONQ#D%$%~|tGtoC2L59rGD-1VP|%6f7bG4O0in@ZL!*|cu- zXpr74WMHv0I?<|E%M*CfR}{ieq6!AjRz9hal?$^K5p2@@=56FfPv6(Yj{hRJ!D;<^ z)s6&o%u4buSSA<~gttW{UNC)P0wW+z(X-srW-XLE(T#~Z#B@zJHp(>ls&*CXStsj( zpC_+P%ITosbn#DUU9*35VTZ2a2hZ9YE{JD!AfFd!-tDV`v$)nstLz5%WQ_vnUzW9^ z;UlDhIjOp|JqSN5#1r_1pa!CJEnM@0gS+|3DjN(Mq^8NIcy+GvZlc7A)7!7qp6Jp| zQ*DO+kBbh53d$?~hrm$Mbwx?ZpfAQwF{A59C~&u4%$IIG_2|)zdGF0-l>| zG(gwoX8VAnd1*P6kYXI93$iVhiLAX@l&84?6DmVnQdg$UI6l9Bm{RESKD{`#($&cz zs?|@+s&Pu9x6Cq`Ic`kXl~si83?DrRZKCluDa!F9jwX5U@CJ8n@GE6uno972#I&fG zrbG9`vE!UuM&Rnrnf{qG_rxNM|ED?jqaY;pNoI!7w@uLkE8D*2388fd=Yq<8KZ8qC zBJF9IhNOLs_a^*fz{oj1Aay}9WcM}+IF(bM?SlKa&tW@O%KWSjC5eStIxKVce^9w$vyvzJhdRxTNu=|TZi5rs=-Ey9* z0tbd0pBU!VWk7_3e;RrB7JSC#}~ zb^=2v1hq1R0)`FJtn+9IfliXN?s5TqQlHX~R%9y#I2KbOY#}Mm6%CJHi-YS$pLKHa z%IBs%4Uwe@0{fWo$tgIG00kEI#%zib8TJFjkPs9KRuVtW!!V-JXcw$tf_g7Cni&&pAduhFgc6f}n z84?^akPw)Kh9vKb^~pcEx{B&kPw3nRQN~V_9fDJVe)|u9Iqss<&qPzV!|QqCkbd89 z0wOKjNc5L)KjgcXeE1$xeI;>-VFV*?qG!T`27VC`vNpcskjfcI3#a^DZA+3^ zO`dgG=KOSPMUsrUTqv!$UBE0}`F&bIra9$jwJNP8iLcQGI>ZLu*!s>Vue$KIyH^o0 z)HJsi{YyV>u*N?B5o}~&TO+Y4@m?)WTn`D(DXmZ~4kcsrjc?;S=IT)YF|yqZV@8}` zYY7gWmIa`88?2p zr}v~63_%{CO89yV{K4Cm_hyi7iuLA{Wi!814*lX&ZJTu`y_HS9M!i|Yfid&S;YNI0s4Ai-rnzN(NU`o?RftdFw1=` z3hdh%*I1=F7~e}c3eI_cc+eFF+R`@L>KmJ^t4H=pwUSq|d87eA?^#s~olZApE_zQY zQK#f3Po*63DR*T6PWh|KPoN2V;}Z3ck|~vST*=P3{KsE@ieBF+yQ#8?)~6GOu}kEu zqyA-+bRB=1X_^7#jP|`I&LR8n&wSJVB5?PeZ#~~$@WOL#*1S5~^3X>6@>jlNmwxnb_MuOH!QQ{@ zU+s{UbEPdJz)*lp4z&=39Hps0evNLqPt<9&%9R)Wk@=Jwc$sW51t^jQVwJh_wv_HZ ze7Ew6LV0+!2iT-TKJjLqveZitil?$g6I4p0R^g4J6k@D9){e-XYr$=dkq$oLH&zPc zeD5?&hE+aJH_tul2h5heUs8tH1%W&~k8&j?9H|5c$eR~xJ4ciDXCHmKEo+}=pZ(2G zbXc0SGdfzORUwlmJp)eby!vQf8mRs18)#|QwAR`KLw)ve=Rv9eZ|KZg;Ppd&F8hd-Yn`b&xp}(Ua|#@cJ5PtlvnkWWpw@!LYDYBI#0T`C@~< z7dycQ)z|=50bBuByln(9A(_Tr+>Zp(1aKK!n`|Y{0N%YIauzkCzyVAFR|*i@!t5E^ zyO>2_JmWVN&NXPo7vXdB=FPK9F1f^h{_~&PcfRu-TfBI2Sr&vr!xz4^a^*^U&wJiu z-~RTu?b>Uvb$<@dOghRQc-&C`fVFASZcW=<%da@fYWs%lkN4kUt&8S~uS*Woj*Y5n zZNb*Ki%nSH$}bS6S7~e<>a}g1o9)IO8?7cDQ?OBv7H(u#F&e2J-nhY*$=>G+Pdv-o zt6PfzFXnpq705@ty{5X(cI?=tGPk;^ha@Q(Is;CStjmnOA0`h}Dk%fq-nW;u{FTlr zHJm11^L^BaRfMEWkqF?oE~h1nIZPf*$0&aeeIoP^*9cY+lH?u`Pd%qRQpKlOen`C= zt?cDNgPakrInSldU*CY10N*kU)lpEZ?;-l($@?PG+fTv)^2PlfCzS zU$7f)*ln+SZJ+(yzxUZkHtK-FOAfz$;Q;5PYyDC2cv4qETa$bwwdG+1;K1{l` zX(8t!TXV$WR;NDRKiH!Mn{K=EiYx68f4E9tSY=e7IR5zK?OpGBmz{g=xk{&?{bSN3 z)?Kk}sM8}aQZsf6R$7P%@?_dH!(`90nba9NDqrTtq5xkpaG$uXjuj<)KjSDskI&o; zpg`zK6E8~E`6f=YnHqy{iLk`#XXLQ9H633VtPOw%yGk_`CGdpk1qc%hy)|{xGS}6@6bcB~l@>HDh#!uvSVpMBqMe{hpEE?XqqXnAXFY4*OF0~80S;)1-u7=)Y$9RbR#1n3NqhVtQ8l0ZC~ z2Jq-Qvw>b}7it)n+n>wFknm>zUn!T$>nL&befTcHCA@=IcHIvY;5!Uk7%wYF^c?}r z=_h{JG}|0?;P&S}^L2awhkj+7wrUZDI0d9deRDo^O1AgciSJJ@tkQzco-PjUt+E!S z*&~I4D${u=(%x^Vo~1Mv*wRHu*_JI=iRV15U8FK;wI}Ms9K6T@YXvDGRLk7gsi&T5 zM<0EREBEBJBdb>PK=KHEKDtWglPCVgNeK6uI{+zAi60(-)i{WYoB_Nu{Npn)FhGPs z!eytp%&$&QvsF_3OXDM~Wf$oHcaRSk{KJXDgd424?;e(pHIe95{6TAIn`JB49BDhc zZnXP1JZ#miP1e7Cr%x6}x9wHTDZ80w3a~;RfH~%Pw4cmP4F#YjLVctYZ`P(B^XT?f zHv5?`wC`X2Tl>|G*O_MQ(!`Pmg|^1liJ8@Ur;Yk=x)g!eak6*ONsFsV+v#>nE4WMP zbkxdDBB$TUjXvoprZDLnCjIr&?gy~IbUa9>taDb| zv1`w;4}S2!WtLaVGSd3>ucH2#1bI;g!ZEi)795o3?SSlN=Ni$>Z!>G0ZH;raz|>o? z2k&~&ZvETMcGHc&v2Xn7a@(;rZ|{D`#YF+XT709whrFcEBiq0K@}@W0YhLqOYwu`R z-YgnLa_jWaBuh1;Kt_S7pg;!jP6ZXGDGk^)gKQ*RMb}uiD&`o3tfEKiCOwJwg}*Vz z?BpPyOs-O1BMKJEBQQ#ccay!RPLz2> z+Dac;J0UUrA7q-v%L-+&0OBf2ItI14AK3`?`+`HL6{E<)Ek|oRR zEpPc(tDPfl!|fXdSZQmKpRFn(K&!tTo`%_m?ygQZHPX+vL9P7u>0pqi`c|C^-RjA< z$mU#+G+sD93(+KVGl&AQrU0LR#yJW#CKH%(rS<|*bYC3Nl@^bAmPYl4<7?(v`{@_h z+BHYp_y71i*-O0N>eOf2WPYJXeag)%cnT4ZNb*CvDS|!%zNaq+*EgUA6-^up(&nK5 z)eB5O!xuL_JjRpuKly&~15F0d@_O!bo@K#O5PmAw9ifCLN7>E|#q5; z7UlO?qYiOoq2`_Me1~0h(M7I|(51!k>b5rDvDw+L#j!dm45)G}>qZZ1PFA zZ~xw-rJcRoR}?tl0p74;90-Q6!fJ9(T$Oh``%e>HuH$hbO`kZ6NjjH{({bA|N~@HYcG z&GjDp?R9Ur=RN<4_N-^0Bb(6$>*?HR4KlXhvs)*=3XLM~h8hVALp4gTQ9y6AHMTd| zL-*~nAAS4hwoHr4k3I2Z8|v$on4&(*$}~=G-A{Cm`yQl@`g8QUd{Bw~vP&5SfU|*6xhB6N&y|1Rr6ZxRwmd|V z7e^7In$vcIW1C;K><0hxS3kEVZ5_V!!~bb}WGI1;k}jlE@*D0`dbdCC zO7&DLDHa48xqP7t7BuK#?t&V+NTe&GedU;O66NIZu{(a_@)slwyQq+`qO{}L0S|gl z{umv>j;YID$}|TuLm|RI9FOynj1Y35y{8ILdZl!!9qC7MlV6dc@Z*P^m>NDylp$tt z_y#JJhlt>!WVmo2t|5GgANGuwcZ_lcFiCDMB-B3Sz>Q| z!|UzLGf!9IgHpz+u?`)^2%rlN&>(QPv~Lr->W91-RZJIv=Gdf!zCQ~`h@YWdQR^^& zABxS+q~&`B$=trAKw>^Hq%kS4pT3%GJ`Y)|zWu!5^+eW+bZ@1bH ze)t=^?&`d~^IgxkQ%*isgTOY+NubHx%%DK%OOz(v2KQ?Ti=bn&6w}desP**LJW1nf zBx5nI8s_l2rN182*kTI~d6XTwWVu~;-(7aq?Kj#z+qT%&-rY7!6KPDQ0Jhgi_^S5A zm}s&%6S*NB&RH+vxn7-|LwBeTJd~f~_#@BnLK1s*Wx@gvsuD1O!z2avy0OU%$Qn&q zN6um_z-8o^Bfp_MfV^lCvmlnai~<=2G73xu1psAFI_V_)pa1zEx2wy-`}^MaK0h?= z!yo>zAG(&z2Y?q47=DOt-DS&{xeSIMhe2Y#VKkSPhk+Lx#pe0t^95_3v&2@m&$puv zJ>35C;5~NphWl*O1G2R#d5^`mY7H_20ECj!26eDg%9+0QG&%$RIK+A;-yZBrRd66V!S_)T%~#(NaY{BtVwXU zJY*yr;iqXE9oJu1Y4>BUs#m`~Y6!q;>T{YyVJam`ON;n*(%jW9DHa!Mbr|Bcf4RwC z`ijrly_;D0kABM+n_c+S6YQiDj;YmPO~?>=?%8_*rTmg3qpYTwVEH|{BS)AkOKGpW#SKtIuU@4)X5S=pLK7!Pmpi7 z>cF>F`I8F_73F#oEYT;Ce0DqK6o~kKNERg$c5Z+vRBMMwWA~sfs5-@F*+crS!UG1? zE>Y58$F8?eF)V#$85`24)ksvOtF;INR=XLKhlO0T>r_%819+#Bn)@NGB#?9iK44zF)if6AA+i~kM;f~ER()U;Z2-F!Ro z=ykS5=1uPD+9s{4ZFb!qH(U2Wzx7GuzP46J-U#S*o|0p7Cyi`@sl7XQ+0LG7dw9Wv zws8J(@6S@icLfj>eagi;MVmqw>`MY*Jm-1Ov+sTHyLR)bb9&g!uB>(8~-s}7fmJhP?-DFD=>MXTLZ68;9Y zd$U_RChO~#IM1P-psjzx2`AY5|Kt63#8F4vnxocO`>bXyhUcwb3OZT9dZ}0w`<+Y>N5A=4W5rxt25atne^}-{1H226g*|DcxV9QUqNZ+~qV9~uS zWfpw`6vzPH2@q>02CWiDACi4TwjZa>1%ws$G$=aWdW87n2_FWZk$}|kq%l?)7)S;7 zUq0}~iBKNOLLDYs|FOz5UGJ$U!#-EGy(MLU-rio>N9wk@bLRS?9BA;iwzk;9MGJkE z6|GsHC@5M^l^+@sh@v*q(KcTzw+*&_-BI?jOW&ofbXyccPFpGV$VSu7&Rw=!8f~2T z%;9NU9^R&_v~_!B!fAJpb#B|P(^z-gj@_NMV@tPf?dtX|KL`%}ecBi(p6+XTT73T6 z*Ke|4{_iVfn17x9^D8gZ?hPHLc4_c; z3{=~kT)oxm1UA53=a2hwPyuqIQyBM(yR==kAiQ}+#KW_Srm~HlAZzvG+P8Jq$5x}bh`vm6ZSaZxbxORBH zkKKc6WUd{?d;{h4MvW0#ob0c%{>~#6nE8C>_7?>*fOl^x4>L-Rx&T@G$gwRqfR_v* z$w{ye>Ty|OrKQTH!cdA_dh8&*B(TV1MHkmSFlhiG|D^&4&Cry0S((%CDm}-=^;cd- z-G#i<;VB1>`g5PIBkz(l<9VX4(x5wP8tKA{nf7TVbr6tLgKNKtUZd@DL)cf6D`=oD z+N9NULsNsoRrv%UFF+?R!%Enk_OK!jgK?NpV1IN-13bpk8OZ7EO$~k4P}`}^G8?sJ zTg+F&!K}tAo7D`lYV4T9&=-Ve0LArIC(U2C0aoZ!Gv|P(A?q(R+gHDNm3{hi-?L3S z#P}7S%@}+SN^-_Yt@hg2J=ab>{!k%Q5mK*RjDKiQzU-s-E=;N30t_9RQVROW2f z+*m6Oc(WcBKI+s1)ekY7o!ixK%itF~dToJDP%qTT&WbvCu5oUneA1p!HQ*<8B1f$z z=e1(QPA&bJm>eC^wyeD*avJFtKH@(uJpJ&8Ka?q!Q|yah_<~(SZLyuW_V{5@ z0fTq%*=?&=ueR5}{`JG&?~Moi1MN7#UCNcYi~<=2G73xu1>h~%;0-*cR=bh5ZQEua z`p}2$;~)RH^Bnk2;61z#I~ceE=5l@l<{7vrogmANSiXwmVH{fgo&z-t(!lGHT~D@l z=U8mhSTI|xuT^#x=ZFWc%E_*KLrWyY#Ze2A^B6&qwXnZfs~zHX+IFj>6?{zbc}E5I zDGWI}n1{HD;W*4wh?4M*bDORUHJAz)N?f7BA$k7}SfxfJ`tH!i)(Rpv$^X<4`mwJ(1^*3)q6jTNK({a(foogrMu zS)b$?8uUe%rDB;%%t2Em$#X1#)9yGgL-V`_nbPy^S*n0Gb@}|UMt(V+(k5MfT_c&S zLC^I~Z4Tm6*V*mOmKRUzmh9EgBu#xaM)*&iwsYo4(`84|fr99NIYmO-=ZX`+?0+K} zltZ~_hxzVZQc@`BklP;32%`hyl6@eu$vf^w;W=f0rji1*qlo@|$>`fiDMW3hx(IA z33JenYdxthX!6nlgRn4C7YhI=)h}n(QO6GP)8t7D41_B(X?1eXJC2kNn@($QpUd)2 zB^K)B^`D_*Mz=I!b7*=r%Ira(1bcl)fL=D|;AC>C?Ay2{rn+j`=M?ZN&AuVo!et_W zCYtiXU1xf9)jw^m1QEE^?5Vn>=dL)}y0dmq1 zFAND_k|FioU3;vqd6Dhe`Jm;6!?r3avD;b&>^}B5yYMMz+o`7>r8qk!O7u(fcc&&9 z%D1sz+XZ#tNHrRLJwApED!muI}O{s7wL)T$0yCZA@WjC{Gw(3oX!S!kw=k^1}^y&*5tP8i&;(*=f+8h+L9SGRL3~ zL6ecjhB(?@UjQ+1XtZ5?@x|graf!rdHa_@}{aC-W1KdEP?XQ3RtG(`ZuS>tqo-+z$ z6v!wrqbUIY!UeDwSsi=4Xu_eL%X|17vU{+>OL)Lv!mv{-q=ElroRXH)#9KoG{GWSd zb2rZ;-p}|!kRZE9VWP2t(9xJ#0tl;OIg@v|vN1LCipG`1eSBLKKsX%dIz&*bWI-{6u#f~Ur`Z@2LY;}AGaO{cf!(52Lgfbf z>)c!eOal|EZqg<(FZ6_ye@0kj@?If%&zq^PUS7%^WI&~nih=cg9 zoc4johcSXrN6X@z5O12;s1&|1;Fuiot@3Vl3_*vB3Vo%1wjE31r#tKf*76>1QZ2Cc%WcF z5PhPU0YMNIK|!7YMO1u>U={;PP(Z@M<}msH{Z9Ak+qdW5J9l4O^{?<-ntP;o ziAy_%#7O{H4eB=7XY4rFc7EO*scHC+{pGKJv>msTVqRo|;hvReR8B~QIQ?PG?z5aG zah*D$){;Llfs@_nF41p%akG|`qn>Q$%bNMG$ehx^M2#Rw=6M`m^6i-P;ycS1eV9eI zn0Wo9X(7p4nz#JeTk6G;&(g@-ACBr*$-;lt#^s7MjN~dj8BYK$S~ORE=tCcpI+#S<^ig}@fd}kQ zfBKU>^w2}nnJ%#m`3-;eIk^M<&oT$5VK1)Nl`xe|;K-6mD>>d8#Bsd$#H>Z(WJV+m`zwEfW@d`%>2 z`XLGHBoeQ*n*>lpm-dhVTVFrrqgA>9ygtN;ze06`pA1#penL~>z2{2mvR!M#cAXY; zb$^OuNRq+6ew3g5`Y>0k`INGE)}DP$;^CB?wjpg;Nlnbk<}UqK>hdlL3i--Y8(qZj z23RoOQ>%2mNxvG8@uoo5K2`6kH!fLYj)?IkYKKjO7S#=H!0`k5Wm4dj$bjlS2@%iL z%QRymo073M;yXWXu3qfxcIXDcyB60k`fvtxL<8xljb_ZYZFQt|=CtHia3+Lc-Y8;` zafo8)2kfwmXFhz`6`y%#w&)fev0B>rua$l=KE8{2YE@5Lm zAT$=3jOo{8+}hc0J8i#}ZM5MGJM6H7byQ1fi{9}3yM&eVRFNT=`P^Pkl1(w$P!uJX#Q#L>DhPdERE(m=(^u6du0 z_XY*ZDd2r6?JwmnmA+_Y<^8piuQ9Wui&e^-^ppk23k2X}f}XU0-lP9l3@bm`_@{R| zG~dN>6sPg%PoWN>SGsnOKKhLUHChE|=xClcY5+D`#S8|pK^`!S$&9PaYy4Jsj80;0 z{B2O6L4gJZ)+`0s+}diZt?bS_@3fOnI?3+6_gc<*?} zJ8a67(Ozm)scco|Fy@+cF7QyyJJ_XwO3Vc@`!Ua~j>+bz)r@C#@=!Bu4%kTiMagT1 z8dJxzxm+5>_d%_{e95MMBpDgC5kYNg zBM$V+4v5q_IUWFTJl&w4MK3H6#nUE29O0;la(UviUi^k@629_Z{6ycTy;4%-aY&bZ zrX}~dP}6{JUkz_=T(n4Qz!EPizONb+n*>>|48wzQFkA2 zl3D|Q@E)mWW^(V)B`+L?pP&`bsQWDJ7`*tHI0o^oFHtlA-m$3Zlqnh>m3NQf)}ygu z;LQ!b#h~7i#1yh{-e$|KZLTifVSw?@m+Oo%2p7W#n3o@vqLb$R=H2u5wE)g)Ve;XF z!Oh!^=G}KQs|1L_fU;N1%j%|ItNc~6m06Sa08+97I#9K+VEIy8I=Iv?vhZ0l^%?cy z6R0Nmjs;KH{o;*c@%wj;=2pm~aKKw$JFWUZvM06$x$V zR>~VCGUQA^V%d+12F1%z?eEr5o_h%1edA1PjM>GH)(hc5LXok8i? zz2e&2>=U2v zly*g`MGEl@+7TAu6C4ivH2=y-RTn&K^tP9eFz)5qrw}p&P ziu%Wo8c8Lp9FNplDM=ESPvEU#s#r7rh5B zkFmTxA%TkF#9U&E>!H-T#v?-3@RD;XPe1Fwx%6|M$Kkb*zNb*K3l$oJsa4Dq{p>Ed zNuG%SirR|m3t6zC0kuo_6aG@~jmH|LKmpxZqiL`Gi7^gl^@p10vKM<>N)AUHq0N4FlSONKwfGf4yh$u}nA=Sf~_nbiSU9rQzSP#q>NhbVeo zo%D$-52hC5uIJm2zsRH0RE#)BB8Jd&k(!_%lN9BJm1|D;r<-js$@)ccs^jI=1vfHJME9lp3}m834Wt+t?; zUjXga+ia~HHO)>t?RwqlJYcVR%|X_pQ#-vYmfQDk{E3}<%2kpCHapx11RY09ba?zMFF?y(W5plYnf}0K<}O(U9sAY4Tkm zK8};5UE0h7f532BbO2CS?yTpUgb=dv)g*zYW_UWeA->~0ZKcRPmfW90IdU$;Uh};E zFvkiOmBvBrMjKBSbs?{1@z*G8&GZU!W@(M~rAGQ5RkRO{h189!M!+FWKc5Z95PaN8 z^t31IEn0hcM+4xU3{~%)fsAO9j*+|)M^+)P-l^2c zW1vKLAp?pozOBoIGIsZz>-;qbo18z6+ImM!dm#ftmUPZvwak>IU(Hh$d!iqznbA@} zhcPSUXLZsi_RxV;N@;X&&7vPP1Ske<20%?NB1##yq2T}t!(IT()OFbeQWy=rIbo*> z6;cHWc<1M-gyjfXCryrA2PmF^yNRUTFc5|^4fWW;uYA51G_CfgH+|CHed>*N@x?dV zMjOwv2Oe5rPdJ_uKbc!TSYdk3`-u$Yy|2dm zu%RLvQ9e!Lcl4LUtI=^aGEoRRQVR5dWTPYc1?m&7^`sGR64o; z{O3R0|9tY3_G?MDJ$T>!w$n~K*$F3{VB2lCor4^>&zm>THr{w+m+q|;XTz>Rfd&N{ z6qt+@2!RXo#@NS&X_zC9IKtS#nm>QO+atM)ItG56SRfp{6>1&ORn7+2FuuxT8qJ^8 zn=gtfjcshj*eB>a<5MF2(kwTOX9zEPmV70a312MR;{00t&c_jeO3p6-nn=b~j)3$F z+*2+nu93rd=A5g(=KTf(7SF1IMsSEY(b*aANs!S?^p=NDS1==_KN*}O->Gztw>j)8 z*Dd`FjFuCb)MK;OZ`d_U=%uNNwUKIGV+;JDv#WBeWITA@A@%H%-{@h;Oyn=K@E2^8 z+E4PS6>8!^pUIm}S?L1p4(ZqHY-^WbSu)o7Smqg4o(1l490WX60gd4S412 z*Yf1Y2f4iu2jM9cGkfv!ri(7d>EmJZBsG)z2=o>#n=iZu$Q0 z_Na93?)vO)Y_C0bwgdNlu04I5jcmoD$Am#Iw<(IhNopYo&2LyfM_`DD<)dMdqX6Se zd0j<6N<3+S$wZn>-MOF@U(4glyyO5=j%E3n=61u!wq!FS3t9GpZx2D9(x9|CHbFS5 zmd0EbW~7f?R~X4_W%r06%}-@d4cj6GsAtqP-16O9?6misW{>{oKR$l2idndDp`Cc* zi4JgFdg-P1g)e-;CEecs_P4toKnEL3wAEx0U5vf)-Jn2&0u2gGJ_^LR#pV^jEnqKd z8D`9w;h9B+LtaG#e!9B4e5`}dTYZ3c^3h~+<|yEbGLO!+`nG1BWD}1N z{0agwSi_|Gslst3elmZhvNZdGt~#?)i~k}`&z_>A3^D(bFZi#~ z3&#xSKT8>Jyj7xrUxcPhr<hGuL<%eo zyp3m>YP_tS6hIGjx^?3N#)zZTDi?DY9{jCPae5k$}=q{*SUr?|BA9gl6f+W-+M9P0{zD?;VLm_3>tMYVz2c`qKyF9 zQ-w&8Cr00TZ5opPo^ z87gfyM0pBc$2X6YH}(K_J32ZX*f{3cV+C-tySqF4j<{CkS#8D}X^JO`Q&^wI%pm2ls} zywcAly=GryekWDU@o)vGSrc3)o-_qx>&Q`wE(xPh5|{}-Ct-Po?YV*fF0vG)J(59P zT4GA^=EJb*TA!)cU!~aU$!Vo<>ylm2Agzy#r_48~TzOS5grFY{EShh-?z)Lx_w`rX zR`WJdJD4vwjT!QjtoG%z9-bIJKUoYaX4ZoWr~XQxr5!|0yiLfCHk#zR)bdC%jPAs? z4XoYHJ?40O$KgkrFkBz&{4zCpqeuAuf)*8kyt4-$w8wvWu|4+JTWt0;UBcauI)~(B zX~dcIsM1&V7Rg2?Gl$_X-crNAoXaqPYV803KmbWZK~#dszIID4xdxb+TcQt7tb$D` z207;PQNgRMO1`06{=4kGT0CWitB^vqaGBRq`U&mx-?{e+@l>J<_3d<1+e$VkvV{- z@$WjF^FecrDtRTd$s=h8w0$H;SWromSIU6qBN+KT7?VaD8kttB>!VrH{r$So5rFrk zcb+7szTd9C`f5Ayz?b>Opa8si1L%A0wbx2l_iOC^?|;8tdF7QZ#Y;PiHk$X}cyCak zL4gJZCMyL1Z;w9uXuJLP+wGov?r}9V0KJ?_K<{p-Z;1Q@?xLI(XIsAczXTR0ya-3{|b7POEyT!-1ORxA}|>Z0?L%%B;X>nuP4Aan=a# zM=x9})2X~J(6dz@z%_6W5LK_WP0cnoJScgnW};sOW|7ZB;4OK(h^LZWxbcjzwc^kF zAO6m|7m8u(>@%aJOYK38H0w;?)0N*?!+V~SKhPl^)^fIu*NLD&1K^zqs;_}G>+Gg$ zASaC^MoWQ=`6FOqqzt#3N2^Ag;5hi_h)%L2N#yKCYwy=5c1M)DN{Zqfg0A5^D|L7p29K@M1O)x-`L=fEPap0C+1dzsB z?}IoS?WtaBt(rdo4D_*aEcpjN_<`RZf9RoyNiD+yoB!mpVUa%i=%WsT%$+;e?!NnO z563ZPfD+nvUccqAzzvHA1sW7+P+$^K0QC##-o4|FJM6vheXpyd0q~7W)yOJiTY$QZ zb%4Ji5zD+>0_i4E!Pe2N1g#wabLJr7GwLZWVfqJj>>a)(Wp&_BGM*?Gi3?~r|0T~d zO@12AU;LXt9)&YENgnWtUGPHq!33|De5HR={;R)?|0?t0A5q?tONmEof&tSfegNb8 z`I8zrM=z(msEIbrm{R~gS6sGe-`BIkR`xBhrX`D<|3N^;nZe)Hee;;*-i#qe3&#R7#x6fC}r&tkZd{1?N=ceQxZ_=;mv1ZzAemI4ic zcVg*ZGxd6gZ3cuGc<5x_(Q=)biMXD4YG$P2xZYA=wc3vu*!;wXz9-tE>wR*`T>ny| zg}kIBlT9RYb0#&lPh1;+D^P$P@lN64ZNkj0S4X$>U`&AaxuGCe7h>j|F&uE_4%)lA z1_8q@gvo6xw2R4Hkuvs3&N>@Nq#iz(8HJ;!$j6kvL4Qhc(+0G_O-!XxD3Nx_#iG}8 zf0P3-nX>TSthn3ywBV3bgEiMaw$Ipz${WAP!xvNJFlkR*UZN;VCI@zt5G2UA-IB+~ z+h{1zE}hzdyZn&eectn*FJO1Rf5)amF+QZ%w%Ted*G(B05CMAR+hs9Kg&r_x`qiJGoB+AU#j?MpU}E<+H_m7Y^mOLYE9wMc{+9`&0iK_UVMYf zOdE6Np1ja5n;hItzK44ItgSXh6Lq*p(L?TCa#O z2@AW{_qa-n#AY=z?D?@M^q+Pkhbr{MO5w}w7)V*|r}1#KnL#(vFG7=E&4D&Q8S<_? zhw%<0AfvxmH&b%-M~h`lFbW|dxnk*ZThZBNy?rZfNXNU#ZWNWdmlz|AM^N^MR94+g z^rCxRPe0lsOeZRo%32#0>l0^Me_rWWj>0e?XH#;8aEcBeMINq#&oV1=0Tb%^^DHIl zWUu~$d#*&Dm-v?dU}W-TEK4Pgzi7vI#GkrFtd+dx?Z{=Bsg-t_$BFr9mU@Lvrp>l3 zTc?TOFlI1+NnYsleULGwMb$AN7)};Gx$%csrgu%ZhdZ9MrHdEY%q|_n!bqQ%gYU#k zSUsUHlgZq{4%>eET$|B3Omf?zZBuCQKqNhbnXrDmg-o7U5(X&k%ZFVF($rodJ>1Y(mYOat@Xmkm( zn98%eo~isFxz}UJO(o;U94k45o$7;QdNBF^^F9)etME+QLbZ@Z`iH-2`49JMPbAS= zHD^)=5me1)$htZ@t#4(I0~{Pzrad5;izFNV`2l)x+k8m05r465R!eiN?8#T9U#tI>=F5mxxCuRitslqf-$tnrvI4Rc&6`2KelKGyjj3gFDIKcf9eK^ zgkfQ3Z|Tst)P~8=fEc8q!8W(X#rxj-ZG;U^oohz{orcJYLrp~h2LoL=mrL(ML z%%{P3G$6ZRE#9Z#=G(OC76H^J|Eg|aph{o2|) zZD?hmb*inF^qMLhv@O!N7C-9sr1q@(1ibgtbH)6SB)pn8l*m~rk$i4xQCaU07TwXR zzJq1n6Iqm9M3>)~pbbagh?Hbgdd3cQr1d!?WaJ6!Lu~7PKx3s*-0m>Y|zhTtuV1BT!H*&{h&e_)8Kfi z?wU(uU4IofHKVo5-u~*N?F9XbM*O(n(NWGa6n&L{XXH<cc%Z4n1u**l5t5@oS zrRix$9cw4*Cw|X)lBlPMxA@FOp%95FCXugi#zcNBtm-kzC2u;?CZ@G}>U5hmV-p+f zS*c>^R^rYM8(Puh(Rh1>xpQt*R(dDzWsFQc2{!x;qX7DM0RUJq-g@h;nchi&joGtj zyFK787Yp%359MF}@)y65h7IN@-DuL)izo)KB_ zMe!YcQGD|xjBc)Q9!|<1Y`S8X_zq5q@yl`3n6xQ+^udSOk@~HWD?h@+lU0g4>>?lW zozEw;EMJ$*fMhWu4EOc;$y&T@J+AKn#1iB)Pc)rnR8(!ah6j-D zkS?VgB&8LRZj@%|?ru<|yFa?STe`a&hVC4O92(Ae);a%Kv-q+1e&6Sg>uD#?bxMCA zsYx$?&6q;$P!&AdBn2N zfVMmHBTe5I{*Y4EuKrv4Hpd$j+1qKwC@FBdjCmdL<0UiR;$X1FkLI-`q4 znRBWt=jmZkv${ohjow=Y$JHFB7FUUur$t&O#Gu6Nh1OqD;&Quda|s)@&#D!pu5%vm z(vdXc>5I9JUJ(hs;R>&X0osP0+l^zWMkp+5`Eu$`lkG$6Cw#F!H55M?SyCQ zUhBJWni74w7YMGB9ju$KCc7pCaO~zIM3ADd?CqqRFDn>uE$$Sgj`659aOCxqO3eZ`f;B$P|;b5p-8bl@_o~$j2N$)p(q{DgNTs zOjjoiRf`P|PNU!F#p^8Chr^6>0W12Qh@!B+4=3QRkdAy+uRa==`=9E#gZs1B+W51#2_~Lcqw#^PCaNHWIC1%t2Rd@ zfVq9SmFe)gM9Iw7^8mJ0)93%Ov5aGp|Dq!O=1nB!WFh9IMsr2QR>0XKFDqEM+KokC zgmG0cewa2)aQlAkch<{n%PJ@iW7jX&?j(}`(#Fn;n{6MAvhLtH7(qYq-yDtQ7}ns2 z#M(GNQup|;j~Y<37pJS_RX0qGy-29^@W~8ozE@;o9ptUKTo@@-=MZIEGv?Vf`dVKcza{(T6{Ml{*| zyuUPyqOGl6C~BSSs+0|bO|e00f66R26}Ev{)F2nCo)T^297N<9ZrGN9ju5J>K-rik zfgJIY7RKCPmg9YBiAuK1R)uSh6mLdj-7CW-CL;OKOP6`rER?okRKh~cAAi&K30EZ% zK3x93Qw3^s52P;N_zLHKUAp6aOe;NWRkZ}PyT)0*Q+jYl$+1Y?7AblguY<~ANlyz{ zOr))a#ry4w9jKWh`#L`(F@07gIV13%S!WkX=&0fkvv5*5E;n4XLfc!_tfId7AtHic zbfjI;EJdx0oI-m0=pF%z-i$75niYmELL-*Ts>%ENBXH0OwV>M5%;E2;0BEm!-BI3j zA)no8(mHQ$e-_0d1-`^e;WwqJnaTH0M5$j70@V&jaLJ>LW)YDS%8K@sHaw<~Qgyo* zf4a+jjG#}}#w}PYIM#3&3PRtjb0cs9IrX88nZPVey8)GK7UceKr6!wprC?_V-yNkNfbJO~}*tN1jh znrlECmrB0qyX7zWe(G|DLg+y3H@4hDXU6mku z!ePf&=Grh`XBl%%kj}}E<;tsE%^$<^w+N}Yn-V&xG3%`r93!R)3UMGNew^?6g8SJ% zCdUj_PI9W-UlO%UP&J5D%r)3TYtGT_(76*i{&u2gpEH!ElZ&jm={@1I)u^_}nqp6* zH&WR-J@1j~={$Ia3!kR&{E{W9wka;7I+6l<)UyGb##_zh?w(@2~MU`}>|4cUj^YkL?Q2 zFufygR6{H@^ru`4fb#a5&-pdVpo0HcAJ861C3!Hbjivqxdu|Qn*Zk6!GcF-=F^ zbN#C4j-n($HsaxoG~KL>b?LLhQ_g92pB8*~?z~;b-E4)BsRrW3E$(AYkr}CyRK;8i z>oy7mmJ=K3p~mqf%P2>eX&Lm&GE^G;`9$w6VzZTRLR1#tm@GlL9Mpe79LeO#xj@9- z_+L!ii5ac?C*sQ?d?8+B`RQEJzv=h0ro1eVt|}VkFHf6qO@|`(8s2w~)y-s8Et8$E zjliR;bfGgc{td%wj{$U>x0Ev@jumgB%Eu%AN6WcCI(f%x(v#m>aqV}ts1U!xv(Y-< zt=fw{&8BpkEa|ytmRnO3&U3H1%qdj4UxDrY+d8h2sv9!hH&&9_^|XN>F@cz9JdD#h z-L&}tV<4a*Llv;~0$ZnNbINSD_vj-#ljQJqjOy-SJaw1u zOCiFe2U(Fmn!{``9ozUtfWmPWlVOOcx6i9=dgHOM_hME03WwiI&Gq|>*bAHsE|d{+ z_jxzag~)u>(_@Stn;Q3Q&^^-yQ*D!K>^4#tS|F+e|E?DP>9~k!xu~<_{)<439BGgb zogcGMEb?fo85@-z(m0Y>lDo@K7C8pb%bV-09T6%|QhL*cPx@>T%)LBWVFb4z@&6k9 z;M`YC;MepT403(nLd0t1*kAvOK&wN}1~}HITBR7#pIw~7)s6EvRQ4A+tyvIdt*@Uw z`aSS>H$d8Tu5_C)^`>sY#4k^MqwbR93|)!+hqZ>7#4KAtwZYemw4iL8eVaY=1;*}4 zUFL-o+#}yYaaN3>465g)geGv)>d8-okS+52pczsec-CB;lp+M1VwEXv zwB;xl7dAtp{3s=maE$XH9<#axZ%K(%iKps_gE%g;wP)s}3&Ja-h zizI4H*#mSKC4(E+ezI)V+EGQ7DT*mGUYK&)RDX`Y4jIxSC%+Hx{XzzN3mxkGo>-D~(ep@mJW*V#qyXA<~)CDy;KSBU0857GJdSrRyhL&YbD zWeVq6_Pt)3c1y0Dv#!a#d(ia_VAPWa1PJGgjo<{#t94s8L>8MI*IZnnV>(G96pKLP6$usdt(&1<_0KA>k$ zs%`)2$dVYfU=pE#rugn-{!K&(T51l78{!$bqPBcYoKr}v>cXdnAK<>tez;8m z`7gTTaCDiWeo_q~nXpc%UgQ19*i5lmt(%E$1s)z4=z5#5_sw{meV-zyr`<;J7a~^? zflkZ2kvp0HzZQ0Z>mO}PVaI(|7#^M|DCWUdX_ByFc;>ZY@sn8 zSofZ6GSiBU;sr^3m4j*E!^`ie5yK`Yj!8t*@c<*%{m~BC2Zi>VcX=)5Hh&)Go2O_> zUZOIv^J?UgVdfh_Dk2ANm(Gb2Q(KqHER)(jtiXk%3z^0jop=sR#wbUmsydSRpI?ZdkfMzxGs(Voljl6A~&Zw^{Ai|35A9+A+f;l0!Txu8Bxt##p3$1&ENOPzgSx<6gcp&+HiYc^dRef ziZ-5xaZPGycc)K_nr7neE$!^^4QA7xPvrBzU+tds?};rJe_vlU=57O3NK_wY`UPO$;r{Ul%ot#23HT5~BS{|Sn5n$26i4vwK9=QZ-Jq&6h);x71VBd3@fiM9a0n$eW zw8?-FIj-a`Q*h;)*LWwugNwZOVAv&~KF$s*&mn3%+W4d__hp4Ek$=Y-{Uz*vhb)K8 zLC79ON-)lkOMZg`GL4%goUyYN>amBw&pvFqxc}urAP?j3A^3q3U4jz6fPw8M;=9(B zWx}FIff%8wX7;Xlc$VJ^XMQoj_hAh9S_fqJyc%I3>t+)ig#6vb+MLhpniI=@y(wOW zfo4p-Asx+`ux|+L10C`pFs(eJoO$rlYMs~k!)lN?FX$91^9WUX`#WUUd3*|u8WS_V zdoix<);?2sT$|9p>0L*>VQ*OVhZ0?mo6YgEb{!ArX){PxSpA1^VW5N$&dY)FEKYepg0CviOw{t_@Gxl5p;bx*CL{ zvDaE-tjY8+F;X1=x)FM^ zC-R;F%Ymlki!O{XC$M!`UtLAwHuNb8Sl#yi3?4$MB z*`m*t3tm6-n34T|)UP=Zo1?+vy^i!39#W7T*)$X(W_cnSOzqvw;_uJa(eU!^tVd4; z&#$H&-l^*zyHU`^=FWv%O?}USA8vcw)wqB2dVY#?b|*d!h*68HWxE#Pbz8F!Slqfl%R7D{y_!apvL% zh$%;AJQlRn$pPwV0@<*@wu}3lxNqDBP>tx;6v*IRHm_8QiOffP^^t-9>?$=*@UQpQ z4@tU-__7S_VM&Wb12zfXk)O6#qhG?Es|-E+J*{qn-d;A}R+>9+*_MPn`*Ed~>$6|Z zFrFn^iE0o4IMj&{6d{!cfG*;iWaB#nxrSYj@0|j&5hJU}C1E0hGYChOiwCvU?+PRZ zERLxi(KhhxH{Mm|=n!AwZ}*90J+ueE)p~q??nlJ@jWrDbWVBMuk6>&Jv0xjkMBj=; zmke1bHHgX2A3iE(dyP&NT1lVJ9$<^yC}*Oip2xRbA5PtMPA-3adxl-P&&#X<9{*Oh zKTY|+8c%sG){_c)SmU%Lk*Ne_-i`we-Cqu{q5IbC&G!CpUfx%=4~rj^;VgD7bs9if z-e5bSqpwqZw$qL+>&9=7=|@5q$Y*!*tL{@^+deILIzi?BoXZ~o^)AIQh2xW&eU_(6 zxjP+;6nVQI!|%a6ZGfD2op@6hSt2yLR&mH;(%-jrIoXe`v`WgTWj;_mvYIXbO;ai+ zC>K5uqu=U?eM24YS*9g%*&E@gG}DV?B#*?L_+qUD>FVKDXzWnK!z) zXgf=#@XBnJR8~vARJPkAvdRMGlDXTysdr)~fCFV_4Wu)A}5+x8gvk<8zOf8$0Y zJ#QC>W9`%vI^-{GK=pJ_=XT4x+3@DIiV7_|ST1U8>8~zIShFyr@JvvVFMJui zm8-kE#)M)cQ~|Ifo<=CRg{`g59k3d?I}PXpY!plhiIV|Tcmt{Y6vbp_W9c^ydqp}u z51xP<_8U_cn_(fsYWk!20V4ZI<7nNnbwS@QE#@rlsr+H04LZ00?kqa%U^)^SgN8KE zRU^}n?@gP3jPP9K*sryzW8cwuCo@0KBX5c}O_;k7UdVEGKxS9R;rluUEpLP1`Os=N z$vJ!x-Yp_?S^j|BKJ4bd(Ztu`9e&fIRJ%W>u^g3mtL+Bq2*~6PyX;h&@Epc0z5{BLhRfam=%*7ymG1SVo3dA^Px+1OSx49%UGrbRKU4KWuSWDjz9xY)8~azB7~L8} z1dEIvplMxpnI*po{~a7&Y}nStxqZeftuh8|&0Qr&+qyFA3?Dq2ftd%d2MzRf_6E zy7RMag`*s}O?AAG=jxKSsnFxP^9n+f)FBP=S-w5*K+~MW^HFwB0$gD6$S_J!+xz<$ zi!{nV9y+_WZOmBX5eSGmEBrpIpwa?_jw)ap|Fl3Vzvo3N;-WdFjVSSnKaDSgzk$?Q zc);S_df8+t#EvRMmc<=Qil4zIZ1mdT`-Bbl(szy6^vW-F13JAC@^znzRz6gdKg>85 z_0Yv=)&F-xC_{rC+7f$?1P*PExKk9zU?1!zU^v@xCv{#J0&Un|1t=_htqKlo&L-Wa zdCBYAasP(;)eSs+iS#v(CQ}Q2#>9iQI1pJH5Y2_hP znN4j9QMS%BdIRilNDC`{s z)*pG^ksfS-;mx?jI!W-ee6D2ffR;Wb90QGGL=Cymw@syp=!DB+Zx2dO2mD>f>3`nt zhr}9^ciyGkf0fy=YuGMwJ7T%3m4MD!W)=jlnWhUd65LLa+rFaj9#FkEk*_4AT+qU|{7tAusg7)&6y_@RD>|%< zzA;PmxH!qHCEfnH$(Gf3%10;Wk9xtQv3Nv+jPFPza~{k-Zx#Vf^>BzS!(!n0Ptzfy zCSuPsm6JmbVrXxm)zyq4Yg?s9q_NSfELh_K1jG9Fn%Qu!7RiXoX5Z&To3Y8NQIh7= zv}YFCez3-;X^i@+Q#E}zEV)7ObuaVTw}2o7QL~sWSPh-kTKSfnR&syZN-~CzrP!W2 z%dq>vrY_%xv@DapP6l>|3oGy|m=Ndj;lWtq@;eA&ot^NHaa4^u`k4SvvxK?Djzk}) zHy;#{bv4i%Ef^o%8tmQ<{w8#*S8O%p+9G@`R$rpzQ}%=`kz5CBr%=K3dhjV|?p-BJ_nil1O#Vd(ZXU zh2D8n$Sv=hd8dAGTXaVhc5%aXl4?V-`CKfhb+C2O3>3-F%Joriu}*$g(5Q(&ZN`S> zQNs$ia~aF)m>oLqnZ8O?-8s-B#76!$iTc4I2vCo^}@PClgW4-!;tTdBL zLSF!3qzS+)U=TRCsP zR(v`$7dsVS_1?Iq6uKIw@im@W)k@>!Ie;giEr{4+*T0ZU3s@|<`9k$6z?yAji&1Tl zH*0xFXzR|s>q)QeekbBzYVq=u1pW@)n~kU_weOhN%drwHll#;~arCl&5)G>XFM_`Gw3ATVbsyWMVUKio|3-2xN8ukfHM?c--pWgZ*Nb>F*n zQ7IqT<=E|Q8gH(kslBJ-v)l9-c7sM2l(%VtaIT``PV^MmK3>GfR$m3B-^8%CJFZz# za}Fr+_{F3~r?AqKNpONamJH(4nXEO37(lOA?FbS|`;4!$&{f8h1D2i2na`A3iqU`r z9!AUfo>pnn_CNZh1bOjEq=w=~`J-iG1e*U z!4}(A1)R?Ui%8I(4_VWRJl~x5;klhs_LGgf96j!tWP~9xiXrEKxslEbhs#9zj{8RC zrKlI$-m%PjKgT;R2m7Qw_HOnq-;}4}ka*L+iVtYhZKP%yjNRJ43Or}T@KQjGuT^2> z(W2&2I1APbG2cSP4J4kg#OZSYS>P5ITXE|FCUCmRA2kg^Ll z3`yxS(<2=lJ}_$-APG%+Ne1;19K+A;uf$(jHZ#o{exi+ab1;Z=xtplD>$vk+H7gcd z{Q-RY1&oGW8K`Qi^{;=(WWJ7yKDEe!|1;iTUH4mEK^D9CavblgNWZA<{U&v?8l&I~NrDH2xKZX`K@eRC58h_3ebPyF!mew43*t=7Z-51AFZq zES-tE@=bal ze$O#(9S)fsXXWy+7ZDviGemhiCII&HAs%^3`%wiX8tm}ar~DARXI)4A%@It8VG z362ljU`sI82-kC;&fBUxE`m53Y@1dFzv$EA!}-}ioU=jmsosX|+Q57q3J!9PFTDu~SLb2dnJe>!RWJ0q1@dwB)%e_Jg)Tfy)+ch1&n?%yXi^R0|T3sd7$ zC7xTR^}1`@1RGP|3~}-S?q@Y-7M@`9<@j<2G65FTW@Fu!tC`>d0$*Srgo6keUPcL^ z8x&FW6mNqi*2ln?5q7(Nw&L=sAX{(CjL(ma)z`H}GGGrVk8P9at&(;2f|(w?tB_02 zkBCjDg3q32mfXh5M0sv*R`vM`>9O!70;+HDr7riI5I+^~?n=DyqsA&rJ&*(QgrQ8W z?6~UQ=Ww!H(oIx|%3~SyXCy;s$dCr5#2(voh_w%P0wWgoG57QPilybli>oAyw!u%e z+Nep~vkzC)yjX7|!ONqs`&F;`9sY4Z^7TAuFsP<=k+VFh>kixaQ1i-Qx^03|LT=fI zxV3#yV{oeUKZ%1^+BcwO@lQ2bz50+qE7zC?P2Oc80Pr3kTdMXdkP$uDAdQzpuGOIl z4&KJ`k`TJj!ru4<>6&_Fwyd!|oM?JQg!f>0s@3EmPYCs!z23!#;`AlcYPFw*}r z0@z?(-7%?uTh{jiy@vG~7qOP#O!>?MKf^0c&!7|YeRrYwb!J`8Z9r?lVKDcd%+`d% z?=k`mJIf*vWIY``@tQZ%n-E?i@m0VL#4v=ytfcvVSW0w<*#qz=HcjMVFQE{K z$CUOnCY{0ZynafQ-SaqZ?-g*AJGec{Qtfgqzk@(4`u(v`%38VS0_M@pa)g0k#-83N zehuY_-a#H`;`{E$346bKxyvm7*9R;wxO0Sy9sVwXXosl7Wz_&E%CKCz(cCuh?J>n3 z1u)88_&q&BeZ?Wi54G~%BM?{ zyHyz3*-E;>_--59f7xs*3v7q$ec^K1!7g%q*Dz#}9;uXJVNf4KwHe6}8Bv0kyY57? zPw5S|toGg|vn>X@Cyp=2s`s$oIu@w$SGc-ZD=s~+7k(D$F8>f!tX5#oKbZQ#w(0W| zOOCrVkr0M#sDZ_W43*rfP+_wMnWF+rN{Je6tYp|hk3GF{{c5qWh2M4gUUoA)hw8b2 zjWME^!N+Pu$Vzc>aYLWK0PQAhYFX7TBCbl_ymTnf=)&_IqM4|T;67;}Fz$SX6I6lB zO-t3RnY3-+elx68VSmXrh6hzVyrveDt!+R(tU)z zN8p}xvIpxl$e+(v42ZVol8F*^lGR3Lx9F0K3BmRE;d$S>K=c z_3a(=Gh7IA5(vmdU&1>V^TGlZPWWs|2LTfXm^mu2C}m?&b?|Olk+`cREEzv|7H*ejlgKof}fo&QSxf$c{e z>(rg;0&;M`FExb5bPNFn(WSjj#(KFw@l<~BE}2Mq3P)>fQ~0knBw&~L2-AYs`0G~1 z_V1p%E$jrNt2Jm})7h>28a{A+sJrgmmH)kQ!CjJ{J3H3qBw6h$TSm0q^u9F!T#Be9 z?d43=c4SPnqEYo&rp}{#e1f3(`Su&Igv#;WnpV%HQqB?ng3{}#{Om_a$Ed#Na1T2! z$-0#u^=Vo1FG*c#i`tw$q9JQI(tp#Ap!6g9@s6Y_P9sLZ$2CT=8Tu>K;qD1&5@A6= zIk9-Uu=t=6wgJW#yo6=|7cG<%c)vrWn@s^R&+;sSdEcIjDFEl4Fd?K?M;Eg9$kL)> z)Je$IUJK=Gl@eZsjtrOp=PS6s>O65>dhY8BbX=Ej$4dU71(S9a160TPPkWZqfuB@` ziMw?R&H$lecP&#b{SI`bwQ#nwbK^4*GPR z1}1#vz%!Cmyi?eC5{$fMwI+^iurTUZ80p( zK*Y1-T~F%}AD?gG&yCf-m9|~GSc?UY_To||t*)NhA%iLB$8#0dqeeP2Ec&86XiB*HTs$@E&jHCpfr4)AMRQB)rtRNUg$ipwgvEI2 zy`Li(_a!B6#3vR%)~26j*{8YGv%kO4V2QePA^X{h#-rGsA?T#SWJA&Ob1dRIwvaea zv?f>9yO7i#G)@99k)o5p9t3rb@R)E=+I;r@_{L+2IXoWoY+C0xt7ctANNjjp?fnKd zf4U+Pv!C(+39qX;{+>t^9r7A5Z)g~cc}EK~oM}$AO2;e;1BqG`jNs1I!shO8riCg= znnAiqex?_pBf<>xvaIFLQJgkg>?bs9!{Ae}GFOHOeEK-&?2sgTmkWQoN{=@veNqqF)O>Wxj6KLzk(2 zhbaBDM@kd>hn&%0-$fPGVjE?-PGbZ4Y_`;;A~1 z+28LNXN+vUBA`Xkb-wk-!siW_9@2ch3*L)la^%AmF4MB|<$XI?)>nVkAs@5QdYl&3 z(5>$-d08R`XhP(7A=30z&%Z!2fymb&-JI*@CTCtcJJVnFMA!Z@-Pukhk5d|KP`6Gy zb+G)Q$iHNZyqqN!%~r@6+aFO35vh_uRKx zeJK02kX6}iR&GwcD~Ij+)S$0B>RI~t&{Wb$RY$$#OQDj7eYNZ9z4=q3qV@Y>F4H+n zH|W!3ogN|-J~Y#FaK$fm{um-M2iF{kCAeJmc5v&~=rX+V42B%_)=0d}*1-HV>02Q# zG15q5QCWQG0GWYS$F=Go9i6O1ba7N_8lW2hn^yl{aO%k@D@e!>;4*XF0&9&w1Q|)A zMDtJMI{QvHXcApUb>(fMsDVcu-C-xM3v;|VvCCqXLoe!Uw5SoSb}rm;R3g?;a~xsa>jb;FRrpC5q!baI93~>Du!C>bpq2C)N0O(mQOs1=b z;OB^T-=RbRPq1s+6^5+5pS&nM;M=iXk>JweQf9R%eas^J5vMh|J>y(ViePb2S=Q*imNa>$16ojYWjp$*RMqUl#KQSsb z3WvDR`<-&_wMa{h;=eHPah-%&e|%9cPDFfk{5WZe|UX5w}QZl61Pkf>a8sw^Wm-(K%JCi3ip zF)JzDmV3ZLbf&9CRI0&hAi)X+akX*~6vsD5ftkuB^#>3bqo{z^4o}f-an8ZdM1}LB z72nQAuZ*4r=sMA0yFeqM(xhxf^|y=nP}kcpV%5Dc@Z_+3^ekWV9xO6(+kX{lkrih3 z)Bw=|npHts1-$y<+oN+-xIPo3z(@ueCy$5Un^Wecv1S`gZ&`&&&E$u7M!w}DA-SgV z3;Vu-EDNuD)mp}-WW62&4->soe@9Ix77Yek_18b4yBzeMOeUTwC}@lS&U>hRGD8)v zm~J$ zYZNn3@736RKLICfS?jgfFc~(8Y1P?YjpobKC@-&#D=8rZe>Jw#MY&7FBpzkt+VTiF z*fxqJjN1sZOST7=+tA{w8NM(_6`jh=ENvJKs^jcp2kxs=FnEFJ)2&k^o*1+lw;8(E zuO_zZa$Ui6yQ`p{S7Q_R`5T?iR4pyh?97T+kY6ku7gR}M3Ldqb{u9zkK>GP_}|R((rPy3a86RU!U(w zYh28o+uB|IBT{lKk=+_>(>ws5`uK$G`|C4I>7;1x(4Ey>TDCub4WufUiYFQwmB9Yo zU)8>Y_Vn*mN>N$62cLqE@?)AL=oZ>^hvd=+*q&z0kl&E{@hjWZ90EiG$O8!0JfaiX zXRW$k!1fgF`$udsr9{Juc&wV>v9eI_^R<>L z28yTW%gLDnDdGP-qUmh{I&x{63A^Rm;p+au;v7qXCxkU5BKeINp4Zs{1l`i&gTjC1 zmOsXl7y=%9qiz75JD-Yj%qtr%mAr^|_Cz{q$f=p!gs*cr#El}NOW^t7zQU5VEdZ+>%-{O3~KaB7rrOk0L zYV1*XUy<9UKt^mB5nTRk9Bqjl6xQwUqjtz+n{)UC<`+xb3e%Ar6VuFPq zF~MBpeXSL!%0vz+o3L`Y9^}4l>J(OX<{_!lyzMH{#&Jx1(kQOiFX6-7u@esYrQY#yQ-0d#m?ajDKJ&!@mR;4z zitXwRadzP5qlva@zTzVMgmPB+Bz3fa|0Fhl_uC_QW>gQmbF@ZptXExeT|rwlfRgK6 z^arfF^u2GYYaV)%b*?|?p}*Cj=<=6mR$+gdH@B@S#5=H8vf{-U>T!c zl#~2HCy`Djc{J1PgI-%9@K)^m9AT(yflKfJku85+V$(KjmQgzUV~oRq`5%^hAowQY zIFi7B#y<&C(f(s?DWLIMc$@+hK&+}`V|R>8z-1xU_ClyrXArlsaG+VjE3bf7(N;c| zHlTkU?HTL39H3$DmiJkT{~+C+xwcTUZ8@Iv7afMbCna4l@n`;VZCSXJ_c+et=o_5# zzFs?05uaPG(WOM%=tkO9`HmQ^CGh*gP2Jj~cP8y@n__&92!$>SI6y<9!UNDvd-LNv z4_Wi@}4Q^E1%&aFn;iPm+Cb~uGp># z=6rwB%eZ#_)u|yk?foBZEP!J<;d^sx7y7$W(~OnUeM7u9IBGb6<1-oH8KywFS*N%4$ z-zUCy1n<5`uYOp62G>A$V3k8)OWcjWh25R#M#W50)@XkQM{Y@kE#rm4WE@}t zLR0mVmPE4nMutjVqlwfD$it;FVDO{qg!My4`BqHuQ*@C6PE&Tk0GY6mrgDRtg8yd$ z4D;vg!(vzDK40UBn5;$vM3vRmsV%X@#|oWFrgE};X^LY>dfy!(9)-ZIJ;IUzb^)+( zosh{^%$|Nt?ko{N}jl)+;jkcu40FkRBV3kKsee8=dc!j6S>#aHVb5&u2Bp zcP<2n!)qp@T@Po|g#U~4;FpsSx7yo9oi;@LUO2C~m zGLxN)3}6Qbc1C~Wehke;y+(+i0JP2X?#kJa0wU-5(%+reWf_UO=3~BBv^UapZY-VB zAOo(cLCx1wjq5)0>eFMwqON|I+?HwzpQJW5uWe)6bbPj#7fqebCb=l&Yn>;BWy%zi znfNo@X4aPGZC}RPY()L33)KEc@qlv_1!gslZ_+sgzAXa?9^%RS@*HPdJ@wU|}vHtIU{?sf;!r^IkD z4t{s$C{IR818QzHg6c-7GVp~j!H%3SC(?MEOeKZ_k>|pxB8%q!Mcc;hXqn#_==jJ; zzg#Bs7jqqhlrDYWb6-tKdn&&8>>{P3zjc5-`>htTTx#8v9Sovc4*Rj080a=s5~>|CQ(nf0T6lhnA{G{hRrS6Oo@Q!NPZ>4;T0u)(QRU}XLmRs3CgYN<9CQRnA+1+M|GRhM_Ig?hgBo=wv} zha-^r0S+NjYKUp}OJpD^L;J1=91ceVQTrBRXXvJVm+x8f+Y7Mmr=S$%`8Lz7FrcHk zbDs_Bq^G&(EWr*QmS|)!=mD8&t_jr_wL7ztyoA9N;|he%e~x{iGC(Il0S*ZeY_OAl zcln=Gn7aZQfW2EK3m`0gWmK{X5Y}K&>cPOonT*P7>BNWa0DLcb$HTPs-js|G$&~_cU zh{Bv2ivImj;oz|PgN zapvRJd2bLpH|31B`W$ z<_@qemRo+^AG!5Tn5-=jwQ`@Ud(^UFnxdym#X@Flc*TR-#(PPtCqFMGs&JM=MTR$z zAohq2t*!k{u?aYtm$9+nM7oN80@)0Ufm0okI$Do(iq~tu;M?LUza6C8H^Iu2Df(R* z2V3exFvkj(7yrcnExS1Ct22m@OfWD*^N;xP{LyV}C$^eKBOiWU+?{J8d`siPw?u~( z)m1Q}(8>bPBi&Uo-E!;+FTlM7wJ@gstUmiWP2?tf2wZ9J5hh~aOi_?$QTj+(9F=Bs zz{qOu@?$*9 z_~m(iGe%RK{WNI_`b90)h1n@#9({ULL>>q3qT{8aAV6DQ2NO>GvDyzsik#7fOIZ-r zcun-ijYUY^PwWx73bhlx`9r}9|5-ei^0ER&4dxm=&LIixTw(*L5@p$s zD8^2$!mJA=%_#*`GaK3qlNa0)whrPwt;H@+M$VVR-?+y!lrF29V z0Mag(Sa)44EiKgsG&63bu641!X4w1}))6KQL6IBMT0pI8EsbRa;|_C1)xkSlVBlY? zZ?Lb?pW*}HmAS7JMMvI>MQ@pq?E%-Z-}=^jGPp?P;KDz$9i{S5zwZXu4pG*_WbiR) zT#QeD=H0qG5c%D;Uj!tvSc^gIA1ee|x|Bm)0$E2LY3<6C&Zo2UBAXu&C@JC53kZNa z_q_S;MfyPu~#qY*%W!GPv~;r^g=2XP>uI)F4ze{c)i6%JtAfbN$V;$sN#ADM$)et@Oi92ED94t(^xn_yCOZT9yMnA|jii+d;a=YX$qAGflCQNfYc`wz4a1r$? zLvn%d%kzkKDGbvaZ$GI<*8tT_NfiVWv#CrC$#e8n6bm|!z2_Nj|EPbq#VB3-$KU<8 zPowolXtw>v7({J6y&XbzzQy4x>uo{~pbqII-offyE}Quw_G_=Sa87m?**4;m*_jbB zoZr7SRy1x3l!Ksyw{uAeEzyN?N$Zt&-Or~Z@uC?_5h)hIt07<8hqW0M zROljdGW+#cXCS=$+x;psxRl}5%KJ=sK~eA$XVql4bWudY4<~u}Cc77S6w;uwUX;O5 zD)isu!_m0(F~JU*iIsf2z2><#|fTWfwcR|mh&~GDZQc&}mwi8%zDn?zgv<}RCSc>bQ%aj9} z*s)(y(FoqzsiCBFnEN%?uLrtlLv*;kt*#+y&9F7FN@vq=orZ2$OQm~mqe3?!ZQ~oz zNu?EkN}=TqukjwfLM|q-f==&~yC?g!lI=j8hZnpq6 z!=3f+;jxR%u`-+4YMW_ydrCfEx!n@UFAo*CqG83jqDNY<s~d zFERgQZ|G^;c|-j>MVqdd`hH^j3ds`!p~Ew0m$2nkaD8^ixZU4vomr848lj^tjmH5a z05^H*AfVy~U4b2*P|KUc)M2)(cFXJ%8>Razr8qhOQ#=94%bofq%F459h|&S8CbIQi z1Aw3hYucQ3lstgMF<)@Y|Mk#+vEIjLpR02IQLY8arEb+ao}a&&`*VTYw(KX z&x3t{02L&5gMVw{5d8{=6JEdw*wW~F{DJJp-g&=+p-k8HT%?$J z0{9?)1Ph6HzI_*bzd#Jvw1dRxJwliAx8S7n>HKsyFfmCWPhUAR;0ESZ-Af=BZ#*CH z3w5L1%afT4cVliPH$}VHOIsjUULY%nt8*F95uI`AQ-jH<2~81VeNS~ZPae|v?X&Dj z*?_rZD~(Jz&YTbMYltWe8?c}2Q)Cz&$969`VB+6mZ>@&Afg^_-6}|X{?U6cTk0u~T znZssi8PlMEF|_N+;Qu9p5qopK@81?!Z*jbK>GdqvMXjdmNg;DGr0M;CT=>8XX{VjL zvhpe?YC$F7&Npm+1EFr}hd+Mi8IHll6b2uUXY@RxbUxk(J-PX8=?P0u%R<+iYq3qE13E*n^_ zu~#-&&jE}Q_gh;{QfLZp;uIZQ&myTm$$MYqN}c5s=^80L51pOek(-ezLTVW(JAijx z7Ws&lqZnHXj4ByK%(KPaavC>kF{Xo?8kfSXnIE%(qD2_^<6#!{9^u=462j=0L9Pq8 zl0E*7SM!;E$p(+A=FbBLT3(VNpiNU|0V{7d=cu`O)8t2LQ3svFE~Fj-p5~``F9ProG6-Xhx7>%=z=nubwv} zE*x$Y`kw-`#2-X##`uAQxwhDj7)?E7+o8YsjV{vGbq*}wyYV9;mB?c0>}9+=EZa5aseFePm7oOuW=>3XaT=ftw_FXDFBJNC zlThQVOAq7O14od-G8?!o}h5I zh<@&qhcSt8;5hQBOSq_soPWV%QB=^mWi4&EcWIq}fKn!1v_vXZqbr*{ELRME2=(cS zt4+ICKU5`sZ@KUOa}zD`eJx|-i1!%5iPhNBN%GA^l;1Qc1?lw}(sH|8I4{oX-vInz zv02ILF+b$UMPs>(W_4q2dqPe83ino`Dr|E{4hd0?z);iFM9+h&63sB^h=p^``f&g=uog6|MS(NXOk&$OznnJ>D3cfjX&lVx)B)Q-q@Y~3BCI4XM~txI)sBZIP{7Zd2V zJ*-wOy1ry7u@qucfCn|^6NnCcLhc6hn_{iKHROH}q$B{i2nt)ylXwLf(2&1>fP0c^ zYPfIWZE|6OtmA3wt{a6?1bmCGl}Tg;M7G6Ox$i^K{{9QZ*GM2NleMJ&h0EjL<+^rd zHWc!?m;xdoF?%zJP#F%23L=55g>OEFcYZf4V{Vk>50t9D>%jyIFs%6Q$-U@!;sXK= z{QK6mC$uzwy5l1E-Vb?|FcLwTYI=Z9YsA*3Zrte=^%at=eim&O9fZ}T;Pj)x{SCHD zegsY=d#niYl|kG+g`GB(=z)>M9ddV-{cn=L5?QN2HN%P8G5LJC$q_R|a*-?{eUR zw}JjVFI)^2+Ha9>a3XPVk6GIAe=`Nx>ZvE5{>jizn%xq}3J+WtM|AC+LVTD$<%;AP zsSxx(XKLE_MmutS9D}xG+`dl-^h~D&Q(YSByuQ!qwIYVQ#y=34%!Zo(ydZz_{d@I2 zUyWEWmJPwLZQ(AuM-AOTu>sp)N0(W*+8etu%Oztna}LJ9Bb^Dr`XG zu~)OaDukk01@(H|$hUY8H$FWV7 zg`35%*1HA{Kk!Qps;WiLFua{QM3407TUG0DnM$+uX9EMdsBOi0e8=U6QtWqPO%1tU zCwYj5nhNZT5V~{SdemnR>vw&BsQ;(ZENN-MI)cI5Xv_hrA07BFtS_*@KL`B-A?GKd zF1lTryC4T#$+n|dezR|_?Wz;M)dYIFS?Y52A5pzW)>$>eW8l9X*!3T>j9A3(ANaSB zNiEZ@0ABpx7U$>9qJ$rk6k^^XAVI48$eZ}@oTf>lYO6>2rdA$mc^_3bLMZ&YJCMbG z=(+nwK3|L)G@T6F!;+?6b_=mLH0gXBXnY?>q%t#)HhII{E>NR~*lpTvBlW==j-P#e z!5+U=Amk9P!iXI!OKeK46Vg59N0JfFdo}F~tNuS6KJ6KVojoyd>OBAr* zLt_RYD6+b`H{ZAD{jyQM0(DV)RZN|r##rS%mBzWLKG5jnL%s;Zu5$C-J9Y2tznb1; zo0(@U`OFR!12R|h#v&dsNckd+6Mbk|+b{7d?W`NZXJ$ym2lRjhC<9bMG* zB>sRSMu_XVlg^2FK6)6`7aev*H~G3JuEcqmhy4~*x? zV)=9h#j+He&&DvUpKa*ve;TM^XLj#heK%6%NVxK;t-IUDU<;>Hzjg4=z!yn0sYgMv z=WMPtQfqdqT($M@yTzd3{1+`j`NG;Dv=pCeD$QP$!NkC(1u2)b9EI41(kVMLgW~7w z71?8$q3zGeR6y0#s&mfOj?GlL^q=$vuCMoLBeUI$I%Tms#_xW5iyLCweGZw@@t>!NUJi7lZLQhz{pVuGf%|+XWyfj{v zsNx5x0@4M=e!s)002?3Fzc++VLPy7ckaZxhT#NG_ZP3jdI@J6u`J$Y!JJD*y&(DZG z-1^ICCi5UFLg0p!o;s_oYG&V7^|@@XE@4PxAseY?4yGc?pZmGZV`8GT6P~C&^r1S! z)yMaKjmmpxhRp2+DY81s_bJ@%h2Fnddey1v4t!c$`HSH7;|@ZQk7?w>@ohzZPvva?i#HF589$3M=}{(_|)U zClTdKeMnOjyS&RnI zzNPYwVzK>z+?mK!V< z%0<#{(%O0vYgMehBjfN^zZ_Ar(sa7Vw&`x=_~6w<+Z{;6;)-6V$aD6kc(>IXQk z_u{6&O>YpeMHvuUZaOlM9*F}kd%qBtOeE3d^#Of!KX#BCV!wQAF8LWaJ=(Uwb_?Z} ztzO!*i)ACiKI4k>;`0FWY59t~rE)9C?KkI2phkCxkC^s2p(OczWG${)flxEGQG|8i zS=^XK(0OR@x=Nn3NEW^4?m)B5y`>&=I9S_J(Oc(PaxY6Je(I_)Hm$M=** zR+3NJ2o5!S9s+;T3bZrl0_p58LGSa}P1}{z?HI7m@>EM^s1yD4&P(j}qiTIzag0X1 zP{rfkUFY(QsEy3nq}mHe(tB~=?zQyWn>U14VHH;s?#7<;Uy3deM5G&O$Fsug?xBLg zjn2}VUzkagj1u>$*4j}2@V%yiX|eM*&(fVHl6tnSQP(=+UNnCUAx(!S?N&VPlFfE| zr_^QbTE@N~@{VWEXbAOL{@5Y1OPeh@I%e6&NcSJ_`LL}1y6^Nk!DbO>I@cHIrSbb~ zKZ4mIt$NGC2DB9`=V28T)@+~HzfXJ_2w$7w|B<*p8tZ1jb*5-6*cd#2dY#}ivQ9}4 zQ(gQ_y%Wl6ZiwfFdX69|vl)d6(c=JY(h`K}GG1aHao3$SB4*_k1BzJjQP@_JtX)(2 zINm9+H^8(|U!`X^vzT>8Bd!Jh6uZIFi&ita2e(u&E)7h11Iv&|m)2#00$q#QdHaeD zk@YRCIrhF5^};Y{(o{Ym9%UTgbMkpub>j7lFM)zK3Qb9Z2K#Ew6VH-N0kCOjzv`4K zV8iKs6<}~LlcMl3RQ0-1zOOIFK}GV;qph2;SwTm*+Jw@#~|n<8bBINE-iB?^WmVEm(6K7FXO!$KuGtW(4?C z5H1LJrC3>XwY&*}9-PPe9SfQ-OSHrff(nZ%C0^zs8z3ksdsvRGOaYN`jJNH6gaw2e z+Qr*WOqcPu7wWYKwAhRyy=Jyt0u^~hGq1_@oUn#btdu)~1`P_8xR};ALrMlJm5dxh zF?jMbO?D%(Gb1taHO2|xKdK+!a-Qe3i6T&dUC7Xb8nZNl;<=MH*r_^^%+} zqs5i*MTFCZUasGqb1ST*tfxc3`Cq&v49n;cM{@?o|IhJAs$&=>r6tc1c7v!#q2hAC z|E@~07r2?x8b#;*p7K8Fye{`#Hr5C<4q9v6&9FFPG7z0@G=;N+al0YXYa(K}F_iB7 z2>*S`()YDUp6jgDQLY>#b*#XIHF!FEjrHAXKe8v|c&;}~r#C6-HMi=ACv0j|Cv%?G zq50or9hcTL-jY=`?S!KB9BLgYn7$p}U_bG${h*-(T~7A(uO7ad5H}suEOGv#my$&2F*I>l(5usIRgkp=jJP zL5|)kHWMf*(P}9u%^iWsqp=mzZMT18xw;+xr-ptkknAMk9O1iuNm04et8jEtOi||< zK3c2R>J?W9|*@a zG!0UZW>?!ShS{1s!gX$Z7LsO~n0W3eBr5k7(2-^DJ8a2%6zg zlsxMWQH4_fJIR05(IrGiq^m5v9(+47lg>(7KDsrNhJo=xH~?O#U4vdbg6~KXZc8-^ zYSO9lWeI9Er6B+zz`lh=3%7Pp(EC%@IdV!rzD%K>F2Y6*qV5S1$c6V&We5cgOjtwE zq@<%L3{0cvKXvDq0~esPY{&3$t^vVA52U+z9&Q{X2F_E^ngiyrdHXW5U^J8VF6bi$ z4Y?8QkcqRoZrnJ{M!>G`)D3|8KB=oc5>xqR$=T7 zAFX4@EJm~x7Uek>(h4S%2&T~pR=_-qG-Prgc(H;>4zc%qa^7WE=ofb>8fjrjhL_A> zg>QHoE&)aiL2^l~=Uq`2y6hwRpQ77Xd`wYq@6CqQC>NRv)Nn(;QR>@nI_0|uC)an=!4!gF&pByN*G&E!jzB|>=uY;xv}Mo?F?QF`wR+yvL}qO24q5^f5M$*X$k z{ia%dFD&fBdF>hE)p+NQXXHhwuo zP4pW5KJ`q);3w`_J)Hjlb&nHzCT0M}hszj0gE(HEd{ zD!6C3dh_8+`x`NEJ83Pub2PSgtYxb{%tnoQRne2cyidB+6~vDgeWTvGea_O3cHr8H zLhpeCoB&>+B{-k(Duj;+{UA$%Rj5ZBhc4&{r~stdn1Ob*bhtGlMY4%MieURd`P>w~ z^U{uH#8oQH&u~B|=KHa8k;xyYcw_VKICNz}2;F(d_t|8m&+j_po8I6HW^R*OU6A*A z$lIL!LNkbbolxdh$s0 zQ%UM*ETY~Hy{fkA2q7HGc1`!AIG^%c5{2dbE2~T`eH6r4n!;80ShJii*-91QpJC}= zTylkTkNd+d4&Gd2I0$BVMr{&_0QDqDAjH=zAP~j@*}}P3|L~oILsQ%&5rMDw<{v$n zw+ASfaLy#d)16of4#7b(Bp_)XhukGu0YI;*Y?Aal6^H|=Z~7xw#8Z#uD`O~-)bIS4 z&9}O;t?5~(uA?3whCEQJ-y$_esfK97YesNh^Ed2Gyu-rOz!-_N$}`s`kc4=YS2OW(aP zS!0ifq;0*GFrRYz=&eO326c&$6+2yZ_IwmWZSvP4zxbYmb~EIwkX-zuQmOQ(>d!?2 z^2=GKYV}sXmk(@0c@?YUlpp>mWxmLeL0uy4|F+$}AU)?fis=CiTm^t)(ws>b0i|8< zPa=^CnYi1h%Q1IcB-{HI^rk z0PH{GY(g$kNJV<}5_qJd`8rS&!(IoPOH>kFc5B|` zFWVX;>>y`Jf#D~!^I@;RgK#C6Avzbh@E0x%vPW1fu6TDSODuCNLlPEk-?Z&9QLn$9iT-UAS zTfV_AEMoT7EdS<9;2}<#ccZ(>NfXdqAeU_tbN2XeAIRa19PbaF&1neVPiuR9J}s*p z5`Ou49OZjta?{J(IuW2ZnI`OA8`zk%*s+9cl{?}MYa2Xt(iBJ{%Zlj!=6wE9NgmlK zP12pqV-V=VvZV}(6j z*DKGVK&J)BFS;LGuBSMpLfUL%=>+IocC4C#*$K7M+qelx}i&Pa}!MpZl z(-iS0ebYvJUEiE6Y*l03UT41%Mz+|uMNmoJ^}|8WdXEJ;4P*WSy&#fA6C#a;lFH*9 zO3i`Vly3&p#3Q|+eq@}|1DkTFTBq{guq{v@I>Mk&9#di|i7PTzWw*kBx zK9IuriRmFviqSYpYC7pc5Se~$cg}>M8nGIw6he_J3-m#N!AmCO?eP5U+s3+4#>^x95Ni%6b9 z0HY!S93m2>d}e0`(?V+=P5~StniYYI;v>e6$RPRkkKNYEjiwKLWl8ky&libp(0^>` zuTDLObfWECy)6`M`L(^EJkZa4kp989U%Rk|*ddRXKdd?1l&;n}RR_S{>enlzK~jMA zndty%lM93d6^bS} zkggJiM5f2DB_h>R5IsB~?Tq6f`ovzS@HvgnI($#Cx$9iI{7u z?692?IJE{#=)5yi!`IgYG3#mk#mw<_zxYU9rqvckz5FA|+LSf)mY-SL9(FeIz(9VB zhJhHmi)kEcDi=Ag7lCwF;guVF;2!tv-~Lo6#U3meF8hnC5l--nd-=$!(b(Vhj{tMB zc@iln9Di(8;Sb#g`uAA1OH77B`0yJ5;{)!P-hqn9=7zrCHvdt-CcfX#+*g&PtaDNS zVU~N?G6jg0EcieWlGi&#h%)VE(vM*I&2ZRQH)_aQq0p?-86J?n$Wyw{_K6 z=Kywx|9|kaOWSYv65+sm^gSZ91Ahrf2v69zT(56q$hCh=){cwg*0^q#YM2vR9|+j) zbJ#iB7u8{rR>T3zJdYEe{?%jt>q3I}&>Gu`ShJhsQ-j|q0rSsTtuzpKK@@~4=RbG{ z`lDv3gE;rssO#j*J-5iK4LU{)*bxc+-*Po($l%|A#J<^!9&`38FZ(Eb)gQ?Yj<8lKer zkSQh8QU!#m@7ROqC%`YpY6P?-g@Q@Z!$exFx%jZ?Za@wIU(gOMC5spy3m`bCG7l24 zoUQ!$((xB&63hbI`@_mogdunnqq3xqLI1FRwXc4U1a%vU3Lqf5bn1U$QLx!Z1lW46 zQOYDzL}Jl7c$7>al%qXY&Su;sKU%CC$+U-sOYG?1g1;FkNn|@-P4_TuZNe{xS91Zs zBBUn_=G|GU!2|UI7G0G+k3Ts_&-^evuI9)5W*#NSBOgGds+zXy&h^B;NBz`MF6~^C z>pw(A+v)srk0o^89$S;vhDQnCe3@N)fzf#OD0y|3_fMzM2n<<{(N&TJT>sd3Ub5`< zNnp3LTs8ay_xww7F+43X`@=EV#PknFbFi9?chlY7AVdPO1imOCh%BOa%EW|G7Y_NN zJznP zV)0G&X&GI>!3ER|u8C~Za`EX1bAZ;*sl zjkMd4-QOOBWPirLWXuq5%eW`dP;6@KTU6qG?R)WA2G@?Qq{1$OUu~9|!%bzX6f7!f z1PvJQOn70#7AMq_FK>~d7Crx6|IpA*vSrf+5Jalh>t`R+d=bSjYz*FpVtE-DeGs3P z5^`l%_Z>k1ojQMbx4hr%{{c`%bbS-q$8GsZcglKX^T2lWTky=}LYbQSZADnIy=h^( zXPEE#7KPkYk+@D2jiQasqJ4R@YTV(7T{E=Fnhn&UA@oxakB+%f{)a9pQZR84AmKQ< zNhPR^+Ht9Xh2SMsL*ACXgr{BP{^knz6{YHPmNlJ0AfryhQ}zBw^u{n7+4!r(l^&Me z&2I;6sKxR8I=wEfid-E!doMr>*@8wSMVt{dCsN>mVRjgS5d7YxiAj@g!8QPq3+w_y zH`|Cpq<6!ZSds=u9z!j~;qEHcQ<}FK+DtZT!xj#8#ikwFt-_LoLe12JHS+&e3*rbCD{#Gj_Hs@s<1Cl~~&A2o`K;4p) z!fFl%n*;^0Rd@`V9FOc+S1HW?&yIHmM?W zUvez7lgk9MHY(^XLw4Ap_ucd&>n?${)5*~KSaQ*J`Lb$ z@vnVJff~lcU{?jhjPQ?S3%Vi(^-{d_YWPEJ9-JY=^Da!nYUB`IkHIevXPSd>1`4+= zgn{umuPzkde(n2p3X-~R%6fGdj-UvBk<}j0gvy05?NC$%pi`{#x}WH(y!>OAAH%+* z6?&k=ieTg44pfHM0V9?-Y$EJA&qe~zhZU70YJ31HHF$VyPPPzn%OkHHZ-5UOcEy!??)$`PF#!hu=0= zw#v9jaB-bhz?Y!FQE6ukqK1)ya{EsOL=Uvw&)9_?mN>KZr8vY`2q`k&M`4=BO``R~ zbcaGyxl#8O+}!Tkhm#0AW)UbFZ4ajNiP87v>|9@4SpM(d@=p$=e&ok7HhC3`#>Lb( z1PMELqOZCveqj`fgov8hAD8Gt389N~cyWp%CG&6cGH5^J>D}b+NmHa07|Uyg6RG=i ze$KX0A8Ygakcg#KpzsUp%h|C`CS}f_F>^iS5*CMX`uT4KXugwu(O;o8oH%ma9UnW0 zzvz~oLc@-$!sfYF@D;%4dXJ~3QO#FrdJgA`N=b=UljB9q*>*Q$g5*%m-Qk)M9?Pl} z{CRu0zXu<^hE?`Qis#^sRjz+{Ti>ceRw5%NzKT^O_JXL?WUy3c@SaYXI=&zXajA73`$Z z^0YH5zB6J;(_76)hN`mrclIX?iPl1C-^I42&qKW2THv1k27@XY)F#XjKSwA=P4`D} zOf!H_S;@bD5X)ysE@SaX{4~1hX@zE}wUzLnX5Pp5fvaLn2saKvYYs|wVLEoPVB- z5w_txf@La6jeFcxd6Y#DA;n~_gQ6XeKn<28arY`WF$EW37@(7dCQG9#*H;+dDm~Q} zfPih^MZvfRz2a-Yr@1%EI@Poyr0alLmKGp&{PQXEp4^bK9*vd16Yz>Bqj( zi{8s45rLm*$gLdm4M3kj${dJi3WA-%x#gcUI=!E^auX~`t&ur3Jfi4Kf<_7mJi!8+ zIixX2+sReEZv|m9XVNrja5B%o1d&Xa8mkwv0KEt+ME#%b=e_kzYz8=N!&^6-+b$C0 ziL zVNC68z68g*ci5s(0bc*j9rCTwrV}nPuR&!JKNu~@HR9N*8ELV5t0yVvCnU&fx2oj; z^I4u1zm1R{)4C8G)hP_^83t|Z+131Qj_RYHzw~|&VmdU%rd7=2bXh+rZ9TpT(=quf)V-Ry8xRofio}Z5!aVi=Y^4qh6tx$Xo16{Zo-SN*I2zHB=9si=tO^w9*#iwmXp8;Ez``t9Y zgQ}X`Re>#3EOxT~rShZ7ju-~LPRy6@j$b~dnj8eM^$%&+GTN=X7|=(#O@4?ymS{bW zCS`c#*`DQdbQL_^Fqn%QOKx5Fo=+_(H@8E}sAP=emy`%`V4dDbqhiuv-EH}CmGj#4 zJ{?tNg|cp8&hgu0^Uw;0G-+WfrzqsYR{~3zmo6SubL@6Ic zt>Qn;t($$=jZb&$w*Fmku$vW+{B#MA5ir9?)itPaspRJ2z5jKjZO3^umT+j4k4y6j zyII-$_qxrZfbH^t4f1>f5@+_e%UdBC&cjyq#ZCe!OEs+`I5r?q2uSTXVx+jX(DRCF?|<=% z%nFQLvQ0@IXF-bV03-s5Nrs%~=BZq9uC)i?vMejts|ndAC4<^UIPz>)JVT6q;Bz+K zuxlPEuBjO6V7SZ|%UjT=e^kC}<`-Ksq;VQjF#y74Ca|04{G+fY>ht|~CcPt-f1cRZ z^oCC<mxaA;^klj~+~>j`_{W5I4n{Zy|!GtMxsM#&QC?zVC&&;KiP! z?N4R`cN0%iU0?Iae3NRy44u$USObh{c8k_uY!LQaF%>SB`UVr1^hynL3G9HbnHPoV z)#MfqY?1jtov(zjceLm9al$oZ!swWVNbM}1RovvvU-kkOrV_dM`_V@%+c|s1W^gi^ z9V~|>tBE+xJxC^a#>uh69UyU#wCr@$q+tIA{`!3OuZ?sANDlXz|N*$21~9 zsy)DWAVX9{;`VVJV?8v}?ddg)}JFH31{PEu|JbERwUsO zugyPHY-b9sYb5|WcPK933k~bi&L*&i66|T78Nfb-Y>7@!yS1-#un=h1wI@j4pT&b; zdk%KODt*=^p1I;&h`h>DqeVK-l03uBU_=w+^bNgff*ES^VLX6r|Pz%-CtUnwddkSCKt4865djgX4%ZbeMvOM1$I8j=XKQUhYf*` z6+oji%h`de_=`b*d83Q+rNLJEKen96c=9WIwFNS&$}pJ^I!FY%kw7TIV|nE70*9tt3#cUfggEZhU=f!WKO8B?`t5$`h(e`2y6q%+)%CXlcZ5OL*O%F6 zxBYU8D!rA~TC%g1mi>Kcym-9lha&+02uF>0WYfPS{fuX-=LB&PImK6GeC55^|%m z-TWc)&(+APOjL~!X=fq}!{)=moWPXGl2U`W3wDAtL6$X3MI3aoq%GBOgVUj?nbEpM zsZXJmS*Cj_i7GnL4dhL7nwNUGAwByQuIS|ZOBAoFWNNzgri6PmyU8&PIpa|9itKz07e|IJkyuFO@gZ_n_Wu?;Yd@bcYSjD9AzHjQta zTB+Q8`}kb0QY*iSuVs?h8_}DM))dNv-1Tl*Zcd)djaH&DXi_ayJ4o9dMp^LTj$*M+MYK=EceqktMtRZ&BZA@U!9twzpjusw*Khk*m9BLCSN5djnBCCj-?&?Ic!LK?)lP{&~7fvkW7cR-yJ)y^XcYbg7F6 zla%&yyvmhu@;?}2c;)mqy2oClTAqeWZ>l+?K6)ti`#4E=QxXbGigGa)HE}WEyO$V^ z93GpwZu0(Za!iquhn5AN4&izFZIk0Qb2r2$I*r38SzwYGUnqNLEA@1zx$DleL})Bu z&l;RUaEW~Y8Xy-SI;SS#InbwT?cAnm+!EMs)IL zb2j^Z0of_FK`(JQ!x*wIt^WbL>%r&K%?aM>^ITS(sJow20}Ck%$?^nV<8FCKdnBH! ziO_JDGwSpZar7=%QXphaL7rmTh^9{jz z)$u7Sr~fznX$(7C-D#%D0jb}2azcaX4ujti7mm(2@aNcCB*-3|y7PTOz3j{4kKik~ zZ)&Z!o|9u+BT5#cuZXV3RG{-F@1 ziFb{;;MD3;yHVzu+A|aqgF@-gpgd{K{yN#-Z8kAbHHWmuDm3Q#iokt=_Fud9#^;NvW|bJ{C2`oiPb0a@fP4u$1VJY{l}{v*D}-4@sn%?A82jaLhE(q+6|Vn>YmEBI78mg<$dZE!{4;ZwwP4PpGG zLG+KIkEYjpGTogT+!=WgwoJbQu^ zct5G4A7@$}Sj(hbfQ-NXH@-PHo4~S@tI#C57D7uuf14-oeS179!;9k3?+)Q5W}gxhnvi?>UXn%qKv`hm9tx2x}Y0w@D=bDKsM@B z^nt*M5dr{oH7U)c3D-`a?h%3r8{%Zz_n7BI?cieq<}`W$pgYkmg@!i{eEe307%j{1 zfygI@8((@KpZ>5;ku@y$%ay@uio70Ro3ug*jq#Oj=|rGZ_#);Z_L1voJP3@Q_p*e< zoMmtoEJs}i`cGUzmuD53ciJXTK?KySsyry@TfJYVHOk4Nr}~i>inr&$3+T5uBtXA6u8r09GyKHnM2M;#|xi zTH$t~Npu1`iHEbmHI!z0rA`U&3t+fQI{BO$3s{f~@mgo-a$EeW87u1A+3MfT$g3}h z`*4=47*j@{a^EGWh~kj1IDwzXp2E`m&NoVa;~&nFROGuc%2goi?RF8p(b{R!wBatC zxfc+U%c}81rRCmye1)&w8vkbPI!VJ&JX)T~7fs0mUpf@bx+A+dxfAryk#~S*0~zEr zA3m$Z4`zP{mLQAPz@SgTmu6nnn zm8TQ2XAx#dXXaVd(IG_@=u1R%@FYP13870yoJn zQ=R6Do*kRr{>k8Qy9cP1HQqu1bA0fPT#c5zxTVemAUwHiG{-W#VSGu`ba2o%=o2-s zO0lecS;LHQB4T$~N}K#qbs>Lzs>mOcgW%2z#o*dg`yNL%*8e5?UunWAUkvk%Mf2iz zl(e)P#E()h=?ETWO`ZRf;s1*~*`jVf?R=9R!!*HD7`}WDOZ_JI%DQBi_#91od}_=Ha1qhtDkJb+R8O+-jXE=f#=e{Zep_Q9nG9|qp*6* zRkculr5lYS))5CZK72>varbK~74akk=}b=*RbCfWV{D z;q*)Q3`ni7nB_{ShHP(`seizGGy2iuOBJat)!Rx}M&ke8EOqnJOnM7mXX>jk>jnx? zg{}!_{jn@-o#=pXRR+R4>x+s~DRGW*OeJ>Wz7>b)(A7*YpiKHrz}J=O#l^>#&BgJ~ z|JtJ1X3(X!1B_2vY~Mtu=BX}Ck4ugHnZ${eh<__Jc4q@ePK5X0PWINUBpL!qkR(Gv z8(pVkth_tBs+hhN0w)T7I*17wZJY#xW{%F*cGgAlGEV<8H?_q^3&2BE#Sr{Jk)Na> zZ&%inmcaO&M!V6}TthHJAAVi#-Mo-$5j8q7 z{klrV6}XRLWU=SzX{1pU=4j)gT?k~=`3x7|j>1(v%A|6g%pxirNARP(KJG%pE4VqF zl>FO41vtOlA%+RMH@uuk7&d$7Nxc;uFmp7W0 zxSmZG$)Yh|b%)>?y>RH#$;X~@GWMO2Jk;$PW8~*3mHHG`6NlOeHO=G&2l32M8)G!a zMal5VaeYxOWZB58*sWdID_|ENw8`92wg`pTwRy_L`$sA~W?EWIyTUZfRF!m(Hz1^`{n2yXGGD zaksGR_f>FhXZZjLC;E{W40IIjxd63l<%9RPh>7k&?CcDY59gRF^Fga94=1yo=3<7v?iA%c$%2Ync1D zR{PSM0cmIW3%Ioj|C+hM01My7{z4y0w_9%Hwd0xf5+&{jku0U7g`^;h^{lRxK83(0 z-)tqzXmfAhR48kly>tt>8Nss<3wzeN>`GsZ{oOv4KUc?syTp!FsBN63=}40!?AG42_dJx1;w_@lEGNyD*wc~?`BqduUtBvy=5zVn1%5ORx|>$E>}4l z*4bs|>(X&`6$asBq~Zcf$R=v2gb&?{;?6zjAS*0bhE-=LdEDgj61Fv7C5g%3%{@}l z7ix|l=f81=L0pDH0s+=Q+sF>znPK~!_m1bV5N{z44i^v}mn%?=4|cao^i^)ti8;~y z<%P~W5)#Og$FBQauPd%LsY)h7#9=>eK2eNi0DUBLWxa=ai9MKLtTdZ0i)~!U-Ea$SRyQ~`M3crFT&W?9Rq28 ztqd?ZuF3Z;PjOf{JNdRBw8B3A#CEe4uf%SI;$7d_n7n#>T^Re_(!g=@l&TH7Dt4lI z{q&ftJ|4n@cy)~LknHpG5&z^t!F_IpKy>TFMDWtLvp)CqR^+Mzj@cP^IUA`Q(_JZ> zU-&0?GInomiYx!7xE^%{4e)#&iZCm7Pl=$IwTq+Q6Dymcm18=XI~W)QzspsSQ1qR$ zu63{qO!w}fIDX0$G8>`lLOc|{*z!kuM^uQyXp^rg@U z_BDqMK=NZCJu|}fPzt|hgNbx7{&;+;_XZa10pP&2Y~Lcc`}2CA`RbTOkQxUxTiQ4} zyvY(%TmqrMrkKe{cWX!D;B*kUV5 z!{!No`w(-Wy=FhBf`}dwTxUHfK7ZeF`y~_%HFFGixj$Eegc$D<2#yqH z`Ze0D8k-Qjpc?dbcm0FQ&XN|q?+z^N9CBRR?5hKb4i03ylV;UjYv?RaTdAejT`&g} z;AAv@U3Z+|N$A9g66K(y__(e078&)83hlay2M0@DETotL+PIYK1kA2S zM+wl@$H%qB_9K7pk)YTA-KO`(g)cUzT7Ok}@3@`CWmmIT)j%auTGg#f6>)_3!vzE9 z%9Q@^jjXu|x`wt=9Se9aIiB*V@?Z8G=9F)-=UYF{a^w%R{Pz*^vc~+yvN-P*^~c0h zvEviGPF2YS4A4ylldx64x?j!Q#7N#&k8~6KdarApdAB9vwn5PPwe9J8Np^9OG%V%KzjXdt~l#$zU%L#MrN0 zP0buP34IbP(Ob#$%YfD;?jb$)FI4G<4F92GzPLsOl+_Uy|J^@-D9(0uK$5_Vwy5;6@xpxqct0cREoMb(e|5-Z=&%1cU=zI@I>P`x zl!8VZm_fy+I&t2(8ANj&B|(32Qcp14MaCP7E(u00Yer3}{aASI=Sj z>qDa_fvyd@wLtDRf~fBrBn%laNsZG7zQoTNFSm7VHSd$HQg4Fw9{{N8&r5-vazrI- zT`1hc$xL9VQkye|OaA}DB$t??SK$*BzW{mXreKxfZ{?z}{QhmZ4~j7uZS~=om&2&9 z(mc+r%{=WY!&mKh3*L~<*%+71%Xt9*v1798_ErYVJx@}@x%y@l(N4BGu>Vqu#LLlK zyY&mom9GqjKMh>29L}lDLglZ-zUi8Hg48&?U$r^?BAwcaJz{~r)U0?BQEzUs1sa@> zak>#+?rHl`34cBe7_bU@5FXKy?J=Wo#QawNp^4h|Q}mEKT(@8_4hQ2CArzcq2iHt+ zIx%CZN?P&PAK+!h9KcJv7oyxE&CTx|PN)p*JwLc*kGHPdIy0?ub}JpnUQzgIhnFLle@g<(W#Ox;&}LWqn}H~aV*b1m(R<$ z0kSvz_lH47H$=pNiBDJB^*(z!4Q|dp!-I*GML_+aYN9pL_`~aOaJm4 z;7(S`7%`b1bqxOYk*D)+@fX&a_-4#*tUE%~vg@T(n%ipbW=u}5b!4A-gx54zX@xG< zn3<&J%I&g+dU9=KEot3YWZfUbq{2-&df__JoNc@l^-a~?FZlG3($miYemo|lQ3IutsT;I#ccGe{d%%9nekCl&m4Kz$V z2ZpGWnuGU5gDoTBixJXE&!|i&9++6h!rYI1);{!2X5 zXYV+6e9!tT%9ai3@9Mj)X?hnS%zx-s#(hqTGywI0e>zjxL}Je1xq8Yq$X4eu(}aLy zfE#6;XE#C5T`t@T$n%*9!FtFE%9_31pcOiBIL=*aL$yN!L4-iVHU9$-uE+wFOf5=I ze3Lxl^pi#i$ds9hj0E#@om|wZMWjRs6k|A-&htkQBg6o7^TV_P;)ZL za!@l&;%!N?g|YI>W?PMdt&h~_GRO2bl}h$tIQhcFo2V9x01Jzk&=Jd~Gr1bO1_H!~ zf^djxvOllvR%n)WQ7k{@TePboB|%%fMg6OOkx3${U*#`q-Dpl2F>J4x$Wuk!Gd;hg z533QHs^{k4w$o4Wd1zE6FPU%(q90lv374h#eTdF^_M{S>vE=VkuK4u=pe|DXnt@6l zOBSQjKEQ49;Am*8y~^Iogrrn5>?GbneDc(L>CnwP|CoI|OTCp+j^>sWWX3ArIA6`( z4uzR`PdrtVXp^|BP+dqG?HiA(+Li1Z8C=3}2KIyUE0yDgCQWjM6hEz95ehzI@p@4z zXTFjXTWbFGG`Hfv_fTJ?*5d4DQ{kT2KX>%2Mah^be7t$Ij%vrXHNT1U^_~N+io>^C zQc+M&+r6Z=Iq7&m!sp1p93GNu(^|}s2?pr0mgwMk+(Eg$a#ks{S!a~+8i2y%!Qn7m zmcvM zjYwylH>E)a^-|qGi0s?JzB1!%1V`R0=O4782TNrE;ffn$Bv~;s)Z*;!E&dmN?S!>1 z19fH^@z5FO6Oig*>ykT!*~R-p!dwl4L`pR3mi~T*tG44@4(M7G45s0wru29gibe`u z<-MKnLe+12dwp8bhracksD3GXdt`tMOT320189E$$kO#ocvfRQ*jO*VN74fj-g9q` zyNO1P+HZ?yWeX7Cj}=4feHQOUH%4g>EP%)3(H^cgkoX|RDqxvCUqISB1u9zxh}~>W z+q(g<+)zAU28g`~@1pYHY9d;ZqM(%DuDYE%yt+dOq7d8hE$e$tfyd%FM^F5{S?utN z`bI7@8AxOD!3&iSBG10yb9QPbmF9SM4&(ac)g8F^yeI_Re#p?S6W|bFCyV}>Wm_WeEfJHHH=wnVyvd=?X}*-ZMU0!2{I0&CG0k#{M1hWh?e*CsYoM2UXLkHtnD~rRB9^z6a!BwJ7 zEY2PVu-6@(x8s@Mu*3JGq_MVtVa^gB#VAXIavuufPY5sfZj{DcTm+dI%(FTLOV&Ly zmzsXD13Sg2FS8mVL17Z_`vo@{*!3Ml?zKULF2^{Gn_S;lB?1P~6f5f}d(wU0D&Hk)+b6{=iRYfNh6na+j10s7+{Et{R1PqJ4V*fPJU4 zxa9mSwMa9%#V|vrV1etuhDo1=x+3CbDnUhpc3eQ9_n>#}S?g2dEgcC%(x+$LO(H=S3`b8u8Q{~r^UrLjWQ*F0 zCCjD}`j%^X_E>>M<4Gd-^N!lvMKd=wxfWB(mubH<3>TfZyLX`|0{r+Y4S!5YN8uaPcL+=u)^BC1$s!=>-Tq@v#h`Hl^SthcK)B0zZ zwi&Idc8upLqy_Tq8Ku$tLZ=SD`Jr8c2gM|r)hsS~x#2ygK&SF%9=*TTM?L}NqHH&i}zT)mi#bX^+CvOVHpQFv-o5;sqwdL#+4I<=Ua)Orw9Tg=jeU4&L z20?n@L)j<6gw3-_ePxC>p54Inx8WEnG!Ty`vUdd+Qi(0 z7KGcgL~?PVufp|ddcGOr0E3BQbw3t)S%7@aRH~gE>Se8%N0=N5Kh|-Y^I% z)A}yYhaoLIN5=|Y6k?w32%+b57UwSq(5IX0!slN1oQ?~?cxc*P5bHC?y=n|)8n3V7 zt;~;I+SxW&&U##OH|TOX*~x*{!E`&sm~4UkUWb<6$ND@&Gm|NZh@E4VBN5Tzs5olr z&9Mx7?W`PmmU+o5oiG=kV!e2n2cf5^^ifR*QQ=rUKi$e{rD8nne!H}G#Bj=7Fjcal zUnA`pd_Z7!W5+-B6}U9$+q5Ij#nI+U_p>IWg{Jj(ziVuv>6Uv=N_3wC=c4q|B(6a7 z0o-tAn+>qo4#xKDg~HaB8B%P!uOFFrFst}M$FUzzz@?R$`P4(??@h_Tj!2gt@_wEk zNYBwbDo8Ch88{oMiqzFO()LW7e9aJJ{DnoOA@I8DVv}<M%!5naG0PonfLdH+dI)^)n56VVabh5!0MNatSsx%70~Hlc_Cwb+1~n3m-7tudSU) zo@FruX{SPYsaMbZ`(a4HrSlrOEn6xFMhrA?)v9n(PBq9O{`74dqfF!k??m%USDN9F zJPCd#qcqaZpb|TNOpTos?^Gm)LteNDe66oJU#00u~>0YfPF(P!GOkJ6cfP6L1?wFCGqQx1O|aaQAlS zWeUY%XM^JUQCH#igAicAYS=xF%QhlQ5a4hMP9<&t^tfLi`YQyUWv6`ysj^fy*zG;K z-BX|_aMR9L?8AI^!M7bNQzH_t_x^L|6S}~Ev4WyQo7T|VoGrK! z$v?>~I$)Qh{Kc|nmA6y54A-OCb_%|KoLr$;F!EgHn$V4$lO(+Ll2}&IWxbFJ&=M&` zGqHkWi^wdSeLU^xSR+CCZzzdUs%tI2i(x`sTIDES?QD`+1LQ|zW>s1`+m7ZX@uRZCtXBs^*pV0V=pFVG z-@{vs>z=px*|rF+W>}>21XV)D*#|-P43Fl4Lwxbgw3nvD{9WEQ`cYa7M38?4&Kl#+ z^@U#@GB29SZ#HTLf6x!q8sr>GbHnZqenjxx!2|t(KXWz#q7M!2Qy5xVIwr0?9As!0 zn|PkQ7fsYwY69PS<>qzOVznw zcNSShv(GL>rt|y`@l5*=EjC*m-n{!cib=mnnVr=f>yT>Rd(iPo5v=BdCGr=_1Adw` zX6r9`Fj^RR_(Q4~KVBUG9*}L^<_WQo+GXHmY(%F&=cGa;Q_1>T7^E}UOMN~KCkw6K48Yavkh`Vw8u(yLi%HKQB<-yj0tJHl&Y>N|gz$!2h~&vN zZ@->MTUjyA3Mn4fk-zpZcuurJGJQ)D3@!Q)W$Kefv=rOL_Ioh0>7p0^q|EE3Xa!<= z9j)3(J-23Yp+;(AU~K8)eU7-^$UPTw=6ikaR7BeUb${(ZWixe9Y<@SK3pE>bbXl4m zpPj!D?__iN3?~2ttW*xdu$b~&EanR3B`a(75!LT~g1B2otBtf+o?Z`#r*tRjxe#pP zcI5_qoNGDGMGTWHBybiS)e}87&!Nq`YVaKXJqh+^~+#iaL^Fe zc?=EsSV`St#}G}La|d(nF&Od&WdQt&YX&Z!KZY3xdA;25T7CtSCv`>=foV3FU~Jwa zyuAJIEl*S(QYrIWBA4yR(>F-(b#*E~WPVIoAyi>=3Z)%Nss<1bv@2_Sb=}Hf-2^&Y zXQ1KPR2hv)-~Zt9)@dZBI)k@1NT?i(eh<}Xf0PB=jATt~6T0I9ur=YV^o*e0KjCi? zV7fr^CQI*Tf`K)Nbkx!*#rpUU;NB0_CIhb-LO;24TtzHOvF(Ocsk(@r(=E*6e&gQPYtP_u>8blY@B4y{`o3PscU6bm|-u zyQgaUBFX?f53Y5fw=su#rmetIp2A#p@F?dh#o@<X(kV1Mqp_OY;ObC?(O4*@(u49jGsVZL%AedYl>e*>_~}$rjzurv99T#95K_Kz z;}1>#6VYLMqHxJ;iUYAw6T8RWi~v`6rmtIj^7>;G|9(Zo1+6ueYyJr{L+9A=!lM!n zOZ!c7ZzmP@^7K(V7s8y0sg`G(Pr~!djQU0)Mb3I z%(#APCFP`TSz8eL$fClx!-#4Px$$F{hRKzgSraq1MOJiK`iGtLOH4)afp!ewaAN?hM|u<&dUF(+1z z|Atg>Y?9+vd_;@U+YEP923HyPLXGP7T|ltYVZ?Ee^?v*UWHu?oeNsJT{`PX`q~2sK z{(PO!??7?`VfHuya3XwIKuN%MxRMILd%`p40OAnG9xP=DER0}lDT)&f z-uiP>I>O_x^$8Oeg7u!lW*l#;=p<8z^W@%3XfL5;%V>fBbb zKF@mQIeOvK9)dt+!)TxATitpwhGDUKYMpT^V|l^&$Zry9r=qeXoK(DWi@IN;{a_cX zzR#4pT!nqu8Om?ygl=9yAcn$Uss}s@a&(4=DVH$fm~iod^Kv(@NIluraj&OfnIMtT zR95S`zd5Nmi*aczi1DJ7p<-U+Pbzjija|pbgVpz3`ZU zh%{?*t;b8}48O@M70b6nixJ#nAoEt@v7Y-4x%$gb`$Gm(geQ+v%=MGQg?KFQ7_VT&A^Y$;gT%1B60PSo*}$N%0>N_(?wN#3;eGyU?fTifCzNE`Q6TZ zv>3;{In2(Y*sMAr-F?7tg#BrOoPB;OoX=ccGFAI*@oI$4A)A%OU)^Ew5S?&5Z>HV5 z-aLS+dnBe&H|QXay0?I3|63k2A_GbxlItD{Z*z z=Eh-Csa!KZf{NMjA?O~Oq~Y-o`-A-S+mrqn9E+`Ybz~ij0tMJ;^=)5--g!Ev@HU4W~8| zp2NcJ!Q>C}hN1T(*Pd!!O>;1aCJzLA#ASdObn!e$qh?(?$;m)}{#qX}*NIAv`iyOd zd@S1j5xNQEJe|!ViI{9a8h#mxp;^1iaHR5JD*wql?pUSWkU?i7;QYe+<$|&mnkEa0 zZ&B;zu}D2spV%rY>Wt1akP|7r>|(U-YI8=iuzt_&#IC{M(ljz42C2~`J*@k8+GYg3 zT)@wb%|Frz0|rDGW7BJqv*P&1ue`Q;g%J-Y8Pu|^3O6QhUfg)w)WCb@7?9KNebl7+^KMTIJ~wb+tAdAajnw&+lQ?~$f*I3nlv8fVgB zKOD{dTQ~N3ZRqf1>G0}%AYXF=Tiqw%&OJ)zLD13#@_E40!E4?;_$};7@ZWX=tK@O62P_SAp*UV^7HZ zH$`h8-ql75dYMN-2At4G?6qvtD|Tlhk^l+&_ER%*VS?|<=ewL?s9GQc3SPZloshM9 z2Ia5!xWTOR%{{N|JcVjF2c372#rtS3Ev*VKWRWsjR$k#Y2XbYB3cZ{DY1II#<}5kh$gahAKPu)d1^=Tlc}9H3K%A&eJ+xa_ zHV&!EfWShn% zqwfzc?1!LXi(Y(3VMxMl<&#P?(W5HpP6L^Myq`&WCHJfThQfL z=|Ev}$yT~>h-Ye1am>s~BJFERjuwhPs&ypm?lc%=Z+fJBO}>`H^{^J11$Yro?O`PY zI3ShQT8CdpV&NhE=-vF;)W!_t0pvjG0w~tUv7wt^PMzA!aJ1Yj%&uTnfa^9Z4s)Lc zXz&^&;%KR-TldgO7)+pwBj7HwX~Fcy*I9x7N`!_u&fzawl0pQK#Xv1&NzHK{9Ow@j zAFS4RFp>UH0iXzI18}NcWcXj~|Fel`Uw;yT)M3RF95ZNr9byQr{VRck8PvmUht~I7 zlggLAzaSm-y5=3a9@qRX*Y0PT_Z0%^FW`MBG!9sWYso>C0mIvR@(jRTwi#ppIrdtrYDarAX)UFl$TjIPR{EnAZf2M(D(0$< z69!fq_-Jr>^t&xa*vf6UApUs%jdMgZ4r+Ku*`v#G%?!t+TXZ7VEQ&(FIFW`&TrkP1 zrtQ`T+PGBor!w7e2NfJo7;#w43c|WnBa0r1P4?mt(#{v7xl)FhFKxxmn13B_TN4h` zHMc5N0#~=qom9AfqMV~3eSVVZ?pQ=jjdR-Yk;{MC1bNL7#jR-dceF4hdlnw2ner(A zGb>9Wb#MN=XSyZA0~(5Bxbgf~r&J zI&CGMTGDoL#5hPgq7gWXL3yN^kzB1ct5r488;DXAdQ>T;$-iK_0tJJM~O zLIBj1UZ!f&r)g)N8HU zV85+IuX!5$BYH40Kt%wi6hLI01i|XHW>vsKGI8Kg+IAQgyZ6(Kqdj;0=CjY<-Lx0fs9X`xMSdm-OJy*%GX_f^bOnS;axxr zaHDY_)F10{JUkolghIZwLo{+sEF)0KZByu8sKpvg4QT`ifb=+`f*S>^swI3GrdcOR zB)i1|#?I^zw=p40DaO%B{$4BE$M(HWn2b+jJEeap3AT7jLk>(gD@lQO5W&sDC}gA7 z1hK`WqT05THKEq7II8be#h|YCbYk&4lY7q3^_U|&=uG_80mr?=AgUc!U)<8w>hOqq+#1kyOt(~`SGW4g&~&lOujb3*(8pTNnA7!-)d-T(D~$c< z*2zSy@FtC2H;Mj@LOdv}f#qt%C)l<-C1x&rQ$_Z|R<8E~xm67O*j1Kfv{PVuk*wdjQK@i; z&T-1URF-VR{etmz|LiRID6ePS^Sh)0jg);%+9Oo%&4u9;3>f##Okb}+9iV>a^OGjR zw2m%~bR9wOt&+X2euTora<<_gs64saI;UQq^y*_b9F?h)b?KXHZrkU?*?o3fHg~QP zPtx!LKp_IuZy0u!4Ug@ENNT(3j;GIO{_;?pR|;CuQu}cRl?XG11H2iQ#@CLAF_(DW zQQ(dckhyT+LZ=ZE(qwM5=TWod{S5#Ck&`OE;3Q-6A0{;*2(7R)2WreXPKTHsf)C*Y zkN^k)pCOVnCb!E0N{j4JHrsJ*0Uu}8@@Sr~#HBexcz{-LRgPbB>w-NuWMpm*q2f6{ z?8GUBfG4=|SxkTvG6dM+@@CI_znN5i}1s40)Z6U-o&59vc`7g0`*rHi7D z`$0!?OjDgW6VqRY?AgwlcE$w9KmGSKe#R6HU$ZUfbL*vRK7=f~9AA6Bp8-OF8mjl3 zpI>`7F`7>xw58~l*=Df&>X((lntKAd8qGIvxP_x(xr3G!ycJ3gl zvs85de4dxCZ$xn~?eYZ99Jjwf(8#dyUM63Lf{s!;F1#fJgYA7{r+df{$7zTD;^#`u z&rF`GH51ZFXzZ6BO={+l2kR4wv)JnVp$qcGpi+{L3L7x(i1Qy!N?E#@F4G5oVbydM z{J3BG3r>DK9$-V+U#ohRb5os_VNXu&B?u@7kCgGF+2v{%YpNGDR2z)qxnP@fMLg`$ zDD4}V-uK^3X(vO#_^Q)Lw<5?*Koz@xofq5=!N?>n!Stby-~PdnE`QGj;Eb+Q8ts z%|nXE)Z2m97-Ky+MLj|WPQCt9pz?mMX0>b-?m!y%*I3cIMUwASwU3B+Ue-OZTZ5AL zu<^2Nvm`DiY>BZCoN&&4Do1fI5m&i%mnWMe?c#j>#?V7QGp8PM&S=PEP`bcU{P|}z zJ!P|A98-rhW0HKvz(*P4)$OgOD7)N``#Y4zlf_v=0U`U2HPr{xzfC0R)%A&=&pjVs z>gwA*92zi4xH%=xu*=9A&qocj6mRDo7U3(t(SINZ`QFmA_ zz9HZkWPY6Z3$sCSdByyz5Ni-mhg8MJJ4ai2E?r2Q7WP|9gL@RHihM?`<}lKO3Y8-zEHg}3zY!S!LP7QGt^(X5)?RPxR*r!I z7{Gd4nERb-CxTj6rg%ok+sJpjXb>PU%&Q)2L!j-?ubwm!)I~wQ z6dw*@JMsVd>(0LFtOpm%e_IjGX)NdtVG}R}Py#QbU`d8l99|~c+H8%0SFfFWkOyk9 zB&N#h790SZ2q%NK8E!<=Ga>!~0y8tbTeb_oB1ch&cv!kDAgH1{3X+c-ObT7PSV+qC z>ybHPFPnP|SNz%H-qI$&mnxOa4&nmKElN9)#jykL+QBF?HWWRhQ7jx`H^daQVsA&_mY3dP;_|KRk5^J+0 z-<|1@^bLPD|0|W_&g{1OHH~a^3G+ad#7OT!#FXVn?AIM2o{mN5nb&L_hIC+!!J(+Y zoeE3%#i{ETB+p66Ca;u^cJ8f*a5z7@Z12~Q`y*Nud1D+^C*ma)YcLniq6UUi`r~;= zHzo0Ag_J=Kzx{Ts!@;G|+Rm6#KwOB~Ka!k(iHqZl7Ll@~vL!_`c=qRNn`i|NYUa{a zkPBHRrjG&wSj~Xhz^7*sow3l@qo5cPIAEdp@ex>BkG8C9yS@>vjHrSz9SRs=o!qR} z3eTiDaB0~8`Er3kKF-)WFsoum__U!|H$S4hrrUI?VFgD}9RO*fo|rH8Dn#FW@NQhG zF<*)a1d=b&{ZNNRl;`0Tmse}8c>UA+nFN-ErXAYRd7=^KF9d(t6mNur3bhO>*Y+Ms z<6W}2QGyr+>csT$(BiIFAs@u0Fz(ZRxEvjcJIGDF9(owu_l{9mvFRi9fR=dyg){Yi9q7V zFTjBxgrJiw-m9-Xn3@7^Zt9_Cr?W`QC6?w?hqev_2?dSt($R%N;c5c*0#QAoazo5x zRQc^->T;hz3K)_G~Y(&sSL>Dt1}r*h}fa$B~ zHWNbo+8PxFXi9}G^onUK1+TyAo6KW55}J!0aUnGZbV{Uj=XeX>qRIDu+SQYJ5Bv6- zP@k$j?JpHEKn5x%DXBt zh-6VPXe%?3kETZGXzWk>Gy74Gdq44J`op z2@to25rBE#Stn+h0)V?>CHPl!uewpAkpRTtp7gipy_fbcD3;LmE09SLC>tkK1LC0~ zGJ=O!$8m z0D@YMi+T)a>h7Q`@y%XRoeiEASh#*rK+HQB!C%xN1bntcc(ItEx^xodMDQU}7%lqLB$T_bDRux{w6P1_vdBb4T9xk8jBhsJ7Z7@o_vU$_t$l z=Ylh_PL%mbe(g9d!|LJdiKkgqN{%2&3Bt7lV}joylxzyUh)fuY*d0&YaCuDR;M8JP z(PU9Ux_PB!30FCd+>Ln8pjA<}fS5W)&r&j>xF35G!^}*<2_tE*L=7m_NpK=1HI$G2 zYZieG>)Wh0nw4GpCh;T3s8i;rQpP>ou}ES53wf1gOj>6GkYrlpNw%|Kn8X%>d#n$UwhKAJ2Y5@Ml<4{|hw>sqk^0|&jEYed1zE!yn z8R&MAzC0__Bn5Pvyf)9dmtw)taw)!Gy|F;tj=vG4Jb}Y*v-trbF4S=x4)qpe>%upa z4JChs9v$?VP)mgG+Lravm;!}=1&hf0_LJz-5wTx?(RRcS~-oJCh`)XGwy61P^CKE#yFZ4u~Wmm$513e7(|I-6A6% zV`$wlLs?=d;lM%sy7kX0k%iR3S*GZ>ZIXVN&h4?XMLwvu?z{9u2=CKA^le@8SFC4OLG5f~@_g@6S<$y@)vG^Z)3VX2`hkVN zVE)Zvk5=#jMMU*0B{?_t;N1Y9mPvD9658E9)LS1|iQX&P!E6g}A-8YeZ-w6}>`nxc zmX*{l4)Q$=$Sl4-ot=Rh3?BML_;k8gc-M9UoQV_CjG?K+pS?8G85A`Xzo`bJ4>cY$2+fwP%(um^~zZPaY zjjDj)hv4SGXrEtQ9A+j~4!+vlXDmIfmpfyN9(S$!$}c8)bgK2DNG*Z94}axe7mr`` zRy(nivzS@fq+_jG#*m^(=s+lQnb|Hv+gdL z@TqQJJLLj;Z(%q5%J*e~@?7|4QKN>&AS&}$oPbC9P?Yo}+HK9wo4<%L}lrDI* zMJnzm818p0yQv}|Zk=C?9pnTb$GnlrCkkhl@@(eGPim3qIg|mL9Cb$hN={09otk6~ z4kiL~_7v)g&b`8r9f3Q|TgyXUTSLh&ip|UeSOX(e)oTm zK^i?xu1rzNS}rrMt}n>wt*0kA>%tLMXyEq{#8MV{@XNGgr5Io2i_tuBlWDvm<2t?I zzL#a&N*xZwR?T65^PVdV^6>70Q8`YCTjTiG*${Vr+Q)D)+cD*%Ms- zVqHPOHL2E7dr{6Qg?s(!RBh3eP|6r}UzI1|jJcWY9%Jj<=|ED=W)ITtl8;MOv1bhe z*0m!-V!i$HxrCOBjHhw;dYqVDouOC}oqEYioV~-m^YQ-eMq?9Ow%gJW1J4(?>hLAHx>XBQzQ*p-Hpf7aTbEXj_cKoNYo3kc`qOFU z%4%(R%SGtYyx~+`gtqT&>~_>Et}w9y-l#G?An#tB+#5+2${N!e#~ZbsmFE5v4u7XV zBmRLwdN#@i@tARwju*~S6S$T@_K!cotMO$P(;cLq^|(m}MRn`7vEXin!gDV0bW`SK z;V$zE`LY0+o9oQ)!5r^A`7FTx@fehD1=IODEZ8!f*~2D#y$3piB)v1kx)QqMvVp;o z%>ixgcL@=lYeXK`a&QiiZ1V6X+5LO6;EMUoxsThO2|*TzuQ$`g_?us?{P~jUQ0V7? zF^^Nbv{_BJ69eM=b~y8+lDgUv_O<09KTd){Va_S{It>pchc;s~<1>B)DnbjpW7V1| zjBuVztC20ot2L{tCCi4@MVxAG>n7QsOWJf*CG2I<59ubi({^PG*)sdGS_4>8%maTu z`LC?iMiM?;yqI5h|4M#aESjw}4-AbT2sMC;NqYJoI@dBEM{imoj6{ZJ%(g-y8|P)= z5XJ2eIRARZ0YA^tflV8m$R|lQEgi#{sMobHn`PEik5mONy5v~KaNHC` zWIkakes%w%_UOY2W+VK$#9F$9O`n*ryvUm^j&1Ke_2cU48}<5C7wrff0M<1*?=p)a zBu`?@YJgom4H%m%Yc+ZOTH3D~SSqa!mF(w&NmxzeXxn*5MNX#kczn{MKe*B@y>03} z^m9YlJx8Urqxx6l_n2_hwQR|bJMkf+3%5;Xa??VyODl0&+qhmaUNM*a{qho!{lQar zRdCNYDfft-7D+XM6HHVCM_iONZwKSj=R%tUKGvMq-%^G}DhZI6Mj>RB=pimxhX9mJ zsQ#02)7b}pzCIMl=hRR8=U-r0{AsKxyx^&VXjNmT;cZQ{^rb~rH(lM%$G7TaW`iKe z4b8RpU_aTi(d2Qae8uT+g9V$%cbwWMiE8Qf56^}kv4OsRrLB*LuUQ{YTL7zVYwlGZ zn}1~*GKiwVEGJ^PCWLq3B-#1_*U3iL?hmCaQ*yXy13wFPwJKMOL$EqE5{3-p8QreRliDDdnp zBEWJCeD%^(WM==QDDZpQ4J>;@@yoj8$BPS`u8@vOxmV+cJ$f~hZ&JW((&`?a97{%H zCl+I$&&n6BxQ|yw(rVm@TqiA?jx61tBbB&cxcRHTA9@*@NvTdbt{BKHxx|AkAlKjc z)P*|nWL2NvSWN_jm}b9Qd?+XQp)iHDf0BL`&}{wwC-}~nj-(y_01{h}`2&~T5KNq8 z2ChEl*&wXe<7Qa|G-pfp5xEt&KqOp_5qFnE40U6kw^h<-@1;zfBY+B2O5Sk{K8}Dr z=SGUt_84j(>t=(4=rJI;UIZ$Q#LdV6VZr69Wz*7q%NomVU5SKvs89KtIp>6wz= zN(>}cNS9^c&4>XZFo?X{2Y#{}{#78!z5r3)NPF<;?xi)| zVYCraRDUwNGTc2)d>Z9!I&-*HbvrqOoGYQ>p$jlLT9$XKS3=aI#suTK7eg{0m8tfd z80Hye#uAOewM&Ht%rv>e8ia#M1c}P@XE6{&oN4763dgxK{xMO-j0ERT6|7IV+|ub1 z87S*vm--9&;|W8L2Wbr?Z9cudX|Jre>q6*kLCopD$HiG~6HyJmc|E|oEmmoo_|hP% zB{<7+hstaJ3b1oY@j2`&+qS*b5q?Ot&ZQBDcCN1HGII9YmfOrtabdNs>LqWy_1IDIFvVez!cDE#c!7p@9P9lq@vZOE?I0BD1fO z(50$OSbz{g67jQh>At%j zWOu?^`I1W~JO0-dKoA4e-l@L)s&OOJ5a+XrY+zP_v9hN~X!jzLEj!YSzI~?PVok>z z{k79HZE46+DwQ)Y94# z^*-~W)|HZ_-jmFc@I$xNVll6dw3`s2=eu>%zQvij6`eMZ^10I1*E+`wVdeMt3cWSX zy*9w=Y&>DM|k1l>ECzgl=T zF}$F_#+a-EQkz@Ou#y>`Ga9q}&fhL7?;h;8#kek!GoZk}#9`$eCPV<3KCBEJI&RPR zQ72Nw{3gskOw7YG zxoW>nXM9X1xgg~x~j z>haa{Q+@$(XQSw3$xgFvt1o9{by+oTgVg!+x$&B&@3rz12*jlMeLjHlsEZEew*8F4 zwNFCI=Z@CSZXQNi8GhUmT(;p8VAn=^UV;04lh=;|_olBK3MEOT86}agvt!Harbvr3 z;|b{`ZtE4IUa_0POE~UuxN}(22_Pu;RS3W8>E^v%?Spih15aA7nY>F-=SfldNFJKC zlTX|FYLR8;z1_-~H`*$z%x)*Aq|1#6ymGLunk>qAA1!(=H7zFc_>yQO&7(&aybGC; zb|@-C^Tr5i%y-7mmIjeUOsuCl-~>7uwl(2o1zo=wtSS&RL9)gUsU@Sq_7TJ#wxc7qF3-F4+ z+JruJ%!&m?juKnEto3$L<94UHbh-^Q7$*PNF0ikr`^VR}`rRNa{y4b|lJsl!# zoe|?QIs9S9^9gJg)g8UDrq|afxdN7dDL-J#myY+q=l$W8X#Ifi>Nc@ACgct`^%V|Z z*F4jzmY@ssrQI|2;%kpNw>}USZd{ho>4}OffQdE>{zubUctzc{U3i8R5RfkE5^3pf z0cjDC25D)8p<$$92$AkCk#1&yp*y9!ySw@3dDr^>g;~Ej=RWt|``U(a>FMcqT48^g zx-JKBO(&8WPiOqcG~0ZhW9xnGHM6xaxoUUH%ynGCw123qTipdigpOmu(5_QIgMvz! zLKi?rEZmqQqEDtg<%D)$+BsI9?B%uEXw~CrJd=Ce=MyOi8V>=oa9F-f=-10|>Pw|9 ztMJM*XCQPoSq97cpNxBxZaSkc<@MzTPR-A{-nq`0tAz?jsQi49KBCBkvku|C8)ME8 zSn>%BjP^dNOR}|~beq(0@FzZE3u0y3!y(M~=>>k&gVU)K%_q$G9A~8)-w#!SmLvIe z0Xd+dY3c7?DH7Ws$C?j$S;LRV6cRA?syAlyn=Z@O*D?s~R#436iU z5a#01BSi8h=`KI*(Mbe=x(k00MDzn<4_g7wuDlN|lxig#9~^A$y9c)FJ<}pTL`wWM z6n0-YrN5cS$Dl#}{ZEHO--1E_tCl@-7Awm;(#`c|fgna`Vw(M=N{hAG{wRWq;_HBO zbuKqJ?yX}6^!ta1hlFNSHE00>!={7>W__+RXhlgN|9}mkk|Z6abtzi3n6#CriC$hQPc^GKrH)eDju#+U0&JM@&x73IU z=G*kRePj8G$^5x2Ri-=$y<3+dk`>`w%P2E$TfQUQ=v6j2y?jgvjF7l0QA>y?IXBbj ziuu4o&13!F8_57P(i`K!)ACzFlE3iZ*S~f>LP{)Wp9)r&v-Yr>IoZYe;`BvSSqZ@i zb0J->WG*1pdf?2oOSmK}M7JOll?=6F=W*jR%B(F9PCJA;GS7gbE@cu)T(iBbX?D0f z!fP`{m#0L@hA74RnNJDgbw`3S?>Z>1Q?II_-Qs#wsm%gkNQDSYe{X{D_{n@SN1xE; zl?qi+gMRm1PrB0SmiA#_`KCIVu+zw?ME??D_2`sxu4Gg5oZ=K?yb<=MyNMOC-lN(4Hr>Ybl0 zp8J>i{Mx&l)yVWl!YK>hlXZ`h(dl)I*dbY57jXLaH(XP>^wEZh$-@=vk?5_fnvKU~ zNvJlD1}7>$J{@@WGBg>brv#zCWGT+;*;M;f>%Hy3%fw3t^1Y$E11=-=nKT301!~d8 z_E8DfD*3@iyv2CUCsG~WE#?i46t=a)OZd0 z@?T>MmQ?&e?n2t~CDTr~@@mRh=rwaTE9@sF3h;FMfvJl|kMO zHb(vf?;W?aRN|g*0w0gP_uyH@+IR=%a1G1^vy|#ofV}J|sp_~_!fS>}EJd2-e&jT> zPM6hH`OQsC(V9M0Z}gL{{QH;V3WXE~A~_Y>l??Hek?q!71d_X>@iEzI2;eaMc=Lx$ z+6BK%B%hEynl356pDFzEvwtyXoYU_^gH$@lCx9nbavq&62CV@k^#&NPA)Z=a;Hl$j zB-{Q)*7I)5S=4_t;c*Koimg%~@0Cu?7>*sLCiNzXVM zp5x556QfJ4iL}+jYc{emUGfR0QNg3RKG8Zq3dqM(Lb%c`&c!WD%1_%K+M3_Qy&fF_ z)5-=W=-ks?L}ZD=-6IW_=a!l6jl@SbOe!_b#X89ac}MmQjV$Walm?4#E2g$OkiEQw zk6))53sHXKmZa+bWZ`ag(GfT4`kFB+Fo9wgzH>OmXGPE@bYOVnSI_$WZiv8{?u0c~ zM!7<3C$*O2Z@i+3#az4%+It4#L8aKP)n879`Sx7mnMJ-V+7Ff-?`ZaF36THbzb~4M z1+IRIc89A`jT2mcJ$ri7_6UI&3CTN0HT&0p8r~9q_)2tyf%MDmnkryudTJmBMz4G` zMoX3Fr$Ur9%)CD@+mX0i?iZ9op-P)cn#5Eg{JH4Z`^_dV?V+}O>SSni)P)}qrd;fu z#96~Q(Ls4)>_DkxK4E&143@9ca28@ibn)QCdWM)CUH((MQQYJ7zf;O8=F#0+@8^dr!Xf=q8`GtGV*#YfV{KR=A%a%16Jhtxu#N zF63WpCUn*@OG8{s;1+JlMTwUS33vAI6V-0V|B0jvQw-t&?mJ+)Q@6!n)cL~9|0cw$ zcVJ8AY?{?~`nI~8KPzBoQI40WIWo9nlgvt%eIOX5k2uJZB8 zO0y_f^#JyE*f4f&+54o9`sKE_b%_+)kXR1b(kVraQ-D0Sb<@}@R$g0!oQ>?{zR?q< z+B=*k@bETUpS3RN>Z<%~DMVA)qpZRf;uhmjdzNe0g5|V#l3DH0g4J_unxd))l{TyB zd2ThLT+K&a9^tSdblM_=&@}=ehm3n4Bz8B2Z85JhW?~ zdGqCZa5e3Q3x|73wU{1bxrmw4N+^zK#jiSsQJDLuS^cRkL9^UCr6YH|M>|LIKPjnJ z%clF)t9rAjDHr|i5-KDQ@Y`9{o~uQrJmPUBMhPrcp2d^*TZ#W+P>q|M%J2yEXFewI zozb%sUDyl0qE5Ya3HYvu8vx7v=3~JzpmY)Tw^~M--_I|1kVnEAO-4oAsYr8^>|+=9 zi>Rb(yC@~; zPUtZwpplP#^0ckwufkNHDPgNDWv6!{-zq0Kjox5@P?ONNa}@c|OEx7K*^#1|3r`N< zs`|MyHBRD_O0Rk?e)KJSn?U??vu;@j@`okl$Lt0nK9+97+H~Go{k7UpJQ{yi3hz8; zv?BX5A19ZV?tESJuLaiEb>B-Ir1Xrq|NOjIy*NPq$6S+&)_4CSkuDQiBD-!*N}8n1 zRY5d|4e)l2eu7uqDe1lXZbI1)n|ngHwr@_7CEoTnromC1hDDUWpmW>eHkgW@mUhZ< zQY!PzweWwOW8f)orakKj(}h??*0>9VS##7LjV_xu{RuJ0)%b4nWI2n2kK}8?YNBVd zGsnRK=6ARqWMCAW9K0yzAcI=$4mqD&l|}ilJTnTN`N9Vs5(hJfVm2jeo4WRP;rI9g z)Ku4lNShYxIPAGgU`Ff9ZwF5qCq6|jhZdYa?8V?leq)!2L!sfIqHX!70UX4;Ub%oJ zDCrq5^;GTCJa=8UQO7b?2=CAYJBss~))ds3t^dnmb_3vi7rfN27}%DXxl^kTLp4~N=6T1jpA z=G=l+M5nC?^b6ukom(_%9CUK@@AeE8@Kb)jj@D+pz^vDE2S~CmCvOIsCX70M@%@=N ziR$5tBiO9P<> ztjqEh)i`M)BwpF`&HiH(oqtTT?IT9O7n`zw3eS{sS9r<>jVW078Os?je}-Pj?R9R% zbqXrelk!V#vfiH*v85-w&9Aj-bq{LuwI=>FWlq5d;_1CQJ$lTfBoFCd8 za&=@Wqjf~%(&T(|&& z^bEpOcb*(7Erb(TX)%k#xqt3(Y4i`Wf|Jg0TuyT0WiC%5PH67s^nOlk@>~^BmsH%& zZkoF8-e;r=A0Kjr;v63iFP+Wr(AWz1jmn3m?_hvR%W!2YG)eff~scfULae zd`ln@G|`BU`vQ;wkR{Ll^RosvMjH-GJa0(UbNZb@K!_OP9WDUy&?^WhsJDC>r+(5P zTblP-?Em-tuMnPF09bj~#@2{c8P0x;&^bFny_D{PqwwuJMC{SjV2$WhO;Y_9MU);O zb9{&T5_B?Dgw^_yES_Qu_~lz75Kdf!B3SFWa#ikrM@KZ@eJY+QG)oXiaQF#u`_zu> zN27m=>+O+f<;C>lk-3LPQh$GhN0j1TTC=CdnnZ7s?eAIVVg~}Iaa5N>*}l`%`-mdR zLEdc8&%7(^y`IDlsfzhPAIZBZH>Vuc%y8hc)Y*-$No)`^tUq*YZBhwQ4&#S%! zAIk*AKF*0R=a`E&4qeXpQI1C^yJ}~Ch&sv{p{g^f8XfkFqXX9O%#k-WRFIiS5VqGp zRy=&H-HH*ZzTyS1-DlI@JBle0qvpNd7U`S(sPp>53SexmwmI?zZOPPztBnY^Rj93C zmoE%A=&$gKMO~@{tb%Muv??X+NDzvGIuAIZ^BxW#N>L6m{HCyTj@7+MOeSP$n8Ckf zsnT(j+P>L1$pp#6-)1&f8a`}=I#i9zQ`O5;EwF)gQc3Ywk^7Y2zh4t)HQD-Z%*jSk zCpzjwHkXX$omJugEM)#TCK?0kax=BnFQ~1?_>^#Z`sa<8K1o21N)eNdZ9Y-mUJkmQ zxkt^8ILnJHaB-#>?GMr;SZ|(t-L_@FHwp>7y-e?#5#3@$^?hm2TbcX zsUMvi$>+6|2Y@u=(oh6@(dS#tN6oG_1j-6w?PceE%e+->Na9_CSb$z26U@fq6YRZN zRls%BKYYYi;b7XGBzOae!vfE9n-)>Nni}IeIhCoj_%IsNXaIBsxHKYdA0wXq<^7>e zXuS7hL4m4Wy}*1xcKUx!XRpzR-!&Y~6IyGUIDZD8#5=|xWg~R)hNp$Lr!;E8P97}N z3g08!BX&nm$OVBOU14A}|a2(a=BUBbSbPo!yNM{#LQ>Qwkv0xAM1xSs+cB_Td)~21krbz-FBYqPjs{y*V&p6aOVi(KS zoX?10B>kvI=;Ti)zp9$bVtYf%)*5bmpjpHDnv?YkcaoP*+q-B6l&BBUQN#>tpt)q9 zj|_afW2#GxsfjlJ{R61Z6~Y#VeTKSi+9&h}pPMmbJ}W#}dTVB9qS?GZ0DV$Dl$DaB z!uHc}!2BA3 zE$$0b$C*q`I0maE%5_unZf@}PUn{Y}%!~A$+b-eHjyU>`5&fm0hyT(BkH-jPOZ7s5 z@35`WM%>BR$-&h*_^hY%ws%HnE{asN$sUnIL^Q9-WbTH8?fc+VN4KgqboKZD`XC@(}4IPg9q#fh{54 zxek*>=fbZgY3;#5FKiwKdf9u+jfRckbfj_YKigoydwP{aGJnDn7Kr}f$tP)dD)LEdMT*rjspQ`RrA{{J@9*|33j-bb zV=Rx&#?;n7^iOeqJpJz5kZR0>dc@!yz$>Eo1GHqBAMnGm!S#K;sNJZ4`@f@KfuV=< z2ab63Nk;-2h3|l{ymcAnT5d$Z@lpQ|ao!>)XTIY}Yl7qVR5Bt zkCOxo_I`1VmujK=jU-)>cxzfGDygBMgQzUCzt`yb0e!o;cK$zGTJZtncUll(h!OW}EQN`#MS$S~%znxR* zYQ~(3tcTG*f-?|pnq{iUwP`@#@h{5=h#=)7jrdi!k=bILIN0bIqsZ{qxjRRyr(Fy{ z*BFLdmVD)q*lC>T=J#Pv&>GbLwck@ZafZ`VRgL2X?WW7Xzw!&7AHOAkk@j z7qW&bC@J0Qmbp}(Ygft<(l3nVZM|gVxs_1fj@c%|>fRZGZR8I(xP1!=66`^KD~c(f zwi?W4Fiv&La2AeEn(lTgx*l}xcq*9LpS1%}>nD*nIu!!D^f2v46rN?cFYmdv-fgEH zP3t?l`H^2Tuv;LHxPTcUph%U6%1}PJ+2iJeiz;m;84EL2Eah zmC?rq{KDZHq2`e#iE2w9{xDjvCME5ik7EDc0oCwDEa7Caqwe>=0yMKG;mkW8!{+$K zeA?{fPRd%l1m0=tUkW|pljvlAzp3Lrf6aN3aPqqJpXzj>q&kg2FXl;MdC6nUc9fv- zka7QQ0YF+akqZu{BPsyk6r9JyW=~Bop{$G`Ad*Q zyc=aF8E6vge4#REBU6#A3HP-o5 zKZdwL|7D|bO~{Ta--Wt9z9VQ02E|iKqS|-AM&%C|QluQV0O-JRRm0@-uuntmVam@y z@v*+VFyR_3o4u>IG;sp4ks;lT1 z`?J~MjeG^3tW=r?`XYK4-89kjt=e(fXTJ7DTz9ap)#bafe>Frlx9COj)Wk92NNr*Z zi={^YT>S$@QlRIp2K)K+&z7~%F z`Q(6>4Dxo8>m)guZB?C1B*&oYIQ}WICXKM~@<;pO0V?lXV2w`eWsE&!+PrErbCi+n zCqB!UZuNL*V`9Siy3EIFmGjr0<^4Xy-E@9&ZcAjjQ(ik^;V>m%^435*>lPeBQFG2a z#U^-dvP>2pU=d`GS_CYuDqP(!+-ssU(F{Lzz$8{} zA45e+{OXA&JH(k=dtdjoF-Xk2i{?xu93jwW`l5`uL;65p|a0 zY~R?%yGU{dL#5W+%hj5MTuS*-v`yL210>zg$&Tfh)!6tx!d=np^F$i2n92qN3Daz2 z@?V}e`bFaQ)WdPW2HBO2}c;P^R6f_V=xbOMe z@ugmteit_b7=b*5>RP--*QU3Y@b589_xZWA0il(daBI-?swipc549vAGd0*{@VDW^ zP%<%``UFPA+Im(TOZu_AkXHgus(Fz8UU7yn1IIcDO1w|`Jjk$&-;Q{$hckPLVbF_1 z`HT=i%!dGyY;I8^d1xTB%a#yaE3=>ax!2(r^tgHOINp*&2W-&uh8PC$Rz&wz>@0r+ zH|hC4t1Q>#r(1c)Us?A$YHiCJ z_?#+Vn)_T}EAmelCg;i)Ww6&t_F5&U!UA=z7s873QZ?BC-@mCFV)wim-7&GI`mL_v zls>Xb%>3a3-j*KDrad;9uimN1q5~QW)q*F4HuCP7Ty)7K(T- zZnmqZZ;IR_Sh50o$RrebGSY-j9691L_=yb)kH|IPDFICTBw=!1>>X@v4zsBi<}mgUN~c%Yg%;p3ZcH={3*-98S~|ch540-*q?5F zlQ!-7jr%AY5n`n$R^Gq1BFoFX3$nTOB)u-gvJ(WL}ux;z7K-=->K{COEJD^_Lj*^Ek$( z+(l($l<**~4_nJ=1SkihfAqx3jo9gU5QJTXP&hbArc|xnA5^$YCOrUpN{}B(Pf3Rt zb8=|gb|Mh<(HzfR=*c0^)>N809?Tmg78+T_1ZQ)K0fQH--_5M}B-9ossk0PWws~9P zf(3aQP>(^bO8kVw`+YtfmhTpSygB9Ps~^V^HmPf5KV}MmmoM-a6+RDX;bT zit4TYlX@p}0V{a0<&;Wdr2=^1go;>=a9#C0+0z2RF|a|Hk)fVOsvT4KqhXu0)J@d@ zfS{s%D~Z^Oq&x)@iPsV|OeYpYLFWW@!;!CyPrfJ}%E+&V!XIHUF+; zfemx@G2?9c4bCF-kKoboX@Bnxli*9LD(Al?*C87peN<-?OVQ&kfnj!{kKQ8QzDPs) zDdEC#uDlf9fRfKXTMJrW4?=&(F%t>-os}03RCaW0t^sBw}?Y zat8#I^n&sh6-A1GBx)Ig@>q?X?4&H{L?@Ds9xYnTbBRPsOASWfMI#Lg%+zpvypzJ3 zPnbeXK$*ptMcNV>6(=p)o*Fq#EDahg8hT|1i<7z9$B-3&`J5HIgQiu-T(q_mVn@9? zm%6Wx-}L!!->tCSOtmecR``}kdb}0IKAB>)M4bikcnevpRQpZL)^TMY22~jqsf>?QC`4d_CNNtAf0K31o9$gMc36 zW8EmoK14>JmJPxKseJ?_4d$gqvc)9!i61j|9N2sEd{vf;tr|VtMQ6Mm>mKQ2yk=y1 z^LI*lt|}_O7480EKf9H09m;XaEi^96mXlv1iddRYQA{RANsx0kU-TE9*-ceMSgd0S zV_;|;evGj{LxsyVgxr3Q_^P9xlD=gWKuPIESUxXXwv*r`AW3Je_r`L zDwc{-)zSgES(ojxTmv2No22k+YRr7a4>%G`J>Xx;$4%(T=GByls>?T_At64WqY{bZ zoNzi!DYzt<7VY9t)lJs@5o{Ze<;OMN8`kp14tpV_O!2dDwM+I!YddbTB4xlUri*w= zHyZZ4`U&v~t`b}tG$$naW#D%JFW^%;qHF;4e7Ey3ZMyD=5Ql+=wTYwR4|WeD?yVcw zAsZ(A`@mc5+h3@7)o^wT3n|phn)0c72Ae(3E7V>)vZDvYY{`QPnp`$Gfi4?e;yk}H zUK0l!dc1|j@2r+gW>KGg`-v$&Ed3Tn1t%0AIz~Cxg>{&g12d!hslbaO7A`;p$&kI0 z6=;%t&aE40@2-3WIIczxcqApM+*2YGycN0nW*1(C_N;6`Jb;vs(@Jiiw|s`vxYHnJ z2s#eZ1GB9v1?nKEqTd-M%RXTn#^L*KyG1A%+$DhLzl;oRXN}6G1%4`z=kWPpx~6qA zpPsBC0BOQVPlWt_JRh}TAv%Kiy*ay>F*pL$EE-40|7 z`mvShiTPgVR0E^{xA!*xCZ&JN$hC^kk!~H&oGbHNgNh+i(UG-FHm{*qC$mTsb#`Dg z6GLFZ18A3`fE@pUMSd_MT-15Y#Ufxdi2BIz@cP4CZ#PPM^M~*@1aWgI9htzRwm$h% z0)hW*ppOxq0IrPj(o@=46YSk~%ZjfY@1@SP=4ApfYNKG*1nA|!!*AaBG~S3@vb%m< z+si}S$xZx^t0^6kKFF|=$tN~0Czq3Ujk;03x+Ju>1 zr3BhK-_-u3NxTYJ6OCMlbrxWq!OPK=?xCr-wh$p&CkQ zF>-9FoSnjAUx{{?a9QN}IZ@OO@8EXZjGkMvg#YbpQ6d#RocPT#`5~gmLo10NMwe^2 z{0R-_4)Jbuxjjfh(O7YekVFkHz|#C%r#Tr#&(y z_W4IVn@h(=^M?zl_1(xuJ1;p3iSXIswViTw<<2XO#h^%jR7-`Q@#rHSJI0y4(`zL> zf7b#S)A0Jz9R2AI`1E&t_AUR_Z45J`Ry!8dJ->G3v>rEi`BcB$zVmgKVAS!%PHkC- zV%5HI#p{KjBtj2wTAPXlh2fH|(H4saOZ6P|^S@V1C9681U0NUBH8Z}Pkt+KTs+h~G zd!=*Gg(V%dZccaTM5?S@o>I(XiaxuAE(zyj+EZRVGs_+2Hy9EUz1iHER}ok-Pcj6h zP@8{*h6U0~z=5^ZW3Q1#0fhhY<_;I!glH*;jc|m`Y&mF9oNHFC7vmi1eF_`~WtiGpeI65WczcM~seY-DjA5aGSJ#pyxnM0ZcwIPH8-<&OrX?Ra5Q~pnVU=3?hg?U! zd>RtFI?cVtWf`xHVpw_XRHU}4?dDXZJ8E`(sUweP?M5xj8^XS8$EtrLs=7xM$gjGW zh3gH9LKXP^`1kFH0ht=E&jVdTwhaOuDe1k$hVbb_l#Fm;eJ~IJum=i{M;cbS1gkxd zdqo{J#!{d45fGpw^rLj@)*e|(l^SD2-uxMgXrdJ3ga_rth|g!9MOu0I0r29{)#2L4 z17;N5N741jUsiFy0+OYc<3me=sUKI-L1*10PziglJtgDs{}DBi|J6p_+DUDTpEGj8 zZFwh)XPS004-ipTly@uj)*FikWX1&O@KI1Dj)W_xPm^kI|2==}C?gNft?_#u@nnv;{or?QM_kW+CQarDENLf3ZBc^hh(SN}{J|~y`;L><7F`k; z(|__d#Hul6GZs}KPxSn!qq9X7_CJA_bIzCbn`C*9QSapqQCy1Mhm*Fp3qB7902`DI zU^YI|J=u{l|MP75rgi2H!DFq1;_kgA@{6Q4;h?bt)Wxg$f$L8VZ!N$T8dm$E z)Q`Lr+m}(%e?Fcmw>E~6j2d)~($}6^Fi96?DNa# zhab;pUuiYA&TWN!&0&&TI}aLuqJJimazmhxd`}<`%}1Jo>tsgrl5E@crDtfVCe0u< zDTG(M}Jb(x=x zm}=Kw(xCmFo!4c<@O^?3%K87CY6#7O6m@lg*5Ct%(fbD|l7|IsMb`}?iU6Jr%L>*0 zO^)Pg+96Yq#}I_G&~PwOjIDM6hdxv>Qvm>lWR+Wx5DqYqyf(&rabr;w364bGGGp)~ z3K@KwIS0y;n2ar38f$Ph z9eaX|VvU9qm-0D1Dc{bjc?vKjTW-{F)@O8%$p1`Ty&0LA^t6w?aBr1w9o{qE|5v$d z>7m@Iyv^OA?6E7VZIv|G{BEtVNbQ0T2m?bI(fzZi_CGB{CsYuZ+-1FQd4bV7pyJTy zlD#2(4V|}atY--WLw_j+XX;%ACUB+?XSuC@*=w^DFF^Yd1WEjf7b!M~6bkZC5wOZXM^lU!y3V!j&peay#u`=MQA- z?;&4bwdR|ilEO9^%N->2qx9ULC_;%3%(4-h$X8D2D87UWo>`)=tbs@Xbf&6)+=SP z@LpPH4U+O&tjc6ow?L*YWn<0fL(%Sp?t_2hhRlYQyfFYCK%Phronw6mp^b zQ;7ZZ;QzA#lF4+W6%zF9*@V=^e*aOqdTZYU3}755NUo8o{l0n5XHS^rG|)F$ox)Hc zC@25U>hpIN$R7dg4V9VK*#ZT+aT^9>mGOUp!+H3KIh3_*)bC}Xp2_k{?%`eY=eAmA zJ`EI6=5TE%y*S~VPaDfRtPi4EaXVw*)vSN3SE(m^vebQY?56f3jP^o(0je{;7DpZZ zuan4!8P!=(7JvD%P@tv66;39aD8~&jI4nA8Sn+efNBo1tdw*DOo-2Eu=e{0T56yc# zg?O#$?60gfWt1-zaa>X_YsTF3@dr&=)m3YJgN_BXLjJz}TMry$Qe0pMpstH`dE2OO zezDdry2z|{U!<|0GND#qe0d|lqcfA(1Yp1vyP8pNpAB;-UWPFcPqA3%}FH)zjO{#Ts+O~R7Y4q*CeInJ<1tP_-_i-N6`{o7IoY9p{ z1kp&kkf23WlN6m7)@J|w)lMq*B8^nAWB8t-(fyTf^Y`CzwPGFgY*~J^h*YdbQ7{|S za5|D@bww5iWk_O~gR}oRA@M>zzAk_FB57$bw#Ymbka`N61wkZefO>`q4rxrw2IYxU znJzp4fCP9Qh6|SzeGGKW^@Jp{gOQw1(bczZB6MIK!_0uoWdw8|)QK%{J8h=SnO)Ko zeq?>?VxkY|2Oc9S!0x9iK&OJrtU)EOctgU~#5y(+7u%nq+rTKreJUFCLOOu#XlsDU(2~CE%_SD{T>B8F zwJTPC%`FKPN{}_H;GK(C6AR_oY+0aUw_#Q*Ah`E(i5axvPPFV9A8`5oT;&s9|L*`G zSNyT{ECVmgecKltI*ICZlgyjtF91`#R06~9?FU+)Z0Ip#t=Q+Pm4Z)N?FtHIrPK@h z78BtWeF{3QozXb0P?h_rOJ88)Q9AETczFdOqtHPgCcD2E+qvwVL zXvN)K9skLUGtiUPg;m41i9HFyJRs&%qc!JQK5BAu7j{G9xz)5wj9V&*S=MbA^C7du zUAXTV#P|}G7yfPHGtSJe#JGIiO>8pUIWt!rtq=CqVA9=azw;)`+3sYTq)>cF4M^S# zp{y>b_%}7@EL6~4XfDP6WTcRFbdmy_Z&*k}+PhtaNu3O?@SU)GWOQJZ4hB-x{q(uG zV}#=6w?^37SMpb9UIqm*p=V>WgE-%QOU>SH{$1J*i@kdzJ69&PjRy$yu53d`Z zA7;f(aXYf!c+2a8vtQL|&TX}sJE}-Ev-VvRb{H&V+9oas$JBCRi+uOlQiwX6FQ3n+E|BesL3jnfyc_R01VDg?You!~M>U0F2VRoc1;L@r{I;AKsl< zcx{+tBo9W~DI=AjxREGqBuo@}oavI79w*@vzCiqvLrA=u?Umy~s0^9MA`T`%t}1Vt zm@Ncq1RYQf@esF)^GCtsnQH~U>7E9MdJb@Qr4$*mcL~Sk?01ebP2t<58Tk`d2D_<$ z59GQg&nT(!7~IOSHm}=wBUxX!2E02@<zpI8(O^qtj^D~buJ_dZn|uFz?;j=VJDhcb zXT2hi6g{r*uMUn%)O`O0Rv|uCDE03Ud&PluFkh`BOPm*yk2($wI(ku+{T(v(E!Y;x z|2!_sb(Q|`38|248%3P9~&2KhI>Ct!?( zIc~zuke=4#ps!LTYS))T*Ma%1YjVsNbJA>6-+ipd16U_j)0)r0Ny1Prs`IKmXHA#< z7P8_$(Tl5o2?9=!?JvDSZmw$cJ!e4u)|nr5*Uui@MnCnxwU*Bne0X$=Esh`%wXrhx z3^(+8(qfD{&!m!E7gqIDO+Ww#ojtSmV=h(hceg{Ss>Bl6YHncr*y00MCOb$01xID? z(0u!$ni~R=*Fom_nXwq&Yva8O z;qug_?zx|)jx7uRsH%2w3(W8Ot~68)Hz+*EKO(QBCn#a!7=_tY+o#EM z#hvo>rPknEoD@Wu|4Pu@1!nYo*A~bz*qQ6+ofE*MO`-C5u_nJnh1v;wtw)g$SFESy* zpQKgptNhJSDGd0`aaW$eWpCf~>5nC$`oj}^1GR+xKDG|~t0|)OudK@HO0eZx0BQ!;C!c7wRuh_z7eiD{&R*PHD6;8l(%qZsF^q@6YBzg>R7=1QITakMjq}uYWmTjv_8*gFcIC1y zT*Rq|u-nOp(d^=bEu~*I)y<2E$AaHpt#5UnkJg>zyo`wleuPnE=E#)ZWoxaf=A{K@ zcvT3_Fd-efX&30`@WAy>gx5VM_WdU=%{qmH%u;v>py+>Xznti8sr4V5+~C^;*!y(} zll}5e(vzRbj*7xu+|xoB>$Pw3DHf`yyu1$7_DV^XVVp=98=?flhgr=uJJQmh9ni7( zL`Tgfty{^5yAR*kqZ9)=)Jk5|tj}3tqGWxakW7E*)|uH$U^W`;bt6kCYW`8{koNyx zX}rQO1CL%^6+5M4dJ^o37R5zKqCRPjg5_NsBs4DIiuO??^({D(5adKLB}X z9`M%X>zkQD4(QN-9c)4#A!3R{B5-=4K5PI=TKi<63k&Cf{6UgvA8sq=u9TpSho#Q| z68Cc0-ECgDT5MGPUTdk{#cWx9z5oyqr$`LP6vcdAX#N3ia3A={RETnlGy7zF zQ=dSQiH&SCFz%R=Dc)>f`egY}hJR6rk;W}tZNI~^9QFxpU8hVD@Z;4E{l;_=Nc_)@ zxa3PoqK1|P@O_#?^_j|y^>|RL5EB>IoaOu)?}oRMH98bMY-olVH`}}u0)UC-O?)#l z<98~?c0ICbSt`8}|D#>l5f^)Nu3z3T=+LsP{Y^j*Ydi0|t-XNZ${%N5Cb!}&;yU$+ z#pJl#0=apj1o!)(if}YgpY4BJ8+lLVgj`V50jBMU0SZD^?iZ+ zh}3e;3W9WAo&&j~P-Ztt^Rc$>bEiaX-}|~9P>JDVyNr&TvoN#q*IXp6#E|cz0t=S= z7$=Jflw}q1(TrJFf2&&=4|t8e)M2zwySod(2tFD;fH`(Du< zs%*f&&1+-?C7l=bbZcKn{IMwxO&aB!=n_cqRP8MbBH8II-Fjl%F#PlL(7rkpg!emjr9mD{oOf?~plmn`<~m%E)0C`Css92Y zkhTkQ&c=~u84137RqV_trav}Td!M3eYUO=uAS{W=+nIbw9~oN8_y{T@A0CTj?{f4# z(5F|N;~V{TF=`Q8DoPWHVAM&tx2L+xik;;xhD4`O@i1SN1keXm=Fi^$dCotQ3Klmk zkP>N1@Czl*VG{l{(>_g0@-a7_lxsuC{pz_S!9XR%SvXqJ1a`XA&j{K=kSfdE7t5|z z*{>%FuK#%(I~dM0!8|wO4gGt4K@au~vkrlI*MEz|+V7|Ter}dWIUP~e^wx7yJkZ6T zveXnH$F9)Tq)Hy{OZu zm)7S+y$`FF@9(vAw%Z~Iu9By(Q{yHUQik|YAN4=A($>l`p_*K>;VCdsz`P*OF=Led z?G(s>;dA;&g8um=z{23kk*>f)UEjk^$bCjUU}4g1gW{*)$}hn-CfPLv6D z0@Nl)0se~J@8&Hfa@6|(72Xl_)?4=Af#d$Gka&JRd0AGGg`Foj$*d&5C83;+ShDqP3D}!Jp6&`4(?RdG6#R+gdO-Ms{^1B8JfCokX;aV~NuD8w zRLFbmTAHbfdl!txsa3k*uk_g9RZ@=k*WF-u(p;l$IvsPyY5br5WG)7|AiN2~(Y5K5 z1xZfTVkE$*;7C2MKz|SXJtn4w?mYy2ufr|)r9^atzpd5J2Fhya-?r?Wp=hP$M|D;b zv%HgkZ>FYJbQR6augLyIlY@;4i z>r*ligOmE^f7BHUemf`c?b#vJF}d-dqFgnAcaaz&tG;i)^yg|%YlwR~EW_qJm%B-f z4^24t0!XYj8cq&0X#ORMrnc5m&BaqMF4fz?Mt$tdl`Ud!?|84y9Ycpxkr-Q%Ruo!q zR#nOy5exj1yM(6bZRzr=Ki*dHHl#%1RuzPId>YWt>6a}^KLmhhECT(3bMkU+l4i61 zGKj9 z7BJl1qH{6uo#p2@G2A4nD&ucg!(G(+El!j&vHS^KxEMQo7aV9=XmpjBG^6hHa5uj1 zZ7AIPY&+B1)+G=8iRUNLJR+7c{Ctd3I!UNZiT=haVaV?|ZxW zj^1g8H0@{WN*tg8oC?sU&P`nl8k#S@!Ta~DoAIJnPMx9MB(l)>qQWTE zm@^g(BTe7y-Mr+;>~4kEsQUFj!MpIy7LJB($2a&rBvW7o)ZqKL zVO*UAyDUK%$+9fF$IJE}6dBX+%9TZ4$TYzY39s^-y-0pvJG5yBdgtT4n|*glZHY*c zsrLOJ0L4H$zu1u%Is-(dYT2+X|H}dkS%1><9DV`zVqyW_2=4-zWy>6|n8jJMVPBztn;9qb@`M z^bKI!6U@aQ@oJWwFkv%3Icj~5z#|(Ym)sJW&8N;9dZ{%>62>IL({NCSOb{a}I7wz33|rn&!gmKPo@jbT3Ndv*^IM z%npgeQ~Fdf5j}%jd&zppkD&RwXW6@_6%9G3fd0p?fw)puUCN?e99HL!F^yjBR@<4t_ zY_*bShYIN#FaLf@k?Y*RC!rRH=K*AQPpO_IAhu4zszK$n_CwVakE?XrRb#ZNk$uwl zQ#Fs-c>>zPufk?7;4aDpXzOx?pRgFk!WDMNS&ZTUO0Iyg*bj#f(f2T|!0rRU;ctB7 z8@{+j?a=0?4QHKoRvEv_)|_t){)k=P4}9PQ&NH*TEgQFepQAv{?jasiTw5nrwO}gY6;U0; zU%cZ&948POx@mB!j>J#s>FsdZMEzSUx~bTu_dx9A9q=9tn!7R$P8LGN(`bY0S|(>S zRlszl`Q-o8rBvS4Di{B3825{XSI}1|aojEcANnX?Y z-u*s1<1gM&M%7wd^vWXDdB`#*m;=HnlWhFa;LS3!d^W^E=ncDzG2yP4CZ9*6XX?bn z9l@EpQD&q3NyC9Y&wlo^T??jO8IAI)m6keDAAW$~)Qf9*e#zt!KlhJ;fXS{V^+nlQ z{Lk-PY+wKOdD2!}jNbYM-wGznJUom4;SSH72AbgljUr+#Y39>3h)_MTJi;f=T1fd}nv9Rn-uTVMFT z{qtw9wo_lQ+FtOSr%3^8*jBDyqeBIyo$LqfM0pY(_!eA{s9J;OJXvYR(Z}6(l(e5# zthOI~=Q8_0pZ$hRqH625Z0u~^%w)7+YuB#UmhWDB+S8A+-+S7Tw(nj8Hu}(S1c++| zPw!xQdsUKcmhq=Pv%5&(MDGc(qoZ?)J+{^C=U19N`+0gNszY&1lL!||lXeAPKp<rqbmM*74^q|$|TLU1RCe_uE0$sGL6gh!P2Gb7o$4-PYbY+uEQB) z(hs3SG&QbK<#>+zXX)uccDjS9M}4bPJ6@7Wd258KrT@}RfpyW?9IE~w>ohvNhGsZUlTAPSL{_3yAqCj@Nw4#5v+ykJd^oMLz zG*cHnRXTll97c(`jZPtE{F|BM4NTYFcQYv?E!}8r1sK)4Z~&z|f9ju=1K}5ZfA9`m zgmJ~iw+RnH6Bm2A{I0$BT3={_r?6{)a6ntY+vhyzIW{yjRoFqR&KCifxLYzFwuX~PF_kLH;wxKjHwkosjQaDE&LyWnI#WT2+Km8-rJEZ&E=C~7v>kTK|=EUM8QOpQo5 z7@HV%vW$&RXs)=*w#kHES2sI?#8c?UybJ2&eLA7TY^qY8HqFn=NkXvwTaW_LsCd`5 zZBkT{T((s2(9J}ZCoVlvPp>QacsJ{Hx_1VT>;<>V5jtpLPWXcr(8Uua>q|zY8w7z;}c&9xxWfm_|PJ z%DL#Gi+l$hJ45EuF}^V_A(vM(H#C=JHOi}qa)iw9SkFokBN_XnHy|5Ub94O_R4ZdK z`OQw6T}&;Rx4OiD!REr08Fgyl>Dx^f8txa*-6n>rL1>1~Cp*F9YUIJzgy8c4T=VI& z42125^&AW)Ku#v@xNQ1*X`2ZnfAT{^Ck{2M$JFw9vSkieuTHJxm(3q--KbOK-%MT} zH)TE@VoJZd?k>CS<_S#b}%H0QpW6(tE(8{x{HcKezva|pD`)<4OgCG4noqE5@HgDQb*rbH0Z4>s@ zuYTVJf{&f~ceZl9nC7sw)N$Cu1Aw3-h7==j1h*LrGcl+mplR?TZw8R5kx{$jt_SUx z*GXHtf83sY+@J=~F59+s#5Qhv*lxLPqn#(Zyy*E?*k7M zFf4Fo=B>EkQ528`3Gx6!s}(>%hVO9&wBQ~g8Svt1PkWku>|-Bu!Iba%um0+<{QLgk z2S4b*EOE1Sp=`8!G3&q~M&JGJcU^#{?xaV^MVl9)^-XVjlRe`Z&+zi1ZHz_KVeTCx zfBFmLwDgtFeQB-aNUd6OJ{hi+g7LoP9BXCPv_BI(2fLe2F&&Y#@M51NPJ1_X7rLVYBRU9?t6Qd;)o!V?1L+#?;a`0BxZou*5r$8|Pp-fIdS5^S+(knd?Ob>XVC{wt z8zcig!45m@Fb7uwWRYdrae!@H@?=7bHZSGEhv097MVa{VgP&10c^-W5!S2Tn1xzzg zhm%h}*~_8s{DNNw?+o(|*!CsAwwomd=)YR@>C|DVg}(JRwMFx_sZqVBn6J_A=;2lR zAKyof?|GKBY9|*3O5kqFZxU_V4%@OxXSrx~r>}3Bjg4sOXp%jIdKa=&!|o5xGVWX| zx%-iOH3yd>l6J4)+`NO|wf3hdP_nCQi7UUH^&kIj|NUPVN-KZLcHh_Re|`L2wsy^) z_Q-=~*{bb{?Ijx)4T>@o zIf1FjZYdY3GLvKK&pp~j(XrG91U!$)cKc*eHu#i`1-PnxJ5w@qw;7K$j2Ac`HYzlKvB4zHx#yniW+2G{UW=QYapgU^ z^K|$x{FntcWXfj7Xi^_~=%JqGD_{AF%cP;)Fk6{Asaf)uG3vPEj&ra!>tE}haqXGU ze5Q|gAOHBrOJicK_^tYrKVuVz9-nf`DXk`-)A&W1HS6pZxa;#Zpv>ryb6U&}omm^; zZS?wlpr4L3CWhX_KA%3V==ZL(sjUx7CeGNMb!JCb6Ni;!@{usW={#D#OA#~}4?KIy zw>u`e%vQ+G=0%Q`TBx;a>-F(EskQ0eNEjvlbKzj( zM!WzLTA^2@^{Wq11zXWGX#IMpXa4P<*^`dj-?k1XS|QuEblO7?PuiQ`^>O>BfBmVg zS=VVVf64D_HJQ{5rnsb!CHK|*(m3kF*sPtB{=#IjS7rt7v0h~ z`P5hYC+)u9=pZ$9@0B`)&9-f{$28gK?CbJ@%xyj^OK2H_fxl_R_b%!WX`ND=zX{J( z*~xgER8@Zydt07a@_nRf(cURYpy^+-(v~kXd&BFWV}J7M7i+bAOsm3MG_Z`@1NUvS zPkrJ`cJ^0qvttjt%wBxjvlK3Xsr1#AQ>IiX#ijK~KPf&YK2o{LI%d(kJ|E|^N zbr!iV?-W%&z;~jJrv934LC)sUzF&M4aQ;c(ml)c=REuXay<-JVWAzVh@m08q8qKAE z*+g*;Y7h@zim;3j$o)YF9pv`hL&FUXHniX{0fZ?BfD__o9S0Qw!m#!El1naet=$+0 zA|9G||MXA)FGYoNW~FSCaxCgBA<8@i4;_^Zwa1xu}cdv5bQ* zc$KEvk`~s{_|)#0DQCcvzXisRj&<1CXaB%%*BKyNHjmksZQ2ba`{u~>pNbr@BIN=(yv2WqZQCTeIKq|fqy)g-b>Q{gwJcYh3YURb8p`~`!YN8gFmqi z`3T&=8ub%}lfTcO(t8wMq+)#2 z&3D_`-`=PN{xR!3j2%1af=U%xiC6DqP)eU{z5HHU%X{v#xBZ{bea^P(Fxfa`C*t{G z-0{%j)#tGq_~5(nWB7b1g#|yamSrp?VuPJ?czEV)3HUbS1NqFA14v~@NSuol`PQl* zZTZK4{6{|*i8|Ga7x}b$g-(oVT&dTBx#+)9px&F7TYbJ(EG*IYC)1n}rfU*i{3Z3% z$ejt@b#CXbC+;FN(H2ii&*FwRyTX+6D0uX-Q1)3epg6(4(ct%Ag|V92^U z#r*!0wp1`6Shu zS@L3i@YXZhyZp3U)*gC9@KvJpqEq1qv~6}$C$T@K1&eOmB!yjeNsVb>?HL}iWxGkE z5$IF{`MCUWoj##U@|O;cJ_iAs@^|3`E}H}qRZb5Qu0p<|4lyBjutb9+`6m_EJ#jVS zkL!RM>e{RQJpf%n+WH@w0A{p&xsqn>oQ9kl;`DhaSC zVeNh}4S8WRTh3Rrp_2zwCS`F27}8cPbzgdl3mab9q!C`%l8~6B^SH!U-D!``tx;Nd zfU;`4MM?qQ9WL#hdRK5P*W$yTbFb_;Em><*66)#AZaa+kS=N_H-*$~C07wF0!WAGF zKqP<;_$>l3rWFug5s0%6fCLZ*O*(*HZ26*zM;_rw%N8*}-ob%EJN$^lJs$+^xfHIW zyoT3cZq*{z;R&=n2}X-NO`LSfYB^Q)E2mx(1)f#PuWG7=_EasQ5poh=oj1B#&Fj|- zzMJsxh?joTCB7d1&0QJ0$a8zU?Og?^Sf`t&iJt-s4O(}W3^9&$HT@`Zo! z{R5zKxGx8ovEai*8@t}{8{)t(0CdrQI^>W;+@uS3z5!~{lm*l!4-QfzP1xIofAC{b ziaf}RU+^ieF^R_?*)%5nJOec2hy%~V#_t(toMER)n~g&w@gv{(k%oKnp%(LK=7KWLF?|_%|<8g(Acgq3E7V@|M=+7(TB6wTt9EN%yzt*6oA~pYkdCV zyhOh6aLHX;q`>j@Z(JnvPNqc>vlUBwH6N0;{iu{sB&vV=oL|_XhxOZ=-tsyv(i|kf z{!!(KU3dM{w}T>!&nsNEry6>PWeZTr`}J`|#g2_^v8Ail>pXzV^zG^_lK=I7d%`jM z%Vfj-wnQfDes|x5o&DW&ZO5m6&>lP{FqyR4^Rq(~cbNa99 zhe<_@L-;%=tz&`Rf2nhYgNy=~&wpj+r*rvIJNKf??Q8$>TN~Qnr2Mkdwmi1URt$97 zP1oLT-~9R&w)Z|?xA(mJ4R-7k50SzPI|PQAm1OvH+UKLNQlBq*WNx67-U)n2N^LAc zyG!Rn$#||8!po!`yjcBux|c`^X@IGy%tZ_)NiZo$yb?abcSV}?=dmjP(00b>p6*`P zmL&%KF)1wK^E^ApLL9h7-cf#1UOt2-Px)AK#1Ti70@>Be`-6ASm5VyWj$!J9Up*I# ziNt9Xw_ZN^FhA=+?;2WB9~9^KfrTujo03x1q;@3tEZswWN@pABnVERG%RiTg?kZ2y zoMl&OakK5L!Z)00X^#lk!j$%E^sKV}-K2H-;IMT}Y_dKdq>|1aHh+R2Hd|~yUSeck zzLZIzGq`N5*?nAj22hKkWlM~E7+El&@XJ4k|4gsc2Z4L0`L@!YR_Ztx;4Uw8n>xt2 zu8&Hn-@5TJ$tq|=>+m&gCtF=uZhNj?ZmT#Tazq=HWF=wgvgNkQJ!nrrY?kat8DmaFdG)I%qi-e`ra|9ux%=9M0T; zd|2Rb0;t9iR9K?8*ooXKyH+Z{L#u6*vd@IHRscnt(n>ST86iNQ)rvFRaOv)eM;_kdoz@|ID1cto1X-AffF0u{$q8x2BBm#GNVw{uV{ zzatWO1lwsPxuC<6I&|<4hXJ_62(7d@BZAtn>@jvpTVRR6bzVwQwyj*V4gp4o-c4q)&LR}#(Sg_M@J?yPTgiPyGH!3RpGPM_^j*)I09$^K#QYZ*ina%BOqd5 zH~4y%o`Aw=&VBKVUvyIqVM7dHgmmN$P{P4SYu922e^j4wqE$dXt=2bNcGdgzdsfcV zeZnKA{fBQJ{+GIP0MdI>1o<@E1ZzT!H{jDgxR)xuz-*h-hvjA&0tGLgGQ*!p1I3420NnG9szbJ>;bsY4i zDrzs0lWcOhr=2Ha}PiKaQ_9LAx`ilw0r?hSzLm*Fu4aE`!2__>!7pSkE>&`vi@WNqELO|rrY2QKfs*KGH_OowED)b?K2A=|PW?ZHitN%^E8 zuvKSs{M*%b^Sz(3?|$nuw%2MMVmXDKQSos8Vpb4)z1rE~2U;V~$;)Y8yIqrA=xc zGd`vryPBK~a=?@3662ctfN>Qx7t=?Z%qt}Oh|wfO6+iJx0B~x~!|Y&0e5atfO_$8{ zq2z<}1JhBWAs(Gq^x=CSQ(m3gy18|9#CHpx`FHQNjc6Y89-kN%50$pGZ0OUjN#;xcp0UOh<5zj8EI7sN`TfW-O4o zp@Zt;JW}nWUI3+&y4Rim3!X`C$@onCq-xWDr#&jereigGmzP7S1Af&UVU9)U9c6hv zabp*A)_=M^)pBd>7wNNjL-ZF$-0y_aI=z! z$|%Wqd}Qj6qm%vGfh_G|e%fuR(OUnKZzzD?>DnmGF6^rJ;$4^^jhar!cC!Wy5GHV| zYpu3Wtko*luukLDiWz(f{v!sNbj)45gEgXOE8)&qeXwjL)#8ooL%jrhnS8RYf#o)^ zY&RR*#-wA^di!+>z3fokefKZz!H3M&uMr@krc90rgw)FDrY#fpi_3pu`>tJQhaRx6 z`l7&AO=JpP-CAhqwR?WE(a!ndrS`OAAFl%@)=Dd6#J>7(-_WY`27BoKanm^h_M;zO zY>(V~gB^O%Ubb04?ju{Tu`L3Uw{9G^?|kn(fw*_uVMBY`5l0>-K^UX{Casw<`{jif z+O=0)s}=82+38)c74^s2amO5~iITJ+QM=WOx^ID!#?~!Y-e!+%++v3xd$`?t)vxV> z^Dh+!J7Gs3d5E3#>=PusFgR#?q6;*9TL@p%-^|G9q-`Fav_6EPKDJCvXvKZ4-F@A? zcF%96=`0y*UmBnnA-jk`{^mMthZsk5{pD(42Z=gKZ zTt01*iG>rN{|X0U@08gEv~mGK;Q<_?%Qn%Wp&>()3VXNk0C)m`F1#rCi%&`gWU#XU za28M)a2MW0nz`p42^Cm;LQ@!E8C$%c|NQ4&YnVDvr~UWe-_1n;`o8ad@3Rw6JkehM z@|SxXh(lZ2Zsrt#&(OaefY71F*0EZ$o({U3+$5f%E#w|rtslo%neunfq*}b0*KJo# zInDbS50JR@PxNJKHz6ep=~(DIl72hzaYNR#Y>O=u2)w?yLUR3p9sBeXZS&~+?c3kG z*8cC6x7Z#h?e9ZgQDbyba|wXgE|gI8-E;tK^4{bq1t75~P!h$M0*5lC_H&1Iwuf|8 z?BET1>+rTM!ay;(2#5(-nLtNV0zjuGx7fp*?y-LHi-Pi*kRpnX(sZCxq1Lf=*w2fh zylw%VIvrY#iatYs56&iJBz_ne>D@$Y}Fbxdv+9;UNPMgYLI zL2YnAb2}-bbxY>$k+MF^LxQZTQ#Gl@ST@D_g#0d-vnDI@{f`sr`V zr=WZ3()f>eI;A-o5Fk)sav;1s)AA5iXj45GvR%D6a~;OGQLo&3VRK!V#h%~%(;``f zrkk)!cH1`fy@6FSC#m^b2fK@P$h{MZztM-?Lw}Nxc-j@OZGgAQx1O+PIF|oUQcba9 zw_d{HW*Z*cq~EX&YK4K-uq>0FRYxlg5K+;*z_)!n_56R}(e9n^$^f@T0*%I7oq;Az zx8Bt@BD8nOz&Zh~qju3H*V`A)`i4F35VNNqcepL-8P|bm(mvR2FT45nyX(*lOoBs4Ew%`6k(r{d1*Ij?3 z{r~@bmi^-AkJ{ckV{qBBZrk#Ud+f`f|An1;%60bV|KA_#0?KKq65+O2n(z2&c;X1lLgD#2*0$}0UfeB0 zIFQiEz>lT@K%EPe@|TTh1o)zageCo23GUlnS_%@}5#GH~O5->g-5_2a6H2!dIZ1#?!uqQ z1OEVov;2dwdM+EcT0H7V-t0*D$VWcn;PmTX|9U^@?d-G9_A;ph<|Z%~L7XFxJkm`| zpkdr7|7vBn{SpfJn~T>+e`PDY6mt3p_O)$~%1ndoh4yrbOIGd6*?w7%qWO5$zUX@C z7dmW_)i?8f@vQ;Oq9d|NN+!S6WMK~}c*t;rjdx+1Zkr9RTWu#j=kfNfv#+&Z|LRuj z_=Ei=$LSRcwC&P2-GRobG)Ie5+Ev@D_l@$Xm^@|>x-|Fc;DB@~m`FfTUl2&C9flKI zA2ZC{0z(e4d(TZKh)z?o&AoKo$I+q`0Se1>0OOViZLnY9s%!&qe)KUZmq^3JwOUmP zp|&MyQ9v^$+;GYG4!E2cc{PTK9Mssln# zOl+395$&2BnXplTwuAeuvEz?B(xhW%w_JaZ>MlBHA~&gdPNxo4DRzvB6vH+kg_WYr z-%YT1q}kSpY_w@F6{Us+iu1b5cvdRb@uIHUE>ehmf^=tw$FW@ z*15qVzr9m7xE|Ctr73&h{zvRr*WGPbU19dD-&=35c*zT7qn86px+M5? z*t_5RQTy?Q583lxe1JXg%Unoj*E$LLd+R%t=C}Hf zcD%_UBqV}GlBWH)843Vo0I&e=;%>X`HoM@03mjnK6y$^_Ols@pXMKw_AT8RuSw~v7 zFQf6s>Gm8h;S;~~I}5M|z!wKtLwH=Xc`fvP@T}&`o$h1hY4LBJlhDuk)doGAui4zk z%(jR7>9$iAb*={DHm*kk@@8{yoThq=nU!BXFJDZm5me#L_{UH0E*$T1ySOUlCg47` zN#B_!{`2Jm(B#T*Pub)C<~P6Thm`TmcMgw$cf>tBAsk>SyaB&X$yTlbr@+`qT0QTDEPy~_`IyYRvb?X0uT@^Z1;yJ5oy`_P#m zve&%!HTIzoeaOE4^{@LP7$9`p&6EQAsC0;ygyVFfm-*VDbuQb>Z1cTRK-i|wPD%#! zYxS+<`=f8?xyk(X_nEeAXHCcVg5z~yuB}(vdO{~PmDb!^+MyFNtumqSh4EtlA<1r< zKTPO*#{9sT;$3;gFYWL{5407#4cO#k4+#R2c{s^>Xp)X=>)rLgyxtBu@E`%XOKn8k z7;nA)*Y@zXZhJ^)k8B(++I2VGD?LFyXbv-Jo9z0VMx5&vi_7fJ>wjbS{q`<9aR0T^ zr0vt8ygG=wbBWHP9rI1*dFV5w*w{btX{vHO=(UxCG&Fttepa%hsQLh zxa^$MP%WUN_qRmH)X-1WiW?PB%?PNChHCB4;-<-gU$e#IZ7L; za>FqL8VVB@D=X{d`|sz-H?zN&<;yB%K>vYy@_?7S)io|gy>JSGa{>jz7;7AFT-H&` z8Bmn=CeWZ!)u_g%IPzcvlO`LTxA%@Z>Q7vSuKQzv^Eb8Ld6{_w)@%92SgS$aM)Q&w zMCeE8yGk9qZc*4SErtLNZrQRA zrAHwSe&eXQ3V0cleJ#-6TD%Bz3A+O4A17n)v%N1ip|%%}rSE8X26t_J=E9@`=SFI& zG{QVWE(lg5j=faW!tO1Eum#XCed~>nWfW}Aj@SU^K%l9bY)mozdGQjdf(e5{O!PE% z&VRI16M@i@T`n(<2{mUWHl?d&r=7P%-|YvJ4UiEd&z*PMMr!BvN4yoFlVCuK2Sc+L ziy!P}?>22B`tJ~_udP(#df)vof-w=f-CgWxH$?J;2cEXyX`-BS_AwBm;Q|pb)!Z?TBzemx_s!ig0zyus z{?4uPU}E6?#v6;|`yZZztxad+y$>{pd-jOR1($3myH4E}8pAaDjvG+1EB@G1&&h*N zERw?yTTjmY&5^h)4hun0xSaX|p@3<}sTa$hyB#anUH^*gw%1fyZ-ZXg=+2cHpL{OA zKIdr}KQ2dZ`O|M@z`%T2vEXYd#PO%$BQ}s7cikE#{-fOax7TIz-(QfQ9lbXMRCBS2 z1ZMK(LW8A3rFL3C|PL^KO7;MoHKr$ z5I+omK{KUm+Rv&+D26MhF#I*0x*egGv4QYtgn6{|>Bj}qup%Bx0%6U8WV&Fvnn{z^ zXP9-C^l7+7J`rk$DNSrTgJ2CbKjguuS4|%Xe?C5M?={%2Y2~L>2e2s&u!TP+cL);( zskSYF(4QLoXR=N2002M$NkllmzqemyhM(hSt9O?JIZu`3`yIl~-i(;>E&C z;r!0Z=JyNVvtWSD$j+F6h6AKtm@VLezJj?Q4Im}GM@!xEh}2YlD+MtMeqoxdVg*cq z@M5U+KH=rS@j}T}nhTDQw7?d8v^RYVvPg@vy7NoRyx9-(4m{|Cy%!lc0R11&1k3`& zAk@sy?;&4)woGoj=|LF@InR{I8>$v64nazB-`?`rgHOrNj(k}DcX-7xu@^}m1eQDQ zJV9Q1@jdhj*yu&6|NX)ja^#U`BW$^xdDgKq73-q}G>ZT8;wSRxKc1C0-i9_YP|F|J zb7R(r^2x#b%C?g?lKetw>(*655v)wU`*N;acI7QHZtP}QlgyRB-g&2d{uxd+;Q;3q zmz2wnJ7A3l&JOy%ge+RN+H8azebhd($@u=jJ1?wr=w~fV zPVyAyx1TKsO?pV)dgl`;{y-N3Ys0Eak&z?%%ET?l%27v6lOAOdC_}+0mm6xpHA}94 zFkWc7u4%_``>~z?K$kW#I9{3lHb;LmrD>1f>dvpVWnJCnGf9}0Q|Es)A)n3h0+uGz zQ=6abX(+5Ek}&hu0D*3_TuKIPgy`G^!8e%+b{BRyK0~((20{VvDky1%2RoHgbWpv# z0T3487)iqd95IuKeX}pPJw}cabI~$#817aCSUtO5o_D z4wapzY$7!flyWz-22B6rWmWR{6EDjhcl=Wp%=$>qI^zTxGz>SCQu7!4%-pSQb^^ua zQb{l@{0V=&h>Xeg=lJC&YH*JkFG$0&H3p$0!s#OE(Yru;;&KJr9gg$T z0x%Yf7F9zF_far~B8MDyAa=2V=em_rhu!Bq*j~%e!-N34QzxGCD>?k68{{wd+$R@b zc825?70R3ceN55uv!joNaJL+rG@mIh8tcKd#88Pyk^Ji96XY*{e_igr_X#=f#A77~ zN40p=i3>h5fOS-r2KUAKx^hAm*>kL;@h+Zc_|mBuAinua^)GCHU_+&fcGY0^Bfk{( z+u}Vzn$b7d=iUCADijAv=vspT3NC1Rfm(JvR(;Y*C#kkBFGRce;)~S=Ce1D|P9|qI zdMV)Lg-NCjY;h^PF~P3+t!}2)^6jW6`)7NY;f^XtcY}i=>&zc`rp7pH!I02(HU_-8 zy93ef2zPOuS%n&8aBu-8gcuZVa-5s+mcm{CxDVds?GGa!vI$4C`F9lVXrf^*PC3zT z@DOMtH|J=c;pLZKR{P_F2M_k5cXm0wjMkPH3Suc~^MtiLNKRgkv;xdwK1}ODQ?jyR z9^yf21|G7g2nJ5=vMIxZ^=;$hGYWYva`@=oRpXA3 z1?!YXwb1~56f8J+VRa+T3E*9MKdhDX&I~MS8u9-5 zS0am-KywD#sRJ>-rtgJ&Kq6)G+gqNL0fWlr;QjYN7g;G|Hy$SaAYh(1HzuF{@e%1a zuuu+wz;?yrS(>f^rX$|@@5ge@AMU^!JErQxZ{H-3BUUM8+rQi7ja_pM!D>= z6J*Miakv;NA#t4WgSlV%=p$&-zV(W1F!}=sN{7j`U57|72ua6|g)KgH?6Oz*Ab-2(OPRX!m(p+emVk$&Bz9UKiZDfTPl&a<_~`|NMMZ|jUM-~4vR)zM|M%LS1{*LkK1GgllvmM@VuMAe0?q@ zQEUSfG#IA@E_GYD41#0~gGDP~o*`C(%gCU?3jozM%V17Iq;JnM5!{p;2cx4$;?fvt zGA}HwT64m}&1G;MQa4;db zSfR3BO2~i^5fI!ap@UooTU+DC7t7bPDrD)rWl{)RR52_Hz@a#xsVTsGg3iW0V5JKo zEMhn*87A8Fw1t;4!GORv^PC^I?mO)6MWUrJzcw7g{q?~-lxo{^GlK2y^gx%PCY&^Q z9^mjL0|qEmq3ufwVR%dZh!G>?_~Vb)qtLX$%VWyi5IOC%)38}n?P=xtK{1?{syUiG zq-%}~3T(1z)LHy&Pr@-_Nf2hXG@*Zpfe-_02m|1s&Ex_%k2q%aq!M@II>7~Lg3%fC zw{OcjsL^Zw_T)wu=?!TbX=8Ja8(fS}8{|f(?Act}Fx)V}`5QkK^5PXd0+k{hJ{-I* z1Z_DDIQoy%ZAHa6y;DC2(J|McZ|RQ$-!vEKYbxXowq+nrTKe##r5K<_J`e4DI78&V zzdwf!Uh-rDo?K)FtT{ohDigOTmBE9CLns)>QVCkRY*_q3WB& z1g=xCmcbp>SP#IoLSCT^8{SvWIQ?L(QKItb<4?-S^?S+bC+>=itQTVQzf^YHaiY98 zvs!Mt?GYJ2X@DGe(r!|k>skM$UM zAah!NcWp$$79|aTzWq_TKT12pPnEdRhpFpEFfy<~NY#p$kJa+HX za@XBo$iatMo{Ye-^Uhm97FIJA}yKdU=iG~MwV74{rg zslhm2VEEDr7)S;du|W;qj>=Ny6QHw=*s;kg=mELz2nZks13ZPhc<8#?rMdnt(NP6h zOWK4cVB@#6EGhf!Jq)&aM@tN_RJ#>SO9?b&3&`7Wa)*GK7ndn<=9tjD z+_JH-{AQUnYmN`FL{wA8?X{JaxJaP55KJEI5aXE#AyQ2paEserx#&uQaiD+>hcoq^ zw(Ih+2#mxji;o>D)#_}HJQn_r2UPQ;s0ixV- z<}ps_=ti;aTv(JRxK}~js-UPIu#3TvK%)><&;_7v<~SHk|7-P2_YJ?=g8`2YIj zEb**PM}lr-D4j;%NMamR)ZvEplD?8hTTar0T|-$!oZt&xJ25~(1T}7{amORac7YLv z92ELeBaO^Gg%>nQ!yD~0IfnTKDnX%2rZk~nh=I0aAOnxpc3@%34#j|@wM$Je+Rvry zP6AK6O!7OZB(|j7vu4dw(-GACrRj#2r0Gy}uT{V}uE`Ljk8cVBdC!G3LSwh*Xc%8j zNOi@02!6pcVxu0bIV3aw)DYt1F+8CX;sjt@?C{xA3|JdE@oArR3Ctyo93gVSc_&KG z@>lu}!po2Hlcu>OlwoL^c{#3dNZfQV&(xLafmS^(&VRHs}wN43IbA^0izZ!(3V=H{5cc zP#5Q<(~gq?!*TkvYCcXi;;t4{Di-T0M;vvay!XKjoKBo62fp?pPD^4uK#;@~f?}(q-acucr@S9=b`150EO}%pjo6FQn7MA_Vlgt*Hf|f|;OCnYd z;qEX;xQh!il`e-g*O1lh)1iRPM(er}Y$50`A$R!c&mS2i{XwD=+kup;ZyLpJDRwX^*x-+1UbZLOI>F1j;8udUh~r#R z0ybZ(67`Y~ja1$`S__-J)F`bX|Eo-e8h&ooOD9-ozl-jN?{Xo}-~Thy;X})zn%59NvkVfbHBQ zjF@w8qN0Lk3ShV&3b}+Xx<{(cvY>C+;FAH;y+QJ=sWpE~r2 zPZy`yyojJL@pM=%8mU2wBvv?5^2*jj=RrS00^F^Pr_(r0g_%SDYCBq+!6)T1dZvO~ z`4*f{TA5tuw$`TU%=i?>QkYAPT{?~-)9AhFrkhl_O93x0PotJDX(FH0ZsR-C5hrFJ zvpL~zbK$|dGCpC`wKi=KmVI;K6?EpuY2NRG-=I#OvouaW_PjgzB}i{FfKQ`Iq(}T^ zapZgaF7W9O>e}yCPpfM-JXv^4YvYgKe&hs8#p%9f&R-6v|LcaBKe4W|S`5&i(t8i~ zZ!e(DFA&SPn!5zi(E`tG^fyQwvgI3$7p&&8Wy@sit+&>TBj0@UP1$3QJs^ZB(Gv`e z%S|qpf#w@#&z>#MKKrbG_wV0dX<`sw_}vl=`1(4TGdSAdC3j|Xy+hJ)>M~~}nA?2V zc%2DB;v5W#3arR5%BU1UItYH%5l&HP2rm`LKQ@;mWyCrPm#9&LmkX9Sltk)b*E3#F1d|Ob!FynJ zLVh3NxwyP30mYgG))_fiA8?HyO~#>}3S|;3;_IuE@+R!{e*3M+_wy>`j=wz)vu@ZR z0N=y4RmIABDJvNuZ@>4I%!bzS_jBgRMpb>Gxmy4^be>*tJ*rop3>jPsLGxnC$7#Su z7{nzV;Dv0QZykXb)?%1n`BNbQ6opD4tleqqMlyNgK&*?FgNMO7$>V#W35z>U>T%h1 zQI7ziKwrOJ^1*xW$_HQ8!Mws`8NU8NS-E(o#FMpp2|O>BCeIZsu9hh~Pm(`9GE?69 z^b48xNa#8gaKs)pv6leRs`!?(zygC9v)GLt|oHf&()fAx8tBxsPhlC zIT=<9W9;nT++}7*=k4!C2sKGSx@)%tx3vn{7~Gb^%f|S&G<|o>pBIDCnccS>cV+I? z#a$pf0((qkXj(FU5$+@-Et!TK>~b=GDQqt(G;FiR^2;S&*hlRc( z5-yZNnp!Xs4UsDJDddPU)x$ginOOEcwRwj+6~hi$T>`hUf|w@A0HC~sae={)9KZnz5g=?;>Xif%757$b)Sr zfi`UtyX8DArG-}{q7p8?SSV@-6G!nlccu|UeDO#fwSTE~!L%ryGTL~f=57SGD&j!$ zetU0#<4B*&qmMiz2OP0AgtAL8{D%cK|#8gGyL+dX%w&D@{-WEup-UTX2 zM70aJiwS({1f@^6yaB$%2@3zV69Z%;`13msHue$cNm^PD#TZh1;3Gm-6y* z*=Cz3ro7ZTIM#Jx z?FxBye(E$VcL-qPtBkI62+-98gmmwSBaV=lUV2H+Jo8L7)ljxanO+oUMW6|a&p-bh z_Bt<->?=h0IVFCeXCeTGjT8_@zGbZV9{(?pR32125k6Y$&apsIF+@s-cTpM zeAvp>c^*OmPCHTbC;qq=!V*WF@)ODkus^v@al;Zg>9tI_DpXrUuP@; zPjLMO1+zs9=L@Xs$$-KA!RtDw5NVH$`2jOnk{~TE?e0GS$}aGmzYxkVTx&r~_xm5_ zQZGkFZ7>2Az`_SE8pZmJ(r)CZ#YFkb%X;X#s0tSp@*>MdAb?6k6s$K=c?dx0mrGzQ zJWnNXgB?GOOMk4h7@weXc_-Mj%KBu%x)%o6*4mlSlAy;NbqOeL<-+7* ztZ+DlyTj0z`zXalFLC3d#qr9uohI#GHD0Zy^;5RhnsN8f%q7BpfoXjdHI$9YBAV7 zDIhJ*E5e0oMJOva4!Gk09)R|2X_|o>2)VF_ln07gU$IP}?28GwPJ41LY(n)dDwAUl z+E?EC&kb_o(dWt)S6(QiM&dRzwc(p1Gr{Cs`};r3oY{-yp$Gne`s=K%`8L9g0<(iW z6l#>{&(J0o-1@2o;CNA6ynMn7y@|qdnzU$WSOyJdqLStd?|dldo^hS@Ks%j%+6huI z=S!G*NTMAe2nVt#z!#L6+?{U5fMKM~HzfzEq0G94uER0l3_Ay`_6N5=DdM@2hJJ_8 z28_SFTwH)rfB}=&7j1?9#|;A;sEE^EI@IBHTMGbEyUm2W6w***Ub{P070 zx@VQ!bq&cRY`>HNr5HcD({XHhY>K2 z-1?`mmVOi_(Xa2+JF+!Kl%UwIns9*&Ye3<0@8ENfC-u3^RRP zGt#79F8Dq*mxs%(b5)TE-&B5Kb&x)n_0=cN&!J6ou?}W9hZu|GVgMhR975SQ0bf;(^Q5{QzXk!gmns!`m+ zyw{Ovp@6r&4eb(V2MoI+2RLnef*3>NFt?Dxu3TxXreKYFk<>`Uq{R!^@S`>-3278U zUrtGNus51SC+EH)Cy5*i!$8nEx$x&-pqkdDF~9b7%F?pG?Oo)yB~O3Jx?lbd0rSdj zlQRSdbf(R#?}&;PMB9>GK)CzVJpnQfc&V>h0wIRT<(J(E%z?5*f> z4KTfcZPy3{wAeosj6xLjz+cKHZTD~?w|`yhJ!YwBVM_y4Pk6!t*l%&2Lf*3 zp^-b~)E0)B3mjjfja}u&7#l*L2*w!AKO`Y29x}9_Tzc6_`2A5%Ir(~-Jf)9pu;KcQ zD6hWsw#@n*7aMGvBbQ!$1~h5%WaXmyYBQ^MNgp}q%wNhmE3T09&wW^)c=~bK95>1q z=jTGWJ40T3sT#JxM6SI2XEJISgy;=Rr3jh?#ke^Z1~!!G%4x*r4RFgkI1W#%o9E{S zDUGoA@Y#M05Z5rYz$kmU*FfzrLI`0M z3~=01V8S2iC4X05d8It~;Da(^#0c4EpMB)(ufN7^&7Wyn#{cA#Pvq{q?^f;JYp=al z?fBBBZ+qvNRSMVDJz<6_P)(3e_nmZ=e&iKG*Bk~6Yt6-LOL+SidR)ZVLZ&Sx z$eKnM*z6_kb1YluYteASxBGTx(k^P{PM=nU78U1h@yxVNfjK(RqhwwQx-FCeG(bizb}M?c!3s4aRE*M;kCXN z>xx?3D}tY%qJ^ouq8>fvRVcma8k&kCTsyG+uoj`219P<3OQJ?WCh3le}$pv}uY9h#D$ zPD@Tbcq05F)a=E-l+zBWpAP*;33Ne4QI->r>}!&$vEY}Y*aXtapD7p*wup0isS9Yw z_>ITP@Btz(J@>ZMRUHQf2inM8f&T4iCzC&xh9Y_I?T@7Z;5XZ3Lwq;DEZ=h3e!B^B z-S7V`4?ps>9B}AiSj{y^Rm}=$LBsR~(j}5GTL(q9Zy+=;gA)8;ToT?`y8`RTT$nq- z4@KpjE_6`}I5<)02mC?z;E5;A+wte%U!$X1Eq0t1Qs{D$(Qi$$+v^q9tSJ(;1wZjn$>`dlhJ2D@H8aC18>PxSX ze?IXPCNgUEVi$u%6vBuIcI`;mZNcUV7U4_RUgpUtm6B=#<=E#&MryP9TdIRL7 zleYy^FbS}#DcVH|DSWM!Z622D;YmyK--f%g73}=`2-EWBO`Qx2rtF1T2M$J+%CMY+|LR8oMAU%83!A@0N)`wkQsa=k8k$WR8 z@VVf(PHw*Nv{#2IO7jXoR>gqflxn(kuj zRg>+GJMNIj9(zm6>Q!aXakY z#!+(sRa>aD3)5U7Iy0OE!G5jCNAco|??+KE%onXSuW5bB`bKL2XxzY@*JiwRc-{QtFe`9h567AdH%Va0HUz9PF3g6OXaj8WE?Y7e z7P)ZX9(KlP4v%0{?P2M&jjgO1ccB2QL)R!$6v993i9mQ-AU%5Yk%BU4R^W78UOkkg zVE^@puNH$(e@fndZINuXSw7Y`xbr4wB~IF{kpBH(E)Lar{E2_akwh$ zfEkWAUYIE_zQh|wXkiVDB$$}f#THKf#c;76?ljaBM!Zm(8nsl^p}Dv`sG?%2hCYFH z5|pr#w9{79C{wT&m^@{e-2d=rGHiqUmz*wzy`bgHBbwxeabM6Ae}74C zxZxAoeuqM=88?73PNi-tZL;x5n87HLXP&K;#~yxJ_BwD!*g8gAP_SIwPnLgID_324 zoxJyos)Tt9 zmyzskSnms_`~K3$3z_%i0XQrXon_uX^#!C5itaG7z=jF&4N5h+5jO8 z{L)-k=>utnT4NuS9)_;X7_iR7Cj<;8H6UY;j+w;nLrx9|K8Q3maU(byoC_^pNiGDT z2T{ZEjX}K&-7hqlFEM=x->>DL}d0bP4s_sicQ z-d?@k1#(Kj3*o$rQ?}hEM@*7kr*45s4wEX5v!HSDqreiAbEIC5`{{mi>`xED4zaISrN5#lo;7K7?3>oWD8T2*Cvt;Ua8_O;`ZYN)Vw^Ypr3>;D_ zJ<1`tgxc&Me}AU#q(YjEu8H^%Og7>YiHk2jU9}XF&^o0Cc?3e!CSXG!Tg%iPw~<*tERqFFDrCTr;kZx$c6JLPoUUAk{2;{E zzK>mEFvN!*Fij3UcvoBsm!oiLn~Fr)7Ok{HrNrLjN-$QyQBB%(oJ`(w3oReV6}CEn zGvH1aP{bd5ojZ}DY{V;ZnuJE}BrqylP8uf*=U2+?ZZ`%L zv zx8X3k`nTtSsR83w?c82#?XPo_TQ(<=Q%?L1IMge2ZNvZ?LmR{yH`NfxC!)AaEx$h& z2?KDCAQ**wnq|PMfn(3_*p2}&e{lHTehfI;_s->MfB3^6KpT^C%{A9ZKWO4)a=d)) z_~VaP;qIMx-l@lxQ?xZxitg`6QfBjeT5Vc8w>pH0(9*zk)`yn4wPozA!lc0^(QQq^ z+<7DoM)0HUI3@J798k2h?_gHpcelgE!_uXaJU3b;E zTm)Mf1;MmG&Uo~r_HJmRVRdu`8;y{lMRB_qPHsh@Wx&&71#zs|z%x^OlYF9c;w@Mv z(guSO3=6+ogMlh+cwiXFj2YE(@uio;%mJ1@xIC?@5_gC|6ZefbK9kwsiA>rePp-M@ zJc(lM(SX~)YO5P$qm9PMA^VM#Cm;V>&N%&inLcd?$P=q&#wVZ2qffkraW_Psd-^+B zierj(l|@oifV)Ys9>{@Gz>=jRqoLTMrvafb!n;4%-drmbfpx5a7QPffD2atoBTggc z<@b=|aDwuOg}2Ey*Zp0-oc*5cu+v0cA1s)+Odfss1)1?)rCf2*V4RwSfSPL;DDX7k z0%{6BtE(46@gSzB5VPTja)1c#R)O6I?RU#pEL5`;)zFlU#9;OhU5+$IO-p0~g7wf( z58YEfn(?vRcE`K&)px&=op+faBZl{vx}~-9!GAxK+i&|wMh+6W;M^0Xv@}PSE?EJ) zx@8iF9f?yO5i9P2cB)W(HsBN}G-OgGhN75Ld)!}xOLX#hLpug^9(HYNR;&PN z^|cS$b45MISqvski_5WI8;teZPzXP;Uc)7TD%|b%2wAfg2nD?D!M_S`TmX8^fI?n+ z23&+Y!-Wuh730RtGRW$_7OD9LQ6YSSJ)q-{+gYlXVpw1YBL+KA6n^R95D-CXIg)wc zVv3}VjGx~bBkA{R&AZx-ZB2qe-1c7b)-ChLLOQ)2@C{+6)kQT*fhV?J6m}7!Jl+NE zTfr@^*nf4p2n32b0__g$Qo#@2g3cJEggH3!62Wd)0z06l z?HWaV_K95dhg=9xH(I|ZlvuIj3p-HN%jZiWZh=){XafYQ?lA~99c_xmP*(OaG6%c} zEy54n?yB~iu4c1=W!TgmbreCG19^pN<&XTZRsXwr?qRm}@YRV`G8@ zJqg3PdYlP)k*H`SC7vzEPW5=A1yMQ!p zlIB#HK_ApXg}Zzr2MB2^m#;wDg!ICtbmgVxs4q5JAjGb%Tnr`zc5}gG=&^59Cjmh= z1+f(V@fcoh)qLqy5|ch9In10JxU&we;e1SBkywFa zrwaZNY_;3Q49lm^itm-%@pi)8NiS2FFWNf&>fz6i1sAt2j@3>ZY&J!seX-#%N^(nv zA~Jg&#vI!i3@^u-I{jvPAH(5~V2=mFyB^gG(glWj5R8-(XUA!}X=sE)fG=)b26x)Y< z0XO^co&%|3-f-MYQ7Z9r2pcNqVgr8>+6UU88V)`}2eXY1dm@-_jm;fS_TBj}PeyLM zaew*v(|ORi{Q#%rXj>Q7Gw{p9FxVI;9uGTwjQn`|jxuZzPBm7uO&nK)_p{GC0qu9a zJp0UN^7JFnt{sW=(6QS4z%6uRWybp#W9(Hyfd(rI>apM+l12z2$8J&zWfh+Cf>I6n zL=e0L-uv__mJLTiQD^KxtlySHL$Lv473(9I$=hgLU%Bvtqvfu99*}om|3Y4S;R^^Y z!EK@|4IL7f3(wnA_L{yOPB})TZsigPqan1CMbP|(FnMTCoW`7^Cn$COVUp9m%lzgvs}ceUH%_+i8*F~F|K9VO5lzWVz#t&ZnlcNp5dBw_V9`d$Zt zUw#Zk90IaDboL021vx73@WX&|Z0}&Q0ilLW&T#<(m!63mhfD~aKYzz()xRl8)T&`E z^09)oDDtY2x?BE6yuEgCam{JW>cU9}jso6B*i_B|Ghe%G366R391sGCGDhKX7i#9x zuMwV@Afd+#9aq>z9S-=q$W-FuSVE`A{4ce1UOC$vK|PlsfWN4?={r!MR`z9PRJCZsv{^c zM74{*lQ?!?>sKsBKSlq9hI#@QnMHZafm3}2oP=Pv9#^?iLzx0tycxWdtq%>KOTo_cAJG{B7jPb+sm8kRqrD@sMid%zhE1Kv%3vb|YG!qy*Rd*vZgSSC1T z?YuhuS?k7j!`xX9g$BLt8Yv?@A=E+6>0{tSfHfXp_LWS9ogI`-r+^`i<3*8lL3+h+ z`wEQQ(U!91W><5J{2@Sic5-hwzR zQ3*S+V5xb6D)&cJb?G1PJ1C^HWuTB+}K?ahcKsSVaSgK>HD zsV5zUhx$dlvUC#yVx5)Kn~#B51jDt&~Hd5@^%l{)KYY z#^c7xpn*N1Z1MxvZctFcvWDg&Mvm+)=brsjDT8n`H;%Oj%@zcm%{8bG z?RUqE%dq6CkUe(YPPV}Y`B(^E>l;uv&|vZ&+2#PJZ&u5|0i{qPJ6Lx5(G>ag)0t9J zRV8Jyxb*8=C>w8z%ei5;ApxymnwR4RcJ(+ekKPVpZ=MW-@b-+;j*+=@=VQZtmeQ&s zoQNDc1oa*>49YB(IFb*$zB)LNBTh2z08@R#Mht*(9pPwKN5dF>pKT4Z5|!AvDuZpm zfw0@V`_$p`#aFM(*Pp(Kc8n?Q3!=lnCdzE0A&|oG7x3=%Pqf!s<2)Q1&9Y~zj>b3g z(mitx*r+7`&be*E#s#wdl<_#dOQo|K@cl4{!i#{Y44HsQO7PQ#JqO{=YueGqdM$zV z8cqN^v=cC0e6oLM;t?&+@R=P0p@26#Zki8qRupORAmU0*+QD#g`qbEj5F%F8M{=u% zL4fr+=CuWwps*W9O~HEZC_|Hzd>scFRvcsuZCL3sbzhFbj<3Yt&DSCPmePG117a{- z#?n5UeFp0UX1`S~7(HA5!F=pHn4b0wnsS!Tp24`m{Da_v-(Gq*AT7OR;ylSHMnOI_q`8o%F=C8c3|2hdO!%q}G4LlVWA2!bbiBFWjr#56mZO*JzFzp7zH3E4C$B> zwEznHh4wp&<6=^Yo4J#f1rXTbnn?JQc39V8G!sZVkkfixdYJY*i-lv2u&KTr3$Fa~ zQe56h8iGD+fbHo-U*#`H2q_I0!Vw&gq%cS|T6{5;33tbqG33qwNJUCb;WqZ1Fis;p z71D@J#9G*oCqlR}ND^nm)qkm|lgeLcq^XJHSZRqYTh4mX1Qq5u(s%rIHFX^~b zm|mEZga2szn%o9*eO zWPRzJ_RR#%$&G!Kmws~Fun~)Peivb(oLir-+EUCbiH^g*!gqE2u%GMIfNv<3N#ool zQn;dCR?Ne(+%zuTEf?VXUGsMt7*;G*Ldr0$QHTr$F$hqcqG+hE37LvHVVUmo4ea+A z53n;?hfUAgItV;;L(|~eGnk+G+tZI%$BzOWb9Lx*IoOab$Aw7=NTH+H{5D6L&|XIv z9RY`QiCVX$SF{39{@MH28yC!I1lQ4{6HyLR)1P02%*_AvuZSL}t;$+=y*<@UAcWmNR zBG*rFspHy9q+$62_%z_6X%I%7dc>kdH`9{1kXfM6Q5w&c9kw5WM8kkv&|>oL5abpk zt8FK3g7#xufcU|j7d1k0O^$3bst=lkn^(LW#OV`kxl-8M-Ey-LC?(9lVQtO}!ML2@ z+AfI=m>AY?F^u8O#}1IqHyVT#&ZZ5^m%tqzMG)e0^&Z6OhcNS1B6~?Jvn8s07XSK_P}3kO6cW0DTgE67ynXVmZoOD;o|e zkg-EI(=cA}t?M7$g_6W(61!j_PC6#Ap5_vVwt`7!dldr5FikP2ytizzVPBGDe4z-O5AB%f0wV~4?R^U5DEM$q<9n5xggIFG?@cC-)7==GIY$uOh=oyCTl^1g)0 z&ZvJ<)L6af9lBOwAT_dDl`M?xVIZyNaUe7^K&f%YAGQ|Bz`{5Qj(+rz2#z>3G%SOe z9^A$W3JWt(7zpU#O03+G#7zCK$lC^)<*VOJL(1=q%cP#m=I=$w^1`|X(+0n@Bu(>f zE=@3>;CC>6@Y^!8boMm==AJ=#!EZ~S5w4FIG9J@fdVhSIgmi}X_ZeTiF+P8E`gFt6 z(jv^@^Qn2@j}m5n`}%^#gSR?^$)~@+c=3>yd3oQAho64CW%i1W7lo&P6#OcGI2-H2 z)UAg_Rd|*UK2*Z(0%o}^9s%-QXk+l_?>d8@tg8nC*RP|$Y6W;!<)0M1qL1dWCFPmZeTnJc5|o_ zp|*AhN-i%@%Y&v(E(E*@FceB-0f2qiZBh@XUb}cc;Y%lCfP#zSVhY?v`t<38ix1zG z#fuk98SK2VJt@?n<{OW8mE#f*UOM#7JMTbaWUe0DW4H-;JF%FVL3515+)8d)Upeq+ zXTrQv6^u654}eTvU?L8XCAAMe^H-5}L&!Gb7;&7m4>CdG<|A_UAX z`z`*bj|PL6dJ;0(gt6<7mi2}XhelAI+Hbcqk+$d}4simd@PIT$e<*m!!^UJ4&Ff&{ zmj{-e27f4k#H-_!W}?Uefuj=${-%5*8Mzr!K|6|ftKKyDHi?cZ*&y!lC+zk*V@)E zj7e>~YZ6Dm3xR~0Xv&B1voAnGXGI-rOw={it_|1>j^Hfv7Nf?>jTc`j$-X_LE>$h3 zZvaLR->nw#VqPsl*|zCF66TjYu&F&?9|E18u2n!d`T@+~^c{qI1^N$!G7ji45ygsm zjw$>u_{}h11{xLLHmC9%M?57TLYgSdB%zOyKA8CejowB6aHDF-kRiG$b}hu#u4?;4O+{ zQh?K3g3E~;V78&YZmBNODE!lnE}awDp6u@&RD{HS?LPIJ@9C~78k`;zUs+(J;#v7K zjk#2;F7t+YL(!v(Iy777PAyHamHvge5-EZd%1?p+@tb1%oH%l#T;A#Cz`zK61IL`Wq<*b#~AmWE7YrILIfpyDJ zKkerZG{i=Hq@z|Q>w`?0AN_SB+&`Yc3+hG+;t;n{oH88)?P_P;1rPEPHo>wS>YS&u zHeeJhw*^~$-0wC12$m=K-CO~!-p$49>~NHuGK4e%FUsZ<@M;OPIjoKhXZfuTMN(U_ zMAvGK*epxnKB{N|?Cj!xlt_L*@D5n32}~in8!Cin%WJ-?L!Sy=+cDs09or5iOzFdb zb#|XuwpVKN77m3vBzEZvu`5>tP2R*e7*xxoF^U^0b%zhLibwPTTgQ&Gx^hSz!aZVW z86lGA=MFaS+v3=>73H-gX-^k^nqO-^Rzv}uO!S{#z^9sH*o>;`NL%}J6Nz8@qCNc^%^`YFp52+dPzRn zcTxrZCL!9)Szn~b5CmXc=E8QNo`kRxGCo;Qe@TNmWc#)>s9B69W1A2gG`r(xYqQkc z#ofl`{bg+V08QNh)Dsd@SpE5=v#0Zw`Wq}A)0<)2eEP(@cnjm8g*4pWZjt&(z*+^e;`+BT!BAyhc zpj7O~RtNZ4ZS^(dcPb>UNBMa%+yx+cJz?KJD)Z)lCqsu0^=N@V^_qU?aa0Oee|NZxc0%DqFU>g3oITpIk#ejcI1#=?y zRgQ+f9mPnFP*a^J8F2GgZnVE7a8XuG!veG=F1?aUP#s=IOw(W(u{t0z!u$dfju8`% z>1H;+gSj^K&C)j2CxYR=xSEgeyQyrtVVDUst?2fhMwXQJ`f#9gbNaWpX-9m4H+SGz z96YW4ETay62HG!JJ{HLYyk5SjVYaos+vZLh7ZV;r+!ueppclWbj`sBOTJ?n`_MF0U zzS8_Hg5||JP*6*>G9vx;8z25WtzK*gung43)_5K~CMGY@Ra~Gv+J0>)pVfO!R!jRC z8phS2_xc#y$1NY1S*lE+;|hg^2}54kPJV#nrV%u*F%Eti7Xn6NIJF)bAn}qR_y(f| z1syd>md2~C$qKKXdHXrxwX=p@x>h6zSOmd_rxf6LDms1<3{EF;6sWdpwp72$tO zt&8tF0L}>s=7Vn&vf0FKAaX3g^{{aD_Q4^RGPx8coKQ<|af9VGe|`Gkr%Y$SlLU_w z799zm&SLuI{P8NyQu#~Z`*!14_3W9xkV0yL_u%0@`0ZabF?WAld+Y+&Wu7j6{ZwvN?@zu&_a8&5xi|eIlWcnL;H;$#VXE45g`{3C)vG|t1 z^&d0xZ%6u4N%6)H%^z)Q0q#@bVZt|W7T0rrPbNjXXm z7a*!q%=nIFWZF?o!OTtom@^EAD4OeWsZ1fXQ4RMV{U=yB`*yyZziD%c8(`BdOI?o! zNkB8Z5keuh5i!Zq@J9?dT3~pgb5Y8w6yQ`B;{bblaM2gc*No1O(TouSo~x`OQDAm= zl1&6b4+XsG_*>my{;{fgCU;B0p{*KH6r zbZx{yDBx`);yNZ7yEcBRbq02x2BzaM-B8d=@_UX%7z~)t|3(^cgs`?|1;XpVMBrA< zT-ZI!!*MU{kmT_AvTFJw7z2;S=2Jw*KXkZUaGj3`@q(<@I1_kT8qLo>TJTJhGb4Vs z{Db+}H)@yW)V^w(kMlu#MCZ*HepEO?`nMra{~&yW<4cnEYUMIdW$sZ)|3Z&phNn!6 zLQCr~f6kZXYw)wx!{Ql?U^OjY3lGK*ka3IVJ%jO65?L=w+KWxVmd&c1cLTIPIO#oVU}QY^mV!AZig=9Jhr zW6bU#Z=3bIY=RemoNoPY4X3n=A7BjlPP@8f5`d?AFbOr$V(foGW2~UB#ACf-Zxcr) zu@RQUVv9$6A~4pkBov5NVV1S0wg*1=%|E!#_}bx zHl}wr2Yp2FEZl^l^Ci}A*2{kCk^U!D+;o3@(Gm;cIh#P-JWIhzZGJNj{e$_^ACB-5 z6obaBI;*Q2oySc5{_a=yhs7sMHV907I5z26K3>Tea4Z@<-TWBp$*bKUM$tH{BxZo) zG>(0==1Dq}y(I2p?gu(Wep z3r*&D9_(Jn@@2Q(c2gR0^UXKwMo0vPFexY{jUxS66V3#-WZF7SaM!=)3U8T`hX1z4lKHR5QKj*vAZ$=^zwY8t=KEuFJ|O5tDMP)@5i<1})SovA(yTobN|_G?{nd^l-c!cL2w-j2wIwy5!_ z#bxYzX&Np}i^qFne#^(@eC6P$%E0rNLH7phwLEU@^Q;io6L@3ooKfVwvzKS)4iM0s z3l7IN18Od#?irrWJ#^1HbnjA=D0rQe&A}s}6k70!o#1|bB??X6c+FC2s9A#bRwZa! z1#XP5ltCN8yo8@;F9u%-ydVgHL2lKb|7B5&{^S5VuIl-v83nea4C=jspW=NmJY*pu$ z&I5)?+O3v(S^0xUbPT@1T&4U5$c+&A1hKUo3_|)d2tSyAdTbJEr{Gpz9`RM@N?5IY z7B2|j!mXTEK7$w37;i4U)y~oze6QTb47nawKZ|cqgA>dj9moSkhRon=I<&vl!xu?k zH69oi!vj6dUZ%wj*3JC8+tY3ORIyR-Brh-V92Yncel$XJ4FcX+ZXuY25-Hv!`I=j5E%VefHT$o_+RNdGpOT^~U&q{raiK?w)(@DI-RVP(cWV zA%3__AN;0YXF0AP(5;+)WQ#xRPkicoW{TcinH<)Zv6z8T8T)P?&Sn}BOU+Q9aK zrw14BESvw^7foL?mCb05Jy9BJ!&)i*THe;`W+BKacZMtpD^p&<^0ebSwSdEn#5!A^ zc(Qz?;Z|P06W8Q%G;Xu`T1tP!CR&L3yA>WgcC5Vp&YN=n`RB`Rx7{W$zWAd2`Oklr zZMNA4<>Cemgs14Nsq*{tShMd&M@dsjOG&%>_wTPK6sWa((M1=@vSrI;!wol-lTJEG zh1iTsGYxbU^in7sx;2FX#}2FIZeurX&B5`JEW`+>9o}Mz_sN%d{UVqyU5NXBmrDY+ zq;nuJOJJQ=-H5dtF6x5l9&sH}2RFY+tyYj0y}R(8fW3_>T?zR+1wOFh|w*3 zk}7o~Ay&GFGGr*9f1SWmI(?8luO)(?yB6`UX`Oh@)X%Eq_7i7wBZ!N(eP=AUuYOv7 zHT&bhXWpD&?2AvYJSb-(FyW{cs1Kf-a1B2K_-<+Wy39CD^>+`E+DACwvT;y9sP8sl z{;k8-R$d|mbNI>9wl}?)U+ISL2w&qZox7Df*m_(@y5X{G$g8I=~6HfRCot* z;G)bkx{hy>8C*b&rxWI^R@2MkpA6DNGJokf_XV|fum*>Aw)}(ncqAtn5Fq$gG>dQY zH~Y45d#;jy0KT`-wFH4@AUZJZI+v(BT18y6xit?aNTqNRdXud487#jEAT68b%ID?l zCh*E<-ACf{as~T?!LxE2(_-J2&f+zf-l}Km zSK-erf(y{iY({bU+ACdk%FG{U<5CmxTh;DH*p)D~O0a;0v1nQ*ox0#(vb z6Y`ocm+3MIhzXzhP;j%!CYwlMVWHaFHNkQvZgi%gnK1cW|2hrC%+u_$?xMI z&u)VBhu0qom?`aN(d?VyUgj2z&4zv~)3~Fpvl48_SyDD0p{UcqY!7f|_u7b=_dlkpNzW=6s;NCh_%g60=&O4QPLuX^B7?C)!vo zD=QXa>`=%H!7rW-wJ_@d9@2x`k{Y`_c1xlzGM7757pMHbZ;S*_DaIRL zss0uk>%pFiAV!5iesKO~Y=b;orluw^Itdau0n;Z?}IxP2vWXeBdJ}wY{T6!`6pr7UEhvR>@a6W_iXM&Z$qV9)Oy>(R8QMWaG4&5N#DJ3c0B`FO`r$~1<92!Mh4qZ}8cQ;CR zN_TfRygz)N``!DE!SD~pV0g~nYp!ZQc_Z9ZO@O&+*Y2rjrPl}7{jnB4armvNV(Rd z$qqBYE#2R(1!3s8Cm9}A%8H8zaRS-N9}j*|Tb+!zn#!jkC+T9>9Xi&-n_Q$6$Mc%- zqI2+N9Hyp!KYu6VszFeSyzJsbiK<+0eaoY-!}h(B2Y<+F5L8kDosXEVVC(oJh1JNL zng3qO7%k%Se~Zhr2Nh0T@|$k0j$an%x-H>zm6m_L`I~BKKruB^t-AIy{mJq zl6Ast#+h*^!4qLwgZp=kU}qB(6EdFQA8M)}X?w`L(-an*m>Sl;GHsj=E&t-IMras zxEFVMAiuV0*#P334hLE=@_PX<65&Asv36_spDTH$!xlyZpJ>XUQGcK?2V18pzcW=` zD25+y2CAR-99*=Bd>}#l827cVcsV+=6k2jd1>R3Od zt`a(;pFggd*xatVYIO*rsD#$qU{{wdDZXs-_z62>{U+?k33R`P!6tTFBASlECxAve zcX5%FVU>@*yN|J)ifU;SR)OJdw&K3`8n;V&cz_+-%H^@fE9K8>7BdNn_OqSwe*BbZ z-R&;aBD&~v0?l#YXGG!e5i40nVrYp44l(?LnN>biZyk_fzGT^chV-+hPLH4qP>WhZ5D{!3-;6E~QN+w7N|bfnhA zcpeW?NE0Bv68Uhy{0v2;@KLdMT6UYay$qz7Sl}iMr2z!){P{vE9_VZYtPtb zI{_l7D2nSZ@#52p`ky7vHp;Bpm6C@lbIn5nCp|X^L6=l$o$HQfB0;8;y@yZhE&-qsL1;=T(98hs_RdE$@$kXvB$eGgX?}e-{z1!2VS9jIcoVb2RM(nH zd-q{e@%Gl)V!cV1vMX5_FJ;`=YHO6g`tPcger9!Vl;NJewbfqq{wUmHZp820pShCU z59jBo6>mS#ulGbJiazctdl9WuJe?rU`kW7KrdXG%8Z#;*;sPND7sj|Pi|b?lus;nM zO=RKEpDF^f-c5$SmkFb8?27UZLco+{tCg}xP9%@oDy#fvLA>;_Ku&dTf}2&4&L6H< z##~7*6NXX4y>0XD)vZGXV_J2o*oIpS8?I^R7dtF-M3LzHNDsnbyq~zo+GFbjM{^}u zGAXv`=3=SdUY z+}IJ&Mt;%X8b{M>025C0BT7x<^SCt3rV*Y=AMYd(H7!{XXAa?;7h8PQ$d*cw1zo0< zhW-}i%F2j;y>@tbEPpuO{iSiNDU+C!vkG4^W^i+;NWrXp;;kRHymVK#dr{OR9CYj2 zWv?8su$vyu>ST;3@CQV2fXk!uE-@dI0WEJiOFU=;IE?r(3O}uoCi|94D7}%X1nU`5 zSzaK*b710}rI4SV4hjwlE3W!vCRSI;<0=dsh!E#pzrIEv)gML$2apY~gm))s39~eH`pmjTcdr zuRB%1{n2?1TiQ8deEOYYhn@A@biRqO+Re$#o6ULbq>>0*EK4^k=_(;P^7~?S0kK2B zC`YR{rDP5%?oa-Y`-TI+v@2cP{o?U=d95D_E+>VC&UpzlpBri z{6gV(Bc89ES=y7%IQSNQO-cMSl1oXu*!$mcUi)cZ^SYFut>mK)!$zTcL6!ndxY9D; z5TvsHkVHEtuWbxEpr|nBrg2%uEdaA1PTT43{ZpETEP(7uvmPM$eRiJttOs9!3oW&i zSv_bEfZaSF5!F;&2eugMM~Iay=0uC=@GN;7VwX2M@c-|Kp}q+F3C(%~`C zNdBEDip!8>*Eb^04x6Fq zYsjDzQ~yOXRG$Ji7tix;4;&_ZOfI8PY&%7d2%GM_RZ+=@qHE2zyM*TZqb8@_yu?C$ z+uFHXOVwIFmP>QbdO2yCyB_qP`=w1vAwPve9Of(w`oM^!S#*y0YzD=fY;0^ZL&f=L z?BE((4{p4=JmcKd=0V%ZpQ=v1RAIfqvD0(8iVuBCIiNC-AoxW2TkPe-yc~v$Jz6l# z($S)y@$S3-9vEeKz{%^Nuve^e4doO9Ha`!_yD%z^I$-oSQe(a=3(PHV=9#Y64kq%XA;K2#PvyHPo!k{o_9sY9igbwCBMi3X}y}WZkgZwwJWWT zW}e}h&#d40!P%U+$s;^4gUo$Fl}e|H;71w2Vk(Q!s-sw2nP8)z&& zUhUscsCkaIESj}0wH49=gON2|n4)^JCicyz(lvO4XBB9;lIp#rC>~YD92c0t_5Ihr z^a);0X}?xctMY$e-Y6Yf5-6w>?&)R-UWDMpd23KLNv~af6p+k}Mn6xoli?c>WcUp% zeuptI=G$P!7Ntk~?vt?BvWw zhHukQ?neLj(5xvC6h=w<9BmGq2AvP$Lu2wCoWoNdarKsfOy;bgsqAO^EOM$+@7Hfv zNF1Ii5D9G*r(Ie3h5$DJm0}v=zOzo%N26ct-_n8t8YLg^F82#A`|!(Ho;2`9&xa?_x$RP7mBLv6_fZ8C?)VWRnV%fZm>GZh|DM%vv52=usNN~&H$9?H z*)WqdQG5~vr$8iVOHs6ljtPP{Cf{QC%N-Dk>`gRXq{%iEf92}wa|?(se9S%r5((C} z$3y#%TJl40Q7Do!l>P3>;2W!31dTi4@F0ITt#FUZ(GE+(M9WTuUp`6cmw>czXNdrl z46e}iB^yUfGJy{vjlc%j@-(M@#zsq*(?0|Qz=(8e1xGHog8j(1SPl;kLlkI(v6MO* zT{G=tjwsn<@i)I4A4~z1k8RIS-g!Z2buJwqwzqW5&eHM8%NXqQU8uaC+Ka$bWNbR9 zj6f+Sl2%+E4f)?Efd#vW`eFm#HJVy;|HlTX(Nvng&1t#dmM%Gu2%oJ*u=QW;PHfVP z-hM%$vN(nkz?x$7RWslizDAK#@DTHutlx~6@!Mrc%Tr0@T~v1%kPk0yy|Zc=O0T70 z9Hm9;E}*_T0A%793vU+04eK z0~49RYA2-kcM~}>m>~o5%Y#H@TuIWY-{`bk@iUe0RGSi0Khja!DSAGXbF^^ULxUvD zIW0LoPK>x0-+6BX!M-RXYQ+*|eA|eqa58scVW85Ix+XHfbMT!{R7P94-H)(%HfttuA)V{hLERA?7l zZd>9k3j?f83Za-3rF4WIgUOq@ScwwLFE+&Cup%aGwb%7BZ2$PuTnV!bVXk*YuCnpEvcJf|kp1wDKQPuRBrMfRS-;1zTVBoWxW~t}wCr z_1uq>ceZ7mUcL12Kk2eA?|ConF5*4d#qGTvv5VsqB2(>hCzxsW+kkl9|N9Hh=1Va1 zf4*PNh~ZBXdBFJJ?+RQ14}t{xQt6_I@$Ua*G{8ATu($saB!h?7t3M~?I6OL|^F&cr z7Pfs?iRuq!)h{HVRUMVD7n0UjQ~S~O)}(u9^BprU_c4PQ#y{`6 zNC8VX`C0K>Ms=-*0>Mz^dtXPB~sq%a2kufirZ2f^##IbACO~R zbaESfnys^69_|$hCi))}_?Nj*zZ{H`q#csL|2~&uMl_X`_qJMApA}Pi6fuNO#PBxM z?h{}PSdSY1)?GuP@s@x@APO48M%HLTtt(^sBA|M;$>x6>G550>2n(A)^L&)uYFHIC zST7Xyuo8sXPZdV`!;Dl(^wC9_RUrTko)cGyo!q~9F#I%`cRAJh+x{~*fhgKb*fSb3 zhSa(+-c(nN(33qrv>73Lt`hq{@lC#GdBc2Wf!IdCs`ryCT0(zrjn+3;_~fNS%G)lB zh#rb87Uao&X-iz8FyQ|5P3cU|U*s@-c}u1~H*>)v$AtgD(o`)P=#|I|2`gI1}&sffApO(0>LrB zI!sG>M+l%cFS=kEqaARs6k%+rM}sl(LjsXeSP;+%vv|uhffT~4V=U3o?^j~ODbGDH zQvA^)mY5741(!n|=+>p$q0?WaYE+HXk+av(B0( z>bu~4OH>khemEz7ex4P?1T}r#d(OOGn!Z2?4i=NJABTs}bC-ly6>t3M-$c!TgpY;w zO3I&-;U^>u>8lGEFV;#t7ZD~mCML$9?s(q!`1vvIe(P|tu4#7hz^*q=ux}AyEAiWJ z4e~GZJ0m>()qSo4)jZ;@=$@3RiOd`B#$#~hWsC}tGHI5&Xrhm90Ky^V-bW2Ro(vB> zV2}IwJ2@R>cx;r{xUvi}Dn=^|G4&RNd~;XKn-+1cEJu5;)3qLbA0+~UVUXbw!0>DD z_fp>M*-j1C=+`#wiEcLPR|f4MK&U_^4C`!d50(1vi>?^?eoyC_yl(QCUj?@7H?<54 zS%4S@b@KwILah#KTjuw zAs6FxllBa0=%e@<)XJB+yQ z{?ByR$NY)lxG-zUfZAB!ND*a`Jx+MF?Ciw+$qM=J-76wUR_QIMBWigPvL0h2+deQ# z*PK%E8`vn#L5x`2@-1sYjv2F@BDWa%pPCAdv!&bb4(|5L>b5iemxlSbNtesfuy{y8 zjCwer57Vzf)z}g`Ao!cQG#=X&w(C4*t)oxyp%g}IVDj>fM0>DDSLr#Zl|bJm_C62| zg!be%dnp3f-IxJQXasA-&~K|~LWyx?i|9?vdj%EU$^MTZ|>_1@x87G-Sqc;jJVSn`q&ylPQcBNiE@%cG#;!d;j^&SWWZW z6J`UbN`8k&sHlXy1+4uN5Zn*rGaqvG9sl69?h?n_Lk=K^ry$YLp3*unQGN;kRSH^q zP0_gIH1tU$#oqoH=dr8~SALBiSL>Q_beE7|W>EqU#m8Z&`b&(g(~&X?nir(5syk7s z=CApP6-bl>GnxFdm*QFOkrpub4+m7h+kO5Y2rW#iN!BT$jGCKRg@i+U}b zmhG&H`wa*M-|yHBJza^9YFDAfm%Z4#Rvp8r!L0ljf;v>kr`Jo&=-Oqd4`raUe(S;v zDIfMb9sk?mc z@UcF`fM*-r{wIzX?u-^=6Lp0hf_iW+6T!e*TT zC9U~OGOL?Q=CKlhp@M~}i7)r>Auj#^J(9C&LjUO*>palJ_K5OAY`$wa3F%{uPN^T_6QNJfddhE`GgM877h-y%{R5; z>Fji=MFiQG{0`CZo+qVR_NZTJH*hrSprmGX~zXVImh)Wr99w^*&Vwh>3EN0Y8=| zq@%$z%hTeW*IL(lhyxk8jM-umU8~S6|C+ z>Bzsb!Hg)l)bn}X?R$;FHa|***JNU#3veBI9F1{OE`2gbEi*}{#y(hECNRKX4~}l+ zLo}N4X41aA=wcn8SZ^&Db)D-={pWIE;R*Y?j7s)NH8NH+&Isxy9owVN%iD1#0+_y_ zTFtU%x+0~Y^k6tnw;_T+%0j2Xy85;an^3m}nJ8x?I8<%sawLYG;MO-gU{WMYFr@rD zL4EGo;l$^1N>dQ)+wx+%+g#rOY&jOsLw1@LrO7S{%(9z?0nZm2 zP@|E4H?P*yB4F8eibCeYr0(Yq7h^I6zE)RwxX7mOhhLn+1slA>03-sRF5F?!Y42CT z9fC#u`}&PIc_B^@aF)H4XNZ0AbRssKfouxg4Q^`bVe9?TRaNHW;CU;+FFFLu2e&1J zV+e0YL^&vQyCoh*i0p)CbiIq>sXUtg#HJgkIBm5G)ILhBFxL`qF}mL9qT(^PL6E5Z zl|zZ@&!-2A_|dC7!&TF_O40JMk_v!3r_2@;RR!gL9Ia0y*lkU>6gj>Oba#Bx*w7Wv zw4Rt-kgYh;7RBR`%MZhq1ucSrW zX&|TxUMwCNhpxVDK;Lcl&$7ucRCd&M)cST|5R4d%m=cJjliy$%#MKOdGFO0-AF&Zy zYZx*JiN2h6om$ z2l%;jF^c=I?mk#}D6Qb6B_joe9RgU$V;jwnes?s6hc=2sFUW z>K-T$35{Lq$fG7ef#trx3!r_=rHy-UfC_Tp!99C8VEcL(kuaA+rs$i*>ip+ew~4Xz z#`Hb&M;2UTeSSnoJAFoFwWt>p_mT_X@!Qg9mU>j|%W4Y|QR0exbvhdL{KxQjj%z=` zdTjju(+@G2lPdz+`!o#wAaa`3H_?!(;$a+Q9o2E6M2D4;>SnGmHgRoCBQeSjW;Xf2 zERegyzDgiiW*6`2il!ZvFFyERurx=J>I@88Y5YLwOfD++iDpig6G}o?-c= z<>L*PrZ-CGwMU?nWH>VAVQEg?{<)#pBtVoGz1mgwp97YKG9b!TRCdmxiG8BPbH?=p zeTHbCZF~!1X_~J3O$>4acwbcjk!w`kAY;8C(J{iFkz^+Kl@Bs3NAPv}V`-5_iPI>e zVH>FHW@3XBF?6;*(h0=l-E_d5y`lYLO7?>A9HvB>J7CE_R_^`Z8AXWTl-mOnBU?el ztH_XMU-Xk-?IyBpHb=5|_4c5@3qalNJC5d#enZ&PndsD^43oKM%|9J-oDKW%DkT^s z>brcuWZpyWT)gPGP3AVhQuh~t&{ml^PcomB3jKdNkBsooj`VKE_^Uv8Mr%eVzTS}t zke)?E1&l5W9lLHQONP7Vvdg`Fi5EM<@FSR~(aLFQfl;|4YHkv}MEz6

      hAB6rrZ9xd=bL!!+kzQ;~0{!OcFQ95UbDGmX7Y|K|uX;n%8K4EDmC zvib5~lV(X$Q|9JM=5tO#3rSf#jdCGIwiwV@RIq-5qWt-}4qgAo$CNLurwjyqFa>Dl z9N~uEq5D%55KIX2FtcU1`(_6sg$;e0Lts$8RyT|XjMfPWTJ;p(CK4-td6Y>P_k+c7 z8vlfn(5WOK(;>3H`R6?vsxWgm1A&Z?hu|1&zhZczu+TZdo%79V*gR1487{MbaqobM zyk@=(6=5#I;93!$5ju_{&)Wos^S1weflwGosDg9{$xdB8Lj7qs$sx(|O=y#QK&HOe zZr(7?PR|L-+UgS3No47J>BbXUnC7>jW(80P@|z5>O(%W%r0EZ|h$X}4yP2a25(0J` zy#}EC6|kqpa9qXn5hV1G&ceB%`F+bY(;TK$KeOu7|Eu8dbH;7v{W|;!!bQjkXDmKq zuyDoY8m3(?uz;IsE^9`+dT@1KpCe+*`XugYTBj&S$xTo|XTQYwkIyTTmMA`$aOdhj zS0;wPT6wg5=IQP-{rX?O#w8rY-I4G&5emm>U+i=Rw5=1yXWxV-C&-(>^IsS9=j8E@ zI%*hd zr~o7s-wUc>WXroH%74aDzRJ@`V_?+7g%4Vb7=A-iye8mXXEt&t`iGlI=ImcWZk-I% zAyAiOYYYmTx>a5AS(6)TsOBz-VOQbld{Zc+Q>eFM=T-mBPe!ZJtNNSYpZTv<9X@4n zYHHzY61`TQx)oT*e^DVw#m=J31QQyiO?@uKehDDRx)y-~JJnl5+*?_-fhT-=6|RS^O0o z!-vnTWyEPVh$QFAhv*fnp()mF1acPm!0ZW>8f3X*JNCHf?YQjFAnTjoZ+l4Bi*Qa5 z2ZZCL0O_1-KXPWdrgi#U_@sgSC@ti5X}SyJVbWZ%KS~zMwp={jtb5zN{yczk9|ACF z^@#t|xt~ftQ2H;dSEf)xdiE(jy+7uP$#i3a4kCHxdOdFelyDz^DxO~14U#B%y9k5s zozN#h%D=-7znk|?E@%5eif&AS%gD|kl%F5mFf_hOoLKm(Ebb}4YRNbT_NT5Y>mqtO z8>qS12IY?AOqCh;ocuF~QkH)4n?BQjQ7`~Tf7kvy+0*OvAIWJFUMMEu>!PeV?Xa(;wDAaiIc+XX>9M z1~m~lNm`sUYS6qe9 zC|YuB`m(@D%lFF=gKjG7(J*-D;=x_LV77F?C=Q3-w4Axfn=Ry>`5VSB2yP$ zqIdZgE)wLEY=8)Xf46YF3>@=wS^XNy{b{uS!8N4tBq1X>hR)mr+sM%pBKm6Q{Dk2~ z+u^n|-=yBBV}$?76KJhE3Z4&gE=IyO!~1b1y~|Sm3Ck6h*m)1+4s6R881h^5c1=TI zyBikQP^AayAnxkh$;j7XDmWVvZk3WA;!=x;fntjqyAI>a%uRzjR$H>s6AhMmBwLb= zx(!lHX$jayEKaIXd4niFiV(*o+`kr#M2z1vTs7V&3KTcx$-jrbWMYZH`FC>_PVL&C zyge;VqpLYfne5N~{OxaCMT(U3pZ!oB?GTNg54H?sc9o|6J-s`;7#F3X@7j-Lb*VJs zf(s;H?_Ms$tsHXqHQrIWV5FGNv!0xNE4u2JAcqO!gP|O7=CiicT(lV&_*B(+ib37uxv4#kEby~IjzAn*B>V55tF5&LEFw7 z5;*ee*0X=T8*E8=XE=D3?h_zECZ@ah|dae{E2tLA?We4%)e%Ut*u$#MJx+&(y|Pr?kJ;%yUL`X+6zq# zm`C5SPALwTa314_E#5Xu^#Uc=8DyJ>Ka$vwjZIIz_w59$$S2*Ut>B%IOn6@26TD)V zWHFPoy>1eFmpVSf`?;Bg$3dXs;Yh-1rm&1$fsqGlLr@7b#Kz#?!14wznku1X3o1@+db}&$RJ86G zQe+sC%>C*)B69fee#b}D_5C4MqXc`*#5m+-*4W34k*x%Uy4QSSkxwXBp<9_2H^52K z71ex7+WJLPJb56ULm9Cpiv_xgsnHq3*2Hh_s8j7vkjn#zbZ#qllj0H!9=95TM{ewW z(f+fTGug$$idHothnCcG&Hr*wWR&83QO?tktOFDhtszrrPJ%gj@OXFm5g)^_(Is`bo@YkCAHe1O$7k@q~>oUhc_jhls!5=-P$2>ur8x&PSk!BnCaeF?4{% zP61@|uA7%(34{iD1^4|BWQ6wFwF}#-c*em0(e-}1uGRDf=_m8O7_(aqDMDgy_Mm9j z2jO9WAfU-m4rm}u-nEa6mof4s0D>=|97pmlo!n6XIO37l>Pr^~_U+AZu*PqC7Kmv- zxVP)f2DCB-e@6YZ(HH-yhk}#{wRg3oJ^q&*62Uj_X)54W+|EY(UC_9bv`Y>j`#*ad zzJ(;P1R#@j?zQVKjn!#x{|Mo7@Q3w4Zi6Joie7`-fPP3}%eL@wMds5*fnRw8?Nc{^ zYrAKDhDD>ZTM^8aVc7H>o2c8iPa8hb*ij?)ATQ2c4WdtXiK|$B>G7Ha1cMgkjfz|S zav#=5KmF$DnaU4R$kWZ@DU_^VE2pD&dWsz1kq-1l8ce7a8S*HPG8gE?bBSQRfYHtm zQ32ggUEF<>r7mx`X@@oGT7pU{!R2Qx1Aj2M5vGUALnpBudtdQ5<=EyfM5-^SuH|n! zdxV??eivTm#;rr{D{42CodWV6^y5J3|VW?)V)7SHq$L7 zvR?e7L5KK2bp22SZ)995A(J_VEeg4?R8#mq%aAP5TZa$|+|L&ok z8#WT5$qjdJ%niW?PEYKuKY#!(e6;~sCTh<9663Q9v_@ar*U0hAQ=!gqPspuL|)?u{K5inQNgR)*g&46tX`04%5{g`DNq9 zISfRD&c*|=7nOZ>4*OTk>b=9T6~`+GZ-$$PrdOza#{kIsbbi2Huj|@v!BM7e5Akz% z;ibIcU0k}~-R=<^VbR+7X#B(bHqWMGSTRN4)uyJy+SJwUN1)kQ2?!L<>VIHQCIo%5 zSf9CjryZh*}#Q+s{2nzg43f7VQMQhEkE-85Jspe?F|ZC zMxKPsqv%@xFQ00)76l*9^u$fO!aj{b``XIANWCy7iZ_o#)eXPOdV{ZPcdczTgT7+# z>lx!f>Y_#EAh^xqawL5S1F41aC#U4T~Kt(xf)JkxBw>9p81IspQR1YmJw zn|C=`7VCk<9l{`Mm<*`({mD+3@Ea?-P*jtKdw?Lcut*0w@H>moWQOSlq0QUY={G4t zVAvHJ*0Kc!{w%wjE#fOPH7EhdO_o1{SQ{+qhM^F9Hk`+RHwMq+r%qJ=W(DRK;M{2j z4BkLLf?2b47^;KBq z`lKq*H#N~bqtOf+hi)N02bVcvz2xinfUC0hznjQhU>OELseKdWUH70>axzv{+7*>D zkFw=K-vhH{r^gb@DVKo_Ew68wjN0a~M|Z6t)l_1%OU3@A7aNuc2!}buD)xK*ex-11 z4v_~sSUs&JRodZHqXR#|HO~==!@}u<*svX6X3%mSSC6W8YU03!M&xKZaZc4pG4MPr zTgVsH!dY0wV=eA{K}Tdt+`hP`T}1$DQgj~L%h#=f3kuzwu(i-GL1&B)5pOVSnElK` zPt_s5TBCjSy6h#&V51aw;Tr8yTWNe9n%bz8?)L~`4g1)~9ROOgIV3I8e)#&ZJQRmj zE{2%;u`=FOoIxwBsFqF&HOoS!+`eHG-&@kpU72qcPH-nHh*S(-5?*Wv-C|5%TjfCY zPOh%X0bm=}(IL!qyzO?#{@|t6 zk`pD+|Fryk?=RKX1@JKI(0Ts051{sTU4IuQrAlAQkSnNWZc(K9lKFCiBMew~B<7GY zhu`|6qiy`Hr;+wIa5${k0LPyzRm8XP`XfloL-m2u{b<$F8hge1q?+CMck&QR^3EIp zgd74*@S61~!C`=9A@Z+mC*d#EYZzyNIG&0<)NzW94DT(G*`o@PJ<0n5qs|-gR-mIB z?MzPdsGzpfiXTi4kB7znx=XZoBy721mLw1)z4kliI<>o0G-oSwE$c{^`pq(3DZd6Z z7pvk%YV!os`o#&}5T0qMo6|oh_EH5I?|n;1HtlRG7HdT8n$#&DnF^*iCr2Mv+EwlBVnLn5h zga{%e_gDcpJzmZGYC5~Qn$NO-<%XIzACEvjVe+AXK;hzd-%UCK;oUmm3H|~L9nF9- z@w)rZlDj>Xgjf(*Cf-(bh=e<(gk{4!A2Hj#ZeK;k$el2LNX-zAxM)84oZ#Ap>2`p#>;|$3F!$0Zn_a zdB~9V%0b?N{QLj|D&dSvs%i?2Mn#4lYkjE$rRe?rYyHhsA?24A<6;5mA-fm0z09l6 zUtqGFfq(#eT#HCTr|7LOw&M0TW>=<~x4P;>Ow)7h%l$GoW_Vq^cudoDd0MBP8nG>M z`SBc=oHOWBK0WF8mLI%&#`3Q5P$J8}4F%$-_GzkH5AADY=;Xfxm6-)kc>0{ixda?k zbT8kd#k2jV$$6rb2MY3jP!aa+!+0m?Qw@wq4NIe2$FMc?{;|&sFG-;SOQdF5&?WJS ztWMJz?#!;BGW-R@fqu3hvrZS;=pg)j>$vv&M*=g*+#lf#L4_*o#v~wgAcVlsXH4Ys zm+~wyGhu)ICmd6uA9fTQG7WkS7sCyqXsh>Nk6Q9e{rz%QS=YE(%TRJEygP1i1_7P_3SzyLQM`=Uk_?6K z;hC1)){Oa;zp7fsa4e{ z2yP!XNio=w^bsBEDJ+u{CCH=v6nNy`yac3R7+4u_UDT^EbaP}FBv_FZ&K>Kn{v91A z`s$2IJu_a@1_A3yr4mJMHkA4Fe7+Q;cdwXqX3+E;5d}=Y?`6HNTE7RB39JuDqeo_i zul{!7+2LA&hCo(Vn#nL8#2_;Wm#k}O^f6wDk1?1p< zu`@Po4tk@^zlnSQblrB0V{CZauSt54XWvDyqr>g?9nM3_5LPG(nE}It@*&R2w)M}} zQR^0vVl)qBiY|*UJ>R!IS9zXqHC(>MF|!tI@5;tGa;8&AS`l#}(geACF6lFD*3}P! zw0gKP9+Laou*T26JlCEoTvUwH;1-`09`abY#1ZL~&!yAbD|0e4T`SKXa9Ag7+P#Ud zX@(MPo3_WtqhXVcBhnUZ%!P*6zK$>YZ~U#p1?hb2idc&uUw5)R%u~zH`O#NN)S!YR z0*w_PjycPlvpQ64Sn`$=SRffctq!E#f$X!O-y59Xu#H|%PDkKkA;4wq8R@Y#t}9z} z)=Z*U|4KH#6mS|sEonxRdBFZTSq>XMkeu^T%)Urr?A&VF^AIL;B#d*WMUc6G*1|KO zm)T^C@^w1Rk9TLhFE$`96VQA^AQKziVe`l!1lufSJY%<`b&#KYX*+f_T~qbhms5QN ze=o++fNl}NlYx!NUFsIV7Sn}+4J=n3n0})yT~^jLvk@Q_7)k`OckUWmMKn^3gz`GM zm=67P;xFy+Rp&&H11*1G4V{1kwjM>fLb5xl&%5M1V%eZ(Z6{x#l9&TxvK_F0sSs*>uHJ1`6&J-MxgnvJH zQAjy31y+Iz$4x@GBjFLoChT^<+f}{vmAVBK;yQv>D@<>_isJ0geI_IZre*!DlZx!~ zCU*5>cfXo$pDRd6FYi(#FRFkh#>L6rhakd5rSZ(kAxmwNUHkQ7hn=S&{0?BK*7pos z!5^rc4w)l!2(9579*2qzln8Zk1~d)c6e1_qeb_C7Mie^cR|`567S93OT$pU*agFOv zXfJtGsr!rW+mPyKrNMo>(nDP4o6%0;B5D~8W@F-49gNUJ>D|yd0^ibNy4!I+a2D%h zE|jN z#rn2W%=MfH(*fR&c)OsmanXleZ|Acmts{|%;{c^2o{C+Agp;`VV zrnuuW92(yI-Xii9c*0o0_Uq0iXhv^;)#`H8uzd$EduT@#S!w=$?btdtfgMXP#FJsa zTbGMEBFP#y0&~`x=(gypc|Cwi(R4YjyZC&#Taam&dGnioAr4<;@I5@Ujdy2{{Q2Rb z@U3m*ZvK(Xbwv+Fm3EccEqA=->8Q5;FN$giJ$B|9x!Ygk)-ht&E@60$meJ-KsZ?3> zz^zxxo{+L+Qo5Dv*wIp?=6Le_wq#O~eOfGwbFQLH&fh9c z=UI&$yJ=_Mr8(ONC2CxyRG)g=7|L+_aq`BmcymcIf^RC$x?RicE=%H$-#x@g*JIAQ z^0}Y;T0YoKL`>ye%x{RiI0;t>j2$6Tro8n*agAO>OySq}*3d%*6XBy1YVz$nbveh7 z&$l>F2#G@9P=h%lU+r=UDqF$_yKr>}Dd>=h?>E+b90`YW%ijUDFBIr-JylyxaHF}0 zBCU|M#FL{ibVtdYs^JkH#E7(WWVa{gq;A+F!_QUc#1Zt_GQy)oHlHM|rX(X$^lm!W z)%ZaLx|osCBH)e+xhtt+hy~<52=&gHIi<;C9CGvR&$p&$f8?QS=)T*>h^*lnIXVv0 zkn2dK)oqANDzAX{@_cdxM02zoJ~=R%_v6*exuP{1wJo! z3s^(*)c%x5(tiH%tC$p}7a@p&lDUFt7+@00BeH_{v<*;Ci0Wkb85yu3v8p-J4Q;(` z51VYYK#_5WAQl4;33lNJDFQPh{P(Mv{uU!b=igqiwwGRUqXi3MxOgjgQtXr-j(yVJ z>on{bWGIVM4l7?S`g9uY251EQZCbiy6+AC8pW9%J*H?Lc%ni2mQcZhp^{<@*fjL2k z$a=uwC_!@5Fhy>RngB|$j|3nh2m#j8xu>0@uXUj!QpfN#f4Nw#>M(`fgc~(=Isizh z6`79*<~+7@T+0x25aB5INp;5A`4TjfqoY8MLX32O?bz?SsZ}CyibxoJG8R?@J$y)V z9hjW4DQiCadVjU(wO?Wj2*V{1k8gbnzp>^Hp87_{vFYBkAdNzFW2eF7Yh|vZzUTsO zlFv)G3?$5BDL5V6g#Xp&*bv~-@}}Zy!qk9_<9I3vG9;_h5GFo-a-s z1|N;J6~9jLn%Z<9nAU#c?=9@E@jYpfUQ#3tpt7WISI#Tfv) z*5q~P!AbqzV$>KqoeC{Bs{m4vlCspiA-~TJsZitw2!f0&SbI|hOrgkF_h##3zsA(2 z?U7KPc1=h^Byqmb^nrPnf@`EA_TPS+E0S~D)j+d($NNDjN zZjWDpT5#ayUc{T$7qz4sg{ec0MPbLci^g<_Z zV60)0I)ZFsRslULn3D?9l=kZ6ju_-ITWoQ<_UDKKyDbXR1mc>vX^Px_)viDMTUp}e z`1k4ISNLr*@aanVoIy|yIoEbDBHxNPP{-A`IkYbWS`~e_(kCn;D&KQRPZ)+z82f{{ zj{y>`#mhiIQd7Fp%gRdM)BZwMO^vQ0@J!c;kohZT656V9>&s` zaVV)AP#)mZ@&y=tH)cu2R}Pv2_F;JTAU(41!(wT1yO=*(K?{3RiW64-4QKF-1NwD$ zo5CCtuj#8JE8rWl*(YfcIMc^?JYU*I?O9n6*pk@RjZK!aYM0t^Gv%_Hgqj@N-XuAR zSd{L!UsZm0pd6~>R25JN|6!@T+kSU5;VJ7HREhJ4hTj8ArWRxx`ydTlpbzB>Gn-R4oLr6NO_hGu` zHc^!sSvXH@)X?(d(%WhwIXj{hxFa*TBaMEtcRE#=9KSvW_g1Ov1WN0yl#VF;H0I&n zwAmduD=MO}YC5dWj9jvD{-b}MNyR4UHIz4Rc;q!|*7B5wP6VwT;)&gC59eTt|54hI zm@&=zCGkH6gSZ6vCWWwq$tj>`_rNFl%}QhT7_qha{3ullJdfTC+WGb!&L8I`!q&@z z{e?`DP4_F{vQB7wfS#k#SR>x1k0QWpTnG5_sGy%d+Sg>seW z)bkZ80EA(B_ho)V@Gr?Y=4qF&x%mydP|9+E*Gt2|k;$~1KyS|nnCSHPF7^O}16%P? z$YBv4IN0=A1ZSKsIaxCGbIBIDpQx1i!rO^Ff5f{;coT{mEkR=4IOU=ifs0( z)DNeU&x?Jg-_OpX)hN;B16pHc zqG31ijxmTD$P4CQyJWSe&_s~Md6^3{RI8SKA5jTL6mdXlA1ZZN&62~&o;V|lgO23u zX*|8>%Gx!Yl-ATwT*VI{@ZW7Y3@F8_n0-EGRl^Bj-sS7>1YrL3 zc=+5cpcJ{`Rjx|O>?uEYI)6WKIda@CMo}gcPw{@6VRj{wCGz-QA(s&MgX$>g~puo8Uk8sK&{D6&axL@GS9 z8_@Qom3JB<*$50M zy%EoI(LNhA!}ebg%V59J`k4!rHF1PpZz$i&MLVgO?_p&5yl{R*pKq1D*S^sEoGW;| zHBkViyd%ytAJ0+f_?qYiv)6jJSJZkIJVhWw>L~e^jgD?c$AX@udUG!)p9h;m*XlNb zZNT+8RfAPqd?J)%G}ZAW~z;4AAg%YHl0RlS=+9gS*f zuGOSmW*BqdiOVc^%Qh2xeIC#UkakK2d373?-3MM1)_V##5)n>WCZ(NpXpiuj)DBY) z9U#nPu%32_lS}Tb8HMs!qG8|`tNyu@B4Ls-fOMglYtZ@VlsM^KJ3+s2P&E|^t~ zApH>b|3qM)`a_?XmXJn+^^imdqr~(lopnW>MX$%`{9mPXwGPv1Qh$jYEi~l?8u$!( zoi*ZqE{6MMcbwNS4EtY_LuTn0(obpl2MBy~j5XIw@gwFzTHLjE_tM+nzOdRjk$z(K z8>%>HsmC)eRO%E53+TzH)ee8Wej(}BBv1j)_}W>`SMHg%l$X!u7MRQli!E2z;4d~y zIaez(#GQFF%`Z(S#AIUyEaYw~{1j`UAkFDok@jWInsOA(O~l`xLiz4S5ZdE@2;_aA zCTrX)`zq?0kcZQ=`{|&8AU`oX6~o@rdjs(qL*Wmj5Mgi2u44?0W`}i^5_BpydwTdx zWwcGwQJ*P(WeF3AstlzEjVZecaj(Q)l#Ygl(@7~yKVtcZ2%qCELrf0tU@Nvwo=@xt zcy>C``Pk3&T&o=K?gJNZ0`TveXrp`cgGO9k1CII_Ph_6?z##Ae%rBb|ZNgSy*jKtk z!y( zumWaTrt&ewTH5JA(|I|BD$@iEo@IG{`L``TPpaj#)O*FQPMITALDSz0Hz#7hGqA{R zUkemqItCIO)9cQ`RpM3uFzSgM?pJ!+B80naZ@bn#QpCW=04ut>+IrMdH>zRIGGafw zL{Cvx#OyI%Lz%}rHn+nal020<17wYt1rp;;SFr;0v=E%%SE`L)tx%6vQG!RPb& zcG)Adr#67rv`o)2P1j^}03-FJ@ApqGm}ZM*>(!Wa+I0>~61)O9BtSjv$ZeAG8=+0r zHd%maFUg21thnw{+cVi`wOwqXP`9uKqqK5G$w*q-vWj-M$ zn355cFiMP#g3Zh%zD~JjN*p*w@Fos~PvCp%#?6*Lmn8oMOZPCHuC~_X=z{mG-mE>g z^rAKUF?+)LXh5GhFKHLDkFP&6*ANi$`hWyk41ggq^Yi?FBRo{wsZ;z!&%N9**%MVLo%aLLT5@0T1w(CfK?rmC3Rs&3pIz z*NbkA6)-fXnU`%YyAI)C|pNuYe9>93iUAuc4yFQd}>(v<; zt`R@j;eT10_|%2B$q^thW5X|N+!t&^0#uUl9-W)06K$ENn_x=7r^4;3M<-+DYw^O^ z9sOx_+UQ|%OxxxEwq(;C{440!NwBxZPiv{Yzb-)ez1WwaLmbKst3L)1N@lb&yE(F| z;|L<0w5@0FOpXtEFU6h_Ql2;OD=Z&U zUk0^@Bl`O)es7z~S+~EyYVX^#r@1lxJcB#BS`OGa*3Gk6OqQzXH2pK?RyJqEWK#P) zV0TA6A`5q`0P!qTsp`=x*so$LwY~Ohl7`~p(7IQ}yl*=)U+;9*UuIjkcA zGFbdMO0)UfLbj&Ab-$ME%8GkFD+~BK!CUZChnC~*1_DmVGS+T2lg#Q($pKkkV^q27 zFxwYCD4#|utjm+0+>~_61y-aZ%kkXo%dQQEXp>MALI5?PrDm|s>T{FdmZ59RhM4i2 zrd;qrM^5aH(kW8zbt8$hipMJ^`p`&4BOJ#*oJi6H$S_X=Mar{HmC>a2^a zii{bHgHw?Pc4ONP7X{XkxVhSgc9bY^vjoTaGL^JJ}C6I`=1`CRt(z;3=uaOpX{YleGsE~ zuK%dd=<0^=G0to6igd=KW#h^Q2zYM$vTlpLYT9M|J9Rxv0bPCu*^2N4k98%}`vrGt z1LZiD4A39!z2+2^FyB$en=Z)rl7l0wro=vQ|BUB zYP3R1bB+Au?6M?-qVk16{7ZSV2BWjox4%l`)mtTaA*9oR1mV%;yT<(vo)Jvnq1MG4 zk0d^ZIV6S5=0ydCzs<*2C7r72nnq-m_r(b~!XvEyU`&YQku*EJmAfa{fXA${KM`Sj=R~ zdFaAb>TZ8*dw8_G6pPS#B^r>3D43!Yu4qK|w*a!G~x zUH19Z+_snm7n>N~sQTU48mX z(G(HjpT#|-JV7E@t%t67V(r#uA|mEq36)6_#_qy=p4u}zYIj~8&OB08Lc9(~zArWy zL={m~qSfRMB0y_a=%DeiHn0*FE=CGl`?P;cdx3G~2Wdyu}*(cO#@6USf zM#tbrkp8CA298~=A90^kNOu?%IeSWjB5vW4ux;iHR~bUB|DZi9nmz}*V=u3)7Z~hi z&c((Mzum&TJac$&DxJX%__SZ;orR58`CVeo2^t=jemK9oe()%+$tUbJ3h^mcH6JfH ztv4~%4eXc0s3oQ%pweAuvP~ds2hpfG~jusgoxOl^95|E9tgQ6_eqF9 zRuZsDqZ{@@EQHWsbFIwjmFlI4YAD0N1tU7t1*rYE=HaU$P4-UN>Hk z-h`>+R{r1vJa=qlniB~@5dbW7AbB8~2&@@_zOFDd+^;!n!+Hv@TWsG)SbOxYznwdq z(}yIY-4N=_dry-C0QvAHVUBCyVA_SF*GWZ_2i3#d!^XxaoYYX3zSQxWIv2&&zmtHA zu{=pd4UCrpr9a&3k{yz^F1omY7G?au5cdjLadq}S45ZwW^C@l~gfD`F!&>>-XytTdrsQ~OaqZtrq`MAw)RCyhk{RX24=tBbEthkKw5w-?vWlr> z&62VDnd!EAzt8y4l=*~Ro2ya+VqwHb2)^`l!q5Txr}~51CLHUpRHL5=8FmQaxzwhX zN|2YuiLO}lP9Ri{Svc>sxpn^lvVOZZ@ZMKppRL|7H4GAcRIw&5Css(TT({a;nrK-$-X$otl?oY4J^qet`Zf1GB(L|*9nNPE2k=B#-Pq*o zq>yRn+T5;}-&L;LXj23D_-mu9FYLIn0u}f8U!r{z7{f?sRbpP)_a#hC_2xJtJzq7# zrbv;czI0AbVdkmE%0H_#GZVBU=HAZ~;(ukVw4sY->C9wRngm0XF<+@yas(L?vT zc{p$O0XHoIBp+W2IVx*aa>IE-2Xs*;!fA4#F|h+ z)O2k**rc14VeS3CGIM{qBeQg3&HmlzyV?MX0wq!%3SmbTq3e)V;xXvWC>I5 zEa3c0I7~NBx~4#Zr04}}7V(|9Cj|8<<)6KT-ZO6*h8pPRn=A3e$UAZP!_eUIfj{j= z^up1t+u+u}uD?#NBtIfE@_ZFV=nZ7uh{Klq3+OG>ukTmC-B{KfD-@|tPSyHVL~n1I z<`!d_q6b|v#)k4D@0li0*r>L!m*UozcwS6jgq>qL&7!?5zuXg8FAelVh^q!pirX-A zNDB+7`j&i}mw#0w|J5|hz9jGZtNEl8wD^X?Z#rXSGA+TsK#dCrGCr_8yA-~!c|%$3 z7LAPEEa$3col_Tq@zpTJJceqct<}JuM0C#NXA)kSQC+ZTPpnX<-gAjB4^(v0UN2rc zxP-(;KgpMdd^8!3rEuCyP;N!hU@zv}ofP(<2Sn?n3)pI*2lA-|{f9g}1b*t=A@b1q~3l>wKhgD;#*m2To9zf3A+%_MYXVL!7 zaorIH*e#(fsz~~~{xL%F(c}Fdh)blPKEotG4fVWxPLEy6##*NY1~s$kqs+vD71>cn z%LWOhSQt*xVZdxOeUwBT)tSnyP+_bwqFvnR*n2%97e?oy%`u6yL37WJ@Jm{Y z^Ha?GCfJ+Cg?k8VH3J}W`HwVtQepT<+20`$^}a!*AXG_Kt(xZU%C|e-`}tU|0%3gp zJ&Sy%Pg4>#42|gq>Q-O7ofe&9BXBAYaDi`X;(t!h2bAk00QXq%*mlUqp*bujC0mf16zNwNIT(`)~ctsIJ0lgGnG6(brodL@=D zw)SP37VMt;JV(yLmajo9hoxrq7BdoSWfmi$UX^T}N|h@EP$6Lo?HIAL@!xc|MfGUydgtOc3tbIIA`!Z?*|Yj`X`RPZ*lXhl!vg=5L za_tRceCT|f{_Vcmon;=aS>hf7#jER^%YROOe1dP;TwZf>oN6G~+UND?pLY5VVyqcf zPdjU!`B7F%U?z#h&;(1AY~;MgHpck+4tq*Acy7KUYC40B9iM;;^4w}-gs9UJLy(__ z3q^Yz7bh#Q4~)KSD)(qklqtOnd_`bs0b9DW# z(z)f%;U^QHR-mzF0(bkxk}4#(QY{UkYq{e|L(atKvRJynpC)pgf@X+cAbpDc+chbO z41|Y$Xoh__r7oG&<}DU9z1!AoW@hjP$Jp2G^1}YR2iX z>BtNd%YQG#H{n~m4q(BtE<4fj?l?32Fqi-4aw1BVli~52n7aCuY}xXpB4VcWZCEXE zxw44x7Fs+*6Q*E=o{VXCq?T0wbjet&F^s8246An^^ya=%7d%b0b$84Xu`2?eIz~YB zHxgrhdShYu8=|<)1iG%}r*||T3Y8Grghi*4uGYol#}(e$N>JANx(YjQu3Ni?aNCWu{>rKSdoM9EO2Ha4YHC%<)34Xa4Gub(1)#Nkng`0gA_xNOnAoA!Pv$Er04d3|Jn`t1L_#Pq8p|edV1GTA z@5?ZFnk7m+uzv+%p2VcRZbb$SMGRHr1Q8wZNkB8+)< zs1el~<#hN2A(cdn(vaeo=^0OcS2;YiczG;vowhanE-KLtJ(vA@%%I zZd$zsEw)|br`Q#N;HgnGJQiue6D9+EPuOQ)Jjs+&TFd&J*UOfb<=pV-*DKw^c3c~M zuJB{^pVaBf3XJ91GiujH!W{KrWadPTPrq?0$Y_FR#pIh-s-e0?r*8v$Wh{V>D-j0! zbFN_4Go(+P)-IROyHlO;uK7Ks`{5RaQmKQJ_{e1 zWs6V^QN~!lx7a}_3rLlPK|U7b+%`P2>=b|fA5Yasv-$eF9eP3dv6GO^EdHh-1f8mt z8!UGrq3vSKCf_9YJ-C#2!EHAsfoSK5kMd}y>f#tqWAWId)3yb*gO?o_o_?n#MNkA@ z=l16cI9l+ih{S;tcITnqL0o-L+)pvsJE)a-=gm9FFPpIZhiRTdC@v1v3dP>9?jzD9 zXUv^3L;_sr%>PVL+$ZIAr(*oiG3t`R-{Ze};7=p%F?tfMKivK;&B4GgDNs`BZBMFC zYx8Iiil{-{v!s%TZmAgk^jj>RUv+fqee$Ic)Mn$0w|CksUOO}Uz(ycS)0bfcfg z$i?*g1MTiikX)3Q+s z?a89!ZKq)o?Z&TUcE>2G#b`h4%?$IxvyE0Xo%!{=+Z|gO(!C4bwp&LnZejN3tId3= zf+?&GP;={)%H_P1$Pg7T<39N4wVCRU<#x{IO2W=kwO9AcP+Y4NPp9vo`TDokq)*OE zD7;~~p@q0lb2lm*jzgMQ!!~;KH+xfh3|7o;l!?c%2kR%Zgo%B!#CATr4JR%!tWeD1 zdK;DFc*Io*WuwKUCdu*MRAdI^}ucW^Oa2JPliU`5Ok1xm?U?l(QOY$m#~!B!wyg4Ws!lDiCx6&cjwDBfR1i z3IG<-Gq9&5wyA-rOGuyoRQbB6)8qLbtB#4|q9)AVPsdjD7;A&Ea@HDqj@; zmDGBCBFW1>4gU3%A(zOFUMxf~zD|T-3;@y_)GSi)BV#=Li+Uw($7&I*B1w1;k@21P z-5ee{zP>E0Stb{8ZH7Y7F?OwiO-L(A0AcX-$mj)#Mh$B#tm&4j{h8^1>4-Hsz&n=G zalkNnX1Be}_jjm+8lSXFNr2+6od!oh^IE}1(t|D0uaxE@0-#3$!LPsI7AT|pVE)%` zE6|&hR6Iye`~5oUEM7s{wv76bktgB8y;nE{B^r^~$PhNI;vY#FgvkW>?wSPAo-voe zBOWH;G1AinZry0FTJ4f>ThMgAlll>|pvu|%kWsLaW*k$-0Bo09<`yYqp4wOyN)X&I z1ZjoeS&!X+-0p38$p3MIjoiMSXHhk&4oHW73Sc|2UP*UXIzPsPr_W77R+xWLKO4ER%>dRi$2UwoMy7Y3t zvV4b0-;SS_L%gW~o|={uBGh9IAUuHo)MU`LC*i`x*EoTlmButQDpoi@4wO*(a6;cQ z+0)untR?Wu9iQk`PTf8~`+r5qwrco<3v}ChQ#nSC0S-e&km?UKaukchPvri&9YdSD zR?y;Cf2pEZt`vM%Ce&%XgmZv!#`Z;1tZUR)&9zQKSeYI;ElfxTk5;)vgk`Ch-CT`+ z{1gJ4Ev7gmpEQh~o(+{U60@^m`c8{b0A>yyR>EKVF|{c;fm(lCVFi%|sROci2LkIc z-L!f0mjkOw?UXdz>=IBKsln4)$K?c8D1a81tmvk#L?_+o9*W6sa}K+8g3ocSSw}n+ z+6C%!wEBfiV5lQtP-xryh@gwI_NJDZ?9`C)Q)}Ri8m)A!6fuo}x>0rZ*@U}&BE9L~ z^@sH{iciLS-`qkFlNhA%WJm0jRAcIB6hxPI$AToZ##I`f0=LV?N-g)7&aewF{7iBz z;}geVp9UsVo#V?yozYEF z$pk`qS#jmGT^!e7SRRu}dZ6#O#2-aT@d>g+>>h)gcqlpi0bWO*JVc~op3vEKA%BFx z;v62+XO!-VNg1*5NaV;9Ha=@jAHr|ljPK`)h5gJ<@2c9>kn2rlVdW32yrZ2j8F={Y zcFjtt%=+{>0`=EG?V~;5rlYhja0`ScNuE$iANK9pk|TQVCMOuVspDY{EU%QFN0&4j z2~G$6f`?~mT-jD>=(JJ3!y4NI!3XP^u*X4A!Go&q#fV({DqP&Ti3E*1x3d507p!L44&a`b z0=;i8JTKfWAJ&{7HvC^(qcKW#cEYyRm-qBu}unuR?@7^wBx)P>pkxY z^e-@@RWhf@3-67G#f=RhQigZ0Yt>eqt7u|Tprxxz)WgcRei$cd>g7rdex{B@v^?8G zgHHK0^;D7XF;-d|O#%>@O%zm!Lc{5tzUyX(LSE);Y%2qCK`Zb@1o|@k z4`RUDM@&nQ-EiD)lyZQ_7d02?&M)IZ$5Jn=+1TJG0`DhK5_$d!fvi5;wB|_`<4ZA07X(I1=%@Ca^hn-itLdHPa)}_pgu7y#9qB5GT=qtDV|1JQI z4LwBc5!6YSAk?8>;;4x_v{H6c0)Wb#GvoT(0*$8@ii~tc_m?O~5*VUwN4(z~)90UI zHl=qSdV3pde>g;PeTT5t}OZ zimCYO zy;0{CD5un1w>5;YHPXj1S82vdapQ6@IASCD6vdCF6((MNT@>3<%^3?N@+#aHoCmyqt}DJF5A$Ke@8sGFeT~nISN@noYL})g2cv57jj4*5&Y2F)RwbqJ0v=khKAEK^A$6 zipCZ4C?Z&xS7z}A~b??uviKR-U z^4B}$h|$%qX)`>}+1ej^5zs%op1={iJjix%TT<}ZKaU38f&t8%TqvhAHh==+yM#s; zLZ=AMf9Y5NlPj=r8v*DYw~JEU{k@ybGuHZ3 z!Wp>j<@OnRRCBwwJo&m0bl?j9od3&~F{5JmTtG*oYayWciDf!1n~tMzOup~058#nP!(>NM@aR;9DkBLSS99{J; zE^0pWdQT)*eXSQw)Itg~L>po81Xl&2Mo+<&?!*>MBMLGZ6hxOq+flET>b9(2m=?$( z2nLGZDtS;N&QA8#&m_7qc9ZOoo+yf5mswvIfNQtE0<~k#$f-=S5Ox#nlFo`z#4t3x zL?~faf48SoK6o0I>{hg|Ml}GuB)k|0MW7AkY*c?g#X8T6f=M3KdM#05^jmyt`&@BG z!!N58oAD!+w>Pm-Ehr#!iK7EUo_P7y_!rK3W1o>yfGY_I9n>yg2!u{yu5dF*W9Eq# zu~e(2t%XR4j>fX=Dsw{&@6SlW?la^dQ6Hv!Ueg#aV$3x~FgNrHgXqZRiTIs3Nt?4r>prAV$p~a1$cFQY=5J%MRqQo|4{Klo3AZjX`N|ar5SXT-P}i9BnK#tU!dNcW*gqQL(96us?B&q* zm9Sbs{hZymnWrCL;Ce7JmZ`JHkn>d2sw?V7HxenV zNBgh0Ps#y9msRa7V@BhtxO&f=m;R5!1Cko7o5!&H?L)UVYvmrVUj;D*biXT0k7B6x zGpRdPmeYP)T6qlp6P+n@8h}4OJGF))krV)8qpx+igE8Vu`2tC@J|lM3c23Itr{~Pg z>6p0U+>%_)7%6;jisu3NBO*Qs!3AKVfXDT+zJr<|SYunF$T~7T3QNdq52HOzBY}R_ z?}-|Ontq4_$9W}t{5Lhjc*&D_rB7bpF@^*UgO8TU;cv8PXYF!QE34UMMf67DJY_SA zmk%1=+#_4HZp8tlLFzsU`(}O5wM2G=GfoXH72%x~F|{`+=*vu~3Q7+INKVNRm(w8m zUP7Q)HqtH9B3cy3S6ncRyxbpS)7kW6g_(O$z9J+|x;8aY`HV)@gi%|Ol0xU3xWRIB zS&68CQHSDj@ICH#C(-EaqaJ*=+3dQ~Mmb5itONoz!rwSn4*i$ zkrwB+BF_&`elm)`kXXIP3Jg+?N^C~$ zZ?DU+7LiJXUMF-HjU%L^@m2%UygM#!&m0pg49>5;ozHu5eG245n=UFJWWMSkjs*lO z(~7z}sRsX_!~=TYs`mJL%T@Wt)ccL;45!XCwMXO?TO~%VS?>4Cp@Remk-;aXok1&- zcZz}>@y=u#S_w#3ml@i4%_dT9MtGffB80suH=?yK1kq8&ldQtu)*)Qf+2xiBOlX08 z;@ojpeR&rzM@$|>f%jk#*XE;+zCmQFpO`$9`#l|Ar5RnRY36&VJ={1Mn^U%I+#r~5 zGzV+0DfBKvkRAm^w4jRD4*7lWQ!zX-%=bX+uV{fhkzx2< zE#D@1W<1yC=e~N0#+gFGUpv7Nee)H=rD1wS&j9M90REFr|La-H`sog)lUENI%APkL zX8o<<3y$oFANh7*O?yB~HkM!%tR7r8x0va1Ev~#Mtd=QDWtuR(tf^><15L6?XnFxY zydo!qkJ$Y8@Y1id$+98FzpBQOFq) ztWRh-ZrV0^)nvQBq8(IiZNcxs1K)1Q`TID@4B5qwYh7( z?!f@8{z+eU7*GqyPi*;+ni2+;#9q~d%~#v}${WE)FBWbw2}o3tJ}$UY273n_ikz!j zDeQzie;ZeH5r=)i5xO6T=_8nFr%?gQGU`lUbX4!ks$^Ca7v7eLo|=}Zq%(A22^teR z9#_*U=@vqX5`-T!en;n8sYNfeM_^X272sXPGAC0{8ZAAz+ z%#z$h=w&7z>7l&mTLss86m-v@ES0%TS{_GA3ngBye>PO7D?S#kXNkC z<`QV5DD5{z7?AE*WMj85+=7}0h4AUGL6@r<+PXX@lN`V(guO@`(gZ;HFW)O)W5ba= z3A(`+E6Ygc3YLg(u4s%Nk{;{5SSjsWTyd37=2}@iJ%_^|SDDtB-7mSvrQU#fMY}rX z`R}3L0X(S=7DUoM5%TvyL}`BN0#~7Y1(Uo*3CC&GZ=R4H)Hx(vsDx=~9yq`HBXBciD+hTnwVyJB$%=Edv@G$zM5Y zkT!8pC-p|gDGxA@cPw{oZM!ePd>^kor!oJHU)*1Hi*&Jw!I6eQZ;wzc5hq#(`L#pgPLhP>F=nC%l^ z28x&#{p0%5y{EjAb^er9g1Bs>7y%81!h2c2_E5BPS|7)+L(FrpGp7hG(rg1HOTA=n z;jA`0LuK5mYxPLxeZHJ=ntG^QF#2qCkl~&@!8qt`Ln(eZvsw45^wjDmX}VceJS%Ea zyWG4}`BG2YRX_R|ygS)Bo|;eoE1h`&na={Y?PNmXY`pE=XwY+;WR%fEGgM!F11Ygu z+Gv_GOAlbMP{`0a_?Y)7ZJT ztloL%Vt~wo-ZC3+UaUMmTLG9zL7bXV{)}vDM9hI_4Oa_`e*6Ua@m5;VqUcdy;*==>By*6R_!*Z0=*VI(M*P5AM@r@5#Jq+ z`(-!mnjtv%b*kTtqSt=BK|&~&*oJeIud4gwP1dg1h#>w*q>zD@tW!53*|L4=JGaK> zuy^KLnx^9zVeF3RO83QzLRq*$4ZF~pn2xDx!ZN&434kV6@#c=suuRs_P*`lGVe33{eD6s}TMdVqYoN+X&sgixXOItd*k3lL?r4!U>;zTJa)1<6qCH(bF z%D7{bA|ihGdaJyvosat(Baq?t7Cs^E*~{2@YBs!}IuTe3icIiDOSZkelAB%h3Hi6e+m(33APJ(UD9xMEBFj~%x;NxiRdm3pMu!bI<8)pG z^$#~nY``ywoO)PM|M2iqvGCfYEH9}ze(xm3)?u@E9H{|By67!v*HJ{(r0h(pQ}1}0 z&+D8RHUJR@`7_rcI9$e`+P6W(%%5)~r%~4v_%4P!o`G(O9E$AVCya4<^e>9^0S4Tk?z7wx@(^Putp~_$ICU58b&9S?#h$ z&Hc_pj?CZf{62LCsq8{H%afC8`J?L8zOWQwg@) zAU_f8d#ljVHnvX=crc-dzhFb5B)MR-8T`~WTo8n?m#%nxS~P-jpGfMj!XsxoZ)7~r{Gze`5DhI2+hBIiZnOpPlt_4LbcV+Y?CC!0*##wV49SJ}*!*p)*O(4s9 z$5Fkn+3BwmZ18Y+Tt9vpZNps^a3zS>$>kD`-akRdRaeTBn6c$JJi{~mRVH&M~+!Hat*bYKh>10rBR#gW0<1xJ4hOMhH9UL(n_HYtsy}0wb61I@8Jld zhb_FG8o9sLEZtA-IM3OuFL<}>QDu0Pdp<_5S!|si7@DdvmLaH;4E1b+3$_}twMR@9;%)Ck zqKJ&}t(iq{XiU@I_CJg-=mDp+m3&@K8|oWU?2M?+^n!jca8PvT=Sr&$eltQj7{Nk0 z{c5Xxs}NA|=TPuaA*e}?A6_5!8}02D>=J%^(%1LTt10%z z@pC{&+`4LmqH!@lRwCK2pBBLFSVqh9WXtMJDwUVv*-a>eqgGPgXD zT3NPvy~V1=Q5@e5ph_pUHKxT!I>vP^{H8r5Jz2cj((3m2jd|-Y(--0jg74qbo5mQ) zizqtt;VW5_^2bU^^ry=)Wl+#{pvN^<$-Sx$@(Bxgpw6|x)xmtY%0&LKDEF_1cIED@ zAqbn5_iT;SQHCk;AOTq4HVNSB8rX*czFNZN!=dKXn$;2u1RFlRf3Hf0_zJCuKpzgV z?Y@=ds-2u1Bwp%LW^fIza+kSQQ&#EA%6OYEb~ds^SEFGY^$>ZOJX?%w$At} zsfmI0VR#&@u(oHKTG?eSUVth=Rx`qSVJ!*j&w|1^gcn^rgDnnFj3c62R(-!*b^ zAHVI&KsMF!>REAepzV2Rr_J-UrGul-D=xjkmi{^BOYCIqJ$B*6mlXw; zn!f!M_E;uau+{3Sa{7Vcg9=4sHSqN9h|J@OqrUsVs+fp@DB_8mzAR{g3Pam9Y@*kf zKb1z-?VRJt{I}ne)T`Q^_)q50l@W&)zfDCPMy~LIAFQXSH4n{3%Z9a}S(l&+ z5exsc%4Mv;`Gm7@L@T@KhkOO+I_Zk)X5G$)=j_hgZ)d@ z6Cjf2n-_h{$yC?-1ER@fFdwg}+J*&zlw< z_DPa58=5Nks}jMyln94yg`Yht z1xK)M_i$&7mLA3TdBrvYTi zBkUnJEV!V$H0=XoukO;vLoR}^zO2r?DjwJ6Y`DGumaZh2ReVj}wxv8y-OAU_Ii0(F zFx$UxA*9B65P*9NL-(Mj^rJ@osxwG5D~YoUwoJHwvHpBlAHpE(Q)g0?lzh#iH^d2$ zyG9}JmdyB)>|H`;@IEUzYfG;2GkT`u`1|L9`l|ewgQ?Kz&`1;#-IFH6SGkTnj1D^B z(0#k+W0LYG5uq|6b6LA!oMFFMn5n%)7p6I3`$-FyH|MuIiL*kw55OgCuMRXvifB}# zI{bEc5(P9wGwe=bO)h4dxjQL|vsOvn+Wp4h>DfbPXRRn$@?x3$j8d1v6A zRtqHEHk@B^jRa0*RlgK0-IalFqBzu$5d|6#^tlq-@@-08Ibq=jMD(()#Qym)2mIfg zlVRa8T--(IU1hO|p8PB36=VghUQGzul7~Ddc7oTcjF!jlzBb>xOrQ$qTx5E5jt^flZh;%IHz>ET1E$?NR} z@Ow0ixko#<59X7ZWyQ*EgMQSLKd;S(LgNSDO5Un(5*tK4h5hmAnls-m3z7ZqR}=2? zo=s?>u(H7QwdJKrwgsJRDVJywXExt8T_yPpsM+*R+Ne$`hvlM7ucdLWH!R8=9}DlL z?)zn9oC~7{biPC{s-|aTRcgetaICAs--m)MJ5uLlaC+BvDc65$#UNrzLz5=1IUsoWS9AtRfQd}u!p%z{%OxDp=Hurc^A#7q^mIiW3%_mVpzkh%WzLx*Ra{oz;c<~u9 ztZMqPDqG6L{|_}m%D$mi(={`#~*jCU4grzg*4xE zUG>(zK4A?H$J>|!F$H1@{7e)OFAcmpOh^?ILzWc+dBr!yW3^~?`st^$$U4+z@laen zD;~GJlggkR@%q{0Mw*Aed1TvQ1J8{RmF7TI56VUFBXw@@n)wT%O>(XslV^)ae8%-e@KJ z>$ykUN}VLTRN{gF4ZZwQ8#Lf(8@$FFcI zuD{2Am^>SxeQRXpMysCogDX`?leTlmQaj?X1KdC6(DesE$13KUxEU*KZGC$6by4!4 zH@#txJ@JyA+;?|mvs%iWv@gCGXYarDovl5%vmLPaUIZ>ILJM#@)_Pq~PH1NzzWa`C zw&^<5Ee}Op0lX+@;Llf8t`RU=gjpI9^{&^FHsaAg8ImXTM~$7FsV z=D(6r>-aWVxyAP1`=7H{-zv8)Mzpo9wpa(6HCR;zR(O1uNP`#9twHdLzN5BE8FFR1 z?M2debUpt2Yoh?4J5rkS9W7!3?&8Me{s&&Rf8P3#^^nwDUf5!=34agW>U+rHS?f&lQNS-5Z@;vHrI3c~=wys2O?ph$@_FWDPXBtc1*YnC#cj zqG8Wu0H6WUAl(W5eB=pvWH9gf?pg4O0M1IF`RJpM+z+GfRtL!Zg&acXOK&N+?hDUs zy6Gn2RywCt+gVAcPig>FhZ5Ko_Ss3_>m82c9tefoZ{HCU?N)$c(cmVaMyU}XIdYj0 zy-(+)F2Af&q}3d`qX>C65n(Zj#^!$?9b@CZnProw&$MD9peAr3;Q^cU4d)goJQi<% z$rR9FNUE!zfsmQNuT}?JZ=IuT+}P>%3ckTczVeQZ{&1pQe(me__!F<&`DY(vTMr*f zJH|h60)d@?YFXLpqjXW)b_JRZZA$kRz7i|wMzPqhQ~`@3Cv_1(7D!3QCr ze}R@lInjovTPuK8X;D;SPG-1Rm@{jNwJpKCkpawV@iE(VCF_o>2}MxO%EhFT!N0mD z!C;(2(2{iadg$DB5yE?JQhm$p5Xh zL9`02{1uWzX)*Hm`{w@>eCgJsRsO*TAJ|!Eo#mjbz+EjsC<)xv1YP`jQAq-m?$@~C zJey|}OiiG%Cifd|xS{*(lO~t^CkoURh#FSWwbgTB+|m0aa4pWeo0l!*5uVi;rgy1~ z|MANjGoi6z+7#! zToBONC`r>Slr>myC`jBh&phL<8f4`mHxNqt7E44^jq92RF<`ryu?B7wb!rI0gORZV zq|tmYjIm2Nrl;mu{tgw8l*Qt3UHx>duySm_UDmhF)*Xmju1PlXwYO{}a_sBx*V)LI z9<&2?`P9xk=Wy%Lj`>#(F8LJjm!M%~Xyp>xA<~Fu?$xC6U&W#tnrrzxpe3I)q9-(s zKx)1-6_qH}9`WeZGY=tVM6Q+3`p#WgB>^_-ar2c=Y=+H;_qQE)D6wb$uMGdpPuPj4 z?u5KK+e(mg%g}DlLv|EiU0yNW+7{7v8ckFmXX?_|dvyRY@^h5Rhb%|U1F0kp- zzO^=`olw%4>au$cvTz;0Y&VARiQWjHb0}){cSM7K{`g9IEb#Lvk0ty|KlcEHltY^1YO8X+ zP>v7Y36H#QWlV>qps2;I8UmYZo=2Qe+i0V{w#JbDR@kP9{!Bk=mQ*Im^3~PwO|lLh1rQBY;ralfwff%$I;Q!{H+AskPluZ36ahHP^pt-rujOS zIv<@vNgX1n&U?pFr{|hH2YZxK?|2MYsTF8l3rWw(RHi>pQBNc)uS`{>S?mtN5ga`! zKk6Lbr*qBAi`**i}_u1J#{Bok*dglZ7I0Mz6&%fS^a!(>KSzrI& zGE0%49yQHM(6+U3qof510jah4GRvn+AA9V6NN!UQ6wypT!&fd3 zqJ(BE{eHuZ*KxN6W5)rwGH4eR0L1b-`Pu>wjaUX{*C3O{uLdV&h4}z*0KCbf)&$w8 z#wEje`|{gwY}$;O__eMh_*=Qnnp9z9zePZAtMW<6Mq>r7jvN5bO*UT7?tAPB>V1`c zH*Q^9f8ABBYnQe*pl@GH@c;s=%dLD)nVZZsqAt)tXQ<~V;Pb9KAGN#hdell#5;^bO zqinqmSH`RjlNqVwQ(_lZRYB_D1El)11Wp0PT$ei({8`we1e_!OhTZUw!vVv)@F;<2 zN+Ldx0IW_Hbh6Dh-`36o;Qb$_fFri(YwN7Nveiue4ozY5Ag}1hW+AB-jm`!@)tbsY z`_~;0*+qYQ(fSO^w#zO$(}u0x$4U$Y9CJK@tq{twY5r1X0Dkdb)njQB^;WQs>RNbk z zew{mawjFlZ0e{b*x)q0f-70{a__HR{K^sfi>$zo-!USJqRseRTJ@)56|Jm-m^G+K! zY?vKz-~nD9xu%eLZk%EY#1x1r@JppY;Ee)HH31EaP~y+xp}X(CyYp*-ywW;7@4WMD z%{A9_-?Dl}zn?ON`GK}!lC3hRteT|DQbNJv+Mi-w}3iXid2jx9{_;3#lItq&?`O6i^tK_nl?=9Y+P)5;J<3?9}I?B>RzJ5!w1uNc4rD&nfMr=z zo%QNkWWD;YXB!P$!;SzT9sTC#cHI^C+J7JV4EJ*Xv9r%Qgavp2Qe-t@4-L?Do^t4< zfC~W|O0+pC*UU3otL&tpbkh@&xc`8atZc?NxEPT9>biSKRLG~)v2&X|JL9Bdae;A( zJ^AFTcH$X(xV#v-zl-upVacAa+5xuJSZ>Na2{8cxtG@qk0&>et0IO;C-8U0lmYy{W zWt%BjovJJZ?H16Kl({D=vR%8hw;^lvw|gJ_(9S&V54O{8TiM{%dlNu5-`ck+0chnR z+-5WY*e2wx7TH&YRS>+dzMW)$Jom5me~*vB<;5U7^`w0PP6@(W0GO-b;MJlL@6p6G z$T17}p4@38HpTaF!5reZ%s~#q{R6>!8|w;f-(7aMhaY{(&OZM-JMT}o*{Z#|*nl;A zno+L+z()Q}BO3W2-9-aC(i&HLuRwc$;`Ca3_=)!|$+Su1nfO$nzD2hF zkbbuJuA5q)9<5L=Dzxg^GnoU=J6?rFx#oOb#~L3r9xNyYEJ*LvPh*Il%LI5e9_gCS zYkW#D|Ek9tTV-s-rHNA1*PJCdS#{iEQok2~3;$3I8*4!M3kzBG*rdTG#zl*KW|7bwho9uu8 zea9Yp;t8~6PNqdjK+>vW9>3_*dmoWBE2-B3Tx}U#04>!_HE6jnEUu}aWn&svjmb#~ zEAUbO?YCOjw!*At?ZG{*v_!%xu3DH3)WSb%@qfml z-S(d+ZT$CUC!e;p{cfk<5zToln$qP6wFuypH5)TKmjuMme1H^qs>Gnoabc?>`{dKl z?UBb`aKG2Dz5b1T^~Ie4l(X@ZraDL9dP#99E5u3b(XG3EN_6Gd-hSWK*`PaQSH5T$ zyO~VN6jIr@klg2PQGv~_$^p!O&Mx}xtO#-vC z@qfsvCWQU0G$`M~OPZfjoih{drXIn&u5uP&vV@mr*E8>3ihT)yb@-Q4FtsP2e9|Y@ zyY9NHPny+k1GuZKgK)L5*NFFP!Yx+^H{N)o$uFcdwKR#JJbALc@x~jjX}0RBtNJ)0 zjk5sm27ya?EQ=%oP2-TX#gx>yR8N7sGQU0b)KlH>ql7}0L3Pa>x0Xd2;yhvs#1x1r zuskRr{v3Fw_9LyPYAoEg_t)KU!wvSQKmEzhJo8KkOU2_d6*Tlrrfj}E%BVa8$g6j$ z3_4c1wJ!iyici(+|aZvse#hb;WRR@g`%0rX- z2@@t*&z?Qq<FwSlNzlQADJ$S`X4m*kW&*_jCCq(<5F{31)8$>ylr5yz82xB z4jKiZ*cuDSmJG2LlX60Wm=&|9SJ;~$d};ss_kH&Mdy^ge)EcdRid9j^v{*_7vg-2h z%=?;XM8ZSY3jta0{O-d6#hje80} z9(}p@KlsF6d3Azy?~RMLy$`aU1AE%6XT8VU~3($mr z+wOb(TU%{a;u@go{K=QIY}WK|m_JZHtmHm;>szZu%ec0o)Lt9?x%K^KqP1_Gu&%vV zvRstK zMDv7oIR>=gS4LK%(Kse05(V)LV_NfV$VFpE7(aARrp&KnjWvP4YaZ?jhcJ>lxncA< zDgAnJ6P0gN{GB>9kx|mI5+>`OQp!vV;0=qL{jdGLfLQ+8>#+>WFJ_A$P69GzgF09MCySRKxWc?#6gqHlEL;^Z3N^NXT@+FH<5gsE4TR}tWKGUlD5#uU{(XDRE^w@OvM(U2@|m<%-|D%t~CTw z$RqGr9Y2Adb)5G>#v>4`ku_L(B@-Cr#YX&Y732Z!q2Vg6&?Mz(lr*ZwNHeS^a+xpA zQtSlatu%o(pk~p}c-j;MI0_u5ffnFV6C*Sq$|pD2@hAP+9vV5vHX_K{f%|P>-Me;j zjg{8zI@`24HFoL=CmTu($eqe0&l;<$d3TK!v9NN=2?yct?i(UlPq)uLooHjnd}sIG z{iZ$i-#6_q7wl-e?tnjFw27PO5ZV0XrlE;}eV_gIv^)QE7k-SNwpCW`ji05yh{cil zoOc@erBmG`9P`U+(7QQBT^sl|$<|%$xsgxX7hhwBhYbDXz`0pm!$e4$#opIN?sTHuD!l9mp3Sn@)QmAj-7Ve$(~>lUxDG|J6^6ElPMg`y*NntP(^%VKS7|wvO-_{kS`$U(QboW0i`YoXk*8a^~jtySwnTd8|D8#Cq`0-s$bp~UmipGLsi zx;ay=&w#de)}Icy-+ea0UVr0#n?|JHmtXqO{&MMqcF%(k+J%2T$hI7@Ixf#N{{`IP zTT8)lt+iJ}+pwG6a@%OT=IR^Jf;`sxbnk>E=WJYn)nHAVk_Q3ST>_QzRZ3R(LLjU* zP_KF_rBcnrhy z(^_=RwlN=mY^~Y-{;Tio$U{z&E(hQpGB7VILUWh+c~SbV(%wD#Qftez2k+g}PCoT$ zYt^+BXh9nCbLo}3JKp?8DA0srQ7+?yTm|5P1uUh(3UaXo5q{CaT*7^AFLm9e4Cjp# zJ_5EwL!E=qmy-Mj8@^sK?pu1;u;J@k_ik;hV|&Wf2)2XrTwb!sF`g{6%w^>eHFC9Y zlFtuQ`y9%hr^Z1oZpu0@Sa^tc1pbk^uPijz&b+2^BqS+;XilI!bzc0kUe_tRT#9Pk zk`j^oOy?e@Fn`p%+f&%44#f|%u()po1-4pQ(m6pi^Cqu7_X+gP#bmx+8_aT2DX$bKo}QNo6Q!(- z&Z;x&kS(QXfNbf?!p8l8YuNcPqme_XBdXBby~4ARPXlK#>l z;S|k!og`)j0(C2zfxuOeM;0ku2{)(MzMnAFK}ZQ`8c-snmiIGZD?kzV`|r%gjr+m+ z^lD`__~6Y(xJ$5V-$-9*;-MxQu_7LiL-v#B(7ye<@|MXA+-T~y#|;B3vy!U|iiLI( z^pNMJA&{uawin<0#9kad$2Q)i5JB}E>({+Ct`&&rtk)N`vFUU294Jj{JWBFB^{NM? z&ZFKnwUtESZL$ql>0xUz87tnuqkZ@7G<)^s5A2e^+-BEZ`=|}*)d};kzVr+FITN`i z`bSevt?jknFdH~9-%dL57CY&<8|-hF>~Fj6xC!;g-`kw&2=4%G{tZ&v2~B|O)U_6R zHs%%D)CqO=B;4(CmwzXBe z@wU>Q8{sn;A)85M#heNPA{JQ_D0?p7R^WG%MXZ`xQLwh`M0>mQ){*wt3tq79J<;5~ z^l%%#!79*?>+08NT8rh@*P#KLQ@}eTQ%fJTT)?L+K8SN~Na1+C_LpjRbGf}g(S%Te z0}I$btH0HgQQ!b&CNM#imX5!R*9IT6pN{#|{q73jlm=CMg8zw*n%oB9AQWBOz{|tA zpiQM?0mARR^Nzbfm^5jUYwGT?#~$wH;hJl&ae?WOLk{uDy1q{)up9CXV@k+>X|`@! zMVnJQkOo$V4jtSlr9fu^1p=A{>Z%M{w3Hi)AkZz%`{HL}3d9tMDX=^#ApR(R7RI>n zE*)o{7he&3TqtQ2@}$n(c+L!K?=5qNBaH1i5x|MV&uuZ!3u=O|Yj)q;0eewAOyXU_z z+s(JWYfn8l(pFoer?oF=%@-)5uSh;L>)xZ#cG+n!NK|S6{ojWW?P@#!{FALuPXh4b zzqkQyJqOApAL`pd7oW51yYXMvU3JLBkkfxjG?5eeC6`ypB8q8cT~es}w)T z^;I^3DR+Vx7&Y?aN&TzJXIleib~(tbS;*y*?d3WRq0YA5a#b*}nOOC8WI=?0$~ALP z{9uurZ(2~;)+SA?u}6p((;7X2ZAPqOE0wl~rL94JmhXNxxISQPP}OWsJ6w$m%tKPt^UI>2aHE#6qjWFH zd?1x+HlTG*vW8eVldY&Vb<#cX1Ldc&hCiu0SG-MV9V=eV(=|>v9U217l|jl^V^J5+ zWi`M7MU5Pu)SR`?Rs+gBLPd1PlKxF#+QCrbz^wNI~dV-hL(Ebt<%ab+}C$N(Nvee?*mF6QPR$EEyJexja zj{7z0&^gOK{Rq*t0W)+K0TjeR9kxg|WSB{PH=dwuJgh|&4}3;jm8L^2n(4(w#rT*m zx10WTzg>F8*$C!!$m;~3$R`*C5_MfomKAsJV-MW+tQQ`OL@Q~_yDA#-DJs7T1e2;s z@>3sUQp6%k9=>E<)G+Ux~}vid9Z-|Bs?`+ivyJ`EaV4%qE@%` zNwm_$^^{Xi@rApaZ@$?l#NyRq!W@J|-5Yp)^EZYGw^q*YyYD^+Pc<1n;)o+W@Qqg0 zpMU;&B6q*$;DRRI0#)@60mvbRF(Mo<%~ryigf~t41pw%{ZQHgUrCmYQq#1V6MHi)g z?y8&;=2Q;VZ$WijnzF>t#1x1r5K~}TQ9wMn`D8TYt!w&;hi39rU6(&v@j`*TfB3^6 z+||J5n{Vz5e3}%CU;bPvlv~>X^v;+u!zb&#diAm+k37;=TWvM_=Rg1H8pwwpdZ@el zS!0bg+*iEHt~!Ws3$Rw*6;y8Y=+XA{(@*=ptTmQoYzbwOP^Wx@hOzPw#icye_$8D+A``xAhYZe>)N3$lFn3n`~5MvEm+O+87pgQXXC@O z+*)^8)!u#kGkbN^TZY~Z05G`%tXes83Q1bO!em~0ZjQbC*5|g?hO670@t;{GAaY(l zi|P=#9v4~#ZF<2$ z*z%AS1#mZEt=5n%L|q{VS7aS*+Qb?5(4&v|_v+fCz2)JrxNhb&%f+_7x(>ynBDo18 zRe}K17qYPq+-#Foi3PGMt_(}47lG|6r@J4~Y5=o>!VdQSyAxP2`6F7+C1^JvV1s+( z`VE;}AF-}%HlSCK$yD1o{pu-@o`28*Xu;}tG&N$P(B3$uxrJZ~{UY8s?{{A^zSa`8 zgoVsa#Y8IXgz`~oi9lJ+HDtjjg%bKE^19;Q_&N4f;;8e?eZKjg=FZ~HQdlWxLF4=H zzxP|x<)(g0I#&MrX*>{5(0HNq+7C&Vg>%oL$Vb1@vXD|E?izKkH0T|dQa?`t&*Ay` zw%o5Gs9QBiGoSfkTGMyFn^(LOf!nNjHE&*HbyT&pRD<;q!n?xJ_-;3|)vk$Js*?#gVwfqfU4>#3bOKLA*BAbHW5 z$9McR6ri7^2fx%C`D_8)okE1Mi~n{5nyxi=?1=~19=i;;RtT33xM^tBD&Hy)%*KtI zW`DoxGM8)D9Xi-pWCwh$=05z-VrG&+K%G%hZ7;m~v8~*7Eqams&w(A6cClxl|H%IL z;0u7HFn_daeGyNcf+RsO2XIqE0?Q#?HBvh}`RUw0~UjH(PbhmFes(ZZ*lb8mWO<2j4SM$K(O+ zn4$#Q5*1S{k2d8#cloA1o01B2^LFMLEa*=L|K=7gh9zb8hY(aPhHE*u|#FnaVV<*RV5vZmyZ}R!u=|0@lgKUWg#Npyz(=wXSGNfKxzf{)C66N zh;qlHv2EbMfer#MP8~DL70Mtjukap?b35<6v-_%l2r+Mb2f(spXW}PP=Srx7_tP0d-%p!;jp-R>H(2xq>(ja!)5Q4WTRyp~$ZZNdX$=7PhwhwgiKcjKw!;-#O2^ zc5iPljGbm*emBvEY~I7N=0HOQLO}`QwHH6Lt8RSMFjhkZL}L#iNHSY}U6H-`(z|vm z0dsfQc^fRsHbX|9Njswai6Q{(MlZ9h?Ao=%T8bo1V##!JSJBAzsb^vHnM(@`}AG)O(o+)lX}2n=!<2%Av&T=fR-^q&XNGl zNt8vqti!XZJVM^OU%!@Qe<`ep78+9=2+q*1mGY6|z&P!g zxVysBq6*jSy0UXWzFs#jQq2`P#Q<;fuP@M6tFx&_+x+rU9V^ckz+!=}C3KqEXcWp^ zNt0wnkPS(lL@9I_bqEDjQs)iV!gb%$;iUPej;GGnQp)V;+{7)5?%*xu^HZEt9aUHP zXpujz1~C*EgFq?KA{$2GgZmNzJU>Or<99d(yxnKgA|>FUA<(+C$a?ndW?wxt)=oa- z4pzS3w5_)qPQZ_LR>k#K-y*QuLoeF+uN7RZqiwhKrue!7#H39q8f^|X{JZbAy}dsA zemnb*SL5SrLmR%y5CpSId-aX~*}wn$zV++1l1=?~0+=gcB>lz{?I@>JAwm1*IGDHV z9>eWV=l#!K9zD`F8NNP2p*s5{?%8KX+Jg^$Vw0wr{o%~vw(HI#=qjwh(`D*0S!v3t zMriNFkp8k=bit)oKj$MW$ex86+Z;RSki%{D)%sX#v~wT0`*Um4>TD)V8(S&wedgKc z?Ed?ovd_lMVPb^<00*vXsK(XxnhDL98IR!@u7c`_nmq?0oiys_}N&mFHO&ny8i4 z^>#Nj&?@)LIn&VOy_$2=m$>nlMbL)0z1M{Y3rmNu+LVu=HOnan#jzJZ|Rl< zWFYp{GONpMcrEUGy=bH2GubW#EoDaJ?)->~3j7xD4cfK}Hm1qH{2)pj%E4bILjl~) z`$32kxT^`czzpqQe%a+j^X~6pv_Nbn1$YboAD8C+@iQ?6VhY3*Sl$#6PYe8Q zQSZ`y0wpu$Yw=g{c`fM4AMa_Wo#ws@_uqeicVQFq)U!*=i$fkiW~*{*e2_+#__6?1 zjY%3ev_QD_+G{&Veb!lL*{7d=>H$IJl0x^Zz8Y6l{;js!%C&eEtW7Q(^qkIXEXy3v zTnj5D@-3*`%2#F5#9x*nia{b8%Fq0M`|aldwB|(0HkR>5X>DJR9cRx!H`*S)|0~ztJMxH~ZIyu? zaXl~@xk`Cd*t#1JwPBm|wnv})+73JVcsp?a?Q9^{Uf)lgZa3X_rcIqvXZ?DY+P7a2 z4+cQ95nsC+$`zD53yo1tdsZJ%f?I>Gw#lZa5X@|2TWbii7e05#j2UB(J^G@(@?xcH z#GZHFVQl$EIE&3g%P&g{_C*cEJ;(zLHsrJmZ1}KsSd^b+8?Lu5WnIO#++vtL@yx?^ z(O)mNnKQSwVYttlIcthN`@$%DYUDWU-?yEOmc^+w0(^eUVmX?E(iwRD)k*f!%m21d z$GmInth<`6+@-U<_0}`?$b-+jYqE8Q47OHSdL@zN8z8*agsZrcD?>*~Wf}b}ye{{q+j$vP;jj)`i(thAf(oTZXcVGJE>@H|>TSKXIVA zZ=dIE*8{g<&M+R3y3xvK%>vkM56F+~*oc2?AyHJi2`jMdMzqFxcNT%&vateg06kC8 z?;7Ysar$jjKs>6Ewg5~KHAl7qZ;67`XUs1aq!{-jNmN6ya+m)#u}mfL&A>Zb0-2-R z0;g1G3j5R}#dnKQzUtniJM(LpMP?FEoRM_D<_p?}@h)@B3yYpQC;ZF7SUpcX9-WKM z!|UL2`pE@iS|e8sPkr2z$~iarTuz7Li-Lvp4dz~6VfW^?f)$nosfuT|2K7)Hml~;q zrE}Z+kD$`rOw`%b(KHa&GtesOUDD>2Fcp%{DP?Xombl=Qy2&+LHR_X;rv{VYK)=yB zjfAeb!k2e;I;8*&9$*lfTdq`=3*XG!pBc>N?Pv>nI8~JeT~42mf>%X#fjsFHXplY; z@BFeU5PVEEFQrT_@I#e{Kd|GDKgimxJj$-P>KXeFK6)2B|djw>q<0!d8AH!qfMa-Dm^iF?{AgG%huOa9LueE4A_Mwm@w!Ck#V3I?~{ zdZ=wae26sypw`T(umapi$d^$A{A6XBkxUy72!VND0};z*D!Vb&?f)zjkWEID+30wcI`^> z*?N+7?EDzo#bfQAw;%JBOx(IzIlpbT%C}QbI?D2L6+{j|n2Aw?z+Jgw;d+wMP(jxc zIpw&RX|R3w-vCz>6YS#iN7?bmK4SG%(`+~BR*x^ZdW3<3JT!#)BvrUTc=m~>aOYsQ z?`~__s;dxmW!gBj63{?^-ZC*#1taSN1bGJ0CWxqKHobd7I5_ z@LFA<6M@7)s>Kv|jV zD*f$me{+`wn#2b1HRPMg)55-Zdk8G^-h1!4Z^Sp=c*C~cc3TH=W#u7v4Vn;tLEyos zo_fkwS!EU5ZoBPVv)0EJ>LlSx$?N2MOa3SGS>>m^!z5qtJ?W&A+{9LzUz&tVcvJmD zioxALiFg}RAf~{tkph8_s_f#2;(_AhIv=!=K>p@$yo{93#* zGq2@Ai@^Ic`L@nwrciEyy#jazs;ca=9F%Wf1!5CFzxLW|2@3Xs^Z21dhk9Mb&-J{t zc;##O{rBJZK%YZ~4DrQA(It#yneSZKei&1v;3Df2y+dP(f|BmG+itE&{K_k@xB^T| z`2Yh%aD?Ol>pJ=1rc?@tC zU=rYQB`mwHx#|>q;;~V7%b2%(@muq|vN>h=ckPAzIuS6`tVfrmU3cY0cIe^9*tJ)V zVhkR|7)|x5dktmSbLW-pl#`FJfdks3X-c`PrD@3fn*fXF%(AmjJ)V<`*0w|6xz4I8 zCt{s84MhgxNYu>$wC-y+{PQq?;alwb>prmD8$R%F#FQEU)$E)zhS`3*{tl%CTzBEt zq7mhVWESq(C~tjr89t6H5NDD9va7Az%3X2KaF!*HKi5W%tgsz-I-a(9qV?+86)cBE zbL#3z-&)A3)c-1RLqT4$XuIpqPmoDCMY(yELVF+|o2|QE(ze}Z3)^q+y`W<`WtnOv zxwuq8jxKH60T&D3yBw-%vV15znvVH*9epN`2}(BmK1+}qHQzLceobwmN;V4(vXr*+ue8H-Hto%INNoX zUHx41N;T1-yh+1%eSpwW!zBDhzGYR@KgLfH4P18K3rN9_VvCnucu+^tRWWw{5>V-j zZ>#Ooad|d)==S&&Z9}K2_XxIMef5Fbhv>M;GO@@|W`VR7<9{FQZs`Jj5MYC+3yz|{O{|Nyu z^1-6cF73mLaspZu_pXlqD%qcEV5a%3+#)rQ|;^VldPh) z+E!hyADWIUq0Nh@Su|G1eK(E>r=`}ZEoNUenrum%r443mNi<^1(e(Y?egG6LpiKP+ z_9BR4XY1IZ2;MY>#hE%tg`d2Fw)Vx>vn`2$+I405MlENM1Pmu7a*Gj`0UgK9BnU{g zwQrS&W(0viDu3Yl1~&&x7DL~HT$?z48uV|l9xHXVHeH}KPV40x*+Ep#4bsy;m3;D~ zom>P8%mwQjOY9p0!N~_&n^s9{+q%*hn?C!l*2YhqWb3Tf5lvhqYx;J5QxWthsxhW= zg`le*l7QFwR1F{7z{k{!nFj4dnh+&LOi6Mpt!**S*TS>ZloVu5ZA=1wOZ|Vob1m@* zcn$ns?6mYt(cHhlpVk)g$1>(_p2X~oWG}qh> zygLX@TGbYCFmmKbJN)p&-528Nr=RYotD1-lOps=*%w+{4y#4muu0k{tMvnrp;uQCawkkF!Xdb^!4v%J5s8$SSbHlCA_T8wV&}WL*ZVVKcs; zV_$wX!9M+BED8!!txelp7Jhr!fPP)AU3-9YeA{M&`sNXKP9?W&{K_JW*CdN=MtKvm zTN&18O>Sk@w?{iGF5w!A5P3zN?UT>y2~&X`c6czoElY2F>%t)6|a3Yg^g9PfoIdgSWAvYYkuK;oji*wxv`J{29ZE*Jn z>qNk`EatdA4@~0;%mF>6=5vr90pKosWM7EY08${u5JRnNlDvAg35{I&(T0%sOqFfnZgP@U-oSfi`0O zejcc-1{H<|{H`VdQDur%B`^{#)jXWAyM#sCR?r3wDs@D@?Y2^pbzXU}RSAqirL;{e z%(U>ih5Z+ipCtfhSxv=M%dNrhEP_`PngRU>YE_=1G)==LM7j0q)zLZ-`gV)Wdida! zz?W`TQ4<1Y855dvD@_u(O7AC#h5E7&GpNkA$}P9`)(}YC%H1BwDZSy}tVB{L0%PaW zgtcF*jjg?IXUhg?&L^Vk>>1x%K8rev7dVjMX$=V8+4M=d@)Eq6g{E>FqA$1Zjga1V zO~6w$2hbEyv|iK&&|aQ_P>QcGe1WMvjg-GP8qN}k8?$iP0Po60b3i_~<+oSBZ||N2 zm#DzJG+REwDhS?2If<4;y@PvP;rjhOvC-JJt%gn0Xky#8?X+o|c4OPt#J2lR z&pF@U`w!%rD|_$fxv|z~F}13rElxn>*r-?&uhIZ&KWUqbiBL#bjD@niGtZH-uYxn+ zL$1=u*oZLqyZqK}iY;?(chJW5#X|BdtCTx@pKv~fAUOaPKX70 zv6vG{#msaX?fPGM9eM)@#HH+wp%-HJ2ka(KP;tbYshbJW>afqRe;iS#uhlG-flWa?D(v&Wz2Ro%$_ST z&jtg)RsH8O@IZ8om0lFR4ll<5BL+O@;aSXI68K3svzxmMyKiWg7=pYS`YQ<~P$_2W zi#O&?ekgvnyOOZ!a5U3JwZi&Ov5-|Q;UL1gmTooF9;pegLDz~LO2|fUtoMZAJCl+l(N50JKfO^ZVABY~Q~Q0gg~~Eyt;W^WBvTWwg-nVE(XI)PG2Q^)E*q3>u*w=Srm@QzvCcN+exF!c zmxRx*4yk4y>JGwC&9EKy90Pg<+fqeTv;`kfu|W59M2K#h5};3BG3$xY<5cKoP})Bl z9+(k1&NRjb7Z{6=ZK^dYrZ(Jz5L^YiMr}q^5kw4nf2*#p+MoOes6timzi)e??_Y|^ z!@`Y~u1Bb0suU}{SfD56*o1AbWmsnPW;xA6g&UUYJ z#m~T4)qpFUv+CW9l&-34mBUZlYKSddj)t4PdR$VDx*fEVSeaKvf z_+QJ+$&g11!XF}4L-pabPuT1b(k)v!`gS3|;PfW!Kf*q@8naC!nqbmcNCX;iyA;jM zTkXL&BD^GEP;tNXveZ@p$2Z;&ey|^95 zgFDS0bfqJ`zxAcZ!VS<0=?PMhyDIU;ZU>+d$3Ib*i67a=&8-=8p_Nf>h?8zE{G|C};#UX~-MWcb z0qi3mpz~}UqjaXcN!%}_r%`wRke8so?+l7 zxYeQTXJ_2X?jgNClY!E#rn0G1tEr-?lhu~>O?3f+2c+;>N0y0Pn;T2~@7Jg|eo%tS z894bP4248+nY#e1QTGuJ#mMTCA+A!?5Tf&DgppZQLQ>;HpqVLtlX?3~+Mih*VQ?O8 z2h*(O`BHT@6r^`Il8v=XDb@)=^q$?n`js8VKN1T)8_ie4u)0C<_3n7j-uoXw8BWUW zy$1@-m3c`N>Uub7FwoJ@%f8P?4R(oT4o^ae>(AmcRTgXI`p5h08hA{65F}{U>Vy?! z0P}Go%xu{8@p1~Lt2M6_rQ2KU*w%i>%N zX`Xf(2qP_kny8FHgK5dtSeh+Q`ooVi7AtN8ff7s$$}&K&XHY6hNEmIMg{!>;U2|{EeDg`1(&A24`hvXB#JkpieE5 zwm!k&RI}@Q|NULwr`^%*z-Kg8V`VVN9PqC4>+I{3fVE1xyr3BA;_+g_z9XqWHZ^ zvoQ8$GD`>v24z%H1*%6E+Arqy_KG&B>YHo#RIwUkV6K2=h_i$cj*BmKCY6%K!Ajjc zL!58bc8BJZnUz0L+$KU`<~Svta}}giZ41h9LSfIya2<<|_?Y1U&cMB5mtb)=;v}y$ zVA3Mja+WS2hn`GRV-HJZ8&@*u*8{zWkK7gX#L&7m@z(0f70v{xELHaLXaPy0EO8dB z^+12;c<#trs101C+LLaPymu!{u53%i*{b6jk;IpNLq^4!h+nQ7->+a9dv3|n72VY% zpQsOv+l}8@jp)F3cvmYXD+!g60>sF)G(5P<4ShhpWhn3|nuM~u5Iz>Sa7~rXAPC(Y z-s|F__sMcJO)hapb9&50$$OP)*%&??5U08hcl2$E(UzMHsG_u!;R3#)sCqyWnO3D8 z>ANA@iN{%0N4psiQz~LYDZ6IFe*59=`?feIeX2=pbY`5fr$0K=GrS@Nmp~T7Qp|y( zW#LL9k|~gNsIR99Z|BNi03$|Y44*o3Dxam#azwUGef!#aX$T?ww%5CUV{VRd`4^I| zCWkdkiEGu5Wfcv>yAVWXB$bT?6Li#WoeOzAMK091L6u`A)c~O|S=W<#8V zpAH>!{}3NlBQ<(sCK?Ll$5Z*6{~HVU+iElh!NNg0I?-G&9%Q*P1}<`?9^-djh$oyR znS6*piqbuKtK=YNCLlsAeB?#G+(#=qiCDCk{DD?mvhz#Q<6)_9LgM4eeJjR?ni`E< zE1)K;g*42(3keDGO?^c=PIAEton+1rjU*KjkAraLAWvf2cxQJ(fse0}cI>~O6=de5 z4hqs@vK)I6g!vN(RhsQL)}w-}ta=IFP7q5_R=Z|Xz`LrhODv&S5`+E}zN zFdOqn5eW*yyBi{~ zQzYGD(?fD#76q3SlMW8|_WlQPbU>6s)yG0ZtaIH?j%@3GzQ581cnwsqHQAkB*zCt^ zXqW6JJ6n+Xm$Fk>jz$>#UB(*i&hbr4=3U1fB|ab+m3$?zQI$`W>o7=bbDo+m*I@Nn zxdKDJHo)9pP6JJ>OWdcIRh#M+a6*Vxd39GCR4=(gC*?ljW_wBHAchd;I{HYr@0ATv z)2N8vCEF$({vU(Dvp*EA)7{@7;8|S1I-r`#CYvaceR&y70)^!T8&|}8@@v;4n1(@b z8>(Y|NxP9KVj52#m!bh=Wx2|AtB&S%rG-KbX#4Y_u%(vsqi>r+;v%-brJ+Y)0eNLz zRH#R?IUrOp=n&vlf-`axl2>7Eb*QW%b7m;3N=^v+6VPgurEd(tv3?Yyj!v-+D*VST z>P^Gj{1JVkG4RBBvSM#Wg^~T4&nnNao!Y~P zpGRhYad7xV>Er#?GXlGE^9(|DroYZV^wsjFWyfHZKFEe-yv{n>U*JC|pC|BC*0qu9 z2#8`X<~WntNW?bZ`O zL)&~rB{E1XC?JZwg!`{XI?KE9clN)w>km$}4YNDUB_8puMtbVMn6ng)rnJ4KZ&})g z&)pP=wwaZYTen1tBEj&gG(5p_ebg;Gd||UiTx}4R`kEPnGvYE4s1Jt8S=Tss=1fv6 z%0gviDZM(Fn#RQpK$g@{4sy7uZVpIT?VJ{Kw(-h?FJo&&sUz7C4{tA-W%=eISI*+Z zB85rjLa6T2KChc=7weVM0t<2dQs0BkY$t;j5y5YNfgxQ2$yvxuPxt|^O4-Fj z&mc?Dh?^9h=!2;vKDTan@n<2p$*?zSO-85cZuhL$UKf0j++nJdJV*wV?5}X5*3%lOYD>zmA-{mD!PDpG*8mTsobU+M`tKow2@Y(e7G0HH(2Ok--1eCEjqyi52f zQIFoWbx?bntL&+urQgWgeN7*#i;Ko{WU^N^*KA^cH&h@4E;3>ZO}(1c)CCyQ{mFA2 zGVBIp7(MGc)9k9)<#1A;6#|o?Iq=YJNpES48eNn?Lx9zYnF@EE5BLB4d`a|Vn}-}c zZ>(2V3mR{lOQz8bhRT5I9;GPkArha8+p(*4*6+f*Yev^fAXP*itHV)=-t*=y``^MZ zC9g)wjp4xoU2ivIRiN_g5{s#rSiZ8&1E@ za&7wfz+zSb*?qhTJ`Tcx-xznq>AxM7v$Vd1V!?G;*dno?F%2pLmn=IF=>di0?otIm z@${A&?^t0jV5c>v!Yr^mCfeI^+@R<4juNGIj8_6XWmi4sw`L4N;l5_hJ-+iLcn_?Z zJ8ofD-yl^w)d!j9KC+zUt)?|Mmt6bgn-BD6o-O684V^ep9<0h)o^tPU&IK4{-?l9d z*BUH6J2LyRPPHAQY5&<&A{r^|GtBCjDS%|BPl@vt&25StkvT%E~@A8;vZ^Z6(z zM7NLEGMX99QNs+!(r=hNY#E%WZt^ePSm8*`9s=Tzf$_mt;d!O0LHhs6{@f7Zht+1) zMXtZS_s18)wBL*}uKg2>d?;bZ0PP8(0Z8Z{h1Ay$i9$?x*5wjFYAYHW2;~nDH&3S5 zK`M!-@&&%BG{qQ3YjF%rA49h&fyp@_Pds{;UoC}TS_ zD>x2mtpSGY@#MoS(I3@oM7G-NxGp7~2P!NXThBkqH(2lkbHU08r*q=;uS-Mxf!WXB z8jgreH33}dV*;;OJl`3oethXj*>X3`G*X)gsDr=CHqo%0b|I?zZr8Fo z%e_JQ$Q1=WW8be9$z-q&6M@si(h4n#m0LZ=bUP}>~mYx8VWL%3r)stDHj-a zD4>}5d;L4Wb;49`TMZ6ks?4G!zsX31MB{5K-vC(MUz4Xu7T8f%{gmTb2asuM3Y;=@QYtzv8DuffB+b#0F zn@r;#8!7_Zi2ETlgX+A^t>QLWwJnj6$sBBgBlVmHuphw3jzCStDI`oD7X8#$G6%jy zfo3k4pV{GHhXHsW9q|R?54dD@$)K_~)rA;i9#dHaX{Jaf9Y7XuSHFxD^NMI?PrV<1 z%nRM>Moxwa62P25x=Ls!*HNO0)>h23I$NbF)qFoMT6}oJF;LT5o?x78G}y$rnfm6V z-C9j#+LR!HJNQ`)sTsj#ft4dY7qbNOSaP%vB0$NAN9QeyLweAqc&j`byR=QxOc8Fn z6~F*ZN^1m3{0y}#RGN5N6Sax)PMuX8(gyRSV@2hs;JmoL3G9Pp0 zR&Z1EpTQk=6l4%>+2(Et7*ZDTH?rEAUH$^Lp~hoDO~cC&7}Qm_I4x8+4{suoqv!HL zF!^`cZhJ0wd`D>KU0{)qDo6W}i^kG-7O$wy^>T9~WEi|s^>YVjRlagFzgguvKV9vh zPFm+GPyOgfE*ST&_C4pt;%Z^_5n?>WgL#?vEOpWlt-n!6XP>A%VSxTpT-3nv*Um_v zGyA~|>5)mAKj6eir)*9^?+5m=AA6@pL3L;Ff|`qD_-PPa#8&T?OI%%6N&PGay`!m) z)!IWz+H2}zU%gSe&&|#~ieNgGqKAJRI~;F!fRX;{o>!(MixW;!h$e8p*&jwC2-3sQyrYgKZw za#`9o2EHG4pk!BoXOwUVL1)}s;@34tB0EAe@2`lNeQ3%7y_~6s6p{;{de%Y>wwTIm z>Z}Yjs}mv62_QUz8=~)4u`~?{L{4M zmjX2}bTgj)Yq0z}z2Q3usrhf<$Faxvypw0MSgdvP!C(09;7E@2hT45r=fg~ohnG<} z$NvGkz~-JEEO=j?y}&>nYKZ*RWX7zq5BJoN+_uZ9fRFl+QfzGOG2cJ-JB7MWYKhE% z4uUz?itLBJFGWm(FL%c#V6;|f6p7I4`|U;RVOG;AOZ$swQES9t5rB^MEy|_(L-+pjax2v($-=J2@@gqyzWAFK7zntzyri zU~d2gveat*_vaO!k^ox5*OhXfdRqpu8RaJ|08n;WWtf$Y7M4i%Fa*Q6h1R9~l>p$3 z1>N}vUO3aXK}*C}fo+K3@bJQUeylODHOf^MI=t{SCe^2d_21)eBc4|7$ep&vammZ} zcMbYn;ZDbDR{0}hnj*W6npy&$CW_}_ja^-M@3wWa0K%~o%IdyVt6u^^Fn4db)=hS4 zZa^w%M6RNXYc_}oPfm)kWwgyhYB|SVuL7EOF+%+6wmI-16BpXoDPF+y|5#}Gbm*Rn z%Rk0li4cT$el_j(&8m8zlos1IZ@AZQ*}Z;F=X1(~`(}fBaiV{1Z}lGSZ_*Ibj@aYA z>2=tOQ%y6SwhV_tZ{>xsJn=BxSYZl9H3t3ggV4UFaZ!70gYPI61k+Sj0qG|4azy$0 zw4#1i*aHS=8C;3r@&7;T8LVl~m=R8H5~=$8Gm#8CxBj_lFm4>G)#ZtrCJNyNp+eoS z0jz!Ai>Z);MI*5l^1gE{kMp_gTtXFjbMzoB=*ivLEQIlLnLZk2791ml~T@- zItq>)A>DenQ%+jT$WSjjayBay(H8Uo)&;~$_+F*rCCrn=DOSV4n~f={yjg(;S?zQl#u*C=Ko4o52_1~0 z=<5kOqzg6^S&QYg7j>qJb?I#hCoGEo#$3XNsMZylwCPHLz&C99TD4}?p+%wYek#Yd zPmOOh?_@=)&y6zG%n|RZoY-pj$(m0sz<)y9HyOH^kw>-tgCUplYZQnfRG~Y?_@;8- zPu9Ei0YRw*Ejt%v)9d$JU^-nI%_|}vWu>Ovoj4mlL`fPOuSaVR0@xfUa-P-;8Nb$O za|i~NKGE!VRy2619ppy~jB?j<@h9uRCMxxaQ*{g|t3Fj-^Zd71ixdw6+(Z~ec8|SB z&3?z#KN2)p#J`ziT!QM}FSrSV83!y|BPXyL!C-G#Wc&Ft&G#0IZYt$}YaO(!*6$=j zN1bAlCN^WvADGPf!+44jl}V7jk8_-4*VI6X#@~Blb5coGNZ(2IJoGsDU$fx%Vo_*n zEhj5R(|2vGPg*THjr9wGTHFx5RH58K*x&gjU0}p#6e7ZmvT|PEXveOxL>aG|mAbr% z;gifHaRh$sBY@A)*Xays`yfaBw6C$yzdH;+_io|{qRoAtlOEU;^?>V_U;d<3HBAre z*>~t6r^Aib2ydrjujvA@c4b&^mjV?p2k>$WvzeQPu7_4$T*xW>ZUVbB8HeJEqib9I+6hnnau`z!u=l`lh1b zkzU$F{c+26P7Vxe2MDE6$*%DhkRn~cifWm~7;OLB-MKFt!99AEw3Ym7qfOvUVDAvw z{O^CBtSIoSwW<1(OU$7JW)!JR8x2WWWshP|UcJTCLyrxlG&s5sY!EoBKd&2OamgxY z7oD<3*32s97cp4&%=0bWv?0 zq*+xzPKwBETlt~kVS4PcIF5^L1QvI#(s}GM%x|=qy050X`DXnCj$$uJN&~7P)X_X+ zqPDz|4ew$Yd%n{28u|aRIwJjF`&u!KH5_M5ikF;#eyH@dzw9feI_8CReQU93^L4z4 zD?|r#E2P|<(qmHs2YaFUnLY~v2lD}@w?mIxJftTy`V?pj%XpL5)L7$IKm2!2Pq6Sv zglxuiif{T#P0|ml3VBb-h~bP0DuK_RN6H|hu zrpf=diUgNiby9)0a8-jQ#WVezP~%qa8nV5Zw2OOi&f}mDPG}+OKRseVdf|@R<_+z3 z|4%y41p~RYlxele`muFnSjnZ8h`Rh2kZmgCqR|Cc=v9u;rm(5PxB!8?*zXD+`MWuK zY`87CWj7%f6^+!=REV8w0Lm^F_O$_ZH215?Q+m%f^FFBj`{=w1_g}7Pfx!c4_h71_ zg<2B$)8EkdKBoRB_B{K=Iq6ub*GU=J4zZ1Lskiz#F!isJ2Vdn91%aN$II~>lKOZB9 zWYZr7*YD5TPOBcy>Zex0UV+Pxg#iaB%NNFV{Ub+b=JC+QGKq9%4_Q%goRB>#3X;A1((# z-K784Os?@kQQLr2!!<2zge-v1y4SBIYD<~XgefFp#0Vq;6Z~32dQ5 z@Nm?6O^wuRKeph28Pk+2n`oJuH~WYqR-!%JqN&aQ@B3rj-;Y-vZA)MFKd2!UGl!}p zF71>bD;C+zm^8qe_>tdWL1TIcXAs;|q2L>x%h`W%ALuGM=U9PTy$Zzp-hvhCBl?53=Ch6-4IVH(9GZ&1S#IR>s29KNe42vJi?Mkm-*wSHUF z^U(7G=Ok1ZPWMb=%&@fzH2C4;jyTNjOV=G<_zcrqWU!!>J>0tSu2aECgrk%)fur^a z+yDEZ`mu)PA+XTX4e3t88oD*9{SS=!w;fT>%*%{AdYV(Zjf#HPF+D--Q@l|H2zWL1 z0ahLLjE~LICK5Ka!VT^fzY9Whe|J_>yp>19lMjH4M)+mVh$&MfCQuvspPo@vhaO;| zWU*RPI%4GJmIzS~q^kwvkIPMnr_||nYJmqx2)CKGckl%Y0{6%XjMW{p zPrIZ@IOF#M07pQoo?C zqouXc#2Rne14py_OXdH512E`81>k=|#`>02Yj`qW>I}BQjO>JwvTXT1W~L59?4O@a zHhK~77wZ38_7Mj@fD^fgPpiA_Bg2w)ox>L*J(|~@Bdu=DIi{wS{c>?>SaMb#Rs7jh zPzX;iQrJzRIzqK}zfwhKZw?}de(q`++Y3%S+?$N`Q+ILxe?y)uQBr}D!@scb1aJV3LT z25P}JRpeoh<(dHsfxz$~?8;)>=x~l>eV>U&Wj#H+Ro!JGSD((6UC@a_Yih#no?`aD zO41fF+yCV_3GM?p>tt^b`^tu%shfh#s@!y))nM|!4xlgya#H7=0F?Z+!L_3^Dh)Pc{zkp`=|ajxHMmG@6brzfDvdl`QC)3lCXm}fwvWF)tKR4@>s zixC{to44XI@#={;S@_#G< z;5vC2k#P}5P2x{;7BYGr+t>S3i+ERn63uD8L+IWAfF;SUVh& z+C&19;=g#)m?4q?AgaI-S^Wq+-MDEI*XrrCzcGhDIFc)k2D>S!bjf*)*#gsz<`B3H zK|*w7=aA*}_J7~Br3{28lP@@<`G9~ep3{6BFO^Gr0vvWfA|HdTuKDN5-KJ-{sMW0l z%>-srOdp3wD+hX8=(-=%j&}5RN`!STAcLj6s;x}$<7uy`@NJOLVfogsc_T?z5W3%f z1VWe%heIBl;ru{jJUO@R$s3HNpwAReaytAyMx4R@OMq$KfIh-`Rz+wKY0W|$|c9S2r-R}<^{%@Y()7cMseZw#WWgqN` zm9;qde7*G^YlRp9`3KNfs0e|#7HmH;Jy-OXp z*Q9APslbs4Y>q)Ats(8eoX>7N#8J@3={66uI!53(S`a54fg~oWi-}FGo zCSFWMTI=*LgWvX`n{y3=)7_I;Z^aE3=gk(R{les|X%xP*uhJ7JDMvSB{`O&EmJGD6 zJl;bftq3}FRwgD1QqC%w!yHkzdK;^%1^&@CN7+~c2wZz%S^1Om8TbhS4~_Kp)q^`- zjY4r~m8oAGOoHB^Gy}ej#qx0S3rB=LqrMO34V^ZGwBFn?^A~g$ZC?im}v@2`gXGep|qKmF6q3e7oG<5 z?o(sh6cWJ{E>Lb@{5|n15=$p3x5jT(0x=-+So5Hp=UWoT?}VxDL`j>fDiRuBz17{Y z)5Y1&wVnEN0RjHSeC7~3cjfX;n%D4VgjJ1@grhHWhksvHA68vR*C>46@0Zq^6uE3u zLrp|#n!uXhY{(P&EJ3YFzCGk?@lo^1jMN6~qhMG9nZWvq)^z(eY5%7kxH&0pzxxH0 zd1}FU1AqzO;DPT9?l&RtWJ58ZF3JR3pb1b;kyqrcQpl^{>Hi(_yohYV0QEMg{sCWz zB7e>GJwOv!AU)u&=ctI^u{wnHyS}9wP~8O|{tP|Xjl#P=jf&TK)r*Af>Nf(=!(n2V zV0S%IynP)r_@Ds0i1LXkMj?;E2Kr57HDII$+yjMPXhlU8*S6+ zK5G1L{pT=O<`e3uHcB>wKl*xqkZ0vaFXthCfaH;n`AM#z9&F z1}=`;O;BNXYa7$WWT*6jOPPic0MfKf&31L&#;!}av(vJdu|IYuYeTbbYtw7=WOFCT zp1tcP?%RR2H%xHNRLNsoC@svWBp*D9O18UdM;d33$VuF7>``R2ul!BN*Sy$gGrn?5 z*@l}&3V*)JeLuoIo%Y(6Z2&soKv?|cxB)%*)aJM~do>S-^W;Pjcy=x7m*K}WdF4Lh z9<{PHRp1IV-ODYUj(JPo6wL$DrJ0tYecqdY8iB3yKOmM_d-df2diyJpF}^H9$8k-4 z?YdY}?wf)kvSUGk&O6xj;pfmRZMP5Sv9OvyYA)l>ZPK`8DT81G3O_x-qzcd~;BdjN zRbt&Xo<$RKDkq)lR25_v-^sbXk=05(Ed?IC?>Eq)x#wCf~Z0A`L-b-?DP|?86kyVd28P@(VJe$ zwfOz7O>N=^`JKx>vcfPsdo~vWritrJRNedrgzga7D46fc8ug+w1gC|%+|E#IZ4Klv zq4F=fHQ)cI0 zR9{RuJIx+VE+eO3V&A)2le9fA+HMR~$4%lxQ)@cOBVEwBnu(@(ikGPaM4 z3MCq-jvap)aALIwEP-%i1rwSfM5`+1VYr#=)9q_&Ih4N5Uf=wf`}O}ynH)33ug@+# zD7sG~WUQC?pB;a(|Eoo>18_j@8*Un2;NhiN{R9suAn_Vzb3=>B&8(wm9y&57u;a|P z6*Vg0XYVk|bUQXW(1EsMQxdF_*2@N1U|QM!82!I5K9~ z!!n`g3RZ9G`#SWfT^Vp5Y)wzcg5^G;>o<|;-0MU;LQiWz!K>$;7FQ$#S1ZH)TC{Ug zR$zlRsEB3X28{^Sxt}`ip4a`KN-aQnq!5*&L(=f4~5^<^2!z-vMn!127v)1%=cdvWRTW%b(2HF(X;9c zsHek8dl-?oW-^EG??;mnM+-HdrKWl@=rvxLOG;*M`LT)v2g$o)NETINb=ch@s2NXYSS+1grypEeh3$o2jue9fMtwnTqmf9erD*BTN& zYSANObJ?@CQuyX1qa61yB*+r+1x(*0k1F(C84ZJN(Pee4NEEF{p4$7NIx@4ypI3V; zgu)2 zb|)Ly)wET#t>SW=ZXw}(W3QaKg;>1A~dEE5kO%1ac`b=mkTf>ZJEP8FM#m|pPSx{|^ z!o@imi@8dtriM&z`6z$p`?h*dk2PM%qpyr`tpY|q_O9Pnm9G2D3W(&gFhR0Sk;5+W zS39GQ*(DYn{=1zbl6#W*uRds(m+vcE?E!ZumqH!gj}`%!A2a3Mk5?bUFTc722+`t? z%Y5^@ytA<8%Q`QfT`0+O?DjQs$Z8Nj;eLROy}&p2zCwV2N{ve70}~{K4iEy!k4-c| zMN_IEbe~WG4$CV_4TWWjSESiWZE(wn{Di4Sb6hahkK=KWYN1WiVUtKK7Mb}G#t}1! zWQ~dU`Fp3&W7FyOpAb7jX3(8<{ZK*HjPpzT6T$xt*NGJ%Ji#@iZ?@C-0TEO!s3%j1 z3ou)(He`v!|H){@_jY1}MR2t)a{~XV%mm(KDkdsy7@QVjX0is(Q=VS3F+!9v9;Wi=i=znv zH+`RBo)1MxVsi<*l*zk2dqW zzaUA>mIsKS?90$GWTa%IqATasnYi3?CEEy=AJxBxRkwyBgc(phB0Sekcv0`K@C+FQ zQ>2ZT-J4widLlc@JXJwwsong$({wbYSZ`-vv7kDL-|RRmzrd7}tvXuOu=zNg?N@5+Nh{LRlNH!p%^+|T z%DTn@1wd7Dc(ylYN$K!>Y0TM7wdva`E4^m7Q<3}C4>%Q+HqD9HuqZK=KkUdz{kbK~ zX>0I7sda~&YxIgUa%e73(xb9&WV6^a>$)kLtpk?yssR3GEQV|0w~YJ<>!dRoh4I(YsltGJKJ{3upiIsR zNWeNRYb()$d8%HyplL%AT*OVl)xZVE&HE{FTUEz@>XRHspv^EOkltS@Rv=ApxB$;; ziV_)_X;_ACk-VlnvMO-sZzzQ|rw~ zyIwCCe+%CH)OA}>>rELkrgPoCn%1f}$7hQqegcn^RtLMlx?^?!*R?G~4fhPTA3N*) zv2zhzCu*lP4K}Hx%qZ^Lt7lsEpnogAj*orHo5NfC8FpVgEq_CWQY|*UF;zxti8P!q zLCm4$fbGwo!+=yx5X2;)^>ESx^rKst!i@pG)QEahI};$fEVQ_Ni<3;s^i>AKsdb`5 zJRt8z|47aG;n1HY^4aCE1(ri&Z7eQNX>CrE!+14AocnX@;!LRO;F+F2jn8FQ<*$!D zcjZlk*KihLstInUQEPI-@|C_|`?0b1OFzoX&ex!Q>_i44rEEw~ab}LCY8<;Qxiv@c zSEr9dwa4clBD|dk+HpnfP}z5H_1{M?NGA(w-B&t6h)^k30BLZeGH+hN%Xw zNj>j~k4xKQZQL}(OK)5}P=JQwlO6>>I-J|=g0rbiBu)?<`Ub17m)QKEc+bH#$PKa(fl-!g4%l^#6vZd-U66tQzjRY#%j!P?*qx9bn=@ z*{_#Ax>hjoz-+n3h}a;|yQ%AOnoiiQ72m&JQQN#80}AREnuB1-|^LPxpeU zH=R=5*cvfzi+UpPc>$+x9!6}%GEqAZ9Q5g72~KhTBS-uWZVCfOUp??mD|ixFp!8M~ zprLYQ~M$RSM05Ly=s3`0uuf3o8^4mD`}8&n>?PSIr#^<`Pi28ecEnyx`DLET2(cP30vi-MMN zS7EAkryd!9Mu{a&%8@ucKi3(vx|YT^ub4){bWU(QWA$7FGyWD+ty8iJqL{hGoEwWozkGVzh=9mECo6wb?Uu;%Ud!{dccEJQp zcWCMYd>BTvPD~5f?+(y!VhewMqix!>T{Cmu^H0mQA?zd%`_tk7K2yV7MSyVBtwj3~MS4g_*Kn8>S>W_PMFf}L zjJ*(e#s~s0z`(wCcMfLF-kv$!;(8cyVyMUIVDmVu9TIu2D6Q|AkE*J$Wf}+v6CHDfong>3Cu4pDOb&B{oD>F zGnNWm@;w^>>lOPo<~y?M>U)5w8qiA9FzwJPT=FnP4!xmax-ww;**|kM;X8v4jywHa zyUS(-H>SK0TR&Y%_Sx33dO)2z;pa)8ZWPC2sX$;1jXSGn%- z{O-|Rc6LKn&v18^4Ujf|H}krt;!zW5wx6WDfZ;chkf86w*GgSy51zyczpU>LG~;m& zpjM8F3P|6JR^{D3GF@ega{|m|i>=;Fb=$8!{pA8qU0M?yyuBN!Gjr+W$_s6HGqcsE z5;r^u8MfpzNa(ufCsjoC8r@O8FV^|&AtJf{w=sZyedwb*CX{@Gk2SR3j3dWYqU9MYI(x)lR4R zwAu#zMhi?Kqx|-(<~1S$G1JhU_cic)8(U)2GR{bv%LT@8<{=MA>i?9R0VZggM+B)XgT+TWd)WZkz&1b+u$S2I=K-O}(0{hcU`^(3 zK41@~gCHvks>e*s39JuH1nY(gdyu9hlg_|mF3YC%ox`j9fn#K8v!5Rk2 zqLa|UNQaV0cSO|ho9AB3m;_C#4$HeQEqWz@K~2}6*Psiar0wj;jBc0}8D_dlNt*++ zc~CY9dvHItnM;B0hR>F-DGX35RlIT!#>WdfVCeoB8c%yUknoP>bxk?_F%LvA*~iMP zp-3zQIX+Fi7!-k+q$cBfhRb}I!A(rj#(kvOTcro0{66}2>w4V}<1xJRZ$;*y7~&!lKMk-k+Z)mgYv>C2T5%CLxd&`aaVa5tQBU zujNq)Y05%ugwI)>aT84E#TAZS(t z*!_6RzniM$W1bUyiuOh7aX2;W+>tRQw^{*md;JStt>Hu!Y`0CNs4O`SntZjx;UX88{&FU84s>uVGUpFN zXtVd-Gj~EmsvpPXd0UL{mBndkIbXL}t*-OABNl1ZREkpu*|H{0H5NV=gmH90!H!~Q zJ-^`I^X0AGX5ZH9v&UePKV*6E+Y)u%S0tJU1by7U>v|uq@*5{5ZrMDL>;EM(woM3hTcdOh}Cj~Kx2q#r4g5<)qTiGxx6Ffr9Wvn1@Ohq{% z21jgaB4d$Qs^11 z%9QiLRrCQRYTyj+fT>tp2ba%PlqP$f!3+{GWr}lZ2NMQlQ2sNL^1`p0{LXjx&a0_Q zBb^@l)PAF>jN=+mJ~bJ*ym7HwC%T;*pw}A~WrcxU0WeV!g0;3Tep$}+%j1A0`VMPJ0 zXE0F=iTKl*ID#`;b8j4vv3F#|3+fz)*;yJz#g-qV%y%^%+pqN8r>Z+ZyI>b7K|a_B z)orL0QW9?7Y}{eTG|xqOtw(t+5Jz-q9z*zZ7n2BJ|xAv&=S` z=`hn-@!ayN5du5O@*}nhffKYUmKY@nErb+Am%XA+@xv!GA5W>uC~v_6kj54s$K0~T z5h5TG5uxmd0fof>kEeHVsQZ1~|Ievqu9m%)ZCh*E*0Ni6CoQ*Z8w<pvj8>_f)UH}5 z;8MiPJnyIN&h+`n2o;HZ(p0`Z#u=sKUsFG&m5HdUo?c7G)fD61;N+|thU$M_BT-Gi zI2PPSS;_C18fCpY>tJj|OznmWjCTlcG!{aX9-};j?6}lhG6K7K6>J3T(YpkhSF}6d z-@JGpcjHZ;&l($t*ew6Zje1zC=M6CZ?`rl(89V@B3JPpXnKz0>mD2!Ej4*YsqGCbs z?yW)2uH^+UQucktP^3o+`8KEtLL8JiZ@<^)*@i~vAtxJmopMJq<2cT>W;5-Hf*r{XvHQ}f`(CN* zcIIQ%`6D45nHG(m7F_=5SXEo}!GYkbmldw`s}-hoFQQ^TS!pb2(9sOLv)mL%E5%ySTN1stnzflffT3t?zqp&E4E1 zb$jhf9g_4*zQUeA)f&w3%6o=ayv`r}aq{_w{bd0@wa1dR3rZr#q#%e*F&S)e84$a$ z5N!QvmCrt;NOHmTQ!$Ye+X9X5Dyjm%Q_|S;K8=FHr=U`TSE&QLC|AXzD+~7Bt+C}v zMZ~}fypV6={u7jT=@Vu&J}b>rnG({@uu8VvB2JsfLcI1mnO0VbcQM+D6w!QkLd~aX ze04^!qoA|D`x#E0mcft>UDM2l-8eD_)yq&=ODOr>l&azxtks+% z{Hr-L^+k?ZhDpQa+S}A<1c2P}$LMh z%QSrZ0TUk@lr$W#)xr^j=zT2f6nej|G7VnjQB`A~fUDwphqWgq(6p!2wtmvm9IZ>? z@PFG)sI5wtRKpRvf5|t!tyfPBA@-3S7d-KPJZ}wiTrMphQp!iG6&qSa9NCRe^q{Pi z2u4il4%FU4KcDRG4(f(HVQA=nR|m)qPicPEn?6_<*-2n7sYkvZM)rWvZW{w=5QOzl zv7!lirr96DZ~|NL;IX~_FUiNEOp4fB1Hg5CJKfscAsh{B*9PdJ-uIq%{R@wRXh@GC zg|E!nP#8_U48}=au{x>Rig8q`c8D8>k^mdozZw)}%^SI2>hRU4l+I;$m~R{SVAs5t zWTuhTr(Wj)N#IyEU)wj1$dkLV%sj!A@u}!(kA{?T+_JOP zh7u1BTdubS0pMx~;GN+2cXEo9HbDHc?x0HxM4Q5uRU)Y)@4MhZMmbv^f zOLIc0@-iq?PFJN5ZTFIRhy;n2?U}&=&f7ZL_Bf?;Y)Mv$I0px8RPf;^HhIEbLzGT0 z=nN==@B)oaV$RQ&bE9xzarEnWi4$jE4b{9RiF(dQ{5i&nWNqKFrocR&Gq&n3f&^Mz z8PbwB3;QA1HVc_GEys~wEnb-=Y+oK$<_va}YE)_-L6nF>ah!lTI0;FTB+G6p^uP?AKUUJ`TFTD zlY!Y0c4q4GjJb^OngXIeExHx|>a{{=x0CgCL_Pf_>-rHR$sUI#>a8@AUQqo(ff1fJ6mhFzI?v`HW$%upD&Y8zh0%j8!AQnQBgYm z7$k$C7R_f?C!6^smXn>&#!!I1eRHxVgVvBmVf~w5I9lkAS!bjDu587mpPu7l-kJy6 z;RMUbC-G zjkxx1-o~($n1R=a^^^PZnYz!aFk*f@pQ`3|Ac!4H6{lj%!o$($?uosb(#dg%VRvfO zmUJ%|%Du~}{b!!*Xhj1F^kKb8IJASktC$)z%S0Wz(+~yluE^5yIeD1EH;H-cqR1Af z`ePs71ImS8OUg~8bA_k&9RV#A!`Z`8R0pcfW{02l#Ui!^i?6Im_k|*PAV+qY zXANF3QX7A{Wt6pf+Tn4)N5_3VuWBZ5BsSIl^w0sM*Ds7$@JigDW@%f09Wxr7uNw9_ z-*cA*tJ3&9ZGYp=SsJNmAN8z>5pnzHD=(|ig!vDB95Wx*3fMShc8dGNfh%!)8!~X~;g7Xy&ypdDPduzn{o?hB0xID)d?qJ|78}{r z2!@?3Qz@wc>@rC^Q?Vy{25R%hi>0x9x+w8g)eg zm$;%*%x(AoQe58=>G2V{C#CH!od8Pg9xNUp*AtEfUE>H}u>}VFNX}IOh_rUvw&bRsu zxAknb^m}ZN*F>(}tMbZ(T=Jh*qfn`AGQhBY3_WgFbkngp590&8LLtxtWoOZ7ApoLEM-aS2S@Ns?I(TcR44UJibdsJ z6eoYnhFh{3k=o$iOjS*UT@*r@c z+G?C9MCS=sQifC_`N8> zqTqj7p!wRi*3g+4ufQ+;)1FPVU!PljocHW2Ueh!_tMNAfk9yH!+eUp7qW1HFEJPEUT5R$zfg7d{v90m+>LcE#~T zrF$F46iH#`IWehkXk+|T7{L0#)3j;m$*AMBldZ1ifVSL%8N0{zrG3@ZWl`;2Gh*F0 zTgpM@zlgp!5PiA_6-3DEY!Tru;grZp{I&D=OSJu<=&DtDh1r1zeb)CWxD05DZ!wE~ z>ASxXBy##xI>ytjkuR@oBjc^eS?!AaEa-<|>tck6J*@mBMJkFU6|B`n;9D7%h%T0U zTx*M0@z%w=;MLDQo;omI0+Zaialca*g_58;6zVf%!8ywrOojEAYovEJSv>8hKI}X~ zK7_XdCY$E{LpV;-E?DG$wSJR&l;wVO9jo`mglro%?Q!7n z%Iw&r3hW64>lcT2|EXXE$vIpp2l5V7=LDx`YZTzrm-96$mzkkVI^VL-nGsf3R|s13 zXKEey7OU=SU=PYQ3Q}*#l_tO5y=1QbSaE+mF3&c7Z~r-{mAv`|C0OA0!^IKe7o2a& zHHLIHmJ7-=)WbexzYeSVe`(uLniv8Y?d=gG z2khn)^U;ssjVbT5nc`no^;uN^s^(`HUA=fAws{BgOQis_shXHvbofyn;L zL*dO~(aNvZ5)`>iXjQA0TZm;np0I%(a_v$^xb_ztTatr5J=kn+VbGZaNkjnugnN)f z>xG`{h_xDS8(ovoer~nE-#yr~qN*|$D_KYX+s#TkWb;FZMvr6g&S}p3u~~9#sT2Ve zm$&_eTGq3vj+KH#JkV{_%g2Z)O2wvbyE-!DL|O66oYTPwWI5tn^E^we&t+Wax2kAI zAdGghHg~jA14uYuP2UbX#n4aCto%bkmlXg$aN8ZM*A_8CNCm8B>DI z;PiSUz+pAojt5)lvCoemPY0Ls3ckU?G>Wx7R~NeoUGPBn%&jg`pivJrTxNnf+K$@Q z_(+n~+g1)0=^uN4N*_W?DDVSYghbF^c6!B^;7@aLG~KX<2-7ATGiysVsZrh*js{-#_{Lr50;-Zb;r(qP*6;=94Sugkeo#LqP<- z4A#6}j)wz=tLqs7cny|g)If=4;vv-ejAl-)-f`~MZG>E*ov!`KCHcMMInDd6bBXmS zl&t2!*DofHu?#JjKOeUvHy)4jYu}UgFH2a;`pZ96oo~yhyLo?t?if*t#h#jZh4rSP zciXqa>1IcJYilM1pBImIQ({R4D6cPTs1-&zesSymcaL!PMFd~yvCRV%V05{2M#tC; zf8i(sTN@XTC16pRUKq;oTj=GqAoKupXsacUZjl@f1E2a;&yxyHJF*CJuU4Wbi$~mN z+zkidq$)KCeUYT&sOV%b%kNEPoM*=d@AkZu^{INxu~(@xe!cx!Ch%t}&o!0SZL+}b zZ?=Z`KL1p_>$l zFL^&m8|X_`oaI_~b3U_-Z#D>DwaZV(&AFt*$NJ?pN^k_&Vc`2%cyqB~V0w(WLqU%Z zRKGi(lkvSy-$+SKNz~9^V)dWkwjYi=!MHg4X0HFy3)KfhwzwT)-@pcl(P@kux)DYE zkY__p57LJWPfnPmwu>&J{{VW588WrixR@y>`HB{ihs*f+@M)lIJwAuMHC=cz#S2pq6{#!i0H{bWLOOiOv1O@qo=SUotP7 zNL*KhM3vjb!@=`fchi3({dLl2G%1QmG1h`^9wdq9Qc_5G* zlfp#xZ^R!K2%AoJ!VWMQt%IEZ8vpO`d`ILaaU+F62l6!vc-_@Oiz_D9tSdXfHa`C_g5` zUUDkNo@gIyG+p;OKP(6kUZ;WZhy#Xtly<3iG14i>E7Mk2QHq#uDv{;KEa#}o6V_WZfh8TOJJQsa!*8TdtFpZ0!jDT=%tXJAYpaR&E#6{e{G6-a=~>I_W?@Q!qhP| z!H%2ZzebG3GjdNs^EiY({lPZkVXsk8EMNj#_$Am?izZahN|jpVUfB&~bbNu8 z|7X;FsD6Lp-i88Vfxwv+J!&V(P0Tw~TKQs`pCTVB>LZvs7EyV-iILew41-xv@!1E3 zF_@az_~3ZpdrEKjT4>SWG0cLP;Q_cQjG`=onB}n+!T|bo^xsAq`&m*ATsbHPy`iN;w1t~@bWlVfTDB1_?R*FO@#?24-R1JCI zy`=$Y4P8wrRb^Kha}@edFSxC~9NtEu z<{)kq)4yp&TbBU8F5|rA*307k2Nbxn+_Rf1zgHQ^J_}#%pTP$})%R_J>l`y7sTbD& zE5l*Ic(;Mx*_bZ>0#cI$7bda2AnD&x=l{QF2ZG*2GVb4YNWcLn{5e0&|CI!3qKQ;C zy&yJUygl0>U+cM2YZ!h<3HcrE^tdLn4uztUAI@Tf`R*}KFTqWqmhJ}Kd^(M^3t3uD zZhd=9%W$8Tb6gY=Gdb8mi=U2LPm@ZGbyMEuWzFA05-~Ec> zsP;u6&Y+|Sg?_;7P22=+HwmiH+BX(z_DRfwMTr}OkwGHU-fVcls-gzfk~a0G_&)p? zqeaQh3F^`;oU5v3dL74=Mu{}(#pPZ}`zJmV&j5-Afd^6Sf~K?jllA8UmD(Ihs|ZyL zI#gwfVA;LUqGlUGu0lxqzLa>-YD`IT#nmN{SXY;9LEowTW#i0#Xz*-4u7yi5Uf*Z- zL+IT-;Mkrq>JTSScG**LwAw(Cm`R1bux!{Wvlzz7Xn*Jw3YXI+&UD);5oM6^T<|i@d*;Pemse56Z$IDZXbM*LX`a;m;RkMF$wGmM(!3W z1DAzAPXSMp0mpF@4$p#!a~tVh+cN8`gF&iw`BKgDfHaI0<+ z63Bqf0CGrP@@cN`TpUNwEH0Y$`Jm2wgeUd!>D4Ktg57SG)0GY;pG;O=V5SU z*X>!CmhbSw?KzW*%HV;^WB)8nc}6j3Q#)~KYUre!OkLX$)23d)96hopAhglF@{bPT zC&gi|8ojt7#qW9FFuzou^u*P$i|14CFx7oS=n$vcBjyz}eamlF{z;jUGW4c!VByIajf21y{XBD-P za#zVHZ<6I{nKag9VQ}zxkSa9tc#v%mI&b>1WAg{WAvvtc8V+-Oph$KoKb1pPBfCVum<+&p<;gC%8q{!a4t zuMPu5K7BC6K`07<5A;5{su^j?TJLVDp^I+wAzZep9WyDfXo&L@da~!E{^YkWDyaU5 z%8N$Am^`O+;Ms@BndHh44;(>ZcCu#kebR@0$y4mw(c4Rtxlbat(!UQt+DKv>e+%(` za`~ewW3Od9_U1U6vSvdyv4kT?iuuN-Brk9;$g|yRzW3K$NUboP7}hEcwi02&KjUMP z%y0Vh+^aDEO5Wl@4L-ETlz?K`Fb$=QVgPlL?=dWMO(1pyG>iacq zR;{@nk{o|}+e>rjj-H`dXbOtxx1XpzARxRXn4CS5@KXOwXM0<2NpMFZtcLMhSvIeu zNc_D0*}yR*pcU?-)t?yIyNJFs?`!?JY`Lvc`6>aGM#g-<1FBEI9VB0V>*hTotiuOi zKB3*q*rEs$>kx;PFvgK0k;KG$6Vclw$$%Eg_I^4l9{Y94y`C$Kah9ut^hTHDPJAt&-}i{QP_*Uu7qLH2w4^45Vap?g1&0aN3(_&54; z6XSH7ZB@k5%139WMuPFMOf%rR<1+C^23{|c`l-^xbuvfcfQ`M$W8CX$p$erUv%6z z{L7|aUC}X(82=J1D!(dA><3pz@~D42s)qC-OD*qeI6k1`2;qKMh}*_!6~%`o5JqP| zbdaVrQ~6lsEDZ?WF^a|pe#Dq=sAjBsHa%KBNHRI)D-G&=g^f`uzxw-l>qMMjipZfu;GLhZM z*T$H&hWA8d?NZ&Ar1DGUNFp(Y;-6zGXKPSwT(JUU^KBwv(ekaGUqTImbIfC)ljTBd z{HAmUxd(K9m=#DB*d@8Ty3ZdjgxIs>?5p96@WY+7Q<3wgNRZCFyAq{#ukZhHNbqR@ zerQZLEK9Df(RSCFe*?p~;HrLrf6;t7WOHPDi^N2l0fOA|fZUq9{O3(OySEDl8Y_8z z@moDVb?^WHZ`pjC)Xd@VmNBYdSO$7u8I{rV6tdS5!_{P>h>;VMxeQ!i&djW;oGF@* zl0tSje)&;c_<45_dJdk~%%F;y1d??o%`JGyWVzoFcwA>jnC{!iNn)uSXe57C<7fb> z?J~pQS0cs)?M_hcA@n8`jIFZgR7!6qNPX6xCU3JwP`F$u*+JT>aQOL6gSR0Bs_E0d zzI62@e~BV>-Fop}GG+QlMwkZ=?&OoNj4K*{es2p|ZuAWq+2ZjMRLpc{h9DTU-kp-s zp=m40^P0*mz2^kZVcX}^D~@8yM4xi1oD6PlYMoMy(vkU45o|ObH3S6`+wOAP6bkD5IKTgdI8XLGp2@dR z8AQTv5!3gBxXU|(NAO1&m+ccfTNEJs5fXtl+PC`3HoO6EprvN%^xEO`jM3`~>jwc+ zu)1Fwt~?6kIsmRd+phy{;yH`Vp6-sU@7n^2m885aKIm=M7f9eKi584I7#5bH&)e6ipurw+K zv313lA{}@r3i>7p=n6VuU%3fkZ!KyKa((7^24zjL#*y6Sba?Arb;%N$(N~L{-dtWO zsc&yG%Pr>&8_F$Lw4=O%^D`8h8~=8aBHU{iRkDCvQ-rus8igCL4GBB*YD8hHSQk;Z zX_%N~SQU&BJ$X+)N)eV)?W~izt$Kt2XuFxewVJ&Jt+QY|-SYB#;XF+a_w9B=ituSB zIn0C`>k|FH!y&s~us298D99+V4PQfupiv?}h#fME;PQ>97>F*K8afOJ1Y)5=A48H= z@4JniuN4A4kw)uPZCBxYjrZ$b>FBmhI13~X9sym2R)5fu@<(-~K9o+Wjo5cSexvkZ z(ULsl)0F2q`Fg~c(`t1(FKyfN#`=~NvX}Sc*KnpMQOnT0@^Z22@xFXwoG3%}w9?pE z`Dmqj?@a=nHg`84D*}}5DK8p~Hx!0Nt0&U6^^9Y~XtFZYS9HF=?Y8~B36jB|baKZE ziAn@I^I-KJN@JJ=pVXgjF&@|C(}yn_SCnCI#B-)42w_~XX6nV}ZgRf6)NuxF=t5|; zDmd7AkU||ttxxFL<-d^g2Z@JfYs%mrPC4n3LZc}_6Xb>^=jHZq=G6h`Erj_e`W`=v ziuZvr!;FB+=_1pA9ipT!h|K|XlLtyuk^>VvUwfAIAHK_5PPTeDA!*qRc#3)E~Y);cut{Gyv z^_{Uw*>+8mBKG&KeQQY!$54_!?(7!KEL|h0iaU8IfXXB_N%wPPv6KV!%cZ?!k~PiJ zL%Wi&PMco4z;iv%GVI5bf(~t4$zQ64F2JftAcoSJy=G7X&}9)`F_?>e-dLv;NJ5ho#&%sX&GUD?gtPTt$h3UEzdNFETj z{sa@aBp}6`(=)#Q>vjEeq}M%980c`Wz24O~kq0*+<2q%)ZbKMA#lONbaIPSN|9tNt zwJ9O+3aH;5_&49~rALIV&WmMfGunA`3sl!uEV#%6yGpSE`MA%yqQ(n7#!?D(i%})a zHFXZ1N|8 z)7&q2brSBg_zhu{qB{`#g;#y^*M;r9!yGn+QW2NoCpDQ8(M0Hyq(NE1uMPP5cr4_4 zstN6G^nHNHD`Wg^#UMI37D^VZm$KDt;8i(p0dO)Bw&4jx_)ZIy1d_WHf%*whQFvkF z;YhqE0s3(MCpXMcDQA%GsPQs*5lqH7p8BJSaSMwEd^_gbp8Bd8MNWJn{Yxa0VJRaf zEU7ooA)z+kaxKYGV)~?#6qE@?wj=$G$J*ofyLB~vSY+r zoUA{sUM`a=W$S?*-_ZDqY-0WVb0K5+(H|@d$fLO;WY!+?Aa!Rt$Q$ga6AP9Lj!j zh{5r|Q|zy={5O3>fW~aw@4el2JsxlHdaN-d&G#2On4AQJ*3ZTM_>caj18)1jhLJRg zm)_5uAsKDKlYv%PUsb1<_Z;iEG6M_lP6g%9k+QljHD44V?aA-v*j1l}e8&y3i(!E$ z;~t(G6O9Vj;BX*?XNA6^eS^Z#FNPalU8;T?Z&p4y4Yp1c@9S90bSERy=x63Zs|#I8 z>8~_284O>sM3yY4bOkLhx=ptQ6Q#77^W&yPuQze#5@gT_giP~=-HDCOMrvSs4LaWg zhIy3BNcHry&gIpF%Nx{jm~PmnaWagtYp!c!`{PL|Z@|?(#jpG4gt%dNI5taA&W@og zkUN!~Bg+wshTl#U*Ju{kzPVzPw$m5~1IqZE{{(Q{aq$Zx2KR7?E^-qBspnB)@-V~% zH5aC{M1}0fbQ^x&>Py!DWRsj<$ost)Udy_o-lpSyN{R%}p#`$vqJ!7{06GB51AnFS zd(;F2cgR-7=aOK8Br4&k0AKt3=>WrVI3X<1oX{zdbQf#t0R)b1#z!0)QN+McjNbO15yx?qverQOjyx2Te}gV>uPtksJi(fOqo=~t&b^(hCx(}bb{v@f4HH^_{*Qy zmwc?DO+j6kl{^EqXg`2>gCj&oK^j0B@^R4dhkRj*?3!-Si$G*(Tl>o{38TWWGi<`+N$dqhQO2=*-A@ctADuAF< zYEy#LffMsMg-qgmU11=4EI=%44Fd5r0M zw=M6uU>#&Gik4ely6`9%+f^i!iuy$LGW;pc3UF2B4ffvACbL@>{YI7Uyq>w~;|1Nu zF)5#fI(}VI^zn6CLP{6R!`P{urGfN!OF0DVSX0`k<^(nojYv=}4gSgC*~o1K}X{;A$E8`%we=A6M~0N|~q;|L8g+ zUCY$-Z))XJw=LIylKR9<-zUt~AI(~V-36F)%Sl3Q(4Cy6FNRBZ_;DRX5J5+WC*fAHYP80Rb zoNdr5VHSOP0*KY&VbY(n+Ezh-{2qEpkOyIQ1L4T7Kd-Di^q@36)Is?7$0DIPDUN~3 zfolz;OIw!Q+k_CczevvW`hAW5`Mj}eEHm=}2%9g5IEYXJKfY;dSUFCJ>2RMH!Dy8y z@iY^eJ&GobpW{400(H;36&5`H0bwsoz389(d~7J7&C4sJc{qYi0KYMjnYS*`>?xL?Ou)R@WG;ON|FFC zG%ATPx(?t%gQFp<2f5lc+>QSsV^6PY+D4?a#*N7Jej&8@*&&|{C_gsa+m6hGSoNB3 z%*SZzsMv6tHsU8Z;PkuA)@+8IL6ZDjPCNVM2JT$#wmQA_uKcJ*4AN1!{E1lS8kbiX@ELFW0kH&AV)vyt6lXQ>@cBoI@3TILRW7Z&yg(dsK`MHUErOvnH87x3Qe z{TfqM$6eV{hR6c}vfDEUa0HU(0+YzXNbj+?wb5exnC@H6HsDFRNc8wH*6fJu0u4SIIs-=sa%r*h@_G z$A|<1M{!E_#S=?p(YY~gKFY6945^3)Bif z)JBfxj;QgV|Gy9sgb|+r75Q;+`aH%Oj}IV^n;xSC{Fw3^jev8s`!gn+`lIH~55B8# zjL=^O)L~7L(aQaDTx%w-`+6R^6_2c}*p_p6dm{i5?8B6edmeKLpC z4)BW^VFvywozKlzk<%v#_WT$U$wlWwMS!~;x;!z=kS>=QaX^N*WNn(3f*^ST7(Wz2 zI-T1%t-3Vmb=k~^8_Kc+A5+eZpM-z#)lzg>F!77AWw5xFkrA~ebH{v$k&buMXbdTd|VL3o~A zt!)Jti_o_cj0fMAFS#yJkFOuivqys9sBp~!_F2DK8L|bz5SpHYy79RY9w31s1y9Bk z5}|$fd_H$*K31@`!{mbHDL3rfBpN37B)lP>V!ES2H11#5@U2uOb)lzMCVJs7D4(#nNR% zwIZ|k?T872!B9#j*}@j>F6J`}Zq+TVsYl4{=-8(&2<8}>{RVTvsKeR8oLAS#+oG-p zN_@aP7|wW=StEtDoQqjLh5WHP04APX4i}qFo*5A=3xXGxnr40G3&Wxq?zbWsC888N zFpGfEz>T%8{4SnxhHH~+$Ol#S247ep=M7K?o2r)>5;Hyky{o4ujxpImfBB4`A=-ib zog|&Gc>_oUuAvC%uCbu5)2``ZoC5`Nq3j3rT45sNPU0ITHvzyFM6GwM%PzY;;drB& zb0WBGfARPi^+ld$VJ?$^C@t>?f8JBniDCP-ECc^SZ$o`1-;5^Wn}@|TZ-Vf%ny28F zW3eGW5D;+8&boTo{3RZD7PDKYpS0rAIhP45_~r*uPMJQU%OtHLgZUJuFL1Hr z^?TXsJz728P1GR5KxJWZ_i++$o!dwL(R282IMJ_$_f%%flaMgj)7s3k_KmftmlpS} zu}sGJ53J3x`18V+9F_96swNRC-ldNzfKi!1 z^qwZ@^K&$XW5yfk@*}D&2QY{}SgfnUNDKYUSX<&pD|yg-T~qXJcVmURoZX_GKj9l) z2a*aH2MVf_@W_f1oBijxc#j#QlhzsU{65gQJ?cka27Cmb2O`u=HZ262o4NB&k^XK* zw`6fWcjJ?l=AxOyYG2r~kZJo^NNGK9kzO?=WbtSgFVjv;?Gptd^X-qH_6k6v1G3UK zeOMYdq_DkLms?UK_~_gaF;9gTx=CC=xN5R0)^v(^@BWys3K2|^!+foC2@941DQz=X z!Eph0UGVVmaNl}gNJ8T9jAFA9piuxsTED+XqaHro`@IMtKw&1sFkPZ7=+_5{`Y~^O z)sl+BH*JDG3L)|U#X=guM6Q~e4@CE~Sb)3em5b_@KhU@{F@}tzTcz7^cP|n7v(67=+o}y?L<7||l*WWjNdeXtdp%Y%d32>($yx}4PF9OK@E-449M%hNP zcX@+4iuwJqnVfjr?Z;@PlWvk-FaEXgf}Kp-PHZ5TimWq}iL?Y+h0P^ZPeh`6grGbj~i|hwxcL+VRR*f1Qw#&^!Fn_D#@d^f+{DSMi?6FoHNd; zvfT3?(eT;7;++g@Rpr-7gC^gkv4eiA}++fz15McpQy74BCQ!KgAgNgNgn`Dpy;1CBk^OQ8cnU z5;%l^BqRBej~|w}mw(wqR;!UUmCwF%EI5@3TAp>F&!}ed`X;mN61f^_T6`*U|Y#jj{+l6-bKHgIxgYLAypV_w0L!b<^6xBq@j(|%> zvH1+ZM%m4s@ZdG)7HLDmGLlu9#GQq`8wiGC4rJQgj7{)opK3GJ2NJCr%_QF_()TFY zRbw1N2`H)W5Oj~^j6imJl6%8*@b;X+srQqx=vf|cE<1b={IG>=!$_1}xtRq>M9s=L zm4=YsHz&xE*mqM+pOMKeLPK3LAs05nMufP5T+>Z#pwa`fpi(sh$H11p+e4dr`iyqB z;E7}B36DM=k7>&hM=Hzrpr)oqjae}{u51jZE;n>;sU7DZY517428pLZM`_RQ8C5R! z=U!jslS3gJ!UD!rZ05ex{fz!5B}IU~V9bbXf{@#>+rgCn` zoPa|4t0xKh@gKiH9TpcLi$E!^1)$6L0XYA>De!ceZ>9jZi7^8=K4=8XELRs`L}=gn z+E93OQA&PqlMZ5m6iQP90=M$dhKEuGNrlm_z-Fk+NX*|VC-`3@S%`;2#&R6oap^pA zUL#3m34OVb&qoP_w!21~Lc*?5qbvwX*Eb=zzc*VrD>zOQCUdhzu$)?l+LqXE zFKJ;c+HhXof}h1n=MT; zHN@1WGrrxzZ>7kDT853AAwHg-=8~(ch@7LWnCHQgtG}?+p`ZS#^}C@;EtF$DtGN9A zNxqNhjB7V6>}S$2_O6@(MAED|nt|uM)Tn#?Xt$+zsdBQje7~e0f;12!)s4A@!!18* z+J)NTF~+g5xP@`a8^l1Ufp+4vp3NZqLiIoYK^jqf0IDA~W2hvI<-=0!->@HW19v9p zO0s}Zku4~pezs{OXr!zx37Z3w;ULM+#%b>{Jxbw*r^Xtjd?ztIK+-`z+MP1-N+-YF zD@7yc?!J^tHssq`URngzmn23{UbTs7PafxNqnh^RrB_S!L~o&X(ScLXR3ViK#;LxI zR9-h~nhWv^A9iW8iSjw;S$#o-5_+4vZ(>Uh>Fn?67znCGlYjWXsDwmdhgGslF`rAl zuH265vhw)Z(iG0F|DJB@Gm1xY~}h=HOE^KcbWSoxEubiA1z>-729M8WS^N}XL> zZrN0$wQLoM3H*1ojZn~kpZ=SaMNQt{q16;$#P>}|U4d1xWis-=?!iK{Q%U^nRk^0`fVu?o=?TOIN^j%pukHaAb4x5|KgS)rue`#jQu&}F4H1%$U!M9(fi|uI5j$S z+s6RN<@)wy(|o1O$U8`A9TX_b%pRQs!~L}wQ0KEZr(x2#<`!Q+uPepp4i1HnLhnID z%Z!S50OA-X8A-wAvrYX4x+BO8{OmGB?K;~BeD1b6Dp8Wst+V7gr886ZMNkQtVHSpS4GR?GOJWWwYfs+J19gdCZxW57Q+~JxcNHwgKwU$*!vf5Z zuKyu2_abgsM$la$ZWyZ}maQ%Jb`QM%11IDf zx3yS0AR-XSVWT6NB=KfKC@MeDNuC)sfk?~J(QGr(V9(xDJ%L~$z7B#4Lf|Wyiu|1G zNhybft6k{$`-zOVaz3Auc zSg9;`t&{S58)MLfs! z!?aCr*xv}|4CfEe7{=k-1h?sQzxJ17%#S}+Vzd*`(9W!7Mw zsNL*(259(RVz{o?uqE|$a$BHiVaLSjbpT=#LW1#K^M4*JM)SnfjLa1vr~|kMq0qkR z?B>b{OUJ~p*|cE8HaKk=?FLFi)qPufQt$CHK%sNKyQl2=-Sg$(~EvM>q*V2mA!FNn|rdQ9J}B0}6VN>qzKgK!%hIt__?i zZ+`kTENDT*LD-1>2kITA5*ziT+`4%!P=(P4neKc&bKA8_W$Ej6vDh z`7(c`(olg>W99w zDYZ(obn|Zg(5Vb#fIlR z{3Qh1N~`UgM-^XdtX(!ReYvnPFx)eUBvCzM^G6u?T}4}1DFC{pp?$+c<5diCZSlA zrBCpLm*=>q?g4+tT&(WL{;O>pDV>!iS^EDEW|Eovw)1hGDi zH_l|E)Af^po-u>6Jf2Sy@V7CI(e%A znIt$&G&gGxf&i?PEoiW(`2>Eb2HB6pWjf*@KDLho5bifhB_a|QTX`y+8cbd9 z<3$DEaE%p*iDcB8UW|j*p3%H|1)MA|ja2~eN4Nf#3A-mEJI)Rbs#BOGJm0V<*iyf~ zIh~~qC&;{|`E?}}deVW~Un8+hl)gLk0Q`vY`$M4R3ir>_g4rppZ_(f9!zXN)O&&W;KmA9_(vQC&)9u5% zDW{eVjVrt%rDy2-P|ZWt+5YNI1j0+e71z5=1yxp94k2SxX2Rs+gv(|QGiHbUyK)&s zl%Ns8gAOK_<4C4NzU*UEn|5SFfe~r2C62sjJTB^pH7Sk4rotm2Ku``!34%NL2eceH zY_et}Qd}PnsU&W6YgEUb3HP50Y;z)YPgxIma!GLE7c)6B2TrO9WImRJb&z#iSYkF6 zO6X=#X8C;HpkHLP)-gzjn){9iG24N!5!W=Y<=+#s@J!L+2~Uvrhw`U62G0f*IjU74 zb-3pZwG-1^a;>)%lvk&o>)xvuW**9QTo7GDvc8eJHF`@KDrw^s&g%#%TK}>5tzsYK zO0T03ZrfRSChRQ0_ps-)Kdac8ixkQz`T?hbI>4SKZtQZ4^9s&gza|DE2n8}ej0etRoKgA;dsp~4J#4;o@5)=zJU)zZm{Kv~_yTqnbVCsze zD@YnICN5S=Zcp#s)@CDfC#=!aD?u)no?~pXFs|uyd`xJWLT@iE0s)b&!QT9hbTO$l zsc&@TTWR@%$3!pzY&{n?b}e?7O5>pP4@)IXuSOR)WOXlHY{MATSwN?`ih<;=p~$$1nOMlrroJ9WbCkE)!tQGvD;o*35c0`h~a9SY7F;t zNKFT@0jFMUGNa1H4k^0|c&wNtEuziij|;6@hF=1;a9zOSxfZwwM(G-t+`w_%4wwVO zD|r8S*lr0=_c-~$*-ul;(su7v;wK$tnGL{Y4oRV`ZRkJlB{~+M>@I$BFA@0(0cyZ#Jku!KlpsF2*vSU6;dXkyoCl1KX(mvNbb?0^c5 z|B8(7<`-r10mW3wU!Pk-T&|S3;2_$^G)}h$(e9St8GqhqXaz}LtJid^9#ZQA z0s;?yiao?=G+&H^bp?yMG>@?FNuW_EeL3kRSkzQGcVPxt! z(UzR|6U*er-IiVnwfr@kJdWLh4s{Re!FVNb76($1I5CvG8=hz{)M=~c8PP;sBCdru z0|I$tMx8@GlXn1OK`aD-4=oc(A(BCon`wb3F;}=8)1eiZgbk=Lj2+TILimn9WX4`gL_78fR#P3Y`JCZ>6-f_HV2`uJ5S^bB_r4R)3-#@P(_b=Hz4h$Xq z@C-O-J-_HEf7L#UpTXJ|d_g)vEA9{G&N9SRE((q&pipLdQVq_^CwPi|~LZmnYSihnaz8?r2QO2Nw8yV|Q-GR#;PH32uV7Ll@fZq>- zMJ1~49xx9KU>eNN5xaUQ5N4-*mHY5JKYs{L{#Z%4vrUWj*Zy(TT=gd#Ig-9CO!#8^ zbp?82nU27kw%Py*cG}umr73iQnCieQ;@-wMCq?f#$?vhgb6m%E2)K^k_AISa`Mvtc zh>dbaYJuse6~8K$obVP1>*y{hH{8nHO0uc;p#+4we-kR(VtN;uoY)SlMn zvxvV-@mUUn?Z>vmZa&3>uV`pIt=a7kj75GE28MwY_oHUmL&xdH1{}8`W3=MyFeZ3K zKg?qdbNUk4qiPGBv0)hy)v$iiDJdJWAi`XdUKK~XUe>EGJYba>G%hhrpUTZ%S`$2LYAxTuo z+GdMnGOixxkU7YAg=w|#w-FKFJ5vJ~v5MqIlXp)m{nLDG_Yy10>%4y-Q`G64g%gfH zOh*kz*=5__6R7#3tC&+qy@x*!PEU*Jv|4MPR-T^cd1yHV_&so*r&hoH?6|qVY(kn0 zh&d1F&UhLf%lPM>`-LW5tg@YBb-qIHezq-D7-o$(jF8L37D4Pd2#)CJ4g0nFPJ=}@ zlt)@W5FOWuY2|72b>q=@e1f;vid*5`IoSQP5Iqm~nJifp1Ia&&RJ#P;bvtSqET)5v zVlamP>A^+XP5?z25XR`GZ zwKIQonb*ss2Nsu$J|zjTkG4lp0XIx}1IzLx&Kla1u_9RuO)X^DCA!klFQXLAM9E@Y zd=^>{8IDe@h1L){5t|Ah5G12*sQ1|_xU8|7j=(VM`X&qHXdp)o()85dF&s#j-Boj)Bj!a?Eb!v-yGed7-`j{d{WCSHc*1{CGPR-92 zLM7hK7STN(P2?5K{`X%MeQzJ80&frZc+F}rYRBr}k8%r9V~FnXIGembeRTqoLx=D% zAyZR^P1)(j$T^$f1D=g(4C9&K_%NU1@0clLM3DR05L;wDAvfK*{75$|Q3#Q^zAYe~ z*vzcPFanJT55lQ-_eXp}uA!mHDw-2l4Iu@1;vTI z00Rw-cZdE?Qu6`%P(my8Z*;`;IzuMrq6i}%6cvHPVOB7Kp%0z^LJ>=PKcb9kSe_gP zt@Wu_xQBA0j)(uSd920en0LJo$hJ!*%|d|p+J@GMhQ3Hod}L4X{~0%$BU=glovkHf z>m32J@def3Awi_(T)_OC98G9qP@}T8vf2KJms<576Fr*>=MCG)K^bWaY(*J}A7xoA zxN^h-{f5x`ncwlIxQy@hl?zZ{4R!0K5F8qc2kay=RDyy^H|rTiSXRw-nyn-x-hgiV z?;|XHW9OdsZ;Y5d790LT7T4yfeeId03MCpHGfoAVgeQP1aK=We8mXxJBK(Gw@npBw zGHusIu_=lXe2t*XTO0FmyuximtThNbNDUJ|Ok4G7o8fs5G{!fx z*jAx3s|0c2hJK zp@lN2EoV%2YqSov-tsGszep|xMLXs4t9T@mAA4z#7J$g=vv6H9-Ka(8CgPSR&YteV|ZVO#}HDCa8gYt1m2ut{V z?l{CryK-m#oLdy4hKqNc^?shcGLs02-Sd)Ug`0J1fF&NEyO%T- z9q3@i=FImT)hPdGU8+Y?U{E*a68vvI>+Y>VCpdS2POH!6=|*SvfYTSto{~^#VHs5@ zMVg;a7$lhZ{@WCSU~x@Cap$8^1*&9Xe>rg?biYj)XnYKdP` z4fPER{-Dm%bh+*qmJQFloPd&kEFMyknypd~kyB183Qq4gBXKHaNFck+vZdCa+yO?WLd)9?-VCqZnhKs|(8Q{9GLQXnVw~GTJB_)1i9U}d&`z(PdR!#5^Ly664Mf2PER#3NTe6lP zr>3N@W2&XH#cm6cKr@==RuMZko1v=eDrqKa*0*q1a(o?ZRetO>jaA#~fc^UOlMk@R z>ylSjtpKr0rqDLKBKsX(^HDf@D&*X-!xh6S*AZgWN`e_nd?Mp{Fn;YNV$V zbbD{n_lQOw?S_=XF&)e>F5b{=x}<9)88F^1(0)x5B*Ux7k{iSOI{*}7Foi5KGMoh& zFk@t2n9CHxAvT&QkVpbXfBz!w-R!{9M&?6Cx<@^={>HuGLlou`Mc_)AGY)xzf{z?O zx+FfhO>zyl&)A3&Is#UKv!g@0G9iYx-6J2i=q=?!?mB7p#ER6iy(YW3NHjKF-{1e0 z&|rB|rCB?(wh;x>W&wIfem&U?nD=lw68*LRpF&i7iLhM^HY;R*%W$JpK+kcKM+%jE zd@cn5Ewm&5sh?I5IfxlbkIU4kq6&2L?J^_5Z_LE6fJzw396&My-hvakGSNl$*EofT z5OGia52yn}1}nuR5f;rKUI0%LL$Dl`oIMx=Fr^sqGFuhr+aFH3rUo^XX2id&Ctja= zImeE!oR_60ZzTP!_Zi*myzLLfO8%ZjLql-ZU&9!7Zmf&!N>pknN>RDX^t2Us3?P*Y zLsKO3l6f|i{wkpj@NsW6{Knb^zS*GzKvNHGE}`r#!8QeVXBMOH4cO%n4TxrR8ubB4 zL-BtVQYMp%Y=zh~Prx>Q-Ghv;_}+bzo`Dd5O24;d=~Qq1(RQjfDQ2~Zx;p>ilKXbh z-1VFUTmJ7=(>A90)CEV<4GSe7dWL}7U?zdd8V6x{TG z&hM%*xp2c}osA0s4RnG_9hN`ZN6vm!zhbsOu3r`q#_}wG<4Ox7Pg$O#@ZDm2-RRqHW#*v%w7 z)WzF?=dtdAtznS&K}m)FtF>-3>UVbaUB0`2&(Ny*RyTYPB5Yk9b$gW?Y47jV2|zu` zO-#(s(X^Fc4S9dDL*v?*C_K+-qE8L-6K&vWLB-bsmSWj?JvSO0CcoinCjS#wa72wa z{q{pbwqROr@E_^2SPL;`8>#fE6VWdGv!O|bL6;#WkeeQ|b`K4@8zDW!rI5XMd}hTv zi@W*8LyQ$lKDC+E6z?T(5ZHWkD&vH(7MQ*`RIz|KQIBMvM1yvF25TcNNgAl$YI0V3 z)}O!9SzPe1f5EQFcwZBT#uGLDJ`j+5&f$gE5c1ybSZ?T;ff4r`ZKEScKvHAcMAU@2 zJe)UWhw0OnkVOL$j+$cY{ag6eU);!Q~z`LyRwgu!LJzk<6cWwtx& zHnY7_?s;RiSI2Jbxn7rVWi^Z3T;#!@2Y9|Ajks49equ{mcX!^21V8L}8ZxE+CKPBQ zhKD69vF|t$j=^HsaPm|7Qm<*0=$7Zs|IpeUmwuZyqf{F3^L5o?o7f+Q>ygAG8~PHy z)DeaRLI)iPpFUbG$wR^6^*uQoff$#E)-7MhQj93=G=Y;cx%y+=6I z9RY;)EG0lewrA@~!Pl)Jj(uQ}k;J2On!?T&x4aR5)NyE?PY`#kFxnlA)7L+N6LcgZ zg(PV}6G1!LL5mG-W@~hyEz8-Q}1Qi12e zN%VT_aA`K9ebrDxf1~nBCgA7Mkx0LvbeQtFuOY@p_s7)IhKIrojjwx!ago*&eD^3j zW5)5Yx7mJ78~CBBV0Mvk^w|Q0}Dp*u2FpgyWS^;WI>oUGc{Pvb6wumpMUxP zc@WUU8ry$+oAG$k@mxwllSY6tgYi^95b)IQ+_ve9C1g{Y=^n`G_Z?{aoABMe-I2X3 zNyG-u(-8(YL&R7jWi}MvCy3Mua+Hg?T5bxqel!B6QVP@rDlUf!-QheuSuE%KEJ;Dh zxeowlAdmMfi8CV|m;zgR&~&<&Zuf@>?4^5u0v4k-EhB${o)vcbkjCly@Sl5#9}$|x z$gu`OM)n4ec_s~Pq<0pG3y*o6zCY2g`K9og?Tee;+x^xC%SsNRwJb9RjeRR;B{1zu47bI(=60fPrSIVt zEH|Ho${M+yanuFd`KMnC0|IATei|zR6THjsJF$w(tD7$@5#L2jyt3#QP2J>B>qq7q zx5Oafs_s~I-yM4xWykhNLZ<`BvOv;q2I%p^#8yw{9qFa8B`;ilu;?!sz(9JDMA_|w z%h2|KJT#9l+`Rke;u?am4u*!Z9XcHVik~tqFim}OeOMOg+cta(!F^jwG`{YRCGs9Vw$NzHcWmT>Nv}%av7} zhF@9buPku0pWt_oqR1U3L$Dy}-HB?QLL@W^w|$}8*SdLZm$sfMGy|PNYGZP1BMG4Z zhGb0APe^M^Nq=|cV%sR+h*7&jNTtj@$^{|NIj~*0`Q28 z1Ibvl`)P9=#hZNKilaLge?n{EsoVdNbUa7?1whnAOvsgi=`q87X4gt0?Q%?kB`Op+ zV0bG0hudz0kAI6s1GL-( zZg2adhGrSzY^m1d@+y6PFfMR56VqvK(D5Yj5RyM8J8b{(Gu^(S_|loD{f;nw^^xV( zLuBSQMMk9m!l$_}lyk_Oqc#3}G+@IXe1|+>I;h%m$(-F%QR0~fu9VxO25r1;=o6j-PZ;Gf+&J;XrSDTA&DxIL zD=RI+&lE5GN;)2XB-HVAzUd4$$zA=SkM+#2R}inkL+}Jli&D9jUIq*A_znCx47R3@ z^ppz4lD#2$4%H}$xx9<&{xU>hSgTtOo)H|bBhoOlk3u5oh4&i0R`t9lar`=48PifZ zJVic95iwl93cm4VFTcxRib4gz;zR}9uf#*81~wC$xfh8bJ-d297p`8hJyg zbz7m^yD3Q3FfMN7*CI)Ick&Sf`X0WA_Nje=8%%Bh?e6>&c%s;oGK|3?hDyFENj(_# znL>VQ30(h1myM{WE5J*GREp%S1h_eD|M3pjBlh*bEC9CNEqMLCagm#{&f(Rt%9}%C zB6OHtI9)@P4RuH2=d^$&zai-?z<{rer@3fEdkHKX^#nTPS)`$YqSX%J#$@N)TR^R% z|3S>dZvLA6?uv-#!05*uiSLKcs!_yeAKDs1ws*JQ*$CtNI*I2Q@%eIs$i_na^n>IO zNbrrHbAfhE>gyYNjm?nj3sAsxQ&<4}BoIkR-1slb4#>2sOydL6X|_x2m2@|yl@P%P zWpg3b;mvREe+x=MQ4^V$pPMby@uogHuea?B50hN5sP|^U!ST{t#Q$-RHjq1y>faU5 zHX#0&=G-~7p@gPtXtADA+OpGB)#t#VRLn8D0cF?XTkNHuC13UJR75)tlC37AJ?n=WbJdui^ zGo;q&5fkH`#1l?e+O@Odp~W4^8YB^Vu>R6U(FCY zsxRae$Jnzd@Mh)rA%ay68KI-)U%h&b&=cFYM(L1n86=Qy21#$o)!cjPcA&NMZ=If`uOx$_Bmzhf?1}};7GqOgEL>fx&=SD<_Kj^4s9g$=OZ|VvXaRDLG%k( z;}O=`>6E!;sT0yB@K5`D%r_V8ODa>15jmI2nn6F7%l% zv;&xUdqKrakeE80mdGaIPy|WBh7yEF_+~?-=Pv-N>fBQ0KzGD%w*HTcE zdhgP?PV-V^{KMMeMZ^;0=Vy%AaqHcEiy!8wZjr{*@&ff6+xZY}3Y^TWpiJZJ@i9XA zP#JT891!W^G(yX&xN<9f$=a! zzf8GhJnO7TgN3Xl?6MN-CVx@LoFjSyLV!Pi5t&{=($$^3r6^uT6Jj1YriFVvz1 zHR&gpyqdC|17aw%2qddU7UOz+;fJkUT4C$tz7O{86hCr(j7|ZLRc~+;g8Gj4EPAT*Irj?QF#)qw465D*ik9POZ`OFe#{>$M*2JM?x%v@y$ z!H7yURJ<7L(UP6=s8gX{>v&ongD_=RZ?ShK4{U+iXk&9i$R8|*23@V&3A^~vPJPD5 z!L+4uNXtv4D)khpcIBA3@V&_a8&PDHBzffM;nFqEODYyDr8JgH3Rz++PhJ>GtG<2( z%K0=Di#!$t>D{yjcnVKN*fut~ZI19#|L;_rY(Oh;B&L;k3m&i$K7qGn7k4u${PGG7 zlGdm@x&?AEbHwz30ZA~D)Ct=EIr9L6plzQwLn#u3fRur15PYfz%|9belxNf%%r;Zv z8E$SLTB^9wCq*y=mptP0?V~`4FClPxJo&4hdJ1>zsBXlUIW`^kx*2QAzh^v)x(=6b z-9zUVB5wgr`|9Wyi*eml zZx|+iewGL^J1BbrkJTvb7Puo;YiR{nn;f1yJRuu~IcfFgJ?|}+)=(yE5O_}SD<8B4 z=hf!YUa8iIr=r5nF_W8|*8yG&tlIE+?kXNqxeU4K z*x5GPmKq!PI#fmDA^A{f>d|hL)A+B=16_^nHMK#%3t%s-rPGNVHtWU#DIU+gE$04z zG0Ia^#Y1qz&~)=}3K_QFYM})x-3WAv8ArHKoP3<7+$`Gco$g;0(2KL$0ps ze$Qia2->gmL`d)wRK=^P)C~AtI4BKcQ|}0E&061RMToz@cT^(3Sb|)Nv^uDcvT}|& z+ja)NZ*u=|@;M4g(0GNaS*b~BQd+_v#Id_DWf8&^kA}iBBMy#SpmJ&ev(pwQ)4cSTJ?8P$Q4U| zGFCC17z7Ty%7M^j#D$5G2+%0BF|ujsb)Nt3Vhm$nd_zaCEgbEZ+eeBtr#7nGT$>QK z4e49o-}3|>+uBFaYsf>~_Ya9n^Pvq31D|Vr#TuWrV~ACYN_#TmtS>IiqjDT$tnmtC zdccBj&G@okl;~yC>w5y*9dRc#P^Amb)`mjrqbAXo%1zw(dU7_$y*z)$l4f$cVH?%b zliA@0#WLJ`IBEM8zb+A9xPufZ3&QdiGZ&SG`9i`I!Vo0pt}3}8xdO~%BY1@M=4fZB ztw$HilacwJa%f?l4`5g`P7Kw4u2HBfDsq80eGjsQP_#jJkT|{Iog1hfGWbENQ7wzy zeWclB1^$1mzpg#MVIFJeh94mmGC2Q#3mFQL&7d@z9A1a-cLdPS)?~S8zX1pt~&B#B+hWe2c{!Ahg4tdQBA1FENGER*BDj>Y8~We~Sc&D0Z5jJ%H%{+4L>leQ7*64VP0!1`Gv9HP%(fA}@{Y@5`P!r%8$o=+IbGfd`K>O?sV zt!CsZrW?q*4_=Y9In6!T1gb;?h8=D~K2gAkO*#`}8{%U+J->*&YW8;NhAxZ!8ui?_ z>4$=?iRirts=%C6v)+CJ0Dl%|5sRSd2$nlC$x35i-Iskx4{p$7o&G+M9T7jEZ1B zqAosl`rfhcNk1Coy(CFvh!C-$(jsV)9`x z1U8=)UQfH;ZB|>Hifvws;v@xU`dK0MI{v?>y_r;y+VCg>UxTYd>psqEjZsL)$&c!r zYY6t!%K@p9wV~<VxbSL&rS2BH_or?Na(Ex8Dz;Vv632}lsSqH0{MwP|HIzMyJ(e|6Em-%@l z*Y&tS6{w2ECSypOH%1;x+jPCKC%SMvjNL4 z#mKri?{}Da6(5M@kKvaThSk2mVL0+mrQc zNZ7Q*g5RLcl4kJBorR@-=~GNzeZPV9r)^2=^xvd``zU6>b7h!C^(801~y`= zXFypgxkL2DN^4?%EQ@{F?=?P_nbRq0PUaiYYQP9*x_Ou}bB(e6d&p8CS}Z*rPs{;F z(LVlWEypkVD?dXr%Imah-ric}kvR)fe_#I;IkMn&$146^Ci@<^yTa$yp1XfWe$Mr; zJ>K-8ELLcf$_1x}trgy3+Sb_De#SOY9m5=7_^Ai2&GECKchb!Tm1o^Gc@sjxTjZvH zi9y5MyMF9%*MHygpGL7oRS0s{?BtA3rDyc9zdtmaQ{e4x)=vRElis=aZjtZTw7!k% zarG*sa=xkmO*s$qADYeNjO}q^YduIf!Csw?VMKuH z&z+yl$$NwAAjTF<8^zt=JA7~kfq{sBb)CUJb?iY%IxM#xhGQ8R&$TW%Ylf)R`-!iq z2v~~K{kn1fQ^WIN534hD47y2SC0#c@O-Ez+YLe5fO5t{-7#nTJhu1cF2Ic4? zU2~>YB(sEyzQgfZqs<~m^Jo?U}ZqL%h9Iht| z+_9MJt_?HVg+Jcc59C4N;8BFpr<5BS{@w?^eK<4TufoK)4oL<-8;-dv366SOcGI~L z2aGc7@PEt;Uk9O$4{1?bkWFcuYqfy%7M+Z`ts56AhL#977$6Bzap1e_d5j#0R9%`#|JPD*Lt>@da;* zxWu4Rw{<2VO#gF$&I0O~&Ar%*SvN$F!lWVCc+2xx{e)5Y3w(x%&x9_$5>{y>p%GR? znN<3iBxDYluB!9?zrBa6O2%cC*^K_M!zR(c=DA{3{!Cr`2~jUZ(*2Jq@dY#l&X`DA zNAwknJ8%y*flpn=qGkrOXoCoO>>HZF`vxnAosL1F@oQY<<@D>jHzctnE`B4V=xMy6 zc=#fTo(CXWvX@py@v7=U^QumvVkY|$jkbiH2z5KtW6OS-w2GiE`06r?pWzyH@^uro zAAtOyfW1H^2ldf!&BfMZ(BZ+U_DOJc$c%(cBd|6m10O&fo=?cKm3_L_9ub zb=%Frf91FPIcZg+N9Q@Q`+0zYiNV{*fcxz@&uP)QU0zv^kU0TR(Y~2j!mmwDE6vBO z4GpeyS?7UgW;_KbYit|onCH?s#AQ#3MjR8*CH=gv$$^vx5fCnDsgU50F@KvkR1wfM zHuE=}*zqY9P?77R(w$2Z`OY!LB}PqI(HhTi{C)Rdc3kmKhC${zF)k#|B6A%SZp%Od z%^`e;h6Y@zYB2u2N*`I=bUzp_7^{<7+aiF=zJcI5trtef=nvJmWtHiq_O*EqG{EgP zy*hOyl0%Kb$E0ELx!Gs}B^gu+X|kDny@%QTfyu96VoM1&_-5dHczudXnR;zkg2vnA zlITBlfPm6d4CJjJ0W1=g{NhN-R*X}~@>bRf-d^%(X+ehWxk&K8qx6|<->}>KxbQgJ zl0dE1$a_E(con*Ae0$U}dHr4^;I}HjAky(~l#k9wPwtLwdM-rKTk09wTRq=xTZ5(| zwaQ7-(mz_g)@PlFjx-DodWIv1?j?y+M#>Ywml1Ax90t<5yr2jc4u(_&A$AG6_Z5%D zY8-RkxE2{)(GF02ONt>6+_N(7B#fZ-#aZO;E+05Lgwofaw{FmMOf1NUSm|Vc&2Jxer$AME}$gsZH)GF=b_N0)ui8r%ehPbYfi_`-RxNp8t=sA z`2f}>Q+ifUJA<8kFLML+s>`o_Ou)pf#U=dimKCp8N|}qku2TptGU!_4TZ9CEOLPh> z{m`SU70hGC5NT?vuOB^?rwh|AzxmXcA?N6Bz5sw-R$2rG>&)S%0_C$*8@#=@bpMx} z?%J_po)TL5i3RSba#HFRyWR9%*x1%B8Nob;ibZ7>i-JniveP{@-#7UDWSVY7+~?*# z)}8U|Js6^cQOm>Z0i61W;sg@6IpXu@NuSORaP;gMXFk88xFCJz{x{S+D_e(j;(cSG zLBB97A>}h;zpbd2t9FcQ0HjUl0gQU{gd2DC@XsGuS=3<`Z`8B8e*#V`f4rk$jF5i* zU_gXJ^_HN}G{+0urPH!Q7Ps2+Llgz-AfvLSqABIV+mpBdSzm>kNmw!UAwFg02EgRx-cyq9iMPlZ;4-O0) zY!HdjD$d)0{wadWiZ7b0zocM(fa!>6;EsJg2tO zi$%7C7UPjHv?Tp_ZZX$s<;!8QQpRGcFh*O%?=ChV*g|KmZGDzww%v+8#s078Ax*e1CS`i|VeXD=8j!BfSQ!YA_efaE zAUm3cpnR4+jhKZhQBXvg^S*39rFvVT{=f2G1iJXK@a`#Yi5<0dT~wO4B@A8Bj4gVG_fu zu}=G{|2uL_V};$~b{ey?yyQSL_J=reb1#zs4nvW|x%kIa^TrUA8rIF!9+R8>Ccjdx!**YO8?R>AnSzfb|aa z4)YItgbjer#r#^pEBBMBsJ`&5WnhO6^Rr?!i)< zzwr49{Mo%I%JyeLzxSlY~! zzaH?oyRdgm^R%ms-4NRzbw2(GYNFq1kukqIaUkD=6}dTtn~e=bK_x8{4@{(sGUUoLv7$v&rB3&{)w-tl^5ZFhxu|$)_zMd(J&!VMwmA4bV zuTOI~R=vz7-5z|Ux*nmod85IA26+t3L-zj)8Ix$WbF z9C5mCwdQ_$?Eg1L^J%YlE*6FNDlaeB3-LH#Y5eSfZGHxMexhkt+T2dhKE1eg8lWBQ zhbTqg-V5(LJt3y2+m+bRUtiXG`@#*%Il>duTe0Gd4z>?Fww^hrue|cp6+MUhY>iJt z<6^DiJ@UGkmQP^xjPk-UEiwiBcp|?3tR}6S8wZIWY@Tm!aR!T=1J|9SaG#p{%J~F+ zi-7!2Hg3X9N0`4)vqIKO9U?X-9^gLcy!PUTrS^AWE(&@Z{Qss4?3`6tlr^?%`>>d3 zBfU?4fD?ocOAh(?m7%K|e}_@}WpkS|Y{jkoHu>I4VYc4vd0WFFH1h=Fkzk%-eWD&` z3@r7({nw@U!eZoBgiH4&FG1dA##~|BKBaZGUN#WFf}VJxf+@nnH6suAin zcx7{~6&&}5GL%Nz9P$h9A=ZO|^T|B3#Y*nul-R)vn|w(Svre{)%i_{2g(brHt8;{iHu>hE5g}ykKmEx(M?p#36ZwnDwu+2qYwwY04h)k z$~Vqc zXBGsR+wlo(j&KnHW)vfu*)Mu~nXQB7Se<%;gjNK};%4(1kcZ#;S@2YJ*s+Qjb zOk{fniC0twnXaP&dUEV@!V~RHdATt^20Qdm*GnHO2;T3~0wh#Emeyx^(tk~tXYRyp z#~JLeP`aHQ=F)1hJuHu4Q)+9P+ex?6g)6)85bl_u>;`LFqEejpQJ&CR8`LQ_nT-Mw zT&g`*+m}9e;Thf%Onw`=2YIIJxn~L;@=ezEp^2<@J~UP@lXFD%_NMRJI`wmnD3dJK z2~sxPC-Yv1Y**?2uPVHhiUm*3%FDJ{I6f*E)(e@POGoO6e|($wDA>LgjC5Az(eZb3 ze9VZ`SQ=x+opLRX<4Zmtww?IFc6!H|*~9YWdv3?0-z7H;$YghBCpWbl8PKUobq6${ z5F{ZPxXMtth}2th81J01&6!zYoK-&sXy@vnjp2T14N>j~;E3c)NA|=4UlzKuoT9MA zlKD68)y5s%csoijQN3OKzwh^XOA+F?;%u+{uqn0b904j?ey$W7j!(+Zc+pg{ayi-t z3l71#(x2o(lN9WEy_BR9w~T*0G$X<3q0a;1O?Q~KfAbWn#7-VKby%pnKJ$CVfpToSe|A(o!@N4=F+`XkkQo0qC?(QKV9SYLj zIl3DurIiL@G$PWSqr1Di*=WY-XWw(q?>y%}*!J2!yYDOBSK8jjcTEQIlm)gFx3sHY z>HvqVRYJpWfZKbqt}jCmDC%-jTv^?7gZCcTSmns5Irwe4X(**D3m;zzHfmZwc; zlO-`T4>Z$lz6Z=)U$R-hKcf3`0 z!tW?l+&n5vx}wq|>4JtUb{eE_9CF*bDYMAW3IbckB%dGA(87|JCH*8p_xy_C+^it*cq zUA(mK-8w=i0xnC8iab;)-X0Emcg6+m=R5bF#MC;9yz&B~1U+TUnKLSsF9~ryWdc{c zTI*~D;^!_IZ0tZ>_rAL|YxRLlOpHJkMzyM|TZD`}rONM8P5D~+o-XEVct@H^P_qQA zI>FPRjon{0f)alY5TO7;&D6y1-elqrk-a-LIqzeG1uzW|kvp=dYBwPBXG$V- z;IwtD9g1a%?@X;7oBm$u`Ux2f%8&Xt)-S(vonsUF<7+ExO|c027b_4Hgi3pP5z3dP z(vv39gT4?wrHGy&3A&*)o1Jke;rWq;B}27iEi_TW^6N20Fv+>lZrU5Y5wdat>*9E@ z#%PE9_igq>n3evl9!gsaS4@?VZlMT_$M*=6#@wn_3p7w#;vGQr_(LSk;xbD%sNw); zT4`P2V?H2}hiErti>gYlFNC*L5^ASjdlvZO?*%ubf}qgBL;hDoEa(kBRC@LP=EU(& z@S-aJU~c&%2_v*OvyREyVcYo25sA-Q^tYA$`F{Nj0i(Y z;^@#qh?7$NzlP(lh5Z^joI&9jbFGSWb{b7>C2R z8sK6%+3|d%{Fr`}1iH2apkp({Op+pTog(Hcnk67+`0nNT%AC!;@#2>XCW|$PpGL^a zi>umCRKCQr>B-G){7-m8mmWKO()>rKj;T9#p|tV7L=dKFbI3KtgimA^i|ExTAL8;D z++lWlh#<~`OJa96+6jJ%-e~e<77&VO`yI1+Qq0;wDxZ%lA)x0Y>CFzlSE@4ivVCf8jaU_MSW{}{cU!d1FlgLjaJ=WX~ zrtIpq+Jpvn1HWD948Te`VPb_tSxY`n({&su2EERL7@JnEh8ncc)U$erg@?0T*8 z1Ayu03H6tyj*B?wCYM{pU}zQKaVx>=ybGhH8+C6)EvW{3!B?p(?jBjYNokNOdGQHU zN7+xgJdryuave3yVS@b?dEITT7f)#^I4!u5T$E7CE6n-bA9k{DrN9dI${GfCWS_B}aWsH@36x>eCtSQ|u$bi~oQ8S2=;!pUTm0qd2wMgSlx86xLV1pzy}F+A-nmNdab5l^3-S@Ngb<2L?Ai@Luu@*2_Y~BA}+qzFLZrQfyRIfqj(c* z-LOwT;KoA3Jb%c?m~9u6B@>z&i*Jp>-^S-y%`k5xCr!3Nh2Z=_h1WN*zkU-CzJrT5 zKH)Z$$9lBJ7!3L@@s115Nu#PG)Yv+}ox_vp{g}=_hWMXbrxS$)B#L%GAWyk{G_HV9 zE^Q`8M#GR1WY~X`o(^LSkaqZX4D{3++=?LG;mc#Hn0N&AuHE7@T7(~bc`E(?8{B&+iR+q3SZjvbn_bKI#j1U)o}kmbrkc)gVn-c5FCQL$&<=)ZbH z8|3)d<_@q+dzpD@AfkPqj1A`WvE4TwyIB*xK-1JDdZPjuroFJiPa|p5Tj3GN04-A_ zex`V?t&h3vt@ZrZw}mzw;7^;sq6BSV8#cjW=ZPqgULy6n;D^TGTDs@+DA|~O3ZO1Wo zsqOveCL*vckKN6SA6x9W;{7P>6HJqP!VhNk}6flA|`YkZ3J$uSNdep8k&MbXdN zLT!}^D!b!(c3Va@-b|9ea5@bOU{U?`9=FsaC{>nV)T4=c`@q-A3VY*ycH44lBTweC zf)BT^?OOp* z-S=2?m2>(!nIG@N&+SD@^<^;?finR+4ka?-5sftE(47!e!>8%}qFL$>RJlL_)hfMT z{opI=y4G*m`?8YP2C5(I{kMf6oloe{S5NkVS?wQAjP>(YMGNero{hKBIA{J(i z{L!j9;bL?zMdkRY{aueTnWogfHjkAzz%MeOU*&%k>%EVb{A*bfIDJ=&%g3EGf_1qN z5jn3TA+S@W@0i1X%fOn;nGTHAm=R0u3Gq?LbWrs0R@;+H8F);XqMSj_Me7{(-p}i( zXVEmaeeOPv0j{2$*V-hA?)QHsF^O#C*rd6|CU*yl96nUp$>&KKd{0`Ety4bGoyvjY z#{GQbG6UV6LZd-2ViI%0SIj=Vrp~U3Yb>0RTAU?mUfJh@FL_6|#1c|K&ha z`6TGYm;&paE@z7WfqsKV!~cz!ME^8ybSiIq0R9Uu0yYOG(TE3CInPrEUQfXgikVS~ zEdCczjs`%UsX2S(*iqYMbF5atb{=f~2_yVGeX+nqb)=OOWpK-v&Jnj&yW_guv#y~- z&qOWE;A-P2^V{N=39Bo*CmNRzMxCL@84$(-6th#MB^#zNtRdukgVoyP52G)s%c{Ez z1SL@jd7_fvRP^80aLk{3%3nC4{zz+?g{cPuPu~mpy6($}x*{xAIa2uW9iRs9A$NTP zI{`*tL09jhgQUf?)Q@HL;qW!S=vICtfiKN#smLDxyo?yBEMxLh(0_G<*I}mwrkC$iy&_M7(^(`QMG(@@&9 zPC0$^{k|68gbJ6Bf63&hHgdomzAh%Lyuso5z#G;tlw*dFJt`3mFQ%hkBXbQ_C!`2F z_xNuaWc{|Ikq=r_?XOfx zxa-Chwt}xE(>-g7JyDF57DI)F@mYha-8B;&(;hR7jDZ?>`bC_4f7%H%7^o&AdZwk+ zbV@p~7&C_cJ*qT~AL>J}vLxoEw~aPsf@ALc6W z&#!_A%@`lH7%LV_&D+pYbB6mfoV)MqQ}lWjr=gb% z0rDQ4jDT}P<{na)?IpIh^ht&%X;AyV$#~_$^$6~ZoA3VpX-6B@&Q@lNRH_Mgq8g@Z z)U@AA7JksGQ;Ta%w&%QFe^Er^F#9m#APYS%tMmdIkuP%v9x20eG3?&4lpB$jFMZQY zv)P4ubGDV8|KGEqC^ozDv-MGQEBo7lN1;DS<|rDBrpB_hWPIiZjZOli(Os=>88cyY zkEYL7 z<@0^2O)84r^mFAIqgkbjF=`^Ub|b#+MogqbSpYBiH?3*brnIlFeN;@<`N(Ko;a*Jo z(##CgxnG}9Zhf~{cZYs98z|q&JPwiGWMz=d091dmY z{fl!8+nTxgenhHlzxaO^Uh48<_tK;G3#9M(S*zgQeB|v`L`jN$QV(yl#pC_5Gr^Z# z7ub)aGRo_@Ir4xz$X#Us#r&tUrCl0~*lpB*ZWqma0QrtH}ja38ImdHM^cb4bw-@~^Fw#rIug56G!p;X-s34I2kFlTJY3v+(WH9D z;(>ncSgRA&n=>S=JM=k>rNN6a>bn&6Dp%VkjcO5*1(2wnQn;q3vwkg*b&#HbK)rmF7o`TX|wQ* zkSoVdm+S0jP|!qMfHBwKcXLBH$2QF&Ir<}YEn20VS%cxQu`!V(gFoTx6{(N-(V}_( zb~3D>zC%|=`xIZlGB1NNZTSg;PxA`%+Kv$aYm*W*5{=+wvR{f7MHXb5fY`;_k!Y+) z@^g#C?7rV0>2aK}N-qB~S4Dk!d;hV_hB$>Pnn$s_(nWH0!SQn@HlMj@2~!j_HH#54 ztMeGashd>|hQ}-{@5bA4NaJ6A+OeR$BH#H{kC3`2{?iO@WnwV<7ZS?%M)i?`e;5z% z8Rk2(R{d}Hf02QOK%7jv8!iBhJ^T5=|5~i1(&X;02q8HW#8fRvhwL$PO?v zYqbN)^p+$owsl+0k(Apg+L%rwDR<$VO-i*X+@s(OZ$S|#lJPayE>SI#3e9aap#*2R ze5K4k9(inX{={{E5li>XJ4FT{!x@Qm1_l98`&-hB%jjHZ0(iIhg7l+^ib2}Ao~zeJ zGhCYw{pn(-j%odSo@!cog!zC48uGM7 z6*u@lowZ>sffrem&Se^oGtSarn)@$l8L17{ubxKaHf`x*ZVDwx+{7^c{wAN;kqBNf zpo@@xT2~7ByGnxQ^#LtF!;*F+W6|h`a*Fq-(DpdwETIj?e#cok^o5lVWWftTFfMQd zSorA%W|J@)|0}=CzZ(m`@xSFhSk7(RfPlB2Pk!9j|EC7=`ODjaSC8yzhWLXD49M@` z`owR}ZN_(hh@d|vAgg9AyyNC$*B0<9|uMs!etuqwvyjySLAo*Yjm#6fe-T6dzk+xR2fHodOffh+t zZfM9|(cZ}v=&S~PzuX8ou;pq;sx;Rj*3~9H6z`aI41oTri|B_YfV1k{pZCieCOX)l zyP7#n9P2EtH}eKXw$U_uM zX!}37vi>SB;CREF=V9vav?ypKbm?s$?b+^bAu1X=w)Z?9qCd9|I{> z5B%%!@J2FCW(8-=loV#r;8?4v=3Cl+^^XtQes&k#4nb2-d_^8-7CuT*s0-r1KpTJE zI9y&kHgP?Fr(Jq*sm%2`hwao22j9;tjEbLYU}NoQ!7yh&nR!^x*yOe)zyUW%ZKIP9 zHSQw298kN>LKv{BO1AS}5@TZpg$1=aK^W!&MQE34W>AwOn;!xBSY#;fhXjCq= z`zs%fWD$tgH)nWQ)3Ef|P-e{?dsgB-%EWWK&txs?kNZp-i75n)3MH@az6RHa0(%=W zShD|;i+5Tx-qveHNlZH4tbMAj$8#_a9vsBdvt;IebG^-a_em}aFPIIq!4zdKy+ zB;1VDIlBW+WkJ+2Tj@BBW}Mz>yj^jlaxTy>-$Y6Y2~Q@{hmIwI8=3nQeU7~)z-d3J zS$X(%V(|NnGk5iVz#6sLKy$&)Vo_aSTf))eBD8T10`U$W6$A=Abc&o+i_tcBIknyh zu8lr@eHc=I?pr^P6#8#uZ(m+{pugV($77VNgxcVw`B{u!8|{Dc$erW4QhG9VN%M!} zoi^U14DH0YeyMg!83FOL$uS6G+w|XyQ6~xMb=54vrAS;Vh{48vs>7M&RmMb}dcZaj zY?}<+G`R~^Aj67~lOxzZUaZ=8UX=%268awvl`OApTr9v+!Qy%)V9%wj_z-*?g$()> zBH>$cjLO7N#|NPu)z&>RvQ!;|_g+b_;1hE9wa;6F508fjj}Tr?|K@)YqBhZd(CRiu zg&yzu*4HDppbGd*cLtxeOxou$`fm z{w(VJYxn%%&F_X`rbo)@)Gf4Uk#9dEC$8ILgOO^JbG*KUrx4Y{Pm|?i4o~MyNqOn0 zP}E?h9TYx87GAtpW6KTQUv+9*xF_xj&+YXD-Vx!m39eknk7?{n$VoPzuge_VMKvtV zE?o%{y|v@BYgOD}7&nIhLv8zVwEo?ezs@=b@oJ^W^w|574A{+F1cB2XshtINHNZsg5=_EE4|~ z6|f9!B0{n9t@ATXH+xGRCp#jP#2HU!jeL-ZPSRxT0)^W9If~wfWLrjcIh+O7i%z&x zydM00A^W4@XWbY3pb?_E5r!H*dAXpE&v2*6DqBf^;^*aDm$cV|B1*H%dpmiJo_);u zbB=B&|8V0VHeh&TaAOSkna*o8@YH@HVBZ~bPvu-gW*2(5L-#T}F1&+7c3zWsql_En z@g>&rx`oE^l2f3$h!kG@+J1P&{^MFEUW)Vo*m$UD(>7&0)1%OwUG|ighCp@l0N4+&%}h;bcTA97$q@C~VY=g9!uWTR`J#_vj6kVx z)_ah)kdrT{&XZE#`aQo6g9T)-68f9Qwzl}Z+iE^njn z$T%~8Xnb>?S{C$!%2K2?e#5?yqdzS2y}^gGtengSPX~F3lXq*D$^8t0Sw?WL(-g0~ z`_Lp6IYs@LLBq=KY!wiE2LNs-fERKW(sjS!Rlg1XUDcJzzIjOAyr^e~ueRLVDM}@L zxTQ_CAvB85!lHPum9c(4E=2p-S!?js%1Yb$`&#@Uh1>elq_Etf-Nz>R3qdshw|6fZP z(t)^|m?z!}z|Cmu^{9Fw1L6t13>IO3zrU}+ms<){M zG>r5OrfgfNy<_M}##O=;GMGU#d~^BcS48d9ugPUTP9n~M;y7>G<+Xw=^ko*hfqKnw z)VN|>T~y~nzLPY5{amt>e$1E~OZg)YS@eikUvrbtNpoKNA$fD*=jui6ymzG8%o5b1 zEyFZP4D*V+R_^+;+s|&N6DMi+*uK^N=3HZ>4u{_)@U2dmOP&;eu@M(?LG4vu{!z~V zumG;qjO{CT9Co>q-ap9%Hkb!|RXLvgh9EDh?7uD~|K5UHbyB+WAwPy1SCk<+b8g!_ zKzD#rdzf-+f1@;hjk?{z;?`X+Vmr^Z6Sr{HzB%y08X8P%`2Y>3GwaQ$Gpudo2trO2 zb>4wu4DlaV1RV6M1Mlt;kF1o}Zhf487)lwmc()I~mVA;vy zl!WiO%x0A%6hBvKhhmo}uZhtgY#^&wV8?|GQ?LFJ^{Oy@${HQ;mH@1X>$yPUwlE3u zbQTDLPDXIEn@@2s*L-TVODX#wj{u>>RZ|RpFvTRC68{t#xgQHA75i)*7APjW~jz zMsvKTXCO~>0pP%+*MJ%u@drIvsd|7$gRgU=B!}UEc3i7oWdBZ{vJRm9EknSc|NBf~ zr$w9G#0}l)Mx%&{S~x|i+$p+RY8DK zWTnkO&5Gp;&(}3`lT*31;uxQD;v@Qhe)BAyuf7Y~gOp3GfkkzdBKzE8EM^sa#7k4D zc35X-oa7b61}}v?JZx9E^;-Ak$tBXH`gWdmUQbk!g?rZ49u77y-vv3||4Dk<>gaH_ z6HEC1W+_qYBn@4zJKT3+O>r@8XidNK&+!n$smf~0yRMvY!1C&KT+P*y3Dn{Qp(tCo zP0asGM(bd~^0g^;yz2cIcjlkmevL>_!wklr_njxI+$Hbq9cVz*K1D zYtQgpS=~`hH@ami5|TriK_r>zdg_JKZQNV)fPhu9F6j2xXLrqz`)x0;aKeXEB1Wfu zHJ!dch=+mcM6Q&mHD56=sVQ=ZD^lp5;XWD+hm~E8CHI4u$5fk9yPqctR713JIL@V{ zao;7UQ;Lc_2^YbeFEa|d+k^d)ml#IK+_%1K0h=gv=f33coQr|w%9&LsBlHSwSGoK( z7#N=PPIHOCUvGNX5=6S=+ZX&=A!6X_ji(IB*dowV-rQyZ@aA2 z;_wq{5+VYeHz`!KoY_@I)Az^Ui!kQTxXdIn)9H^UWvOzdiGrBv z@{B;!gGV6%??1L$WzRWJN%`@~>C7z&gqIt>PNBXW1PdL%c5AV+Rv8`{S_AZOx?0|A zi5TCt(xhv&6+cp96f&!Wzg|47k~%e%9=PF}kf&|dk9d82jvcAvYhIIeN_*FOjkuDX zP0ut9=jBT&V|Oa|cV?^BWtvWXwo07lY7Kb1pfT5^WEiP)@X@=!Y3a>L(G5;HzRmah zC}`utM(dh2G9;QD^W2ar|xuWaQ zexP~}VE=~MKjT^Vs_9l@jPrSW#c?e&2(VXW{j>$Ui|Me@dAzdrpY1tw0z4e4L)lF3 zG9(TcQ6TQYw-lc3K`ev#BXR{fEwd5MqQ>mH-Vv`%Ep?Zl-pJj1gvh+MCCl7YiVO@~w=B~sum1tnJs9ZJ>@_(2 zoyU+yV|dO*_`xmitT{({!iKZ%(LKeq;(Yu%i_J5TghyvoL$Y+!t)dWHo>OkkO1vQ4 z*xsdW;!i|hCO-Fw8HIe-)yQ8dW>t-tVn0HU&o#@MGWDat>k)Lytq>me-ix_zDS=lNg$0~E^l3z;%nky zdQEN3b+WuXuNtCAu4e1eWj8VC+yu4LRRm+1sDG7vMN%jbc4V7!IKJg^AJh*1fV+%^G`#n)+!yTp za+|hMy%fr0R9Ew9>H(7JfFSY)!s%Cv8SA{pe!_;_xvKp)Mj#e+{;jGi&WH=G*0D|4 zo74Dc7Mq?=s;Z-R6$B8uDFV3~c>7UtR(F}-co=ftQ-qwy|B3SwWS0E5(EJ*oE!smJ zDSX>up~6$FWv#?M_DGsrF^J&Oe1y!5hjMPQ4ZNS}00F(+fz}LrYyb~>LJQ}8TtQPv zvmWX)pm1zkjC=~_{m(~MywZ6+GJKg}CScDO2PoV^9{TX|AliC)uy9O&EKmc+e=#Hz zw_3E^K-no%#axtpIQEG0iFntT@t6BxV|(d)@Pr8`-2SH$Tr(0~=h+F<@uIpgz&gX4 z<8p(=_}$lIfm-7Wt^>{I(`4}WuwY>g?h{RT ztuP<;Rkt^3z)pPS+fX-b%NsUhsh8f;mRy7@$lJl0MNA0OKs1tx`$3#=ErZ1##xXq_ zn{Q?ltBLW0mDNz*GOx`%?}jMJN348hNlV4flc%Eho46mH@k|xztjy>$B21#??>iPT z5x_V74%_drS$lT7K5_JpP@RA@S2Zx-;@=YSGj<@WWhI~65njp-0DU_1E9VrOy%-nR z5nKaLYvbUu#yPE$2AG7%MYf*nb$n_QRTBpfcdj!mn~#(_9q9aG)%x%D{5nr6775e0 z*%3Iy2>h*6;ehSZ9icqUo$-p};6!?Gg-H%&%ur32Tdtr!hVkq}nxXlB9i{z~*WxjV z!7uI+P0^piKChbabmdP~sgiTOeH4l(=HnO3)cs4_HzR@_=p=5hV9&Q8dNu#gY2_^O zTp8hw(_I^#2FAf-<*?wOO9?9KpIH1@hl0)^7fC+&-xfx$X6p))N)ofN2~lTR-Tovr za8^ozLfAKXq4%ou|5!l#*aT=*cwCD2k-FumQ_yX}_lDgG#xBPnoGA8+>+IU+PRf4& zGdB6RWG`798;A2y+_$IzlGsJMP2@QGH#Y60&#^bc?R_d_`vXI1%uMow36$u#R0BO5_o`+}ZTDKdpyWqT%0MaF*7v^ zA140LrSlrbDt=M1&F=enu19pmNax5Ua!|}*%lc|yvWYoS_4Qg35aqr8pOFh3ab#)> zKjL>Tm1!N;Le3q=!m7)_#tlmcZJgys&erLUfQt#J7t4j=oe8);?v}FpwtdHA#TlBL zW|jDw%?i(qS*+*CU168xN>TIxVR$*0=Gb0T7&MMg@>^WAlKXO5e(CtSXUN(_%_Qd% z!i}NIGurvpnld$dnPfE@c_a;)=d_PV<#q;`g~>{IX-*%)-Qjw1ozQ-$KI%&>q6Uzn zIt<-E?ZTj2m5MVyC`^8SMgJ5F+1T#av9Hx~psMpFyw34IZ5GUnDdcK7r07zs)V(DhpNargLg2kD(LRfYPYdFmhO>9rRrVfTbOQGcFRWKwg{4{i-!j@2^(bdvCMID5jYj3KJh0vMqe(1KC^emb zE)i!eU%g8a0CLLSj`X;vkGxcq$7J}ijGVN2oq+wQGMu0HTFQ4O@5+B~1Z;VXTXp0~ z1~>sElDN zSSSDNo8a8eE206m^*pmiHh#B0b@!Hrho4&pgmiFgVNOVs`z|(_4~ww5)kf;aV&Aj; zDIt-ah_Nqy{B^se1+sHmMcQNFvYRL-CG@QgXIJ+8FJ42WMxzd^;NjO5EU}`vgUkz7 zrBirAS*T-nl*3s&LKqvmavvTT%WKYg$#|r=C3)kvMLYDw%&IZW*|Z@$ubW?kuYFsM zweik7RuzBW@{YB6?!FPl1Ff{aP1!!(VUS@&ZRryZf-zaz2rSc?FT`C?k8?S8(UrJb@XzAz0g6H){IabI$H*?mg2Z_cOn zP#Ko?O0?+!hJe!P5?^TaEwf3E_r#uaZ}Y97!7zCAUFtanIqmst|CdLKV5eaC8C(62 z-0ZMb2-1n?!AOj#Q!lV2(D#8Y&Ea%avAVMVo5&h-+R|T!@l%J!)z1m4OIrsgC((Yo zH`<6)SZ6oO=W;hvwVn?VRjOY8@H{(VW{zDuC*0}5MXq88DTbX{1a@j!iRx#$#@Y?1 zHG9?~MXkl=F5HWoP&wKWt<3>@7EP~dCsF*#JSeKCy@K&2!6Yj zOhrDRR_TKAz4_QNN`0h&m(9uTwbLnemh7ZLm!{^BAeS|3OZnlW97f=ou2xz@F*l4IUxd&BX20?fj zVD}4clXzmbhK2mD}Z2j5a!dbt|9mo zhVa&OV2F77MD{I8RgGLzj3SL~2XIBO@#71^ha1L|xc=&%#+Y5)RI400}=i6C8n_1E*?`;Y)xAV%y zBtDE88TknpRr4D%F2+~>uX%?~jnBDn&IP}Cif|z29I6UvnfpI|!-1+W1dt~ad~;t~ zyxr3Hc2xR~isr9klYb0!viix4Mt*!8$fTmCNx_z|Cl@MhDx}_NSsH2Z*Sj#;FA7OX0X-sM4OD09uf{P6K6@_K}(c4jJS*7 z)zns5;XcdV8fCOFFm?~!;|g!Zq7EP4#idPWx<^>9&LyCrmWjZ#fCDka0FTHx0{$Q4 z2zr`d>Ex~r+O87NTCskpG%`!@-<5&n2e%C(sx2|l&|BXWT%}3DkdG`n(L-<&K~5w9yXeur)aWsooJTST%;TRD;ab2ucqdy^+szZbJ(axB9^Px=H%SY|Mc`-2Z)Vv8Tx8yS)QsD2 zNjhOVH~R*nX_Ptj{YB$e$%KU!*lO}@=}!d}@mj=a$WVPsTZ5I2?0Sdc$v93CQKcj2 zh0c|yke26E=r&74AlJI!PHQNjwQ=vedQeeW1@miAC-OGI&(3ZyvI!x7O{lt zt-84XoK2_qH}>1pvaUJOESG8~#YVA?f!pq&fwj3MINmE|4@)C`Zj$J&-{+utb&8I7 zkAqQz2u4c$I^DVe@|oiLNo$4C^lO}re%DrOhJxwwFZ*dm_nHq&36=Fdc#ztKj(ZWW z-8_eS7C}2U9o(au6gxYtBXV2g3Q_E_Ylj8nW4lxs2kL9~COtP+j<3nYW|K;CcF4(RdSck$xf3nA}B9gB9fV=G(? z3A(#?I{xDm%_=n) zjgj;`v~YYUBbS0<4P%TygDPfsNGLqZ-eXm+Xa%{;vMyu!FgxWbeC;O^=2%81RqvMb z6k!wWFBVrl4Uo2UGRv>aAj!br?NNn9GQJuwi^-0vmX_2TyPF`0v?GG(OPQcFmWs zTC>SNc+}OJG(Om86ppZcz-Ek4DsY2ukV>WfB-=+9{u>>+v_F>hlNVNa_H{e&L!kmchKYSL;z|hjfMx7OnDeQtN;_@#2 zI$`ZmJ(+F`*J{sSCpqC!b3k_-9-o*da<-eKPC6lGYlEn{c&)xntowJaxDnfIdrpsW zDXNR)$yN<^BZsPtn-bJSAcgYi#$dtl9UVK*2l~(X@F3>8v1ksa?dTY~->R_CljcoB zWw|l+2H#>4NE0e@tp-fLJp3)$zs}%eTJt$=pF7JI6c%>jm}C6z-{~XUgjmT0<*)4;i#$s>nRwQmEVw*3Loxr9kFj$grrg$LelYj zVY7bOLrwEQmxFaitlOXjM zm(ZZ;N!QGsS7U3Yc*tN}Pb#DJNNuORh~|(q8W{}g$1kCn;@2&Tbmesg%L7TfA9Cud zDMT*QRN*A3`lz}juWM3rt5;rW7LQ}hU{+7H$3>)DA`46P2dy!4nVObMe-|^RU4S|` z`}9!Fsx3PsWba`F{bwR~DCTjH;L55ofvZx5>OQD(6@>;TE<;b!2IJgO_>T>LlQv*q zQ_y%=!DgGMLZpx0ms@3d12v`@t@7w0XHs(Wf%0=iZ%5E)`$j`f@9`i)Oxkd9Ps*t2 zl&J>cBYgos0}dW`(cX6C@(h_PD;0C~6e7B7oa6xkA9nKGz?0>E;>#MkYCvB2n*@vw znSDG%7Vp(-Rf@8;bO>|Rbh*jmmCJ%X$u}eJl-HBv;@gY0R`1vP~^I3gaL4YJ*lM*NL-#?cBu!ONWK)E4OjjSZm`bYvng+T z*6AtAo`8a!NO)>`DJKTme5-I>SSOKaSgU%)yWyASHH+0j?+MNjh0Vq2-Ob9^?*-{5 zhSdEN5ku{-)MR;w+|{mQ7nW}TIcYz=$_r^zux<3$Yz%e=pKs%F;r~ukWW?%^1yTWW*yy@q#6GrE{96%`yJfT@b|tNE97k5ma7&e`h(}W z6tC<<`^U-U@B1pHl@Q}jC2L6;PV7p7&!j9IhFqFS!u*BYNY<}oBnY(8k(>i;b?HKQ zp^9C=QXbB}VM2mS>6pfkKr`Dh#*06D(FJrr?XD^8a@9Xtc-Qcq|EP>$2}cRU zrWN=3A^0YSq?BuH=o)UPfO*AlWK>1dROO;OE_x!oWRMr>|FPH_yN0ZLjb7A)#D7rt zb!H1@1H8pr&<*LgM_HU~TPv~krTn|frP%jZKqB|rqNX*pOyW~o#FkAN@TM;Dm9Y)+n5lbj5aNsS(!+Dx^=02p%u~HRm;212xV;FX4{gO^hwU$8w#+; zJ5opQk*gr6&fdC>gbL@w=*utv?062bBynnItk+}We<-Pbx@<1j2HpL*tTcJ*2zUf_ zyjS;G)Z^vj2|GJ<<3`IfII3P;2{+Ysn6lvfB&mRwDSm6I?OV~6I>T_C<8=9(5>}7`)x?L*TGkTJ_Y$ks5qCH~nxf_4^*9*lA4CjuAkKPz zWEn}vOFNtEV>??mSD&j7*l(bCC!u(L7RlnT<14Mewi7kxIOfxL!VhVJ!_At}#fC}| zv2W7@ld$xcwsJfzpV+yx3INU=sJ|p*yl^E3m(zg1!+{DBx_*yp=aMsYLgDdKtnV{3 z5=hOc)NnC)5aa9b^=y`U)qVx=_i$(Bc7^h~8KSZY*?A{{VW;JE39%r#O8`TtR544^ zgk4OXZq?Eyv~6Yx9o4T+Xb8G&^0-a5N3{WuY(s#ZQvc&FUB=po%OGOY#J*FNCO99d z$XC(6dzO80R0j1&(R5eee+!Cuts98OlxOsVa))&k?Ld}e!w;xvPw+1#diL9d zHp;Z#OqtMI;5m{CZ}VC5c!ppKBd^ef23a6Qu%QLWsJeT%#%4Rr)jXY3HT)I^n2#Nh zyJLGF*?x=ln!~60ouc2&s66+{7hXWebmn`uF%<0o{OR&gzC)qoI>URAf+^HsNrGVy zMR&D))~py(wR%9yG*~dvTKV`SUzr?}^C;mvoHX8ro|Dxz5u%ryiwq)s%KM#8%OU(w zf@iirtzP8!01j*HL>BY0@?_bV9@mWM{CM{|0PHHIV`r|vpK9iI7#kH!4FujhnHGN3%2;O_3hJ-EBO zLy(QTyGw8h?(XgoT!TAobfb@R?z`{4ulb+pwWhnOx=N=IVhC(OhN7@oLN8@k^cR0x z7|vb-%p$yblR2v5#ANwQ2v2_@a~vsnj)Ls0G}1snr?Y-Tr(RUp#iR*?4<>PrBwpTX zZY8Z5xQD=~VuX3*KI|)jQo3be*xqgct&kbv!7hYCU7$v`i}wBCLasbxZwXv=2$#pf zs){EpJ}}4zK#HU5*lj&=D^N5K<`Lm^gbR{ptX67$*+m6t5`TK-H+NNPmx#vt`G*B& zHWmQk@uRe-1kaz8YQd+qS0+NeVU9iM`Of^rh!tb?B|8ZQ5Z&WD{NK zmO6-;|44rujrqc>_2WiOt>jxlLQvqi+nw31h)Z&&Jw>CY_nKsz4)F%DN#@VRzg?f_ zkNnDe`(?lVtU5{2=Trq)0CJYaBuR-p%OUg+Xjd2x5mJia8*(FfDvEj#7UhZb4vvmCsD3BLfDo?KjMTw8Dtt)A-Cbm6|tc{_#WIEv+OUnPdd`b^a092>e*TJd#BWqTM!T)Cy z&ce_x`wN6Zd=k{xB=aE)-j2}Hoe(KExPUr_L|9FmXm%pwBBX%gvBLx}O7d|Qtpye& zAy%}>tgtb+eG5=1U6?0=6P?LMcbp({1QS&lCeT?V4u&dl&onVl%v+(eWrZRa$z_5N za7aJdOqlhd-hKKoLfChyUY)dX`FVP>;{4$s&t)S2zD$)m?$Ojbj1H%5XnpN5!F*au z>ys2+(bTIRV`K7(Jtrnf!x4H+J5Y{$V*njX0t|?K9%#^{L|3$Q2aS|IcaOWoa_?mo z<0n4eWW3f5IyI)fk9q2kG~wjO8j>B4t*r=!=zkPS5juc(S2UdO3=pv^`K5PAA3A)* z%5gdbA37-cFvpDkNv9{h`T=I3kH1af@+VU6ici~g+sbYbYQ9W_KaLIlGFivrIZ{p( ztVN3H`dnn9c?h!c*27MV0+r!Ct~~l3-?R6iUmtE^28`Mx%xP~=Co+c=Vz#f36WB*G zHXmTa?9qzV9G5mrx!*9Vu?wM}cpJUtTc*}cDdX8=xYrPxF;13InfH~AN&w5wEm~si zt|3NTg zFbgnOBlvyVw|`)Ne<3#2JKLFlX)Cp;&Jm6ulIz63AZY0l*Y7|&H=m!f$&A@z<5`jH z7lQe24#L)2gy{#OoY>b5abDIeHRfD5OZ4sQ0lq%}r33rYi=acuNib5L6V4H?xar`H zrLd{I>)O+x{-TVS9g@np1Z=^S2lnx0;$(Z*!hLuzw^xY|a17l+h*9q6m3_>goV>}_ zCQhA-JoZaNeudGXQ%l*AaMy|LQ{e65XB)orS>Ca|KS_J9uzI#1HjVM>9OI+Bu?R`! zw&Htu&yTl#7Ua2YFZzHF9-Y$qFQ>~B3yiNApkvq%scG+(hjHzQQ zjK);(d-yX1lX(<%VcnLX{4nhbtRICBO6v`~;LSh2ukwdM<2OR1sZcYBFvAJQf1?Jo zQjqRvbdzqd3vto5C8&{Aee(BgQiX}r{}qpVm`7@d;N6xuNbPJ}h3U7`Hon7V%^Z^; zLpJ7s{JT%~XW9MO_@b$K1yX?EWu3}zWpM#FgRcl%?92Uhc~G;leuR%kQ8=~orl?g0 ze>2x|Ej8|#rd53-t{Q+cBu+ur3qCleZCt=``RdpoW{ zL_uKgggV=!lQCm4T$5uXA0XDYHwDX=R7y24!u6KP$}qoPmf3}78llVAr%8sC+L%id z=bNbaj?R;~51B*kM2h3!42KCSBc!$0*8(btv-bWhgj;VP&Yw+Mxune&cIqE>ho3`$ zYLmNB%gh#rnc!$}dX(jv8g$g9hGZ|g<3CM(FZv2UlqcNcA|~p5c7L-ITwkw+`fwC* z$K{IL0d&<}oe~WrxAn4mhFAC}AY~Dkd$w#k(AzQot!p(E}EurW640V*Dg(UQW zAvkQKprm8ba=1JX@AR8K~8akvRe#I|m4q?R9WE5;pM`kInhDQLoneWtNP*3R9lF+s27Nx+vG zTmddd?b3@x;Bll}kR5sdTU@-;! zy?PYD7WGxT+so?!#8M90v#tlL5H?8(lzdylPd5pG|1DOgKO~LqL1;V9A=Ii@{q%hA zyf-!BY}oAI5KDzdWLPoQ7@VD+7eh^WiGABBMg5t9>!%IbDPnmPkSuBeZZE3fDjO5po5Es({CaqxUlN7^=YCLK{lZ> z;zUgp#Z62xbDB1&btS9ajrE;(!wEE)!|1(}GRMUG?pU9Pz_x^!i((}jHo}Xsv+Cdm z<(kF%%xvcXB$u-jx7W+k&P`k$>gY8x68{Y`jH73>o$z(WXS(nJUXfpgvFEQ1u&D;1 z(f#DGJYFgE=*08Roj@h|8?F=AlcgBDoVJ$e^L;lBD~plQ!;>=N;4e z-o%vf#%)vC?1OV|8xkiY80r((%N}&1qYI_db>;~IYE&mbzD^xD5B!!2W6*7RJNG{a zS9W7Y<|nKwz$ehWU!oAH_Ayx}WGyWL`^dQ0_*1DEyaqC>NverOSKfNlU;Z@Xo^@oL zW}?X&z=7_d+)(S|eZ4b5jb1Z46>}uiVv`Vjl>{K$6jNRNo}q@DC4H3+H_D9XT!rqt zmWt0baH=HzDtW-M@UpZ0{bL7}hyrsx0gsRJ8F318XM8A}@_b%7tL1{3qeuf5rl4p$ z;M!8teXO&vaSA8}y_6;HpLX`oMeR>sOOCd3d?nukluJtElqt8-3Hat@JHv=ar;&f` z)gdRxv9B-Wk=^-b$CR-A21pxiFt{W38d(SE<-W+q1s+m86#Owi7{!Q9PMYrQc@+kD+P9@$Qi zJUcwyNuf1dJ3~w5Ec;b8$*Dzb$zj8r&Ohsn0q#Kbpsj``mtn=6`>jU`@TihpgW1E$_amz(s zce(jgeIr8mG$-?|qwZ^Y&^}cU{dU)>`AQZx&+{=WHq&%sY3_$%+p<7x%BB>Xf*`I! z)8-s3qP31VoBK0ww^H}jHC1lSD~W0`p;ia4I4@mA{dppusWczzn$ub=1+ zbkS8v%zM8prQIlg(_8ED+eI{>KFW59UE42)fWEcnLj(`0`)TWu+c6(Z(?*f*u$Mtd z;aQIw$H81^vOwtu2lZmmoN=%D>(g4`Cq&YPkpOv(G-=44>K4TBr@GfKFu4@;*Q!<( z5-()5f9)60R#qYO6^?JVbI5De(3K=`NMzTEO-Qt>>w!hPqIhyw-1;ue+GgowbTJGf zLA#VyC?vh=%`f9gc~72IDBIr*UJe0cM?J0?xg?W2v0l4TWY$sFM~1d;9n1Qww}>>Tn$Cvufn>E`jbC%%M1=qW;}U=6_W+;tC8;+ zX^Lbv@oyMK;>Y`y*$&G=ln`bN^fYOmNP+UlIrDQ)V!<#TRTy*UxshZ~3EziCb5GJf zqg+&ww%Zh-Gbm>{a913TCG!KckOObVGP>$DH!0Fy?>`b+y5F2vdkpq>OWS7ZFWyqFkPKCki!D&Pp;>O13ZTf_%8aX^p zw|VjGjiB_wsF5Z^T~>KHl%0!wm@c zZ&E$p2^zg0$7Btj*lkJwy8rgt{8{MBXJ&|&G1M=t2p^_SKXsMbGBiSNn@mp+W2Mm4>y_CQQ=R{wNjw!J#kN&V!miz69 za~i9j#m({G!s{i4gA(}y8bHgUXGC>l=!K!z8o9aQ#Uvno(&i?8?3qyp;sD-_K^Gcb zvih&yS7}`DMk)$C&v2l;tJhke`*8}qEX=kG_XMu4lPgto;Z4;`e--Gt;E&1qWQ6+B zt5Y)LPT)woYMIoJc%0LNFV7v4b#RkZAZ5=?&&p8xiIEx+WF)MgxC}+)Z5H4X7*`#< zgUim2%KF4+EJPFe(qM;*ANztk!W_shec4b`Y>r&SEJ&c8iUS>AdcOz!lo`_E0IoZW ziEr$)57nVfGUinb2U0?OwGRoi%@`O-D|lL|vY+mVuLtXlX|VjY-vCb~8e>7k&IWLb zeJX`9Lf0*6_;5jQ94TGu~E?D;CGt^{9Gs0{alkjDsN`QisWA2 z=kTRUJa3Xfy9Uao!FWv1of{Z&og&)l!@c9_%VR@?7PN`KJ#_8iV)M}o%w-(9d7i#@ zECGwBzi0~wBGM{xSiix{`Y8McTF!epcrii|JyBWvWTN02LZ-`6zSX)hjZhcKW~ z$io^@&HvH#y;S)-QfC*^DG$w+H@DD~!gxRa-=v9%vPWVA~Zd->2FO%yEA3 zy`ls>SoDz9ya#zqRO)|#w;j=7IB(v|j!!TO=XgYSQa`QX`?-$7=qwp6V6R8JiEw)! zy{egaE^I%&PREn^ehF+ZsRv$+hmQ$_)T2uhLp5ZxeP@&{1>VTzakD144zJj}qUrB< zfxNXiP$jVz?K;6As4u+H`aUlfhqki^wh^u;Cv(INNm8->Gx>RMI;N8pU*t#_$Z<&$^U7T9l!pNrf_w zml~K$6iPJWcc&w1!M=*fFQ=URP(A<#U-+!Mm@UK~*5?f$77df)h0l6~pUJaSkrx&6 z-D#t-b#x)r@pe4Q#7KlRL-6-2rClXigVLeVD7)9(a>FaNLv`$*x$4G0?D_+z^iWL{ z+2qQ(P>*9t>RKd(aOlt0q>Jq;z~n{&R2#6{NFjN`TBT5dt-`ofI;^UqrNgryHF9Iz zD6X2iy2+>D)(`3j#FB~=CvjaZ*?wzX`%7G%^4%U6F|~@Ja@QvMuyxljhb=x*4npkMe zc5eG^xuhRMG;fk!Ix3#0b=uzApqkx*#1GKJJA6g`mFfJ)YUgQHcX=4}HLoSzh}G-X zf2T%AsAhbK?1{M$FA~pzB8dGInpc}ZE&8A*|Y_4#0vb??&4N6*tgQw{S}BBIXa2(6ar5t zgcwM;7h5n7?9B9c5qhbTHmJ}%T5*k`^jsiu7bu22D3zyjS5in!(JB46fCo-u*;}se z2+*pps9A~6Z>l$`);G&X>|oZaQOFa~B&g>y&E#1vRSI&5wee*3;p$Zx!p^<+IjUXM z49NAC*nQf4%-E%}Ld9$W2iMTpR3@2;G?NvfKW&?&&;j0#Kv! zT%kMnOH+*}ygZE0T(&}op{MSbnZ)*QIA|&GgP^<)D*a#)_)OBpQjLJ9EK@}hE6$cn0`@GvB@w?-9ik9O*(BW`^NaSGQ7uD_?|x7nM>TSb(j>?jG}It{IL4%-Pt z&lZ~jB<&hze^DvQ+9qDFFW5@hb2T^Uv*dI){Z2?k^EXYngJznL^7Hs>-uv-br+49N z*vW5CEzl@8wW|lHGPIKfWu!RBKm7gECRgYc&VR?`Abt9X_$;+uKqGI+S5$963bnEa zgM|s1mWjHeBqLky_De9r46vlzrD^le7SHANko>v#283zZ0Jpas% z|KNOQabO?_Ugp5@ii7!t}0xP^ejwovg zn83`5T{=s1Qo7{EVt23ixvB3Q z8ZEVO44X7<+kR`zU198pPV@MLg~~Qg2w^o-TVBHU-5}0#W0dBPvOQ*tN>blnUWIi@ z?h-x-pfnVw@h9T!{LQja1}>6R!24p)=*!2Yiu1w@sr3PwpVaN(h=9J|qlcf0eH&4r zA)I5{b)uYNUndD>AVT_GjH2*an!AfvI$5I+9rt9d)P;LUt}hhh!PGrG@e*UJ@%$=S z4upsUAnl^`f`0=gpZb1Za`%YO z;`AV+o5`Z&IxRMPYmI~MK>9HxT^bVcR<9(8(s`1uN*aFvk;?PAO$F;n@`}?MZ`dBR zp8d2pvb@$BtY-2)&Mlo1OH~ep^gFUu>wdd;Fsi}Fg?DB1dAunI1H-N8-P2s%?CAK) z#unv#ueO~SwsrO0a{sG9I@qBelW^<$3WJ8OUxUvzWL|>HH0}7%j2~-iB;Ah=XxIiL zn%olN%-Qeu$=D()#|<(p=iVG`lGQe-03K%hxUJiB`+(09b0GaA1prY!SEqK3PWL5>g-PA8Sa^x+F zzoqaIpMYy2BOInSepj8)fM_cy)F#)!JNlK19I3!g6#0qDrS_Q8G=Z;2di;|!)U^g% zW7iXn+G?2(cF*U|LE+~+{z37DHqj6#Zw=0V;2R8%m$?-}(=l*)ys1)rLNH6H20gUy zadacd-7ud+YEAWMIa~4%Sz`#A+7u%;GtMH9j)CY9<1YHw zFoZiWD!vo9Xy;q-7=9hafqK8(2z@x!fBAr;yz(cQdZ*npH$iDo=$aho<+H2VBmCD3 zU}xc{{eV!p0pXh03{po6-T}4~>Z#oJdTFWcFOMI{n`OX;16P1M^!(6!!I&@=QWLpA!Brz1Aw_t1FF zZ2M?dcQ?~~UVyg;=_Y+@8_whvEabNn4c0+icMIX+PpNNw&d)dFPvU)kh){Qh-F;G3 zcXk_wtXkQ3ebo6q~H_B=N?z9Wa75!Ql$)k8xKC7?_F zJv|~J^=CL4)z7M4v*5f{Lb<2J`KC3kiZZoU9zjbzr&B!SUe~8UuV1? zR)KUgK9PjJ{Sm;zyh)hb7Bo_o(ok4OMY-rbfwGA&?(wIbUi;DwRG>GNa=G`~oL>*@ z$iKK{w!P8juEbBW!X#IC;l967#92N-e}^vwo0OivBOeBP=dQk&&Pm&g1-gJc@KP66 z9`C@R5`2R_M%WzwO0d2H{(=&HA{hlOLDJBDkVFyilLE#fp^lWlQRH!iZL- zrG-!O+!lkZDcm&P&D}Sh!Mzj-qNhE{#_(8aoj>Pnhj%E1EFoQNntyzOdsj7Ut%Sw| zKg98?$Oo3>iy=aPIfqReDjWUmq{m>sQ7V98R`)nFtjNm0f&9330^Q-%B;fQs)d7PI zeAv}9oK$c8;Jl6yN3Fm!v3BJ10|a?o8W&Nw8N*P&_E_QK#cqNBfe^n(tbG~s_Dz%8 zS9+VaZV{-_EpFn<=J+OYx2|b%g^i#f*!>&0_sFr}$h*{R+$=Bg%Sricpr{JR@4T6F zSfc!G>3^-NI|Rf&Y&1=8*QI9AQsG)5bSa-egCzFRVO>fWHIvuL!ak0JPAZA6cY`h{ z<|Emr<|X3wsd>N|XF7=+snWeP*QZEs@;tHf<4p3WpTF}2T6KAfvT}3IAdls&lTG z`ufD9^G8t4L}=pjC!LW(4^#uPiHdhoH#bd<&rsgz(Z>O&*H*(7BM$$CA9IZdUE}N# z%XYM-N{qKhL01c*#iqN3F-i?Vl%z1)V_+y2AzN zy6H*UCdxDBJkbr_+%28ZuoXrxHP+Bcxq;CG2r%at!0Ucf6p5Gz``ECbEp7CGZS3@0 zw3}C+g43q?Hu9%capbX%>(~PMEf$+kQcWp}waGwv`%)vhhK}AO=Y%y8~6bp5)J;HH_WbtVoLd zm|1iM+G>AySV4mUT7@Xh9UhyIGc%j83OyY>cmiB0zm&!**`+d~9 zKavRc$3NQjt6z41J1^FGe2PQ4<*VBcC7oYyM#lIj3)BML2Ze{+Bp?9t>SIrL({MKX zV!4$=9>0}Kf-Hh~jb-9_5kmXPk8t!3ajnjY+}V%}WOu2P%@CB1cpUS|YCZM-iry$L+e4Ts}y8ziu-h0;rWuStaHub$4XCbp8q{qCWANPF(~ zB`1_;A3n2dhP~*}u2Zv^V>A*AJ#hB*OCx;FKh3<4mWZjiHr-)g$yP#)6Nu*}!I_Tp z^QR`Ohx%J>&!CeC6Rz+PJeMoNY8oZ&RY9xHa!NL3=Fn}MrzRlN1s%c7>Y&4IhBwl8 zet$l9%%{V-v3FHF`iZ;TNW%6r2~ZzklV0)XqEozxD%nXOjidk(+w|C#36yTH4Ft1a zgxhsUnEu6m_%8mdpr5qdPCucWTCgwQGbu{04V`&@Y?#&@C49~yaWKu~wx7?7R-Qei zogD1Kor}|JcB}ewcK#rp(w0D{fmF+Gl>sl}_Bk*nc57Klc7B{B_Za<=0Kuo>BE{Dn z{Z96E_3`{}EZ!*4EcIj-OS@{bfBLdpz}E4-HE`kyjULCyl%%u_yB7|#PM{1z>@^UB z?SvL0RXyj-mTFiEgo7|QiVido_+2)#E>0@4q-#K4d`pQJ;WU z6s1Q_0N@jNKD}l2Trlnm)idEX^t4r;<~hXWdFlHHgCz4iumts|)g&h=t}-e50ht0H z2dky_^CK|_Sj32mkWY>HlZHv!<+7MaIlP{OYp#T-CQ22KDnKHaHgJP>n2$n;%QlOQ z<$!s?HHn=Rd5pT>kp88N1DGW4{QqP|B%%5}VS^4Va!Gq$G7XZ*AC2C*ye-maL}syc zl@AXOO_!^!+OB*73b}55InM7jy3`igUhNxhM~Y@XiD88tbv9qjpO|ZCQ&ab=7GmR< zdtPaPCo_KH?ur}qA-=Z>{x}_iWDTHT8CMlH!&)G(x|{D z<5t%_id|x3XoYkX9^R(9)BaPLj}l?5fe*NwK!-Sv4HfqU{~0ab)hUEoTBhD+S0h_# z_q5AINnM{3-KS(1v&4ldas2Xqc}Fep?!%s>8cx=8QzTZmE5TmVuMfV_2AJL73g{>e zpmd*z5A4dXvRJ+{GZi;VpJ;fJF94q`|GtJ(>JrP(TIH)q5wuvxu%O$*yNC z=X<%9APH8=Z!>H+-BYEM6jWk1*d;zFyUx3Z0DKf#+Z)L{BHUAWgNx6(1c=y4W+Ti? z0tF};(bz#ly-k{fsW}Lf0i+F-P}SPmxvf#PGSD~Qr}^)pKrvBQ3@$7B-}x-VP#H!i zyM0+<#omYJlL>R*7$*_A4(>W~s5?f^YG=GL6*@Ho9nIU2vG1oy~0D+@l%Uw zxD~wc3cFlcxHgN2_{E)mVf!2kZz&3q8U&aSoCM$1~@{MN;y;sN#N(Ca!x28sAkFn@{@ zC%JG`bqvpsX?TETCI_FSMa;&{31@AJ zRN?(l?ojjENU$Te0Ji3haQOf;4X{M$Jr8&jw@o_JeeDX`)aU^_`XwjryBp7_3l%~X z=_HzqktF4EtibbVgc=W<>uc~wzh-(HN}MQjpt5ffBH4npiil3k5vD0#^fsRVMI-Oq zO!u)Bt(-lhMgk1hZl9e}8TvFvIbkZCr<-#rg0O_s;TqX&ZJQ*e{_+SmXPYwcS%hmc zEFw!sGuA>smuGIRvwdzEo933;l4rH^Vs(CVb!L2rvC~VbCxP%LM4M#M>ckb5Ok!$Oo2h?zc zQF{5Urr&wckRJ;sD!@V;KhnIXE-=;kMYK)Zyd!*ELE>nPkGNE&WnF{67x=8>@TK!64Y@gmddBAq#u!lZ_S{SvV-B z%0uayu=c~Bqv>~;jaD4v$M;!1_kR(-t`dbC4C^AlR>+{Od8K04UdKHprUK4lJC(-C z`}iu0mm)I&IJW{ImUa+KEZ4OL3-sy}nf_AR%(&Nmxq*KB?rVG6c821YmLNsK62Gp$ z?x`!lx_TYKUB)DXmg+?LP>TyhE4_H36br-tYSj$kJch9uKVB*a?}_cvkGCCU=XdCl zu`_+&IzlI?Z!of9y$7(XqIGcMS>{-8{Iil?{hH7SthK*&`oULJ_Au(BBQ)?339+cR zuZebEBX#!w95Nm1#dI)ViLU-k*U8!d^Tm0~<1C75>_Zg6S2R9pI_|sGQs4WM++m&| zqO%tQQnX0)w4YGW+wYN?*GwSCbIPOA>d3#}Su^TH^rmI-b-cIM zaTmWcr)vPZQ%i0hoQr_@l}%&x;W9HZkMEWn^hDK7x!v&$T482qRI7^SSQK_eyY4UX z$^tV~n_=ng19fX`eAA z;q~E5F+X;#T95rj`u;4ULU+S+VLaC~)%qcW^&R8i8`nULVsV?21XvSuReo|TkaR>~--$9G%7H4o9_{3Ev1U(jR^=C_Ep+7RN4VIT}o0CAuM z>=`23mt`~Eiu;0AxLNF&YTT1;TC8q{!HwGlJ=MlWsgsH9QJq38 zfIOV7nHiZgY$i5>@MF0jxK@+lkJ6!Uzzg-Q+IQa*0?q*Gu^QF#YunW#&^=rNYu<(#u{80?5l zZsp`8+*eY3)K;?ZEe`kIrO(lPjn0FdR#t)6Npq0*jd7HAu|Uv_?j)GDjy7%$(SBps z#BZM}#UU%$Zi`$ZXyoiQ@_+Zc{v#|6Cl`KK_7cAq-{%?)O3W*eFa}4mMCT(;K@H&4 z^g`nFu)0--?hROmLtta@qvG4vK06(W+8TaA(S-qJK9zD@B(D9&n}L?1EgtsN++N0*yWwiu;W%uNlQwVG>r-Kn`6mj=J(a z9#Bgd0h$FY-3BwPEkCvxcl2hK<{=BP4~Q*ts2{mswrWWdedLXDML0Z_-O)y&3MM0^ z3JiJOarY$$L_)f!@N%@CSt0IJg%?v>?}cGXN+(j+*?`zclRGKD19nTfoxJ%sxaH~d zulm4*6nKN=(1db35t!_a$Y(n2gFZjJ<=26BM^6Rwe6`nE*iA_VL2J~qU&BnzhzSZF z17!Kguvxe(zltd_*S2Dg1sVkBI3B6(bqEpM2bky~t~U{{^14E-#*4sHyapd6%tTq* zGYq>K*7pOD>^VCNNpLUbQUYyC0IGLrsKGZno2KdvCQS6#&4;69fh z;z;HdH9!+%lKHz~d>uzoY>sIRjEBbjZw7<-ky}d^oom#yzxEVx!GgsQK6^H+%6*wj z{wE$a&jGJ99)3jb&f@k$o?)FQire9OaY8|$xglo7!c`8eYsT>jgFB?Qmy8h?2pm7@ ztmM@equ5}^baOa9^0~S{Gavs5lOH5|p*w9&^f-t7%Yw+?9cTXpq!nHw+MBwNCN z5t?bI8JD=N$zc5$ncWMz@&Pg43Xzj6aS+4YmH}y2Vn}PuD==eutKdt`-|6jiWMSBd zx^hCu0Ys%Nfqw<^;VFqky{S-qzwc=ZZhmRBo@WSs;rlQ9^UoK}zu%++$gUS(#LA#W zWBi@_riq0ghh_O0SjytJG~Y-kdUwR5g>Uy6wHxR~kE_;Eie0vh-DS5$=VzHhR6n{; zGFP(8<3cr0(8{+&Ar=mu?!lS;?u18#cULe2+$39udD^9Q=gDNu-JvzxUl`qVggo9{#|$r^BONO_C;=; zeL&STqbl~Nh0rrh=z$3+6X1M7&4tmeN6dE}NrzqA@LRvJsj^xmf%=+U;uTJrbUdyZ zL84=q=%}OLn_*^+0aI~-LcB_1p*-|BmA~9G#IwqZwV9w9H?_nJp6YVx=#2ZjDULRe zWou$uEEPu1r@4X$bCx`@@K-kOvVc9c&qQ5j9p z;Aq^%we)op!}y=ToyQDvDWVWLRyl%#=9mrUy$I^9j#IgSear*?LDg|v%e}@>PtNHk z)B}V;!ra+z!#Y_pTehu2pS9Ne@IC+I+|zr>+r&CVgwNJ#_$iu+c*L=qta8}C9-JC( zMO{IGOj~O~SHf7F{8m~Kx<*=n4=P!!K;~ifdTifj>!iuR6W=Ln+jU^1A_dMx%|Gh0 z#24zX-j4pK%%VIBX;dsD@>>jx)mdNcoFJ1~kgW%20)lUWbTqJBZIUi(c?hM#Tq1T7 zehAbFg!SE`b}ij@e%hid9!oN{UB;WW%t_RSmm_X*1gFeoJ!ay&+Dc2V3kHS#@ZzfH z!yfNY;kNq;W3~o?IMlgVB0*pcDe9dD%ePYpH??_^VPaav9U#tN9YZd|s(xer*oY?~ zxm%-u@9;_$eC8^01|8!H=XqxMGYiXBKFDxA=YQZDHv|jBIM=*MN@j>tZq`qSOq(p3 zV4QAjxUyBY^G@=Cs?3d@Wh%q8f=@yOvQC!yw||>b3msYD)!w(_M zjgBe!P5BgeDP7d(9c>9LPvTb7m4!t7y*w@Ico!&8CV5WkzmLx9CL243h+4F8o*w$z zX`P9!Y=IBE6X51ngwoNXE#6G-fcS>LB|#JO@U!1jLNMcuNSMacALWxAI4?k zgABBth5&ujXI`0Rpp(E#p!th3C`85>IdfwH-gsUG8xrR8H%l=4ck?v7%I7kx(Z~C7 z`~sztk(^88Q4YP9<=d@bOMlBxCV3lHAlDd<;r_|FjNul7T0&5P)horCu{?r$g9E3~ z7f#ZmK$Hgscj1>NoT+bh6XtkfH-gPQMWt(O0)R~P@p^{4=uU_>1^ zgSZ^0FWt8f1$l7OAbJCzTt3}MU&&EN;!T+3UFX%>m1LK4AnelA?9?&HP6&o_4be=ZZmcbnA4FSLYZ&%j zjjb17NgMUeo9b*5-#FV0jpzIm0y7rG-9^AElcI`BoP>_m*1PFmzkl)Kq>(yc3U~XB zC2Gas%JYheTjR+JRXh7U<-NU|)t-BxL&xe~8NDPRWUx@+U5Yo%HOn~sz06-)1U$OV zk8a+HI`-)Czk<+DCGssAXCdhj^+leZJPPuw40cUl{%Grc@dA2?q^L|Vk zr5U-G5c|a9nPZgTMj?v7Fi>XgnpA%ZczEAb@@%KAddKa5J$^N<6$Cy0ejG0|cu^(t ztnx*t8)L~SeJn%B-s)I98^!?qzFI61q*a&Wy2y)flW`sYsGc&vL{x*ghIX)G~{G&V4 zCiGWx_dIs$TYd|4GqEu}#x}bmO1oWkRz3LiW; zMrr~Hg%HM!BE794Kg0(G8`Wq+5^7S{y^4__Eu8v5Up;jpTsc#d&Ytt%mufRI2cQa? zIdMn>Qjx#-?cOEg2~D+9W-JlyMqpVg)}X>m?jN05|Zc-em+(h0-?snh2T%6}x{x+*)} z31pjMeqU)b`sp!!(iR>1qd&VpkH@Si>03N1SL^MfNy*O1uvg{lZ-`fWJ#?vcrf}Ea zRrx6a&B~O?NL-?8-Ot!A->Wxquyz@S<|wfc zG9C#-@vsrIq^{*rW2O+oHtEh3zC?&3KyqBfk%<3~zO@t3>&p?!d7u)P<#gyRswN7% zT}o<_(rXeabQUUvGIk=6j}p#Qux^4sD?a8BC)uxLpk|QGlyu*278ClcHEI!hi*UKN z12J&-uxjgyasrwVK;av*yBcdI^;wLd-zJ z$xw|rdSdG$mmpjI!FHDC)_yBi8Vshwj4e1HRz3l5j{ZZNIULy5KoSvZRFH-fTt~X#bIAJVFin4!9WykJd0BV!ywrtk`?<3uVd38yk zJ1kldf|4I8S)1vkh!2Yw<<8!{q!jh`m}g{Cv6-@)=B;wnj{JCL9xqPcmd>{{iP!YR zs>#f=(CSxXpExZyap#4-3A*Fp!d8@hB?-Uhv0JU8LZw*Y9$QW8ql2x^Pb(dGaFgo9 z_xWMhhpG|7w|Wa~a!hW^E(T&eUgFpM+Jp$XW?wTQ8Fle>Ec2Q5i+1hb<^vAJt)>$s z?jsx8X?YrPO9WzvbGW0a4*K|y{~Cb&`%VKGV!@zA$M|QSMwv%sJ;6guMalP2l`QTj z;KbMSYg_jv&}&5Gbuvx-nVZ6yub(gne)O-#2-!_kUj#sCh0BeUvFK=ccVb$l#(VU! zF7SL8g|wu=T5M1wMw+o}_TXT=jRxhOH%$2^CL8+sb6sszcycPSvAWFc zFf)$6LIZcrw`G&UVXo|fK&e0ThrAaAA9jR|(Fc#fKhKgi`SxIXtr3@d^nXFV9XMMI zsI~65ss)+uw{c}C;dBz{PUw}K5KEPoEEVC92qPvx&TrGNe=a8X6)k^(Nx*#((GbbN zfs@K?yq|zb=z*7d3g@NfGx59oKXkoQc%|*OEgB~k+qP}1V!Ps0Y}>YN+jcU7ifvVF z+c=qPuQlg?Hg3no_?}lCt+(E~X0nJ6qS_hoq*Nk$!oFD+r*x2x{(B*eFKCKF-%Q=L z(G6S1=gGZ!uyeOzN(PY)Wpsq(X@=PysDw)uV#^DvH8Qxxay#kb0fdYU0g>ywm7El7 z>ugZ8u~~U5t|w#;2~jOx$8rSA#^0H%9lO=1XuR6^`{*nbgTn8tYW~r}<_bOd=bjZ? zHCrn>`XX_N7X9~7q!L!>q>X5Se(so>v-U-{tNSlG(AxS`7`GSu?U8YTSH*qEz(5(^ z8Ah*Y?cUhZ|9jnEb6?Z^g%1)Bq>PyKYq@J>ESi74J08V9=;a^IKKFG_;1LG@y&PJ%@X zl=T;u=+wu*GbpIN3k)c(e#07sT!Qy(u?6?F#LLkI0@9_?xvNaHC8a?Ccl;K(5xb;uOof*uCyZ(ko8E$Fxm19E2a>hbtI-EvMawl7jKb@4hYS(&t725y< zozdCW@psDKjwo>#sDR4sG<5juqivGJDZ}nubhj580^qR8tZsa*E@y4f%uXv-@jlTR zVqd*rlq>m)m^L<~6eeO|YuPqhq{Wq$Gxr}WEd~yJ8~D6=1}wi#!Sj3(q(@4#Dv{)A zR^a!gj|$vuc5`;462548gUBBJTl=*oC?+N2#WiVC=2kn4Y2ap}VYIvse?PoY8)n>m z&ytc*?ldOvlD-D@mh7rp7;^(7Onuu7o%k{L(st&S#Y;@@3pS^TWI|zFz{e=fuLicupFC-Rr7i7vP*cr$!cCb4~JeV z!{@qc!wj#m&&fNCv4=$}?Zlp4Pn-Bjr(_g}+!{mTQ!@?zEBsOD5Xk5|Ce8JQm^?h5 z)E9I_9clYLf@06X{MB0I6g+mqZF%u{9l4B^jEu}!slbcRi*M>*Lg|Nsr~!D};sB=j zwRJm@=-`jAhwB=sHoxV(o$<2l*g{60{0~h63$cqg|CZSe2uJb&>3s!r56_lZ)386i zqkgfek5D;}jfA=jY}j6bJIYp{%LVgxl4`&6A~G1A3#DPgLFzH$UF=5)3qLMrfML7eT+K zl$JM32|f(K>+&BxiaJOahYA+zD&Afge3j{WznGpSCV+_GweoYc*;FQ!CT`kBowYp| zI*!uTzdFRPpnhGvHLf0D7y>(%Ok*(S=-o7r8%r*QhF|HTcvHG#bap%9 z>9>deC+D}K0(0s~&eo^QmEce4^b~f6Hr9y7qZ!@UC8Cl)PvCBvO;SHrT1Wrt^8zdn zq?DQb3BDtp2b3S0#!X*@wq;B;`6auUak5*D$;Cd7s)BaeZ4Xc6vNGzb(J@O)^8{Bp z*s;qSa5B!l*ejoQnBmXWeO7amKFH67K9l<6S8LS!QO15vK&PrO=3c)okkAu#BEN97 zaVHBpJb(BJR?0+o!TSpLS`1G|mW$c*eHvLO-maoAg?FlEAPUu%sAVL7 zeJtCG&&wL*n_<~9rt*NmS2LApkze+rL#-_Trn_D9a4)l?WN|PG0z1>9@;n(x7VaV! z$a(GSA6A0V>+_qt0-sQ+u)NRm14^Hbj%u(!=K{5q=<5(QXK`}cPllUz*R2NmEtgXH z1OlP|r(a*JWaKr;00kl8FtyL}LDXs%w~ZKK0=lkRb@)gB+f(fL)yL()!7WV>-5q2v z1q{r&knvu&K>*A-!Jby{oU3W%k~g@KZHu3C<2b{uT?qR!w%!Uw3||L%@AVAi$V5#7 zE#0)#PYQ^I3aFDgpF(#=YZ`B@md}6t#&3xwP~h8*?OxTAA}>S5At^Mj%V$Gl69;Jj zPNVvfY{S3lcPV}eQM?FW)nHTKZ61d`S{5NKl?Z)>1FCCMNA<>sgj3C0-3gB7yMl!~ zI-dO8!}BuwJeY%fwfr}Eq};fPLA_}^j8)YRL_UO=1zDm1oe>E~O*X;TER^ghP?leM zBSU7Q{)isS%KLs{;8f0tKTD?p{fD}eD;at-wleCaB(_a|z(%V5#rZy04EKj2ZN*R* z@Xa#-q8C&Yn9fCiiSVm@%uF96+E{*e2idzo;b-bhLOom-4NfkQp|2+)Bf_Ko?1I$b zNHnp=g0=`BvG?QcFuK?jA*^cz?}9+>H7h5C=Zhr(|6sJ!_%#YS&wSan<|cQ+>^uuu z2E6D3|K6v;H}KMNGr65SRkLGW_;ouVNwRv_aAkHjfc?9O;FwfM>b%$F>aeZmRIHwb*9Z2tPU^?EZc^s8NseRp_ zQ+t1_dA?CV9oMio48XJ=ik7W@3go&T;b6bC0*7ZIb<-|Yg(Z#IrM#+7)cG*WTSm!* z;B7(TNuH%Qlz1V&f5pS=IJZ)mG?6VOsHMgN&mSo@I2pv=YeD)S^O#*vSKdH0eIb=T z#%r6_xC!>#zxo`rwOm!I{rF&>3kJ|o`fMy73j+VZui6av4ms5}*|@_#GP+y4ysZ8Z zv#bY0zaJ8jJw?2&sQe8a&7k9-%ZHP0mkI4I!um1p%H$(ZNZ9NAZ!(yFwZEVD1(ixO zSA?S6LFBgjHw8=8=u0==wdVn0-xLy1>xkiv(8G&nM94A~ynAHDrKbD1M)#FmbQB|2 zY1xwHGhHO$LMu=cdy&#av&YO^A`oL6q~Vnb)0{jDs+Gy2`B|loA8qf`bwyb!jp# z)*Qq5G_eNW6(lZh~EK9Ha zrpWy@T|pBZPlQa>tHXOKE-^%$%cR%fHS0rE{HArC?gr?tOmo}OY`#B%cFVn6#STtu z!2f5IB6=?`Z_E1jO7qn5+bcc@VGsPqtC&Am2Z1_BbE7}#_h`!e%7C!-zLkTW*1zv+ zoyZwgWi^;7@iYV{?h-)Q_j(RWx9C)_OUYYTF)jX8sfCv><|%UpHX} zS*w(n({B<9d;tvieS43L9ftk6un01TP@{ORKp5iHyWRY=K;Ulk7BpCvV=35(O8!ji zhRRN_?ptU_NGC|DA+8!I^(_=4ZGmywe>})Uhviv$|9&qpD;JLYA-#t#0WKcl$mj}a zI^0qV4MK>zi@BPVvpeTF?rzbU@`_?*i6W?2{AO(=Qn>H%n$7;bh8mpj7bAhHJO9At=a8>g0&{8cOC`A>hpz!Q zOQ54?7+G0}dVd}Yyj{NXuGRhs%>6xzupT@VF>_qK+cc>*Zz2#=9T(j>A8UF)J27WO zuzWv|wOu*QMKBOu=;bJ-sarET`xZ0E(M^QV@r8AJ#^6Sqfnma#Qo;E%IhW$g@7@|q zxacEHSbMDo1jq7*U-1(aQs!bEK$-@n1C#0wA$WZ6BPEi>m}Bh(y@F z!niUGhnWaXCXLI38+zKE5V(o)%KV_sBPW1W6)yK7=* z_(ul&PwDx$gHR4#65o%{%KZ_;lKo?p*wlj;Y@>r0T0nRd%% zOhj4LN};AGZ5mYpm)uwMUm4op*98(HeJ(~j5<2S`8CX!`hl%6>c zN#~*#4MHhRc!7ISR*3s%S^c<6iUEue3Qm5yW#MA=y8sb=7kk(ph$Y56YL^`oY1})X zM)Sd;Y8aQkrYb)*d>(dFM=v0+6g<0J9-VNRptW zHtyDNQYVUb-66`}YW~1l=btn&Y#chkVIe>ew8!c11uBH=pVwYhhvp^wXEp3x6int? z12}e7yC}t&gi`++7V5I0F+16=XRQULh4E{RRTE> z86QB`p88w6DvU;f?2wmJeLS7Nmp|)?fu9fBth(Y5(!#OFv@{Yl;Yzzjez{)tczdk9 zecJ>8G@%Te(>P;kRP^nsXtC?4C2jwS&PrJD65&`jZmnN^-%)m_!>PW^b$P8mTI;WY zVFDScqxR11WQn)fA9dV?_xC&KP4P}To2e=aF9WY2Yi)q%@uQqSdz- znryn6IzH(0MY)Z-+$}iXUAN+W^z++3!B@!2o&SKqs-ygFApIv7sEEPrf_IfHqVupa zu|`xDHRiYZcv+e?Yj`t#SSap)S#QCrREr<KGco5IyZ z3GJ#AdhbXMZ-+kAwg{HLKs{=P)tsj0lu*Pk6(OD{`jX5jOuv4FsHI9@_1}OX8@zU@ zTvpjkAZ_ZN|7&$)vh{6&AUrg@;X%>n{WtW>xqoBj;Q_njv$r(FgjXx-Y^;=VAI&y3 zzTb*vX9okwm)Us)vA8y<2lVtgCoq=tb?Yeql5bw_FLQMsNaofX=fn2Vx3Z#nGR9`E z7!@iz$@$uvR)~kHlo|~pO1#A|VHYuAMa#5FhYb9q0DF!Jrs-ug2wK(!IXy%G1UEEJ z{n#WJF;{V9GVRiE-$XgtyAI}|+(VmNV7-U)kZ-L_Q5*K1%UNN8u^TvTF|Sn7 zEBog%qn&AHneu~t12m4abqi3If%o64YyafK{Ua-g0W66V%1?|0mlVze>9K`F}7*mPGRsZ84kO z3ugD$?6n$0xfVl!fwX-WFjVp9mOZx<<}+|j!7D@3b*gJDovW8m%S=_t$;sCmtXL~JdGJFe6yIfW4grGo z!ZY4*x%}T?`l)+Ls%=rBE<^{$Hm> z?nj2)co)gc9x^bN$igC1XWprQvATI=!bEEq)QiYW&OG}Z$A&f}oL;LLEZC})61@7e zJc(M^pL#!Ou53rX{38puqNR;1;kZ!t z4|5m~iQK$TLu%IM4(43ULe8x~NCaxxg=Zbrp{K{`xlnBOW${S5zJ+0^LX!^>Uk-YUAfbnWfz`Y%yWzDs+}=|?Y~ zT>iWC=@FdGYl!x5o%4LvbxB{-h(2!(+c_2b)B~ni_o%KtJloX5m<%v{GPFksA{$JF zK!wx+8Z$x9Yc6MM!vW>*JT`5s3hO)Fx$v9Q9gOjly(Xlyd<(D?jRh zhpPCTo*GS^_0bXnSOh}SuxtSJqyB>y^zVYxN<5UPl!^U^tR3px!DgnIrWjyh4Qb8R z2YveBR_Zu)bIxe*VSIV+Wf4h(4GDrEk_!gG*{e(RH~?h7>9 zb;4A$3+tPp0UdZL3kwUV`Lz;Mm*q13OQXPBBOAtbgO5o>K>x)_x%o4zL8>cOODIfT zt5zcSxthXVI(r~d?l8h*#J3_LLP_P!dvGV5$TwvCtBi+#Vil_J@tojN(GscjUk?61 zjHrJv-3}~7#BZ%(VYgUzRm-;9xNQ)0P*+=9g-#bA+vTjf6+RAxSh{x7{Y1nkZv_u*+A!2%2< zqHPe}fId|Mi>nGd47?Qe?$YZf!)VF?TM`=`zfL4dPptoN5B}R-)O{c+$coHQi8!-B z`<(&#O&=`*t4uo#B871rR;pRwPd$)igQ#XGrRC}!hpo-c$OeRrq*76%pi#x97Hd;h z*0lh)Bls?yHxEp5Lr3$LB}?T~;F)V%X110e+NqfIYV8&an?Y$WuhsihF*N7=f@pRw zKhz%f1`ebMZtp6cw*$Lt_jh}joc$RCJq7V$-^2lRe4!a-ZM#zQaOd$KX6S#5#{Q3X z>=2>gJY>@wH{aj&&vU7={C8rrQFdbkbDv{c%ImD0l5`M7>^pd49PW8la{cUQEd&mB zcJf6F)yBger$m&*Hij_`V`4hF$?M&4&DrPLV7$*I?;%=BtJ z_t|;u3|H;1^^Y`!m6o#f@@Ut$OZA$3BV=6&>zxJZ`D}YqT!Jksaf)m%Oij_Q38CA{2{=_rZ z>l`o8E9_6^l0a=z@V(Qoy}^_=Y2q>U-x%=Jk>1=_RI%H!Q#I(8arfx0>&eXciJr8y zwC1c93@o-G>F;G~)`Bz{%sNocH-+!UT}fN;f5#lY7QZjk=HgCkU-`=%`@#3%t6pu- zi{=XaTs$D|;5l!IEEza*iecwWV7+Y$J}|CAk^j*GIFWStdfV!)1)4vfxh@*!xoi^w zAGlEqPKKu@=RBxDvOcKGLXqcH)cw3oH7&dRd|sXQQTzPZ`qF_+b$clHjMx#D*n5BP z%5q-%b=!5s$AD=@Ko1jp@&5{Tv|#;v8e-sIOhSh%EM?L~)#^G=WgMCFc!&O)k$kUV z)+QJJlyb%U^G?q%nfUmszeut?3yW}WkxZI93vS25dlh-id0FRMc7EFPyAf_PPl)9B zXVOd0Q@8x*aqOqWo)YJTDlWWBmtJ&XG>#)G_T9_--E^v9ET3AdsD?)C8A{G|kd2V# zsr(u|9ye*N*8$BktKRg)|H};E`tyRx0C>XScDn8_TfGtOfNfjoc)*HQp)%iO(A?Ih zdX5Vs_xoWOk_JK%TA{{@y*Ys`+qG4M4!+H`8XNZ1L?+xNpBY-BYpd&(Zu8H(x8AQe zhaoJ}k?$a8Vrpt^Vd3=rP{B^>!T0bhxz_9F=F61psbq4~sB-Q1p&tCs#FAja_% zOhX=Wp?@=LSn*;irjyI`INWd=Z_V_TLl1OwM$@RZ=L($Jwa5L}tMd$G6K`zCnUW2_ zq`1@71J)x5E|NL z__xo~e}Gzfq}Q|7^-i=CR&J8M+j4r2i1M1CC1BznK_h)i@`>oB-z zWa}3y3k@Jk<%Sz+8#1Fu%Vrz7+&H0`E^$d9a42?F)=&A-C2B-5rTt9>@{go|3ro(W zvY_3FWDgANEiWipTey#OmOFT-r~d^2AOm}WD3cc_U#-QWR(E|&{l{1={qV=*kJX}3 z;nVuB7*Y(>e?k}-p?$B#S*o{YP@C3W-815X|LXCa#@J|*29@!;zwcMVodw|~j z>m;;mqkZE}U>;Z4pX~EeQMm$de%tncU7F$Twm`ha854r|${rh^uBJ5FkSFLb6w<7A@ z!SBlqFe8eq0l`qsvmCVF9LcRuI=uRa;x;B9SVlrtt3Q`E2{K4J;Z#kbc)Rky?n^sV zk+5h8w|GwEtc07%I>a;%sWi;t1LJPxU+KB{Vq;REH9KPMKn{dkGr^Ni%sL^wwx8t2 zw~v-DHp!if7IVzqA1zYZ*>c~TiYWv}xB;i-#+j2qbp0@eWJN|X`acJ7EDAW(qn`Qm z)YT4~57OK|#_3oXB+uP*GU`<$+cv(a_1$-TM(y?b13hb41H26RbKFbH|C1{ zA@XGKu3k%R6Z7h)Ysl1Nt+^Z=)q^e02uYt!j?PdP?b52HIEQ=$o%PY_cm&4M|GLMP zAwXD4&>r9@!p7)c6FMwbd?f#qaOh_vF)E6%aG3^f-Fd9<+hqT^KL396`pVy2U47;7 z>wbz z-wpEq@jv|{J*f8&O6_2K8S&T5%=P@@i?@FKJJWUH1fhF~O+Xz(qr6ga!Vg{DfFepP z{&S>fzn3P%pB*{upJ}&ss+@0!f|njsFl$*9BeFDQNy+xoP+RLMag z%~{UUmS+D}ZZdHv`^gHw8%~q;vzo^GUPjB}T2fM~^!sPBG#YAbuH90|Ez$WwZ}xf1 z$9msD?{P%&@6rkvZtO%Env@WDq!;CH$JfgWQ)bCANk%ICAHfOi& zh%5}bz=bpWpfs#YKHYBmui{%>AR^(_(G=2g3=sJP;w8G>Oh$W4d5-(*A`&Jd@H}j6 z;Tt7tp}!}ZaBE~znAZDv5$3UCGBadE`>RADwK`@9{aDxOgMOsfLs*mx{7$#1cc#Vy z3Y2T6%G^$yK$?DP+g#9q%K=qOH`q6e@9%}(f{40=I?fDN#NIcqF7bl~WD=rZ%S+g_ zcQocU@S*a)x*iRxQ)!|~;4n~6+bP2=^e<|@?BwIP<-mDlI)-Uz-@)Q{#mXNT$&59{ z{+1f6z#a=HS5j&9Hvz#q))H{zgV}#O@iRXl_pY0p8|RVL#tjFkrpealzlTrQIpFsQ z(C%k#91NAXM^^96#m$1fHqanT_$B2Q(CWn#ux*jsV~5kf)|C5v*z?v|-1QLwWvOYWc2O^@5OG@s5$ zN%L@$J_5Ro1DpwoK}lOFqzy%+yYppw#RGI+R`E1_)@kcGYSUnzxhWwm4A%__5*7dz z5W!mBFOMi15k{OB2JI^-8cg{8qeL8Xgdbsd(7hbB`TASW<5v%hMi=6`xRgR}k2X<+O!UiyxP=-vTY(Se3Y#u!xu$YdDur7cOL?J@>2P^=2ZDz`x zAMB?P0?_1-f?qA`=1xpMY%Zk;TfW;?MTE?4Y!dY2u#?$40$jEBh=EpqIApY8hE7xJ z7PUrJOAzA1)3{mLfjeHvslehyi`Adtq$#3afNh^XI$1elVPPQy0Zlfshq|(|v6**1 z6j~$K9dbCm%b^{twAlByyK&QizJZOK`D6#L|N2CiZA;U)yoLtSHM^s(TaK={HptP( zz#$KQFkWE~uBVM{S* z6FWF|w+`3Ywl2gKw>C5UAsYQobHAqw%Obrvv%-3G_+pSg7j3n1*Q{42hxJy`Mm;Un zR0NX&=&@D>@Lz0m*?_F{bc2~`zgc+VlA_l3&>)bkohn*CJ5Ej}8w0K-EXn7kl2qW1 z@}}ivX@*z}oOQoq6v0bs_#+;W?DqEd(_+Hfy91Xt(;O(9%17ooW(!+gm%DJ0|w`vZ!W2g@gq6l+_NsE?9 z`Qd~4G>8HY6WK_bw<=wuIw{30zd#*aQqMiiA-TQoF+Zwzm<(+TLCkF2G@geCExG>Q zfEEren2@Sl=an54_nG$@tSPYIr*_?yxWN7NNCoJ$-xGVe(y}ERcBA9qm(tLxQ*@!x zD}OO3s4O%-miGCnU#r*abK3_=P?kG;cs}LT&q(0=UQeslB4aY0Q|)e@d%E<-DuUzj zx5Js){niNFrhIfg+3;5bPzVPr>&V@-K<xUvo4vwbeRrGZmHcw#^nn(~o{KHshw zq*G{TJ#Sy^{aks311*M=2czBBsp7ar9G}RXS^@#RsaKLL_+U3T@<&Q!>PCN#lc zY+Mr2!_PKQ}jb&8=xi zb_B8aQh-=%jSz1_pyK$&2D~lok`O?C#y}wgL+=hR1sB5D$)Fx_OwTT~2lp|We`CJTsGFveMZR$hJ5r(Zr zr9j%L9*`E5hw}Ah0!aGQwS1HqGqzFs=fs*s727y(2bdXQ*<7tJbZBI>Gt5p2I|YAl z)~TrOw#UDW$1RnZfCE?@#hRmzl3Uu&UFXgwgcC4X(< zxcp0BF!w}$qdUsSE&*RG?BJlQ-ohsei?+1$=7ad$!wy@AV zqVvQ{%+RGor2f7|4>9zB5dEoXPKN*-*YlYs?})%) zo#@35e*+)|M!3E;4n6Sw`jGK?sT4Q)?NL23%HFT0Y4)WRb)6>j9=`zE@2cb!L z{sF8ETEe95iidIc`fx!b&-aj21s;Q<3THFCJUG@m%Vg{B?p~4Hv8?M!h5^SDxwI_} zmUGqfX0y0KW-BodWl`GRh|t=fTY}Zl*ofxlGB>?X#3vQP^|GIMu&Az=+-$jAaj|*N zsP!hJkQ%dAlNI=a!(r3)>fUZwM}xaoJB$&hk9qhIu;Op7;=yBahPQ9KSfSeC2Jm1x z#O8)-S@*ao&|KxZFW~n4`us4SE%=#{*BDr|??=c0$;7Hp04c6jg#cc!M2RW6hkD)y zop!6rtld%jcCM5+tVCE0r-kAoj0JOuIj!sX+Z1&YZ<_zkO7>K)Z%>+pY$UMs1|(7cq$#`^dgzp`y2{lDZxlT=Wn3 zs|)Ab*rQJ>2Hvlh4d><9Vm&M%&G$${j1*&)$ULgnh;`@hX;*wezTn)}a4J2+wdLh< z%rFiI0=Dn+OI0W@7u{P)jZ-{w$PYOFC>)X^7v#Sh+|VXKX%iFWaESzZFR1-Y*oa4B zn?OXZ!NR2Ol{1m|SYu8Yb#U%gZF%pu1;G6-T8=`!k{MS(wnzwlOR_yc0eSvu5CIJy zi<<6t)nTxBq)PlX0zkSJwBkht6=H|p=<%+)AELUsH^Cb|XKc(qv;g|U*ibq1U*+Xd zWwP&Wtt`=&T0MNwUUzsO&xTxa@O^yzb5&s3!OxmcF}(|6ldz`99I3*T%jM<}ba5mz z=o?BTX?w3ll8HuJk_$4byPUtb?^1XI!P6^u?m7mRnPBgXSR=(_aN7^CRst9%x`w)L zmltt|yO65*pSJ^B34^O)#)3mNs?hvum2di5w4zP=y&6~TT;Xh=qb6*1e&hKBx3$IX zO~&a%UGM=_(T6n~fv;N=c+vqZxvAOQeuX&4EuKYYi0t=0=Ay(#YTfN8-3fH@j_>FifF2uC;^`eAAHwh@uIHIbYTn;|w^M!3Ld8(dEh-d~4VIkQzM%>s~8SnM>)xlMN0_gPM=W>IB%}Luy^7b!{JdUKG24~l%6u(52z#d56M^klHX94? z#u^!NE4nc&;J{wo%2Q%1uao13o+doZ&SRC}0gWBIl#qARTQ05}-dIzhDv)rx2;7Xr zAbU;znV}1n`4nt!k&%(AI9zO^S8fhbtlfozQ%ZIIiAY|D>81;r)6h!T3W$WZ4P18m zd!zyWqz1K0?rqF*^Xdi3XbAwdXdtVNH1-N>amb4W;NkSom^Kc)KWoT{!FUgH6$%uX zrtmWSiDWH$*uH1+T8bwUH1N77*-GYSVfx+)ygTt%oDvOp%6s9|@9W%v>jk665C5CH zVK?^!zk$W|8r(bKPFC7bEXq#k-r_5>r|3W`3Q%lz(;klkWkwD+P2<-o{e@oedO&#fMK>hG?ZU{KV=25q@+>S+{2m+V%5^jAcZW!$lz!p zrXj+uTB3xjd%^*iJ`T0QF&`B67tqr5J327lmsFM|JO3+usCt~NPwm&w*kf6?%tQq= zspVx}I$bHAfun#wh&DDzQ=Y*%LcEf4!$>@;hEAfe6i*g);A%RHbmN1olH%cFnQTfo760-Aa{dXGjg82To<^|fixif*q}hR z^Ak=;%mE1=9^gAeAeC1~Aj0iYx0X;ANYl9V>FMd^rmPcNdbqhz0cZ%^I;`nO7heBF z);uxEj=hr{YZ?t)uvz_cUMiNjPv9J$3gYa2Zm4w~25_9!u4v_aD3neCS|QJPLTuYX z^ZI6mwFAI;pbMB}XM0f6@L~51x?#xW2P5d~>oBmE2e*cq8YoBF?M$@!dPkb1LmYnSL1+l%HY*0JCpB+GwHCqGEZEJbx8MBi zI7Xp)NIAix62_oOMpveypA7_uL)toxNQILHj@SsCn;%0ZFNnY6&IJtaVMl4vB#fri zAPnY1IX$xgB;yhU&35=YhQ`@O8$|Ywc)pG z!Mipk`sAhb1P3;P$ioao6`83B9uX2`vG|&N9Z7vtU*(pKGqqqFji5FFmHKUOy0wxev zLvIl}v6`)^uI6Isvn_T{bufH!e=8%47Eg3#>u`s}eNOBroVc=BqlC7*+%^8?PHU_> z%sEVOlXol)LUudo|C2K3co!s;bu9jgi0b6UEAXe^WfyZ9Z<3M>`xffF5Tfg7$Borg zNSUyEwqE?ng5&ofRCDdk%`n0E4~ncfO*K%Dxn+oK8Iop@f^wN~lS424`)86iWno2RL9)&kKa))E+tPX>fq-AsbP8kM|S%?9P-Vl&XAA+wVaKPFV9`R(r+tE#c@Da2Kw z<>gXX%uTub`-V83%X7DCKoeX~7Fe|8gGJ^wM8x%9R-XT-X{dl^P? zURNr&ItH^Iy~Sg#oxnb3H~>n{s50}H$H6rwhD;>0*I~lKv0qaJq+@t^D^R0 z?LIt7ASF6=a_21~{(`NKM)4;U%Fxz~R-jE*ZS}^*kL7q70vxya(!gui-*YyfcS5xy zx%k*iZHBa=?|q;ZDkHrJtl@f|cOxCn>xU~E24Xn(OP-V)T~p$^h@IGOD71v#i{r$5 zfS`byfC-a>H}W?!I(LdwgMULgvjlJpb@4M?a?^ zn2LW;y))|D{nl)8yEuq7I_N@EO2UfsA@h!(!hV_lI-~^fcs@eAf9Bd6i0>hDoz~SO z)6>SKxb;+R&5%0s>%nq4hL25ozoC7rc^fkuY!FiQJw6cLiq)wc02>)HHCdwA=fdK$ zYo=@yR<&;WeCJG>q{H}pKjAY2L@LX84S8y*cjNG%k@m^7DlX7(gCV7)Ia$bPV2K3{ zxt3;V34Cwc0n~wRfvXY&2(LyG=s@RggA6!Ji(^%>hasW1Tk@sU%8y(s!5@&At^7-7-AY;KEJ-NvYpMT_d*_gdWkHumULul%J+Vp(bhAEV+r>kTqCjfK9BH-CysW)uq@3gep8AOp^ z_#RRGabXJz=wprCdUVpBH2N%&Nhjv@cZ|e66BZ|y8LT7v36K=0*dVKG&W(bnFxOE5 z*!yNZd)GF&&_Qx#@%#T-s^fiK#?rX)mQA5A zN55uqIJ3qWZMz|16}aQqBl30M#!45kY^Ue+y*nS+&)j;ASUl`&($ET@Km;ifh14$^ zkRjBw-lm!_ZDclWAuJB)EI-gUtC1Xm)!>@j)1}(Wa+8~G51%5BE%2G&}@~Lfct?&Q*q=?*4H_c6K>~YVK!DQ*r2u*mA z^f}l(r-n{@HTnopG+B?FJ*+7O;|gd2Z^AA(*6N6{RDG-&E6=n+CStf^GJc-=hZ0%qkQQ8Yq{?61n0Q3GHr7xEHR zf^pQJJpT88$^v|M#4*RpyB~ihbLMOWXeu(etUwMsa3b=I%pfo7P;Qmiw5>wcp+D^n z$HS0Ea>X0&gGGy0san2XUV8I&nLjToC!R7~29*z#s_J#ns)d4@H(NgXV7C1A(SOJ* zFRhZk_#_u*DPoZpS$KC`#!nEr`kE``z=I|r*=m(?D0?MvFx|3Mb#niMf0I{dE}(QO z1N&z{k);`CE1=vAx?FYX@pAfCkHaDgOcmBwY8aYLU_Lnx>sYLrww_rg^`llN{0U&I zX0H)k8NhWe1=1;yl%+XtqYI!v>ps~KI-?&Lot~e<1t9i#Yl+B=?$#GpyE6)j#FD7+ zM}C3W6`kTKde?jY7<8uhlqjtvT;(q-E0fn=drfY*;RXzLX5>V*vo&<#5&)MxzG4{L zDaQ&}1#IBJ?hS2fno}eUJ?Zg0SnfGk%x2wd*B?|#|= zjC~T8MF(d`q1Zuars4SGk5^mA)Y@en>G?|k&Jo3CZ}~0bV_N0qs0*&D7fH{Dn4M(29d2p>2fTRywnO=Y81Nr_nPs_NWBGXQo2*#vRiVHFO$FildB}1;b z`Yu_yP~_wj21;R{fRw?`Q`C={9jnIJ;G>L}CvLCUvM+eBe@CwzXxmP2Wd?MWo!}as zc0uNa&i1n&QvMR$o%!)ndHJ;uu-(bhz6hZ_#I^HFbEW@)BH4HEkuql30EstOK^qh8 zA%grdel`tFCnd_{0|7n-n2}#j1ga^`{OKw=d{9Q+*2VH@Gb4?ic?Es(j`NPkn<)!c zw#uI#Z9#aN|*r*ID8wyhp{pHgI>*coJ&X7Y69}6>0(_ookttuLA+}J30 z-t{0fa97F2=j|&~kJ?{G3?B$B-WqxN<+tUJe|TFOD*qury81F1IT|J)JWUe7Zognb ziM&5|ovf^S1sb~p<(xB5m7LrxjA8vC6C%GVb68-N%DIrWB$IqBHGxImc;jOzEZ0kb zJ1Jp*x1q64KA-i0{P*1^IbgELMdzK21&vl|fwERGBNui?m&v)`{VmKgh+K2ck=Ti} zuM8g8U)BNOdhf%J;bu7I@1WI(T{@ zvvbQn?-`CVyd|@$kfl|nVab2au;b*sXI?64aPOu{PKz=8T>LR2f~VPo6F>1m{l6B-gUJ)oefZH&63_)Uk1 z!iEqvYzc5~i^gt8X{7rZP}f}3-={z4=H|*i`|Ojn&88DdH-O>vc)EHfz-xq#uEKTr zF2LNgbJtdKG2ZZFZ@1G+_?mq&D0MIrv+b+W) zAWj=qfD&!jW6y*J;NrCDoE#G{mLCPk<@4sqi!27ROklX6(z zefM*jJ4fVaHykgQ-U!>14I6-+ZbW)8>aQtdaz-9f#&JB*4^y~?u>NpTr@!^o7T$`5 zzMa`Yqvr0+|GY1MnEsjc%@zE>uppuaoOoLeMvY|~Q7&@X_r4`ZA2LC*$v7}~e9;y? z9y)fKF9c-JB#lg8Qo`W{uf~~s9V74!zKme8CrH3hvdM-aIJAwi%_FAu0?-iOn6jH_ z-0Lo+0NO|#6%zGkLBpdjW_oJy!lVYqFdq(OG0gQY%%Pif52C=vjT_~;=bn?VeeG*1 zjO^|VUk@U!n-!M=TSWoGV{*ZT${Vw1&sKn!+Btg0MmOgEtxDMaXg8w(cu|0b;^Q{n zK3Lm@ozZx;^3_%oaT`j{s8H5}T^CTzrE*aVnCYgbW?8aqIe6(tX@WU|R_ta>;Iy-X z>P8$iD025*7sxlxKSmODtMP4v3>uUr!zO-R zN`NPP{W}lIt-t!MOq%cuDJdwH+PauL|Ki)&-MLgQJbz#L#Vr>AV}|Sp<)9*%d>FUa zFqt@cA35_o*F!#gMUFgpnCy4ZX!I`vxA+m<<<+J&vTY&0P|i|~UHJQ;sly|YC_l%+ ze|y!Iv4#XrY-238VAdu#TJPPzy8{K#r`ce)i?gscFMtlp*>MkSNoN*fJqQYzo;;)j zMYC=+67Epvj_W8)D8BKZH)Qzm;cDv1;6lBPzwh+`FBv}ClBEXD^D~~8Z=CUswC3iV zInSp!njE}$`mu!EAsplg+} zGeDIcf?A}sf5t=Ks1Ip?UuT}SrQ`5hr$#WJO?;i@i)ouqOBykMbOd6JfS?|`@Q<~% z2(1aH2jms^!-=t9%CCNNg$(YOFU^hhScUb=`gI%R_5Xbe?a*2BlUp8?C#K&dqlRQ- zg`0L>TLB(&#TNq}%R!A@qoHnV%UUJ~5Ja0c>In>z4+jNATLbdu+lyrMIFZp42g-&o zK9ex`;7}%Q*}&fMXY+I*J>!gNA{!Q3_3Pv6D+f}v38brW8hcU5LxPNzri1rbZKqS< znQcNdh(D?U&o;#{yc&mhX#+IB&A|zo12KkQalt8AT|PiUJ~BITS+;hS%$>Jf{&4SW z^3$9DD#IVTR>n;p2CfjpKl6ld39Q=ufk`BJwH9eC@Y%`>Uy+HzML}NVhh`O@9=i*3@D7KR1v4-PfSae5wh!`& z4{$DampYvhkMI)SUft0d3`cd2XM5oR;a~>!nrPktuo+sj0GJV@n(&H2+a}uTmF9-H zy!GY>kbOnIa`LG-g9CBZHbL1V0}A3ex((XFhaP&6{Og~u$p;_2BPX6TLNdbEZdb~# z8IZLm?*pZj>}+Tn15+Tl3m%mDPs&iZrM9B(^hBP#UhL|``>%duihTR@!=R8*k8SEW zFe>Pie*KDNz`zp7$*t0ad{Z=wvBw-H7kuY)`TZY1g2{=6GWF!+QQn9={^T<%3SIO43)Nn4OT|VQJb?_!1Is-z zDeqq_C!c(T-296d&gAQ$58;ilYIkbPNGD3aiCyEs0;uv`j4$4ZhAWV zvAItt6R{sEFVcPd$;WEmiP)gQuV_Yv&frV>up9sB^#Ctd^@j}`hVAV`W#%h0I{BoB~*y#+nE%$2g z6cKc{k(0*!$HCitW3Jnh59haA1IjvS!&Mt6?VNG%s(YUH7{|)92Ov7_HgR(AKd?YX zj4Y9sS{x1$2+G(A1LUBikB}Ds({j&!GvuXL-jreIp8^|HA+*I71>iC5qfK%yWJ9tB zD6_?!D_%&%@&!`^76PXV=fXZyRBGx)3d+E^cxuHLibw>jT5&KhxqY%>p9c1nFjD#h zQE6_e$EsIcvNLkP8~|Xo!CM~FW+As^mhGX$1~+wKK-G{@Fl>`!A5ur~l-blc?4g+G zFALB7TiI7ehT)oF)ZdQ*eKZbu8#@wPDH2VUZhDYS@PyJJ_ zE58VJ3=RQ(6AR52pJdWSk}3U*`DD7u_q?~At{&rF%Yt$-zs#40cZ8!C^Nk;E@$y!; zAE7^Jd)0}DfVAs&IR&)7(G*das^Q_u$F3f%9I#?_++tf~MZ*Ud3IeG$t*_iadl&^? zfBkhCJb19~n4w_enrYa>8h~5e9!-I(e|WWmyX>1raO=?-b~CqADF8h|OzY&)V2d== zM`DfZ@C-kwKPJJXoth&zu`Vf4RcXtSuoIh;DP<+Fot&rzaG8PKav~$k3uOOE6D2>? zSAKBSgYw`X9+Dqje-U_Zn05dkM>#4IZGon56J!?14jomp^7WKgu}2VmdlaB9$Bq!2 z3gsIBKH+?51x9h!3FICsjL@tDdHZMthch+c0J7FBsjX>}^-v1`Xr&0LqP0Q&S}+r2^ZZBcQ~C!_lC$WfrC^kk^)# zKq%VZ)!mvZ8!<1G5%XpIq(S2M)JRq?4pW1*C6U2@`ZNhi%dH!PJnu_tWO+uzQydxi^}h#~gDE?DxEf!4&{4 z0U7(}ETHjoUIASp2#kgU;LXUvDloP@#1fozY~aTk!B*TG(8*j`A^JnImiA-kk?KH+ z*PPhQuk+B|f5YX(#82s1E+;IemnZQxjdQlcOolU{ePrrx4tCcqH1 zM(QOVt<@{^~1#$&toZXh&+|=sC211^ZZ%gG?YiIBaBw z`9OXURwLypxoqe})cAuVKh9o9kUE4C5wnf8f-3>I8)>P);Q%!-^uIzH>NmBG+!BTpFKQMQ&u1PAJZ>4D9ws1J+-7&h!#k{Ezj{&4Ip!tsPzEZkU0B5(MXKhRCh zE3m*>VDhqYY66J_+sH8Nh4N5y8WlxN;!J+bDW%ONv<#Sls3E{M!EvUGe{Em7cUwgP zHWdES9LG~@0nXG9>KbjZHRhL``~i~JZxrej2KsI0fr~7v)sd_-jQRNIx;>Ku)a2oA znZN)2?|L{Z`3P#7nDN?N_e|=zrFJRMGbli_VC3y)%$Skn_1QOF=Td;W1k`?Hb4W-S zJaU-?3kQK?gFRHRZ${>FDYYvpKtUzVDF91W92H2zu>dcWX5eo#DnPxI3E;_)ZsfGQbk8T31qSEskV!F88^kC zByB-QOCBXtPJq|Cs-N{~7k^0*jZEX#h>fhsE0u^GKf{-~R|^yBg0Ma2RZ<2=j#TzX zwVJDQhXq=WG{XjqA97np)?mor&`u+skR|}K#dPM(8BG8B*T2+s=a|uBbO$WT?MmdOS33@1G}Ca}X{X6O_uM1TKKrbkdFGjFvY|T#M>hd*gu&2*nIoeJAXX7p z$Lav`VAYKr1QDTK=z3FQ+=XP^B}ud8#5p;E12teiHN?F5`GNpaVTK{d`%tD-S61kX za6Xt5FDA)A7)%Og;CM77%a*N@9DtT3#Q-Dq06=ga>d4T9Wx6-Vf#= zR>*<{bFf>kMken&QcC*fqmSayHZl-IxgD+0GW`FnxGVubZ8hn zzr+bH@g3Mil?C+5dzJ5z5IndbkidB-jg2(`m~bu+fXx8|M6SE;S{YaZUJ{BY@d&|C zfU^EDc0a-oc}_slBpj}X`~%qqW#!IIWjSHyj`101&Q&qbjK|XG)sNs!9a=aW04MnC zCGcx!mJ1o`pw&ago*3k?p+ozq2JdH|&XKdeb&ASPuw-n>n~+VsIBZISVOhUn6AsoE zDK0KV9|0{Pw-ta@o;wfNG1AE5cFq{WhoPJUdB^96917dkYgTWR4eJ4oWcdMplSo)l zfz~v$NZ?xW33fu}oLBl2*~Pu`S?XNPP6sN6YjE9Zq2Y06+jqL_t*P)8*{5&(>8vGJ{=j z>%u@}I}FWeUf9MF?u%8cYP2it%`o5rN3HZh$T0|;ICjUY3n1^fTI1kmY{~nuSinIs z48!3PE*5_rc9H^_(pcLls}`=7=l=Pu41$vGgt3F5h>I0wPST-#X|3|gEjRr?Daa4Y z|GW22EIz=RzP3w3onSDek6d&0HIkp}mFW*XC<_Fc<*IPd*Yk{j@7( z?TQ+Cj<_UTKqg6Oz6~HeRX)n0w4gjnhSK ztPJ}A)W*mYNO68hzJKX=<$weBf-T=1Z1X`LVasXlnnoDi%=_VOzmWZH>GDC0x(H)j68tvY`;POqo3Z+0;>kxQ{92_Iog616R>YZCI@zuu6_ zDnZ|#4258yG@);XvI@Y2Wy$OXE9IfTJO*1kYt-fyFdHt;OAg|@=)CjfxT6o0%nX>P zfGsO>9@@h*3uMLWYI)$^N9BJgAL|G^zOXv((^c+iCmt(jo^^s`7eMnETgUuVKn8g3 z!`2a+k)qvX8kV9TuL`)^WpOV%f&%s!t&44l-!vSPRMOw7-%toA75j0p8iYM-OivRi z%y;f}kE8%?aS=4)kG4H|Kxb)bsoLnGCJ!}TT<222r9jW2z(p5bB=_HcKMo)IOeRj8 z*mE-G=46*sK*ccf+2HPU*CozU@$zuhR%lvsD}5&T2lC611=W#%=$>lfE-zcJ$bce& zq&*}W)&uBI;71+bLD@q2ee4FL-B^EK5e{Gr$zAt7goAQLjyY}+nEGlQ*3~RId6_sk ztV}-pbggXqvPO#g7K*d>^|iMfN*zJWN$9PC7|J-pw`)t|;n+&mkHsW0}yBY%GMPUp6%KFnMrr)lt_pGB!pwN?c6C z-E1PA1z=n*0MQwBc6)!c4{p05qKWL@a0c6OKimG~f=k3p+B^#I9 zZoO0{Oc;s9>?Ub|CT?C~2^Ld*vS@Kaii)9;fECpQ4Cp5v&4@r_an5X!l0I>KmnkJB zedYKQCP__efxJ8Gb=ZLl%E>2vO%~36L-L^Ydgzn`v5GrfUVd#UwAWV3(MKK*qy1}S z$j+a&Eyt<4 zYXRVXIBO9=RDjZ4T)}|9zN%XO^Tzw~(sN(RK?j6zP{1KrL5^XS9yX$2{^6~67R!%r z`mNwh1UYu#!gogvl;e*%LUM4L?uum<^8S0D%ALP{R%XqcEw}yZD(M3);O2&PV6v?I>k2wG zkg=bOi$`AC9yY^5vP|yHcB6n9U-b$?E3;r?XPw}uCn$pv22&hM7gy$e^Hh2R~a?LfT$z_*4EkF3db#lk;SIh8WP;lTu@vy_2&=WI`)Bo(Rb571i>u=U>Oc z^&iUAV@m;+O~%g21_^?oQ7n=}M3n>%%ZmbFM`zQ_8nB6`G=j)0HJU*x0QOP*u&@r< zPfbQ3@7;Aqdzs-4*t;d5V|UYW!x;*oatxu3m-DLQC_L<|wo5FgBy7~3+Up*gFElcn zE&EdytW(SkVfi^Ed&$Ti2n9H(&3kjO?h{3S>WVLCL&9!6NB+2*|M};iSNxBBLvMGc z&-L1JQaxNxX~fFg!3Q5KPt|sWXCOC{o`W z-`;xN)-4R3ww-Pgj7=<#0W1Mqcdd60mJZ3mH4v*63!zV zoxn-BBKsaRT&}v}94Rg>l6V6OMmxQDO9spD0C(9eIj9G+vSkB|ZQXtUO!;TYOr%G8 zp=?+IhZ3@U_0%%?=GTvvabrrQv3dhEii@RUQ=Qy?=Y2}MTW>oDr`e8|oJ=r9V0c=N z87ilJ{ct(|qMysJe)nG)x7Q#!9zeAhKy-Fak^K1jyJXHRky~y)UM@K2G#svgl_Bk9 ze!2MKlL4~-QGWgV*Ja3vzsXN-{yu;oXb*s{K1^cmrW*EtX4V=RaD_uS*KG$1D1(4_ zl8gd6jFQ?gxp;^Aj?#1n`as;9j|GZBU=I4TQK4Tz8$kzm)|zw7YrAi|+Yb44qu6aI zKq2GLfBv)b3sa^{QNWig{X9t2buI;53iKQbkpH61O=|I;eDcZI8MHtTROUhr?L@j7 z_dTbsc6Lq_GXqaR@Jwwv0$3oA?Fp7hA`kqa2aCRuMc@u9ku`Dz(1t{N>WD>i+JfBK z<>-~^kymp#KE{E1`eAw@jzf|m63D%GKO!$Z`w!F)z(~{$T{W~u>mUOhwNJKu|NG}i z|56;5>uJPJ+ybfJR0*(gANkwkPsfT_^!dthQD>xM$t6oILo=i^!F)rYG`R0F!w%_s|G1*aL7s z0&uuZLIOa|a4C*`xy{1>hxD;f~FX#Yqj;kM#A>GAFA=Uxx+ zS`&8Lka6v`*Ft;nyYl9nZ^}Uj9i#&IaVf~d{fBo5V+%X41+ZJ1wIB?d1X+<1V7LB6qaeI!nLJd%hcX+8FGSk5i>#InU zlPQ=eJbS=A2^b_CM}QyK(fGa*gEO>R@t)vrz*7ys0&IJ#1$g_moPFlOa_N;nVWOSE>a zESkGa{_>Z3a?COP(x;h{xF6tdXaMXoW6XvPJ}eF3hdOng zOMwn3z{6Nydg&zvd7ZD#b}px%d+$=frNFME0Owf>Y&>l3@WT(6tFOLVUU=aJ&5#T3 z?z(#EktvfA_5{EtW@9HNw#q{@Gf}w?yjuf!WB?BU5xtb($oHlI4L!0ny&zMmyc@aY zmfC6=QeG;jeQf|(?kt!+fGmQVp&0^RC+=M+d+jwAJNAx{Y{(OV1hhZk&wn0im2sm= zl+)i9H=PO`Cc?{H|CUV6zDQk?w&ncyI1fWJ6q1df`!5g9e2uS`8ABvbYq3goj!0`T`k>pKB8)l*J9M9TY>%0Hfb z3R#HRU^G?kGz7Ci?UNWI|Q(BaR_$|4C zsdePIIH8A<0r(s=uvAVwYJeQL&uEyT!-2sK4*IF6VL$U;g-a;60i@1Ee*W+O&XLh0 zhe7N1RqXhjFQ0shL)w6`jT+M*8oE>E$fNegVi;sO$bmdao4b}N8-DGi!(>!>xjgmM zQ?e4}d-dg+D(}$b3<}l(Y#f|AZhY;M+f6+>rd{_7lRlt#|I6MK5`!4F zlG9ql#0kSN_!I-!t;d+yid9?d@PJAC4VH^8{f*pu>tix%)Lt^Y8~|@C7*iU4HEmQ) zGT3xUV^AGqW^z?j*DtytBPZ854C`}3rl$87Uo3;YCTJDcw92!Oy(}vh&ymdRgfzgk zLL;^hC45=3bn%zM#kpAv7Rf38sbDbbbSnjuq50hG>}(v%+9-83wR$jIS9xIiEm#FE zMN!W@#n=-C8h9j}TMC9D6O7$jJvnxtea3=`hdGPN zwUUX|Z(lGAho?1418h@`8#YkNf%h$%S1+x#9tjrom$%>ghw>udIP)v8wcRW=o7PDd zRvpP$lIe*7051Tre8P!G$?dnlEOS3!BO^wR0T7Qka563NTnpHRsDOt6HGfUrbU|nL z)mBo#*{GE%CG$Y$lvgke)PC^hVw?uJ%M(`#?((1&K=-P-fxqnlV)b+X?5Px>*5B>7 z->!?lTmW|Gr#-bLxaHm*D8T-y2h#!^xb)IX^+3c|Uwsvew8!XN?DpT?p%!;s9xcV3 z)cAFw4os>KXILaM21rW{7V&WaL<}HACCeXlsl+(t62nygm?Q-D3dY2 zxEyDHY>@hD0HcEk>Ukd?&w7|BI1A<+8YC2etd1?-lx-3)*Kx+FQ{?NX90-}B4*md* z!9%LHDeTS;>X$1I-hVxSTjZd1BNR%oYXJEqA{Brva>S8?<$!%IkuTS*Q80<721@%{ zJ1%+9Sv-mZ*`f6diwKf~ZHFfwbD&H+>R|Cxo0aB4oP2CLslnHBV`CI~8ZxYp-1p!O zs$ttyyFt^Z)~jVbDh+1yOLeA-QtXPtU>bM=Ha$s9eNi5fF@1O&TtOc*9viIeec{(zxMMSvC7@(A$H9 z-r5enO9v438MJct>2^9ahL-7aSNgN&qJux?`wm{YUa1sFuOGf`#8Ap(D!4!bQ`693 zAA=;KzyrS>}dIMzQx;R5l8ha%E`NCC<@ zurIA)HdCjR0!^%pj5^(^d?71caL)v;Eq53c$q$Zx>h!sKms!z zGlOg6S$q%Dy(s5|B zhBvzf>lxF#Z+EWzt^P}?ShreNfq95Q3pCps zTVW>)%-D!gI9;}Xp+p+$Q3fzY=v!o#^766Gh|9*%+RezYHFuNwZGXc&#G4{qG#5X% zHpQE3V0s4{J2mU18Nh8On7u|YDw&*7a0nVYA`V$=lrLdQVq|}j(mn-Xu3!TgjJq!b z%zRUeWZ}eEoYbZ##twn@?fltIvSP(r8Iji?;580OK$_V(8RE$ZV4DfdPhfIMpigsk zF$OvWLNO`p50zrLb?Yj?Xl6*X9;fF*i-@MeT(^@cpzRoh2ZNyYc2P%U45+yqOW-79 zXzlvKp{nQ2uw1q}Yxvw9*y4o6Uc5^M0hYMOt9rHk=J`7vQaz6-AVSd>$EW z?)``Pl|%zfBl@knq?|+W7eaveaU!w@8iTooVJK(hi*K*KkPCP&3C<|dU5oXSnU4tT zVJ{ZCxYSgosf1X3llXA@vJX2Z)hS=eb^xfM75Dvcphpq2r-wPzDL$ToOI#Ob=PJUbXuR`f&WB z$EYbJ}3ALI{0Z{7rPaWoYwAJdaw-GVH^gi++4L9_Dhime~XMB2jPhF65>gK z)=i*gD8mQbyancNKySP3EM$HG%$YTBfVH!@4Cot@vXTJ+aV^*=9<_EZvyOaFgo#4g zCJxY?TDzH{0Awa?zgM8xswc`;#!<^NjC#E4(lh16qYqU%(ZfX+tw&@5*T-v!bQBPR0&{ zVvP?tfDfAF@Q>qwLaTwXF8E_xGWA3mjunrF0@_oC^S~UXKU{)759KWnw08;ca!lwZ zjS1w^T+`yki)F@)8FKEq=gN>FLlDoN;jEszXOvY-$pN9PtV}EV7r*$0uJjKaI8X;t zo}f*8UY0{6LK9Q_Hrkk+g9F0o>O&9)`-eyt(6UCX3|Na(*1>E1w#&iSPG$2KXS!(v z(B?nxAcQkMzGgYM!Eq#Zy6sbFXNS|q0 zljM};pxq9})}TdLg^FM`*^d=&Oz^PHK+KfNqFyi_9vr&X)YOde(JPs`hzraMn>5En z+Q#9InGT&!X7+X=$Hc}JTIR)vrL@9=Vx9X>{_2r(>35HpW%ECj>`<6IjkIF1BZ}2% zShH32D{XfIJ2WLW3bfKld{w}8GETm&si{#i78C$j zCR2k+#OtHIW8{;BMp)Gn(XznT!PvCxZq9e413!1&<|$w%c&=`cX6Q$>*AyeoVbiNM z8j_}Ht`vob;28{p59;G!h*>vI$2zloQvID_-1|L)0&AdwbK{LSstKow6DESsOdFd( z1{W~eGunb%%H4+orcZ`(4Ez;WTmkb+=gaSY_d9Go{vqaMa3tW($@@eItdL)D-5x`M z6m}Aff&xN3I9na~VQ7VAmtu2&Jpd{kCLE37+z8dGgaQwFGnXH;X|;3M;Ih79yySqRCq`Y4Uk=)bxdX$3l-pFmHp$_i=%?24>dy zj%R%k;04emO^geQ9t4-L^Uwp3)#pcIO?c1k-^LMH1iI>>imdJuq&xyYn8x$)5D>Nv zQx3>TDgc20&&)&|z!K;P!fLoE(xhyF@=%_zz=HPtMO%C0I~%yhcV2*>36zOIHUVGN zu%#NAf?+J5O&_N`0q;bN<-=Y!8t(=8hV7L220xtnLU7vylYU{yYVn8Qt%#w8z zr%EOqi^?6PG^3D!vW=BDTp%COWU>b~i+$R!0PykzIm0_S@`F+*c`Ab5UI6DMeS$E- zHxO^|&hYeluU?i|$7kyif30$wG2cAvLV$*`>5%-w{?ZY5GAi6q`6$IXWM9kQ#FrAP zi-)$Hb`hvozj7*%5vIjyduTtv0MJ%-cRTBlWY-_=*`?54RJ#`sJddcgh0ARjlGS$z zRD!WcBb48A5RfgA&d@oveo~*%+}S_iNxq(DMb19^Z0mEzz_G>jo1|~=x?!K&qZ|hY zPVyICc%i)b;*0XbAO27tf9!G90APiuIFB$C2jL^6fP?*lfGD_UbbwB z^c&b$`W5VpNYO7i=w@N?#lZv7dN5SbnpHjv%#{xaDvg^nJwLWp`6*!G$Ez}6$$^>) zr#YQ$b&5bY1bviFrXM1#{oa}=_#QMxn28g|OY_5X<)e>3$H6NH0MNyO3i#&(J$(SA zGqQ?h>DmTavrgoo1A@pp`V};%C&4yWFz|*v^6(RK;&FRRepX+Jzy??|G?Tsl5HygB zu}!H}o_g}%Qi5*3&;BDMMy+B@5Hiw*lofx2-*h5al3xl`)tucecVwM1ldC!7oa%)-FDk;QeIw;6Q?f&V;IBY z4-dZN*ylQz0xku12MP=uHcY<#?QhF(fBRe6Z@>NIkV6i!7PgUjg1hX`ZQ#J}P$D;v zTStM8_%O`!DGDjz3Ak?_b|HhNZu)G}Wd<~Udf?k+ym(ZemHA;BOpCvc9P^E|$b)D)$xB;c ze$OQA_iZn%I}gER@G_cr&44<$GKgLWYQ#6hWFa-KM3amj}9$`&CW6CB?y~m^Zwk;=A^kMXn3$@HF1D_26tY@u1 zePg~fSLD-ak} zQC_wm)&=`WJb<;2kOX}Nu+a>y-OxZx+}JT2hrw-ao2(Dgc=L2-3f$TA;DZlh*YSM0 z>#n<`v=ni|8LY3j^8@W$uNJ>||KZ^aMMXtg;j?GYmIwaufS$NZa4p5L;j#bohlbbj zK#ZZt3uEFLqp?+dG&BIAEe7N8xY^4xO^<+vmBrF>9<1Y)!y1*7Fz0hzRK8O#!h%zXKE@#f(j zwi7~NFSi5u7G1er+p>H zujzwnQXz}XL01lG#<6gMMQDlnnsJ4_YagC{fNSwB;A+s2OaYEpqYY*DKY}{T@!!3{+uz(gvBl&jk6(S0>53 z?^McDe|rrKN)bLQkT6bq&B*K{4YdJz;?bG%*({M$P8%e{$HAs=Q-y-^EDv{#bvSv0 zj>>=Vn(mzMI(nZTeh+!(T;I_zwfLxW)fBKMd$tE^?s|NEBqKZm78!=4&I`c^LMsII z1M~m*oZ1K0k6X#uiZ;^p!+l?BXq;Q1&QP;-}cO$}ZC7@wN;GC(Hf2<8R(FlpIl zzpOZ!Kl?pGX&B`k;je&|kwf@w|CnCNk7@G9eDa6w>U?&F!6$8wX*z#MKZdr-pVAH? zwH$3~axUPWj;*ujA)~@TWic`S1(a&t6>pj$bld6yf+Y z-*s^Hz>hzM@gr=?kA9o^G41s9H}{S6z4^XZU(tUpARbKvt%q%ZKj`S=$~Ixzg=rgX z=aAo^x$Bbx;Az7Eqm4zI?T>~7fLGfu+Zo%D6;`iyv+C?JXU-hC|Ni^c6e|y5r6aiO zA`#DoVnc9XdI zXlt*IbW@_(Uh$_yy8Aq^61T-|=5w7O9c=~;1*uu>f!kDQv50MoU=wYJ|NQ1(%FrSG z)C@xeW(?ND&gEXhXODx{F}6fTaYO|)jH3Xe6X*|#NCMj=ksiR{I9Bwc(F7Jmx*#B!2CuhP z?f!Z^$gDCFZT-*cQ=p$RC2Ze#2aVdH!*DX}556Z?|L_613LTQr=fG7ZAfM&I{UmiuiR z1HA70cPSjo} z%{6g*H&585qgF40U)Q-5a4E2RQGh#Yc!=`F7hkMe`8;#OX`+F=KYtwwdGvE#@27xP zGpkzp2{6bWYyr*yfPjs`KFV*6Uhu!Q*!P2Wq6@>MiN$Ml$~&aUhnoq@==Nl1__>ts z2AUcEjdZB{I$#1j z3Y_BQ@V8>v{9p6^=0kQIVq3rOIM$x!OR}Qwlh~KGgJdn@CEJ7btcx{RJV{{jA%b&L zGBTjGn>`fcF_a~#oCC~+yDLn1tIt~TTkW{>w)Nv5|5%0$8KOJ(skC76kt4vneED(}?w)w!2^lkH4DEg8SDVqdChqPog(5|ZQzSrfD71KSC%6X>4y9Oe zcW-fu2dB7G+zY`ePH-mu&CLC9*In}$Oul91&3@0>=jMAknFG3hTp?3l8+do1OO%Lz zL=8>CUl_GH?OR70>h+x*5Jn!6xrjc(OJIu64M~DSOcu9&Qn#MlE;bdm9V<_`7&QSK zfxQ+0J6^C(CebY~kLyZj+s;+j72JzJ-De6U+)NfKJp!s;$;x#yD{m_jyb{jBWpz;HUj2kO1-lk6%NP@rLD<}rFPdV)MXnz9!P@_p;4TL->Tp;NDvWB#18T(jN2o$Xm zDU5cbZ*+OTQfrVz2RR5AV9Yb%8ET_lD`9+?zu_92>Ra&Mr&7wV8sQ~ZoMbogFUIE=A9p%u7HYYhW)q7oPm<=~Wc8MOo?3 zdZz-VKdvvwkZAz=m?ko9(swi2kOr$8;mbu{Pf>-%@}oY|@DW&vzC%X&?qFk|Gc$RR zw_{MLzS>os^kf`H`r59I6ax;!1zLYc3Ux7?Vs0l$&FkgdGqB|~7h_zG3-D~&~Yq#@3j zn?zmP;uCA$;yjU55OI^pxJ#dVni?M81Ny}cAr-ogpJ7?-WRx;|0pdoda2IK^Ovng);~(427rs1)(b)i`3VR86IEC#GmC9xREa;Br)o7qL>vNj_)EG z`vuLaWn>EFHHkd^WLthaWqW5|I))0)+E>OG1U4yy!Q3+r!H#qwwBL0S!OR{jga()ycK+BF zFU!^;h(Pwi5h`)%IvNqb$#Fz}2hpl|6#cR%5p$}evq^2< z*)C*l`x+tK-Yig9Ga{anEkL=UV3;OMDe}$m@MJ`<3l}gw8rl7+9yF)F$igzB(yp=J zGE_u|Gw7C)gE&@n)TWNuM^Ot-Bo=!Ras2E1cB0ohmq$G$BlNYN2+}jKrnZ$Dxn=as z5y9C;sqK22(#rP-pwZoy98y>)!`-?=B;g-*K~4~L_E^Y z!EoAz7i`VzhXg#Cqh0-9s^U0jxYTm6$2)#N84Px1vnTvFK)qB-!3|;8*QW7u{fxx- zH9ZEeZtbXQbC#}Br8zr^=X~fwJ`nM;ld0njr6jXL;g$^ryUg=PF2CNLRmZr@Xrx_b>J68`sF5TIZ zOLCbEdk`QFilii+InDsMAO-D8@%3|LqQvxFeF1!d1A*=do{72a%~6x$WeR0N!Lt%# zK7ZNpMl4uj)g)DXy{;rooQ`0_Ci5e4>(wdcK}DM4h`uLt-l%O% zGG#XE%yvegNyRQVi3fq9K|&JCuHSoO*DJ-u{8OkT&R1rV`vJtZ7t<{A+=gS>QzP$)fHe~#^- zz7ck^F*h~iNC1fWYCu7n#SNZPj4>sx7bHtv*zh;m<@(H6rMHE>^C^(1tXFuRNsBd< z;k^U9q%)eYzsere&iv~vH!6KnX@H|Jp&q@X>pi>e+ozv&nW=!1Ps_qafJ}cc+7Zaw z-@9g}Cgt!xYCfcyRg7;!{Oa;}N305Tl1QrNMdLJuCua6t^i;^>Twu&;waR5*`HHu$ zbydGiOBufnHwA@%X&ZUjSWg+y(@8!#^lP^%JvgA0LZ<-9p|~;)l@Z0tf1x53OrOMP z=|4d){1jzxc!%hUxDD$Au1*7E1{4cfzD}3Bs$5}5U=qlN$7`pDfA;h3fPp73bvpLu zD>}V=BMyT9`sE$2N|w!Q?umLHLau)sIGgn%@_+n^8;+N_OAkaV5N#5QiLLJ2t>^7R z;JU~-2dvr1Z|}fD@C&Tu?EXk?)e3nwt?(wa>C; zc_*rF$4XC<>YwmG6g;7||Gdrfrggx{fEkZ1PMF22gm3N1ww2+IO(bSIs@3cF$P+JjbZ>vQfQSc~*L3`3W>9Q1{$ zZjPDR*Kr!$yV3B0I8Z^jvpjf7;o)0sZ*^+Ic#Gc{_w1LzK_M3@Uc*&GhTN>KM^n!t z59`IVRs^gYY=@7>k(E4!aNxc}VK*eIrP5mK?StB-uuIg5d-Koug8f^NoQ$$PKr)B0 z0EP=+8gaAujLzP_`azd9&xMTf3FF?4d@S($%Wlu2_mK#g#_ycI%P&12{;0v*cvy#- z_8tcI!tVkO%G5MxZ)ss}tn;;|1}C#FIVC>IQ*sBL<*Xs{17STv zI{_5!Cj-890pDQxEmMWriWy-Sn1M=l2(4oY-L-tLO_*irvfhj4`a4wn66l_TOp8iF z74s7H9(H_2bel}#4BQ5r(%VaJ5r<|xHR_DJe)%gHi%gClkv{X?y$XEt)%Nh*&{q63 zNik73*JE6_kRn27a8Ebe==(eBZDbZ8RZ{N-mzcVFtFv-^R~=KmSM?~E8u*EEuca1r z!F)^?NHyU6aSU_G?7vMNdzw$P>E{4-d(IEsh|JwA z5yGdnIcs)UB9gpFh)bZg#{1wpOJY0Sm0Ut z;;Gpm_AyZxX}=n1N9thM^Hq5%b2VUS`1ku%!lx&N(&zoWDU@XfQqkyTUVbEo+}#RV zgd~KV&sRPK7_KHvQxiFP8R0C#Ve*OjwzfEL>8024Kn^9ZtrKgpZ)0mzR7ht!gcgE7 z$r%)Oti!3AYU!DIgb6Go?EUC<=E3L#kJPt{PCrj5#--tG})-h1# zg^qVG`Xu-p=&f;UZAp6#G1Bwo9*P~6cUG`hI*5~wracN)5H2xi&*ESC!!biX{|=`R z$Tt&vho#|dH*hFx;O(IXq5ZZg*Y+8R1a*m;K*#I-r<^X*f5 zJ#Z_dDF{DmEqvae!bA*h#_Nqk_b!e56pLfE+%j4m3(!1-1#$1h5Tz!#;|lOIXa7(= z`!sgNArqgv_ttgKoEhKXg8HuNG30$yaxgT^%byha zmL{(1X4=f-qJpAE+8)LRk1nGMPch<0JpZ)QnZv~XJTrasj30RWe*2ue>)1^NfKC@= z;q|O6yuSe}VD%PFnv7g^`W1JQ%GI3C)d`2?&G3c6aZ?@m5mYUD^GA`bR0{HP*!^CC z)KmwVyhe8t>C@+;X}vbHx=6?U&=na7q?Io5wFs644})%0$H#Q+7CwHpWRy=m zuk^_I$!@WHe5zzfjEpuSoUZ{}4Seskk_~)GxR<(( zoR0fRY$aBUG)VXpwPB-QEsp9t(9eOj)I~9yX+GP<)R>7)Ba5FO6U0F&*=>yi@>l|I zH<+rJM*#m6Anqnv%lU4pUeeGu>Gm}nu#7SmjPYgN!85lyV+MD^8r>7SZZjfb$OgJC ztTkT9!abHox=kf*UF$S@+}j;a2NG;6@3m`ic-s>8JS`>L_Q zI(j7!;SVNGZyrJ~n12Ux(RMkAUckrEX6KgZbJq&u@I;SxT!67ZY;{FPSk@*q7~z58 zd0#|Swl`9!wHk7v%)fMTHyOA-WYlGFV^DHLs#BU^f`Jj765l;2c~d9#EA}5@@0NCr zZ_0xznlRBCf|;% zy~4vQBJ;}|=D-#h+fw7l$r?^&H2UPGpMY4fKc3?C7+1Fu^h?-uP6tmr>T!PX3?NNI zQL^s?%0BDI5E9q=I!#q=!!mz~`ocf9pI>H7EcNOk9#3-DoFhucX{Ww}Z?4W|8R8tz zyX*cSqENJHSKjUd%1$%i9imjW8LIQ5<9>HnP$9%}H@2*gspGlT5Zi`q>(IwA< zYi%ZT&)`+K9O9Q5lLFLq*i#yY{HX$L5=_*LVoaxO2R3D+OtjmTlgPHemyk(&BBid@ zBcZ3T_wMej7{&xr45^vSU#WinP{x(|iaJJu$<@hb6*yQpPKmgiTN0;}z)ASnDy0GD ze9UDlp3;8aZ+waLc;d{1#u7sy0RV}))rO|Dwjk~^bU)KjGB!wCNoJb`WMA{ch$OpE+2*X-7cFZQ zWDSng-k;w*{j%_f=+rE2d{Kcok6b1o>^(`Bv-XT$xEQf^Mm zdw`A3qdUh`B$I>5*+avB(>S&pT=_=7-J0uXc&!(LjzGXw5j`&8=|J646UtHD();H0 z=5F6CaI2p13Ho4ipCRu=%DDNbU^E&!G(tirvI$pglrO*ukN7T`NyIh2<8|-Yd)}NI z2G!FOyps;zg25EuRRrw%Z8jZzxz;2%ROm2UX!WZw9WQOUGrC)N{!nk>&F7!GfhPB< z`XiGiT|+IFJ_q;Fho6&^AZn}EaoYRaQ3xpw?=}gBEE)?WR&3Qw%bO^F7^?SUEzcIz zML*dsf1F^Pd@rYXL7%P1FIgbmbF{;)yDKD(rrES7{M>OG;C%bA@#5rX2RZoQT(8{) zft4%|<_Av7oz;_C#`V&z!2tp+Lc6n^J4O93lJaoUjD0I?5w-iP_RjG}Q{UwWXP&SK zEDEJ!&AwfK$B}Hm3}&EPp4!b>ihd2KofdNk)OY(MAB%lbtQ{{*$8UE?Y$?jia=dnr zgohy$tm^s`Mwo+4dc^iU#t)IF+VLX&1G-e-$(wWGUmcaz7IylKAv1G zrm^4ZXYy75Jtcy?h1i+s2&;Niz!fKq-T#n`Jwq|1ed}xV_|$Mz7Wm9~5R?vm%7~2= znWezgH_oH#f1K)8{VD;`(oK!#jEJ&7Yx)#t3l7?R9pxMs!6tjg!i#q|CH;_@K_4o0 z7FKKh6vf*apRpl$o3L2IjwVU(_uSnw4$2fIgiuoS`5o?z!z*vx4VpaR(UxV^7ty_J zi+r4R1Q7_=doM{Gj2(_E5sK`AP9&43L*2fG&YQ-wUh+;}cy+?{9NtQdyI4S6?5O(F zBVG0O8)?&oW24_I+eV#FY1L-enmwlExRRXzO{Ug8Tw=E3nT~E2u4OpZl zZiwf^#Y-R=x~}$m2Ut2-0IQe6%&tnO(nTK{vdZ)d;*(l|e4=QU4-Q}X#^%Es9nL^V8;v@(OLy;t53;-V8f zfci_qh9$Wyx->&KrIy`Hd)}UJf|9$}c(t#*l|%{zGp#KW=7$74OgTfdL|sANf4RSP z@c!wdr>^UPdwF6WnQ#?LScbY;aGr!k#Ktt9^mn6Q+EDa-syIKL{LFQF+hd@D6Yte> zpeM^CCf)XBs%oLR#UJ1K)b~aaC`}4}BH|?MOwa#o%_gXD_b8veIiB}=SkoThAoiOC z!BsgVEr%Xe6&%J9oMgK{6QBcSs6su*AjX_$WC^Tb_P{3YQDEx-de4U!+ODC7Fs;(yx2Pzl`h?EG zU)MY^L^2eaBjN^;zYjU$RY9c<)AeKWo8=4T`nLdw0E^SG&PqV`)`m&dy(U z#us=1I|g&E`1$0_m^5nh_Zd^Y@x!L&J9JZSam-T~RjiX{#T z%&f(wvl#z0m>!}=fnI0tSMssIZme`%s>_^W{5he-v*Tjbbpr232_nK9=~T3 zuX)q}k~gX)PZHD88F4`-6(mblU;M;3avi5fZ}{3B%egmqp+5S6i^Mt$HFBOqgK)`_ zq{nv^bWeZ7bzLg$7Ol6U?Y3ZJ^|Or-oJ&9d)Vi-W6qjd%kB`$(&(e)PAIYEz8$I$h z{kBsq9!{@*4SAw^T42xOW@P9d8u}VjPqfoYF4&AQFc2AC*f4HyqkLQ@o~M$l7UO>f zd`dS5(Lr=AXdV|tg!ln0W8*cLMfSoGb?3OeCST4Cl9WYx`=_2vg7v!zuLz5Hi6)kpL2+m~FR6$u zIw9m&A1HV#lzr^&@!$5-Cd!w8yTTUydf&Ko@eW?D!}Zyal&;Z%ngODU;PIz#B|Wg^ zX`eiQLkQ&$LyCS9v!9oAACQ$L1Qvc^gm0Q^LFU;#|JjhPpFSKF3sX-AkApc%ZvZ>Q zUOx1NGrPtFz{*%a&xb{hB(u3m$R?O5!8*LKZw`>P2UQ83P+MF|;>Ea3D8(eya#vEL zSF}HSAE%&Yss5fXF`Y`65+5>2H{Oh?#rJo^bq3`;-GT`raW9+n?tX^iHV^kxk&>3} z2k`x@M$jz{bafZ-)7c$Ospd3*tV)xS0Y&F667*r*VE6&Ssrn}8)PGP_(8U4@ze)CO zmGA^qES-3+tc_W0R3-~gPKvZ5%H-4JX}hVq#oa%5Y zWflEX10PymwIE;Sy1O$i5<`LC+Nq^#L&i$(!ao5I;^HF72AmC}98g@WUe&J`)a!bX z3;G_NKR>Yy&ZG4_jyr?-4ZYSpw~CY0u<{;O_r8BfRC=5d%uGPP^o9OSe=VyAM@GO&fpDmX5fj z!iU0>j)_M=R|@KG0brW+YnpcZHpo7+_%Ru?Tuy28{FqGU<*n$r)KdFDU1EC}(!-Hl zxjq%}WwI+4?j6A8Z8TjzSZzY#Z?^}65s{>cGh1=(4lWd`3^^}*C`;>HcKS&R&!ZQP zZfZJe;$zldu77u$L00V)f;J4)JQBZbd1VO+up@xCuuU+wx3`0LDb^o%g?VO{jZR=y z_tyW@hc2QqZ%}B@ntZ|8Tf0X2|HXtjv*knFY$k7!0x=G?8jcy)fb;JOLE|VE z+yfvee*0$&QN}#TuN{JH_0_jc2iscN^~2wSp4bFY#)qcn4lW3k%NcW!KG46BP+U~_ z!d;*sQ+@2FbN4Ll>n2yOCpSUcM};Z|;lnXrDv#$Ld@w=hBa|nSRt%K>^%k0DLYI4y zTt-r2wO&ul|6Ny4PF0FckKtEO8YX&!A67vyf4}v16f>_)G-U)m3W~e9mJH%$`{?8m zPeR$$r~j-niWtQiYreUVnsbWG3n>M8O>9RH@XXx;@V5{vtIdogfat!83@^R@yJxKR z1f!>Or`Dj#uX|L$2ZhrW2;N_w#9=XB3~@LkVQDxl*4e+$!LAeTux)<5kd&0pzJ(mCN=y)i<*nW(xjcC;L`Kpr z+xY<@M@FYu>hGSET1b+E@>k!A2~)%AcFxIlAbllg*s~2ZscZ$b+&YW<5ixF7(@VFIET4LsOsZrYI(9UMOqy}B$DM@|GU`YaaB8k zPo~>DZ)m#;s;fpb9^nuzc|ya=Rbo!rAZUv2^4Cn7Fi6MFo)1+H+-?|9+)&`{2B121 z-oWo}lK-IFSQfhekk(+pfn4itv0UeJ@mZEkT(^b1*5iFlcDFj>#yFZs2o<;4N8UV( zT>3a0_=E)tsQ)6(011TWfPflS3zj**!S_PwX+c3L1I|AnJ>~Z@rh*Ivt>sQ1mK3&Z zw#BH{43HB^A4;EyE5A_Y7uh}tOG&uWJ*Y}|I?GUtjNJ3y1=adJ zpXQzw(Lb@ltMoVU+r3V4|G9$tv~`V0;g~iw)YN*rSCd_N%-(AQosn;cYeyend0)<7 zuuU$B1U?;^WJ%rb0bWG2J?B3+aqu`^Cmxr4Tx>7+q>C%dHO9G9on$RTR^B64PZnZ1 zD@5{V^u5iXY|vSFgOx-|-l+)-PCw_EcFx>ngJ}+YY#31Rh=bi?W^9XluJ$d}HOHPtYW7HWih(`+*$pRgx$&gR^vx9l{z zAEYcrWzzziFY1;V$%=O1R)P`C^cCUZ&2u3o+KtcgDPHMhk@)}{hLJ9)*uTqhrEkmR zIe?(ISkxx;O*q5{Xah2VFfJ34bpmLMq=VFFHdHtogG{BFD?A5;D?i~`4wk=w>0bd z36brXv--3SPRR3h4gRFhMBrAt5^?cIr~Wsdc6&E&P*}si;jH096IDeyH4`_&UR<(=vJ>w>2z!RJFp^<8oYhzBRZ z&Dl+F@+hjKWe9x-D<1=EO z9$=9Q`Bhr(QQx2{G6imymN+0~5tk1<2Co7y#i2MZbY<>v`w8pbj*{|n(ITg4$y|R> zQ2QbmnPIj}3VVk0DxoPFM+kV+PqMc9FM!PZ^1A4AmYTgjHgTx^$VucwQU-7m}U_y7@ZEoYa^g}_}ValS5VN(vaamJ^XjqV4@px! zp-TpHjn#^dZpbrgj|$P^fb}<%^662KEbc}*l-m9L&Pxz)YfGI+;*rfsZu#NUrK5-+HZo&48HsS`p*ui zgN@|cO$?awp(lUPIwd_qLM20*bK)78583$)#rXCJt&}+Nn7$JRiH0>o*(_4}ejNTg zaS~v7cWn>yMLRNt60$!0n32qf@sF+W07fr8td=fn@KBTsLmfk^_2)8f(Isu1v2$6} zZ8EOZ{~fq$u?X3^EJ^@fufaSV4RTSkPUd+P={8Y>MxsBgsXszX7AEnv;(bSYuA(+2 z#`b&}w%{-swo99k>4a{heA)=fI3C2=-5rIzVk|Nw_{6JxgW})ar@R+nSR*-yuCeZQ zA3Gu&%2d}D_e}G&sHoEB&0II?0;WI6-ZzR9yE@`h`26NQ+y;B(y1U+kk3GvCou*}R ztgg?`T~}OUk@cSUDe?kWM)(#thH>$bT(1dp{mu=QZtugS)^o2UsIqNp#0Zi4zDm_R z$LXD|3_i!P%mLdx+}vgu{)l!ZY}fQ3MWbkA7q0-v8c{7CD8+;f9-*FL(Jv_JleB0WDoDN28n z#mtF>9^k|LD?p$YC`T83Q49?Zf&t)d*JTFXxf3<7;>ch4@%)Yu<8)TPr^EF!m|DXQ zmR1N7Pp#3{tE?mxazOMb- zr6Q0J7dC0XX3c&0&w1fLZ{narB!qC>pO^nT2LI=&|J{K9{w4Ss9xpwKk2U|FfBq-_ z|H@n1G@6cANFhf0e@eVkkfMS-)qRTj*WLgAw*L|*V=BkQ5aJ~Vpws@(E$;uk<$w6Y m6g|8n|6ksJS0-G4XD>W~*_A__A5#(FA0;_8*=iZnkpBfSWhI;d literal 0 HcmV?d00001 From 18f1b9a72ef5264bbef60bd22111b62092e1eb42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?S=2E=20Veyri=C3=A9?= Date: Tue, 2 Jan 2024 16:35:23 +0100 Subject: [PATCH 192/224] Make Neo4JIO streamed (#29853) Use result.next instead of result.list that returns the full result, stored in memory --- .../org/apache/beam/sdk/io/neo4j/Neo4jIO.java | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java index 268c6b088d89..1dd95b44c2fc 100644 --- a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java +++ b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java @@ -814,17 +814,19 @@ private void executeReadCypherStatement( // We could actually read and write here depending on the type of transaction // we picked. As long as the Cypher statement returns values it's fine. // - TransactionWork> transactionWork = + TransactionWork transactionWork = transaction -> { + long count = 0L; Result result = transaction.run(cypher, parametersMap); - return result.list( - record -> { - try { - return rowMapper.mapRow(record); - } catch (Exception e) { - throw new RuntimeException("error mapping Neo4j record to row", e); - } - }); + while (result.hasNext()) { + try { + processContext.output(rowMapper.mapRow(result.next())); + } catch (Exception e) { + throw new RuntimeException("Error mapping Neo4J result", e); + } + count++; + } + return count; }; if (logCypher) { @@ -846,13 +848,13 @@ record -> { if (driverSession.session == null) { throw new RuntimeException("neo4j session was not initialized correctly"); } else { - List outputs; + final Long count; if (writeTransaction) { - outputs = driverSession.session.writeTransaction(transactionWork, transactionConfig); + count = driverSession.session.writeTransaction(transactionWork, transactionConfig); } else { - outputs = driverSession.session.readTransaction(transactionWork, transactionConfig); + count = driverSession.session.readTransaction(transactionWork, transactionConfig); } - outputs.forEach(processContext::output); + LOG.debug("Retrieved " + count + " elements from Neo4J"); } } } From 7e95776a8d08ef738be49ef47842029c306f2bf5 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 2 Jan 2024 08:52:40 -0800 Subject: [PATCH 193/224] Exapnd docstring. --- sdks/python/apache_beam/options/pipeline_options.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 7e5954fa820d..030a25aa85c2 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -592,7 +592,9 @@ def _add_argparse_args(cls, parser): 'use pre-started services or non-default pre-existing artifacts to ' 'start the given service. ' 'Should be a json mapping of gradle build targets to pre-built ' - 'artifacts (e.g. jar files) expansion endpoints (e.g. host:port).')) + 'artifacts (e.g. jar files) or expansion endpoints ' + '(e.g. host:port). Defaults to the value of BEAM_SERVICE_OVERRIDES ' + 'from the environment.')) parser.add_argument( '--use_transform_service', From 62de5b89c40b4d99d0301dd737d7645030900a1e Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 2 Jan 2024 12:36:30 -0500 Subject: [PATCH 194/224] Revert "Bump com.gradle.enterprise from 3.15.1 to 3.16 (#29740)" (#29891) This reverts commit 543b013a548a933afa6886909a4c313e79d04b8d. --- settings.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index 8a60d377f41b..ef15f2074a3f 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -24,7 +24,7 @@ pluginManagement { } plugins { - id("com.gradle.enterprise") version "3.16" + id("com.gradle.enterprise") version "3.15.1" id("com.gradle.common-custom-user-data-gradle-plugin") version "1.12.1" } From 9f7ab3a6e8e37d303957b298377e7b668fe392f1 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 2 Jan 2024 10:06:17 -0800 Subject: [PATCH 195/224] [#29848][Go SDK] Fix exec side transform identification. (#29842) --- .github/trigger_files/beam_PostCommit_Go.json | 0 .../beam_PostCommit_Go_VR_Flink.json | 0 .../beam_PostCommit_Go_VR_Samza.json | 0 .../beam_PostCommit_Go_VR_Spark.json | 0 .../pkg/beam/core/runtime/exec/translate.go | 4 ++-- sdks/go/test/integration/integration.go | 19 ++++++------------- sdks/go/test/integration/primitives/timers.go | 3 ++- 7 files changed, 10 insertions(+), 16 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Go.json create mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Flink.json create mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Samza.json create mode 100644 .github/trigger_files/beam_PostCommit_Go_VR_Spark.json diff --git a/.github/trigger_files/beam_PostCommit_Go.json b/.github/trigger_files/beam_PostCommit_Go.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json b/.github/trigger_files/beam_PostCommit_Go_VR_Flink.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Samza.json b/.github/trigger_files/beam_PostCommit_Go_VR_Samza.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/.github/trigger_files/beam_PostCommit_Go_VR_Spark.json b/.github/trigger_files/beam_PostCommit_Go_VR_Spark.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index de1271ca878f..72af9e80c405 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -527,7 +527,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { u = &TruncateSizedRestriction{UID: b.idgen.New(), Fn: dofn, Out: out[0]} default: n := &ParDo{UID: b.idgen.New(), Fn: dofn, Inbound: in, Out: out} - n.PID = transform.GetUniqueName() + n.PID = id.to input := unmarshalKeyedValues(transform.GetInputs()) @@ -660,7 +660,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { } cn.UsesKey = typex.IsKV(in[0].Type) - cn.PID = transform.GetUniqueName() + cn.PID = id.to switch urn { case urnPerKeyCombinePre: diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index 452afd1768d1..160af1ab741f 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -103,6 +103,7 @@ var directFilters = []string{ "TestMapStateClear", "TestSetState", "TestSetStateClear", + "TestTimers.*", // no timer support for the go direct runner. } var portableFilters = []string{ @@ -125,19 +126,13 @@ var portableFilters = []string{ "TestFhirIO.*", // OOMs currently only lead to heap dumps on Dataflow runner "TestOomParDo", - // The portable runner does not support user state. - "TestValueState", - "TestValueStateWindowed", - "TestValueStateClear", - "TestBagState", - "TestBagStateClear", - "TestCombiningState", + // The portable runner does not support user map states. "TestMapState", "TestMapStateClear", "TestSetState", "TestSetStateClear", - // The portable runner does not appear to support timers. (extra elements) + // The portable runner does not uniquify timers. (data elements re-fired) "TestTimers.*", } @@ -188,8 +183,7 @@ var flinkFilters = []string{ "TestSetStateClear", "TestSetState", - // Flink does not appear to support timers. (missing timer elements) - "TestTimers.*", + "TestTimers_EventTime_Unbounded", // (failure when comparing on side inputs (NPE on window lookup)) } var samzaFilters = []string{ @@ -229,7 +223,7 @@ var samzaFilters = []string{ // TODO(https://github.com/apache/beam/issues/26126): Java runner issue (AcitveBundle has no regsitered handler) "TestDebeziumIO_BasicRead", - // Samza does not appear to support timers. (missing timer elements) + // Samza does not support state. "TestTimers.*", } @@ -263,8 +257,7 @@ var sparkFilters = []string{ "TestSetStateClear", "TestSetState", - // Spark does not appear to support timers. (Missing all elements) - "TestTimers.*", + "TestTimers_EventTime_Unbounded", // Side inputs in executable stage not supported. } var dataflowFilters = []string{ diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index 96588dca67e1..ab3454b64ce2 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -35,6 +35,7 @@ func init() { register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) register.Emitter2[string, int]() register.Emitter1[kv[string, int]]() + register.Iter1[int]() } type kv[K, V any] struct { @@ -80,7 +81,7 @@ func (fn *eventTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state. panic(err) } if !ok { - panic("State must be set.") + panic("State must be set for key: " + key) } emit(kvfn(read, fn.TimerOutput)) default: From a8f0586b6e24c174369c2e457c6568083a1ee8b5 Mon Sep 17 00:00:00 2001 From: JayajP Date: Tue, 2 Jan 2024 10:23:21 -0800 Subject: [PATCH 196/224] Record mean for underflow/overflow values in HistogramData. (#29808) * Record mean for underflow/overflow values in HistogramData. * Store the Sum instead of Mean for topRecords & bottomRecords and compute Mean when needed --- .../apache/beam/sdk/util/HistogramData.java | 44 +++++++++++++- .../beam/sdk/util/HistogramDataTest.java | 57 +++++++++++++++++-- 2 files changed, 94 insertions(+), 7 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java index e1e01950d656..b235e774feda 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/HistogramData.java @@ -46,7 +46,9 @@ public class HistogramData implements Serializable { private long[] buckets; private long numBoundedBucketRecords; private long numTopRecords; + private double topRecordsSum; private long numBottomRecords; + private double bottomRecordsSum; @GuardedBy("this") private double sumOfSquaredDeviations; @@ -64,7 +66,9 @@ public HistogramData(BucketType bucketType) { this.buckets = new long[bucketType.getNumBuckets()]; this.numBoundedBucketRecords = 0; this.numTopRecords = 0; + this.topRecordsSum = 0; this.numBottomRecords = 0; + this.bottomRecordsSum = 0; this.mean = 0; this.sumOfSquaredDeviations = 0; } @@ -151,7 +155,9 @@ public synchronized void update(HistogramData other) { } incTopBucketCount(other.numTopRecords); + this.topRecordsSum = other.topRecordsSum; incBottomBucketCount(other.numBottomRecords); + this.bottomRecordsSum = other.bottomRecordsSum; for (int i = 0; i < other.buckets.length; i++) { incBucketCount(i, other.buckets[i]); } @@ -181,7 +187,9 @@ public synchronized void clear() { this.buckets = new long[bucketType.getNumBuckets()]; this.numBoundedBucketRecords = 0; this.numTopRecords = 0; + this.topRecordsSum = 0; this.numBottomRecords = 0; + this.bottomRecordsSum = 0; this.mean = 0; this.sumOfSquaredDeviations = 0; } @@ -202,9 +210,9 @@ public synchronized void record(double value) { double rangeTo = bucketType.getRangeTo(); double rangeFrom = bucketType.getRangeFrom(); if (value >= rangeTo) { - numTopRecords++; + recordTopRecordsValue(value); } else if (value < rangeFrom) { - numBottomRecords++; + recordBottomRecordsValue(value); } else { buckets[bucketType.getBucketIndex(value)]++; numBoundedBucketRecords++; @@ -231,6 +239,30 @@ private synchronized void updateStatistics(double value) { sumOfSquaredDeviations += (value - mean) * (value - oldMean); } + /** + * Increment the {@code numTopRecords} and update {@code topRecordsSum} when a new overflow value + * is recorded. This function should only be called when a Histogram is recording a value greater + * than the upper bound of it's largest bucket. + * + * @param value + */ + private synchronized void recordTopRecordsValue(double value) { + numTopRecords++; + topRecordsSum += value; + } + + /** + * Increment the {@code numBottomRecords} and update {@code bottomRecordsSum} when a new underflow + * value is recorded. This function should only be called when a Histogram is recording a value + * smaller than the lowerbound bound of it's smallest bucket. + * + * @param value + */ + private synchronized void recordBottomRecordsValue(double value) { + numBottomRecords++; + bottomRecordsSum += value; + } + public synchronized long getTotalCount() { return numBoundedBucketRecords + numTopRecords + numBottomRecords; } @@ -260,10 +292,18 @@ public synchronized long getTopBucketCount() { return numTopRecords; } + public synchronized double getTopBucketMean() { + return numTopRecords == 0 ? 0 : topRecordsSum / numTopRecords; + } + public synchronized long getBottomBucketCount() { return numBottomRecords; } + public synchronized double getBottomBucketMean() { + return numBottomRecords == 0 ? 0 : bottomRecordsSum / numBottomRecords; + } + public synchronized double getMean() { return mean; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java index 8a518ec799ae..42d02541540c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/HistogramDataTest.java @@ -357,27 +357,74 @@ public void testStatistics_sumOfSquaredDeviations() { @Test public void testGetAndReset_resetSucceeds() { - HistogramData originalHistogram = HistogramData.linear(0, 10, 10); - originalHistogram.record(15.0, 25.0, 35.0, 45.0); + // Records values from [20, 50) in three buckets that have width 10. + HistogramData originalHistogram = HistogramData.linear(20, 10, 3); + originalHistogram.record(15.0, 25.0, 35.0, 45.0, 55.0); originalHistogram.getAndReset(); - HistogramData emptyHistogramData = HistogramData.linear(0, 10, 10); + HistogramData emptyHistogramData = HistogramData.linear(20, 10, 3); assertThat(originalHistogram, equalTo(emptyHistogramData)); assertThat(originalHistogram.getMean(), equalTo(0.0)); assertThat(originalHistogram.getSumOfSquaredDeviations(), equalTo(0.0)); + assertThat(originalHistogram.getTopBucketMean(), equalTo(0.0)); + assertThat(originalHistogram.getBottomBucketMean(), equalTo(0.0)); } @Test public void testGetAndReset_getSucceeds() { - HistogramData originalHistogram = HistogramData.linear(0, 10, 10); + // Records values from [20, 50) in three buckets that have width 10. + HistogramData originalHistogram = HistogramData.linear(20, 10, 3); originalHistogram.record(15.0, 25.0, 35.0, 45.0, 55.0); HistogramData copyHistogram = originalHistogram.getAndReset(); - HistogramData duplicateHistogram = HistogramData.linear(0, 10, 10); + HistogramData duplicateHistogram = HistogramData.linear(20, 10, 3); duplicateHistogram.record(15.0, 25.0, 35.0, 45.0, 55.0); assertThat(copyHistogram, equalTo(duplicateHistogram)); assertThat(copyHistogram.getBucketType(), equalTo(originalHistogram.getBucketType())); assertThat(copyHistogram.getMean(), equalTo(35.0)); assertThat(copyHistogram.getSumOfSquaredDeviations(), equalTo(1000.0)); + assertThat(copyHistogram.getTopBucketMean(), equalTo(55.0)); + assertThat(copyHistogram.getBottomBucketMean(), equalTo(15.0)); + } + + @Test + public void recordUnderflowValue() { + // 'histogram' to record values from [0, 32). Values outside this range are + // recorded in the overflow/underflow bin. + HistogramData histogram = HistogramData.exponential(0, 5); + + assertThat(histogram.getTopBucketCount(), equalTo(0L)); + assertThat(histogram.getTopBucketMean(), equalTo(0.0)); + + histogram.record(32); + assertThat(histogram.getTopBucketCount(), equalTo(1L)); + assertThat(histogram.getTopBucketMean(), equalTo(32.0)); + + histogram.record(40.0, 48.0, 56.0); + assertThat(histogram.getTopBucketCount(), equalTo(4L)); + assertThat(histogram.getTopBucketMean(), equalTo(44.0)); + } + + @Test + public void recordOverflowValue() { + // 'histogram' to record values from [50, 150). Values outside this range are + // recorded in the overflow/underflow bin. + HistogramData histogram = HistogramData.linear(50, 10, 10); + + assertThat(histogram.getBottomBucketCount(), equalTo(0L)); + assertThat(histogram.getBottomBucketMean(), equalTo(0.0)); + + histogram.record(-20); + assertThat(histogram.getBottomBucketCount(), equalTo(1L)); + assertThat(histogram.getBottomBucketMean(), equalTo(-20.0)); + + histogram.record(-30.0, -40.0); + assertThat(histogram.getBottomBucketCount(), equalTo(3L)); + assertThat(histogram.getBottomBucketMean(), equalTo(-30.0)); + + histogram.clear(); + histogram.record(25.0, 40.0); + assertThat(histogram.getBottomBucketCount(), equalTo(2L)); + assertThat(histogram.getBottomBucketMean(), equalTo(32.5)); } } From 9da9b8dd0e0e2d46dd81c5c2c961b759688a0ea4 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 2 Jan 2024 13:50:08 -0500 Subject: [PATCH 197/224] Clean up settings.kts (#29883) * Remove duplicated and no-longer-exist projects --- settings.gradle.kts | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index ef15f2074a3f..da170d0f8467 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -86,6 +86,7 @@ include(":examples:multi-language") include(":learning") include(":learning:tour-of-beam") include(":learning:tour-of-beam:frontend") +include(":learning:tour-of-beam:terraform") include(":model:fn-execution") include(":model:job-management") include(":model:pipeline") @@ -115,16 +116,10 @@ include(":it:truthmatchers") include(":it:mongodb") include(":it:splunk") include(":it:neo4j") - -include(":learning:tour-of-beam:frontend") - include(":runners:core-construction-java") include(":runners:core-java") include(":runners:direct-java") include(":runners:extensions-java:metrics") -include(":learning") -include(":learning:tour-of-beam") -include(":learning:tour-of-beam:terraform") /* Begin Flink Runner related settings */ // Flink 1.12 include(":runners:flink:1.12") @@ -317,7 +312,6 @@ include(":sdks:python:test-suites:tox:py311") include(":sdks:python:test-suites:xlang") include(":sdks:typescript") include(":sdks:typescript:container") -include(":vendor:bytebuddy-1_12_8") include(":vendor:grpc-1_54_0") include(":vendor:calcite-1_28_0") include(":vendor:guava-32_1_2-jre") From 1c1cfa85a9ebeed243dc569c3f744c219bf7437d Mon Sep 17 00:00:00 2001 From: Mark Zitnik Date: Tue, 2 Jan 2024 22:02:09 +0200 Subject: [PATCH 198/224] Add support for tuple ClickHouse (#29715) * First version WIP * Implement write tuple in RowBinary format * Added complex tuple test * Disable debug of javacc * Move tuple preprocessing logic * Adding to CHANGES.md & auto generated docs. * Fix CHANGES.md & fix docs * Fix spotless syntax * Remove :; from parsing. Only adding ' to field name. * Fix CHANGES.md to the correct version * Change new types javadoc --------- Co-authored-by: mzitnik --- CHANGES.md | 4 +- .../beam/sdk/io/clickhouse/ClickHouseIO.java | 19 ++++- .../sdk/io/clickhouse/ClickHouseWriter.java | 16 ++++ .../beam/sdk/io/clickhouse/TableSchema.java | 26 ++++++- .../src/main/javacc/ColumnTypeParser.jj | 47 +++++++++++ .../sdk/io/clickhouse/ClickHouseIOTest.java | 78 +++++++++++++++++++ .../sdk/io/clickhouse/TableSchemaTest.java | 51 ++++++++++++ 7 files changed, 238 insertions(+), 3 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index cbfb005d5ab1..3f48ebbf8795 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,8 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Adding support for Tuples DataType in ClickHouse (Java) ([#29715](https://github.com/apache/beam/pull/29715)). + ## New Features / Improvements @@ -98,10 +100,10 @@ * TextIO now supports skipping multiple header lines (Java) ([#17990](https://github.com/apache/beam/issues/17990)). * Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) -* Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). * Added support for handling bad records to KafkaIO (Java) ([#29546](https://github.com/apache/beam/pull/29546)) * Add support for generating text embeddings in MLTransform for Vertex AI and Hugging Face Hub models.([#29564](https://github.com/apache/beam/pull/29564)) * NATS IO connector added (Go) ([#29000](https://github.com/apache/beam/issues/29000)). +* Adding support for LowCardinality (Java) ([#29533](https://github.com/apache/beam/pull/29533)). ## New Features / Improvements diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java index 52d520a0a191..7ef643488a23 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java @@ -27,6 +27,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.stream.Collectors; @@ -109,6 +110,7 @@ * {@link TableSchema.TypeName#ENUM8} {@link Schema.TypeName#STRING} * {@link TableSchema.TypeName#ENUM16} {@link Schema.TypeName#STRING} * {@link TableSchema.TypeName#BOOL} {@link Schema.TypeName#BOOLEAN} + * {@link TableSchema.TypeName#TUPLE} {@link Schema.TypeName#ROW} * * * Nullable row columns are supported through Nullable type in ClickHouse. Low cardinality hint is @@ -475,6 +477,15 @@ abstract static class Builder { } } + private static String tuplePreprocessing(String payload) { + List l = + Arrays.stream(payload.trim().split(",")) + .map(s -> s.trim().replaceAll(" +", "' ")) + .collect(Collectors.toList()); + String content = + String.join(",", l).trim().replaceAll("Tuple\\(", "Tuple('").replaceAll(",", ",'"); + return content; + } /** * Returns {@link TableSchema} for a given table. * @@ -498,7 +509,13 @@ public static TableSchema getTableSchema(String jdbcUrl, String table) { String defaultTypeStr = rs.getString("default_type"); String defaultExpression = rs.getString("default_expression"); - ColumnType columnType = ColumnType.parse(type); + ColumnType columnType = null; + if (type.toLowerCase().trim().startsWith("tuple(")) { + String content = tuplePreprocessing(type); + columnType = ColumnType.parse(content); + } else { + columnType = ColumnType.parse(type); + } DefaultType defaultType = DefaultType.parse(defaultTypeStr).orElse(null); Object defaultValue; diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java index 8ed62eee3b59..c8c49a656e3b 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java @@ -21,9 +21,11 @@ import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.data.BinaryStreamUtils; import java.io.IOException; +import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.clickhouse.TableSchema.ColumnType; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.RowWithStorage; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Days; @@ -146,6 +148,20 @@ static void writeValue(ClickHouseOutputStream stream, ColumnType columnType, Obj case BOOL: BinaryStreamUtils.writeBoolean(stream, (Boolean) value); break; + case TUPLE: + RowWithStorage rowValues = (RowWithStorage) value; + List tupleValues = rowValues.getValues(); + Collection columnTypesList = columnType.tupleTypes().values(); + int index = 0; + for (ColumnType ct : columnTypesList) { + if (ct.nullable()) { + writeNullableValue(stream, ct, tupleValues.get(index)); + } else { + writeValue(stream, ct, tupleValues.get(index)); + } + index++; + } + break; } } diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java index 06ba2399a3cd..b89a88b3fae8 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; import org.checkerframework.checker.nullness.qual.Nullable; @@ -111,6 +112,14 @@ public static Schema.FieldType getEquivalentFieldType(ColumnType columnType) { return Schema.FieldType.STRING; case BOOL: return Schema.FieldType.BOOLEAN; + case TUPLE: + List fields = + columnType.tupleTypes().entrySet().stream() + .map(x -> Schema.Field.of(x.getKey(), Schema.FieldType.DATETIME)) + .collect(Collectors.toList()); + Schema.Field[] array = fields.toArray(new Schema.Field[fields.size()]); + Schema schema = Schema.of(array); + return Schema.FieldType.row(schema); } // not possible, errorprone checks for exhaustive switch @@ -168,7 +177,9 @@ public enum TypeName { // Composite type ARRAY, // Primitive type - BOOL + BOOL, + // Composite type + TUPLE } /** @@ -208,6 +219,7 @@ public abstract static class ColumnType implements Serializable { public static final ColumnType UINT32 = ColumnType.of(TypeName.UINT32); public static final ColumnType UINT64 = ColumnType.of(TypeName.UINT64); public static final ColumnType BOOL = ColumnType.of(TypeName.BOOL); + public static final ColumnType TUPLE = ColumnType.of(TypeName.TUPLE); // ClickHouse doesn't allow nested nullables, so boolean flag is enough public abstract boolean nullable(); @@ -220,6 +232,8 @@ public abstract static class ColumnType implements Serializable { public abstract @Nullable ColumnType arrayElementType(); + public abstract @Nullable Map tupleTypes(); + public ColumnType withNullable(boolean nullable) { return toBuilder().nullable(nullable).build(); } @@ -265,6 +279,14 @@ public static ColumnType array(ColumnType arrayElementType) { .build(); } + public static ColumnType tuple(Map elements) { + return ColumnType.builder() + .typeName(TypeName.TUPLE) + .nullable(false) + .tupleTypes(elements) + .build(); + } + /** * Parse string with ClickHouse type to {@link ColumnType}. * @@ -339,6 +361,8 @@ abstract static class Builder { public abstract Builder fixedStringSize(Integer size); + public abstract Builder tupleTypes(Map tupleElements); + public abstract ColumnType build(); } } diff --git a/sdks/java/io/clickhouse/src/main/javacc/ColumnTypeParser.jj b/sdks/java/io/clickhouse/src/main/javacc/ColumnTypeParser.jj index abe29aff3f8d..5bb9ba4171a6 100644 --- a/sdks/java/io/clickhouse/src/main/javacc/ColumnTypeParser.jj +++ b/sdks/java/io/clickhouse/src/main/javacc/ColumnTypeParser.jj @@ -17,6 +17,10 @@ */ options { IGNORE_CASE=true; + DEBUG_PARSER = false; + DEBUG_LOOKAHEAD = false; + DEBUG_TOKEN_MANAGER = false; + STATIC = false; } PARSER_BEGIN(ColumnTypeParser) @@ -99,6 +103,7 @@ TOKEN : | < EQ : "=" > | < BOOL : "BOOL" > | < LOWCARDINALITY : "LOWCARDINALITY" > + | < TUPLE : "TUPLE" > } public ColumnType columnType() : @@ -113,6 +118,7 @@ public ColumnType columnType() : | ct = array() | ct = nullable() | ct = lowcardenality() + | ct = tuple() ) { return ct; @@ -263,6 +269,33 @@ private Map enumElements() : } } +private Map.Entry tupleElement() : +{ + String key; + ColumnType value; + Token token; +} +{ + ( (key = string() ) ( value = columnType() ) ) { + return Maps.immutableEntry(key, value); + } +} + +private Map tupleElements() : +{ + Map.Entry el; + List> entries = Lists.newArrayList(); +} +{ + ( + ( el = tupleElement() { entries.add(el); } ) + ( ( el = tupleElement() { entries.add(el); } ) )* + ) + { + return ImmutableMap.copyOf(entries); + } +} + private ColumnType enum_() : { Map elements; @@ -289,4 +322,18 @@ private ColumnType lowcardenality() : ( ( (ct = primitive()) ) { return ct; } ) +} + +private ColumnType tuple() : +{ + Map elements; +} +{ + ( + ( ( elements = tupleElements() ) ) + { + return ColumnType.tuple(elements); + } + ) + } \ No newline at end of file diff --git a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java index 33fe9467d45b..b31a19236cb0 100644 --- a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java +++ b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java @@ -138,6 +138,84 @@ public void testArrayOfArrayOfInt64() throws Exception { assertEquals(15L, sum0); } + @Test + public void testTupleType() throws Exception { + Schema tupleSchema = + Schema.of( + Schema.Field.of("f0", FieldType.STRING), Schema.Field.of("f1", FieldType.BOOLEAN)); + Schema schema = Schema.of(Schema.Field.of("t0", FieldType.row(tupleSchema))); + Row row1Tuple = Row.withSchema(tupleSchema).addValue("tuple").addValue(true).build(); + + Row row1 = Row.withSchema(schema).addValue(row1Tuple).build(); + + executeSql( + "CREATE TABLE test_named_tuples (" + "t0 Tuple(`f0` String, `f1` Bool)" + ") ENGINE=Log"); + + pipeline.apply(Create.of(row1).withRowSchema(schema)).apply(write("test_named_tuples")); + + pipeline.run().waitUntilFinish(); + + try (ResultSet rs = executeQuery("SELECT * FROM test_named_tuples")) { + rs.next(); + assertEquals("('tuple',true)", rs.getString("t0")); + } + + try (ResultSet rs = executeQuery("SELECT t0.f0 as f0, t0.f1 as f1 FROM test_named_tuples")) { + rs.next(); + assertEquals("tuple", rs.getString("f0")); + assertEquals("true", rs.getString("f1")); + } + } + + @Test + public void testComplexTupleType() throws Exception { + Schema sizeSchema = + Schema.of( + Schema.Field.of("width", FieldType.INT64.withNullable(true)), + Schema.Field.of("height", FieldType.INT64.withNullable(true))); + + Schema browserSchema = + Schema.of( + Schema.Field.of("name", FieldType.STRING.withNullable(true)), + Schema.Field.of("size", FieldType.row(sizeSchema)), + Schema.Field.of("version", FieldType.STRING.withNullable(true))); + + Schema propSchema = + Schema.of( + Schema.Field.of("browser", FieldType.row(browserSchema)), + Schema.Field.of("deviceCategory", FieldType.STRING.withNullable(true))); + + Schema schema = Schema.of(Schema.Field.of("prop", FieldType.row(propSchema))); + + Row sizeRow = Row.withSchema(sizeSchema).addValue(10L).addValue(20L).build(); + Row browserRow = + Row.withSchema(browserSchema).addValue("test").addValue(sizeRow).addValue("1.0.0").build(); + Row propRow = Row.withSchema(propSchema).addValue(browserRow).addValue("mobile").build(); + Row row1 = Row.withSchema(schema).addValue(propRow).build(); + + executeSql( + "CREATE TABLE test_named_complex_tuples (" + + "`prop` Tuple(`browser` Tuple(`name` Nullable(String),`size` Tuple(`width` Nullable(Int64), `height` Nullable(Int64)),`version` Nullable(String)),`deviceCategory` Nullable(String))" + + ") ENGINE=Log"); + + pipeline.apply(Create.of(row1).withRowSchema(schema)).apply(write("test_named_complex_tuples")); + + pipeline.run().waitUntilFinish(); + + try (ResultSet rs = executeQuery("SELECT * FROM test_named_complex_tuples")) { + rs.next(); + assertEquals("(('test',(10,20),'1.0.0'),'mobile')", rs.getString("prop")); + } + + try (ResultSet rs = + executeQuery( + "SELECT prop.browser.name as name, prop.browser.size as size FROM test_named_complex_tuples")) { + rs.next(); + assertEquals("test", rs.getString("name")); + assertEquals("(10,20)", rs.getString("size")); + } + } + @Test public void testPrimitiveTypes() throws Exception { Schema schema = diff --git a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/TableSchemaTest.java b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/TableSchemaTest.java index 174761403471..f560d6268afb 100644 --- a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/TableSchemaTest.java +++ b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/TableSchemaTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.io.clickhouse.TableSchema.ColumnType; import org.apache.beam.sdk.schemas.Schema; @@ -196,4 +197,54 @@ public void testEquivalentSchema() { assertEquals(expected, TableSchema.getEquivalentSchema(tableSchema)); } + + @Test + public void testParseTupleSingle() { + Map m1 = new HashMap<>(); + m1.put("s", ColumnType.STRING); + ColumnType columnType01 = ColumnType.parse("Tuple('s' String)"); + assertEquals(ColumnType.tuple(m1), columnType01); + } + + @Test + public void testParseTupleDouble() { + Map m2 = new HashMap<>(); + m2.put("a1", ColumnType.STRING); + m2.put("b", ColumnType.BOOL); + ColumnType columnType02 = ColumnType.parse("Tuple('a1' String,'b' Bool)"); + assertEquals(ColumnType.tuple(m2), columnType02); + } + + @Test + public void testTupleNested() { + Map m1 = new HashMap<>(); + m1.put("a", ColumnType.STRING); + Map m3 = new HashMap<>(); + m3.put("a", ColumnType.STRING); + m3.put("b", ColumnType.BOOL); + m3.put("c", ColumnType.tuple(m1)); + ColumnType columnType03 = ColumnType.parse("Tuple('a' String,'b' Bool,'c' Tuple('a' String))"); + assertEquals(ColumnType.tuple(m3), columnType03); + } + + @Test + public void testTupleComplex() { + Map m1 = new HashMap<>(); + m1.put("width", ColumnType.INT64.withNullable(true)); + m1.put("height", ColumnType.INT64.withNullable(true)); + + Map m2 = new HashMap<>(); + m2.put("name", ColumnType.STRING.withNullable(true)); + m2.put("size", ColumnType.tuple(m1)); + m2.put("version", ColumnType.STRING.withNullable(true)); + + Map m3 = new HashMap<>(); + m3.put("browser", ColumnType.tuple(m2)); + m3.put("deviceCategory", ColumnType.STRING.withNullable(true)); + + ColumnType columnType03 = + ColumnType.parse( + "Tuple('browser' Tuple('name' Nullable(String),'size' Tuple('width' Nullable(Int64),'height' Nullable(Int64)),'version' Nullable(String)),'deviceCategory' Nullable(String))"); + assertEquals(ColumnType.tuple(m3), columnType03); + } } From 6751a1737c19d7022c30e850a718e73cd1909e30 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 2 Jan 2024 12:10:59 -0800 Subject: [PATCH 199/224] [#29895][Go SDK] Update timer test function names. (#29896) * Update timer test function names. * Remove leaked debug print. --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- sdks/go/test/integration/primitives/timers.go | 9 ++++----- sdks/go/test/integration/primitives/timers_test.go | 4 ++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index ab3454b64ce2..b4443296ecce 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -35,7 +35,6 @@ func init() { register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) register.Emitter2[string, int]() register.Emitter1[kv[string, int]]() - register.Iter1[int]() } type kv[K, V any] struct { @@ -139,13 +138,13 @@ func timersEventTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollection) } } -// TimersEventTime_Bounded validates event time timers in a bounded pipeline. -func TimersEventTime_Bounded(s beam.Scope) { +// TimersEventTimeBounded validates event time timers in a bounded pipeline. +func TimersEventTimeBounded(s beam.Scope) { timersEventTimePipelineBuilder(beam.Impulse)(s) } -// TimersEventTime_Bounded validates event time timers in an unbounded pipeline. -func TimersEventTime_Unbounded(s beam.Scope) { +// TimersEventTimeUnbounded validates event time timers in an unbounded pipeline. +func TimersEventTimeUnbounded(s beam.Scope) { timersEventTimePipelineBuilder(func(s beam.Scope) beam.PCollection { now := time.Now() return periodic.Impulse(s, now, now.Add(10*time.Second), 0, false) diff --git a/sdks/go/test/integration/primitives/timers_test.go b/sdks/go/test/integration/primitives/timers_test.go index 631ba41b91f2..01f2706c6293 100644 --- a/sdks/go/test/integration/primitives/timers_test.go +++ b/sdks/go/test/integration/primitives/timers_test.go @@ -24,12 +24,12 @@ import ( func TestTimers_EventTime_Bounded(t *testing.T) { integration.CheckFilters(t) - ptest.BuildAndRun(t, TimersEventTime_Bounded) + ptest.BuildAndRun(t, TimersEventTimeBounded) } func TestTimers_EventTime_Unbounded(t *testing.T) { integration.CheckFilters(t) - ptest.BuildAndRun(t, TimersEventTime_Unbounded) + ptest.BuildAndRun(t, TimersEventTimeUnbounded) } // TODO(https://github.com/apache/beam/issues/29772): Add ProcessingTime Timer tests. From 80450399472ea20542d9566a662a08aae0b951bf Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 3 Jan 2024 14:20:10 +0000 Subject: [PATCH 200/224] Add `svetakvsundhar` to Healthcare reviewer pool (#29899) * Create HealthcareUtils file with shared resources * revert * Request to be added to healthcare reviewer pool. --- .github/REVIEWERS.yml | 4 ++++ .github/autolabeler.yml | 1 + 2 files changed, 5 insertions(+) diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml index c2a57ac8feef..71692a7d4828 100644 --- a/.github/REVIEWERS.yml +++ b/.github/REVIEWERS.yml @@ -60,6 +60,10 @@ labels: - igorbernstein2 - mutianf exclusionList: [] + - name: healthcare + reviewers: + - svetakvsundhar + exclusionList: [] - name: Build reviewers: - damccorm diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml index 57c8f65c6ac1..a4ce41fce073 100644 --- a/.github/autolabeler.yml +++ b/.github/autolabeler.yml @@ -57,6 +57,7 @@ io: ["sdks/go/pkg/beam/io/**/*", "sdks/java/io/**/*", "sdks/python/apache_beam/ "hadoop": ["sdks/java/io/hadoop-common/**/*", "sdks/java/io/hadoop-file-system/**/*", "sdks/java/io/hadoop-format/**/*"] "hbase": ["sdks/java/io/hbase/**/*"] "hcatalog": ["sdks/java/io/hcatalog/**/*"] +"healthcare": ["sdks/python/apache_beam/io/gcp/healthcare/**/*", "sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/**/*"] "jdbc": ["sdks/java/io/jdbc/**/*"] "jms": ["sdks/java/io/jms/**/*"] "kafka": ["sdks/java/io/kafka/**/*"] From 6d534fdd802b93bcb1bd8f1cb9604276339aaf36 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 3 Jan 2024 09:35:17 -0500 Subject: [PATCH 201/224] Bump github.com/aws/aws-sdk-go-v2/config from 1.26.1 to 1.26.2 in /sdks (#29850) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.26.1 to 1.26.2. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/config/v1.26.1...config/v1.26.2) --- 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 | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 65d70cfe702b..1c5874a49c1e 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,8 +31,8 @@ require ( cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.36.0 github.com/aws/aws-sdk-go-v2 v1.24.0 - github.com/aws/aws-sdk-go-v2/config v1.26.1 - github.com/aws/aws-sdk-go-v2/credentials v1.16.12 + github.com/aws/aws-sdk-go-v2/config v1.26.2 + github.com/aws/aws-sdk-go-v2/credentials v1.16.13 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 github.com/aws/smithy-go v1.19.0 @@ -124,7 +124,7 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 // 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 492fa999d6d9..7dc312f9cba2 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,11 +86,11 @@ github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0o github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.26.1 h1:z6DqMxclFGL3Zfo+4Q0rLnAZ6yVkzCRxhRMsiRQnD1o= -github.com/aws/aws-sdk-go-v2/config v1.26.1/go.mod h1:ZB+CuKHRbb5v5F0oJtGdhFTelmrxd4iWO1lf0rQwSAg= +github.com/aws/aws-sdk-go-v2/config v1.26.2 h1:+RWLEIWQIGgrz2pBPAUoGgNGs1TOyF4Hml7hCnYj2jc= +github.com/aws/aws-sdk-go-v2/config v1.26.2/go.mod h1:l6xqvUxt0Oj7PI/SUXYLNyZ9T/yBPn3YTQcJLLOdtR8= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12 h1:v/WgB8NxprNvr5inKIiVVrXPuuTegM+K8nncFkr1usU= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12/go.mod h1:X21k0FjEJe+/pauud82HYiQbEr9jRKY3kXEIQ4hXeTQ= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13 h1:WLABQ4Cp4vXtXfOWOS3MEZKr6AAYUpMczLhgKtAjQ/8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13/go.mod h1:Qg6x82FXwW0sJHzYruxGiuApNo31UEtJvXVSZAXeWiw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= @@ -126,8 +126,8 @@ github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96 github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 h1:5UYvv8JUvllZsRnfrcMQ+hJ9jNICmcgKPAO1CER25Wg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 h1:HJeiuZ2fldpd0WqngyMR6KW7ofkXNLyOaHwEIGm39Cs= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From 36bdff42e8e47975252768b74c0a18072fec3fac Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 3 Jan 2024 07:34:48 -0800 Subject: [PATCH 202/224] Support key iteration in Python ULR. (#29839) This fixes #20878. --- runners/portability/java/build.gradle | 11 ----------- .../runners/portability/fn_api_runner/execution.py | 11 +++++++++++ .../portability/fn_api_runner/worker_handlers.py | 1 + 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle index fa7af9107260..69b93e9d3676 100644 --- a/runners/portability/java/build.gradle +++ b/runners/portability/java/build.gradle @@ -200,17 +200,6 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = "" // https://github.com/apache/beam/issues/20374 excludeTestsMatching 'org.apache.beam.sdk.transforms.CombineTest$BasicTests.testHotKeyCombiningWithAccumulationMode' - // https://github.com/apache/beam/issues/20878 - excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$MultipleInputsAndOutputTests.testSideInputAnnotationWithMultipleSideInputs' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testMapAsEntrySetSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedMultimapAsEntrySetSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedMapAsEntrySetSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testMultimapAsEntrySetSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMapSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMapSideInputWithNonDeterministicKeyCoder' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMultimapSideInput' - excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptyMultimapSideInputWithNonDeterministicKeyCoder' - for (String test : sickbayTests) { excludeTestsMatching test } diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py index fbcb58e16133..885c96146456 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py @@ -988,6 +988,9 @@ def commit_side_inputs_to_state( window=window)) self.state_servicer.append_raw(state_key, elements_data) elif func_spec.urn == common_urns.side_inputs.MULTIMAP.urn: + # TODO(robertwb): Consider computing these lazily on demand rather than + # anticipating all potentail state requests which will be more cpu and + # memory efficient for large side inputs. for (key, window, elements_data, elements_count) in elements_by_window.encoded_items(): state_key = beam_fn_api_pb2.StateKey( @@ -998,6 +1001,14 @@ def commit_side_inputs_to_state( key=key)) self.state_servicer.append_raw(state_key, elements_data) + key_iter_state_key = beam_fn_api_pb2.StateKey( + multimap_keys_side_input=beam_fn_api_pb2.StateKey. + MultimapKeysSideInput( + transform_id=consuming_transform_id, + side_input_id=tag, + window=window)) + self.state_servicer.append_raw(key_iter_state_key, key) + kv_iter_state_key = beam_fn_api_pb2.StateKey( multimap_keys_values_side_input=beam_fn_api_pb2.StateKey. MultimapKeysValuesSideInput( diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index b0421a6e43af..11c8ecd84f21 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -955,6 +955,7 @@ class StateServicer(beam_fn_api_pb2_grpc.BeamFnStateServicer, _SUPPORTED_STATE_TYPES = frozenset([ 'runner', 'multimap_side_input', + 'multimap_keys_side_input', 'multimap_keys_values_side_input', 'iterable_side_input', 'bag_user_state', From 3f3f64429fcc42b77c591261dc1677be9308472a Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Wed, 3 Jan 2024 11:00:55 -0500 Subject: [PATCH 203/224] fix indent (#29905) --- .github/REVIEWERS.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml index 71692a7d4828..b878d2f81cc8 100644 --- a/.github/REVIEWERS.yml +++ b/.github/REVIEWERS.yml @@ -63,7 +63,7 @@ labels: - name: healthcare reviewers: - svetakvsundhar - exclusionList: [] + exclusionList: [] - name: Build reviewers: - damccorm From ec3fe3145e82a625fa8f76343734b2b829da6f3d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 3 Jan 2024 13:28:33 -0500 Subject: [PATCH 204/224] Clean up java PreCommit (#29894) * Clean up java PreCommit * Move SQL tasks to SQL precommit for Java11/17 SQL precommit, still run a small group of tasks as before * Remove kotlin precommit from Java PreCommit Duplicate tests already run in Kotlin Example PreCommit * Arrange and remove duplicate tasks in javaPreCommit * Add timeout to java-fn-execution * Add JvmVerification to sql precommit * Remove jvmverification for :sqlPreCommit it is done by ":sdks:java:extensions:sql:preCommit" now --- .github/workflows/beam_PreCommit_Java.yml | 2 - .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- build.gradle.kts | 39 ++++++++----------- .../fnexecution/EmbeddedSdkHarness.java | 3 ++ ...GrpcContextHeaderAccessorProviderTest.java | 2 + .../fnexecution/ServerFactoryTest.java | 3 +- .../ArtifactRetrievalServiceTest.java | 2 + .../artifact/ArtifactStagingServiceTest.java | 2 + .../BundleFinalizationHandlersTest.java | 4 ++ .../control/DefaultJobBundleFactoryTest.java | 2 + .../FnApiControlClientPoolServiceTest.java | 3 ++ .../control/FnApiControlClientTest.java | 3 +- .../control/ProcessBundleDescriptorsTest.java | 3 ++ ...tingExecutableStageContextFactoryTest.java | 2 + .../control/RemoteExecutionTest.java | 3 +- .../control/SdkHarnessClientTest.java | 3 +- ...vironmentInstanceJobBundleFactoryTest.java | 3 ++ .../control/TimerReceiverFactoryTest.java | 4 ++ .../fnexecution/data/GrpcDataServiceTest.java | 3 ++ .../data/RemoteInputDestinationTest.java | 3 ++ .../environment/DockerCommandTest.java | 3 +- .../DockerEnvironmentFactoryTest.java | 3 +- .../ProcessEnvironmentFactoryTest.java | 4 +- .../environment/ProcessEnvironmentTest.java | 3 ++ .../environment/ProcessManagerTest.java | 3 ++ .../environment/RemoteEnvironmentTest.java | 3 ++ .../logging/GrpcLoggingServiceTest.java | 3 ++ .../StaticGrpcProvisionServiceTest.java | 4 ++ .../state/GrpcStateServiceTest.java | 3 ++ .../state/StateRequestHandlersTest.java | 4 ++ .../BeamWorkerStatusGrpcServiceTest.java | 2 + .../status/WorkerStatusClientTest.java | 4 +- .../BatchSideInputHandlerFactoryTest.java | 2 + .../PipelineTranslatorUtilsTest.java | 3 ++ .../fnexecution/wire/ByteStringCoderTest.java | 3 +- .../fnexecution/wire/CommonCoderTest.java | 3 ++ .../wire/LengthPrefixUnknownCodersTest.java | 3 ++ sdks/java/extensions/sql/build.gradle | 11 ++++++ 39 files changed, 121 insertions(+), 36 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index daee33b303ac..1614e9e64c05 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -26,7 +26,6 @@ on: - 'examples/kotlin/**' - 'release/**' - '.github/workflows/beam_PreCommit_Java.yml' - - '!sdks/java/extensions/avro/**' - '!sdks/java/extensions/sql/**' - '!sdks/java/io/amazon-web-services/**' - '!sdks/java/io/amazon-web-services2/**' @@ -79,7 +78,6 @@ on: - 'release/**' - 'release/trigger_all_tests.json' - '.github/trigger_files/beam_PreCommit_Java.json' - - '!sdks/java/extensions/avro/**' - '!sdks/java/extensions/sql/**' - '!sdks/java/io/amazon-web-services/**' - '!sdks/java/io/amazon-web-services2/**' diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index 5c2face871df..da28e6201f1a 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -92,7 +92,7 @@ jobs: - name: Build and Test uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sqlPreCommit + gradle-command: :sdks:java:extensions:sql:preCommit arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index de962ff7c16c..ef0b758ddd5b 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -90,7 +90,7 @@ jobs: - name: Build and Test uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sqlPreCommit + gradle-command: :sdks:java:extensions:sql:preCommit arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ diff --git a/build.gradle.kts b/build.gradle.kts index 511b537d7fe1..d4bd0fca4fc1 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -235,14 +235,14 @@ tasks.register("javaPreCommit") { dependsOn(":model:pipeline:build") dependsOn(":model:job-management:build") dependsOn(":model:fn-execution:build") - dependsOn(":runners:google-cloud-dataflow-java:worker:build") dependsOn(":sdks:java:core:buildNeeded") // Inline :sdks:java:core:buildDependents so we can carve out pieces at a time dependsOn(":beam-validate-runner:build") dependsOn(":examples:java:build") + dependsOn(":examples:java:preCommit") dependsOn(":examples:java:twitter:build") - dependsOn(":examples:kotlin:build") + dependsOn(":examples:java:twitter:preCommit") dependsOn(":examples:multi-language:build") dependsOn(":model:fn-execution:build") dependsOn(":model:job-management:build") @@ -250,6 +250,7 @@ tasks.register("javaPreCommit") { dependsOn(":runners:core-construction-java:build") dependsOn(":runners:core-java:build") dependsOn(":runners:direct-java:build") + dependsOn(":runners:direct-java:needsRunnerTests") dependsOn(":runners:extensions-java:metrics:build") // lowest supported flink version var flinkVersions = project.ext.get("allFlinkVersions") as Array<*> @@ -271,6 +272,7 @@ tasks.register("javaPreCommit") { dependsOn(":runners:spark:3:job-server:build") dependsOn(":runners:twister2:build") dependsOn(":sdks:java:build-tools:build") + dependsOn(":sdks:java:container:java8:docker") dependsOn(":sdks:java:core:build") dependsOn(":sdks:java:core:jmh:build") dependsOn(":sdks:java:expansion-service:build") @@ -289,16 +291,6 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:extensions:schemaio-expansion-service:build") dependsOn(":sdks:java:extensions:sketching:build") dependsOn(":sdks:java:extensions:sorter:build") - dependsOn(":sdks:java:extensions:sql:build") - dependsOn(":sdks:java:extensions:sql:datacatalog:build") - dependsOn(":sdks:java:extensions:sql:expansion-service:build") - dependsOn(":sdks:java:extensions:sql:hcatalog:build") - dependsOn(":sdks:java:extensions:sql:jdbc:build") - dependsOn(":sdks:java:extensions:sql:perf-tests:build") - dependsOn(":sdks:java:extensions:sql:shell:build") - dependsOn(":sdks:java:extensions:sql:udf-test-provider:build") - dependsOn(":sdks:java:extensions:sql:udf:build") - dependsOn(":sdks:java:extensions:sql:zetasql:build") dependsOn(":sdks:java:extensions:timeseries:build") dependsOn(":sdks:java:extensions:zetasketch:build") dependsOn(":sdks:java:fn-execution:build") @@ -312,6 +304,7 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:io:sparkreceiver:2:build") dependsOn(":sdks:java:io:synthetic:build") dependsOn(":sdks:java:io:xml:build") + dependsOn(":sdks:java:javadoc:allJavadoc") dependsOn(":sdks:java:testing:expansion-service:build") dependsOn(":sdks:java:testing:jpms-tests:build") dependsOn(":sdks:java:testing:load-tests:build") @@ -321,13 +314,6 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:testing:watermarks:build") dependsOn(":sdks:java:transform-service:build") dependsOn(":sdks:java:transform-service:launcher:build") - - dependsOn(":examples:java:preCommit") - dependsOn(":examples:java:twitter:preCommit") - dependsOn(":sdks:java:extensions:sql:jdbc:preCommit") - dependsOn(":sdks:java:javadoc:allJavadoc") - dependsOn(":runners:direct-java:needsRunnerTests") - dependsOn(":sdks:java:container:java8:docker") } // a precommit task build multiple IOs (except those splitting into single jobs) @@ -380,10 +366,18 @@ tasks.register("flinkPreCommit") { } tasks.register("sqlPreCommit") { - dependsOn(":sdks:java:extensions:sql:runBasicExample") - dependsOn(":sdks:java:extensions:sql:runPojoExample") - dependsOn(":sdks:java:extensions:sql:build") + dependsOn(":sdks:java:extensions:sql:preCommit") dependsOn(":sdks:java:extensions:sql:buildDependents") + dependsOn(":sdks:java:extensions:sql:datacatalog:build") + dependsOn(":sdks:java:extensions:sql:expansion-service:build") + dependsOn(":sdks:java:extensions:sql:hcatalog:build") + dependsOn(":sdks:java:extensions:sql:jdbc:build") + dependsOn(":sdks:java:extensions:sql:jdbc:preCommit") + dependsOn(":sdks:java:extensions:sql:perf-tests:build") + dependsOn(":sdks:java:extensions:sql:shell:build") + dependsOn(":sdks:java:extensions:sql:udf-test-provider:build") + dependsOn(":sdks:java:extensions:sql:udf:build") + dependsOn(":sdks:java:extensions:sql:zetasql:build") } tasks.register("javaPreCommitPortabilityApi") { @@ -728,7 +722,6 @@ if (project.hasProperty("testJavaVersion")) { tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") } else { allprojects { tasks.withType(Test::class).configureEach { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java index e9428cdc2c94..9a7cd2a9cff4 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java @@ -37,8 +37,10 @@ import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.Rule; import org.junit.rules.ExternalResource; import org.junit.rules.TestRule; +import org.junit.rules.Timeout; /** * A {@link TestRule} which creates a {@link FnHarness} in a thread, services required for that @@ -46,6 +48,7 @@ * during test execution. */ public class EmbeddedSdkHarness extends ExternalResource implements TestRule { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); public static EmbeddedSdkHarness create() { return new EmbeddedSdkHarness(); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java index 187c68decc9d..8455b23fee90 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java @@ -40,6 +40,7 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mockito; @@ -48,6 +49,7 @@ @RunWith(JUnit4.class) public class GrpcContextHeaderAccessorProviderTest { @Rule public GrpcCleanupRule cleanupRule = new GrpcCleanupRule().setTimeout(10, TimeUnit.SECONDS); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @SuppressWarnings("unchecked") @Test diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java index 6e19f9d6a7b4..0671fd1eb3ca 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java @@ -55,10 +55,11 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; /** Tests for {@link ServerFactory}. */ public class ServerFactoryTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final BeamFnApi.Elements CLIENT_DATA = BeamFnApi.Elements.newBuilder() .addData(BeamFnApi.Elements.Data.newBuilder().setInstructionId("1")) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java index bcce5f55a994..b7b7b1cfa58e 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java @@ -40,6 +40,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -51,6 +52,7 @@ public class ArtifactRetrievalServiceTest { private Path stagingDir; @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Before public void setUp() throws Exception { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java index 3f80c97f4336..51c0d99e5933 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java @@ -43,6 +43,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -56,6 +57,7 @@ public class ArtifactStagingServiceTest { private Path stagingDir; @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Before public void setUp() throws Exception { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java index 9494104b4c68..ea2f00e7e23b 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java @@ -24,13 +24,17 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.FinalizeBundleRequest; import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest; import org.apache.beam.runners.fnexecution.control.BundleFinalizationHandlers.InMemoryFinalizer; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link BundleFinalizationHandlers}. */ @RunWith(JUnit4.class) public class BundleFinalizationHandlersTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + @Test public void testInMemoryFinalizer() { InstructionRequestHandler mockHandler = mock(InstructionRequestHandler.class); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java index 7adc78a22ef8..823af895d655 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java @@ -77,6 +77,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.InOrder; @@ -88,6 +89,7 @@ @RunWith(JUnit4.class) public class DefaultJobBundleFactoryTest { @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock private EnvironmentFactory envFactory; @Mock private RemoteEnvironment remoteEnvironment; @Mock private InstructionRequestHandler instructionHandler; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java index b36cac381ac2..b605ae407e19 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java @@ -48,13 +48,16 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Unit tests for {@link FnApiControlClientPoolService}. */ @RunWith(JUnit4.class) public class FnApiControlClientPoolServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String WORKER_ID = "test_worker_id"; private final ControlClientPool pool = MapControlClientPool.create(); private final FnApiControlClientPoolService controlService = diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java index 30cd81ec5ba7..293f847bcc18 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java @@ -38,6 +38,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -47,7 +48,7 @@ /** Unit tests for {@link FnApiControlClient}. */ @RunWith(JUnit4.class) public class FnApiControlClientTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public ExpectedException thrown = ExpectedException.none(); @Mock public StreamObserver mockObserver; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java index 47df7b7c9735..108412a9dafb 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java @@ -60,13 +60,16 @@ import org.apache.beam.sdk.values.KV; 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.Iterables; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; /** Tests for {@link ProcessBundleDescriptors}. */ // TODO(https://github.com/apache/beam/issues/21230): Remove when new version of errorprone is // released (2.11.0) @SuppressWarnings("unused") public class ProcessBundleDescriptorsTest implements Serializable { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); /** * Tests that a stateful stage will wrap the key coder of a stateful transform in a diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactoryTest.java index be28ae06acc0..a7ba09f8fcec 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactoryTest.java @@ -28,12 +28,14 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link ReferenceCountingExecutableStageContextFactory}. */ @RunWith(JUnit4.class) public class ReferenceCountingExecutableStageContextFactoryTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public ExpectedLogs expectedLogs = diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index ea23e28ddb66..d827421d2e81 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -157,6 +157,7 @@ import org.junit.After; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.LoggerFactory; @@ -174,7 +175,7 @@ "unused" }) public class RemoteExecutionTest implements Serializable { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public transient ResetDateTimeProvider resetDateTimeProvider = new ResetDateTimeProvider(); private static final String WORKER_ID = "remote_test"; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java index 6ebe5eca3e65..2168854c863d 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java @@ -95,6 +95,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -110,7 +111,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class SdkHarnessClientTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock public FnApiControlClient fnApiControlClient; @Mock public FnDataService dataService; @Captor ArgumentCaptor> outputReceiverCaptor; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java index 11973385fe16..61090e9ffdf3 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java @@ -51,7 +51,9 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -60,6 +62,7 @@ /** Tests for {@link SingleEnvironmentInstanceJobBundleFactory}. */ @RunWith(JUnit4.class) public class SingleEnvironmentInstanceJobBundleFactoryTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock private EnvironmentFactory environmentFactory; @Mock private InstructionRequestHandler instructionRequestHandler; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactoryTest.java index 2d6ff53b78aa..14bbe751d3c5 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactoryTest.java @@ -20,12 +20,16 @@ import static org.junit.Assert.assertEquals; import org.apache.beam.sdk.values.KV; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class TimerReceiverFactoryTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + @Test public void testEncodeAndDecode() { KV expected = KV.of("123:\"ab\n'c:", "456:sdf:d"); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java index b04ef8881070..b06c38db7e74 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java @@ -54,13 +54,16 @@ import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link GrpcDataService}. */ @RunWith(JUnit4.class) public class GrpcDataServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String TRANSFORM_ID = "888"; private static final Coder> CODER = LengthPrefixCoder.of(WindowedValue.getValueOnlyCoder(StringUtf8Coder.of())); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/RemoteInputDestinationTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/RemoteInputDestinationTest.java index 9b2f919e8a05..40f48d012c31 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/RemoteInputDestinationTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/RemoteInputDestinationTest.java @@ -26,13 +26,16 @@ import org.apache.beam.sdk.coders.LengthPrefixCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.values.KV; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link RemoteInputDestination}. */ @RunWith(JUnit4.class) public class RemoteInputDestinationTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void testConstruction() { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerCommandTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerCommandTest.java index e7d294fffc5d..7d5af0514dac 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerCommandTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerCommandTest.java @@ -33,6 +33,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -40,7 +41,7 @@ @Category(NeedsDocker.class) @RunWith(JUnit4.class) public class DockerCommandTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public ExpectedException thrown = ExpectedException.none(); @Test diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java index 4b2948ecf9f8..16d32e46f010 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java @@ -48,6 +48,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -58,7 +59,7 @@ /** Tests for {@link DockerEnvironmentFactory}. */ public class DockerEnvironmentFactoryTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final ApiServiceDescriptor SERVICE_DESCRIPTOR = ApiServiceDescriptor.newBuilder().setUrl("service-url").build(); private static final String IMAGE_NAME = "my-image"; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java index efddc7226089..4ba993251739 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java @@ -40,7 +40,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.RemoteEnvironmentOptions; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -50,7 +52,7 @@ /** Tests for {@link ProcessEnvironmentFactory}. */ @RunWith(JUnit4.class) public class ProcessEnvironmentFactoryTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final ApiServiceDescriptor SERVICE_DESCRIPTOR = ApiServiceDescriptor.newBuilder().setUrl("service-url").build(); private static final String COMMAND = "my-command"; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentTest.java index aefcc2954a12..2b35d81a5300 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentTest.java @@ -22,13 +22,16 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link ProcessEnvironment}. */ @RunWith(JUnit4.class) public class ProcessEnvironmentTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void closeClosesInstructionRequestHandler() throws Exception { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java index 8108cd4349e4..4074bf949439 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java @@ -34,13 +34,16 @@ import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link ProcessManager}. */ @RunWith(JUnit4.class) public class ProcessManagerTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void testRunSimpleCommand() throws IOException { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java index af8a810cef77..9bb9eb616117 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java @@ -25,13 +25,16 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link RemoteEnvironment}. */ @RunWith(JUnit4.class) public class RemoteEnvironmentTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void closeClosesInstructionRequestHandler() throws Exception { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java index 7a29d02fd355..2fcbde5eaa31 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java @@ -41,13 +41,16 @@ import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link GrpcLoggingService}. */ @RunWith(JUnit4.class) public class GrpcLoggingServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private Consumer messageDiscarder = item -> { // Ignore diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java index bef964bf186b..f0d37fe34d3c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java @@ -34,13 +34,17 @@ import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Struct; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.Value; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link StaticGrpcProvisionService}. */ @RunWith(JUnit4.class) public class StaticGrpcProvisionServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + @Test public void returnsProvisionInfo() throws Exception { Struct options = diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java index 0342ddd67a55..fffc0321ca2c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java @@ -34,7 +34,9 @@ import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -47,6 +49,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class GrpcStateServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final long TIMEOUT_MS = 30 * 1000; private GrpcStateService stateService; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java index 5529133b5b49..3ef70756760a 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java @@ -39,7 +39,9 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mockito; @@ -50,6 +52,8 @@ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class StateRequestHandlersTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + @Test public void testDelegatingStateHandlerDelegates() throws Exception { StateRequestHandler mockHandler = Mockito.mock(StateRequestHandler.class); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java index e1d6401d5e53..49ced362c29a 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java @@ -48,6 +48,7 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -55,6 +56,7 @@ @RunWith(JUnit4.class) public class BeamWorkerStatusGrpcServiceTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); private static final String ID = "id"; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java index 49e621fb85fe..0a250ef0b8d3 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java @@ -29,7 +29,9 @@ import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -37,7 +39,7 @@ @RunWith(JUnit4.class) public class WorkerStatusClientTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock public StreamObserver mockObserver; private WorkerStatusClient client; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java index 1f459456d13b..a5f23db5b29e 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java @@ -55,6 +55,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mock; @@ -77,6 +78,7 @@ public class BatchSideInputHandlerFactoryTest { COLLECTION_ID, RunnerApi.PCollection.getDefaultInstance())))); @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock private BatchSideInputHandlerFactory.SideInputGetter context; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtilsTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtilsTest.java index a0fbd26d72f1..40e8ab96c615 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtilsTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtilsTest.java @@ -25,10 +25,13 @@ import java.util.Map; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; /** Tests for {@link PipelineTranslatorUtils}. */ public class PipelineTranslatorUtilsTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Test public void testOutputMapCreation() { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java index 4442674f4e6e..28144e68118a 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/ByteStringCoderTest.java @@ -36,13 +36,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Test case for {@link ByteStringCoder}. */ @RunWith(JUnit4.class) public class ByteStringCoderTest { - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final ByteStringCoder TEST_CODER = ByteStringCoder.of(); private static final List TEST_STRING_VALUES = diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index 389cdff232a2..5a775c2c98ec 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -99,7 +99,9 @@ import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -111,6 +113,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class CommonCoderTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String STANDARD_CODERS_YAML_PATH = "/org/apache/beam/model/fnexecution/v1/standard_coders.yaml"; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java index ddce63a19c99..86917e1c6bd4 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java @@ -37,7 +37,9 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -46,6 +48,7 @@ /** Tests for {@link LengthPrefixUnknownCoders}. */ @RunWith(Parameterized.class) public class LengthPrefixUnknownCodersTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static class UnknownCoder extends CustomCoder { private static final Coder INSTANCE = new UnknownCoder(); diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 8a22bff69b7f..6f34891c2d3f 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -240,6 +240,17 @@ task integrationTest(type: Test) { useJUnit { } } +task preCommit { + dependsOn build + dependsOn runBasicExample + dependsOn runPojoExample + + if (project.hasProperty("testJavaVersion")) { + var testVer = project.property("testJavaVersion") + dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + } +} + task postCommit { group = "Verification" description = "Various integration tests" From 2d7586b3526799123795ed333d6e3cee4c83a359 Mon Sep 17 00:00:00 2001 From: johnjcasey <95318300+johnjcasey@users.noreply.github.com> Date: Wed, 3 Jan 2024 13:31:51 -0500 Subject: [PATCH 205/224] update bq docs on stable schemas (#29904) * update bq docs on stable schemas * fix typo * fix typo * fix whitespace * move documentation to better location --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 8 ++++++-- .../en/documentation/io/built-in/google-bigquery.md | 2 ++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 686c29c13e33..4424b53f83ce 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2101,7 +2101,9 @@ static List getExtractFilePaths(String extractDestinationDir, Job ex *

      By default, tables will be created if they do not exist, which corresponds to a {@link * Write.CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's * Jobs API. A schema must be provided (via {@link Write#withSchema(TableSchema)}), or else the - * transform may fail at runtime with an {@link IllegalArgumentException}. + * transform may fail at runtime with an {@link IllegalArgumentException}. When updating a + * pipeline with a new schema, the existing schmea fields must stay in the same order, or the + * pipeline will break. * *

      By default, writes require an empty table, which corresponds to a {@link * Write.WriteDisposition#WRITE_EMPTY} disposition that matches the default of BigQuery's Jobs @@ -2539,7 +2541,9 @@ public enum WriteDisposition { * *

      Note from the BigQuery API doc -- Schema update options are supported in two cases: when * writeDisposition is WRITE_APPEND; when writeDisposition is WRITE_TRUNCATE and the destination - * table is a partition of a table, specified by partition decorators. + * table is a partition of a table, specified by partition decorators. When updating a pipeline + * with a new schema, the existing schmea fields must stay in the same order, or the pipeline + * will break. * * @see diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md index 769b05741345..64d82d8ab959 100644 --- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md +++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md @@ -502,6 +502,8 @@ fail later when the write attempts happen. If your BigQuery write operation creates a new table, you must provide schema information. The schema contains information about each field in the table. +When updating a pipeline with a new schema, the existing schema fields must +stay in the same order, or the pipeline will break, failing to write to BigQuery. {{< paragraph class="language-java" >}} To create a table schema in Java, you can either use a `TableSchema` object, or From fe1627db9472733fc42d6f027eb955ef55fece58 Mon Sep 17 00:00:00 2001 From: pablo rodriguez defino Date: Wed, 3 Jan 2024 10:58:40 -0800 Subject: [PATCH 206/224] [Blog Post] Scaling a Dataflow streaming workload (#29619) * Scaling streaming workload, article skeleton * first version with content * Update website/www/site/content/en/blog/scaling-streaming-workload.md Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * more fixes from review --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../en/blog/scaling-streaming-workload.md | 291 ++++++++++++++++++ .../scaling-streaming-workload/0-intro.png | Bin 0 -> 678815 bytes .../scaling-streaming-workload/0-pipeline.png | Bin 0 -> 41826 bytes .../1-default-ps.png | Bin 0 -> 80386 bytes .../1-default-throughput.png | Bin 0 -> 177003 bytes .../scaling-streaming-workload/2-skeys-ps.png | Bin 0 -> 95852 bytes .../2-skeys-throughput.png | Bin 0 -> 199363 bytes .../3-autoshard-autoscale.png | Bin 0 -> 94179 bytes .../3-autoshard-parallelism.png | Bin 0 -> 88744 bytes .../3-autoshard-throughput-tr.png | Bin 0 -> 149305 bytes .../3-autoshard-tr-input.png | Bin 0 -> 74404 bytes .../3-autoshard-tr-output.png | Bin 0 -> 100177 bytes .../3-autoshard-tr-overhead.png | Bin 0 -> 82261 bytes .../4-format-transformation.png | Bin 0 -> 27127 bytes .../4-ingestion-scale.png | Bin 0 -> 43609 bytes .../4-throughput.png | Bin 0 -> 27300 bytes .../5-format-trasformation.png | Bin 0 -> 20761 bytes .../5-ingestion-scale.png | Bin 0 -> 38156 bytes .../5-input-size.png | Bin 0 -> 31216 bytes .../5-output-size.png | Bin 0 -> 46655 bytes .../6-format-transformation.png | Bin 0 -> 21764 bytes .../6-ingestion-scale.png | Bin 0 -> 37699 bytes .../6-latencies.png | Bin 0 -> 38562 bytes .../7-current-input.png | Bin 0 -> 35557 bytes .../7-current-shuffle-total.png | Bin 0 -> 5101 bytes .../scaling-streaming-workload/7-latency.png | Bin 0 -> 32395 bytes .../7-previous-data-input.png | Bin 0 -> 34791 bytes .../7-previous-shuffle-total.png | Bin 0 -> 5388 bytes .../7-resources.png | Bin 0 -> 34148 bytes 29 files changed, 291 insertions(+) create mode 100644 website/www/site/content/en/blog/scaling-streaming-workload.md create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/0-intro.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/0-pipeline.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/1-default-ps.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/1-default-throughput.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-ps.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-throughput.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-autoscale.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-parallelism.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-throughput-tr.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-tr-input.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-tr-output.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-tr-overhead.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/4-format-transformation.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/4-ingestion-scale.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/4-throughput.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/5-format-trasformation.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/5-ingestion-scale.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/5-input-size.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/5-output-size.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/6-format-transformation.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/6-ingestion-scale.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/6-latencies.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-current-input.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-current-shuffle-total.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-latency.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-previous-data-input.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-previous-shuffle-total.png create mode 100644 website/www/site/static/images/blog/scaling-streaming-workload/7-resources.png diff --git a/website/www/site/content/en/blog/scaling-streaming-workload.md b/website/www/site/content/en/blog/scaling-streaming-workload.md new file mode 100644 index 000000000000..cbd642153f3c --- /dev/null +++ b/website/www/site/content/en/blog/scaling-streaming-workload.md @@ -0,0 +1,291 @@ +--- +layout: post +title: "Scaling a streaming workload on Apache Beam, 1 million events per second and beyond" +date: 2024-01-03 00:00:01 -0800 +categories: + - blog +authors: + - pabs +--- + + +# Scaling a streaming workload on Apache Beam + +Streaming Processing + +Scaling a streaming workload is critical for ensuring that a pipeline can process large amounts of data while also minimizing latency and executing efficiently. Without proper scaling, a pipeline may experience performance issues or even fail entirely, delaying the time to insights for the business. + +Given the Apache Beam support for the sources and sinks needed by the workload, developing a streaming pipeline can be easy. You can focus on the processing (transformations, enrichments, or aggregations) and on setting the right configurations for each case. + +However, you need to identify the key performance bottlenecks and make sure that the pipeline has the resources it needs to handle the load efficiently. This can involve right-sizing the number of workers, understanding the settings needed for the source and sinks of the pipeline, optimizing the processing logic, and even determining the transport formats. + +This article illustrates how to manage the problem of scaling and optimizing a streaming workload developed in Apache Beam and run on Google Cloud using Dataflow. The goal is to reach one million events per second, while also minimizing latency and resource use during execution. The workload uses Pub/Sub as the streaming source and BigQuery as the sink. We describe the reasoning behind the configuration settings and code changes we used to help the workload achieve the desired scale and beyond. + +The progression described in this article maps to the evolution of a real-life workload, with simplifications. After the initial business requirements for the pipeline were achieved, the focus shifted to optimizing the performance and reducing the resources needed for the pipeline execution. + +## Execution setup + +For this article, we created a test suite that creates the necessary components for the pipelines to execute. You can find the code in [this Github repository](https://github.com/prodriguezdefino/apache-beam-streaming-tests). You can find the subsequent configuration changes that are introduced on every run in this [folder](https://github.com/prodriguezdefino/apache-beam-streaming-tests/tree/main/scaling-streaming-workload-blog) as scripts that you can run to achieve similar results. + +All of the execution scripts can also execute a Terraform-based automation to create a Pub/Sub topic and subscription as well as a BigQuery dataset and table to run the workload. Also, it launches two pipelines: one data generation pipeline that pushes events to the Pub/Sub topic, and an ingestion pipeline that demonstrates the potential improvement points. + +In all cases, the pipelines start with an empty Pub/Sub topic and subscription and an empty BigQuery table. The plan is to generate one million events per second and, after a few minutes, review how the ingestion pipeline scales with time. The data being autogenerated is based on provided schemas or IDL (or Interface Description Language) given the configuration, and the goal is to have messages ranging between 800 bytes and 2 KB, adding up to approximately 1 GB/s volume throughput. Also, the ingestion pipelines are using the same worker type configuration on all runs (`n2d-standard-4` GCE machines) and are capping the maximum workers number to avoid very large fleets. + +All of the executions run on Google Cloud using Dataflow, but you can apply all of the configurations and format changes to the suite while executing on other supported Apache Beam runners. Changes and recommendations are not runner specific. + +### Local environment requirements + +Before launching the startup scripts, install the following items in your local environment: + +* `gcloud`, along with the correct permissions +* Terraform +* JDK 17 or later +* Maven 3.6 or later + +For more information, see the [requirements](https://github.com/prodriguezdefino/apache-beam-streaming-tests#requisites) section in the GitHub repository. + +Also, review the service quotas and resources available in your Google Cloud project. Specifically: Pub/Sub regional capacity, BigQuery ingestion quota, and Compute Engine instances available in the selected region for the tests. + +### Workload description + +Focusing on the ingestion pipeline, our [workload](https://github.com/prodriguezdefino/apache-beam-streaming-tests/blob/main/canonical-streaming-pipelines/src/main/java/com/google/cloud/pso/beam/pipelines/StreamingSourceToBigQuery.java#L55) is straightforward. It completes the following steps: + +1. reads data in a specific format from Pub/Sub (Apache Thrift in this case) +2. deals with potential compression and batching settings (not enabled by default) +3. executes a UDF (identity function by default) +4. transforms the input format to one of the formats supported by the `BigQueryIO` transform +5. writes the data to the configured table + +Example Workload + +The pipeline we used for the tests is highly configurable. For more details about how to tweak the ingestion, see the [options](https://github.com/prodriguezdefino/apache-beam-streaming-tests/blob/main/canonical-streaming-pipelines/src/main/java/com/google/cloud/pso/beam/pipelines/StreamingSourceToBigQuery.java#L39) in the file. No code changes are needed on any of our steps. The execution scripts take care of the configurations needed. + +Although these tests are focused on reading data from Pub/Sub, the ingestion pipeline is capable of reading data from a generic streaming source. The repository contains other [examples](https://github.com/prodriguezdefino/apache-beam-streaming-tests/tree/main/example-suite-scripts) that show how to launch this same test suite reading data from Pub/Sub Lite and Kafka. In all cases, the pipeline automation sets up the streaming infrastructure. + +Finally, you can see in the [configuration options](https://github.com/prodriguezdefino/apache-beam-ptransforms/blob/a0dd229081625c7b593512543614daf995a9f870/common/src/main/java/com/google/cloud/pso/beam/common/formats/options/TransportFormatOptions.java) that the pipeline supports many transport format options for the input, such as Thrift, Avro, and JSON. This suite focuses on Thrift, because it is a common open source format, and because it generates a format transformation need. The intent is to put some strain in the workload processing. You can run similar tests for Avro and JSON input data. The streaming data generator pipeline can generate random data for the [three supported formats](https://github.com/prodriguezdefino/apache-beam-streaming-tests/tree/main/streaming-data-generator/src/main/java/com/google/cloud/pso/beam/generator/formats) by walking directly on the schema (Avro and JSON) or IDL (Thrift) provided for execution. + +## First run: default settings + +The default values for the execution writes the data to BigQuery using `STREAMING_INSERTS` mode for `BigQueryIO`. This mode correlates with the [`tableData insertAll` API](https://cloud.google.com/bigquery/docs/reference/rest/v2/tabledata/insertAll) for BigQuery. This API supports data in JSON format. From the Apache Beam perspective, using the `BigQueryIO.writeTableRows` method lets us resolve the writes into BigQuery. + +For our ingestion pipeline, the Thrift format needs to be transformed into `TableRow`. To do that, we need to translate the Thrift IDL into a BigQuery table schema. That can be achieved by translating the Thrift IDL into an Avro schema, and then using Beam utilities to translate the table schema for BigQuery. We can do this at bootstrap. The schema transformation is cached at the `DoFn` level. + +After setting up the data generation and ingestion pipelines, and after letting the pipelines run for some minutes, we see that the pipeline is unable to sustain the desired throughput. + +PubSub metrics + +The previous image shows that the number of messages that are not being processed by the ingestion pipeline start to show as unacknowledged messages in Pub/Sub metrics. + +Throughput + +Reviewing the per stage performance metrics, we see that the pipeline shows a saw-like shape, which is often associated with the throttling mechanisms the Dataflow runner uses when some of the stages are acting as bottlenecks for the throughput. Also, we see that the `Reshuffle` step on the `BigQueryIO` write transform does not scale as expected. + +This behavior happens because by default the [`BigQueryOptions`](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java#L57) uses 50 different keys to shuffle data to workers before the writes happen on BigQuery. To solve this problem, we can add a configuration to our launch script that enables the write operations to scale to a larger number of workers, which improves performance. + + +## Second run: improve the write bottleneck + +After increasing the number of streaming keys to a higher number, 512 keys in our case, we restarted the test suite. The Pub/Sub metrics started to improve. After an initial ramp on the size of the backlog, the curve started to ease out. + +PubSub metrics + +This is good, but we should take a look at the throughput per stage numbers to understand if we are achieving the goal we set up for this exercise. + +Throughput + +Although the performance has clearly improved, and the Pub/Sub backlog no longer increases monotonically, we are still far from the goal of processing one million events per second (1 GB/s) for our ingestion pipeline. In fact, the throughput metrics jump all over, indicating that bottlenecks are preventing the processing from scaling further. + +## Third run: unleash autoscale + +Luckily for us, when writing into BigQuery, we can autoscale the writes. This step simplifies the configuration so that we don't have to guess the right number of shards. We switched the pipeline’s configuration and enabled this setting for the next [launch script](https://github.com/prodriguezdefino/apache-beam-streaming-tests/blob/main/scaling-streaming-workload-blog/3-ps2bq-si-tr-streamingautoshard.sh). + +Key Parallelism + +Immediately, we see that the autosharding mechanism tweaks the number of keys very aggressively and in a dynamic way. This change is good, because different moments in time might have different scale needs, such as early backlog recoveries and spikes in the execution. + +Throughput + +Inspecting the throughput performance per stage, we see that as the number of keys increases, the performance of the writes also increases. In fact, it reaches very large numbers! + +After the initial backlog was consumed and the pipeline stabilized, we saw that the desired performance numbers were reached. The pipeline can sustain processing many more than a million events per second from Pub/Sub and several GB/s of BigQuery ingestion. Yay! + +Still, we want to see if we can do better. We can introduce several improvements to the pipeline to make the execution more efficient. In most cases, the improvements are configuration changes. We just need to know where to focus next. + +Resources + +The previous image shows that the number of workers needed to sustain this throughput is still quite high. The workload itself is not CPU intensive. Most of the cost is spent on transforming formats and on I/O interactions, such as shuffles and the actual writes. To understand what to improve, we first investigate the transport formats. + +Thrift Input Size +TableRow Output Size + +Looking at the input size, right before the identity UDF execution, the data format is binary Thrift, which is a decently compact format even when no compression is used. However, while comparing the `PCollection` approximated size with the `TableRow` format needed for BigQuery ingestion, a clear size increase is visible. This is something we can improve by changing the BigQuery write API in use. + +Translation Overhead + +When we inspect the `StoreInBigQuery` transform, we see that the majority of the wall time is spent on the actual writes. Also, the wall time spent converting data to the destination format (`TableRows`) compared with how much is spent in the actual writes is quite large: 13 times bigger for the writes. To improve this behavior, we can switch the pipeline write mode. + +## Fourth run: in with the new + +In this run, we use the `StorageWrite` API. Enabling the `StorageWrite` API for this pipeline is straightforward. We set the write mode as `STORAGE_WRITE_API` and define a write triggering frequency. For this test, we write data at most every ten seconds. The write triggering frequency controls how long the per-stream data accumulate. A higher number defines a larger output to be written after the stream assignment but also imposes a larger end-to-end latency for every element read from Pub/Sub. Similar to the `STREAMING_WRITES` configuration, `BigQueryIO` can handle autosharding for the writes, which we already demonstrated to be the best setting for performance. + +After both pipelines become stable, the performance benefits seen when using the `StorageWrite` API in `BigQueryIO` are apparent. After enabling the new implementation, the wall time rate between the format transformation and write operation decreases. The wall time spent on writes is only about 34 percent larger than the format transformation. + +Translation Overhead + +After stabilization, the pipeline throughput is also quite smooth. The runner can quickly and steadily downscale the pipeline resources needed to sustain the desired throughput. + +Throughput + +Looking at the resource scale needed to process the data, another dramatic improvement is visible. Whereas the streaming inserts-based pipeline needed more than 80 workers to sustain the throughput, the storage writes pipeline only needs 49, a 40 percent improvement. + +Resources + +We can use the data generation pipeline as reference. This pipeline only needs to randomly generate data and write the events to Pub/Sub. It runs steadily with an average of 40 workers. The improvements on the ingestion pipeline using the right configuration for the workload makes it closer to those resources needed for the generation. + +Similar to the streaming inserts-based pipeline, writing the data into BigQuery requires running a format translation, from Thrift to `TableRow` in the former and from Thrift to Protocol Buffers (protobuf) in the latter. Because we are using the `BigQueryIO.writeTableRows` method, we add another step in the format translation. Because the `TableRow` format also increases the size of the `PCollection` being processed, we want to see if we can improve this step. + + +## Fifth run: a better write format + +When using `STORAGE_WRITE_API`, the `BigQueryIO` transform exposes a method that we can use to write the Beam row type directly into BigQuery. This step is useful because of the flexibility that the row type provides for interoperability and schema management. Also, it's both efficient for shuffling and denser than `TableRow`, so our pipeline will have smaller `PCollection` sizes. + +For the next run, because our data volume is not small, we decrease the triggering frequency when writing to BigQuery. Because we use a different format, slightly different code runs. For this change, the test pipeline script is configured with the flag `--formatToStore=BEAM_ROW`. + +Thrift input size +Row output size + +The `PCollection` size written into BigQuery is considerably smaller than on previous executions. In fact, for this particular execution, the Beam row format is a smaller size than the Thrift format. A larger `PCollection` conformed by bigger per-element sizes can put nontrivial memory pressure in smaller worker configurations, reducing the overall throughput. + +Translation overhead + +The wall clock rate for the format transformation and the actual BigQuery writes also maintain a very similar rate. Handling the Beam row format does not impose a performance penalty in the format translation and subsequent writes. This is confirmed by the number of workers in use by the pipeline when throughput becomes stable, slightly smaller than the previous run but clearly in the same range. + +Resources + +Although we are in a much better position than when we started, given our test pipeline input format, there's still room for improvement. + +## Sixth run: further reduce the format translation effort + +Another supported format for the input `PCollection` in the `BigQueryIO` transform might be advantageous for our input format. The method `writeGenericRecords` enables the transform to transform Avro `GenericRecords` directly into protobuf before the write operation. Apache Thrift can be transformed into Avro `GenericRecords` very efficiently. We can make another test run configuring our test ingestion pipeline by setting the option `--formatToStore=AVRO_GENERIC_RECORD` on our execution script. + +This time, the difference between format translation and writes increases significantly, improving performance. The translation to Avro `GenericRecords` is only 20 percent of the write effort spent on writing those records into BigQuery. Given that the test pipelines had similar runtimes and that the wall clock seen in the `WriteIntoBigQuery` stage is also aligned with other `StorageWrite` related runs, using this format is appropriate for this workload. + +Translation overhead + +We see further gains when we look at resource utilization. We need less CPU time to execute the format translations for our workload while achieving the desired throughput. + +Resources + +This pipeline improves upon the previous run, running steadily on 42 workers when throughput is stable. Given the worker configuration used (`nd2-standard-4`), and the volume throughput of the workload process (about 1 GB/s), we are achieving about 6 MB/s throughput per CPU core, which is quite impressive for a streaming pipeline with exactly-once semantics. + +Latencies + +When we add up all of the stages executed in the main path of the pipeline, the latency seen at this scale achieves sub-second end-to-end latencies during sustained periods of time. + +Given the workload requirements and the implemented pipeline code, this performance is the best that we can extract without further tuning the runner’s specific settings. + +## Seventh run : lets just relax (at least some constraints) + +When using the `STORAGE\_WRITE\_API` setting for `BigQueryIO`, we enforce exactly-once semantics on the writes. This configuration is great for use cases that need strong consistency on the data that gets processed, but it imposes a performance and cost penalty. + +From a high-level perspective, writes into BigQuery are made in batches, which are released based on the current sharding and the triggering frequency. If a write fails during the execution of a particular bundle, it is retried. A bundle of data is committed into BigQuery only when all the data in that particular bundle is correctly appended to a stream. This implementation needs to shuffle the full volume of data to create the batches that are written, and also the information of the finished batches for later commit (although this last piece is very small compared with the first). + +Read data size + +Looking at the previous pipeline execution, the total data being processed for the pipeline by Streaming Engine is larger than the data being read from Pub/Sub. For example, 7 TB of data is read from Pub/Sub, whereas the processing of data for the whole execution of the pipeline moves 25 TB of data to and from Streaming Engine. + +Streamed data size + +When data consistency is not a hard requirement for ingestion, you can use at-least-once semantics with `BigQueryIO` write mode. This implementation avoids shuffling and grouping data for the writes. However, this change might cause a small number of repeated rows to be written into the destination table. This can happen with append errors, infrequent worker restarts, and other even less frequent errors. + +Therefore, we add the configuration to use `STORAGE_API_AT_LEAST_ONCE` write mode. To instruct the `StorageWrite` client to reuse connections while writing data, we also add the configuration flag `–useStorageApiConnectionPool`. This configuration option only works with `STORAGE_API_AT_LEAST_ONCE` mode, and it reduces the occurrences of warnings similars to `Storage Api write delay more than 8 seconds`. + +Resources + +When pipeline throughput stabilizes, we see a similar pattern for resource utilization for the workload. The number of workers in use reaches 40, a small improvement compared with the last run. However, the amount of data being moved from Streaming Engine is much closer to the amount of data read from Pub/Sub. + + +Read data size +Streamed data size + +Considering all of these factors, this change further optimizes the workload, achieving a throughput of 6.4 MB/s per CPU core. This improvement is small compared to the same workload when using consistent writes into BigQuery, but it uses less streaming data resources. This configuration represents the most optimal setup for our workload, with the highest throughput per resource and the lowest streaming data across workers. + +Streamed data size + +This configuration also has impressively low latency for the end-to-end processing. Given that the main path of our pipeline has been fused in a single execution stage from reads to writes, we see that even at p99, the latency tends to be below 300 milliseconds at a quite large volume throughput (as previously mentioned around 1 GB/s). + +## Recap + +Optimizing Apache Beam streaming workloads for low latency and efficient execution requires careful analysis and decision-making, and the right configurations. + +Considering the scenario discussed in this article, it is essential to consider factors like overall CPU utilization, throughput and latency per stage, `PCollection` sizes, wall time per stage, write mode, and transport formats, in addition to writing the right pipeline for the workload. + +Our experiments revealed that using the `StorageWrite` API, autosharding for writes, and Avro `GenericRecords` as the transport format yielded the most efficient results. Relaxing the consistency for writes can further improve performance. + +The accompanying [Github repository](https://github.com/prodriguezdefino/apache-beam-streaming-tests) contains a test suite that you can use to replicate the analysis on your Google Cloud project or with a different runner setup. Feel free to take it for a spin. Comments and PRs are always welcome. diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/0-intro.png b/website/www/site/static/images/blog/scaling-streaming-workload/0-intro.png new file mode 100644 index 0000000000000000000000000000000000000000..7e29be96bb68f40549ce50c3b2c6dd8665433cbf GIT binary patch literal 678815 zcmZVl1yGzpvo8)W?(S~E-Q9vq&;&?u7I#_P3C^-WaCZv?g1f^m!QB#qySu|b@44sR zbL#u1YHE7A``7(UPu0xSGd(d{8cJB`SwsW)s09cZMW@Z*D94y1;=4NKYH?%q1l(O@03;ECRWUYx;t z%$b;9Lr4%fI4K!MOdfKgfDULcY7zHQ*qC2OK7VU7`&8$j8|BI7aHz8S4NC|+W0MG( z@*AARkL|D`d}L%rEL!n`&Jw_24WM2jHUt7+?=N!&P6nLfvZA8b4RY%1T4e< zB8y>_xy7P2WJXhwpo+=O*3H87L>Ru2sPrxDlgYw~wj#|TQzzvh3nTp#`fdtIWG+i) zewUw+=OZ5>uO(jzza>u!4-1b7yaGC@JE=~nqh_Q;q$RQ>G8OIb=srMnysSBx+6%P~ z*%TSMN>rw*6W5|I78CP#EGa3XC$7Z^nap00g_+3DAGCEWc+^GUHG|k$a6GLebpiDP z&D&dWnw6Q^+EDL#%l_Njn#bGQTSs_k=mV3M3mw4b%$zF$l;Hd?(t1_~D%Kx90NDP~ zr~tSqI{@N83hrMe|5pJ3gl~}mlz%P$zban<|9_?Mt_2AHmxgQlkD{!uoQlf7Ru|}Q zW##N)>jJ8QxE}m7wP5$r0A%n%T@2{r#AR;jVqwMQY>eZ*MMdel8bx8*W}vQBiIlK5jlf&VL%59=^^Xb01D;4~G9^c$iq{^Q-#)8?@UOCpfrtQoX@i zdI;hc*T5Kyr}8~UF(%~{ak{e?uXjBJU9!@LJ`eRgC1>OtMt?m?=jD|oY9Qf)PQPlX3zq{o zLAWO*cSZGY^m@cL#eo5ox<*&C@gx^dZ#zBFQ4~)X`oRymtrw3+(^Mef&Et;@7Eg=2 z0qM9Sf^zWDCk~?^{=Adi!Q;&cgSEd9ahD)2sr&t8QU`U|?J4qBKE%dBlk~}wqsAn@ zEp8F$U?W`fH;5rG;E~5-D(1p|FhWr@FYr2f|NTJi+g>nkHi{DOXQx5amtQT38}kO; z9_)IxCQ;m}OMw^kdV9oMr+2}WE^EG$Qje$kgYpi{k%p5N%_b0&r+3IFLiJz3hlfo| zl=3NLE*-qoUoRd+L`Vk*$3SBy-MC-%xP#93g2`%oyqJlz=6$Y=(~FdVkv87`ty^!GypU>UOQTRPoxOs#QJpF;1nP*%%b*N0St<7K8B|L+0yJL(Pw7UmJNN}*Y!S791DI;FXkJ)$HoS} zyxajX^Umg(H#eJ(+>UOWd>*EsF3R7=gQ;3b%7Ez95^Iv+-#w4xt+yDa32zTQPx+fG zI}X}@HyqA1)v&RPSyJ}9&Fy{Knpz|9Uai-bnLVrX>tE@Fi4nsK`#q~`Y>mywh)PtS ztM$Lhq#f=IE2&>7y93Dl6KIB>Q;)Pio5XNE`(19EJl{$`=K9>byr%C5KLW3d@Nk;% zHg_rV!p+l}qcU*7MA0@YKBdfJ80N*u6B`3^ZX8?sj)fk9Zc6fX@uoC1nBq~aN=J5r=3{* zcddoD2QxHL6Q|>L_AHP&XzwnF`*qXsP2Z>^h#*Uf1c!Pc+np6kHt@V@)QEjo3=#tm zymYnR?o#k_U;&8@XPjwVmuZ~~M&B-b9!Yq4p#g_C@M^^6<>lI&#yEM9$ua0mYW77$ z?Te~}98>nv4~#gHt2g)O@EoOSXR4vi znTQI_NvQUdNcXct58mCWRM88cKuYl!sd(pdq8)cpCn)aIwkDd`qp>5)8kSPA&(o-B z3~X`%NcnO=dBW4Y%90T2CPwM{fmw@irDE_xo1F^?4BDVri^h>|qqrKani3|CBT2J( zGn2Nidte@$wl5`zJ;CbQnlo=p<1|ILpMF)wRK)i|M|gC?;O@gjZ?63EO-+BFG7 z$Ik&dImhf}6(1E{EAl(D2WiGlTB*k?BS?8~jgMi|uX@j$wQuO{exl9ZCUn{8Vym#d z)Xn>X_?+{Iu{N~lRk)zq$T!DgicgL@Z8{Kizj)j*)^IMYI>Ay~A$zxnk)5SgAL`s2 zn~T@{mnxCJLHNbm3Hf+|U`s~sFXi!k;Si07*){FbRW`r1gxcrZV7#lAo>(dg2;QQm zhWa{A{?IE_51etN^m}s4_31OMykj61In*M+k5yyzCd06eTz~CVy|IoxzRo(}y7&xw zv*-*x9CzT0aLXUuZHO0Hc5!pzXR@}j@G=+r4%}Ii4j@W*F^PcAHWI1DCD-{Kyno*A zpc#VBQms^#o7z#G@a`E7cjI|F&eL_fTjRfV+|fgDaJFu^0|dKoJ9;RXyL1Ac-k!Ja zuTl0r&O{zhj05`MfTs#A-sI_AYcZ+w6m^i-N$vu1*gm$vNwEzEyYSX!{N%bc&CtnY zH}zT24aPK;*FiCbc&&zozXt{NF<(MiFS35IghafJ_Zf8akEO&A_gobb==i;ak1Kcg zbCqeru1M^nC|LU0<&E(qkglfdWrtL+%mO|)_`GEe^27g3>ByEHc`Sii%v7$Mb9e4} zGhmFH6M8-Ney;8_^NJRzCj3W+%$=!|6K$ldQP(SZSwosn>Dt*K+U*A6!?k~IMS@^a z45r$1Hj=nR;3JQQ+P*eW=wx)82@TrwbeqjP zAn*6OZ{Fyt)%D7vSMxV)A3u^>E|gx_o@HYLF1CJj+vSQ33Lp8A_VutS+fD-FM$FJ_ zj#SJ;iCt{O3A^8xKR!Jvm}A%%=}!t_;d!2AQL&qfyHU!AZd74`{}}yg3zY1>W(;@r z6nuTjf6exJJWn<-F^b0D%77(-amz4k3NZ~>wiW#w1r>i|fnnqGI~i0}ik>0B!B&vJ zF~D5N#vo1K{`|k!i*ao)h->!i}dc@&dtkI`8t(Bn(%}}16|_i zCH*e0e(PL{SN?2`O_$Cm^K701$KaGB*Yvfa&s;cjuw#;XXhlgnjsfs1JKvvKO5p+u zFd>q86gepZ)M0%%`TDh@fA7bSbhn+u9RW6t3^;hWH51jUT>hYHg-hAIL{f21GP!S^J1o4$vt9Crr5MOX0sBAJps{n|`+A61^f zhiy)-CVpvBBf1kvlG_eC(WA`Ob6WuNrua@({sjY<4yRU8K;R8mCcu~S;rfqG(3{`h zMe%*&%WD&L3wAEv4*b*s&vZ5&X5xzm8}K}1OB8=yDg-#(nLrnCD(eH*He zRKC}$w!{smQ?>d_=xGEJS9*~mO2ow;em$%}__?hq|md_~G@ zvaAagLiK|2Lc3jlF@Va#ura(8`DnhNOGae=(Z&2C=97?%=+8)ya>aU?cw2vcRf{rJ zRY-HG7i1ep`IX!5whMUfZ|b=ldWX7czaI|N7T&ISLzI49(x-X3eJqikOWH3E_mEA` z>{zi78+K?*J$eej?%7aK*-EXafYU>ELoCS3H8uM)Mq(?SI{A!hS&w6=WUW&B9J-4R zyx^CO3)K6G!U$cG)96ND9{#zxHg7#0W61jTZGiRpT51Rou^@@R?d{*NNlACv*VaT7VEAPt7y%l)nkLmgws>Vl3yM( zX|~U1-E6w%YQ0q9$m}78?xLP@yg@9A0X+2f_Jq9f#VQ>i{uV!D#UaiNLu;Ct91Jam& zrwfos3SJBTD&fBQ{JW{A#ryU%TxV%8rmu38L{;VFCC=vyn`VB??gtaY-HMl*BGZ1Q zdy^n>A*9UK0( z;+vg>mSO~AFfjP;6MTXjIFF!U=urLVN8mYnkhL?{N=PSOSOwoSlkBWh&K~9ogp|}8 zk9nbw&J#~4k_P4SVl60P4OXWy=VXs>Ir=UKO4o#QR}|-w7nRBIIDMZbBe z2s0TN3vZ!D30i@|Sk%c7B&)v3JWZTxKOdh0p@KC2C;$?@WE7aB5862Y)pTF9mm7V{ z+UxrG#>`X&vSV-f#JA zanc7Q(8!bU_@a+P+I}6%Xz7NzZ2{8J^h7?{lIkTP;ngxQxn%y@YGrQSb#)L=+fCoV z(blD$EliTt`&oYEFdcj=kXA&-_e0NDiRD=*21^-&kJ4zcChseziq#=P8-f|a-3 zJ)Cm)XGKPWzY(0{O5Q%APuTg{Gcms?FyLM5Be{~!lU=L=aw+ib4D!h1oUV5a-yVaJ zEIVe1MBrvz2Ms4%TwBXI)S*Ch$&7HAXAupNr6|3hk&Vv3gJ7a!WxTy6E+gE~AnBz; ziYwFDEYr*UzDsxz^;u^$hDb}22L9`z%y-G&i^pe>#=P$C#IzSE`n%M_Ju^U}h>0|vFVG=F zO!4!jl=JqP?q3JOc^9gbiG`!*@=h>dpF>cQKwb^>v3U`G&M(;fdfMDkA!^?40|l=( z2d%yQHN{kIF`gsI1VoDN3qw^OLE;=~g4hpjxyso>`~67r_Ri+d=Q0eFo^s%N&|G}Y zMRA{6K}fZKwewEChVmmjso_C&y~woaTv9mzHVMiSweD^9%Yc$lqxQBhkshSo6&d92 zZqHfHICdzWfl}b3sBf!)A7kNA9Vrv8M+e}rC+j|y==z8D-Kw=u9wDhPBzA>BaR@+l z`zbIqfkHKMPx2AyQAz?xR0u{7xDY=}BK9pirNl*3w?U)2TzLXfv6|Kp{rrw$0-WQN zo@~ZQq$QoM%n_%jDrr=1T@N07B2T&j68k|u=RI_JRFA^Hr$?D`34i}`*l0L(cdp*M z{=UnHHwe19^nJKrf5otm{(5+Hn_$a52l)IE&fuFx@XT@)4n*m2J4S|qO3Zz>a74{Y za2s}a*|2X~ZaPG)IRn8+Qk@ANp_k(+c>4x;QVqQ*gGiq9KP6K?arcG+P*yE#Ry5+L zEW$V+IWD}#WNLR#5tURrPS_sbmvw6fhIS)fjoJUcW7?}cVY4~OdgU{nQ8YaK0mL^G z_ePwJ&{%IMaKP+YvZOoslbxBXX6#ltZ*ejtG8WAT2qj6(KGIxl7W& zBSj~3v-Q`mN~z%R$>x^hEO!&+l70+uQTJqXkI@VF~xpT ziiYpO$1cc)hR^d?B5U7~zGuVv;iH|Glf2~l>C$`3?Ls)EQTDo7Ey&$9v=F3IU} z<6rqxbqO*cf@m!v_xE(xlas);8Pl`YGK;=RN2`syEEBKGXD99kn8A4G?0#KLh40?S;m#00)*aXZnGD3_& z72&Vo+4RaBqZ9foHkqyHeaA4teM(9i{9EKHz6)lsZBia}%GA6nffT*5PEF~r2NA0q zYc?xPsc8dB3qqJHKzx!q(K#5Gk(gk+!UL{OY1_$hDOSjiXetN{Xt!j~Pz{9o$xuwi z)mUKhkyq(f%rd0`9y9dE^h4XHQT0m)3fTj|a94L3DWKHhd-XN~3Y=iY!E?l7G}sgY zmLk?8I42%BET~9Mk^4gtnhxbCrdOX6gBBtFthIROP*e69t3`~I-8Lelzt>K{b}=0~ z!D1?=W?R#rjD0y(Ayx}(L<-(s8b zVB@8k?F;5U37tZ0hk;>HVH#Xw{>x<&O*mFuGB<&jytQs~=sH7NZ$|7PpM-`2k+dph zI=HDC;))SHI!XRaeK`yRrnu}KjJ=lvBS=+t2+Cw`eHW`u<-AL)OFkq+-~m*$Vz~bD z_kXW*+r1M{y5BCr})p)cn+lBjoP4c*lxR!ZF3{PYx&mGe066Ufr+tPN_}oM zG7?8Ag2`9BGdek0D}1y*8kCyL-7~t+*)xykIdH0@R4|nOtRi~;7b(FB-Tm54@Hi%u z&E+x(HVaL&19EE5K(5E^3xr=eT^CSB_lmeri>1GEU&~W)!5fTS1Xdi^RT*FC!^F9BtQcAan2Fhf0*WT``JTv+nA`DHuny?kSq7-{66zo{< zx@6ytb7v!F<0Ag)(5MgX0il+RnRb%%r!ZXqJfR9hBbddod>1p~7TmUy)AzZDXjb00 zhM?;+v;B2$VQs7OI?Ib636(LD+S-Ov=2o|DCjPd0J6?!OTX{ zaYr`yLmn>I+ZZRk z3-B)=E(BxdN%*P9Phr`H3YS#iR1V-DXVq6uD-mhc$)VmpV0#Kxf4Nop+^4Wgqqq zN+J}z<1XY@-1k6BLNd+w&8y0W(BEgJmDI=PZw$l>_P9ZB}Hy z@Kd1;+q5&{?rX_?WYA3OcnYgMg*J_y6z&uZPO7xy3GBlqd!#jxsSaec#I z;-LX!fA(5mYn2-&{kFnLz8^5CJQ{3DZjuWW`>^}9iSCynW{E}tgn&fZ7FY1grC2EH zRkaPd5d((#L5P#U+7`GGIwl<7da{qJI;>$+NSDSn`>T7=e4CA?qhx#}=wOzma7|Q^ z!KfQr0P8`bIv^~R6C%-oE4HgdaUz6f%#kqL$orr9vs_LB8whRw-n0#OkycDIX42h8H zT2Ls*Q1j)W0a4n)4hCu_&Skn6eeY%Acv7w4!b()ck6dySF_2-E)f{drUr6KPJnVFt zbJeVT9P{DDeJ!w>fP#;CeBLFE%(Vfc?xuTqQ=kAF)0IddUkI&qN{e z^wh|kTjkRD3+?hXk8%>maDud-WYU+q^-W0?g_=i-rGNT~A~6q7`lU$2sF(RA!b79m zuQs5;##7)1y*;VnF|U9Y#ZSn#Mm)GQr|qw-Wp?AIPDD9{?4h-QQ=c}C2g7tL>4^b# zV~-@K+_jS?Fq?02062{tNl@o-^a#=QFwx90Byet0MCk38=8eZUup$Lbve?+thc31B zMb^oyVaMCDZjthXfu2db6Km;x-Ly{Iq__y#88^kB1*-zB?S)*#gpZ}`qYuRqnoMP! z&73>?_o+S4B!sy~wRenw@V7AMGktFGUvC3aGCK9Z_bwDEaFr;BUcmZcY}MfIZPCpn zO~_tAk_RXp%O>yF*KINo#m)^Q7LrTs13RZ^!bFf-RAxAT-JS&qe+{&8Id9PBMPX_X zlU{r7vV`ftmCp48KgU+rUnSDj%O0s5M3_Jw`0qFQz4}~Q?I7GU-=_=$1bJh{B}r6@ zjS@vb7G0CmmLtbL{kE}Atudmw!k2xHhA&KjyfR7Rz>Dj7rY9|qvLehXgmch>l~hP zHA2{^rMGJGfl#Jm^=nxF6wceOX5}6HKFIByrMM~vowdsOjc2xv%6?6Qg+uZ*nXCQb zLBzO~{HaXozU5{=NTTb`98wP9rA1c!VkaN214FC$Y|(&nV~tX>f*rFyw;sW0eIit7 zLFtQQGHGluFuqx;s-3T4WCm0slH%YA3e~4!(1q(|;Xc~Ejp@$; z5JIA#Q(2h0Wb6av$-fo?crs>KUT~2-fT4nV#AljC?<$zrH*x(YQwXhU5G-02PYITn zqZI(3$+C_BQhft(o`!Nv_CR~eWlaQ*bih{(?PA-mD7n_Rb?tzQE^h6Rm)s?T;5>qT zM!=4m=?f#sj!i1?nU?xTkIZivnhM2AOKxV$8zRyHOnpi%y><{k8~#xJyBugdMi_aV z!eP?=Qt#JvyKFHEfqsoCbG$zzw)XYt^XskYvuMQamV5@BJYh1IQ7b-$BVt8rwwn)M zY?D69ISwt@B>hOYkw3nVo(L#_Wfs`jqDel#BBAr7eh!CPjG*KGZGpAA#&tCj6-WZ? zNsu6Sg!rJ{cLxCl%d)JB>Imx>{Oj%-&W3QhZ=x<@6-GVwovHG)D42VJ_yp5pW!6r9 zVW^C(!vwDjKIn(4`}Qck@utXQ#ipM^cWH$T-lYhTPt6d};ys+o8fJbJ=~BePwX^H} z`1K1Gz({cv6hc}@jpsFsGC|^?rYP{YxOt}05lzlBR+ii*3CIQt^R=W>TUMY_cn#x& zi(4P8i1ifd&e|#o|FI*|SE!}whrWJReD{LazLul>SgF22fJIkjsuz%^vY+z7F729} zRQs42XhuouPL&*@C8xaCX5sZ$*q|`2fC_6z{K&DbRwxg-5q)@bMgS~!hzQ(&e1{QA zr@e=V^sM+%eLABamqo+|sV^)U1OYSm5%n++Go@s3BSjCCgkg+8s6wh{_h%v5=pd2L zSxRdwlTs2*q(;3KRCOL7%+3^%bOodnFnMT@Sd8f#n(12S7F3lYhWCEz%PT$`CYmk5 zhO;IFa^bGUaXeQvoPUQ5$*UN0ILFKsVKJP1TSr(ya+;B=74`bN-4$lw(&+RMb&e4= zD1vyukhtR%z}z)57;P1%bZSSsY7@Y)+n-n=Oi9Z(9yS$0D`x#0m7if5%csZ)y#Twb z(aH|)jITa9iE@{Xm5!K4QFThP=Aaj&HBoS$p&qvhSt$ErUp;DROMN;YjxG1Qdjs~b zEJM<-rUj9izTam_!bR1Os^vM#R&2?YM04wtbV0ajN5Nkr~}t*RAPSi2j0dfBB7TFTcrcMlLx~ z=^e?da8kMF)ng*Lr-8NJWke32-{g%Cq12_{zE;tB@6k?AW1FZNl^3rf<=3-=+~%Dh z+vIs34k+H;|Hb07Rw5sUK0R{$yD-*(edOTThdjmKlgT#xQB6z> zoT}Asaf8PKaqEFQyuCcHoD3UrNa#*gIDjE!z#8sSB?2y9d8Jyz1}(hF{V zB@}0~RZA-4I3HI`ZZqjo-wKGrJ}(wpoKo7`44I@d&5aLpKK@gFd;oA2nT%{^Z*gJH z`y3pQz*i=zYKJ-}5!kJ9jkduqHS=!me%EDaPwFl2q?iy%_Y>lgV)YMEIwz~^QK||m zrmzkcsiz3;P26YdpuoSs`u0=?mtb>|8{n71mBRCH z?;U2h-xl%E;GAON`hvBw-^uur1^?jNOZi%V*9Kx~Xbje9gF&YdFF|$)LO+K{F>*)4 zt;O5HfJiA0a!yCnpZEoOsy z`Q89D3`<5ZTQmW<@#~`0sj|c6{iN~s--3lB?C^rf7@aND4@F9Z@CxphmVGyhDDaI~ zlRRK;Ee&d#FoLva;^r+Ve$Eyh^U%guf9A z)9?`PF<* zfM10QE~?g^J!O`^aF~1h0RDQHfthHTGKO>2SwdALuW2n1c_2j`54BvxsJe2&H7(7GuL$IBZMHaElm5YK0f+Y<2&V;g;8s_pd2PBc)8sw%h`gFq?xCba^WLm zKOm7N(xpwHMn*Jxq%fg|*(|HiS+$(Kv$I&r@muRNE2U%LtO)AJI<~O9W(cev+v~aJ zI^m0=C?f_FH}Q2P-Okh_T^GX1cu*=5*)P1wHo1Gx)(x{^kFM z{Hl={1tF$Dbd0%WTx#|?XidHk;O^N@nI7ez>R&YB@=48acClCM5P%b*+Ui;+rgA0n zA4%bWBId&5SQUGu>~1qdgmxgOs4QSd@=w|{v3clHYU%6DxrB?4En93w(g4DJ8gj>V zuCw_}l4(}^IeY~Xj@jPx1h0TvYM%t!zs+ESk>Ccd*V`vYl6IuD2eJUIwbBpNWfD7N zPTw0KeOP`oNqEk_lintKXW)+m6sBRAFIn8dD#-#@J(tNtQe5x2S%Z`wz}A} zjdG!pka!Am9pM7Q)gruSPzkDUT`fkd(t!}FxyPX(l&Nuz#|hpJgI}Pua@V|1lp66B zF*mNQ3mL_ze^YJAX%mf-Eflbf7O#oFQkzqv;09r$y0V`J5f?PbB?7r1DoAacjkNBu zxsjtDvuo(d^Y~ zOa@pBv_z0?BbOHiERl^H?6xMgNWD1|Z|M6qen5B2A@4v=UTNG%g{_uPay4m3fo+tr zsQlBmT*3WLMM^j{Qs6xt0PDY`Hd4F9Sk(!E4(~hah=1$%)i7$yOO;Xelpi{cl8}X? zlWiWOLYkmgqYJ1y^vcLEn0(RsZwRxxUI`K~I?P!|c!9Ce52+iQPlvGrH0nw8E|`xL z7`1(Jk1wu}4u`+?wm_qRG@!%$ZN^;fmag-&N!xouPdpj9aF7+_As`X~_d|Yk+`+XF zx^YPbFDc;8EyWkTi-liX_1rmhwbjXT&+CL;zh=9sHgGpr1i65_#;E*-A`{(&k-Ql$ z+G(Jc;$K24bMRBL!Bd9QD%cx>CLG~$+&`OUIJt3AcZ$EX{w<7htL;0O6Zss^m0&z}J4FfUSC*9+&DJl0@g zGlQc_D$~+tSL9WB(XEtbrkrZ^{TmX;zoF%@Fe=;CNRp915t!&7oDM>^e9C~xzM%DT z?Z13m{OYw_4ujx|xs2p0c6~SZxodO6cvZvMU1%b;Miq|MqR0ad(ZD@m36#}nH==+b zIe5@+@V#J>oRWcsoN=%F-yM53!!+JKu&`d>^$*^zw*tA+Ub)k3;bHG~D1iRRsoKM^0YBjT-crY!6~2dU&EuU#B)fC8G6o7&GKxxY!- zapVZT(6T?daVX{BC!Sn(t#=I3f1*A#ix`}7`HkiS(y_r#NoX(*X3$>2RX$3IdquY? z?H_Hh5YBs9)`KUk;aA-)BgUBYv@Bach%u~Y7x)6aho36%DzzdB*@LVp4ET~SUx*`T zP0)BQUXYh+>yf2~F3wcYT(n^M_XzCNXg}g}D9e=_N1m>sSS}o-h2ttoAVOWfeYY;A(e6&ZD-df)(UN| zeU?+9n4ptQUfWB+BdQ&LW1xD@XgUg8DvF#r>qQSMl#UA|_AMVXh0DGgnoLQ4b{mLpy!X9?H#`lvvhp8! z|GKAPFO{qhf1lHUYG9o19Bd=LA9APc4~gru{oVD7m~U(LYjrBj*Eg@e{DW+o)cm-( zB6K@$eHZ@a08L1? z7zFdU%3w5R`5n>9cO0VtwX>rq$#L8GdReGI#kX9TKIg7BJ zVJ(nJYm6yS@16RZtIr}-s7x8BvCycK?Ik{kYUPI%$@2V%>w|RUH-~WFu8Gx{VktU* zDQ_ZM(4-%0ZqZmiO3KJ&aj}>B&un)hXlt?;Nx>8%*&Q5$5i}vpEBzz$EdWu8T#tbl zK8`^sptsRH3aC+GNB@St1ZXX^C75+Y>{ULCu2jxn4LBP`i@b~BS*3^9m0FHhS>c}b zqWYK9`HS3IvgCfPSWZAjY!%JBl5g7oe00;9xd161AtvfGi9S~Yf@f+xQPhq+08JD$FhogI zhp#Z%*2W#V%J?+QOZR$#QRSxU)Aqj35-mt<1BsF`L=9uzFV|zIdi8QlYF&5Cbz&{` z;lRi3i~;J0??1Uvd1MhMYJLL*jbP!=h6FOSEC`=83|uERZl|jKOA6okx{vEiiBIR& za&84_&QxOZwWf5^XP+&Ztd;i<*QKq$*beWjn3xyk60$$(^0}Bj^T_){ z00NSlu2SJ5&QkQY~K>x4tpo`cvE0Hy;LkI=h?6 z3fO4xCzBBe5(pfkBC%5k(_(Qc@Aytj<#=|2dtA$q-uR4<022>GXUCyW0d}>XQkP=6 z&zY}&0&Q%G^eMQ z&Q#`dE^uNL{V&uc%VjFRq)O#SuW46oYd(eVM&SK-X9bJOBqtv1ix0uSp-K5K!``7y zA)lLV36A+gk#j|{Wpn(POhNT3%zlYx>*|r0T5UvvIrr~h7xb-ibFV#U^erhaIjklv zWT;8_K&(jlb4X8xoO`_w>{j%lBM}Kyu>vF9ugmEz(rbrGFCY!dcsrx+7nyK z!^mcfgkL1mF+6h4)Vr1q!KUq`%}k6Y!At4s-zhJ-sitmUkl3 zViTEy>L1=Luf^_P$`6ihUnSg23d{*)fQHlSzbmR1Rip4}2>D&G&1<|cH%t4 zOz^>2wV3 z7PNgpSzMT=#Yv~!$nck!O%z%mE;*0h$8XXR;ngSpgBTe8Q0YcN-XzoOR79qzL71p5 zJ)6a*5Ow4wI`>nmjquZiQ6+*U2j!ZL#Hq#!1rd;vDBK9)V+z2E$ljd;!riddtjTtg z%N;Cuq=+q)2XEiJW%i=2UKd6NH}~b3ry78Y=<4jSpw&)io)v9H0F$P7--J*nKtF%{bxOI&`MwF)jguqw)82_PO*TDs=e?OpY)YV8Cl|8w9qk>8<1N>kdquh5q!)VW^lB%%| zDri7lW1ou`oQgnW`gMNJ7#9~0{_AU2(}9jedm5 zOz?7=2rmhjNL2xYN2xh$sla+)9&cpHNh}t}9HX+E&HW^1?nnf+p8LrX&HQ~mCmbcO zcJKXgu{+3TVo{eq`V)Puc`%{5K=5jaZ67;Nf4Z`8_^WMo928>^~-Cg)#G5Hk#U4@UJU4O=_!? zN^eV|?<>jov0t8yM^-MbGJ~rcAPR3a5cHtjuL~PwsXBQo#Na!Br+(C@nv*&j*nzgF zcV6@Ht|Sio6pqpgH71X03|91BSN2r;o9Sd`%MUTwy1^2K20@6jOvT( z#Ep@3{_NHcW?#@sK&q2`?(piDk)J{Hj^hA$p$FpDZby>Y;^(xl|BAqEY>R1#9rx5N zx>;yQukjcbQqt1)Sf#q8BHda!%HKJ^H#|rX&XeBJdL*f9X;bVnjKk+pbw$;Qe{_`E znA!*hoX+fwewO|SGdHLZ#Ry?TlwCQjzO^_(ipql5Aumt}T~5S;FS3pB2?zc$Fx;}d zaNxy2kPtoz?>}bfjnw!sl(GGn2|C}H4*?4@CW$~C%;+V2-Sea26rOXad&V~^VU7C7 zQnK2-QdZz<7$vFwVu)acTC&P+GEsP&J6sWSJ_cCA<#;aZ9pO-xI96|C2Nv8{r4S}> zcb^fD^m0xE#8WOM-H*y>Y9Ss)w7TuP6|C8&MmrhbgbbOM>g3!&sTG@mRw>%<<#I@k z+=q3-B;mLdNw$6}12*!acuB;-=yTPQ56F(gtwIUf$1)5P*mwrz- z9cm&jN~<)?Dp>mK#_&U=j2kCx-qz7=rP*e*tK{@Uq939UuRA|eh`gKW+nC|pYTefC zFwykn&Vyfd@a0x3Do;YE zj96$or==uPZeYkFTa#Ph_M#=DD;h}9n69)Jnrs)M9fO7aJ3-of03jbw`)IQOkK}om z@a6Yn%%I@bgv~*^Ygaa3*&du*$tF@iP#z+Y(i%=_T6a*|)P<*(v;Ok@novAamP9_{ z|Ljg6N?}YfbVWiDGU63*n>$&*l?kGH{WEo4tSKEk*>g7p0^~*ZjKS|btvz^+C8Jk~ z;sJa+Q04^a_ZYkoMh$NodF0Pi;t{DJIc>@yXLQ=Yy%k1E9V)X?yJuE*0K6hJcfO(I zX!Y8BMESwzXQ`k4>bNojyzqx8O|A0Xz9yyp)pzSbd8Q=O6Z;!FbO{@0(k7_@}G zJGOg}2iEGi{M+U0# zfrXZjvN8tN>gxErBIz(^sw{4*s6g68keT8fI;={L^KmYkV&iAC z+SA7nwNtll*>XHW9La?TdoA9#<>oc!M4(X^qSf(BGmOGAg7d2jWbtVNF6za)9f6ry zv!3Dg%;q)_8@0qg@?lqvmTtc;Xg}o9NNKS5Ian<257l>Yp!wov zA=(gSCCo|B<7TE8qb!K_R7LkgW{7+_NL_It9iVu=Dw_|+|6FTjxz@gdsKTmyOz?bV z^&w3=w!aUk2@*Ni{VbyKbl&p+0X{&%zk2R*Pe$Sr9zcxONnI}N1)bR-W8ps?x9NV5 zr_gcMxz;|r_!s+35W?7uLc7aRy-=R^>)1WVOR&>KPBOrg0Nj=B_wr3=E5AI_Lhxbz zsCud_?Z@$)zXTx36P&+eQ3mHxB5O_smjat}K=*g!#+GZy&R(9E=BAZGH_qe)UzKr2 zrszk02`Ck5M%a;s2R&z$)q(I*kGzim<&XUpeF_MR@p)Z-1~b)%eDOQW8JNqAf)r9JtFUsms|;Z*N)F`5!|eqhsi@1_rZZDV{q9vBF~ z2dRb05&AQ8mg9i90{j*<3(VB|ABK%7J?9L z@T-#@YZ85HaCF$6H$NYju3RU8g2#XspI-6dkG>N-8&5b0pL%wJIAyRvjfl$tA7BK! z%^WFj5m9<@_d|AEu5%NQpWw!=7i$SB@)Ts4=;YkTs9U?DWAgC3oQQyracbvIFvd-s z$jRn2B13>28b<=DbWVg06CzKwc0i2MiR)Qf*4)%lj9Cnz5t1v{u3`M#C5Ez~j2ff? z4tK<)5?nelJbBybm^CG?Zk)()4FgfEwl_A`Wn(%&50z9wRBEG&KJvm^u|3jtf}ck2 zaK!*31kb*6Q-r)$0W@i4gxms9^`N^p4^p0*cY9Js+ztdz2vtS27OF*r&>ddxigb{h zM3JUBOJ;TTGW^x@Wmo_JKmbWZK~%$GpcqZku04PO_$_5__y zPPcj=>}(-4itzzig+Vi2#gw9t#p^fY5@)`(NO^wn!3Pw)d!R5WO8VMbdlIV`iIP3P zN9gtaxCe+UK}QFtU1nwAPRdi>AMb5ISH#1QK7c6bV^V40*ETkyg(2NzLq;1^D!{73 zoEo1k;S5gDn|XlS6luLC?N*sM&p-Wy$DF;yb9Cb?U;jFfRJ#x#{q%1ML(>6t31(E_ zHeh^;8aod{-Xo1V&3m6{Kwde@01eOXo1>mk5WdDC!LyS%Ug{5LnP-R8&snZj&%%8- z)*Qxk@+ELbOpJME{?e{qKLN3obgDHS;poJn3@K}V;K9pn=waU6&4^K|DqE* z#Awu6L|Jd=j>{qX&}Up_=H&9N*EpB(HO^?f7z3`MA}P-N9`cqAoad5pl}5qX~1ER(~S1CGmHlw>o_u^ zg)B9m(qP0!%j&g=4t{A=(PwDZv1EFdK1^*F&ZDqTIyu|(w*b$`1efu*c1Jq~f3P7w zQb(hnM~l#(V5D$1fF$OxeVs=~5+pLuUFCVd|D-pGW2hN!pJBGE5AvP2-$ zMuGzF_g>3IbRwpmTs+F>G3;7#KTeQqr`SN|!K8w{30C>GuX-I`Eero;pd z@J$vk2Uo{^=jxpkfqGP-1&r zn!Hs95iop^zkoKrmp^%d#Lk%{IK>uaxn5`cnS;bO`0=_XTLPpQ8Rs8J+_} zuT4#(T|ZYB`5!+8ko9+_{FFbs={S9zEVKY49TK|Mm0LLlLIps`N&I)Y+-pWlfK4iES zFJ5N8P%g&e-)zdu^`~+7C*Q>nwb@6|ERMTE4$`l$(%-0K(o^>FQjdT3v-qPw{2$`e z4}OG`?Z?`K2f0e32rC(d)@Gr;XngW40)h2Phd$s)k!g_S=H2f{Z|7MoF?mhP^kKjR zs}1YPEa3B0?LLFjA-Ho=!D=)vUb+C1KF&u%8KvZSyr0jAOz zDn&AgwTKQFMpbcULA6TD9g}XyS5Z|qADiJpv95F{pjD^Sg*iw8$EuMtY|axIh%TAJ zGhK_k0Yq1_u(8YIkxCVIHum^UB(BamXA7&>^Qi;dY#?wqh>u6|m30)fYsAG6tUE_k zL2itBt{HGDKE+==z6&5lClx072oRVnN-j(Qh@!fZb)H?hc9jSwAiyRvS@s670up0wN(I4 zfR0Z_m|+sVdj6ag?k;E8QQroLeD@cAF%SP;0#sRFVOn>ev+JB}P9j(P6^;R@qx@|E z(HvvF{LVYEc;j}Ah-!|ro4C2K0#7KeiLN3jRE51xd#zD2|xFl_cCH-!Kx?rx8b=9kqRr=GKQzhWK+Kd)HP%>firOv zgL<}i=*9^SQfBh*Y#;DkPomV^GvsW)8c|Lo8>a#W+OQrk3vJQPwofc!^15Nhddow` ztiHLscbuJ{Cp-%45ZNrPT+Zj2YAjp3d(@3;NQwO5$ZIT&>fOT_J9!o57Kc{T&w%4_ z23S9jXPJaQuYLB5u?>CrM4nzjXq_WOI%7P0v`X54;YM_-b1~q#eu8IBan^3K5ZZir z7yes#jNX+A*8|d=@ShctWj*8Dwi-%EW9s3jZq$rFIE)8Dn1u1YnZ(sXTluR zc&XQ8@QtZ}5j5IDxd8Qd*1!pNjK&o}o5dVD?|ssk4x838dX~KxMt?{@rj*1uwwF#A zp6d8aAL(d3DB5~G&sL(P{BZ)+Db?tb?ysQ41HRM7WWl=4}deH z7LMO;)0x3AjdVJ5KG)ZHin=zw(BtrU7q%m=_%w4 zCF!6Ga7;Je4M;M2Fy#fqOR(k0LjG$ipRC^h_P74B0zGy>Vylx7CZRmp?z9vp)d1*`60E> zXl{bA^SVo#hi1me8yAP#e%wF&Pm1ZP0NXvIL4Y2Q0ZAUXs>o5OZ@kU%74IFRkHk}_ z)xAVK&~pxIobcLG107k~z%>c}&gMnl4?RWAV)0w~S`R8%9$8;Pr6XA^C?Hp z1BUKG$XP`r&|BiE4IHtd-{MBg1o9Dq_*WOSbPM4|KUI;0e0-qYC?cqQiLRmx*fx)} zaZssN%8~2+r!GpAfXOm0^}j*=vB}G1Hs;OE9H<=I8ua80WW( z?@YU@ry5JH75quBw0TIoj}bJgr!TNB-pJ_?fo6RpG)hD22psUySGk~s4i_)Bm(f>v zRfBF9Ehq`LelDgstiQ)`R9(tjDm=|p67eC>3nW~q-_ zy?lkEFDPn6Zl+TD7`>S(qFp0)#qf6nt+3eN-E?yc6#0R0RW`fP;9E!n#|VzeC=@3wOp zxVfM&sA9DYo0HBQB8uSiQ%#xXDuZz~iPrd}f(cT0BNRv^KC>{xV>T|u;gOM8*30gYjh6)?e1Jt4W8F!G5#^nn%X^R#f!BK zfI*D|oY!+VQ5gkoYB)@v<19MtWcC;ZqjFPI=nG8YYk5q5O_j;!l**^Ch-)PqkYT=SB@H% zF7wPy3xfT=izF8^Sfi>dHFp0VGapCnZYs}QjM^N#%}MpPP%w{)!2&XW;L|>jrolKG zL7G|MdBN~&Any^awp2Oza*p}F$##BM^_jK=+_>x3B7cF8(TCjG<&BI@2g-jrzqAwC z4*eT-6HsUuvSk|A@w%W(Kn5>0!bVrTCP5YLX;^i7Y3lhrQ?PU(nDLd7nsgvI8;qQ* z@M6;ee1Sgib0>8IG`i0IB#_zdk8S4)o4<7U?#MllxS7UFfGQ&}J{ck4y@q`Pp6I|l znnAmi5fJ)HKt(%_B>^UL=;tX(E<6WeI)qFkGDfdU`^J0c6(c#L&4n=bLf-P3{d-;3 z*rYs)^kEXf_kNd-2HWh|i(Qw10mkTgR*pLLR{m-e_)$hetc=>wegR&fgE6Wf336#G zNy0WL59!gVoquOkGov<;O%r*oASpp1H2GlxT#HDipe|vZ5pAL)MLHK>fj&4=IJOpmc^rYdvXj*C81Cf)-HMok2Jj?2&GuzYd6f}_+;KG#Y5 zJzwVc_Ln@zPUv^kKT~^0U*~@pvc?3b{ELWUUQt|@a&SD}_ILLb?RN3Qh0QAFT!J3#w_H|!lVrk_ zt5HyCwHU21Ajs$HftM8|^s ziVy7H(vf3ui!WK|LSG%{=F~;`;5<49&@RHv1U4x9@e1vr_0S%=KgCAI3Qy>1HfH&3 zlJYI;hc&9C>6a)sTe)^KmUzasl=3(>fps0f`JAf6_lY*dBY$rXAOEc@P57!hMuU_W)SI0cieTaj6`!nxmoBAtA z%w2|647tEyn&U}4Nc512nTAGo!c-BuwOMUI(0~bOSlUM%8QoFfhM4E)7h?1II?Uw} zT`+o^182wc06+#$^r4okbXnx->4fwd(=^W_-QM1bUCx8EhR(>K${|>?F0cVgpJTi4 zh@&PJD1g6A;p3+t|CH6qbb{7#jxssfBVESiwH9XS@}+2(=q}gw5LOlbIN4N`&Gi>Ni*u8R4LNzn>zI6Q>9I7Bx)vBdy8koAc@(d`@frfJ z#Sj1TFXG+zeleDL^oG&KDcWw5>O3J;>CRx!QyXVRV>HQ)By&X7F0HPhSG7!AHklC2 z6qL6}o7GXiV-6~YYUF1jpa{}-hBNEZ$^A;Nx_DHiX^T@Ha zdk?s?7N30b1fWrkCl8)bvuMhh^!bH(4zzs|+fN=v4FK0=lffxbqzciivfu!aai9Ae zrd6o{+oRv#{)gWP67VSdW?a6wNZW+PF&M-m0H-{;0iJEq&c)Sb^pWS8GWJ8Fo))nB zqz_lvQ^@AWKm0yNp|H7tXtD0yqhS&xpzl6L?mr?8JRtHr1gsw;>rFO-y#B`9)T?o{ z2JL8w$N2md1BRlbl>lBxY8?-t)d@Jyiq+@j9#O|;k+#puGxgp&B6Wg0Ioa`ihRB#feb(=(xEjv*Ejb_htM9B$ye~1BfC#w^BD(4Aj!l5g!oG0c< zp;lSIz>`xVa*bIQMn)2#^Ad%!ZoDyEV6m@7xvSTxn?Onds(c_-}b4Y zcYXdE3A)xnPtY=~cD~@F33X-FT#PaFdUG$wBi!`?oERFk#rdiutWW z6U-%u<9=Rn}M>(b$KfmsXtqOoEjZ0+LNXVR$N`N}5&px55#_?5MK zlQp4vjZPIX3(d(y7%4l92o<1JhiS@`NIg6?nwbuszFkz}va)UYpD8MtLJLM&vgVO5 zyvL&>+zfVvEwX=QQ^s+dl0BzQP@)WT_aLnZ^wisIk^b!8amx$Gm$ii~UJ}52J-qfX z`#k)YzokjO3MT#SLd(9KtNtz?Hzi;rnkdLu{~5e}S5}xBIeU2$X7c>ig+#sSY*M|#qGunpS z1Yrq1fxu0%(1Uu3=EsO7 zZ)l$&13Fv0&VU$g!7avGTrbG8l*@T#34uIyvEqC$_>T;1EB~ow;3jB-2{a?`tbw=w z5}6RJNxR1Nrt3{(>H}Xnd=WZS0 zXl1nN6(b@R;QBgX1A~s!!ux4J(4E(A$MZ)YXG&@chvwtF#wd1k;{>KNjZ7Ds46jf7 zBnA}FTCo9Xv=izMJ!9;M=!?qZSftllk3G_u)(mNIoK%Y(hqB405?2c&?%yXRJ5OY3 zo6p|;{JXLBaPZ*ilV=<~vP-R- z^>?M%c)G@pRTPMn=Qx1jl$B(Sx|s^jwafK89C0BX?iw~S>!k6yrXD*ZZ*+3Uq{Pd} z;DaB0j|~!svCq>29^Si;!q1S}os2s-Za{l4mH}@w3sdp6uYVO@Rb%b(evA=5 zVto753bJz*eY|$#RzBsU2H^BLqx)=-akC5dnJ6D8Rd0Lzj39!QePpgyY-dA{`$gV zOy0N?jZ4?qr8F0xJYJ6-j{R_q1B}49Mhm&Ri`J(CfFOxr6LDn@`Sg$xA@gk>dp5%k zDC_S$+io0#xXD~g#{hrZOaM(ri!qO3DBQT9ld#<~#%vZ~1Yz-l4IBbH$fghG3E#V8 zuuZhpCmML(SElS4gS%fnp2ymtAq%Spk+>nLsvcm|^8ro2HqdbmpvTmlLYvWB>%A;c zKTl;ERdbhG&NVc|ll_~>&{Vw!+wi`ITB8eD2|#R9U;*{{y#`&7k!BU=qv6r``qKF7 zB)!*b3Zq;U&#R|8XdRzM*SW4E(h23t9KeAc%Q-XVA_AS#AQ$N?Yo&0Iw3)_^K6FT~ zs-&{!Dsni_~YGmtf5Zr@+X+-56lh!Le&xHLP`Z(nwD}ZqHUR z^e)8CqYLTyp-0f-A=o;Ztdo-Wv}p~O?Ymf34}Dqpl7NZ(d=|A1KkK}>SAJiD|dIU(~}`pmftHR`C}dyc10*@zoWrry{`ao{z;3!PQ}q(Qj}rUiWJnR*~E zL+cmMX<1T4HattsetAtb%(wT(D#=dJz8Bd_N++NNfW;EJGk z$~t4NmEVxeZ|4fX(PdxGX#vFF)tnZ9d}@v|Cv^sXsM`ag{TVr>k&Hs4TedH4o?j|v zaLj@?l|*`S-se1aVWS`6mH*3Gk@pw6Pyo9Gn~nq7I!?j!5Cdo?!_q46Qq+*c0PsHt44rir>c`ePLyK~Eo+$&bH_fR1^;8BvCY8zt8x~@2j?ktgX~|Q0&pc&L=J{>z zD8-D47a9OJj|Hg^ykVqFT*|;I&_&j!N>FQsROt$f<|)droO{-&ACnatQX;l~|HD`& zi#ACRWh$R!!-5DOTRf_hg&A^@XO1rcw{{Af1$)`JhV$QHwKB7CF&23q>M4;Ayg;7U zmX6p-#^iJ0Dv3_I!`WE*=p7huYH}8kzySnAE9Rrgs{a*^XoMmUP^J+^!bfPGAN;6N zYV3@TxhqLjTH*;1<3uVZ`SHv-4_lok(p1Gr2mn26txd7HPXUm*Llo1`r`XA;ffUTC zI2tIlCRf`!B9&PAxcb(hSFMZ4Q4OnMNc+*zfarC7x@_^d`qn z@kP?LY>!?sG%zpnQLH}tnV?G?}T*|!!-)hJ-f?YK&FU1#CS-5GXN8f%mof& ze#wy;n;by<`2M~4&Hwm!IGUxwBbHu7i6Y77%I$bXkVkMa4sT`!Iyhl>BDyWSZkqtn zt9M=lh|XsePJ>tm0iV)OPAML_u`We-e;m% zTVO37hJbd50IUH3X9sy2&Fzws9r4`Q1L_exZsiaE$N!$ksw~FQX)R{2zZL6!cJ#IZ zXFXS$etMn$K=Ji{@_e+#OK$YH1>bx@B;Uw^jzoauy*ir0 zaTZaf!a>_7H|I6ZU*4p#*CA?OBwYKE3rBR`9l%DS&m(^VzXT4nQ5+{kFJU8x_)R9|Ta>aMk<7To7OKOe92f;kIc=^o!zoKLBanZDFlblLbezr?09 ztXv7ZtvuC-BErX?w3QK3=u}?~KIHwmz$;2G1pn$+F>czBsH9bndLQG?pMqz(SmUZp ziF4oiymxH!I1esIw~M)LR1tgtz1kfEB922o+qSkKPYB?~HjD6s?QMKIa>ht^2n={y zhKoSuz^uI%fik9;;&(amd9dc5|h=p=}Zh=Q+&hli1 zTWnS!6Gu?!*}aePOB|U+FzJ8;db>x|J>!e}@*8^1h?q@KMhW<#2R5gik=_{})Di5= z9)M2WX?l!v+-SK85OQfohO#Qf_U`X1J+@IMPA~PBRU6w|bx&sQ{ zw*gxdI0erSGdjZ*#WBD*94{B8a^}hkMfa?;*`IgIKfh6`=}voRAJs1vQGG5@{s8UV!H6(a9;W%n&bOPz{g~ zg{*ahY$uUmgS`|PSz{FZCRN31&vS8#?ahQ^2}hrT?jN)5w?~0}b8jo!FWCMIsPing zBY@AfJMU5l$JuhE^N241Xk!*Iz<^W9Eqrz| zdo;|!E|E}HAblJWpOVtkAQb5X9a0PB>T$>|NZ!P60-HJjSVi`XKqIadRwN!ZbNS}2 zIN($oyH0~Fg+Wt$`}&h zpm(pH5p5*3+9WUUYT25E2B2kxBD)gyF(3rTvymDWGt9SX zk9@)SYY>f?EUgj>-0YPtzGi$yzk#v2E!j&cUXO&mt;dl!8@!DvFM* zSS)G7(%GVSepNWh!htw;zmtB?Ci{P>I@2dPk0U=<&pq99-}ej#zyKfs9^fH~(n^-J zTCdmZmF@5uc7#9qqyI0*+7S+Wcf*zx+Lg3gilj(NAc=bp%zfWi&+X^QnwG;24tu8i zee12t%F6szUZqv;lp@~|?^sBGw3j6al|!*fuy_Xp+9JuwzO zQ&9+19It)Q-S6`S-fl|$Ot}{K($TQAMVs;kbsAZBx{iES$UoY2+$DHV@E7yL7y~rf z4QXgQ%tah^W@qVN?px*eJSLT@{mifY46h4X2LOa#&bbZJjxyI@{hxE~KZox416=UC z^4R1P`9pqlJ==e;)2ilqu`yq9ug`QIGT5bIHt%e7=#y*0wIk0sPZ=Q7NIc%>GfsZ8 z#Z*gQ{?3iNw6G1ILzCCEN7kb+oX>diT*0c(4!r;kJzG_|A@dm;WFd4Lux+a<*avyQ zc?h21@8EOVi03e7Ulm#h(1KqofBWtakM^o)!`}pN_TG120w(oLfDF1D^p(bWh|%W? z=*i=T*66!z)WAZUj*zC$rsk=O{UoPibY3^DuPElP*oOs9&fHShh?`_k~pufA3 zb)#fRR{_9cTQNt`^Z&P6& zE*zWloHr9%*X0NL!5p~K3(%keZ;SG~5!rt4y${mc-}xi-1)i7{N^C7Iq-~N6_cm7n zZp4UVJir2xmKSFUU*j{e4zeg?5yE2x*{mu1E`Qq}*NDatWkl5_3jM7g{%I->jHTA0 z(bRYAW>m~Cpw#OK#IuUNuFq{y;u*p4mfX;FU{b>w)`~J#5-0H#WWK>M52pZ<29gkr zoipBQpZm3pWviDi3QK`txj>;uVP6EO$3!r4+NnxwWMLfQw%poUC17%hIHrNLNpQVq zV(wyvF&?OF$qpDv-m;8I8H<%g6EEDI^4Sr9>6ExS4}mrd(co(dZ9JPyR}O{3^H{WH zN^o2Ojy%;t%K<6{jFK}}W7mZlwyOn`GnR})+@afKovN@kjcc)MBz~}pou*dA=C)oY zANu$VT%VJ#UZwu=JE^JXM!Nsuk5VnBR0{{U5;y0$YaZj(2ne_UL|^h_jWy#$G*9c+ zZEQesVudRr$aIZG;5$rWChrzjsZ08F|AY5%LFc}kUc9^jxG{L2nsUOth2^c(I`)uY zd}07`nKc0tJ@V`l;nT^93`|2>cs-LIfAI<92N+aQ`wW0$eKU(2He(6x1Plq>j#1t_ zxDQ*Xj|a>Jz!M(`K+*E!Zu{s=*Xb#B<_OAMTS$v5lhh>Jr#NwEFrnF79lb4Scw~%g z7Sj5&uhQl0t2F!j&(aR%K?(o~W5GP%Q=X|~PCx(R)AauP-%Mk}qY-bWwXnSp{=$v_ z*1ZYh!f-*J?omhQ4VW8i2sH8R*n96hB4xV;SN$f*FVrDqV`0Yi=`mTFfM9fP8tUy# zqrLsOYN_Sbri+-ENs3E%Q}EZAvknp)YMBp12&Pt#(!D?W7pddkLr$n5xrLop1Dg#O zmr6tax!~E`gb)-6Ts^`XbqdW;d7M&Fm0AIZXMo95GD^F!>>g0tXNOq5YEE+Ar%%qw zF}x@cE|YDm0MNoIk8ksgBLqV&NeEhfPLWWjSR*gJp0ONb(M4p11-H~hkZdPwVr4qD zUJ;|n2_VjKh(v!e+t1CGpc4K%;n)_Hu13023&IFm98peapCp@Q-11u9tu+Uni0?d! zqYl0`Lk&?G-+R>ADKv<>aXhJ(ayb}yL#VnTh%9l&rRBoCD$Im1()HzTKLZtGKlzUr z0E;n(@?e}QTwV16=4e}i9XnKZtcE7=6~dHfA){1ix+Ya`0w3L{y895-A#k+@90IW2 zhzf)Snbus)NgH`A;r3z8)#)BZPQ(sgj{$s_j~7&EBQLN3D<9l|DpP!}$tnR5v^ww% zO9E*5j!*-h6>^C2ndQfY%pX<)ct)46RI!ANN~nf;XPAsx1*-enV$ih5+|;^hQgP9L zU7cnVy3sO=@&cu(QADe$?K4G@b?0VPr4n(c@QJ(|eYmC*lMpe)Zp35+0RTER%KVs7 zZF|;~q!S@`qgEwzDcyXC^51i@Rk$&4uHEaoP`QO{=WjhRKvTNL;25!+4FP-%o=;4O zajqZ~JRs+Nxob)Vj|?iX9?Hy11MAkY2i>8QF>z7g7W(PQ z2n((cx|E{G~*@6p+i-Kc3cogokInf zCp}2$tSI}%+`2ezT|MoZFfpsZ_wucCRYTPQ4}Q-UEnR}Rs@e1lAY;Qpf3#@Vk>MBM zn&%4SJd0V7W{5>+uO1*mFX))oE)Qcj*)X!9?Wv#^SxOCux#IqQbe^>Rgc zhqUmyf-Y$y|D#b@zl=gK!{^3#JMXSxspUp8LJ1(H8S@$!^jxu|pcjNjHmlF%-;h7Z z@gwFyT@_v<+I2l@n6h+P4f5KPftD93g@*n#17E-?t?ZZC_m(%)${MFI(9wG2pgbY( z*Fpl;Y--u*^|sd$jU$TO-T%ti{I6-xJ7|7Z0r_brfeQPCRU38aTcA{mNJ&I5=vAy4A$cY(Xyyg*PHJ- zfS5DCh_%{SqeKF2cd@hY9~_ihVy8PbH5GAFP56x)P@0pIlO!cz)gczdMPi8pvqr4N zq~JA!)fz%FWJR#)rj?0ZfCjc2%Tn3xt5DocyPRYk$0U=xYj=0x;&P|rLMQ`FJ%vJZ zxmhRmxT$1JI!=}*-~jsdJ$yU$K72Q|jNVI)UBlGns!Ox8bAT<+z62bMjV3G89YB;K z1%M4EtwCtC5!Yw^v^s>r@%jc?!GO?E%5;ir4s;Vj6UTl;yi*IY3*p+rGO`M|)nW(% zIAUuESLJ9V6V#&M9=!c_dieeaX$f;}k@znQ%I*QUm&rUfi)v*9w=^d@+AkMeTy!d^Nj^CW-@}@!UiDT}NER5sbdWd>mjc zYN!1VdG^2hcmE;vkKBmt&@;@+#$!Ey`WzQuJ(;_J2bgwj@X#QJ!sU?H%Mr}9#lsN zGOx?@$%3}Vf0_PmY}v}xWSV;XWh$>z>y20ikLhto-$nhpUg%O#uq%9!3IdOrC2A$I z=+6bKE-q-GaA9&-2P`J$!FV#Z0LYI9HWy84(^x))hdagw1iEJHiGhA57)67!HV*Va z@JByG(ZF9-TvvySyug|}0qE3oLd19f{Ku37X-M0|%QZ1Z3pZN|ChtbI48XSt1Np!h zMJ?5}@XR6Ba4oh5CJS<4C)NY&g`W~NiB1E2oA|&uaH|?)>TYs?K3b-Q=Au5NWk0k& zXvHZLA0qe)poP#>K?|OJW#z|)N}-2f4RDY3$k_$N5u{aj_m6+B1KsbClANHhB;*kF$NJ3IsQ$9Q2?GGF9zaCdkhc&KAj z!E0F)Fobo3F@`L<9)qq!0iM^Yu(CA--s3aJ>pco$`&O|^WLLi6(i&GFu_#@RFX5_L-BCjQf|PVD>u>7P90?; zj4_DDB?|HaJU}D=7Sw2Iv_0wO?{*;idOgp!7cTH?%WBY>{s~-cLwBAoHZ5!Q$O`Ec z);-5S|8rj*5Alh5b67JKaHsXh^&|ZSGSW@n{(mn(w>pM-aqjYu{Y9Q!|9Gu^x1W$M zc+qG06V_(>;RG7^F8>8s=E4dvqai+Zy*U2-!;J7R^L@wx{&IfZOgJ~0&y{htSm`w~ zu-;ii(mbqVwCVHnn#&evd*NqZ8$gk9=6OC#+r(U(9Od_PCFWkbIUZlGQ`cC$_}t>W zhRq0`1Y}kL@VOPsxO1YFU#pZaN#OU!-`S45rR7RXVd!|;lvnj`_?>h%b6F!v2enj{ zYx(pG>#vfK2Obh|I9`oe8cKXtcFm$Q@H!cWQpGyK6q?5Gy%kxeMLa4VSg?U{qKqCm zAYtKz4MvKRIXeDP1CDr31A0~-p5=m>qAuDQ-dkHFp#GDz&#vd*dmqNZ=}qk@M`&}Q@`m;U6m30NUPv1+UZhL< z>CW3jw2i6K9PiG~bnO!}Liy1G=$x!Btx#KShrg*rj}N~U8Ymd%7v?!RV4Wha1o(DR z_Jl+REoLhIHW<$Ap%&uR6q>s*Z#^_^=^$GM;qQ5#y3^e47~5kRkAcxKN>Q|@QyCcM zIfZUl0Od32yTiQQeCxf`hb?Vj20H($)Og(*;Msf9DV%=SQ zJ)1gV{8OyM7g$1eh_c0QF}a{iaLX zZKJ1ixHEJ^;U=M(HE_xz>}Kr778$5oByO{cF;36SgkseM-vr1)#1W2=!BGRpwOL!x zU8u2Fg146(Vh^ke08`I+ceZyqwuKWvh#@@&P^yCJQe^F3bd&>o{&^5uRk+r3Z{umHzO(xf!Jmk9@hvs(NtgNLfEHhe= zJ^#2Fx>wP@0z#ptP;rygKz~A+@;u$2#^;4Hgg3+TeXUSjds;LqN<&Pj3q-|5_d=Kp zw3H*h2QXt@DdZLSkx55eDntdS0$O3_%WLUUyi`C6U_p(tA`~)0&LhmcH)j}L0~E%h zkkrZ&u54+{vyGo}g0oCxM-LAR{OpU?m~g>FEi?nt?H~V;g=T^pk5^>=RX74Rt>psf zu;MUg6@!~e`<#{cb^jT+uZnfYXM0T$+OMFsia)?AV-@s7atO2#9QvabUOB4k&LfUg zT)L%wMGP?gaE|_<4r$l^Msf;kH^3XTi*Z9xfkcc|a&WK3Ta~NU+Vt2uPnkFa%z9j! zH{ug8C!OWvPr-vr@A&8f! z2D${M2<0r_dW~KX`>I~hUG7FJ_uskWhkN9OJg=cvqJ2M?e*=6uBgi5FL%14g&o(2j zGCE)j`Ff2cu9pnj^5=QYiwo&oju$V97NsFZr_Xu~Fe~X(?X{q6Gqm z5S0lDs6*cxx}*G)l-6ssO~I=Cbjc<}W9u-$W@B)`nT%K_mJV>yiG5i`?8AKA`Tk zZk}=ECiYnkL;0-*)HNE`(N5ZY`N=OS-LRepx|-9Yciv2GxQu@J)1RfsAALk@4!fsA zlmLX0p*&zcVhvdMK3ouIe1&FV-wmuccs&Fh3r!PhBr33AB2FzBGf$4kTJ&Nm85^HS z?GVKvTZtaoUkuQrO=`*JD3jH9MT)L5ddmdtyW?u&8JZn6vFaLa0_0g^PC@OYS*6($ zBz-*`M9*h%0xkwz9x$W8t^@;Wx|GFY*TcIC!O2n}Dpa(*2o5{|-TPExm?r?68{8v0 zW*4Ok0L55C+VCldt(PwWH@G>BM7f1!Yj_Bk9Kd#u*toql@1q(zz~&TMTGNd^PTF6c zkK(v@2MH$U_^92PSGYd~l?}m<*+sA{H1%W)!KyW9UIJziEijiQ z>H*cAeIR@k)fx1tLOIW`Qepbq)O<|cw^|g>758WfI%5n2xX@Ry&Y4xaM@*me&`r`o zv0P&scMv``xW%>FG?V0U!orh@bf0N8!p4IOlo%_)14@2&evuL(#GGLGku}g1fKC_m)I~qtglWut2buKZP!=gmB<+dz7uaHDN~$V=z=f;if3(W94FOo{qF? z0^}?V?d{ej+M!fR6(HZaP@!6s*;;i9YR_s_$*a7w1xewrn+xld!V;m1AB(>MO{^W| zM95S7$nyn#c0e$sg&5Hr{c*>wVDgg!{dI%rF4qFj3UxjXh2xwV?*ootZWu@WU}xYn zU1P>GIYz(qJ}o5?=ZdfvklTMZU-rfE_;6)fShpa6vd2_p{cJDBpi;JvbW+7FNMi_@_CYiHgPYQ@7RMgPurV&ju3~mv)y$#F z#CQ81;4S)WmTfNg;xnU9G#)hKIo?lK<0rCd0sX!rusy~FEkOqv5HlX0Ya9MZ4wD0_ z;0+bD_4T59fX|`7G2hH#k&Uz9(GT3PI5)wU+^@yWAxaAYt@C4NOo)R`_%VqNz)hR{ z3Vqo`^S27qwd2da*crVPT0*?u{z`L56Kw}S^M@`t`_06Lk`GIi4f9OtZd1OTKQ~4u z21-{!L%8H)9P*Sj$k+OwfnNp=`2l{H0hAjOAL&4YR~$F<=-fMQfo$-WzcY8PFMn*u zM1~k2w`ZWtfOzR{w%e}l%o~iE51iL{SDK(BoKAIL#0!Sm#J`dIY#M!!e&^oBGriwF zI(}cVS>Si+6jowB(+Cq@0XyJ-^fh#{*U&HH)ZM_=AfM~N0}jPLXw3{=4?X-{+~N9? zF9ktv03xQ6cKkjplsv$BwNKJKEU|p%yQe%EMpeSoqiizNz>S^j&v}+dWN>o?pf>*d49OX1STwKk3tfdDlp8z^<9B? z$n&+x<-k`bn|5ImUn>6B)34|+7OYy&5v4dF*^hnr+Fw3DNQD*2hh1=FaXnl zu)a3c@iOlw#KHvs^I`J@BJS6*M_ zEK)2-%;(JdPFf>7cdHJo>DaxrL5!X*!WwD}dB$iBpwESI^X41r>65S1CIx!0IALI) zn6X-b(5tyQ&YHy%3IDVL(ucAs+szFJ~&*7%zsHxp8=Hx~nSRHp3_Z2yYtDUT(DtMtH)USB1 zhh##d|H#1t11u^o2=J|Ct3YKh9iPywDvTRu=C0q|? zaGM%$9O#9@M}l9AM>J+)LNxviO-o|1!}fH z!8h!j+_fJ40Kmk2=#ulq0(r#^pBo|piC`+1!{B=csAWJ)RJhdQ-Pkbi(TX5g zL3s0gt%Ljt7UFPg=F1J0O3WR(b3*q+7GrPm2ou(*cbEqE(9a2(M#1GPS!XTiptIC#fSyA>g#I{3z5+Nv1(lAn&$*0B1As9%!@5vy zTv})$a(>)o$s@L7*n(s9eig1EF(qPtoL6P8)?vLh8VoA^m)9JSbWr&Rnd!g!L<^al z4`mov@u9#0aKT?HV6TyLG9YJR@p&k8XkWQU_gEy`@U1{6=8^WiC+I32&@=W)eRb~e-A&0djT3}eO+JXu0x2tliLTT6_<}Eaeg;H-cCFO; zPMu_OO{P%>ab7EbD98FS3z1i8%g zHP6}=H(}Xb?lscS^&}0X7Z2th4IVx%|ESx2(Y5$K#Edrp06+jqL_t*8d*pS;E|B)! zm*ck`Jtbaa{IYRtW(oUaE7D$C=rWHr1)Y6W;w6povkxIlm>(^jx*r?a&^w3nU*<#`k^o~^$C6?O~!yN?FI1Qq%bBF1lBvSb`52TM`qcB1NYvw)^e-16W2HMl$KyT`7q3j8+%{^Rm%uZ@% z*ILJ7T6*$Xy4aje=Zh~>8z$Epb_7=dDo-`=yjjK0CH?Eft*BCQr?f*nfpvY_nD_>O zsEg^w!~~Yj{y_DuAo>*m7h7r4nv3Zh-~M6HwF}GAHo@u!*5ADUAl(I$;kx}$o~-5y7zS< z1i4u6%4~lE__zU5`Jvzy_6k^qrh?jCwYyNQAu1Dhulbb~lh%9N5!8jE;yE?)JoDkp z`O(#<49Y4@B`SE#iokmvkDIw%-;FhkjJaaNfE|Rj#ydP1$0^Z*vJ3S_%eHU)A_+}w z_Ou*1P64j(f}@C=}=VQ}(m6Id^m8^}!2_ zM|#CroO7A6$9%oOSy1ZOonNnMBDBM`E11z^ zc1lLO09xy?Jgr+XHZ6=!(CVf`U=xG&*{mg#a7+M*dWZDOaSN{GLpMFqUvBU-I?eac zo3?B>z#A<}vzS}t7nj7C_&C<4-+2t9i3uJBmrp;yy2^803%eL1jBOVzIyUDg+`Rnm z+-mtX%)z;`?{Pv0^Xs#{N4|AH>V5(C$c>wUS)_x7ePJKGI%qZeY@Dy2!UGI0&BT;9 zgGGwS_0hZcFd*DWJ9u;su(r)Sd(4^VJApf_PT&SeJmQ>05*2GKx&u$SuEY^q|B8g+ zSc=PVpB8l52%bqd&k^I!oS78SIY^=@MvWT}KS=!(Z{rQ>>PelT$A*qx;+$@!1!9M; z5L7#WF&BnKUb_kE*0kL>JcxNgA1sz0K%9YsBH}PfYYjlCw*U;RO{GcJRQv22HA{wi zQv|=th)!N5j=>;31FYTQF&+j9W1(D+VE!_U@5u*Fig9Yzt#Oy>09o!R2NVY6cLx_m zgCX)e-}n|jY6z?4527v2zfO%jdus*7iQCc1(%R63<-%Q>eU`}-WEK#XnfCxTh!w7+ zuYec9TLERKi>rWAsJT{-DjJ2Z2c0u~2+g%vA3O(8B~^>y9Ko6~_Rjn1!S{ZUp3Tou zkaiu0<47$QgUZzaXp5pY@_e)Zu+MNB!F*c%{pYCx6yJ;>HM2~2QZ18cwwVEq)(2Y$ z3=EBoLzBa_g|*b;wuH%Xfc%#frX>iUfAx&9h-)v-=woqD(+AH~b9dj}2Ot_6;_&lv z+`4UPX=xq+)XxHPC&sQH;h-z6ghk4ofbPF@&Dg}nR5t?_>AS^!bt7s4xOMj~B~w)X zmuZURjZGN+{SUu|@18n#SRhxQV@aEQn(7f;jU4D()zF^m+WOKnK-w!o6LdPjQaJkl zhY4phz?Yq+!UlI}Js|6tIbFqqc#4(o(Ffm(1-lC<`t0M6iD`o$Vo}mhgw+wk$au7R z;v}>nLI^C9N7SU7Omp;2#cF}$OK1a}pa$J=Y-A4L*>#Hh9w9JVI*7TX-z9i{g96FD ztmYyD{gBPZ2>{N75LLLYC&YUwU9*$PKX(8$6am~m*&bE096#bMsQuC#DH1=E-g!p9T|m4=Q_p&&jlc2hu6lYz*ktT zSzjs*cb1<07s?s_a`UBtRM5#A?n+gR{%FMu7X-q^>|>^o4a_;9oMOO$H%<4jf}?af z@)_2cX-<);Z%$&g-|jA0B*8mm}eLV0n${m;o7$PM}(8&;m9 zK-Wse!*ic_j;Grwa~!+$HJCp(Y4*|Qvd&DXGJL}M3z|qwX2h{N56*X3jcmaCprPZ~ zho1!!Ds)3JLYc(9TBNWLAzK7>wifPG+6x|Gj+|qaTv%A6zll>$mhxP1ox|}yL7s9$ z4?tF=@E7_aTS*ze{DM*xyf_aU0i0vkssJS{S)3oVBH5 z4qzidgMVDl&QXzh5oiYJ@_IYRE8@9mE{O zco=gq2i?)aDX-{eS7#c=At;PQAo{3LCg=z)5M}huYjlSLY-p$QJnwg19bxRyqhu&R ze8w{~pw3oF<%*j#*NgfyKpq=G3r*)ul7N$qr539|C(>T4wiZI;y5O^zBj<_R`Jepa z_k0p#FJlB$1#xB@`M`16jRg}{H{(MOl(R58XB*gn{ih9}0;2#T7SgXyaXfZsvbXG?%t9M+Dr!0r*9c4FNP zS;Os+m@0Z2ryv+euaK0v*A4K48QXZSE?1eRjM6l)zLB7F7o>~d6nPoiBBW>Rh8xLD zUFBX+29UWdfVa0!wjyZO(k0fP)Ad*b0WXQ4s;7j7mCE}^ZV}{;YsWO|V{xra3`mP2!z@UjXApA{5auE0MHs;tJL9a@h$}p0 zaUTMR_X%pRZthJZgCprVF2N?OoeE>iiX3sstw%2%V`1@J+8!qMX!b>#`qkg0GO5Sa zJcs2N%yxvO%X4APf<0!@g|GoY6tOInLjzw?;P(F?e4p$>L3Fe8?W z6l=?AhJ~t`IjmzL9Wx)gId#eF z5^QH9wfy-%q?J$qF6}&_y(VaXibYG8mRZgh&})xE&N?BR0Cf07Q>o@M z-KN@hJLKzXZ%DmG1Q?d*GCRmN=oz&DNfR#s>PonRcL4+{g)91dz@q+`Lc}dt6+M>6 z5+H)cYFxt#w@ZXT2@9B(v`d6)lwn~5;fWt)$%IR}fdm_Yp{{nbZR=>$cryV8-BwPq z&LpwQP@%38BbL)m2E10~y$EXy_lCtprGm0nD3$@K3S)tQ8#+%qQ3sijWN8NhP*~oX zlPa`{afE^>1)(4>K#qd)S_!Jm1!-EY6#D+~7=94a^jW!LU$qnqLM`{9Pz);%=f83V zqQ*=;?xFn*5a4~rqwHva-;I$J^r7lPVHxjIU~Jm@EotJ@LRm8=6`BgygS$;65Kw7} zQ!|=Gq18<##TU^BS@D<+a6ScskG+TW7Fmk6%yXra*6)Zb9NjT1D@V0TA$QKnkP125i&-@{u(0T3^yr4_q z^*ZRG&hU8xbRNU6&biNX^y)Y{Mp2m+tJex-EL3aX{3qocuSYfpAd+T$SA}d>G3Vz3 z6nKUpD)@tYY+pWtM083Zp=CDH)aUxD03*7?7Xq62lwX}2KZ3f!BRmh9*;!9bG0_M3 z5nd&ASb%Zvj!gg$F=@x&X(2){oDakU_rs0Mi!#!Xr1*%gWZg$a0M?YTU-FM@D0tE9 z;F;h}`4N37u=EHyL3g-f`CA%E;V6g2=Q*s3K5~!O`{Ud?cPz!wI|Y=h8_vl5lqB?B zJaC2#)Ai_l>&eSmr0@qFmL8reBek?7x)ua^ZkjSsWW7O4HKzhjx@l!x=iE`-gly4< zvCGOQPnap9le;*6(fQT#Dyom#e#{f58g@5^Tu~Pj9oy7K*baEfu$$*kpQY)SFG6l? zZ&DQkFl`^)&^b=ayTmNeXX7&6h#ir1XsmFsEX<)%iW`}4PdxFY3yd7-&cPcaY4GL^ zFj;+i`sztq+gc-BtSjBRc?+Y|QmX3c>_~Id(+t8g1C42D;%+Juv|fW^tieTgF2D)q zqnyM1>-6g7v-Hll|53V2*5Tc^-r}L>xKRzxClDNAX+`_7v0DJPJtqDvl*l15Xizr< ztqfY`nzoKkEP2E#AS4a|2U?FjkXu)TXFa+TbYj|3rY@hTA`QD5Jk82P8Ib|n-bb?t zag^HrHi-r&xN{-!9_}J{_WeEm`Sb+9oAv6Bv6@_A#kin?=_O2~8NC{ghKyKn5Wsxq z!Z)3_mRdUx9^59=lmdKMTvU2thGmsm^Gjgrt|;o2(O-q0yK))z=j?KGW8h z=4Wvcvs2>84YO0-aceoz3Rg=zYs5cUoV9;wIIM5mxZ+;Fo}%C~!kNC;cH*LBELYT| zW1qsjT%@l)`#2SF=kCnUV_l+PEC9$!@9Z2v*#c^zXA}2UAw-+HDM!Nl@Yo1$WNP6# z4nS))2^rW%vDP%=rgDeL!a*UN!L+yErJTw4zMED6HmeH@Y5wU~Sf4J_{M>7-SUXrG zkJ#C)r6M67p-(5izr z16`C6L3uR{^rR2}Ifa-xQ*YmTRM!Id6kZCNyBT5bPqDO};%dL3zL53wtjA{^Jwbp9 zz#`X{ztN0uyr#m@_)gaXreR`P0=@=4YTBq;mJR#vy@E)AvGj5y>i799O+kR*KoDRr z*#w1b0o*|gF6H^`9_Uh5N6Zg>;>lSVqT%hQ)*bt!A`I6VG9WTBQ3Se;y~db4rZV~> zMQl948X(^J^a+)|s_^t|*l&S`&ve5r2sTk6EK?GmAIIdlU7Ml!>5ISHc9dG;IksV2 z-r_)|h5Zc>L!YEouH%Rjqpx8ZiTeafv;&X&($b_=uk!L**i1%>w)9Y33jElt>X!E_ zX(D}H2ZAo=%KLmd7tsdKa2)o>&!mU%KFW9LQn_9t3XZ~}?Al^bj>UEiYta(s%P~nO zEwus}+fcFR_O9KFJ|Gh%`WiR-886?ocm-d`X#DQl%kh8mRtBMZd!p#DK;G*av{q>8 zpK0!xgYd!2t_kU3pZ#9TvhR5vVgZLf_A|Q0V=Md_<6#UzBl;&0tGLnS4Rw@YFWT~X z(&c&}l`;F7mSSI8hmF|{>m2>|d0~CyJFk~_1d7ICwu4jT>);z?pX4y+T)H}b!$PbH z$c^-Y=f;|MZRWA%!LYGHR#pJ9R`6ywHe73IE|d5)!XPqZGwnwirT-+3oZym2#q`;WhyhHed~ zRRZrlD#A%vKruH1D%1xnOAF3)oPlY!U$`6~jP^U7Jc6s5 zfbv5U8sr5747J8}5Thet0J73Iz+4q26kvoEk}t{!WLc7-XAGVI&RxG&C~Cc?4ys3= z9}%37a`iy-0&Y<)Pg)UbTzIq*XZ+EZS_HX8iVG;)T9kIH$9l0tch_u5m2fk{T(N^@ zEGkD$$DW<4aBBsezM6aqcx~edB zxUXjXVQRSbFxB@Cr^mnleR}rEZ?U2_kV?)1Mi_Yn$|2UqeX@%8PnZ*c-O}Q88tKNJ zOM%=1b5mrl6%?KsT+5v5o^qagdx`Y}$PBOeU%IFX(9HOyO2|*t3Ey z`l~<-3kP(yAa8+%qtXx%gc5*u#=^K;*EXPlQt%3}RBYDP3abKOL06Y<2iG&Lk+HiG zZW+Ok?lxWOT2I`i8&vOQ#@OW=UT%z341UVw@`}J<`V>kcEP)xYD8Nd{3RY17N0pAx zwv>(H-_K#;*g@Nl{qdP9OE-whCC9-Pe9j&GXk7>}CI6$SvQlCF^DHe*$Be6td@#d} z<^y!tp2}7wp>#TPS9)=UMID z)Wzbva8Dsm!&4zov8Gk3(%Lq(_ISU~7qCa&D}L7%>l~`+4)CBvIS1*;+#BzgtwY96 zAv@GHSdIgTxP}Cj^z};qgO>#Y0(_ORR$pEBK|AjCet9*rh#9+c9v!ir;D@&fUZjM?AinF`>Ge)2%c>r0-myclYH z67v+FJJ-gXITqzEMi<6v*7-T~5-8Z7Yue`s;skEq@o9Z15{I z^l^?`%a;Hy03r8f@CS{N7t+(VE&UQPlFX|zFy@laMZ@L&e(WO%ZPF5?}t?yBodK6c}J%sg9ru0Wsfz5DI-#(UqS#1)=3=w|Gin_xH4 zmF#~FGgGf#g)zWdZ^k?yfemCS`Q3Z;+RY&2m9ZVf(gnar5$cWf@B-zK#sm*7=?Qef4HDhG^{O^B_(d9Gl1uQpVZR+L>)Zw9#sJ{OBuYZ%qhi<25Up4oD_&F$x0rQ1Z1X{TsI*0`f3_ z;wm%I!n#xKfILe<>~Wk?6R~|I6jK9>d7uOTeqwUS_S{C@j6)2cgs9V%>b7P%n;NpV=&44;!f~?onfC3jtt*vix zDglSG-@Z+7epkBv-n*#>v_FTnYsNZmOy(I$9wiWW&%^iAHt|k~T(u~mltQqWhoN9F ztg(Yz+*1a4^>4M{05HU63r$%!v{=ZjPp& z_U6>f(Qfs-E9rRcHK%v1u}K(6Up}28_V9q6)B9^R4b?$kD%D^b}_dflc8K>cew|^^k^%s3w4aK48QCx zl4-lKP2uI8^!3bQn%je4707VI1v|bfz+l;PV`=XSelyHql^vn6Eaejb1;9*Ks}mJq z4eN>42!)rHRe3%v+X_5-zs^xE2HjeSJ5DS?i0P&{VI!wsL5q=deS&+;xrYq^p6$Hj zS_na@J2aP};GmmLtGMsPOm zL>(}ck_yJySRNK_=w7jp0 zmaZ8%W$Q}-F4_q|$9K*vodc}qI7%%I_9Lt<09(gzBU(}f%DR`W4dzivT5FsGLl}G+ zlMMCpJ-%~K1uQvdk^BK2!o$M#f?utDDsqp|lRm~eR>Oya60KZXJS;CF_;4Ns^sYfI z$i6$@j?a6Y14$%rs6V71fAXDH7ChFR#Lji14+0jJUifco@a8u<5 zF#x&EvtAvK^pKX`ulwH(WL_&7oC!WFZF{}*skg}akw;t?K115mYK%epS;fLX4f=Pz z1+Op`t#v*VQqY#`k!UV>PHUQSM*8_nnKc+0skU%WD8W+-CzGvB2Ew{OhWY zdXbDZj4JeBfGItKXBmwAV8w$53{AZQgVDbWVm;;0iQ5zD7NJhR`}>d5Z~pcd$fwhE z6RdFi!NXLg32}XMmE)4ALB_&(j(h6@w?Ywxt%}WHD}`_?>)fwzaf#I>gX$FqxCA;Mm#P zBfwpgF2WE2<%I8(by8t`T}SsG+>dO?UfiYI0K9Vo(i1+-ykJna~Chk$MXC>EhZy$iv?AOO+n$d!@CPFDT(p<|eUTXELJ7 zpDfZRvn%ivyTA2KV!(Ej*|1f_k*zOLkQX=UDT@QqP2uAw6oe)f8ztB~G?dn%;g0S+ zhNZv~1ev0(mGt701w!0n;fgpVcBO@_mDC6j?IGEsj?KU_r;RKwtO24|Q3hnou5YE8 zm(%I-@4n#BVXUTPnQk+Gy4jCtTMK~&k|*vzpu9(0D#0gbxTm{D@6eYCz!5@&1^E1< zU*PJ_i#G&_$_zp04p}}vM(7(4snBQvoV36vtU=J%-k1DJ5oFmPV|R5>1QaIGH9 zQC5b44d~>YIp&I6%V$?Wi`NR)lzZ|uRJ`UTz&5fe6e;7&R+yZzn8%`3P5R}z)e{n9 zcf9bT{o|2XE|l}RuM8vl!i_wK7jG)KCghaY=h|=Zf%K$Z`MEL{$ImK^&$A^HO2xcq z&=x>Bx5_mFRQd(4rz8cQg>q_?`cK-6NsY+FC!c*vDK2Vc0-QZGK;SCfuF&J_%ge|>!dk#PXDExPHo3vw6hDBF8DxMwYs>Re)2#6SDJeM3M|ygX;otwzIt-v*CD~S z?z|hwD!lc8^CT&rc){*?m}8&Z0V#Cn7iWo~G7}AJhprw4I$)qcz+I`jIlcbsRr;91 zzgK*3NA7?On*kT5$GYI!NjUJZUlFn(&Iva|gN+h&!i3mO`4W^+Q>$($6fvhdc&6nR zu~)kg>DY7jU;?vFoCt-D?leJ+Lan2VlLnY@SP5YRBho*QxAN@BJ!JYkU}K*myQ_e? z#`>1jf>NwTs2C#@$Q$8mknAyl!1y#5ibV_S$ab8aTMUqB+?l=gh_GFX;2Q6ywM~WO zf_E{?^z!8qHCrqbQ6vb_qrtRtElgttdiiCV{q$qpmA_`P%MqYxEgr&b(+Tds0#?H< zVz=tB&TX*MEORcY3Q+(05wvK*zh1=-v7>*80LNTN*XOizF5STRFmCTol$J3@9zKsi zVqA(6>&Mgsz|GV(Q9?C`s*0nqwzfCZv&X;3J-86N9RsyXF!d3D<^mVl=GHnV1@usG zc0X+ZI9$HR066QkY2^!{CP+bH8PM$!F6S)z<*&Xa;L}~9R@iD>leG$ly~EUx zE78y7g$)u%t}sQn)RP5d+#lmL38lzs2J3acDYMg}t*2VH*x z2Zfu$MptGvV98Q8CEBg`Z+eP42MF&|K16;;kX5V(Fi;i3vVl3Xd`uly>I-*xhm64l= zuo}=e6{rAAV<-wZNn<`^UH z&Hlhwt|I|LWFR|N+))AP_Sd;lVJfqINfWKgt^yoOu<@Cw@Ii}Po)$o0!1)0=#c^x;PD z%~mk_&pi3um}AFchrHtt?b`qVf_~eUA4ldhzv%YVQmQN3wVl_au2BpdfPd*?|AWus z_uN*FmDB2K!R73Zt<1F`Oa2MI;u+p64V5jykI>lfqVx%CCDwiPkQw88jW_l~@MsB< zDA3NPA-)XWbFQ-6*mi7No}qi!^GDbWo-pTHaOzm+W?lvT{mk`k*pnMA>EZ^}a#~@< zgGVZB)sU&C4yrW_j!+T{0|T;uab_mGUX8S4;*+}EiI%1{7%DE+ImjXn4XznCx&rqK z4ehHu*+VT+X`cBn$LN}nH9kI;y1_IHb2I55{_Yp)^WS_5J&NfLp;n!Eu>|Xzc<)ZI zPF4|)!)}9ko$-lOO>nrDnFbW;#R0&st9OK0vNkMfnjcX}fU;uESsLywrLjTI^tyC_ z3OJ~M6{r>f*8-T7*ac#dbeHYyt%b0TiGpwfu!xa3hFMCz42EnJK-)gCZhIV0WdUGP zGt?qL71M?SDku^(dyD-2XKDD*`GCl zjYEpW4w7Mb9$7(vhmrAA2ijl1BD1h>3=l*;Jgj0hv^&@_5Q}q@IamSkG(ewiK*<^4 zg1w4biYPyJ4hl-fW9btYiLS;oV#CZtE5eXX-GkI9qx=fv*xug;^x$G18{>P$ESoaP#SLi1I3&6K>RGqpf-bkhPA+mJl z(%Jq-9AbY=Hm!iWlZ`_Q8-(4pX+R`=!l@6aD!>EZ^K8rJsIQON1?}OAb%$3hn#_Y$ zmy$c@?0iN@;F0fJ_^!OM06@d7%EC2nNh@Rxym|)D#*uS^8U#xnAodh+ZCMrph5Xmn zz;Rw2K3{{IQGPOG(xeFX4pGdvuwPCM0Qem&rlt0-G&K)h*^ycFx}LV0F3YK(1>9t| z-1#BzoY+D3tSRnv2B55oo*2g6b;bq_&sj@acHt1MT}P}zYu1@OqyW_&ZnDlPzFdKm zS>X!YE9TeII1)x#A?Q(92{dvbx3CGNrOh2WSd)&CAV;tcV}aGBwT}f}cUOP|#_!Q# z3^#;)#Ov8!^obV~6-G ztbJS_?l|bEVld;E4umqWeq)F>^(q3mnk;vShkTQEw8NheG(e_cIG^$C2^bk92zCw5 zR#{X)h>Dme(!M`Ai_mKX*KVF0bO*xgDiet6s8ZuaObPe7n2w1wQ{ z$5VE^&iZja*UcSbBF3TY6VT|n5U23&h2qaR53MxS2nbDcD>{bNQ8@ zGvY`sm7<~+bQ!GP$Y#m9p>&*f0RTLQ|KcYA7X9<6IBAGjq_18F&}Gh;#JEtJUcmcV zPoNtFcC=B)+K^6K#v_qMZ7&VE&lu;>-ckVBUn2Cw&>n^yn=mbRCW+~8jDARS?sYAz zvz!C|9~ti^w)q?_ZC)hMa~uLqlk_ah<(Ro7svGz`?~QT#DK|%ZbPv(NC(<&1?_YUy z!I*fk1*tuj%CUw;jL+okg8Xf}@RU5Ni&zVw&D&qgP`Sx*!{8dP|rU$W*5)9~fy$xx-R^sz2ID$_#x zZt&c27>;CVD+Y>8$YnZC=-U)7sozDuairNIrY^yN}ZR^fR6f9iUU3QiYt$v=JtAe^xg+)d+#9q^d~=0pZ@x{!IyvhPyZsl@#fpq=)_Bd_i~yn`s&W0h7pSlk`!5B}s&VC<{(*=HYPH9MfFY;St>F2@CtA!?yug@DI(?PBfN zSYAM%a8PJmh5 zinPUhfAmM+3+tRE0w!NR<}(%VW7H~lyv|Q>2P~f)XW-egqJY<|z2E#z2@1k)W3A)3FEU4qlqXOa*16bOgxfo0 zQo5sHr#b%CqjZGji#<{Zkw!qxEs|Z1YcJEIAN?>5PTUSR*8%Nakg0n@!b3e4r^&~k z;lc(uu^0;gl@sPill0ocd|0xsSZo3}v!^|~*4Tm#0M2d3-Oo{Ls_hPva1P-M<1GiV zN}dqgH#_^BtlCzXev)`TGK*hLrne{Vq<{S%{$uJR?rsm^*2#$u4bZTcvASd1;v8O& zY|}+%tm7r)eEjr99EE1tmsXA(J0sq$T19{WIz>Ts5HqL}MeL*8R8qKOm@j3M!d6gX ze1t;11kmxo`E%T!QCwOj$~=V#*80XZ7qtHcR$F&+O}PCHq&-AQ!mT8lm?Mi&1%MT4 ztA*U3Ue1Ig{4P^+)1p%(jtPoXHS|n&k<;OqEtZ4l( zS;Iu7NUu%<%x8os1$HxwCk*e74kjT_O(=%Zn28A%_sVV~SsXA3A}@HyOL z|CE!q>-Q0F#OEsVke{p>dC%({n_$NG%8!7@c>A7VZBL`Xj7wb}B|F7_+}Z4L@Q1 za9?HK-DukxEn}`FKX**Q1MxY4pU=V?5`R;h3k!+mto)3TGER7tXZSc>+?nq5nK8wt z;(DDrMlCfJ_n-6;kO)%3Wqz%zw2ox zS-?D40M5MHy5&0Udy~|r~DWEXDAQXWox3YT{{d=MW!|b_P8kIi`PaeFJ!Do zQ(Ax4Q$L(z!SCwwd{m^cK)#0*tf}`lNakXkTNn_mUu!~{&(VtSGhGWFX{qJXu(NI| zVRTTws)XxPeL2i2NIke?8?8aAg&m~Xf}U;VnmccQ6GK#UdNK7f-F^Gb^!-2i)AXJ1 zeh+ZHMF`q1RY2y`@*;XrTmcOYBdR()x}u6cS)2q*Z*dNy4|?mpZxfJCtQP~*UDJtT zpdNUt#$9u;I*Szq7ZeR>?sH+At=Km(O3g58d9mOY2&A@Fm1fC(gx?;?4PQN`xNKV| zLR8lawa!43)@f@;K)(3$cL0%HtVvhIreTRV-onyEX#%V#_wJ6TCU)@z_oUu#EEUxT z_EXk^@*V~Ndd_MyNelJOoUM815eWpwq+qc__}ZUZ+&e|$)e0~~BPP&ACZz#uNCQFp zE$jjdfJK?jwJU?DbSlEk-F`dPsxq-Za{SWLBEp|!0|1I{8fzoDlhZA;PrR0T{194L z*tUg$dXMuE;H+aIQ~7`R&;Chj#Wg**xt2cu%`ehl|J7dx)2q;%p!1iX{~8drm)7T| zV&Uz=q+d;OrkZurVCohW_R;*y)KtX+gYmavCaYj;{i%W(8`+6j5H(yxu&e^6#BN-{ zp=;Ur!8n*1raQm_w2FCnj~&J+Ow~&=3tXCNp@%j4Rw=H_aR+hIg9#&3GaO&WuC0N- zE*dCbTTGkoAo#n8z{1Z9vs1!@=N3q!xWrPs!@@gb+|p4%(2f=_+N1?XhI zJl)`Mb&0G@T$J6EBiULeQG=t!VCF$~T$ZGe-w0}m-A@x>#WPxwmYB6E?#phZ8B3P@ zV4>whc810(oRDB}3eY$Kl-{~Kl6pIdiK9Pdk|^o`Dxd%I@0c&Ff5T%~pzi`+6lN?c zK(Qxw9O7PXX$C}cy_Vxqtkl!Z!d>NM1W-^)g zsOIagwS!HNWoulA?j+6BTBGC*UuI!OpXf}&(U=+-cRfGGj+{eBPa-i!M1T=T9fc)> zT_KjNYR08Yr2+Tu4m6N&q^-fnW_nrOUa%w(5{MZ0D3B1?25ORKqxz;&wf@njZHe6TE#ZOfAi zZL^gH;sQq(r!x{sBCZ8!rlN7pgaw2CWF-$EQ_*t2MUdmHT&_{*O9QhVwffV!=!^Wq zw1u@&Wow^&34X$L!(0f4e3l@N$Iw((NWOTpve%mo9J5KnZ4`D^QZkr}T#zYM`X$Zd>+M*RFA9-v`+Pp4Pk2$_2NfqxhSG8_tg$E^LXo0$?NjAdp^eW_`5TvKKLSYR?gUSGe+DQQ zSgR=Dtp!L0jhu@Y_seG$TDlelYg!j&JZWK{q>KHt?Cml+Jg*D7E6CED{w zCnQK~6Y=9;ei|Dt-lc!6MaIss;0-f!(G6_sw4w#SUEds*LHh0d1?{*ufE9mB@1TL} z!)J1j&(Ni4oU=S<-|U1*KV~~RpMn!N&05!_n=xayRhe(AR5%Yl&*%CIpnOfEaI-T< zo>-!5-OYsMgWMQ79xa=`csl%K%$f-cTGlWkAP+rpVUNxGA+cj87=r}Kt`VPMoSZ|~ zo^4q%WuPET>ua@MF5|J$8|p*b^MPiTN~ap+l-9hmHN0pfV&(W=ndBy2ufzpzQDcC$ z>>4ktY)}>o^fhuRuN<=V|16^=PKcZVWX%ckx|+?4G3aNP%n3%3_TB-)(g+>uA4%_i z=LhLqKm615==(oNci(-NWQlPMX`G?n(@inoO_Ct-0B=B$zo6h5-N8!giB)@B8)j`}zwc z*ipJUGJtZ!YIe3m5(Fm(baPf|RXKIH5dTFH)Edh~U;*m{m z{NRT098{Rp1J7%4@13(KO+Jt}t6}(77G=rB#W)0;GH;o7WmcKm0M6z4g{Tf+VHOMU z%Bb#A!osF7V*FOlH#?++E5Y4iH-YaiDvucI^dt*tv4`^f;LrXvj@KH$F`n9(6otSs z6RZ`qlcSwZIX`Xg+1HfBz{-R*$K#E3NnvOv34)-|Vzf3j905V*@{cIfCnWJ;SQ!}o@MuhJ1`byMF3#H9m zj6Hb3=3@n5L?2khr!4%(Up&_Xly*#%0F?Fs+$<)2>>=>e|RD z#FH}K002M$Nkl$5+FT84n68GV)3n(D@-TmGGqM$vSCXE6_9{ zqn&|_byL!WBUr${%lr`;{~PCdiCWP!4{g)lC}OU!SeZMGK#bK?P#lf**mPRyfyG zC;`9ZKA-29={ir*m1SK@$Badb=LMML+5ijuU2)T?mUKxg7ju6Q_KaCzr}Fou(on&M z#msSXWdKzaw2e!TkWf}hZ4wqOR>V;!08tJOz3R_NmI0UV?R^teRfD*FT%@`eS+ zU2oD)tCw;)TLLo(hvVW|HtBRge)_8gk&QBq^Thna^Me(xdCs{Do^yS3uY7F#w(WH) zcQ+yaE^w27^x*jNa~}jNK_Bmr5d{sTm-k45ICI=NmZxop4#~5ES@|ilNNtN(kdM2f z$ggx@?Mrv*8t;tL=Lmjn#}EnM{a)~od+XwZZ z*E@b+_TOuRAEgO&@^9a@(r2F2y`1~vKplOB27b;eX8zdNAw&E;HYdG|W2Q9vX@0OABWo_}df-36ATo<0BtzMw0tL1^c zg380ly8*l+ce68Fc-I}*0x0nGhXcW6PiK1g?GMRnWVeDf(kxqJmP#Exse?Lt(od`B z>f-BkYhsMab|KP#tVoSKx07d9<9e>as_EIBJd)T2GMi9}X4O}lZY*}zMcsU#sYY>%qa_}0YJ8~wgiZ$AYyYv znz(f~hwb_woC8fst3lBdmnAtb-WFRtUU4Hv{lIPhWlfN$Mqe~l@}zy0 zwJ~{aa1`7uK>E(kp8g5$9kXCVD}Q!jf{#!<>Pu)R;c8S zt)-<#C!U{{W4-E&1$N)u?GTA?FyQOS~Ls^uhUsR$4c-n8s&L8DkQ}WuR^$Xh5Xvi1q~{{s^{hUmlT5 zqAnMf64zP`oZqsg1|l$S`9~|O_tMPuyUJJog^r9b-0%5nzDH?eg^D?amu=JM%h%FB zVnKaY=rzHLz)qk5Smt>?CoBFu=nTN}d^zX7oI87$U-r?#^CUj^J82x_V{U9O#>M9r z({=9L+z6WOqd&g*IzE@K&P}w*oZA=IhHgXK%`t9yOm4KjR-Tm>%8w?1otq+`DG0MY z!z^?;>Mlea%1!)(_neFCm;L?&uf(^^1G+xyvoeSBg}maX(G7)GN(+12+NmH3z2JAA z+@jvqN~*lE`hxn*&?d*>ljZS{4|F2j-{?7C+3M*0y9uFhj7u+weCb+quI+b(C_q>B zX3T|*VLy1Tv3TmR5+09ELcL1pvBBZtSci_c6(DANZpJ;G3IrYU-&(fuc%jkAI!M@%}d` z-aCzcUZs93Mi#zfqfs5z@ZFV;j*SIa`R(8TBF#;`3?=qz>ILpWEE&YUT>|9p1Co3{ z!~)cUOY97VIy`=Z38xq-73G)aUkBq{qEL88*t z)}8m>{V?@)adHCwW=}6TCwTqikA9oJ`0V$9Atnr$*sYOqtP~Wu<^C>!TNlSL)etm@ zDn&7Ie9=1YEdrpybGS)K20$P?;Y}nT$c!>uEwN-fm_v_wzr^xHN!IMUrlXo+73Nyu znn*Ti#-eZzFgl}cvmT8Ja`$gwzW`0psq*Rsr0HJLda|}g2@(X?9$?krfjB+~+SuL+ zJHrCjsEK#pN@bXH^9;s=d#vSkgj@BD#bY4*AZ~*J0_LB7@!K>$JchfOQ!zmNwYVh< zm|;(F({8y#3 zUjSmhgl~%6Q$xE)Sjy^P+Jeg2-3&vt0CqMa{H2Hcu&K#|Vp6f8N zHPdFEJWf}Djl(UH6DW)FkUE3F&vXQ^JfqZy!PKk7%$2F(=K^bMGiD8NVG;<>8@@4~ zUU2M)S+hs*=MH^q(K0FUK_E2t6N|@lS^&cqCw6|d7968kc35;%Gt=~MCH>WZ`)_0` z;~v$*M!UzGoKk_n)Rj<&`Mj;OD>g63DAsz`?YTQrHUn2EInMw_LPXMt0>aZr6e%j= za0j!nPbsxAy}nDGpF3%7elBXc>Ef%#JzdK#v&7nLLkVfeV8-6gn zfo)W}^Nk;*0?8z0idtJL<`O!W*x|Kf_0}S;06t=8cS@G(9&|9n_6h)}s~$#Teo(yh zOa-m7u#>1V_?CO2jRG{{J)pC|Obd%Zr?T7Dwb4rAkVTENwZYM%6nVD7+qldC4E&CC zMqs0$Q2?QkxyJQrTNQ<+98ox%t*#OjNvS%*<)hZOf83+MjKFB<+M@4WlbHMyaimPW4U8sn*q2i9Uzu?E?{ z+)sfPYa7{Hu|8=<(NZo@ssI`3%CXM9yM?gPGAb#(f z(8?1VB(A1w@p<$GdfI}064+#bSg~Yk^PBDU)69blXy<39MIlJy#K_l zG<@ea<<2Of##(rflZq~o;a@(P0`pKpYlqMW3=1o-o~G65r|FAd{w$sG+(F{lcL_H- z#5;0C*&loK@BcFOo=uWw*PY+3ER$7P-gfz}YTrFQgK-=OOSoNN zKw=qMl0hl7D{bhFK7&FL`Wn)P-Y8ZIk-HlViXkjP0tCQ;8;tE~@2bmtZ!*i*{(k3H zVmEtgsxtF=?sMGTtRIJu(^YC7M z^Jdo=JTK#CVaQGa^llxQO|G;0vb0rTXyGdsO(ACou28~)bi%X$`49h`__i_=GE0^I zt#pJ1$^y5?l<*i~(mQ1cKw>NzGe)UvH}>kgFvSJv4^Sv?W4+P}f$I!V>mrrW?1%2) zEO22?V$BTRb`cr3- zCU{WD=qf2vy5&HtAGB)6+S$yeq6>Oy2{RAy`~T{HOx-WtN_%uMzzA&(0`82L>%vm( znRL1mds!%x&z_}%yv$hESv2zQHg4or1c1B1Ily{1_iW*E-o^FZNut960MJr8`v{){ zcmuYI9baD9tQE#p1S}zbQyZI?`p(|8M46i z{q&RH!jp(l)?IA}wGP?z)KWZo2fns$p}?ks<)+B`G0mMt2j3%*m60k3o_)$XCi{Z; zJra`Wr-=^rD9{|k;;z-B+t{Qa8%&#$Xv0hlyTBKYrND6pd>k$H( zvy&CR^`KfUAIb!khd(L}0gRRuWrxD~n2m>)1Yh!&JLC+gw7$?biYt^4_bYT&F#H+( z9ZG}i{oe1~0jo?xVZnY`NqJ0@eClDFzq*e9(rK z9AkDY(Fg9=Eo!NNINOmn1eZR;-F{AJ%9aHIs*2yf2pUyi7v83`~%``NMyv1rSF`3z4@@cGWMuWGQn zay+WL^9)^No^LrNRuH;oD1cnE=Mh#dA@bG({C;c_c3GkW&d?| z`lGeiGlZSHP~!B-XW4)0p`49)CVR{Hco={!%R70<4%iRpktTQxtV70G!_CAxfl{>1 zcUpUO$@vPfC3@ni;jvKn(5^~fa1uaMD?c*c`Es1bg8Acv++g@pj>=~S;yWiXFLA8^ z4;rxs;yU^*&lr!WWzVq~mf{?QD_bTwf7!RN3ek7@Libw?P6~0IV-#qHC5-#!Ir|;J zpSO7qm|NC}=BWU-+$*qmys<_EG`zBP6#&fNwpT$Xt2=z2V+*Srf9De$GXFEI+y?C% z4;^3(3vKYL{o$4AC|_%3%jnO)X*hrz0zB5W{V~2#5a!x*L&i-!SL>=_S=sQQCAkT! zm&TC54ba-}tFd!tMJgw>?3qkpyLLrh7iOCPk~er-!+Il0umzW7yqF95)ldxqtQIdf zFd7C{knf%ZWvR7%@)5=$3_Pn#bI?_5d3_iXG>o*OLjoM4|3}6zqf2m$4vru{;5#-X zVxBJa(HOw(^{>3g;q-6d^(+#<-<`UK2GZ>E3P(ELPfs2^#Ta#v7GFG0Pe1!jy7Q}_ zAgTd}=)d1`TGGnG47iB!6!dZr2@(a90lL_&3kX)`IH&Ud7sQ4^P%IuhBoRo9eE^mj zA%tpCd

      vfKpf+L*n-(l=Bci-hG6WOzsgtW?*0^OueJm(j|_IIRyDtuLHnrS z{`|>P;4grvgoTMr(p5l41E{x+Sg@|HegMLF2syJoKKbZ(>7$Q63I^UHkg=^5vns%C zXMGu~#};j?2$*y_WJP*2T$z(t%mN1iq9*MUXHvQn>Zu2JTEKj`zVhSgj@-0+9~(DuylE~^ZbL2m9#PUBCX<%T)=`v$YWY~4vkRvB?M3%$7IzG zjc`^i*}A%MuuiSq|BSg}SI2_#IHg9EW^=2;3it}+7GWNhy1N7g;|YL-kD;pqodtzE zv!1yafAuwd@CY0P_Av}x#(gw6c8Pj(6f8!FxEuB8vlZN^9>i-w;9_fESoQX>XtH0Y zA9ZPvz{>(-?uNm3;U^DiS6M22kXmUIE81Nwi%%au#*Mj8jlnr+1yG>O$1dla?w=Y| z&W;s;aRTFZI2N*n`*DeC@s)OCSH?-XUV{f~v8;7OHm%CSk_MJdsfF*_JdJ}sZUL+Y zIS_aU3uPmVwwCOlPO?;6SzMkCTdSa8v4%x}Indm4H#cEAh92XOH9(!BZ$Ro@=QOCNk5`^~n619UTA-o~El@ z%X*{lK8HmRipn+W8Zp*Ku%)HmN2tipum;4s=ZRX90{l3a$V4}(t|dWBxYy46X1NIC z?HsSMG7?wC9pFk3=1oEHY!76 zZpd)>LU5o(4x!5cd8Fe8^tqv9`M@vm8()TXhViJwT?_Wj=Qmj9gr{3`Qv!s0*UcZY zN#JE^2<2w*8h4!NUgLZBg_Z;{YuX3PbMk;~EB91z@~!;~VC6H&h>y52u8{_@zF1p+ zh7{!5bN-@Dd8$F#0%$Nv#UmYSxnCk15)wiEjMakH!I#h}zI2nx{r1CCfnYiMujSOu z3D06YjEsNi2)sDbBY4ZTUL}g8;Jv)ZYz6RO%7cGgn_e0lJH{7p_)cn8El_Tt;8MN< zi1IvRVPig|Pt1Yf$TAh`!ICser$*@2OF zi2td z5LHjjEq)L<3LTOE9+oKGaxOw+2Nj+wIGMx+uj@(SSB)QW!mJkqKeCWeB23giz{fgl z3h73aiDzIovM^UT*L7~9c#I#jYNlet~mqvX=SVfWKMnlaoU-kW~dZ7 z2B2*NEL|WG%fxSmtvj|z`G=uBQp6oqEDpwAgtli0+NMFuob=NE7IbUm_@nw3j!63w zbIVF#yjoksvWMl8Au|4hwC+OXx}(#2j?TIY6LTm#U?So*c=qzlWE9Qa$0FHC4V-4~ zZGl%>0l@88#tJ;UUh5ZhIw1pfd2<1!PN^3dKI%h3)5}8_UD5Cb=WN1@T`Uk26LtY` zTew^;Pqt0b*$RTem_mV*v^H+g(>^)@cuVuM(BXhugrss43)YOKtcMK;#XFz~A{AhC z0uPx5?MWOru3us0>nf>GOKy=EP+a;O^J%z~oi7>EWfs;E^Iy**+5~7?W~acqDPaB8 z<#2`|Ft$ZS=uZxEMdGQHpyv+X9bs+WB(|&-E8O#`Su8`iT%pl6pS|+VS8=Ucn-9K# zH!RxQ4t;8Ay9LR)ytbN}hQ?A&&rn*~D3h$C#mLP=a};=PM7cGfM7prb(jh_#vS5z* z4(Ov(atF&%?0OME5SSlcd?&9c;35fz_ZE?^hnHO&#)TSV=k8QcrvTN}VBDx{Um=<; z^4Y?acim1Oz?$9dF3AG&h9FlZXNKYErAx>OHVA}Uc)dG$m5~WIzSk8N0EPCwP9@}X zZAY->^`V%YNB%h$W}nYc5-J`*i{JY^`yh4-V8&H`u3OU&70_X|25|XY@73bv@1h`# z>%&5c?8@;v0t>?fP#RL_91IFP!jeWC?vk_Oqi;M%d7!e=Drx@&a@j(Hg64BKI^h!J zb3vTPXgMMk5uK!uS|jaljzeb(Edt5ikiy4M;bKn>R|eTdei9p6jG za3%A%^z}O1QBDgeRk}X!_cY^XtyR+9&#M*|>EyL4Hv8+`=K^=b8O&$0sZlv59gH?KXYV% zXvBcC?mVQC@hium9lNSE=8DYV50Pv?{l0NcT z%tg!}-#~+Hi?%Z+2W!6svnBz$jtu12;;mK91O&uvtT0=ZS3P3V(@5l-;3L}8#UH!{ zKlwZNQQ)Xs9-zm3>Pii(C;h6f0iWkftGA%n{(6tV-db(WQ?5x!C;^r>pYOO_kJ3W= zIHg*xqofIa@-v65S4oB(eE?8mpjNj#y86DPm*8 zdNWZhXSX7A0dHk=fr$x%G2N$n5ae$UFVMqugAqex&@qONVlyXa zVE`yB&0_(97X}Af!XOWYJkG&WtUC`dhHt8Eja~eQdJQgU3%}N4|+NP=N33 zZbjv454tYUhfXXZ?|=V~5l&1bI|Z#vTKsqbHL5sI6~Wq0eKjq3x_kwq^%OVVo|_~t z3ya+11eJ@|*;QZ~hv77bYzhEik`gXtkKPF;M;Qy?PH}A>aBp#IA&n6{-^flO2b|VW zGG!#~V&`GPLuEXl0;xR#IU3sTiZrHQ1JHrKqfW3Ti9H0LN zt}zw|$X`b5=;;AA))QP00OcX;xF}5sIi_gw9xldH80!=(OE32?O}>aM$$EE-v|p`t zW3gZtPa7pK1Z~Z@l~vju02mdofY{wak-naixN=CQ?mCLJ2_S9N|2Ev2X1dnlnrtH; zu)PPzMeRDvPWecdW5i;{;{av-lek;$~~IveL@L zWSlbK!rtQj5xbl^!1v(zFw6HSO#}LjU2+FzW~F6eoDT)WE+F;x)yrw({{5(b_Pc-j zCrr){Q51k0&N^+xTG+rQV|sEjT^=97B8H+wSXk6_VSa|J9pVPq031;^=i7h$XQ^>u zgahjV&M5Z!1`43MS%L4yFODH-sndqb^zOa8oFvkgMteHa`qNL-=E6jJ@9nn$i=}k? zt*-!>TG9r^fVXE}U|L&G!xuQXc#V39%xz?-B0N=O?!pur3O5&VBUbN*W)F{t9{_xw zVcUgUc^7`_xNb+xI8X54nj(1#o{~>=1sgcOO-!nm zK;5QV;ylJKj__d)LpC7v6qKkF#z*mJcuVCMiix$WMWY!vej~0B>FTw*Y!%LC{qCSV zbfef@x*{IpcaHpVsT(hCz z_$)SWj9Vb(GX)>IhZWr1!RM}7-Jz6%fjgB;7VQoT8e?csaB@v#4r4_M8mMr%-_Ly? zZXU*}!cvw>G7HU~vh(m6UZYYAw=fTMU3lD?V#VY6f{y&1K^AevTGvo$T1|{2wl7(3 zK~viCz4xeWU2?kMw7loJ5D>V*p?h4Xx2Dg z0@*H`2w(Qcc091$@3O9{iLxn#I|6+BW;lv-Esskd znx?N>p?FIC_;AFT798}3&-On*b1uABx$BSZRud@VerPUOkS4CfJd;Y}@${S2W+*H}#7iwg@wT z&%KXk8Hcsw0ABD<6%1PZmk&c0gcX}UcH+TtbEo?<+~$niSHyhU-&n7Lz`W^kj-`!s zvSzEG*>Ok@AMDs7(S?rc`SEOV!-NcRvJPXEA@Lr@ntvFV#0PY0f%WCOwOo}z+!J-= z8^F=;N527SaWO_@Ueqc z08WTOmQYfX)EY{<5=tCHWwBi528J~*1Lfmj0T42kfNu2iRqA=IaMt7~o>$D6OhPlD zu^aPYJxt({71I;c#G(dQ6K>bZCzGVtuF_U}>KUL`T<0Js4FJX%i|3}cqcC~^6V1l! zZ4f_1a>G88Kfs}|JM`O{O3VWcPmuN|edl)>O$C| zwwKJehaib+)+!|ir=WBduTZO{kzKnxjWT@+A3TdB&RGlP0J{N%3XI8^vSone0;d;f z>UHOF#sXd7;PR>Uef;QS2sW;j`CtyqO*G56nDE+KaXqv9zA=6^ZOqPN9XcQ;u9dhY zvPhw^g=`~UkNb3Y3$B;|`bD5SL7OU$14^%0t8AUIv|^%eqdrYNK-{xsKlsJZVBS># z?*@PbFbtix);ZS|Aaxnb?GAyIdn}0YS6@#Xw7EkYdjQZ{XuqnP8}QxEGfzm3w#e=J z{8AdBPcski$AYL~k*#B8SO(mff&1rw_7{{eS%-F57?~VT1=#__ZE$9)b@u8R_rmNf zKa;74??&mbZW_zPY?HvSLE&99lZ*{oh&Tm+KS1ULAan1NPjM|#qmMZ`S);BF^YpDh z{$c96aw9DOhPzn|78{OCNvvdh0F@5o%K(KBCuZP3gdOudh9!9sORvRhb=U2&s8GL@ z{3tW7&r&lM;S*vRb*k688DJ^;idsU*HN*~<25#l2wN%HcC7-fkV%-4#)=Xw$+%wRy z^V&5MF0ysW_`5w45DX=-fOaPW37HW)(poZ+LAW8fEEo)IirC1-1yax{lB^ZoVw0-~ zq9PVL1w(-i)i$6uTx%#+dIxVK0C_2F1u41}UIVRr1g0{A*;x>p}#c7PDab7?VVjUMDA@j$^A>u?)=DA=fbYh;gR@xus zhG0cOA1~hL7?`XmRGKO-6`8ypZXE8ne2w(?etV*eM9|{Bp|Fu}R%v%r6_yv)SRLy= zPLe>Dd3+wt!cSVMQ0B~M6%eQl1X20f$T?^nRs$ZUML?R_cY#aRrqDyh$a^!=v=F!+ zrAY?Qx^|ccZCyI;64f`;xB&wRscF7`QHgC2d%P(7y$8aa^Hu3(wgn+n27@dGPNXx1}ssp=))- z;_O5DPbe{%-NuTu z#?Uuf_@vhkF3iUd9+P2uiE~KD@ZR)@Rk?=^Df+xjVxMwpugu!U@Z`~s3@9we$805E_f zL#jbX4&s&Uqp*I7Jb+fbZI^Do!3KALjW&r>3(M)W*hToy?R1FcZE9{Bsm&VG6*xOJ zM;QvT)EJC0D@9z8TDL^h9^`t8OK_8t2NL`d%6E2ViV3Gy2}-`Zm+}`7ws)W}kh+L{ z2lrzm69el5JdSDqIi|>*CCU7yb-<$MnzpiNOh{D-B2vdyN731p<@InImgsv378adF|1RPZD;3R59Hvyp0I^*s zX#jt+F<|a2bTztYTCqx*CTytQ#p|~qEtSa{lvdx zvCyk<9Wzdu7gh=jjLann@Vyonk8UCqA{}c@gi&{~$|21#yJC?6lyvlu5^res3R%3X zE3^u8mZ2*am_uUUS`p-ZqhnmlS*YAs1O#zX7cMvc$08K8l*6=r$Qf{05>c`_Xc)oH zE>w#JTuz_e%^?&>u&|yO=188~mKIjm0{mF_tpO(0V$z1S$+)N$T#~wQb+7Ha7_e0J zf9ch926*1)vn>Sk4hv*GI0V8l;^NK zhM8^6^i5OuF>xF_q^p}fbO3PfqyAWl?BjjrD?ljxqM~R5=o{lGNK%j<(ML-`D07Vi z8NYlvz`EH(E7KF{^B@0z>7&2?e^M_Nx|iO3J2hZ&oFwa19&l5#fz@w|wk}-0nTBq@ zM(98rJhsA-Zx>1K=_aWGzUpmH&!0VF9blP8h=58+yr@vC=RrDHTa4qaEVkTEtfo9# zPe0sM>K-_R7gPj$#AzOFEMd9Z3{a}lY9jgN2?-6OoVnV1^*V_*C@WmU9n81&7EO#; zp|E2YZtfzW@r>i;YRM)&gP-cd&00w-Panb)6b!~XUWY)egZ@e_%QHxm6NM-e+)xCq zgYs`Ea|D@!FTetQ^mkZv=~M7;tZix$s<`1XL6=33!=k_jO@$Qq&}Ux)83mp~+U!oP z-&(PcS?k8tR0Q2@{>)MqKiUU@g?&(2*ghTSN6qE;p_A#a)-z&qnR{Pa5$umw2^zF= zDsbI(>(WroC=;@K%6|Ge@A=-gR8oF#mTi2;n(&zd5$DOLD&TV)(#U?&FPB(KQG_A@w4A8gqWBGD!*urQs76-50`qmv1o#e02t8N z8d+wT3i6aa^1ky_LQVz6_&nF6LMG9Fm1ls&_${!?N;dzd|H=*5R)7t`EUqxF)lz1F zx%}yTI5sCk3xp5Sav^=HXoJk2DZq3oiX(-nb?JbPN_S18)lk8#yVZ z$mZ1m{^I~RJTpQCbQ#+pWkOyf8a3ivWFBE+oMBO})1W;AvADg@7vwsx{y%ooPjg4R{tQq{k3-q+hM0%L1FD-g|UQ(2aR96MxY&?*4+Yyu|5&_-JLt)8KXrA zy^e*>g0B|J^{lccim+O9OeS4gn1g-_YGc-RIC|*?jEG{#ilF(^pP5-)TEwFr@I9g~ zE#QJOK#vx1rIJlYJ3D-DjWa9G-?~npU@~Kw&@(&-<#S9yQ0_49!81&)y3d37&s___ zyddQYR6E(17*C#FyxeM2&pfIuqnSxvMs=E|plp1-Cv3F*oHWH7wM{s_} z*jQ>DzM6V(yplSHucTFWm^;w-7{)4MJ!)b?7XZWtd$m|mdg@uK1(P~H9&WKqGQ-#17rQieU+!!|d9vU=I%FRjO8Z!t3KSN?;UQ*WKwODR|EpwV zc97v>`IQdcs4P4S3Yr?-3JncbU0zs-c0Jo|1)k}+K}_P?fFXd`+6MKibPJ)dYY{+4 zw1;h(;>*tpRk;5Ex)no{a!H#N5x%VTGD}yzgeg`;yj8WF#R)STosrw#~w4{~1MgwZxX8 zao^xYvQPmN)OKqcyOQerM~D^dj3euG%X|M8ec8v#+H?fJQ(XHLH@m_1XISHE;Oi}7 zC|SK>jnmp?_M7}5=rz05Gi%RKh6eM;xtAd-Rp@{lguK?D2CS&2#kaC1jd6|52y38% zl{EtpYa|Jv;8l5CWlBOz~eRCjR!N_1IWEU-nt$>!z_<>O>`KvXyORs}C)}8BkGa0kbvJWbB z0lSLS_F^6YM)o^sn|s826=Vj3eq@$s6#!L%gWqRG3n6%Uk8Q-5d1GH3m(TQ7&2*Iy z9kH~JxM9|vuc)Vo+_o=ZhX57^s5>`)Cg2Op5j-e;gGV5U-)VuY@|UhbHx0o*^eOm~ zK4|&X!szSwKtcWHd-`PCj)xnAe+?DTwHOH$wBtD3m`g{xAN`S6bhl~&in)cR)!#uc z-rFDN)c*K#jYuPbmDl>b%-ej)`}51<+A2t?-Wy+Xjr|w!__^RZ=7MKc0i*NaDzVSM zZ=1j5CBrD}iQ{vH*^k&gA&27pRKThiXtg^d*+HFGVKX5erG@&$w&Nl$bbhusGIq#3 z#r)AP*P|EOzTjW^XIyi>FS~u^OASGSL)Vx;jzxXtXPMrawq`+J92<`(`@gtWI&zOZ z?Xv~DTJ3#C{A0W(9XW3K(Go^}A4g2`44>uZ&135{CNzU{PB5fcEdq%^Gc}ay!bmmH zPjZ6Q4Em6Z)F?%-b@dIAnGLpJlWgeL*wriP%FP?d*LrZy8lU0C0()3ytc)x6@x4ds zvkyN>_ddIup4_{erk_5DqP&9xBXQbOJ)89H&Al`>c9XS0c{Y~hAyz%hhxH73?mS2O zvC;4CX-?nx)?4YdH*TaGFI`E6h1uCS;%XQ7Qxjlq7t}Ymw1LTPiSiM+J3yMHwn1Wc z22u%xSl84YXI@IM^~Krr^ovh$VGe^fNrz>yW=xm`N8MlQO;KEydM*dVlZn*bk=E++ zf>6_Qi?Q(5<`+2__AspzTeLwUMKczP7P1?c7iK8aF~`IcJdYK~goaZDn^`p?IHs9J zUP$kL<6E4eI)tkgAh9r&IvA)-3p4|)aW-UUe+aRCCl~9|29}{S1k%hr0E7j3f@`(K z>`oUFj5{=XISmX9p&S4#ES4Rla|ND03G?YD2%O`fP>AhVB}DtWIm`+?W06Lw6~L&q zlL`R4BS6>ji7`RAc1amN2Dr)oaaTL-VC+JS4STFlA&2!f1-o z>zA(32WU^ih&%Sj5AM@H+6O$VBn6a3c43}lTFXKajC2ymrmO8(APjAykTBohdWpma z`d$OjECa|}*%35zBEuO)o426Vh08ZcD%b-!t%n=Oc&rwH)XDsGs@c>1O1TvH%J`mD z;tP9+`qGtKx7jEZaeJ=fy6qPekoAk3HIJ8jhYv~9{zu{ySP1lgAHW~k z)%2yEoz)f#&mxkooKvS;+xUpP4gMguPp0d*^6LSiyYRsopj|=bq3vsISY{r5 zn%V(WZgSi~mJwuSc)7w3Lctgt92Vr_+-j=r>`MpC*}gy(0da^BEifOAL+rTV#RK?! z1$T85S+r*W+SW#rfk>VxZ!D1%f+dFQizsl!W}H}2tk2f7B7E51Gm4c|;DR7Cvlo64 zc6m^9nYD1jXNbs%%fLa!H^#8yE`*<>JPmHdI3Sw7`(A6CmJQ=&1k31LzBh)?Ldv>X z2s5BRmLjn&t=oc$1AwLLMa3YUFwG-W=@&fZhRvE}Cjc_U3JL%v&6;vvwfZA}Ib{Nx zz;_CFvlCCrgk?_%Z9O?cH$D8#^{^`J!{jn5by39BVI4IC^NeHo1P3Ne_|n3!us0@6 zWha=_m8W}&rB6eQnQO!9&U58UWtbK}fm00|uow#0x?wvzW0L}P520x@alv#2dTR~N z8(S4}w&>|L%~p$0&haoUIx4TMJfq+#nRjK@i!LXI52(X6NTXFiD}d$}6r&bt#t2y> z8=Jo+f#6(vndPZlOWA#>}2^kK!MJ(Y3|r|`tw z(q}hJPL7r$$0In_?REmc3d-xqb~dinOx36>M+^|#Ig}F<2fqq(8sJOpjnE1?uly3k z(T1CXIy^rxSt>J z?-&gOME96m;>zH|{W6w2bjca>2LRYnH2t>{p&8f6;yJ!)?x*G{G>| zsvkPH^j!`)WDOhR=LsDiXQ%}+fG9NIBRr@`mb~M7f?iywzUbW4bE+4xF=p1wti3G5 zh{47SxP>v|j5?iV#=W|Z(F9N7AO7HnsSSOxMuxuO7%#tZn*^-^#!;IlpU$L@Klm(7 zKAcStKKeXOJ-pADi4z|fg#?0178&zm;+fgOW)B;RVt7jn#t_%e2B$C86!y{|{fqCW z_kZwh4nW|{ZnE}2`}NOEauks}SSASKkB(ht@s?1UmjGRze1q`>zqYXoYwY}6J9^Tq z?|l_34&^hL_%*CQ|M2hs-}IcpUqIpX;XmF*scD|wt8f-6lUd7x)iR}pDyrpUiFo6CgeV?4nj2i@Johtp{MTg{A1xqxZm7%BBhooT7!iJ zFw{-AhD$E0&%=D~&de^Uhq>Ci`qDgcaqfot?%_iBMz_TYz|B(qp1l*&zYWq0KSZ;yLD&g6@c;Dj?WBF+mHY zJ9^`Ptj;ai-GsKw%=Z>^U!qX!7S@41%*yk`u^}4MZ$9{0Is&Ym@*F{mi>ko5b-i~m zZ|ls#Ay&~EtV4&K+ja){*_vHJ!;qQ*{Tu1y5bBSS`9Uo(XYNHfLB`vS663<0UBjXtv9S+y}Ht}wjb@52BLo95o^K;POOy}V= zGK3|OYgdU8y!*RP*eEPUoZQ&RNa`n+Z1%xJ#<3Y$(ffN8%_S?;8iaFm^YoR)0AoO$ zzgwZM8_H}S{%K_c@a*YC1plkNcjlg^)#=Bq6Y3vA6BAeptfK-#{m>XFXl;SVCisl? z$aL2Nc-a5|KK$p28zYx4r$ZKUTYoPgQ1^Mls4jTf1#WVCiY%;@d^lc_5L$i?pk+J($Oi?#))E1i zagvOVK65>^aqT-+=foceW*@^U!QYmCaHANTMTEFxjXG^ynqzMSMgTv?>ALpiv%+y0$kExfu_L5m+n&miW}*$ zw9rSb90rreydX!N3oYTguN{M9R)&QI75X}Me~jammP}#HZ$)dllkIzc2k7H_nL-d@ zyS{?{Jk!Y+7)V1cC%$rApW{HHGub6mCjbCI07*naRHk_T`B?os014v_&<&5|@l zJ-LP(e9TyP*<|F{Jjdx&Nq061@H#Mr@t|gz&wMU7iWt!39BDCf4M+*c1YE{gyC%3Kc)%D|Gc7}(pU<7NavaXz ze;r$wL0d+R{_UF65*0u2OVAW~Zb*qp_FFHn_d=yPWmE=8`7@1R>)+tll{C&<>D;$~608sBJf-MN`bYLQL@T=gbo*RN|w z+DdC*Zt`P1@B@T{Mux9sH?6!c&se3x=Vtj}HH5B^FV=o^jp}u3#2VI!mqM%}KbfG= z0hUnA)f2kXh(2t?K=9_*zmrC;-bxQ1Jcv`KhDR@8Od26Wwi!&akv{t6$LYn>7g*dL zr+a`m!_=S+YS0Kl0IX<$EjizI;9`3^2G~_(w9Pjb;$A9vDDm`2X zde2U;q@VxYuhJ*Kz9Vn43zkR-uj(v8K2Xv09jE-5;NCkr!!M!;WnHD@Hz4h+b zu|~`Q1{RUdBsT#1?%uzPTZz&MfQqiJUNQ;i;-ma5Mex=RZy#|Kg{BK7;^C z0^OHxqzXI#U4Vu{q6G!qja$(qk2tvjCXqK~V3n@KS5Y6Kq>oVgE)GGRZc772t+N#C zmcG>@?CYVuZVuiMoQNgx!lg8P`yE`7y%8hiX%-z904mUEANOld|2R8CfFJYLi83sz z^oWZqvAf-X$?LaQ0Zr@afB^9}7FUV3&k)FFuA*9^Zx&vKGea>8HeCauh+ukmoW`6r zwqb!Y9*%M`1uUioeiQW?u~Qy%^y({bq{#<&DYrrpC_u+RQ{$FQNYPy^56sL?rYE2O z4tFWpwq)`e@1S`1GzH6^9F=!CK5ZdA`t-BdZCjRR?BeApozZ|xaQeaLsfn?z%}%lF z!>x&B&A6$>so6MHph)rNt`>Hl6k0ASOrYhZ8#g1_;RxX6f%Lt&e2oQjyk2t*aFBil zguF*+6#(Y*Pam@x@E9^H(d+t3sUPL5#cpwOg84lrsC5XzPAwr8;WY7<775;7nn{b# zK4)V=v1sV!CZU!EujQ~o3lno@0*YDDZjNZ4Skf)}$8MF+OVDc*pxZq-93@ru6}T)o z*NQa_O^`5Nu88^m%kO_Ll_{}uitBxz`7J86Fx4kcXK00}^}P3BaxJD>Z%+Y*tL~iH zU0UdPcNOpq57r|b4V4H$qP%tP;UyKGF@#nj7X-9o?Ni8>&NK3~`gq#N^_RcI{3>L! z(DZZZ;yZ<8c4e?Bq~AHaExS+L#JCCcWj3+(nykU4l2*`WP^iKI1ke{i7t%h@j1Rt34rR)rqpQs~hL zVk+2Tm)}^519FVoEP*Au*v5rT-&iv zA>*06h%2K%TI6CbZ7bU38awHH^I!Cnv8zOV1>NYIAl!M94tZ=?P@zv%N&DMZ0HSlc zYw5~h9|G}m9+jhD0Iq}Ad7#(Zrj|ut@~7)j5F}lyAtH`LS*uIYSAbZ)|9!AMZOS9w z6A2TH$!YezbjxvyT12$;W*47H7WP{@#j^xV{FCNYcWmyzGr&BGD$8Q|D zeRFf2V?W&h7%F5TYuA<5T`c?bQF<^l$RD$#vka1dlsi5{z~)9`dxPTh08~$0q9b9* z@&Eu2qBpx=58es#b^N8*(&t|hw5K=U+0cRZql@|eb7W}Ja;-?;Q?8D3p;(y3$Tw9KbYNtxPabUV@4Bm9-qMNph z6xl&yeA@bk31sh2{TFXDNfcif&MW!U+55-=lz-x#t z7IF(5X^7)05*P*|ie z#V#3&&#)#QQKPNYae>nTFqxukDwGm9pg%H?^@`jpxoC8?s^k<_ku3_dC@5vzC=!Up zY>Ny^OYA+GBo+t<3f;m3^*^a3s~V<1;E@MOlSeb zHu`(IzK~8(;^ysk9GBDD7r#xdn0otU5E!{axsN@7ZV$|=%hfby{ql4#VjJ9HSiII9 zu<;z8%UX}JbuU_{l@U;S%6ojrxcLOTxTSGlZUA_O05OLs;{$e~b;Nslew|?9kZk1! z7Uj^*8|mJ?F92-R=Ifx`37NJ<9$c%?QV3%$E6~R>KU7MRs zqnw4?gZo)4t~H0`ArF<0Vv3BpzptILB=i=VdN}X^K&zjV7dGbRh=rR;>zo_c)Y6!~ z{Oxb1E_O%l7cZtW5?z)7bt*+O7A-Y$ieP9*xU|AU2Rx_3PR%3E7N?%2GM26=&gxph zBDD%+P6jBE-v`Ba8 zFNoI~?W8obnOK{fA@f=*Dlw#3gHE`vxo0pnF}_ANOBL?l)Vj%9?WjlK!Uuw`ea6#( z3%Wva>UHY#9URxBZGvD6Y_z)J*@M*u`Xi`Os2Z!~@mkEC;3JgSc_|O(B!d}T83<}$ zW_u~P73%VdAS!nC+?&gbC_fBF2LLlB6^E=|fG@%-09^zuDhd^d7brAUcs|RvJY!I+ zmex0>lbiXQ>s>3{#Qg|ou2s?dSy*>?rhrDJpuo0&3gyrS^g(6o*i>R#0Ifb>bwSz> zl}msH!5Kf976McsNk4K|=^u0M?$~b8C*$pk%p0AF@#alqrfY1-Xh3 z@odH}EqsR72o1d*_}TqYu*8 zAHjhA3;@G;f`#ZyYzBCKH4e|Ym0rH&1uv8)Ii^zZMi>fwEN^7GX8HuU<$Bwiq>B*z1 z^yJxOy8roS;hA~+-FMQp>(}slwPKuFz@>UWyhfg(U4t$-JH}&&SlC=$#C5t7uG?;O z+O?Y()4N}J2VK$%CR$I=?mtUE|J%Py&p*G1OB7=pAymf|Wc%FYWVpbJDE3x>kOy5` z0-=OLYR0Og(yzytyhmqMY#(u`>iqO1G=M-5m>`xkIC?paV-~cHGyLS1p3szdh`@E= z{X=89oCXuA0M(-Rm<+yd3dgo}UVu=YxX0KnVm9pJvlbLY3&3SqOP&+ME_jZ^T`m8i z>vmn`%fx6Uyh!H9rHjM376pjJWo=NmWfRNXDQ+IB&8tXvaXAWxc5w9#4-Ew%?V*sY zF04L$*IH(*Z)Upj`1loI0m6^xH(+_1m#~7iXSx?kM1iROH%~*;LwDPFv3(r_0n=YlU&|{rVSV zECQ-f%(V!`BN#x}>o&k?8*nKD8{})OlWwdcacwfK^{pxh9}_r^>?e!&2*I^LJXYT* zb!fV~=_f;=kAjmj`~lm*pa3(JX(Qmc1#9Id*2=x97isC<=eR@fb7UKCVVJhFaGcs| z_tPtvM^YDl@jZ?dYc{S3rPPmGczk#$bzw2RNSz@;NF8(8#h9*N9*6l+LTsV;0Ea`w zqk;aa4HqP@5OVcUVMR(Et{UcbV_?v$P=Eu`F%ycc2Fyu~g z4Xg5#&mX0&iN&9b7fv~`~YRS`5 zdCd90?dE{M>L?6tn^ zw~E*2O9z#+eHV1Oq4P)Z8&)0K^BO_4B}yD$SUtJkvFPfyed!^s1pxxRpexVw8P(@X z1N-T-9GN8yTivYUy(ADWjaDs)f|{eD5n-<$v}uzz4tWe;vA6H={r>=*>JT z58*_tH|Pql^Tlrguk?->LOu@K(1zDUOdWKwD#S9!1loeMaZ}*at8Pd?3;H6z1YI6! zW#3{`0Z-W|qxOF>jsV}$MxGmGg?*^byEPOI<&kH7hHi7m<&U#qZCPm?RzKFC8;0o@ z6VW$`sT>fjI!yZ@?Np&2jdp+)%rCr0nd7s~67_sx`>2f2^C&%~feCLOjc6jm#cS6i ze%+G~%xWz_=XR`S>g|y$m!l|e7a69OZ~DqNzLsv^zQH~Kk3xHNtd%M1VkK-NxQ zw4f;*EUB)B6;39TL0?S1h~qaDHtw3-k-Okam~D>$E!L2+5Ut?pfN_=& zu#TACjnF)U;oGrRGX3e8=j#UACfLz_g#{GaTBocTz}YHi@KRLq@tu$3uy@BVpKKyn<^eP%gyHPdCt>O90nlw=S&K3- z%!l)a_`~wcLU%#iZ`=+CC_I0AG~1y(gN$3>2H3Nk+h;x$&~+$=TE?#RrmdTrgN#FW z`!O`xWL_KF`$+!iNTWAjO(iVEBR5F6;NFMNXVP)caO!{Ks}w&b!l@D{Tl}4ou!fCzSzB; z0$3~%S0qA~$^pppo?4h07%ylIH(uo+4ZTpo#*Z(uR^Er?PT9q=e3X2t& zSP8#bODF9J+;kavjlj+O?YqwtY*&F2AJIo?mX*0O#=3D~fjqw+k}xJ+gKWy6l`nyy zT{2!vIV{KsfWn84L;i?bfAFeGQJQqJc@Jv~iqn5S!)vp(Nzj)qKz{BzaNOZ~hKAD4 z>%zUs-}03J!5WOd1R7QZkQc2a;99ah8VL|2y_8SV*7jAdf;Qg9d=Lu{MirxNwZvE?!UMa-X}Z z{gEb-9;*1Ys(jY*%pxnVuAF*g~=g9PkDh3{R*T1&N{$EhyTmUWn2tej7M zVaV<pkpQOM98lgbvS z2ot#-MhuhTOwO>(O7BoF;Y(f2qqI6p-}vTN!A+fE0QuX0|F`MEooBILUwQdPdi%|n z;HT|$@6KoGS3mz*dhs0nh0f7sU7~+}x4t%mM+y%SxMc$)L=7&}*WPGG{t zaPjP=$rl_x&(TLmJlI`^rqWu_r)Lz7UA=)YZ%EH6WV$y0j4~!z8!CH%02edD9YNkg zTv{vl7qBiB05e?>vxD7Q8^q8&s$kFrddRM8ehMHcGo#4bIPX$xKoh@Tf9pNjIFP@k zM=aeSXtDnt1jZxnSx6ZKd5-BCAn)SsR{<)xd+A3#0`LIf+C;xCpJEvdm2oUU7LL0e znQjehh(&eH?vq)}zEL?FZzoe*$4twK3j%~G? zV38kFEV+V9wL-j~gz;=TW8&b{Lu$#fWo({Pd=sC z@?sh%^?Hkhgk4;+5Ryag;UQeW`vAhFnTfChKL?0z@g7DDpsCRET+~H2B7_p8NfKVV zu-?^Sm#zbp{rE>eBDQiCS1MejHPEYi|P5?avFla=Qx$XOrxdcC0xplX@)Sfwu|Fwm$}~1bg$c)7_=kk zrPa!0mDueuhgw72u@^bjqPL6SSrmjWMl-Z?K~oA31CHfW%bgtK>MsMJy9Y0({#S0M z6AS(E-66ZUMs|Q_0Kd8E8FsMJya-^OJNxTz`;h^TXJW!{$ajIYbPTH= zmk3&8P$NW|CR(66dq>eA^XZm>A>O z1cuT=&>*dRNn@{buA^aEbgc5Dv0%2RE47Vv81fMM_&bk{Ic7aY(g(Cr2l%)1>{{>? z51aLK`z&AjJm*=grT()-j5HEZ1z2MoRoW>Vyw>6QKmYb8-UfeV3$8rrX4_;O+me>H z;g0|%Vl%lZ#{=fRI0Q!ln{W@;Luj86ME*GEh%H(49aSa0=j z9Q+=0&Sr>)V$))STrWKZRlK7)06ue|<=t_+{wyx8^{szsC;+%gJFaJG?eo;>{%!2L zWsNL(R7NlCvu2g2TG8B~1R&!n&a-@afX);!hT9c>jke%f=hX@s&aGophBbjB+_Y%5 z(@WQgrS{MM^?#4&ix3rzHqxv_jP->fa7de=kPRSuWpId^ZGgI&$rtIvU;Ty>Yb0+D z45xu%!gsLFZ7j^BJ0E_8maj{T05Oju+{Y`of#pnIpP!qG%}d38G47!&*V147PyciJ z+CTee;jx;Tna83xk?!E_oO=8PvV#di_P_JZucvSS(fd&tm`uhrdG|vqRN#_YrD5xX zxq~gjpnX`*%#QTP67&n*VA0{Kvv_Me%0?w&HpmVZCxfRgvMT^s!Wy@TTMnt2x+r$n zLM+aUXU|bYOh)V=Av7*REjwekUyD;06c#%u3o9jCbo9mJdkCc!%B8W8SQG{VpJ>g2 zA+#LyXr*IeHZe&|BkndB48|oqB|U%sGyv4D#b2>f73zqe>bJxRL5e6#0Z9q>U<-A% zw3KvoVwuss13;@MLBLWA3XF~Q?bM02>x66SIqpb-QUeHWz+$pBI}Ok}KSBjYYwzlz z;2s${00M*96-};`Q>;tYYCENhvY@~e@|LDHlq8vnSb=0_%PzS1JRE$NSi2epsjjtv zoG?G*^)TC5m|SQ47ppH>_+gFG5Ys5DQ(gGHa1?23i1_6nye?`0yLjYk%-A zbBUF)ypcAaka)1M6gBfK6>xy!urvUk%V6q#+_&~s7ov5*OhJ!=yNsb00PW%Ny577F z0;fH|%B5Scr5984IbHz?VWC~h%B2**M-Fu~>W+&oDH zEdaxs$W-paMO$H)qw8gRbtA14SE(X0pnI9WpG?lAc8)07VY5K~reR53fwb5d=X)2+InSe zD>iIA4kaewgjb=x+As2w-Qz~KWE{MT9 zRuq^D)R)Ivh5HR0y2r*#L8?O35^AW0ico8YZc7zRxKJ2>Rx~I<#-er5`e8?`EnVQ! zip9hwFdY3lhIc)_FAl~=E(mNwfkG2qNuHjdvIxbBY*W#w@J^5;4$}ABh2J?_-dA)tA!vf&i3rher%q~-vsdQy*^vUkgL9U@s3@)zsGXObF@Sj#J z+OrwP1leOg9JT9FMQv?M`6p*davyD3mr@H^wS0(Y0G}XZg*P7fo=-4<&U7YQkW{X? z!342fQ-HTE-{2gCJB_)46GF-BlhQ4${E1Q|_M84yU15S<*Of^eA^GTAEwqp(0ZM7# z>pe$QfTKGrfG}4`%VXk4Y{Vhw&2fm^t2+wz} z{V3)U`BI`JlwjBOXdkt>3C1I%Rh>v1wjT?Lew>{5#aLdyIYCZ%eUvHTGWlC8q+uxX zvGZq)XAOVHW{UgeS>-{KAP>Fai6!;u8xsp8uB73jm~R^QUG$A!#~O3vOY7X@c%TjZ z6>EZfO@wf+Vnf5n#@k2v6V~QIYl-r4xS!;PW{J+UfLsZl=h!TE?i=gzmh3`wD%ruO_1 zPd){}&4-bqm3TRir*xCEFgu@q`IEoLm~fCLA3p+s5&E)9&D@PSyqt`S&68)Sk6yWw zZoctGdhegVpT74O|2n<=-XDTBdei*s7JB9hC-mGQ_L_q5Bs$!F{Y&WwfBGlsov*wd z;cd@;`-}8<|JVPXe)?DcHWk|0Ra=Y48ZS=9DHF3v3|qB&`*t8+6bqe}8f7NFqx}K| zFhm=n05ROPZ&T3J^!o>Q?x3tM1QAto9;;(;q6<;v>8FWtQTCQTbJ zgz)i10_)_E^=SEk9zd3sK;!ewEYt{Lt>vf;i3evQ4umHk>n*2^a!Gv;xYWpnn3pJvX!fs&P>^0pgkvhGziF zrH5bO>iYt)fPy^WEJxf~F1lu{eKL+QB3d9+7P52!P*DRIwS-9=*37eI_#Oi7&Gs#^ zNKEyYDWYODObn?I8phc1Oa!X{`~>E3;R~!=1}~=O@f&HgM0pR~ef#wB3=75q-)oIK zWITOjMb@#@Fu{gnNLD4QR#c3@e0yU!mdr8Xvd7H7Z7rU~6KYO_0#CV+|y% z&}knlULzK{OP$5kc}mTzg%^Me%8T?5rX`92ZzGfkuioU`x4zWJNg6j^z8S%#!Z|FPnuA4L!#({M==Tl_jl_l7Ilu}u1i+mTSJedgbd0DR+?9E*6ZP26IFP zbT{Vx7It)o4eTPUk9e;2^+NeEC;Y*W8Pf<9K*KUtQ0x8Zh7iD=!Se+yw5I|`z^MFS zu&?EHDp<`rbWKweY;k!mT_v7xoCFfiF~KEHhypy)#3S0VR5t_Mi_~c0G?;oVFCz&C{aWJBEXn}TT7XnI4#7kZFwTB4XinL*}A?Qm-DPU zO<=4r6^s~f)Wx~6#?)xBbz@frKMH8{P*_P+bXrA#Ewp_eS^@6}3RP-aj9eE!$5&Ww zSQjc0m6k5hM!W<90xjIe&glvh{Mn$^HLW{lif7A)7BT*&{~?!XL#vqm@TIbK?Di#; zK0Omm8UrZE6F7zN;sX0*Kf>xjKLc>F_H*o(u1S?TU@H_?2Crd7gzp`jN=UF!1(vqs zJo!a_j^CRsf-9OnIQHy%Ql8>2b_2~bjJwrxqH9Rt?O3FnfHdm6(PQ0h(nrOpOmcvN z0F`xQAMzZ*tS|2uP&=1Jl{0;fP!HzY-x;8gnTYHtk#M3lEvzPtG~ci6KzT9;jxjbb zT$8P-f>&tATJZVa;f5ewg2=;cA>&i%AN{mHT<50Bj6}hn8K{B|OV9|0lr^FBS#S1T zi<>X|CyxYxf#J+{^hMKo%+5^614-_NBWEvb5z;~cR<&O*wp@oh6 z!^?H;n5(Yk{{Q%V|5;-gGLme?U>~kY2b-P0$uy> zOa9dI={!dtoMULL#XN&m)-ca-ow!B>OxBTc(J~Kz&bj za)cKM^?3Y{Zj5fSJ zr!|D90M3R8U0Ej9dFI6=ojAs2i@xKyLqlMWS#NDv;y?T7&UFh#vu zybfA@XC@}n$G`jl@O2-)bu$BfkgrF0C_UlAjaV3xNX&a3@7UMA{oQm6Pfs%&Z0mfk zZ|$c0ckcoWC!%sfKjCEm0$1txzxRVwgOTXRfBAo>|Nr0pAL)~y{wQtDPen3Ap@B)m z0i6yAX5YaTH9T+ug#d`dMdTt@AP*1>-a&noT@>+^8@Et6R+LB4Pyp8;{UW0`J2k5B zLzn}ISms&c)gpB3RmiiG>4Z49 zB8#yVCFtVt;Ba@ly0A9aSLi5cNO)l3HDK{*+~4Nd zDr(lvaNGv&$ZaOC1tqTw+IFz)5);jUx&jL25c94(kwX|!Wn{{*<90Z6*SA;A&)AU@}1_J2?fU_Er|4CTl@y z8+0wABs_Hd1woW$7`bn#KVtL{o7B+b$TD1c8ZaI|qNW;*T!Rv6V(z=RXJeTR*^b^+ z*VUKy0EjMrm0}IgvK-9fi)ZO%mGgJccGFP9X?pM0we$gt02!T{xTl7l=03uqO#h~F zU$z6ZHZhY|=<}za-id`?L&3=k;O}I8D)rPKq-%X`{60&!aEV%C<{1m>m^o`h=^Vln zTH$Kp;hOf|w1<@zc!}jrQ#~MXgqTYfYFB4(xb>Pz#3*!86u6E`*eJbKKJz7K;zUd3RcH@5 zW*xSW6jaMb!0i~C(kRuBLE|kMcklu%pDmz>jSTD1N2}<~CKM3qAPWHLoWjE@s{o<&D>Mbaf;PxT$0ZOF zP-cY-=-_&f_Hi7#s$6;*EJ|K@h0g;V^4lieM7lr)(ONBxgYz1}wwu2g5981sq(YM> z(n;5SRT2A{n@Y3pjsLP?6}8ITc2suC05@&6El_I)7>2u*HXWaRic=q;wSY~TB>*&j z$a?|+$`gDi5Re8ce&^cneV(tp5#U)co3!;ilXQ%El1AQd+dj{(0&8Qgjj=MRK?`J* zpMiz~l;8~7IFqKd?_3Ea1TD7h*yu3#0)U-w?g@HBTfgTH=x$q7NkE2mg-jN}u&kj& zfI#{ZeM5%(@;qHWh`Cj+I}Y8b-VEu;`106*GfxY@|UoR%b>AV{;IoIUdupK zIyWMhe7RNyY4*n-!J5~(-hA)N{#8MrG>Sf*U+?U>2CDq++&Xq+Ya?Efwsgt6@vvR{ zBwd+0)@`;TpAtWBq2!wLb=rkKv!s~GO?-6z$~1FhWHI+vzL$rTd(p0dlq(JVcY`T3 zb{&+sPVbpXdn025gj#)xFrdZsiy!|Z#vZe6p-F=>i29`%6AZt3eE$Iytfgk?IC!Uv z8msGR1@Dhupy!VskS)4QNYoa_$;0%N5?gu<>%c08#2m4X;0BJv>Pz4F-uKf>ufIkX zE~UCCxwXeJUQ;iovDiJqNM&5~Aqf)ir2qAQ`(HRcXgvMu?|+%Ivyak2~gHgW=V~1B9ySg|Ad$#6|DQVM$fM#yHCeJ5oH{y{7gmx*F-I@ z)FuGKU^7Q%Og>EI*@-lH=fm`Z;QKz9!nR{~i*=^Ft3oE!y8Y?1jT%7Nz%1p zi5Ta##Pj8ezG0FUn8ymr%ebCyvpfN-f-Qs9hlYkzH!ju!0{RG0ZA?~yFE?;w4#4#F zBm;B+SSkp#3VdMDe-pbP3;vc#FE1T)540`9z}iU*^v5I4ihzxJOw>iJv6eA;`ur(^ zzsxyv0|BBQvRDpDFnRpx2PEK3r&F>wb$5qr9J+dLtToQGVpw&(1xwx1!bF+@kXi%i z0Jnj~8h2Sp^0!sNjSKBWVC>=UYyb=&5O-Jr(zHX8n2!JMLPs9^tgP zzDfcD)?{{Fn^+iHiQ_y*;2=`MvZ|m`iR)IgHlT`A7yL28w=9sxGImIt3o9HgHo;=$ z8J1`{1BhZPOJ{(wp-VTJZxkeLnw3ox0d~wpZrE91gFzSrG~`bD$;Jd%?;aa65Zo^#;il#&)?w_ND8xDzJ_7vP6c@RhVn3FzDYM zrtcNP5jRCUX1sd#sXWK%;YBT_;Ko zKdT1_`Av33$--F?gnNg61^ASg=#=v>D0KHPAoer=cIO^e55~CfX4Xvz^x&@et@2Vf zWB>{{0-Ok_bPcNfjSsZ%(GmKh%iJ*vkaEnJd;UzT=sR<&H9(~r7CHK%bzVp!Exp%g z2}tCzc&>a&8$Ki~ee^#U9L6JoTnMlNJpx3zmZdiB>N;yNk%aFAa#~VK#4v@m3Of37 zJeoDM7=@>VHJPn-0$r_?%m=_oS8V`T=3mejpqlpVj}{yM39#jH^@i65c;gzCX(WWe zvtkNmPUamHEudL?`{Gf&*rcONY{P8*MhawmzaEMVL7Bw6sc)*CN$E*+B_g5V4K<2-5UlxHG2#ozoZ5W8diqKut+ z)PBfsf=Tq1&*m8dUv;9o0dA%LxlQSAKX{NsiTM$H`B4V*(nX-@7~DV*)9QwYemYhy zhW-{5`HIZ~-vtmpr+bz;f=%yd!@_kDzlj{ytti;G@V*({_S>=88~c@c1|H{gbyO76 zr5&x60^Dq=W4!$B`=BY$m)5?QkAfeNjd?S4ZY^fM^{^R-wlF(`tjP&XZn_T9>3et^ zn(;u%3swP;UV=_Rn$OPHYmxRz5R3oZfICJ{(U8|H>a9^iw5L4TCeBSOpY=$YkqCS0 z85l^{Ub>b(|MZje@b1GjFfw>jOiWOZbUjXV zY9yJnh`a5-{a628`s&xe0+{?y=|BFT|2BR4Pd`f=99FPH!b47`g0IjK2k47Jfw+|- z0PHFSI!_tc3LR+2z1K+*NnLntxa>N5ds010ynlEkUB7lK%}!24+{->n(gmfkG=s$8 zM6GuEwp*}v#H6yI+i1TLCF0K3_=T`=n3@a0Mc8Ku7VEIJU?KC^rWQ(x^ieLOrL&(y z)mK9(FDxvC+0y|4c|qbTT7NTd{-M5h|2mjM{+$+B@|c-fU7b)i86=41SOs_ z`MW6C?WHA-QX-ZFRDKB9>qE)9Xj>6%T`2oQ+O-6QNBeBAET%rJDeOgXnc@fM_X*&! z<(QHgTZ^ewq0&AUmCC{_$0ywd6|;B%1Y29|Tw#(P%ajm1CeyVNo2B4W2pBwB!n&!O zQUEJR65wj0SBSFe36yLh5#%9Rog{$(m>aP)l=1Pe0;t#5w>a?rV(PhgD_yws3N_Ad zqzxu?k8!o(^4rDou};~GB7*3Q__q=aJ%r_^7QxsL(Aowd6k%?ytpbYuGC}Jucz*Q# z%Xpf7%ebvE(~>x$C5MCw7W*bUlD@$afB{JpxWUURJQk~G+3FHDkzs9R4y&Z@qZ2G% zOX=R-`<&2moSw`sAuLdw2pknl#1gT_TA9}k+@Y(`u?;ZaOVQe1fb`?LcY>!B@Fm>U zgM)*_d*PB^U*Je9tY%nZ$8X%C4i7Fu1WtvFo)hX2*08hEDh$)H)4^(~cY{UQ0{Hc_ zeSk{`=CGdjW-@|S1GqeT_<{gdfH!;+)!Y%7jqqrh_8f0V|8Od!YzhcHCMs=X8C6hh z0{ohXFHs*3`e8Zt0;(nIheTi8sY5K$1WgNj1$fa6{ES8*AswG zJRCc%REN9R#G~C857HSYNL(Zy;fVN*y|rb8l-Y*3JW*7KfWpXf<8fNiboUs)0^^3; zR1jdnN|lKM^8^=iD0Z%ic(za=2weLWR-?Ez)|f(Ap==>iX&e?X1wM39L1o3JOABzr zJo&8rT?Sm-%u9u#0_CORbWI}o4aDaLKaIatKDqXqU`QcOo6u8GV?R_@_Jfrj-BX6JuQ&TrxgPFp`svF!NCB|p@<+uT0MY(%iS0Oy0Cq9fb3g`Nq+b9b z_%-7A1SDMV<~5&Yk+03}-s}dI9%sn2L*kiNC~x6e#1?su*#;ol2YJtj+Al4<^054* z8#%jsc?QqV&*IYazE`ckRjZM_q(Q+Wqqro4TQ0B8spIlFk*q>{(#7k-f(ad5D~``= zq_JOmi*1Hwm3!=y-Syc)F#d0nLAXIBApiMtT=6%*Vtm{aV?F;)UJX@7xX5y)17ykjmI zU(39QTN}`;&Jw%^;DlBJH9@QLSg`FlgXg%%^P2_Ab$EqZ2rY7K$^heYOC)?`}GP`@FX6(!N;K}FE4mLxy| z#Be|W<1y2inRe5zy1e&URarh$&vV``Kni6~O=V@i?|rX1_nzPV9dpwG^R~Q!U4e&W zdrq0>d@U1|hD64Bpe`s*3KeuMAZ8g8>j&@r0*CQjx_$G*IHWa)FPQ`DDF`0r(L+uy zcbz)^D~JN4O?-V+itd`+$A{>Ig}Ny1PZ{%*$B6O=U@?Ja=3U#{;@pI!Nyj`KuY(18=txu1>(BfLWg@St*r- z!mm=&8Q=6G<@9frCKbM0nVwE-u$)G=HW3Lr_{{SpmhDM-`ge`~_1LRg%%96VcW_{c zd|?EstSpflW*O697p&G+TA78h#4oP|`_*CF8LS2thAJk0B(04_4VK4eqoDMo$$UcG znYzfq&d{f(JaI?!cziqpE{<4aorQA~k730yGr~f3_4ROi2WH*T(KOG&$MyKq?V@;Q z@tc$}JZ5{c&)I$*0J?o9LN(NR9RR!<0C{d;fRiNfQD8!K0s3>K-^kHVCmRcCl@v66 zgM)DtS`8CircK3v{rXkp29Qi&Y!KzUgUQw7t7?$*#FO(~19Rst5y>kH)A(!> zlmG*3f68?2!4Bo=mpy>IhanrlvpUDwr(>7!9ch9+-G-Iz1gww`iCydB(i)R$fJxPd z{~-$%lU$`O+l7Rs5`9#3K}Gfi9D-nz3x}3Q{yjrbg)Wnc9hPXBWICos|lB8>si$2|9-wv|}{8^q$)GkBH&5;g+s zi>aMliB0SfbBtFleapranafOXrt{GM%0<)#_yATaOrW$yxy%lJ7vRfO@jHMH?R05Y zegwJ5dZyH6JeUJ8&Q&}lz!JNCO`PHL39I7E#XZ9OX`ASS>AKkprNw*-UotP_ zDJokPq}K{gO@eFR>*E*op&wP8!J<(=mcMwZ+x|YZ!I%YDgQ66ov4>AXM`iOt_ zpXnrMD*cJyCyLiT^nJ9$@up1QyG|L9=5u|9{h<=qv}=Ha%wCKQ_eX)z8i6#M6voXp z0A@g$ztO%628j(^M8X2=r&vrbO20@+Aii`Pw_S?Cme0T-Y$#Wbd$L7?3P+2o*CDMg3&#d>U zY=%s8Z;T0T$z;_wb+#_Hk@1(QRGtJ>ZVDZz>=NEl{~1^)bEZ1)HH|yh>_f*{lV`YcAVZQvu0ys- zKzs}y5eS+hOu2NdY?G#JUk18pF17xS+m4$>ED)yPF?qhHD|Get2awz2OxH)BPf{OJ zGWGSRp;7$%%k|BbWi+)3Z2)k3mhb|M*)rtM!^lfDX^fxcYAs-)h$@zpodw|?*U0AJ`bIsnN}Z@=}^H2q|fxwuOn z(4O?%3ooSMi_dWS)cy4Fd%sGLZa>D|1`>>`?FY|R-^7Bq0-)T-hk1jwssAErL?|C$ zuGw_!{htHE8qzm^=liJ^-y?mSwqS|u0J<(tJlNkQig9H!og3~+YYQ`&SzuF8b{}yd z_#$S?4QRzW3Vh||SMhH$x#>bMAyW26w&^Rky;C6Vgr>=QSSkjO$V`2$`fyV8AY+e` zSX-G#>5^Uny0VrWa~|I{Ix;}WEy_bbs zb3j85O0o==>=^oSdwnMjoVmoH*MovJw-ONv@Y~*4q_Y5)l(RBFnT9%X^*>}`f#RqJ zT&#hVN&gZ;R?UQIB@Kj$WgBq7uihoC1P95tfw)1sfFhI!J0hc41inU*7Pg_yIZfiu zohK1AKxa@E%9;tRS)w5ayM^Nt8w4fQ3^oMEV zlTXs4ci&Gr7h}woP7dwz&}4wo4Hz{JcE{{lW<+i|tQr1y2Q0ce*?Y+IF~KZVObC|l zEcq)a#A$?wRT^aXv`2cFJrww^bv&s^qtAUYZIR=zD%VafLD&U^)QNYDnQ#NY!`_zW z)WwctACvgT_m# zlU4xA7oK~GfCXfWiMYP9nLhpSlXMF0KKBAslP+Is@R>6D;pj2v_Mm&SBLAkGkrDij+4TxtcA|L~0fbv^pqc!%@ zSJDFgxkbMot7LeVr(854lf{|4skHJ0@Cv=(*ooVaAdPR3R{y1_+>{*xV6J$dsUB?tmHOfHMzFR{G=C;N(G}MD% zk?tqVD=KKJnCQzGZ@~?+Ngv6UOCl&p5)y(R$pqP|IA4{T+2#PzjEnsvfKi^SQ0@W} zE}-6k4*I2_UcrP*@d363NrEnA&;fD0X<3v<(IJN?FzU|x*r@P)L4>TN;MRi5NnU>9 z4`u3)sU&++k#pa>WF>W1h&==MstX)@pPjj0uwmez8v;Me0Ds0pS|0-k+;r;ZpiEF~ z$e)E!G<`)aXrujYN!}yLB9Jz@Zjrhu1A=2^#Ghah`R#Qk;`enzzi4U880idLAuO={ zZU`tx*@-!6L?;;#n9J6|=NbG|#x*^CgXz^TeJzcQjz#}aJzlbGSi>My+OY&`)vLvV zRi=!)S*353FMW&ogEiO<6nKI)&<%H3{w#x^1J)JiPdOm8oK0z^j1}_VGkPv8GM;i| znjLe|xlyUAG3!G`dglCj$Q<23Vg^tSx*W!Acb!uk0E68`qBb^Ef*(}Kl=>v2V|=r0s1rlW+@!dW7e9 z35Lj{kybZWVCu*zN3?2X>43w_#{mI42*dUi$dQDSsbeBP_|-?LzXjS8L4N;7|2{qW z>~88BxsaZ{a5k+1WCmd!rTF)tL|NT9a%(w3>Jl9Svj^)^iC`NkGJXGUYVPj>HP@$w zH4rw+ybZswnJFSa4`C5mFq)}nBkbW0_H!qNi-KssrL~v5H(l7wR}fZCe1MrNvOpS6 zuOHD7Y~2pOJ*uplvGLfjS{8vx*tq&i45b}5yhkni3OyEU;*&wV}2{5$_ zC97{^H~wAgGjpjP7Ho~sikSCjW|jyM z?*#m3rpwh8m_mF9%SdxlkLmhg9YC93Vj|}_Y`!ZkED%9T9dexmY4Y>C^k-8Vy?8m@ zWNg7(t}XmzOOi^Xz_Z(s(<;mjD$Rn6HeHirvLCSU?7I-L_~w{n9${BiTMi4jMTBmD zM1bqV=`OaKl|Rpdbp(T0Gkhj3J-C}%?K9-fXC4Dw>IvdGS)O1~#IKHptC@tuM}VTn z7V^TeNt6Ao@t8Jz=4z3@9E!9SK{=vqb3wX0Hdmta%+x!1&)8=9+;8C~BT5 zQ=`(#SPZpe2A*UYi>5pRG{?Hgb+%LBct{?eU=vv*Ei+?Oc%EzqYn)-n)~$<3wGHQ- z|LlLXv~qsRz^JT!{vl&$yKF<&e~d9z5wtWiCuHSpkA39YWj_j{oa-{6j*SUV^)Wo4 z42B_IGA}+uQ&+eXFc%%Rq=( z431z~eEGSiK-nhQvkKDUxPdp>?JgT6`z^+j8re_$;_r+F5>)vNuW>M~&uKfX#~yvy zRGp=U3Si{s$;Hd-y~&Gt`O9*YhwMd}^qZ!0>!(ck3Rddre6auG8pn~b(?si-TD0Za zU)IxdB@P5ff=gwSRz`VqtkGMfB{L|}0KxG2 zC*FpiVslNpcl%MSOSQxX?BVO{b5_X%dI%w)W$yso;6VZF=xNCf^T)P!b5=WGu9}0X z|EquT-$&4+!Bxxi%jt*z>VN0-su>QrXAOj$aQ$3`oqP4m-=v%^eBp2p11q^fFK6?} zl0~1h;bP9IZ*5+_kIe_y6;MMwSOL6Kl#KQHzxTgI*L1$Can$@$s|>$}_T} z2PHp-l2F-;ZbkPkm`pd8V1Q=E!`9n9Jepej`_ss!OX*AB`kmB2az0H`*(Vd@RD>+b z04JTg-PLFhRzaH3)`?lgE?uG0Os6xl(MUagz1VwOn2GuqLEUql)jllECiI1D+!QRz zHaogQc60)`JQV0EHsd|~s+xO;(=h^;r_*-0%!g=+vV~$1n`@)d?lqjM=UX7O9D{s{kV#bK|L&+;JXD zRAP5A1n?>o&DXUCWv0?DP+h6&VMIdhounu+Eu>lx*! zCyuZpy^5jODzQA^iz$hTvXI(X0F9ol_qkBPZM3D0iB*BH*w$m(kz+SLGSp9E-TCx{ zBe1mb*LavZz?vz;#9>GQtuE-1&H(58n(798iioyB098nLxXy>R2p=`j(p%`DjEEL8eF9@6J6OrA2vJIBJJiB|s$ z{RFFPT2d!xo0|KnoO;U?%O>s1)=;*mLDYBPKZxwFvp`e;L_9{#f%BLt{d#8SIKmB| zotB|9n6k_9_hZaaHXDq)Y%@qHRj>)z;`s%}*ARAAo4ZkiEJSoEp@%6S~SJFdKvcB`8JJfxBUkSRyUw5vqc?v64Tk&RhE_5_iwv>Eg6 zW_~|)j^Qgw3Y0~Z#-U6X^~uw3d!91F{c2_z73YO@$g<<32n6&j7#Z7@DG?Wa>*Q|U z`*aT1$4(O(YLUg0e1$A@z5P8XG8TC%NL5%%3~oTFGDihN-jBWsB_k*k%n5FyK`dlC znP*Bw#v=n`*4vjZ9wBgk9*Q?(JS@z=1$QbiwFgT-Z4>BW#GsAU{IE47)&-?Z+r3s{ z@>$%S-NVhA(SrR6poTIcqRe&6GJUCtGKGs0=a~WAsF%vlm(PnA*Xc_aK-hceKR0l) zL;*Z0GXtSnnJO}szCS)mCAy0eQ7E%WT;w0Q&r%~fUaTE}G+#bbCF+{%GwcHajiAcU zPwVcuIF2g+EdL$~gs}`iA>P?*I=QRpEk7H7flkH-$7YVUSSQPE#FZffo|y$emdzY! zGMYZqc6yyaBkMj!6rl)nBK7n0Q;;D;r0-VLf%*vzbeD{#FixS!QemI@V7Bla-a-e_CHnqm~tI zbA0(rUtc#kniah-lbfs_0%JEBQ5N_8|6i8C%klDum$K$KIe+4B{^wj3oC^qJ^TqFA z6S>yUtdp-8!;nwrE&m)R`^7Q^MAlEgPakEy;(L2No)0KeS0fAFD*&{A+|YQRb?{~1 z`raoBBHc{5%AIZoTrZSEpGn6?InMbNEO^SPeQo(#Wi}ueWMn0ewjn$8K~%5UR`W@k zr%;y9Bd=Valn47-AZl=#vTA({3NlG`1FlUw1Z8aln{WO=)s+6kLj%X_-SnG!0)^>egD7suhVa1Ds?0C z*{7eRzyI-1IE~>}xIK8hS1ataXRn`SqyJC-@Q)Y|K&6{6z%LX~dUEd$(X5}N1NY*D z95+hGj6)r`=PY@1f9F5@e)`To{ZH_PuTQh+xj+4Pe@Yij-bvM~lc}3Tsb`w75d)mc z03Z!CJVz&b2IwA(#>fC?U$W~INc46zgDj_DkYIL3&VW*h++twM+4{5(jBw;rU+Nh; zOJ24%6e|&JC{J^LX(BZl?KTX^#Dhn1P^%Gej%6Ja&)jG&EC8Sp4)lH!9as)1Xn^G? z%XOqp7W{3LWC1ja7e~r*a5vHrQ7WsT;pOEugepVllBg~)*hs$ z)yJu7;bCeg#;=S_sGm% zyq5$o0q*J%qzZsg0cE(kFc)UT9KyIcH=C9wo}|ZjZe#LeBGcG;zL$mENBFDpg^Cbu zgwBaz1dh8<^RMarxsAE321U@(2D`O5OTN7&CL3)B)a_Hx*(VPeM4hdf#Oxq906&MM zp{VK^OZhUGxXFirnC&>|e2ED)Ms)Aw!^fP6z_TJ4V>FM z@eONlYDjlJ_yw%vcBE2w*5fu-EXN+=f}%B$rJin}YLKu7{77 z$C~JOrP&tPl4_uTYl%p0z4B@*j$BN6SjR2=54MTe6})M}tzZEt(x=rd?EL`f3KtE= zsGddUWOY5QJ$XVx;A{+92@I*C@9P0q25%H7V{&39wX>K%{`?Ni&lGI#ahk>)TS@Q( z*^5#YCevC@GZ1j+o|*@B1~;R78PD|0U)e@YaV|u%56y$;QvI3d({4>yT4vW&#yHht z?yiHqP+2Upi#n$N`-!05$0T-$j7sOL{F>Qux_BHq2iT~XlEpM_`a{Ta>vLt`h)JAP zxT}PtmjUoKr@F9q!BE#Qb~VV85z|Egb#$xP(>88U}5B_uER8b2?B5;6(IY+{2ea$r0%E0N<=b5?Q8zv+aQUnUn zUKy2);XtXfSf562XUd2Ln`Z{NrM($QLxyRC=6>1DP+}?*KeU$A#dn&8Y@_BP6^bAb zRl*ngk_p;TOfsXaDJmY@5+**L7s`M-``dXc;~?NxDOzTx_%bDH-JFk^!?sV(?o(-H zZLluZK_93LFnWDtvUQ^GWLS9si*Q_`C-kV&)0BcQNczOdE6tLrR-)V zYk^6epg?gdadIoUhxBo(!Q-)EHI-{RPq}pDe%SbWKnVgiESHUVF z=hne?`m6}Rpw8CE=PGNyP63jYv`>90%cr2rPkf%{%<#oxET3K$cpb@u9W>1)6D-SqqakOiF>p<2lI-wk_K4>%Zty6fe}ItHW@ zT5wFH*ZR=U6n@GD*L)wMLwjMwn3K;XJyLwF(Iu;w*_Nx!x zCko>UpoYZ+W$e*T4JfTLkm@{&*K;vDVXRu(I@6VxUr*(*Sz9o25apa>2@q)G95M#o zJXN~KG~*{XKK>-FL*eW6=rLMHbifu1!Y+fY03G4D-CJ{%y2>$q9l*9=+AXawrF}wl zTUfwl*p8{oSq7}kd~Y1zRk5?e%7PCP3V&mLASN34!yR({0Rt z8_=$cGc(xgSHnNcd@{Qz`COa6CPgS_%_k}o7&Zi$C69!F963Uok#-dPCO&~%8)S4w zNH1P{1rWhmcF@ul*z)@rE0byVzzk_VKEfe$Yw`*GL|_I$R_3%4e~n}O{5pBRO_0XM zqtBz?+DrTR{W(GK_XD^(NvCJ$nLzsG9EZHL*WH%Un|myntqCZV z-0bp|uv|t*{`d!fjSm#QUi6FUSlW>l6Vo;jF~t6i_N^0{yq8+^k0jtl`^W}hPWHeA z)(yzi+|&d~Bsq(g#ViN-D#Pr1;ni=a)-%_G?OkJ%Y1WjPZffsf5T16~IpEyGQqjD@fgFz2pQ+4MS(n{&5XLrRlvd_c?btJ`Ou zoxgA%^S|F^3cM0i&iuj(=}AZ}0~kAM=}L2Kl!`dF78ze7u4-%tbN2)W_7Ic5zGfSo zZ|u3cHS88E2VkA%clRXwO6_Ad%DyC2-fS&^W>&p}}e&#|M&dH3W3nNvHes|X# zAPxoR;%sNANCiqN4HsxPB%V906TP_&4ZW9{jW)+MwAW`ax0^AyM`S0@bDpU51JL-u zD4zMuf*nzO^qBzIJc(S!6@oP87IQ?Ff^I-rSXb|*O*~g+LBL zDNYy$=A+6^7R>8>g#zaqnM1)_fGMs~p$TmLk;N0Nd2igqz4noHh^jD}K1XF0%8tql z=Awy=g`e%SiO{@hr|L|uvJK!!zxtU`F5B<}`7(BfEIra$2C03~(wn=gS|7DebdW#EbnGf*Kw;ClVu z=suWL01)#@P$!5m)rSdNl`X+tfKJL(xvPK$aHp)+?A#p3*O^jk9acu))C`2!2bzto zyHSY-RRlO?vmBsLriP7$4fGk=>o;Id*@{R`)=ux!VO;iCP+~oN3G9Q-^KZvYd9oMz zJL<=0j+bL;Ta?x4lemwzD#vjR=2#g+Dr4D>gX0>qjjUv^Y;2zX5hLZepOzV)I}Y@X z z7f4msM@d-mxXx%!)1A3i>2qr@ zp{s8Ie_v9WEG(mUAdb+t3)7RaUfZ9VWVJk)(Lg!Ys$e}gAuK$%ysLMZym03^z~Ekb z=gpsCom&F~5G+@Ll@UMEho9XJ9sb3y{#Lql{W>H|DJ?H7k=qvEdn|%`OQghFpGYr1 zdnN)g`+7T&;hi)zHk!Wkz3-*(eE)lC_|jQi=*!b2tlK=UD4%|GCw=tx$N1{leKl=1T&fWSXbzoi_Ad7Grj^E$=?(bsG zJ4m0t`;#<&>my8pC`OcE=Bx?kK>>3a<{nc;(8nwsj3jpE_H^y~OX|H&%(T#5DHaTR%uc{R6NUFiI#@PZ(&%*}R-wSdFdtE8*AG3P=#a z0hTehViHAA4_K7eIVB+n3pmT~%Fe;m)H@RAhzgqa^bvC6Fqp&a0+xtWZDesdARkK| zi&{04nd9YCB|cAUceVR~jy(i%9g|}b=g9-W z*R7wwohkr??U;jDHmF#)EA{Lw<`*Vm6EKy)M6n)5*}4KBVSb*$adMkQZDWU| zgP7s!QQi$5IZT>!sRgik0-IKZp}9@6-xH!Ti%`qWDiU^CvS2wtT5aV?T6lCXHKBa> zCLgCVn7~8w-{tTPX>WruMJNr{2tPd(o)Kd`M2EJLcMfYJGI5X|;@?=*&d(xJ1z^&D zs%_{(Dh*sr8*L+LwVG2UFo~A~P73%$Ik_`HwuQpjCg^}Z0^pnIxE>0*2|vd=CgM8D z6|ZB)KOyhh%O715ukf`029Ig=_h=`hxj{sVuoD(>9{c#sk_P$6JxA zKrocW<-!vak&FkyeMK#DMd1%!RwK%Px!^?5g(h)jQZZOry_S zjKBhq6stkVNAYFcU?*lG;6t3%PslHH0?;Z0#CqOn9v@V58@kA7l4ygK+}Or6O@iM8 z7dz7A?4hV!P?!QNO0ahLjT}){b8BfyyNRjB&PWDFe&i!y`V=6*qyK8Y$!Pco2MI-5qp)PB0tm61 z`0~9k>l^(^dwr?oWO9t26HxeEeN3}+Rs}06ET08)A^Sp`)&FtEDT-WBYVd+giaxu$ zL{zI-PWu5fp3KKPzi@v6< znyEF{IzqPFm*DS|Nwi$YUU`tERo=Z=z$#l6ED`b{Q)|5T`q6vr6Qil!%$z>cpc31Ux=N4l-3|HA z`a?FGc}rKwOzr}lw5n8a?L2Zw>wFJJl{VY6)?u5? z$qv>PecD>sNVl-QD<5++vmEES6DRica%g>R4f6{ugJ44h;dVm-meJDEO4frRFw1J1 zni!9qvi0=S+dugsB+E3jnln_f&aADh;%AMS7Tsa$s|)8ZLUwS9!!O=Rk3ao1PLEn1 z=SWLZp4A+!rE3=k$Ois z-ht&aKH$06ZZJlTIAr6tv{}NboRZI&2oDR&4K21#HL#zCWphaRMmJVZSOix2GC%nB`dT z8X27m1hEbQXk$RO*Oq8+McSO6NClkutC6&707ng}iqKiO! z;zKjOb)HA~_D|m;vU3Z7wwVTd2Gh-VKTflcX1SKMCjcH~3($i(0`shj73mAd@p_md zC)GB9OFx}X6t2R*3fm^zGsyUu&&tK)>I=_v%m^9$*=2+A2-qkmD1>9Z+?BK7R&w0O zAq(K(*eDBJ9l)07VbUv*Fn4waFa~?p!5MHzJB%F*O&j-AGx^Q}z}8Tt?vT0}qiuk8 zIm}`|3~T|$^6tC;07d*^oK0J&*^_bUL~(3m((#-<&BWV)fDJ(D32ayuihFBiC9Ptr z*2nII2*`FoPg!kSYC3;Cm4=>4yPO5L1mHf#Orn2OH}%W`)Lwn*O1kmJ>i|*eOr6>= zmmaYL+MAe68}}cijfp3i-2nsaq!0}Tf#az#748w_P)_7g5v8wBp;4vX7sk@)3(o~B zt8;oaa@_1lI1Uh<}-0|09z#YQ2nkiV{s^7YA!yhah`d|!YZhuKT84+=9r7vX*8dI z#aW5SWfnGN^BEYQ^0>qHr`#ctfw-r!k67K#es+(}xL|EGh#UGDrL{7Ah;FOYJjrB~nm4FAjI79%1&gz9>MIsb$(Rw!xR}@E#Sc z9|_j7AC;U}0L%bJGE0I;+iLw)+P2d&Rgx;!c!vGWXSP-Fp&7~4J~9x34ug^`SQF48 z%(L~kA5VdZ3Q)!E_4c7@YfO)$SyBK=w?&&Wf5)tke3rxQ%l@xqoMkFZlx+QsTy}%& zy|&%C742bt65ySVHQG1ok&RzA2DTx91nr|ysTF-DE2L?PtniL{{^6cAqVhu>vSXJ$#>y zxBcpPaibT7%=k>%NndUV1Q%8+t_$H2q-C-Z7oGku`Yx-x*DAaI2u!kj9k(bd#@PpE zpHo2Go6Uh>Mrpcj@!27Vj*T*fd|8MfUfEW5oWBtS@KjFR7}4@%uliBl?fl2C1d+Z<0!8>fU&tAA3teZg%`pQ+a<}A$3k{hlG zGii7F>OX=dyLt(O z2?1s1fEU3=U;FmA!+KJVF8=$!`o9SvTuOPEPfHQj?|PJt7|JDT2eF+*|Vof=wvH3lNtX6u-VM< zM*F#m)O`MO+Hc_apfgv}^y5i(-%zqVQ$@eS>QjXs`xxeI1>h#K?`$IBDJdE%Ya1iA zMh^d^Hh@eWpkNt5rZ3wfzHTOt^{B6tG8Rk92{1>Qqt9Lz2)P%U8gbT_qyDX*=)8%D1-Euy$h>rg?$#Q7(W@hKo& z{Nl@O|H)l<%L%zd>|#}#8J~orZi6LTCK{)JJdpU9Q0;#F(DrfuEHUO6d;F>A5pwOkBV*=FTJJ!KmN*ZjLZxvb(Ewg6uKFHQVh`XHB$jG+hKu zT|j{>m{sY@t6xm}0Mbt$64s5QY3G>{&ZF%o>a{yftj2XvwQ{ScTw1p|mh}{yOwKNi$Nq-smP{XMjD^nBn zF_C^OI#e6B73E$IK%?A!z}|5?5z;P*EII42y4A3IEL0L zAuc2O{Ke6LD6nl@6{2q3U^yA6p$kw1}$FkyPVW>c9s&7hiag*6fZP~{njO5%t$$~O^mmP8g9g$$H4cXW<- z*CLoVJ&iH|PGN(^sTVXSj`o59m&rYS2}&I20Dg{z17futACU-sCr}R{g?t#0aSRa= zyQ8=;D=8$Od%W3ZDVM-&+CyFV383N^ycg7(XR*4j580#s^xJ8j=>qzh|4}c$QwIy8 zjTBXe+(?<`MLFTX zDBDKO3g!f;wkz|oWcffQ%K?G*m+OEVwtX^Qc#Qznv)5a|5li5JT3D>5saaA_Y3;&i z4f&F#HMguF&5eyLv|SQugLu3?=K9vk_IbX06PX~^aXZ<;Cpbsyjz-)x%uQLq-`mLl zuAZjU3aPR>G0#2O7{<`jJ|4x;m2g|irORzs=s)GdjW!XwkU^#)(aO*_Odh$B=h7K` zymQUfkqWDFr!1_Dwdmmd1{NbMNkyXno6w=7qi68}#PWpS_QdoI{Yu~l_~P>OSAv*& z`zLP!FqhIInIoF%ix*ycC3S-#+PZquJa}tvb|zJ}U;APjCwCYlld7ABQ}Z)p>41|A zc1Yncb#NU?Ax?IGnCA0W)6JVV(<-Ld@taHO z#_M0^xFph3a6Cj45sgOl>4&k$V!aG_xbWN!Dv5H%Nk5k(-3NeZu;x+-Ullv>4yf|| z)!A4e1ug@_XGwfJ9p*&(lL@3Sy7*z*#4Ibgs%zu?$~76jW|Yr%)R~cF;7I~$UYUzm zQV#HlK`G5n-A_G4q8_4L<`$Pz4-7^Qh3z7|i(oG>S%(S2s0D0PVUw0gDKKz(>bnKV z;mP={HK!H)z3N#cOQvxE>~x$NNkbQD2L;tJ@da)T=SQ$t6X-xWhnNWWm>A}*X{4QI zWIiI2w#Xu_ecm)9$6Q>yr&wfpR&zlS!?g9Dkh%)weFU;_d?15l9)YGUHg z?i{4n-u~3W?&OW%`c@kM^ltdvbQ6fMN7*MRfs-Rx!m2fnEJIQ7O!JGGBysoxOem(q zHYn#@-(adD0pdOjn%S%;Kflc*Soo-Cahud8^m{pZ02?nphu}A*#XBDY8ejwP#jK2&0fY~w4Cv1WCwVLGf!{J5CYupfA&c_M<78fCiV@|ob+H}FOXA? z-o$5(ogTh$DtgbG)s&7*zr@v!8SSJrc>P>W_ZlX`khtpF2(5{7Y6&&>;^kol3Yfb33j|H z*#^wcm#61pEYwG6ieO+|tqz6>^01*F5Xrm!q+ z0ox|-)AvNBs`8Ox%3~&D?WIc%kjNq>Bl?iKs)Vz-PH8dc87ImRND6$KD*>;}#{=f9 zY)>#g%wbKdo(drwrh>{?F&F0SZvmj2BGsgAQx>dC1LheS2o<&gB{DIlFF0kaTyxA5 zX!?XK%2ejdc4SPWCLoh03tHoa^75t9KK*fymSA#2soHY>+k>sH#}`%b;Rm7QX{Ri3 z1_U!dyohELM01UGYT$ca0YbS)Yf$jiS%7N4m>W>067fgjTMypw`&4P$9y%xd55upF zamm<7Hfb;)hXMlnEOa?V6zNM}*6G)54}V)0G^;j->_3~U%2=vzounYZv{+wSrq5U86pyV`aHMBOb%!f*o{i zLDu|K<|<6?lpTHNwQ+G)3){$5ah-K`p#0^18J5qU=lEvVS|&H~SbaB7$2KC^z1LzX zKu{C`7;Ff$I%JjpMpj~ z)CosB+D6N=FRfD+x$340AIvD92PoTY6sjOc3`95pym`*7^V(pZEmC)FT2^}BTuO0Mibq#*OtQnm>oiRsd#`BmYd+@= zc`SNmSQabj+#J@a8muWYbr}Gr&q1`OI77FB;z1?WEmB}{64KJl0x7drBSKaX`#j)k zm=kxZ0E>@r{*mMU_{y?8BL z1m+j!)4T7!O|Td1EbV&f)i0!Hp1l%&{nN9v=~H44dh`v%g{^##3rw23LWB32V8`_x znAsS_hHmm~p^Q-Qw`XzeJHq+BqB%{h;51v?lj>p6WOe3XQ2yk9{vT7*-dy_9wV`xr zqzeUfkUoC@O-yE!>GfAGrT^}~`F{EahqHeFAOB7o8|X~~{k`eogF6fuu#my_5LnG@ zQsEwV!MNjp)P{*~0%Q$CK)Qo67HR_`G==RzYkDsu8H?2>z+n}3rRY{&B<{k)_qF&A ziQX$wI-4xU2QX^ookMB(`Pb4u*UVrZd$PEanlNj%_VlF_1lk2Fk225zRbmld0}<=z z!5T#U$T7sq(T?Et0bq*MIR_29hiT8qzY+^bJBXp9uNR*{KnFBwH@<-@oRwO&IGL6| zc_;N&a2+(O!lQG&X+4f$A@l+00$I<|W~Z(@VslF^12{VYr+fHmRp2|;3JbLZ`&7jw z`o>qk5fQ+HqZa_3C>g}rWR}~jE6~Lk06U!739#5k5cBl&D*e`sIrK62FS2_+qWpdO z&||%1p)?Vnq5kE7rnmm;FUge#*wsIY`^x}=f|w`c57}+3qp-@-7jN9)^aN<-{VkNm zb{aZ=HZ8ymlw<<@P#TjUr!wo zq&SY=zLmcFyT70M`UlXAduf;x9D3TD0Tb+)Sa4*3?C%EJ6bvwniBX77R?TKL^h=&x zK|{!D0A~7jpLVS>5xW6V6AvFD4^=R>>yg&O{AuRwsQ|=-AqlR*EQra+kP;W_#bts< zs9TYRa0|0R6KP&50Dt@VLY6VV+5j|7*uRaI#V{VbvwHjvHDT_uvy)NQtU5?=O%t+L zgFj>Mo#4`@G-pJZHWp-B_7G4(IDYF>LT`vs#a9?RDED2^3FiJq1v;N<4G z$2w>hv+trWXfL(#>j0j78fI-3vumv5WB>cN05j6ec>hxnt{;?OoQvrXwjBZ=){qMN zO^_db!}sEZ)3*9VBPoOJq#p5n>*xFf2-Y$;y;gs_%xq@=@dWxo)>9_QSJubWkZ0Ag z77GeBAKO!>uMAYO?owU=Bm0Ga%7Ahv(8}b``J#C++l_#<$x|urJ5SL+cI9 zuq@4~f=%_5EUj`Q*!M^=H(mCF8#gyk`rO(!=c}gO7S{f1)&%E|DZ4y2&KztToSvj# zxtSfR(8=ZWX)|l#ZZOl?{g!Kgn^wxx6Q-X$N$IgwWIT{1##8uX{l8EwPBaS$s`@W&-=&>bVGzH20{^Z7;tQFh&z zQ`?mi%98=tTCbmy1vq$1hhUwfu6b=R|S9g)Kef15x1cmGHFw}1Sv85Sly3hCOF z%P6G!)C-Wcq z8bQZlDwvQKwbkFVzl4`A`N>-Qq90pBT6{wKmPm{dG%UT2dtHe zV+Ej|HX%^jP53LOLr~@Uh50|M#JVm3HIv>J@RpZC_$}^*YE6g;QwYLE> z`YKe?|D*UR=GcMc*&S8VPn+~jImeD1W45ko#D4$_zeay#Fw7QinQjoA;H@m!?etweV@uJB!u4B};{s!qXW=Qq%5HEn4P6t z$URtFl|qkuwq5j?*F-Cskl0TcpxZIT(%Sb5#az5?w3h4TJUTEiGYQwM;&xYX?p748bA$s3uQ$a zw$G?lfs{T!wnx^{HfR24^gVxDN7rJ%vlG0}YxM;@1s;yA&kWPC?@_*CCgvW?613e0f;7A!AK^1&J*jSSx=V7YjiQFVWTDp z^Ii`lR|g;w0SfFtnKWt=jH!NzaRgb`&Cy29vbr${I9&^@vtOcbvAaSU3TRC9mobH5 zD@t1c&=SuTylw(yOv>5~v$T^o(i}<}4MZd^+5PE*qd%SwCc>e=CRdto&#j*q8C^7B>GZzLCB zVXkK9F`F*5Hm5~|s2btar-G3Nv{a?BOIHxuMU+@|T7YV<#>~85V&Y=z?d};&3q+%q z7~ezsau3F`86eTc__;jKd&n^!59i;vE{N^u-)*WVHs7ssdvRd&;SO%ze^%B)gpKYKZcD4a< zIHs1d02lWRRD$uJ{R&&FhrJcmcDRR?Zt zNKRaXEC*A~4%5lK#b(E#2Zo7_0Fk~diVfXoB51+>Uxvzs@kMW`4AJqAswY;3RK!hQ*5!S%A|;w}|On2(w1 zGL0&YFeM2zkSUe7W8u942vIgFD;1m%^9j*V|Ki!cf_3rv$T{dfz>@&eYpt8*8(( z`pM_pXBhwjv@*sWD1Hw&KS1%De{P?Rt97*?$4rH4@R{IH7NyRJZ2HAEId%dG-o}qC zuQ@5rY*0sJp#7#I*8K!X8U2(w&CBG`Cwv>vQa+*(uI7CJOnuFK&sY?t$$9U~>jMM? znBurO);H6gnEJ!V$tHS2jMr(UIH4naZ#$F?eJ9=cIaZd*yI+^<_t~Ge&GF5isXS!T zu1v{{`aQqmPW~6yX3tTkm6Q01udGWp4#9%)e6RNiv5-OZhjmbY>@$YZAKd3S831A* z`1Q=);|fNE24=I>C3%Z0Og)9PPN~Y=IGGA1@H4 zN_weQB43dQkJc<~}XNQJR?;XZ^2BFF$vg zh*a`~u8kAu2~mO``Q(H5@J-)OOSqaeMxuSr>1N%nA(KUUIWtDnnW-9zc%!Rm7H22v zB(nMwov4uLAe^CVb{qJ*R%h|MkQlG}gM$;=7`qO9= zeqb-igsfAhht=j$l2zEBJ4{Uiv$V5;V>*+S`G6^-kS?BkHqsf? zVLEI?sqK(M6Ne!@&%NAp$h+wi1GMWPcHCP+Fc&w}1S@w9Br-M1k9UcK~~FA^^p&scLtP0Dxr@CIh0_wQkQn zNjnSU;mfpV^do{U=wNANp~0yahNOjn2Ce*X}s2?K*)-4=>{lStYlc1`UBxpcvvdi=*O<9C1~Xc?@LyTSgU5&8{~0WfJ~qBU}2 zKqYNCFd-s9N7F=+U0DO*PJujx%_)OzYQ&b`N>pqsK>dK=2#+|cXN=0(ovd<%PN97y zRW$d(a8;8FY8&~(pMdu0W77q2{rvBKk}kIxnTlz-;wYUt)0fugXOX9kbpDx3Y#KNw z1yE<+Hv4x4FdqIdEFMOEY67tSI>ig1JYH&=)Hc(PIL!eT1TCBTdb`6Ty-~8+ZW6$Jf9P5#>{K8Bc zJ~N7Wc9$RmTm=9r zKlOq~UfAinFh(kdPzqUU3wP3)5|jDK7I@E<)HI#>vB2;VC9D6CsU7rb(YGYB?IWj} zO+5=y_A-l5Q?cX90VD5X9Hy{P90Dj8{m>q<_=l+uMWQ0IUYT-}UDE+xmIH9fm8K&> z6EJlA2oO*x>cvyQAvnpDQO5eHOjJIRmkf<&TY?qi#sG!XBLEQf_mzPOO=GetGE_2@ z8K_d3*8+e8IPqDSkSw1v1z@LUwG4=6fHcbFK3_gp1t~LPeU(}3=Jf&=eO3KjplSd6 zyfB+0rvm9v(3BAy0Ayb_EBupK17)FXx60mEuwT(Rtd~AMs{+>`zkW{Hf<4uwdExAD zpDj4?Io@NteD8DYH$Dx}Bpc@Uj;R|z>u>rA`zvGJm|6l3)?LeuesOGRHy20!EXy+d zu^v9hq}>5z?I*@qz~IXZwq=il)qzht3oiXYkMUhm5`3iskY@3NP2?dQDW zf0RYPIL7vwA6d5YVV}$N%Am!@hCZ+_vwCH1qx#$09c`e~hI&yaCGv=-`l=ZgL1 zv%;i`tk~ziVh+%cHYC<@t`jT?@=5?M%drnEOmni&Qbz1^e=CcTPK2_Z4>m@otqJ!L zGga)6ZtIY>#(t8V(d2BlhXRS^AK$%+n*;NS>&)S%EXnZt+Z$s1Xv?X5DB3)R{HIut zAPP*&wGL3ypWO6F=Eam1mhF$M1FWSplj!YRcPI~w1TH@Y@ZEg>y)^#$-Spu0O=1F; z<4DohU%!!Vy!--IvwWKR^xbqke~)~*hcIJ|6*%Sb)J$6D2*)dzo=GoVyOgfrR>Zas z>jcMedImUdVAt?z%aON)$Yxv2Os}ky7S>r($u|A z(!*Py;Qg{ry9s>20weTyy{;#vSv`SlFm$G>C_4-@H&N~}%vTA;DZSq4&J8Bx0A!pa#lZp2-gzC@LD=WaV|8r_WS&j<7Sf~ikDcWd1a_yi z@fdJsRHW$~OmWhOQ+XbGHIK0DV-k9_1`tE}ts=-|(*h`6fDIv<5h<-^kvrOkt-Pg&-n>9NMm>#aD87OXYC3#F)Ent?Yv>E{02`u35#v+UyzAdInhT*DaoQn8C zRWLqXMEV}!K;KM~*m5FscM7nB2;l(}(;R6e5kny}Ia!3~vCnG;E;6PCChR_+9qcm6 znRuQ7SHctyb%t*trxLJ$X-@UT4?I{?VI6bpF~3)5(l!Z&J#wsyWW>j9XVY>2g|uFV z<2%4(2hjT%Q>({^)iYiL1P$ywdXV~X(l0Jfq&|RJ6@YOaGbqbYM2jBc7v||5)i6Fl z&DeD|abJR&f=^Ce?0k+2064&urr+?z+NKZIj?yTgxTuMe1!n=~arEpM^f`+t%(T8} zB^DzcYs;%yOB2-0ickGOH zjAlIrI0GW|A>w-%V-qMxDjxb&_HK7u6J)jnk0ibH0j8nNEYu`2Ff<_)&>ndaZL8J+il=PW7+Gp{ z1zJIXU^Hk6_Yxz~R8DnuJd^cE#U2riJYN)Sf7tF4;1BL6Ko&5FPgpZq2dsx+!IyoI zficPr2+D*-!59rF*ikV%RdfiiVIHeEjdU(Y#wu(N_0Tsh^UV{u>d&W4^K7aUzKGO9 zOcw1@Ho4C}4rbL49fJTTT*SjPHyg-dG@myH*rxW5n~L6XHNretuqf2=K}eVfj&p$^ zDErlC#xZNu*Y`#{)1pi^=wE6|y&VJ0@op?wLyK(e?ISlNPCm!pwmKeOC;Lv50a}{b z`Hpedzf39F!;L@_efXtuuLsNv9HSrUOHKMQ?u_RtqwCR!`bw$PO%89;@vP4?FzV*O z*D(zcXt-hUCAe0g%dr?>c;qcI$oinv%Q)&s$5WU~nQ2iS*ubWY%F>^J5acizU;eiC zzka!1bH8)Q#MiOmwAUE_IEIc*iEK5Z&#*CH8HX?_Qh=NMbG61q< z0>X^35)3iUW*6ox(Uip$OKRZh|=V&?{8~!H=pDGj)kE}Sbv;3 z%vxT7E~&u}HUuh~#b-r`CdOaz8Ey`gRZRl+{Ca_G4GU~oS$FXn-e)b974(4mLbm816(*6N+~7ytTy zBOTY{^p%&#(wD9ebDsBPdUE%6ntJ$iGtu9THTC13b zyYMB>Po~NqketT~m9cQOW4GOh9XMjZX6F|HFr24I#~PVdb+|$FAB%mFNDfS)1wh9Z zj`jC%-%ppGdo?vMpa*n(MLh|2yUs^cWNBj|T^t^a+-mPX9LIEdfbat^bHmC3IzV!GN+gXhlRAekdLVTHwkiOn_pEDUVDLg4n9 z_&vRy=`owQC#}Pl8$c zsve}j&v+J8u=HO6Mz9~pQQ4m(7?NF5dDJr5puMaeIs;~l1!Ir)npky*%;$#yqDh3S z2QXu1_#J{4hAv%B#qqMV1%tSo-%Ep+&H?UdJBwc{yDH5|S}Mxvhb?wj`lqRMODv4_ z92^`#oA#HX?A%D5tRm=j1Z?zPO`EVbs{|}GW3G;j+sy+|-}qh;1?1@-%zAtP04M-{ z0Y)Fs%M)<%Qh#S!1t=Fu5^Juw>Bo=K*=sLyKIt$=jIE>_XGhcByLXVAJPDHVaU)I5 z3_vWu#wiy~BVk&uz#MDTsD6;O{5Tg3HYwz%a5Y#;=Pq4<*%XlPq(02Z1^U7TTUJR{ zOU2cKGk<9ToQ%KD2@9I}58H{Lx_R@%^jl{KQy+)RS5fx@zNLCAY%FcTa&D%DNzBzS zih`OvW&$IPToiQ%UuQ$Jis`kS2y5lF9w4_@#_1O%4!#Pdzbwy`2FMhUlV0Rv zh6KW8D6nHU2F#5DvZF(G31V7sb-;5>=4@hJclF!+tpeVoeN`w>*8>;Jdf4a!ky5@i zd1|UtS!$jtun6ljqb?JOgwG_Ohhpb?8GK)|39-0Rw+7~wtd;(6Mx5$!Y=TaemZy*m zve=zQyVkP_3nqmp>OZK1ygT|}=a8|?hYrAuzb2p9e*1|+db440Z#^4$Wd}gU>Pbv+hQFBQhqKQ5WW}~f5ZV9;r!xaQ)ZqrP zZ&Ku^06^2O&+|Eg9DnS8K@yN6#z$c0CPhUqLm|^Dcy#Vt{{X@G=4Nb?fSSYZ5!5|o z$d~=HPaoO_fz7e~$v;7cK+4Y#FqM~~TLgH5nlLLfuat=riqV7Ni!}nNOySyhn#;Y{ zItQ4gPWtmHYl1TYNrfOyAT40yUI9)G8@cE>{%>D~uOaV!Mc;E{$N)gfaTVws>@qOC zGz%*)j+tu`4+1P?K6l{*M|~V`L2MP9*f=kl=Q~#fc*xtT&`llW7#-uUVbXTzsV|D6 zy>4*qSH{(L^0E%W7_wQ|1vu4Uov{+uM8?xGXl?Ic9h{;}^8P`vm~u&r*#Ya7Rt-03 zhv?dNEFX~uj5S}m(By6VENxp03u9|V;;1oNR)idy9)HaG%?TrHdgG)9Hjfdo0a;YX zW=J5t&S^h+GC8z%;Ma^9v=ZHFO0=6YA5in0bNl* zW+(36#a!B$Ucd1&ZbUe=fF+iW%E_cbKHZk%bnpH+skpupvhedC|5eJ9p6z$P_7#A6 zB|!3xp7Y#`V&Urt|=Cd(rM3s@V?hpG$Bxy!AztC+s~-EZM{&atGWVVVd|=sHKh z;r(BxVJfmc{Rj|;c@V!CcdA39qp>J^hFv9pH{k2k4)V+44Bf!u<$>&tP6PrFK7IeE zm=HIjQx0)BU)!gVmEBsNz0O3xjEq(ZS3KJwD{W2`< zN-Eh29Klc!qCi9?=E32MX)o7H_WS*`MHxE?KmpU?gVl|+gd#`fGB6tePFN0}W2EYl z3lL&^V68X5Gwsm0AvS%p(;$UrTIyY;JQTs2O-e_SQGOa2~$s7TVcdz zAKp(nCSQMlAHw;Fmrkd;N=HO4Y%Biv_zL-@-Bg zz&e1r=lZK@on6WfK%)}BKPN&13cdq#s}y$+puQ1?!E@VAm;ieST%&zMzvS^>DdMwu zD5J(AQq9iN^GsEg?);J1R0C61$yhFtCdPb4&G_pXp{g@{0}DzCrqs;W)t(7A^;zmV z*<@@vyOk)R9ZiKWkRBOT-rk!IcvhUM(LKNsVMe}{!cTIE9ZNlJZ7iVNFbQ@5JgO-7 z5Oe1m;OyKBFQqyX?^eJb@yJN?QG-${(7t2(Z4uxEZVCq096y>vo4UKw8ge27lHXVh z^L{6QZyRRMM7Nr?4*|&xC5fM@GdpmP;xZk?*!i*8<-4G{xO#kWis}Kg2O6{$@<&<@f^{EU?3(#oQ%Wf;zJ{xf`uRK8>K&*U;5tPGe+LI-k>p)oCui?9I01yYb^!%w7kP%Y1_X#x2kLVu)GW$qBGyREt z@8Y9Uin6GKjEu}y7f0Smkeb`9Z310#XSr8Sm zrfQiO&CHr~;aBnweB=wl90$Z~8aR z%wupU0aW-bfCtz3JN%fq*1G!20104=y6EfY zx@I8rSpqKw zL`dQD95Xk+*^yrvV`S+@ZnJ=g;#Su9BOWP`Rz|b>W^!g}ZW?Td&$3+qw!Ya*e=+>t z7)L)hLfZFXj^m!Jc_FntI0LDc<8x#DxIkvf9BbC?blgB0G43)!79z8#&XKni*ij9B z7%xGUV{JLv!=hRNvtn#Apm$nVuL;@bVVSv_KBWw!js@P?IA!J8r^=(_6y-l1FUKX= zR6vn)$Cn_;e*2B-Uf@Tc+jPb|05)w2|2E20=3ED?vhw5HJe5KF(|YjG0D7`_d=TFo zJzmfHqRfY-gLS}YQsrisO?oSlqV|gg>r3YPZ5lEA+jCQcNkooXT3dN8b619n=lByc zPhF8X{v=2aJ!opKX55TC6u?tv3<~lJ+u=CNS})-nUSJb+?bTP(m%j1!bmf(oIbfiZ zG$Bho;5fbgw?9lvoJ!SRc^HxZBQ4^P6xjtfvD93D{(5@$>Q&Zqm~nJW&$BO(b>VBN zj*J&*x75)!m?j=QOt)`-fR(N`z4pR2EF_ER!T4-?G_#pj59`w>Q(I}SKo)=l(u{Eq zR4ajp=BlfP__=WLOnUz6D8A>Pr~Qpt^3>L*VNzLD*U?wUOB{)G2m?U=wz+#Bb8tJA zW`aL>^e7OZ`Jq}syC$R6@!ees1ca+M9l|LF?2rt2(3zZ--pxCQeUqoE_fo3qzmf{I zM4LjLcd%fdZ?A{dVqwNqXwIxxUVAxx^=ogWGovFcsQ9J<0QLY~2bdp^nGo1j*u{}H zgaxz=kd*C|oJX)jEU?uKdJBtgC3SOmW3s*?W|LUN5MC$82>wvh4?j(3x;XF~X04q2 zT)caRhbbFBCd~1hm>Z4(U?u6=zEMn&y2)1pgbHaBWBud+2#}kOp4I$U{$o1$E zomgx6-QCw0`Q8dl6gBh#6JyuY4rC+xhf>FLuMkz-o)#I~d<6zY6(DVQ2d(MKGg@;n zcO^{93^HkB0Q%J56Ev!O2ci#3&7IK1Fmqh*;r2S+o6u2TN7Fm(0o+W_pn~#Lfm%Sp z2B~8z_^ci?u_-X>*+jHdkwXrq_h=={)t7_$-62>*MyDRYvxW(Gl?}r1)#r&;WufN& zEpySqC=B5T*aVwcT>xMLNObIP!1SyWbvHTY+S>33+a}VJ4u9p<*8nL1Fvi6sye7|` zdVG&XgvFLc&IP4&aELKyae)=uU=cCVbR&L|Md^3Ita9Y(3_CcvV(noH;0D0B&&|x> zQ_8Lou*Dh|KBJAjqXfyEkjJf!GxCT~et0L$u+4z1a+s)N`f3wuyxiljkb!;r++&_} z+LaYC8EOLypUO$a^8c{)X1|eV*`42Ia0xQF2KRkS=2DrJnYH#(-Rf?YB#dOM+wE4% zo*4;{Uj}2q#`wj69}M^pFyIFp7!M$fVH-2hVt1=YEsa{eSCwk%t|hA~m&%P~lF5DF z8C-&-KcDkU*35&WDw1Tx?{|N9Irp6JJ*yyVW^SGweM9MxK%!HCnc4xFzNU>mz;P$( zV9MCd%b5AL)Eo`y2MLo!(|U$>O%sWHJyQ3uZ#|^BvF&q1eKT6|9mY)%5NjCw9igss zyUx-w%P7I)^Y|>&kYvKmM%A!{#(?WQm?lM&yBn%h|?EqskrG?;B@L+zw z@Tp@g$`6Hz<`}gA`rWv{4Lb28uIYx0L%UlF_YAZr@Cr@6*0L33X)0B-ra;qqzK#>F zOy2d6zWE$CBK-=G0zE1y;%~mtRY0&Qa1wB3V_@Gx+a?UnG=2gE1Ri~NUIkmBwTM;W zX@1DdHB!*{Qpkn2%r$2GY8Fjz6Ph^s;#~N>e+yjWZmzX2ozS#`+~+9#;_bX~t#fKW zTssAC03&!q2!H0m&wO5hlx*P9zW{X6Grz;k1%FW%`rPo3QrP=HZSxQRD=7J&vH2`t zS#)X!;++@!BY5+BL1f%Zd*1K-7%j=eV@~ap>+4$khyYStllgfGCj4IB2^}C~yZSeBwj3Y%TfK`?)E; zi+KWkcz|^rYMcEm8_D%x%uEv_vB@->#RMv}lsK{(S6ofzi-T24>F>2H`WQ zDc+@uXWG8~^B=Y9jE8xUp5riJ`?L|hbZT6^((ywJj$se2J`@&*yk!o=o!!VNeBC@y z83iklwC#Nkn>!qm>j_@7{x?U8NR-};MZ~}s*I7DnY;izg-qj3dW^JNhIeh7duIxO- zmmBh9?bQrwK|$exFPw*eWLU)Ysya>Izn`8USLY_5q&)uF7rRMo zQd&tvZMBqa9LMUmp88NmT)XyMS|zA+7FVJMN*Z2$?RQew$nzYW@S6t{tHsM!+7Y0QXhDuh;Om1oaTIT$3{@TPYQTt& z;}+B|_2_?-0tapo;TBLsEJ6imn|!J!gL zm1sg6NWywBbvx~Y{12CAQV+gsogHm}Bu+`pdo6WNH?~MLs^0*_vN}7%+Mwlt__kIz znU``z*9;Eyq0J->4B$4;TvP#`@@S71Fj=DaO$V52SJ0rX;rnn#zFtg-7YEv`HBGV)#;3K>Vl7Z3UGarbyts z&05v9bkYg}3{6b~DN{60)dEll@b?pbcN9=RlhS2t?r}QYSW2Uq^4;VGK$xXl?tS)I zm~~{pt!ODLU^M!gbdsQ2ZA$OJK$w=y?PoM10<6KGq6glTw9DWTZ9ns+d5&&=T2?2=OH+Uz)Lo*M8Poc%!oP)O_CSaMo z^dUfkRHCYgpvAhSA-q^8fsD_znm{;EvleO)6dVeqefm|;Ve^bJ)B%i5Y^~HWn824H zLWVEn&*bGZz=JRi%>un}UhJ>5qHvWyG_8f{4FLlSjG&Bg-bP?Z&wH$m*N0|?pYFYC z7_fW$Tp$Zn^ab)I*wSPiCMG(fR?nBf^c+0+cK|%DRmeE-S~U6RU`_s|_8|fkSd+|b z=mRs2tsq4~ZlA(b$2eRI=Q02c<5U<6ECq53V%O4V`nxF=?7!nz;42jU&R`Qii@*%W zPoLE?2_Q6+`aJJbd*j@z>G7pi#L{tsar+=Ba|~Xr_DJ9)@DG7Zo3>|LYIXz?teBnQ zK)^iFhIs}NpGd$G6ysFg3ml(PoEL}`j37( z9KnDFI;_vmr!DwjHFG&KJPbe1r_s?t@XZ?5 zJd`C<50Oc^^x}<+@Yc%oa0WlP!HcABdLgy+U%_lT%wc~mO?>=ey7lHS!n!s(fc1#E zT*uv|vd~9<)+;2s9ZCCD9cf1ZNA`mHVlQJMYXsUleaElA`eiCFyoAz>ybz=*G5PP< zrR(WuZ{JIA-+Gdkwy<)Xwx`@2YRVbXZ`9-9T+Cy4#y6}7ZAX2vD?KEQ#NGMzboc2> z`t-AhWH%;DF=Zeu(_`|{Mj0CjSEp|MjkZaLaDW}T1|k2HdZ|xIYq3Xx@_ff=6!5NP zf;Dd~^4%}~<}ZoB_&XBf-ARA@@4lCQ@c;Zdpn%d!6Av-lwGyrO8`NcOPirtK>y{QT zK9?FWkshHX+5yGR;3rcFlQ!D2z(#4ouHAA<2^NIDD${E(ei1>4pAf!7=Je9X?+|Cn zt~T0ap=9zAu}S{76WVOV->e2LS{=Sh?EpAytDd1btif^q00vXUAIXwi%Q%eJ0MZ`K z%%?)nWt_q>Q4!Q&ij0Sk9*6%@GsrmyI?vH&XU`D+Lxes93QB5ac%uNrp*)W^j!g^` zJw&K-BuGo3e;NJq!feX%*~({kDO+}!R>to#pPU9}@dE@j)syO>7BEA|LE1nAWfIw7 zDoW}=V=DV|2-T>9q$%Ce$DuqR1^Jj0~2i}nz*0;=&u2rqyYeYoxzy5F&Wpe zF`Mx{Yec=Rv++K(&pYH;9VR*^uO!v9SDtC8PHfJD!70)t`dKP z{l%$tVRV2?F;$vL8yeB%Tm!?Yr@wh|uPtp=bHA%1{LUxrX5jmph0eKlQv z`6UE%9@hb~hr{?3K$;OvaLXT@4$Gu{A%~vkHrB5Zx;$Xc_F;79FRF&wH0g!_olbuI zK`N~+k-P7Dnwoi%331lm5~S5$0m|dj<6C1t;_k&daZvus=6p#HpY51W5`60cS+8!Xv^g$O8@emr47*pdbV+ z!o<4BI$bNUGBBfHunm*JW&+&%0&KEo3aA3XJPK-~1x=)FQF_Zj>DMZl%fJdQ7OpSX zs6EtIEDE~gBdg%kcP9fMK}HCUY+U-YX*%^ZZ#oTyo&bq1A+$KW!tX-(LMX+;sgBd(5U#?A#Ma*%{-xj;yRyOjT~^U&JPa(9H1eAhcD+dqh05R_IT#` zxc!-#6fIdN#VClK~+8M5dTZT&Efta-BRul+IC#)&|Nk{e}eo6z}?{h<|#BD&wsJV}&JA(yn$IG3UR@_Ot zdO6%8cO7#0$U)N{o_4?+MzH6hd$oz({R1Soy%{C2^dB`Dxc;x_v}z=MBib~lOf6p2 z+~qG#@M=Go2#%9S%P*y$M&zPAp|iJ>v}RqD4_u5?X7l7*JHZtvk8irN<7{UI|8=rX zkcMguMZnhx99$(6#|nXO*S`1_eExgV;Mir-A{Ek`Kl+>WZ~o=~k{+Ybv5xpW=}8_f zY^Q-MUk11hr>&E^G`Wh;YQeNz$PHv!4e6`YR%Z!{%V})v3f7`slmQ<_e%R+Oji$?( zW&hWI{k?R1YAfxW5X6Xfr4DzlT#+gSSoPvWy$zV#*;r=q+v&pPOXG7}XUECRcXBm5G!;sc^W zI@;UQ#r~c&)PlAMXY@9-Csq0y;Y(-bf>ZpKR-aBFFmPBT?^b7jKS2a&@L1i)^K0$~y{xUUV`%&Ez)Ifm18tYtj{e|?>fB8qL1Fc~N0$-4L z3P4z#o1vcET)H$kfP6nnw|@2OY+l)16>Nx;9h^=9E;=)}pz2+~w``Y0y5_Ibk8uN` zqkp4JqQK^^R1d=nYlR5t7 zz44SVbsqvS37ih0ATwLX?AHWcAU0ALn~7OZ8xxOGB`SN6G4sM^W$ZVvJs*J^Y8hm< zhOiqIr72r;nF%ZJ-~AZDWs+O8N6^PHG<5x?SJ8s78VIs#1fBpuAfR9|C5B9jtVX%a{IbCMs1>`+NAf zLFY1Vft1=2BLp*Wk!fZE;IUUtysXKqz~MtjW}TEZ<>>!b&G+tG^HwPo{1Nvda0C!p zEk~Zu=hCIYJy9o=57de%;4`6)g#c6S7IPZ@w1C+ffNhpAmZ@}ZxmN+;7}di0oE!j8 zErSB860JO6$NdUBX;!!+^^Gn2vUAm@Tm1PEsf6g3PZJO3T&U@S}Jhur#^X^ z)3@+M03yg>d|pReQNoM4RA8!+@H@3 zX^KD>?l}iE(I;rfIrchVei!%9kMl@fqf`CNal0m((gie`fIio1W^)*Q;5p8N+P}CD z`Y~#lp~QH6hGmq14)MGY{H%K>=vi<0qK^8R_R}Y3phX~~2HkO(>WPnIZ>cpG2s#9_ z{=0XaH`hzQR1W|GhoB4A(KYwt7&C1M%uQb-(9e9;GO!~sa^0m1XU8_r0fYAhm|-3S zj2WHVENkFCaD{kAwBf`)8*3I+A~416g2Pn|fneaYnnlX!CJY=~}?hW^t5#y{1E5IDk zcS{0h?hlpe&L{z2vRGgzRM_*+Q-rdcc)OCtfPg) zu7`!Ct#>F=Q1$l@rjCwc>cQm&R!CBI{J+;s(*Fq=8p`PS+9`eYXBrTG&+`6o<2nAccf!JFC~0_Sd?v& ziAso(^bylfBc|9RnCmX)KgN%spN|w9N2@F@LSd6M7psJd4l~&9#x(>?s{ z`T#jPqu0{sD$>-St{@K4&g@{mDpKEd8!cRY8+A%sar#4pv$AYNBaDcRCo^s!7g}s& zP6sDM6&`silmY=pCv5y9fTkNmhjhyyosq7ASb8Em0c!}=h2^!>Lm7;&0f?2HU%Po8l6QD5@b2k*z`n*qHK^G9MP`bn)sPP*=lO#syaT9F!5#Iit(xHq7En8GaAH9Q*HBhzFkbPoUA1fN zNIju%U47kYW_6380nE~)sI3pDQvqAm89DPBFpFzGUm+Y@;ixgL^U{my!TJ&)sGibL zkwN{1K{hOsAegt<~4mfbYY^XCVO04sM;(Rqj_nT6a`mHJ#lx!OqS_VCGPKrRW7ksml^0Xv2<21AJYIt!lBsGY6<;_L* zftvy}BC~`$G2WWAxpTnt0MpR$%G@+FL)-NCjFau=%-u9OnNPzn{Z;^$gH24n$7!&-ctrx)U zp@pRR092=Fqm7KV>Pj;-cS5AvSsl!q&kVS*Z)RW@ z7yh+y7K^5PLbpDyIqa&@Hf5T^OkgWKR7fhE1%aB%&NVpUY|Vk_OeaYCs-{iA3IvNW z*cQL?nE@gQOJ?E7fC%;A~+Pl2_^(dA!unJ`o{MPa`%I4 zkmFk2Srm@u!DABX18u^~%@cXR;jIQ5Rxqpm(acwk>xFY+A@=|U^h2O2fI1KqVkWW` zzNL@Z7_DD=L_%C6LIpwr4(LY!ROV2JYtA!#t_f7Sg;bg2NUOp83v7&z%o7lHAf5A> ziBGGH*x`Nw#}RwP!!+`k^}<9epx}?t+)CjwH%u$v`y3Cv%nIK#|5y_QH0C99E?n1` zXFfGrQMz(HU8|T++O{3K!RM?ImKa+IUpx3zn0mRu`Eew1F9Q_NW$VGSc&-PEGv=Jf z#plw2_vjnu0|Yhpu@*jY1`P>h51@N##f0bF#B;qgp6+ki+YjT-=eEIG1uT@t>@;&* zVM-1DJ9aJkHfssM#r`=)COWQjAZ7mqY0|LsVmUznR(pTqeu2&e%v&6Ur7b?-xr~)- z!Uce_#{t5<9pDvr1=kMxKld$@{$kCXNodnP#)~&<^TG`kJ(U(;FDAWM31+TS8u#nRkjd?yXovA<-8zK;dz%)6Natd?}l0os>` zr1AR?ux421OYjNrW-kcdTpxY&7=e?m61&wGgv1!36R65pQ!9( zK(7I5oqdCpL+k}uK{GJ4H-7Rr5uxAJOkPi@7JU8bbkX>N`y2J8e5?hL4V@79B~NV=1SHCo0}vGfWO8x> zQy`H?nEYP(+SkbvLX@YzfoyQWL>+BTeehQ5V7-o6n*&U03ZM4Q;WSN3m}Am$ERyJV z0qx;~^%4M|4AqRSj!0*NS%wEY)0bYpnnwE25CckD5HJQ$Y^OVR1h*( zeTF#M(@z;+D;mjKoZ)4wp7i#aL-~J$CqvCeY_QNiu?BKl83#m!%AEwuXsS<%EH>Rl z9a_g`N?R?h&^I%2qp|HJNJ2IK?N2e=GVVdj#7%EvEC9Dd002M$NklmiJ8du5ghbYhv-#&$I@!2fRUx0g~V=0uiHnEq|q%vuxjV!C<~^D-WZ#vI8CB zU~~{>XTgttR|+mG4hZldjR0-ttTRnOT>=_TFh#vZiW`>3HZKRvP z#=Kt&3j>YR-1zcyC7YTGw>(1#L9E8XI>>4)Lu*OjBf8dMhj7jWx@~Yff=CYA3_I$M6L^Hl_w#3QHJ^pXmS@?%>!@Z^4UTjW+D$Gy zAD!B&c+sb57s6#$BS6DI757#gh}qXcJ6=rZ&!d1{2j?XOEYEYTY{z%Z1GEzDI@!Eq zUis->%>K^i!g_W@>9R`9!8zD(n%^#v}^d>`tsMR(K;#rj2)w9T{a&v`#= zSAgiVeaoDq-^>kLB|gi79-ssJ#+Wse3HqJCpf#b90NneZJx5@Y4dXvNQ{X6Y_2qZ= zLG6Fg4t@4J`);5amrTRP7jdohlg+F28O;Ul=rUm61cBjFA&v2hXGvSxr9r!~ z{_=xZTQzj7A-~hDXU|85(WlT@GEN(@eRIb7%40#8c{-iZjB`a>2C-?Su>amGf742) z3^adVR}XGVZJlX)axzTy(J%J7HCzJ-3fy5E`?y2NqkX0aIfH`qk(Sn^O;et@ZXNJW zT@PIQ24vXFFI-PIu9JB83ooXZZr(^a_?|qlnKD)*O~}^-PX`d5a_DIr97}lcg8+jjBuyJsYfVcs^YUxi46|B!bN>iWwJWYS_x2Zcf zmxl67X*fThUhX|gUmwn=mwHG)lHW(>q8!1;edn|_4PAddz3>~~OdI&nnL)y|OIkSk zhelKG&YgQ?<)39^lIunRM*4{=(mLb;PLDqNMVkHW{d7obkxtZ}fBfBllK#j4{(lI@ zwnHAcIv9VB$uSw>A=(;9F*acXz^aIPe4w#9_2E}nJlmy~C)zInLv^8;K6o&hK7KHr zIx&q_0C4u_CQ>gEh@-W%MRL1~H86JkzNSBTGhJw^N?kbe6{$aZLUn*M{F?^*Dfa__ zQCl^3iAWQ=CrGIxVU(+e72C2cVjppNAn-kS=8f4NS)D zyCD!ucvy#We(Euh1g#lJyuEjjwmJ~l7~`PgV1f|99w0$YKrmCrUXO-x15na3I+p4N zE)f{enLa~fw8MN_l-CG6qq8J(O{P2iZsOq@QehN8^oVD&@#EKm@Kdm%>k^ z5*2VK4)6s6N%WtSY0&-Dj;XMgxv%F`*+{=jPaaTfbv~V8zh9?DW;4%TnVb#N<|%!f zd-MQ5K(u%=a0s{LW+n0O9*JK$yQE5T?;O=tpx&-YlkfZ_z5jpxX}bHvKTETpz6VI; z88~oW933SEN+mhHrFqdjYgNf{=q-_#}q+C?COx$&wXGHcP*0Dvk1HkAmGO8TLx z?-=u#vQi(GEkMf@K5*T`W9jCLUrN9J^{w>Y#~)$lgGo@Gqn)%h$M_!9Q^_Jvr3M+0 z%F{FG=!7y*g_~bZ_l|MczwkUh1|(O8v9(ejt+%H&0tV(LAEnPecqcUjM)9ObgS~xd zSGk|{HWyhH!uf!CZDG@y6KE4(KG)TRme*gnN$$0!_36OE_|C?{FHtc#V8?hv~tiNletF)&W32 z|Fv%bu1McPa>*UEOU0f+!rKX=Xr$5!j8CDNIjv`Ap%RQnKM?&n3k|Jk?Yf~EgcEdi zLiAPvrgMxI)2LH}hMqpU&H5H$GJrz>+!jfmD>xbIkEBU>o6z>Dd!IxKm+hzH>9hCV zz`qo27xZle@x;T=(k#qZ#wQ~TA&THo3$Mn<-~uCqLmt^CJV0h+h`!pj8V(;v04^h? zneq^H-eYN;Iy7?nwX74}Wr9-2s-{A-7Lyo4&XcxzoP;}?h0PB~CoqF{ zfuHB|u69IVr;6p9Eg3;gYxBZQ}pVFgu@azB#^vmbime**`5@;8&5IBF4 zCysdv5W;;{H!<0*AV`zG;6?3QfG6h2zN)o7uWy@a#WD~TAci#vfz5Tk0{qa1^=>mW zC~HA`&X=h_XppBm7yiq^!DmC$H5`?4BUO(2R0Pd=~4=wQ4EP z14!ziLHi!1&uCj($$SsZKWbaF?4bXnm)E<`eA{>F%4Y@*#=6B#d?vVMt_a{0AZCD0 zAf*P3m+e63(!qI~5y5TS&%m(sWUxVIqCY=Su#(J?`$-TM>2_EH&FUV8q}j}jiu|B` z*VC7Q14fDaOb-wT$Q^nB4X`$7`@(0Bahx#M`qAIL^=_mO33=)q(zbaeJ^bk+)J8Di z{kwN)6Ni3yt%qs%w&Nw1_o)O+-ooRgEOsii71{l0@6~D3Qaxxby*QLhuU+p+y=|n5s<#y7mDDwQISpO=Li)~s_1~r*tW8bb7t(Ki z>$~aYul{Cw@e5y0U-_N?6g)ztEE+@df|9A2)A~AQIm37zcs(Y?s~7uI7XYk@0TcjF z6$pfPe*V++fBdU|mHyxxUryD4se}26)P>V@JsVzvKb1iuK=-y$dm?;q^faVb`Wn)! z1C0b3sO&92nJ18Qn6VG?9c$FkW_31o5X?Z4U5M|5jcAS6dNhuz?~SaQnSPS`&uqbE^1NJi(p;@tf^_ZdFfIb9q5OUPttoIzMpz%&yrM)R3_NK?C5y)d)f#5)}y}O zMN?WyhUu;v9NB5FK#}+-@4XSphMO4I(*1j>gx0CdX~Z-IX0PATbOiuD<5ZzX*8YAD zfqs_8XBN^&w?9RAq3vV;H|Rr$5Pyna;?)a7RKPG|ww4?{*z?iI*05QdU?%LgG}_w@ z<|0J}8$~l@WFcoasqwUQRuz3&AYjJ2t+2gxucu1t;7u&9vNcIvp&0_&s71A117O%kW2w-eefl(fmp0Ix zPEStp_e?O+ZPr|&)`WKPkZ~Sis_>a6#`QTJ(6BXha{x}0PM4`P(c0dO$#XY7p|)x@ zsc804r5|u|Yb15eqsLR}@$`JUfTL=jsNfGjy~ny>;zeV41PH5^F@|75am_r|;#;N3 zsDL(AC;kI`A?H@mP!gSc2(6W1EP}~qfduoGAG$CqLY zM6BS-0d~}^3vetubILrphMrPe+uQH~d?ox+)YvGbWuOWx)5RPC8hx%>=UPnF1>UOd z3(&f6nJ&c5oN!>k+!_8i+^?u5iAmdUv@J~EfkY$2tou$=ms+|&O36eDzswv{) z=yPaj7^kT%6#jx%$KzY4X!)i*ML{f>6L2^-_qz`LVeaHw1)Tsv;hF^uIE@(qv0VXC z2q$+4eGNdSK;ms%A(-Tpe3%m>do%>b0^r=z;}+4q<$u8ou<&{pPul`V|- z`@QX}t#qvfF-#5fnXN;%u7C@!^QA`CS5{jW;KY2zHGFRVKl|mqf=e4Ty4IIZj5%ih zoL}eBcO82eJ9VtP;8htDRSK9`fg3%P9AmwtQEsxi?ByCVC%*hCSeK@Jj#_8uzz4e~ zg=YX!`t>FF5pa6FZ3Z1dA0CdNQg|NyEZ1zo2jD%zYW-r!OAZ13)qV(zFrbb~L~`_jyS3`wl?S++$=ZbhL|~x$QeQ$2?d6 zRLz_NBNup#!8-7DXhfQ%(~%a$G+Aa2(Db=UriwArcqtdH266yivE*T~*qW{Y&MsZK z9CE0RgRqg#4OrxKBam0@W6u1GfBnCw4}SW?bg{K6T?4a}Nf}au>{*027Kfit-J{pi z{OU$}`&aK_t<2%OeU${eui(zLomO_S7=GcaX%^l(KQ*0R7#<<7?)~)mSN}e3PrQ}V z!aM0&aXJ0r7hBRdZ#JYCI``6WVLc7E;B(x7Z+B@k-TdM!>0AHoU!-fl_m9)Sm%f== z#$HR0*5G*@F57#0P zKrK;fa^eqHrZ4p*w5!3M^5MI0P_OJ`On5zMKB^@UPAl!0 zi%s)jJ;GuL5Ao_|m~F zAx3=@T6w;{KW2ZvG)&QSGDNTPw$xA?1h2o)28?iheI)>IgDEW57SNI*v?=){=slQu zkoG6;r=#gdsf9KAyZ`%7Sqrp&G7L7nIkHxXRINcx-9QwOdB)0sp`BfPr_g3S|GVE# zgTM2wwB0eBc3Lny<1~49f=UE{{5H&iGT%;4FQzmJXKIAM;Ba+efy9f;!FUA%2Lx(V z%RD$8s@-ga$?U2bV!kU`a|MGDppBf;?%w${J-q)}lyU1K>T{PiD*^lUXpBxvn8WZD z9HJgxg_RaqixL8+vu_|&*@M@}3)j|5-BdJ11YNA*XVf4Ffp%-SwgP9k8ia-BU+dr@ ziqaQfdO7WTA_ORz0NK({rn)%;0Le3DMhi33FbJ6uF%vy~Fiq%^6K*4Zc4`;QKrgMz zygdNu&$7NuwFLH!f=oZAA`EF3+H~(!m?ROvF}Fcln;N2<@zF%z1G5D_*LP?yO9U$J%f}()l=iY0xq%G^Jme_!2<58$Ve=v!(LRFjt?U`OINU$)nZN zk=;2`Ix5KQS7Ql_eF+}D*Y*r@5MZf6Qf!!!EAAYW;$o2$=v3CGU^u=fU z(>@CJ11#{I4D@gh&r<8-SezdLh;!xhlwaPL^(pI|eT+}IM!xkQUV>)6W9+_Mf5Bk} zdhBR6Uq0X5iqff|JKB%uWPfMv^K8cMoC)rH*-t(Q;AsEUCI?M0_E_^+2Z234mypxB zR?1%cJuT3MY&`Vz^NzY+?mdB#z)3oYerH{d_4IxLt3UnEYqM9@$7qj7*vIqv%KYL` zNC<$S0Y7(c1+6p`ediH;re+#>fy~l}QkwPU7@e!|3+1Q%^cBzdd*)Ai_-JYA8BVse~R z77L(UQie!Km@Cyl?H*ubMmx5InJJ>u&;T8yqC7>&G{ZcbIq6pm^0-Em={_d1dHjAX z*3Hf&3YE5ad%ABWB2Gb5W??HJNb{qu>c@jdpi0w-8;WuiZ2i>R-%o0argU|9F!cfg zst~kw+7Gmt0SN9nE4w|y6{*klI+2Y9s(*YrLoK3GFU-l zEkxR(V8(~40&Pwk<7@;J8-~4)usY(iDztc;tbv8?8)Y!8QzPrw%y%PbC{LH?Fhjqa zt`GEvGrZBdmM@FMk8Hvg0B?zBX)>}*PCcgQ6a0}DV8F-SqpABhelHbnplv0;O?}%? zXcF~h=Ojtwl(_3f(;6AN9(UkOt&b63T@D!4T_Rsf`Ryq7_r zBJ>-X$3DRDG$y)9Hgy9|&d1P86{%PbX{VAyKqa(SUSFkrS&{r}TWJ+@Uo$1n8m*E+ z{+xmltV9^Lk6n#`88rr)E9%ki)-q2yz>_{!1^yoY;DfX@IhlsXE`&36BV*n_-U_p_ zwf0PzAk)$;nuifuH&3Vf3nRmPew+pd``A1~nBeefLy#GMU&)&4>lKMrVPFbZeYNUf zfSVJKQWFQB4#wHdX=)K)y+#<2(bbzI0tk!*E9=|J>>Mfg&DG}BRoL^u+?W(MgYznqG}1~$N{JKrt?4tQ5L%To%s-sWnKOk%tx02p!Do!#`K;kkr-Tg? zo%g#pXjg%+mPRlbFEtPztN?N)XhUB_{TNN~swP$NsIXIerJzyUa4w`m0Me<@kU@`* zRpFHZ9RN>^4`EEd{oJw9M1VtuNd}m>E`S7|aE)&@NxlRb0;3EhW^Q%acC=fpw|T`pkIOy`{wnwsg^_lD%exw7661k%Ip>BYL4u*+9$8`WrY#PEzQUa z>{BM-(d^Ks&vR`Jl<+%W!H<|fb36j`&>qqU0jHYGOe2c#D}DF60H=KJ`neX~7l1nEmk09_XbMEU#_`xk zf2xgg?QO?#xNg22YXDWHC@|PceN=H7IE3<34Uixu7ewK}a_Iaj{ zi23Du=UnjTnn&LRd3^62RkL5u+vHK7k1c-jPsbV;$1jX0YhSHf&lkz8cXUi(YnuT={aZQdbChMg?2B{c zGksS4jP-JkydQwiPc?;_MKy!p!&PP&y_~;o5*dl_YAP2IrR6lxhhIBNg)_ipTkjyjg{N5W_QFbG00Z${$XzQ%p#|n3kwY=D^5kJE zuT9|!b}ubF{09>LK1rj^2kE)4iqzlC!5Ls!jjw+_JV8HR-9~2S!A)SE7A!7DJmc5z z-by>GrS$Hb@1}?M?x2;dO2b&;TEQr9piuh$U;Y_dIAlKa-PX~HrJ*vt{dfOw%B?>5 zIGwG|rc2!|X{5gk1WdtlG(45DATVr3YZ;(3h`1i)d5n-~?7x(b@yl6L69UseLD1A4 z;w-;0M?@%QK!kb$;o45FuALleN`&mGsW#LIa4NP)y}<(Pdo(iJs2-Qus5ry3sJCbF ziK}E0c>aC-aTX??g3v+9wL}C~)>DS4KfQeOWt`|M(ijPUhX-2Im%j2EwMeg~nVH%2 z&U_vD*%KnxkJu|Fg-DcnY~`9>IPzxU-`ywqg_(t z#>BI^SE=24BIrN>8KtK~;sNH{LqLU^`Dz5uF^>MT4*H~P$=t7%|!^e4zO6p*TCSGTkpJ?_EwjpMx74n zJsct??%;!ksd4b~7gFQs%c*H#jND}nXn`721Ata1`UZGVfwAe=SVIMk3u9L}z06`C zB`AbK?R9L@$CN{}3{N#ln_DsMj*VRoVKt8rR}KyH_T=N#3%I|4@0tFEYb3_i4=7{) zFe-e|Ab{|*f+eN#JGavP_kKwUIkNj>f~==ZiU*#00vgaUGfw??XoKHP6j{SMS7SnK zW8dWP4Xi%iA(Gp&RLliQs@DPT9vHy*X^9%T&1h{)%-h2F{q+3Rt7y{7=@&o!QI?(s z8mwYp3e*BzauBEhP*x$F72Y{a+{@gXFLrZa8N{?qB5BIPH4c-Sh7&DGjS%bzOiGQN zQe#goL0N5ck&_?uSi#|Gn{0MxJ+ptWu2NK=to1D`98F#^qW zFQ(rAE^3YZVxHnCZ*N2b<1S`}kIh1V7Rc>&&YH zG4iD$#4<3YMhXB#+W{`vd-jDDrG03OImyw2KB0se_SRO1YWH1j3Sxw$NGHVw; zXEdQ&HLq2xr~hFDa4{E}rua1O&%h(&g+^;RaJkl+z|Hff_R09bm5X~_Zkv(giqZHb9^ z`}$lX#=RA|xA1h@MWe`g{f!4b^ILU{9)tkcI_1)UV~?NsvC5>45O)3jPdYpES&W5g zV}H;#bLo5twnF1Z+a`GrZD4duu0zT1dQl_Eg#qZCOXt{qQzJm)g8=(nf1Z{_O}Ym%`RhIh*zvpf2r?tn z0g4n$I2OPN`yB3@+S=1U{jdK8ZX~y=)ac%+*9gLLi2wQ$qgR)!+Cd&pLCwlR_NyR)g7BN(t^kCIhm z__809dlpPkO@B@ZVniN3An0-*?VEWt8;flr@22kGP7gl(hqOI+hk&m;X=m!Abh7j~ z_2-Y$&7Qh+u@jA3f$>u})1>U|Qmj4f2?*aqb_?!Z7pMkxK#*D$y!x1AzCZr^Umzzr zEOQvY`P{`&3jLq|@_$ahdi&=nrxO1DFQ?!8)*qnED5W>P|7U3hfA^{an+jth32;}^ zMa)liqa#F4ZgCplC1=Sl9S;_o)&M9~XlCernnAO3_4>6`hpBRRbrk}jq!6F&t>YgC zU>F>|kgCaT(us-l+6%9cYitpui^&e*eFy+qS)3)b_YNkbiPQoB8p5|~Ve$!)Cas}) z(792i-PP+?abA3YigYGD#<_Kz5aKRMI2B0?(t=+NcBwRaVGz?1kt(mflxn{6A}UuB z@Ij#4Xb{`mS5nhus2$CfN&wBBtpK>~ErcuDnt^_t#PNSAAyD>clPKfVJ2HZ1ibzs^ zZ@@Ock05Kr-^zu91M$pCB4ImOltwgSXh0g9(OT7$M3q{p0LZpZP8fx%G%-avES`7D z&MlDWuNp_pi#K0Qds9>34D#Fcw5L5b#{tZ^0At9(X!3$P7>NE2J8aP1r*o;dZvcU& zHWH!7MzOtG`qc!OF5+jjUAc(=oc3|D&%=bA-#yw~W4_#-i}=S?0p`vCz?wUcNj+h; zhMff>4O!Esrf$-xpb}=Cw-KcKEp<_*Yn*&v`hHab*kuT(Jo5mMQPn*X^>Rxjn$7W? zea6_1o&E?W*nt+Y8DKw0JIj+3snA^}@ZmyQ2J9ZNCTB=X(?3*HLsNj9Q|1%8B;^M| zBlsZ}Anr!`GPn6SG+Jt9OHl3^Yh@kRI<#-ao`E#?@DZBcQkntOHhPd)C)${wIfU{W zA=l*t(#OyjQn-e|UgVIlx-geIF#DFsvRwj%6}k5Px86uC*U$)3*VZ%u9c>+eIVx_j z*3+zq%)={U5~#&epkU&kxJJ;Z&@!61%2YA{0`u3I zIz`QxCZ9aQ2{9XM7hp`msx;@NQ1+!}MsO^UQ=2PLV5=jbO<`o36~;v$%q!<%!g!>A6jf+y;sZaUkJ67CH3N;-mmCNE^T3(VCxS zo#%&hz;_-B;yV7$)>#d&pdF2;2dJ25H1*u-I)vuTPwAVV`MqmvqG%6j?sJoE>xY@c zN|N=1{xPpHpY$fHv;d89jS;A!WZ(cj{^@$8^^#`lsfOTuxhAi?_SN(Vp1K1D@sQ&j zZ*#r#V*r*k=RsKwoCg?FSebj%iY>Z1_&puZ_7LA2DAM0Q49|WadF8%2psZ#qxjM0m zq%Eo^6v=8(OdVJ}Os6$DJ(osM2&`bGdpbKya{Z;$J9I5vDN6KdVfHCicxJHx!sEd` zIrcu`WvSReyZ}lDEk2ed+-LpFFROc)euB+d`B-&mAL|x5r8e+4-vFzWai5u8eUj!B z*SG-<434HjvV0ia_WkdDpK%?6r>>-tOV4o#$dQKZtu#A5PM+S&QGb=Zv{|H;AS_Di z;A7ST&}wZbcNM0|Jm6#tL0zOe!`cGT8+3Sb;t`C0F0=zWyS9_uu8hxF9jE+4)d}Xz zyVMkGM59tfaA1puur<+0QRdsKLetSgdj2Bn&Z-lH1FF#yj7yD28B?r;a*W3A2u5Cy z&{G?-!M2bq@3`iX|1ydQxoapjTa!AQ$T5D8o%{P+C8Ms z*|+}ypthaXF}s~zyvRo3M9mIZTScu(t~SsZ34@18+pwRC0H!lcXF0TXr!Z|#krrt` zu6p1Hck@1NFR7?SiR23hvfdKX-``u(i zDELYC%N&|D-+9idbwPNSSxd`^4PZ{agEm6{1kC}?|JnLVI>G5Ohi^b51al0--vN|u zH`YeUr_sS-%$#hlI#S5UmJpzGIQrlC`nS{JqmNVRA!$o6pYO5hs#~c;N;c~iQqRoI z%>!b)quB7=qkE|jhSdg*EkU3MfZ-AhyS2XujVAO0FnfyD=jGAMkz(Y*hrfni&}aZc z*P*o?*1Vex*}H(C78qV0hCGLxc?439 zbTG%9MvoEv1=RBb;69?U*C)rB6LR^~wV@U4fN3an5FTvW255&fEIdST!7tE=DLrag zC%(qVkz~%4MjN8^iWm;pYNNvE(Agd9cujQTgUH5npSBC>4dOR zN(pu{A*F!u9)+$^!X^YY@wlMSx6y9(oaEIyX_8&%a2CLkaoZ=n9N3%q*a$e-6AGdh z_Tl{8Y|NjU3b14dxzLi)-w4E^A8M-02xg;SGp$$*<@0|s(4uxkO+E|FTY(v+yZoIY zS!*|dhIGM8!R)*!M9)EwpX*;H2vhsynEjp^1t3W)hCOlc17s^zoe#B~nuyB?6-&RV z4OQ5e*)M{!m`5AtM~MV|H!?N&JX;P)p=1)3O7I=wJ9~@|hq;dNqxWCO-W^ z4=L-X)x!03tX6q2!8Uq6@5t5~Eu4D+7MZOjJeI5Z9y;;)(Kq%+tRwd_CV{@DM7VdP zgRqb1Kv(*(j&cM+(29|ij>R>#-;PCaTSLU+5qr#jc&{c}Y0tJI5C9+@39enkSchy6 zozreaN%Ni8JI7JNj5YNEg0G-kc!s8N+YY+rTd(zHL})DlIzFe5_SgHu*OB=$FCh=6 zZ~RcB$NKXxJd(VrPbR2XGYkzVL&@U6;oZyxFMRPesGM4%0JwgF1_XT`=8Rb0rixMt zQ>p9VK|t3X0iEs!$6yC}^S(Z?eDA(clW)I0NbYdJFN0;82tvdNnzk0!h%hC?0oo%| zUsb}#7B@KD($8xzy^Rihz0yykNSct+l1FFx9~WI`CUB)0aG;wZQK^C~t~b zFeHtWuyQG*b(1id1gnk#Ylm0|2wFQJaC4pP^IK*3B#MT6cOMbedVv4lQ2OrgeiPsH zo%GK8Z>FbYW-!R|<`=&j<9zb?Gb#uzq=BLS^w0jAf1diWz8xYjayu-(RRZ)As=-cM z!2FgcDuODkX@F#ob*QbIIhB~xIyimc7llTEymchZBjGB!&pc(%&Mr~Pi{x|wqzZg?Rsk2Mp!MA| z0~^#fq1j=vYJ5IsHY4md(WKNeLk~XuRazlZ^>pr0>cKhi0;b?9ICUG^8iODneeyBE z53_0E@e{Os(`kVy5eZKpEtz`}t&4#SRs5~FwG*}N0nFOum|1b`T5@fXo`S_VM9Aqt zDluB5Sa5Nq2?wxr$Y<6>%#l9gWFNmK=0QLj(Yb82U3`fu@$aj~T=f)nb-9J&>S)C( zaKzN1_fRAcb3Q^~t)s54z+t=$6JO(6YyZi(w`s%3n>ra9z~E?)h(^YG3Q<|(bOWdO zojhjafzforvv#oE=g`iX2BsO}tzf~S{8Wix%aNL+Jd2iWpA#^f=MZ33i^_fz09J&N zZsQnwgekp55qtf|4A|I*z_%JOvtR#08u?GYnYID8HE66_@a=l>mFs{UK+GYDo=Jw> zhvR*kf0Cv@d^;Tz72Da079{)yCa9FLMf>EyL4|zA7_DnN@%T~HvEA9g9ISvr&AmJK zB(2R%k^81C>Vq2WaKQIq#}DiE@ee*^}3uE`-d^U$c=OVQGZpdltRRbbXPpr8yG(hs7V$mD&(;2QyB zo=i_LwRNLOTYNf4u!Y6iiB#_CCrWY$KMw#J8a^X0vE@S#jRci&N`)2_U;+U5$r0-< zQDZ0+vn z0D}1y9--Pk&mrxQe0;4W*bJv^_OaR!lVNfRtRF$c@5eANg<5vn=R4*>pQ3ol*Jb1| zdxf{(RWLsxbP%8(Fcb<3`{NKWnuXMO(T>-1Z)gO#KR_W1rC>j0?#%(GhR2>LIM0Q+ z_u4N3kb>W2qx$e=J>hEFQX{9CQ~{-_QqW*O1VXkKQIgDIJlp<2BaXqB0#t$#Xem4d zF}?*PMkEUUGUxW}`WQc#3Lb3FxA#TlAiq0Tp42_8?P1XfQ!LjiG$Sh0XK0MPRZy;s}Xb0TdRe8-^aSH(!S|O1fm7x`z-Dk`i^yA zwb4aM8fUYm{tIIfxG+1rkXi^d5Xc!2qaW)gE<~2eit)r=qkqr#x-KMUe^{EG;V`!u z^kY9bvmqN<19O zQ(h9+qs@H|#HkIlED^W+E$QBJb$YT(NzQUtS}IfOvxXERV=u#NN79?`yqmuN?|zV$ zSE##+)z0Dx6}0;ZYvuO(QW_lSPT%>?cTztJqhtE0IM0=^%Wad&qr^ga+K{i+01E11 z^zHoYAp?9DJ~p)o-&0IcJ)BP3P#4w_?O5QuRu*M{bse)2i;RHxeW*DS5V8sLSDW+% zlhq9VVB_=4_}JlSjxAZgjUCKJhZXe+7dNyl2suxe_^_Pt-7y-l#?A|v8PR-zT(!YB zk~YD*nATzKA&j<W2^3r*lLnkJ)FO^R$ZDa#tZJ`HFSn=ofK(GLSCT6+gm1NBlZ)@w-qKE!_n)7WEE zPZMRg2jOeh-KI@d)|T;5>Lr5Xx#LD+Qpo>$!0nt$=@Mr*lPUd4p@f*=9@Yg z=F0p`Izn4kkLIm|_KqeeQzM1x+m1I<1?t;sRJko|U^Rgoj0KU;X$2oCK`cS2|9-0N8zGQlBi(xAXXzs5>C0Vhsh56M<1F9CVSo|=X=UP3 z+FxHymnjOI7hD4bjxnRRqV4(j(KdHq&7MZ-jX>%|vRfm0J| z!vG6AFr)(*hY?EH`cflI$y{|COAAa8rZ0q#V9L2RIH3w{Tmxf1R`?SMTV9+_XMm}C znEnaO#dSVLW7)*XJ^Z+2{4kk}nP7GRY~`8%yi5}?WK@)Z#QMUJQvmcs7{IQ6ee`W_ zbp@^H7Uf&|(!;wrwGzZ2(1k0;nhIKEj5W}XOjBNO)K!?)SR=JuSl)?F#s3sPn^O)X z3XxiX?;gGrZKSR#&rhXhuC>0Yeqc?l9nc^eX51Y%U?O?ObZKN5lkOJ4ewuo`x6|76 zG`X87=OuLG`!Xl*bD?6MHRp<9@j`GZ@Mu4@DD2xzKq*X^bFR_(-Aj!;+yM~lSg&vp zJo)!|j;Szl#fd%>7-};jLe;)2NId1zl75|P0F_P(e>HN#FE%XCAUPf+WUXGv3V;&Sxp@rZ(!L-8W*+hhZ_c|XJ zS}oR2054dHHu*9@4*)~3WWrH3YQBBAl`ZV6;K75V5xmV(-GNmrV})SUaf zW^HNPY5@VxJlxPHT|aEkxzRK(I0)^P?eYu(;yD17E=>091H4xN5q)EB&r^}Og7U)v z<+S0L)J&QhMnARy*4*S+&gVyP=sbB~aIJlwbD*E5^C)1H9({gjwdkkU#bJQ4`qP*F zi8lE?G-L6TeW$M&-L^gH*8V#eKGXw&&rmb#`g=e-Rto!yYqDR@Y4f~Y=RICD=lly8 z!v{6yM2okE4cUh6+7I6zY@LIQrnAF~4)R&Qd5-fS-(bFd36kBH z(ywnV3~IehIi&2A_H9>dk+kPiV%*TD$$IN?6Iqy<4F7inxn`y&sqH-jjas3=fJe*9 zVTnW5=J?r!(j-d3Nu$yZnlzOzer6z*n*B!1k|wq{&)dl8g)};L8D9GAKtg|xQKTGf zEfU~|VhZrqU#ubzC}z$Ed|%-Vl{~kTL;5DxC*`kuOP}FmG-eg}M%FQxyQ@p-jI?M4 z4*7-Cz0`ry@Zvx-iF*mI?4Vp@GnzN#<{pRS4$5aOEJA;~xPY9spa9~)eb}6CO_$P# z^HhmwypSrou)OrYO3n3)sk!?S;O$15B?Z)-2M^ON`~vqm2$>q{2$}9dZV$_0N!J*7 zL5X0;TOT1OcgtyYX*a$5#s?{P>4h()L;Sf`N_@Dpk588VCYm>3YD_{4r-U{zOu93HmJIVz1wyNB_(GZoS6_x5mk^?W@ej^RxP&kZF-Z4O;9#lI! z{wS@@P7wqE6TvaNgJ{xCOn?;-i>8NKOqN4eZ<1=G179YhX7Cx=!@2($!p@TyD^7-V z<`jZ5y#L4w6#!sO#4=aaz7kz~2T?8I7mD!Y`v!pTPzRa$v6U|X0Bdo|REu@%&ZlXh zr!Sap4PdQ~&0Pct^$;pNfS=zsK2xTlI06J{^9KosKfv`s6Ra3$(I;A{4S57T>p`4{Q-Uw<#<*YVdgQQ`JtI$D?ltg*Qr3+?QX%dW(U zL`@Pp6~=^fDeG!N*kjGK*!mYv0GxeS(%{#BJM~|_#OZ`!m+{Ax?aJYMc9uSQ@14}i zF=&L1n+N#ceg9YZrJ44HNWuBJ)B=zoBQQ}r_ekFRPU;&Q!=JF2{_R^Iq`p^QPk69K z2=zX6>6#0q@@)2Q%(I6A0~n;triHn?F+7<1YcVU$Jf`^Y9FDM^Y4pp#P0&SWS_k|a z9P{YmXQ_nAyo;1K1XiJEBCKrou`-JEY%6pvI{i4b=%H zyULhvPC0=IPAkzYo&b!i0f=P)M_G+0^ig2l6=I59106JV3KT10d^Iq%Q>p~Cqxs7- zXL5nOBOtNCMy*cL9v^?h4lkuC4 z|HR)p+3sOhFGG*g#10_6G&jqHMN`;JqH`0Qz2d$MT7*aU9F9NW@%>Ah_ z@Dy+UJcWTNW&B&sQ#WH)gRQwr0bE3TE|bTJhzx6xxCq=fFg>MwQcX=t{r4EDet1u&tOgKKG|%ke0GB#x0+>>1VEWjD>w^ z-<+fD&3rRI&YfV=HPi<*TMOr!mE_uU@Frl3XE=?lqpxfo*s$?peD((#GcmbiGewLv zs7&%@KfK4!oMZbQXqrcEfO&&isTsLM&HZOSF0{r)lj;5~u0oDo^ElM>$fBn_;_`zfNP(8HO z#&ZFU(Ci+82;Dv1p=FMWEiAq%pO_9JOXq_g58%6I&6s@n0H6uK;~?v~yg)^#_Ii|3 zW$H`!Hc`5gH(D$$XYC|ok2GNTN^?b6EFMnHQyJ(eU7_A~3*(>r?9)`58&B0(q0EPS zsjrp5gAQ;8L4qigYH|D6uOhIs&<9o-Oh5Vu0vb+PORQiOEu<~W4B_d&R?;i4zLviI zkN+fn{dd2Mj2}*KzxPS{s~`L|L1&nh|IM#INE8}NwfGE`nY?vWbFxF{euD4WGKjmxfGazOQ@No7t<(^n%?g#Z9R07*naROwgmy@%EdEsW+649<)APZ_CZnXX-!CtHL1oJ*+(tyUXMp#d#J z4^E4%s7+gQs9D#MO+J}K)k@J`z@mOsj~+k7{|tvp9QI3qR<%}#EHdmKf@YI(T2okBE6|iI zW#-99#uc9HreDLHxQlPtCI#fJz96%zX2b2n5c4p(N=$V6LbWrVcKX$de_A!y?N3br zz%W_it5m}#H1)zFU{(MUd(e|9LUe-&)JE3*48~Ad-H_&pT3W8de-3r8<(aV360$u< zF0>^QJYy_GQ?|RZOtNCC5inQlGt=n{&r{lpAdU{^;s`L17AV;Tp$=ho->N~ioGYX$ zst=S*RW?BWkJ8l6nWBt6iccOB?Chb63+Tv|dDmJIuPOQQF2&;^Kv&)B-r)#owVBu*rm@ z5#Ga}LH`O4F9XBmPb zYK1s)hY1blECW^G7>JSY zav)%1t}Tso!uqZ+JWVfO>P_p@cLGSZaN6Z4oi1Iz5&~QSV??0>UB`P*+2^3ds7o2J z+8@m{yO>?AnxGKQKml_T)WzgLZ?>g&HMARo77ix|fWJNF!Ot|4oC|PGUq<6b?M3!O zn5Jk;A>|(O+Snt{gj5C;G~c4_1dywh;!@@;0|l;C2DZe%1Pm~@YBUr^3f|C2unz)& z(*L-ggM*2LO^WT__ZcB%d4{hHa5zGS$A4-CLl84pK3gH9a4Vp#&7LnH;INE)h#EMB zq!FX~&#GkcB|uW(TS{=7{jP8o1bZ#B7;7TvvwWRJz!j(hBSD*BPeB~Maf~-AOgt0R z_FG}@PeDL*Fk6-o?g4V7Uw}<$6?vBCcm;m|HrkJQaz6Ra@qv^9LzyG@L$k#50^Bjr<~sDbf=EB}SyHUvD#lK`9vnhT$yb7VX~;3~ zOU$zmaSeTzpj{fUK51w^d8YfuKHGL2DxfVN6XOy@v)9h&8MjMjng*%dA8x+bf3f?;(jA_YvI?L>6II_9%ISI^z2Y!+p#zNnSn^>!uCz% zZU61B;5YV$&yc40%;z}wzCD~Bz`HF4Y2U5Fl0$3PiV{NYTP=IwOJDt3x_A2yhjjd+ z!3u4p7;7ow%c>#;EtH9y*O4QObFDy1a`*Aa z@57%bQq9pOF_5^d)Psk}8MncGb;zY66_--+LaOP#L57C@^wHfV{M3h22LTT|l`ZMB z4XkLGP?ru#gx)@cjHkx?eX>*_o1aeHi_8)4z4a5uyd4w1HQA&L?=m(Z~60OmH>T$MUSVg_4ZT|x6|wViS4dX{>dWCH?N2V}2 z1+kx@Rjb;!N`nzHFff=wHH>i;4dTX@In8vaM1#pGx(EQg2$*VEo=o-I^Qnr!hXdB{ zB3jc8w4|!F&j87qh`Z1bPJZ-Js>d{L@%l3T9bYM>hud(Nu8TzD(w_cYFbMOf#YvC; zS7VauV*}rM^9@YiYmrR1mN{)ELL;|MJ~st25vScGkp1#^{)i-r=yXYdeA?ENIxu5n z8w^b+)Q$!M2l&4JRNLMb`OIYKhqQUTvP5!eK*S~|y@^L?N@3_|cF#J-QWcuQRT32M zG4H2qb7>9r{q8!aC{F6tTvv;BM4?^6->3is=xQSp6`GFdKxkh-j5-2tdY^wG_1yT4 zbXqw@^B4IzHW zo>P^ULxXh6+L$fhZ_0TJI1_}l!&p7|sKqIu2{qAn6Hd4%v}J_w9s=A{8=4Qg@CPX^ zk|XJuWYhS@o*}Sov!$y&DoXf3T8^|pkXQ)A5UkA0shpkH5j3N)u0gwN#&d82*C|vm zDZ`uvFJ%Ih%!Refp)mjnKsL-ojLI6iIfP)O$-sY)-G`81i~vW4A-`n8lDX1{M**!* zpr$*2*QZGV6M_Uf%t@2%V}-MO$Tsxl$x=DM-`L-pq4V75dMV_YP^Blpkl-NZK+$AM z1ce~-$VuS3ET^FY&V^Hc=|gMuna4!&g*9NUn+i#UFg{|fbQAEUIamOfX%#boA&6BI zC(v{{1rq|1Jv3~B#ux)W$bt~}#1VXH8(w)tgj+pAFmrm>M96dLg}zpnS`usqnBhLP zPp*&PFtj@y*nL<9ISaEOj_fdEANLThrW*2jCa5-uL~tv9&>G=hsbG!yFa$acLPtJZ zAmRE5!kGw&z&Qtl#5^)+9{K-~_1?{qU1^@)NhH$3d+)0XWm!eC>6&bIPkS>ntC3b_ zV-+Fu1@xyri4=-Zgft;(HQG_k?6{tpooUZz(-p~Lc@#{+d!IlefrQcXoLkkq5z1~5 zsLY%99{aw(^Lw3sf|}k0Laq_o%lM@6e~u|r=&$$D{Fs0qnIwUS^ZX(j8D`lp>l>hhUZ`YDrCybd_RS$lsvDvLn$}TSX1>qxRf;X5nXo!3(siS zJj#bBlZzaV*+bN2UpvZ%$E(L-a%Df%qQee34C}jm!eMl6ekv@@P10@LJXlu@Wf&N} zOhpIEWkEWbxLDU0S?8-)uZQ80OIRvPU>WuTd(Sa-F6;5RDA!8dmd-VHwb0pg%=6r+K4PK!T(u1VbykZ1&L;5-IOtTGw2hgRLbH z0$;^7n8`ER?ETWv+O9teJ_O@K)c|g; z(Z}rzOyS+Ip2PjD7Y;YkJjLcl<3*s=1gHnDFxN0+DCJ|o@kz0X5ZLapkDxOsMg?TJ zyJ>vtQJ&}Qvn>DCchrEU9mcQ>3!VjpwE|unV3=SAEe`fC_qr zu5+vK*Gv}uvQP#l-c^ChkXe4q??)dfzGuoEnY<$FF@;%KWv=N|a1l*4wi$`1kBx`I znE#A5rD;oZpFX0Ixuj>&F`V}gqK^&-D77Ka5E!6d0Y3l}4>bTH0{{|ppUP2I%S5sX z6R94O2@-t9AO_yo6iI`kFpY|gUeo2IUjk0X!b63?K;@`XjIs!j@3_2lcvtZX3RLbY z*TgIayJ&xT-^)YfbGI+kvv!c>vyjAR)ht0L7zmelD_(+(b1Z;R80iUKd{iD2R z1S538G4O_I!2tJ;&tRMchboFV4jdK)XD|K8oIe3VtS!oO{a#`^Wv-cX$HM-&E(CVK zhuB`3Y}+LO5|9hV;v7exv6lcg3%i2lA+B3N3Ooudviw}t%g@>Iadfa(-9l2KRy^ORkfxB(c zEL$M>idD@X<`97Xay>+UHRpx(?qN?WQ2{o%2uX3qTJIVj3nL`dufsBO!2CX&d`3`T zK1Sj%E-!>Dw_gj*=;gt&QLGHz)MtNA`i`O4H!ID^*clz{S{=V}BmD4>|2Vw<`rGIr zbjA5`*j#uLR%h;`-!DQ>Bm0=c@hIlb2K?bOSU)Q4mrX2o708DHl9ZRWNXvCxL;l#S z$CX$Q_+YhX~a4Mhr9siG|39~T?rSB!{H#^4nU(;{Pa?oefEs= zBo!QR$*JT?wthkFZ_-lL@OKIqG3RQJ!~Gc~j+G+nA+<8NAdOtR6~6x6Ka9zOF*q@o ztU<=tT%7x~F^z0o;{d<1vcR~b<2eA`dGn3%{`bEf-g^J*;nwT786Tc=hEmCqMxq|& zZmIE&<5ZZ~D1{FwIKOmAxOZzO&iq#~uOjIYP;rrP^If9X4%mEkm<$a|Z>BU<2OS-Q z)wO$gvd7RmsX3arz% z$pX{ws0z@@Goe=jGsm-!!s5rj3>kdMYVZ}hNs?A5xUe`0Jqp{_hT=cr`le=V<=m^5 zcGR&VvQWr9oJcGt-`_U`$iQJ8CHrh)0h$>z8fLu=49Ec!dE@#`7RfB>9?H~P1I)2u zIHxkvDo@PzU*U2+Ur4(dW$*v?GoPMPL(H z0BFo3^fdQ_hlm*NAO{e67IFMGZ(RoS_4e`;nA>c4^ZoAs&92WOxo<)W)AP!` z--Ye@nb6zakKoOPnI})e=O29lxP)!tNr&8zMFv>o)XNw*v5quB=ID2pli(%^I-9QC z3P)6ZkR6O=Y3O%7YvX)tJJh3KI~lu6xDA->vKfZ+%Lfx-X_JU*PKLc>6~79_GIhrQY0X}Oj-72?Eda9| zk}CLF>Y^Z1Ae-i?)F1?vp$f`p3jhT{uK7BA`?3JxP0VMtoNlx1S+)qNk2*+7O%MwI z(7#FL?RPc(`ad>71S|gKOGOmTYFsD?6Yzk@qcXP5JU+G~76P{oC_DZ93|fgMU1U@Q z6o5=Wi$I08M>7`pR5{9=M}H#rZeqSlfST6`m?DeFz5U&UDOA8H=!?L>=K*pBnS$B~ zplPE(z;=eV9O>qZ{t9do5EcPoRM~Wtb?nPF#9?|dZYq1h zSptL;rS6z%U6Ij@U#B0w=XJiEcVCWQ6YNTCC-=?b1L`~r7HD8p_(mN@Uoe?KZl<|W z6R`&r=iM>&8A-qiwyc~1K!2Tx8B4dI-g>9CJi>r_S z2u*gu;2j4fK!1lU^1);7PUvZzrzuce9 zA?wuC8@5ki>3V!QF9K}8=XYEa(G7w&I|jBrGPbx#q_Hj~k)YHNMi)h2LY{3N(ju0f z-jR`zL4JFzyZPqV!>jLoD|Dkf3dqaVh56{`YVNg}nP-?yv3O8UstyaoCi?5|{@wqB zT)-WM{oIRe?c>L`Y~aGvh+Zfmv-i>CkH7jN{O+HB8WtwL2p0#d;hV2r3*8OqspZKq zN`AK*EO1+ZCu^}5AO-dyAFf=z87AjfNNaTze)yw54nydWt*OW1VCKs(-i!q$hgBp` zz$BK%J;;vz8j=hT-V9A+Z-m9eeE9J5+3?`2nXo)X$|`~ns^Fj^(f*lsa?0Wlx4H`{ zfPAgzu)jqPIqUy6;a@*GhL1PXkwXW+`rdcK_^WS|YnGsjE3YtCm}0fMq*$ZuWy>8d z>)*#7>E*EBPl~efTi5V(>!GYy2eJl_FKVB+cNy$N5W+d7h|V!{oU=$zn9Ngzo``w| z;3Cg(afn?SHYvlzn~ZO9eK)k@a9#oUY-5tMbd;!GL6#-&mI-CYNDkW7MDch);O5R0 z@RE&!qLN|RLcm;X%%fNd9!MXD`PJv)DJQMDxy3LtcA4}Imx&`MFO!TLA{k zWWc{5)x{CY)VhxXpzX(BgaRjq915N|4HQCC*g?}8`S;rBrSR~pd*p}f!q*Hl**+QJ z5iHm%P)ubaDUDh=4(pQ>EN~o*0WB8(wU%p^1#LuOD?SyT4mZ|T$@C0M1fu~S0&Q<% zO1ePNwBa1S#NPxTus8?WNKXO)`r7SVur-6>mmmH%tN`G?fr)Vrmh=J!aGwRJPB%Hy~7OTZ7JN%q{Mf^es!RO(JS&+dd%e!NfCyI;c&X zre6S>9+c?X`dmoOKMX^Z)5@{6Y?PHYK8t4p5Uws*V_NV_#`0juHFT&;5 zZicTOKMC*r(SH#>_{;w{Ozf?PURXOL^Nb`Dz#0)K_*W2^$$kr#v^X|EQsb`B!RhB* z_G_C3|Kv+@(G7(9(d*HdXAU!R9X@MkeCIm;0QxYrz}P*0{E!HvO&mY3kXvpojzb0C zpu@RoYTDI@mAQvuiGzY*`3a0w(+O1xnBybXaAP|sCDwQg!lV^Jq>tFr+B_+8FuB4g z@8R3lT2}~N1ij=_fc%<60y7Zen5HYm9vET-E{B6%HgK7-+ydAgm!a%ArLDqb73pKl zh|YLxxj9y7t9&?ZmsvZfoUk&O5-s<#LBzI7;M$FAJOSZ}4`~B)G*3AF$fa>WbBO~7 z(Y3fYEG!^Ap6`S=zwy1W#+aF#(daGd_! zg+MBWcGhx*ypc5sm=osJ#JZ+HQ32=lhgG4YCWu0E24}AMd&N`Q{rEjMTwdDzjtt$1k!7aC1VcTRLA<( zZ%dX;KN=mon>f_TE@@hBCe2V(U?>K`qDoAYWdYN^?1Rc*aC1ak1T0qRFfWwAP7ol= z7ZoJy+X#98wpxHHR^KO;8$IOx1h6G6B1{VZi-~ywnSuvFLS#9RpAiV~+?R^Cy`v*W zm#aj5C4ePij${t41$j)H4q5kcU(q+mf*bJf_!WL9f5-@xS)-y^zQPGV0m9x>HYkoK zhpNmmfd%Xn=dgwYnNPMoeqZPC3gh5h$fkM-5Q+#)&081QJ^EoDIL9giRmO?gil1#F z|3)s2qEHD!L%!)BZ0<`no2`syOJzH(N{06NT<1-|lVWTgAFE{?n&_Ifm|~nGuyRer z@uQuLf%_q*Hlgo=WY>?t*H;8(Tc9~cD5&O=+U6`&<^QUU&_T}7pFssG` zh5(o!_E7}5JUa%vFbP_hpGlrP=|94S}WP#Mj<_FScpw>EkdrIe|rM4{R_NgXqe8 zsqonc9}pqB6UHy&H%uj|(LTtUG68u6m|eXzfLuHX&zIIi2RSP%xDIj5iCG1H_RHVK z`F{Jo*RkH*4=YdalQjXC5r>jm;sw~dlQYMb3$U5frxY&){CBPB?xT3la{Unpz-r7oD(f+7=_W^pnSc00doSZSq8Cdl}lKXFR zQ1I}P1AK3TDcT5n92)#yBw}>C(8AN**B8l$9W837&Bz|ES3Z*$uC|pwj#vQEF_?vT zOd12^Su`8eGP$(|8qm1fHteo|w?h_b3lnd?s5-81VFT(Tq}jtwt4Uwxnf*=>1N9@B z=s_lEm;JL2XgWbb!8wHne6;cuZ5_tKfoi0J!)na=w!T^e|Sn511^Gjv|Bk z`HTgd!h~m;t`!t;Gc4IL6JFpXx(eIXkMEiOC)@bEl>|bxwaTVB1881ANvE30)K9Lr zZUE>GC80VC)E7l@7E!QSYqsI5wZFL>0g?%2OE`J5J7J{=*Cq|f8TR|5@=ElnLV95C zB(iL63W17pE|TAk(@2&G*h6Zb*7kH#S)wa+UAsY$$wqkm#TO`<YG8>W;#T##j z#ozrR4AH)AQf*kQUjGn<`VnK&$UN%AUBO@H^G{b`SlADVyRqE3@Rt z=Csm(ISgT1S7@&iUEBvSnP1xqw?|+uw>ew@HZ`Ag^g)ReqHZDd0#tPhfPQmilsTiS z1Hf=kK~x#TjE}Hk?Hci8L~sEUu$yEbb9!s8Ei$#IoR|(^q*hRnO_)yW$|Rz__jxp* z7cmcKF255pef^xIU=vWnC6uSpOeRo1;DnbFG_X0(NHAK1W;3POfsbP&8Z&taU3Ue}Y+9x78BGm66}(L?g03bjXT; zrPdG>{R8-SGEl&N!v3i3?;nU_Uit(hL61VGTgv%+3inW<``5`U@a zP(mi~J<<|Dp2${|PgE#;9;3lA_o{HK9DaCG93N$kdRK)dsAelg1t?<^wIrVlCo};m z&-{KZED?n9J@28COw5r223bl$p5{9s0M7+?*dhMI zFP`sroIvj(Xo(7yeIWpd4&@3~E|1`V?9J3HOsZrV%fIqdF6Ikg*^n-Y(Ey zev%~-|M?&P$2Al83GH`nm~}ybXMY^y=!eLh`Apl-GK_5#l-kwU7W!xkka)mw?wA+f z7f_lE{uo{5zkNke#@}i5aAd8KIjFG+s?W}KD)PR`{&u|xV14P|th z1Go!8aMZ)_-+1#)A}t5_Hse6w4YD(!A%bc8l4CBm2wrqgmOLB-jH2}of}soDbA~*C zs1GL3uU59hAkm=>WSK|-EEyQ|FtQijP!E7In9%p9Ca1&m`MEHD@Zib7I;3_%BQOyM0q_2FIgG?jx^ZcY8@CWck zRRzsdpGN)-fBf*%eb9`tpsMUSQd=C$q2NC{v?@_G(J$s#n`5qnY`U`;+w zXDtO+>oCh%r2QDllSV0f6y9WT4F-49=@jYWH^V#n5r%_kD&Cj*F*QRQ4uNMp>pAvg znTGVBvS)4GgYksk)!WAar^6!Q!c9=KjnJZ7TN@-H zB?^=0nUy)Md4Ogi4Hy`ZA?Xq{haIAHiu&O(FcpIssx4ult0_EKI*J9+)4hXX^2z4` z>Vn)b?L>``s^a+^rY%nK4F0Rw+*f87Lo+7DMU-b716l!u)$^Q9lqm6@VH^`vEnq7T zs@DI=5*{Zg=tX?X$ng*^kB`CVVSdBpRiKo|8{c}5NE%W;(7qJPs-e9rOi}yQbP_6B zssB9~o>Lq*(>$k+zAa-)b8b%DgiK&5)eyR1$x`IkJH-^0%@?CTSS`-!ov=8yOoC%+ zKARH&aTU{X9ZbwA;|dQ+$t^PUlZf^dW;AtxFvavG=qupwbVQrWYcrUpS)e>G%gO2h zU%%rG0R<^cNb^E|7!o>j+F08T8z)c0*qu9J_?26j`liCp%nW8n=H1f@N=PPB0cg&# zn5KsNr3gxxo|p=+ymliD>U3AUhuzvIAV*5BL{iowI$;77LC1KkZkZp1&XX42Z51o;5y z+k2^fOS@{?>Y_7h<@qAMB7y-{mZi5n6pk#cG}`*$YbmDh+L@fe$Jz!Kq*;I?VGO zm7Wo>vJyrFGfGh**)u_kZ59BkSp8lcCjnGc{>WGEABE+Q1A++< z`nJ;xZHG*mAli25b7d536*IcP!)K`IqyHLxahM}mi1$R{!Z!P6yp99N1!E=SqS-D* zmU+K-Ok1M-M`mPBQC^sp{I{U!9Jd17;498KOrD_D3;effe+2b{PZ>kOhrVSV;Czk> z+u!1>6Ap_qk^1_SNrj^c(D{;qvtFzB^tt}n&j=Xeb6IaM+TeS>yr+Oni&FISVw|ha zCu1f^bIjFMzGAx>1Hql9U?aF4l*r1lN1StCvY_Yag9vWoS6S=y(e=XvnPcXT`o$5< zM-b?>$=I5#R+cxKc}4Ahz)#|X=r|;6FHTT=+W-GQwn7%qwmFWL_S1@U&Ud3}-nP*1 z*xna?6R5>!(MJ31p&*(;X|E}1eAyMhrw?xgxP13N066EChwyT4_*!Jh&dwkr5 zbO#oZgIzLNJbDTaxd{LAkA4(0Bh29ImOt7IoshppfSU(|8vNA`Ih~!6h_@5}?@#VN z4JVTv{&{{q2jK?vj}>hsU6!!;HG>&44PB5o?BNSsQN&BEojm)nuYUk$jsqU@K=-ed za)C9PL5{d)JPf&ROn=tS;h3)gRJX!3Y~bz=Wf#E|ZxNY0e*HSGEFdHqHD3Dc)<8n6 zE-d4FHxWyWk$sIp0aO+6UD_#=!obao;&&1?SJt9nEX`w@h$Bw*`xNrFMrt=g#rxv= z7V1!7DHeZ;2UjQnYo!AtR!wu^Y3&SOo-|5KuvO*0H?I!isMtkXgy&&xMZYVMFVEfG zS*MIuU0i7T3pFwcC4mV(NQM@d0FiIp83@bs6DT2^VlnBR0f4%QSQ#GbWg!$WyDpJZ zGJ$xt4rgS=w|4#cJZd+i%>a`1<|ObLmz%>lk5xiMmA@p93_rJi7{j zXhR8`dA}CWkY{7}a`GrLal`GM;qm>y4#T%zkHER6ttE`V^L}{p~ z1(zA8LYis|3-|HCBM03{iFpJ#ql^?HmK{3Aj3vu=4*Ri(+3oVR+u>wpA?&lLiU_^J z#W<|tXxKyP{_gXS0pz)G=jsUlgGOmmhq81M?tb`bc>Db~!_{}+3m<&-Fx(&@ql%C} zBiEh|`FY2Md9A_-YRarh#5{5DqbPeaz4aF%mEK#`O+jH~6WF4*iX_+#!<5#p3Qat4TOvj!SDM6?uJ z5gH}Jup5YkHXI$s4Zs5N3bQyGR!N7GBPX0yCF*iWixRa$ISVu){1&!}e#YOfg>~V< zCJh)q0#t2IPs4-}NJ4$v3Vk~T=pC}o8v#9sfQ)8*FwqXNuJCo%+8SB9Go3>yW0ET~ zZ~9lRFJY!UAQb%whHL&QjN(N2C1t-}y>%7f*hW67LajzGdk`90ji_1pVxDk(p(Vvk%0Yda67OJ1Y?l_m4u&mnR zLB~Sc7gz>zD2QnKVhCg&VtFBf9s2=+OSfbf$-AkaR7g|`BRK&;9*1C?+- zmx+*J1GL4l;%^>S%l9J#=lltR)D0fMc%Jj+XZmGoWr*Mt5Sd1viy{qRNq{@d3OYn^ zc!S^7S(5h+Ig#dl#@knsHCt0}bTQzGagD{@am8swkC32&4zn(3izd?O zgH z#;jEwnLL0D+lan%1Ntko##BWRESM>v`=Z|>^I#eeb9Z{^5%3#*cu4NcfX}(?u5EO_ z95d(6>k{zfYU0IvMqKMQAYc+`Id`)79;jjk62J#{;og!Xv_nFGE(m(hV}0A?-!WgF zn-a^6=IB!dn70?0x-VJE>{$p9#>a!U`^v*cYzY(X^Nl{fAmxxIPwSn=_04*eR49M} zYH>8lVyn%f8!cC9q_CM7ZoKkZn4NygaB(ezHwGZHf0oiJQNpT%jz7e8C=b>s7$XMUELjkK4d6!I5#_q53$xAy2$u6G~?^d0quid z{3_hKdO7^$yYGdMe))^AMlfS{10V}BEW>(F0g9^ZO-lWQ#!k|Z)V76Be)Bojrmb*k zcp#kDby63e>Oi!2mwoW5jm4!nhxq=w`WzN*+=N!1~tf0dm-PVNH-+e#) z-QWCuIC?^EuEs5teL5^W{TyfV=J46iHV8NvVRB$Bph*V+6s1KR{PCl?gr8a~V|c_w zr7?$6T`5L{p0HSJIFa??bSOPN1aNC?Au5GUH2H*#%WQ^oSgvn=^WE^(v*}nSs~^Yt z&mKI$#Eb2m1ijlV_B@lPzern;!i3-qOn0XxHa%l11?y=xivqIbM%&<}P@1lWX-s3y zuxLeu#yP^tqPm(oYB^0Dp`;rSQqtKvgPy~nl}Y8$h(ucgxE&*`i|xJP)9LvzcIj$Z zUS0@|uuNb6`+pa1{^&mo*T4OJ0Q@@Oit;7|2c$?*(agd-B=txhg?;a%`{CBD>*3Wu z{G;&T-~JzA6yeqZi)6{K@&-O42*f>D#ReG@Hmm|rBH8g`Y{1@hgMFezew-_tLEs9)fm42r2Ru3aMX^x;I9 z1!N44d?S20bsuvk8BYkX*~HwL!6g9eByFgHeC4#pW-&53)|6!(^C#@q3LuAizof9q zof8g@z_QXvg?-XQkjuvAT4Xr{2#q8ep1XfH^e`q0H*@FJGJk0V6TniNHw&Fo3FhS zKK{+mlgKSX>0f*O^)U1JDaoq=^8j%ph3nc)EX^8$&8TI4bReAeJ(!?$juHAN_&0SA zUJX0D2q%PkCv3w72ev9}ftd)s1S36KSf+#w$(irG62ZwFzHqHQ1LPar#5oyd0`OZQ zuwv}it0;j)5y@6rfcXO9y|=j?{gpJ=D6nKe{KLrJEk4sv%}Nf|FD+tr*hkl)GG z$Mly&4<-t8?47eG%RJBWd7i+HmiFay1RT-yh=!9j@xJyC-AU_FEws<~Wc37AK8qFm z`|Dz6eD-Bz@wm=wl!-=FMn5R7kBkQ65q*jTemvi#yj`T{QBg-`i@jq`#xkJZb~NWH z!cA_)T=W^#zg0gwSah7fw|L)n%f!mM`E0?l%$^`cGpo+|Do&Zs6bHMQ{sC|h6*%K+ zpDKW8O}%xDX#_zi^T?vHpQ68vSe~E4 z=F$hFg{uHqL6Ku<9}F8YK-n@r#tdjdU zh5{1rXPX==zw0^?R7LiQad1BTpU-d)=tnKvC6U3hiSw^7sIOY)(bO`&oOACP0i2Wc z;)U8ohIqYW5*HvVP??2nA#lf$snPn|WYQvnOk2bUu7W&YTLP4E-_rIrKpFpRbU_^^#Vc68reT`4k&hb!H!d0* z8z#>y?h-_oZY@3u?T|b3(~prOv=~2a{b`4gq4St-*Or#UrLocQ!|%Tze)Eri6E^?z<$HAzD+7e&^KkeiHA$b zW?dRI37^vz$Ir1ogm5u0-Uj;GfTY!>)d>EMU@0$S^>PY#cUHp~IaO=*bIUTxKp)UG zQKJBZUYyhw4b9M$OE3cqmsiU={R6<=Qe@mV0cmygr@~QW1D^s5{&IffT;MLy_X!4Z zt4uOs3edvP)MFAX9LUc95W#g4_yySm@>;-)1d}1X@T*n*;SYXfF~f;NS_^m zWEVbxp8N}>L`mgXh_I4ba)q>a5b})kaZ6_#$p^QIOr@%T@ z0}v<-8LdMoFz%Ccv($TS#<88^xr}oPCa9AWp3yvQ+%pHj+TVutf$BIx&^AyutBeVI z;W?)h7RkuKK&ZpFt!JJhACHk|K8C!Fh^--hr=>(BL zoPxWdbIWYvft#<9UZNP5p`TkX6P1XP9%Y`M-Th72Kyci6^AExdX*A0GtrAcz?BJVr z03|+j`F40VvjX^gCA{@t{x{*Pzy7mO1~iXgKF7%-o=6muTSOijj^#q5ak)`1Lnb!R3%PUT?3csKZlsbMS>s*@ZfJNb)4h;T4?JB#gnIQZyBZn za7}K*SjV>=c9$R+D?w~g1fB&GgH*oy@K>Rg!@vc!e)*8SR*N8@(uKZ^#h5f2Hik9Z zSmY4IK77tPX>2BX3WhGn9^F_^Bgi_!gO7fVAa*aaW;`vQn!^#HjDjl>Fk{d}iuH7g z;5;J{^9qTR_lo#5vUbk0n5h8_$0`C8ULBxfm;jwTN;eCz&>TK^c^p%IE9^7CgK^VW z!*yu+J453I2AXO7DbtQU1abvH78A!JoWMqr(Xo8cIft7@fZP&p0-YRIS{a+NOg%zR zHs@3yRMv?~Q^lltGEe(sElr*iovN8<{q2lUufnKlX02!cBo6cjCE+YjBv_8It8A@? zi&k6=$O?Z}-o-DjE-XyVazN`v8LKaZ%wgXp&%olPMSk zL7D<1V2EQtpOK*|O`lQ5)T&v{x%aLKNKC+h3R%`ec1Gn;W&PLzW{~g8o?zPNP+ftc zR=GRsvPF?CwS|Bn*@5ywQgI<+YsfptI7NVkytO}$q5br`QeOfDL8oIDD^YMDHn_w9 zumnsJkqodg>hlnPYJq~FE|xH(g^?KoBpi{NCypTx5VSe=X8RWqD|cglF8bih;1OMT z&SCsxzzpLpFcQ=X&NB4F=;uAhihjl$^F8N8P?@K%Dq8~q1ZDga-%o%mpS#xeXSM$V zc)?Tzg0wO54`c5zCRuZm`QD2;c24b^FW>SzwnwwQK?l*D#QX7~2b^TZB-bbZb*=c5 zynL=d>Jp#v;xVjG^`)QDE&e0eb{>*LliyV~0y%hwZQ%d$`+nj&64+L07k7x@-@}0G zE+)Ko?*fcu<*F$UPMX5mC$NB;J7wc!VNdCkz{q*`npS+!j*#>F+Z)J764^t1$o6Ye zmQinRz^8d?Isy{&LAq}1(UV;qjN1r4(7}Hb@TNbUITO3O+rtQ{x@ys9hdYaGvcoVx zOEPt6qZ&Y16YfjJVqz_|`{=zaD^RSL>AbhlOXo0C}hgH)rERCS9D;7f=K?p@C{Y|py3o^2t0NLW0$c`9F zsCe?Oof4f29z!A2a#AHP-e3>PiK+zL)6+T+b2q=WHI=+QKSPvTTY=Z+(mK=UiCiRIp?J zSv-eauL;fZ!ypkEIl=S=(K1bJz&60`38oEg_?rYvoI!854Gl-rYAtkh+t_&c-XH!6 z<&@^b22=Nw>6LI;-x;ox>!y?^f^wH+#`qYd z@eQ!@0b)J^xD;!m3zNt(x#5;5F)~clsLt^hm^oX>$CKKb4XqWb9<0uUa{@JTv{e9W zK$O1)cZ6`-6E*Q{)1&NgYB@l_rV!Tth6DfrKmbWZK~yw4^*193cGw_Ap4Hco4a1}j z*;uAc)RBAiXpSVsnTe@qQGv+N zb-_3uz<3+#zC{p+!7C>y6MZ)g9%%xA)d3dzNZxn>6LUc+yCxXg-jS<-chW4-S5u$t zkTa+m@M6@d>a4&a%lu)2i!vvu7kyuTYbeVXYV5lY(u2dqzpw^JYleE<{n`?=;jUh6!>F+w-W++R3PtaHDYfUcKg=f8baGIq|XFEa~h zhWB|9tiNc(f6U@}FV~FiP5>KHpEVorWE=R~_g{j&2>9ra0WOI@X0jh`lKqU(lJEJl z{fRq(^Di^)Si9~#Kvp0wqTq$z+fQFBHj>%k83%u2HsYOH1?{S;GOxwO@xWnSw}Z@9ZN56iI!N zuT5kJ&^_Y-mX%G3w~FGcgHbqT({Apgz%c)v z>yN|6x!w*lMD*YM&;J6u%L70ItO`Ou$AbLe-~TNRil^j@sSC4@KgO@-Fx(s;0pPRv zP;{FMOQFg_%%TtwZzwKV9oPq$j0(BWE?~R#vB_f&>LrTAoLA<~yZOp(*gFCW;$}z$ z@8}r$#8_xL`leB6U87^vStEZ9*X#o7ENp9?(H!8n5hb6)Pv{W8!NFeYrS^^>bO3Qs zv#TU#t}+(K_$$ut;$whP+opX7D6c9TPG621gi;Q{Fmd;8$Ozc*{W>JsvZc68UOe;V z-Ox{@>}YQfzA>cz0PYq>E{CuG;3we%zrfjF{Zlxdn+$zApaO0OuD=o*dIw?ip0I{2 zY@LSHI)ND_T>8!T!$o_4IKlibtA(hHz{8}awSW{)-7=9nyq1WtDWXK`0WK{#*|%Ui zX~RFUieFgm<~-Fj4DrCkM`Z9h;OT%(SvkeeivWZZaK5YS5q0A3nw*mEaJZ`wvjSj9B!0+lI|O#MiA2hSfYh(=eIo!E52P|5_-F zzZMEGHv1fKE=a_ylWc~dfEG3P7+E865gi28>ovh$c!_@x_G1D8Xnx zp}nRdkOdLksH`k(ejE=uFjVXptEd!YnUFaiq;w+oJDTtH1=a6O-!54`fv#pJL7S$z z*p8^2QC74qMqJ8Zq3F?X@ki(S2nOh51TBn(3OfN)^%9xpR!Xrwp9y$ycJl858?J`ywb{j!ds2E93aZG35Di zo$$+OV{3N`fZ}*FX9@U8#@FXCFv|eH~rl4)C|fmE>NnyXcPa8euI9~&-_ok-tl&P0Ob5HGO)bk zT8nHjmnY-qcepZs*JnF6J`edHffDWK<^FgHX7vx%MCx3`HNgDEetRL`^|~|HpPk_i zMvOIVmIIhMbhd3bHO2-e#|HBKY5L#5bp?)@KGuXAuifD{GMaU>wz$Z?tVBQNQ_SJI zSQu#4-PeQJ^g0RcNsq+A%V^&lufB%$is~#J=nuEZy~ZK_NVbc^nU##H`1v+t9m&_V zgeMR2vt#~#@SXR=$G`adur_f&4B*~VL`Lu8Q(bAj8dkG?p;FfmC_>dw&0)F0hZ@~j zActleonzoZJs72m-|Bg;Ioy8xyW!H;-o>r#B;0%O6ca0c(-0#b%(Xg5x~RtuFx$eX z%`@i+K=n5L)n&l6RKr7q09kzJAZfHZt}QO0XE{`FkQScjHR~<|@ze-lTf|(NHHH9O z64x4Qy+~g*(^_Tb2tsD`%JncheuF9wS3=EWObjN(z0@~AE|fwT?QF(0NrWGDBexE> z!_(zCm<=d>rtKT!mq-rF;z7xuV;Ynibs=dYk$IS0SfF+(ei0~w6B<=+?hK3g9!yaW z>{;n7Jl(5=i5>F2l`y^FSZ;ZrD$bWP_@X?)nQ;kUq;o(xR_)PFs1ONRf3y~E zcjUtxJ&hrUS*lQ43v)#4HRE&9h5~M8z)O4N>q7C=k}qqJ0UN+~ZD;K{OwSR*ZU=f3 za}ef}T?Wn#sJScp>Y#P1m=8K(%^H@UhWyjtg}PP10?M)zw!!+93lpCc!8sL9mY#)1 zQfv(Mb%p=r$3MhO%p`*to7tGVD~kjetVRE~R+vL0PV$)CS}=XG7dRPE-iyW6fY4ea z5TO}6`FDTv6CBgGX-`eqO~+uG=)==GR^)#RV)Io7gB}Pbo%E z-ol6P2_|A`h60VPohEIO_ z*Ws7{_x~0eXlqw{Yk2M5Z$)SRr;i_mE&7{-vDyPzt-*R-e(MiHiR7|tjQxg46Xpo1 z5W%dcNNbO#P^z1=bAZysjCaA|;TWH$JWj4PB$w^M@jkmcNuD>p%BE`uK&=S)T6*vs zgbD2E4aNijJ2Rj)31stn$<9$YX-T1S=lm;3__#p+D-N&4h^^pq+oDS`y|tb!31b+rCLq{!#F|pV|Pq^_Y!K0b|tp9|Z&{!4rlB}FB z6`a6WWhj_XOGeWu3Q)NvYvcH;WL1PlvI@FnZv-Ak0Q9Ih1qRIP@0Bpjm8_oE7GRB& zC|DRG)|lZINRq@?kU`sd2|}VCNAbq?2n@O29~H5n zOYj)~&AolwzIyKhLfmz%4X@|^>;~>K&pcItEpaMR(FA*$p)944t zIJ-X*lbg(oHL>wE;;@ncVc1^g&haytBDzC3rhH$35g8*|Em(>ETM*fr-E*&KrP*H5mItf9;<&KO>vz^{l@ws^P>rX6_x| z2)ey3FAoG|EEoc$1VjS>0k6q8C)YSO%pUioFFZTWfNSN&8c~<}zn8BI_JG$IWEg+N z|FQ<@zhKZd*e}iYzn=^4rTlscJiUSDcQ4d8Gg)t}Z~pE7=o)_`vsjxE1oOVvoVzYK zV0-wm@6N4Y!-#+Pi0j#;y`~@2>}Hb zz?T^xSk0zASkpTmwo=p_zo^3!L;Y$F z;dxx7G8g#5a(LU}?6H@w4|6B^Fu8jg)(BGSx&A6HETbXUJrJG(z#6Fr(TF@5ymk!& z=@P6O7D(jA9)7jVXz0M-{gqc=L7pvyzyI663lF}$7k>Jee;NMbPyaN0`I}#b5C8G+ zsB_)|J{iF$nQ~zq>X{s@hWfITG6h+d!L=vLA$)8cSE$7~g6(>F)?j2^J4o){GtdVK zMrtcu#}?*S2=eI}qVmF4c(%LQ!gY3CkFu#MA!!p3R|256X=G7Cbl0LeR`I@=F7!${^h z+_~IGstni(0u8QTxf*`^4}XQ(5~YZ_Yx=AEG5tsfpvBTdCg$z*zAWZEg2%jXCn(-C z;W##RbB*b%qwiWBAlYamg>`3lFN%msPWe=i^4e^CrGWX+XpQ8}p2@)O@|iyjd_^aZYeBcpD){ zTP}#U%+{$45IhKiL!3oVVdC1TZQIU?q=R-0b(7bIdYCz~Ki9x$l^E+jHsw3-ew*mO zjqu6e|7BQv^hJyk9RURNj$FdYau+-PbS%L&O*HFgUpx#~zxCa4(Ki?l*nFF?725c> zI1E$}0Gj^l5lG7fPqcGVG54EAjCWlJw4+nda6oqSt=Uk&MWitfsa^P3)KNEe7h#-+ z(ykIUxyUBiMu=9}7&RnjJfY3{ULLVYpw5WE-KOHeVR-iC??|=6;o=$TJ2*6Kak!WP zpcc8Go3jS+nQH0@E97ZgtKfUd8a`zlYZ%ueVDkdTa(8hGB}DEm+Svx}J@fQw6n;5C zda=Eeh}JR;F+d3=`r54mHNd#uN+w_hc>@3+D?zw=!V957XYLD#du*pp#B z;Qs_=B4ePh3u=}_0Hy(|qSP|KdD?F(p~46(E^ACSyr~7?gEBo}{LlC)uy%VOc1{qa zr#q{dB2gTG5hLML%KD)RW_c*60U^(KI{>Eo2dS)7Bq|I`xK)faV?2$@(oAXu-5LiO zq!$W-;?vi!BG3R;tVeS|2~KQ#8u_q*|DW}Ojb4lgM-+}q+C;nEq$ZesKE<)W75Lq!BY_oaKpcvm55k^xJpPA-F zMHX2_K$`%=mjFxPB72|`RYM9E1VWl(^?7ovltD6$vRDyN3woKWSj3zAyT4^q1aP** zw#N}a4A>~uY zJhq#Axn7ba$&MfQ=RUd(&=oI?d&PbWCY^X8pFqx+{g3bYPyS9)!tkE=KtH^EhWGV4 z_kioio7)e63x=IPe+!yrMja!8sV~RHN85kbhS5x~ee zj^pHS1p*IDn&XQwVBLT)OKcecrjOb=tUXPO^Xl4lFLlAb?ZAe8`r${UmYO7y@4M)7 zK-%onu)911+m*}V`t3WR=7SDgKZIe+Uf?rV zyGsfZlGUDj3ed5dO!Z}uv7@Ad$)WttVI|tzT0y!*V**mx-%K=ulixcm<|?!(D?8>k zGh(w0ojE!%L^8dbQ8=7oUM{ha_Fz!903}%jLxI6mKpd6ukJ#rFh7BGq#5tc0fS@to zl#b*%0yJu42tScYD3htlDdr5J!lEvtAQ2^u9YUSHH(33E#b#*{gGJJy_6^PPsek0vNK zZawolyRyJF>zK5*@$qB)w{ScKTtTzKc-4k0Fjp;m3n7jDeeuDQump|Vc;j}s_3ai0S*-_J21xr zRgxm|z<85rW+gR9`W_#8oElEh;`Q*#X`bj7U>IDJGV6F(eYgUuanZgjIJNrQ6qp*r8iuj~SL5WmsgO@ajQdu!y>X9RK)P`tXaNzS9o zoEE7k4AYl_(Ys*dmx(IQV_Iw@7lfonpMIa9SLB+9U|r`*S~R80thLJE@s zdv%KwUJmBgls`2b5>in0gYUctZPgbhw>VG$!ZcaNQdwMMnYj!VL)PzqLeR?r`?5AM zOJUNmy2dFJSK?%yV@^JTowd%aJ|3D`&kQm^H^^R35Vv$wVN**jme0pC0v}?4e58XJ-H_?qxxQz{~3ebVn>unKtJtme)c? zx^EN5clsOe5p^kRSY|`!NM?xZ=#UCUprbYuq{w9K!jkGzkjDqegUd^1D%Js2@iMm& zF44}c0EPFA=2K5Q9=shNXvzmu7ezB)Y`O1q5kM*dN;H)T(GSpI2FKSv2^v-6j70P) zjLBjBD(^W04o+lGxwrjJ1}g!E5qL0GXSgiT8swrbF-df=<0+sqa@K`l?OLmJm`2BD zMS#pR1R-?r1%UB7NCe(DiveGx?e0MRuo!bdt{pZFih@!9HOI?l`jz+|pT~Ja)_4c| zD#K(%tKcT_+u{oSF1-eD;QFev76inuS=;8k=a3~4aFY@#f+g2X1)b(7L7MS zdYo@YM7BNYx|-z0i_d+gW9B?vAj7x~W5@i`K2~GwUySah9dT~?EY2f0Hi5Od0<|u9 z;MZ?kU1l;{*Qk3l=lIe`+2nf6e0$pmVSN!xg!=%3XXx@6`gWg##!i@i{EY9w_!FF9 z1g3|{HY`yMfRjzw&2_AIM&{mr^$lE=&H!$6VSj~M7i_Cuu`&TQ5j62p#+FXzP(Z5hrs0VswoS)UOW~`$s#_AmzA&~NV7{Wa$OMj}& z$rirhjRd_lk|wGi+1&vHc=eU*$a;M02t=&0{DX9JnT1ou7bb=C{1L*W1_k8eYJhc6 zd6a2o4nQHJ&_XVg1{k;^31FLGX8JKtHnSnk$CblxDb0;py`X!35U%!6@qC6qn7@Z_ z-U|QvfA}9mcHk;0FHXZOl^(WPP$vw^zP@E_kP5y}dvx%K&FN%W%--6i1s2cko0rK! zHV_I#&9;%g;tFiU^}1AOJ0?xS?8C4z^_Y$cj1V9UOeX`)DyCf|EQbAi?e^=D;gdmg zL1u^|Y$B`k1`BeU2oy7tpCCv}3(sNxmST}~)-rx$XGF1XqM!h$OPmUx+@~NqK#x3I z4S=#TVZJ&Y$_x-##holohYlFju6D9FBOupd<96{a zJO@0iLs{nt_Si<~901_<5K^9cj{0oh1MS}daMDhIYL<0D0BAVI9C-|-efX4g z7BE`vrKQkmz0~Qa;qkNSkj8X%{p;TemjH8HiwohwuYXRtvF*@<3A;c(x??ta{m^h2 zfAj6|w-2`OdxNpae%g?+Q6Q6v5q zS(wxjGH0L8KjIK!=se%$eTCvFboeftxvmo*hZOzKH*<1nL}3#F&ARLaL>C#m8i4#I z1V=53d=uumj=GA%>96@b@{4?I}^ZX-y85!V1I6I3iAU2F$miu zz{@%N=iq>RaGro+J2_fKv(`EZe`{c!3)S7we1<tEM?qg5Cb$$# znSmBDlR9gvhTKcm#=0P>4Bf3n>He(u-ZqlO+ z2Y>@ivSkiyDqbUA8`-1<4v-mrPZ@_!l4DyvB}I4<~sfxaRPem8az4Yz(FYcowd^E>-C;w&+l^p_<#8k%C zU|wE=GM`10kWVj6z^pw1iGbDdwg2{8c1yXGus1x1Hu&=X`WOnr^-C0}LUb1Vp z-w~9-i6(m567Na-3dk1CaalS(fy_k5s*c93=Xpzk(=H>ivH_WR!t zBbTp+_I?iJV3MwZKFU3E&}2Z{0c>5p{n0P58TqLDhrWLIKKl|sI^0zBi4hIfqL_9$ zIj?hC??!mn!unL1+!lnZ2{}1~gn7VASASpF!0Ek!uoe%Ij7RTX5wR@}S$QTX6DzX<>7zxvmq@5-&Pdq#v9PJ>l8%Mo;A zp2c(X$|X8Ljxe6N1uJvG&RJgH4l~$&Klto(gbBVu- z+tEpX@|avKZDo1{h;tJ1?c;d8h%eVt=)>G%WNAMM`1V*79b?0xd8iw1?-=`Z3%VC` zEfF#LcdgG(;Ja2ET5xPvf!1~eAU9BvQ}*u0YIugv*dYq{5Fn@z%YlFn1!tOzTIQw|de)5m^C;jw z1YQT=1L7(i!M^TN-Fw1Gp%}*A`bK!?2S4T%wL*%NnXoYVEZn#;9!4%*2~#VucsJiPctunS*Po2ze9pJ?x(%BY#Jn>U&hUv^B zIBW3-EfVRJWsRB$^8{M_aAh{Kp&fmFn2~z`&V!-v+AHA+k+%A2jSi2ZfZ017DmGy6 z1_{1cSzL%pM&-B*5Hgj-A=i2!*n&ZghX4))^&Ae?CmxUrC4`~jzVOC7?~@Gj^Dr^- zBnsp-%Fx_fnx(Blt2=|~7|fq$#g1D18|yGdw*%xd9D-7~4w%wLC;#4ov9Q9)^RSKx zQ-YGFi5}LxYCwV!-e+7_z?ZPHI1wg){SToQ_l8FNvDO!t!}9Z4azw6%2M_MYh-<4n zOgx)NSXRa@!#-%?Bwb;hZ<51LJnGoWaA{7|j45L;L#|>#&qbvzgP`&?cqsxn7z-Ig zKi8bI1>0^SO^`w?Vb)J-oG$j>DyRAjfOZBmh78;V&uDC;hBGowg|WfjQ}!jG#Wp0& ziEW9>4rMDlCUA&+2)`pxV4u4dRm9N@$G;;G;CD2BvHy(#j!}R10lDUVyk65|{4Qha zcQijLD^$pqA}Yj_J4y$$t3eimFF{CTclfuggNj|Htqg47bdHM06vrB+{SlA~wrQ(C zLSJ0jpcuGhe|RaMBEVvu`S(k9(Sy1m!fW*t)|sD=?Pwf-9^*gg4^|DpgWuKh-QUiU z{fS@-Ip;mSUWFq;;5cb+jo{Gs@>wJ7MFd+Mu44o@qnGrJYw4S97f<^=0h9+C9k~UP z-qYv$KTVzfZSv}9Cg)%v;Fb0B<@`l|HNNloJKm0~^KQGGQ?JxrK>*r{MI*MIG2q+r z-mVD`mkEL{ zuZh=jDZjlYvW@)HM)^O-P~he(0o(q;8syq&`sH(@gOhtDpLwnAvEObS!ItYY1~f4) zvZ`9Uj#(4BY#bpAV!#7q=bEyQra$pOEHkcL(dSQp!)UVQp8J?U?RW3cFs4(gFJQ{F zIQuf-ZF^-U21d#Do8_TzXaJv8_A#=@e57sYt7eFxWn7}r|4SBz|19lmf>qrew>n|xCheh*%ME!=tUyP;#4 z8rSILEIxEO%%<7`bfwZSzO%2FC{$!G^0pnHI}aq$zi_`q{=DVzdMVHiIG}2?mhGbO=i-m}oLO$Y<8y?S-y}q@DB)i?AooL8* z6vPXU`sD-rJAmINo@jjLNfsL{9LsD6iav zrY*97&&f?h65yI+@|$hWAW%`DAo?>FsPtt4VAhIIZX=b*^d}#LG`?1E-oajtiKz-x z_|>O(!^ES+!3QrM!N1j3mmJmxYAFL8$DbymBn!Mk+5{ zg;^W9dNVvgnf~3+{vm?H9ok)w?;a~3Ayg)Kr8@%9dKQsvXbV9o9c)4!^;L-MSWY{$ zVSVapxJn6xn>{d_859MHVCNn_BH{>U5`p~AAN&v$OU|1|kHY64{f;(~1FwA~Jbb

      ~aJ zGrn2G?AQdjFKjG^0f5{U0DA}Wv4PW;!ex*3QoxUGiAolym>APg?#-ls(Xn=Q1K`aw zRH$v{?Snx&TqjV66B=+AK!V9_VLBWvKMz-7)J$#DND3XV-CF@*0g7Y|GBDJ~2;OtX zSb<-m%uAm2Tj9Ww$Gm9DlQvG~k3ap06Vn66ygodEWlQ0sIRmTt)gwxwaDvLfa_u7+ z?KgBdsqX;q2u*eh`z0TpRm%u~Z4NOf2-qTE$)w1qfYlU=N)wvVey_axCc=XQ41qyu zghhp8qE6Tfae<$HV+H&K&FpO;K|SAX*e86R6fL)JT?2&SSUmq6Mw0oQ1Y{C;!kIE; zdPc%NNwHG^>KyB#i@kc8wR-~}*&p1v90oYB=~7Vzl$iwC$^#W-$otDBe-DPc$m!e)5xWRt>jCcLw$L2?8XMV11f8I8)I(O^FmbF(RFI+b zwE5T=ex3!$kR7FuO~@!Cyf4^OJAk^FK8XI;@VTbD);3PHG77w8nl(!bZUk7eA)*Qc z2y_Zp!J;WpTs(*O`6Ge=St|X_GPpWKvl)Hfr!OkGL>XgcU_O*Djy=tf3R4ha6|^R4fmDTq zm&!c0J+@hfOV&rwm@q%GUIGN$Dln8~7Eo!Li}$Ae(IKApdY{Np(hmEna`!o|Edi0a z-RwsaWl8@4Wxx!^7*L=kg?5>kSx{)sF!zw9k7NKf?dv}o&69lZm_24+^c8ae1qiZk z2{<%@81|I8(=W^CF*$J_olhCG2x53o#?*tdWB(Go2-1ALOmXA;e(u=IsQD}t5j*0} zee5UqiIpjMRs^ZMFHlUL6Pp;ISwv-$JzGD$oaW*9GO_gMF49Xoqzi;3+DXzY#AVdc=EpYqaVl@-_g}U-@POn z-La6l<<72Eo*(ba^)aOi_jV8ZJ#&0IcdlLci|^Ora#3Zi_<0qb6RQwFZj?CqT2X}C zGH&z@eSP`LHPjfEJob1utd?eK>wa#pFHv{7o5L^b1X94H>)V4lI^H8aL0D9;h-nCTt} z+1@ME^S%;Jn#l!9YOGUai-(8%AOAM|^Pm4&_}O3m75Pv*2|ny4BDIO2f$?w)(xZbE zCHVqb6zGp~whZx+BFKypWsW#_;^Tiv&1Kho8gslqW7l5^!&h&Gp3!ltJ-v>8Y!3$j zKmFTGW?w;;Z3#pA3~WBq&qNqs{Ck z3|264FeL!B7R*~C?G2%~2}feglwGg_1(aGA$L$ut)&-`R5~0D@uZ@L~kwMHG(CaLW zV;19M?BVMyiWDd0ZPFdBolIV;yLf7;s;gi2=x2dD;4O^jZk>cRvfquvA&y#MG@z zqik$xv$tCc>$oV5Q7iNH?8yUSpm1Ly_}kdD2e4>+beY&)UxE_;KCC}~${Z0R#1|Y~ z7y_KooF!7ku|haCz|LSGwowv?2qOys$Fr(nMtP=T3e&8Yhog;+VvDmK?q4NvJ4Xx$ z1TVp&>tSYkJd92ahoK=9GuPkWT_q`^D{O%DWh~2pg#(n0nSS=$jMdcv1?L<4VezZq z;lA7qKlwd+wWUb3AQo-+*%Jaa>5Gaon+7bF_Brru^pF1fuW*wAG}!#7)ZJ;%Q*#Yv z_YE1D@BZOG4n#ME^$I=n7#ZGl-p7H}>R_Lk!EglnEUo8)%!Dsv6}zjsP=W*CNb$G8gR@q3}t=xYdZME3i-UWEK~R ztxK2I!{)=g5sa6Km&njxFVCLH@%Jn(8_r7f#-q%L&FHHD;1-luoohmaz(O|NV+xX1 zV(v6rPO;`LvmQnEV*+5NJ%9Pe?T{Ux3hmhcsE`fZG`i z>Cngsf|Np{Z00inm`YwCr4pAZQ6Z`5{F3QNb3K)@?SFt4ajvQcHe-RG`Z*MYE-&3* zGReCz?|ZPrx_Jc!fv_R37m|VQ_;sltbzlSm!j4< z92t$spr{ZTuX7h;nmB{#)uz(ptfKoOh_v3HF9o#)fMu|ZWs8d1vy4DjSPLQ|TZG(c zXS~jjz(K$%7!^#W0l_gI2;ifu%W=f92mly!ybouZj{-;t;NcbBn=hW5Ga)dxepduM zkzI>#<3E50oxeNVte6gYUM4Z1DziM9B@p}z{+lxKNn9wJ( zxjvhJ`e79DnYmrOj{DM=E=19lVG)ke{Ukt+J)-%$|BlQm*O6H?9KdH2q}dux*SB{t zuwIwC>t8`f*b1wIy*PRGa$MUZ0LZ~MED^wZZQ(g{bq;9)h{4#kMP+GclVZQ<>?|QE zl>4exD6G$3C=QK;49v0#2FJ<*?qx|K5DW=+mb}^xuiv~(5qPjdX-jm8yrtG#RvC}( zpxju4&?3Ek=d|22JfJ(<2k9{!uD@}I(jP?tEM>TOu6E>90$sDQwj9GuF48?DFN_3Q z0`dQmI6u~UY=U4yV@Obxt(0)k)hyHQxzM1<0LF7!LUUM44P_Z{J%?5HhozaESB+WA zs-z%;^lI9uUhjd1>zSSsFvA&psf?^YLeB0(l&vqWP$P9IJox&nu!5Ik599WPv+j9K zJbt_s?mb=!&$dv0*)hJFHK`gX3>mRgHkp|hHsl^6BjcPblnT~%1_d%sFR~ml7iMYM zV#;dm*l(~ExY=-(U=ET6h;>Q=!5Dh(Qg3T`c^JkFK-Y#FCQalCFsj!|;)ibLg-73q`3LvH60vm|KuVJMo-JBQk&zJ-7PQWfQRb(B zS^e!v0^d7WqcP%!uA{&&;WC{W8;dcBH8wgbE*3BpFdHw|1;Fb=faFos%g<-S>hfHS zpDG~ajT_nloG2pupuSI7_nHhz<_BY+@_(bdHNLOYOjiUw=%t zDdkYur^h=hu)JM>dD_kK_pUC=gOCB*Nji7~<}p1qY<3AO3oW9E*@Jjy{}W?Yfn_+{ z;A@+Df&w*?PNrT0lorA&fxNmbYOX2ZADUOF-_Vz4{;ridQdr6gC*_SpIg8bK^fO;f^p;v+_zjy z7n0BD^%-6qk9%C!$oUui_^giIYc^xgm}9QnjA5jV4m!l2BT(i&7DM*cWHfXoHo=?M z^Jm$+9$a^kiRFE9?zz7EL&nN^&aa- zK|yy!UGbv=xBu@IFULijAdq=rY+fU}gnzhR1YoR>hN;i)95x5hypmV+e=iIYpU;=i z=*wpl+{MrGa}!Vs>YBgvJ;vL)rp=g2KJU-SbX|r=$U(zBjES^=?Z7xOT%1RXd8g5T z^N)#1=Nf_?-NnwEY^M{GCqsUHWf89eoq&)5z5KfcgT_P!L%8&CSfTC6>^3jO@EUxjD)eishs z9)*>AUttI*!}Ro}@cK`F5?=q~|0J}HULncuR;Xh9AG3aT3$jL=fVK=Nq7~jjGC?~G zV|4p74;rKbb%8PejB&?ixU*34JOkQaMCqN#{$3y0<8{$Ps8>#2yV2(vDK7!uoSH1zrL{@V{VK|vHZdoioQf_ z(ghQ4kJXrIhr;ZJd0G6sUxmlN_*v)x#Pl$KL!kL7>bUf?c`TM|abf$TEzCZO*V;pA ze)PM~Vv8r;ywg`MMG!Q5_v`TC&;EUA5sa)Y5qne$pMCmkHj3`FecX}+d=sa*PrTeA z?8zx##wUILn-4+@^W2H`c#0b-!3M5jv2@{bI^5Zz05N4MSf4@Mv%{VB@baZ$VzsE6 z4P!dSbK4ZWox{*geLdSi$p~Gbz!v}>wu6#VCWbVW7q-H0|Ihy;jCR$+PygWM@YZWL zVN{7N!;&cxk5(i;@%R4V$J7ktV9{hc%icI8=rcQUBkUtA22rdF-+ddt3q6J*Gj!TkE*ar{43NPJ$70cJA*4zUIv`rT~{Fm`!CQQg|@C_F{ zdwyUheDWNiN*qcn*~R@BJZrd@?PuA*c(D5ZIa#ozb|JA+VV@XP9x%+2O!Vq|KMBLc zQ5y4cN-Ssx`?_;)DeOP`B9zw`!szs5ESPM&tt#<>-Nc~T=QM>ua6#~)``Fz9;O~x)eb&~)9KmbWZK~%?> zB^9%5p?k2E`&Cg?y5m*S{bU2{ezaF2mtoGT)Q+MSA$wO~5bJM6rGdV6!tL zxN#iZFeX9pH{$?pK$5@cnlW?$QViP|AICT?dm~sBQ3!@)H5;52-8X{H7bPUD3uip0 zz&^&{5z-I~f}^~65Uy{+8%hTJ&|q!aSUcyz-&y}o%h@qIe=2`26cwEF>htZkW{ zFu{fFk=nxVWHVa`h8KG{r=HiSFyr&H4$OUwv14xt^aMJ8Kj#8a_Y!AFx0hL?rqc;$ zxin+qC-wm!i@?MEh3ooO!1NUv$3ztmE|&=OF>#>b9q+B zVmF0I064##8^byh$T((@=MmWW?2MZ^VW&mV#}D!POktceYpwFDxF)hd{KUQ4AB-c7 z!N8^X*}W<-We)%bJpX0|#eEA{kzGE=SZpwVRbzvBZoy>?E%1_@7w29>AlAcV|C!X6 zfZc9IHW^=5XTN$AGt^_xQw&1GGXyqCbo)8-XBXzpY72+NnX-Xt^o}9LzBD#34}K`) zI=pmgikR6I&JN)xWTm%bBsEw+&&1HkFdb}QwTN*$=loUxiTzlsNfM`)AKeXmtH>H#mvN)$HOE}X< zzxWI06Ww?DI=!s{hMc)NanngcUG^bf%o<(AD{+j=U-sRq645Zxm~%7o<@#z+bRow& z@$j7?JF2wlREBi-fJ6F;$<@#u=p|;Ep6C!j8_Nrkfi&h-p#R-B--ORU`Z#>?;V0q2 z=ifr&%+X`Lk7UF_$N)Sm>I2ryqU>jk#Xj_7%pcf;QWG#e9_~GqbUVsC+dCX-%9Asy z(o+mJgX=`vkL?Z%wio7i3Hm3kb$~C!+A1&souENw)1@h5sSt)+O!x_b*jl1Du3e*_ z87u?{QVZCa9UHTcM2O#mgwP@~|(m^0q*v zBKa2Ft3>utxOX(VVzvlAe7Z`eB$T90u5$n|%P({=jy_nEmX`f6iw}K|RzicGE6U=d} zqr_g_VhxLA=H&Q2xntpcX+1ptn_q{%SMl$|oDB?eV4<$9Z6?FF!N#1x65E*l9wh=3 ziZO7&q3EK;*73n6b=X?Nr@#2~Ff{SshpSY){>#7odDy(@4nMl_YA9oUjIz<&5H|Zz z$zOc-d3bYXI&43Iwe0ThHK|qt7U~$0e9*xGY`_K$jpYahfQ_Xig%a8% zmLvzLi+v5L77F8IWSHWbM2LZJD6c~APD+a$ufT5m1_ZpUl`%lNVg{i9%?Mqm@MMBO z+OyIop#%Uu6wTF{t0cYbM8?ZLj>g!@HbmfvqODT(yNWUPd~cry&*{*|x$8pU>(;W~ z)diWbDwHFPf?kwGKMbQzU4hC5`VmVOpgxD$RXN9icz&<^l%WvxD$JYlR81Bs#o1Ho z3K9~mS^ON=RW-*rJY*ZzkC_KjBr7cdiTT2YhBe0R*g@j2*XOZU6xMy8~GubC_VS_-q0afw> zF>k+^R zCMTyS*pHoDgLA}l<_i!+>`&{eTGpzvL$ULAYLu?8hZA&A2FB~!^fcEd91Ad3BToDj zAlM4fbG({==K$pqV&SYZ;94D`<24a5KiFr2F+x)Cj>x8#zzfrOJ`T{gX|lFAcSt5g zCiF7iI^>Zoqw*$09Pkmw)d|_}MdYDwc3tzvf7TmzNzp*71Ew(&_?%^YWOV&|iuu+c zT75*9C!W=fwZYiFMbDk$fwB97Mz>7f0N1uUfh|py^Tz)QkaY#8s7m0MXGXcv)<#j> zn0>-Vv@Xe`^;JqI41rk7JT8bkOI#d`%^|K+X+-N~xq)phv9h%gCP0_tWCG@h^+|E? zk6DBxSdKJqF76Imtm0~vjUw=$;QwR(rr>Sk0DE9j&ta;+hfKT!9v)c@bL2yGlayf7 zau;%Rm+5#P14L53}ezMB~2oo&iy>==Z~mSA5e z@rxZE)FXg-jsU+P7|P;)Wflo>4l~7bF^^g9RmB~7|1bX{2unp)-qxAbm#z5S=;q5oyq%i4Yk~kV+d5oCY4)<&ws4UbXvVI>JVG$2bn;lE7XY&)g1QTE{0s%y2M9lU z{ygj<{LgrQPYUo)vPV~;k4_M{g}L#{!aNKZJ!+rPifL#v?2<^*J~+k3Cm3_(`;bqT z!f*cFzYjnC7yl~!gMZc?{_X$$ZzFISxcLtGE=RDDDF4QGc=BMD#_E-DjZ!0LFk^@8 zojM@b16Ba2?V~(1C@{`#=pyE9WsPFDBp+Ph64i1U8y$*2mq}S~V;(XHBM-Y4FWVMj z+8!2{S6FKlHNsEVr1cGVpFRjJ2%I)zBrRpq4V&0WYc>1h9_^C-h|m(?DoWb{(t7|1 zQ`tMyxB~cm4Qp03Zrw2&Joo0x%vjhE?3w;WJm)BdW(HwW@d){L0!LX_B-%fAQyk9c z9a_J^zN*5q%f^(Up0$=^G6nZ+1xU!W7*H%Q5Ip!f6-ERad{^kOUmXZP|kb9{=NSH36dS*&}=YerXQ zL~fi%E+9k1@ObGKRarK{mDjO!gI*9OJeKxjS9J}xFekx1bxA}4{eZ}|2Ys-pZ404Vm07Ox^C%lMneNkhtxi5rZFBn4B zo6q}faj!9#y5RkvZKWJT1iOajaG@qJ=lwim)2++(S>)(4XPg%D74UgBB9L@1I!BDv zHTUxo%%g{8-QwW*n|uAmp76O0`{5@3$TyiKfH30{0JdPv3n;eHZTSI`?wH@M7RzU8 z6p%ngylw^Kcy(?zx)s;R^e>|C`Y&BU4?K(Gb$)NZdXtc)WzJWb{-*T(#88jGkgbw4 zObd7FK3=XA^7)9`u9OCdNeN}-Odo9iC`?%!XX%WDvwaw~Dta*u%Vu(uiFvwRFJO_+ zFz_uyA_H&CbCUTmHr+&mQ&~aUB3_-kO+V^lm`6{|Z|&3DbOSxkUgg>9jCCEam7bnX zjCr&9+wrOm;5BiMl_ly6Tr2T=CwRcRrtbMV?-g*JLWCTELrfww#%-7Dne3s=>SJ!Y z@B~yV+ff%4iiD8>;>yIxcN1n(prx35U4zZ++D`Oq^pL z)Y%B9C{4s7HQ-Qk1}U;Qx*ZXpj8!_7K8FOG7jU^1sX5jlTa!D%?9gZ%RMgETIUo{# zbq5QXbl0em*tj)ZGj*)){=t!Gjpo^?TPQ))3hBTdutcO}^UyMDx+fG&4Stpd(CSjK z_rRi@;Z{xP>O%Q;^)k`01^b}=MHKBQ#Y}ra#aX^qceiORRSi#JPbP}_kb_w{MVSs` zZ8c!|Yz}_L0U3b$Z5S`bttsix;-=}WFdF6`u7rUt5-5&Ycsf7WHY7tf=mqo9hQPM0 zf|;cISRe6_0d$O2vPfkH1vyJ{!Wq8ue!ysh7Gg(a?Y0{TjsmtPV@>9h>r9Lc(cE?! zH#7x!0a(_ZntJJaxN`etn%bwr;sTjEu# z#9y^DN4n5>9M*%(J1FlMpUC>EgwCoz6nkZbpWznQ9kRK;MS*0x8ZcHZy#hdc4<&a7 z*wKZ8IEmSLECCykKB0ibss2G&(q%q_WyxXNUtbAJBo5SNO<;4zhNfww1*p##34AB; z8*sR_@jHt8-VSME;VO_p?SP;el+@i%KM7y|@)u#|-5-ZP`O|+9e(~483=>l`p=0Ki zu+O1w>)H(+ThGE4j8dI3z5mnqU@vRo@z>w5PmFH@kf215nHxjeRV-QDzt#2G&`aVA z@`t}O_bBad4xCC@<3Z~=3SvPULQZBzOHoDC#Xc)z{n?&PAd}=d`UeRsz!i87t9o9c zI|TD!E3P&yy#~n9La`=bv=k>e4!vQ2j+%8nBVpoJKCH|kzyMeU6!>|00}GrCQ?g`# z{qZ++Lg)=6S6>dTfW=h=-7$$EJQA@PG)Knb%mR#R0AhP>rnzT5i4nuTJ!RgF6&&mA z53d%oB>pVp?j*LXGeMk{5%8XN6 z&>fI8p|x*@mjYBq!1{Rh-W|Y|k|mS+%8y<5DyNcs>~@sZ;+K zu%H;cpyOqhiDUaz5& zm96sg0!V>l1dzy9)vyZFm^1$_Ky|#9f>541cN~E@CP73Kl*kqWS~1pJTM1d=GtIa# zt}7~x0D|Y_bN;MycP#`j)hmn@fHmp?NG^c6;Y^lI?4F;(>z z4O5pZshXOaB$0QWz(knR= zj=}#KTNObs`@y-jJCG$%nt;{$#c1Jg)+yze%mMp~!x9s5*h!Ir!V9D;Jl4u%D|7+8 zfGIG!hoLxp8J(O8r8V>zphR}bc3cMMrvS9x=Y;t0mu}zU9B&}cCwS}o6QysnO2CIJRwRY;$J)P zvK<1rj*dy9WAo{b?a>|=|#cIKu7d8(V;`BDy3SQIwho~-yNIG@|d$c-i9;bC~)8@jtoxIiu2 zWlg*uJRXK)DO=0uc=eF~v7{G!*K4M^kKb*trN_dvpG1$l?P6FA;1n~4b5Gtv1`sm@ zdd**;+>DpnhlL=EBf;dIpujBjtANwJiFk`Z?vw#l6{3`=I3_EtOo6*Y9x}n84gnSo z==xp|Z!dzkfYonCrRl?r2oy;}VU+FM>N-PUq8N;gv-?5L{Q#*pgdzADWktpc2-gQy za2x_`WFag8(Vg#swXiy|DeRUxp=1ew%n>dT!?lcyrUYF%H9Qu^2{L!%u0xA1z%I6O znELq4C|~u3N8xn#+tANuY{3d|Wqhp&5!u4~-#vzb>V+}ku(b7J#lwD)37JAjok80c z`l;JOoL?&kGY6aCB%d&$6=J1280RRP^GbInjAFs`viPfu8!&GG2LOP9-4`fCBsFnq zC`1-&l|au^0N>L0_W;FQmkqK{tV;^&v^YABk{5l`_i~n+bXZMi#1>?rcBe>OSR+{T z5TQFh^HR8AO)hYC^bCy=Z)E>hEHgHdtmOt^-~{VJEAX5`g!MCoyP%74DdZ9yv;wS2 z7aOPmm@_U;x>=XFPR28Y$_WQ}acKzvY0M34%O>n$4u1O1kHUZPpZ-y}MpkQ)+G3fb z4dQvI(9aqsuqKA4FU9QNz1at}!Fo)w*8xgYOyhq2akzpDcJ<+R;gdiAGwS!X(FpiQ zp^Qs*r)MIRhHi%HmAAwB$PL0Dh6*nCrdg8yUFJ&XSBCvyyhO4>a>5yG zF#_Tofzsz*MfhwlJ_V@oUJn^oTLWP#fU8i4uG%{h)+v*MX&Q!Ky2e$oaFwhGGK&aX z|4fVT;nVf7vPaSBEE%Wli330?1A(bVfMGqlw`5H0XL`?$sOLa4&t_Hy`x z+gHOV7GO)2Yg5)k_qjr*d3ISp1bwvhc{YP?wN4Y9xpRQg{s|y{gX94~Z^8I9&To=+ z(4uYtOfI;uR%?PHf=;pjP*4Dz)k3S+| z29|(l79gQr)Z#oF1*X@RnTApve+6N?MaJ$SF`E?_z{Ab$us-)3!A((bz=x%0v|774 z1G+fUD0zz%t6XJYnywvspaOC;7(sJuo8jKQ@51b3m;j8Hc37n>nO-f(k`1cB7qM(T zUy5bjGzJ#iUlN29V@#5q1Kn~aAnDqAp;%oy=h2KmcaGpeH9EQ9o+`YD+k<}8I z39@9gUbxWwoUTsC;qM|l$Y%r^f@uvmf!5z&3mKuZXK~{AjY)d(+-=BcfnR(cp3Ubo z5#j|~<~6NZ8382^C5Sk|U{dauF>=k?^**}ToA+eC7?|s)46zNG?~QDR^CjR^nln)m z3^|{?KY|h7pJJ^ns_ke6N`e<|5a*L$aV@XU-}szzEtdD-tjjhzcC#p(fUEfoGHlJY zWIsiart8?!5XL1oujM!#qjTbW1V!H0dw5-cHZxJc5J4sH(R0CU@FhT`m^xz?1iJnS zfE$VhPmuxaf-jBACXkPd#$R1SL@yubp8U^y#yRG@FW>K<1#qs7V|D-dXPG}=alHJ^ zYefJW=fhaQX6RJ&d77W~9O-)Y67CV#%<=nOP$&pi*2)sw?|T=5BpUe4L5?tm)#Z8C z24fmoV9fgfp1f{kjB+fA-W(noB!&*MWp|bD6d4Efm|5*Sb(1iX?Z@!MI9_~okMJ2{ z;fZ%k;^sVM5A_d^hHKYv;Qe4s#B}e_POyf4fp(%+EjpOQ#WVIom1`YUY;&9KmTrCy|Ap=nCH~Y=k;-W*20}7RE1!$r;KiWebov&KH@uggqS-TdTn@D{dRL zj7eASSAZ*U7o)Q^OifM_-&UmB#S+PqkHXiV+$F|+IV?O|!sTk&AYw<`i2uc4Z^f9D zpt4uEvZYhEva*?Z5bT{`c-s-lTCiz!=iuZeo_#z5N)!e%49o;cmd#TxrI14=^U$XR z#aEyqlYq7Xq;0I~ZvZ_0`UZjdTKOuAjDUbm+70toL5s|;?}zV~ar-25VG$CpRKrb5 z)nOcj(Q@_Bo8!<+tj{C}xeK6pOxmvj*nJG{;P3=s)&z$Js^$O*R4)GZWJVQ|9 zlgCeBv-ZOQh`)QNKNKd10Vl*!5V)9T1J-a&;wTC`1a)`A*x4^Lh3jSt5LTl|U}wN403`(ByQevj+B_?F4XcGl!=2_mH-H371liu^MQ7%=~o0u;!qK z2iUa8CKMomQh|bC$uOq_uuxg*pRsSE0`A3KmdzKU@;@Q))U0D!Axouv_~FN4?eViv zfn};odoKfQHkZggMo?j)$!rZyU&DRZ2+QAm99BO0TWSZ=G#ECu|Mpv9=Joe*MID9z z@UQ>ZFq6xNTUV*ogA%Q?-*(%m;nqDGj;Wem;b+ub`0Vp9!Wf_^MEtm}P6Wa7ljo%_rfV*RF<<0v$VujY}eA&QO>J00S*zOpp7>!kb#5GPYHovjJn= z!MK(d7Q@*NpxgL5?xlO*8f{3HSme7-D1u#D8W;!}x<#D8QdeQ`hF*Oy91u^rP0%|~ zleiM9$6+Mp@pYyTR>SNE{|~L$NOBVNum>!nYrKc;c?^Q?0hCz-sexiMUe+~lM|jrx zjtL(@xyrA`^9Zcbfmqi7T-HxwOmG_oF6+4Te1Xz4KMNoK_LG=+)HgUBT`9Uk5OM;3%ZY8>dEZz?}P8GyMB<%niLhT&@F`4rkQE{i>*Wvv3TeipZnj5h=D1`#CK zhDTPZ0I255O;-`XPM3{!AXU5qx@JZras+lIDq-fy?>?_rQ<1Bvy;cMWjK}OjnX6d( zgt?DwB+ukr#|%%;3S*5151>!w9|0rsBZ4Z|fwL2@$#uP+9vut9`Y^i3WC<<&s!J}; zC2KB-5+F1)Z2?#?jmH=Pf(73r5aR#w8j*Elj~Mr7NJJNgM+Bx32n(nEA9rIcOnvj& zcvkNrh%>`ku%gS+H$|3d( zz4vYww&e$2uDvX)-(4I3ZrqeFuOCN@5*Gjp1bu)A(EQFWj9`rSx(@!v`}lD70d~H9 z&gL^wo(pg}f_=5%jf9BpO6<$7r^XnyiUjOcS^O^!1uOqP4%^DL|(A}|Xh9BUi zyY3iJo~Mu+|NG?rzFE2<0c)RoDOf`9e{2NXOuiD%|GJ% zCUrMq+&KOn{O15!XJ-3ei8BoN9)MIaGuB#@r|7h@o)gYo8y<=TfUQv&Mb`djVd}LX z1KeKW+)?opmhuR>>fE0I0^MKTgk2N}!)PbAErns6g7_&APSQijSgxCgqZ`mizx?a) z%m45`PWdpku-jAm2hyN~~J0buL**za-oNEZ*F1CVG@%a}UO-Z?_){MD~N z48Qu|Bb2T|y6h&Hf(ikL_sP`Q1LV{dBp_q^X>M*4o0X;cIvec*p>u>Q;@-2TFi0E- zHhn*qg9@-rY*ZF`+KJ#M0-EBtBq1OOZQfo4h~1o+h==a-jq6nNN9gcb4^5dmE$hUC zRB;_`z)DyId4QT~XTBjBH`U9IdW~UbMw10r5R33SDW40OG0;D9la=K%@(|N{jm$jW9ED4g_2^>{Z9kb0eK51V9Qq#S zGed^rOSrg4III<7`s_R~JTV<+M)B>_3hU^r-vX9XVQG~^O|jwWH{QlT=m>xPZ~k>S zB$G6=w;WnaYp|U>16hxUxxsJ*5IT@OU>|xm-9WtASWLJ=pDTLGo z&xA&0Pcb%v^`-w6xLP_Xo+q+q3NZFv4WIKN0A8h`WRzX7lRy=)P{V@642(618k~hH zLggHHbrNG|Y+@MU#&}@p8gv9{M-i&H>L}?=_Hk`^jPfVB@aX%eScM3&iAkCxm&h<; zeag&zX*~?~qEs+@P8iER6#L}s?@%^J>kq|;71(YcVdXyaZNi0?w}#X@054CHNQ!;7 zhXJ-tOlm(uCBaxmU-`xd1B*EP|)@%J_6_^IOk6((KGpze;QI#uY|)3&K!VM3pEXhEP~yK=$E|n z5f06p(S#$|GPM_Iw zH$J1+@toM%;{}j$+!0i84Ru2kh?sz8z&M~-Spg_B2GeX!-Q#)#Jbk+OxoTvvE3B`; z#W*$JYw`yhqdAZKSL3q-nPHfUrAJ@`9bz zjTu1UU)YsBKU%%@;X;3LrTJ;p*iPM^)i^xx5C$-fNrkFki%gXhzI<7Wh~?i&q8 zU(B?Z;6BYcH}OQaN1!FJYR5wo~%nrjvdd5kuJBY=^0$P#-yb@@to z_G~W9J|Ni*0J$~$n92*w#2C|AhZwo5x2P+Mcj^>!V{dsDkH;}&552hwFX|$e+P$j$ zHdfV)@I19rt9X@8NYXh0xE(^0^l)x8avI2<6Fdo)v)Vp~L4^=7m*<=^R_cXAVkMoE zEQHM{;XRgR!rv8>-@1_ZJslXnFn?Q=EBXB6UxnX&_)Er+C0_Lg!{BVfvRU+aeSHOw z1O^ND92y-X{+ql4_SCVugjYAR${4)MFWn3iJiqabdKLEZN*r;{n=CCD3*BupXRC%9 z@{VPS*v5KthFG-qffB8N?$?p79S&tH4`AqP&P~Z8rVz&axOD)K{F_5UOc=5k>$VXd zgP3i|ohHk0ew~}2z*gaYVMhTZI5@f{*qj#4Q_&biim4faiN!}-Sco063~ObS7nz~D z!pz#tFu=WiTyY4>GlcM$Uq6hDfCr$!qK}M>u?bq3a7vW0fi(wI5L#+AHfNIinq?$1 zZw0aLvoS3wSf=({7vs}cE_hT(bnyEMo5CjJfQZ3PZ8MD?NxxlVYo3eOxrH1 zv=gk}fENDdgWte@0zd(wMU+IEXE?$|_+;~LXdS|BisBJ;s_-hr5qZEe(AX(9hGC`K zZ0hOhY1RZVON*wh0|F%h7gKD$?PcPvu*&*aKhz29Nzf$%M`me};8Nqg?7eDW=OK!* zhr?)WknU`ey}f4#Xtusb9W>MmOdRnI9NTA$&nXC;hQ2Lun0Mh^2r_5C*g`6e;;@Xt zDMGl7_hb?J2Q)q(%8@|8{cBXm-dc-S$AZkmitZ8&+fV0*jA}f3sW(XLR^SzfY3G zT-YV{wU4zpAa2lD8xw09?AOz!U5p%B_O%Ug4s}h*iZxxrx+maxSw9jzI7`;V^ZA-A z0}zgdGb8bovuc)_%zirpxgFsR1j2$Pu8?!Z2ck03P|(e499=7NpiE3nFLAeZ)|QPJ z!OHs5Vl-T?U6}#!p2Q@lHe4HabzoBiTu{8snL<p5G7`x`g@$&}3 zqA}r-wMRbK!_(`rq5A9%0+n| zSx3ncVVwKNedV=W1Mbgz5)D4@IdHo%Ma>puGJxpSA3!}9!T3T?!&ue%5*&Ia8jL3a zVPg`F8R1DT5|b48E(4fBsmIn}jMW%OW84@5!)-qE-rjn+`R0#8FRsp=y)EM229Rx8 zzzj9lIPdmE?!%QhG&Tr>PhCsAPi;I;1|5I>`V3w^z!7`L(jzjHSy?5#BI`Tr;lZN^ z+#goUvPFcw7}sjT-|)mFosc%dKE@>hc6co?P;kBac|m2BaWiAcLKtLXXIn7@je*-E zX-d#;>n>wo8$9>6q!DsIq$y)kgJ`IkD2Wl!PMkZQHJRsLn6Q4BFfJVpbi^Eb4f(=; z+uPweOXV>4Y?1qC2*tR>^A7Mlu(otzXm&6GoZ;2##l;yrZZHPj{25}GWxi9EF4Akj z#JVSC$4ykQERJViFIyW7qF_GF#u5ivh~2&7*116VT1G_gJIJ>gs66 zCG;bRmAI?~gp9PoY%tZowk-h)dcgTT+>3OxcTuRtr-aH|c%ESL#t*@;dlM|`~q z1!!FsTJ*&pT3|&5lI}@gHeV1$<+|e&{K%U^DHGR59 z2HOtrIfEk4O-+UV%fw@?uM*Qo=A_R6fX<;bPuX-^uzV_7dy!gpMFvJ|UiX|rLzb!n z+oH10ao{?jnm5N0pb+BC#N;i~##sAZtLX%1H&} zKDdb4L;^j80zy*ulS^TAU~lp`y46%X`JN84e>cehg9F$YuSm|>AY*t4YaTZ`>p^ZS<5K`6nZHsLjCN-4|LL&`xwF%W9vlF@30hZ?2^p&u-U^Y0h zEfpC+)|UOEW!R__$H81GEQ2&Yl`iWq%O-oqG)K!imZISAF)_>i zhVYJAuk1~QwX)4?$at8sW^FcO#$sm#K@9U485U%c*K%x;S!9n12)#yh)$upyNEXgL z9@#$jm1A?weV=YNK{3DL`hJga!I_Qscg#WvSv8f9VBc8e3iIQ2TpO6tJqf)%n*0TwVhN!ndhL6obu)Soy0g7T+i#rb-@T|x|;>I&AAs?zhD~8B4(ug zO-UrN&batl;9m!b$gpy||KCeFNIVR*O+SlAQ$ySuc{bq z^k^SElF1aq=u5~lGWJ*Lz|>6`jIQ7+;at`!?8*0Qp|RgC(6Pi*BV+Yw)gX1|w~6mX zS35V^Tz{;Es<+Z`i9M=R*!#3jd8lS)u7nG7jz;$x;@)AMWNp(BNvF)$iz3oi*zlaS zQ)9EaCV*=2lrh}K$0gD6#uVypSe{>q`}P2WM$o4&Yo5O&S~)F03D2mczP7pqvqqoJ z>$hpRdM%u%0abO}3QWuegH5xDmxyJ%2CX>^s!NdoQQIX;5H#P-rtC-&4|G5S*<&_Q zvJdbv5?Zn#=wyG`Y3~d3XPI!f)EYkCCZHb0mmI&2B0LRqD?2bf#9sldS^@HDZD`u;>nH$2Grw6rRz*oy4&?A;&1KUfd|1)=Ju63cpX0+iS$KU8a>* z0XNxgN?7Djpat0_tjIh@e=q5JuTccB1!^J*AZaD&Z+K>m=JBvJX@mvnzU6|_DFBrV zzSbYy4c&aUpA9wvh%11wk8!si;o_`f1+?KhD8L>JQULi7WxBES6w8DSmO^N-NvsEE zLv`bSj$mhwU>{8|S)$A3u5+asV8glIMm-VR0BI z>s3M6A0YrYq2#4xN3foZ1B=b<5wKbvx=C`dbL?K~q>Z7#we(uT-12fr_h-Y@otwnm zQQ#I~(SZwF=ELH-+5Tb#KSMl6H(83e`>6A@{9U-jT)xf0>_KT7kJfL@C7lf#^WTU2 zKmRlKl?-lAxOC+X#b1f91BkvpRtS%N`w4`DW^}$efL6ax2#yUqYhs>qh z;S3k>9E@?MNc;{8CV?wlE2v6_RR$(E&sQJURiSPG<_j;s9bW$FABSD`*4*j_Yaj#R zIp83;hKSZv3=ZQf-g)&p$qdA>!HATW=EA$9*>JrBmTmQMxFBfw9A>l~fEQU7*4E?} zmAeh(6M)ion%DPGU(i0ReXtXz=@)bO0}ipZv?PL6XBJhLI`w{Vw&F!q`W zHL>_G;K0^cmT*zv`cnDo{%asaRZg)Vr=S2fRadcbWM-}Y^MCp$;S^)aqEi2S4 z+^hSq$&fLOD4^L6aU3cpfu8Y5F$ah&2nH z^Cl3~{b~|~NeU5evFDpW*{pw`F}4t6o_z+DwC6_VM%kc(kFiqhER~DzS5XOO9E%d0 zTQV;&gUsm*w#-a>+jhLFi@6!aeV}vj#a?KTmdo# zpxh{4+y8h^LAG-&L+lz0%u>k1CJ@BkjUgPL>xcdAKJxhlV!phuOlRza>q+J3=I@(- zX9(;i#= zu6jXX)dVp19ta@h$nx$bB=GhG z2_C|fnoKRvi2QSjEeKvgJ|2MybSu{|etIE6oY=L!W0D$Ki~cL2@7i18S&1V27@^ni zya{;hB>rp(AgRFyJH{Br zmoRKhLK-FnWo&v9R&o&0j4Kx79yrRp5kpPA+yr{ALQa7RkS01FQ4K?Ps#)i@;`40f zz$-lvD+%`3IZX1?@9u>!KKdYh{=u)qGK7&tNf`p8hZw%zz5!$%Ma%!uXzqXP1@&3L z)n>9A%Zp)^;K4F3zZ}XpgD~RieBsYWz#OA6>;bB9>p-CA5C)7t9w38{W1Sr+(q>&3 z&@mfYrEx~_+w(+cpb}J=e&;_5J=boq>7ZF@VFmODM6mx*|tZwW0|qL5r?N>>;Q^u zFnCg<2>_~YDKkj>@zb}$!t_$jzpa5F!L3?`HtxfPH{M0jRh~P^AujJ9!*C+pICajW zKJZh7z8R=f(^FxvzbC9crQhceWhtl~)&j^phQZ2naCFsLpJ*GmtCXsZ&c}yF=$J7d zUV8V`_^=Ufrx;S%!l(?gTsw2nBa|6~d8oMtqWP zhBQ{iCWrH#*WX}HN>Ii}VRf5p4-bSZH!ekMs?LWFIlMK1VnyZ3^>^qWxEvjl$@$Qr`P zIN}*jND{FxsP)Ir-9tRX^!Nmd1Zzn}#hfc_4`5@Ro_lad!PELArHtsOw~4A5qh&2iLB#W*f}G+CFS8 z*EA?rFxJN2*KKspnMq=mwZle(6~h3}cje}*;VFz_!sT*&zUNb74Mg z!OGVWa4n4e_A9TE0Y~#{nEvUhE3qhTH`gv9{QE~}UB-AcUI!4q!((IN!ToPpd*bK- z9J=-LeYpr!+yh;lmvsQ3&FqIq2f_i~ip6L1;o8eLQK+)C$Ue@HR=ol9LLz~iz)lb> zdmxChT*e0d4L6sUIrsdHJ!!a!>=avv{Y9t*$xkJi+^Bq!2`2fBU%3h>L^h(V1qRlE zD-j5-Je1KBAjs62AkxiV>LA?2>_J%#-Hx&|#^f9`#)QC|YX}s}oEhg@#y|`I7*^QI z`FWU;Ha>I6T8hUUyX6=x%dpS9$-LQOHZFfkSYM`@j)C`J@!5|SeqQ2?9Z;gmau~)m z3+OG2Qrg;%f#-&tX!su@Zv=6j=njj)H~2cQ7=nUU;tXX-qOs_3iRZV}jC0z@xjTp1 zPhr5wcsI+V0Nh9>Dl<;krLM=Mjsb_nefQ=lbPZtYU{7m29eQS1U&~}^ zu(%)3(8IdItw~lG9^srUa=q3tbBziB0I=`xehcs+>`}SLK#Tk9x@i3$dVUYG(2h_4vzFC^f zb3VabkJ7)rhTQJ}44RR925`3?pt1S`%vBm4XXuKR8`!zLi1FvF>C)|%z(772@2kK) zgk@WJ`aG=SNwfsX+QKr}Vh)T#C>dG#0Bk^$zpQ>N;Wc&j8dW$*++>ewFtnu+dJ6Fr zo2C;bOwMb-~s*I4|RO#&;Cp?@PY#os}VES})0fl%)u6_#fQd6N79D6ueb znL~a;To4cyr9o{ffPj`VD2@%%tYM;C(hQ^++$`-5z?mddOz>DerH>+Yrxb1(4(RED zW7>@ZKo+13^=Wo%h0n9+0ZPem|G^VlPjrW2;&ys-#8@!XH~@&tfl?(#uNBLG8AT{l zG~IP6>@KdyrruipQk6R#`V;`C#o$^r1w;hZ>C55p>BAT|f>aB)P;?u+uy`zdCv(;S z_*w;ij(~l$UV3n=mJV?-Gmk|BECcaKSRTfP^}v7__-;)v(s;wk4r9lqX$=_Nz8469 zSn~@3UD=_A9l&KI`%>t`+G{6{C(ojt1E6~asB8+jR|F}-f;lR{_IRMReqIB-4ZeH} zXh?cB0_vEtr6>F8cgc0we7(lD0aSX}JX(oq4nd0bJ;dS__?9^kW%fjjn@}@0Qu+F@y^de86g{_eBz${Vj>5oALbf#ZU+&CMMIRcDMP?Bu$}Kf2l6#BH!h zvK);XLX~}Ct*$KB1|L(xr5+X@d>wMM+d5+3ln{U!Qoc z;qFjaCWAW1{_8*)9I)ROF+g(L%uvdJfu*}&g_lRi!YM*! zZAGJmLew{JhhvgHQpZi}SOsF49RX{k(pl-f#Wh^olVNb^8rjHml%}EM1vKpe0fWUl zNdpJAsKaGR9YC`LE9?d1g=EdGdxVq@#mPzBxfm%Z9GSsQT7J!3V*fD53j2NBOfkNS z$-KaAr6g0?*kwWH*4H`L+F))A6VqW7Bj;%0S=d=zi7de>3cMEpw}ath5c<}=Z$bwG z>4N=`RB^Gd)Q;Au6L>elug`4u7Aye!M1x}x;b_*DfJzx|_M=MDrqcovm0cfo;d+Qk zY$cf51Ol&lK&`nn3d~IFUG`}LV3WjsaqX42!ZHTn7MbU{VY*kaPA9P8Bjd~m9)<== zj9Vc3WL|7Ijj`037n4Mm#sMw*R>{m!$L4#u zVgv^fY4Ew2tiuCDmn9Ep4Lp9^ya%5($lozq#>PD+ct8+v4_r6AU-nIA%rVJ=$|fPe zBarR@oa$bb0X!n6%zH3mKorIr*LIu&8g)zg1jCK%m}uZNjAhfHNWq9i*DL!_K-6NK z9{-E+VN9ezM=+_kMli_#s#CqhoJH3y-(wO2 zd)n(X-{;yhw}?WYU)D#rtk-v)W41B#@4cPt=o(`WI(}ciujzK?30PAZRtNH8d_KST z5u6yO<^W^Tf%Ar9^>6$FW6?e2wE!xZkW-(#bU3sT*>V9C}{fwXVI=ZQGbBk@5IHLfV4?*PSODNG@%yxVh8O1 zX4rW6MMwhhbltYH7A+CH0^E?}8sQDr;0%Lb;9=bMIbSEl(C=Xxw@?SO4TkD+|7569 z3T%Fd*mj=V3{Z=4@8cRRQ6kOk`@+yz^x7Pjwql6D-hMTda1S3G>lwg!sRL{mmstl0 z0NgFHFt&Q1F=*tZ(eY_w#Xxb_M0-Dl|fowY51zGn!cfj#HBmc5qvX=l#c()43HXJ=Z+CsO1L&U#vH zaq~ej!z!HYZZcU212+a6Hbv3bF|q;+>@Ka0T@;!{7`2*g&TR}$9)aJ*=BkKFLFIWi z%sypGQvCa9_UkYNf@j-?bq;NrMd~N6=mH^o&VuI9<_F}nraB0g+*}786WmxN-U~P3 zk*o`fG)o$<4B!Ttou?cM>txw+u3JU2#@s3PR6W;@Jg|X7I(0WSnTxr_=6d7OZ z2^KC9MF&{;ZJ_8B6Tb;aE4gtIr0o^n3rOtkg?-@AS#wJyJesF*{K!yvzJky%8PUc( z0LV60tffW2P^$CK&OIT(oqnUV_bE=%ZGlY7y>%>4g7o`}SUzl|8cHHZe4j!@FP-(; zEI@q1BI!mr0*!yO&1PSsva4W_Nee>8 zu*hMv%Y-#~%wXz~m0@EUth)JZAyUT**}i_PE3B<0l%WE-#-TBwR%`cyXG*d;rJSK^ z3An!=e)Gx4VH}FNjrX<#%HO&=9)5+Jxjud^WG}srQeBU%XA$P=d~ZE$efmM@y!Hy^ zJBV$}z>LrW<=c%%;Y}2ECmZ|-#cW%xC|S}*suwT?5T-s{Yb)7>`)gcdi)4c>_I92#jdhJx zRz}$m0ZtE?uN;EG%Hgr$JHqV(S8Ww5y@H@ndBy}BfR}Z)4q;$-?tYJDI7A;-gK&ul zIlyqhKhb@M1my)^mTg) zK3lPL+t~Y^#JXv$bixQGDB%)~6Xr|iuq>ScK&m3!dUCLDqnBU`S8=Ck6&njgadPf0 zt0KE)XBci3U+Pr9WF=M3FMzGCErqsckw0T3H1}5o0z7AgNuF=6>2vu~(R>nUR1QCa@6+kOY44-jM zyk2At0cYHWIYaxRxCKG7V7lxyE_B_(x5v3iScl5bF*hmtNP7)#$}=>E53Lp{9h=7Iv?Nc}YUl4C4tH9->tqd&gKY z4kYK8ID6`r9b#nf!x&!RStt%*T#*>ic20b18M24|p*uT-8tpF9X@)F+j0#^-H@XS= zzmKkNEW@OB6ZUh0t~XKf6fZ@YT5G9+sgNZjG?_;~uMraN$dWY$j6pq4Rpf69z-9uBgonulx|#*1 z8G1>V0j2KCG~s0}uz>C<&!a}X@zYMNOr){&o_!h2fuT_tK-gFq@f`Ot!`K*ZL)PpU zEQmAyR~c`{vIw>fLPG+l3*v$$X44@|T;Lc~%S}YR#%ORfIZz;VW>AJCOG$uU08h2F z9`1kr5pIr8@guK=$*JjZ>&}~4@K-|@MLtuxB9+i9OvZ`x4=TsCO^cWWAgCOgD=-bY4=QT1}*hu>@a|Z@N zvzcyB42DS%zl8#&>rY`$Ht1pc?Zesd%U}H+e&fwp=WK)^a1Uyh_^JzlM-o5uP&b>6 zV!UQ8D$_E=lN1RquY>mY*chu=j|b4TPXJOC6hr~gS3*h8&XH}mzZs^eG4}TDn=wea zK+M-6wH}_$Z-u*G&xKcRyh$+mop5gsCI#kU=vr?WzD)wh1=+sP!bxK4LaQx(KC;x}>e@VmS*arA&4MbinJ$P!*kr?O z9^i80AXd3A0(9>I zND8;E&=zK@={7}ZoE)yg#ytof%zHhRho&!wpZ{O~JFV160%4YRvD{p%Q_8iNc+r+Z zsIeARnpYo@az1|bHDU?ZVU=;CkVSe1Lre!Nghe%6nHUe7&!2|<73v-#0ABxx|8eM~ z_Tb2+K?0L;;{y)Qs2pxVX1fq*#I%wma7^~|0RnQHvsFhq4)k}0NsNfL(n@#*=DlP0 zS*Wj(@kI+gTYag(YA{mvFKi|D>kNTznULYjSBT&1hJ~6U#%|3(DxR|~3ccU2CWVDh5TE0!swgtga%1T-N*(R8kPjPKoIc4d`KeP zp5Olhwiqx9b6~6v+m87yQI=pn#(o{~vt?cc06T0jT(jU>b?x0=#w*M$pUHTM>dixFAx39M;6lUcu7^M&Sw1RG>lj3GS9Eb7FIO zyDl7IxOEdh%OfE;ara>0s8D5MG@PK!05E_Q6rKPVEy{Ys_#?w=Dlo>NF5(?gM)3hm zF0*4vU=rxbtgyeetOW-R=FRvpWtwZ)2H+8}34T3$hJv&Mj5+@N-nv|{s{CC_P>9pQ zeZeLe4#)^VZ-<@$WaBwZh%q-YmXAad;iLthtqB=f9rnHZRF*5o#c{1R&b;iCz)E&W zyz66*_ zp2ecaa1-3O$*f+WrH)ZpvVbxiDinqUPA7Jiv*+iSG{)+>upayz3s^fr{5uvZ=d*g6 z;tN?;Wn-xK+PIcsHniis2m)o%Vk3C3rSZeN76(Yi8zdoNl4=m=dn@6Bex|K7p>OQ!Js{3l)aE)f_n!3& zU6V|&bKl^H{1DxI*w35jXxkMwvNVamOl%uQzXiq@&_y!v9Txa!EcYIK7uJZ4?!)!0 zm*E&$*@jo{rI&7oOH-GrsxUyo{1L#~0048BbBsR4P%4a!66S$tigoTpj&u_HXSt(2 zbn@craxA)Ttv21ewsN#hV`X=r(1@k5GCLcdQYUYFaV0$e{!w^*{{dg$g~jJ|6@rb+F8Y zcAvGZj2}v}**FgIrQ5T?VK!Z<85D&TkTZO3le~Z(MvPMeAO;Hu^YGG5V`UC# zFpk?&<_(&?htE`SJL#_T@C|^PJ1qdlL22RO`7G|(?^(FdKK(4#^BO1lq8-7Zf?iu* z1oUr$^r;w53ox5`udlDcWE|2T^d*E8j3B~mkyyD+4*Mf|q*8u2QpRT}%V+ZoVfNW` z$}Q}Ko7b*TZecLI_4;dJ5clmaF>=57n@_@*Upxt8#YxzmcDf({TmXzSH^))-By3P+ zdYCwYTqj$Sb{ZxsSR^oM(gX2#h)dYHWEjHLG>Ti=E&`W_hGW|=_yK?@<6|6J{s*vP zyZF#g6_gynE0?aYp46W~SvFvOo<5onAN<|#!q;pbONOLiGdoB7!>zaPgmZ-P8p_`5 zly-JQ7KPAH{Ecj?hq;~m=vqZAL}pgTwF}`BnJTtXmY=6c_qOIw8=EeHkTi=qkCJ}x zwL767rfZLwlVX8*2x6UvVWbp@T8|r~o~MX2D^8EI=C)8`&DbO7D3n)=^vwM3BM#Yl z7<=t~vMSjrode;$_kY3|X-9(M9VOH7F@}V5zfvXEpl>`J(HW!_rQ6B+pHkw&z~xTd z0O{UhC=pn{f$%=!WOOn<8?%p245JLSs8R&g=fcR)NbFZx2YBX~lR;v>p4^`$9+WKY zSKkjit=-{>SkAKqwaDm;o5hW~wD=4m*3CT%(F%R^r;KKa>d>=*1W*ACBD8b})^Pszolv~y0q{wFZ9>>5lTF6rrqr&%n2Czr>9iKC9%q|8pF<#dm#s-M%;%X{+ z^c&CR=hA{V=G@|=mcpp8?p*{!TPMnAjALM8EGOahV{>xB1>;io>R#lQ{9T1FXhK2o zH(57d&Z&wZx@dW$&tknSucNzIz|DiibDcq6I~TsZX49qEWO}@a@mMjLgNwRvoJZw> z%+>*(5{+!#RDRC6agQ}Y8qk%`2)Z1%^WZiU!1(v({KCrdd@_&T+wr-doj?13Iu9{R z^#A4Q&3Yrvu5`T}!C){LoacE?CUdUL%2HLSq*h5PwOZ=d=`#YyjkXONePbAgZ++_@ z;6Gr%fDIVFX~3`zS?=SOx{txfr7_`HWL>ffDX-Njri&UKt-FT|JEcP=6b<}B4-1FQ-X9XtA%%kkWf9dODR zYs*xd3*#?krrl?b$?N&$9`qTq$gTLit}}z=Q~YF(0gDOFTsr{E0h{kQ>dRExOi z8V?=Q_aR9>zkk6(#l7P^byzMTV5QY0J$?Fb;)hCH&@v~}z}W7e%pGUm6l zZ!r`o9R~25yf%Z;Myx&h(E__>-5byEn#4pYa85K_n1qUF6WL+Ato4<3SiX5S1EDpz zV)a&8kwA9cFqC$HViO+Bc7Rz11G9rzTxOhWTHH4as|Fc`PBlB&y=(lcWz{q~jYXCz zcjL<_o0!L5XEQS}>kP#m!2@f3xJhCQ7~Za1PT#8=Mq0EDfp6etGYj5E7KhOp6+qDb zYLU#z_1Jo+6`#57LRxWY*i9jWU~z9YVle0rH%rLiKHay6#1QE!F&Op~*YPmxo>Q>+PzgH{P$xy7vz7m~+Au0&a>nPs z`8Dx8!(o)_{6l0-mH}RZq6{%>y8Mij8ALJe`P8EeSHAO)GPAf%ox5MmP0O2-me40Shoh(YiRBw7$UZp(B zdLEb74mG}7D9B3%X&6gOog~>{Rbn+_wRWDjAH~1P{GISD+iZjuTn*j4U+ejt!_2v5 zj+-c-jEkD*vChyIE~C3-J(hO2!!p;r^ZwniIRAv=zvVFZ`faolc(tjZ)VUsf?TxG%FYttEF3paVz3@&MwndJGg+IMufx z-1x)oYgfXvXO@lFCdLv00K-r)vxzl2XPxKg&cfB{sc?psRbag1SU?Y6zYY_<1UK)# z3Jd=cpmzU91V>MY-@n)hI|MJcc6PxCAw*z9Cg?n{{Psm?8W;^r^cNi*;oyxZd3x z-hKHEmenEsX>WuabJgtMs9&^$YjqUYz$zJ3R`a$*KyKq5LH+dA8@q}DLd`C$a~nWw zp_>G=#@_1$sGLKoG?Ia3U9Wl$*HeO3&k02s93;>i5VAEp8|sI9vBvE!s23J={Lzp-PjIX^7eZ+62*uyW# z>TO93Qh*mm7e>bTKAD#W1mHz|5tnKz3}3$)))rRiz zB#A=BZE$@Xnd%p~B|Y#B?B6n>5xH|>MXuZc_!_9sIi-+x5oOy3z?boDhJEaS(a2LX zaini3eD;Iygneq!HDXBdB%wrL`TXKrWQFcy=9cvV%K?rpj0t!G&>DYZDH0V|6mBTc zD&GomSs>+(aW=qFE=dwr6vFIrz?^ZC7XG#5jX*I0ODd2%p<|0hlTT8dcU%JfvU&1?!9*iJQ>#u$B{HQk&~ z4GS(5%i)+=>)G4`s7041>mwMDCGpF9L}0^x5xA@wu`>YOcyCyi2!QyEpg`8eqUlxS zwi%3$wdUfhmGX!eH0%`#yO5ai@84KIx)t` zHU85)7%meB%WG`acwb$f&O?H@Q1yj9w$*7WR+YGVkhii{$UF9sl?@!TmvH`>gecEM zWEv>0Q4R2ExHvI<`(EG99J|f}BQuNZ&^OjyJ7bTVfMTZr8Byu_?AZx& zskS?K3)k~HvE&MRQ7?hAow4yj*Q>T*aV>PsUjkHFQAa4w6ROXz}= zaP#JsFfh~%4#Fo+VI!M4JB&JdSixo-Td1`gvlTQFK%pj!iy-7uH^ zAbGD_fKBgIx|`VG066h_h5^JqE9J}NHYpIPQk;MV+od$g5%pLs0(r=1>Nvzzl!vh; zT0AbISs)%XefRdZtH9DIcugrTF~AMnfa}}FQ_y^tf_qcA{>rC$+*+g*lPz~hfzk>> zv;h}b10buFYd68doufExfn^-dd7L1W0}~^28u*)WkZmwy_R{U77SP5fRvgJ3Ehzoi09dPC#a2j!MPl|;WmN>* zCLq*I>3tSt*88;`&@g%oYcd-sBQ)8%+V%b8(f5h+r;y3BLI zOe2J^0L*t`2+xV%T!*PzrhCa6Y3URdEP7cRy($9&*l2h|;yM(06QA z1frGuqZ~tVA8`g#oQ)I;NPmUOydFjnA&C6({Bgz*)b@OA2V82D8Mv)dF*exi$S@d4 zZBl~@S;4I0i{Onp^eTP{SPaznnt}&ZaD2@<7F_u{FGfHf0VexVr7x>zrf@U!aLV2^ zkt?=F;_oIg)e3|P5Sc&aFV|6a#!vL$*Z_7}*z|)qfkK=C|yTg+l#XmGBU) ze-)Ht_6tcKe>!f#v3rBJGi9hc#;6jLJ*(rn{fNf2RH9=L5$Iv? zD))VOfskL`&G?@#l9rAMt&m}mO+1Uei~&K+x85ADUyqztXT{HQT^UJZNS#Yxs{yat zPnT4krSTMjHY~4f-4Sw6M#JYupNmU zxby0KId_65!L{J*4A~%1j-g4s#v?@C={yQZy?2ap=ins%vBpc?4cZ{&7|` z{OSpZxN+}hc>L8jvFGMFfJQIe_Qn^0&ZlP?=rHu4f$%qf`xu_ zl!G!ORz}@0ggp`$i)0I%<=O@gu_%dsm}AT*6Xh957C6lI4m|`I@Mqp&ECQKOfQJ+h ztVQRBk}>n}_#J6%m!AdKI>@lGtXOQL_!Kq2I*G5co>dm0P!EHat>(9gd8U4Q|xP`js4pr!EhU%U!mJlrH`A7F$lv61o` z^^}!3HAWAYtrV+W6wF2(A}GEGkk<|i(gv+N$$Ra^It*Y*AmA#1mm?0}CJG>jAZ;U< zvVR0B6Gmi`jo92)7yk06_h^%3KMLa4+7k32|%xj5i*D^913I5${_w_ zQn*d^Pa)WQneZH&2rbU&9yfi)n*zX`ge7XzU zXm3`>;o5lyv%|*g1Tb}>q!w~J;r?eIg*uex8Z~#S#ACG)ASxR)1TY;Q7zyhD*Ol^d zWPA3_(nKkj5Da!PVC!&ajt;+!6Hge07=tRYk*jRZR^lgmVNeeMEYc<_zYBzh^JHrl z_s|?c?7G)h*pF?vEf0w0vO~tF(FD*88fk8%iyaBbmM zKcQ+mU}c%W{c9~d1DioaULOff*_AB2m~qrXZxVM z*?TRl!9I#pp=^nU3U0(64pALO36UHzE4s><-_0%%KiLzXr5%GHOP!xC+=AUB+#ldr zTzVDiwqAuyej)S(0P|4}S2;qSHA{1_FR%y)uHM2;PZ9!|sa;c7Ig514c=r~=p_4++ z>%?jTBmk*(1=^*dNKX-BW^&sPQMc*zdmn{GGN%GA-awDlLKW7D@6>rF%)Z&9R^n#( z>e)spoD!CRp+k8&+z2Q!Snp22QCD#@q;fDRbcskHFwGbfjH}RdfbpiDzL130ug9fY z0a&jsEx=UMfr9u+g>4(m1dROlmFbSK|!VaAl8ujNszzAHrnszyfC|^^)ly z3xlIq(UE4r!F*x_y?*kTuuys&V|mU$_$nN&uh8?5@v*kK?L{`3u5ZpHV8kNh^(ZG} zNK7uUDg7x1ok73nf)>YyT&gKaWE((3tRQT##gr~@GaU`0H&pF{oh`*~eyjEoX__x<`?9OilswlY#BDt%d z=Q3QkrfeK%9B%{_}7SNoqaay16d>oMbe1CZjiTrZ3h$OJ*4YZnuJm`9&a_gwU_aji6rvu=L|2iXG1 zW9|kNSD>7-MDxS&fBiCoP|ydRt`*Z&w`T!$dhNi35k`W{GnC7*=D-0>FqpM40?Jjv>Y2=7qQc_r+hDBQ1@b~e$oM`xzF1nu zbD)vtp7Gp?XZFy`W{0I~tJ!g$13xl>$PGfuXAiSMh#l1;U zS==jIizi`xvL|e;!32!-hvS|jHsMyhP92+yFgC8kH8_Mef3L^&JJ3Z|DF^lG(i#hG z?KD!Thlj)Wzx!t_3YjiTi;Npk0BAZWBWPhydiwzoWV!YYj*xVq%NF4SU{i@g9wNBd zG|aKty)|K7V+rIx1sY{O& z49lzYFf`{V*>+0Cbi^w6k-K5@RlFwmX<=OJyTsFB(_>-k{vF~2XdxBj zrf}V0IAj2_`+)YD$tf~tDa4Egl7IIK;C}?WiNL^-`@NZw@FceqGNr!IGI=FDVV;Nk z$5CcWuwLA2{zU{lvrF@2A&rN<7U=Bu(a?^?e$4nUFd!OXQmPb1?tv9-09;qG@;8ZD zko+2e5j&|vfU;I)*t9zKl38)~(QLsjdVyk3Vtuxg#4!%dpY^ z+iyRIsV#yjCLIW-tSP9nKSt5jvlmkc`368{$+%9EUK((L zyGJX`Y0N-Wb#qN!{~7@TqDzpa3>E-H7b35#Q2dHLG#PMFcp>AG%HU%ysUo6M{F(S-3Vyj6O?i`CJY>0yuv2K8Q{3ElBoH{-^f5{Nl97 zd$^wjcb5RqGv=3Ljz3ug!JKoit`@|}Je_go+&e|RB^Z%bD~N7dm{JXr%OcBp;zE0E zpiX_&e4er&#P~`OX!KO@)MVM`+eeRkX=Nc28I8oPEiKG*hN<@o>0s^JNyh?!I_CcFX_FOW zNgLM0Jz2(kW9*s@=1mN!k}PE?l6%*3?jp1&Z0kCNdaj zY4Q8fnQLL;>o1Towl`dHLBq)S6bX8?2*n*+DzZo1i|hlFdQRB$R)8=*)O*U58t30g zSjyqi0p}kOj=YQgy;)nqw`7&=*is2LkcIGcCz-x#pTZ0aX&Ss>DaMuXyTwl}YUl!I z*vzM(bgfbLF@r&1#rM{Pl$+CZSuOGgkfp8%g^PY@ea&ZFOQ0gH>35kyOEoikCWJ-d zng)^o@*jT@1x71C4AF$5F-|9hKR$R7j=ww(*RM{4x?97|+jm0+H%ozdpeB?;Cjo+H zv*l4R3hhP|@G(F}DmI0S)q|oy$ZZis%sn#%TU!|QaGye{^qqY2a3RJ3j8c-J9oB97 zb{7hiqO>ch!(}pXP$Uqej9FI=ii%BCN3hW_$a{|kE^^Xj6-I(=Lrcf@5)FctJRCm_b03!#{b(%sDTTnQ~MoZ9h^Au)b0^mCT^m&+= z91KI#BPcD(I?U~m8Ht|N{YKHrdg}01(kM|{CY38W7%VPKCa*Cpy0rSKx7I`9=RLsK zv6MZkG}Z~^fM&=tUL)zFk>bWPAG{AESRWRi(9E8gIZ2Qt<37%<(;?#|OkKMjPIosc zH-oDhqSE00#&jWEC;8zuS*HaQ-N5+W@R|&#ZopL;qu|pY|8;oqn_q@;#vrSxd^y607$OZ0oh}kx;>bJR$zA0k*fEy?gIEF@Jd!T0?v2xq3USz4|(2 zH&(;M)C{q39DelP<{bS-+k3bs@q7R+E#nJ>zyW~4>e}Ns-lIFpA^!*qC?FV^I z@NO0t=`rj!76;ZJ{d6CG5k{x30ro@q>bJiPZR(0#ISlj+P}7n9iFMt93w;Sd)Q;6` z%uO?Aq8Wj?xC&73>&IPA`55N8u(}E(+|QY?CvlcombuVG0?6c*Yq%?OWR+2N1=mF} zhry$z4r{JrDe@c*2){IY-xygHio#6ApmH-7*4Rh{1y=hBsrv|jn7uO;zVEe<;3`1v zoXl=RC3YxRFg|^QT7a$L3`3zFnUp2|aBp=nG62dK3%BmzUhl`{6J46@Rox%xO!g;W zjD#rb7S^?NA$u_5lyB?-<$ z+3Og$c1#%ndnv0Bv#=2bGGqmH8f#2mJgZU2GB=TpMo~n@1?3Ul9?Cn$<9n4q5#T9b zb$fEwZRAey8o$?EWV#5A|7u`OELG4%GFD`gcvfYBz(jZUB^Z*4)6Ht>gA|6V3M4XR zt_$)}aHLzu&7-M9H z;$tx;m_8p-fOle+JD+pBH5N^s!Cdk87?{r|1wHM=OWAk3R)Y54G#Mz@ z_Oc8}ZOk>MmD!H_K&DoM#rU%-?2PEaHSvK3m4d-Dj6&N&I_J?n%N$B@7)KTXJkQL{ z<24+M-G<_X`8r?DeQj@WS7g0KllefLLx1{SpIf-b6UVdUb!GT1HDml$>{INk)FAxB(wh8X4&252|??-Z}V3Fozq z%JiT8@Y?{OWvWBqWxyLY17O<2o3b=NANmQIK#3tYDW$Q!6skKbB;$1=LpBg$W@4KV zBq`#)W8Wf+PcKRoe8*rQhO!JJcS6DX_A%nz8hb-`?_@~eVJTCTJj-4Yl)ZjUB>?X0 zJ?$pbJ3JZ|(Kr3W~5Z!1SSj^9-k6ro2<8u0#f1D?OW`JWLR6-K=wm~ z045DH+Gl)blNT`d1#$5V;F(N8N>~XZ8i{yS7>Fwq4X`|NyAB!SJbN}gJ6()xACH`I z;bsQwt;*olsN-zK>}~X=>*<_p#MlVo_=NQ%bjL!{GNjnSaWA=PeGk?!c!q%GmUe_7 zfS}BPn^9^ffP^CissYPvhZw951O#Z*tVR0(DqIY(b#RURbhG*GXZd4|EnrdXPiieV zvm6r$2M=7D2{BNPVscQq^)W?{! zJy@)l-^@nf(#T=zf?mvGRTQ{?oo*o9g)MDZBno!oum}@kBN_Z@oS4Fc!-j&XM_4Hm z9J{t(0as=cHsqKL!s-3eAMJ25h;C6!l#BnN~<3k*mD!RSn4WIWBV>7b-Q zr`8u1Cbxn9hKa-_)y>Lm@myPMoU1o)(+Isctjun}uwgX=UYp4D$rFF*##g8;1DtM* z4#rX-D!g?Lk5-G8%_%|gtpvapaI+cLk>ns6BiWE8UU2qp7$60_XJ9Bi_@`e3nmKgL z>nQ+J_wa*X%}2VsdvuhaxX>5gg$BTPmi>8+5-4E-ElDoma_}24Guoafo$Vee_l6pST{G;X0l>N$aTAj&9Q7 zal3M#_6~~4j@*G^+`?_X6t3L-ka)JfP$Vv+h)_#$@U3aIvM?Wp$V%-Y=1+ig$ewQ> zo5U@@&4EqDs^04)YE(J&%^>m%_XcACcZJ=x1qunDlc@3uf)@7q&0_>P@tz16G!X6% z5_Wi^VzL=4d6KwLKrcm+>i{gr2&YY49*ynPe&c*CeDffTe)K~x9lxkK9iFg8SMT4) z3e1N7E7!x^?jD6YhcFH{qT8{`c`368Ww5oV>#)jY+-nI9*7bsAV9fTUJ;Z3Me zv#VlCBb+PDkqmzm=i9iT1f@$NC_(TD1ndK>QZE5I8%YE)cwB)kkl`W7Eq{^yobe-wa%4gwFyoU zszEXnaW66Rk0mw1MJa1b3mRiz_5sUsZ_dcHhOW7b> zC-4x_8k=~Ce5x@sFk>*TmR+esZpb7l!@Y+J#Cz+`am<1;uTe{K0GLEz$24&keRf^F zc5Tspc|sVBu2aX%lSkzx2&*HO$)NdT09-8O4rq+bB6}o>9x%?#Y*B;kje)XTgz-}m zXmFhxBZn)Jad|yvQ`#+tIta$?*yD9;ps)tIEQ#TBp(-M|;Ji40j-v)j^^!O)t`((i zO{6N{>*D=o^oagNj=CrG4CyvK$KCHfiA*ZvQ2!kh`0t(+1YQDD;$!0d1x>!Dy)o>aF7>+aBC+5LO%HQuk%<~8@P-4Lw8pyB6ujA61)nZ`{7 zjI4Jw8jw>yy9}=5bo{XfB(L)VUI*kew~~dgOT$)8lf6l0g!%k7F2mMX2XpDsTZkNt z9^9N~>@$-A_6j@bj{WfO{>y)be#?cG)pge2SoL9Kp9y(_4?=@row{-VwIkEE={C{R z04RmQwM5PVhT0hhQHnhxKFF{SoHL8y8j4{FpfU#Q*qx8US?f4-IUX@&X9^?N+M4@2 ztIQAQ0-SLGTbSiM*#}uKMb7}S-ss@c;U@YPIb2V?9J<P~nb57vc4B>9AB_%*mJjxj(a-vGF6yxU90`WmOTa_d#jWOF79-@owX5;^q z3#aItx*EqR8PnfW1mXWt!T0#KAgd&}QdVv7mt(42Fq@O)D}{2UsC{ z(2#(^xM@=!Zl4XyB=%+FW(MI}ioiqYgvZPoWd-*-7jUp~o2eY%MJ7utioO}tDk?2P zRa&OQ#`LyCv6W5*3pB}LsssI60;L(D-GpUOglEk|spbia-bPsL^Y=Bf+w`BGQyVUU zVsPv(yfk<`h#W|>OgzZa=Gsq!%_@hhj>B0GI^N$Y<2u|9&mX)EZ(gm0JPUN=+nUoXV0lU2@=s{>>c24XJF$yEskIUc~&}2_jS0lsqI+11;NtQ)DWOtr$9|disirG{N z1ih}d4%T@6izi|A>8sE?LeJZosc-=Ux`CjoA07`!^t-$mnxgJbS6HU5o7JEb)V$h- zmE1s?Z{ub{n#63mF5F@Y-2|KS6ot9v)6uVelv@T>cBAaG&_ADR%wK-~Iw2lB3A%cfv__D182S9&3p@MV+`0 zlYsu!cepGu4ggfM)Ld((r3&^TPt3;0KsbdZHQV=u+H6&_hX7tcp}&sEf@^4{wh;Ru zLGY?||I(~)k~!{yF7KuI@)~ZYBi5t|ms%&W6C)^&gT;BWQFlG9(Y?AtY(xP8P=pcA z)8cIC%H7aDFbPmChj))2A@qnfgYnI<{-@j@Y#sVwj86)?VS}1~jaY>0l}2JWb}=Mc zrtXEk#=$Uo?^~e?lcn;So|+0f(EJ~K^xbgl`c1CA6E^1FgnM^C3?tKb!xl*N={8+U z0J6<6QG2kbC&a|9P-k%iz*-^3a1S=|INKiPa0?UxHC^;4oE+*84}SR%u_s|ckjI{V z@4Z_wJ2=Jp?O+}*02mjr3t8epvaG!w653&2D*$jMTRTF@46G9bg04NYL|uPbw8K40 z{;;-3ln9Bb^_)dJIwWCvj{(ZpKm9%l7p>v{`o+(~^VxTi6>K7A;R2;<>{cBc--H7N zn35!o%Tom@^A*ceFmFRcLop5!&N4Dqwc1%oN!2KG$6=4m3sQ+WQJ%3MkOv0!M@7kg zZGwd_QU1U<)C_`BLr7JsV$p)v3Kuk5vE6vu7Gz};z*t48@#%A)a2|E_YH+En1rElz z2uyrUP$}!+byNfb6%&7Az%#E2w%C2hDVZYv#=cZ&tAH$1ammcc_E;)K*O2nH#(v@o zW}GqWkoyP%92d8a48GUmtz27Bi94G3pvWHacUdauS1=c44EJ=*&a>r7yoeytaT^=t z=Y5VCGwA)j319wQ_R(<~@8|h;Y$`6F$7?9td`6!^#>_TLy6?v(tRwa(V2C_-I5KDg zF!hJPt@hLPfUld0@64ze)qX$*rU6a&wAY&koKe8JHAGF z4CdYSi0dwT<8`m?x6kUA@AV~tru)Tx=KOkX4QE}kX75_pF2+-Fop?@U+89S{CB|O3 zbOZZ0-TY|?kPOKM!=wFS=EfNA(p{d7(1_vQc+Uzt^o*L57UgZDZmI=$Gt`j$i_box zTvLKta2VallL=Y_Rw%pWyuN(&WmuhkPMNF_Mkv_2|MFqTRCZ|@2%CimV-LV}jB(~X zaSSo?kkwp2kIS+MP?6N!MK|}{rZhr|||DhE}8!p7v90Mpe?eM~Te3)F*G!+oy25nRW;=vn7?**;A&Yc0BO z?X&Fu$g=l67oHK%XH1mgoZGdjw#JZMz6WoN*Ymtu_&V7@EXp^F%i$dpI!l)3DyVoL z1i8m|8QIc+>!1$AZU4|sl*R_`j!l$t2?1^KP&3v%J9*;Kawx76U`}_Qy+s{NtR&Aw z+cVYQ8I1_8_JBM{io+P=7X)#ZJAjYtVZM6VSQCSTWUYuijbkDkjNtqVKyCoK^w00V zPYDEsG(xhD`&mFw7`=|#yVJL36< zaY3!p3ar`ZW6^bAvjfBG4wNtlyNJbN%aqaSsW5znD%MyTEsQa-vKh9XzY3?YH0@YW ztHe_!QT#_O#O!6La(zIdWBh^77HM{Fovt3Tjn>yO0tEaB5av&rc58Biew-9UWX!$H z@8QmNSo->5*nRR1ioY@Z-GBGrhIYVj2?5#!3!7#Bv+SvEl*8QPr!j8uyFdDIR7jXM zp<6m0*IE7gY}k4Hds;}rhE^NG{U7}&VU|qGJ%mHHnhSjwWbaT|vWZqv9n&9%FArdD z=!usaoQm_ELeS;04x33ZS)_*C6FPdN2BF+>5$DW0M4FdyiIxyD)-;mMi7ml6eno&V zGM_bJ@8QO5;Q;2zlDsl~jjjsA{Jg`nP#T2!een1}=ouTwI<)h}CVg|CvRA3Qce;mG z!b0RMwKR6&`X?|RL1y_7+pQ!W_r$fZbOOtJ{|Di#3{!wx`51wGfD;73$qSGYaZa8M4+?_R9MFXps%H;pS7pc zNH1jvU=uBsaLAskS^8#fvEMjLc6=!B)5QYCry_|A1hT`nSFv3ipsZtfB-GO*v5dgl z+u0;5jhI7NX*~)B~AhDFo}(#cK9kPiDzE@YyfHPfdB&SP>P2uh*0}^QdrUTng$-C0zqh zxT+bufT0z@?K4H4&VO`Q%f7^{Mh2;o?EM8Re%8^QSq9jo&yn9xn>^;A~&b@crzAm>M*s&a~~9QV1bmlnYv@AvsK z7y*u3#ppF1qb^`QGhXumqs;pWcu^PJ(hOIOPwpc?i`R0nQ9h#Fc+W99$oo65f@i-j z@98ykSL#7>5BZ?(3+K<@-2*H!zYDaTliGWoSNE-RYpc@>6U&iCr)|~Jnbd2w-tsT#Y++mxA*D!;G4HGJ~vdl5j-y?PgVhX%sTZ93Ty zK4U2y_mSXsU}!j$iATOQ-5sVW^EEd+2Y74)v`_$-B!>}xzWDNsu(!M(hPpa%ml8+2 z^^R7Ei$sxBz39>>gv@Y9hxX{1=8Ks>2ZH&%}q!0$^BG_)``~M=G4_puReOI(- z@fs1vf~R0^;Vs3?FT%rz4?{bJc`YcMZfvK#)F5Y~DY_h6>S*L5~^hZ|M*B zYnn4}=a61XAGLLMhYmufn$W!(Ouo;3D84CS;CdYmyF)y6{@#X8u(z^%JBAsErMRbk zp*B~H{=NyF`YUxQY^Sl ze_LQ(WY(~uaC$;-x`DW%6f;{WUR+w;(Yd89TIzx1<55sX>X!%7!i$aX8LgjmqiN}$ zNELd3r6q`g(xR9i9in7MHy0yQmG@+MZxf)eM2hY8@lky6yoNIEqu8yjJWOkE;(;BL zlWqY%7s5Ea+C)*>fvi+rOZG$DCo9gTy{vp3H~)x;x(vd$b%hne+X&`1ra_jj5uuXb z*$Kb<{HxGH!M{&``eR(knXpBA^#SO8obeW5tE7XE0Eq)^6dTp=13r%dH!UdU3|543 zG%J9gG?vW~R!b*}(XtJBSeYu8N1lyzhKtlgZyhSjbz~8~Hp=$PvI!;um_lw%HiJM- zFu^+*V}M$45A}3+h1Ion;um&s8FG+_S?j=ksq18u4iy$uHs;V^=Ao;jSmIQjUtSIU z+|N0*LVpJ3IyyKESXd@L?pCOy?#>CWlwJbh?J%IwvgApHI4~8Zwcc`OBJu1hGaF(n zRjm(p!6sBddF_KeYUB`;wzg(`*fh2HF2XGF2L@~#SC>&C@a$(WoPzO=9+bNZhr`g! zv+1rrhtfBes{`xj04CR}`9!ABDP$YKNX;(=7co#5Ep*ACpsdyT_c1`~xrUjIDGY%* z*s1LJM2z*?hS6(c4h4`YKuZ_yv&bTFP%`X~1FXX?z;Tf=SrMHf#=!T;Fmd3Qu^dnA z7u?iOqvKmK^_^2h+Bzj+n5QK1l7~n6kRqv~y@6{sGB+3yHc6J`AT5BEYghm_r#i-}gTHR@jFzQ&?9R`;8C2ha2*v@c88m z+=DgR-{QITT5uf3J@(%CwL77uf0X&@p?)Arf&GUT2opDM5=M~@5C8DHSSrXg_Tj;y z$fD1^c^!*+ws(_-%A8aPgJ_058^qn6Bcoe^V!Hb|ih_rY%Fi>CV!!MV4{GLAn)z)- zXz3y_u-I5LYegp5AIG`v=<4V2*8m?f=$LO^)~m~N%p{BfV9P-IsR`IcLngM#3?;$f z3f()hiNmlCySt9iZ66?sWo(A-8>|OHPh*nJ7q20suOQDN(_|w&JJ$2-Voz%@?c?G# z=Ec1h%Z{Mr6q+VA95U}Rj566H^n|o|`U3NE``-I7uEx{?{E?%UGHQYqF&k1jltuBx z@w2yO&{|0bnf>NDAdvmQ{Z-5=C#O!O+5(u(!e&-re>`Kx-U)(R+UX*J0ZGTX}CJp!ydvN&zvW4MG3Q4^NW6I^{%V%HM%mxZzvbuK+Xt-H5DkJZ(J~ zs_bpoPv9$?<(KYaStwg@xgVSlHdj=*x@mo%>zD!byWcLELBnIX1@oLl!INPRXebZ^ zZMiO%3YitiXk2CQA0XH8oH#hfD`;|{wQSxRU{twJ(%LQqYp-#sJhhdRz}eCkxwSQ} zjcgaN0{~>3oD4lTdI{Pvy3822%fONPlGvgoRR`{U469Gu!u`D6a4`JGM>yLV45Hac z7TB}8Ne>Yvh-M}_!p9%p0NiXs+}N4KntA8MOJhjBdJ<+|zluFiuT2bx(&DqQ`Sl;d z5KQ`sWC3zo_9}~B)tE3l)dWT3YYjgrp3zANu=f7X{x%$>>7+osTVoR)j7$LJ{C)l8 z0ZAwr9bk)V?|n#YH(>|5HwmdY<4jxV*S*z5#R&^@i_tlO>|;6_Fqvk5?vaF+!QecH zXgNWSXOO8TPafhy+6TxrAcJ*L7t!bKoUnX<2D%c%Gbc08{rCCl zIkK+q((7l)4?@B?n;np3X7@&82rM9gcT|I86P*8j^{NEi4q#Yyv2l#=QBd2mLl#lts{6{#+(axMQ%U>` z&Ey%EaYhFyp>|&D#X>OtLZxB`;AuT=h1jr33YGFgWCsi$Zek8{Z)QUsz^fUits>at z`X;F4*B8SL3K6f;!m9Dw$0#{7N0~1KUIPl{h*A-0k|UZM09XJ7vr&!dIDoZ1Ctl%# zbv&j2Cmv-sGC|a4Zm}+DHmJ4Mj(NW8AKrvvY9@1Y0ZWKjJs3T155OiK0_9pqJ)snA zuyI?qLhE1+8mS|+JNG4q#xacO$6=R4dPWJ2vlFOwV&PhbN!19nWYrp4OAcRNVmmew(FER`QtzT zbC@0;1}vn)?_bf@?7@Ew|L#BiuR{7Ke;r;Lx($nxVRnI$0`S9V3>{+oM~eg)9m3YZt^jVldm8CF5Jg8od$TG!4RuJFHp#h zeD;LcFqKBlSVFkCr%`qaRqL_I7|0qcEV3y3;sr6cfR;|;R~H_9jpbdUw9c(Cl)4q3 zJbe?oXiCkFjb}b%St0`R(-a47;XEHwFkIH`0-%9bg`vkf2f)`Of6&65b(1JqG|s!g zvOyp#i=)!yjNAKJ<$S^(F!4uL(T3F*T+?iI%U4)Z$&AzzXTpqp>pQl=bT7@qIs~%i zc+UXFjU7xjR+n+@5KjtVG$CSZlX{98qU@`vkhNkR4eKDzifkPF{t_^$91JK=aZL?c z-!4-m02VAnLl+6DLayh#2nbbq>(xj(hixxByfJMe_p?UKOTydi&+5u(PO$qoGU ztIFJI^vVFp7DRS5j#U7{Hs|{MKaSTcJMOrS$U}jp?rpyW8!|t}GWk_ww*YA9f%v7o z0&x32jT*<}wRML&=A?TFW8f4v&^3_p6RgN)Mm@qsBCEtt-|rmp|2Q52frhN%F$Ao3B=P#rkoR;S@R0F~d1Cw~ zK{ytl%lX#;=<4mImZL6v7#!xpPsH_fPcaATpC*#3s?3iW!L=AU;{X{f32l9Z%A67= zl|!}`^YmgKrc4(AYxdPV>*fBXtPpx9bwT1C4y${2rU4z>v1CX~V+)%Ny$<8Nj*<21 z=`+AneV8Ws;e2l+lxAOrv1Y;m0BB&Z(CL%NJs_7YCnlZ?ft`ek!%paStcgV+Lm7}v!$hQsKQpj6zqv}?F5E{9su%00w~JOcDCoZK9^yP0 zFgM=E`j;Un5DQj0MR4FM>KwZovi+l>j@nzv7U)pwxRd}3CkfKtIWPMtvmES_MP>JN zkzj$AJy34Us0?dtp4RpW?n7jBv{KUup9_G5%u#-8JqAA(IpCY1bps8p^gRl=8V5kt zA+RzWgevu{&WO8d;Iqw4em_Egnjpn~;+8ygWk6&J*WU^Pb`M0aHR!?!Vg(;eEt!;1 zsVs(!g_WOIvBo_X3-nTxO&KGz=I-=Z=wYBG{Qs+u=>)*F-=|E*G1sshOM%Z>h5HDL z>db5LsLf=Z+P%Q~Q7PD%RyK00)+xWsNXRhh<4-ZCN6^x1SetNMd<*RO9GfpvryLlC zvxnb6NtLthhY%SkMyx&J6YLaFVIyXF--kEvIMcW_tpkP?i#rWbj)1BOTMgn0)>H!J z-NTq<D2rv0zy;X=0I2Jog2m~G)^I0m zUx{Zw4sB6J1@DY?P~ZOpzQW{{*Ez{?n+0AKv`e+a9uU&BgGuqdk-T~P_Lww@yUqiSH7&1)+d*->M{m|k=SmM+!LPdH~XwEJ6RY`=I!!ieDv zuoTpKI-*rs7KPr|nhxVCgih|=)=0?`YW@+XAb>0Ft%hzuzZKPi+Ke4wh(L_)B`cLj z0mD9F|Ecs;crxiKB-u1wn+9}y?ge?q?HT)HQB9VXu>j^!fM&S0!wT_%SYD}~eXMY= zz<|h{%8p~bM5Of zKF+mp?js=L16)f0C+NJ4j}uh6NBpVqhs_Fm@J=dG=84WJ6vnu&w< zb$|DM5h${5>Ii4exV&1dUTvTFKIL$>B^!DN2VlcXVgAh$F>d^OxDyt$N^;dYv0aq> zAt6eGypx!qzx>I!87JWuTaYQpkyGTN1cI#ATWW2ZPvtsS}vZP0TXlHtak4`LR) zbuMjBd0Y{&a`w^bb}eYfQ+9|~C}zAfkDgl%(N6wm)_Dn!g|*#I(JLwRTMHnyf((lN zcsZz9k{s;eYh0kA3of<*&e+emule2mAxLzsqFb5!D02)wXv6^T8yEtZ5<|`X;-2Jv zGTeH>TnE?1n7Jg^k*G4uy4{Ka{8To@x`+JDV%vuCL^;6ziitXG4ELyCZezzwA`pUu z7t9c~2jvAD$yz(d49NCPWdvQ0;6JlFbT8P0)b#T*0&<5W0TTyybscdyjt~mX{o|nu zY#vPW%HKFX3X)Nr%c`{hhq&Ox3jeu z#%PXxwXdC)Ak>n3@-Sw%e84>ufZc5XTnCE4%FJDWszK_ywQ$1fQT`3YPFcFZ!l0Hr z=_H^s~ExsgEY3IUanuE!3Z zuO3UyBDQA27Km+XWz%7Mk)`;SYT4_64%VDCYr|Ee#bE5gF?~}NeA38T3<*F3e$E!bO#VNlSx!% z?K(R+16rf3Lo+qgnpvY}f@WJRqk&RxXZ~=+ghSjIogCDc4}J%8yhiXWt_=iMf%Q6V z>J5FgI@1tZBLzM`Z(&OkAJ$jHt4BNoz}BL-0_-}3(l!!sqXgJoe;qT`NTI=&wNx(SD?~uqr zrwm;FhZtAXHHPsc)092bYN-IaT3qi#jvNvA%s$DI;jD1hIz2!ryO$d|&(6yp?5lvR z9;IIkqcG{ArhIje37m98o}q*au(WB6DLc(HAb(_9b>}5rPh!mM;Gkvx=J{*Di1A@6 z92J$?%ybtPN%-waiohu0GeoM4Ze#dpqgpTo48 zt?0~{IO3R&*sD=T02E~JoJhSyvSk_&3C^zpK9;zV$q=B~O~ZIKggMX5XON95KtCfR zeStiZomBSu9-D`|SAl489oRBlPry|(24qPDhXUCeu$E~NAOfgVvRp%BC9<5(4X;^m z&iOtCaG0#(9_0xM5gqG+>l^ytY^oj0oD*Y!Tvl%$C-Fi0k_zB!M^fkWTD_#lG<97+=jh~*AV%5h>>~> zaixB*XUw)G;}}O4`v}9F!@+&oht{OA^n_{s5mBOqt2~_b#~_Acv+iybrUz~pw~iU7 zG9*nb+%WZEY^|eWef91QeaBW{ww4iUD0T$-=*%4$hb!S6R!%Ckqk}s3x;VL(0>)yL zm=i&eT4KX_` zLI)OTi3t#7{rp$IggUlF3R|AVKB6_z1&fzpa`r*Q1)j%vi47Fn;9xJ-hgN?ow6|yI zM8Fu49rje)!7W-su$x^nIWvu49@h?Yv%Ry+#sr{|p(r)GLx#)KFTW;cXfVv+3MwNk z>QVNNN9%EVs0-!qDXYMg;ocN5vH?oWy;*c4Fnhz48|lJrd&Wj+b5j9^7KkTV&I6>e z9&9Y1MyYF^$Liw%Z^5^2&@8Ue1c&q(P--HHc`d8m4W4tvST;LX9%&Wu*s2JKYAf7*eM%C}TQ84#o9aa!r&)>d|9Trl> zTo`82?oEz{Lq3;ho-T%`!yhU3e2wb=PyXTm2^nH38hHKY4}Tnf3pjprPz5CPk?P(8 z3ym;90H`SUF%pj1w**bvb$2(ZZn-sV^vU7uA9hg|^Y62h+z z;rjf+=eQeghvhdfiMb=r2bb}}^QWZN=g5*qDBb;p%&InG9ro#pfi-+y3?JN?;gD}3 z2yg=c@JnQ~Zmv+yhVczg&%nge?{)GvDe%)_b~O)6HjPmQz(rWB^4`y%zX)%Z7eaSO zefZ>7UpQNT7K#fm!ks%GhMi+#S`%b)@!k{K-L;{h)26q_>}-`93<{7%1?l zNvQFFG1cAEMe$x*5^b;0<24_iJ^qUQkOeFfbWE(_DK(k$Bn@qD7s594Yq^d)fcH;- z_*rP~?WEhqFc$fHFhs;~@@?mgMr4lXQ2-)Xvm}wMiwZO{U&s-)oGgjJ%2+0i7=xDm z>7R{r5&)?{w^0&ntE=I?`ycVd=oG$X=VUcxt=#AP#IUHe+c3fmVbF3`Xl<>n#rs~e zTzs^4Ui}40rm@vZEZZfkB&*}uVR#W#8TcyOv7^YM0{chsX9MWY!p~p&MS=c`*i1hS2w121?7$a9h62QU?$-wzK#d-KVwS zcjq;NHDrcBMP+8%{r-#~lb=42tcmmNbNgjIIFk;PYL6#xLhf7iMJNxVC`MWSF?G_`743rBOBr$m4yumtX$wd^%sgM^~D^dp(~& z0zCd5*O2?yk`VYHw+1rCb<<#Uoc_&m%M?c!G1v7qU2FpOny~_?MZU(Gp&0c5U*hlfb0B1{G2u|fiqjFtemg3)y6`f%uno!ea90L&4b z#52~DQF=t&-s<9u=q2mHh&duQ`EY$DRJS*XYb6#G0>yT3?vX0GzX#p;_T5}e3bXsu zDp`Bizx`Qgo4gfrXXted=LNvz!!I9UFsbXPvM@Xj03%L}nx0jRcFXPvoGN%RhQ}r% zJJpP=F}&f%?dudD9-u4Ie0+WrZ`9i@RAz|fe3;f)s*8EsMelo7#0{>q`#c&a7y&Z$ zwY4%~fivJ<5wtOn>PM4|TzmKFr8_%@2e7wH$cS}qSv?axvcTXAcFQ=)}DS0y}o0nRI)Sx`BE??UpzB;tyY9(USr_ zHWU(Us+0@_R`pJ9GYob&hZ|Q1!ayJHpdNxHEuDfpQuUC_Z^tBrPK0hN7Gxhmp@Rb> zp|h7XYMy_N`P{<_Tc$jP9V}`^1rh8VbmJ}xxReh8nx||u$D>?shGDg|$S`A0VCi+h zEcDUt%4X*K(Bvmrgh#AZfoHSGY}x;4v9n1mzFE&kNwFal9Pr7$)^KmCKa3H8Hq?F^ zrumu0O?dd`%g}>6vy;70WWLQjYer}s;0`IW!JjRyQ(}a;vW{T@1O*oj+DF*fNI1*0 znjEnBRz*&|t+nm2HaAO4v;J`R!&}4_5(ukJjS!HvOCscd`A`1{>w;qA(5Y}Og>r;z zu!&5><;Q;v8xMXPCIR3btjBtR_3EDtZwMMLv4>|L(H((9`(|l1On>huAvrt~R*wL! z6pNHSvhZ{h_wio)to6fpi=m29P@st8Nvf55lH39FVk`+qC%*rjXVEQ}W*@f_c-}|R z#%i%dG60Mx8NlEB=%Y|&?Gx08tD|+*@gA;tVn0q1SkqVU(5n^8a&Idn_E(7;Sq_ub z2)s!_2!lk|dswD##FkFG1wg;;MYS-X(tiz0!C3_yJ$6gc^ z$$l9di9!Kn^bT~fM@T>+w(YB5{eq+ z#I_>LP@dbYw@uyam>b(-$s!x$l%=bHj91LwWPM~g1Sl$8iwmpltkI_sG``Tfa7WCm zLG$dU$WRqX!a2Zj%jlv&HkR3Qh-vNx+kl+WsEpkOxF?H994*g`Y_k4i1IADi)9#-1Q8ZE7C_r!MCBgKjd1N)=7RUh+L(-EY@lv$A4j*Evt}Hy073BKv&c&L zCCJf*bV%%>tfCB1gg88x^Qqgl23i3X++P$Tz!s=BV3>00c&#(Wq$2ghF-{1>lZ^xO zf!BsH`I<^ofa90OsDQ`MM>xgjF2S#JqcZjH&ZA$KK5?mHmM57f(P|f?e3IF6L|NRqQjQ93EKBMy-zqns?sWIoSGlPuJatHwfbdP%hJBhQV+gSHA z&!9ZBtP}vDwWEh?QR;~p+&21C_t8CJaER^(BtW&&=Bl51X4awH#}MowgVc6w`OOs! ztwi|v{RxhHA%c?x`lW$0r7Jv#r(tdWEqcq22tBlGJPupmd>v9G8J<_jZbm;?!)d=Kgm6ojVy}4Vx0g&+@`NMi*oW zIKyPeT5)IP0E|BWj#?s*sjcUXLdja`} zTqw)l5yBu)`_7L(!!vR#w(=6kWQZla+&8Q`pJk6B4}7jx2sCR}dM{;=E_?Th21Kf} zhb)i-{M9Hoi{`d)_$@b)VsqHKK^wpm zbV4=zB*4v#$qN9}F$=jxj1S_3!~_lt8!PRBWuw{lCU$|qls8aVx)PI?2dDG^!quOm zT)&Ku05Gz93U`kIa}Jlh9D7S}K~*VY`!!P?@cUN#cGExz!1kk1tRJ_RK4)D2X4sz&*0a`0Hr>)$6+B zejg$j-i+M(jMexl*Fi>M6BK?DAKKD7Zr{P7FnRwLb>5o83P9@EEKtUuCDnG7mJ3wxEHndL8P#z*r+!yV; z0Dc{5O0od1F37SntJE-og`I*z*gc99=W!cS#JIl+c9UmA=MaBLEC>On z$AAn4i-{5`WA&KRt&J6wVFT=AbN3F2EYmv6BWCa_GM%QFP999*>k!~VtEpN z8oOFW;>ygKoFP+@^e;`8$k#!CP<&!(GBqc^fdyzW@E?k$k4VI<2h!{D`y$??l^ zdQX9JbRBUopG_9X`PcRD`;E^PJV!8wY`4TrKk~4SUfD6>0{vn^Vb`2X#rGa#JTw9L z{i?BP$baTmO%%t-y4JuS@ApG|$Dhs}@AExgTeqX*iL9nN!q555Iy+Xujf&adWs-Gi zMRmgU;@%1;&SZvh82WSo zNpL~U&1Ra=pAU``w+xYyZVHw!Szg)ZtS9j>Opz!+Y&&O97H@+RRT?HH>9yjiDnY_@ z*AWKA{K;nfQv=71V$IM7lCp*g2Z@y)*ynyJBTw*PsUwYFZN}J^{Bd9Vd1FPH?l^yO z1&~v*uP>O*`F3t4O?YEmYqk7JO1CB^45KYRgl{qX04t!HbS4`p7`mIPwD#$n_DG;%hi<2-SvR&s~k74pz z4h{D*V^r6loe#_~wP`J}%YZB<8HK8#=lt-+LE$+PfX)Ma&6_X|dBA2HpsEg5ZRW;g zxJ_)(@aS-8=%XQdCyl@fxL@Bt3Xh-9#YBk=N_}uJ3zDa~dxwI-HcQ#IOe3W&fS0UnvZ zMt}~}ghC;uoZoHAD>Gn?Dw!qMc3@(fav(Rt{PsSs?mjYmWbDZ7Bh}uzSYUqYhau?5 z@*!-EO-4N~lw&;rrAjR12`Eu)jvn5Qzt}uP5yFa(;t4Z`ArBL8zS!%Z>E~1miL@OUwzZ zSU(29ITmUYU@ilwuR{rIxg}`OPRefxB#Ep&`fca}*lGaH+`LJ~DxiUDOUw{5IJbU&g058MOKK&Lgr;+d*gaILr5OO zniVuEm@BjVWW5APDTIy23`F92k!2PT z+PEBQIt;mhfrZj7qU0-_?HvyJ%cZrjw6!0)hsVPR0KJ(KGWG0r<%joTi^p}-NVICa zawxJw3U}m(zbgQ>!fP#TIEE@4&WcLU_nS4_gh85&;1>C#B1fjk8t4Jg3N};U>$PB{ zq#FRPD93Hw8S90KfFOcu#*ij#{}T zF3z*^0&~OtkZZn z>Rj`Hab9AE9iI^l+gs8&A(=ztuw5sg-|GoRt#_tEi!1~R)%m()S0X@HxiNcYI10}E ztS+z1=X92NDL%6RO$M&^ewY~6NLRl~+kFth9S^Q@_iw&08W@Zvy3&x7&WZCEpXX2G z^xuNH27tcnEJNd(ItGECfZqufpk3|>pU3?u2=`kctQ}LE_jv2LCwv~C+5Y8Kz`V?= ze2;8pjqP>r87F`D`nuV8$VdoSnnngDjmO6_TAe-JVd>ohUXT{_8wJ%dRIQCE7%A~H zzd09fOb>;-G^*c4UfKIw_h1WhC`Y)-1;6jEQ&wd1S{TB^v-$k1klI?N=QeT;RAb>~ zI~lB7a}*sWd13v?WbbI`zVQ)l@`i~s?jvlYh4z(CV9>UCt`-=lA>!b!GFI4q+^DO| zT$3}thhDsL^G1wMb1!M@-c zop=rf_2-a7vF0D^BT$qLSMM6iU^y9$6|d`_mqCq21M_aP1H(0Bt{sQ5b)FZm;q~eP zr@Ff}c=(UY8oGyu(K>)mjNc~uu{%dC$QwNbe%r6sd)Psu!rAp1W#t8RmZ*uD+ng)e zgXI76KmD)a>DNzjgHD9M`!D`3{MAqXD%`*SEhcazWH@+csTh!)gcUf#=UkCVgRy9a zG3#L?G!Um$E#eHNdy&ZgW3|T*A6U*HM54Oanh<#LE#^SprfQzv(TWk~? zl^0PGE}Ar{Zk=-!!7;+E4`nsKI1k%(fFFF5TQ`Q=-}yA${n1~Bw$W>$fq-Zt8c`lc z#0hkS_dgz^as$uXOBn+KF&{mBjN*qyV%&dxNd_u`pl4*r9ROYm2#aIxb%e69jvIrf z2<}!`0E~2?xMaD`$lhI8*hZ-tsHij%1UKN8q_`lgR2y+>iNpP{L44T-fN^7Em0)e4 z8Y$EJST8GpzY@I?chMEEw<%uBvnMI-v1y5o2#8QLjH8SIWGUF3#+PtS4*4H{{g0sm zVEobhAA~~yTkGIR1aj<1SPfl9FppRZhr}M70Ya-Nt!)@M;}%U|vFB@&?4RluEZ#R!byHEcq;K@FEJ) zn5D^q5weJ@VSAY}7p&XsuO5X5fBZbu)ArT z0cVDcf1ld0O-6FMl&cpa9psA^gox|AN|y-53w+VPay2SXbIuku18TDY)5M&!LAl>LjfB?fcy4Q{W0t5Di z06~Tg3j=v1OKy2IQ1^7#)>W0YWLBXC) zmjD1j07*naRNuRt^ED;?VFai~=oAr}CJr5-APinFdS-Dg zOkBT5=Z_JLGZ=dI-WlI-g*`2iHC^Bw75OvIzN;ZdrwW5&iGBa+7hhwPU^#RCFHl$) zY_0@h9A)Wj_8DS1Dq+hg2x?TIw#tcB^aW5@x5ikR`Dk1X`ZEIQYOY}{o%QDgM+#*A zM*y|OodIqr#3but$%ze^VA;NXSb}4&6&q9|ldSSC*wAh1ORylYQQp0=)CCiQ1X&{8 zs48cE#fSMUpN~L*YioFUHf2x@u#fj(Tq^yT(Z(DYL#uLm!{)qk>v*p9DtKSrKq?_$ z?k8hwkPch}#{PQ-%JLTq4FPF1!nt28!Qp&4C#;{!Mc1Z%0Cqr$zb^}%YZqPc#ORqT z#v7j%j{vd1x)kj}Er78Eiq}w?2`ntXA{%P5gYkTI-V1P}i`Wnj?OCT-4soVo##rT~I=hUwYjQ;(j=<0O+9lzsYcH*=8Z`adT z1uVGj*aysKjLmYr8IRAwhOdA`9!+2=`1ZSH*1j2l(g5%`CQcW&Omu5!J31FPB5~1Z z)L391MM@|DK*2r6*ouTR)KPulmp{4}PB9j&^LPmv(nHmUA{puXtli4e0wl;)+^H8~ z{kzY?;q0SO0%Qr?@*2RfVB5%osqt}$7raZ9$7&s&3WIk)47>E)=|tXcWcR}Klf|%} zCGNgVThHc^&^>S?Y!LFoHi9G~rjGraB~$&{y|+kU)E$rE1Q^Y(tTUe#!^JK<)(}lg z{xHw20Md&4yTY>Su9vCQv)~@}Jh;y7$R#m?%%ldW>*$%>2e6*9w{^kVW7&0!OgL+4 zESh`Ef6KT^n0Q@_Ya0R}NOwQQvPG<4tQp9hnqk^Nn2B;jdDJyPC=+2evBL;wPO#>6 zzG*f*|$*bZ0zx(GL zIEw5({*v+%&tsBA_gG7~a;-ls%;5f{s(skQ1%%71>k@Ex?-ojlK5vvS>1yhykm7Dw z!}ZpJ8>JCtZrq*+@0h}qOXq7b*m)nE+hkp0?5@A zwiP!C^VJHPlrD5JAdS|dWxDXGPUg0~Y zFJ|vRNKf614DsyD6PnbwqA&q59OlPgeoibWmM`w(ZUSOkdG0;p$X1Ej8v>LsY~@4W z_!uQ;$WjFbeD#c$Sj@@U?nY>GACq`c-_u8o7cFW$C%837dj?=ODK<%bXMf88uF?5$ z|N3a?h1t73btU}f;fv7Ze!`8IZW|1nfV_5+hW2@$rwc2g3ZdMa;ol|>Fb6GIwVTW% zcV+F#t|GX;pYj`P%;PqNOKr3D^3j*!?HdE^pY^b|z6cxB7M?$P7PKJT5O5R$chxRG(Aat2k7 z3O=(kyK#pXjooAqn5DeBwo1(26~M(g!aEnHdMB9gdW0Q-3RYOA$GypMh%K-3^!=-< zp*lJB3y(Oz+C9UZ*O@0zq8v#u=3`i4+d@TQz`DhR9)vF+je=g`EPK(!+?@g19AiEA z%o|?_t1p9LZ%qZgAXXg6FNpIKba7dPrixqzQV`?M22AGcSxuHzV)KAPH3be*uwaYJT~C@vmdlHdw~f~=>2$9eJk{>6HgTs8s$vjyLPkXU~V2F)VK7O;+B1qCRm z7UagbGcb)6=q*3ubHEgEf7n6fp|wJN`OFotgFHjw#)9z7Pi#?y%!~8Qv&-rUj=Yu% z)#9wqe`P&{ffc#z=BUIs0dRRgDrV+HU?vc?<&l3HM_5IisB`R^c^${$pKdtk&vMSq;b0h=EGcXQKLy4h?>fwybowX4;yyaa9~{;iv^3%S@Ki81qv`9@D+1ZE)_fc20a zxH;)dT!()Di(ma5gN=_sp;n zi&{Gt6QyL-4C$yvhV=9^cIImbxAZ0X6+(J-Mh=M@cEZ`uBdd+q;?E-5TiC6 zHokim4pvR~-Nx^pA)E3VZju9He-dGCW)lFG3row1aQodWq&atnhk%MYSh`l8>40EN z&ODD-6EXytTXQN$R$baUR?Mw{%6OT5;zD+*a?bR_XdnpKMGBf`6pQ5o5=mI9>O$Cp zky{23Sxc{{YcS-9VOl5lu+8eYe9yLFNr1&A*Nx1AXrAZw*$uEiWRnFL8TRke+G@Dg&w=3Y%NPvZ z0LpgedJm!51YlZu@-5k%)UvxumgiYp*qV7s`4$pMx;w&aKwOT4+%Pg0`p0ikgN%&g zcd0xclCN@ECGR?Gsl#=iqYfcVM(^k_`vV4)`_|$jAL;1` z3xE|Fg%&z4G&N;#bwZirD$upPi90b-MyT=oX5u_d7Rb>1bc6WOnnX+3%5H_J>o>wY zUK9=^=x2kC?Vrx_TV&3<#_?DSQMeduyBX(bhje|5ubYr7jLuXBis-wlJo) zqkH}8MHm?w4SSr2Q8I`dS^<2x@(&RZ?VQci>|!V`wvh>j03xQ()@=%?0?$yFY76zP zeZ+DS44us)7bvAejlCj@s+^z(oR&O`kezviVjz*n+JKy|F6P#Z)MKtyIH9L8Y;7!S z064UOC0@TFyhTG~4a zZeEGPY=?3|Nq|oht5zYWVX3>+ABDQ?dzy1qJc6N))-z=}WSupqcvh8hg#$sUa-PiI zQ!yh^QC0?W0D9C0D2*xv{wZh07~X?DTcZ)6%;vL8jU`G{E*9Rdq7G$belRIjxR$rj z0*YXQy&VAwMx@CYDtND}_Lp6XtN_0|;=Bq>V=-jLk%q0cG>Z(2ERZr>7Qvxe9wLc+ zOd&gbL#D_3M^*+P!w{l^P|+AGt8%C$6F3%F9X=p2 zKCm~LkXRx^Aj3E+N=-K;bHx243&QVJgk}WB5AiSKotyDElwd8DeU=%iqlAd-EmLRQ zlu0{g4%+mdp|d93TgHO_jH@%|sh(KWBLI*fMz^j;gU=wNC?n=P>#8^2%s~rW9i#Hg zc@u=`1}nR#;(9PbWLSc5n#gQ%P2=f&83yF{onxOv7nyNY#^~u5_VcI61>MmWq-VQC z)Vi2t$ATura(9|AJux|@$TF91^!MDU*Mug~Mk zawUu)KnL>GFbGD)yufA(`Wb*rRLgo!OpQmdDKKd#=2alJjQnVB?G3xDD};0$&}W$z zj{uU~&N2GWATW$E^mP_FDVQ$+Cej!lMQVo@R^~$yqY(3o@u6D*geT4`XZR4$(FtCm z6TC+q{Wro9Z5W%o>cZ^YYM6UE3jshvBLv07^;@BVX7nadO+SAO5IYLDAG}4!ku+W~ zx*ZS#C4*H@tegaq1;F*5l_{EDWL?@isou~{d7!mQ4Bwgio_W<9vA55uWFNTJm&jDz z{IQ=Y$%$Q5pU{Xi6E~5hya&(8HegvWve4riN5=rzIxz=tm<%F2?fP;IBG%AIaBkeg zaot(dec}L3yzv~Lu*U_o-p9S`xyg};cCCEPO&!~>QK?x8}i1N83_YNC%8&?sc}c4 z!(8SfjNTmP6kLYe(3fiqWZY6m=Hr8%@XNpWLAd{uTj3CkV(syIsD>T71Z`>I?qlsY zAZU$8IpfgRvruQw1&VV3fM7;#9w5h*F@Kc|HVf6dNo51l-A$o~L%R%Eeum;)pptY~ zTOTf{1Wd~YOcuS}04c`mC~I8!OV&@e)8c|2+?c7!##eaQDWa$_z=d?QyBUF96=gvB zi38gQbS2nC^{w2S!2RlT+$7B4Cg7wKX3Xo$g6bAYL6JM&3#|{pUIw60%K@w_!Bk>B zln#j-Ail2%B5&bFMc8d!AsRwkODl9H*$UUM{xtO8dpB$>(u214EM#APhY&)jqK{VA)|h)Lk>f@j zzeY{D!S=9xPG%TmY(ePe7oLPhl01qSK6zNhp5_`DrFU^Ic!2x0cQMAN~s*Gy^`_&{{SeI&VhwGq#WxEp0kBKuW!W9Ot z-KBKEe4yQMj3E;&SlXx!(0F=G3of4T_;?#D{5F}*Dn_kZR2IhKuHXg$XiwuUnC;f@ zK}f8}#a^vsasoCuZz>#RfB-}mq?UbPR$e6z3I)r3t?y>SK&-Hi0C%%mq*JeffKv8D z&{bh`*w@5VswBR7NY7hZaplO!g?x^}K7uusEOJp`QHvsD(?npSe2gp%YFCBL$6_$G zjFs*ei+rkZk_Iock7T^894`<21wyb!c7~e^z%-n_zIDC0dj$+|FT3JY zfD4n&_aor)Q1gm$ZBZY{;Gyt%UvQz^V#?w?2~4A#PQ#n=31lJ@#YHQD^h#U*+3p`ZDSpgpE0x0wdCIwhLL>%u(bAKyob-t=M%_i=eDe_`hHGi z#A3Gx?&F$zedVb?epaS5wyxrRnbN3lINLsd1z0*3=FfA1KDcBL$Gzk<0`Pe%)`?7 z+1IdTJ04uXH6aWTIW-t>1>(-#?fX`4C^v*3{`wz;OOnZ|>Rag=fxJInWiB|!Jj2AD zyTo^Okqw&*b91kv5j1>tk}_mhc%CvPhMt7}!4b|LH5QTe?sE(9?@-38iFQM0JX=@K zeddBW0TqQg(nb@7z^G|+9>^UqRjJ5_+PMzJ=0TFAacBQ_q28r@U z#@2zym`3E1NihCf23}@1ZZ!6R#*F(of;aaf1}ULabW$L5uQfw(Hk-~CH*O{SW4alo zkejjgQuxsGDE3X_RaWqouk8S=V57DGb8Q48-nex?OkTf3o2{1EcWh*wEKR<%i@;mV zZHI0A$)*sWv&jrJFQUlo7?4Kzw5sn*7X(W8^DI?tn&K{r1?$C-o!e(2$UN(q z2v4%Gj2-<`3O}R&a<}rZGAW3cny9slG zCP=_cxcIVWTI~C-a*`b|F=K5VhRGqJe;v%xF+x}|)5hUwWS^`)dKzv6dPaw7 z)Q-iqc4#d#fDH#t9j?ca!9fN8g6|7j&ASZdx6TPp2*&x`1qF7ic8~vo<>& zo>N?MhXj&iN{UpIIIy)z35+!WBVUT zxL^)1puR7v3Ffap#+qD+AhDj!U}0R@R~5jG(b3R^pqT#TZ)4nE6422`0qG>*-k3q` z9@wyzklWi1*X~p6jXsO}SZHk%H^R_^Uxv;b4?>bQVriJZ+gERfrPm7tACo;j|03j8 zUcjak8-v`^wQvkbzsN3yE|NSFdlXiF^vC!dD#kfdt5h~ipQ4INh8AHOrRxtg;TqEZGN9;K?shDi}M&zH|5=yXjTzlFJ7ugHMK zxnL&vy&%_eAhMrtKI^>7{Q15=@v{71%y8sALwCHV6&WlF9T_-`6K0NoV}LX7!dKVR zYx#=f{C=(lRbH!d9m}v-j}W$qd8)9D&WYa#xbdGElDc6l|5n1vN>{QGa28X$aw zInp?yA_97=fT3yayjr{(COp*ucx1NwNBZ%a(G>~ZnaL15k2h!+mdygv%P+nS!@Z5+ z!CO-?_Bw^J*SR$zn&(k^QU01woJbGmU11b%EG<*U*j;@p(a1Xm5^&YBc zm_YWGzmu0_&!BJ9gfKK<*}CsM6UJuqRrGab&qmq6xijd;_dRd^oFOYuF)$^)1d>&( zkIe_%fEs=o;-gIhX>82Ia1rO?2-*7h^WTNl*Dp9nc+m){`RsR}ghRrpWYjgfG^WH| z6jH9dnB`);i*9h&+x=E;M}#c`0QZT_v7CUcm<#W_TIRcetvoKob294c zL63b5uDi3BphzmXgC1?CG)G&iM}U{lpU>d30_~&J4q@mB+6kG-o8iCvAO2a$j1Gkf zz`^yQ5z0s42IZlXOmZy~UPADnQ<|ZQK9B3P)LH!Y%P{}lm(;7;4m~5+!}Yg*i2ryv z3cB_V;=rg!)j?SW+k17QtYdcw7C|cYl3>iJ)v(eH>fgk5Xi?T(SwaM18n=~k76Ok9 zF=ZBYHQr83Fw5tc$+B$3g;WJ7IU%MfSxq(xOp8`{9n8wBM~h5wcld|@1LC{I&wVG@LXl-NVTAESJD6|GPdIt=kF*nAk z*lUiZh;~<=MKi|6AN}<J%FwKh!cpCWA8A*_mXXLpMd&zB}1>a+xqA`?_sqM*z`^E~&?Gk+x% zc){#K6x$I(>X?m-2n}cK?4AMKYs{&2<#za4r-TKyTEaKPRW=dZcIVazoiYeyrBH2w zM3-%VqtHg7)PRr|tZK0~5|_{jAU@<^FJK^CA|zD!8P?o5n54o?mTqM|)NtTa3PPS~ zb73yry9!7lYjd0ZIy*NTPGEbO<{0SzEy(}_FhNxmjoq60j$iH*|M5Hw5B6YGz{HXX z{8xYZuW((iQtYw$+oy4KGSn?(PVJ}fi9=HzTI1C7W{KvnEEM1xXP@v02Daio&u?^SxDMk@7mm{R> z58(#k&_#FX!A>^xk*PG8E`|K`SD}Gks@*LJ5B^g&$OD^j+m8e^-UOzQ&Y{VXi5E)Yw_{zo|U_w`d3ijE68x;acg z32B(kb~0(1Y$~gp$xYE}l(;b?!!VMQgc|57J_AVaV>$0qJL>Hpz8{{ve1fon{zu3h zli*Op+_ka?Y?&o{b%?B)1jwGS?~2HpSMzfuVz8$LpPauM=8<0mEGn&N1WC4y(&|rR zJSdP`GZ;ebANxScK$vh<&H3GC%}+2MOg@NiF{W47mTTaTLR~>vjT}>OU!vFqz?HZp zm?p;Pk7@iT?0WLpTdNkV;d|1lcKQGH_jR z69dgryn+Pxyn{SMtnsT$$+5`7nP?H?1DPa2x8T!PbnSA_3h=6o$KTsLyaIA7Oq}3WQ00HoO~wsf z7ZX1si2t7DhNTB^2m<{uYv&xfFPsOJxjzB`Gaw6uDHysW8IU66!8Pn1 z800f_Nw{?r!+$rto|^`!T#{@xM!{*yc0gLx0I(Cqz3>mn_&w$*8ZxFfK9w zo2%K7C6o-$B>*;?=R| zFkg+O!Sqm7Q)NIeryyTfa3{pqHpPT3X(H$ph;vK!1M`THs&VBWVOadhJw1!ev*S7> zOwDdd30+Z1+>&k-v&ZVw1OtQgz@R824j31_fO_6$V<^ND03+KWSwqQkAQt_T(rrVi zreT=M03&QI4yYhWtD+`6r7&N2{{-v?S#_xHW)NuugA%nKGYDhn47}EYZy+l!4ZBcJ zM#sMNFB|sQ7#PN5s=rTD1f+QMk9hDB1!P+&!C@>)4VzX=JvM~}-w5)gffFdyQ~cO_ zutW(Mz*Z)Z*#fM3aG>IGi6*)4iJ<3Ypb_tla~_mWmxOVo%XK-YK*V|+U3>&XM+rlema?$`sAc(p=_18BU32V5bdC1@s! z{UX!2g)N$Ftvl&n*$&5|@=GyqwfwFfMO}~e(Ly;C4+df%w=#l)^*LoO$@M~s!J+~> zFG2K;fGf+h90T^Yqsz&1BVk=bWmE_kL$L*s@#!eCei#Em6LuLf+?u*c3lEGT>1oK<7V zRRM$=0U!bkY43h&(`~RPkxej{6=uPjew$mnfRF@Pq15I9SZ!{fki;@czf+PDNMtC% zSef{-2h*2_>b|U@sy_4g(ZBiE;r=Uvo0g@J#?uF?wTeFS% zJBRr+rmQWMA-l3nUAKeKfj~T`23|EamvSpJVU5_X1KuCT#i=4@bH@GC(?5VwlL2&O zW1;NAQI4|$3;Ndk)WFMxd4%mTS)9qXQfPt2v(D4E&tH;=(M9OH5VW_Ew31y0WFnZEw@#9!axkW;eVAXE0D*iFft^ZGw9*(ZT*ODXjJ0Z_#myQE zG_3E|fB{qHJXsZ9un@a1a8BAgVFCq!D0PdNdd_tF2{57(WTs=TH;5VQ%koFH#4}!&DQ&<#jGdBovt}F; z1;MO>mobrPhot~GMYow!TOsYg`Mcxw=S{$}b76d68;rJKpoyyPC%RAgtRO!)N@CAsak8>xCOH$~Ny)Q5q^H<+O9;OMqjXjjp`NqM-x?eF8({Y<+$e zsJ%J8Qr%-$Y80EWoL}aI|ad(oZ z+z)cy_3)YN_-+KM-;dGnN3e;^jf-W$&F`2k#si685aTm3zzY#<^yXRrVp* z;JU6ow~g;tbS2ke-7CNJtA0og0kGho0!M!(>l+zp#_3)%-mmr&-GE+>2^2iHy-UqL ztwm=PAxUJBae2jJvXTcQ!?dYGcCO$OEg@rSFAl=r|23tWBpSX4#IE@zap=}4vqonP zhN^whliY8GzNbgXqFa5mufs#pKsXS{lA4^gbco4?I^tmartXI_-n|22BT!ky)-Pk^ zY*L160=WdE$JzIq+a#zpx3`1`AO0kI$_g01k3Rbhmo4Ec=w91l?&D$LsThxpEMY;$ z%(E}d%xovtR`4YI=s9jA{@r~SGf`b9WQ{Ky$-AG`r*=mYyj$y3V<9H_d=LC0JIV$q z;K{3^lu7K`!komg3&9^}+Pbq9BSRxcnNryskv+pT+Sxx6)=uVQT>e6tWo1o0%Ull^uxHWrFEIxSK1(4*Ha;3xJRkS<$9TGPbbDfReVIcjAYz-a z@PeQyWVoOobnC?6POj_5G%d_V?dYYrIg-%RNdSVvYJxHekrHB3T^$rF*EaL*d7*9H!qY@>~E8*uA2 z@V;4_K1V&D;|(s2FhZbIvA`)dnni(4zBpvVef+=wA2DmSi@2)Q`Ss9EfO;o&^0raF zI{=UTw(j7L=<>|NkhKi8l6lx0UcQ*7eFmAA#Qi0i-&Qt|LgR$%DsXGqh$Cd;WVuHz zLNCE#Rhk$VxPuZe${xTN#@HP;XUF(Rc)dXf2doJ*KlceLO;d`(9)cGvl9_$Rl{EsW zbsz5`ST5O6?SQ9Dn$1gXw@Uzr)#i6$d(y;=B?rdB94t*T(;d2?-B)0=%=~EqyzK*o zJtU_b9$QW&_({W3>AI`|$aeOO#CbjgbS-V|hl$&F5ay?0ah5Dflx`C-4K;v;F2KUV zIweqG<1D?R`@ld>>(E)dayTMFNXG!5dv)Qgs(n+$TO_N-C5qJu*u?GVFAuU6x1(iCMqj(UB!u*q12)RJx~3!qp%k1H%*pLWSzOoDgylgNRv|R@QO{5O1N} z<0_0L&bNv^2BNbT5jZLPSPQLmLE{NVq>9uA(yCrTj9NH;kAK%Q)}aaJ3uPzpQHIL= zq{&+MdIqJd$o;)?QCE%`))5FGb7OG3W8kcNeHpn}u$D8+HzJ6%^_KTHyB2g0u&D@i z<%aH7!D3AWe*!M$HDHA2;I0gnA5M-5O%~c!VaPN_Aj6m=tHgbMIp1%Jn>rT3g04c1 zg(R%6z(*E9SED5sR6vG7sAwvQ02=NXpxobqsN5o0adFji-~U+!>!5+^dyULro{~Z) ze#q{rhvGQ=49~|u;~W`R=-4aIj4WXstVVoR#~$aG_XTF&NRY>a#D7aTfiPC2kSEe_{zWhD)N(i@*^{Pj=MI31%5BnWPe-lt3HUe?^y5h1-b&h}I$eIu>Ezw(Bots+ApbjhGGtAYv5)-R?xBNGVN_7 zP#~WOqq3Hx#)4qa)vf^mpG&y#Z`(Zz@Zw(L97NYSR*(jn>+Co^BPKp+_>?7BFn}#E zth@CDquktpV}NKYj$&Nec zb~{S^_`!W#L0BdTj6)XdkocQ@(sFnBiN1L@sO_~pkZzz$U9HT)0YTgffYP>`(mtnM zv2W@*nl)Mn49-?qQse7TuZ@jlh621MC+{G{k3)80hQX60fD6C?`z8i??%!dG@+dj2 zor_v(rP08;dwi6tF-#oE2FK1nd|V%_$OS^9zKLqhwf0OU83ILAO*~POhihX&dfT9K zUsJI47)$jGrUbUg&1`u#1zk1wY+ym_K;};N%*twvJ?dm5wy}W|EY>NpXVq*NkFCCJ zEQ>xfM|goha~4;=3+j6gP5I#=b$!0Pe_{Cv*+K0zmPn1&UYH{%;Ssln%5dF=x3FWp;71h|D& z8>z4LdTu^+-n6g!zXb1)QK^Y_u=7MC*dJd5u*BY0XwFGuL!W z$nH>H1I5J(^#B%z_?dq8BHSXjthI(VORJP>Y4*@jjF{Rs9UNH8(Bxp$bZs>eJGMbE z=sDvFWG8~$0hYvE!saA;_$(l<4G?(FVYPv>F08`?N~6Gl`r-Ko&s*21$9R|XI^ z#<1}W7a>CP5>^P*A6D3lX1LuPyBhxR>^U_G+X1u9)Whowe|Y*bynkhkHJS{cemxUD zxN#?(!vhZUtUHF= zC}e#^sgZVoHKjxl)>YJAEMai`^yYZ@Wcqb@{ky*kKmPCkF>_fJt^?-wR=05FKL>0L zhVJwg`o=zE`~c+b94x0+V>!Hg?+4*Jp#XpQfByq9hfU$uTR#ooe(|3}54GKXhAVb+ z>nNOYuPm{j`4tjZT01aa%3*%`aV)@m29O_^oFoIbIXwI3YnZoun1HoEI48!Ex=srV z(_v(Sk|D6VWfa*i@hj~$w5*dBW#9HsUL_W@Bh1e{4Qor&gk>bdU;N2mg#Y<}`4=#d zT`fYa`%1_Z%R{u0j!8GtErZ zHo`mz7KIdm6=VMkI#it7+a-dDDgMhDR}mP$V%LzE_0G9Cg+;VyWYqfzZ_e|QjTT{D=b8j0Ulax}?@>hui1OW2h06ok5Q5Y$Y zdGQSXS^0;ibCiQ6n8;K1f-$_@oPD4)LUjS8jn^q-SZiq8mE{7^tWq&Ps2*8bjf~+V zoOzKGiVG!`2k_czNQTBlnM1(004{c(V1R~n2$0GEwo7C1n%=}}VbsDx`kstwnLTV< zCF7?AyE2HVHHHre;qTEcg)vnNm_8xi*{~J#6~GRK#a@uvv5n*E(i(-XagmY1og!qV z3fWeT;ynSB*Be@5mmcJq3e#>ij!&H-BsC*c5NZ~&*`Oj6ThQ_0sC*rh;MQ1UjfR-n z%P38nvQKrH`BY~$QIz}$qqCOy=u30~cVw=}l10uqcLFUQipyh60D@g(~_%lYvh_c!B_M~Do~5w#gzJ2M+)_8dDO;7PeR z&&QSpivy<7p~?fdiy=N~dMa1f;B7&^ug=fl!W)P2Cnk&i*o7gUL`EJ1n3{=wZpO&8 zChFYW8ZtQ>?oJMex33RVeIN(OJBlSqtT$%p%_+v_@(Q&cNn)~I>*m6InEw24L*{~H zEZk_uqxJ#(Omtg=v>3VfZpchcg?UN?>Bc=ro`3ttuQA962^ktB(WgH=B}AjIe}F>n z02{)X>H$zDG0Z%D7WaE231zJqoW~g5x@sM_%&mq`3Vmx$#3G)*Jv=|f(@E~k+9PlD z26YhDVKQS(LSXOJum%={*0t?%fS<-@PBMO->ODL`pFV>I6ko7*oL}X~tsOCOF>0 zxF;wpX~~W@7J*4gA#hum^eTjkHQTb;b-*8W((un~t=Do;%e<$XAyV0F_%uqoz6M5x z#XDmnwn$UXF3dsmveEfz<{|d!nK(!pkQAj7&S<@{i`&C$<)^q|GsMd50*t0#JO%yk zN9ET??Hh@?CWJX8DlMF_n)!Sk2VCE`!e0im4U5~reoJaZHivO3P-8|Gf7-h!0N*&Up9ANJug#MS#71D}mF|{V1Si>#*@~eetYG4#WxjN!CJ;@j^SSp2y z;ZXzvF+Hs9DgpaxTnFo{Ne1Oo1ADdpYL2x!rBlZcN|!PotYwKhWp!O6A$N+as5(X5 zCHn|=dKDJTax#8y00!{JXg98eEVbrfQ&@-2HaZb7-o>@;XhovtW1YeKZ{3H{(e;KP zLD(k&U@11SfvxY{xEYpbXG0Yr{GE3ngc-UebPtS&=iff06v`FW`7GRi>vn7?-%0XH zBk`7vggETb!mP7rFg90Sq9llhM^A4LisCE^BkS6A432~pEM68y#4;^1R|Cj|I@p(I zGfR}l;OqkS6~c3ibKxd(VvqAzaIc|2?%fAm&Anzl5#Sg%GGPW)s~lJ;o*4y-8#ck7 zF($4ZhSZvBwzV=g?TpyM9pY+taQU`k!53h%E;JT-fAGo`K7+xC)or%*DeIqQfAx+| zM1@gJEa6ul|DNs@BvirNH+NzQv)Al%+X9P!jKVpB0XOa}g$rFFEgPo0q^q|dW3UGK zc7n@jlYLE%DFD2I^=4grWbAgby|opYxyUxLF4?6;;&R7Os+aZ2HL~5^!*T@fl-aEQ6vkQ^oIUD9(&fWPGQLgVtxb8NH4`?ki3waogUCZ-2stN$T-}4lXUW!D zIz%0%`&tlO2e?Ce@Nd`zzCr|A9aNJlHDaNHy|mZSOd2 z>|yXD|6>g>%IoOjQt?_BtlsM}&H{T*nW-CE5Kw~6zL3#Ihgfnc#hFN9z`5o%>_M|J zb!F9IJ+x3EfPY2s~Q`p@^11D7>0FW-k598 zQH+bMu48}G7nwaBI~XBf4f!bH>O7~vut{ddLz*!hnL7Xq3X1u{MauZ5-g!S5MLnK2JXPJ630Y^&Dh=M()>f!d zI*MM#K!>y`ah@!>Wri|?V_qq5giMGnz^1O^;h?S~fK%70461#MTSF(Z7j_Z^S6gS1}hOh!&Q)bhIqkXvvca%kCy;SnLu%ob>@;7 zvlb5)@M(xx@?{_6-Fx&+5SMP9R*VPbit%yk;Y1Ugq_C4CE@cBy#=$~>X`P(;4>pa# z{b>+!lEj3ofQb&${x@f!zuOV;_^Z2%z&=lXdAZtROHZ)>_tU=wQ3BeLVq!VS`ZtMq>^EI(KPQ>}n zHY{E_tUUUXBDDDWsbAAU3|=S2Vqd*FqA(sYG614;fFw7;vLbdU2Xbb7W;nI!hTzW> z&l=Od5$5Mun$|i?V2vgDeQb}zMu~yNPQF$z?0*GIzJKmT>ueDyqR&OjHVkg^0UCJwN!nAay!~ADmVIb6g>e&1@zW&^a+? zof%@z089$yV_3i=fEr850YuRQVn%6QmLrprs5g{0F-i!_X6|p2gjMjDDJELYxVzbV zWfW2?EW$L(f8hkpRz?u6m7VXSsa zrI^5g>$ZJ(BK&ylX87y>_?y^|v#?=<>+W!4>IOowfley8rYSXZc zBieNlK7caq?4{cTMM9eZpH^Pq2E;c}_s|04vF9kyrSg`RkcCkh?z28+gkJ&Usta(V zv1dGGrXC>288TLPd0~-LiUm$`NE zJ7usMNlvktcw^e^HxrIf)a+%I2wGgb4*?;I;_Xo8DuC}5g21tg7IdBum9*p0T_eyE zz1yu0ysNujk7@EqfA;AwE+VWtL13vW$5bl*#lMmxh@a!H6+E zUduC~A!uBmmcA}Szi0A91P`c@r*{AVKmbWZK~(AhUMdTfSjENUqu3&dv7l^pD>;0@ z7}AGpdCv;)@Y$_nC8!bjozW>Ih8bu{aSdZn>ydLligGCr%UA@2==w9hIhCU9kMe~% zQf6s^M^-1wag0B1?0k7Yi|aC!I5ye97-Y`-{wMI1+O&*`nX@O1OW-OHV93lfV*o(; zl2uXG7+b45%X#-hKBvG{H(3N}?70$hS;H~LQS;o!of&3i5WkP;J`fyNGJXZntbxB{ zO$C1pgCCxW@##)0xegA{Z@l96LEnhY|jP!I~fj ztOsMR7Gw2`-~Wc9=wF0MI?QNH%2w&>Uq@c|V(=CLAbHM19qZT9*d4z8`W5vjjenoQ z0K5`r@H%z(kbU{tr(tYrDl9Lr#JRQ0z%fKk3d3xFFBe8J0*r$cz}W@Fgd*M2X0qFL z1!^$rWer1tv;@Q%wcea1&#%c0f*i?$ifftVALL09IU}~Igh&`C2$cg6G?q;x$Ok*yujtUPFJnm)RNouHu4^&!LXwtVC8D*;m5AsKn2+cUcBl z6MW`uT((4I!(i_)%!tCZ1}+XZh}+r@TWeL}^$TO;sB6V0m098f(bd|5Vl)=TM&nLo zJD|TDHS7ZA7l#za27IMN3bnQL;H&R1xG!Jcxop zYrTPk0-PmAPEZYtMX)G#C~*WZkATh{6qrRhs}O$MfQcG5O$tFE3tA!|ys;Xf#CP|Y z&l>)&GFOR|d9DlQgQtmwY?1d?m+xqOo7>k%191mR+*QnbDgSW{y zSPR`1$Ybj4kr5;K<+>^2?COXeDZ+A80d^bGxPaIL%_!hgDs2}Jv49a`M;w}J<};0j zTPA^I8=Ae$nDYo}i(tx#YK@ct_0=#AwG&%p%r(86prCN@W(rc0nSpT!Fa7dhBJ8T+{gv((s zTtJ1_)OLnJ3?2pTJ~3@${C(!-SE1O(*%>Dgn7wy}vLssomsT^q5KNl|;{67$N7uU` z;+zsNTWgDe%$~5mutW?SO`NGSwnQvsCv40~eme}MDJwzQmB)YizlQY9_rra%XvuXWP$4fGHX$np?&{kbFH^z_3t{^_Q?nD6R1xKlz#SHBVh-C<+*3mVfE!Sb(pS( z8$bMUSa`V*jh{^ny?uVJkB+epoL^i4JwsP;nGVAM!%D7h;)-gE8L1cLB35o6iiq}l zogFbV_8ix3T>}LzT~h#jiiXzBsm?GyIT_>UUeCS?AN=W0qNTY*9Xg197zCd4U=3w? z7)JVGD%pcc0JT8!ic0N(IJ~R16IjB#p}xNtz=DuR{=WfC zW}jOmmR?{}uSYJf~*YosbOU{Tx#X14=LzUkoUo zWyWpNSp*g;7g>k+o&ZXhXl3Lo20@`rrl7^{1~F-auo=W6^MWMfcaGPjpvpXPC%!Hd zQDPi2F0qpUW3g?VV^)#HA3o?fy{7=m*Z&8kDr0bt1*%+*pNbN!7fiyg%gh=;ukxzH zGhG=ihi^{B2Bb!}#>+B8W!%`}Y`|$J%juu4gPF z`+(26{;Wz|tN2j7?|a6f#~4EH=jRx_|Et`e-?<~4Ytfv~|^XwjB?I8 zAq}-y@m}RlG>?vM=bR@2ia;)Ev%<{xjl*QU2rs9XN&32h zj=&2+jmkzmV;T)tZ`{U6C(H=;Tqa7EsS&2#6;5%Ub$2!(TP>O7vl^>v-PvQFp#_q` zy=G#etWIugo30Lh{GJdQzHfOO_gGw0$~+0WT>6;F3{Z29Es8B5_Vf11wb!Y8Lc>Yp zSNA9;0B@6|Brs;KjmwlZHdb9@MZ(FmG}mfUm50xN>VfwRZmcNRe3M?X4hS#1q4lYpxC@XHsu zSy6U}MbVArCa=eOZ81gyK+uZ7O#^~jaIy zgw=gqlm%RL0tBpRS+cei*(a=F@{*(lSfmWhT_Z{(MZCxb*7xiA*ThaFaaB&x?7ab? z18~62umxgY0iQbgtpFd{@chNOLOpSNv4&UhbeLcwzpG`1$M*MMEeBy zFE5a6fbdL`UDlTZz;3<{yN|yJ-+lJC;go&RaP@8|kYSX^>M>uhqJNWZhsl4O(wP z7+B)MtX_jdQ(DG+_ma;%Az*$D7O)v5*u(+ZrKxg(__SIM)*$g8gF{^~MCowv#w1MN z0mV=OjVSak!2=T6wn1H7yM$Xo82jM|?}vNWr$Tmd4I?2H-nx5@{h1HLO?6=yu(6B4 zTYB~+JbFyrR`*o6Af6-7IM9H&iwK0}5ROq&W`bg~t&F>j#d?~jWg1N1An}^UMclk` zjqBbBua`M1d?$ys+22ixkPHbvB=2-LP+KmyKz%yg;Rr+!3YH~l{un@2taTp2QQbqi zkLz!d7{HptLbi60joL$@=dc}~{`LPBzWsOqGPIG%b4CU!HUP0C9mqNA6fQ0yJYd_Q z{X1|cYk=h0=hK{%D%i7Atia6pMA)ioAoc%8Av4fJhT3xY?Qj1OGK5f66PVmOIEwHn zhDV?OfqHiseC)G@>6g)s+<}WGk40a?l0GBu743&%flG-ouP-l!V}x7-=f$oN3b87J z>W$IK1CX`=xD6zzB*{L_vwy27UobM%$KIwFGHcv|m1^A2^T!Xv+!Nwy8c>*Aqh2PR zbJ+j@M@ur_2i%{wKdd#lmMGk;2nDOm8y#3^EtsD5Tlc zyboK6l9Bm3#Y&IC=j?qIfXX>CAbcIB}XaWna78R6Mdc{z!0G2p*-c|YWF|Gk)qk`v8 z6~13I(^lnlh`UK8X9$OD<$Hn;m0&C@qB0avaSyK-KfwR{9I`L67%CL!p#tVAN+>da zj9(Bazz_^+#K-$1s|32bY56n&gNt#WSY82k(4@E{WV7*GVirGK%OcwDDAq&+C>XGB zOfr{``by9t@N*0flV9`0fBZ$Ts^WIOT{UKgD`3DgCZ98USD2^B=JH*WPu>8}crAzL zg(_E!^Pt`k?0cpRWswmvmXd$OF$&xT8t%0Ug$5PEUOJ~E5VvIBP|&<*nH9Y}mO?RR zP-aJv>iqd2vWbq-2l01tKACIB;%fMN8C5@Po{A-+EUaBeo5i;74Bb{%B1o5kUtL-X zE0nofo|z9nesC|mfAb2-J%<<*tTF4G!}v_FMhC>ruCHZb?0O*{aBHFi)?R!`Ve_Lf z)Y-x~Va0g<6UIA$@m5O+%PQX~@l4l=pZ*%~_W2jjFwpt{Z>iYKeqj6xY%*t*d1}Hd zu)DoQoGUS!;0ik~wPNI(q!Wwi3Yy$A+~*uGo#!XI&e`{_U8SrHKnK0--^Coxx9)ja zJUb;6bcsXEaH!&n8-H2H8H#htynD`M&teQ4``sFqy3u1xEp~@VE$T6gQ%fqOanHJk z{1K>Y7-)d_UWHu-xUfFaO|N?uFH;_`lWSzCnKj)skc{8nq0+(fJYAlcgFHG(*}&!E zv$-F9`5oQ0BsP=+$#l&Y38AZkICNK$BJjp zp|Me@e50Nl5vjz)6PU_FTYA9&{W|_cW5-nQGc2YHz=s)D2A%KW-`}=|6z(%)DmqZW zeZ(F3zCn3{Jq51Wgt{yY_}#-5wgpqQ3YfVhCZc2bO4z`9&r>^Qp964cj0}ZTZKIVU z2)~Jp%0|Z2h(bHTO_oG3jEzo(BdnCWWSpg|j1Mi0#>V z`6$f&&A$y>pZz{0=;E+4M?>=VtKr%Qe-*Hr3e~HW0F_x}7*ID%%12*XcY-KyR-vEYm}?IZO>v)`FO_9)iky$RM>} znxz+_)yG;|2snd+>jCh!60~WJt`-E{Iabcx>|7KowPf$?Dx}!7ySVrcnFno@{tgZm zS)-k_MQb5@c=3zR!r}@k-&gO2sxg=z;=OjusqlPd8!3v*8dKqr*cXLB6Uwz#77ifT zQAbNQtU@zIO+zafVB$xxlfO0-3l0`VLs;mIdohL8WvzYitA?Dp(TbQ9dYI{`~bHY4#RB|uX> z2gv~43*0VE9H=A|eV%NpBtUk2U^INmHFj2s!yvGEBfE_4R1Q}sCICb=;r5lQB;KG{ z5N-#p17Wd-c*`o{qS&8V#+(7n-oEoTItg$O0L$af(Lzn=syr!laDHHsK5kj$RCznOCpE;K*2HBt|E%5c5KU250^l zD)o~2Jc4-`p5lA}=vs$}d2H?=V{|2Sz#eA!-o)GQVQr5@_saaM>G107hs48VVc02> zOc=p1v2-2iC8A|v?C2rNzS1>1GBV6K)~SQ{34boZQXc}mn=pI;UOZzpmiF`M7u1C^ zzRk`M`S73q?Z0F_H)2xCE?M36xOKbw24ZMHes_~|R{=oQ22=DYSOxorY9Q$lde#=p zV~pv-0$g#1+5o!?^D~@UtX<^H5zPMy0n{qhOU{bTx)ZP!#)iqxYs@MaOsJ|TdO`-c zkBY%KCk40tMpf+sE|sg9x<#nN2+DXPW?&*iL{d=!^I2K1_*wR>3f|Z%*C+~cJ|`n1 zSc~8Yg{ML{j?R179jB}X4q)-d#~1=$*Uu>QRZ4;wuNU1c%!BL_zlmV-7)52w7J?ue za4LyrtGM3dW`UJ9CS67AoA@-25czx&Ic@W@vEfuLCH}*P4(L|Pw zYnu640Uv@%k`%NqcnLgf+~|zZ7_nGjww+jB{!>XBwC(iy?8bMQL}GHCz*_gUbKuKy zs#tY_`K)mc_`M8t+#AfFGSRXOm3Tw%>9VLi%NUoGz~WCqY$HZ`tf?n>a_u7ktpI)j z6@y?r#&gQ(2r>oQ5D{EbrqYji50gkVl4Jcm^oP%Fl1wxp1e46OdPBhO_}>6z*UmBX zMg+$KcEP6e>YT?oI@X)FS$E7&o>>>NW9u2{4}-%4;ri{{v0#2IjI5D@k<6-);DG0# zr6#;hZO#HL+nYqi9awzJKFQ9nF0XK(i!gyfm(MQJhVYY6bFdn^l6Z0mV=z!(y%pAo8{LH{X=ZIVv+IN};a%Zb zitMQ(I@?0eCFG^XgB}<|NSJ1fZ%G#RAV#5kgPqHM!*D=;tA8D{Nd?ZIhDd{;5IABi zn!w0?qMOzIP~*PiO#z{CYMe;CeeD&)h^q;r4TaL#uc@6}OUMv1^A4v>7 z2^vbVsRHBHkaTY`TntO}`3Kjo{d4PY9U~O>`S1z!t-+CHE@ZrwtuD0St$}1FkcZpF zbyMj=XeU5E=?QbgaX=zN=D&I6xt@YTYW4)>>w?5oTRxXY!50b6cGD{KQh*0*s|W^t zAP%@V{%v3>}78%zm*&Kbl2vkurQ4*-A@gZI(-VEW@(lZ@5St5o~sjut|cvl`ZXD-C4greAkrrDIV{j>+-ps` zXPKuHm|p?R;`$n;L?E@4> zi4o~#EV+#p;x`Lnbf6Q1gMG+&w>DwR*pMk)wp|z#dw}+J80{9%2nK`mP|LZSna8-m zMVuTM3T2oqJ4#faz$(JPE-&G(;F{xuxMj)MTxXoF`^e<=n5|ug9hSx1ML2PSBXieD zVviQBjCeaq0gYs*9h&m4fMzi^w)aAopU(avzEc%uVB6B{{SNj*53E=kA+xzmA4Jw7 z$=HwfX?KNdTad5;aI-&eJuzg)z*@8D5J0Z$y@Pw%3($SilIbMzp`Ez5Y^>YZH`vGA z!CC{fPYB77HAqt=*9)Un4_BtL97s8 zW=zI_8QkAS;PdSBrz9ARMfaaI#EjY5gGtm?Bl}ZPI?5Od|H7UNpf=fyT^MPPKKnE( z5(BUidORQF<#SA3XvG*X+<@&D6BS0UPQ_qtc6c10;Tu}yM<^Y^j7r5=F_Q{(F?Y~b z&M*w+>!l!p`&x#g9;IaVpvow^h&{8cVT@g4w1O}hJ1mT7G&G~IHYuvEqExxa1n72S zlMoP|C(Z}C!`M~o1LI>MgX``PU~Gvd0gdv;GB1XnWH2%jqtSRb9<9K>h+!yl8m*JYx!dW-n3K$sls&I{%9RS=KcUeA;G>8EY{=)P=CtYA(3~zb)@+Hqpwm5pTS#b9JqpRvpPcg@ zOLw#l^oC8Ixv_Vc5R3`>j2FUh|MTaZ7eY8rVN|IQVL6RnLU7#2Od??w{UlqM!OEUM z)_7JdXx&VK`a@b$3c3VluD|=)eP$g=J65QRn*dzV)ySeKyZX_U&5#!+;aT6#q&@YE zF5?vAKV~16&^3l)dG3z)@n)cxhpDD8er-I;H4AXtuhbr{t1FxA#|j21BGUzHO@vp; z&RK9*3}d@d>rdMEwtH}Z{a9hqJp+!@WQp7gT_bW?FcMG0+8HBmQVZq)fDYV*{M>;o z=^L4fhIDuTaGV$8?(Dk^eg&M?Qs#xuI2h3VlT+ZaO6+YrB#3Ti*>g&_un}-WLZES2 zTkw;2gvG_zQK(o1@|X=$%NJV#7cwYpR}Q#V5!1(yH)JQ_K4f!>44qa4gKd7S{RT0C z&@@KM&m$0J+!O#DB42J$!3JyV(Tph7nrs5NM9P=}YGZL=tgm!LtX0ee;#+3$oZ+r) z#A0vcQ0x%6zQ=)XU^r#co6FTXS~v|e8RsxI^hQ0~VIa`nq~ogCkM~fF@lYewI5z?@ z`)-~Kba)oCip^}QCtUjC#X@phJ)v<8nNAHCf-@FXP zg>?=e*Jl$7jZT?sW89ih_|(^pn_9OtVmb`N7%k&Mw$4*5h{;CMXKzA>!Yrn*vx!Q8 z$V=9>9yFU~Q!lP8gsRpaKrilUXkP>L>+s#5qEs!OEM=R*6?Fo`bO9*XUMYqru%X@W z{7JYVuB;Y!ogE409)29|lW>wJzM&OGPbe6`!ya?670#;xlf+FOBb=;_bEpALU~Lyc zvK#Y@(E0Qdt-l~Ei9USXCpL-ZD|U;BT(^bIU78|uA6%_Dga#A~zz8sog1?}|!8-$OVQF_GEbc<>V~uyx z>S+T-eaUCXuxhJ_b2}sZs-G5V<$Y4V0e6c8>0gj6vIp4cof-@4R0~g_0AxXH5t^M| zhnh<>OY}2+`3yIBB24vlhh;(!40^wL1z&{|^-x+(ffQ#}N0npP!Qc54e{p|H!l|VggEpc%M?kYjLhDO7W{ZVbf{?vCl7!{X+2&tGL8IHdFC{AF~zBl2ow@ zbdCYrCKw22EOp^?BZBxY@&RC}QK$Sf%tYo%p{6VCT-TK^j7nvx%7y9R`>+69K<1~0 zGiV8$Q@jU$U)P;(Ju}P99!GmufCiseZs|$zA#@YT+A)0A4p4=mqVaf6%QMxOiu(d^ z?so(c${>TDts_}+D{_eVu18SiC{tyuoxK!&7Bde8DO`+uXna+Git)v|v}6FlUNO^N zAnkbTSsz_*=PFA96X&mnwUq_*-Z4<0YekvQGYAw6=8r5Qpi&T6i4)Wn1-r+Yu?8I0 z2J=ZUJ#%P*_$X2x8`sc{uJYruaj*h2-N21p?;NAjR*;oz%F#F~s;m zt|c&MF#OgrqPtAQ0v)Lm?BA=g>V#C>-`p4QPy{gX2Y;7+t;C+`(!Br>=+ZqsuDE#Z zV|#&%|AIXeuLPcgyom;+8sh7x}HN!H4jv{`Bgt_(5-q(+AqwI;sKP9h<`|IJupWY|rf+Q=LuUh8$ z5D;=s=+Y@`xP%_i;OfD!wpv4pt}Jt({x%E}gKW)E>#&`&Cf)s==&(9G4zRx%+(Q%B zD8!x$fB3`4BpzK2uNKKDMGo6`&LZcY<+BUq7G8@qvgGBX&nTyq#Fe-VaPDC3k78>- z4hNv68Dj;n1?xOTsECE;>&T*&k+efXEbF4KX8sb)`5pkTdl*9$&ywy>lj;QWCJ{uP z&U|UGR@~6+LBODm;jK~WrkCX{Ape|N!%93)u_!RtF`Ockp)3go=^1<4J(gsT=}~dr zWkh8RWe)jgB$;rMBU`~X$RPKQ?da;z)g=fV<%u9vrmX-W;odafj+;h7A1R#RKV8t`64D0g9aAXk7?F26>sOnj8h~0Qt3^&#FqTqM2 zW9I=TSLI*e$M^>oSQ{=UTYQ)nY+`|vFUT^MN~`Gr7fn6mPheS{;r7e30ltKZ>AZ0> zT>sz);ln@uJK;M7!|bPI1kP-5s4m0p58e;&|LPYZKerS<{c9*}#=>+~1Ro1$?>UQ6 zvbUK_#zFC8*LZ+umE9`v4=pcCZ|q}jCm65AGRp{+1B7md*t0{xPYdI_(Vh--BnK2x zK8c1*Smr>PyfI1v+Fd|Z4F2|TphQz-4K{QC1DK2Uww~~s_3gR#PG}kcbl$lamI0#~ z(#MU9tRwBbhB-N5^TxV#$QvujUouw-<_Pfyz+ioQ2115|PT&QZi?g_T$vnpDInr7t zQ*obt)`)&f!5mmza+ghST_=+d24O)|C>jYev~VX_D#Yw~q5!FNTTtSFBNfashi&M} zNXRbH9*)ofaY`)Nz{U_Hj&Z$9_Ua-j?SiGQKDs!porhU6u1ASWYuuxI0(-oR47FqS z-WKh~*5>C4IXMn1FX=1#)_b&784R=Ee$IGdwE!*~#IJSGB4+>93;>NXC9~^coM8M^ za}BNHNR;R!kR6M!g*D8RoPgS*1P@FT3bu{Koqn}ReZ?w38#E%ULjl+D^k*N1^zc|1 zBxbLdo~|{7C>$=nf~KM0aJ0&{!^BuWbAZdC`7At~{+4=QL%5Bbi7_Q;7lyGOWokC5 zY)guMy+w*9Z(@mNu8c8Xx+-v!F+bH<$B4bKw7kZ(_X$_%#0`=NOV24viR<28%Gsqw zKmdS`*g9K(*g=Nc z1rp0xNI{Q)k3wJFO_)erGI!`sAwx}K zjBbW1zTd+6+ajZNc&MA>ix}tk?J#C&KNtiUW!6x(@epvS>&!NNF%bZ%VjL7l4#7>< zCqN;xEC8Y^#ynlD*)z|VTgsu&cs@l$_`)rQbW~0E6uA>7*bKZ zG=z1LSsS;%B@2dqa|hgMFsI2G@ILWRmRJj_|wRehh$Vsk9fw;F!1p zyG5ed0s7bkjn+=G`7!3FUwj8Uy$<<6cm;`(mbTi4tscKN1;LSKuTj1Oj4^U`A_64W zidk}!m_rREP=q=IFP3LhX3;FtB!$wI2a zP4{a&PXKh2Xx#ryLc}sVc+XfC@OC8*RAyITw?|yHYgnmCsAr3NaerooGo^bUz{&f@ z_@5H;<-Ij3G$76*XcJGcuK$InH*1bGOVaZW5C8(P0Ej?f-`Cu?TC(=)>TQ}m>=~OG zNk$u^Nby1@{sCR_u77~8s5_ZSG@~e8jEQWiIU3IN-qlrERb7=?m6dy9-+{mm1eO2- zfS%{Xq)?rm2_WK&?>onj`|IwnyYoESChtMM@)rC!i$p;Z?2Av%fBa5IumL@zj0+GA z(%2WX)&c7-*w^3Q4-Q9<`-%nKNyP+R`&lz5G8ynXJ zr>uq+BlCFBPuI7?9O<<(f~ps68z{-)M!$k0D(kHiHQnaA zML_HtoU5+IoZ z3NQuO-J!Qk1I-l=h@Q*B?6KIj5H=O9L&CLU~is0cn}&O z=a0Vk!|?uJ{BzoX&_V<5B1LbEeQFIkWN)5O_%y)snCRsm2W*RjVU@5^n>O~}gEt3o zI0d}D_3nqEX=E&<`uoEU`+z7$)ptZ{Y;y7?X&06Xk9d&?*8!mPnEQ+#IR~qbNX{MB z`ixL}hR^Bhphp>=>LXg%>@QKRNosj(>IDTN=i%*l?}lL_Zx;Y`6U@5{%VB<-`(c3m z0f5vIaew*G{yU83MYw_t^igqibU_hD4Zvc}-hRpOU;gkEkV?x8nmHQ*F2^1~7y+tt zThf*AC-04uCftETN#kb}`^h$Iwp(O$+t*?F@e?B7HGsIT*ckcJ=oo2N1}GyVTZ@y7 z!MYdrUVa>|wN=9Z`EUPCc>M9-aQ;wml&|YBy_lE{|M2&}gojB}Um}XVoO6@j0;nRq zwoTT%qYs6T5p5fWxCHaTMJ(K`6*O|ilktaINP(tjK&Ol#@0fOsk87` zfA!-qFw`AWh)#UjAOd(7AXg&lI6&HVV|fwK%ajBZz6Pk3&)J1{vpPLPg{;vm(y|*& zEa7ZMutrqU7Be>56&MI0c5zl57%idMv#mOoJg?WU@pbN~a@NHCIuKmS!j@IXT)>&c z>IEy$jLgaTGQIZdOG6ZmG(RxaqVd&;_;W#w-uNqJt&+}A2qXCNSv=pScKx}=-M)6! zFS%HF_UJF(%xB_kQwU;Xbu=ah6R^VXymm|h^Y7SueZUQ3>llQ2jZPfA*{b^sTc;B3TpUnmKGbDt2Eu(ZkyX)Zl zy_Z8kv;hLCHPP7dxq>D>DuN)NBNkcs;3!w4RRT1>eU$KP*YbkK`du@o%?@gGbt@7e{k4_MKr^E;n z(%eHmS&k0QfsLIjwDGxs+gevkwt#2NkY4=ojT_<5e()xZz2T3>^~2u-EbkGGU&1MD z;(Xf%rHvLvM%x~L{P&?|V=`QAZ;B%dcZm*l^^+2Xb8g(SK1aXEp_|x~j_|+!Z~r&C zj!8tl<8gj@ir~O5_O^txeaPN87Sgnpv+pyWjjHSM(~3EvQUt> za8DG=aV_O1o*|yyY?5nlq}<4sYiXt^${1(Vy;Giz>{&Px9Tc1pBU{RmpL@)CcfPhV zqx@{J2Bxi>vHcbxIl;p71Dpw4I4V{rD~|^RfYcLCB#Cl}9r16+bongGnfE8;mNFpN zH=XPLE5F{e0}E`~$EMs%N zompi4VQ}nP%uh$pk*AjbwnG&qq8m5o5a6Toa|W3WSle5fK0ton!y*W>TFVrpg%3_h zV7aidJRl42!<30$l?i)V>0`+HxNP+d;Dzt&5YQU_UC+aC($$9i10YSP(YW#i!~+b zfwl}j5-7QM2hZ;U?j#)>!K<(dMGT$Yqf?N&ZI;*oV6u|7fz3aI`>?+-8|J@x5IbmJ zzI!uVew)xSpIIhCHUr>T1RS^b55_hp9tH`aDuZfznfv3=8+qtrgBv(t2N;qPd#GUR zAxNl4R-)i4gr-GmxW5*J^_Vp{La-h6lgBW1k^)^MI_PK*yQIsD$icUegCXZ+|L`9F)$tmD&6R+_cj>GsHZ-x!hT$=#qH4J2Px;1q0Sq%WSRg8#=wFiNB-^v`C zLli?6gJ(n39jbb!W@d3hAg3A?;qa><9i&~LN~xLk@%wJLWP2DJ$9TyTvAX!ss4l4k z(+T#~Ti;+J0Fj0U&V@bazNL)xVQk= zI1i7Wz6kHqPdETX_$JSSD1;WeFYg_v@_HFgPy2F?r(MFp{@22z}z9wK4+ zFgd>%QgnFFwf9jhLNN#X>%}(g2*Bpib`h_}#bcCR3bYM8j zNEy)52G^!IKS4$EJfnh!!|0V;d_G(p944dGHVw_Sc2gzyfOHP`kH@QnCLeB(6mn;G zFDJ%oY$9?fRp%#1W65hgW403T+^rHRws;|jqiZD7_B}S))rGSBU%lXU{G(&w*)ct1 zD#ba^HWl4~_t@Xl9Uju2!L+K5s;wC^${Mn9#NOm^YT}4Jbe6r}+Q^W|S(8UwjkswX z(1qyPG78-&j}b(TTP==TTTcgROP`$;=*!0Xw5Lug^Fw_G8yg%b3*^XhESvrVG%xU zz4Q>)SR0}?Hs!VjH?#%5qd;&Ur@?V(63cD;dO7QQfhvtk-iw6V*yI&E7!sPYmJ;Gb*EsgnkEOVv))EC zO+W8)4q^s?d)Zl;j|TmMb+tKoJ^SS#d9S4u#r}ZYhrrxEj#uE~*232U!?MnFdVoKc z<(?f|b~9ji$N6chI}UFU5j#6V?+_gVa?^DIE-at30~jb$z?FednkVI% zB2%?TyO5)2@ZM^mJrLPaAJoAKEKsC$NOrD7WUqDjI^5#+@bCY_f2OY){Tdltxmcv= zthIytAwn0U^#lUAzXHyZ@k{xLhn(Z}g%{xx4)(kh9cVA|%bW_*3@^;Se`P+S4X54I z{1L$%8)`?O$2sA_kV81S=s2AW_3qI4Ft%DS3s&ISD!ktoB^}Irmbb0=v;I} zc#mpleR%{ zfFjCAiRD>xJfD7YH>gCyZWWvIr}nC^@CQ34z6#V-4sz+lVmK(sIf zaLEA6L-eJy@$MdLqNlfsx7|oIA;q6V?lHxFROrRFLlp1I0FwuH7Q*@zgjv##(-2Y9 zaQnKLiz?`L9Ev<6g|0x=QU$)5!NOYFi<}Wj&C(8Ly*L{ypBqQ1fNUl`-$AtZpZ=J% zwuR zcK|6zb8rnwS2WX9@eHHd4hOQWw;n13$zVLvJdw_$@>Y2BjsEaxa+6uUcypuxciG$a zrbd+JQn>f~xiCs}s+;JuEdXE;hxrw_jIFE%fOD%jLv?MI_Ap(H!|Dm2O+KP1Wr%TE z$Q{w@diLBdGJo82n@P6--1kuul7CwPT;&o#a(Xr@yXP;7)T`8c`a(u|Sy+nv zt5%d-k%q@9qUseCTx-u@ScVvkk_dgh11KQ?3g>B=euszzAjn3eaPJ_uHxZq+o4iql z6^b}|x`qIp6JdI;NEI-$Pn+xlEm``;$3r&-7{=;Wi7azDz*lFOc<}_~SB;8Wy>>*j zzL#j~38|Y?_9R6wLP0`Q{M^L}j!uS@BZoWcFbUxTc_T%FEj|9Ij{tUioB?0gh?6GB zw?CWSbxhSEFBTreNDW3Y64sHiSV)AtRU=J+CUlNQ+~|tn;{X3%@WDPQ8yYHq?`KC( zzJ@+hfEg!@2%P-4QL)(SgMXho6$gFh>v<%uA@Jw=+`j-oeNzt@b$Wo`tH(4D33Rj^ z9*tCV^n9!LhF-IxGTBfyG8c^hsosx~DZw}f1~6zuPyH7`AlDQ~*1Na-8&!fsD~J$n z#RZ4}Qm`x_Z_aT7eVK?iu8GlC^aXO3#F6tEP1UNFeOCFrf)0D-#e3!FHm3E<%E1fH zz3m)~;Kcj!8IU1+b7>b${T@=OeSz^ajUM^9Xmvl^`*j}$=0-QHYTm;MOe5Qo3&&Xy zEC|9BG}C0UfDcEs7Oq)bd!F+GBX2N52TkWLMn$>qt3B3=7@1YZ+-IM|f4Jtp#-DgC z*EI6A;_rOFpf9=xaShqB_OGwIYj*ZpASxpgYfYSHk3H{}6KgU7+cD8J%^(eW#(SYy!&Z zj{rCJDrwVv_}jn#6i_!#5g}lf2XCZ`|IQolgm3PDjy&T0vj+$4cM@vAA%%z~^g(fX zmNSUohtL8+|qXzlzf+*gNGpSVZI=TkU$X4o2w|Cus^+hw6wY;QX5MbPwGd z&*K3OOp9?NktgJc`?q_+0i10?wiioccUxL4qUa)+qZ~u2A~?_(&p53~a8g$R zn(mA8Xv1Eknw~dh#|#3CTir{}oxtycjEgh|cTI|c4waA7hyxTO;rzzukTdUjuZ2IJ z#xb|h$Fuh8-ka%d7ptLl%y0~Kw5-JSbItTA+qH8j{3SfoSvGYNr8>JoYLBSLCW3ar z2kY_b!Q|OsQSt$YOvJhGg}C8;JpU3=uQl@8oA6R>#wx$8#D_U!m1m(lGhtp*a^`T; z7}plOlQt^e2Jsr`3b`KJKQI^Z5P)*d2HEwv(E|w>l)L8` zsxq5?fM<->4|xu9On&g3Zh(iY4)h*F_AIcY!&EW~sm8$Qp@W-GxNis4cLLOU+VSoY z9#goIFH?F164AME{)%CR2DVdkM0&W7{=u%;BhDyH8Uv9h z5@CgD{n1|N)X=a$lEW|X>>y>q9fKz(@IxNn=5vimI#SFkMTIuWA<1x^I`%0X5LMh> z4qK1D3XN1dy+Z`&YStZ)~CbjAK(E#d&YV)Rp9lz;oZOfSE2iz_h^oh2#+Thh!!Ro5!4T{ zHAlpYF+PG!0816t|ucNHnG~p4AaBEL%h$s>Hqv^D* zRZ{Ge+{kkcpL`eqTb4%2&Fw@6A&NVbHKdQnHODK1pZtbiidkm54&q&s?nO0MFbWNjlT{CC+Q!ntOpK7+cd;U zC{Xs%rk5gJyUHI|2_7akvgYs**jeFCOxs|aKlD-wYW*YLgGgP=vw zEpn9MzmIw@{>*FtBhcYR7E<^n z56JXs{NnYm(!qZ3do^@A-hwan8)_p0Izej$gWRwBDMnQUFuYG5lRP*79ixkU-mCPd z_awk_%>++zO?iIh)HRdp630{t`gng8l)2vAKcC}uTqAXPd_V8;-n_2&<9m#%We6Jh z8D=O1l=9-dhQr0ZCcor5UKfajMHWz3_tbmg_ZV8&RX|~+PVQ0!WxO}OhxM(#j%)B7 z-kZEY*DPM1Pw-1H>>j;pNbG%k{{j{xJpPF(ZnnnWXIY|0MvM(2xb~zA$ll<1VVCRZ z#W^z2g&xNNLBE^I+(y64aW-JGMYVW)!3hm5oV&g7+28$)mLaR53ofKWXyR5+GLOO##^I5c)W(k>Qe=XqYA%}5#VgNJ9<>=a7R(E-^c zG7WibvXQ9^{zyGC(6h(q+jP9zT0`5ZEGeTnJ8|8s`{dfVMhaF~ac9#206+jqL_t&* z{Zy4pWP|s*POqp3u4{CHlrh%DwU6yQz#)Dwz%0{R$~3r>H*!(esUO&M|Hr-Q05y@$ zwd;U;=zd2BDgsaC1nLn1FW1RPwXHW#P)2uBb$9mWYk)jTOs?TEtSNVXLBatU^Ot$W`Ng@QV``^aPT+KT87= zcJN*l@+#34J_I*vI8cwuuymushJutkQpDJ&xXBC$+=JtxwJUL&0}?~c2!1TX)yw06 zm`WiT2C&gf=05x!-e-xf;rtv6TOpTUVl#K}HY-FQ?9flZG5qMqe-_Ft&N;lS+9ZT? z(EBVyx6ioQb<(PO;UB&}9@b{59_(XoDu-r=^r&8>1}m$V$rzzHri0H;`rL$HJvbDq zL;-P;E{H(3b2z+S1#&RXJW)Z52x#*FkBntUI!w`;lE91NUVWBE-g_ii!_XRDMgNnI zw9i$EP4kuX+w=2S-Pj47f8%=)l|yhLb~fTMJl%v}K%0*DKKLk%U%4DU{p7PSK?{aH zq5>~hzoBY1&w3uxSIg$R4B5rtlvWPI5-ihwfJg!GU132Sz-$@|hlB4C1b|sTf>cD| z%tP|0TWaJGIoZH4{>lnGiS~4)iwd;jIAOqn5XT)Cf=l6uT7%6&ImrFqB#d(Y@;nndp|>(84=dJ zRiUgNGJl~xeDLRg9scwG^|SD&AAZ2B)Pb=0?I(oP8E-|@@Sp#y|32Ki|B$l-ErGT% ziH0`FsTAIjrtR;=@nC;WXk@)k^X$&Pj&K!kHc8aZ*{~}Mq^RJDb;7MPU9z>|^Po5Y zl2gy-!tkvg`u*A4h_d;SkvN$8%Rhb&7>CbRq-UM!Y%4lh1ABet0G>yHT)8vLj8khi zeaz$&qfvUuheXGB04mqsdWWeCb4(Oj#iO1HZ+`bpoHdj_;O=;7A}nsNKxrT)*+@~q z@mjdtI~1OLI!(H&hl()(7uVY4ETxSEVTiLt85>bIazn86$Q#idl-Y|XkHh4XC$y4a z2=>5G{Mo|%*P#~&u9IkQsfmaf#U+jCmRwto0+DYpR^8!i_W03*&+v!1hf4#YzAzjb za-?TS<&F*Yp|glYlQwGNx#bM;ZB*7Oa*GsRjo-PT(pqJ$_Kdt6_F|t%)*-1uJF#n6 zEZXwyYN*W;n3C2|Y3Qw{6b!m{rV<4PMz~biN2E+7R3|x0Ha@PSrI6GN=Zl)|O8~}g zF6?|z7(~dr96Z(xkZYQ(YDB6=-usIH5Kw(Wl=%eN6cF1AS^bQYE7?E6b&QCqAaz7J;^?+&@$m=oZLx9YUaBn;V_u;H;*W&~S_JYhsY*)oOwVjg%Ep@yW4T|id zUx}WWxoN`STET2YMlf|jHo`~=_yrwCK^>LjMn$Dhobwtm1na^*qPwjQY$S!nCv#U? zcjcV)H{VHu*TFK&!&@<|rgnWUK&(+c{x)J7$?iO-;L>VO-y8uRKXaxkm*n&KIbTK) zeeT#3jO)7IF*3_CSP^ODPu*XKg0_tXi6rEdU$WkQ*>_NIe!=~pl?k5kT(-9{Yb5u( z1Yg-_CgUCUaiUAuxem_9hj*{ycylqXUczu~Q1Eny9U@i`zW(9~ZH7h}DMz%ivKuBo z|3yd;t-lFnBTGhLVQC5K1w|1QYY9G@==*um(iI+j^CG;Mg(KWL&LcMyEuRl>e&;*z ztl^w8OcUVs2aKUpwUzG{SR)VyKG=v3vu0|%Yl{{k|>kuPQoKJ%?F~FeX!ah}4IC~Ttt*TI@=$_LxBgaUzELjr|h9n}Rq&=hHhKJw&vVXuEQ3QPjm5j=R|i zUaJ#%flzzSOzECN4)?R{SC|;+?*C~NVGfFj&vIxDyc@33cVI04jKXRUB&#Y#kM=my z>rvquwaAdy*DLT4h$Q{8%0*IZ2_DiGfUM*pUgA!;YS5D!wkq){?Sh~@9R`#$4*NSh!OkDtGY z!^%ystrA_^1Dqsy#tw*@a`$*Ds$@#^=VI0?o9sBTsALhWb#mx?80-R(9XTqliQExD zc|_!QF_jyCZ-BT{qRI954ZK{`K4R(&-3$L>CIAk=Ec+P^$}20X!MeWn z*KlTJ7DLiZ`p@VQ`YkFvCr9|XDl=!o?g29G`bR=E+@ZzqCaqb?SPm=f6~GO7W{@XT8P)O`-)oe#bfp73l# z9UU+{GP%Biuy12UjN%b%BuF?0tfevD`@D|LfL}!W6C!JyG_RJsB9yKGNLy@^ z!gXy>?Ce%~L|Mr4_I|6T!K#W}#odSu9#bCP2;i{=Ll4(nq=NP9FF(UkfB;TvMjqGM z5h(+}x)VMwxTo&HaU!QPp#x8MmPr^%Lh93#Yw!^nv<(NvFTHNZsaYjA)Q@~4Z}wr_ zCoNJYbyVo;54C{ZrS*;2UL}!$Lq!#FJx<0t_g&#UG+57H20>aWr`b1W2W*LBZd=c8#bvGlP3b!D8IEw$?%yyt*bF#^t$L zqM)RpDKcr|nPy);4MU^D0DlJD!%GC=g{OaX8U!g3&eb>c@arBP$AIi`P9BCd=V%enPS!T(Pmj8u zw6JX!to}-py2=w}Hyyb{%DB6)7l(j|8S=PAI~f5?J-W2SnU~D0BbdgaGBQ@eVcNtg z7(_Ooz#%OIW(6-6OxXX>D56tN+WYA(hb)@vjp<$tNvofU0n)j(RfmxQD@5fY+NOfo z791!d8GyF#fgV!U=qyCY>T}bmv0#VS>iw(NjSj{r6~@7cp9O8|D${_iI4yPP1PdzU zO2tt~7}rP)$2eLPROb1d?x8F?y{2#M5>Z^7UCzAnAy`!p+EU3jA~u0FrEA+3*CM9t zxs}zqh1imbcd!-M5Z3XSs@~4777DVU=df>S)<`1wA^K-wgOP+p)2%?y7=hyRb*^=s zcQ$A?%Nh0B`{*I(*`5KC%#M`cVa1j$*r)_{rV;*F4&HAwj_@+6vNVpcI;@2%M>8iz z30V%Lw;KKPoU23Sx9+18TvtK4=fkf0ex~UzvolWluqm{9H~z z7qG{kwbjKrj_!JR_wE&BsD{DzE1VH}kW$p(UN{ynjV`dj%J(>1bZ?Ua0@lzdl-L01 z<5I-)C@4!}pZbS;DU<@(fXZC+Z2}8r?3riE=-2_#!wt@x{o$NaR4#d*&&VcRs)9e( z7fT*x;CH|I6sMJG6W|VO0h~K+|5YPr76_?_FK`C@vrQ|cuA#v=rf|#519qwo{dP)P z^@L#7Hk}yk7b$3W0)y#Q_tnA+51%u4lLVh^r0b-Ix?XE|4}?~Wi`nw^NBF?8() zn291G?!^V*xJHm!*T!L!N8p!=K|oTEXMP44s)nZ}nqs7QNC~wHll+Dw9>6abHWmeN zsN{8JP(Cf(r~tvRv|(gD&}05KylDzz9s;FNJjwqVQmT$WeUX?M}EhI2;=HnZrh!?~oL& zm6j|$EhuQZy0S`Sk@T(i%I6q$5`@_ZRzhWgQ2HEcWFtuHC?3}&i*OlVO9RGi;#Y$Q z4qWDjeQr`Vgg*7MBfpPub$vZv3(qLZkF(rRUJdl+vTE!Ibe}=t*g%$(>l3l2fCewnw~YfruobpX!s@zIdz=^{PVOSA+qgwo=e!!=mNPbHk* zOtU}IW+$W&&)@*I0o(?MvEOpHf-pa z5_q%cyZqjgY1=(G+g1(ZY+6UI*&^aX4giPOX2;H)ZJ?rd2YIPI;d6ARt`|6OJ z?R8R4rGfC_+pop`VWy=o4Km>XkTFPP?|{_g$~T{dD?j;f!uADWZ1@OA^ay(ICx0HM ze*gQhPGd?3oewckeSU2*WH>l|$ebbj7L28n8#d;q!&x41NJZq(?b~5%VK(eN{9|ZB zuFy;Iuxw*+<#~;b^wWEdiqDlI2NWZT!c^hf=c@(9-b7_+Yzu>oC_kt03T+B`L7_&u zW;G;F_Ay?x>eyI~MH$<8SAC?9*Vflip2wkUWRR8`pFs4!OT?%@RJKTm0_OTDzi)#G zygT;*%a927AFX*9QAnHN_AM$gQCtlffIo^?!)WiO*KYTQl@*FlrjQA~H|?9GCsRH5 zIXCw#v=c>{nB0J1d>*&AqP~k+T?{-TPe_yK$3|UUzBX zK$Rv2UvJXTiUx5Dz_z)uM&rv%p%wsEXro02vgEA9j-GCYQeO?viITZ@ndW+`qie#| z4}TbDxb8WWfpY-T=@zY2P*SIW6ZxM5qeEez=wxwuKID4)h{`cz7Vy~LO=oxZ>@~>6 zb91XPGI&DNse&Q@#ee*p@WGG%D|!!pfF4*0S4m0l%{~gpt(;d3@DXd!15c-;(h+|5 z?Wf`PO%!WAo;c@?T}D=`z0bCBY^)}gLuBJ*NzdOt{FV;(gPb{{N_youiMs%KHa05O z1ZUZ4AV#hnJJ&{KvOv&~jw90=Q0`^+MMW(dRF5{|^c)b?Y=9Et_gp+09bSNvM*h(a zQLh3jqF&^zoB{kfR`FgnQl3>K5dL1h>-9~e`5O7QbvO+2V2oB7rSmfcOS~LoXDW)w zd`t_WmoP}Nl2f3AqKyWhHISek=UQr{IkV~zc|H=f)m3SzE*1h=;j5hxkO(GY3W$65 zZ@-L&dmlcN|MRot%NmXG%lkKC5u;YT#b@w6I>qWx0ggt#49KVlRK?5(*K5G}RB!R> z_%rv>B7j%dbYHyB*!~5*ipVs($9_S^N;KN>V1M%pT)h2HqCv+?(lZj&_{qO z#P3+rxc)|14b|${#T2*(l=Vm?us=QrV4^{t1qV158$^7JO#?J&M@Xht^{;xAds64S zMzOgoAL4)h)a;W4cLAw9KBJ3GL<)B~Z_1fdgw$NxD+}x_nyDqVX`^m|ZA>w;H|~sm z-_+y&-acuuIESCjG$a5`c zT%6=N=YhEgDF#*08b#kb;^$^KBNva~3nX2?x?zadE|9`%lSr#ZjIKCj{+JIt=YXo% zvWD&utoVV1^+m}(D5s97n#;nCa>0EROo=}k4uj2mO~GX`yl$kzsPR8XIPgE>$Sk6U zaNUUVV5Hg@cIy!K10Jr?1xE-_QxWLdH1Z7Oda8h9RLdOB)~>!7b&&5?gR(MxR1fHz z`|1y2^Z8c@eKE8XS!iIKlf-W~xPj|BCg+d}C)+9HxECv9*>dJGx_R6xo6~|twQS%}jd__s}lYogNN;By_<(akWTp~oy4J~lMQNEos` zBw}Gx$gF8G6rIxw5JC1rP{bAZpH;nF0$|K*06q-CqT&!E^m2j`UvLjF>esIf6BZw% z)x6VeRfoI+`kPE_t=jLp=+C{s-)Oz))6Iu!X~Cl%8jyWT2*9-<))0lkO}8)IymMIhW6A&16D2mc*RU!do2V8U0yrxe~9qJsM z!<(r=|0RxFa9p?)XUlVKl-`I_gJja3wKexgf`@R#%#}-_q)GOS-Yoi{rTAo-2 z@>ES1dL3Wy-x?!SeT+WvE%yn4H1dHwxc5fJdlLIDETO*nx zIAB$%&m|bqcxjOBmnmQotav?x0H!%n_-I#fG52|fQ*A0aHf83$I??gJsB2uCDv!(rCJG&-l@*65#jZFMn` zLJhO&GIg@{5W61tQ6sA1HTC7+wJ}X6U}o>g0Pu45#?KSr+0{UTy&&Ce4Il1$u%Cig zIlJyb?19HFDHjp^A$RgO4F-74R`|B85_Zf@Laz8v08j?>&^ssGzT6*fj4_1+ousZj zN1is}qsFcH?!Y)cCLs2u7HS-6vAT~E*(ekn_l|M!owtd3DvcsJ;gCa4z|>`LpNN zby6;+q`AI(1ZZk&HkRr4LgP}r+-|&yaZ(8nrBAkg3we z7}~7XNTUORt@PDfv`n898imbi6#@oR3P@2D_*ypj0FGA$Z>N=;%M(>BLrk9qux`yh z4ez{tBQ$pd>PVgKad=4pg)M|F3!o`+tt}40jz)@SQ%EIkt8_G`%|u;KwuqR3@;l&j zsgUfh?}4^RV2^dFW1s5S2fe0#fKXmf8J&h)k@7hX3jmlk!q*$L3pq^}LjLBv;r{k{ zSUPJ7yEJ=TDMQvrAv$-`YUC~y8u^z{LO%A28|a?(aO3JFT8Qk1nU(p_O^NB&H(!Od z2VaCd&u$83=SA zm%?bnX_)!t&%?u?|16YA!yWMXHAHaR#;%4k&53txek{KU!_f>0_2#7^czuNMS9ggb zqHtM{3=YB?=b)DTOk&V2URfYCUjsNjM@hHQN@JPSL;`SIL-lF{+{A9C9dvfKkRHr) zZK}Vvmc!J&e+WAh_rnb$e!Zk3S5&fnV`2C^e*wZ9WUm>Q)5nb4e0!K?>cz*O?}Veo zNJwP*!a|Yen^X~(Da@FfTZz%6@qv73O;BllvIcMQAbjtmk0^q;7AA|wVXC+XU_J`Lku`O=verRfA-fHgwF8Qr=Jtq!l^(hXv72xW0x+4%U3TWm!5Sh zR^f&$&rQeYZvPhcrW`poxg<)enBaJ+Nkg4AT;u zOQd&j!1mUbiJ0NAp@=vnkq^7FG#`HT55I)__9VQ1n<*u%Pdq-Pl=Z+5i3m?qX__Ow zlIb16m|O{+V|QraOm9g%;N^uSqIJiikD?9gb%+?#=$3z6&>OKHXUVhZzSrUOM6Mdo zDR?si_fYO`Z-j|QUx()pK4YJ0odAfe zA+nf|LV;2?!eR=~D1ogIB5|H;2^a)y{w)CUbpkd)4B9*Xss5bvfevBSJ^k_DMmW`D z0v!#%QEu^sV8db!4P?9q?=cM*Kg)fK_d{x*wXXskA{=BI1W+1O&$FpH_|)Q*V|*ouOgm!st^Sd~d00@QfJx3mdWO`M3@T0Axw) z8bNiOJ}asv?nkvqtO{o1ePG~xpSKV}4K^WyCa#mkVRArvta4?|HMCXYKhMJ=X@pII zY5c0jvKsZ|f^mIJU1v#&H=rv_Lu${ehT0UWyI>U{bO*MH+l*hXyWqoH=Z@mHX9Epk4ux=4A~VX)rY(mRFOH zA0ZoX$hcpFLjLVu8@+M|but9KHoTWxC|^u)6`cp3-{)SZ0=-1UQEGk^9R`4ttLM4X*^o2rzItDxU=>W1im_y+zbcvB1_IOV?mw@s^!+WV$gj-GOC6g}}*L%lyupoC9BiM1?(N-u># zcL03ah`L!B;?QZqMkMW_1m!@ufL3ehJ;*JnBzQ1&}SZ5;I?`ZRyfqeVfmB?!kQ9Aj_}7!$I` z+{96MIJg9%w`T|fI$ezk#|J9qn%nzBZKj3KHa$S+dm5+Ku$l4{4D+IbaQ*-@EVLn6 z7gkVpV8oF{U)P(MYz5|ji`#q!R7ai z<8@|{-xTtB#G0=FHuva~eFAVk0D#3xIGh$FIr7*Fa5&`ts4$hi0oKIL-D9sjK=zs2 zLV3#HJVOyRB6}9_sBl}T;yQ<;Rm%Wq;&*VWI6r`jWw<^C4*vYq1Y}e=cSN8%$1YL9 z$zBstYoIaq%Hn#Mo0^KPWg2PB`;t`Y{sqt9+D4k62sPJl18hiQ+X66I8Qex!`xMTN z9H#qUKP2ki8lzbIM5G-AJ=E84sD{~~@Ck|3)i*YUCxG6b!J#m6>2jFmewG&I!uP)S zKKtW!kV&3hATTmE5{;$Yz{n3&URb9z&dnCppEk2ia(3m+PCj~w@kr1So<3ZpoEw;l z)P`};`MCe+350k!D{wf!{_1`h8XO4uo`LA7Z6I^w+{ft9aQO6BzX-qj`F{%Uz55=D zwmO@%NTbap9F#)W0P_!rG1_ouY3-8dJQ&GRm#6^epMAshg_$rkG8Bf#$IvIFmEg1P zB6}5_yC;f<9g3RH@mLZbnIH_mtrh;x0VBagBk z9vkNjRl?fL6XwzGQt>@PsvG_bVBVr3`+Tv@{K0s$@p2cP`k)fuCjxCBxNg>A=IPVO z1vK?^4gl;xMr{ZTDJV2ia5DndUe7JzYow(N;R=QYs% zz407rxNJOpMAYO!=Yq3;A~@qL8qM$?jTRUMi2p{mEwqx;YP3Crvsv217_rFp0C5f0 zF}xyqKI+U9vKaymY2W?qq1DeCu~+HIX82?l$s}k)VzgbuC~!6kp#eQY=WKKCD~5!5 z2KCSd_aFyJzHbg5XbPHwx<(Ul0g!pQ08N1bf@Sn=1NZ2cy4MQ&3TIIOX5>temjr&dxsa zVZ&vkfD{sNUYrZt#PcdkI=azDvM#onDG<@w!CB1{L5uPMfRJ;%H1U+%6J%kpxKCv) z$^d#xu&NDp=4&1M=t!sD@vBfmI*>D(0l;r{B2bz$d0dV+y?!?m?@c#u67B1tf1yBb zALj^j6^79bFP_nQil*l^wj09EiUZ_WX3=qLOG})~p%|4Hq(!sOs<;jaX!!UoP`1i`FJJ?o}@+Sx>pBfmCC-(FjQv)aiS9zcX*gA88$^d#hHoGWo+Zz^GBhA zgJSb>kpUY=iQ!g6SyYC!gE0zBhR^<&%L`NVxw{-jZrqG#S9vt5 zz6;+{eVh1xSN^9g3Hm`017Ja^*(>LOl!jhiGYC`Cx@5_?U_@PP_-aqICcuvkRdXnB zR6(`R8h;yh7jee{%PI#JT`gm42-A+H*(x0PeY}SqK*2UQZDh#NTY6q`crA*;eqN?! zZRl#xw;dKwMQx}#DjXiNfojt<-UfLzD(hB>?2sEq>Fr}!vs65tTqMH+m5L{TwmQO` zj&UMP2xw&3F!v!qN~07xYdn*AXL${DZRfN6oC6GE4Jo&+wXHC_MDIEjWM*V641Vv= zL(}+`u&Xk}lQ6|wQkfty-kXsek-yu3utC;CW7tj*B}$KM|+eVC)sF`vDZ=g z>Up@ADhjJ|Gg8Dpum) zd7xzwx|X>28+)bjY+)sI-+C*ok+Ru9@pKG~L0TUKwCu(HaJD6Jq)Q!Dw<*#cMs{r~ zZKr*EzO^D(3q)mWNwrqsmNX)t>}a^~8H@8_iD*n`cQ+nl2LObu&=?z;dN>YcaW z23TwmMWfjv`%&V+DdriR1qa6y1SAX(=`_JJ!;xDgl1BbBD&hv@Dg(HdVC6nq@lVFa1vfTBO!Ux%C4K?7sR{qR-h09bOa2D;k9(}_tAJALj@ zFeyGO!THre=w-n+*c*b5`$V_^iKWXldyXwicQXcoG9Nqx{U9fN0z^3LmT zMl$#;5papLb}4V7qq;2tCSE+}$TAj(eOm(<+w{@~u{M<5SYHP$UBNphiU>F!8ygL8 zy>W-iZ|0P?u!d!-WnX+7?p(hKdAkrn-<>yZhoAjFKj(hog;-HKFhc5lGRj^yt5eVz zCZA1)m(Lyp6gB|P{V3JW*vR-CIbJWWa1V9W5n1em1v>)Hb10@;r?b7aN%b@5 zXJC9dI>YtIrbKL0bCLSOy+x;3rNSS*{c1W&^yIm4EEG{lz(Wg;lX_xfoyaU}uio%8 zZ63VM-lODkL>O(y-va5ut5;b^D(wM`Iy;f;!&rqlbm%OS7$W4o@^0?AjNP- z1Uv`$QKuQPwd%RMmq{!{#FOk*3(j6No;XP_=`Fa=!~ruYO;B`vlwBM=DL6yn-Qio@OV;yp3Ll7Rcx@$b&Wju}^V^Tq;3#Oqa1de!f5f zYe&!ku*iA{@+~?e6cMvjrjCs)#%AuUt-MO3KCWpJ{gcJv-UHxc++x&r`Q={Qy-BwKG7M&u?DSBAH<#f3qt~y7 zFCNW>?NA8YXUT949&sZTs^gcs&@J$aIcMvDx0Qu;K&|r`qd#G)4-QNDe z7+^W3wNke6A`CK1b$XJvV*t2wZ6{794~xE}y+}KJT{1yVXAz57DC2pEf`jlh0>q_l(#El19n zExSNr$~pTOsZdCZ!5N3tS0ikkH^C)#v?A)l%xoq1Ahv^}!7_I)XGxBjlk|H0% z_ov4ga}2w4KYMwYGD>R`#P2v21vjb$457&Yy+k1f-7 zjU2KD-WOY6AXxIacmXLvj0>P5BVq;sgEN5B0Y+%F=R9MDlq{a5GM2i5@Pl;L>~s7_RI*ndZ^ zg$ny5Dmi$92!$0!0toZ%8+fEy0EEx!d_>pUNSc7obcH}ikZL)x!=yJjI43CXj?obS zO+P*LmKf_a$7rOk(7|MfIrvmtHg@G2w#L|(vpbhc%^{o>FJl0hs$GZA?gU|peb=LoH-SlO)2e#q}LaX z5$`K;5Tf!J9AY--A~R0c0R9J|n^d&Xp;m|s9aPGg3HRVSqmyFFwb+58(=F z{k`oK_;`DvnX_#}Sowt;ymxFEPUF^EBvYSg`F+}L<76L-v<51HyP~}MHUYY=%)KBX(u7Hqn9S&C%HMw^1SQ zTMpT)sGhfNx=d$M&Rzzaal|!@b{R3zajjl4vy%j9EW|4=c5#%~H&Na#= zjX|L7I#mlguo>)y&ayAmDDvv+|MNVird{vI8BBTq*f~DWv2I=MOq}6*YqVi9kRaF= zY;{6p;&ruWW)*l$uJ7LQ1Uk_;h5`WZ)z|0%nvsd! z0nmNg9F7HFaKDW<@xEB)%J;3UM1=sX%Efnn_`R@;uCaxZdsj!q?X9=2l7V8xo4rQ) z4DGp&C)Sq8b&3fCFS%xd3iht(7%o2fGQ2*}9Xe`=w4+aLOJeG@!h0K#ryAC|y(bsG zA(~&49R-LHslw@LW`=Wt$al8sjAL`x&!Fwl2hK>Bfc-L{&+2ZS42$IyDlj27 zf*{5Pl#T;&3QvgZN{?2UrEq=F)YAf7oVOK3}7H|?RkFm0Hi9aRnp z!?8Xi>MeF*f8&GZd(+w63^GCsCeBs`3PyDou@S7N(P*6k&ys+$?35kPG?%TW z>*cyCm!1{D8TyEavJ&24(viiS7X~1R077R=HV`1--&Ts(g_|EyIRk3ObG?AKlU3;P z1O#&#lwKR;@_G!x7NM_N$lD%bBWR*p(?9F*#V)w9EMCYasR}!rUm3lGB-15*d5K67 zVQhq6Qtkl{eZ)c@x|n>9<6?XYLx=u{pUcf7x4a%T5y8SH#x=u6r9iwk)k8ucjmu?W zGq}b&9-I**4QoRITj7eFT3{Bi;3xcXT&^xImL`mj27HcSH z8LHW~wpPNx$S_7=j%p~zQ-OX{Il{wRMb-tOh8#S*HVxNjnK_fBI52P>Lzlwv+aHEy zJh&oUn61Mdh}OAqm!4Cl<#`muI#q#maAzYe3^CG}rh9e+2K1c!kL^o9+X`h3+#8j; zsoYbd2B!8}6n36b589Bj<87j1M)dP2;tD@QqlJAY6tt1*Et7J{)9mMvgL%R;kKY&x z+j#FqxM3cGLILo=q1qrd=}5FA*1HDaWM3-<)oB+-B8`Md`rIG_Do9VU_ZkUg5^o0Y z696L@@&M4Z&z^P+QN;w1vy8os6k>^WI6+Y?t!{>vE3bzQA`R5I`16;y++v7ECBheYsBFLWywxffA>5KEB!1}YLkko0*Lf@=yJpYx?f5%!= z2}PTb0}NIp`|<8O?}YmgAB0}K&dY$3oo$pdAP)T*I#HVTuuF2EjxbSAmMDP1WHKl9 zc`UzgM6T>jv_^U&gA-xu$Mh)wBo()dVHOy&-bBFU3mMr;62kWk+Vd|7xGilXmpLLf zv>K2vcf|RLgjyoTJ)KQqnsd{I2irq8T_hgekft|>1ifHo;falrEZ#mbM zaCNAM2m^|c3B=8nA__hkhAI0`;<%imq?$PEDP-rZ_udV^{ky*l|L%YOw-h`46H+EU zL@X&j;k?Ve+9q`{JTic>ASwj0c+ymCH=W$^xKB~YaS{V$?j|^J`zthe&J&R!HT3M^ zBPz&q;qCAK(->*7^LP(@wQkPI!w2_CvG2s`9<~x#CF1kvfAN>$XaC`UK^LL4>FHL% zVHmmmTDX1pZ6^LQx~ zu<5JqkdEL^CP{-NQ1rcnD02RuCu-c@Ge8uo1fOS#HZj8(0V9AoLFiQ#wc~*VOd2D* zu(uMGZ6v}J({;$a!?We~m_pOo@Ft865fTi5JgeAtg7<5{qj5qrBywv&-dDaR}B8_(^4dwI};Mn7I47s>m6 zUy8VG@D-(E3o&g<8P!}G82$KZk0{AGbPvcaQ8 zJwA;dsn23hYN+-^hqWQ2s1>Ly?PR7Z*fpF@0wSF_+2`oTBle{oCu(MDA{GTinN!Z$ z55d;e@w;RoR%0Qf{JTZ)2ORaY4=e%5lF#ykKcQSTsnbTgAVyQg&N3^@~w@H5nJS8 zhv?xUj->AuyvvU_BW6K~SxNU*dP+;aE!sv~uIUKvlTmE%=^^vC9_S3RPvFU-;vAhA=t&Ua(7m;XD8OfBI<(!t?0e z?}x!V?}vs0mRf}4;n$yrb()MCWwQP%rbQ6g z6lt`m3J```hPXeTrD6{-n&J>;*=XnaZ83niod`&pi2;t@D3HcL-Gu`|@Af7QW@p0; z+?$@k&hYM^{y21#udeGN62`_f^$o^S>v;(N+e84H0d1%-gppa6q)c0R7NcIoo>0yR z6_F)9TfM&==`mX#I9v4q&p@wPC8v-bBbX_a>^ef$%=}Ef^;6y}VBRin5dA^0xZnw& zVK1b7XAZ8@P!y2I(~}$`f)%WH664{zDUf=C^5Q%KM%51ZTn|thK{-a)HxbM(46czt z#|_nhrkpuzrPChYF9E)stZ?{M2}R)r=*Fa(#BZsVd#xV)amtm-|V5@t)m9o7G@z`>A=f`7 z+&(fo9G*_k#ug&>j5C#NI;F&!;AO~{oG<~cvU*HvpexSNWP6DS6O3}&1}ms%ngbA6 zHMZ&KCuG{r-^rxL8?FVTxL!V<3=0d3;nur%;cmjSs4IjrDb)tjPBxqFz&l<+*0Kev zrE1f(O`(?&G9>pek7sN|RxsX(%$Qc5m}lTS-bw$Z(deuc7nzztfPpfW*3*>*1_8w8^$55AuT=+5A&FQf#1fnYDL<}WjJW0FVBclIs(ab&nkxM zlaD_}$<~B7KKKs6zL7C+^kYK4>o{i`ksJnXeQ7at(>6yS(L&m0@!{9uGHGeTzW^=x zy~tK^gLXXVF9A-H>+F#((SS)tS0OhL`ppx0EpxuMINKv*S8!CQdhh9{9m^)VtO6fs zl47A$XvYXN0}>kmVm2i`B=uEBo<0E=~l5dr>w`p~^Lu+F`#-`;D3DP|@2g zTEM{LMZn>6%L%pVsL_1CKSc&2rwBtZd})k5WacQKOU?{852XU04u{7=CIMTuf>Sf7 zdEjp@F8Mn~xcQmJLcIg>=01>R)?Og8N0pxA)}(@z;31z9JyX~(-Fl5EWVK?_#;jGY z9Y&YW)^qmSo*^AHqYb7oE1b2(i76Zr+MXa6MrLc-PaE9!^>-0Xr|^O^+JbR5y~wen z&m{U64(J(?45QojLh~~OcShkgi1IW|_xf`=#g3Yiw-?){U=tjhmc+?xrfS({(w*oC zd4gMHD2(LT6V-hY^i|g<+E$+5^+!)&?-Iy{(X!a7pP#!9aj-gPT)Su{%7Pa4xw=OE zogyW?g@cm7$Z@&HAxWD#Qz4`t};iR4{3OcA6 zhpr=C2|s%8I;a*q#MzODB~7Mms|#$HV-hVMGDrYy319x=Z$s_IQs}1u;TTp5fBh8mY~bza_yW$C>^F%kxlCbj4(<~sQ^Bc z6u=39kGW@chncs=WP)rUeNp-B!RyzmSO}_yMI!ctvkz}#=>323(+xn^95=f_DDi|& z%grOVLYBFdr!DjnOF}?pK=*EaA*>$NhvE^alW_e$X|gh$65Dy$h_jIIz$?GZgLZPL z5JnWPos^5kHTqWVVH8n%Y(x$&NS=d(j}b|w=RCCC$CCgDqijfDjOrM=RuLQaj9#Y; z1ZX+pk3|6z2c?phG2ntdqgQE=vCV<^QD%#zh4ut}C@cra=UKc)fJYYKS&!m4qfu=$ zso7pOe-p*gT_1}>002M$Nkl71+a_8M@FvencZ&00fOfGX|TkkaK)l?cV!mn%ZSw}|IXqgd>xQ^EMA`27X1O<_E3 zHDZ6Y8d3sD4uL7$ECO1?J?>EGVLCibdxgP~!Ei!#-h^v}=U0Gt&_qP4j50PPZ66zZ z3>kq?>mCC@Y~EUfLa=p+{28%xiaj%x=rjX+#c2$joq3D`w$(}>T#F_E$mtpBZ~n|e z>_i1Km$Q?UZ6eHUac~BD!&^W2Q@Z8%g=DUis!RAga05~(aHD=Y3l1NaG^!Gn1JGz> zgmo04Yec&t2QSjh*Yz9j>*kgj(hO{?gD)l3{ts~We6Nc&|3 zBzBa?@U@fHK17iSKF>+_=SbTmSYOu+HVB0U-(q;LV+3iR(pmtJZo&n^+vPJmheuh5 zK+m$F&^|`vV=8lPGJ% zM-MWi=Z8}4?CpyL{yoC|HXFTi;|5-1NBI3O{vL|yUU>8FTVV?Bj2xaWy!n}l8NB;F zcq<(k9~3NWwz0SvKK$ULIGeVab{!egu8U0GC`W*PLZj{PMVWTs075{$zwz=~iN=jj-K>n7_Ag#rayA}vuv zdy!6*olc>VA}0eh=kpZhz}4d1Cy8Fx@|sP6o{e&C(UU`Wm6jI6;-d%Q#$b2IQc1qI zwG?$xf>ebOa1Dyx-vtD6dvpNQNg87vghLEx6TmBt(GZ+VhlrJNM1k^vV;vrQ33_e> zLpFfj1;lo?wT9^@Pv}!gu@P$^(6WfhXq{D`ahN?0jA;}BTnn$)2%63mSPHgcl^uJo zk+vWq_TiFOhK{q#zIwsm@d(lAl`sH2Nwq4K^HqHwQ#fcL^)Pbczje@bu(Rkn`+1u7 zk7MijnK~g>e8#gxe=6jKM?`N_;=Ja1^uZCpR(_aK3inN5W%|!z4q$iu9NlzOCGton z1?NQ$sYNn}oa;hQPc(L&P%iQSo;oVJZ6;o39nQGs3f0t`Oc^=h!D4m-XTTKe5zl9l zot!(M7XZ%o3j!s+&}qlmA_1I*4>lf75>jf$=}Z#+Z$;j8%8Kh7fFIjT*b)W=%a_ii z`(PwXmTiOmz+9yWt&ySFN0O%#-Wjy?Dlqyruz6yA)8N_R(h{tWd;o zhGBPX;VYh$fezDcy@MlR0p}~zQXumbLf=iISoCD8#eTGtf_(d0cPwDi`0pU+Isq2o zxdvtjjQkujhx_jBTj5tf`x|6^I=p{*fWjh*7N({-pXo3>+D9arf*#iV{=+B8VRP6n zv&ZyWoH~I9P|t)J>|}2nb5x-QC7aQsdjwVBa$dUr8cqbQrT}&4*!%-hxQ-AM=osDF z#Hk;p=wb(_W*5Mi#J1Jqbj{%y&&{oHzjD@~DFAZKin&s)*iJJFWpuY0CuPm+>1GIy zS@a0}rCcS940aWWm;+`2M1t#m3U1n|f&&cTsPMiQP%d<9e?qj5wwf8z@ zKIa@DwzfFOeHFN8vQBTj9p=FY73`ooK)Y|ZjCz1u1HmJE+MRK68~zfX<2i^#a&MdP zeWkVxUcb({z&X(t6HpPrUGQAcn&9aGD4bP>T_c6D&IK@j2w_umAMO|Zq4bn#&*9A( zIg*rX==p?$dxFucaI+Oq<~DQJQfWFIlM_B&o}+@1@H;7o0~F~gua6Np4wy=!g@v!D zzyOUC0Yf#c2ynot9fEZA-1M4i`F$&MSQXGcB0z#T6{^6)kvmCVlV#&9II!x+hQKan z6Ng?xxkJm%NtZAnMwd$RgaAB(iqg_9pG7t4&ME!Ac;C`G3(g^~X=2VIozQ7hz;3W6 zgfbU4AomkVY7d8x!jtLg z>5xVix|3A$5m_oOF2GgbVUQt~AGv36IC5udTDeGprX6Q;$b0tWEAaO@Yut+0M))U-(qDW6d(O2XvWO2dOXQeaXa*1849Df z-V19lpMNr!j8^L~u#eEh3lg?j*(kAC!Hs>evl@cd3` zn49@JRy_)M>n?CmQ0AY0@=3V;#+$4eU?b0@g{x2>_SQq`Ns)BtW1=clcftRYqtgPw z%W*#D)}Mw4pZzXeL?;L5jMfCE;kVNm%cGspkCM)%i5{KpglG33!J|2#-&Hr>Da}f$ zFa>6iY99z6|N0wJyvyP4yEse$tZkHaFQDF(&pDv623aZdx0RVrPB}#``aRkUWN0CmbXk;^8|xPZ#GL4po-ZY5`ngNf|Rsb@*j(S6B zgDp4MD$;uH({(UaX+O0XX<+TDX(Nmt1!?kET{E9Wd!-X)%GPRgGdZr8tv`HzQ`a_y z*QUAF?vDUiZ5Ef2|1*W>zl{o-X6BZwYZPC=E4ilpWmmoWUe?id^ON{p++*&=h+TC* z*-O{QH}EBh72)9m#Mk&9xr;Fy#b;= zV|(^I+#c-=qqGg#gC25#W9?@d@bK|XL+7J7TaaDNBY+V{eZ8&{a$Okjg3U`w7T8T;*; zY13C}3@*^!B4{J<@!zI`H5z7jtYTi6TL}47Zx#6RtUh-Y_>v948P0NUj*zVy_=!3c zg8ZuN!x!W@7zN9c0kg^%@0s;upLiY}CIRmWpW&B1DofZh*SRsxnz1iY7Pu$($4Iww z<&v9CG&^GCnqS4W@N+FZGHUKw6STcbXL@erjQeKSjZ5;y=kgj`t(|d?{ydGth|m$; z#OY)|Y*`j3J#ptd1 zln!aty&?h#qBNqyBGfb92W$}^w8DW)13*k=`l%B0&5sckhj&C-F9KmMdJSW9K#IYT z=?Gr`4r#D$yjShH)j5VE^}<|CRCE|6U1NbmEgpr<2K62fF_tn6yrz{>8Z|en0dIU| z4S;Dx>}(+{0|xB*RLjMCx(7m;C{=9=z6-*)wZ=06B$@yhS&Vs(4KW|SLus%?NqJRS z3|5W}XhEoocp{L;!uj5GI5Y}D!`AJ-(QxI?wb1?D?^7{L8tP~>)NZeYy+wF(gqqW= zbp!Wm&#h(%vUdM3H+ZP^^^&mYYadgcJJ*Zm%9t`QFp!Xt;vkgi01 z(}4=r0nw#q(r)`0@&@)ckEhSgai4H

      >ar^PYXwE@+Ahkp1xekHR)!=(9ikHtftz zbB{bbe6081x(l$w3xiYHkcSt6G5y`Wk3)BUAe^$c-Rx-#vUhy20{B7jIqfQUiMA(% zqng<3ZIqnEU8gVXqXbSFm2z;nO>e)q-G2WVMwu1+k2D}=f>CX?CUgT{5`D#)u#zf@^S3lDRLHwMZqYCAXx8+b|4_jfQPB{D(h- zzd(Hs8<>9v#=x*W)6?!5%m}USF11=qt(lc8bK$;|5!`}|;O^h=Qk5kt7X~BV_rCX@ z^PJ~A=Nu~Oc=of~%NXAqw3aBdk7yArw}}k%uu{8J$=QVa zi!Xj3W**#QtV=okyZ`gwgn#?5|M&2h|NUQ4NlR;iw@bX2Q&M)r*!T@t*es)T9)>AfLkwCSk(X&VC&Vtiw+yegFun(vh7MDGZ+}7OO|wT1YRjZml~2 zea_hMNICR%JqQQu^RZP>wha)Du)0#FeF}T+hY$M0tM?0GcXbQ)WH{{39fyAQ)U)3` zhjsW#tlUcHdB%(oTi>KFA89-FDO%if_K?K_gurOuNcT9SwT+f5bJ1gN-B&Po`vg=1geLD_#pagDs1S)_GbuJIb%&kh`SAC@`8+&CM$=2c zc@F40ggqNWd#V=OE)PdXRNBL6Ch#Ryro$Yv*S8Zz=ARCVkho@mnPUHT3al<^8w8j? z-bF_XDbUT?-bOaX^6;AdF# zm{R7oVvY#h`JNh1e@@+{!GO5IVy){v31;M8*s-vcIER=`A2Dzfx2t7gbCsf*H8NmGk(+T<+W26?o3vjVZ zlp?*X@yl7F3_LJq#tIMk~4R?3)Ml6as zB&Dlxy-rHwCZm%~A-$bL2gCk){3&TRdI#?5_u zGs&Zy+b0@xT7vB|a9wc^SaGZ_Wa7~WAv=15*Pe!z)ioNn7IFB}?z!-u^h5!p)ecDa z3?+EqF2HylokTb3g#+}1wv;*MbWV3cr6!Ot5J3KV)k za7`H84A*28TL)*uq6Qu5C2Lrmxf`p#>a}Gu>U2RzGr74{rHSxV><@MXyPbo5UgtAE zp&@Py58pKJ=GtPoJ9QIQZYL}EZ2^vq?f`IO8jkleed2h+PEj|cLq!|&g$;10l;!2 zeE4W0Oy8_vV9|~-=^Wa!ccAlWNm+<7PpOkVhTr1lVDI8Mst4~Mw9!d zuNE7U05Hv3FbkG5rUNzLz*Iy~P{7d%o@{$^fW}YV;mfD!KVeC9KK9x)1u82Bo`As{ zqsq8W(Bv#s`WW84&A(qesB@(!yw?GrEvuybZ7|lJ{{6 z;E<}a2Un+{780QZi))H}fc6$mUQ?y2zX20zn-dN-KPw0kYO&4{eB<~DwvQzzWcT>K zqtve17+F++5VlirDwv5AdURIkMAcAcX+2aUfU%e&L2$52%LxoJ-(zjqTedMV^>6W? z{<4LX=HoyA1wAC+hR%}%z}ql0hGCAeD{JgswQl>Yn+$LV`^SEBfA=?k7j90^gfr~d zum9%%p(%O`eK2#d__W1%`C>9a0r;-4~*a#Q9Yl@u50B8|Jfd{}TS8AK^bNb@G z%3zdrv_dqPKCK_11?KGU@Yy-eZf6P>9n$6ixyJ?P;sm=mb?-JB+d*u#g+H7FKES@L z-+uL$^#t^BFHU<<*Z20tOInqTUI%^Hfta%A1$CrZJ!By+*c-`Ijj;_? z*oxsBx6t3xCWNf|95M|Sh}Whgn{Q+0&J;F*`*K8|qu6XC;&dhfO>2aU~hx` z4?xJ5q~Fv&)Tq3W23Ch-UnO-aE^vy4t3YY?v1v0{M1hRz1a)tGC zHaZ5LieSp7g?YrPoopzhy?s(4E`p4KDPjIEf`O=)VBrKW4Jyc1R~ML#xr(D)jfD^c zMMTUPWSy$avMFHO=FXQC=WBe0wd!{Sb%|Lm@m@;jAqZMhI6>EhwAl{C+LZ2 zo4|WIUd#rB4OoZN$UsQrXm&MZY%Z3SjZ0%$4$^h!dr}D+1FpmCRsc5v_3= zb_kTl=fNJ*__doVz$*l=mP**dAy3-wh=Xg7DRnco4ce;^`Hhba@%-tc&FfZ4DOp8j zqtsM}IkAA5Lvs5k)-b+Ec^U<5}=_XVH1y+cJV zhO>afuvgR*n52c-ccj$YNdv+haZk(36b=kAXA|b9G&&x>efcsn2}!_e3q~Y$|+#VU%YrIe-}>(O0DB7|-~eo<9q3pCdA_gz;N7t<2-5XjN!eetT7BMe39r?aF*jxW&q?{RP~;diZ1}VPhjiVzAQId5gZhE7r~

      %*1%V&ysL)vLpMgS2XUO?`{f1{!=w<&NqM4keDs&?ZbMMgOJSXwo*|C8(sO3i=*~ zY5L9#Df)dHg8l-aNwZTJy-qbg^v@d~>A?wa0bCJWW6abEEzQ5@Jd!4C6W9^Z@Y%LF zc{fM8{QW9g$5C`6qYU8A$JA3BYj=Q?>g^whj(C1{jvdaLzz{SD2rfyxIO;8jGcf>b za$TnY?-QPptzjJZ)r~E54zeQkwdKXgG)>*SNx*w@J%^ZcnC0`%+2`5>nHO!s8H}(^ zYXwyXuAMwThsv8WERg1mt!`kuWz7tduN%q2>cT|&j9S>IrmO86I5axSYXAtmj{%bX z-2`y{tbvW^(S0mvaS;2&%p_;iMz*x(;Oxo>C_%AmPY``=Kd|HMS=kO*BvVsH`-;l7 zDS7p0Y~W+&{u7a+-_Eu4^cG?BV7n+P(V5mFfWvlT zr@Ve`uxlZJlQ|Q}3IbzgOYy?(;(qktD40vuX_WwKA7B)pw5`5Ipt|U@y2oSu>eX znPm4@JFiIrXnljSiE{7AXLb6Uor1D!EAwDEbY^IKSkEZL zwo|kex)GX@XFHpv6A#PBe*l&VH$S4TqK1; z^%r2kC@(|GGsm@I1ppP$=pb{WQ27+8eWP7dzk;_ci`0k1Qwx&e?|xsplqVh2W#S?` zo!f+0qs#x|ywp}udzBU)D;Vw?n`#FZCB>wN1nH7XHsUT0*{DywRu^m2h2URiNfC7( z0Cf&HzT&X-!78R%&n~ob-GHbwBFyDCF9CE2)k%MK_kjE{a+xv}aZuRHTZrm8@NP0A z4S}r2!KTKhjU#Q3QDC1$c2o^uD;M2Hb)KJXl%*D?9VSw+&|B%jF(3r*qdE|Se!zX4 z02mH2gxy@zF~Bm1?q~?TLuH7=Nu{HWj0Jse+9G;CXDyziCrWdD0=ZbSkAVZrL`Sah z*~t_}sv{J%Sb}fEUfUiB(riR~&ia;MGX{u`itMQ*-)lpQxJK2^73*tr=uIMwc?9>B zo3}#W1e+I-Ugfh7VZRiQ3#+J% z;{}f5f{|9HKH6D(4}FfaV#|;Nz-c=N`+!O{jg#qc55H$cC5?Kh%QKm5dMskBY8XKK zw>9~_jp-rn0K_YvA*Tocu|X6z7Vo!>fMpJY^5}!dR9M1%@LoOqoFef4?jmF5#$!># zHpb=Atx^=N| zO12koakelFti#~MD57RYsFCU}Q}sGMGZvn|d=YN*x=T{{mBFE~yY@bO`T2k0Tx^8V zVGLbs6XtOXM+M+S_ONReA3)Qy5!ow@;>HdEAA!Zj`g#mJAOHB%_?c&{Wed;1b7Doc z1q7V>*fBp=SYy~_ifw*sTFv$=vl!qL06{VfTglI8AI~_`CdF*_YPGwKLzOmklY+ef z7$|Jd!0yWgTK%ZwZ>j;NqOcpAC3~kVM3&r152aE*ynn@LFf^%MMEdSIe@HECil8vb zbM?BNwP=fuL6{6H7)=3r_PG?;2CfNPtR~H#Yu)StCo%9(yq;i?z)O1wR*2VURja8v zQ&PT6{u2AL+?Ne$na zCir&^3}mnayq7k@U$!{O5n!33v|3)LY0#%VS)ZSSxFDm$`<&y@ltc(ri;xL91m|fO z;taOgDr!NW0Pc`rDvmf~pSeb^t>SLMoQ_mdp+P_y`+l+}9rzHd8YLWfRIpnFmt0FP z0{s-WvH&w?!B(8x%NaHu=Jg27TyMz>wN|mW9cTVpzsJ3?wlbpnJb!l$W!mh&7VV?{ zh<_7IyVlnX3_%nZVM=~?gQ`#VNUT!kGq(W7(|7KZ`e#NoVD`@ahv9d>{!bL)orS;r zi=U8Pp?NHyb0{0beZG4?k3CWEXfHNwBmqGY-+Tf){Q9fkhI^G_xP^vv8C~bx@>2MS zF@!eN-KW}COtZt8YPL`rdrns1=BHulJVz^@QS|3b-C11#sB9o9qBmn6fXn?`gmZ=6 zGw?&l6n0=@)mz0Gw%T_pSb&k%Kw)Koz85y_(uGhdpMkmjf%j^-0$OcC$NHw~n za}jOQo3KZ}w7oUDyl<`2-FZG71MZFy1*3)pd|>p5nsO|bDAB%bR(ts2tpW`gEjOp- zxK`U3y-4X2Msk=j2rS4sH+9B^CPDX!?hP z<)OAHz>0WhMJodAP*4M{9uPlkIYMu9XEjrn-XB>nR z)2{5r(<*&^))&JtsjJG={jhLI#TibZZ)BXq1*k{QxrH|5lzX&_RKVwT=K)m#2x^&sYg@k8>B3!rf$=^VjZK+j9q}w z5qq+ZPDKz`M+=iAy(>6AhjGKIM%|umW#SB5_U!^G)ntwoBj=)bXb8JST7?wm()+mx zGK&C#X(SyIDqa4TQ&!Nu92)gn#;nU&9sJNX241X(5$9`5_6m?tyVv7`gSZOyjJOS7N@N;H+x{0*gIqy`x9qiB= zhJbEt3|4@`s?*jy#VI@26`7n}0a_wD9wFM@Ayt*f!PiM?@4^BU zVOz2Y!Axe9F3yeCTR!lMQ{L!-#G+eP~)E^DN z%4aDo5xn%|07L`?t*q_AX#$2H1`ekwGu>d{2#^J|=LC5cSz(&6^?W|pt)0<+opL<} z_pXTp=B+&JrCOdNibJJOAm>YcnETLa9dS`uAC`*ulS#c!xA8Yt>rcTdYoP6OazM-m z3E+74?WBnPIhF-KVL*nIh3i)wf3#nZul4Ws+6D|EIz;@D8ZKF=4?g`VQ*Y{F{Pt~T z13$s89>@8>5ARH3`(cFvtlC^zw+)JsoCRwYu7E?k*bdY5fBozKBTSMu{piMUEYf-Q z-HR|b(jU*3Y*~{a)N0dx(wVjl>!(-eR%?l2^?m4*%MlbG5-9C)ZU!q8u>-_9g$f<) zW2aE4X>)RkEV~&8SHvLJ>dXvlm&G1$G8JWDXe0s_1BMg)UkCn2B50j6FN=0a3ci)+ zoq{R%p&9UI_;Vd5Pu-!ctb0xdt^<7Gcre*6g9d|@p@}gF3d+j>Rr@fCk~lFK;&;Jd z3-@*i17|b-bJC}0q@nw(4aM zQwCF|bxL1lh?GCYe=zZU5dM$=guSr(>UAt$s7&8toY-~@6#B=9qufMbIrruj6~JWl z2-ahooM#rnKKsG5hhZcI-39;#JPrpF{LW8peMU|L3evef-q8xOk`1|&_B z(FCcq^TQo5l;UxEE0h)zW6G1KtuLnfU+3&G;E51MaDq~ zWvR8ap>04k6tWC5G9Ek^%k+gpQlqB}zD`-_4k|_D8yYMgcg~P_ng(JF%9HoQ2&L0w z4}TEOvj{SAbgyA%){Q)QzsW(mNfW&~S5uvl24Z}OA>jZ31%Uz+X=vb1eBi=#AG~P; z^(pKnN~+l~9I%8?owf5I90CFcm_6u}0wV_x*0Dg3#aD|Z3bw6;>9S?6=N&B(I?>bg zPu~e0gEzuD*SSS`ynSs<_h#7~rxZ4H@L-aTj$(&&V<)RCfHppKyq5?97A|uI$b$hr zsIH-zqJ0UpegXEQ2})lF7#?%|Mv6w!=KQ6Xec<*RI>e|R1K6CIXFr;59IzYu7>9aH z1hPpK-NMf)TowEk$_zp8FNEQVJ|c+>G;x7BvfX^Yjj>IuY$!^)xu@;5>UFdT2*1FQ zo&uz0T6IWPGBr3%EIXHBbV8bhgJi0~C`_Xi#ns%Sx<#1;d*ldeJTzBCJ_ZE^3|=RO z@NjpB&54eM*L889m)zgl@)9jGXo^T%n`C7?B#3yHc51P|mlSuFjb6Zj$POAQ3q*{9 zbEFx{ICTH5g-vhz{d~VhU$HfudOl2l{Ijsaa)0&e4b)7ANaq&)j!N{i+hsz+A`N#L z(Zw|x_2?KK!zD7M0e>iu09J;FVzj6JM9qc5Tnj3^cE`&(CFB@GX}VNEZ4+nNtvsLQ zIHoh4_AYv}>OResc|Pr|g=e?WR{^}@kcr@}a_mIIczFavh`6^EgLQ$y8vyJbu-_d> zU%`NV{p4x*@lSsq*$I30O`*Nm$7nr&{tA|h(RZv#j#R6xiCR3rG>N?ch78Q`+%l~S zkQt0m-ym{72><-6UsK$)8$Lmw_J`kp5gR(2t{nzsO$^a?08rUWqvCsaCrKj=hOfT< zLp0(O6E}Fa^ag}g-oW_S>ZX$-f-29?6tbyZwd=hQhsCRrE>cYpYoR1;4UFfmhhWrH(DOAm}8rxMW8gM%Wy zM3Kx8hVQH2{vM}t9+nmt5XYBD?E>Oq2z25@l?p(KlqlL-o9X5N9!2)tHo>dR!c}`7 z_Id~=cqOKo`(R+z=_;@nxQ>pF#q+0QTAY82ErT5f#MwYqJxh-B=Ng+f>u1d(NaAd0 ze648hL!Pqy?oCYV>Dbh}s=KoFL=je48ATgwxyzcVVXcgf#t7eZUL5R=volyztSw6D zBnW8&GW{j^nY?);IuzNpeNs-&{*)QxaB%hnQLtw0OW#}e!}o}V6zo@p|8|2f7&K5I zaDrYSh4xHVOi-fe|B8Kf!dl4O$9;u!)!F!^eR8~*``Kzue@;*n15NCq44fLD$OQ7b zUE~Q)HgSGul61KG=p#UPH`#;kJoXGKxtp`2zEsxkh-}STjq!siT9NP*hk~f9w8raBKCvFPWbf&}2T?<=?2&~PZ*bnT0G_5JrD{R6sdp&VM zuni06bNX-p>aW8x{v-vPs$^uAnWYt`nSA)@77QzlDgfTJtnD$@nAFn4>kLjZ>>c}_ zRtQl4_TT=SP^J*#NB5^ghBNs3yXW{Wiu}g(i)syN#7Ain0544Tsm%w&mjckoLE3guSj7vuNqV1-}&H-8bqyf$@<>p3MQYQ|2jm?>?m z%=cm0+ynM*OaMS-zSovKXYM6HD|kU{;C>ev_h93_Cy{;!&r02|fhcq{97@1w^&dnV%y($vZMYv`{wtGH zbRC#RBIJI-qHv~^(4K_^+j_Qgr4GA=D>;iT^IeA#ORwC#mH!Mri!k@2* z=C2NFVgBhC;VB)R-+cQAy6evZLMZRWzx9wtv3aG{5*_@G9Yo*7V({Pu90EPjv=2iO zq}5QdQVQeK&O^;a4PFmX7*(4%HDCf-7LY1bYjNsM=)3ta0=o9%V3?zq&eKJhLN;1o zWsGV*(n);2f?a7!d$$#N*kuf}<+mQnDD-4QXasGXsKGOp*@sR}16=|PipmvtA7PjT zCMDWZq)Jqw@jjcd95O9BWTP>KnQD-97H{ZnGfygR9r}If!ykv!@)%=)(E3o-vtNg? zR1bxbza*MYBWP{oK@Ioy0FKV7a^1v0!6u-$$3DuED;3rVcb{f;hpO z(c!g-SDm8KKAHiHeHYI-wnt!JWcdsuDGyf*8<_8eUV?%#C~>Oj)>+u z0T#RHNrpxy8B0alVz`1cfsJKtySV=ZhWH%8@6H;GB!(ajpjszg(23!6!1_P_%fEy^ zbW%=D*ss%z4q%lnl*5;^^Ed;>jA7wSNeC(q01*~UOpOjALf(L&J7wie8+1O^cXrDw*$7m0Aja_?O?ZPbk`8jVEU& zqqFXVk?ZXr#yMe_NN11Tm}cDpFbIHUU-F!}GGOID{O!LBcQ8^iUGp@~%+T*lXRaZy z3=hS~zNM9Cm&k>ATKe4m;75p~Co!B%+1Ru&LWh}0pEnDrW}Fb4yl-d-ws0wm#`{Mm z=!-T(#p^h;UFXs9Sjm3H?A9raGVPOKOVjCWtQJk+*qlq;GcXMC=NKG|K{&qvE8TDt zW}v`xHW-n)%kv!i0HYe`I{Vljl{Oo;ADB&mZR-)6k@9(hI9VdoTzy4nq0YzVvTBUg zOS*Pu2n=%Dk%y?|(NL=s8eq)@0fH0Lka^C5DNPx09j9Zia%2OnYExU;KQsvF8{s`* zSa4plnOF`0+I~x-39ZR=N@G@)f3lY;LP3#9QovaB-0N$g&etcIy-rwJXbEl1ybM7E`i^z+d86t=P%2FE7C zA`+YW?s~X;dotX+#o6SVE&2n_gmpk37Zd@q)&K8iUt!nVskr|peD(R?hyVK12cbmq z!U^Z$-RpPZ(Y^7g)83#c>Ix~y^M&o$d-Eb$487AIh3A{dCx+-Z26$^@jaHCIB=O7I zaf?d?z2#n}6;Y(qja{)mX7>Fpd^ZS;N2sdgWkPK=on-iu`;Q_z4Z zgjvTHCwczMbBoc|9HM!(&z-G(98o4=BA6DyTl64K(B{}wS5l}GW;)6USOc>~BR3`h z{VB2*uh26N#(u3iX2sOCs=jzy;UVPXU0NVAjfwz^fVvY_;2z>DljAt*IgyA~``s*$*!5=vBGtj!`}k)XgRSRkga%?7EZI6?Ho8o%?+;gltbG)Iu2V! zRExosUiV&Pl6=h;41)rE`;*!5cW!}}6NAv*-Gd?7M{0x=1IX1!+>sU1d1%8cxbaeS#TcW7629T>#HJfahaapfdUU-btJ!j#)vy4ml4f zP?W9_KgN*AB2JX4bu|#rpS5*E3S@1lsvgO7wT?oQr|!dXX|hckFr(5lj2Wc!1G!5l2b3G(=^&^ zPq9`UR7J=KJJpclnjDFiBD&v(ef!`e7-#mJ(Y9baR{rrUZN8fa#M;TvOLm8!)&S9v=@+zk9-(!dhbB`(etCVW?aif9AMhlhAs&eT#;m z1TzV2&%*Ko*Kxt1`+->LZC^u!o%y+y=zKr=6uUq)ZlQ*HRb>}xo~087cGz19Siv0E zC~I;`S9=~cY>JiSr1Tk$b#D3@MzW{S6JB7``Y_<8u3Oma+kjff%vGoklPNg@$Tg{Q zw4re)j>lFan@9mP_@>>|%*D!8?1n(e3e@$L&%U zKZ6`oeKvundV}*T3yuzrweU0S-DZ`#)rT7^EBqVh0o&FKsI+INeWPR#EL<>^>a+o~ zC8mL_I`^7nN4yA~r$9)(mV35DFqg)eyvOP2K)-M8VbRDo`X*;%AHsI3e`;r+4~&gr zW4f{RWL9vRvV(%9F3y%Y=IxW3H7x$X_x&0unK;V2p>rJ(2Zb-5UTggJvDL6EQGVtT_OcYtb%1 zRq*A<31Z29;N(pyE9JRheXOoOLfhor-yW``2hiO!Si$-Cv*xd{&r9KlpF9db{qaXp z?qgO%t=}sCF7`gdm|Wnm-psy?{Wdoi=fgkzr~eTCmw)vOT3w+5-U@G| z2s}LJrJoQ0UlD{?7zKFu;T>!fnGgJuJy6LJqSamGS^#9=5k8IJvM=_BRP!50Y4JWX z+Fs`Y{=gJ!ifd(4#_x;7)EloLGm(5492pCXG-qFV&-lKcZqm_vVe!>0Z85jV>~IFi z_?WqA#n-v!21HIcviQ=B&&KK^HhwTnJ$S&zVh?gYl_eY#=ot7Uel}40ECE}f(76Rv z=~o(On#H(xz#8 zQLxHOJ)N4fbC{9B;8^IpaXSNJHF52cQs z$kv0U_};?@!5csO1Y@pF&QVd)=t=U0NS zCY#L}l}R@l_q0LPiIqTBoh2FI-Zd~vgb!Oq`+>+p@%}a$7PuY}^*PJ652qq?=(W@j z(EBk{vjwd}g7fGP*ELvms>(UNg!s(`ms{@n(*_Eu_e2oH27 z+`mYFVq%CW1BQmwM7fNfQ5KZ7GAcZSGQXr5uuSmb9+j#X$t&0#$BNgLZ}9I4j7<|Zw0Q~}fR2)zsqJ1&$)9RSc%H-^JcDapP#mAa zS+au3J>QPQKIWcl>^nRBAK~a)NXuOSq;0WK;A`$3BVLGzhi9hP2e!OmzUt60>p&`v z&#G}>2QbEtX&E8#*dXQ9%5;GPbatMfZa{j*R2D!@7T8F%OvUv3x%n6fROl0R`~IEq z0o-#ldx%XS%|Z=y3{ zUMXNqz1r^1Mwov9Nx zHY@Md)XF%>J_!yDymZ~^9+(w~j0!(18|D-bWiD(Q95%j$^>vo-+U(m{y}!LfDzL-|F>Ff{+gv8lknv6t zBowHm$2-QKJEYm{a@Em_*UZc@1vyJEJ zd#heot@0wSMQMq*uKrSXM*zn|3{^(1ogH7dry9L|n`YLTnEKmA>QMoBeMIUst*(>( zDX3JyZYIfzLQWM}US5nrdM|7B-B-U4Lp_~gkP7ZTeD*irJVPdQ8Kx_x2wo4c?K_YJ zyZ98!J3^{=D74Z;@ia}Bh(jzh=ToLUuPg~WsD9O;4}-~PqoW3ht0z>VVAK0v08AS| zqq?~pzt~7HhDe35Q=WAW8bLm+B*BXEoc%6MD@Xk#c4TjtLL^$yIB~-Q zIG{oFjTdM-SKiLT&K1Mx_*i)H<>%pmLY3+&B-6VES`V4VRc^ryZ|e!-I)OXb8x6G1 zq(i|%066yOHX2~%I-|Fz(UxYgi7?fCkj>#w*z<{BP7Hr$j9}VOL3j~>Wi@Bf6b8HV zN}3ei7OTM~LRSRRm5Ir+&|MfX+d&jKgdMzTiVg2qCt$3b&D%-JC5a=X{y!Y9y=Lay z^RP%he{=RlSRe&^w!g`t;^C2AH8s@=P43|qWwMJn{@fJC$4FcUXcd#K4sgkFlL??W zX+&ZC37gqPlnE@h9keg_$zGBaewBSoosNZF%%4% zSLU;&LOtBFM;=ghU2Xim3B_$soSpS$m@>q%T&w8=Qyxjyq95Izm00T-AyXUu9D)MI zy~)p9)OXob+u^jf%~S;L07lWu8y$~_L+W*x-Z~m2H?Zke-`|v^1LIEtSN2oW5DB=c zK>&!hfzVfpG-hE#?HdOGC)zm0`HjNJ)G@M-9y2{~#38Bflk(v+dSP63qQ?NsV{~<< zI{JHQnZX7$I@CDt1K!$NVFNhuR~&Xz0|ksz)V1(zWSUM}=*I+Ww@xdV3~Y={;1&yB}Ff_c0Ot6SUH5sVM(E7g2TWGrAx$(-_>PWZbZ zzybyLc8=)Y#>&lgf(-Vn^Bj{!+F9T@w@CHbB>B2k3sD}rsFoD;zR@=CZ$+)mjXggj z=&w$c9c%K~p97d=zHX`P9=J}=xIQR`SVL03Jd;zd z(ZT5ju4kXhO#wwKhS~N9t8d?+8$3j_wHQWWr`AXd+UdMNR9AtWefjiDu4#z#NfjZi z(c@2kfT8A@@l5BKI?^QVT;Lq5$I^h+*;ir`(}L~jG*6C6^8xUc2dr&ug*O-gB^k1lQoruc|fB=Z>TZop9`EH13@zNFlFwpJouvF@hq zWoIsVE>~zgyNdncEh*1``uw+~3mB(lH+WLXU;O?%gunt@=A%~V1&1xbAVfo&1EAEZ z=x$iIS46h_0XCS~j-<_X>fe6%JUn6oio@Y;HzDK44-+sDbg?mq0qt*Jz2RQyCuX`H z;OPN@7zj@JSIEoN<1mY()kR zJ8GI&@qRn@P#u^Ns^?R|egm7a2cu*5!`3@#*r!VzwR36v2pDWcon(Se4}q(oMMh_H z|17K!T-sO9^C8oxX6*>ZjZ4C|vp;xmd+eF=lqu}s-p+B@+9O|MC4Z9jHg#GcRhy9@ zz#i@K+_hD41}wizN%?wbR;oMGff^XAwTfL2*mK%tpPN1D`~W)w=-647_+GA?T38EO zWD1WC&{JWP6TI>E{LARL`$xyZ$nCpSI0G~Y{M#-Y;io_O2rVtTSW}PqB~#N?{EO?E z!>+%3GmFNRpo(DN73aP4>NpG&gdLMQ+(DaN-P#OyZ&a{5kPDo@It78-`}h)ApEH7u z-l@l7^OPWx#_3x%)3%co{!rlv5R$EnQ)swGuibvIrVjV8iMEO=_YVPJ2IKy2V4JHn z2e*e=3+n@Nh&pVxUbN>v8=LRucvgpKy}^R8&l~Iko)K)9QwU7;`=#_COS;~f!xAQ@ zaWRm*4*-)d1CChb&GV409_Ab+*dwO|^=b`eVb$BZpY!+ze@+RDW2*MixAvfmzCk+L zf+pLb*{@E2pfu{cCvUiN**pL%KFnf@-E9h{@a1Y(qwI^bqx~|Fvw$MDlZmf~B;r0> zumRfyg=yB%H16W-m-s9IEo75&3R!mdg_9b*?^f_s8lMv9*m6A=WI9d=UhJc~zB(7W znPk)>(+5#vmGKp_5v3#Q(-Kp;^k+u*0817#LGQJFoIE-&98x=AY!lG21EN-7Yvr8Z z+rwzNEp@{7D(f;r!lDQ_9+D9CX^M%#Oa+r77sE2enchmDRJVGiWaSRs}P&jXuJ;3iWcz_Be6mW6?8AT@k*dw(#7Age+XuVxeD)1R9X6>v$M7AIa^pJCkYy+wftY|f zpoiB61Mf*UDXgqf^_sB1^_vTV)hDqVFeJ!k&A2AgTpw$*?53W8_VpPf_4~lH^l}` z?$f&uq>u~0BKA} zfV2(|?oVJZ65-qD@405cCF!i6{OBQlpx$B|5QB5mbMNQlL;{;-TG^K*{r8)H`VIQL zVz_=~warBz(?$v<8G--+M$-iyDljq1E&?<@{FusMY>mCP?7=lTJ%RJV@zS6e(jeTt z_b}9WeY6u^%k~;6Y|avE=wOkX19tWhJ$D#)13^H(g*tNCA1&lj0F^S^cFJt3< zF+<^=Nb_2m4z;h}_l9!6*auFQU1nSMO7LlYsefPNvtX%*3>A2W*GwO{bg4=2^X$z!BNKK8>w)&S?Vw(L#F39#|H- zWXSwvV4!*s<;PYx*bxET(k2=yK$99ovm|~_oCm5O;Ln0f`{$X_u~0@^ZAwtPW2K|b zz~u z0leWaET&joryIjXBGg%vyFd9k(w<7FxAjI__ww0GfEz3`>r~yONjCp&0wiO*Eu#Q_ zxkrY_sRk!--a-0NxiO9Q?ghQy2qIvQeZ&FvNcp z`z!b+&MntxZ&%x-7&M=e4Kjt=#HZWbJVy(di95H!Do74sZ7<0J9ntSD1^7$c>@9tE z!tRVi;@OZ#yui&5ER0M_L6pd%8HMuvX7NLs`px(wx%p&YVx8M#3O7a9|?s+)bpehs1%j&bQ(B!2<584{^g|Q!- zZ9fD?Mpmt`J?#L8GY*vzX>7@WL$i;HwEZFJqDv~3PH?Ub3`vuFY6UFy-F_JQ?tc`n zN+V(Ws+}rCN^?crMAN6d?wt3&;z5`eHPz7IUaXjtIS`D<6qy2y9mt6sOi$W!LhvD| z5#=jZO#p_iV3D1%%=-*rlm)z~t;-TUC2@wPa<+*hB9t(j`#PjTnUyQE@OL8^{6E01Dm} zY;Us$BxhraxCJ0^&d=rh&{Xhq$W-D6HI*pPcTaebZBDDkQUlun5Xp>rpXw5^8If_6 zX|=-Dz(O!rWDn&4Z3Wt(v=B&i;H1(?sG8?Z%W2_83M!$vVhl z9K&EbzN=8Aq@Hu6vn+A==MV*7HW>j$R9)&pT#d80S@aU(-%4eSu{rebVW}T|ayFilT!7u5AyN3@GUm%ZsG1@-3TGnjewM1b@w z0@HThPfeUbWxmYXaGkQd#{|8l0oW)Ujw7|yoT*_G1e=1g4_p?Fnus)CU`8d(a30N# z!uszWupm-4S^0vRJcC}DBYp(-=o-MuROozV&D0E~*zamncdGF^HZND0a_aM_^-%2@5U;M=1)yX6XE4^HxBF?`iv+>e_l_ZW{dD752{o z;2bEln)nPrfkg~O#)Qe*nzD14dKLS1g6;1?pQv+pZG6@;VX@GFy?adBPL}eNy_MkK zW$03@gU`fvHZpK&?6lS4>azsFYD8r{Ws3yhqOl9I5_?>yj3_aD1fQ;h@*efII&f{5 zRjucwAg%J(2Uy(UIt#o>W8p`zo$L+UzMNp2+X+TD7GH(&>1m3N&`CpH9JAlc`S$SW z&N#LL7Kj2S*%`;r`7Fm-M;<*v4Fym9jB~L6;XnTS@My9>B#<(I)&XzN!mZK4Fg@Ih zA4V#~!I6F2r)|g?DZP;)+2h`??Pu6DtR=qJz=Hh~hX(L@rm07#nDhcmYzqGf;>A9o!=qz- z2ALYpqXEw;lA|NQTLR*zm6kLmm|gp>I+C(TMj*-aKLZ#!9x#pE$ulh>WJ-LUw536U z_T0>g`@uf3GKo>vAcGBir>@`oJcj5vf?&Sh|!-1+2PR6q_b(Ag#cJ2-jIQ_I3?ntSbeG1 z&Zxr%wHeTLL8pu;Kg$CsVO-Osj?QaL0iajQ=G*VW>dSA!0ln!Oh&c~7=4hz7fdfbA zXT-=ATaxW~77@D|k~8z-AX&s7@!@NbuED{z_*H;*cnM(WD9~$eDzx`bF!Jkms3oXg zCH1k6Q>wZ_SztRXr%Nz-821AWCI>#IEqDMXcr8NkzU}{`p)h_;xxqN}-9!W3JTT5D zhtv%RYX;cdA`;#+T4T+vBud-K}liE)p_mG6X~3EC>7eI+K3?8Y*#pxBM|-S$}+|Rut~aeWf`UbFr5b=U<!Gx`Qr3kY{AC{?A0U9mzYm1|BSHcuh&OoLm?8&X?D8#C;v>V2pq)ROXk;9mCVNf__vb5ob}E@KuZl=Q&82jOa9B;;rqeFj??CnIoBWw{_nxPIQ0 zkChqefbF{EVR<|`j_kifI;775b^yxsMCt^@T>?%IStF}gJ2B)Pu#a{2l0wuIuY(9r z?PHU*m(n-Y*MN<&5>~bsPXg$@j^bGnz>({SqsO3gvtf(0hY_kNd)wl45xiKzL_`9J zl`6$RU=fS#tEWWEg5hBr@wP(YXcTrRTG`kE2yiY3aHN}6ewOvi!|V;-7$?fd=)hw7 z4DGd*8?1!En|Ek^(g;hiV~Xv2Xbob6P)E6>U^>nKM?2_Au4uQBYHNlE_Xq~q4_EXr zsvVrt`0{;pAli%?{mn*3hcGMze*|In(USq5xphA_;eGPQKd_(M!`Q@lOc~i%twaII zISy`NWidSb@FVRs>qRjEy2Dcfv4iRsdl!utAU)BR4G-^qfKIQBY2oy+7G3OP4VsL= z87bCFnwPd^Fq#7VJjt93n^XTli07iCv! zoQHSlvTB^ALa{`Ol6AvI={!1Ery|WYTVW6IUXX3bNdRMJ13EYZc~}f5zBnC5z!FzXQ=x%3v!KLcU}!L=ugz3sVLR;wW-mh@P?CLjh;!5la@u2nI?5pOIu6R-VLCUn z0|w9*P1OO%G7_en)T#*_`sm}B<*XR=MNmWPMv=Zf7#-ash!J$kP^szita%SIa8Vb` zy(S43)n`_>YOo(VIavsR5w^sPh0nnv3qhyFB~Ax1s1N{|qPO75^r{&g!J5IX?|*&n zEK-4W@O9V^?Ww8omUcQ4;Lz$g$N>sBV!DtvB1ITDBBz+9ssTpUN%@@ttYq{o;xaX9 zI#&=7{Ss%w=Pqlc-prM3z{o|J2={);_d6j8a`b<5td~u=V-X#%JtF-s=v3YjTc&u= zNKUL6^y0aV2xL7=R`Xlr^e3%3N9Cj=S-wJceH8fm?o z5AVkbB{oIwp{248R(NK}tNH)WI+efOCSBZBb%y zgpZEWquaUMXY53`BnOq-|C570W5LZYv-|iCUzIC9r)c1; z-om1Q+=;^2)Rf|cV_23Xn>b1Tt)^%iozOn51dc(tTjEeaR=$6jhk#*bGdQ!*!>XZI zL+1mHS^@4&;U|(R+C3m7W`pH$ECM@k(@kVQG2zK3j#dqnOr9HI1f1cJYYRX4H1UDiv@baq0zsmJW7uK%M*T19&QGvuR$CJ9)r}y8e}+l zHa=d(;qK#Pz+zM(F^3NDdy2uP?_SxtEC%vbZ@xKL$IODZmP?QWd#I#7^HK4&VyHcVTts(0I5k4~7Ir zVvFa}#sRiXgaoC&hIFND;^HGCW!xSO!*O7l7Z2)=x)visQE` zHG^?EBI52Oif7BkIl_vXV|tKt;ha^SlZ{4ol2Hr7{ZnDWNGn=9@IyJSN0!50LI(hg zHl9V28MZQmoq#;SzzN_y2QZdGch>0$(e)vErY2GN9E@B$4QEe@ATLH zJwr1M0uRnV0L*Ky6L_To8@B4ut~`78j$_pw+x)jp_Omf>?i>!;8KZ)7p1<=)Te;V;K_!nI!pDp zYPV$H)WBgL*t6~IBWB2I^Lf3)*iAPJR-2qBTQQyRC1~jez!d;+rZ#2Yymp^Ahdi!< zauZXN+BeUF0i%w69~Q=8Tiq$U8LpLDK^t?`T=je(_H2_x_cB_f? zHG=p<&Vw2`2dLXh$bNZ(Ms-h?TRTh?^mD5 z5#sFW9A(vvw*_ z@!9==QCUsSpU+YK)ymc$!6&w|*cUSWqhXtXc5ahir#Bu!$`Cx#taj$!J?8kXlV-;j zv*s3z^be2l=&1Uo&5^|{{ry7`6w4$_j@W-~ls=*cTg&fh*#jV}?W}W;^m2n$V~Ke7 z>bjv@*l$UKQYQ@5(3INZBSXcf0i*2nHM>PnCUE9nano_nnGF-@0x>Y#;%W=Qi;_%uIY%#PR1jw6@%^5un0i@oozjFSzr7#U5YmT-i1k(`dk|%La>b z;_F#$4X;d054}+;(Ar%dQ2tG^s6*7_96r7;72AYOvQ3HOfCO6=n$;@Yw;csUOVk0K z=|FI#Z5Jn`Tie1E4rv0ToQCD=qJw)cvqjHf<*M(VhJ|l`8|>!0L-r*?=Y>t8 zdkk!jO*1((8U>ghyd!efF(t@-=D8=^pj`clc|3%&SL;wBO=J-WH;<0)g8pxPVVlT9 zqh)WONy?h-Xc<5Qak6t zXJ|>{6oM0s)Xnj6oH1fwqFk$o?8Gm_Ri?9a9wu%JW?>!Sa{=RG0ZDqs+u*y9{S)~Uv`O2ROirjC)63;`dcVNipZ0R&hs zf7PIM28MiPdzUo935*YehH)nE86WeKF-{pk+&bEiiyYvCy?M-m-sQk#IEGxDQ?*CZ`- z9M_d(JuivCDx|*JbRM#=I1sPDo3%NB#o7}fLF?xMO@dilRWyjS1n3S)#wAnT?c@y{ z0&IhNtiIkn>?(&GC#ihEF>QhztB1Gg)pz&7eFBKBP+fc*Zr{91dlW{fajnW626^tb zfO!7=HO*)@(9PXqA2Y>dYBIJh8JHNunByEMuDK#@mth?gO{;M&(o&*Y+olKxa73`y zMd|++Uq6iuOJsWxRTtQwD*&Y(9Mj!LccKo|O0tQm3B=YoW3){D-JSHQ`p-0p?xJdP zfJqP-trSKW@Y9ZA++k1FkQ8hq3FzURoWtzzpf@?Ht%o8-Pets31uP#w{$Y6b^~beBvn-(;vS;yfWk;*eQY5RdREl6yE zV=q0k5RPR7*+gJv3kHSx78}UIlmVr$zWoyY(jv~N2FT#vO0+fsE!c~y&1yI^S?mv6 zHr&2_CswW*a2O~OS)r{#2giC@6!Cq7=C$Q{Z1HKBxH--1Wt<6``A>m)eV1pcd;!hh z1$!@!|H1KiZaILQalEEBp0%?totj&ADC{ojZyg=ZT%oxvy#-uW<||)OR-;a|cW5MZ z_gA7HXyd-^8Ry>Rb|a(Z{c97nS619A>eeRNQ_qPqwtgYbhQX$&3kKQt4H+~yUF}G2 zU@6(eEGORsbS$jVRi7X;0h?@kOUA?k5ob6H#%0yC^|FswI7NfTbIy${k*vRy80@>& ziOse;G7B5mfJHY*wOGrUo3b5Iqr^3F(m0nq8*I4ER^#c!A7D@;s5Z5o;S8{RJUf7} z=$CI>rEF^v-Ztvj6zLzqwD~?o&V;G^m@X!eyETniJyvtm+J`lgy)l@z58G&^LiVd2 zd-jYb(t{KM%)@%U4>v|=Cq&ih_VOBft6G>w{nRZH45T-2l;$_aN~GC5*AH@1KUXL!bo=l1kHHKT6qt)_eo_5 z6alRQ_@#IAfJK9nIczpeA(~W~ST+~?oxR4nm5mfr)FCW(Au$X%?2T!FEFckk2%ME! z7%=#p1UU}6m&h>7V5Sk*W@97g5BkAGw0z&nrJL^ke1+|=z zLTnbhQXCRUkKew9tYjqWs<%ix+rDX!Y>S{d@fSn=pIzY;+W7(X8r?jg9>mgCo7x90 zhoQE8MV{a7gx$u))6kiJgH7O6<^gG04x10f$|yI#4NDJ+Lj1Gzw`g6%ULd75h?w4i z+&&B=GpnTbIly+X-zF-t>|RF3A?VkA>G*k`xy1u{HifA{uWJY~^j*>&DGXIRMmmLa zD&zb+k6;eg7Q)=OzYlM|`dxU1p+4SOBo#|hh*w8m+FE&Wb_V^ zwBXl1Wcx&hB@AK@YmxQ7xyQ5zo;6CqQdAd~nR>Am85MSCY|~;_W~>rXG>j6dY^1jl z#26J=+%OIkV!Ewn(Ew|2YZh57!wFdjdql0EEwSYbM+25gmWv8H{=UWatP)lAB9iar zx~19GJs53SjFN^Xx1i+&C*&@6?fmjUIRG4N0uzUBkrb0DVk^kJ_Y>Bv2f?O|H7z*E z%O>Ff@))T;O4%=nz?N`urr=uG;Oi7(G}y>a={V-u8@U@j`);wf56{K=ziaI6yndhAm;Kto9T!L`s*V4HMF|HwFNK!sZuvjFM#4pGVb(i5q7B#aD< zVMKOfu|Sy~O!`FipSA~Ru;5W9T@Pj=0g8V7WsYMy5g9{Dk1ZDkqy|L%?A0Egk@@ zvgU0VD$`_Z9M&tGVHfLf=k**cvlVi07dM&DT8=F56%}iyX^a5ZmgYh)uUTXLcK~U9 zeKf+wfu?h$H!#vCfUAMtLRjGY{fMLYDN>S+wdrmS)>b{)9Kg5idB7pwof@OMb)9WV zlV|qgw=d=~V5E1bcDpw-81CF2#nBNd5?Flu%_|J0PRDU{%>L!M57Yg`zGskpsuQbC(c1%c_5 zRoTJWxlaWEM*gNUP0bmxn>xu6$W?jX0`Dw#Vs-v~Ocizm9@JLahR31am+Y_hBpO;? zr`W!MBlNi_Q81M!TWjAiQKP1P@|+C$_X(^ zM9?w)38;zH&+N|_C}=^jv9{pIGL@wWn4hDfSaG@9S5b-S8=r|lPcUPuQf-@TWRms9 zi~uZU`8l`v3BP9(?sFWVz($Rq5{%eR30v62`MM_d11qv$WdTjO+9bU|yQ4Je!5+Yi zSqxLAd(A5*WKev99f^R*GX@bMm=yp-R-f~0-#3wyc@IJFId&;#LHIX=$S&5T zauk~hAY{t-(Yq}A7=yhb9g$1U?uK<7A2lu2TO%^?(=|h78`xfRob=!jV*f&!ajD79d@%g)deV#TT?jIhG?3oic^qqE=aP680_&9AnpXqd= zyB~bQ)Rk8;*y+Hg=S%%Dt7(5caZ*<|`bVC%y*ce4dI8u~?_8_W?-P*U2B^00Yz@$> z{K3A1U0{sF&j zNn!xZ`)l0QipbE&6d8d_pQ?MZa)%SegG00jxFPTby>gKGv!Z`Dzxp3FWqahwWyn%2 zmZ>!R!o|v0fSM7NqbyQ1wk)z?M|l_{An_rvWSJF&If%D~fxf*nk3!GPJ+x6h z2$^At_}OlBGNvUWE5kiFB-s<~hBM7*gWnQNR+e>10Ki=1HjHGFwQt9fq={+@oaGb- zDu)xl0DRbr#Z;U~y8w%FL>g@yM$Q!V9uM#UkXc4_8rdn<*SUB`Yl`6U8$3%(=>b=~ zukYg;I;`8YvNW^P7A*R(G(AV!m@N>g*(RUY1z5L?O+KIe_>f&IiNpF(@Z6*;yt% z=m2D;tvHM$hTI!GH$kwh^)ALS&tWytGL2Ez(Qu7N6pEZtq1lcTbN*(9C~t__k@mM! z$eaQ+Y^|&_Yjc69kHN|qTG>S%gMwale?^>E5gl3;ZCoqo$lhq@Eu^@ZkGQl%6v;K6 zaTe8_o)M+)!!`}UKEI(#^q7%el|CBG_IFUYkjK#Pu%FnQd)y1!Imf!ae>WfQ-n~U& zRK<{zlH*+J+}V6EHDeei3^zK$RSGHkU?scr6rKREUOasTi;6=*z`KtKSO?h;K(wb- z4}1QC=RP=o3+9V`jX^xa;K^i~h7iOgaK6BB03#KRtZz(D!5o_2nYnc{M)!S8s(AP6 zHP0Ixg0mUzS=P%G+yQIv`Bxx)W12?Zh{7!@5cr#(vctO#j}3auuozt!cVzGuQAFu_lfKd;QsQK=?l2QcCuiljVas$IsJovq!m-=6I_d#k{@mDvwgeV$+!T46D5!68fKB#aFE$wD;kf@xhF z%*yr%uuL0bfmkE18M|a(%RP$1)Yl0&^cMRBjoyP=yi)>b+jYn^b#)hcuIPfech99Z z&_G?lBl`l3VviwHVT~|-*FJ#tY_flMc~9A2Gd!-B>u;JnM<0!DfFy|p*b@N|Ye+hC zaUGk&KFe}1rn6*H1gkcGuk(8Z8UO|Di0q``%^qEj@A8*@F;jr+6fC7+D#J6BWpN&{ z_T#!}!;^+xVh_YN7ua?Q3H_AQLJZtxcKq@gbfTpa5a{C^O~=9EcNry9ZdR%SQ26qB zU%;IEKBojXWhP#X^io03WRo1(Cc(@O45X}-Y2-cjh^>)aYr*L6KK}<&r!d=n`A}P4 z3H7a=Fxf{&0ODkqfDf}lsSd2W%nlGX&6E zc|RNXe;PK?X5Z!m)rRY5{LBVG(JE`T{Cotqz7F%ci{H8%W*$7G$k57QKsm1!z{;v8 zemOk!nYnDc<;RLk7EKh$m2!BwFuvR{_CeE0c4G8fSs}rPH73_X;Cb+T-v^|aJZ@kP zp1LA?cFF4bJ;B(upp)trHhi5M^Pt*bmFvOaUoVhC&kPUNMLFU%b#CfVEdanljhH{< z0*W}IK>>_uNMxBTkPsjxqJ{~l46rHjB4p2@jk#^Ri|;~xWj-9QEYe(Y!CbW9&>+}E9^d8TRr&+q*M}hdC3fOcMP4K6B1{-t(SwxcKK7trHBv36Ap?v2rJxfaLISsIkH4iEb_jkphMx zg+b)ixeTlxYiUGibdlokvVbFXLRN^m?X5aTP`A+^(uWA1Q5cXmfT_d4y0K@l2hz$$ zfhj;w5A4Iy(K+c)%Fc;|O?MPT>a4lCiE~pNyo(W}y^B0p_PL3v5{;w#$vWpTE{f>+ zHUJMh%AN5T8u46lTddBpy^O%m3S2Lk^A!DF0lPXR88cI=?z_zg?K$L}L$w#OHr;%F z3!vhLN+6q1=5eH@2~vpcZ3Sj=AID$pCzZjPo$C~NcdLz!!`j?SA~B}{=xE{?GtQAz zwJCdb1R7RHSgEztF<361WG!tyQrX+1`5_k>BeGN8gxOFek8XzdZs2sTVN|oE$}ccv zCzS3x4;K@_XPWuq@^aP%z&ds(kF--jbGH4OO2=9lx4#l8*@~bM4b=q<6J{lx9c&{H zz#zEquzi(N0*)IPM&4HjJ1yYF2)e$qjss|h_F=4!cp0xHm=R_d@CzWE=b6B&M#=xq0c!-n>48CKzbll02m9d%A5!^QCdFaITm*E5uzwCC(E%t$XJ*Q5abXES zLs|{SRwl1fL03mauc00uDPc6xs9;!UaVtN1jFOP4!%sE)%B){OPk#W7FQk z#Qkuc9>T$L&H)~_NV%bB-t`sfTC|gsbM%RXS>GWAy1x99#?SO?T3U+2@@pLGAkInP zXL?O1WKS)@v(H!>F2xzLG9pSOOdVHzm*a$?<_ z$!U+M%~foW#V?#KXEa*WVE}BC+t^czJ0{~MlhHp!dJ8+%;Eq%Qmbcgp@ACx)-?38) z^x>_sQRvgKTZ1DbI8pW=ka1Bx3jGDsP&aYRo|1}nhsiKhK=VB^bNUObDE)q!{}>Rk4oV$*DPHQqsTz2wJvOzg6V`6o>#`02>47bA z?Hs3-1_TM-WdE(?Q)g&8DvH^4o_u}*wpi9V6UhZ^0`|lx*kW*tP z0C3Ea!CM?M&u7>*z^NO`S=3PRNZfKp7SilE19-ENQjqC$t98>x#I_ayH|WZE7MjHm zd@JAEK>%#(D;9oXBV-x{AXZyPIT7~HRHUIOpTIq!0-ev~`eFm{pJaqgp?ZIBD7qQ# z&%of*q62w^GcYRz)dJ*POfAA-@;^1Hjy3!I&wmVGfBs4MZ~yVf;iSCEeB^To3^EAp zfs{yvR#bOq=IAs~4tq#B+>Z`Y#a*R)$bhPVw=;YxV46%qXRe=&1OOheDcd9KXs~3^ zR3+u7?^F9NldQ(fAjfsJJ(EF^`n}kGjEvL)_DB7%y)?}P*hWKLvFqfSG5u(U=y<<* zmpnsi#~KD<{Ct6qUzg;0WI6rEl)mX$0bFFX-8eQtV#H|#YSVH&Uk~s9o1gtN&*I3# z9IW(Nta4RP7c)1Ua{=5rg(H#)S$yRfz*AOOO}}uQ&gJN5FJQBH(yO#GX%(nmT4>1?Yc{LoG= zh=42V-FQ&Ne6eznZxDeBmZUMQ8j4y8EP&Bn7pOW;Az+PTzc3uWdN0$SDGsU>zRa2i zLqWbNB{e^a<7JBk8(v_IcA7tT_MXUJeUb~*1t9Q1*laP*H{tVT&jLW<+F z9Fz=3U!iml>`4OSoj^C&WBLQ**pHX&M7wr#xE(gW`66sT{U)@qA%Z3il5AQH-GJ|5 zZ@K0r#HsU~t*n(g6(i$IKwleay9Uu;YH*U#R#g8HAuq!AS+LZrwt}zk=H0DeHH~x3 zArj~aj@2Ft?tDL^ErGO@s%cW!xQq2KQFYXZKEkTfI-0jCnz)lgbOUtN_mJ5hL@}?L zp(I5N1!!$AjSDRhRSYd#rlE6GkSXjq7f`T-L05MON{D-`sNeTGf?-P&+1SzFU+Q-< z0CD#$%b)v<8hs7E7b;#n46KY%3|Dy_&E-3UR_!an@fwNzz4#; znPG}Kj>8jv?=H^HOhL&PD0py05R^MV$5bWeb;+STakHY`pCaAU}HA2{dvqA5QQ);x+4kb4YU<6o`rY*zl*p9*$0yXqx zb-^`^Q3(hB_~9%mQ2I5}H&T%M{a4?jqoF_rCeKRZ<4T!ewjS0OUg0Fs5@B4O;b}Fc zgV3`uqE5ZYkxK8sAa#Od80<^W2wa6B!J-y&;sS)Gi$Npw1 z?&>)`1TFaKB}0*<3h@9f+*5sZXmUo#ALp2MZW_(G^CU&DTT?8h4ogn>>IsK!~h_Jv$qdJ+Eb|Lgw@|I7dU9~cFA6pq%AI(<4ZHfkRO{u``Z^5%QsYeBgbf=oeXWQ&heK;f}w;yi2uo6-hJaSYeqmo zsx~&L#~+wFci&~iTqE_1g7q>aNNJoZe^Q(V%6sUuZn0%`s=8T%2wu#1GQpo` z#bQT;N&`?^jHokAqhB_IBe7u0G-|_;MhM_MZ)Sn~pTW9oDw$wGUm8FwBi4&l;ORGC z(MIX5$mE{kgH2_svy|MBy)-j)K=AEIKt~zQ+`Er`Ae(`&h{Xr^!WYOoddSk)lIDbEe>hH4-59~4S2G*4M9LL6?RH}=mA+ktLbqPydNwp zMRjJhJYV9LZua^N0DZ&y-0(aGHpl#4QYTyJizDh*U#~>aIxsvE*}*exVd5Xg20oHu zi^hNpXahOcgE)@!$}j(4gcqJl{@Vq14xpRnMp$@YwNfXB&dVj+BEwMN0R$-Oyt*+6 zabyb{;NRu>-WI@v&dWTulLq>Dfu3$U&XdBlU59<4tkU$CxoyE)4O-h9-^TCVK!YO} z!2Gq0kyVpMKSqA)&N3Xf3=w@EgKJMY1@CPfL=R^Twxv<6(2jx`hB(I*n|igk9+toP zL#R^W+KIs)E3uKh7xyn9Q-N%T|yLo>> z9RVq64cIru@)!;C_9HL@YRC@xdNm|gRXIwhgTrcZK!Y_(^XI$vkO7D`1mj+M_FQo3 za(bw|plTP-*c+L)0#S_mMAum@i{XvV9SfY0RXySY?GVlr1v zV@eshn}b32-A-{}>N2J0Gn}Cm|F#tSgrD2#XW&GKaEuuM%NR~UEyCi;DtbTwOmRG% zbRbk^4BH-t!3NeN^a{EH%(wB5YEl%*+6T_hPY|_r)NMxniZ(Y3VXhB-V>``G0d4o* zehgc49u}WI2?>5y4&$=P_?lA+9WDTkr<{{2hN%U^WO`+fl+99|is$5DXwSjuk)E@? z4h1Au&G^Ht5YgYH?|lU&bhdu(4y4OtX8UU?X4zDT|)EqYH1Pngfs^1 zOFL|wGl_EqBfDE$e0>p}A9Fte54$@QJW%}6-!~A}(9GFfHcgeUX|l_kRFr4R9Bk=ZS{$@xwdeZ$A7meEBsgl(p@sZR=yyR*m-`8y#j?{5Zxa%jeRJ zn2~H(Fee6Iai$*zQOy}g25@(dPU7K4#0@>GLzVnB|wfK{^H`4v|Al0i?Tdh9@|NIK_Z-Ay{#pjO^|4XL@#@^HZ9bfW>374n+}k zT&)605$KW&&R!0RCurVj3$EP!)@eP;t~gHu~m*qFOF&-frd)1lzDe$de>|M2T} z2s~sPo#%XpxZdwkZ`h3v)gF+_5KQA8!%nSjW51QW;2W$SH65oQ-jtp1p^wnN=yvHF^%Z&v2vbqoo@rb zWYGl9h<(8y@_h!QrU7|Pz7i0~pVWC*8PUb(-@>|+Eun+v0L*ckqJDZk++>KNi0EDftEMkN2#dWiCS52b; zG)HDZ@Z?!Att+s%Aj3-D9Y_k9(g6UwRndpALm(c&9@Tm#u*vtN# zO{f8SW#D9>+-GfjOxpv*)e`HEnzN>5HDN&oZ+wjaS`Z!!WUwFlgLW}MJc(&GKU>|B zRjoE+WQXJDb^^p4^zBQlsnQr86G=5#FsBhjCW?o2ywxci5Lq9-->TLNex@kdN=vD5 zUnB`y;h>S*{UHd@pj?7V4mRk^J@E zM%a7xEF6$#y25B%c;MPdyqkhi5UcJ(W6%UNq@p;a$1njl-o0|8XlOff?=5t5=y+fO z&M?v!7y-O2>w!b&-Eqwkz{y-Xd(8p^SuB1N*ID$aq+d;jM$X~_wPi9~9UeZu{}4lG z`Tf4e8^>_Vx;hA3R;rc#NRry>AtjflittPa4`m&hH=b@f2|HKw;^^8|QCR~}0Dp}7~V6QIXw34*s4Kr)XVBiI6$)pj6i=k|8@ zWA8PY+$@((L#K0JVOC^uayayk4Z)`H{v0b=M##M;NaI|v1o0V8S#QC$YvcGcwO0bU zZuVQ|a15}LmQmp1?!nkxRRQq3YXHd2IOw>Gm#c z<(#Kz{_A?NwDC+QuFRlkGnAWeq@ZsXGRz_3)P5;LV=z$jd_q0!-c zJ9=Rv+9+P&OjKdEEQnY}m$JcdSv7HLxc1goV_`#@^VkZg(NU;ZlEtpTwv8}lBMH^7 zApM5-y5Vf8ugo9{ri2dwfbk`8@!a>JEyeb+A25kH9rx_Tv*(DVXJCx`LvebZS*n1C zo?`g$M}JT84i#ma8&r^%!#nT%RajkLhgHdQrmEq;{%`($`25Q+*{jYl%%3eTF4MZ@ zFg$+r4v}w_3Qp3I6e{)R3(UlZyKP2)8u(O z0L_2;^ouZYXAXnT`Qi-i>@bfKPT~1i-ym;DhqnM$>uW1vgZsFN1Y&q%l#02*zoa|R z0iofuL1>L)5+@DV^UvzKRDJ*hZwrE^KE>3XX*7E#I#2cz5W5MJV1v;?*qD7l@NRiM zf~7;Ec;_#_US1^V0vKYHZEvK`&??i{qS!W!`D02qz7i*76k^vSu{}Xv_i7_I>i~3fv{RlJyw!YfEFrE&8gjM zGCu+do#F{bK5A3(Vx|uJ0Ke_rRjd4mMn@t8YhOI|kXNwZivO#$kT8X+lX9-}4%#78 zvPA^!W>{jQT^zW2*9_-xY@{ECl;RAXE%)5kFR~|A(V4o`AJ`Wx7HPoh`|$Y_|Fbo^BTK-fz_kY(M~hDB%u zO&K-da@86l~C7>Sy#1**PytlrSpPj~*AM9+fCKP|l!0}#H3Ua(W2CD-jV`1&} zGHVHOLZD_FAA`2Uzk2J9k18BMJH%gc*hF|y563}u(TwozU*p4xD`e}`!6~*D9mI8K*1#%dk`0DN8kErWu|_vK?!gh-O`U}RA?oOZ9fEWMqrlJ zCo#NPgvLJmF+!h1(lI)b>kiTY)A!KIjl`Lcb=JFzGd|;)e15ebEC_qmJm3OmZ@pka zXC@$OaF89We#|?lB{Ev=!pPfyt$>)>Xs3+O_z+?HIRbP9Mtm-`IL1g;&ZyxE_NXv5 z64R(1fIOKcb&2-8anJosX=$sp+~b%7wlbNw#K<&TohX_%0#YaEnW=KN1ou(1!^P=luM4oSrWtOCn6d&( zSl22Jbg#S)d$kodaZkY9jaSI<+1@sd98hk59N-C1Gq^-b?cIB7#@=KPqN zxQu6)Gs#-&fD*6`-Xj?W9fIe551`JX12hKWnjXQTwW>YAXLJA@v!tPHlXAq@DN8UV zuVD8EK=0U=UZVRo1iSkfx`D~LFicTSOBZZJhDs;Wvj=p#PoQ<%<$b2cX*hcbpkr*- zD6I6`f5qI@omK2bBaDoh3ZQ)ng&`fJLk4Lnuwf+^&nP40BM%M2;NqYy%hw=3d-8;c z7}fyop4t-kb#wQS^k*Szz_y6&1*azzFy?EN(}M;I02bTzu;*vq_xk`~H$6&^DO!|l!&Z#KLYhA9L#|L~yw(-=p_RfayZ1xjgypSf z$~JA@5B+BYxrPFE#|~C;c30RM$9sM8hhGCMsAz{p9U2~r1ts>(6o3qtXdM8E>_i{b z3V5=|n(Vmar!0E0v2~m!$-AsA(G~)mXke9R?=Ak&UM4vstydHn+#-zTzt2gZU9=KB zSRu;&u*$C;XB$Q8d~Y`((6eptsw*^Gtvq`_`mBwWb?!4dQ$AGI#H@qtmgh>xCa_81 z+-=@k#A#WnZt#a`i8k91yANP5+n6P_T(@r-%zKKxtHFTp145i$V2ZpOKyL-2jbnQN zu>-K>S@bZrn9(i>A`St4yJ$4?0Q6>CD0;^A<#6h@RdotOT~!Xk?xVzCRh(q5g~O~*tDRBXc} zaPYGgnp^0?e?;Avn&&)in24&-xU!!~dSNnBermWsuJbWLNR>710YLO%OAiiqqd!(a z|Kz)Gs04o=e)2aD!bq_#eEr8y$N*4Lyi0Rk>~#zOeMAFz3jqb9>#R+qvp?)z5VT*D zj;pZ|fDrIr%>LMnT3;pWDCh*y0ir4VklkmU)OxiuyHk6i22Eh>UfUieL2%;zk~x*o zDUFRqux`_Fo5kCTOi{jbkiAo~rpMTkn1<#Y+;XPQVIx^kY=V11rjDS%k%wkq{A|yv z`{o1NzC?yo-*1}IR5hn6+D2szW}60vM)@Pq33p5YX)}A#hwIpZXTH*xpNjzW&snf`cQ&{9J9a$Vm8x zS?2$V5gc!3OOSoE_U&XpJXJ}u0XC|>h1v7#I|vXBnA9Hc=uCiREsR;&_5@J-}<|cE_zF$uDBS6paKPZgT_Wq=7g8OQg6dTopA@LtC@+%_EbDXuc#<17kDc28YttKN7Ch=Wt<| z6CF3Sll5%laIhSJWSay7axkVU4O`Iz`1~9nA{CfMfiK2aBjH8pfwdXMa0-eJ00`Ga z?ro%S9XGZ|dd#wazfbYIRgE#F&1aheHzgq)au=;kWKp=}^kw3+ZOCl8)z241J2$bx zy4mk9>R_ap`92F9G`i~Lq}tl}+!7#05aAanHczn^>LyJYn>LqqOkq^W?!pWSoQdX% zKKd|j15{MOm4uh8YheYJ!{~43?rf}@J%_sH8{&D=`1mYco<%-0GP1l^Cx*$3B}0q_ z1%Q!u=D&LR`cc?1>RL&sH^Lxkt4^M27gp!xizlQP*J1DYEI{xPBOcP@k2(Kc^!vh? z;uNfcG)l7Yak)^8W~P9R&Hz7vYWrEqU*5|>K(Kje35&Sz!+|#@3N1dMLq?b<~8@wqL8L0WG|jinV~#Qd@s@QQ4UP% zbT@RvZd5SlfBNmO!hI$iw9v%2y!jG`YO2o=js9q)DFj3XgwjWVEz?w!6C-i`k6;hf zkIak%Rspq#v|@R9XCh1v4Thip?3eWDTZw(t)M7aUb6^mr6qd!n;M0$PAMU*QI0k`= z>o-8 zIxu}_m(KM7(;Iq9Rd8m#L$lncwm7bAZ@mm~v0VjCUC{w=K6??f4krX(V>qXG-hZFq zgd&IK1;%@!btCOx0@PfRnmpX!j-!pPFQ|ef*mK-keq2-xHu%5vP1X zr**`AZwM@~7Xqu5#aGzeODf+=G%Th|eFOG^G`bGzgi%Tj&YZ?Ux&GE+p={h|* zO|iIamg!M@)AdtCu)!L&AOkss6^KD4-y?&kgv3gAdt!MPZBklTc79rlLL63m(&leK1%B9J}h9?ASiT_QF}PwN;r88{ovc5nwz34ra7hY8_q z>9F~Bp54kzb>Rlvf?Zy%X@6Na(^*PmBs#{B0SGb;R0KB({6vK*LL>BM6ZrHpTlof% zI^9cgU*bGWjul9^)0336rD;Xm%oJcSP0wlb{H$~C;OBq(ZFuwkNO*I;Km6(AUxy5K zz8go}f}QVUjVeyf0qA9M?rW^e^L_Ngg(;eIWA~A6I4V(evv?tn@d7Uho=m?BUIh38 z9CdDfPnLA})s^hGt)P&e5Qv)<%OZ~uz**dO3lmnS5;X_NElrF^0B57?6ACtDTEF=C z4+M?qrSW%%NPO~r1RcD$Al83m<0P|`9C?;yxh0;=;z?perjj#fM*{o0rASd%sg~9D z%5JxzN$f@cre9C6*6RIC4X2Qg9m2o~;LMbWi5jf0>D?@LZu0It^U7(1MPQWUnG7TZ z)QP|UAO6>m1|R=zSgR+Q9f{%V8x6&WKML8ow?maGuCpEc327 zu=#h02u7fc+XOD4^!+^yE`r?39=-WUaorM`&2wEV#f4Utw{jYR?_Di&>I9mUOTi2y@go|S6@w6k#*UJ5|BySfe| z*%Ah3N9lEDl!{gY;JNYqNqBf~Hj2ofzIw&b`Jd8@3Fi+RXn-?>*6E6i8YeNd)R@YfSUts-y32$VP0!_4#qhJsW$d!abF%6gRm zSO<#MX=vNR-W{x=-SS=l60cZ`4jj)2y1nJ6-^S+NBZI?Ky?p_IZwvS5CIF74uGk0r zB^F9F4ebU{5%J1~8i3d!dvV92bl9KZ>qLFYauth0i}B z&HCm800_1KLv{rN(oRIK+`zP)DaF`umb?1<|NN^kIW->M`@0Wdo4D&Zoh+BP0bRlSPf~wfN&Uq^!=0XXx4c`73&>zV6;;K(ofIc#|brrM8QKHXLnsGQ>}`|1t!WWQG2P`+fQ&&0}!ZT z92%mc6GyhZw15LihUqy#Ctp{?@tJViPT3Fr{UqXn)Qg46=n``}>E0 z76W+uY#osD{5PY->+ssxt@*s?TCvT1Nx9DCj!0_+W%b(WFg#~&mN zG64q;8E?my3XHAj=Pz*XR2vf%WqKqdaGC@oQOC<2FrCd(@CzHj^EXqd+Bg_1)=S-; zjyzU}^I20fbCC^n`iSXoQ?Lg#X?6sfBcE(kd<~$OF4EVn1;%8>Bw?9V$()k7{V5+X$-muq~HNXPIHT z%T@Sp<1loN-3?U=c}{Btv9!Xnt(O6U8rlN@%8&l;r{PyW|BtabIk$`e2k=bbYQduW zk16InA`7CT+O(a%Rq$f5hrf3{1TYpQ%-wr{w%O`V8ae~I?3E}fAOmM{fEE80{>&gj zNsr?HTl~3cSFAp1L<&hbZ`x=M3qKqGgZ3`&A?xpF=+kAvq8^WvARAZb%n9VI?zURo zIkF_XqTk;|b1REySBzeQOLdyJoW0%6g~-755Zq48&qW((#{*N~PLcZT~xI5ZZqgw?LfFAT`U3f>QcmLV{Rfq>aSpGDxyGRI2S^6P~t;SoPrbu8gwHlPFB zX5Toa4(+U`@|a$lh?ns_w>+=Gm%&GkV9yHkR=#Fve2QRqJFdN>$RA|p?&&cxE>;BJpWF}-1#6RCf*FUBlF?9 z#DMGhzY0fmcYb|CCEmMh`%D$_1)(a1?IWxg@f7G6ga(u}uPB<3({y zkSRGw4_%9j+4N3chMKAGB0nPRSDjXCqv!@QGC$*IA8^S}`8h081aqbft@2g(CR2%@<8m1Vkp|_EYV+zOUPKHv7}gi=UAe+M%kc2QXrj$xhBk9eu~<`Z|UNTD%C$IYQ&s0>Q-I zZe*_12AcAhRqcpTWAoAo;tdu&4>EEBV*6dWo*B+d>~#frnMGL4)e1MHv|0e@`%B-3 zDfZq9rdtH@Q+FTmeiVsZ;$*uA!xn(Tid)+QsM9pPCAF>Y(~37CLKXew4k=W{)?;&b zY4S_|t<$}*#+={Qb9(%YPKGDnzd*ZGgzcH3hade92@uj9q}FMUJIk1wZk*gI?`}%f z6i^o+$reA7_8JDAy6G%ou*L{W6P==MTMl6kyKzMQ6lBzR536Y#fPI~N3a4dPa_8yR z`MFM8$m4*yP&n7aFk}k}*h+O8Ru+Ew&5N+~YKz9tw9nydPch&VGt)E&eS{H%eWthE z5#ne?y%am&Y1#d zP-AoK7eLVl(9XAQvfPBLMEq9MYX=WVJo*xrJj$BAnE4TcCF5RrES z(o7Ya+Op|0_8mqCwv_W@mZT1=yh#)CDt5s0YGY8HvEa?~V;?pD+u%wak3~b1Xs_~p z88mzpwxHYb=h>j#h*H+sQCfD5k6J!#g>4h;R~IY83GVzF12(HEoy&SourUZQwWp^H zm$C){tGy_Xx$|o6I=cwuxhDdq6wGHjO=^!ShE~zJ}aa^Z9yWqkY&2&yt*ev7+B{gXCMWV7Y8Ampu#i> zGfPl&NdUgXyBBHfn&M2h!=kMc_+7x#wo@^^%3SC9v3$5UTck7xV^1W1<7t^$A*-^R}$7$DH)J`D|1?Y_3i`x9Kk zQXd`@s58MONw&=ZT_7(g^_*!ZJrn*CpvUy4eh1)M^Na#$CZ?xj6JTev9>Mk-2(opN z855YL@yoWkX~%E4UKSHrR1kX*a@V)XR&bb`%qHLHXInH7GfaGrMJkbu0q``_rv?=k zt^i{24>D+*8*8C|l(UMiP-5e72k8>~7W>iR(=0$}z`WhCt``JN!xIx^bSUD0HCG4P z4f|?0jNkw4=MfkR%mvg*K?wW(#0?4?bBl!sqze0n%eY8|tFhsk@q`IWy zp}p&!G$<`hFbJM#IX@Ya6K{mu;rWoD>ZOH$C&%VP`{-O~0T{NIM#4>T6zva}7iZYo zNh$|`chyQ-jh_V%#5_iToDmWiL?)-5-LAtL$n2PhjyeloN}-s>OyfFu7q1D( zChda5vtzoyZ{zcNF+d$0#spF96-+}XDU}|aU)u>{>h;CYgHSblU1dfcMvRF42FKlU zO$14r#3*Y7biWNC@%hqu^fVJ=q*VbX!y}}rNsUnIeF-42IU-+#V=zH%hLh9=>mZ|O z1+a!rW2N&J;I&|^Z9-^MO0|0MQ80k8X0SF^jAGMRJJJl$%*T|>J5*gAlB}szHXqMg zVzdBKex}bGH6H*4BY)GXO%}A7*3&pkWor;E8`nt0A<$f;xXyqLQ{CqJ?X+)oqmh=n zlGvn_eKh`$fR8h91)z>oEHI|0yEau#(Li;GszV$M%$#G34l(4duqtW9=L5sc#pEJh z!%jIeO+l&aL6F0XV2(H&GQNUMDSFo^>KfQud|O;s`=~aEYDRh2BERV+GS2du_I$f$ z-)tJ)!h5wc@t{rt!T|=-F>7@kux)EJ5VAz-JvmaWDW`~^-Gwpl5|o$A+%ei{jEyju zcpRXKma@D~qv>iGMh@WoL`A`-U5*&{(n93T4~qpR`{Yy07~?k1jBR%~4WxRY*EI_4 zuhIpbl19XTH-T*Pd=O{EB9{w3Z*+VxR*;_nlxn12x3|mjJUdI&fMBbZFa-ten(RBa&-3nh zFjJ_#1R{B*9GuZ+numMIXYr&lrKa%P=TaEr`AXZ+7H4<@v*sDe()_wGh`a>ndrayu zS3nPjPOgtBBDQon1u*sk)J{2D_Be}kGkI=(m=iZl*=oN8Z4Q{1O(W?R?}Pxu;7T?w z`Wb>onV4Jb#t{1{17`)Q;1$yvnM_mDwj!}g)%S45t(^`qyD?Z*_p7``V`CZ)i%jsu zTC@=813deOkjd!701cxB?hq;d&T9${xD%QuM_=`qMhOIIvjr%TrPBuyA(ljzb|p{M`&{OL7jjk&t(}2uR1KCjoCzcKoBRn9wNvmtXu6pKCB#)1A%o^g)WYJg?8@zX*f%fh>|5VT_IT16JV36ZI>?#%J&T!9> zMLQzUlZ@!be$61CSbhDPAczWXe99$GTtL~Db*QuK42H~?&@ua_8cyZr&YL`g>U2y@ z0>FheBK?S?!1;RNvGdR@AY-@R7okMGy^G!_q<&7QTDyg1aJX}x^7+~M_rd__#9N$4 z8~Nx9n2sc%;E2fOg0y2hdb3NIpajvB{laqHTo?|BO;0nhGprI6G*J+XrNCc+6~|6s z$eO8H=~E03?{LHPbSZlsY7Y+9@f^0=IAy<&h#Y)pOd&}W(rgMF`8=JwhG&D;AEVIx zNg~!8bOtde&7zu`Qh=MLDr)zDh?mdr#sOWzqIGldE^x{X8l7HK^<&07J0(FxH8# z5v^4fK;y_+L;J(AH55$9=#t7OTDx4o3d`I!_0iY^%r4?!S%jx1z>exixf88zKXu6j<<#LVS2ZX zf17x1<9aF!;h%r?S-4LM^+ylKm{b9?cuU)e_G%a>ZMbo?9bSL2z&M%G+vjbD zm4Hz>J1o;vivkr=9xaSZ%jVIa0Q;y@Z)lqqylnKr_l_Lv)xcP!W#LBU5*g<}5 zXz5iVK{YXZU#Cj&^n%JmG`Cp-#47K3QzbA-Q{9GLbL>_x1sK~1qDQd@w&6MDOyZkJ zjl;QOu4|?mVR+$VI080Q=P%iR?Sa+lEu8TNPR$|ZR-fu1{AHlrY^*M*Y%(7f${6sp zpr3VivQfUkl8?rIa3`s>wcUcLFTN=TpV}gWGyjn>{o%8$&=&w{4|)g;O%tk{^TQ0X zlJr7~MQ7L}t9|oyT;Dw)HQbE6SUUnl?1^n44EEwsdIF;ae{VIed!hpuOvV%z_E2Yi z4(Z^Zn`yy(#lP3d^9tM#>^Z;qf(RH`B;J9~REvdEznwSYY_`XbmqOVjwk`s1zF#T_u**fBR(DQQuF6AQ!o z15jCG1M=oNH}{o=q1^!XL`ITJ&LWSV8Fz{}?P7NM9qr z@_D2XV{wA}2+GlKM1aQMx#qD=4WH}htIgxd;=gS(BY?x>!_*JNzK%|A>VqXVNHXo3 zKwhAocTY`wo*_}G;XAHje|OIrZwy2*&TpM>!c6kE}6qK7`j1NdGD9sX9aGoFy|Y^ zGXCT5Wg-QSu9+jv`rKFU6(JMn{46Bi{&)Y?M;D;J*yWrry5#~~bD%_&u?(7p>n)X- z1$rmspqukl%nYHSv9iMhE2ZehNjslZK;iuIqKYhic`521D$_15TEo)XZdjmqRoPPZ zZhE1lIY8*@ZU$iQW*&uW*t+(KyCF-OwtMPfI3SumZ_kFJxqTwhs*w;tp%v$o#(0VT zEpBiixxLZaS-2EH%{i^WU=C}8loZID%@Pm-tr-DE9~GK}LNO6-BBErGoUJMt#iWsD z=?$Y?KF4b~c)Z4Y*$+{WAtRU}9VyrlBucZIV(dfr)I)^sXeySEmKA_lo@pE7dx!?@ zN?^hU6`{}>LmrZ7Y-$?|p}O%hobzjdD1Vs#R)S%p5ZAH|+PLO_vsE8__%}Er+7{q! z)DrqsM8n4c?`Fv8bUakwiu4)CT2a!EVakvFl9uOPt-jEW!;qTiJ z?^XpIe7@k_=kQ$GXw=4=o=?*yR;h9W_$>etaB)Cos6W`t;>Z|NNihwZrwIjT4-AQ~6g^Fv$>F zJaXdR6KLdk1}j&4pt8^%!!*!g8{qV4VwiO6O;~vGJj~u1M|g};=3P#S!kt}dlxiB9 ze{t>vRt6QMFNpXymKR7jGIaW#Hvs`u)EzPjfyTYJr0yJTH9vhfe(g0qEC;3=?)MJ?~-Diot?uNu};wKRT)Y54M$}`g5z-#oVNzRe4MJ$OT^;3ux)^q`#xeX zH^Ay1VC5T{emZ-#mj=LAk){xm>;RZomY1n&gAGE%I5F<9=#%jDyYFesGC*Z&0nori zfa6Wt?9lUw=G57IhH5aTN)YUoh6gDusxm0fJN9!v;O92E+Q+0R<{M z;qCW7fXSoU6$Z}1*N!{N!Ly z`H<(p`A|eqBCr~V&9f)eK1_Rxs?CAYD0lY~R)b!56eaYRM#9QkIi`~w>-F1T{wAjC zJGi(H-ndWd)@dQIdZbYId9NaslO3JRgoT;eCuOJP;T(fd-oue|N9*cKIeR%6qAbjr z-O{u9d`JSQoY*3Gm!+%2l9~SPCh|UmZL-PguYdms+GNnqV0;c;;uMwBq>H%&HWuwf zH=$7RKi8zlii5DUtflQp6!YIiz(*A$Hc&v}xV95gy%%UvyRa>$pDnto zkjk=aLOa0(+sAV-*{A0w{Pp_}sm5j`-1pyyY0|TAO_BDZ0LXN^{oSz8(O%x*z#UN* zTeq-pwb}vu`5h@!{3b!+(f$g1gfFvj3byTR`7q?JF+^m=PO#O^E`7eX6*ef$nfu8< z1v+aoE^q|dLq7lx(9G+ea`v2o9lQGTv*VrtM$9DexIb{xQS(VJZk%Kv0$-X!+gGe;>V{ z?ZxttDIEkH-XANlT?6|Y%FYQ0WmXFW#2j(fkcZ;@3u--c=h*k1wI$9jX>87l%w7K| zedl0BWq56q(hGo-h4YSBwOplAw}FON?BMi^J>(po3&(Pn#wNnzs}}&i1i+itTHICL zXQDsZ_E9T6jIvNW{iGIJScGl>NRG%yZOoH9>bTmx-M3xe8f9J~wS2BSd*fEtOZsp>#q z(nC6|12bkNE6o8hT$da+#ozs0#3EqB$rEY5#`J;u5z`SiB(?*!1rMTR7u6nijz1~D z0^AVMR$$Kd5r4WNIs(W6H(vagWpPW1sYIVhGt*Lf!uX95p_S&T@z z4ED=2YA|x{3PS0KC1`n`Dhn*D4I(^C!^ACBqNV_26{bQ}VF4Wx!5Mw@`!E9qDMKNk#0YR{eQz5li%C5{8ix+{(H?c~r~1h*jGYxjQkz-y6jleV(Qw##fx|e) zB5B<1{cYG9B&0>sI5<<&PdykXb)B}3IM=K2L)D+jK3>r7U}JeTjLeLa0xn^U>ZBTx zB2W}_dbmOApTW~JlVO7XMpGrGRUB^Ov@0=HWrz9U$q}3f{hwq+V6P$B_=i>F9dJ!C zQnaCM2Lk_2z-0yws-MublDT6!GW;B@}ZU6?t54y)``j5O!F4?t~2`14oK zVID}2V|^dK@n%?h`HG^0Kj3(g4O~{}UxicO-)D_UZ?G#>rYs2JP65AZ0Q3#uI0MuC z;>8-XRo7!WtgE9D9zK|fj`Xu9&q(Q!qKBz|>)rQA@s$Bs8=SYx@ZJZHIioG;<35X> z>J>g0xKEuh{7=4Lq^-hMSbq5;7Ad5$Utj;y^7928{8o9LK76;~(ck_JEnG~mVAnW%wisAgUJKuT`;5DdQ~-l7YwnsJ zSapa!Q%G)W4x5|ngp^|#^mL`o6wreUXXE6Y9@t(%`l)vmAL+T*PQ*e6fQhVwch2)K zqo8!rS;eZdW5?yu_2p;8I6*a6lcqyd33N7D;KB7&`DFGwMSN_1Muc-UW=m6)(=8A zgXrsroE^Z4;Als^I>FJ~|N0-pRX;@(Nve)p(Vi2COwQen0lHZqi(2fQVEA7Ji1iJR zLSh$5`O|oo;6%_C2^)4rJ7V8D&xF;y2Hc(vWiA3NCvMoQ&AZ{Cc7f0tcEjN71k%DD zX3iH+0@@`U>>(q9~mH-PlD^zCA_Bd9Hb|N$IhnO`~tk(jNSxL`L&Y;Yayz z`@of7zrxnrn~;0J_ho2Z;+~&j$89f_$rpH6$qq6!1a>t7^>eb4rt7V&-PwH2-;>EB zm?=$8hLxp70!3Qtkk+koKfEJ}|Ni}lALSA2UO}nW%4@Jf%n4n670zB$S-J8oG|I2S z&B1y|Fmfm0Sn6BL;fl(dD=6MJI?;DCa;O`2!dZ1U80~g6VBk$e^fbk#6~rhGwzsbt z6&L6_JWDRu(GUe90A@g$zx090iI@^eQGJ@@Wz;>%d{l69RW_`SA-N)2Y(dL%#HU_h zkduNsiYMy*(;+eOFx2v+;god4X_jV!S^5uk5xHSx&xx9=YTiKJrZ7aZ5}PRuP>~C0 z#iNA*2`;ylKt0f3Iez6y)jCApJhrVm)Lb|^NYQwSbfL3fuedypqOv4CPJQ6aA8}#q zmFDqx+JP*jtuu_Y@_8=gUjRXjDEJz?Q9G*1jj&+D*`3#HdYXfH14Gx&;b^Bf)Bu;k z;r6|V&QAr>{EPvB@~Orb!+~YNSu%hbL+Jr)$2cVL=4uFK$j&gd>SYwN#=dgAx8Tjc z(^^Lj0e^PkdMl12<5~ir06KbXNpG+pa*2=CG`wM^yB1*I~>2{1~!ugkF3{jcNq)HeaA_Z9J z#n=KKVLn<&KXbQfWn+_RSW(tVN2*@K3Sl*jwhm|zrQR|=uA@K!0qJ_vVcvm#G$KEw zm=AFt4F=e!8FSS`Wt_8V?T|k^H-Rz1X=5ypXzF_kNWC*V5hiIVa*LL;qn&m$$Se9O z{x~CDZ8f9?9Z@I6;jnAF>4{eKf{)+%5%1eh0nG~FALqk+s?%~Tj8cYI%NwK%@_@$z zptBIx*3pz=7>SF*qr1~#`P(lE9;x^)uVcW_MWLm&oZen&vU10J+eFcb=!SigvhVHDd!_ZQBO|{>S&3#GDrYIW8V;?(U@pORKXq_BDKSZ=|Yn{!_9qtp1 zKF{H_0ND>a#GkW|rXtm;nhb0QQ2P81h4<^U35T?~Q8VRz^ls`9wM|y*sh_mEPT{%w zN>jokWL)|Har-c`?jP5he{A4eV@-KK?>e9F%u|7%cV%Mk4r~G+UA>GSBQ@i7G_gRs{!IR&avEl4x{ zo`SjYzb#@JE)DQ~WP~_RHCQL@T(fG>VxPYNIR5Q3$Ua&GW+usrID;_K_*U+Rp9LTW zfXDt|*nRg)5GX6zTqlA8)+(kE0Uw9lN5_?o4`#x9Z_^@Z3t7Y~dnOjcqxr#by!jN? zd?gIAUs>+0rq>4C>J`|2fCesmi47{7hA zy9-5(jla+RiEr9PFPA<|#irH-NU23)GNJ#ea6uXH!j_vLX+ksoff^zv)u% z@e!Cz&7f@h#M~T#a1P7?+f63Kwoo>}Zri7~(lrd&B~H7>g2?5#p_lN%cFm6Hn+5gk zm^9-oU=W&Vx;7kf2N$RX17~BvDuPVUR@jzE9$E!)|QSc(lRdq_~J#j8#qpP5RNar4CA?E3MDHj0ZMD z^$D-cbD(h}DzG8#A&wwT^w!0%2A^q*3aj+gXl%m9+0e8~mDL3)fDG@LOf@6kQzAu{ z5>SFJhVM}qa)#)+h3CwWQaeSXQ^(+0NRqc-6Nd77e~VE|fHQt|3Nm1kE(jx~qt9SW z9Xoad$T>nI)t9He01yrm3ISJ1XZ@Mum!*c8&<2sf_hnq zzBTq2{Jqs|R)T5}O^sHGcr{`gMd@^VS2Y_$GxwEt9w#^}#o|J4*{T$0$Tl5z*XQoh zs}Equ=A3c47ytZSSY6o+Lqxq(|L`CzJ$n_lnf3UBYOH(@6>o(!|K{g&X3%~|ajVg9 z4`sp)(y|w@na;a)24@TVYP9A$dGn*tnP;}mO$TD&4YWsA_6bha*T^gyRj%St)f5&- z^Dr9pK|(`UAjRsOLi;p%(R3zVu!3r5`T;Jd$LK>TZW+d)?``b>yy-L#dndpQ<;}9+`VY;t7!y9+{!|HZ>cuuPB-tefQ5-B>W+F<-LoDav>37D%eqNcXg zF5aNm-29RTsib9+G?vXjgeBtNi{DcPy>msk@Dk~j1GF*7B8Ul!=yDWfZ*FeHa(?eX z2PxjIr=Nw3<1LCwNb}B3a?a0bHkt@8mYx$CQf&qaC;NzfBCSO$h|x|epc$)2YO9Lo zWo&XDzg>lO`ZSDDJ*<{g-HM~ZoGDmiT~E)sL(u4n++bKeyruQW*R<={4R=}3uCB}Q z^!e+s@il?X?Augky%$<0^8jeTU<+x53MsVx@n{fr1h-@g&PY2sip!4nTdQlFpSE!K z?jwNrejFh-Jv|*Z78h`M+3*G_?WLs!`T@QUWAq40PtaE@3!6oXzl!FhI6VYgR1d4m z3t^ZERi!>!{)`ZP1DG!O`(B2T@2u_s%5c89LdcFxaEE&-gt5Ug+Cb7G-5ms~6o6f` zmo!D9)dN+hME*(MJIiM;eg7>^7`^br2ei283j<^{u9FoU?>6l}h~i0O+OuxIvPYy3 z1A;B^4i5|shNC1cRA8`lENy(g0)EdzD`(O^jTR&cJiMQ3K0U`e#tv+fY=)OoO_R)8 z^B)}<>&XACuuP%-GpM~*Plp}xYtN9A@#tW zT?Dmh?=X{hV8(CJEzVDu!kzKKu&}s>^k6AWjt+)7G;GNl6Mfd66V%b5oC{(`qfXB) z!JnBC1G^K%>L(QjpQo`K8T{cn1y3gw^<2~Akql7-ty^Q4fa!!afPo_l!RMuD0*>aE zf~xM3JK^c^Wte&w=9nhUhkX4uwkk=ms{d^zSaN*S(cx-%|HDU&)S^)@_O-Pg@QW{q zf_wryL50<`&3{~PqF&a(6sBv!Zz2P1^{QDn?8T%Kb}>VAg+ne=NZyRL}1E$dtTJm=_BL4;rC|nWbb@tGYwIPPp6yM9)!;@2=X7( zuAW(gAn%(6h^{ez0oa1A#Shql-~gb1&1a?{ybQKz7sV#9$TxgLk`GI9sKm97x@aOb z^3pUTa}zDG_F=iQ96khk%o7b~Bsh6M86btAvMjEzm+_2?U1Ndyn{Y_)xE(g`nlyz& zqy-F}P{21huKdt&c+8M&2NyR0HHxr-EI54ji3>g0Hy_4{u58F@q<4mMu`~2O8(_0N zoAuUV|GbEqT#?^Pe`5@KCd1Lz3YxYF=DxijhIy_e>83-N#eD!>0XulkXRUlR26CZ;7tw^cBo~&}YdMOi6 ziLg0{M#WaY41?-Bi6)bzf!i@Ej(wUMqk$jwQS0 zu-Ik$hM$oI)gM>4Nn7La=`H129No=#xHI*IzXiPmKk^Dra(XbCO*Kf-DEaTbcu%ryYt1?!|}U!Z0B zw#!+c02cLvRRES_y9AH12{6x)fZgn-5Jq!qO91Dq-kX`>eMqBl!L~_*rg#?9gq}kg zKwB^9wEFlN2kiYJm$4UMT72$hus zv}>e`IZS=Ud>EV35Jdq3JAWf7H!8q zjZXu_Pj|u>pMH=2bO!K+4w8a}BUna_(ilU>vn>Yzc-UAfJ1a-rH=Q5p<2H0%H?WXE zB3ebR61l6nxe139Lm}J@!}Ql|r&92ms==YrhvDU;XJPr(i}2!$&%z<8?Yq+xtaoqN zdW}|%RM;vNzf-dh(9aFV4*FK2+T?SBhOy3^%+Sy9nCO0=DnI8|)}auSr#L-OlD}`q zHdaaPU7>T$4bgrBZ4hA)!7uBASu;S#;Nk?aZ-PC}`z21?A>%CxG7mUHvj#RYmUX#c zZmfDUL7dHm1vTE2*ioN5qeB*ib#|%QG_`AJ4CcTIIm+vYYS$V--nujy(5BOFGWj|s zwSO|#kE?qU~a#tDXQ_rf24{p)b|!5w-O7XKoaN7{nzkc7ywclIY+UwH|$f;<2X zYV~-7-i*5d!)->p(QlGgA@wp%%d4C*?1*Qn${wc)D9hO7&WSrAIXD%10qlK)C4v%q ztKow(1Q2`syR0SSvbcMn|Mpkmzr!|u_SrXdVqn4!n!=HZiBR69eH6qC%M^VkDuK9e zo&!8XTnmF*D??e$_s~u`j95X1~hi*RR992log#J0_}O~ z(wG&H0BM4I*T|#CdvKlzLE`88GX%eJ4Ixi_Zf&Jtw1f4sy48`C%6yD4Od%TtaST~Y zFb2PrH`z&|6<@1%`bqrh<=iJEwIfr#h^75yQG5Y-DUxh#Y;-ABJpZqR7`}C9Wzf0$b5bNR)SezPg zW|eODi+g8bm-NWM{l}qj=grVJb2lUh#=;ikYIccg>U3%6CL<0t^<^3nG%R>xBjZRW zF0uWu3PeB7*0e#dRW=$jMf+B}Hb_HmGLl4w=7`9o0v#`4>f+GI67JFPv`#+8Xi-7& z5g>4Pd!NDJI~aix#H%x*h(;ui8>!J8m1Bc)kIkZL4&dS(w-$_79vw|5&gz78mn{ee zp`80k!^O%XHvELNgu+TwHO&-&PsC>dM!9MnI9VhP3}>r(y!;K90Ar<5v=xL0_Z)y_ zFFkb{vP7Ko~O8gaSu*9KyvwkgtcJ@CS zZ7!?@4Nf;m0N5-V$zr4%fChi>d+!}EJ`2`WfUr$rKmul^9TwMTNn@DnI9pi|BhPaI zE}w5Tt|{m;4BIsUNiX+LgYoV6&#)J)8jP^(WfYsGP{#EA1w-U{D(38!!=rmMFY8q_A*(@$Lf!;-ubZ{)Vofh-N0Y$e9y7Kp1#{4erl~?TW_A!A=5P-s>N98yNzY>t z4mY==cC{DH(IxMnqj!|-!!he7b39Ce&(X<2oJ(r&I0e#EGxq>;CHCA18J3uo$N`SQ z0qm&^?K65=Yo?K~1-uWsPNL~4?%xj0r5)e(?Tam<4N|SwMyuU4*t~-?{4~7r_TN!W z2Q$ZKmUmCkf?)?p-QAlRjVY%CbTk#Zm%njuilQdY6*oNvh_wLVj9{ssV5yZma)8Ed zgT7ajjFIc%oc;3Wzi6gMNPP?g6xEbf5MY}cX|VpQXcJ3ArP!4DXn#Fa(egS=(>_`` zGWw(V}HZYx=8NpW2hKbZ)n!U2e6Wh!>(f(m=nc^oW$&l6}4ei%#tS*Po zKKVW8`vzIbz3`KN^AG50#$ZP_!v+CKX?&Wr?M>LgHhlHEWdSZ-pf`>wLAy7X&Vbm>qZ5k}kb-ay?Ww2+Zen&GR*eEnibR}rVV0#G-*JN- zDg(Sz1eX>*$oi!T==Nc+wV1plL76}kXN-*&)VJ`AdiW&4iNMX-v2EBEi#TSP&1>^_ znQNs5uA6L4g4}BxelU(M)4mIkxl>)TXK1S4oS&q+Fc<#(DSoQFMS)x{yv4+XT>DA* z>QBE8@4Wpej8IHs5sG8<1k$#*=w&}{0I`cNpTfYE!}K)GW@#keN=5b_lN&Oy?-u19 zuVHs~Nw;1=5il9%gy$Qe@ZytaE1^!Og2Kc1L$wEXnnFOI&50*l8)R>I|LrZ(`C=xD zp@#ZN@gJZWUEYR8gXHKLuEdwO=rndfuK z??xTsEni!J5uJVTfGIfxydU?TwNvx{>;Lrsai8#zfH?hUo;&FT6Yq${JrYOWHvbDxe_KxqJ>Jol5#hem>nCu`XtHG0Ax z=@ZqDACZN#3fHRNUheag&;JaNLuLW~k+c!~CVo6T@R6$`Dvo{D@R|G#iz~`Sugj%W z&~3F7??u}IQ;#r2yo}QiGQ5~ojj}M^+;rq*fHD|6q8x{KCpi#Zd{z?rpX-mfkMdUx z*f7Sup9&+z&p8_DX>3QwrZAEtFjxIi!(s!#2n6uz>WB8$H^b`}ufpmgX|jzKBF|km z+0-S~VZ4}4P-)`M1w|i3S+qT16#NPJ>`hd_$#jyUT?06+(y_Zi^t?-TY-@2mbd28# zx9DO54r2>Z=P~V2jsbNQ9A68pk`!o?l%ajq6eX)7^T3IkO{r!9E!a6Ca+=>_sIX~K zXJyYJ8^J0b@5XSXVJr7BJezw5RO1aWk_UE$?^A@V#%v!=f%It}0r)7!Kq^=bRVN0o zhr)*<>9;m6ffYasLvOiwIvs~&%P7=Xi!4ss!xn)Xhrf;yV_Bl{HzJb(@d1qx1an4I zjus=%iOj2$9yoI_O#m(qB!3YBD)*3QB@*O)?E>#SH?<3DP&DiURSQ&%R;9AjMC?BE zUxFy2cXW6hZhONC9Df*pCVOY&R{KVo-f{k_QHvMI#?6lI%J5k+ism!?JQ+FJ-Xntm z&fhtDqiCGjuPtTP9~ywQ09@U`hIX?q zrZ|P1CxELu&!FM;A|%L7D{MC^ztZ_}9(#C(6ak76Xdi5-U#9~*0if?$8B5Q^L(XK5 zee9wql!nU-DJYXT_gU;+==i8QPU8^P_MAx2hNYd*?Fr6N2ECBZW*>k0vo?D+r^iBrKMYo=CY zX{C%_4A}&?r9z;QrY*-cL0&7D<`^AMZ~rI^@>s}~kN{lSd^=4bLqP<^6@`KE@Xa?b zcn|vW4b5<7a9}u%#f4YwK}-1X{kH)nbo3|1^z6H@aP9;ov<1nMX4QBWVLXa3`&M+8 zMg}-fjj#;U;azf&w1ok-*05sOb`W_2AX6VYKgH4~R?U%ur41TY???;KIeqczXR%T2 z#N=$$OkL6xv_SxHiqlGWlTIbqaLc_^U(3a1-Z6nLAn_dMkk1eDOa=B9W&rs`D^7Ko z^{u0ETzmD58LsQtsKM~YdwsN#BV%JoNGJ^94qAK=i^xbH zIfcaPI6;hHNC|@Hu#alrBzCERzOED7l>=Drvj5PHQE%ED(5=B&-$dmrfX4gR#lN)| zvc_5D2VH#cHa+&7#9-iM)u6>n>f3ZyPCRfD1c#eMMLLWCG_82ISCv9?83|KV2EqJi zOsNarjFDWEV{EtWR_tk3CRJ=_eQj42us$+9iS6X9q>ux!$9{iIRRK^o(06s=)UBjf zP@gAwKSW<@I@b45Tk1S+*>P=~T?h1QYa^*ZzO?|MT=--H~2bn&yEA!dsYtcbQ1XOd3%X<+{35>Q;Ej%4608)#ma-E2Jexd8fE9{`H_ut0 zTDUtmf?cKDeFAR^S)N`hLIuu!br~MlE-27GI%#MXHYu??7j%I$vUiPC%32x;GvE_`6?o-z3+4Wej-|3c{lZC~#}650)KU z?$*_w8FV`ts~x#zH|$_bVWjbP@`L#CJX8iQANPLP{vkG-MxpeMU0oYK)Z1x2E4s$sFQ$ng9y9b z4zWptV-xWn>+p3G?Avbct%IO8cH469u030H$~q6*CXvSOwhtq&jpt!|eVNm3Qv<@o z#7z3`P9K<$4^S{>Qg#bkYdz)SxnGp~MOJ7J=MWgpi`@7SMGm^zga&rPo!0Y3B7_e4 z%}9!uw-jGb5MyL&Sz!ccU65}_$*0K$Uy}`9Sfb#Dx@b~BTbV4G&5i?8CgfsYkwGGE zuKX13Bq8z^AJ7C2b#Qez(SMSFFIgDi;o(4WMteNu1ct;f$5ABU9%%IKE>zHSX6`PA zhc;snnAly0@jK;SMEJbM&e8W{dAOa;+rt6hVpH@U!cjQEG2qX8*vuI=u!qo$!420= zcF2XsS(#=A2@LfBhQV^%5o<{@WfT-)`y579HRgp?X06%4VM{lO=(X814$(};w62WdCyi3@i`G?Bp1B_U18eTzeYM3mfEu1JNc2 z1;fd4k0P@+MKsBH)!)&uxj`H%?>yKp+SL>`mIcGYaxap9xfkDyAhws1mzy_1kg1cf z0YH>!pCIIrGLl)gWo@NaBvJQ(NLoHP5kkkoiAbgS9M@3po?IR~Of@OF^|Vr@7<#NmF)b?z+;f)MuFks z&=0S#moZm%-R(Eo-*k^DTKdUveoaO@K-qdKcBGC>P9s;d5Y|EAj@u8zSZ)9m6J8gn z`Vo^swpZZeE^LJNK71HvL2*rDPmj5m*v&`Q&mqcjm={30@^s%Oi~R)eq952X&SW>p zc{}BhG<*FV&AZOpau{VzgsXQSd)z0@4{9e+2)ngsLsj6O#~gqcZpL=KsjME5@fKC@gGj}@w#It zIik1nprl>&^3|m3A!@-XjrZa27<@T>fg6poE*d(^ecCARI|eO$o~Wc16&xMQW zdTcjvf<&&mzQX&_DGGfifmUG?s)I!LbC6v}_3hDddPb8$nrR^e1cu97d)RI`eAgib8*Ft32uT`EI*5^B$Dz(S5})m^ZX|*rk(1Rk6W)H$`t2>^7<=+ z922EZ_NU;0Q$++6d}FXCf;$?<2(WFm@ZZK4N62oF8S-yJNnWc@l?U%0(9zv+)S!E@ zk7CKv1|jy4<@z`!12o+0%$Mj&KHUM;Lmh(T$|l8As#f|IuhT%V1AlG>%z#y`*i(Xb z_mnTOc}Gfsl^;iS`Jdc7*GAM)nolg%vQRO&=XbnE_qL1ST1K&A69`Vybxc@q&}|)< z+O2Z&ydLjE3W5oVD7JKkcAl}R+g99M+84WoXEP z;dO2>5ILe{g9HiNC-5Z9)=wL3-gY8P4B`ofp^eCtge&CvvaHo45b*KaEs0NI=r)Fs zjdDUOcPFLBJPudb+zn{g!&o~~3`i&7#={;9Y4=!yY_FG%){S#ia9D$C zc}#sa{6#E7cM(K22rMoUHr2_=KbhcAHaQIEWDA?1G!kW9;}X}UFs|fv4Wm2**Ug{d zfyo$={ou2`Q%fAYA%7C+u{t7e4v21GD4fUFP1dUXg`{{_r$po~)(%xid# zXP{WyaLfww_qYY;yC(U38wF|v=+265$Jxk7aLw$PlNTqaQcb=?)}Wh%;a(DbkfiOj z5GQT;e)1vw4At48XogMhwVj~iiUWSQjOsr~Yn@=OZ@dtOrianef(!NKbI?ECY zYbV!8ob|UkrX5UJC=rO)xPBZ64*sjh>*3be0HpkJ_SF#_8qk}~{m{kUY*G5?1}UBb zeR;aD9*+3Tv*uz*qr`qPI}Ddju)KY-X&nF(Ctl-g#o{Uig1om*{sr~1q$CsP+nkde$TZ) zP$jd^Lo<-8vcw$UTDS+l@U0(yh@CnNn@dl_?>_k=JiI*}e)#=&!{?vjm)F?{-mPr~~@{&CbD%a_nh zI~xc6m?tO%5o#ll11UcwKw5{2u~$35NmB-eADCyv)AZNW?rTpM!w$%niOA3xyhpe^baLRROJVi-lh8XfP9gxh zOBZMdoyqNOaQm{Il(=z-BzYYqY$lP1bgP9nkjl~^dz7wd66~E#x_V>K3WAHnyvBK3 z9@Rn02hxrh*l4TtSMqa2)#QRjeJ=LP3AWFUziUbJd|$-UfX3OO(kvuu#|bQ)J5hrZ z5*`yxZLkjA>J6{H`CRr(EInb%O%^@8B6o2lpoul_s|dEJj7<~H!VKR|GjNxa75MX6`}<7@)ao4{gil3R5Ex3hyAf zTcR|}142(V7v`%dG|+t3rdU2_-Hz=_p2iJ=+~c_Ge@&LL%C_9`f|>Fmj0)k@qfN6Ti&2?2HrdX)+TbuIJ+j)kq72V^Q?^@{L=uV zjjne~&IV?pyfF|b*|!7UphK`|AF!lEuxk?4pptZj2)^Y;JWza&eikodFUTe1v-vVM zFmUl2_jY%nXAu|^C!E1e^tF~&t&X_2+(Vr-!t4?Fh#(3GoKac3xj*+k8N1icH{?$G zpMEypi@_S77t5EtMy{O|O8y%E?whz5-EV?R@jZPv?oAZ$nzQXW^Z%mciE)v)!=K*P zIq02%8-I;!WD{Ky4b6QJyLhjB--IK|lOQ$Z_vHDvuoEXF{U$-ak?}#t+xgr_s)su_ zQ1Mp}gnTU*Umqo^k|2YKfgEpSB%a{smQ_Vo`goXfD|}Ps80_XNQ8r);MCjTmQiv4W z!GEAU7!4h?2%q{VI_^s{f=hxSMLxSIFLY9?&U1M3mrO)Jo@9tfNU>YM@=zcEO3|H@ z=Yf~u+EoZ9M7k<$%oIe>wBxK$do56M%K;}FMSJsd*3=QYJvRqz&$I&!l$u=q`7%h} zdPGSSgWYu|T!PGr5^4vXX(^mdH+A|?|N7VP&8s(IVGI7xK2C{4v;;Z)HN(!=&$`0; zDcVdJ(Orng+jN!HK(z>u*(f*NQr#RF4Oka{+DU7sSf)grW6g48j|M?WHs}dZt~`!1 zjS*ubLzLJ0;I|biw+zILRc_lhqb%+g=qM~Virkn+6WACOCr$8rP5^qu0WI?T zvoq6ZQLV#SLVS|LQ-QyyAf`6pV=*DhJZ6* ze>mB~*5=S-I+~s5xz00qX*-}Mr<6?% z^rml5#y0I|IA9xNqyz}-uB~mu6`^cM=1e?~GXmkl*$u+GJ7(S6a6AW;!-i&NLl=Ae z1go#H6p2u`D}(xk%;6`Jy_RPuNfrnexQ8x? z#7)*=9TNY~|BwG2?Y)ywW$e_|9(r7BJ7KaifsiabHD24mp-%F<)m@}E*#D#BWyT62 zYe9)pDH2OrI_V*JF?KAu2Rm4sd76GEXd>u^m$D1;(M!Owvx+l_CmmtUMpX@LSN zxzPdQH$6QamKPRjkidw$e^*vwQfH~&xq{=o{tTmMBGP9BR0 zlD0!r!Z9)ra9VL3JzYpBoZbj|l4Viu*yF>>oW1Jv3ogh0|%H}oB0x2eeJj_}Gy@(!QpVM#y4Sq!S zMepPo*({-^WsbyYYh{gn1bU30kBA!fQy8Nxz z#z6**>O9C%7@cN^-Q$Gqif; zz9C4WT33gIsvewrwQ9Ug-?|m=4+GBr3gK7(<9~48;XjhFD;YC5aSPg2A%3o(C7LJ) z)KaQ@u{1bjR1GJk}&S9FULI`yOjZ`_ObUO*-D#qbM2P+JAN=Q^pPK`dcO|Y?oE+2;W}kI zgvI$b-xH-G8{%Od0Rfs#C>h{fN+VD8#`b{#j=8ZW5A}?npBtU>fgTLeGY?)Te2#8z zO5`r-!UHFZ&?ULR6dbW6PgjI4A%exTy2AM#%Jre9t^rTL{TZd~HOz)BB12N(%fRl6 z7U^EJbNc9tZQof8*N7l?)Hg#0d6H9*gbw&B?8Z1SR$Z2np%M6d>G0~SZz3w8c%QNz zB4|qsuc%*dfiR%~$3fWIT<73|*wHl60onS1>~H&$_J0t&{zv~lgwY2fb?--E3AAG; zI~G!8(-JFpOM3TbQyYd=XYe?uvn(gOfe*$7F21pj`>B!{PBZ zHYhH*b#ZinqPDHhR~&vd$`W;DjqCXEQcemf`IUp zf#X5=c!qXz@;ESDlqgwyg@U_z-m!ZZBW?zJK*=>hpss|9Ijzug%DO$z>wvWpIY?sY z^8_wxuC0Tjrg73d@K`dBnLm3*{VNZ7lfXcZjlrRZs)W-JY4kwWxJULqhli3V4#Cc` zf_i;?t`j&6-h7{=91k)rPCEMzl$H0l1I~m(^v(g`39e5=toBb1W1WRlmcp2IlBrn& zYf#?m;AeO45jbE>>Yz)`?Q|ciLnR+A$6+^X(}5Y#6hJo7lm$6-Zt=tva9%~3MDz_}d~m?bzTSrUr|XXTWDafdd`J`8Ld358i(nssQy zDL?)EYlc&gA>KGc07yoQO8xrcI%pReAN;Z${EaP2z$p%~s7{^Pivy!Y_7~hV5Kd9f z>h31b;1~lg8MqGDWiL#EeqHu5{JVKY$-b9>r4~jZy00&DN4;qwt9K`)=U<$|QSK&2oQ*{#tn%#MjV zOLMpH;LNZylv{=;rf9Gj#&I#hgJ}k@p1;5_E`?c0$$6Zwb2AMTG@weUoFXfvQ0hLJ zdlxoCG_sJ-kl4(G=ih!4re|je${z3>5m;uA^d&N!1xhsRpYkxDqc$DdQcs`0p$wX% zgY*{0dmKcTMk)Av8+CX+lxCBqNzf5c&Cz?If8rfBOc~ViZO%U(yaaS_UN6CWYlI2b zeFr|&-a#wc%OXianOT|bEMBZ+U~H8}Sr);IzG^aQ(U0B1q5g5x~}QKzLDTLTJ!ebouYZa&uhMvN3&&G!yL zc%$Gs$t6(_c9S#1vyro88A^mjZkd6biB1EV5&ejQy=qp8rkQ9pxYwJ~=?&`DL9(b# za>g#v3o{>~xnTDW+2fwQV-kia6UArRw4^9>35r5FjZMO-dge`-JO^u_wn;e7NBq8} zJsq9XDV)*jHR|l0Pb_~~zG9>su*cL6^d7bO65#u=UzIUX$aOVXFOz4h7hO~U?EW7cnTm;rfu#QJ2+sET zF&sUr6w_nSBQB1RN?8ogL9|Ce+VyQmi`9@|yxJ}$)GPpr!HjZTDc0fHay4u=NVX>C z!sxp{30-ow2~H)vD>T2!nNq$<*$SIRDr2KzWoaRfA>7+mAQn)g;N%MY0zA_zg66h% zNuJ7C*z7?7$NMOzYoNvQv1ZsG?3O``vrnY)uG7j-ZGAO1W8L_+ zX4E-z26??oNtyi^Q9f+9m6~OEpIMT#EZk7{ok849*_z~C|6?G@4(XGoW2u#E6qPcL zFflMLF(@=qJwim;;Bt>fvbg849lJ;~wYWY(6NAET?J{q8u(OVRX+`n$0q!FY51Pjr z0SOykKKY-&CULRz6P_pMZS>Xy0FoQD&i8TO``MyzmQ!=^F0Dx6#JEOc9D;2Tv%@O> zmjJE@eu#LOZW-C>A~u1XuzwT~_8FLjH4orGQs%e3Dj%^< z0B8J^M$kJA!h1w8+41vA=XN%Wsq)sUd8Tqr{G2HN(K!6IAKnj$z3T*gJ z9$uzA8W!a()&RiZ;_fb6>1jYFZtaF=Xtkh)+F9#30sC3M!; z!zqJ+k5^DbUtbCvY@Q7_Hqp4{7GYNxqs3uoC7CEO`ZvT!eIi6f1IbgoH_?~;X0b018H zAt<2a$^8ns8@=ySz7wr<&7J6w<;zxTMTHLuGIlYdd!Tc2sv-hP@KxX|L+M2kUT2gl zHVfN?fJMzYoFVVG32729KzRyd!)G0G5JikUSkS%_Szy=De3KY+&?QJG^7h)?#C z1H_T$`B4I34_H!d30?D_P_txUiPP7tpQw*U$35aZsnuk_?;08ein58IeFukI0_nwg zc<-XK0?-C&YN~l~`6v5`G9&v(p4`6jtx+w%QO!A=KGi2TQ>*ol9JY86+I(3h7|&UQ&77j9CCpJU89S%52US&PVX-G zc0=Q$hBURn0W0NME6(F9O0xqa3 zWf)|G8HQ*$N|;vyU=P0ACfu?UoNkuyRWS7KOS^RobYqL|OZb(1pTyBR6+$#|vsxt` zA>qV8VnHG7GnoQzp+P|lG%`(j%I@Xk!<+0r2Ul^Bh71FfUTlcU^hpvHI{|OqnxdRT zNs1}}+aU^%TM9LKN@H|ubx<8j;G$V+9N+U7tL$~I&3hqp?}w;vfOgLFeEsre6x}^I z0%1xH!sQ)=OUL@*Y_Hl8g{JgXBPeTu_-XG%+WQE&JK>95;OKQwQ|`-<002M$Nkl zk$c1s?wP{A0P8z zdVncDnpG46;aDAkIHh<{m`+eSG#3xf||N@3(dEq@L~-vS8gl}KKLM{rXGZI0vh$}lyrIh z;svF7gphZ3qFucE);o-^+ht}ovwyL9%5SAeYz%6(ui5f2dl-}v2_Rfi41D~Vr9}e{ z0UMSl21ZJ17#LYrRzbi)@JeYl*~8~nI=D_{TMh_JoKI&k`YthbmJ`BWc<~Zki z9XmI1`#wnMaYW^;I5glt+P!;BIo*6v)@5krZq1nep_whRy=Y$^wik3Hu5JUUO8ZFrlVjLUm z7)}|&IYt%%u0uDGW|9{83!VPZFb4=EYn{?n zp%RXH01K~Q^AJJQC6{k! zbK!NI(lDi+GwuuU5#xn3Ay84Y_5uw(GsXr@4whZ6*obHRT>_?R>v94s1YmYe8t9yZ z%EXRM?%&|B#347rx9hXv@AG-;vZ?J>Zi6_?`k7S@P-=CgOXM@~tkk^|Svv<|S^}-P z>cC==%Aff_-|$T0ID-Naf5o{fQx!Hd~3mk6x_)|PwXOy^fARX`z9j#*~YGM)tZkk}ypkxLlDO&%XdDx`vu)VfQ z;{r~NeQEc0>lKLwNQ5J1>~0hN>Et~d1YpV~5c{zg;c>yGsbkPc+iOnX@K)&?WukQ*plKInm)MR4S>`km!h1boXNni$v%=AF{fbJ$p;D-Ee2xI6Va zou25IeS~xH?CN zU159`Vr}jtq>Ig%m}2tBHM0>>zVBw#k z)e9}7NSFNp4HWyU&qDphV{Dl6Ga#?*?IL?bB)#eWrX$f3w4=L@DZ^V*A@d+ObvzKk zcPGdd0ZN5+>EleUtyd8tUM1jgOKm(Ieg;LevYLN`-sM=XalNn3A&f;tlz;(89|GXK;GbVNB zb&=bopTRH1pp*0D2UzmE;643(Q4u4}_;-G0U}A8f|MEX68BSv`c#%8jmvg}{LDgnJ zy>6k{+(cD>d1X7I!io0NFgZ|!Hh_mA;8zKtzDEZt0fH(HJ-p8#oE6N` z#?m4UADdwmL266RJ0Qyo^KaNs-H1lh>Akwam@JS_IBul$oH6*?8yoOS2g3%)MRoVRtrl;jn=Q`98*o3bS4k`F-|2#K>p!IkIMc&oy-I zrMn!r&gLq#o13??UoeG}ZKugUc%C1ElPTPHz;<;Ud;0loVdI1XqJNfEO zVhAfs?$`J;zVBy=A{spUd9I7kN*}3YiRiYUqtb$$Q4ZV9y74#v>-dLq?jwoDTsZ+8 z-KRKIiK1=*8wMIg&}O>$YCe=F%J;YtwTk$}<_H|B0>K6Yj)b!1-nLpp##Nx#WwQ94_06yVdZr#(jq*T^ zL%&0C*nnr_cpH)K4l;a4W^KYzGYGOvX^T=BR~p7fQ9p;Yj&%+d4&ETnzT8EIY7hWI zaRQQX%5eFuXMYKs&przY9P}V+`lBdg*B2k7K>t}dLipEFS9=7JJ*anxYZ)-_?IK-4=Eg&h$bnitPRiH?spux~CZ+;zrxe2>-DK_C;ZJpt`4E5L z2r%?)!WPC>2ViF?u|g~b7c0YI6=>UF%Ek#hl;t6KFdZFc9%y%PffA%mCw88CR^%pn zeFmxrmR7j53qGR>(qxB@W1=(^9$tA#ath48oaE3(FqEPTv^Yet0Vgj zJ5CuBXe99fxKGtt>R|75y59)!QS*qwY?Kl)u*5Oz)E${tCoA8hEqw}fLD>@nJ3B}D zNS-y%n|rj3}VfzOGv)rX@fRfcY)9e5_T++X4>+Cc}}c#gJ_-<%%; zIgqFxjL|M7D9>0sd+wO^Z{x71xMvEJjBeVFYu66nDnBqwQwZqS#7tO+%cq3G5ao7f zI#%JnmZIZn^s({24{N9iCq*o8=7jD*-^JNF#^k#s<1=M`!Fo_-~yA{aciNY|>~k z7U+xse+N`RoyH_HNiPXL3<8En#t7;dO~#qs0ojZ^m42oekjWd+s$8B)5s(wK4_XT3 zOhnOEi1YQGeYD=}JUa=WfAM)d5A&2Nw^)Z!OUs}r8}Lq?TRZ@w(?c-hOv?)bqq9S# zi1wfta8A^5OoNPcV!O5$U*jB38F2p&_RMU%Ej<4GY54XV29g&Ecz7xT&}yR%4x>v~ zT_gZ2hsj%aI9rU4BN#CQKj91vmeQb!bQ#l$i1D9>E=}{9kUjbjsITs!uklB1)Lbh6So;uIJBN=Ga+YI2mH zp;XC2s)w~t(~aGv=d7PTMrou{2?;3B(;_Hx4*`fxUvf=FZ#Q`6InWVNTy^XO4%BND z(d(eRERrO5(*_4S64eAeWE#&9W%gpZM45jBbjDtPuVs!bXfBr8(`xiCJ!?_5**hPtq}wo2lQd9`K}l{f^+=X!ZVr( z)_6u#a<~_}O<%uw!g+6mJF}EDSC**l@qX}QFB)gz&iG_Jr``B~WVRSS{_H7K8918* zx5E~b7CQKhv6<;uo^8-jbBPD|6{p#7io}l zN`PIIF`U!cjxTY~DF>qDhmC7lN{%Mw3vm9Dti5QnNLHIcBl}Mteh&zk!JYGve!w|0aOBK$@26BBUK?m+|&ZqKEyVO!pk?^hX!SEXJ>l``iO~7$3Nb_|IsN((j}TL4V;)kkO5nZ zi#=jN9dFa(PxYllN3MAw!s9}!hIGb~!rkhK3DsK)?PQ^J*dlUfI%a0=z{nXQWNWyF zeP#ekG6-=iMS@R8R|~lh9tSaqQ6TXSNEb#`M=B@DQk8}v4IeE<$BIs!%Z(4+K!+z3 z50i^}z=3LGA;5}2%mRXt@K|sFSCm=%(u^&ly*f)NWq=@Rge0;^He<6yk-)Bx0Pcj& zom~Wk`;f<|VDQUq{`Ql7M2eBe;NZ!m4qYvkI0GacoNhZP?IoEHKI(ZETy@3HTq-nz-PT#%--zFDo+}jce-635_W=fHJ zMRt5f35BN-gPbhuxeX$;jzc{sICAde8N05N#T+OWV>heiP`h0%ug1<`HM%IpS+X=s zb-Y)aFU!;xaP~Qndjtw1P8uzrv1JfN4tqIlmTnA8A}JTccfxb;C#cDz!LtYgWEsrN zeFF|*fB)c3kd7lnCX0ZS`{0?#*Dwt@WnY|fNHYedJYNI&ZXCK|q&Y~jwpClMRZ_x$ zTx3w>%&cdSIfKIeq6IhN!e9q<9-M#(HrZ1tjQjzI#}_nHwp zgVj=ThlFxvbFd*SIX~OI*<>zZr32+DzQF7{7u$dqP3BiMsSyB;oQnB(0@NL zw?@%JkH?8xUan;k5^5sJSzuq=5!TD|>ck-#{5EmU3JL3YOrA#Wy%i2AHPk>s&-gq? z&Pa3c8v3}#4%TdQWdoFsQp9Bs**Z`?Tr_%p!t439*nylQF_3tE3Buyg`Y*nn5A&-v z%BeHq|M~bcgr^&jd2chU{z07WdW2z-vLKD&f_>3JqI1r3aE@j-C5f@wY0j>tlJ;0O zGQqN$$>Gl_&SxK*RdQPzAmka&=_kMb7(F#mOwOHMqN4 zfh;U8htEF!iohDgmSD=~ut@-M$@-e9j*X(>G%*=V4l2E;McH}n>f)Oy2(1Y0!-o%} zV}AVSPg(zy@b;rA0_#rDxE;x8`nvs?b!znMra6)W~v1aFk2n4mxZi z@Ht?ooGzox&YcJEK@$Plr39Yj{dNgFRw(`KV-xc1ow>V@2>Nas6x1jy4i+IlBX>fO ze93(ugUT92?$Ma=^w|^62D~Ez85@|)@^f^dCbJOfL5xHOoY^Q!-%iL07IQi@WC(ZN`AWDiiC{k`gij0Aph=3ZDZP;>yN#Dy6Q-h-;><=md zCLJU&+7P*#mcY~yjN^_-3bp4>Gm)>_4Vk@-dpaPYtk-H>0|{+Jqvc-hQk_5>4htnB z64m{A{~dxxN#L=vLlP%{))J}#w9_*TI7}Gb57(@}BIa4jT{(EbCf4q^o@9y}7BV>-gEH&v_QC4G3I2e6|1b^AAa)v0qs0{rV2U56yJEUoY?9$yyHf z55&NAADW9&Hr)g=b$mp1V+F2kn&(fs7#`f$Uwp>CJ`L~9jfLfBFTyp*u1INvhM0-+ zPmP`>eWk@2be;)PTcxEwBF`C8Jh%7|}iDzTE zQeQqWGJF$sv;G&@Gr7>pqc}a{lmObUVMTs>@nh}ydkz=P6I8eO-4sb|7uAs$k3Wq* zJb`Ufk55!mJBtyew2JsG`&S>+!F?$8rJRXk<*}QXpNVwaI5BgFnWmFd_%?Xi_}AgF zNkE@PK9fQzwp3eh(BnRE=JpN~VC+7Bh`@3mdXS0`llc=s0UTjtKmOIPuoH4bDWhW_ z%A*qi777xp5lS$MESdXRUZCxj=W`t53F5u7E8L~2NDIA*5@vQZkR(na9o zwWm1TPM)|H5dj73A;ZGV0cYdTG49=LyO2*pM6QGq3^J@qIQWH%S z2gB%{yJ+}82*$wFC3xbChGF*pa!j24cUu(I`k5TOEx zLLtc}WfcjjHkq_MIIOK3+u+8?|AJ?FhH-Cj56YF85!qR6%~RwZ(nQw=#G)`!ne{VQ)5Z1m2S!qdO@7o7qT;x4A*5b8Y~{pDK0B z$Fwc|62lPX33v}&TYR0mRPr#mb#~~DR?l}@S2jr49o59}x$i_c3S`zPc$jI{rUR$u z-re5W1yxyt)6fh<7%v+-&OwZ;lr+x>lFi_H?3@M>Jp-))$m8?4cLQtgwMJHbVwTGl zOhT@||C7IG1QSUM$fN_i<=Q29E}h)7Q&M&|)}ztLV34lqgly9!FskxT4rv|@%hL!yj+S}SrUGGa6ISm$mEd9d&!azCE2eAZfyi^9UxAZ2;=%% zr4bbOlovXEdzOX)Pz~uFobwcjujsNw^=tM(l4gW-A95q;WfkFAnH^tYl=tbx?LZZO zq}Uto-5vw!sMD;aERTFU>?G$kf%7mBy}}WN*FpY2$n}Im31@_Yk~gdEF(vwX1~P$nb0t8o$IAZy{`BxWOaQ5 zBp6aH=vHBfQW?)I4PQnv)mn|tRsu)MJleE80i2p;`#YOA2?usM%5Jbf3rr8l$-g7$ zX8V9VQKlp-z=X5oDT(4_Bz5;;7pZH`KSj@T8cFtuIYBIa_8a@fYmJ3|^p*%ED*O$s`e8rPj6QR*SN?gBM2 zfD`r8){FQz@I@&cn+P|iAfkC9d6o{fWeWG&b=r$2XBu>*4Xv=)SuT=-c{x`9%j6h` zU0|&)T6J{U24m;v3j(JTBwYk0GSCtp-5F*A5lA`v;)wI0=2?M#YeHdyCOBc~*aX)p z3zyJu_I5YIBKv+6E@GCmengYkGGiQVWO!?alKlF5xZrsuKzBEnYfphNKiir z{I^gm4wxa_kr@eF9XRaD4Dvv;1m=STE6g`0VZo8abE2Ido}NXNyBidnfRj$;I)dd{ zXbk6|*wxJ?d;vil&%~wjF^gx%XnHC{NhH zNWj8iQeM6)0S&+6a}{QDOJe;$7US^>fx~o0%o{)*jn#ZzM08T6cPpy z*hM=w%h;}e{O5m){8f`FD*)R5ZJx(pe*K%+G45zi&%nUwG;%zH_$NxM926Vc^edKo z4G>FI1Jax$D+F%Z*Z{$P*j`yhw{t1J_bK+siilXD!v)%CB1&YaKClNpe0J2@pdsMp z+Dkt($*8R@!PADbi(Qew=wuS7#iWp166BG7(DBo`yC0ppXrL={DzX@FGqJ4YIvl6b zw{_X*Xn8226rvcaxoZqOq-J+Qq<%jRMF);(;ZR!ygr{U|#HM6Qaz==tG3YGYBq--+Cc0!sP@=;M1V z1i=R{CQwkz4dX!2bH(RLOz+^cdk8QO=;S*F>ORAf3q4ClbU=BLQcyoocY;!@M#CV^ zvK9CyM0kP*{Vb3yfF2{n2r}T-G|}+cqfE2A^oF32kz#b$c2J^eV*@#JwHxP9#y}FT zF~FR0L6nDx@i3rdfov?VQ+}4Ua(QK&pmHngH(6dE&mjY^VfMZ6hklTfCPCIRJfB^h zteeKt!8L}}5ig=tmJBT)C}JplO~?WPn9ajI=FK=ni#T*?0zV~WbZ9o3*r4E88l6TP z!I#eJi05Gm)&NKzf*F2-KtMj-8PCE4=Cf@PoYe_TjtM9(*b_FoXy^}kR^{PQHa@s6 z&({OaI~(=0cOAV2(t#sT*j5BCl1;h46Mmj!lUiO;3ZfT-&;LiMiXPU=-S4c|3+~(N zJ7fRu^HVTOa=^^HU7uEzhy9ykZYWvV^@4z9h3*(B*X)( z5fe4c;;;>9?WFbRX1prJYpBdjJ3M^cM;a$p4S>hK@B=^1#DeC9^VX zH{;6bmt0B1THuBDeR z!o%rNM)c7DvsWW^%u0~Hz}cuw;xTjA+T%UFBi zXMg`6*wZ)>$~?U|L>@2##!PK6C}2-25wKrf=QBZG3FzkLCMo4oI*#^>W}dA&jvBse zevsgw!v}BZoX<*wQaNSCk#sf}DNArclI9R_(%5kK&J=oo9VWN4PI4 z@L@$J)-l*gp1&RC22eUBxw&(9=OXgf6g9x<^l*Joa9~@DuUJbe0Mm2u5-TBzUR)2( z;*7HQF%1vrl%H!PA~l|89>MQ2sCqlPlP$_#$JmG~9;9U$$LIN3y`WgBe3_(~Bo>>e z1H533i`b&Fyj1q@i*LV$lCm17OPsgJ?%5YyFx;sG0Uweb?4u}b%i7y9m%wW$y*u(c5HHtb42n>@))(MJiDxyT&QClqonwo1fAdK zcSQEE)G_!sDA0+uaZT=RX#i;i+$&`o+Ax@%00}0Mp7{gL$2EKPl>2pm+W{>aDUa7A z)dB8o2FjMwP6QMrzGA&4khear&Cb)8FX2A-&K7yzlPMQG| z35s@_F9Wt{0QF6*Co31rnk6`#N`T1`c3fGkywS+e;QXCau~PKeB%%l83Yah2Rk?5+ zAHWlmS%ScYXJ6ufas;Xsr*}W;f#suez0L%EGwzh~}rE7l#ZYtNJhN!5ZYqCS**cYFC91%FL z(GX;0>JIcKsx6!c7eHl-*n!O+6gli49*(HHobxyX7`w0U-CU$e5?kCyvUvgmCrZ!% zu>Y_-B*NV!j!M{AfsFcx*c@XiAu*qYN!!6T&6gm725%z!hsf#(T6GhYC$VGRU&nhR z!yk#(*d4*s#yE1LERbkYBy{p`BW#`Vv5txlv3tEN`w~vd25y`3I@+UMApNpknjKG$KlowKL~?d?-D`YI>=p?Or)CwD#xvtZmvE8E;lc( zJvtXN7LC;jWug?%(~_o0ote%to7ec%!;#@YST0q#^$=riz;n({W}kzgl6CWJ%!CGt zCD01m=W!HG5RPsR!K3eg5T32leS?FsX72=rECy!;6mgBsoQ97kP#g_3A7}H5j^_rE z-q%?$~h?64txz5MN3Ola$L6Ifr zmirAb_+0k99KsIX-^Lo(Lh`CGyjp%~)^(x`*_lZQ%W$0J;rJP2Q&V(+4aRx7Du`b~ z*2tzo;y}r+A)a4SA{P;@JHCo%nBn?bpfzojuntyMaKO(4rX|e%$G;D=@4m-<1pG(n)lo)eUUsocM^q_GqR|hELs(3(!gcKtsNp~cKp&QtR@oyLbnxAc2F|nSJj-iJGnwKD zzfUKh$|8$ht_?>%X zOO(zS{qyzXXEAxOgfdKtF3Fm$ugp_UraVIsrKE{NyYJ16!xh}7L1LT0YcjlEU&4p< zqD(&(KKuMxSYAg*t2joR+*n`$Csr&Rnxmc)~n1QR2`p~o>CcA1Ip(81Say# zGVB448GDsdHpt8whF<Or+j*|1B!igun2N<3r&VEhOB8pJ$uI zS?Sz2Ddjq7-C4gy0-#ImZxU4i5mpM{bcpT5Z$Pd^Z}H1r!HxdaSuxlw?aWd8~8J zoQYipaT3Y3Jh+e6V-!EfZ>YQ0OO-`Uy-S>JopWTTwaKOB)hp~)C-3$B@7;m#Hco@g zOU@EHZ1}G{i4M5|BMn2Cfk#Q$3=ft`DA*-YV4tnz`C<+`$oo0ItOUVpW|_Ub~E zJsCjCrUSJ3@mHS`^bx4hu#({+T=Kk{+fZtt$>^-RI?#RFIv^Q=@5xzQrkt7?nhv{N zg+Spf^xuCUUK`UzaOPis`FR{gc|iHpy20Id-V0L?-VUFA@q`4Uhdn~~I5Y}7I~DCd z#sAc6o2(o@1>2(MO~I`@We@3x`#^Qt3A7YO_L&&u>2FQiV>1KaziC*wDPbqLM!P|e z2!Qpw#{da7$taUz!hluh_dwgAMQA5|FC{;ukK}38DMMj+_W2)4jG;#n#5E{=>f^1l z#l{8Ra~$8ch2K*W#V%AS5B^;EO$v>46`R8>_uGY<;S8@XEky+G1RETiV~AM{`U>p* zb9^Be66%zak7+0@jf_$rohAhYRKZ8_GJ@Ql@Wsc!hi6UM8N1~^A44Ou9Uh?wC)<~D zsFgD+m0Y8hm~sWx3E~Vp-nC6JK4RUg*v36LmIjB%?0Km#Y#8o^{c92es;G>L6q-Y1 zkrD*(QO*tj8siu~;{X{FDOaM5#$X5d0Ow z8j%!r!0PeeHzQ0k)XKep9vq-tD+%&V7Ue*PhA{!{MPw9oj11t2P0fTEvPjWAx+=JC z#b+&T8j^cMA{%`K9Fj}@d^_?CX8cOAAicaUO^U9NXo?KLT01X`#@4_>hjGkxhzbh0 zsE20R)NWZWlt>su!!>A3OEgU=a+#!t+zq;+aHm=}wmnOvE#!)1yNBW9KYt!H>QCP+ zf=Xa`_ugKWGk2CBq$4FW|g&dLYew6;@TRl_I21FG(@-pCO~5l-C) zL-if4sWLD%1ux18@&P+6ujoNCgpK0@%-F=@HOLvBgG~>yrJZ1}57~<^zkNd|>@`yr5C{dGhg8!%92A7hoD0|V&BIglqAJ^ zuE?%u$XURmEtXkC7;b}Pov}tu)?^RfqlnnR07iq%v59_G3oeVs!0|kTtdl~jrP9q@ z$0LHR4EsSR=}4z?in0`j^3&gZj9%GVc=*Hj!_1?Hp~^nlK}*Y`rK7^SaS&4^A0-Xb z(mJaD!15xGjYyhA_Y#nQCjr|Lg#V$*8781igePBo73cZ=^e5j7Klq^Qj(>QJ{&`FxDw8BUaKSl zPrmvxEQ09GQobyK8ubu(ofAZ*pf{|6X!X+saCZ(FAA%0$4V*@T)!^B8g>kOy+cz8G z`9eLsSfDwiw*rdESiIr{*Ut50G$s5eJ8@n(AiKoNT#Lc#I>GZkB<(2(@YAECv2mjp zxb9$2ZMho1S0#WXS%Khf+1@g~fkTDYkFXctykYtW=Sn`TauXwToVVaxwbNNTUO|1I z#O|-3{5~QFW25l(So3kt^)$%J_^rDnzMQ)&rXdi}zFb_T>pd`iM(plQJ5 zoZn4KNR^Sv_{OS#aY<|x%U zLo$hjRDV(l4;~+95D8K`va#D-Pbb=C?4IaDf@eZ7; zXV%0_4Mt^RvIp=10cm$^lctAyCV`(j(<=9X~;&gzl$L-sr;lm$)FD$&G$%Tf1 z7?^>Ub%Va%nJ7oSwl)Z*dwZMlEXz$g(bnKcdfAgEy-LpH^PDu+g6JFxpan|GJOlQ_ z5DgEXfA%{#t4N{Hh;czMbx4BJL*{je4rH3WGC)~z0nzTPTq_XzH(Pt5$ZYOrx)NR- zP{w=fCvemY*wkheogZX?`Xz~id(3iXabha`>+e3nPZ4p#=dwaofRDFJ^UD=B>k340 zcWap>P9!Zt96U!<5=d~5slH?j%u34&;m zZ_B54qe{crSz97tb?wMwu^IJRm3x6lOz_=5J{1wmjz{JGk96FAfYEI?j>>LS{>m?k z3{gVi;TetVcr5kF;Q~90Z!9tLM!!hEtP_(nzo5vtej#S91$!ZAzx>=-svc2sEcmfqU_X7HqB4EjJy(#XszHUZ#9EK@JWZ{E! z@%4Gj*@8Sin#hOk383uRkFU%Fs>hKDtE*(Nf~Jrn+URWJ=%R{snm9J ziR$tuEG1gvw39VX;$V|0A#(=L!w$7Pdt{I-rk`wdVSXt-U#~yS^Xp_!IE_Fl21__f zmT2@v+uW#@d)?4(Vn5g5%x=d%_KPz}2RX7#aUVix) zJ;S|Fx$`ho9=sb;W7FY~*R*);C5EvLqX>h9&gm-@;0;bpm|W+p@aX;w$O8MC|C1-0 zp{whAOfwiBgtK*p}7b3^hpan$0!6!PTq|?#3lT`9B9-f6$1sf z3j~CUB&#;Rx3dLeN;f;mP?54@0lG;p*9~g50_W)!-Kt*@ppCE{XmSBTXWK9V<~EL@ zGb}HyL>nq%CW_Zbw|fEIzGN3l?HE1FINyBn1;(682+o}-2VN@jBcrp(y4(rZ+8oN; z{ewgC9LYh`>DmG9REw)PLIlBJg6wwePaeK+s?Zxp28y;XExy6j9ECE-P(NCNj>&Us z2@xZ5C#6`9%oWRY8V=!1u>TnE0s)gqqJBUWgw%_D0CLUmZSdTWN%HK>>*VtWE0r*N z`wrJm(+WtlI(;UOI%1IoE*0*dC2LAha3O6Tk|trz7b`G|9-ZxijYY?=b0ydvx4#K#WdVp$Pwxwgnt zZIYm@qqI-Sy+pF}n_vHbBm$dEA))*WjiwVD&xAO*Saqrq@T5VHH}Ef6_>XqmS-sfC zp|wDr|1{5dxoPx7`%00bViFS3Z%Vdx%j)G)OqNzih%T^)j!%>?ZrRbfvLZHicV1$D z_fc8^xQYRQe$7r#yF}FzQ|C=8@VR{CtV@pbNR6MNi34DR)1B46qTPE1cR&Eje72-lV#5x{vXEqt8QB(6Bqu@jjo zO@{|N1illXa8o3B_&kz71HuCk(XapX2hK90+XQFceuaq%9R#~}Dr?`}m%gz9r>`k; zhMd1D;%K7M!F{lNVdeRYP$#BPm8xXZsf+u@h z>-c*bnm{b$?BFJRUVKfGrl&L|)EI={$rnHJf`o(RP?@>JL&DQGJJLC+)g?-1BX1wD zbVSd%;RMd0jm67w2n|~D1Qw1+Qf)pD?;^vdmh+)trDSb`C6OBiG)3IJsS`@8Ei!1M zZwcuBso>KG3@suTuwGo}f=jP|dH;1qi0M==0V zX@3`l?Etdu_I8bIbq^$fvJ|>-a?dct3HnDuG8NvGcP&4(JI8z{1Oy@p~y+%=>4NZMk!o z0pWD1V%&BpYbz@0CNOyNdCNI4xOKSkA%yuovS@jOeH;=cVTv5I6tBNQk#J_Lubr#J z4y-3A7}sZ++F(#JvcUz9ml7RtGJkGiU^J{{pmP!44h~)?YvD8kuOpINHHhpvOOmd! zrFlyxQ4fcm?xEJ?7KX>jY{`(|fgQ7Uc7Ev$OdKw0D#;SGIsWa^<`sB7DN0%WI9#(O z?@N^Ky}P$rQyV5I?Zc0dTq~+(303Ze>nW<6AOlp&W|!dSls(7`BR@#=uF3nmd}i=! z*PZaZ{5`MPB*CsI_) z8iAo0N+3OII^vrtQ9`W7A>;{&QgF;VDJgFgG+p6v$Htiq0je=RSVD&m85hL6Mao7A z99Nd-u~h`xIO%Ij#wK|D63!MkNqetoTpZT~qSnRup;UQ{=Ly;asyg^&4Q46!NT<^sxicPSici(YSdfY_@iA(j*hz zWc2b=I|%GbG(cnsx;nYHF6^OVxFW=&WIY6i7iYT!L#sG&*AoG1ssiW8CY33l@2oC@ zzOF%8fS-$vW|8R5haP~Wh+TjfsEGLXTyri!35jgj=TufW(=nN&bZ($^u)77vipDBB zHK(U5$Z>!=jSl1dyvHq^I1Y@U&wslolt1!ZTLx2Zsz4waLm_Oih>56<%Je34EjduE zt-(+aPS)f}2@;b-k)0|?p#sr5wG{g-j~z&Yn#ZnF{Mk12fg_MTeSsV$g=iJ@&7!^b zv%mi^+(#y4aft+kb8e}*ivVka`Ny{?nGPV2(;zXC1W#lYxzU}RE!7Xsunz`<_NiyE z_kFDWX>%WkyNt}tUqQ6TH+jYf@w8v2mfnQnrv>W;D-pyu)& zrTB2I87L=zn&08TPKNB*92%S(Tpu%zk@897Thz=npvYqD)m6o`gb~VtJAe&##>>nSQ_-xBPACE6LupVa$mhIf5MdZ_1R4xwsH`avJw81d-n@DiCZ=Yh zUEbTFk%Iu)p!^EoQeTaJyaH7F%bt8QH>7Q65yT_^WV2>uT zrB{@x&GpHII2OFAXbr{W;Z{EfX{)K z%Fj!Kqbwe18$qlKuZ)R4;|gCH8#=k?`W8(gBsuyzHB_5Sq%s?{dp1W$yq&on|9N)y zBLgluGR`~grL%mtZ=Xs9fAl`u+0zc7Ar3>G(LIq=5N z5hP%|14b}J06YF4=8y?@+y&Uoj_-o z11_47Aqz{ga9OUwMHFGNY$8`Hj#T0`y*O;Y?DUi8;f&5Y#;#8G?*o>c$hPBpDGTFd zR8DqGOapYpU`iPe%eg&la05rtJi9edxZazo0&IE$IfG6ENbgI+tp_z@mcaMj5pE;{ z-(K!dGh~OHO4L?lExm|<*<~rW%%+lEGT-VtI(1}M6`=D9(C_3uI7d9QMGo{KNJWl& z>Bn%j6Zjo$FcJwT^x`E1X^@Bye({Si`RJWchxELF(>Uf(I>xLWgDN_w@pKazh+zrY zFgQW3^~ihYx-2se;b29s;?Q9NHK#jsr0ifkb5z7lQP}|45+2G@3!0$e|7hcW3 ziW#p+nBmwNnUE)6WKSdED6Kp^0S&}4>%2qxL^LZ(FksN-U^sYNw5!D4yyTkX#_VEP z2^$H*NIY0GouycO>yZQv>bX|QKtm}D*F(|I3G2vNdB4w1&xpsv`X4$<2 z1s9+Xt(_fI$hTtvni{Tzq3LlNgX9H~7{$+k|46WFLy2e7Ze9`46dbN&gsypOBqO%D zUlKFAwx^L)a2mnd4kaEqEvq00dv#_3@1BI`ZzzpV&xH5hz7J5uIJVuLu=)z5nQN|X zA*MSqi#)?LzyjGmhs*Ip?U0X9-!8KpC>c&t8YG{``k< zNFa2)FE0yv#tA~wAeR|zQU(V#J9mrcjbY=Q7ML$;L(T(aIeO^m)cCI~FA{*Uo;V0O zR~q~TNSU0M4)$J(a-IRYO)%=W<;oe#q|1=b2MPAN!)vspjuHPoBEg#&n~KlJ#+}%i z2I6AJ+ZFncZG4>_*RhEr)s!?rvDs ztiBEb+_}W0XuOtW38e9wYgU)$fO5Z`n++0xCieH71Oqc3o%g-FcVbyqv1sSMx~Gb8 zD@Dpgds8_6ESis^>z22rrdSR;Lz+omSp)I)I*ALGh&>H;gU=Epi=?xQfX%8&a2LJ#-07*naR8I_oZsyoCZB#;7qdjbq$(dEC6nie(MPBc~@-EJ9 zAA4Tse?S8L^4rH@<;{!m!S_MI`Tn=aWHjp_Z)8~Q1P)n(v^L7g%ZpS7@CjWsqioX% zlpsME8y!amW(A){DG+_I$6Hh^vJ;_`v1~O0-qG<15+m#&_PrmwDqW~(sh-juw+P2; zm_++BtgfxagsF&plO<8PY5r$Gb+}8%D>f_gg|SNm70@4F!}Cn%XoBJFiCp>>J3={( zqFoxTmZ&E7?F?TvF*nCrC)vjaIG~cecN2sy$+CK>zcPydGfR4RtX8TPfdG;)qv50myDe7SxgNk@Zxzshn&5nh7(#V5}qJtj#~Z~Sm( z6^blSWRjO+ONZbJV;Hz^%fmMOh=6i4@m#3KdQ_(wjl{sj#Cz7a3Ra~`gCRrU*+ml9 z@#B2rBO^q|xwJv(lz-GkaAHuSGc-8LWB7(FNATu~$eIP|1Pn|RyEMa(ZMVEn7Xh0( za5@zsW}Splr9p#ab;FUzl@ zH=-eWMc-c(O9V_IMIX=+flt#wk08v_Qxv1*!OLlqPo~QG5k_T?z^Bf`-2fhc$mW_+ z$dT)}@gdKB^}}WB#`!ukv&3e~V{mk+^22<9n`ADV=wfYA_Ojdd0N7iJl`RhV8b)xB z%)W`?Qu^SCjLmK-GefVJ#}wDifWq=-f@qs>!|4Dm!XOObARHqktgAq3EZGr|Y!i{I zF=XKP;Ne^07ytavVf@~GjCw6Bd3ad|6m-H61nfaFwdL2Z!oU6cw*(9q;rsvLXW{){ z{HJh85c~P-MXsA}FgB%!#Er@B;Xtc{CVBOybZ$_z3qQ-U?hprM0-<8jPX`Z+4k=F} zs#F6pvFRnn`wbIhS&~j-Y~>r=9Qh|F~fKT#zZJmG} zy+F`V2Pao?7IH)YZa|U04Q72tqL7w+l(CV+;PvY{EDdp{adfO5$eM|Z!J4FDlarz) zQ}%|OpU5|agBtl7pk)rTzQXX^z#(N|kKiap$^M#W-VVR~{p0Y(t5@Nj4}TCo{NWG6lNXEdXwiQIaoD4j zRNvbUZ@>3W)Q*xkzPSKOM;4sHNq_P6Q+QH);e+>o#JXjWPGQ0YC{z(t=FART%7fp0 z`gyp73zKGkX@)&54r6P-*?pdj{?04`B8~+=9oDw$I0VWKIDrX7O`A{@6dg8r&l5ms zH%_rdd7@OsKf8e?HIAp}$Vbfd6L7tx91W7hds@ofts;H2ipcIV?fW1-qm|K^OxVHL zM;R@4I&x1X0+KVFz#%hdJxi|9$nY?dC{q~jW61F@C_|>WFPsuwt|$VFq{IQJ)$m9K z-p+XVj3$jscuv{-(dR_pl~K_ za5!fwNTz_uFz2s_z-)9VOBF$hAp$q{yu-|&e*KJzF^gegVG*gFU2KOU!Ntfqva_{T z+l`zq6BWzkX9PW!{u~kuYs}2upu9w!&^A0M-!L_Wpff>O0%w28^>5QKB7$u0IURl_rIt2nnL&^!WkWXl$8zA6#@s!|=S-L$W z8#Ttvtz2+_1NTCUX&e0&x-qeRwbcdaMkps>!-^Fs4EWJq&W8ih6<+?+Cm#do;8zLW z>;Szc=~aI$j;O+jZm+z=R)VJDYn0}4%+)n}S}u5@e}rJjih;uIHdaBZ;58 z55Fa`5m_Su=ULbUV?{{eUj0Fm_&}#ii;m zh9r^u+9!aw8E+roWB@0MWx}K%lD9fKJxOznNB93{`4@ax46$(0tP3FIS0{Xf`vc?|7S4lCDJ7*bK z{oBgaEs#Aa2{*jLZuV0<&r)E;=Ob^wiQw}&Gn}0+b;X&M8w$h+@+zGKUa0>2=au1) zEMr|7bn#0K4i27&O>vWIyU9I8nIC}t`GhX-C;O?%cnW`4eD-L>h*ok zdA+EHWmQ|<^FZin{_a$*YRoJzj61q^66l$M{d2sU4=q259~mA+vy@ zO+-BrF-m|^MdBg`h-fkz5nX)|BVwm#9RJ1P+{ZE7jeCL=10ZPFCpFq5yV*gLh;tem z7#aS+ap=@>LpU&HQQY7nDYkg)v?Wv4a9(mbJVc^f{RBQ)&;~*+I6bT%wegqVd=no3 zHON!4VgsjPFlf-3gD-bX<}RYD$g;?(4n@tZ zb~all9I71adWG}N6YQ7ZR~?faK@2r zM3Y?KDR6v>XFEAL{QoocUd@qSS-Rc^kT9755+>mj-WI9~Rb-Jx(_}X}ZB9>j&ve9` zBZZDace+r7{zc#EKj`)-LRUu`Nplq4J#5RSC(9QK1@CPlVKNim=y^7VCxo7wDgwyN z@7v$rYrX4TYrQL`S(LM}IJOxFP}(nrC7ebEfaP3Q)9(fy7euiubOfcj7gI5&hb+3* zU^F+iAQIX~U6vrcHmoA5DZ?O8xZ;CNZ^$+`n;NG!!fRR}HIoCVd+56;o)yH&xZ2%X z+k6UNxp)^JAeHl3_gX138*&A_sy*65$u_sIm6#@S0qA5&#OmwYba{N(i z4`~w&p!sU0RT^68a$=22Fv->f##AB3(g3EKI?OZ9>GafW*e2NMe(+XkAH7A|61I&! z(#hUPv#teda4Jb%mGK%ZcDUwDDuJT%eJ%O$i&G2Kx5E!Vd`M9^x>9s&?jm1MqF%3i za13M87XI+BpM~!p(_C<8BK+XTAK_f~aboDh<+j4qslZ>F`5a}S#rmGz0D zk#PC$(G2yjnb6VQPGNBq4j04A>kddkT)==9spV0dt$}rn-On&Dp#q1_TCJ@vMImTy zSRZwA4zWKX;E;`}W1n7=7V|9IXwYnM$iCI#7(}MH_Z;#+Lj%;4l(DafG)%YWVP|c& z8W#{s6p>BYo}Z#wgc>z$)dk?V89j~l!(Cug>Z3q~|Bha|vxF`ZvSOL?%HHes|pczC~^X}LHSq1_}8V?l5RjR5HyCBmf6LLf#g@X!h z+#5zavPCu27aixSK-^w{|sxqLD4}Wly=rb7cx94_UrBzlPqu_ zT`esUTy3oFvv09kht$3@=hVUSm&_f0eZ(H@rkUt~Nj(7hD|}-qM}tfoO;glUwq!d= z_jktz5vS=?lA?Cza0BdV|L{2L2g{31l--h9v`b%tCAT)sg3+)iW2x=M`O9jVa&Kmg zq`t^u_l}*^#tZoM$KJ2(kHP*a&)K4V*>>xGecCM3bLyo%V8s4u2sbU$H?wWXiq~yP%AVjbA>CYJ8I--)6!jIIy|^KT_a~+N&o(lvLR!m0ke> z)7aTd<^mZJan(<<+244$MBhBC_~kDP!2Ga!Zr)8$%R^Bbp^-l+3{E{r-Wn-WoffqF z5R7gMI3g)tHF{!tk&R&Su&vReY5&OfxLJ(Md?=1%GSaLB{2ATj#rYWzUJ%LmH37N=lH*$^jY(%9++A(%5JiH7)i#2~RH}yCL(^{t0bj)sK z3qFmMRVMe*sB?GBL)2083DNl*T^TXlu#9xx@*V;w*&9==ItsggtxFWNHn3sp*_6i^ z?tM}}$FN-;ICI50tB8EAct6E>&p9NuT!R4U6smi9W(sD3!d{$IQ-<{D&~T`w3-N^C z-v=}lB8VWxjbX4eRAXS2U_FAYk&_9Gyabl2*PA9bT4*L>G8J!qBc(C`;hUN_fw6KQ zDt5=c0+11St6k?*1^1k}f0;+?&#d{guFTF~Ih&0MAAz($Ge1*0vcH_5%vo z{X93AEjG97yoy|gDH>BBHYxBxW4c%)VQD}+K+NVCB`qqsw{41m?L;1b`=_z+dKciZ zLPwHdDvoHw7^oAZF_9tjtO)Qd8dhG!x-Fmc5zmO^0r$jHkM@;rb6w;_K)M_O#3HbYZ3sBYJmF@-!p~m+T<~|-k(6kTGc}i_%-!iX_^jBS?@F6 zyAr0Rf^>#s%G5T5yT(xC?Eqa1uV<)%B2`B1nvy6A>2i@c9MZIWj*}g(Wj+v*O>oa= zbYvC*4ipTqpV6g4{w4)@9vT`#Q%*6_JZxlBcsltijNiUP3iU9&g5BALO}IPpPGny! zXQ*e>uDqU(!>+jvWJgE=v?GZ@YR}r3W5C0!NmA_;jd%1l18UD;!k8P`W17H52AFWz z1M=-SJeU#AooseoLmFu+V1O=(RF`I-Vw>Ctq#WyODA=dK9^+|>Uf^LZPzCE$oAi8B z_;OHfBjstpt^{FlwMfLqNfRsDAXjmI!>*Em3fmEOVrD4WyKKSnKE3-xa} zLH51ZR9jAsB{gUb4KN_E*XOK*O=1c*_u$kk>6lFMJLhz}FV9VKeH4=C0T7jR(57K6 zlO^^x!5Oys6d-DogN&VJB5FY@U!W7~?(65Q5JUpjQWZyNRl)+$t#=C8PQpw%K|m*n zr46mEl+LsgEF+0gPaUG^PP=q#VX95xRufEH>t@-3w%#Tq9b$CUtdkveY*Vr$IBwF5 z*00&+XprMMDG~Jf_sw`Gg{Z4w;Hbl-6u_w&v6h!LI0UF$N@LyQ5nxyiu{{D!>$>86 zP3zt0LcVurAiVeH2mu9*6pZK|0m0<6C)nzW@b>-D(9_OjhTRPsYj!xt)N*4NWpNY@ zmeDiIki?#tF)5KHu^H{m^`2o;Lw+mVn;65YpHVwGg>=LWwcrHg=LJ&N)NyLRHu3-> zd`xRIOUQ^v(>OzrN+5Jrh1OmLb!)Ao;l(|FGVm9R)BfSUxq!{7ak zbA}B+<+V}pn*c~#rQJ2~jhQcl3+$&^44I=7_I2k#4qWx<@16*IG84B~+Mr@$KLEJ~j17o>+s-e_$7FYJd#j8KePJ>R<|N zyf^)%4P|EZ%xtI?C%fI)f|xSrzY8YJsVHr*Q`Y&);MP+@WT^w8NPO=C`_!pJ`R!$f z$q&T_tww-Q)F|ZIROu$3`w*PDe+9N|Y5=Xi@dO60n>~L6;ADGa zw=Tis^y6=LBDA7-;?D z#_CEGUrx7Xse@xZ@!5%2=%W!cV)O(oVkbZao9&RuUj}BA=*9MhBtN${=YR`o;-KG< z05@r;s0#-O5_DLMeu`0v(;IMrQPzW}=tGrhF+$K8rdS6B2>Eq@9)=~N{F@F)0a&aV z;(|0}oc|jGz>T|zK0w(*zppGsmS8DIJq?7fHZ^wQ>Rin+8lP zdLO2@xC;O&fEr_1#cN6^dgs}+JuN*J|n)}m%Y4-{Z((Dq4BT7-H<sMF=oZ?xWi%Z36$iMnaDDn)d5L&gMeS_5M25t6fK|h7hofOQL&JLoZuYr_kX7UTa-SCfZO0;MfXe}L z2Y~A|qsf4=p>?<)r*T9r)h262y~!$dNL+svn@%y?BJA9CeOVZKXB4Bph(YNLFXm|H zM-=oH>5faH(XAEILelY$1;PmyD6kiZWt#xu02{lfeO6}&1YsDZ>RR^OF)|Kx(E%B8 ze5`?!900zh^L-p3OhWfKwOSvcmVX%T=Q_E*H30CNuqC}Pe(aycT{M?g!o87RYM2Ij zUycj+{^HYbNC_mv*t@sG$lU?f4{%5-^WNLzVRmVrnmi^zG`6BW*G1C+69ZtQ&R`ca zlz0eYme*&(3-rsfx7WkHcRvbc$uneX=&Zy@w&LVZb|=x1V}GaGLQn5op{B7nM)KA{ zZab@y=hsn=Kr~FYqs1ynTeWjN$~QRoc5{b72nJUXbH$)@)SQ{VPj(K6Zh&14HCxj& zt06Ia5(dY6(LbcwhwocVZe8a&XW*E0(gi@Sg09^%1cR#qWeMINs2#f5CntMJWCmWs zaB((itSWJB=NR%+YSaeqyb%WPQ2wwmPYJ{;N>T_!U~wF`X3@w#%*@GFj{S=i2Mptb z2e)xZOE^#V3~cG*+7fmfKU%gL!-w3iY1 zGxHP2Rfeuqi4=b|>r###E$4iy`Beq@g*XcxfcSGpmw_a3#z$eWzn$OB0ATkCs%&~O zKw>SgZx|3olf7Qgi!(N5@U=UeG&ebNtfd`9&6zB~bXq$*gn0DBH^<1%aLxgs+NtH0 zWg5O-l6vn7_r{o(!Two({T+5Sz%X`Vdu%AVnYp~Uw7?p}!jbB;sqNLPsW3i1gdIWK z@8x5F!cn+$ZvuNz4j_DnZr28WfPKY$UgQlaC8-3g?Z!sAo@_h&HytKlEyuO5#ir!) zfHz7SE_s$!1PZ0Ibl5AS+~e-O$OJkowWAFkM)VEW*XLPN?w{+NdHhW{AZ>C0>-knk zA3hCY0f1!5gLTQ;R_n3!lQL(A1ow(lU$QauCAuzCgo;$_4`r$RT~pA0@z41GOr6?T zBA_#M=zp8`u7&itf;rzKRc&o;9CXjmFy1$TE{kbqA&T3g2}$b+wvau5FEjA0s)A&| zx3Pn`UTTn8ixYwq%pvb39ixwi<$~lfXf_CPN=&S0!)Hqvae@2+_`*b6r_U?#b8BH> z=ni&N?}!?m{(_(Qy9Uo<8#OSwVOTio;nGaK;M{cgcg8ibHrX|>Nys%3%v%drZru~F zaY9Hre@un?eqtG86v3)lzNnSQ&n)+HVuSZ$knWqu_v3Hxi`(R0P$uYro#VAJ)5ab+ zJ|la@&t20dQrx~yAnp3vkm+X%avkaAx~Pj=@jv}9|LHev_dle1csQ)G;jS{-&~@+K z(Dl{_VdSGN$ zI*k7ys=}EiplprAWD(3|3vj$qFUW%I3V@#p(-J9>tLBc-{{X!xQb*S8y!qpwhjpCU z;__;wAkF7TVL;Y1);@4}tx0lRmvc3DOyNQ8?vk#M23K&=E)b(RbN|*ds1zTy(fG`p zIyeo{MOZ`eJi%eLBTiL!>+#szfF2f(MH4W%xCs{Av@E;`+YD#VzfB|hcaopWsVQw> zr0>|AB;ZP>t`i_3Zo2@i9l;7XTo;>0gojE$2QEp9L%~r6{v42<<31F#1$VGPWJFB{ zxOO-Ie&4AT1=hn>dKK$?s83UNp%NidXty{#`~riGX9}HTBjLk;{KwEUaSu6)^YHBD zB+i3@&lFZBxRxq5)5h{*m|t89!*4wdKl*R~W4QmTzYi^UZ-qrtY2QvSux>P|5HaN0 zm`b)7H95M^ztJ&bCsEt7rZV|aFS9V&mo?{94Hz|4?};(1@P);!uU0+8#$`R|dw zxB#3OJvzEb3!QKmb(2IOO)wc4|RZfIZ|0IS-#;*}rYbspI{V?8Op$ z#JV7Z6^okHn{yv)bh?^bNtsb34k)=vFPDX3)a{i|sbf6NHcVIHfaVu+^Hq+`!jUd4 ztpOs~tE3-)`rrRLv;h2GzI;WLNx}c^k+8hF5hXZ`&T9xfsMUt40g3T!VtE1kY#%U| z<=RU!O{~|(axQd|0w}P)f{fkmd>9+1=7Ru9t;pTIV^L28WtWxV(KpXY^^$sJ-EZH& z50lRX8}}BUrBP?2MCq4Xqut@{H^yT?XD3|)wUSB)IeS#*MMr3Q%gF@R2RmYGnpD=> z(o|TSeN7sc`cQz>AxxqUVvzOkqyw}Jpu4z1P;yxxKKphC5o!gzu~B=lFxS+_ZBmNC zUff>KQ7?q}Fe$Hvd1~3H=R3j1?2&3bVJ~M$OSZPPhT-91z%pz$`^9NDCzP77^~tah z5CTFC(6E z4^pGHPmsd;^K4!|dPJf6ba?+ke|R`Svr5?s%pqO61Wyz99`Ib6;+fWGXbYW2)C@R3 z2P8b9DE+Ws53U~_z#jass?r=+n>$DI)L9FyX< z3$;q^JOWEooCo}_1@%hS$nMK{MFtN0egxa-evTza><2a49KBYD{Fb$I=X{`H!GOdQk!mJzbn>mAb>sRJ!HGo7ku*h=e(Y588%c|7Ipsw<$^YWQ4{Io z6Tqz#7@X8$bD%+(St`r1oV{MczqNH`BXD%8!SWobe7PtUBbLpJv7x#cj!Vfy0pFT+<~d_k)+ojm;rbqHV#@l(OoO7Pb4y6s4DC|(^ zOP$=>n(3rs7@D{h-g@{3X*NIvhxCHaQj4#i7u5qc=bkps8Ad>Y=PII?G+I8&k4gc%RjlHRB!qEVYp zmd;KsQ}->+bhrYFeO4Y!0h)DZf;nNb8e5e-YXP9?Z0jPewQB@eSPv5w)!9>~Y7e1? za)3JQT^PIF&sxJ`@;)9f)xejik?@Qev6^Oy!=3r}Az-!{cF9^wRkLlh+2vW=sXFoD zkHWpb`$?GIMBI6Dj_$wx&_6H?*kbLGcQD20bCR8uL9`a4$a`BbFO%b*v(r{bE6XS& z=`34kb&BZiqTYvLv+Gt-WINpsxjzV87*s>yx zKN`~|K;k)PFgtoHGOBsfc6L1y9l+q|9Er4v)D?tQZbD&gO9OWJ%*gr!S~UJ$IOjLs zz7H6z1~4wOwhOV&%W>(MrZ(&bj~j-tXJ{naOr72G;v(0{6d0bdDL!Qt*n$KFF#lY8 z5{f3%ax>>tEyH%4v-YHSY@4j z9>wAXolJEXJJ_nQMyJGayB+BIr1IsGwcRZhbByI=n46IduT-AIY2CHW%S2LiOO5<=C;CTfBijmxyRv0?~R19 zKI&`t*1|dsDjB9;^bZbTJ+KL!d4*JWchD#_kYi7qc3)eX3lFfHCDO?+9)HbxCt!L; zqrHDcty6Jh8RiWeL_pBO%6*BvH0oZ{OAz)d`>HJI}Psqro(aLVyFOEA>sq)j`< z(C$pOh4~_B!|q#Qb89y?j941d+Sv*?M~`ux^fb14WnmsUE999tt3UeXuOTW()AnME z*kk2nm)vWXl{g6|Mc|=GxD1W}*pXmYG{Y!`qJQ84$Ml^l1{s1=U7G1X54P-;44r#a zz;DBv9iW7uVfgMi3~3I#OTf)D5S;AMlp+asO82vSK=E>?+|)JFh{*NWJ=uWW*6x}X zuO|yL`{HS2jZMwUbeXbLbXowZ&$F&kSulU@ADJ&piA-gBPxXKtch5HH4NJ$^NxxNuhe(>>K+AIQ+4E<|WfrD!>qy8lA$(%i@{UGqnxJ z7X!5B4qgkk(Er)MX3G2K;rJYBRRj~Lt>@fe2_*NiOr2I~-^tV%LMs zA_%JkKveP2RKV65n0A&q(=e2CZw*9AGC9_mxhYBGHkqw zi91BSJPyER?iH=~)K>HpDRoc?z3|EJ!pq-%5}MfHy*T0F35wHy{SUOtPlwessk_WE34FZAwY%-i~IA^c`@|WyCZcs>m^NileZIx&^h)8Fe!LPd`K( z5A7uklVh<~CjlyH6uVE@fBfCho(yk&@D5B1s`$)1#bbo_J9Jew5#7S3ofFAB9#7U! zX4-kE7d+1*Z1nT*Ua;l}jdr(?W+06P+mv7pZ%>SbMY^mv79V3!7Fat|DqXOSZQ-_{ zYMV%z^tqAw2EoV`;%Wc>fB26$!Id!oYC3!|wG=w3-)L$}1Kzenaj``A=pv~-!kC)X zkZd2ourkW1xhp()_#xuj&nY%%uVLIOujvQ|7}Qk=Lg~h9s||17?Tp=&&mR4i6jLj~ zK^xAijdkx0siq+YzITMb{^d&;n}skiz`E~X97*AIlM=Oltt9v)_`=rWnDzk8=V3Yj zlsc%EFve@I%bLR|(=nEpUQv5QOFw6>p2C0G&W%HuoqAHWg99)tG#&8#Ue77DK{qX| z`v8(ZsFl(CCfLVZ2 zsrYbtChY9eJV0r{8Nt>XLD%No2EnZ5B{J5!7tYh}3N^wNVZ6U7+!-H)l^~hUweKv? zMJB{Lp2}**HPLi2`}`>k;Wn&YKs%7GQ_BpH$*7`p*kSS7gd*qv_>#Imt|dwBqv^y9 z*WQd~EBc8I8O1>VK(|G9$9POOp#g z#&nk5=g#!p%Ol}Lm#(#jyQ>ICzj}cUsbjY7UBCp5eXpJ&Dm@?CQ>+>1yCqXgpimuF z*K*WvA)?ItG}NalUqN>i?YAaCp))NPW;Z$e1o745kg^0s{~WLMlfoZ zsn=xi6+f51JqOl)SLi!=m8`Tt)_K5U9yT}E3A4xEj^#xB+uw1KWie}EqvLz|9ep;} zVamFKYc`YSC#dk?J=oD|ovtM}KC4R7%wE4AY!=2;v%<%&i=SsoTQ!1;yQ%bVc4uFP z3-p%K7?s<%hL|Bpkuus!Zk$?bK3a)f>d^vf0KwO@cxJVe{5#D9O0(nYOb1G5T1$mP ztY(vyaS>kiHO|%gHpeho04~^a(d^XQ?D$XE!0NI zEG147#*b^at0NYu^LhN$13zg?#Z$Zhg!X&NXpm(5=bEGkk08uBP zA%~XRZ~+?Re!XW7C!5a*6`O{0VME}=O`$p(#1)BrvNbBZzgaE+2YOl}T0#yO6z$i`fS9$&)w7KnyUxMnyoijx}~wIYmru#(mlmT-9P zlbD8QPo{|uWfYV!IK-i+iTW?|G<_^iQ{WGG|dsw6xx2BV^UL$>^QJmb+8l?pmm#BB7_N@)@dH|z*j_7QAcZTMc z2D%&(jvX8e3AzDKSeGh5>vcUfU|gH^e|aL$uJ+bgL#RAUiZwGGmt?K0Fp_qU*C;Fu zz_k-H!PE-}$6uc@|8afU-5YL=-^Gct4uFW(&X&kPI>n=v)RFQU^NTBCZiSY9ShpJ3 zlxxmaXJ;QkgmvLu8gvx^(FSUnW|%_?r#rG;aZThh_b_%zl=*8L0IPJAl>utn&~a-Z zK-+--G_=Jl&rJR6#=V1Lp$?yoy?(-gcy3aAHQ&`!ph0cF|2H!h-!vu74X1uSAl$t=PE=*ieO{w>NgF z$J-&*e2DB!2ewa5%VYM4^`jJQ^W5EQvfv6q570$Bh(25`BD{tD%{b-YV0R~UwjuR` z9jc*LumT&c_^pjI=cM7cR!L6+9?x(preZ9JKOqfc+DZABY5=*BV`KrzIG7GiH?^`Z zl$UV-r)8uF%NqcTZ_*jpPdX?=n!kl^Q99oNTCGfRXhPa$c6JKJYZ3bkoTNzvGZP&@ zh7o2FokRlXTG+89=cl8)=O&1T4Yrp0g7OO`8|=Khz}7V(onfX#nHRGV)@#{iEpecO za@?4mtyS!OZ6wYFwt+e*epfpub1$P=hn;EQGdoE8H)3}brnaVO69M5@JmFz=qxU20fvSaxE!Vi}n@Wzh*5rVXB?1;RFfq^$LGg*97>7fvcg+ z$@)15)<+>)?10^tW{xnaG1X~p*a<;XSN~94FJ(L&LNDUn+uC5CAzMPnsy1X9K<-9oBR(3PL zd&5v20K{9+!?OlXK(#}yx2a=m#B#j1bGQ$&^ZGTL8x=CX;5?rk5rFgTTl)rKve3P? zNrXMHOP%qnN8bQ+9SVT>J=fJqV^JNYU5-l=C<_AJt10%V3}q`pUKwXu@E7+dte+I} z%dfsv5?c18@q%5gW^k^>xV%#(k zxTj?QILJ66n=I_4v3|+1SJp+zcY=m>);`G0uqqB}Usn+ALj{vZJUA;Jv<1mX(UIm8cR@RyfTF#kJ-DUK`=dJJLF{(tkq{@ycJ}w8u zUEx@psPpLK9!x`#c#)tQ)+hQi{C&6umoM`wY33SX+{8C z=1D&q-&=dU1ikK zW!Nvx?AJnZHEbWFwbj8{#bBNitaNcb89>;hZ)U^l>?EI;3unwb9U2%8ZPtHceqYjaYVTCz5rQ^1xJoFDR+HDg@ z)X|Saw*d@6?RloOP}^seKXGT2YwE+{FEL>J7IlD&q%1c={{;*f_g6+xm!x@SeQlMr zF9}1^_6^j~IJWLO1|=cfyPpkjJRF6kJqt@q`S9}VPngD$LC|%Ov}HP$F{Bfex%760 zZytS1a|R#|JF~rk;{&+X^1G!;hX$v6$p!i_elbu9gs2P@VwRYNirXFlV^cC2fjg>h7Opzh?Z_v;+#rkdoBx=Gt zAJF8*Ia;23Mo0f*=%~dXT|sE1>89qMws#5$K(B=kI*u)YS)svW1902SCTgc)!%~Sk zbPo^CE5hi5H^S02K)mmE`109HKnj5={z}o}GVEA60mmsdPf3V^=@q7C(4=+_C_7qS zrwk^+{@DPa(}m2OP}zLDiLE=1a$NXt{jqG3{_)r}H*0^B9g|Z7NbvDU}6(ki- zD2FgZ60a}57imqw3c%Fthu3ILb;ZB=M1hXiA7wziF#cQO!sfqNQX&8%U=~d23j{@y zL1GIfXf6PN2J`Xy`MK+G8rMT6j`wA+Vazz#_)LNbKuW|cUY~!={20}n?iGOfJ@3W! zvmRUW#LxBfJuB87TB37-kFm7CF9QuV?F>jFxZ|~6Q=FyCxpUpCsdW{L$dJaM5m#(; zgJltMPCLKjwZ~uw^1#p1XL&9DzNJ@*q#%k1X};HwcCb1UKztaW4b*skMpqo)x7a#G zTF$x|(}30}nHEz&MulgkGDNIy1=%j2wCD1a?k0W(hDz0Zoiz)BvA@YrL)i(*=Z=?v}m_#bv?xxokUb_{x`Rg22^%d5ct~ejaMTR+C8cKP zmWNE>WZe+EI(Eo9I!dto6Up+?j zlKlW!EF+@2gWzb9&)HHlOX)96^MW;Z?amJBwr^)X+I*UCy~bU*AD0t`>$U$C}E(De`&F z=kBmy(m1AC0Q~`-wly$)77BAhqCwcl=$7av6^wVl<{l84uaO#e^CuD%_QC`x$B}8M zCk@LGc7hXi&2rSdD3}}dk<#a3Y-orMuyyKRkUJ2h10JqRP~MYtHF7=uy(8h0nvf&* z%gDnI!n3dc3{Y)E@&ks4{alTq6&Fg5e zv)zL+5--0Z$N|h&BTv#$W!>&N^-v?RY2=LgsYlFH?eFUf6R@DuQ?J6)=PxkyXW>Ub z`~iS{7deQ*aQp2a#X#hcbEyb=ewVf8I%j8}$KRNaX5at-KmbWZK~(LA<>}5+3rbPF z^+WTo0hrjYR_cImkF+47d>`Oc44-}ZN5CreK`_VCer6F95^1cz5iHLnA> zkHS2Hq*IeHS**(s-n|pnS7!+}vZ22L{Wy-56JK($f~KL%07Z6AR}rio(+p9QkcC3T zn$n~!DfsH^JkA0@z`496C>*0~;QsqR2JofA^gQemsXi0V%a-Tsz&wlrVL5-zNRt zOMpiAEUbDtnTi&sUDzdS3CPa+5@M;GWoo(Z-X0I1fAuYa3qswjO=q?TI}do9LBDKw zIW$#KGYNa8&Sb@9O?djv0zr}`9R$|aV<8n{V^K9_2z|Ydta7X$fj_3U6={mC%1pRO z_k}-y@idH%P2hI`K>`BU-Wsx9<)m+Oq|+6vo}QTt1sN|G@7rV9=)Y|Ir~t?w@xG@p z_EExxzyNa`!Hl3mR_Yig$&A!FYo5eb>a=jT@yoPjHNm-47W`eomS9DHEcmfJ=rV#f z0f(6{q;&|o9oJ`uO4dl=;_v#ora}dQGI=(lxb6{zVcWH-YS5XIwG>LQB}pQ)Brp-M z=>zyb{!NCC_f$5-pV@%OyGH>p4pz>L zz^#^gxx$XpJ~s5INeHtB9{B%*SQp06^>YMqX0*H>!I&%O?|Mx)px3OJ0W_I6_UyHc z5;tI3OdQ2UI^S%MDbaXi@f~C<;nhl=Av{sG`0HsYo6? zO2nKz$8tIMR8{B`K2LoWHz)xB=^>&7q74^Zw0m`Cu|Mz6#u7+%0R*g_klC@5OF=yW z9X1W;$A9zLY{lqotz(mxw#KiF8#J=8i{+*@8bbsq^mCL2(WyD0yKqhe0(W##Ljf$G zDMCJn5rl}w&=alGY;bVAL#@MJ1RJIoJ4sdc4G%%7qpL(Zi9HW^0MM}^r1OikzdP?# z4K)kCovm2MX}JQ(R8xOa;<+WLH90TQtiYO$_H|-7h$8r(>!?YP8C5di4Eor6PQ&n3 z0zhSkonH%sB$dg3<$kqS1+6tu_GE zn7V+8xnv(Glj4_qK@hqM6ValM7R+3UO<7O+&SB?=Gz;u=KTQBk0hb52l>&U(yZa5j zNM^u%5&f^x(Z!>RI(0s`Q=Em}f#m?A{cD{nK%CF7BE?XTLp$Pm*YONjatO$BJ}jbE z{rmv1tvI8#Tgj?IxI;>Eeer2HOQhov`3pMo2Ht)<6qe?~@gfJ00@QYlzfQ?Wu7UJ9 zmIS8&kXzggyF{;c%n@6#t~juDqA!?9*Q;-kXq%MXSsBBaT^9~;8d22*fY5ts9Uph zJRO5sj4RS4=Zi362z3HRyO?I6lf2;k)+Z4nURVkp>~AWC3Dl@xq_%WvlLGrz_I-Mg z#*g{%-B(|c&S3--;P{Z}u95diR1#pawUeB`5+y4u^Eo=E+ruWJuFC+}zTOtVQW;Zk zkj+3J?J&PY-C2$`$KI6Jg`a&e3NS)r6R^Fw@G?}bFUIL6RXB?AzVJ^~t z+~={e5oBI?ZrI3;4Wv(KOF!UMQ;lPo_0g_j3KH{dRC)c#(;M1 zi=DXJt7^rO`hf*gORO{7M~$$Zcjt66u2O@`>`2b?aS0WIep1?K^wk4^nvjp_%Q6U^ z?qV(sKXuN1IGdU#HJBZTo~!5CSWiHoI1TT<-ye!Qufq2JI`)JTmik_b?;nu*BQ;I- z!oa+i`>%q1HsyUxU|*naQLtkjW0v6R<NPcfLLac#c^sY%GfqL7>-X9iC-ed?J6?llTBt z4|vPS+M0rOG?&r$Nw&IILyf&5j`69Kxy!QfES`6fs=V{Q;FtvPq`U$>^|f& z4y`xB)&YFLMsZQfhB!q*ra_--mMyNQ?3!?{Z4sP zK8A?iE=t~mg*SB(Ya&RmDDdaCILFfKQ@Y`dpF7Om5lJqDO8M2?w2cJiqGj)u&Fgkz z32{2!zQ7aX%x7>>+z12RtwEtP)lE<&lHk)}EJ#^ckHU33u4|tfA+=*ofsUvqn6)fmuA`H}ZH&X3fQJ_;#CB8LtWGxddRde4jcJM$uHF304gQRtacIGss&8;k1bEkk?C!wSxWO zGb4Ti#G7VjxwvSh9;$IXc1zp7ua4m=hSqhobF_bGkjMx5ob|P^wX=pXqwwMg!N{i(>s6KxS@%vjzhU+i_VjPin3;H1v#6o55rP&et|wb7e4Sdqkg} zi`H(sJlXS>BwTXVE{(EbN}Jef8;=%31p!jaP+J%t>Ebc7E!n@BdOEP7f zXO>{hy2HpDG(1$ZFG)#lQ4eTWnBa$BCY6h10sz~f4NDQO8eyGstlt>`0w*;`x7h); zO=I^a!h@kP7_{-Qu(}k}h3Y0|V15mtjP!HVAX*1_PS^s_ED%tlATR+E<<#T!7pH4>Dv^r9y zc_%2)9Fu8-HRt>s0`#r5&8^L|?+7@t2X!za3W>I1XIiePnM=|zLC|}GjXI%rt~1-q zIV1In&A3Q3!PLT_CTaHJd|3Q#O}rg)*90W{n+4W@LFVkUKA2Pef?b7sG=AV}uq()A zPy8dKw&3e9tn$$ zb+iq!oi@+i9vlErmr?Ucs&029%CcO+CRP%VWc%*pEU8Tc&{{&Hy|>QMR9iLoenM)> z8cwGL9P-D;iX63gHcz$Cgy2kK{U6awNM9iwyuAWQ)M>jfZA9U3Y-pNhs%C7>zCkN% zQJpGBpqV-@Z1n*t>(U9fuCgiA_dN)&wg@!NbnK@72&}0ww*&rgd6(L`y09?y0)`VC zTXq~40kugO{b;GEtxU0}D=CrTNpe4=>Ga#!qu98BY197+9#sFgq2dH>#{|z^kRl-g z%HZFEvl}62|JzakIj)-&p|#mEQ2H63Ehr~;`a^K=p^L{*K_v@r$SUCsVNk*h?^h%)rClg!CLy+`lm!h*<6fo;srR-Aje@6 zp-QpxkrdH2Mdr&cQ}f5B7;oMb4@4cA;Oyo?d}!V<3*-Co+Sstb1qg<$jdO_vavo@) zBVBjgB;PeH$el+A7auB{(~V&}v_FfRfCDa^x{#&ebE{yZoKLDlwPRG!Hlsz((zhBN8M+#rYz>>xYen3L&zI41pRxFCXjwhm zf(4*S6tYh!U}m@Rz%k6EF;Dq@r6bgXvu(K!C(_OzgvT-dfCvt!yUV$wvXfExme*cU z#Cl~KkB(pU<`e_OZ2;Dc6dbc*^zP;|*RQS}D)-!9wB#(Xbxv(i;<2C=z-bY128zJ+ zrrL76HO3snz%?ARJ}UPYU^WjtoLsNN?VaPc2Q7aD7!q8pE>YXVde;Mj8mSW*9H17f zxs|RmK@8)R0C`2kl235v%^0x+_hb!%2SC;+2b(B6XPt^O49mIx0tVQKXqV@y%J{%7 zm;!2Eb^traY^*XiJ{uvd1Nxd_+m!pr07?{E+tr^-$%ybAk31E zIO6^c)Muy}X(B4W5xYfMzLmLrtg$Wl&ZTvVf-#}{*lW6Ak*zUw&p0c!v86sVlg=w} zFx5U%s^!JZd|03csI4s%?x3yK_uyVwnw}1`01D_J3f{S{>J?a?*)aU(+u@yG{Zshv z55HsF84V+_!ztFB7&kUKxDN&Rb|Ko)w?L;_nse6HF&L)ir^ERioq{kCaOZr zt4I|Rtq~Zhe{>DAPz58|*kSQD&;E=94+Cax#LChRdk1YWYJl2!PJ25iFel7xL_(sZ z(IR>u>>CXq-zS~<@>%Q-%`n`%6-s}T6j6SjbHMuyjo${$HN_NNk5jhj|aqYNVIT9Vi=)~PGQn4-$Cy0R7? zefw?L|HGePpB2CE3-^Wx2x=(EChB!e&kn#yCU10<=iS#v!Tx3_+b8nny;muEPV5rc z!qyJZP1%fWOK)E*UCb~;0IMpBh!aMgm7IBiZp8(`2Q6Uk;X^t>vz?@AVT~@Cw2&jP zsUdhc!(OB+8<9X+p#Jp`A=(knk0Xz$N5ltKqWedz&smjOLbw{61vSn2qKg=?xdg-E z7$J2fm9VI&CLp5-yQjB1Huv0TDuKet2b>qjPgU{Rwb=Y>f}0B1_Z0PJ&2^n&7vOjW zU_4JCuY=T29C-&gNb(>&h6dT%<~--t+S8k9KCoGU9mmu;VI)WQ^BD{eCV?*keuDWF z4KUS`0DuNZG$shH9Yo$ufS90Yzl{-VHDyk1u_hTCqQ3zoTykb}Jo~As*XTYn8<%?? zp**p@TSKkn%8N&=`zmQt#t-%nlNlI|4f}M$Rx@AIu}?CNPCa@4<)@*GfY&o{bqzD0 z?Ix4L_(}|gKtoaeIsml6wy51^NJ-+SqQB*t z39RCz1~gIyC^jbP-(~M_zur5mWb8++8fFnLj&?Womc`@^+5U*^Q z1PLD;!DM9K0ONwb8rVr!6>vNNimZel0M-X^JVnsHQmk22x_hpJ%B8f*A*ZwB+=w(`;)ow0-r-{0O2BHha{H_3Dz~opLRw6pH-TsINUjX>zV7PVw3tNDUb&Ah(M3TD{ z7UW=%ioPe;orJ#6_S4DQ(jI0=9Tj=KXx?kpdDn0cSNvWImLkCcyn>nH;$w%Zk%6fI zN5P1)&9mEZpxCO@5k%iUL;;0U8qRQ(cF|f~DQGQF1Lq(1Ndz3W7w0kN>YSMyKLR8i zkRo$Ey>B1zIfwKBhQ%>nyS$c#1N5P~4I8D1a18*WJDUwF)RrCd&}>$4@U&v8+Zf^v zn7AE^xNR{nu*uxiQEmn2A&5JpmT{MBIHjhfjA&2em?A=a^ZrQq0eW9Q|K*2aw6i-# z6Z;s|0>*chXnLD<-X&W0@Huh9ZY<}OW~g=B0JPO(+_QAo`DN5vAjMn3TDxaFmOg*y z*dFnI&Q|rcZul1138yf)0apNkk6+Efmerw8_b^=WY}P35PXXGh*#o9Y?0h^TWg6ov z_JOH)b_ef6$_jw@<>brIz~JsQk{t*9-U?B_b9rsLabE2JoujXomLtnoooOUp)EURL z5tPO#P^QJ_wNI+!1mN1l+811h!hUE3Og2zUbPWUa!;g{V-~@g8bSivQ@XM@_M`FRjBDNm$MHdj}p=H0-=-SE55zGe?&;JBZ1oUv&FjYB2ptc>@IW1%>E ziW={hP9mT=fh8#@>p=^2ewk~bap1B9kOrU&vehut;2zU7zSc0=FjT3)ipSJ;1)AaITfGz4S_ zlJ>C!qqpwH=T!pSUE+9;*%y|0R8UU{&w#v9Wf<*m3vb>Tq}!d@sIbpTCfQt^I?{|; z%k`GZa7Or2Q*dT>Av~oiXK`*W2J1g~`>k;QRu4@UtW!No0L#dEY%fqofQT=3ts}SJ zp;M0`@T7VVu$6WISI}u&LlZy~VAR^@`zk)if>u z)<>x?9eWU7(!_9zICBPjTftdwWM4h|%kK#=o5OJbK&YUTd4;;d3Vh-7R|s*l&&aM( zqJwl0wbOO%gK|=?M+C7hdYDj^3iqa|#A_BiebJ3GGsiY84dro262KJV~dfW54Q| zN>DTdh>g5yLV&0N#M%{WSF3@6*S(8PvAjxF&DqBdWV0-95>Pp5N$|P1wak51U}w5v z`sq3c7dQv?6#ukH{p-r~BtC^_P7tEBSQ|Qf3A*J4XaZI1wb>|K6MCcsiw*db4E4tn z7}4)?y(L)K^|^W01jdd%q*mT3LAM>Yal{$VuAF;m?;DQnwrN?{ZuapPv9rB^|ETB8 zogF(1l}rNI$_Y99`5cTo`zDD54Vj)QyHLsAQ>08@p9^DT9$K4}S)6IAR3m`yM3L}O3zoSPiI^=42fHEVFy+HR1C&D*5JUA}i>nav>^TycEn^YbM zl?R>!nC%p|csN7?7+QhjIUB}oSFJ^sJOpP;`emn(Oc&pZU5VzS*%$miwLKWvJat}| zIIi>g<+yo3W-&S}QaiIo3%7>63U+^)B=*+R(q|)gG;cH9L=Q!$uW_`EC#g^Lmg98tckT= z3bw{N3LIGuP^6HnPJMl26XY#QM_iVsjW+7eKK}S8G2-3W+=>9y^qz*p0&_v8Me<&M zY~tW~R#GcuU4}YI=_)`bk+!0(h;Q&eW`;g}6{a6OLxaz@Z9p%bf_KK>WwSF7m_xov zN+^yZ16XjH*?h>86fVOvG{S0NV>VV7NpGOdMV-$GK)~+U0)@z?RGj=-Fk0iUy5*F38U9KZ$ZrBQwJqx)>^hA{K#V;VB(rY#~mi^ICpmqnQR0ybwY3=kb1 z0Uix36m&erFjKm3KXSzq`cfA!Ogh!uEhIby0g%QMr%W!_ht;LXq(H&%5*z?Jy za7N*H0=bce^(j>R+ru!--^ks!!rJ_7SX`WAL>Lk-)P&s|8Kb*%6(F<~9?~Ir|NRGH z>g8k5OEJ9od^&vcFJFW^4@OC?j#2Y;E0o=;q`qt=yk10%7+O{tg=C5|s?nxa609G- zoO%SPt_VN=Z~i_Mke^uR8BM)-7B&HkUELInGvNHw-%=ab)5lqFf-TDOI)X^{)X+$O zn0R9lpj}`F>{R%_{`vQ@Q?#d#dO*NdJu(uO3n)8sM7`+5yGK*5e5#i3UtLTZbFHHq}c$QhWHqO>7_lR?YVTFovTV`J4JwVyOc zb-;LDz!VK-qQGw&Hme3kkgZCs9nuQBbCLB@@U{`VehG70j)vg+x^>LNLfD<1?Jd%r zjP$csxO*(ld)}iPJkL4i6cg|OKndb)92$H37Pg)`zCh}U_cZWNQUZWQhK=juJdlk! zLh`6U*DjL@aX&%mUecK9Ln#|WQOz(x`ZjfW`BP_HoXv~TFNhIJfRFCNddm$ zV--GVWn&v@kVeuf)EHvZcCl>}4|-wqm=C*w4Jg)xmka2~-F_padk4be&JJT0OQ9NI zQO$KZmFDrEevi+<&*7xkuvgm$_*c5QbClTBGxJhkczHo;jYEF`h{@t>&9LNg=G~1$ zJRj?`@e@2#!G-MH_skRD{x?E}^NLMQ;rMk&kuAgT>Z_w71io3ag3SRa!*B2`^tNWC z^p7#y1Yuw^K?>go2~5C+zaeO_Y0$K!PFP~Y5+$!C)5<=@XUibTb~(g8rdj>DuLWHC zd9q1Yq%duUkU0}12^RIW`o1gPo6C$CLiUZ-;Ck5MF#EL-8+@tBmBG1bpr|aPtD86EBjQ^7e&6Gj?@NIMwn|#*C;Z+`Z6Vjn z=a_x;-hEDv`ZBnw7OSwnu1gh-Ul*|IuAwRlN}Bk2k{eF=r~o8-zjc5yJAkddcXqJP zKzSfpIav%90VJ0c7}Z zBXn=x6}3Si(z@KRDN=91{Jvu){0tvv5{B$14SFMXXIHBfsYQiuhKRrgI$Vbv!F>OX zCK)R07^28Ha^$c0x1%ZQU>B`oL*bCHJ>UE!R4;EZu|-P;Y4^ox{WG9lsmWh|FG*emr4) zQXI@{n9oDJ%oU*V5QP5acTeLcmCZWl9!*r%LY-UedJV8?W42l~o8B=w&Idf8=)6D& zVIG>c7K5L{&{wjNM89@`R-@W(7i}Z|bq{k-6Aqy;PhwZ{M2j<{kjA>;)%4xcSd91ea_K9ZWqD(ithWdJNzx ztW*Jtsl5yBt0sEcIB=(=*Z%k~e~FPz4^FlcgX}@6=2=?&Z)>+TGXi{PSJoOW@l4AI zQ`H{ao|z4`9nHvYpzYNTn8C=NV_@e1Tkf4+K+-XFCEWmGYkd?LEoWV1kIPw)u1s4r zY<79&0Al9KtY>JXzD1*bxV#d!Sfi#E3b9G|hyqq;m*Y&#&OY?YMzV~fJEF#H8#bbV z;OgU0m)js^nI&>!An{f{EO9X7zUIueR3C@C?dWc?P8NFCa-WqrvI2m#0>f9!0WVMRe}h&HK- z+grk*kG;?8Yc?koP2wCQWj=ys!9O2OBCl`6XFlKZgKmG1eIJ<*E zu0ct@p*z%3++7KKeZX}t19~f~kd^3};8`LBRMC{e73|rGyjm;U~wt~2Wfy2Vs?L?z&h;_*30h?x%uI z*>W`Gthx1^>;vQ-Tz}|!83DuKgw+<5f$~nGa=x~ml37MJ{ z&*|34Km>C-pnb{{WXyH2IvnTx;@Q|&fJL(?I7*f6N4Wnp%2!M!@(VEkIBfz{dM;pQ zN&At(xF928N1sw9n~at^B5l0Sp7BfBq1M)>&`BTx?8h-Orf+=(Ta|~MIG}EhwP9`V z3=I*0TDk(ej8k+XOgSh+kYr6>J$1@_5_Ui)8_{Ukx(m*I1K&6EvjuS9T3!zA&GpEG zY-6_^83$m;wtxNP38_CoKY^BA&6}{z)d1)#6bpKKXg0x??=w|opKOrzmn|@9OY?h3 z%kaG;GB+LU-_HIpLsP?n>3inmIxfwb+VhVYc)qh5?mc)XY{9x-!9HD9(=CZTGEJDM zFUDf_bPLTvIVPOoGiuN|i>Whgt$@g>HF1U~|29o(nG+8m`%Q2KYvY%0xC&oo3NZc- z&&v8X{bQ_$#+UGo_p_eo$Hpw(d!AWTnj;_ zU^mW&=UVh#rY-$>Tp!nnd-p?KUkQm5_LR&ZuZrYMit~PgHl7nO)XT$jfmRvk}~hZ~X5@QtF5|H(^ZG-$AgI=r2hl zK?3%pmcv}h0@`gR+SUMEv#Hc7s)cfrWwY)=fW#BVIH`F8TfqZ#;RI)Pc9`eB#^9MQ zy5hAlKkdTeV2l9VC}5RX-&M|nkmUfB;TZWH0hHQoP28Xhct9j}fI+h+!pOr2!Tk0i z*J<6Hpv}i2&=Xun?H3l@hv}mFY0*$Xb}ntxZT??&qK zglkp`MS6ek0$$5+$I&$LKIt7MV$!u&{xw9<44Oh_fW$%9k3~# zCUJn_Tiu+-=(47OS=TDgfQ3n|!+BdrSK@)MVjNE?!u4?2P(hXg&_Lu)Bm*(QCL%Rn z-h>k(0=G_#d$+!@0BcwvGR$+^<-GqHdtr;ncxY@iY~u_%h_w3qJD_(NV+BYm9>D%^ z4LX8zBDR+9F7^-XE1zq3C6f9$DefPK#%74RZ#x{Op5A_nmj_5&7Q@Wa3dWql+Ze@c z15p#qPB{l(36x44Kk+Xhl@vFd#paGU%ot`M#q&ENg>eXQO(4I4B@9XGo{ozER)B#` z44bU&6^Eb!2Xe*dJHiC`9`|?!8gI^VO#&P|;Pnj%F>?<)+}jHGWvYjkQ=A=XT^x^M zsvHQUM6yxQ{@Hg(b8zns0N)h#P#x4D&BKtKaJDu`kGHq8Rxovwi`#TcvW@_f=g{z5 zt2?0!U^4axTsxU$?zWgf^5nYT|`&me|U}Ts|vYY38c8!8#NynWh!r;hg=w$uA z``ag>5`C}e`c4IeU z12w2U>=EsdI+X%73+mf?2BZAX;Mh)h^>i|hE!)en*U%PhCP3%{^bZ5DlZ~~MO^}A+ zJoXKY@&2&Sr1}*IeevX5&fE^&_h<}uwQ~(j2dSpSBZCd&oHwDbX~RJ*dghg}j0JmO zNBc2==LB$qWrmf9O~58zlLD&7cEp*KGQ;elSRR7iOcBIgvkx**{s{_t)fG%pf28g8 zcP^;6)VazS7+}e^bEkZ^dkmQ$fO{D>(@w#4*spCE{w+oTno_)~BtXN<$41Nd4?-Ts>#`1;>8WDDJg5F;e5LuaO=4_K11(+i=&n*Hwo{%7oRIczE* zmcJb%X~T&i=n6*OjKdL~{5eVn)O}M+ZfPzLW4w>%8;t(waH#7Z4GX06bIiR<(oj;_ z&_SnsYk2$iAAL7i_bK;Z3tar~Y@ z#eLxaJ7w+6Iypv5R!P9jg~Xrv?A_ZkU^Z3Q#G*D@WS00IPXeq#$1Po;SX)r({8hDkrDJ-tXK0r z;v881HW-S)gAX^=th7zMf9^S^Nd>{&mZK6QS>@;B`~33rTs!OQ1%a}Mu9aCujuO|x z)G2(m;JQHOAa=pBE*JQ!o00<*39xt5kH3j!OE%P8n*MeOq;Tc&u6jH_(pJI;A_Sj7?>yp33Z}rZ-uYS{Q~hHh*MOyigvA(ycQN^yl&Ia6QpK`d1#;CXoR=&kk36z*s{jAz|Mb6-c0=2UNWu+Y z!-lx#-bRMUsZBZ#PhW3_b|U>e44f^%qHGu2ZnRc%>;lnU(u<}ME$Hu|uy`BCJ|vya zbvj048{qNn7oW!T>^A8RHTg0zRl>Ccf|6|hhD=N7?dcC~TpJ4><`x%YN25mN|0C)> zn;T8DG`$BTK#&AL@4d-n`pnA8@@=Yn{Ot7fs5PP4h}{Y8e%KEi(ii;>{QxNx`l8it z%qr9~v#V%Lt9@5*`KqkSsx(Y`Pl6uwpaY@nde9}YIy;#p@Zxk&*(KEFy0FY9(S(X_fJ{Q2MfOPpv!n3;SL zCO-d!OkUO*Yh&Bc6^`*bnOdGoPcR-^ipZHev%pfU$vPH&5%#z`MJO2uk|f0M25#YW zT*M<(Uy)S_kXQ$lMiDNn!Y{oMdWqyS_)Z_*{81|U>V2Hq}>y+>hz3JAZSP0fWj z?pzDQRLh^DPTLySXr-s*?2BnaXT4!;WGL44J7n#3kep$r+}LP$=p=+}tFtdA$p~FM z2oG=Gqp)u(ygl?N5$FR=4ZBRC&4c70K;ZB6*w3EVF9I}@7@@ru?zZ~v9 zqCaUhh~y7SU~B@+&bKLLF+^o?bWw+lO#^^?bAA)xT?J4;^F(4n zrU%fogxt%_ytYR0sFmUx5JGGcS02%6x6=F^6hugV$q=m(DdDZ{%-(3)R8Q#gO{}OHu z^-w~gH5|=t0G@I3hh!Qb!unF$q+uDI4mz}vdt{Iv;ry1SNTRU$H9#GQ)Lp} zy%z^F9bIP!8c^1$r7IdZ1oaGEBV2S9Ru@)iFh7C5Z8MBNS)+OR7onS$T(7KUghSg->^ydJnu0JSB`WT+uB%iq`a^z^YmaK2Ua?j_1nMa0BRQf;VC%P^$C zPY-Net0HW_mRaVai8_Pl*c(0((00Tf8{=ocktOiUU=SBHIlxtEEJx z2|AWF%szdOTBmD_HDp9hBcN#nU0g@_@VEa3Aaa7u8wx9E;rKA>8jj)o)8QZf=l`9E zMVVsYUHF_7tnxNxO9gI@~Vo%gO>_)EzQRdDbidy!ZM|3VqWq z4g#Zla44kP`e1_E!qbm_5tg4l1AGpJEL|8*8Sg|6W}8+$%d0OKlQtp?*hk3Yr2VApAZTqGFu1B6==+5HUCK*_c zqliW{OSs=J0Sm5~&k#`&0%Yx)>B>fk;6|;M442ZCS|mtU*qAbV+&7hp!pUyo%9(rC z8J{Pc#vOCOChmr>R4XnccT5z@Bpkp19||tw^<^@9-M=wBiNh99r6DlvyR2kE@Fj4I z&nfWZbLv?)+xa|6D*$yPg?#=Al7vR<5PEokSm{7B<~3dSH5?Mu*egkq*|S#af$Na5 z6a30jCIHHI$gu>PTHFeL5ikO#J@jKQ{EFZcpIgK)lt^ZAdg zMh^k2(?-?r9H40jt&s-L+BAzR%OQifx`P2$gJDKy1Mg4ZOfQ)Tfs}>~3&HoIZs<|c z#w~UW@yFoO1_UPToKuirhDrr1hEC7P&@|K}fZ`G3@2hFm!fIi`gUE3_CL84VBTB6@ zCXFMr4UUf5Y$w5*sMk^)Iv%rQy(`hK_f*S$&&@A5OMJMj94GH81eyF>y42Kv5%aMN zZg-U|qH}8ZaDPXH>j;5n9b6WlJHk3xi>pg<0je!C1Yw3v1|xe1BeJ@&1>=IVhP7zK z>8B~>z^6u*FTv++GZ*RwoH&cwE^u1gAv8CDakf9CnM?QYdBYMTU-wB{q9$6R=PiMI=*$|Rx~Pm zI2H>uoBDjl{I}3Zy*|^5(;#IY76j7+3(+24>FNc{op2i81UN2)zx$8>D}vpQ@Bl5; zEketNrB69%+#u9wNN(-ri!e*r?~0ISrnZyXZ2(t9q&c#|YRF`>Jc;We&APQVlHpGe zDJjzM*U)~M*kB^X`6+4aY(i=8r+~0Vgzc8-7#`?kj1QHT;$2D`Y?D}! zCXCjGCNhP=)KHHQo@(4B5n>DB?dmu+|7OE$ufB%EA4TMw4O5Kuhm_nHgk9=J%k)#7!~Ns@#FWYeMM)Ap<$}wV@EP=;exQ<$SbcBPQ_q0_J^T2|1gX{{uT36 z8wS7iMmT4Wo?)0TG2~rm)Wl+(3zTHpT_l00Wiq_=U;nopv^dnUulDE!Vei8%YtWiy z1=!GDisQci$A5}AUJX+N?D7`Hc?rSbXiiy2b}M-D=?weeB3!4J>z};!R?IG({c?gN zg~>2_b2yCLA0gCkYcp7?EXBbO=?jZsxSsK~)QhDi5(>L1V^M^OYsZmalCU&HcCu^! zYzMYE)Q3O#vxi~q?r`|=fA~8Xnw=yRVnI0<^PsEJb2t?e~B4GZGoPn3EpXUKjK%b+3?a zoMT^Gh!|uUZ=K7Nub(nrShF({RpxPS9C*_i`tTB7w@+vZ<~YaczX=U%37aE3(@l*< z53}l@tcz(gbTb>vfZ5f>d2C4~`6z3dBbtVgf$Tj1*3=I}}j%^AK+i>1~w8c}z+h{V?JQe{& z_MkOGyT7-e!&+Tj)8`~1oE}gZoXYdXBH`&w2bsLB_!)FffRn0Ix<<@Dqqf>cwD0Of zJFu6AF@E*;KLaqPS+`BdE@)E)TYCTDD;!2H!cYIhe@2&j8Ag}~%XIMcu-mQS+5iW% zxwuA$kDeIXJ#xOdU)f4o&LXxw&DxZ8+aYV#q%M6Z zs=Nr4Vx2H7u*{ZD>WqV*ER^8sl6e&%fC-oz{yzdV&PqsJlCYF|SQ#oaGR>BBE&6}z z!R!d2RKgn;bf-@OwKgW<~%v1$Yc@1S$6$Hau_#Ee>GInb5 ziufFH36PCzTRsGS^Y<_0BN;Q_uhv*Cq2b*sY?gb+aZyhgb$7hOaJO@J#(N~VYOvcE z9C)+779Z@m7^+UTqq#7fMB`-3B}*nW!JZ`qSe&M6n1E*9a1Ic;-~=}LmC5C>;+UQ>~qt z0|s{!QSkKA5}V&ez=CA3?WgG?$l+|$EE-QJjp*F$+gcAWHF*O&N7$ItC-w?uG-pfmGx3zLPbSYa21rmQrCZNuTc^)jQP$obne7#Q7P= zJ{(*3Ku_qd1{|R&vGalqYZoj`iuKmtNjg6VfaRA@(8{oZ0Fr&3x6vih%4%^X+yF3S zbacZbBvZ7|lW>u~mHRMsB;(+O>tKwVXnfralTy#7u<5bw%*tfd^$?ypgLOI}JnK3X z+3iH|JfEGxCR!4Fg$;E!UCrWeeIC2lwSlu;$$1!IDrGAtdKs~Sr5ZA zK<&3u$gfVK*UJ*}Z4D3J#JL#C#h7j~zGm^`95!-keJ$L2?bVQiX{>8)4TE6Yo(cdPLisSP z?SKsrbOjvi&iVlvqV04Yu?#~9J5vi`ZDuiKI;~kY5;}${X+o&c#3WmlS?910;A6kS zDo*K`CH61i3N4;2#Nf3a9CF4KkYImh2>UhRWP8vwYL8k6IWWOkTksk(n02)h zo^i111laB}-ZqKn#{k$44k0)l#?p=o_a8injyB1x__eNuN9S?k!jM{m3iP`n7g`@eohabcnjLtSJaGKPTO z3(@(B#I;AX+XS)N z_FBTtK9}K6%cBVD>~vzXjO+N2Y|%7(Mt^LEt*=#|$6fOW(8p$o;Zw#b0a}dOH^$RK zyN=yAXy$eOsI}{_S-Vu#gczGo^xK=Qn~w z?)?!tG`DmQ)XC{ZOR4|eG@Lk(O2Q1QTUV9N(>q_WXmiU zRuO9X1i<5ssD~>-$68oeq;2N}HC?SW089W9&(8^!_%`dO%mhTSAJ-7Xxq>VJYky3O zwB|5qo&qehVRK!t)k?Ae7&;0Q)sNYi(ryeQ?P_XjHFK&{&~Jt#rAY0-iiG#Ifp3TS2ZZn(L#aR!!cJaCa_>eeR$}=NC#eik`kQzQ4 zu9q+a&(5{2^>cAm$u4t(WG9mt>_{>59`COX5vv)gWyVYj+;E2UtvWe3xox;k_&ZK8 zBjbQI!g_hu*?aOh?6Z(%WL|JRnTio~N-0=POBcwVT40&=Nn0Fln~=gTqHh;dBS5Z( z1!N*hKbI{MTH3`pZjoiRgTZO(B%_qj;=?zu<7^uUztRC=n$Xr1qG5#lYLpCOx8t-9 z)@V(&LpYV6opP_aCTb?}fp!vTY9@u8P-m(R0LU}x88%pRlb?Km7OXSe9_}UFLEl3d zyuu2f%W2+_U9P1!WO`qV_gza^WoP2Eu!o^HW!|<-%{Zq5Ou-q(O@U|=8kiHp!k6eN z4I?yj%1xJYv3A`d<0d>fN z@8$H1aq)pH6=S)TR@j3q&aMlfq)cFSah>c^#PHnb1&JJ+fFMXCLJBl#$C+&ts#;j5 zN`50_+XoxgEdx!qUG>t%~AnuhqvqW~r^U1K>Ucgfo0-GpV6QZ>7XO zYy=uW^-4{H6lI0Mtij>57xzLNMVj-pXsd?Bt;V@6fBN&VGx;f@Zd%8o=V>3h0h`r9 z81ycC69M}AENtj1%wAV$rjtk&fViIdK4M>P!al9NT!y{JL7*E=Hn+yQSRGvJgs}VW z%2IfW5WU2qtqOfx6YHk6<2EHswj(H=4~eZ@w~*mU*` z1Fo&RMS{oHGO~>MSfZke19tb|C_u6?EKXNy5O#I~RH^G$q$c4h$uQd3;oG;vvoB`C zB+-W)%y#FsF2eNe7bO>fIJJa1dcv+R%+qt2mV?+(S-~5F!+>?NywU5fQrEAVWRR|r zk#Np9EYqc=x|#W;jokK1fl@civCL4BmU4IA{j-oAdw`du_8fpAzq}SDH;pZA9IGh8 z&Uo)_Tg%bt5rYBf5FEwYVysJhHFMkCJE3Zxk-s9?(5Lzf0gQG{pc7xm90;lmeVU}_ z57lDImdWl`z?*EA?2urMSK!l_V!p31assKCEWz&t8khKL8NQ=!3d`biuQ(j|<+N4^Vx(P;QryoT{!qWnnd$%gGRg=l=Xy9+@Q63)v&gbJ z&fdTKP!OtLH2c)~^z~LFV8ik=4q=R?jGY+6u}$s)UiNqZTiVoM;0$jSm}t|(@%k9h z>hR?<^?`6|h7e;Z2x#MGPBm3j#}>F00c)nt+Ooy5Q#30tUMmt{p)?WxB1sMdS`+`? z#i^vYfi_NN-Tj@hu8dh)4uJ7(vRwp0JEz9-6zJ4Qb|Xfx2r>IHr-BWH<*FZ;?eV zI(H(ZR-3R?>aZ>_2crw6H;qd0U^(@TJ*1zNelE=1dToJ#~Ri zIB^^0$_#Zd2~=n%QGx=l06jCj8Yq=ehb~RSJjy&A!Vpvuk}7SXEd*qAQfnu{Dbq#i zA`%>$&FZtAapIg}4H&wX>9kc=H%_w`mZXuexGlyG3EeK_pHi(Kh6)C)zBL!_+-wOC zZgfx?eU;vnvoVZP4IsSao;v8cc%3k?T8Bvv5bJ=RwFOR2=r-=ZdOJM2eJ_T6c43$5 zNo067fmof|REKB_WOJL7WEa7-Ugqb>n4|_7#c7Y8a$4cw(bItwrs;J<4gojyr_QP0 zw9DyOCd~C?sg+a%x-hPL0B~(53~6sj%NdklD_sS1D<{#o9+3DjIM5wld-De1hgj?xqK&S+S+nm+3{<$i$OGGlOS6Ocrb%l%hwGKYj!0hqRH za=pz{@@59-)rKIPogD_o2C1Qh6XLyT!Y)iPycXF7)tijQfnbheX&kig4)myxajD|(NX}ofg}yn>z64aynu5u zl+egJxPE6KW=5`}vuogU*I*WFaZZCn-C=#>n0j1uFmbz-(V*PN@E!Dk&tut){`(J* zR;=?`F9_9bg@!X^6)PJdzdjMBzy64xsjr8d_udZGt-YbZL8RyUE3hSplvjZH%+Y~j zdl_aV9bO^pwLm+tGcvbjJ+})B=m(#Nhp#`vv8`hy6Jd)6?y^9`Eon4+=P->lF28dp z)|Y$m=3B@Dw!+NBL@05}&H;L@Mt_Qpv36S>_F*63;Nh{0*v=^$zFnA6Xx6a)oYEyE zSG@Y>E9jUASs6lOFIlYi7{<{eA3lBjEG#X|h0)=mShA!{M+B@6 z`yC+H-x^Qs=>u;tj?YFG`yugPWF zA!P~LuVxNG?QOmM%%a=dFm=>TBNIEzeoAm)-=!RnodZf_{kmAq!kvZrE0B4aU&YQD zYQygBq7~9f-WnSTH?NJ65C9{Gy^I|icB%2lp2?7iaEK<(gLM*Teg^2Xzr&JK*o}`(KBr z^b8&*g45aw>)J>pqoawEHnlK{0Aa>;`pHz-1q9XzNJ(fI>FEzop3DM9=&{XSzT!~z z@U_=Np6)?gRPA+*x6mgU*o;IBd$PQ0CQkVFpZ#^XL`!^y)Z~PDnjrDX(o}|%wY$66 zWwl~4D#qC0-gzE!HFyUMaP=j4FqqK-`hyamaNE{fI}*Sp7kbaC1CZMj;~QP z9y0pjB7}j7E57f7(69`exI`_WHRG(E%S8EdY&*q^!7B-{Nr`9nnvo^tbNw=_mf?tV zRH?V?*m_^GZ-!L$`G#waw?OoODXwUnS!p4r4z&XMyjJqVFVWR-wb=X9htoiD-&x)S z#v_EtO*6bEc2O#9ghS@x@Gk{3hup*~&9KClhfN}mssy0|}r68SQs6DbVYH)l5 z7@UsY?r@G1TiMwP3xqu~O)xJ#ecWJ)B8Iy$Z0fbl=4=8OHFM38kaLu~GczuS2=tew5@M@O<>%q7gN|aVPW`Y~o5G^d86HW|A2xe`tYpV~L$!4x$ zYb|-kxedmPN!yl4a4oVWXpD#lOs<1D-=z9J~I^;q4(d7 zcC3q2)zv1gXLdJ8iKlD>td|T_J6f(kfA=Ab5BjxjG9jtgcgiMtzKAxA&+7y1-5_M) zdYFf8`10}R;e(Gp0pOCf!Il5n9x!?U5=r`UUUFaU z>>#t84_PloK;5r@{sDj}7jDsGHcN>16a$?;#HhkpZ8Lu_pHNa_poJ83v_j1J3HMH$#2`c8an$H(o_lO%1PMSitLO5!cc0%<#L#g*?43DFA$PIHdX99PCX2C%dqi zr(W6Z(EnN|0CqFHc>I2Na`91k{q6f0LB?*Q5VGv!M>xg&i`lR}IR_|O$B=PCC9t~w zj0W8zZZ;D+TB|3U9aH z3sVbgXb}tHChW|u>))Ynm(AEqVeQKo)DByvxHd&om)B#hzAU2fo>#sV*2ljLtNhOW z)xtsN)UfLQ4y+~RDL6nJGWM6uBgOwi0oJq$wz1}-5I$f0B~CmCGnWp1*Zc9ktuRa$ z*5&|!VBWff=h{9Gt2~_!v=bI{6K(0%4to^GU>iD<2#{83N(|vpc$zy z(d{GE!d@rZK(^`7@E|sZ8iX7GkFeL62o5Wh4?z=X({Q`e6kvUe00-N@8KtQugnfw) z)dHs0u`4sKw!l*ZD9c}Dyt~j7HgRufh`lYEedo@fvX1dRIC_0%iZM2GFN0I9hs`q4 zV|$C5iiA`fDXJ>_re3H9kX%Jck1|E4k2YpPer|#YM@{G%9wX7@F6($ReDdK(VE|_J zHjLNm;tOtOE!2>W&$bHf-GkxTr;}l8h6?y(jrWXlfaUKFSpF);U>mz|@5Z%o0aN{a z`2~QB&n1Dt!q3mf=dhKK9~?Z-ImkVD=e1DKrmF$v;<@%gEG+dusq6N9W-hecdKfxy z+z&-;Dz&@9%+r^;bz~sJ*jHIc03ABl61{u%RUQ}|kCdznVH@o6z6~G9H9r zw}f%1Cd?}BXdg_A z;(C&*@xA9g?h&wgq2HN52H#*r!f`+xi?0AZ6FJth_q-eE!`%Y}i;!c0UFCy%e; zIm5BLfb2(C4G083VN}SG|#z_LmE0Lv3jQ8HdN(5m?pvTYe4*|s#B=M=E9g;-iq zp>cFebjbQ$;{4&#*m!(8teLQh+p%&5gX-2~VySBHHOy|zv0$6gLfDVBQdyazw=9?* zCPKhuA>u5|SdQ;onOu9})$m#T7Qn|m0B&z|p#=#o5kN8TQI}4b4!y;F%;7S=<6Z`COCa`e>6-|@!t?iGofIMHr7>Rz^8Ng7?-)VeCVc4)CJ=(H$#by?gnbK4N~yA z0u-&$x4A)x)UY5Y_YM*(j(CeaPvWwYOWaY{)oTKn^t|WwgN7_4QTS zM9oeu5VmA34%0W1(h5_bf5O(ZiTUl=x7FtDePqlg0k#_)wgvz&dF((fYn z3nQqLxOL|SMX^W1EG+EQ#Mc;Um{f{Kn{l^`ozAs(a4kL`W^0&COg3ihhiW!mnc8GI zLVwp`#*UPaU}KNC#}h)TGI8qCE;z-RX{X+*XXFMN8rBIQKSkz!n&clD9vyuZb5$w7 zfahV;TgK#s@Ym+fW@OFP8f!00d$NKjau zC2?bKi|3#V1#2lqQQ?Wn`LH9PFkq#ozWz1RAj9G$wgHBl2)#akJPu=wyo3FE zLC)D}V+9l)*txOomW(MxoRdHKonkVcbR&*9rLHY@8SP!S?G5o?P-H}2nw2l;dS z)8g7*=or0C)Br!nfoPH%k&Q#w=@EQ~(n0mSzk_5Sq6S>Q8(m*LHjR=D5ipx9V%sDE zpdmj2kOwqbt78u70v^}V0a6gxia5aP1@?E9MLqX0!gvbAV(1OK?%ze$mYonRUYkaq zVK~rvuEs~kk|FqKzXT>CjY?Qgk0SfTp@r-DGehcvB4t1kz;LgI^c3D((@!uWQu3XC z$xKgEwT^F7x>?|T=^*IeE`)@f!{uJ|mCLz2oqFzt7A)#OLNc zj-%|D#EZ#7zMdEJnH9h$C>9?Hs&gE=a>zdR&+7AruvUq(bxxI5!i6t z5TwZ)%pnF41dn5ML8y^y#)vhm*WveOOIj4q$yA{4-=b=pY+GmUF>FRNC#xzz-xVz0 z**@{&r9F`4Aq5=Gyy4nRqQEkV)2v0*bH?|l(3A~2DcBb~LTK0&|2k1NW3PhUYewr- z&;2&QnAwE*6yq3aNbXB?f62WjF`i`vr2<3}p2#F6EJ-nA4&iYjZUB^0%w*;QHW2`l z5}|{vOX&6LFnmcgK&O268Q0mx!Jkt`B0+5^TV-@2k`^G;qR~rWc$Z6VgghO4nxEVZ-dNV6AQ zMqmoNmg2Rtxf+CZ>W!@fEV5(-%49a6^Rp^?1J`ETkmx{|9ME@$K5CscP{ngyu_@21 zcvcKtwwFfieSH||T9_mSsg;R zy)BWh&$8C85u%*|NLXq{=G$6(Y1Vp)q!Wt{SHdTR`v}VxHw34!Hk`Wc4`@$OTRqK8 zucNmkqqwU*b|N^0!7`y^^(m~|#148wLP#VirE?@q&}OM0o!b%X!la-hfak$3Y$vS8 z7;TX70jzl#9g58++5jssEXywm2a!G8&^f^KQ!3*+?Oy6`hRuaJm^Net?4J$}A+><8 z6n!@>+Pcvtpl&W#Ye@7hiJ zpWece?Qswx^t!TwQzpp*&}Rn-Tj;Eyfm~YKr~AQu`U~C-S0iJD)2SH=1H7`lOk@GJ ziuGK_c%R_t?KkQiS8j=CfX*Hq++oR%pM(7GAOLqmAGY(MSRV$K=RPmlJj2 zwe|;$H{m##r!Cg<#@Yfli#wWO9pcDZsRdX=9X#6sT7G9>;6_LcR1&9b@?NBzLKYy` z$XYo7c(IWIJCuJR%TiEjR%jKAwa9bUA%kcH49OCn!fsh;-4=m{n3_pA$-;Ua)4`&f zc6Hxm2wIC%$;&CvVM2vpN$g+_*$U<#|G~Z=!>`~E>xsH}a8$R9tBE!Y^TAljJmH*R zUv*|45_NcLBrPI!jvcEfTrT?vB;j`)261BAFz5|3tLMVv99?19Lwz^yQG2nQZ1eN5 zzpzO8iS;ntLphzzX+R=LEC5?Zj?g)PYl3))3kc|!Mn*Slc|ap>5C zDp<7xvd`z9Psci#on~6Mp)2K}xSQv2d2)Kkqkk^fahJBo(l_86Jg*QHR7;w9;|Q+k>rR)b@2(dLE~XreGnvovm$HW zu)jTnjEQTZ2^7~e(TXAVxvr5raNhih;>u1(?HcmRq$lD#0GCbOxw zmkkmtVEE%QKWl(h-zHk1@2~WsS3?J&V2&Ky3-C>{iBpLK;t$n#t)KVt;LDfj_?(eHVIdfT zUBW5UxlnMh=RBAd-QB#*5e|dQ8^7U3Yir2RLQA&_7>frsUIW1%%THWEQRWD}rqQ#V z10>2I)-4Sj6mNp2jR`j3`30dr7z+n4jT0|oKs_xwfiby&#mkjJw*%vuf4+{nqchUd*O zIW1)NN&Oy?3CKH{G}tN}VjU$5oFsdex`wI<`<<`=tc_EL{^T6$P)(Id1p?wa=w#EI zu|$FBRxFz$)LrE{qXW^YyX*@NLRc92O~1sb&(viZ$yfVG0;R6N!PUVQx-IBHF_lE9`aMMxi-B zvr4;q5)F@?Mh+;4qQ1sG+l^s4Bn&jMv`nLOv~Vy=hs+P04&k@6ke}M8nf)$KsTBZ) zcs5&&UalGK4C{)M>;(scJbFGIyw)t^J~-Lpgl(ukOG_oPwqa^E7ruraz6t}wql8Xs zVa4ts{$34VfAj^J%$I2Cx(E-{hyEVg_`!fE(mz;P3tNOvn*b*Deb>Y2Yu{syz-*N} zLt{^WWJ>GklUcf~g@OB;&>x=%Xs<(j+-k*tLd#VEik*+1007owk>UmpQnOE}#l-#9 zkPOvBSaR^*y)gWUp1bVB&HM`5re~CJAk!GnQR{x^Jtx=?Gsm-Sjc5g%!_sFITxD;0 zm@t9rlEcW|?>z`RMF2jpD*?1D6kTSmtWyK=oPxkZ^o(RdaM0ki4`U#co`Uf>H)D-| zTMN-}&Lx^Do1v>c*3j1j=I|U?8CEidxZAr1*z+UoVeAEaacLF}CeC^fvA#W&FIkJ* zbOLB@ZeyJLVj<-Qn1wpDa|f&sv+WzQhDE8%2Dr9Flm((afLUHbyP}k1W0M+{MG}@c zTsFc;V@RUYGttLGJtE}TZ9qqCIVNk`C9qAUk^lf_&KgXmJj5Gf(#d%&WvtlaYSLoY z1#DU@q|Wopo|tLqq0>5>G8y)BHB73FDN@0{Om9?6CbhH@ipECPXVmBdGD&DSq$uy! z>JmModsxRX@~72=^+_7SU$r$>hXt~PR{`7^A|=DrVl--Wu#GJ&R#6&f7CT5p=oM#yp=S0BHO(wqeMcoQw@KuaquFGabH-k{CgBiz~A)=rqbEvz9O zID}p6d-X2%ijpE^r%uhUg)|45sMk&bROH=V{gG3==f33$Tmu@US+69a1SyN-EG z#^AaYX3%F~uk#!kz_ed}FG$O#DV@V}dJn#~mcx(?o_j38q23HruNc`o$Adko)AS%` zwF`f4$r!`G0<zVLH)*mj&vjUJ25X@C1B{6=a?@@V^2^w{_!WtZMvX^H`x_cmGc|nePHalOMwXog zSQ8kewr0vf@P2{dD-T}@FD56+${WNew?RKL=rCoXVZ&fqoO}V{awDe-7kSz16Gr&z z*;54RFhxwZnW3#XSuLTHZ#_64Hj9DqbaRP;uzpn&i-r;5+LVu=77Np2NCiVBYj5->>iDAxnFSzq_YU>DF@X}B$- zs&M@ZmUxH4q6wJLZk$~cpbE=D;onu7-ed51rkl5Jp?Mn&b1$dEEOqe~rlhu!GVH50@%qU4-J+J?jj?|A z#q)6g-dGqI8VbcGI|@{VABiFwpHFc5hs6hgG+~^PZo?3M6h8Rv&)HKq!_5Z| zLqDSV*WZ2{ZPY3i`KKrdOVR^-A`2^`-8mx5I$NIzfAF0*sQVhLG})YBt2PWW~S z)&|Cs#Eufowi%@>gkG)H)x=?7nR{qL;IFW{)%A-Vsp9lq$L~|HP6gxxX1HcaZYg0C zssZoHQMARE=*JZ0sm-yaVG8hhMWn!kB;-CH= z2TyIt2am{L?Tq)6%3-&fJEINEAX`bM`=fKZ;6NQqhhY9akf{%?DqDMtwWD3(K&Bm6 zmJk5}_x4TAOc}*kLIS(uXt^E|W!44n(}sGe_I`yIin_O1EB36kbW)Cekwtz`OPL<8 zr-LG(N5={KaO3!~$3^VQHpvd%9572rIIhqgp3z~$c8Bv5F958W@bJ}}d|oCjF2fiC z-aF8zm5#T=&;HLJqpb_Wiy&RTeXkKn|n7dpV3)!4Kw39x`xqN4ThH6 zH^U%+;)=2hDGdHOA&vu2u%0{^+bcuCY|L6uBRq?tU=t%`T%3d>?yH)4G(4n!p&Ajb zizIG37Kb&P)Vgh*1Im+t1jf}47K1QQr`+#hfr6zwG)rd#2nGba$9_JU#g}E~l`$^6 zuz9Vm(CmR~-J&cYqNTkfZk7YU*D3c>AY^7MmRbOw8$vcP&8F=}%XDk_I))wxNxK|R znFaJmX%>z1cRsZU;4|f(&H~J}T7gC!pTN#S$=3VoAw*)`AA85{pR9%^He?Paue7-s z)}g1%80osMenKlG#?T1<6d*PA>kq^DZ{7n)N`kxuw4B`9QaC2~gxgkq+X(`MjzV5|= zoblWTXe|fnC~!h2iAC0_ZI4cwqcU~){`xQeD17;HDtz_jOUjsp(8T?e%VhZCg%U7f zXI1ENsJlko4@b+EW8P0Ad&2kZHZ#QA7!R2e4df2brX?`}#M8KjQ%26>pIP=rme9V3 z7qif&9@G)8^Y0dvqv%P;}S`N2ihL+*J!fQ-7Y!E_F(*3c>iaFo&c(CU5L`JQ^*owIi&=zqaQ3#DEX~d9)>SI{~D+K z1&miefcQFxIuZp&Z-#-Jcd;9E5#e(WEPTo#p`)MBaBm09>|Xfn7yrP>8TM==j5bX6 zbbt8%tsln3m+9xvC^11cAWnC2eLswoS=`QCSd%UTLsZK%&pum#fo2TT>%465Br|7SEZ>sG z@{4f)E@esxZ>#M+1Xh6^$N51&29xkxHms`+n{o zNjE%3yhqu-B>q6a<@)kDd3(H;pYEFXd3;R-RlL5!LdjgodKu>QxlK+ZN`UXHFo^2M zV7>u9u33p33-ZfYUUJY^&$&-zB!{gNv{CPZgC=2iUgzidC4(pfSjC!lK4h_D5#2b( zfMxzO(rBi>;}wA<^X=d6Sy{vA``sfxl$gk$VbdIQ!MY$U9*(>kzgNlU>jb}QT`L-O ztUQ`f_mSEa@7EdkUXyiWZ^jSwmXQK*1q=@-cPrAn*^PUfHtQ+gi~71PLV%Zu8|`%| z-Fd-6y~5zTVKY=;mufuYL{>w%hx;g=SV`P?PiO)K~`A;k@#5CtJ zA*K{ys>FmJ8eYJtH0Sy-zI`MG46-QcKZnCUb~3n|G)~=y_EtX5a7t-vAw8auNUD{Z zIxNHn9h*Z82_^umG0{}CL3QGlaY%rq$sw1?3QNO_NY_VeHde_G(&$Yqk z5H!}}3~YTSE7rgU&?l%TkhPUPGW*y5rSn^8umFW=u8~34BV}CATCk2^HFGP~?5V#J zZD4J%CQY7E^Jod1<_4;>vnCo5n;SwZv5Ah*xU2y#*6{l0)W6E~UZboE29oFOpjFa7 zj7o|LR5y4=NVA2|-oVgkXhAP!nD7+w@7^X^(X07b4rF5-zz z5(cb>Elv|oUPQCh#QLpCP&@_ypDT z8{hfzL9FTI4@Fj+=!=1sPpK3JykQ_gi{31cJ5FU)&XO%N|s|8 z7C}bz7?xAEJi|IQV4{A-HDuvzLk`E3OgY)GxjW95kadEL(l(sfAq?mK4q+~yL!D9L zOvBFFJ+xXsdz#pIH6Ww8y%|RU0KiT)@|oaQn%pmiEkJA?psl?RfWaQx$KlSdV%Tu* z4LHoPhYwkkgwqL$Z7eT`Y4(qWwac_h>tp}Aw{$-1YqW^^dJE={=IwMsXzih19uHoR z#A2Xtkmtr8aPWHk-p6Res0r9g5n=#xjxb5vw*c)Yg|N0XL-G?zHiW(|R|{bfCtghk zwqv)7(_bNjv1eqMb;EjsVQ!_-c@n3>GROX*(Z!~0vqnu$$zlu4I#mMU_#L7jJeZ>; z4~L!%>!XSL>lo^Zhg-$vb!5DzVRB?@4E@9^`Y;1EgkI7(wfcHO)-{w0Av7-IVz^B? zg_4wtmIvDk%Vw4{ftP6TFK|K{vnxUaIt2SmYFF9-hCO|K(GgxS#}aK9rrPF zdUUizA6~#0Hrw(EGV?4T?30X#^|o*-f(!12ivonO<2cAN>rM?ME;NE8osSHGj?M%H zurr?nlgBmn9W_V=)bM%@9e?Q(i3xk4mO6t1*L}*27=k|IaO1kN=&+5wP0~rR(Jix+ zVSVi)aX2HqYf5WpJ3{uz5<(t;Yg?$(c2bxNH-Pww}bUgtq-dYir|uet0i=9B0%Kl zr>t!O=o$MU0#UB7kJRrOIT1s}#1t@+vNhkU#K19*v%yUZOq2V2K9z3U_6}$#IH6m= zWDiZr%*4FI&UOWSv4LgMOifQvH|Y|ma+PEO0Az!2O(?!((Cgt4}#0t@40o!6No$;9q?FUkIbNpmC~9yimnKiX+Yacv`3h z6uEG$`*q~0g$1k7+CHQjE)OLUEn{eFxORpgFcpNBPRa0^i6O9e|Kh*Fup+w0_+J5h zY>u0N=_?odLkatQeWfM#qWpNQUu~wG**h&*q%#9%8U`(A$I!4$+%k zG1duK6A}?a5>|w0R;XbtHF|=@ntkSv6OzdsYCy7vArO^D){5s!F;@8G%I1Zc5?sf{ z&G_ze^0xso?G9N;To^jrqPJxhmf$-64r7$nz_555IYwTvhzH8qApuxA$OK%bZl2aN-THW;KAlBw(VowuNU%0 z*I|P&+t&Ja=p+*^nYo}0#~nc7o0c+b002M$Nkl|xKPFM!!vAvrF6quI2 zf!5G4P#;z)i*U)fcTfY*+D-yx#k~zUSWE?b2#3m8dFoGteq>}k+19cq(j1)3%(bIR z)Ljq;HsQhI(1xsHJwr5rif1qALs@r3A78){=7e#@2^A^`IV~!DqOQZs7 zG57hi%Q#vHy8qTl*g*$+iA`HV+@8cv)x!!{)HR7n+S9p${t^i!i>tF?oDwdcHoC9( zg`xY`Lb;LfnhX^Vm(o)7a2~K7b#7ph#W8l_d^d#{jps4x|7il`yx(!~r!qc^))x zdmPwK(H8GvRLj`69rjiVR;HK3h<5In=JU>jelHIzY$bsf7(^Ys?WG8NAcH|?NPp?W zVtrW8QJ1u#E7bOmQK7(S6@CJtyj_A$**|K@bAY`p(FPO0j9kdLc+frvC}yz}C+hpK zA!pbb;a3bN^LY+Y#3lS~U@#JBL)qQ1M6&8w!zQ|yo@jffx`*5)b8`q2P<=-h7K~_^lu#t9TK&3aW zAL9b|8YNEZo@jl)lQz-%F`%n0qzIMCLfL7+=Cv1q95cOYxQ8kj0^m&8Uz{flgCir$ z>6DXJX?FlxK&8JOQR;S>X2R;A(NnNvM;KV7njrzxH}L4~FgL$IRrY>{S{911%jDOj z_Cq5xfvg)G1KF!MF^h6ClSnAYVr!-T)`M3G<*eh#*I-4cd4uM{YV!@;>+0fMI9#C+ zCROZjj9m}UzxbRIC1i49m=s87DJWS@B7vK?YX2Ajwa&s|#qwndltw_h_3Lbn)B;el zi$Q{gkR@~y3unSu6__jZ>xeT6lN%c6d&89K=L~_`nk2(`Dx{4DmlbizHL3xCot#iF z*>uM|7lRH{MJt{DE{^6GaXeO+7I7M!!g2JTej3TdY6P5J8yVty7;1heV`O`hO-1~S zjbYS~qd+kwNiJwvb9)G#B%+(_(fj1;4psKs!WH$D5{ScNQ+LMg zHrbWe>)OKNX)3(jq6N;I?~s*;s2DxQ{^C;XLt4c>w^BfMZaPoW!35xf;>gVFtKWVj zyz`^K3ZMS&Q)*2So+2cdCB$osrUmX{Xn2V0b%vR*F&>Dn2Yb51JFihTfc2+?+r^RZ zv!<k}7%u>S&rQ)3CC{0gXM7SO7HBY2fY~581E6?;45%FGTIBr4sB4U@y&< zq2bUwdM|>dC!aqKzxntp4pWEWo!4)LKmXw$$70GaCTGK^zkDx@s68Ti!*Ke0zxoW| z-WtYkj8Oo1A-wnEy>RPQv`ZY8dI$T+MuzQTy|?sJsCeimz>1_Mn3uKr<*+(O*AVvA z#`bTl$)>Z;WFU6n0Ju8o6Xp_NpWaPXQ_*p3TA{FjLC4SSlo zW@f3rUN$eqp7(vSc>;za2{QuVXrp1y&SB`%0B<|$TmkrFp>=>^65y5LaFVL!V8A{u z5dzd9s&^J3lvzWEfUD??SVuazhb)=&bZ)nR9Hi0iE}r=x1b9&fEL zgyqG_&`D;roqMcyzqYtZlFultcs|UcrPMz)LHpTZB8l_1j@{(*wJ(S+&P1UN_Gy6(dcmrt zc@7~Ss(GK~kz7-I7X2>MB9bQYrLM8i&~=Tlu=VN4cI7~Lj!to(17D8JPbMW~AOP4j z07rDj*iTE)#B4}xr=aYL!{i0!u`q$rM>3R^LkD122%xPK80lMayNsov8SljX#TJwJ z314P}Lq>^*DzCENUCOX8#1blabBJr{CE%zncYl#;O3-hi9mPA9=s%3Y)54ENCWHwZ3wzpbm z8A=aA%85>4WUKIFRjhj%I~g>HusCKO*yGy4j#<+YFBku?NYQ~VuwMaE_$VVeWmt&= zn7&xQ&Y|J+coa<~_r{tOutDrNx|%$(s2xlEy|_c5tv@=$pDG2gWQ#yjrcIK_!FFDF zWt<1LFF1z1qI5vq5kH3OaUBOEA(Q57So4-u(^gk1PgHQu`2`wL*76ZIU?y#mLsSCa z;r^4Tv<;bKEJ!r6B6FKe<|kr0tSyFidC~ZL|40Ez($6^!?IG-!84@9Cpv|mo@+L6i zBFYOKWV~URsyc4cMRUPvBgM)y3&grDR&thRPf=#Xg<%~uc6kixGy+%}5VlJlF5|$u zNiSYqUB*d}A<1d$3?rft-jMDU6Vu*9m2a3iOD5QPU{fto7a46BJDV>TS!^N`KM=fMV_ko?z#j|(4_Is+UvUW`;R$HDykAO9US_?9Vw!72LpM!P@=$S~~&pzezBnmV=WJi>RDmixMn6Td}{ z4M{c`J`&_H$LihU$q8dp%WF79xo8^L*wy>I|{Z0&6$lySzjNp{&K)P^Il2G!CWrDq8J+9>*>3>&$7 z6?*U9r73eSHPWW(Qevvq{=V!9ca4vAPGVG~C-A zjyG91tl9h)Rq&bT44WTPl+Yi=i@AORc1B}&ijlN_V}s#xCxYFZWWOHboIL${N|!xK zkUU{8U&E*#f`(xs4>7D?kSyTQq^*-)o#@Z9uzA~*=;*tFNS|&IW$Kt6u_ha90d*L! zBiRee`ZTq-gps?jnWIAhFhuF=bpPP9XC8lsVG!tBIC_~_Ta4Zo$VLq9CZcmD*v;ubouwUy{Z<|n4Z?OU)P*r_yt z!UC#<2*R4BS5nx5@oClyNj`+v8n7YFois&nr*U{Y&E}c6-n*|v_FE=rae6u|aJ>aW zw`YJ-nd5HO($wS6S=ZEKWc+K}S^yK*$yz2c3fp03=N>?^iglcZ#hiRG1G{Gnv(8Y( zoIU$&Iy6%*ril=571Dqs#{Q^^F@n{zn@MK4BB+)fKZj8=<2Ol(9fk8O9oj}AtH54d zoUlJR^fWcfgb12&jDR%EI&(>e{uN*jr(;+~C#i0fa}CZ9_J#i}@1rvehRczAec6i=8fUh9N;^H!SyT~K-L|5456RHN3VddP@>f2=OFGvD| ztHVxvef3cI0~o0sb|r#2o>9G#3{DN2T-*`Y!ZGvnGz`@A#Fqdz!lC^zUhI$cRX~v8 zX}U3d`2PE_u|zP~Tvvcw`>1Yivi1N}FJ{&;w6)>Q@4X(|+imQw;gik)#Q158hIE59B_aB&KhhbSakjL86Yv!j5LGiJ;h#mk6tTw6yV-! z@U5;5*)8wQdyrA8FklrOrl3A3HchZ3sPoHbKE>b3ka>^x0=`1aX_=QOIpf)*Hj&o~pj|X^ZuksAv(K%L z&vg*zn}6r%z5Tf#C2eui&BW>-Coq1u5WQF#l@FQ9|phtX&3qX;@ z8&9|iM@edaZ*zqQA>6{LE{l_!%`bp*p+C9ltxX1yhqpD5;T^L-ON3(h(D66eq{>4&reV01})8EvZq>Y5HhwI_zL< z6ty>k0#0QIc@~xf;{;mgbaKh!vZ$peyEf9~W9m1br&h!h>Ip#T492r;$QC130h~N5 zr|=q>$rKq*n>2RO+cg*lB6B?IpL@LzNCrr|QzNU(4ZcQ->Fj!p+AyHel; zTg2g95K~t>XaM~J(7eyM$@X2r1SQLPZrKtau!k%l6Axk*6&XI)28Utpn!?h|EZMmv zUJwS(0MM^EE!0B^HM?m50xsLwt2S;Z&MACW;e;*%9Cfba0Kv5pG~0^pn6-1p{FYb? zruvaH1f*~ZNPVUs58l8`5pue}3j#;5rOaTNoJ~W>b|y_J}N5#%8Gf?XbHvPvhw+ zSTjHx|EURl{NDRvcn*sKFk3I>@n&F-RCn6JD%Vio=;4~UY2$XSVo|qhFM84U)_XrI|)6Wa@g1a zSek7$1zV8iM7V)Sy%T3XFf>fRPz+ugZJ^nSgnSKgCtN4&i<6jL%>j%tldUjO2}=`= z+!>4AQVtgYV7_C`vIb#tEI+Y@LrVby8d=98zcWe?yrlM4CuLUP(Wzk1AqHI>NrEuA zP2mubroJgj*^v~W(F3`mjQnHbW%h5}WgKatgq48M~YaN?ND`c%$zQwvj{~USrvet3xF60G-86 z*aiSW73*hvjy?K_^~hOi?s*X%R>r@%dXck z7qaE+EA-A}&3HJH@iG)9ATV_A_un}Fb2H@uSS5h0%#dFK_5>iuOg8XXbP^d3sR^7a zCLXY*wg$Qahpm0rLCK^Yvu3IduW~=um5YoK07_72siw%z;ask8mTHC4gtlc>Bb$p& zh{+E4jcUfGx*lhz573!mpGqW5DBCHLLAbU9TdSn!gv6H#J8WmV!YUmX z(u7Ald%EZ?oX77W1>ul(fySzqWWNka7LF-H6Uhcv2Ve-ciMf(Fld-ZaluqBTZ$JPq zXUzRQa}QXP?J2THocV{kQLp zj;{5U2$fh%wc6mu&5^nJWe1R>SYB>>E3Ph*s<}>4@o|`WM zCPc9({9VnU#Ih7@_%nb=nTPhKoqWtUgTYCRhph+U)&i=c?ty_dRah9~Ne1GZT>z!e zwU~}j%VJMLE#@$`kOydU$ev*_(1)AO41fse{Nwx*00E=LzZ*G4*JTMe@Y!3Ngrf*2 zWjTc%@i}X&t5KM$=&u^%;l!1pWos}vib#D{{1J7M&?s;Zu{8)|U{BFajIPUFLs*B# zL68R36%fX>P}mj2Mh&T@B|=r`J+ zZ<0JS>X9ssChXCbLW?h~TmKKf@wFoyzHm6gH+DGWn2;kB4$o*Nrrp&`Rd+3!S(R0} zXL8>Of&fT>AOTX(^CHu%W+w>(-}ip+a?bDk&SnP!H*qO9$);lqF+X-qIEGDQ;PHOC zk0p+IEL;M_e&NpRAk5MLqi3(zn13u~4tY956>=!yB&AA*rTB4?3yLODGA9+30#tSj z*SGF8HbU$KaD2EQ159fN)M3L?KZcPwXYNb^&a%F}Q02RTyWt^%^AV!Q0GuqqZ3qi_ zh_xpDgcevRlPu2)$WFW2-Ms86%hH(WP=l$c5;tkNksR|kgoSSwrJGxDVI4C&N@)T1|Z7>C{hL~Ghl3l z1nM}oqIRpV0MaX1G~3}AFnozyXa*K|XqsdOxI#a<3EZcLI{dI3WUtT7m{0&^ktze^lo3ifLUH|{27 zKeAY?6N}?icOQ#M1D3of0T680)ra8I;Od~nbon*5_St953-`XnRisPAb1R!;&tWsC z?H-}!ZCsr!kqo!7BpQ30)H=%p+}kh*m$WD2K$XLc>Lpv&f zB^GN2R{oMPPtD&9XRu#Y%DUL6wn|x&T74Bjf*X~xH5x)gbHxZ6s)sbSPmqLy0?!tw zSeudX=l}X&0MypP4zBNKkADbXeut|~H#~)tXXi#E>;IBY2G$!q#9gk^xOsCi7NQkc z8i!gV3}SC}C%pLjJ{jsHY`_+t0IH2sn!A2AmLuwAzuf%EM`U}_mg^H5suR0q7nl~z zPJN5>y+K>D^^LH#0^`g5d)UKi3^E~w#i%W6eB=5Z+A3WMYwKjS?~;Yx;CTpYKm7Gq zxP9$kit!5*SLA07J_V04_z>PF#G&oO=E|yr>|P{_(ngReCn zdujiq;y2TAI3^Mxy8tqV1n3I4Q%#l(0TmO&M9BccIR9QNiOeu5LLhBJdXqF%_`15x zIzOd8r8N&@u~Kx5`ltl(FW|N8o5R6T1mxaZbv%a&LRP9Oh>y z**nB$a;7iR#F-K%mxYdYjJBBS<(W^(NL_!ngHFkYS%7i{`32ntIObUaZ^uW}*Fzr5 zOz8zc+{HLi!$$fryhOYeV)mMO5*G&M!(rbnvXI2G`D7N{DABt z46lLsS@v|jydUE+i-g?BBIz=+9ePc0PR)ojzemnmvegRzsN{lOJd}65askr8n z6WoorVP+y<s7uhf(S-}hY_m@UJi3K-Z^o`s5nM_dbkB%niO-f{9Zw0P zF(cd39by3uA?2UzZq)P~|HEW)By4=V9WSriO?s zl8uU#BGy_3E65GiZ2@F4U4#!I%3#Yp1YVOvAr!?8Q*7bUQ|GTh^U`%67)EDN$()%BV3HpSNU%wk3 zfA-t3{p1mBfb3X{5(SwshO5+ryF<$%JD8YF>q@Fb4`4ZTEe{Zzw$EBv1o;v-=L|0B zBvs%qNLe-zy2u=tV6WHMdk=q2J-aN~lrwaOfLYk(k&KN*Ho;U5L`E8T8;)7GLlZ~X zD+XKl5b&r_nvm(krF33#47zDJ=VXj;>{&pPHbMZ^#p?@E@UO#mIk$Ccvf0=26foBV za8GhQv!OZZVgCiPWQ`I&DE#ul0q2E)d)?YF&L^;6BNNwPL;(Z@-CLZs%6lpRq-_-8 zG%n&nj5OWJr!DFM$o3u*|)gkg^ z0H)Z?e1*L>pl#rK0KlnqFCy^Z^ZjIUs%#NI#IeC3Bw0t!tgR3= zG(c#uFL)D(kvzwE>Cj57M$0REuR4YZI*=X4^S~O=uFXE9qsWW`V3~sj^O1##-rd{@ zJIM1UZcP*nvI@XWwQ*g^md?3Fy|)s*efKL{FvjSQK9Vr{Hraxj1P=Qk@KZHZG`(|`CJ=@v7DxdabYUd z`xe3m%;wP2?NFSWBxJ{0sk9M8<|O%C4KG3o*Q)WnDX!ThbY*{igD{L6C}zr-ctVhi z0%Fe=3mfOYeFp2zS||hme^w^~F7zvF#UilAd>X~?ysLj*XLY2&CW1d?vHC9RWptOW zI`@r$)N&jq1~|TW4p?6qH=kE<*h2<#EK$Y#BcsIYk#u3a{@%-(igUv)B`_SPdfrD6 zy!ae^6g_8!3dbk81AahGb;k)QWPBdieVcd@$I5<-UZ^(!$hdhQunCrk*JJ)oP}Bgg z6T94l3=`C9xO6===y{%*k1zy4IdavuW4**E%0$Lu#LS%ugq{UGeF9#)Fi1wJx2-qJ zTfphYTx$Gu=S2oF3!@h^KzUK{6@fAH?PTcvGTztop&U&DT)dBUF&oG>-{XI8*T=J_ zvE`Snrp*jY;wuf~W`A=o+!H0@g?)~^ajwiuY&7uZ{#-H|ZmBGm&>+AdOC1{#vK6)m za9da-1x?un0t8eW+hEy=QLOqTS#VX{A7?y(8z?deFef%Oc2cXv(qx%Dt@jigEzM^# zw3TskobVWY$3jn1$TP`x&B~G)(^7B@u|xuk&_tk`&h3PoIdp55EE9mqxtoM1a3kuX zD+3Tga3MuzQk^&~)*>t+K`H73-vztF}C(oiFxsb`?kZMg>%D{Lt+y3Z!K~}O( zgYiIWbqug37W_S_T9a#+ewDYwd950TQM9qP5&NbMD|kq(p=BUecQpU5&`(ePn>YbtPfa5fVWT(CRZ46I*!|~Lz#hn5*3QnHq(`P2Fo%yKTTUE z0Q)NOS6J^CO&BT0bc6SA?jD3aT#K8}o`fGh`xWJC?uC0l{aMHmP<@p?!1e9jP<{D~ zD)AJpMQM-l{4>;tQz#$8@H7z8#wYdz-j8sd?C#P91NX`3C^f^F^L=WnS>h*$@@>G3 z9^fwBEz>=Lc*uoGTlfH~DEhk!bJu}=y>nxUevA3=SHJr*0^*N9xd#Zl8fM<5zv3y` zp0HRe8(WNFIBddK|5Nl0YJHSXE7RL2RyIf`J9p{ z8I>`6Y6jP)2}v?Twy)Y_e0jimhBe-yVf-0#VPs||)QMp{Lx840{KJ!DQMn5IYV7H) zwH;byZANymG(H~V>848KbOE8;!8$eiaJN>OD;uQ`!g&70zxx*`NPy7lPI&h8Ij+jr zWW-bWmzcfxfBJrS2WDjX`AQ5RH%Vmx#_T9#Wqe%UGn1?hb4-zF%M=|GfPMmlyY+et zxuPZB8@{RkZ7ds;WiD@hc!zjPIsgE)-X%#%gQA7|Q$yznqimV`?;P%g?FTS2-`-~r z<6ROdRWZz*edRT`V0MgPThMt(E*^Bqxed47%GC82s)AbDFOf=d`_!0 zu6Ki&&M}nYt4C{id^qQ4w9CVtYCPiF)bQbh=Jh81!NK}vRjdT zbDVq&@C3lR0ljAyV<7?^$H@3~8Ea7JRzAjC6&XCPqtX;$nJ6R3ad2ENj@4)Ca6jF$ zZ|@O-8~abzO*g%I(OPT=`?~-qbT6JRK8|h^x%QSg$OeKij6UXiR?uij=z|me*Q>>UP(z$`BwKYVwma161*jdVQXM33$gH>|xFWuI-#4cut=k zu(>JAvcAoF!MgF<<|8a=fFjewxN#ve-cwi<11z(AHaCdN4{J!*ZwsO381igJ4}Oe| zV17Kn8I(p3!M#0z8NrmD7OdlUN=9BRAjw9x1z8EuG(`Mgf=r#)PacHLmrp~dQAHWU z$Pv$Sh(f3^-q&P@W=KFVsNZ;wW8w|$oB@>;4`(lPTmjwNBie=dihjUf1=n8< zpr1_32B5fsp*f6`DJt_XvWG0pS|eGdy!t#msO`hTPy`szIX}G+CVujx=vq9%NUHDc zgtgsG4l-^bV#h{cmKqodG7D|akta-Mv5LFf*rJ)~Y4$gzJvei&Dfl=t80!c7%pb*! zd5`4BAp82%fczgeBMUt{F+!ck)9~OAFT(HdKc{2DSh)AWl`sYX8Kn^HF@BEr?@Y|`f3qIh7uqcC)U+FKc=Di3KlKZy#ptHOVIK zy8*gHoUwn40OlflVENf<*dnHJ8-=Tj<~efCIH@N4H;K|nqM-7e=R8W;&K$DEJ%o^K z;dXuU+2^q1xZ5YjF{CIQONWRI?%4r=@M8XIxWa77h8a5}llT0|a@cu&KWsez8X$;! zkh3^K?3{H7jdeAHxI*&N)YZGxxmyfVSMG(I@BJvod)mix6UKX&(oQv!sJ3>$!^oy{ z3#FNKaavQ&%JRd24MEfP<`!-@++4^#W9eiA3`?-Zow082BYWh+IiQeGK0N0+vOfV6 zXPlYcM$B{_qBR;S2Lp&wH-PvyS#nk)Fvia~9ogB)1|e$%3a+_|)*5gn&UljeVKz2m zjCn;_s?06ouKYG5RK}u>VOb>}a(QbV zR1PR{R0eF~LY@Oi_F&LAkf$c9oFVsTXD8_-f-aP`DU8xCi(aUdGOAI(Z*XCraD}C4 z+?kBli=61kSbzEbR~V{*NX~NynViYPwxJ`+tZgrW_G!+jE?JYSWEwO^mAl45Wmp@@ z0;Yv|djrT+HOS}`I?0!1R#SioDocw>By2)yI?@TSN(QtaOgb;HY$iorPopLjhd#4>GWVwceg zqM%;J<{ZRz@R{D+hnFJ$adA_J2nSS!s`m&t*J>#YP2$y^O~>;4%@>`#CKV@6B5JAQ3ES1-R^%YoII;&nf6* z&Tw@U38oeZ9y8817NIPk&!@G)w0EDCxr&YLFG(yQ%hx)4M+ls?&6hk2bB)5Wf8N6?S2OrbB z^eW{TNKjdYrTXFdT5PRl#&nVSEivD439zoL_as5Z1c*!RpTOqY5hD%Au>i2-wS7D5 zC>aHFj+j4Sp&v)hFqMvy2Sf2#*i@<`2F}Fd*OCpc@xzI##8Rw<7l$m;j3B1dJY+km7D%=|D0 zobFUzKPnB2bt^;QI5uYStUmE!8yniJ0CLNeW6PkJ#{$wM@yQN6tDW(CT%-S+E3$%;ykDIxk#0ers^JK7vkLVL1;e_SDLkDc( z{90DadjT}K7eD*p%#EyR3wM2HKrrYXd7Wn$@Wq&3eh2nNFzUJq(gdOLXBcInqW^Uw z9#E_Cem-k=Buij*0T}DdGwH4sTF=#gDQmRk7ZCUkpb9RAX1$; z&{$MJ55h!2$Dg}Ej6>~>U1A4l@z6L>{>=e$j1BXex?N-24i-xHNSlQ?LTJcbITkZn znd(@8G|PQ*Oq4C2WMvvylG4zNj=yKw(6W13CchUY9Fa4BuLESFn-{jM zo4tj=XNk5nvLy&i4;&{j231;j(@l`#+wv$1hEtxk%o#|)7L3d-!W`n7 zX0x2xUBEIbFm!!b{23U7G2%J|-f@nY(-J}7$5^u-K)gT1f1y}CtUV)P?%ulqH;;?7ACO$Ig}W`oYXewh;{@B=t-_eRBG&8(1!9fOIfH2{AjrRawjRNi9XdSm z!zig?sCS!#H(_jK- zLVgjqIE7z}2tQe5h0s3nKNW)IPf%=a6oDlkPKf_IM{#KE7$>RA${Jk8{K+8ca`SnN zyK(O;f(8ND3i5MaW5Y$^eT((99o;^)=2nPP+9qypcJ4+vAOrEvd$&UB9v~J_vk#+I zCrXvxEzb_ZyYF3LuM#W8erO{YvjDYzz|a2PK^U4Msi;Nh0{h~StY$&J zLdlX8_7uLr^<2h1yvcQJgEKWv+#)Wn6PUo=7Zm@+g_otLXd3tG$Q<1q2%nf*f(669 zct|1LZQ7mn`FEaglzEobso~P^Fb6v;+pJZV`;b{$nn8F2Xs9by8l`XQ)%TdM18Vp! zhuxKB7(5IdvJUM%T7u=7ous4@?$hVbpL1r&eje`+^Zk>67Lsh(n@2E>8!xuVl%+l< zuEk!~qXejEQ_{tnhI_c;^RS(p6osy1tZrjXXL?~~0j2%ywPD)h<;IEGnkKfG^WGvJ z>=ZXkjjjZ;V@K?x0!rV8*2={KNiPcfQ(Q3izua4f31iLr*_&BHQ%pio291nUL>$E> zP#WOO$}Z@Jam^9lQD!JN^?X>)LM3iSbQU8@RVgrQ!LWHQG^l%sR}=iRWMPnVlfTa7>b1rmH$0@iKD=K{TgYnoB2m*=}VBTbmUe3Hm zl`_|F-WI>zCw|snqkiUJ*{NZ8%xbZV@i(fK) z8cx1nIoJh-Tob+Tm##|_Xaqm;8Q3^o@Y4Nz%`9TZr}13C$Y_xn?^k4}&`+_5ytspL z2x63P8RC+aXBN0io>4dfwkC*Sb@DL0~g2mGgG6DV=~Cv z_eujFqQJE9yqN=$?T7_>8KB@Qiy)M_@iXz^c}*9hu0c19ib>W@Hc;1wKJy_MqFy%{ zg2sAR+LJBVpcLp@Kxtgu5CXeFyqKFX!=yN2Etvr(KCgR|_t@zn()f&3MxcZn)rRT? zEr$3S0D-H!Nu#UQ4aJ=J3{tAnZ4InlswXf{08YlL#o!@}Ta^!!Rf<{p?{dQ_wj-MX)7sO-Jj4XKcbkT_tX!i6H60`t%uP_-x~H%I5UC7NnH@ zaSJ(5GHv3>6BPX?pL~h}o{j5phUL4yyh0OU`lvEzrq$OE0eRft0=z!oFsxJ4jIL*2 zP_GR4EQ0FJhd%`Xkif9DK}igsrMeQfm!HQZ25S!rY*%qn z-e3De7(yV<0fHxqr!@BA)_ZeUAS5&#kfn%Fs-`0TCK7TL46B2==i?z4X6yyg9&k9S|4m^svV1 z^Be*2n-mJ2z;Z8y>kDJyfOxPyl#y$*Mp=tzurzI4``2*&jNe@dSIBbRN11Gr6!Gfy ze)!_^7sRKv!}Y6xy>|(ApJd%pvNO2+?p&J>uO2^(3g|k_=hJ7;!WUmX!BDseqiIUz z5DS;%xz2ELHAslDhSCZQ#SkSB#&Llf7GZ(r$QW}EGsO#7z!qvgCh=hy_wxwFIVT2l z;j7QS4Jj0E8utD0g6vmfrmkF@4WIn{!zieqJy{9QpRE#e3V1{|eE!Av(LFOkKh9g% z=7JWQ2M2WJFW&}bOM9=f>H}0&Tw^Rj%1S?>=RN40QLD( zVwTVWvp42pyc4*$0CNDxR}a?1({Gk3gqjQ2Z_UBV(VV@n8D3M`!{W~h+6e&Q5HWRW zf>}$nS2C{i_=IGE7;D2E<)>hk=19mOE1WYud-H>E@8f?M6Cj>Hd?pa0&;S5H07*na zREU8A>r32Pjk8?=2qrjlx@-G!@h)$WrOX+!_1hRAcX5HTB?QSIkYFHyY_eGjoSi3+ z*I=Sa-kE2OKmIhdQY0sluyRI>DurTbdh(ftw=ff*z|FYbJ}W0~B=~`LkgjY0gW5umat( zF@XSJlS5wCNwQ%|0Z7zNM7+z03IGd$?mQq?qgh}WdmE#yF33o>?9fp>3*DH}GhR1_ zw3j_}Mpki>Sm%scrJVJs$kV%i*fw18V9K!aWETK~+~A&lq=;IJtp|4N zCCpef45lW+pZ)QNWP5LiMM^3Fa9IYWf@;O@)MgmPg%)Y6l%x15JL!Z9isJ22@*n6NZ+o>_g$l)|xdh6a0@umkml9TplGB!UTXm@Fd#571&bQPkI6 z0D@dopYpEwIYHSuYoN(m?3pC9lWk*GQYSjk;mPm!&63>V7RR*mK zLcEvchpd_R*O&kZF$f6~fr!{5)}Z4$vTG7YUd=4eKJ<*4&kcwY^?cX8C|uDM4C&H+ z81D5(|1z=OuR*4}y;1kJUZOjYe;AU=$~H2oMl5oZb5dvBB@i4}TOtS++~gH78Ty$n zEwIc5yp*-F^TPEtvU!d!P*t@4~&ILu7x^1+`EU)2&_^qGX%*tan z&m{-9$%AuU{>C5~=p}#CLOlj)lGFo$!7@kTc|!+RpdN?_T-q~*z$oJ)gxbN~qYs$) z22ivaLn3g;9OLJR9Z^u#E1>m@ePRpV>YXV2Ar_R)@W#6<9SDMlUN$SxPo4ojRh2m`#UR`+u7w%qTu+vU~QDRxf+7i z=Hh-eI8g2<`_~!!V0@tWvJF)QaM_2V-=-+>$?6Kq1W-}5Je35yExa~!P*9H-uY z4+eBN%zgY`_|c!8gd20x}3YK?#`?41yP1TM_v6BY0$Q zWx%EJdkIEYQ0p4s1|&6buND~dJA*I{k2l$6>{hHv>$$b5m-YJ5Dy4I>VgB|~_^&>` zgR-iJrxes}z&iEN2XddphHoCefQh5d67xGoyippLYyIeqcsQ601=RT}v4#T#{tt%_ z7q4*NTG+uYx&nwCB}?<}+(LA_Rw?hXze2ec=GcbobCh9Fh^cU@WF4K38e$KxoVaj{^p^9N~#(AitoLii;6h@If8$F*=b#`-bZ=~7t)*?(1-`hv5 z;|21;ZZdh+wT5A1!S*whUW3K8yMv6d&^|z_L0bO7g1bIW!X#(7|p^V1)0T)9I09Q*IlSAaKk z8hRsfK{=X@wa6w~p2XtWY252oVih&G8|>T6&@^>IhayNwdlpq0=!pnekn@J$=;m*$ zWBE@V+$}T<3Bk{H31Z_#T`Or~H3du7crMs}QKzUkA_(GFLD^dmlfcQd<8{ypr>;R*xh|%wt;3|SJo=E_yF7?}cCji(c;JBxB!HPA!wxB8YhsKbJ zAcWET)pb9r`w8nn_7CD=F{w%cWFm-j>hH$SId_60@qxy#KMUqiu=(nzHllORYS;d&_-P?wirLM7#nIHv{?^BJcl1~&3R zgHdBZw+qOCq*J3+zymX$xbQxVRf6MX_e1>g`H7S^KA$@Qu zQ5gtXM4M&DXL7T-==95z26yr`8=d47Lc(}HD}$SDCE7OTj*DZNv#CII-N3`{B*i~* zAK4w1dLF@g$~<{eJtWSD^BLQ)@Jo+?i_IddC|#|a$;0e>wV-_dB!^ks$~c0z!J1mq zMcj1pfN>2MpSzHDFT3c5f}H0%7RSU3Ls4E`W{@4MpU+ow?Rbw&VurZ^TgBJiaMTi| z4Ht2L0Du-&HGc{XGRN8k)jRfM=AuO}!305f*8Z!OUxe*PlnmhEjg#vzUmZX@$P=MY z3nE-2{qt8SXMx)Qp>HA5BeKcb2=^kuTlY|h-hby<;b(O+fqZnVVE7{T9WLI5J5;*A z7umE+s&-cZNPFw6#E8MRRrZNvBCZa`q(Vyh`abi8^*W65oWYGXjc_Qc)R^0I4yPGH z>nM*Tv258i0pFAm88yJ39WX-af;(9C6~IIqz*3@ghwEG%v2_X{mO$uBvZHJ)E)dL+ z7}F1XtT3~NTY~4v6B|d$8LS5Ltf1q6vT)QlMF|r$&iYhw5g$=8zMmMG88XOJRL~n# z`+tNHat?zu5q|su?loA=gFP&KiuoEdgzAJbNQJL|O(%)z89IV2kr6u^{>i`m^Kkr6 zPs7VcPs94tSFm5Sq&hgFi^w=fi>#v`+h|a~KcL-1<6YGH@+VjF2?MULK#P;ruX;F3P$Xqnd!B>CMw#4(qxN zI4~aS5&*6tSO5rT2#imi5zob1$U+?f=mth95=;@}e)diV;C2is>ENQX8M zqCZwFL1W0k1wR{e>q1>5i?qB#fRnK}I5-iCGdIHc;(NI0$sixwg<(bRaLuehoO!%} zc{-w?_a$z;1T0mH@u`6O*>g37w!kRCT(@9jb^%K(-+ZYU18@yQk3vRP7^}XG3$nn$ z>hQi^_7|$3QWqpTyjtNLu@*3YO=1|Uub*&z$_bGHeG?a^z(SY5>`n*whprvj1@*Wt z%U<@Cop7Sdi#4=b{xTV^2C19H8WkrpLKoi!7JQPps1}Mcue%BWqQP)tf(*2odfr$& znQ%iK4oEudXZkd$I)J zoF?f>L)#+R25~>V|9~W@bDGHKDfCS_59;Fek&w~KG0uLTg#y$u$Q&c10NZ?spyB;0EarIT0gTI&9bD_UOb1!x3A_OZTXkaHI7fsC3{Y}KFO1o#&Y8((7syry$C)Hl zVjiZO3~^2gU0o;&o52m9oen*?yxWX_7KX68^)xhBeh4K(7$z|k>y+FwiOfuUmV^Yc z{(!q7y8kbb(^V3+vbf~};Oz|g>^_hLQF#j}nt%+~O>o^0SU<;D_FQ`Y^e~t(VjWgP zXpF1!vnp->j`)xJDJmDPq0*0`GVCKkf_s$(jZCXIR!4dUZQa!U^3L7Q>B_aElCgw> zKjmLnGx!dEQ>S`;*$M8ALX7`94#^4KxPlb^!WU!X2d@yv%~*7QX3=%BN@hW8koRGX ziUrh5kxAnkh64z!{Is}49p(L^o0wOfw+Mg$aO&ea;7Wtqg8qUczr+y61?p0Dk86xD z6837BNHX@#*w7{(8@n(#FREQ{Wgp>KmIRllk7;tI$iU}x?F3u`s&m#YKCgR-`x<^` zN1`}KVj9NpJxta+;XQ(8b%|pY(D^(*hhWm0o@TF#gSw2Opd1et_s|fsh`%w9e#SX* z4&okoa~%qAGZ~0&86N?e4`V90;flT7P(kkB99P$e3lQSN(Z|kvT`(6hE*Up5STL3s z@o!AZ+k%b`z}66)TV9rGkU7QRdx>dCVOpGwmASZH0nOo|^YVw61l_oJL$K>QA zY|cD9$RpX6 z46KUaEf!>CTn3=qY&NFwa&4^{%R0m@C^+P&UJi$CbYct@^C0Aln}ZD?IAc24z$g}; z$5?<%D0+qg*nnwdGezr)gJK;Ur_Y5FgvMGyyl{!Y&S0HtxoUAGwZgcrP3H|1b(85( zHCDnb#tQL_T9{lW3iP+FQ!JxlSro3^y^fy~IIDQsP;S~h^WVdLSboMzQULA|!C`|m z>=yT$M!Me^=JQwO_69jy^)OqOx^yKulb8w`wy83ZAz*VJU%gI%E? zJb|MWV;&%`Zh>N@#0*?kZNEfcLFTavja$K;2meRh6wJ--$uM^9U4SidJSeO>jKnE% z6_$Lc5F50+_5#<(dU!>Ce*xxXC`)iMg^R~&4OA?$ChUy{2j2L)DuAv*ypQd83`Fne zpci21v@FfS^6;jRq?R`r9Mp2-Ot8i4jK{VtRe;4I8JEZW&ap#T48}dH+gx}35g`e( z4Qc9^MYj}jXR=G8|(-TkNG$;wVBpghL_ z4nqi%OtKIKk}+fr&W*-^a{HWs;LUakWe8C_(#eBPan4Zzwy zEC|oYJoQrm_x&H=3%8c0unzYL$|rM|c%Ku>nY3UhwQ!pxJ9MfjeHf}LOy>?Yw~QG* zLg8jPXML3P=*1;DGCN9U=QwUqKs4uje|wu!Eo+phAbw_zBns|~_6V1tt| zHRb&S&KG+WcWWLu{m{f6zz|6ZgcKkd@CK0dLPl|oFcFJj_i*jWg%ooPvj8xgM!*`w zpy1yl6V#e-8EUI>U`BS~=iFviaSfVK$`A0j5U52HJeOWz*AHEIW*QX-{tuon(lU$ZHT7+n|0_A=v} zybilaf=q!Ps0e6S5sV(nL0NUbd{7V1U%rTpqS>mK$h{HRPh;EcYuut7$OLhQNyhl{ z*%m8C{3PpnL~Fk^I;+XvH}=>#s190NE0Fz)-VCW5H*~7sn4+L{7%>NOJHTk-$J6`Q^CO ztKQ3}7SK2!@ptto=UdjhA9hc)m;?~2FeN_R@3E{3XTY|aK9}c0z2WD)7MDN=cR_A< z9{P9>cfz2J?XkGG2to44X}71ar9ssCj^mVlpVT^!nv(l+O~1Uhi=oHNu?o<9{&)}P zn{Q{UOH^@AnRkCzZ^uw9J`({k|LU?WQPkX-@PE#!^JVt9Oq_uJOd~l72%!=f%2|-2 z!X&y3cO8+@?=k zgROIyD&@l+wMUOy+ z2fJjmSYDycfspZYQsM^X{#^vkXIV@ESO!Z>w}Ba^BUo!DLlmU8c_v|vhhqPK`Z!1DuFUD;xh%(ltOc1s42v*oXCxylM0t3}8li+4C z#!ujAyq3Y!x(&Ne#UYP#0^nxQCakzA%vg*}D9`NNnh@fcw4?>>riJ_Y#0k%#(7{^h zXMJTj;3i^u3*8pOtdDiRIt1u131SHdH`dI_ z4A6~P*dPGYJ|JZf#fr8I2dwuPac$t3s1*D2-Lo({PH6zuCQeN@F#=pm^W-(v>iO>mABY0BWv|hw1m1!k_-@e;PJc zSHrLW-!H=b-#nt!K`o5K80bP0Bw*0Nh@#-J_;avUO)Sq586uu(pZGcVvYC4wGBYbr z$%rLRv~LjR1?Bw7`?pvp+L+7219Sga_}vf7oGoHnIVZjB#c|xNlegbP=Jn#5CpL=g;xh3y3HI*^^Ls%nHwz4# zt#}TTd|0C+#Dk~AT>%V{wX^e+xSRVi6i&i4wdp2cu}@*wOpa0cwCQ*1+;(8t`_iKr zAB0Av3Acc~%u;Oj9RFl50OSIr%-GqT?M+IL04Q-Y>jkk%_`nc}8JWRYXtbBA0x}P5 z$7oFJk~OQ&I62)&nx5%5M&U3lnP)o7+0A1Z7(}jXUfE%}kI2%%>RJ;|uy9D6pf&Yg z-T#5Uv@2l-w^oHY@R|mhpQWgE2;dzjQQ;>)dY^V;6i-jpcwj3@3;4#5gx(&Ly&_ z2ngo567LAYmq{160g5i^KYfxL4aa?xVPO74S|ARF8Nle~x4#J;+DlSQB8Khk0pzf; z*yl|;p{&t$g$&`v>)}+dQSS(6Zzr^oSv~BVDokVo8O?5DugOZwrg>Y!hB*5vbO3(@ zJD}9?MyR{YKF#2U?Oc#2bwa>&!BazTs%V2_}Wv)gN?FS2n8R&=@DjL!aIDpvIUB2#2_Sbj7Ogxt_6shA{||U8^?inmxPE zFc3`0N<_w3BZ2wy{;aX@Q-37n=F5gWHnO`dVeJd9##nX%G9 z5!-gjqP9s|*%)(m&OxtCWYPjQ23 z!AXtF5DLsq5HTyu`4?Q8aU>A6tr$zl2H~)<&fUYr{Cinm@Y_PbP2OL_N@RAJfA$9l zOgFQoXbia4I!~ZPyml4WqygpI1d3X& z4)zTxeF76$AgDjb&p8yhg@v;msvNO}`3c%4jV7t3MjIv=imk1~Z~zlf!;O3{z}{)dno>I)zJ>~6SuZ;AJh;u@mpHRUmmG0GNb zDOF(&dZ_KyPdN_iy@Ue5s?9SNP$pHb(cy28fBS9t_OtIOBQX?4r^pm0R&a4?Dct(_ z1B&Rr0~p#PvEzAoOhSOMjeQu0;|qaSQ`=*jE$4QUfcdYyo7bt4LC803wwAHa5%;>gH@t+ z->r{+65jjqpTyR5G7~MaJ1 zuIvDCNJ7Is*=Zywtuq~d|NnhUR||@=vxbu}!!3TcO_qkgjKw*+&XN))V;GWScutf3 z-9{!R$EHF#OGgFD&oqfiEs?~P<(wTpdr77r2|1i8i;*AfZ=&3h!pw{8cnW1)AbD<; zJz1E5d28;(a#3S*H#zC36@uPR5MqKW2@dsI9+~d>Ph<2RQ;$+$rCu^5MgV6$MuE3; zE=hrSjce!M2Qdz1`BVzZ+z6~9C}UliPe0UU_n33!^xOOTZ)D}7Y?poN8jF%Mx>o&? z#Zxs4I@F&szR{b7p7&nedV{!$`AinC7JP0TuOUkO4}hsDs_qa}8otzJlnMm&>z0>w z*CUkSOqeaoBfF`had6zJ!2kJNjDzdT5{fMhKo|H#pv3dE0cd9c=N2sW5zOZvH3V&1 z?{&PVWK9e~axcg6sgOPIUPJdO?GSsA4Xi`d&8$IA(B>ZDGa?_)q4TAT@IP5MF+?nR zg7GDz80T0p&b7N=U44h$6Zfw76PGw188_$8cWTtgYAV~kn=D)`n}(4e*GS?-V+upi zuoussy4!Y@$6x{;G3q#{F3I|&#t;Hlowjcm3_8k~cd&S-=z_`O=J#KjE;qfw_bPDx z@g4+MA&12_0CSQ7fK@#(2%8Sd`IyN{q3i|_Y{x54!rgoCli8CX;|wYqO5I7YaGe8O zgq7$ap6zg(QV(3Kbd~rTlxC7Z_v9eL2W5~RoSnLOS|?{SL!@XP%MYALd%a` zgo*cUu(5e7_0trZEHs-y@TRcQBIz-ifcN#YhQ>Q~Yv!2i01JswHD=8Hs31^)>i&EC zrFCO`qW~{Yd``hJuuo#aS^n0IV|WV&rN24Uuvq>bt;%KCXBPleKx+Msz|ieIOx9LU zqZ~ehEFpjoRAe3x`KemT?w$DE0>Skjcnq*pPcDW#ha| zjv3@!x$h?E`?@$vmMsj=HUY~oKYtXy{N*=<0}Wxgl)@xggTsKR$yosC`_q(b!rO%?w^;RE;kL!?H!v1MAljU* z>6>@Me9t@VL5@G`+N98M*|36>!!QJ(%8)&rTL36eO@-e18vQdV_k!3pyVE|AS7s&v zVVqk8`6X4^wYHUWTHs(wzPvDs9hqlQY3tOTVGL@BvCG0Hf4wcwn#x{UN?5G z$i8rI^k6V#WY;|J>}8W6WIoJ_MXW@{Y{FC<5X{L1=+gm&&TIb=iy!^Xr43;FiLf)C@KJmI-Eh-)2U9s34G!Vb!;fDB$BNk*@W z3{8ck1mkr-{@@OKj8<`UH>r_ObAk(glNh@JV#bD;^L!6kfv~`z|J7Gysi(ucGc!?k z9Z=4v2Ubae*sZ&Xo&t2ZZ|#JT<2j9CwNt`5$yJqs>2OrM3Ry4-_?u$NiLqM$K6EzM zLO%uA3q02eE?nE@b`Zm_^EE zV)-mCWj19KogW!J^s)wk^3nS^9(%6}`p!CCmw8lbeBS#Tlh}2?b^#49B0HI9Hlnb{ za1I5lI>~=yL1A@|V2Cm6igxZ~1EUKXqQy*587{$>4<0K>I2ty*E>+oIGkC2H8$FD? zNg!(n3X|>}v%t#hsQU!gX>^G1IYXa1gOLg2dID76>%Awd0(@= z&-uK>M|?krMjWHwN@L)+z$j(}SC8nq%o#Z+Y(r0>aq>|PGcW2d+lfl3rG}Z*QzrU? z&DHh>jLasySsu`OSH@&W8;blCKGRUz+hok0aLC4SM=dN~C5B}%eEZ~Sq#vz%&D&u~ zh+|_i*c$XaML=_a4eafMdc01bzYGs;=?ml2&Y)S>9(@-YSUP4GCa^G1PRSxfaf?$= zxMrK|$0UNN1<=eut!o_`G~NyT7v-kHGVishu#0FMovua&)@6&pcU*NjW3&JjS-#Iu zenl1voj?{I{RYeHv_`oe@&u_*O-~YQ@fbl1$Rd?Jft56iV5m0wh|wYSpZnKfSnM$x zgVq5k7YMXdtnU~A%mOr+WV70=aPHr1=6p^Zi}b5Iif&RYG%f}XsqJREEi_=l5IWYS z%VTM?Lty8O2jX9tFLpG*%Zy!*EgOR|7^fJE1jltk@q;ut1}re{IwVL#_mu~LjppX% zS6njyYb==r5}2dNI&po3ti?awq%wduGqA!rctEAcZN9C*D&hX9@$5~Wy|cGLas*5cR*3=T<0zF;z)O{}9&>Fo zID27wPGBm_D3K2CnA2^n8-#U@X4X5n7m`@g{o|9NZ(RRab}MQo8v+aJk%`8Tt$FX&dG5IEZ*K%XqjBlcAZ7bU(Sj2&90W!ho! z8bY1V+oZi!-(q`I!%OPyjnUSsh}$wdHWrHWnedA%KjOS~$kclt_Ez@86I@d#D9au~ z5K64MY*wCg#r9zS(Ln5R!v+i}wzUvP0(O#;GsuQL%73{2+qegZh;OqG?G!PP)12Qk zgzqW)#D2A3Jy^kAj>7(-n( z8fd2{&r#a1VSq_2z+G0`ArqQ-t6taz?u@)48xc2og1O8PM>XDmm$f>MRo3lOJ4&pU zWl6e8F91o6C$8)MY15Os0b_Yirlf99GvN}v-a?j~J0<{>+0rHCkjlVVRY7Zxb2SWr zJxBv;F(wVgFf00!qS`7JGS=`YSUQ0|poCwvIOlpZO#V3Gn#%qbjJ^ABn7A~PzKF3A zF0wy}%|ni#5C)+ll@Tx-u)}-xdcaFiE`&BjfI+WmO+sUKO_n}nZW8gZ5puLoj?N-4(j9~>K;j%-9dG0Ve&jG2{BNV_G$9oAm(C!-w2f9s%uZG;z zU1A<#NHNYE^t4TFzYaOTTaIAL4Bm_uhG1HDm${3T4*=AcgfZmuFtRFIvQ6{DGw3UH z3Ui%6c7TA`Q_di=7Z4R)jy#>fMBSybkHP*hn+-BrWr38W4qrpsV0_u1x4mNpbh<8; zr-EktefnPYM+93utM`gvgll_0J!^ul$S@&CBp{UEKsWvl;lR1KbV*0A2Ku!NdRTYf zC*X|CAooyy*$m!3+=9_apfFy4_Bs4L8UV<3e|A5`3Kz(8!Cd!!>HyY6K*^Z6EcfBF z{HNjCWdj|n*JKZYAFMCiLEiIA1TBX6=w5GFD-zk?VBg6o`g{U8&ro!YvJV}9FCL%^ zj8rj#_i|3e2A9-nWG1@zW-T;4yos{8%X})QolSHQ=h&ZpPU8W+hD@^Kj2=JkukjZX zQ+O`#gDQ+`64|{tU))bZhYP6-eRi3-SiypSE&UTaVG)D6@`?%s0#@D>UGJ=si5P~_ z9PCk%fw3wZQhTrkE@Fa%ZtAVfnwwu1p_>v;&>q6Ynknq7um+2932+#sW|mo3UqAdh z`n8V{06j0BMYL$(vWV{zMcC`^LKvoDn3`)8XJlY@B#4{D2VerzEOwFw&9iw75^qvL zzv%%`=s>?;aGiD!LA?m~Q^r`~fcx-g{G4NRbdfL{2~-4w67r!&s-Woo@|Ey@IU?cVRm5_@I{G*!+q%aSp>zmFpAXm0$|v6 zAU8Eb`x_*jJ|=g2w4vx%cuuv32Qi z|1bYnVw$cHAGQz~+xP$BKMMU*^UMqV9*LQ=IJ80231VKx24}+folnCp)KrTg{70Yv zD*VkKehULfvPNZ_%s6TuvPMeDb3T_ME@=S3MpzgvnzkurQ4AjvfX<4bOf@Rlv;3_= zrE!S6FgcBjhyC6lUQU;f@s2jx?qg1^Xy0-knKxbNd4P7eASD0>p)^Gi;Udb<>g`8Z z5+?fQ#(9Apw7$6Hb-H*YtnUI)$rwaobpWb| zC|SE6ykgzIti9xH4MwKr(@*Y(g?raXZaIQ2tAy{LuT!Itxdp75VJzrQwCHO|999d~ zZ|7wdxAatKlGxA#6WdSh+};k^vT}62JSf{4A1NJ0uI6EY}1^V})91EkY8s)}O5%@>atz0O5KI zU?Vs_QQ>De`wfJ?ZsR)h@U|AjHgSK}QA`_SE7;x4j|HGfF~= z8R1)N%aMh&zNblPX=+7gQv>1ggJbU!9@#$ z)WcfY;hM-x16Ti4=1Tlgg>0I?QGTF9?d>VO=WX9{ktwpL8uI;@!5=sPY7tZFQ1Fajsh=UcN{Wsf;bZzd=@9bd&zP+dPm|t zxJdj*kQ!fo^O{WgIcs73ogO&#VoVs|IlXQKX2?+)#|G!Yxs(~xn;6H(`(jc9q=@WT zhx6eel&2|lli93s5FA4BLJT)T#zfaFa}}>IreRMD-i#%eNpsxl!3ehb9Tmj-Vw}HO zL)RfblXK3q$qd?8l`1>YwJ(fh1!9=h$T)GhB!U(2E?vKU8<|Wm&4kcWKx>O_e%9T|BPUbk(uh$Wr=RD-Scka=w_}%co{qO%r zoGfFkq~y;LK4UO);`7i&H31)%usbf~gisv_P-z9F27_iNa@ZdMU41fMF|1BHY=LeLODBg+EHXzv8 zl`a!wt0xXW^I)Gxt!tSY>+E&Q2ax*Dnpwlkrt%knsTSb5LA*|awP>?(FKbwjSn(HR znH?i|WK#Q>vmxBaC&U@8kQh)w2whSs|FVOd4y(M6>m>n2Ie={+&o+Q{TEH^4LUx^v ztqU(p*?`Bm9F3u}Tfi7DkytVX1=E7T>%&S&;gZU67-R-jb|8kBjFA-7vbSPV2MR^T zfg{fRGKVmbqr?We4^{qkz{?4YkYzzSC>x%XeZ+ob{73w46?)lNykpkU#r9B4;x-%{ zo(PLK-l3#Pfx2W5L-jsxE1vy_zx$7|4%67ewQ%dxpAc|-6QRMganB})spY%Yfx7JR zJYe$9(rCDM6?eaGaGtNs{P&^+#)#3<4L^kIPAa{82)n}A&R{SU5Uz3gfOSTuOjFc# zn2bLY9Adp6=41dMH_A24LOx**ObpV6zV=`>JoxM}>`o>u+!`UEnzaLTT@tT1#$HVJ zP$LZmySchdLE7bT_rq)Y)&a)deUygfSomNt-2U_z&@YU4y%N6p!=v!SgVz95iVeg5 zi~<_RU}Q?LY?lBEa0`GS9S&jp>?&dr=n?lS&r!x@S?9G4N{(2lc$iqjEMsY5Y#4Jo z1+Y8D5IY1I?QPVzhaF}35x(Ab7q!qq5M%gGMp7a-5Ld0jXPy$ zay^7Q3hTFj^NjKuG>pGOZNq{rICfoyKC;g%^xiDs`n6sh2?>WVl^GZ>o|El~F@ZV4 z{+k)+>=Ng80Wh|Jbq5NjRz|h}=3176Al@%Y7Ckr(z(O0a+(mf$;M?&0``_b!9%nxg zPe}H4i8$XrKBMh!n20p*DS*$A5(DAUM95LyN@SyPj$=6@47oINC?TW9ITQp_T7V+C z7>F?cRbm`ZoAjJksUj1tfL~=D_9}cHWtS4*agN}DNRUA}61OL-v~-=MEn?Yn>^EJH zhh#~Q1C|V$-vc}Xb1;aA4P_0CAxs+QWyvU5%sxuzT%R9H1cp)fa!naQgY>(#%rs=*kPp1}b`COR z6u)Jp9H`($kk<92bziYWnb(iwWPWUsHU&Ulf6aa*m8Agv^_mC-yb+L?%oF zlxeU*>P@d3)jUa$THq1SoB$B$V_&WcBS7#fXp z7@5&G>n2NxKxGa&+?4K$PsIMq;t5N$Pjt`f0)HC==XiKAj@fx{!nTQC)$<%vz&lMXp#mC}vZwV@)_7 zBKmQiB;F@;V$CWu6GfHQV)4Ok$&w*MuSEcT4;H5Zh=;mGhADgR%-Mzx-XKN1YHigi6oN<>Z&i4lJRt56B|4WsJp31xZ7^ zR={QIj(YmyDX;a0#W`FnSn>=WfJ)Fbl7=iGwh}jXv;+`BfBirI z54cXSsu4V8@uO)nsDh&jv?w_!;L-E%z( zcj+G7z1MEv3m4_rxCK{YwS6<@FMPUeQGesL68CyzkkO4S5wh*rV7mUmM>yjo(VSi2C3v*8PE6N9Sb6jyd_jTM-O75nHHsUbz*Li73Rnr%J(v%G%qXpY>peO{;PeKA~pe=gieF0eZ&_X0z3z zjZH!*5c+0v7l_FlBBM47LuCx;0M9Tq$#wP)!?Tyveq(Bptsg*pB#;Omkk$?6loq9%Y}y5-&}R6W2&9E^&V2)J58blh`Y0{2kU^+nR45L(Fq3c$c%?Y6%ZCH&S=GVjzjwN#_z@Vhh zIOm@=(}nl$_1nxW=M?a@gbu=P1+W4TkXaU%)wO2r$2@>94S;N8D0lciZL{}?r%y7* zB2&^K0qY#lxJt;)F&Xs7@Ll*i3da1w^`E0{@G7h9o1kD*aajfdx)$>6VX`#c|{qX z%b_V)XX*gLvX;qS&tNc5Q_7}9sgS*=4`a6JCG*0x;0a1 zvC;bgI?S^e!)|bmS`+mMnS74!O%YcZbL7ubv}VQnqo7EAB+C_zZLa_JI(oQdu>4&GB}h`X%ldhp_&Z(>UWy>i zJ*T#prP58TyYrI5&U$IiF@iac@pJYIXNdg%1{eY$xhTI2Kn(%Pah8c`L{|!!BsyBi zAwigD#WQCXXe2_AX%aeuKXrEmm|Vl_2rwPH^B3oW>qJ+nzj<%oV{BX-&mSj--(d$? z@0h&F^T~w9R$bgfRxds~_c27Ng^rPDFqX~f(H-nb;==LlbpblpLq<3@yE!>O&eeY= z>OFf~P18KTn+lXjiliher;}5is#B>$bxrs5?#%A&t_{z?yK5|9z%LB=l>z&^_(S-W z;f1ja&+Z!2J=0yD4%OY2!wEV`IZ=T@Bt=p5?{~2ZR<&9>6!|>&bBF8tU;ir!>NM1? zl@~g}xnvED`8IRf>qv%lBy1S7cqwEqvBXF*7hlTMhvge!Wep*lR1YkLfwiDj5PozH zEYHTG3|u7!Aof5tO$N|2`w~g*gxWMYY6CGyg~+9>5x4ViAR+`3Y=R3yoYD}3j9_hD zhaB+DnnJG!V?=xjn_NcEYxpd)Gt&szXZ@b%MOGAci3JsA_5Fv_uT;B*O&!N_O@oDd^| zDM=7LZe37i8F#P!ZqIg!H5gboa6wZ&SNRQ&(p@6RKK2-c3u5qkbtHoQkLE&jj8dw&ycRSClfYo1EVO<+Y|MzU5`ww{he5!vTg0q6*;@t933a2ojcO{v z{1q&s$!l8IXDF#7m?Q(wM_`9aSof2-*dE@yfvX5sX@4&juK**&OKpDoQ9Aizl>`RL zmY^6*%WLVzqaOlPiDP2V>7F;Sr*E7H*5e2uZ*i9F)qC#+J5*%-Cf-KTvFGOAf1P%} z{v@?u{4ssG^OS<6W9d4qSpy*4VxOF5=oYYp%Y{p4n>mR<}28Jy${*mY56c zmH{vn*k&Gm`{HFVwOYX!oV5(GUP4hynExBx56 z?GSsu3k!1zn9Xtq*i%CEbG;X?i(0!2zNVxl%|QTr4)_Ki%Qzi z4F$1=D^mi1MX2*{Spt5o;;(nj`0a%=33~aCCE<}a^ zUO=J0SdL*F#;Z*qze|eEW-u~Jth+3JNp=9GFT_d5#47_UBI7#b~7!a{{$Yr+`ErNZXr7? zD13oC_goJK6QGB{>l*uGpq^fBazHTfze=4`)u8ui+4Qr!6pVAiHVa}BqA zW1EsGxUg4{-DaR#sJwWgnPz#2M+((iE!UUqmb+?%J7JAM8luh3#=i+dxvcN} z?%7F*LeCfmU}>c$I?(m9*5okc4F{JCa&0~@fbucK6z*3QK0A!D;He{KJxbkRTnVHs z{5^q3<`@~|94UA7$_cD8x(JvjpCcxMxe&l2Z~*q~dv0e;s5NS3JO$NAV*fnAIE>wS zwiA;bsje`%>x5F3aJyT!tHb_wuJzgo_|93sLp(IAcy^T0LHM|Ko-y`-UQG9c&!sV> z+-pPN_;eO*w(hG&lgSZwS8<(u?HoPnIkS{o6Fea^3CA7#!*w3RQ$|@D0%5V1aIW!wAP%ncO zz54;tW&^6z7EcN~6DD*aDWoa{qhDKCCFW%vcg}0pm5^~Zs7#ScV7HAW!Uq48mP1XL zwQ)Ag5Snb9g|TK)-G}BgTA-#qXZVajp@U+u%!M?%Y?#7USES9@&5$ySFiI$cAr# zoO{sh3j|e{a>U8Fy?G`xZL}0^rDIE*EKJ)zpJ!lLmIQGVG)pcE!(iiaB4x1*^tRYB z50U8Z5w+%I2lIVcQ0Cy0Vz?OjEEjGG4%+3Z3WdyJSVIG=_dfqLT@brb=ierUj6v0w znQvXirG?qF_~0%JTEfbupepk^#(WIHrrC_1bh0SWdVGL@J32z}ol*yBi&|>C#1USk zBJ)0p^_vB7>NXr^j2;B3e_2vpBjXfb)NL&rB7i^RV05|nD1vJkD?Z2JJ0+{|5=$DG z2#d*Du})UW7W;!|%c75y$LC$Z?jfd64J&mF<;JJ;EsG{6;0o|Ujq;FqurBM$4 zJ!;7v!`{dOt=yOk#^v($=V=t??d17qsrlqnYB>ruLU{eEh{ekXOGf1b`?evy9n{0joFx&P{YGJ}WH?$$6A`>I9a{tkn~gI)?}s5@(XRnBwmS zjn@Dj3T738ua$T<=fnD8vdLA2Cu6rMql^P^e)B$qqU#+KC0J~FKW0hcF`hfULT4+^Z zwg&YHxit4T3BZEWoZ_H!g`gSnL*A-svB~R2ch$K?r~$FfZ;yV`ePWJOy() zIzmHoUH&H2;4KKbhD^$ld@_jv(q^2d`kNtYzvVgB&J0lm7R7Xt4gO>u4AVFzzHxUS zMgV1Q2^i~QO#t2}`Hnam#xTj6WMRmI@j$j}km_<#*4PKRjSJFVi#c577EV_w$$%J& z5MBW87yT`->Le*V^KDvjY4^~cTLzL8;&(HzF6fg!=e|) zE;U^Mhe;PI>msaI2WIh@>9NV>kNc{SGa!jg=7Ue`(6Bi zZs9ssEszEFJ;wYUaGs5w7T_{7-Zj1r(-HWo-z9kbivs{Y@e^hOda>I+X8Ql_c}Kg!EJFCE(zy|`QZJcIkAOfjLdIvj%QO6LEt1t z=^z)*h(oj7$P}&p2J`INvcHfWAOHQ|GPj(0m|zRhyRP{H;S+62k@1KD#Jv>-Bw2_kSJ(C)}_xN6wo16oWbD08GH=D2hs*BrLX|)d69XSfAE` zDj^V!5jw-_7X-KnHYe8Nws&R`KlB789rj^0ERttw3{EZznFmK0ot)8kj|JzS95M#Y zCiS5eYHp$y!d{kZoQ-Ll4Mob)7JURaw`t`<0X@)Yqe(nWi&jst0$O^3@R{`EssY`& zp%602JFZJ0JSf%LD*`raon>JG@-PX^R#+QN1v-LQCe9|{$O>2tTn@TK*`TEo3%zQv&&DA`jMv%z)7_*)ZzwR3WmMgXeANST*keE~2-hyqINQ)>3i z%F-fTBZ%FC8dvyKar-LVmWXQ^W@2oYu|^!vIVisakX~JwV+_S`Ntlgw1Ry))dZUcT zq>*KmkJlaN&}La=`vx`ulcU7nR#ha2}+&y8xwK6!`?PV#S;9;ZC`o&X-pxMEe6u zcYH)_)?g>7k`x=Fe$(N?LShEWFNo_`vCYjJ<`QTkxmM-TA`mU$*m{|yF7y{ zk$Q$GsSD=h&Ye|aQ4r1;N>6duZV|JiqOagmogfQTRwx?*@ft7#=4T2pC-Y?zLY0`k z8i^P(dqtV8TLhIK!<4}Km1YWQ4Ul@meyYQ^nqXm_zERFfc^2T!+6;T9aAD2C&JDdc zpWgcCw_xCz(;epc^Y!%E?>{Gd6d;Iewnu%p5#qmQp&VtzEAw;2w;4_WIKVKpG>q)# z1=g^@#{`WnuJfE^jhkx=bfdTft4kkb-g9bM7S7(zb2>b5E(&B7&H?t8VB{5M)on5_ zhlnc#R8TpO1O$wbMasfV!pbsN(c_r2Z#}TbPhO$F(PYS;$`bYP0I)N}9p0z_PLY3b zq*1G{h2e#k1}K;`MUnz%Z3soIe5qn&bU0HQ5OzVZK&^7r=Q$(+DBQLP*DB!jbj!x_ zhNT#)!kGv3MLkKnKeS+WJR{REeMf*C<9#_SSk?e)gQ-QAWVW}mJ3c=VZuFi8>rj1h%Mdrc;oi=R>YXEz_ z5BG1}QkSBC2{hRu;ehjgv`4*9%fFC~dIyHQx{oa7%-M3Vwt1S$>>U#ZlzlQLN3d?j z#|p6RQnG{F+eDiS*sM|JO`UVjz^?%16Z2(4A!y9U{9GU}Zv5bf7;{%?j|7@5ArzBS z^MpN|rp5t`AT`SVhyVWHr7!4l+CAK$5&;>?$m1M<`;gdQz2!}W>3A97KSiAB+)A{{ zn?(NV##=@Qc3}BkU`wV1;ejmDSh42YCFDUcGrX2%UR2PMCIjGN%$OdH0dNx-mtdB7 zfn5z##hS}18LQ@NCc2?XNwy)Mp1-;F_GHyXX9lD$Ho=JiP1mK&okoNAmqqhiFNWp2 zS{ST?K^Y^z)XBbAc`ShR_dx+$){Vazt|Yi~iz;jdRuQ-78fl)#J|Zy>gUUC^c5w-e z1Fw$^$H*M#thj&FVHV8Q%aQ?*=)NDOCa0f^m^`j2`=s~A^Bp&gzp>wtan{olSjrR{ zR;3{%%W6zv#t;MIF&(>ZNv|i!v)$vMz8o+J?^KcVD&QOAtVPQ`#v-B8VUGrg;;zx> z7a-Y#v@!43ES{u-&JG@60VPgEj0rLyf@$#=w{^G!%?EV2+$O zfoQu$vjYAm86|L)t@EcHAsk~cb4;7^m!WRU zuz;BW7v9Vh6_Rm97qond^n5PR&lZVmS_2FqJQ1ZTCIztNF{&^=(xWmm(xXinLVF7~ z4B!Whi2Q9Xzcf8V3H}utGRSI>+*b2X#2OEw)ni zzr0NU>5somx85KDgK<26`7(XWybd8O*{NicmD#9kxR(GI2m)gd7GancmhYq=0X_xf zR|wk{@m5v9TnFK+&;DR{2P+ijhSv}HIfC2klDNDahiw7j$kt6y0VL~x`(M%|Lii5# zlx~uVnPX1MxXX^QltIccptwvCU?;GZy3hKs2fEXBi(>-Ee3FdG^edp)e;n575)6XY zD}TL!**d|>-zAf=w!@gvn6@N3fPG4tSHz$Y%+CJ0bL&oI8JDNnm+YtdC;tyj)E$HZ z0jI22>BIl=C+ThG;G0iBPXF6)KZ4cl zqz~TwAl>DVH+MJ5h};9vP~Q#VoZ)YI7=m$_hf|cxv(L9MUI14xPSXIEGeG<~?yi}I zQo6Iez#his!m~EmGj)_rmNE1IvqkpJB{5@WzIKTJ>QfP(9hh3jC?>{h(nEpwo^B8) z#=Uy%Yh7%MTsKGj-t@|B>QcTX!x^~5njEw7}tj=Oo zoFmtXWQiWCf=S0yB3Z+@=x~{`xAsUx$pRp<)IziKiwt-GL*6qBaLl1hduGo+KjJxY zeF7w#u*g}2k>zM+ZZAbyqfuam4>Dkk7!#m`@&tok4B~z*Hme6vZi(nEi63nt4Ml$G>IFr7a zdG~61lZ?^Ut7h+F+p95=OabFU?_|(Ho;!)A3)W3wtzl+m2!_JZ8h~f#DOg&u5W~<| z4Vl5eb@l3!jcO9eOXp`Kny@m6?|UOk!COPjH$`7kkiV9h=vovn1{sFAh>J1wDFQBs zMqq$AKyDlh1)BAEDl5b!j0#*3UcAi2$-XIkLE>y~WA20)Qmua0hw31xx9i7%C^iZ! z)-^_jzg3Bw(F#;b1c{#~cvde{pDZYp{vW>^sP}W7TJEu+TILxR3JJJ|AIB zBl#Ij%qfD~c^$!bi3MBa-dWsh`&(PoheIKAtvX|EfV}6~a977y#FWXnFtH;OC`bYd z@!^~8#QzAAh5gcfz<)JeDa;SD3#N)VB!pI2s|s#61Dk-4;8G=)g_TgSS{KTIXIpBS zu{qAzM|j@|m7`-bb43rS3c=D6msp7&AVOl8M`n`su_lvEo27BX@6TGCG!Iz=lpFUj zgI3yJ`Ho|mxz+MB02?NY9aH4+j5FaE^qJiK$Cojm3mjh4%zFs-OI*Ph^#fRir)iHX zm>FZ+7lWDIcmWb<fNMx>IbCR+?UjSTuHanl8_iISA9iyhd;^ZrS1}&(|@yo=W31 zmp4%Whi|wpc5%UcfW3mCvYu57aNBzEb(&mU zO*Q80kl2TQmPDP(QmVkXjZIAxPz`H=tMZe7_|E|TJ(^&zq(?vec^U!~UjOrdnePAd zzfRjPzoydu*Xc36A%`wTVKyH{Hw^=Iuh>5qCwnP#x}R3Y%V~{RogB~K!$>)SjmGT3 z03nu$!0s!aFIQ&&;G&aBnM7gvjAQIW19@Ax3Pxe?ddQy!F-0w0Sy7CW^JpC_*UJT) zU{6~w%T_madboDE#y)}dLx95yP3sGQf(gdaLjcUwb9NN) zGc)(Dr-_w?^wv+2{II*5-)_Qmz{sNTPqtB{^g`8rJHecmsn2K!6JuuKW0Z%XJnVK+ zWbf3_k{c9nXe<~{u>4e3z~`;C$dJH zxcxXdq3~sn1goRClCyxeENo733d)n1s!Lj|wZ@w1`Lm~LGv1FPQjWl6jS{N|dSMSL zzKuok40C3=&oJXRG{b?aM6I$PiCtLkypwl*7k9_m)pFM@>gy(|*n`(K)ynQ+pq>-M zsuD3-#Xhfs6wh~`-(_QD`U*W<UxD~w9*qq)X$~#17PG92+>j0Cn0f1A)C)!(yK$#23{Kg^4(G>Y0mrft+LJPJH{S-^RQRiVJftxK0ix_nZz%k`=O4g1#EWfcqno zG|+MW945Tviggs&$@0m*^;uIx2$XpOyqHrqw@jQdxR&;?fVAh#|2b}lVmx9UPhy;S zey^S2UE;i1z+X?|1$x5}4d=d%5$DLqV3AMu(OVEx+>xi_X})`pWB0m-jQE=C; zK`Tuy-DynU1&7ng^M+bXgvSG4_~uaOC$Y%M937%PM;>G|#&euumAmy2tvnBm89;56 zhk!6*atX==K$*_1;Iw~X03ocf3>HjTV9340>S8R$a@hf)3Oh4P(6KRqQpa5v9ASU; zonPPfR-(>4OsED6`3|$&_cD za1Veb=Aey$1$?>(!u`ycmPk2^_yO+MtnXuWACTw*V@2TbCC{vtCSV-G{XC8`B{V5q zt7YbPjyR+#vKz}3s4Wu@COh258orW&uv7tKe)iR$g4w)4h)hvWYMetjMUejV(kub% zo%H1&zu}s+xdOObXK<8pC_mtC5qe~bu@B9Z>mvlq9OOCnl^K9-_O3}PE$01Y$C-{U~`j&@RQ<0)H>WDNxND6CNbKYM;+ps33BivGmTEGszG1xMNwCe+- zM+>}08}Kd2!o6&lj3LgfuJO|BB07n+T)7>}sS3Ngz5WW;mBQG>Uq1Q6R}}Dm4v0g{ zG3O=XvD8Sq5@Z~WyK1rrkC{tb9+jEeK8@QCXnSVd;W5RPr^wQs(Y;H!#|(wV+tj4H z2H?%}tXTl@xV7H20N6hn zSO>w>Six8?3?$~>>04>jR7f~LooG+dh zxg{e)EjRYe^ptL@K6UpfH}v>*s^cQ)6^|f5#%Y3mF@ap;vDw+I0sFDX-mW62&vBXE z{@{buLe4boQiGdThPeTdv`T@BZxq9FI!8<*^W8)N8(v`T7?XNhF!~Zq%I*_tb6U&q z)}6b=?oOobW7>wH4>}lix%q2^Vz@@skh1Ex0CfT{{?nalX{yktvFz+cz=^9BN`d!$|K)GI8hgwA;lBp} z&VBG5v&QQI95Ipv5rb*z`P!FUB_D$F`h9#KTn90`xsB3AjbV|39km=%y?b(isLcE)2}cx zSUZFM{q|W5?_$G-p{<+Vy{lW@=acF7%goyE!L7P=0#e3nsZWc@E$35Eto`6MHEy^u zx(B7tmqdW>b_0T@hP6c$Z;dMU9ylwy%X+Y$EDV&ClY-(l3rmntU;-h= z(-Dg?1DkP%m8Ie|V`iT?A^QsoL^>R@vm=x#pK+1R0FlXaQLIl@M3@$!xjB8DY!{b$ zT?h}Y1P_>F7QIdt6a@AHgy#XwwT4I*7&B0n>#}gVA_O*kh^68`o|%t^Li2Fi-X;(G zA;S}ai3l&Y9W?lW^*G)?WL^~hvXTUn!w%_Y8G-GqQqn^vvxqe@g{#JRnjDG{m=J=K zWn~i}+<11&a*QI-t*7JrY(O4|kd5TOC$Ko19GFEY>J!{&_WV;IWf`mt#yBoA1^{!z z26R&*j9tqyKxVrOBf`NDh#06_CsxHTyE!p>Ao{#d@GrQ#;-F1n6;;`<21WZh<9aeE zkA|)l)7bfpML4rrN`3%5Z7pTVniB%D1(xpL`;&99xb#T<P>B+NC2=FFGl6avh?`cLXr4KB5 zaoONJ;U4nPO-_tHNQIA@%L&fac1C%NTJW(NUg$(b>d zp33v9>Dt|Qa6whl;+bD?Ild#Au%OK?R0>mwtgO;ro>2v_)LU0 z>xOs$xS=|k&oVBu=m5g`G>K`9F`jJTwkBA<4Pz$&aL?Yja}5OmL(2Hic)l)xR0gKV zyxCPkaCgieHhbGnG=^SyPPSkQ$JxuW;6u#q4D!VD>6x^p+a<5*rnW7Y0#}C4V%0-v zeMmUs>By9%JBEdWuO`{6cxCGJ;XUJhCRB{^Im1Bg7^4fz1RLXfWx=IeFrdY-+O zW&i1>(q%jfus0lFh=wd zP(1;8)-T7N$T7|~Y^(*NJr8Z!mvu>?bp{b%gZBurtRv+Ve!0&EfX*=Oq{F44D|^WtkIjPDF(PWmY76I_Y5 zL!{0!G8uz}H4=>53ehl(>ty|(?eAlVD6`0Jb+4H$#he^-7TeSFY3ZH!(hynp&%Sy| z!dJo=*~OF0K1Pn)L8c9hon=pWjV8wZFt7PHLE!Gr9+eMZV_4^%L%bFMaQh{zXb%a6 z8Hdg5Pb{aA+1s=OBrJx1V>+NOV7ZZ9N(tMOnH_shIVOKvlTQV#OIAbN#5S{s&48Bd zurTjM@0j^Oe)8`KP-mWHd6^!5BKII4ipX7EsqWiGlZ$;{2`u@43eEYHY&kg(_o z6+dzux$Byv`9t1mTo*5Ma=Rl$bArfQMHO?boLul**y19Q4FFuWEgBv2f%r(g(3q9MY(BGiigks~Ze<}(9n zamn96UqR4#7Im*1VlZ+7>*VGYA$Ir{C>gtEtu(VvY`0|k`#DO|K?r0-+!r!PjLgiS z6BM8U$TL{pRfMLo9(|Pg6c5gfQt%gWLam{Q0YMmQh2$yIaKd{ncxMX~kC6lBNXCJi zX2C~cUh-sIP2e(7pxV{I4Je&z3`q@OWI?O zq6L3f@J+ecxca&r>KvOjhueoiBcBHW!QQ`#U^=7MrUxS9N-_gtaUP7#>VglxN4buT z!$srPH1cn5LgVU&BHoU)c-rpp$=Lj|17(zgLC;ztDlpwYW;6;m1%US5^m(;jOi%V8 zS;iq;+&qJap+Z1>Y!a^FLGPnDPwA!8z=F@9Sf1X9Z6QSDs4p z`;seJ;-4=043ZW$#Wrk!BHYrpX0oVf6qO~Gj-3J!GN+Vcy;Yi@FBfrPw9{u#o*;CY zTe2>Ww<(W;z|nOwL)?y=d5rZjPSDL+(5LyE}$Y5i%MzIh{M4&P6!H}6tCo_G+-5-gJ> za*B)5D(*LC-{m=BnTP}W55(ro@$5hS>3@}q-A3B{@)H936_~VwgB6j%RdCrzUAMy7 z{u!*BmFA(~san7O%`U){ax#-9Ai#7MH;D}b+bv*S%DvU}$uIwqe)Atb4x??7&zd~7ak8K8(FgP4hd)Gsj;HPI z9g>8eVgcjIV_iHmH&ANKdQ{v$AzrXaAbp!T)HTGmNDJ(b9_Ob=6uR9G4gfPxYrBL; z5U<5q^uIU{oH=_%mM~m0MOXvYtbyBg0(n4yAF+RJ&LBdbOm8+yx;R5g9RbcbE0e6Z zB^V5L_iqRL&2%sfSb{ON{#u^>Gq*HBKUfkfkOi$1#*Xozt|T{tJp)((NLyM%;5x&; z8U@H55yEf=C_jZ!)D3LujvVoZ%sjUdEV3RvgFt#96p#v>bu*4T{44_`1Mr%OiramL zWDg~WV8Wm7-3~yp1}itrn$=iSGx5zj1p<(%i{ZsP!BH5>cqvTQ!S+5Q+xB;V_a~TW z+}(^>mex?489@70TUMHHT<(*Tgq$#rHpw{}!bhCv7oTpiUx_6nOvU1|XSlm_^o2cv z-7#hni!+jiUgCOfTWFj3xlzWo!nwXdxWfhdV3_?5%%r9ufC8Ch;QA>sil^-3yYKys zx^A=S7ax6`Zc-WG^|$M=#K=8lp6>MqArEDqvB3WBU_9Ekun0rnBOcPM`Z6-*3c0zD zS7{93QJjI%r!xK_#+vccRq9GkEZ!$}();QfBZL9DTE%e5@jTcbfDT5-oLRzUgd*kw z3iXA|lJl+%7O)s^u8inGN?63$@DM~6#vlwguQPRVp8Vx|ccn33Cdm{@wwhmD=9=hW z)^HdDz-zIjF~7WppE7jlJM~MfIk)HD{;vM~{_FpDTlxy~%JqCQ-HeWvoqwDCR_-7ypH2@ zzI11nX6u8G|9 zJsDzqXJ#RQDwH#@Mv*nU3Rp80teqwY#U6U&fHWU`lDL);+%g=gsJNcv=^LPJ8({C? zQ=HPk;ljg!!70oh+tr-HLd*jKT!=0~ea0}2b9h=XEgqU0u9^`BQDQOXsR?(;V4rTj zATtObGY{BDS{RSzW;$RT+yM6(N`eFJLhCYQiUD*?gcHN~T|9#c-4xb^>7e9{Et85h zHq5KExUUHcj4IfRsKU?2%qmoQ?>UPtcseJxPD>;JE1!!}1aSa#H|u`kMPFyq9a{-H zR%S{8(PIJiVOvuVoE5B+pv%hb#)%k*)WgqVwpe7F#9`3q@8O5$=4TvSEkCBnQmi0Nq=<3=KalQYT#${>_ciq3;;qlIt$kuiBa zGBagi6(r}Z&z#NZwZ7RCqkwmgE~i-N3YCpg>(sUokEe82z^z&2S-UpRHlva808*yd za|8*&M#8+{qVU<vF(qNv9n7hOnSuDb^A%Kn%Qgs4&t5}6* zkri>PtS&95+55L5dwBPVo{F^UnIvF*cjue*>#zU7deN2!VPT)Y8;{;gH{X3fE#AFD zjNJm2#}}~l08_*?)plN`=eT}umvQd_f{&hjMFL0z;qBpPZ8&3Buq_eS1CTm9#?syA zoN@jHxudvi&Z|}hr?$`uYs|WpQLtyP&eAY$!wHHCkMgriHbenMDf22Yq~Ju9dFW7r zq(c>a6|!+uSL{QMM(oojz@>-s?r=>F0Gpx@IuvjQI8*MYaf-Qiwx6Z11#&a9^b=m8 zh%#0E0VMj)|Kh*>ApPJ+57X0+*I6T!8^OdS0OmO^tdIZni>Tdr;`b)5lML<{+21~y@NG1t%$I^!&#EAGgRqE8|IPI>xJNZo z7fCi6;rB5>Mw{51?lu|fyHr!hkdjf+kd$%LJ)~E_Lco^QvHs*1i62AwaOC7IXq{)H zT^>+Z&Hc)BMf@vc@Ju>Sb+;$a%IS!2jk+%PJ}=d1xid|RGR_*VPwP#3UlpY8b=`LX zE_kg)%%xL)?<9GPB8Gpl#g0sV_lH_zyza4kLg^lLtcOhmXD z4{GdkkP^-l@=Gs{F;y4nER#njkYnc%3i~pAfZl1&;22>guU>DWn^?=G`zd$r?Q~51 zTLxf;HN&|kz@55$5)!)JMldwYDzqeoHOcftTyZuAD`cn)E1r2fYAVw7<8_+sfqhr)5A*QGZlVb3=AgM>w`aZ$wTll&eR$SZC4pwlVG=>E5 zQqh9KH86q2HpYQ$!bC_D8n~|o-hoD*E6>1qG}F-uN}LCq#^TN+I0QrboQ)N*d9&Fc z6#ztJrLYO~m77v+s(S@RRl{#>EYKMnJPwvYB20$YU`DX!xgIQ{!rquKCy5zkq0lc* zFcUk*AvL4b4XVJrKuJKlgn;mD^l|$NDn-&vOUe*5?&8O0cSAB0EKPtX54^(1*q1Wy zPiyW8BCMMy@Dhl5Z~~A4JQ!*mI$b-8>ER(`5ftVeey$UX&b2IHtpDBsdyj8M@QFWn2_6(m>DfAFV*3oMd1RSmpo@)dEX?5-b?5%S$0&_Hivd__%GD`^p z`=a(QXd{EMpyIqjkUAE@V1WbY8p*(hsNfme#2$5UiJxrlaXmn)hZkwqq|faoAkwv) zq&TbzJ0%o}F%4Oik4&LioUUmGYYFl>efmUCb7IQ0&x9(E3Kntir^SxZ<|@zRYCLY8IdzqkFi)`o#Re$A29vwxB;W= zeq7+pcaQgBLXOfIYcL8ZwuRgtduX;Ym)`mz^}%q>8G6v8d`OL0s##p4o|TiGFVjBZ z8d;Kuj$71+Bkt~q;Av}zMY#_Sqsa*4xG{3E6`+< z$A7FZ%NB;t#`Z3>0Z%xPH2H977nAFuQr;CC5qDF`h~AN&W)z8yQZb)&+A-gd{SbuowW6537PU)40X{ zsbNSIJ@*~t0S4?P2ESRMJp`AXCC;HGKi}GmP=tF6GijI-MIC&qW&fzA%>-BUo;DB!iooLH`=&u7`VVPqN#WA`&d8H#ONV{sJ2Y2*2`aP_I^1v?t7 zO4k7sitKewxV}Ikl(Z6;wH17!RQZ?Ugfpf9=5v1eSHD8K$eXc-aqR3v@9%v)9n4Dr zDzD2kYFL+MD6hpiEf6m&>A(f!tV9hv#-cmfI7Y-CV^NPuOjx%KrqXl8-gLe-!oAlx zdnU%hzr2Sa)x;nBx?8@(ga>tv=OAkB`97X6iUY?LtgZpLV=RtW{NeKpp#99?-|hu4 z6Z5apXjoI!zlG&>ot#YPk`M5A=h^w>41IUa=m5oG94XRkYc(Jopv&QaKIM zc~Pu9mziN617lqLPwJGHvG6Jt;l9vEd) zt1xl^mO%T=^CSxGKJKqoYC?8*|0)qXfcP z#wE+Tnn`$xi@JKa2_Pi%?Ip|yZo($+r%M=-Nx;P{pdvTT``CP46BvjqIiM6l73C#Z zo`f;hqMXHQHonOWz%eEZLWF${n1(q~aMyA9lxe`-0bHDru(GwU1&{T|vX@{*%fyM5 z7l}v0+EtmX-G4WI^X03wMG1^5iYe!uLDv*;CFKAj+LWOH06+jqL_t*hedhc9hYwTb z*0nTF+?lb62m5n;H>fps!QIo4e!N(6DK1lVIne^(_ z>$I}G3X4T74-@zmi4WgCT~F)3`8(>At)%(u*V4lEo9X_8N9o;ne@u(4H`Coe`zPtQ z|M~w+uRs4&nqCH^u=j>>5nI2F9UC?F4v7n#0~}8QL~Ll?QF*f$tUai^W7N16=A+A8 zTy48{awo5ZNF zY=e7$7#X2E+%p;(*a%=jqAuWbK;9ni7=^3PG|#geg#X~q8o->`M_7R>dtPNd!??`e zKBoJ?GWF(G2XtzXN96&oh?C5U6wiNOWM# zvhD@Ot7k!WKz2*!HH12I+u`pFlj{MnaS3)=&o*5P#=6KQ@3D)l|3xQf`*Gj)hkR7|N>*pi&w1{Ka~7@UgDmETxM)hXR2k-Sy+sY4X+& zQe&Lfe9BqIT8C_K{ta@E(mkw?afBlnB`RhKfJpwrkdQ&kQ~OMJnETCr6|4*65j^y% zeJM~x59u$_TRPaUOR6i^J@db)|#%Kzy+;Lg#vQMQ40B$i zFt}id`j2~A7An%e5UPL*bQoG4$-x`2R z_g9VBx+^UB4V3a22@DwyT%Y@2z+%`)_Y9W9*o2DXN>TD)uA;NV*xMrI~M;A8_O>a$vrUeGu~a5Q@~p7D%% z5eQC@4lm-_g+1CntRl=96W_Y-WZfol_vu1%Lq}zBHlF=Ay$-hurCFd3m*pphSl`Am zu4w+Y1nZ{-G(-k+rvpuo72ii7x2Wj<43}A7*FMU@+Cc^{PqV>SRwn_tG$qEWKOq1} zK)1hc>ls;xvW=q%MHPJ$TEKXTLs~Y;zPM-4HV;^fLE?e;n_v$=cyKM^v`(?MEx=kN z9?k-$Gq}MXy}y=z^5Mg@vwe~tf4xCQ<}ueI@LnL)qN~F1ewQwO|2vc%a}Cf~pcw4i z53tsA510?*=3wanf1BGok=b@c!Rw7ZTxo#4F0Rb4|8SJ%rfGq7yqgZ$7cX%ijlxL& z*`tT)fBAQR&3kv#(+xT|>^x4t`u#7{uL0lV)G)ht>j5Pk*3!TFXa6+)+)OLGK9W7cAy?7SvbrVCO$Qn)KQa!^cs>5olY$Br%W;%o{=WOHatthc$ zoZl7hRMhRaA+dV)MxFwA&&-sgpXH0^yRd?qItVp3jU{zXQ1*wr$5F$r zK+CEQubs2URw+RI_J==CUp-kTDI=S1{d6ro`l~kqZvYgSyN$13rI+8l!l=X0Lw^n7 z-ZQAb3kbJYsg2-AIa~#Vahr2yT{qo6izHs?>eRR~LvB@`D{n6CCGIa33LWpVK+PD%)fu`?X5FcP|)*S2mP?(1t|=yA??lyHOV1?izY zJDl$#ph-?faAqfxGdvKohMrqX7U>D_^Eu>=piTiFLLMc8nj6Hoa=x1+M6`Jhg}-$@ zb+?ammKLwig>1RfwT>b^Bc>ACj>MrUz%s?4`K%!ki_4U-ffdJ!3s?L=-E3@ zjPNlbFdEI*7H8SFIYL!PG9oT+nDLqQ+r~B9;k#*Q0^^*Wnc+PsCfuf2H7M*{dQG^_ zi;Wi;I^Fb_KY1r=F?MQuu*Lbbys$_@4ImlWQRUi2;wQJ+4?~sJbn8d|Buy+mNS_hk zYAp8GAODGl_ndQp|68|jr9XZCG45JCJcL$MSX*U$m9U;myc|)G9vL}L?a8m6tw$X2 z+UgR8pUF~Xd1O=~-kO5d3u`|hIpP-g(CCDz+cg8aoo?Q}6S}?1rTwk;i(P#!(9}%JjXp=((Umpys-ZS|Th510qYA z48gucOMnmdf;u`t2cJ=gL_0V4A7A0FLlz2L)z1Th1f2lLv3{T96JQCXEJqZdXV2KR z=$V% z?4DvV;z8*V0TO3`aUqD1F*d2LAu!Eb;n^t8VR zTkc?(58bQIkFw79%Qkv_JwNss%|R9jv`iuyw7r9jVg2;>NG1)Qd2sH6&178hnmU^A z=w8LT@+_zz#%4rGcd?xk1Us34-9O6CH zsm}XgW8fZU507XevX_ENm(k$Q0Sr1?4+sPo&zfvzY)C!3!PoaGkjXo1T+k-2bp?vn z1MZ(TL4)7CrhXO|efIh_0w<;SZd_wSUeFrnC;)-kg0`MA`^%=})|E3RW(T22Ph*5i z9mWHTncjNVBpc&_Wf-2le$E6Vgt(|BC~{bDT88$Sbm6Nsy=K5XB$g)6R74o>kdajf zL`)N8s@Lrbg<{~K;EQDm8k|{+h1FdO21sQlIevFC_+~4{Arc^{^oETIWL})fu!6iE zlkViYxK}Cw=Y|FB;EoE*!A-BS$?$~k2!?48=8;PfT?$60F<*OUjuSYF|j(icA0Fk?9)>+l*1?b_;fCX{){ z&DFv!c>o|Jf0w{kghqZU-GA$5di3Z%fz)I|6Zd(D!BGb|T2NWHRfib0*Nd!hQl-0*m;?LyZ15z z)Ze*rJw04{D}8wH4U{{DQz?hAdH5oIPEXw5{ipu}!3Sfqw2eGYXt51y`Om8sXo@1{qQ@3|o-_ICSXJkU_zikWrwR@Hz2L zHvs*kD1jb|&5CQNt~9(jlJ3mkOm}~FEgd|;9Y=YKJ{4KpO$-=f`D(Q6ax8u1R+eVF z(Q=D3j%a|H_&dOcdO;U*U#Sva$-1?5--XUZ5m z1o*j7?B3h)@-yOh*wr?EugiGqOto((E=Y4$*FJdW7b{a)CK+fT zPXbQ9eEbAC3TU63p?)E9ks{7kHu!9PoBQ$lyFX7y*?Aa$_APR*17OXWfiCcGcJyfreE3~mBxn&F1UrbHQFaL27*^;F zJ0f^}<6K)9MV874wup7meeSrwXA-&1;EVm}GYoVbf}DqtWDumrGjnZAK$$VxW`7y> zBkOfWX)pn<0so#0z(vSrK~}W9BJ@UpXax$hJd1>%w0H(fZW;4#@`~=$x`YsW-#vZB z-uM1qLl7O{mun1IF3vK~AqZBt+LM+K@;A?*arYh#!dk&aYpC95Hc{c!z&JFX;*ktlhf`L;8AdGSHpMX+bDWNk4@nC#kFrkd; zLZ*%Xl#}kM$V}!lvZAh?^U0a_LeQMtXoSk`ZX*4*S_^$1ehA;Kz@mUK8$ucTUR@-t z5W}2tDUmkxKP-yELL6Y%CyHMOhL#cmO4BC`nX)F$rw&gnde14)hm8M`# zuCV?r1lci8L3bk?F6zHIu`J3Gj6t|1nY}9yEsJPxQgSI#qyQJFXE&XGWi7KkN+3+L z_zauK3>L&^VqC_@xNuQBjn}mtO!anin}{GhGwV*vIsD^nUdNtckprm3>apl1ak^r*RaM zZe51Ofq*dqoK2&!CUA)t0DS(*Kx<=_UcB0kQW>&i#?l<)8gX5HcBY9~!*j}+=-NhE zaD7I}=N0$^#pC?pALR+S9j_Gt4onH_gWyETSgktZxA>^;CxN@dVUg!Q0l2gQOwNGd zqRlgz?#}~q$S8w=TiHUWwigq4ReTV6)0zlk*VcP!w_Sz3j8;K3t?PUC~bm+p6N zEEXVQXtoKY7nB)~XBBx}BQgUe7>*HKo5Kb~vsGkTnL-#Hj13o;(OCra`0-=@ptZ&; zDDqW4_wIZ5Xb&`&KK=4Z+TPhs&CT`n>uz5!$*P9;WB(Ptrb_j+Yc_HG_K6m?hjCAQ0+o0i>`p1rr(pGsIn5 z9-@c4Y>0Utf{s5w1Q0Ryy>(j50LpU}tZEe@b2ti+T%?^7upY(@Fn5T{4WqzvF?Cp% zAs7av7uQDda9*;WAq1pCSaz6IjQeVY*qX_+ngzrd^Vp{B$R>5?OtvVH>~NXm8Cdfx z_T31ec^HLfc_HgjmEOb!)}Cez3UVX>0Jgq$mb!cGbolxRb`ORDX43nNV}*|a;I?3W zb-86Y3mOv2NRATj{4#}+y*A3+y>7CX!k6Xhun&xbG=bv+nOOh?aUfxz6#mEtp2#(J zk6XAm!~Pi3+rXl0j0l`Gpe>#|$^6Lx1)v7V>GBTtBMuJWoA*Vj0sgIp-&NKzP64SR zz(fY#gq%TI`x%rTDkt`#8Q^?`7*2o@p#r*H!{9~c^trA|*9`B-aAi-75Z`J-%TQf| zihBq^xsS%12f3r7#Z78RSI$^S=E)3SnFcH@|;k7w*0xn-zz82vJwyJwYd65F;be7-zvy_{=o1PyKH@CSzY= zy?&mik=G@SB*5Cy!4`Xc60l0_CWf?a4R?s^R;haaSLWvdXYKTIdoSkBE)E0`z#`71 zdCv1U>v)pLu)qE1f0K5471|Bb737%mBE-)6yt1hdHGfqn!h*r-UIGFISH9P8<(>-toS0#ClU+C*z$kkm zVztOrj()b5U_s6aJceLn1ZmDaTY~-0dF4E30Pc=mux8kh03nhB0O;1o9DpiEGk9Ob zu<<1;8$xqB=O(XlqL6VhHm>8BV~fMc z1r?!!uwW0lsvtL(B=#ozR5?eYKn~o|AZ%g`dDg}oC{Xom*~48=SKtDlk8|$AI!k&E zE{v69K3$hVjYDRS=M6l?*|yf9UxLV};mDc>WMg2ikLT>th6HSc7?)G8`~sLo?_2gs zm+&x+O4p1lf{6w!q6wKCI>y8MMSzhx!@w;GBK63MqIkI?4_nqHz~m}8wT5G{xEzLE zsvz)nubJ&*8gCZiTgK|^a6oKBV?k7#)NjjvAk+kIZsJ$O3SDr0Gnrhp8ZMqw$}te5 z9e~)@gZ54Ck6_$4i*+=8*@+qJP2aJ zo7|MHbK`a_J>XXId2?g9DfqjAuErXO{B^?^k60tF%6papA4_B}PQx59C_th$=)QAN zD-;+<6j0=M*E8Y+_xMOv-8^IeHM!0>%6u9sdl%Phe4sT9-d z$_j!*pbZGI#uVqG1PfC;!BC-ooo&-L_K2TiEENQMo@t*%j+$@f+laH|Sd=p#x7onZL{rPYGjTi<;}ORF!^ zC%^qQ@0mz9uHU8a?zOb`*1PGQ`RVi%7|OH#-PAauA0z;|c34fXzkUpuV;)(nUFxS9 zU_ZfrGBmI%|fJqFAuwuVp#rkqR_WcD__3Ok^o$g{C&y~~g1g-7}eHf+XRDR_y zZc^gt*lT^j+b}F-4cFi)B^UF98p`s{1n8{>RF&ze~C@d&`Xg&b=G`~;LUuz)kHq0FiYPM6Gs>u3SkF2bEH zk4uWyO0=*NP}MMYWV}`Q#+F(K(yUodH(-`?cZu9Z=^6{#1h5>_zt{6fG2h%9sqHjEvdM%Fr;@Hf2`XLZ!CWt0=&(S}8*8bG zH%?LyM}(p92iH{B<{%-0^C2@PkYqn!vy{H;Yk*PZ8JW87&;_XG~g) z_|uTj06ErFxii>5Y(YPZPKb;1e}awhV(@;)s>^S{Fb>|MocHy(055rOy;D{|@WqH7 z=f3JC$L_uKV3aVL)G@xtx~m$B&Xwm*V#Zi{j#-?2!+(qybWDhN<{oyMUyN(BKdV`& zGJZ@}%)J=J`E%^*$pH&C08|Vm&X2A~<97vsExd?rJPGlB#vs0MERI(I=9hXqzPNt? zasKY#d_D;y&z)Gr&-r;>!f_UPZtvl_@~esQY_g$eJ0^`Evbe@p`u_Ml`Zh9;nM>P` zsfWFPge5VzKBp5Q_=5zHzGcx7k}9`B+!X8(f>!pT!DLyP+aUNZSzo#ca>Ndaq$xp< zYf=Ty000rjg^VLDEPOf4=PiI*APR9Os1`i7+g3;mei#D@o>uL?3d2cqq^Wwu!02BloX1?lb9Y^6Y zV*e;iVCQ2Ua(}J;g4O^pSwz)&=&^xyopcb!kaBrf+9C|&OK#aIs>g0a(4M}ZnL`S*Vxeb(EZWldG6e}i7qszkUhjaPr+i27_)^HE^xH2)ulj$ake@5 zT6jg2arX+~%jb5FkJ8`%>Kj15E(Vk<*DbNv=1|&YKxGcWF_LGkSXU45CiC)S;~)S| zftaxe_bF>op-cwrgB^i;5umnIPPd1bV(vTC0(j#9sn41$CwP0C$q15NckFM z)wNrTX_*+LW!y`?Pw$Kcu6KyjGe%9eXNKpvpdho&>xW=jhA{+=3B1;wW`W@f<2@(F zvT?MVu8uNf`|YRwk3UKuF}DOAq)D>;-uc11#6(?7^EYq6ESA!H|FQ*$`!>~gcWEhh zM2n++vW4Q3=ZoQe--XT7QHEGgDtj>Fmwum*{+)A%uH?qi;BB*&1#SW&1 zLBZX?nt874%*8&!P%vbco=I0Wgm*90009S^WJ|M`EihdIU}gZzZNOa?WpA+h^vxNJ z8k9jRpO)VJOF{?8fFwx%)%rRvwN-8gfZf`{HA`%4?J(UPSxyZyuytEhNa)xm)A^hCM7&ABi>PS$_{##hAWF$lX&*o1u&Z^wfh*NJVsNc*qZL%0qP$X@OO=oYBu z*5aqdttT)ZWLX7=T{gm@;SPYGBYL*lamSvg@7|sUTv6uZoF?8Bpq`(fjfA)-n;WS# zS4h*8ml+?Mq+fE4IiU3w*LeVNd*kQ-hWJnvKZL-GCr{Ha{{4TX**r`P+37PFKNZg7 z-Q|V!|NfSQpLzQDa{k`DeKY;zFaDkybf+M@n6Y&CN{m~~c()AzyO*+r7VF22akGJR;s7*kmnW(OUjEY~n$gQ0f`JL0*j`v6MMKf1VfL}w$&g$6xt z4IvQF=t{H9%iwdeo}4$9O%OWhdFv9o&v*~(7nKb3A^}pAgW@Y?oN-!?PcMui4#p=F zX&vJb@F-LL5^N3pBp!Wq1ZP2z79j!bA>*uj6dzVss*At>jN`dvY~d9`8Axylo@A(s zoOu%-R7wI`WB9zL#LR&8LZ5OUb&Hh`1T&j2pOd*i8%!M{3%5VOAG~2!sh%Z0QkDgg zp;G6|8h<}l0hIUg^-<)v*K@3TLj;S63+@jo!QSwB1aIM5W*#*59iL$l>Qs#vU>~0y z5DU(EhViSh~Wb?bR>s~7uTJ)pfGct?$ z={h-gEJ%Et4C1fuAs(OSBGkl9B#QP+bZo~0nS*l{PU<{@>Deu2oCpG(7=seczL8~E zB5=FNrnv%aty%&Bmqry~+<-2xu*r)AY45=pSeZQ<%X1%ZH$>c$wYI7#cU_F&))11hg)Fr+NprUz%#b^sYNtTw&cTm4`Z{g zzuaU!7a~r}I5PWRZsCrLqz5*S?#H<@^MT;E?p#~eBtx1$`^6%`4TQ9fdyffZAHFI8 z;wZui184k(&m}nJf2`NQg=&CkmwloTbDq4Hz!4qe%A+h`l!#jy&rDrFu1l{I)OnG&K?|}Mx7{=Lj-Foc-h=h_JRBT1Qt*0 zrL9|tPYANij@qWA&MNhv9^APFn5D=mF3}navx!URkdS}_3<$?#q2y@*pEdAIpjla6 zjI&{P2nz#0rqxsT=~gIRTSCo~fYCZfc^GcM1D9b@7FMUy+yXThp(dIzSAvc@%GzF! zW*L97{g`+q&HxH&lF!@FedYR8dhpf?hyRL(-*tMi)`CrKpo~sAM{5*Nox`}YV}^|R z0qnzD#Gj6^epSwp&B--BbT3cSI_4A$^!eZaI(^PFWl5S@ytYdG6ES-;(`oMB4UB}F zsYU|E$;;Qc(%6IUc>qj@xXlZccMpKCweH+Pfw2!s9^qVC@x6(Q{c?xoCF^Ml?lXF>F0ZEhx9_Gm5whXt-P=j~uU@1YjQtU%fHGt# z8&9WeF^|=rW8X$5HUQSSe@zRp2m=7DS@+BsL@H3?jzRFzW^4#UV2JsxVuWp9K80(e zYzOWz3%2eJQ;3yj@6$I*WU12U_OOv^V>~1KcDOLjPmCJeY?By!N5lip&P-9RfLKc8 zQ;W5m5MiFS+VSanW@%_Y{F`VtC&No+s#-U8A1)h36%NLowYEQ zbc@RL$zES*crdTQ?4!tKaAku%qaDsA+abO^W4c#WL;_uvdDMPHrkEh5(u+2T7;;lH zlmbB+D*KON7A?2L)NFrO#g1(@+2P z|3sW9`hqoi{o*BnaM{pM_62M`F@KiJYSD|+stBjVaH%VpQlyt!NW}`f$~xPpy3d~T zzWzOc1M97)BosRnt!`CMnfNA9n?TPfTU7S0nelY$+yX|GdX0r*Pg5@s88iSJGFIHw z`?KE!XUbzii>}T~bl(+wMu4IF*lR;uGd6S zUjdRDdR`ctZp9+WY=amvWC_>vnFLDeEM5g@>t5gYh$AdMZq3#% zK$n4&sANH6UgsQGEkHtH4AwMD=MO^!1U-gZ7&ah@6k!#}8d+~A!#(ep>mh~_;9L?H zAJ5Eu3T$Pzto+c%HM_sJmF_-xnAX;id#vx~#w+G&;KGe@at#f1L88D)eP)kZ?{$fB z=^PE#%u*(tbKdJ*IG-2br_f_!9nQ8PV>LQdh`jBn5nUa)PJl7?1tQPGhcQ0@g1S=0 z7EvRQzj^+3pF4Llw?4bq1S@gKVOs)8a`Pw;!X%Ys8V~?uyk5e`S{70q@rb?4(=yTWo(u91MHJ@x}uWkSp`TCUrt$ zL-2k-<3I&P*4m0mB_&DZEUl=Yx(pD?W4)Ziq>Zz|jnk0AcB6r*5f-heysj%q5aUKS z-lha27a1T-MxD)aj#3iz3G++a2G9b_QU@d}$f~ec<6NL-!(FbgP_YcdA|*45Y$!WWOmP4XVcS~7 zApPcde_%5zl{iEk$O(jJju;|aqReoBuCNY(hdlq_V6p-MmP~w@<(jgi1)dprj4(qe z!%$2ih%a%i?9tfVeWIIIi^t$|UB7Nb%ShNN#Ih$fTvcoe4uDxWvJ2eL1BJz|V3VI+nI$4H!=C>sq&9e!HjiD+`bJl1%7nCfmFs|vfpNQLFSNnE1pMp^fBWze)ZNukr=sGaXzNV^epk5E>Y zh#GrH_q8pQBxkyvv$XqFmHDFv9%m5mBw48R870en8e__gO@Vg?g=`{&piseXkhBVW z7#3f}rB!VJak$+vxY^7AwQ-002pF_%eI<~RJOJrzgr$haHIuqToP|!{$PVxDP@&%#!!en&)vsm zY0D^U>(xl$(xaez+607gTqvUz3>||_h~Go~1#5PRF=x-mksfQq{6v@npjt12L3YYo z_lZxd0cO0Q;BE|AKMs(zcj*5|)SEUpnq7H*mskh_AV`9Ha?509R%Y!bl}gfdt6O$Q zdmN6iBOG4BerL}oaD?Cb8~BaChwZQ@93D?cw8vvlNScvKQ>~@4mfV@iodg#EBsLO& zf4}oo+AL9Ig240K=iYPA`Jew;?g;&1gkbg#&M!`D91zbwIHBs-qJikOM*ii6rzs3F z-TK|mQRKh&0FgJyu)K|iYL>BHB0T2Ztt_KcnDYv;a8I+yv*t78ko9rATj&QoDHu0; zdg8S8(3|_O8aNyDd%NW+SMQ~jFa9=Fwmzk56xo0I0#?*w15byc#SA2ZP3G$q(F~|o zZP8Qk$3J<gWci)=cvG4<2mt8u}A}@r<0h zpZ(|mO*-t?(_!N+IfymbJz5T7JF;F|tP%IGA)95-%*1Bu*mh`U9y*LQRhR0t3No~% zf-)=g1yoEA?_+PEUy2r7G3PR{>PYvBpB3cWZpgp2Xj9_;&5pAY7-jn^+VrUYkev?X zg%Q-i5&%dDIgf|tt^s0ro8$9$^Q9SF%te4s=E=~Y`Cram zfjw%DndcyzOIO6Un*uaNTO*mO=qbt^nfF{~Jp?lXM$fR#!{BPuZ)sb^Kq6deR8PDh zKo$6EJ7R2N1c0NmK=^_ITENFXVBZuBCoop=hBNTb^6E%81mrfoh#Woom=I@u{e%Cd4-4( z@AYjuZx9Aygfyls;3Xg?Dj9iDlbBT=8C@$dFo%j8F$_D{YcLJ<36{Vt9FquIXVV`^ zpEEHUM0@-Yxh3x<3#WNjXqL=r^G!Sh7Ee~iu!lJn=AoI#C7Nd%B7{}&HPEQXEvbl1 z&K8N9|8d0wFbz?)X;Fd(xQ0RSejW#Q2ga$y`0PnHK~pQop5uBRjNM1ci}oFdCjuBA zwa`FC=o`xbmQa37D)pG@j4s0?#XvkvTr zY;&>Mj(FY~ytx?;o?j-6wokLpoi}?JUl=Wvt4iz?#kaInXO6mn>I{szsa0$az~t6x z#-Jh?U<3_I`1%+_F$MEgCIorGyoDoX&oAW2`4Eiybe`2Z#hR-mZ6i_PdiYHI4OWzU z7hnifP%RjVGtT#W_F@-i1@;cGlofT@VuT)Fn+pIRj-dkJ9FseCMuPq^p%2-;#<~6< z;F~cTJ`n8I7?D>z1G7QK#i%01g-6BAo#sH6D2qu|l zLtb`DFH;#gx3Q8|@5@%RzU*ULUR=?hrOEtvXy=e;508+Yk-49&eV^w)hYhn*Qp~7c zu^PNa?o|y2w?wOyGL@`-9Eek@Tt|GSOb@@#ByDnLR)Td{A?#L05qt8<6abc4NaI;U zw%PGi8gZV4f+i~dXpwUp&8IQ`Y#RZxOh}h~hDd~AT;Lbi+DiNpb3S1-g9^t92B^nf z9rDb)*M`dxQsEz7%jUAS?TC~Le)0ycsWIoX$g=ZnlD+wYG z+%DfsU;JnPB)y_C`EciHuo2(?@hkR-@(L&-|M-(H{t6vIe%j7A%)VcOg#cCD zUt_DzaUZK%qlmzX#T<6LQP`4{(LpdbqvJ7Sb3t`vy`*U=v^5%;=h|q8dZn zZ<>MCukk#{tiaE(D7WHU01VjUS^?6ug`7ny@Dwm>EB%}x&k$tTI`EFU$`!)nU6?W} zux|jVe%{ZOSYxYO`#4kctB#9jbrSqPuI1cjCowP&{LO2+E73B0N?7Ap#bDYhVaUxaFLqLG_G<_J5)~#IclucVAJTV67 z&3L9Eb6rKr%66S`4x+mM<@4w18Q&Y^P0gZUJJ95HKv$KCRZu-3J)Z-?6RJ2Sq!4F# zGGIl^>c%z#uM%+Mb@1e>G&^;1&37|*QAMj^nq-AB0)ndcL?{nj2jG>LaTg?2{T>+1kBTDk`1m0f>?#uZAeMkvGOuP(TwfAhuf)* z4Om`Th$_fcLM!$bG-q-aAmChWt)P6F6oHRyZy#1CN9fr+hc@2w1*}V){(xZ^aLkx% z)dCQc290pjVuBFK9l~N`oGNqSUTXX3=AaO=#aU8-`wyA4a1V6x$(y9fkqWcMq^@A230RDFp&q zfwKi6vz%kf(#5w2mN!|O)3;~om~Rc1?DZ7DW0`!ft@S%`jFeTBGJ)k1ij4nkO#mVw z99-gk7@!H5OytFWUoApp*BGWYhAfASUeZB7M{5Y3wsYhb?hKY@oSrv0{}qt1M%ZykZvc~37_Wupd6*va;f*}ZnBei` z$y>ky4jIO}LRD>pG>ks_Nua1QtdO6zx=xehl{=|+e>p9D003saizw#~q3|Zm_LZp| z^entZpXC6hvfDCewj&Tszumo}jStG4jkJ0P!#-~j7vPw1?adH&CG9W<#tl#>agJTj zbHQ>|Y>UA$IB&kecra#991v_RGgfFc_7HM~Q*m;*m!AHSR0ZZNbON&J)-feOcD{gI zTWD2aAqyB+!=dIX4FJ=!4tj{D5|}DsT8Uw)5)UCmG0k|?$L7t$N6!^o_9ma7Jd*r?Qn`!I* zz0_b&ACNzIdy1{#$hX!B`i%Eq5rSVLlzmE3%lE(kJuOpGdiMG$dv1`{H`mjH#}Cpk z{^oDV0qmxK{?GoGv_bF1A3eHDqwVME#UH*-RZ<-m*&p89b5$>4d=snK4gZ^y&V7*ivnwIxdgLgt`uZuk&S^j(S|z!TyDs@nCwfkwz}KWY_~i zj5ZYr#b+siKvxTBv;sPuh95oeJc?8wZOW*{6FxsLm4-ms% z@)>m&penp6bAZFlCXl@}RH(i}jbYc>7hngA9F5BGfWcM@Od?+m`Od)>3Z%4au~{4w zax8dsJQkD4s+K)p8;4QgkYnGQ3^R+<_x8n3*e}zgfL7t)3RtsZ0G#K*GQ69piqWSk zdN$f!A;)!61!OMSl-x^JJsX2zc+ae5=Y_&GciTDfs5prlzubG0U#2}+aN@V?!b{8( z$HoyM06wGli7XH<&mogMO@K;mQ660osSDWs3HF^KU{~0P7nfk|WbW9{%yj^Jp2=}| zD1x`=O@sqsNE8SciFYCNged?@f+3FsB1*zSYjh9?SA&P}Vj@xpYRp=1S+s3Vm%UG- z;*mueVeGsQ`XmwH!7Dybrzlntu%R1(mM8Ss%?Abo1H%p0w7VG=9D>dYQ8&qMLv~^0`CzeZ^Ep?+vS>lx_s9G zMLmqw+0j}0<rWUa9 zDGz4nqpSxTif7Sa6*oi zdVm#kFpVskJ>s5!^?lmhypz8F{s51jO-Wvjf2^_=L%L({p6zmG}gj`tyi z!#Xaq(dO~?H^{4TW`;0qKBp&QM2Lv@yZ2N?1C%wxqXKX-J$ks2R%!aWN0@9MZ>&-) zr2AW|!3@|-=o(Oi$`A!*Ke;mCIojvoXl%^WTa9taCLA?jNs*TujK+S~ zBL^6=mKp=s-A0y6_4Tv~!(&4E#mNQz;i!0| zm!0$B$rno`jIR-HQVGjmO>rN=wgA_A)ToxNk-%RC6tV1)ZeT)sQ!q)(j6+V75H#3K z)iIhw=#fvhamn(r$irfP%7Tt*{V-4Q#DfplQiZuzSw>DNEZiY^VTV-4HjXbiit94= zTLFiphb(=M!!>OI%2|WGM*b6Hu8|K*aD{tJ(dgG47`K)#pnH;ih7vt^a}>r%X0K0w zK0Q`LQ7eQrUEk3Kj3MXvI)@q%8B&{WpX{Ba*WbNCv%%cs4B1jZ?|%j+*nP0Te1UDK z@}-pt#Wq8oQ|{|NvIW2*^T#F$nbGO=^n)wbN5@mcU^Qrt*FE?3gyyklg+o5gS;nyX zS1MQ(@f0kKxxGd9ev5*Nuf9B^QkuL`9<7MMHk7qS=x4$cFBtb2BX28~68o*j+{@C~ z4BM)98;~y*0ZYu0xjeEC<_O7Vm?!J00UoH`$4Pe0Zb|zvUBSW*J$;R-g+m%%&&3>I zm@C{{Kr!fI>e(Egy*nBLNLjP1eR?aBCx*$y?6Q{#1O53Q2vq`5?Lk%mEtDwyv7z(E zojJntO{&nT8eO2*8licMSZr8s2Y~^A=^@Tpi=5IoZ*g!Nd!g4>DNgXT5d*+4S?8z! z^0yeY65E5rim;-aR7R!V!VMCw&{x|7z=S}FkpZxUW^xmiMQ&a)KhmHQf z|LOmnUOat~e*TxANB_F#zxxBabuSfh5M~+gG}qR_kR3!|V4qnr=2|^jW9-`-8|nL( zugP^)&v8^$hjVMU)@jG{gq*x~`kR0JACcGBOn2|zg`vAl&;Rr+O>f*|Ez!rUy>_LI zv$VXniDRpMVza4FkpWXagY2!CS+L9lQ2gr z5>s?+m!VFHJTvT-AZ$EF*Yl!nHA;XNK~7ZpvtPg)fP6qYn_nhKFyygOH^?)W51{7y zo13b=@HzoPhyazDcgLpFKo=arMJ*I4X7V0&+!kps{sU%*{n1ut?S&>2#*9kt^3-$HK?(bFx;>^Shiq=X-`c$`TB;&n3xZcvmbC zpCOnU002M$Nkl;X-V2{Dx@_j<6$ul+= zVBb9Ox`XkNSz=gtx#VQRHfkAtKeW?sorFqEKUXEYFzpVM#V+) zs{_dBxcC@`s5EhOOsXB@t=d$f`z08iz))L-Xl_Igg&q||J}2Xf&qBAGU7(W0Wk~=1 zfA>F-n^j25DA_zp@CqTH05#VLpBjz~s*^ER5zHC#rxXyY+6>uFGsl+X9|H{Dz)T!) zjLaAXW1)eqMl@JHlgC!sKkkxwQ2=JK1fDNz)!4f{8l2&vkQsc;K`@4V&SD3T1yUH5 z4#V6z-lL(p1$6cWIy_$wAr`C%&;*Hw83l2(05z-U@&MKw#+-*`=<=R(`_Z8c&>9#? z3?(dz<=e9YbPSR#>i{L*!+?ze<+eRB!Ltm4DzRhx*QW>z=h+foiptt9$`J6CClAIP z%S$}a7aVhSLz{#`9+nIy=u)qcc`=-1p3onTU#1UXqpa+z5KfVyV#dSxhA)V)=n4Dm zIfQcp*`46Di*rt1xIR4Q?9p+0{Q3K|mnaevI!e!W4};MV?8(ed!x9RnJp0w$Ho@6| zF)I6g61h#9FBSmaT;J9o+vNBRF{q#iLV(OYGU=vOp}V0qjOM!8-FudwZa5WMc%zI} z_NTx-be698{Yd4BtO^J`XVE;m>DmSk6~#idOL@I_$jKMYkY7Pak!KVD*egsH(3S!A zfU0bxN-laUp$t@(7Tg$8^k^`v1a%$;gfVP|(Ld>=%l&qG`}JEX zX~K&X0#I2w4G>ztyOBQptIzp2xh5El7P)phPF6~G0cb5``{M0E8rbBR^}KGMJ4=pG#0t2!>JU_r{jLco0k?YGG+NVyv zkrPbv$+8uwe2*MIMF#I2**Gl)qa?V^Q>5|c4GobGB~JkK+We#$Zb71)VBcB4c7SrM68#SqObFVa_Tf;Kp` z0zt-hzk43X<^BA#duftXl2>ncQ~&HN)!4%|ST{O{;FQT8%c!!Z%uhI3fEf!&Z1Hn+ zd6Y0m_3M*=Ma%-2Q|f9 zl-Q%nOJ;OoE&&O)tuYTwy%^C8{p9E%t7y4p=yBU_%>wAaS^5<7yzF%ESpN$9g zf|KQ0=h_$7*)?_2BSgw?-cZNVz3lO073^gIM?kp0fKk_naVQf_LiSUzk3O?=;ksqv zVeiMh@q(|%+OmFLTUqcD84%0>z0c}romW3!H~^r87tThQU=sXR*{e@{4n0;ULxgE3 zW0j;^hzNA+>;Tl{nm6PZ^<5iE+Vr{xs}VrOYAZ1&$Lc zug2P4v0ffoGo)Ch!-l>^dFN=+VM~K+68Q&&gY73~h_%M%C@;xe0U`-Y&k5GubO>sb zYSJS@Gvhmih!*G4F5m@nAEQPhsYzHQMdz8@F~~fc6C(&kWtH`?s>rHSGaqXh%>6IF zPgArp@XXyh3jGq`?|d0fJ>a-4*L=5k8bEH%rk;d(J@&~0h8B#Y>{pG7&^Y^Wl7BHD zr-WB5t`YdHz=F-e8Y#QyhF!R>eV?uXHueg$;KnAtuIBh zCB7Z2d(i4$t1@j8?mol$jL^M6%ufF|%snmxs0C$@5?dx@UZrJ91-8;+1;=lWjh;Fi z0Ann}N}eA|x`)Bd%h0fHx4g#+Z$n9ao^uH3y261R@S1wB%Bck<0DBRZ*y>^Rq9IO= zxBHIA2_~h698U;-xsUr_U|b~B>2XpMT-!o0fwj6tKv;F(@q8sfQiNN)F8@UV88Ynp zbD5AmZxY9TF+b)r-S#dB{lKDOG*x`IAJfs**qJ6VVqY279MZJ-3KsYIlM|lLd^!D4 z3?Fl6w$?mO$TBi(4y6OLnwmcXP7iEp@Oq|1`BqJoF2eP%Hmph zaLl*!_LwmC;yrp8l3S=@l|gy&@(eHw;5(;69GRTqvrYD0HJlq$EMS7Ie#OyQ2e9EQ z!7yS3S#QDF4f1+{4zWk#hxgZEt62-0zMsC{rz!a*Md)-y;CW>oJ@43Fn>t09ne3Z+ zoxteN;}BiKjuAuz0A*=4E8F+dTbfc&&aS1qYYXXwLWfC?X*xfp;{p2T?%i!pXV^#$ z!el*cm^P+@z3O4?J+9A?X8^g7S8N4Nl*f$emS8#5_+4@ z3sBq}hS9Zg+3PZCJ_)QFOu8p%2pqH%8I#9=v^Uxw>{PIUzFFDgTZd8*|(1E1$~56*j#}1?d#JayY@|aWAyOsyb)vQkT)yK z>Jxh{ftT~@Tna2rQL>$n{{;wkE*Pu8<EH=a(#>OrWfrI>Yr4_}vz9U|D| z0xn55c7AaVe8GKb_Xyt;Ol;VEpAFm1H3R(OycfD$2+4`Td5nvR(l>y(8myRuHS{7~ zY@FGiX>w*ZiG^7wWXOzgGb=3Zq&$R@wdzlHK*mh=DjI)s(}!#K(|1%8;1R4VtS>+u^$Y5MVyn0z7nhKQ^6AVp(@fdzH zzV9MADP4@I<^3**Uab@F4-L0D6ICYM$uzehL-Z&x7NDZ%!W%0T0NikllJ6h0S+khE z;0Z7xQ^s60VWC`fMY~CZQ9WGxw8Vkh=5^iLZh*MEEfzW^J1dZk$2ST(1)#Dtr{wV1 zO2Q88XZ&4dp?4M4wg_37u^Zx0=?kLWivDic*o~)IH|Nk|oFO3RmT+wdI?BN>DREf= z*|;|IBEw;0W&f;U@%%X?gmpL6ig$)bF*~3Ds&f&@hoO%~)LvPiLsGnN~UP z@Zt97Tnx}hBD;yQ>c}7;29^IU#?hO+Mm4K+*Q$v`s78UZ*P;RJa=Za#ado%39t#K1$4G(C?IFy{#4M@5}U{^n6` zZ!VHQ!rZaOUG~NSN#9H4oK4pO=jA2Xrt2C5Kz&jV)uaOuEMy?F&}lpeJgsI z+~0~zL5}%0IzKJ<8gtPm++hDVDSI9==uB{NUu2*K{b5k5E9g z0YXtKM>iNa`|uf>?hz`k0sw9>(kpeCSL9sw;)49r9>&;WgbtxP!CtW|&{yFAtT>yt zK%Uba{kMum=7(_a1cv1exn??H=6@`6p!*#PQOtpiK8*t$KLC^6`=dV~@`^fyN!MHD zI3=3a$_fD&$(cievVR01I{y*YWekEo_ppYoA-hLIP2(K5Ig1z0LD5BD7CTPJA`SHUBS4lZtrsB)D>Oj&0kyN z98L|7`IYwBV@blG+jP|&%RWE?Es@@PL_az1A!}oC6HA%C{bn~^;xuTOnlQGWyka=j zJRgfamN{8ukpdU;0m3j$&`g(lQ9yx)d%zl)TMBsP{W`ftWMg?517yW8bGm(RJH6(F z0xk`E+(-}KTcAhbY@BCXnYHgH&MIw5CfJKjTJPjJczoC=-;blSzUCB@n#>e>xI`Mn z2U}|ZR2X5#CmVZ%PM=2Cosg$`#0eXo&8b26gq{{Z=U{%f?yht8?-Kj>obzotHVefz zhfVQ(Q;%sf0X&Knm_M0Hb_Qyur^3=&L@1i*wHX>E86CD5S|>- zr@=9oWk+UUQG4O;@!qEW2;S`Spw9Bl&)3hNrE8pa3s*wSeRu31^af8UTW{W5biBa! zd(+1NjP1=GeoOormQ|o8JI`p;(N6l1wqP~&1Am7EjKaOM5ajU-&K^M@JJaXF8%Z${YAQD@&5D= ze}>(_pTX0dWP+q3XX`Y=WIZkr^cj>l+Jg`vLgqDap&Q2ssgAM}=qU)I1>}gZDEDV@ zf&uAYD_EnEALe`pWoB4V&p?G^)u^w6`tyx;X5}5^QqEyVfEh3Z?IQg zlxPD+qhTTP;CYxg+)L(Dra@pR$ToM!Frbw57<6UHKGP7Oj2{7lWpm=WU<>?@$HvnX;G2V`IxWMx%@fzfmwaVO@;)4>tJ7}Lst(B9V1ehibhfO3FoJv~itc3-8PvtwEg zP{E9{{g(UWM#mUM!kc*XN4BPb5tdmA+2Xpg5mRh-&sDVA?BMh)eT(AOP_F>yRtYnl zlM>+ZUKK#4jOP*w((V4cR3v9=VSXi?61_258vDte5?VSsh1uXf0$Zz? z?*I~)$rbDqUS4AU>KLzC3Lnk^Y%Rd0?1eyaTG_z~m|&gwaI9V5BafJoCRN}T4tPxg z$~g(^DnE-;W?-lUMbj`!wqUWEyG5u~_En?3Ndo$is#hCX3gWz03-4dg6Ul-V8Y-IdH#y`+XGltA6i&rqO*J4u&R61-dGvS5AY?#Y-42>7Z6TM zCzwJt-#r`?89zalRfdZcfS9wSzM5o>)w$ZuT7{fi^rhk3JWN~@=b_FuJVahuc(&i5 zNxO{NEIDFtIZN~?JtCL%qYpP=v(D(Xh`}bD3WPwvA~QVX==?T4`4ff``8+znxq?}K z$i8rYFTnJylVpBmj-g6TjbY0 z|LyN-bMq=yaklL+AVYG6li^QUaGQ2}ES4brQrTqhO*6Mu3f*3@wtX1oHuJKyww?a` z>^b%UCzpEQPbl7~kPL$;>;o7$^$p>8dAgy8xDHn#7gvCSFOez;J$;#TJ z%?i(8IGKH0OO*iCp4}<4?9q2&YpH!i|v*s)SIDpIpz`N?v;$sCr$a_5m>^_L$01|Rtm?#p0I)q| zjc?h9BgQogkkz>pq!lnA>SD3<~$2^q-@za4*! zS)k6i)K$(wm)zkd?0g-^B5WOV5xSjwsH>EJ0g996_|=)=OxlbetYP0)0j_fGUW+_E z-W#}r(OWglc8R^g|A&qCnk4rjd#nwlfpcP&tY>3u`_z{Ku&|>;z_-`ZUdmKv?O3p8 zTtnybhIT>jEsw=>u59U45*M)Ue1P}xImYig`){8?hEuc_p<(9M>eJE`_u}3@r&z=< zL93tjTM#Q_mw{pD1zYM^dA4_ekRk>)4Ldr#A};J^7+w(zMo7%ZxmEbs6o%LS$AA8J zseWfQ{i7fKj4%lbuW^duLs^ozYxjN^-wM+j%BPHiHm^<=$N~5z?O&h*$8!UtjRDHe zLIjVq(hJkuSSkYwRyn_rJQtdm17gT2B2?6+{Y8;7at}2(zJ5wldIuvfn`J*A!&FkP zEISjIA#GWmGKR_${+5Eq1n7O25LV_rm=ll4(%V-EoS0}MqVTbhR(F+|6FZN~7-|r6 zjFsgL_C(J{V;O?uq%tn&vrj_jkWkGU%F`+{Ctc&b>$#W=P@=Fy?u^ggQDB&JWBM+n zDVYk2|8!HhXg%|YSXe8%1dJ`_bqWt^W(qHuwj+8_dJ8ugnmCfl&uJ|D4`DQvR3hZ4 zcMw3AF}k6+eYVMX-BuZbYkSlT5OkBt-AuAUT?TxErM*SbxUsFsF*Md{QT2H6JpJJ- z7)S)NGB+Q+d6qUG1fT=a5K22w$Gfiyho12)%reCO8DWOC!%t^OP1WC@u1B%GG|F!KhTY{umjyeM=iJnjH_|&XmUz zRkD}7c5-r=UcEj<0{AORpg>sb5(edP2n~-cnOik$bvEHA4VBfiuXp!?&2|E+0QMz- zS$NpYzo$d!Y}o3-3foJ*V~tpo22F5pZeNDdQO51fxB?)`k(BRwd9rlZC|FxSv>1co z{4se(GSf0N5jJO>s10tNBUB0-4eO>^(MfWyEnLvcvvINeeF_%E{2}w-mSEGCbF=I@ zoEYSyhw)T4Jr2#hInUMn_Sr6bewzLHiu}o9+J1aDJ^0`=i~@^IUeh$7;Fh&nCplUL zYLCb>SekiqD-S4$=)h85(;W4bYEhMRg)#Z8s~G#GJAhZfVhdn(#^+RoxZ#A%Va-^N z4au-=`n9H-%=wiQs?kFLz9YF6D`%1;Ck+8vZo;f(-Ap0xMZk362!}GtGZxyAf z3Ch1;KBp-LGS%)&L!ZIEGj}!#=B3!9>V;7D=!R(Tfl(Z=_hkN^d8bkX>~2I^pqP0d za?p5Y!3ieZ`DDM!NChk9xOnRTMiG_EXZc5rCmXBq(7o$P3=1@Pwosx1>sUb76>-M& z+(+C47mRz;rxR9sb{c&QV6(|IixS2`=FMDKbLCL2RHf4DhHB&@W#Jc8Zd>JSIL($B zON+&HZ{t2~Q>uXK(=hB60QbROi=1700_LjePrutsIdaJ+BfW$J>PbOZpi;O;*xIv| z&2Kzu9LJp6#63@5)H!nd;`u9#8*|GlvdtsERUlS_6`f@A@;Fz&{-6H$bVM`hW!8TV zeKEo&*fz(Iu|V2}$TCSNr$`f+WRLOm>3SS*o4AH|?Zx>LjA2@MFKiBS zj0?8_AG!} zTi~8_ZiPU;&pn|Y^2_&Xj|1>>KYuzC76j@Hh$pP5t^%%ujb^@Mjv4R#@(M=~k{9cq zLPt$Z=vc`b%3!-L*fplg`^kEhcy3b%OoQrCkZ3-*fAcxb8??=tx>R<~Bh&cw5C7>@ z@j3L7$9Z{f_khpbpnGDzd6yW%+;%_XHJBx2Kyb#>xZVzgdG_C~o6(;f3_X8}XXJ$d z;Q~$bUtMF@o(u9>-^JW9@;Gh5WcVLeh@U(>izqE^vb!722^2M+up$2b%px9O3Y2-E)e%Nv}cm7`7ACZ&keA_O9q*%E$0vx5pq*Y~P)1a-Dn@$PLRtqJ*4@ zRUsStLY@McxbaRXQ6BulG+&+Zfgi`Z%yv7QiRh{|DG1q!UJ%mh_&wUsh8K9vRLbUjBE{@F+ zTy_v5E4h>htKeeZ09)1d`>C}4nC8RN0jNC*z=pIoUX3P-$dLshvN~4LmGE+!mlP1} zy-DMPx2eWjTRAywAEed+4K$hO8-&usj;8_US!kNkAP*h`R=~rvvd|)oTKx{!L`E1- zl`zPBU4f>LS@AQNPv)CVnuAr$*hMRN@$B*LN9H9luU!;8dEKlxn+q0oj`hBWthLV$ zX*qF9sPlqw$z_`9!#KjGOpMOaXEZm)I7VTK$`gaJNL5{x>pF=tP{$|dfEfD-;XnX@ z-Bc3v;;=J98)nGLBe zZXrm|eN+NG5A$4O&ZjE&#{*!qn!}Ol3xp%d+dBjJAPRhGNRUtVU^KbqS$gb&2F{OR zI|L*Y`ySvJ6BJ{xh0P4lYWU7n1{t^-2Fp1P?~ixFx5hJDj4-57rAuP`AAk348h-f= z3>W!$7~vWWqes}k?ag7kXyO-0zZT$d-RM5+qwYviQ_aIsH}RlwoYp(|^oFBpZGsR9^e zwg7*&f;m6(ndsj~L*d?KF1`8wifZ9P+IjsQ@;MCl*dy(1T{9vAQ|BJCUOi+ndIIts zQ|Jrx^mY!;VG-?TX-oC?UobDr#-h1&D{Y;fI>wTZkS>@k}oNaU&*X^)} z+}|?t9&L3-3X3TX_PU$Gpf(ZyEB;V|aaQD7CESMekl&ho-(J`~- zh4z6zW?+TY7Z3@+vOG7gfzCHYNFA5QPTZoeUAwHhRL9w!0ci8OUWd7jy{H`VT96CK zqOzky;ZMJeYU%7{vcS-pZ_%IxjbG8)gsS8@Qb{biGI841d5DJ&PNj~QbE+J8Jf619@yFTA{4V~{e#pYR|A#nhMp-6EHv>sAKM)!A zUqo-)nk!_*-N~Hjd=78KEZ8gb3~vsdI}S$=41lb7?^|$`I>6qg)9k+}Zb2(bN`)?B z{Lb0!t&N1aZJC>2u#q&bfgHU7ZxE=csC-}*mw9R>eo9SGF=1z%#?WID?{f@}Rh$EW z+7K@=y1}!-^U~xbtE;FLupEZ^n)ml)Z(t>Cget8m4W5HtjF3*k*s)P>@sADadwUuF8X%Cejd4twzoQThCxyqr{3^V$8Mv=>yq`sf(&}Q9b7W@-FT7xrP_c=g z^~~1p6~e&Z?N!wVy#3?9`gywl-e!8U_mUhFa*)>O%s-kUB-2QzZ%$C2#8tv+`n^|Bq&WbOoz*)SsSB$LSu?|E3wYcf?0!i}{rT;ybbU<7Z~Z}9dGJ0* zI=#pC;+2d}=Ta_f(T8u4}sfypY__sX6}o<%#WC(S&L0V50amfN_i%*bR+1I8^{ z7^ma?UF3nJa^~|2cCT|LJ7Dn!helKK-{t3s%`s%7mnk1dOk;z%JiIMDDGifrgSY^8 z89PR;0t;vVwL9eRl`sy|b8{G5{C&nXxBf6qJ^YxpVXP`Cl6OY{xDrP;LFaHD=<9TP zxSK}GI*eT(4{$tgrg?H@>NpY8TlWYvp?FwF8?4HD;$8&1b4-Y-i+tF}?t)JKRluCR z=&Atg^_8pzg$l$_qo*Rs)LjC&&Y3gk-kdR$)BQGFVv}l-y1AVt_NEL9nhe>1J!8(y z-BNsO0Eu}()7YM7l~oiF?1TAMIANZzX*FUOMGdLUdN2>j>^W@lEQJVmKYzM=nD$`) zSPYtZ8*b&kqzse=H+)Co68lL7W_T}B6bNC~F^18Hi+N#b*yFv(Yubl(*+rRKKx1gm zpVcs&Fe2+KX?~@iR+l#-alT`*h~W|B#WpKjglE;oD(@SL7w~rj7_FceI)rn7%zG~A zxuK-7`ShQLhS!pC8zV7mBF0br?d*U|HR*@ZN6 ztUbCw)=eD{j0yKpmi=9Z!*M&m43+nzpOA$i@AtX^02s4*v&t``0fWT&1qF=j+PqiB z?;3d$j%=&X1lrG7Q9RU(5Y>+lZTc+ORS?eogBO10N<4vw#08A=N9%*<%^YbYH z>@-mi+d6gGC-~e<;6%Fr(P!y3MNB!;0~#>gc|xmWPT5&s-ypntlr9f(1U`P8${X*e z8G6xGVOm%?^5SR=jm)qcV-BsN*I+9Tn2iWSRtUKcP&gJVNC;@XgB@`mSc@D2V<^p| zpSnbh{F?@Rq+$@Dnug0)oLeKYd<-kY76bQ89`@g)a0{ zyiM>Bw#9p~?h&0r24$3;ZzE5c-63rWkUGU$-y9xr9NFEFc`uWN8*%#hjLeUFLBQ!y z{Bqv~2t~dG9fpv#bG{~-63>9_3QFzeW_VkNK;0pUF{V#x77Ap1+%Ui|b0v%HmwU!% z3Gl-lV?j`P>GbhoWwDP6f+?$*4L*nrURw&<4+%v%R>&(*+sgywv|44{FucXrkAgu(?e)49lm9A)BJaYp_bm4 z8QWM~qOTMOi39pX(<(6gXIVA!_F$i$Q2XIAdRjx9Tn-te2`1Q_js?Cw25L?(5+kOe zH1}qGjihBZkyU&1(7KU`>IMPUWE08#F@6o6Tjt9%TQ31SQt@Y2bD{XA0gwudzc@^Pe)^ogZRDRZXRM93M8IW1>6oovSw%K}(AeK_R?pkAYRWgWN>FD~-HImI=GrfE|0Xkry`xK(muk zmNZj^=D$XP%VOz;A7S{9KqOrR+D(W?i-dik90HEnM+sjO26^!KBa+}bF#YT#6oE;x z9)YsHwUyRMUY?&{CiiBA=CW&P@c}^f<6opr)^yOMO$7A!0MG8OKSU z6E1$sI+5Fkd~{$qWEeG6o_{JhkRk9aPmPqJI~|i5p6Su{gY@2m4`NKOc6TF>@04)& zIpJ7P0kTj^-+up;b0?8Wnt~H^MnMw%o38zyW)CoMl_xTpD1%K!pT z0XaO{r+dFgf^j}0Cr{XhMby)#FhYl`Ne*3`{aa=orr2Lsw1~K(!R!PwmuJsSu?GdG ze%ajDO7&^LVt`{_^ZC%GVKgRpw(?-5MtaA^PpP%80^ z{p6$rs@()oWgH9lga)bybLD;1YZ@G@=`3XNOJ=Q%Oq=th}VhtCOc#Zy#43BhfIYlgT1jEzX+dY}}c}w7dV3hwY_hL#Eu=3hj>13p;XzgKqv=9wTS`1uELH8_nCJw70X1STIjq!;sl2mEJ4-?SvGd6J%(S{c%B; zfK=N|%bO3=)33fvM=xHc)ldF{kTG2cn9nxVljn_-bm!i7tY@GYp79!a)*-H#Mu2^r zWPCZ~a{#+7a4N!XX{Q9$e#SM@m@B7ojX*v)K{uF3>KBoY{P@Vus85PWXgqHM^@~LgN2CBt=H{?88-QaJ?EPK#m zfs0e5P@tfEYvXCza&ss$aI|N+Jw}gXCT*1&l&>$cn#L*&2jo) z|LH&T3CY2t`U>ISAjhsjMNk{55^OOeDE&*qc7h?srK~a^1438Qk9S}WYAnhEN@SAF z?%Am3xK;70=jrrsNX340D&<;bfw?u$MoO7$aWSh8BVz=`EJa?S%FetmZcD2JJPw|9 z0+wZ&s<*}wO;#~_mX=EcA6;|wjysVF9Z~}yh@0nsetJm<`86sbxek9f3A)b)cJt_6 zE3h8at{|!OT`=?0topqL+?l&HPf>^(_uqc=G;ObvpiKpoeboev9t3}hAyPqa0H`Z7 z%9tYFxd*f;}Zut)N= zM;bktgN0*Ln{Uv-7!=muPpe=2MJla5AY?KL_(55qY4M@J^XxXZ-b=sw?eCal z8s5@`nSIR$H{1%aCv^DTR}Ecia))#rP+pw%yFij{9X4xybDbiEJ2(&|6?kJVel1FqY=tovNFm^tNxpK<)KEUtWogI`7<3uTivZBbP zzPg?^J|JZ!6&!oF&Nw_ra}L1k znYpsK71$f^F$vpNz{y$tNMi--fpbRbH&vy4o_!t(!;byMo7C9z=q&&&dS;4&RB||3 z>|^t*Y*-tW!;BO0Q}1TER){oeow>dMAjvdP*u;K`Tvhae$3u^6= zRgrNDz2kK-#DK6c&S+en5%rHhCu!NC@zc} z&0|a8GooNcog^b(p^a1teKxurrl-%Irb))M!1|7Po<5(mxwuWc4+y7LD2TXVulCt9 zOhsCyP$I`O4KbWMR7THiy`Q!Sfqwt&6}lEjg<`1EEvz>!NV=Ct?CTlim+|bsP50jW znAS|M0U+e$7H3J_*@Ka$*yY|L<0CHosVt{UqYOI)*tsO>zd{kv(iYXQ$QB>UuEX!jMdU)Up8iewjT|!8&ghs0H3FK9 z5hk3*8apLYy*wkpsscCeX3`F>Jfw*}3-vu8oB+DiK55=HLsyY702cyuX0M=p4A7KCqRv zUNiFV7;l7W(dnMgdTWFVxffV7DAKfuu02>;*Snpue%rCm4*HGlFF#ay@zb52+#!*3C5@xqns%aHQWq@Lj&;iJ# zjd5c5>mp)yZeR=Vwx`6wMKe|msk8wa1kSm_1Kql`@uRnjCqLZ|S$M|G7cFfgn=Q}b*^ z6fA&PFTjc&OaCXx8@aqVK+us1La=R;(e>2$I33k3^S0p(&w^mC0J06o2?$jzh6nv} zqgpg0HJw)Ah9d z;WC9aTjW1pq_e#@I-6g@? zJRq67bMYoU-+Pjt@OKqq9nalHw&v;CgRE9th)UeMcmIOQ!Z|*70yub+E)I7%vgkNi zv}4$dZX0knv7fdUIf4zA$(9;6`>oeuE4iOlrJS(JA|Q|a+j(j<=n2PqO<@=+8%t?> zg+tWYz}EslLQoxYxpEwj;+)A;%gE$d1A&0KoA&h6C|omF8o?<%OsitY?3)2|KcUBo zp-}OdysNUi;+qd93VIh#yf)}P=PV^7%aVRQ!UCLksd(~s=C8H zam%yxG1}(Dox^&y`27qvcMm3BRUM})Xu?s(hG|rikUkg@<^(U6<)TX4mP83>4uwWx zQvnd(M)~eF7#EFP3;B!a*EIkzSh0ypjp|bK;g*-{+5Scu4tnSTo~MK!nuo=kCzSw= zm1-FMxh2@#m;2E-ugbl103khJ+YxCj3_+>*v(yL+ZMYAEv(g-~q(H*`$K3H>6g#jI z?mu(M)Qg5kqv~G6s!;PK$TC+|qv1V}aP~uXzbfmjF?-jHRYt;anjiI=ycOM+fkp=) zfHQn&E@ZSq;5El8_+)oPiU_KTeFIa-{*_^@m+4;59*VFl7t|@gVGrvpUK1i6;51zz zR~-NcFUIpv6TvWN*(&Bq_o4uUk>BzjEAlNOun5kA0(DXuCtbapV^4S0x3J?SD%Z!R zAfWp_n$hTk_twyPblTh3NAV0TSh~A_eOXNlPn-|)-E4DYL-{d#ZkOs?k0z`#|10h} z9bj45hg}SW^Ym_c_QL3<-0M|Gpna^pcuWtcISIieq*JK$W zxpX&8vBz(zZj<$hF(4By7}QK&S+7M3zif7Hnh#E`I#U=Hi^O`#r1=%{!OJW+wvlih z`(_e3vW1a*Q?MrE6a^*}O@2hf zedb;;Vz@O<=s=dDFoCe{1!*C9`rdhc9Roh)YNKCF&ER)F>$A47{r1C@)l$Fvy3wN^ zj-dO%3Qb=#Tkp5&}#+ zq{U==*FB_hjg{s7jF~wQe9JawJ!>gB0u#u>`%E&2zDZK8Pg)O~j5UHwGIl7VTTcuD~RvREKzd#6+95yt2i9dKosftQZ(2$^E zt0J4vwgZ)pK~p4BDf&WCcjm(~5?9i7;qZ>_+KqF); z5A*!Y<1+EFEUUr4m9u%`-3&PcUMOml;XM&TIyjOv*`(KO9&?h)0O1L&SNoXI^VMm3 z_VhVN-QW@M-UUKU_M%dWcF1kIBD~frK_Ro)QALWN*sRtvdGBj)F%OGJhsUV^7NsoA z!0s{LB1ReY5r0FYg>mr=QZCN_@Vd+V9k7qZf{ z$v!GK_8aorvavN}=me&%{pGJnn1;P{%~0fdl&Vzt6pOXQ(NLe^@z8i7?R@<_Js<=& z3%hnR<`8t&wZ=`?@VqKpYwSl9Y>D?0Zufp|JT%wbaNIdz|1$~@dL+i40SXQQSm!8$ zqbJ4mlF!SuZGUhtee$y}($rsngp%x~!@XmQIS56wCr7Y?zyCk~MXJ4NrezcSnUfNI zzBKgyH_5dM7(E>hjn|EgCJH0Y@1%`G8*o*onQ+a}E1~jAWrg!?3A@4Y4gg6#fZYUR zjXWpT5D91R1Ll}BnW+NutpXn~E}zW<&EaLjK$_5Qh@`qeuH?cZGQ>LZUc;jvGt)4ED09}Yfh^by$)1#YGVaqZpSN{R0id7)6R_~7*gu95WY%mWW1_fDQ{=V+ zY?uPHyN(z}7CqrnP4mq0B74vMJIj0X0NV?gqXGISlDAQ|9m6AxC-dr&X~b@dzvsD@ zhBB0>9yT&8^B8QLr;;#sjWX$~LDb4vY|sh64%qH z%eE>ZZ7N5NWb|Q@&ygQ<;55L?u!VW%+59#Ct^TM}Aa!GkiZX>R@|ZQZt0DCk0KSUD z?#UVIbQ>ny8mdlm{58>uF7oMpqH;Uf71JHC6+BCol$HU-B|gVCRfIR3^xzSWo==RH zj}vP~Fh}MS<0q2B(ogs9lIl<==Z<|frXl+fAaQ4Hn`-$I=cB&GP5~Bh!it1;``Dp- zH9~zivMqJ4ZBYf$7Uu2>C#_6g>hhz9$Qbt{O{RC!NL|ujZb>;3K9>=6Qt0cGGK1MO%%WKpB5Tq{d5WE1FB^|NyuU;M7kxJ90QeZ>0s znPAko<^+!ZJ14;P6XYmw%BYJVP)20bTeUD%M z6x2AsI=mh;RwkSo4*m}S(lMG-Jca=93|9*iv}w^M%l(Ti*=53N_4CFq_aYq1M*Hbs z|GVkKFMdJl%SGCI{yckjpP$K9zdG*$pa6pfoKA*#a ziR~t&lUBjKP{b_qEL6Jv*;Hn6f0Te5V-eP=!v>oM$a#qGI-YZB_d70>+?pd&3g!3KS~f0~v_p1z@tfy~?Y$K?8S z3GpGQGZX==kavb8QKW%lM~F`GMm>nUN5ZLRPlhMVbKYNHCm|N47cqU$1?;4ar4@u& z1B!B>9CvBizB4k=fWxE{U7fyaXf;UJ-MwhY5kWA3qt$RNgm?>xy^88D(MkvzKm#3g;W$fF|k zeL(?<6>tit-h+O`C3Nu_N9`!e8V#a$Tn?bZb9l{YV2FW}pyr=!Qu2@h^3n2yf9t)N z6m3?wyF089^3sNy zSj2nx!1GCf)av>IKH#Sf8?zxqvj`UlQm z|#`i|2m z<|B;MDV)B#~DPb=ADpSe!5X$U84sM5)dg!|M2C|ll4 zxLVImXLv!q_wkx7sW077xhkyC&Z4HR0e3m<9W8 zAvNd)RS-t6NHbyqlQQU%qud2u0@6$N^aMwSjiGV`BgceVHUX=n35vLQpGozG3T1Xw z(l+r`Ny}=PIPUYfhgACNDaULtxmz77HeDT=>B9!dgw6oyZLeara+Bw?xapd|IVP1a zA4aap4O_rjW6cDhvZPn+p9#RA^4~`e&FSL9$iB}+XpeRCykQ*^Wv34?HIL5mtdIw0 zPdtwgLpX3IW!C5o2*cJ!PjTg#mmbex4$=ux0zq|!f{Yc;(%q!Nwal$s#r}H=O9kDn ztni5FfAYc5VN=Yx;d0p%i)_p}_L0o%Y%(_jaYM30n2Q7QM>TB9g)EhQ=S;W3lEDU| zg94S}h~b#pkFassNOwqEka1}u?>WGy?9E$(G08|H=zoi;=i9P-7>p!JWfAJqM&p6UG zm=>I+PQRJ*Jac`1J{=xXl@1`jXdL74aikVZaQdm&=s%<4d7 zN>%yeUf3x0rSlW*rkEE&h>BV8-enz3O|aTf0A;0R85Y@Q-GUm^5bQ-d2O$w(%%uRy ziftofW7xbQsY|kDap*fYl1;a7sC!*NZqsqpK69yUlGT*Om4)@_FKjPPUx|BK6w;(e zWf8lKL<4xpEhT!=p@*he#L#Of2A<4R4LMo?l4m8GPsx%7Q1s1cM#c&FLl&xB;~4hR zQ<8K5OhB{03MR4nP4c5>&ErMR0=!{=wN>GnIc6ltCepT$Qno$1&WMdN$K%DgCt0UX z+g!sR#>U5$GZ{bo*J-cKThlQUum(7C9hg6`BhkJiPTL?F!FmYjEJW<~TClLw>F$R= z#cQjx*BaQ&j4=$b%9H7gq8J!AMJ+rFiV+5v^{X;!^pnNArn$J0+}q~K*(=m_(TlS8 zY7zOi@2PvuEs%q-(r$X%icXhpau_Oji~MJD9JT2dXjx%B=32JxGKy3v5gscs;gcZU zBAaL$qrFHI+9@8)Jb~X(y>kwju#(LqZ^kHYn}5#wevEB}$y%Vg&d%S4RM9-)E%Z?s zJ%k^}hXMis65fiXuw6cXg+Ur2G+}T7J66P54K+(%lOr{e(0qa_%3MqE$3O1S<(`%T z&GqokR#~_lKys4%X~ZTN*C{NUUD4lLT1B~#)IM&f?Z+ED10|~kJCD=kvxhMR9R71Cnt%Ovf-e6t(5T9bG<6};0CK+q1>o2!Pc$LY`CJ_)pbO`eN~ zMVAQ6Ug9z4Sax% zp+KJ_%)y!|$QTln!fiGwcry3t0+8KjEr&dpphWN8M!5}+%u!t3MCSHP>Sek%2;~f$ zOD05P-4ampK2&~e4m}=OR(r8|9jgef+O08e*|CZQWf< zKl$(u9cfoF(l_b(&R%-@>L`8r_y1e^j*#vUmdxtc^{w@E|KYv#qkr#<^rK(?G?B_y|(}iT4ca**|+W-SwSl&YE4psIl>&E{E#jBF3L6!2|ablUn?>%8iixn zu@Z)R8s?=AD9d38HP$jYQy4)RQ$y){u!^z;voH`Iu~o!)_)t>-pSyr^OD)aokqMc0 z-BxIb3OE5HN5b-ugSBN;R|utE;^5TD_jKM)ScfJ#O4<6$+E9IFcn~(<6D+JG9>a7u zFUgxc-@(YB*iify3C#cmE~)4&5-JeLnddeFjBj$^2e5lT<3x@ea()O4na^?Bg$5rP zZsH&azV!ZeY78G<065K;5?GkS!w68K%pEdflUpW>y`;xZ~UdIBqtQfl!$GIcYAnoTx7Lr)4^hkQnW!fcbq zkr^DlbIuT8El6iTDGL})cVH95?=lzW$E}{tH(1VP7W^DtdgqY2?!ccnK{1&zvct#AZEFRu=MC~ z(?_gQ_xL6K426!jY@Z&LcO3 zQmb{0IE5lF?n{Z~`Q*=m21)tIW2HQ$!-rjXFBAvgYuza>xSP zmT}miS6P}S7yJhMYZ%XK$rxF56LQP%lRQHkwrvgBGL0etAos{vBZQu-s-HVtPu56= z3d!N)>S38OzzUzUV}Z=K44ALGw|rI|8n2=p%P1)$!3rYx{zdlE_jq4VW|#!{dWN=X zDjn89mN8nC`2P4TdxO{g-8LUK^A+Iue4a7txXWC_0zl{6*F82f@~YJj=uvHnd(~=n z?bhPTYI^wo2k96``eb((0G|~G%uLg#lk4z65!KR(3+Bvc;rX>~k~Cg(*dwaI95^ob z4gZH7B43sFdT)WFcF5+xVwI2u-V@iy_BhYHO+y#+!*4MsMkF(q(#K}5U2C30L6PNG zz9QPWxv@?T4vG&}?1(0%Et21Fey?)Ft3fG8b4?a|iO`bOZ#sT5GYW#6bByHJnj^>L zGtPJe!{#`WRg@jz)HUT{Ee-Ze#0EJrB2D`@d2AQDB+w}~nfpl>@*$K7W5jl$&>6Ui z?H;zhIJ1_fmt-Qi#uy_n;%AKJK0KIPV$5{(2gsF3%`XizqMR%JNC1wROV8Ktp;(;-7p=9y&?Vnc9U zLGg&Q{8%ySEHLJrJ=r3;9xM`t2vzPWSYT3^8<<(FMrUU~eGLO=2$X@P2it4u;pP(0 zNedQ^DSG|tkRwmdQupO~`VEG|zCk)md7Ot5Mq`qYb_E8YzPZ7(5Q_Z-fX2A&3|}nD zqOrcn+s><3%o}R}II96{t~kT>mTKf{0JvvE<}v7HXguU*FqGU)k}OYC*o zy!q|>U|{k;$fLWa5W#Q_>VTgB5%vjUkFk~ZCwbct9SlPoFkqGHRBURL7#i?~QwZwd zhav2=0Iq(2iLtH+FzW#7Zv>36L)R!E!KeLZ-Xnn0XpdfxRVo8{>TF_)YkPZhgRUVN_3|PqVp2uL34KFV^0N3PdF0GtMUI6Q;q)9+^`d$DQ#! zGH`VwExd@#0K%0IXYE*>A@}pA=aU+)cCMXg6W>jw=cUkdE{$PEbI)u|bAv8$ zEp&21-vb_W)J-#po@fAS?`5ix=``NoFYl{;am=xn+*^Qe4>Z$W1TX@b03hrG*-7QI zWS*-)f-&fX=m=>G)I*Nh3)-X4{pRz8Q^8MjY_l*WGRJ3~3xSSoqwlc+ay$d~jFS@h zd|Vg&k%1n@#eMu;J>P*XJ|hiAAY_3?g!s9R5r8Ymj2Yz4TyCFJz=}phP61?0b&sf> zW2k~V9H>>%0%@U$xy*|9h(6Ah70pMhjfnNf`$kkvn4Tn@f<$+Qe zi}bR9z^>5pog(&E#G`H6IwsaV2e_DKoF-vP>pCa1^q$u^u)ZTgZ*E4{)=jS}-PV!Sy9XsG25pPhz5FlqkgEqPB&cCiLC zv$z;N#->py7EerLkWa~-5UhT@JWC(3!*osi4-ZGSS!)4BXVu>TI=FzRYVSJqB8tDm z0nu+U_q>OB8Fyj6uplfK^ge6Pf)`K(EIV_8xFZl_=0(us+6Vyk2$Y1#so)QS7zNBX zMKv$~j?K^Rtzg{DwJ^4>@Cg2LEUbTL(OF}ZKq#2Z2rxv1TaSy4t(YoEdLtNi)d(|; zA;nmrm_263F84LQjq8sA5HqZ$3E<{gT7BD9FcCndfE5bRC_55OlM#N~QDAX+df7m) z&12}1IarMk*Y=pDTNM-6u-ku$vnV~*&FW~208DZ=QML+`Y?AvK%tV#k!aN}q8zl<@ zl|u_iH0rVzGF7*Pugo#fFj?t2qS$5xLz>6pwEz<`dMYI6$`GZDA8!g^G3MVMouK0{f_*M~||Pt_ay%VQNmJhVG;v z;~c&C=1=MTCFv&^?-FaZOwj@A0=Dod_1LGL@!KNb_AS>k-28hw9@JMqhuMSW;Q;%! z_x~~tN!4g0Cv^;UA7>^{YOuZ9s^n`aLp%ZFBZNt+3=czR0dzj#wa>oj06NH2GvZ)v*!Lm0TuYGUMF%=# zTraN58nYnW*UthvxSft~fD-nmDl!&NytdMFn z!j4*$BvU4_(dFR`dwW#a48{|gSpcYoKSY(kD*?UXVQcFD=OCH%VF%ycIu;-Qi zJ^aew#BUto7>6S!JR#Zck=m`E-bi*g$*N+lDC}z{kO}09O!)UZw=m&kqk+o2dGE8F z=bZo9{f;ds?Op0xcw|j_IU&Z}{nHXJuZ!>UhVrnD>ryk7ufQOT(B7{CUIAQ_&@*|S zYg2I@;W4^*joEe+uIs)@^8q^jk{ihXIbQE|Q3H@dFNP~*Rzu`F*u}N?M;iu7NB` z&B#2n#Ohe(JkNyO$`cZjfo>x+-g^mLt6p9OP$n2MIZ4v7ygf>n-n zv)FvTjPA}4}N?-r>=c)PPm$ZSINNcON0HCML{W9%f5s=XgxL0HS zETy z#A8?{WS1IMeNym^vuon2H8yFS=X98x0^@=;%a0PU&l+msvyM-+`d~eDS1WWbaqkW& z?h8$Dt->sG`zTDJAgKi7YD3gP*bsaRX0EqV0V`0`RIQ0url1u|fQCTI%xvB4tzV0Kx%{#3`A{X(Gi1LiKR}5THrI3d+$1Oiw0b(bv$xSr;2d%itB#6S%nw zB*L@+zoWX&0dWe(<@*b`fMu!>6Z4HG-zG@_;@ZTwnn1Fb zUSL($iic4h25-}I_LX&(Z>kJ>&UM@STY+Io7#_`M1f;X?W>GeYJ@LHA8oTt9bzij* z_k(%EKx88Exd1h@N-JWt^6Ba~pn945bXCBZG%fS>iEvV$!%jN{?}q3 z8-UV-!;`R%jdG?fkD)TgF9?B2A!KCmmdNoXwbcn>xvrbVq!6%B5I^>Fm0z%uAw;!y z304^lK*Qi~%Z4Ceh}Bv^DP3BS4&WtNVe7a>8vy$v2fa@cfWmMLMcEpJHR!GeaA#LT znKLoM92RN`sTF^wCG}VG)0sHh%wETEn7v z1#ey;8?*{A+1}q~ok+Go==bx}sZ0Ad+vn|Y1`nZ=B0OP8?b13iT_ckexrSGmkYEwp z3hSdnDV>%HD?-mot;1dT`2*$yh@PKG2i*Vh=RZd_j;Hs&^+~$*;JviBxgFh!W}ycy zR`P8vBPxd`IOv{M=`*fWAfe?o_LaSs-&*EGP!Rje;FZIDz84c=0ga3EZ@5y(AOroi zCVTFp)C6?EhVZ{kLG~QDNm2fSy^#+~e7dWz!L;#Yd?}vedZ}1w6|(z{GF2q`!V`UkrIPXLdUkD0WEz97Xi+>qqI?wmzEYeFYigVL zK|!sFmDe699u;ffh38yNQ>Ki<(s|BsxR}`!!yg1kx;brD;Ka3%TIk&Gn(*U2bwh`T z100}&u2IXibg%q7ERbC1o{34^g8=QW9r#41<$5^}*V-hq7Dfr@tfD+YOmha(-Jh6x ztP>>0exvxXC;3(4WYjEKAp|eQ@h}tHqUo3gZqkOT2Pm@mrgf;M82thQcDgu73kz85 zD8S~;jt_BnVEJOK6fY%-fqsp7EGRB~M^&GDN%W%V8rsY`gmsFh^VnDlMrHznjLe40 zY5?JC2u6jV*RlY!P#_NpWGx@Kp_&!hWDdau2d~41>((+u%B)Wh!M_C%{Ss9Mz9hN5V{1ij#1?I&in6k;0fmB znN!RYWi9O>7!UUM(Fi-~=Dkl+XKS0vq{p4T$6$OG3UaVaW4cHj1A+c_!1csXmU$5zQrgtP7T2`1*HX}Mm z6M-p8WbT*i$g*G^&0H%iJqHD?gEy`AkKE7JjBiU&U7n-4+2XezR+%rqpoL7Z>XLCk z1x|5*p^pxSMTOKW+e@DaYg7K4R<)-8BZ*3t^&`v;oRp@M<_?$>VhZ#&7u#78cfppIGyUS^lUc6>z zY1Mm8z&^(#ghLTJv-a6kX@Ns^N!bZp-pp%+wj@KwF<|bSn8$S!POx^;ub|ji1r{p6 z#2IecEG`Nq1tp?Q#-v$(_$U#qn8`T@GBAOIwedXE74B7H?^RH=eu!Mnva?p1?S%w` zX3px?ykXA!BB=qFn-P1-CU4DcS-*K$T-?vbSe|0Ij8$Sn>qy2Fj1K??$;8ApK2fO$KVnc(&wK<=$8+3UCs*YH)Yi_h6bzufO*Sog0Coww+AGvvA@>PN1utuRi!4Ex8_J3CD;pS&Tq(RfGJ z8QyB^zyaXc*vLHRcOM=mFkK;YyCeT60iw^jXb7759lRP^HNi`s<2f*tMok~+>!r51H9Km3>fCe2dD;b*`6 zEdA(jew_aBPd^2KVX-4v{9xlvTEqJ>%Nz|+8Kn%+ldrz!+yewrR!(w+v5enH9Y9(Q zt8${7Ppz|+ZjB+c0EX?I*PI31d91d*i324Cw+&s1F99t)alUSBp1R0B8U=Hn@q>FPM^NR`Jqg))aUkGEa~s_#Z3l z#)luV2JX0gN}jCSOm~@kF1rx``I*)rLnc^iR+JC?*4wo=EUAdQ*_VJY`>wzc;^N05 z^D5PTGlKg8mOeu|H0I3sx;Dyh6+Ub|&NBbj9VvZTLd970u(I)2U~-V6>Lcdf8jNX(aN=TFI1M4@4oUi7*8F>rf^0Pi`rgi#xbza zZ4S05ag>M5A>UR2d+*Fyke7N=YtyN;L~~b|@^bSseetVrg3*>S#hzNs55~Yg5Wi(1 zLbNp&%Pq1Wsc6^_%P(T$1?{}Q=fSc<|b7SX& zI3N}_!YbAVw`)P^vZ)c=&W4%tuJvt#U{-h50B$QN@1W5S2x`sKl(~W?S9<^ZY3ki~ z)5@bqXy?p<_1OUd*q+5QDhk&@nV69!%AVZ&7@Y&}c25N3lOb+K@Bbz!1ErDx=qTZdU-KC?HdyTn1h1i%>}Y(|pAw1~CN z4R5w|8B>c4mzR~K%@#GYcam5kt(bNva4mI>>T=Xzr9}e&1rOEi@PR z#P67q8YjjB2hcDc#eLF+6fRbTs^wzLAT;JYyegKFn~m`^VsM_s`J?I{^)zDP61%GGL&BqZI>Edvt)iQ!tAI; zME$@TG&*Fw_Og7uvnwWp2@^1`s;sug-fJDn6K`nUqauk2mVr4(qpqMuMz2!9C}p6q zhH^ysdlZ`MA*2Qfj{)G{Cg?IfV@N{zz~bdWF@Tg)3e!U9VTCbrAklo)_%-20=j2IFRnr086Lz%NKz_AWiJlvOYKdh;J1lkbccg)X#1PJP( zaE_Q8*>pqbqhJgm;J$dMVk&L_QBv3d;vWL^5ZwrT%m4JTbNGA5aY zV?M8+q{k>2zxnmAplJfn5&n0GUwi-kx%8ud@ez!X@;W@{g#N>On|m-i0262U>z5AAzk|eHU0y%^t4cG76JF>nnfLN>k-L@n`Ct zw8`|tGWB9V&b%}qt~Jh5fLw&5nQCLKO=OGlx1CSydJ-@0IXVTMqZ};UqF6HLER;Ct zSo(#)gI;u-DDV{A!K9%VwP^2|G40Gfh|Mm5krqy~ZH+}$xHSNa=agXSun*o=3m;z$ zA)#U;E!a^dYBRzwcnfbgz{JsnfcA(XNF6U13uL2glUMMDMaI?0qpXbA@OO`MV?2`K z9fAvamrke(i5?RuH7YtLqF95?q#Lu@mS>37VqWLa-Lo$qQ$UvD$UNUNEHrw<+kr5^ z0+7ENqfEj{+N5m8H0!wve_aKfn6bKlfI3Uagt4r~yEaL#34nnRiP$H8doIk94vU4L z7%R7PR#6bx2|6Q@fnhu6!xQgGQVhJk0w1+(QVXF9!J!g7d> z>ekY`4?j#l|BwGZ&C>gE8l`CG%_}--pllOUeEYq-xS{Ilcnf7tWrnQBGr(JO8q5Cu z4^h~rQe*QKXB<99GK*fG0x`wb6Rqz&qoD2s)*P7xpRjFT0s1JRi1zC{sYSN^IO{)T z-ddj<#1;PFCx0CPbo2Gg^zonl1@z1uIj0tzogkdSPAB77MU{8gpvQ^<6k;( z!*a0sW-sDqY@;+mnzzhUjlw77aL?&qXR0uN5HVm;&|xn@ebsKh@dvUjgY&g!m7FZX zQy`$l#SEBmb>U{!!scX6AP{)WV7-|sP!fcf)%yn=dV{@Hc(ov@6_hD=(k(R8=F`v8 zcmAut;eh^W>QB>tg~h59+hL%$Ze+8>M9>lF&PHm%vzJ*BR4lWyHeCUYtykOOpt7M9 z0~``Fpup=I|3m7t!!TTa^KD zftvl1dI~g|Lf`AI*3Tsq^5l_L`ISRpG9kZWL!<+@r^`qlBOiPRNk=>cYU11oh zEr+Brg#v}>xC<)h5Du|AgtY%K|K>x1`r7A=kt;I>zJ^^B&-W)Q-_@>m>d zrzZf4a=LSC1z}^%9yRs0NtD5qeukUo6pdHQqRojd-LQV=MjC?#3}l{Mo+I>Sor=}C zL3RPKtmOa@XPWyEfx1bf>*fjL!ELJBV~pBTTD?X{UeMZAqWpjipi-IPfT{g)fU$Ji zM#%CICIgfxA23VSTOP{-qM7~W(7`Blx3vU4?2>Vs>Oig|8yacr9ysGm1jlm}Ev|5g zOqdd8}S$&{c=tN**+r!e%XoAc@8@BT35 zZr%mll5XE4TZ|tNae&MMLO9Q}`qWbE6Z6(V85jWWEL7Zkh0C9IXVa|pB-yGttiA?Y z+v{_|xGVNj=1^T)N+Tp9)wvHxixyMR)9?%swMkazX7nYsas2#BIj!Hn$@j57qi}S4 z1peYu9-%0!)-5f7TGF&+buFCTti#wk3H1%y!Qx09(m2~q zt_K6}V8PCF51A|}K|$Y^o6)eP@@3nlODrrL2fP53Y+%5GyBbQOdth_xa48~`O<XZiRnyq*!Oxh+r$}{u1yeTlJi|SCug= zs!`iQi@E`0)?6I}WZ^R&K{y^Kb5xS$Mez;L7!u4pf#rHQ zc{9y@^uttLy+f4&TxoR5X|p*4_%u=&#Y~_?SaeP-z_w1p>Mhyjm3Bq(4|} z(3YfZ3}(1e8E0081DBicK_`M2>!KO|HB!X2j`G#M_$J21L-`-$;Tp4A1z6Itay3TI zm>suhO1|nC{e~_=<-4?RSdI$3-m<7`#uy;dpnd0HKhR6!Ai2_!##nhmC`$s*0z5-- z?R<o`6hshN*P|o*ewEX zHzw!e*$LSj>l|!@=L^gLx9}`x2H&uC7;=n%8!FI-u4h-)ePRIFQ3}d?f6I$FE7yVl z@eIMV>%wQ8CF#Pr*n#m#+$RzrxJHm0AqXf=0#*%JVR^)`rB&1Mu~PA$>t^^){-qgN zJZ6M1jau!OP0%!1q4~CGpYu*d7yQH!<}Wu00!CQ0`Qj+d_*5$w6MmQ8iT6Ie8ya7J zVj+F^C;ukB*&j`N+s(A|;_LM0>2K2h3E&21VW7EjR=P#U`UqbMm=jn5_>_p1F`(GY zOEtD!zK^z)72CDc#W~MjLHI1pFGeN!bF?LiTEDR#3K_eafdTFf7>9wuEn-dtN(PvZ zz*r2}Ewd@N*4NW(G*45t=Q%7t`#=7DntJ*K?e8aP^}~-?3o@~=Kp5Mm)y$v&r=!URb$7d7FN*|p931vP}s~#vM6)ZmzhMXFyqIAs+MqF zSkV|Kv*yB-3?T^uCeaqlv;u210fwxPE+W1L0SPL#HbJBwAP!f7tk0psi8bhA1!|nZ z1kjuX#9bJ?NedE@Ry74BWD$o^tD-67Z5Wx~^{{(?gYz}P8@w4BCUFrSNZ0KY040OA z_+qhwj~-zOi+thu(VhU(D+JAO~`O8oxn8NQ-~QW!Z`xqdyhT=v{%vUupVq-B|0WJ znJi$lp+*7tAZ}c9x|ys+#d>jA_IGRP7r%IdRTj|Az8QROHk>~dfM=DuTA_Whk97o| zy>vZDB{Y2*q35OspxLE9qN8@`wqXjv1VfE}l$;}DFLm410j(ndnHm6LpBST61m{8( zs{jo6L~z2GPPUKI-~X@wd+HM>TAbzcyAM)v4tEgNi8A*xmQ-dlhNfS_tzX3&I)o8* z$;zyZ@SGD|dl$!P1c7%-O|1^JWBas2EZx&s7NxnJ7vH27&zz>$D0&tQoFdoDxL2{B zj$*--2aFVXfj?+p-@G6P;jak1mlD$3jR1H!4&1{CTOuu&>S@DO#nz`lne_{8*A&C zFaZZ*MImGF`HqT{*M>XI?{a_LrIcNQ5d%_4K#Ce$f-bJpYN9noP-lxDg$Tn6I)_L? znX!yY8TWtGCw}h90=H3~YOTX`g`!uYjKVS&i~@8yfL9I? zC}t2U2zTO~v$lNSWFSkN=ngk%+w~S4GStWbc7elE!{A4u`)S6kvbcB1I61F&S-G(| zgHn8sVnNv@z*~dw_gK?io^#HYm1kDdB<-EPc=8Qa=!xk0Tz~OZ%Af2}V~Zl%2+N7Z z<+S?V18OclA!G;ZDog-7NP2@I=kBL}nx^kON-f-^Ezj5HPRh@dTn2!enZb>mf5EsB zkYvIS8wc@YM(HRuzWN9;U8=8t_c`YSP)AvuE}&Ppn0&4Y?T&HZGHa11gl2APKJ7ex zoQi-+-Ckxa>RL1=3630eAZU|Ttl6g$s3pp96y0v9Q~`YC$pMbsDd@;>f>Gpy09OI6 z(x^hMMN7p<`JiRTd1TcD;NR>On;-mIWlCBXerUPs6F%hqRm`OY?dU3SwAm|7}*V+7%<`T7PQ@(gLQn%C%yBnNt8T?q_I`V>)l^ z2^1K!mnD+77_WTaUbRewIct&hPZbX=($pA?dGKt1duV2d|%fqFk$wJZYYhJ@At6I0W+IBor$cp3!6YqZA&TyN3Z>c)7wbAK5| zk0wJ*+n@i{BkJochR~&TUK(E*O%GPyP49hsHof^~jtA`$JNHg{gJxwZ?LL}h8MlwY z6??J0o!$_9Si^Mgf@waLX&DQp zwa^w)jVH&jN7{yXn_|_xCA> ziFEz_hw0<*|4F)c^G*as?`#w2!}%za?W8@R^X7GW_Pfv0^Dll)oJN=?q3g=O~eIUx_`%EKx&tL#?iy2WEP?fzsDfl3mu39?+dZBP>f5k3V0-wQ(!t` zAz-Nz!DV@iBP@cJlOg0QV-k!-!i+v~(8eVP{(>ZsbH;zhbqa@5_TScF2AvKFde^=8 z^Z)yoF}D)JadmMnee?Vc^L-hIKy$SwZ;Pwi>voCw&AYc(Nwk3}!l;LUF{1@O`0x&z zAyy{nU7gE1LJbOgwpo)BZpL}6R!!=5?UP06ymYmW0QAVi!)>f30__^6q2;EETXY)1 zb%upPOf(JzWU}dZu}dh z+Tr5#y#=g##zLxDUe@W=2q3m^rNQcXl&M+PX>o~6*QGh;TBhC}J!2n#4VZ1wZf=2R z&C^d6x@V4q9F`gi#agUrac7E%jhf-AhRr1 zGFORKI^8yEdgEBUw3rH#v3$Bqw0_7$rDKag zD^RN3+gLG`AuV07iio*oc^Mh*cH&sI+Iw~cTC*sm)#a# zmM>qwCZ-lej3*g0G(pJ7+O0dp#U0^lCti|dwH|!4%cRaY+X|8rB{__dvQ&dHswx?f zIQKvi=Kg|R0gwF4aF*a@@DUrlpUV5-9m}(35u&tUF_}PFf&Hf~l;S<)OO11j@WF~o zy)yuC@AU@#RL|3gpWX(vx6(IXex8<#Sf;I&H@lQ>{m~z%fB1j@uXK}`*#b)OF)reI zbtV1zzxyAtv~ecrTC?%wv-H)At@O!{ev&Rw&aFal46laJ7zcaPYZB&yQ0Du_D{4tD zTP_0S!g#n#l%^`78Cu^j4h~U3NR&gy^^kv+Ih1kMx^cLde*f7o(qI3(|A9Rv19lnf z6?4&(q`^R5slZk4xc6E#2PjvbA>FkqNGi(^5$8mq&bXjeJsgH!1aRYfP*8lo)oySP zfgIDJ9cD>ll1}C4T7zxOg@EBAIcog7Jd`?&u_w)Q4ZZrX3h`3jtH^z^>d$sNg z0J(h_OL$mZufmnNNrjcltSpJb&&3?s7mzJ)fu-Z~Kf#?wj~skTo+VIp94bVvvHNF< zmA0<@9wB4Q^*VIHHI6l7tln2^U|!ALG7Mw*cV&ES%4l?mc_NC8%-UO~SD_8XH@5fFmoGLDTKf@zesg^-ZES6$ z!FezY7FN?hI5?SLmNM*&fMVh>#w;KiYYD_RMyypC4etu!b&BiZ^0JAh4I^Ui>LdmX zU$My@w6Xv6*>4D3X0g~5wvX{pDML9AuEL znc!86Ww?2Ho=>q1iO*Kjjg#&l=5G&-2N$VjERgG?0;jvvH99)R9S7L)<{l)2tv$M? zm&0DUD{ISlBtjmD+s6Y4_*Q6ej}m(!KvcLzDF%g^Ltv~N@gP!}fm)>*f2N6NZUdgl zoZ&(7+iWCjSlK&Oi=36Hbx+Pf4+C9lJafR9{H|ZA;v(^Ys8Qqq2iwFIPFob3&((0W zplF31B23hI9=ecFZyL8J~@a)izsXCikzfB-hS4bd@Lw zU6omK2DB&&f-eav7y|xJVGL~!>OStFb7C{AFp>xN-{<@phtUP_64%ChI`gptv1T@M zw@~)Xa;!_^Q|BhC_9;lE>;*EFROW@iZlAd|Tb z0F!~;TfM~tuF~soC~mn+s~}wRc4|<#JX)dF+pVl1>m{`WRgNp;><5GDz?hgdLP~8M zIv7x+sRgyOCpJ89(YC2apzS3*!mR2BVHcJJFfLO!f*P_G`3ldm{gh4C4KA0qLx8cx z$T$EyqlX48MyqV`nGs8d>uQ4Ym_uk!k%bv?LeeHLd7EWibhE1jg{75es>$0*NY@6U zl;Cx&;Sk~w`ZfklOP}3W@_M#9JEC%CKK6*cpSZ46;6+F`SJ8CO^_ zK?R!g+?m0B_L?x1E39FxxyfEl0`#pbct%F^Icc^%#so(sX>2?_ym@=HCUgJ*KmbWZ zK~$5Ve3?4kS#aAg5{o>_I&1Z@X4)y4nrEkY3jn}!TS!==X3 zM?d`MSSD6zYL3SUm*?UBCS&Z}2Q<*MY54e z?5k5u+qwFZUVOH4%jY?^h%@6X32g+DhOjsW7fDE}#mVa|Pvf&xSOhgDYfSS@y-HP# z34JW@(XK2)60|&VIrB0hY8tTS9w`$YTexbuAgUiT^KmV(yg7s!E+$ZOV5ac?Z z%Gv5Orc4&`+cko>xF?sKO%tw$X2iMc@s60sm8rEDV<*$Yp_gr47Vw9hq$I>N( zgc;}-eYkc5X3XDc;HOl1Z&)N0%g~Lx)9FW#{wNCHjnYdlk6qYm=ifZvq7pl9O%U-6 z8d?c=o`LqG6!$d7r47SgS)NK?qD|%~!h5n)!+%{*(=fDYT#^w_#qfI^XtUOu9IzZf z=Qn`1F47?i-LV0_G(A3l{+fknvSg^J zZ*)ue9k1HY0*yd7zno#tF(OjhV;hHP7 z-HnJ#f}!wdjN3~BT>C5pUZYRJPd>NakBmX)$sz2*jVAZ zxKZ16nuDWIanv|3fE5^-@mSJx7ngGfU}WNg^yo$Is}$(@l25S%WNy267Ock4%6$cf zV|Y!Bx57sT^W_}Sq&zD`tzhuHaYg~7o<%K40!3N6wC_uod7of(=~Jtn9WRV=9KpS) z)rrUQ9s-B&TI|>^D+*&X__dOHZvCxg#>5AEEEiBPwDd@ah9ao+n7n0Nn+X)2A43%s zWX7V-k;PxFq9`%8d6bqe)+-yq&(31irG}Y%5H%H9$Fmw4xPZD70CJ6F1)I=&b`>a| zzlriRF?P%W>yH9VdQy2iAc;jq!R+dD=u5@Ch+?SqK-q6*yzW8grnT0LRm%a{aYtoG zW!d(CuA$ka0%Xr*8QQYMNRuCXm!1Q*TRdkP=6K2;Hvu&R!7MIe>A_@-Hfw8n6SxJu zlw=~ZK1Tpb)^?k@|0orf?(;poLePu7YmfS4>DEKq)S=j%0Bk^$zanGyHh+&>^#!dK z@1z@R?~}z!TR*_?0ENqhuDltK$gF0KE*7l8Mg5SPoLJzn%3PB9twpIq-Oe2>T{#4D z84K4-j1oCAS|@Jc#-@@$2^!yh_5$XNB4h~}3a{r;+I5{5sk2wbD%fgaDbvEnddkmi zT(1>J9-@MxP`ADr)D^TK&4t3t?@RczZo15U4}Hu0eDPZ=6+7ymyx`98&*1yW0#uyv zR;U!qOXEQCsR;&lwz1BbUyDNU3LIbD$ZNGw3mQi8mY7)Kp$Krm=UNf3xJS6Ef~I&P$9}As~2k%+Q>CGvv6m z;3%|f>qjkxu!P{%*&?WTndR%5$&$`mWX7qWF;rfek)a>+acr3q&jjOqN6cUhdPx7~ zo#yq*5XP;@u8FTGBw0smyD<4%R9jnQ5S|raY`t0|7U#~#`arWGEtxMKdetO|5 zegBU?go9wYpm^j3Za#~jHX!64fMsc!)Ncxa_Fy1qF!w1m(R+{X;u4@X6e;%m)H`cZ zSEoZt`vikE?fTyB<#bBgy8SAr$a>oO{mWF}x=2UF%v=!gFajttNZbRhrlD(1(5FT> zfu?p2h@0fl$cRi>uqpDDCTIkynP?^gL>3&=HfUdsPsZteBy2Z8;+)6DbV^ZD7sdW^ z5~AjQgXTrQEF%QPF98Zi8qp&oHS%4k_*UhZbHo#Mwu=u!SWM;k1WD+fWz@ zmbtevZ%epV9({~Zp^et5*0=(|eqjfO@w<+pamZM&?DZ_Tn@G z4I$Ii3I`3WtW^<|Dm<(e?s}Wv=b?SiPnQfiGjwwTe+m_Mpp#i#PGbNXW3n!_0OD$f zU{vUtgL@2!LRi}6V$kPVFt3H#nUs5o`-u6T6B8xlsK5)Xe`ZWa0|hM-jR36SPUJkw zjm}}7fm3V}Bu&H&dWW*BMV^x}$TBJD@58W-E zCF{9)hWy)JzgEZ3p)?v316e=^x<1S-MadutzF9Yp4qE2>W^tcTJ>U0jrrw3Gq2%)n z=!6G_s|0~I2@|%HjHN%MF==&(z?T-ZiWqAO0R?dPTzH5qXqsaojTb8cCe0qTyoOny z7XhZWcoT z_s}v5pVq4aaX`1o5av2)oh2f^NRWD!rf)rpz-#suE$@9F=21u)I@Hpe)gq-)5Z@ z+)NK(mjO9>w(fcbdz=BwXLwqJH?wwJC$RHONRS#YLTTmR(yNwTWuS%%`LUqN`xgLc zp0BXv@JT2U@B)(s1q#;C#L_|>*D8rjhm6_pA`AJhvc|h#|5lmx`7u`JCJ%LfaqqYD zk#AUB^qq&!xFc_AACWd4S8c=+K%NDO{6l=#jnbKZROig9rTm$<$@-~AE< z2T(&9wK75xrK1lY&jaw4U9MfYgE1b+hnWrEBP@!!#~Q**l&A4DC;#?Gxn#zwVN}k= z@A(t1ZPs1-@R}yxyv)Lx%`w0-SNd$nFCWWa3`hZ%grP)T0#jkY#B-bNi9akt!Yj3L4qY) zyht!6C&&OV*p!=ScDi@8uu0tlGMQckU-FIxGdnO!^$Zz)G`IC>ta+JDcT?5OE4XyS z%_yNj=swe;z5bz9Qew3A6l8`VWGx`Xp1)(B0$Z0tSC53sml>$vhGh9SYoO5OAY|=Q z7#H7{m}E{YXShv(st5p{%U(M^c?nbvbJGN!F$js>b@h9iq}A-NAgX!X8<&7EgJR_g zf>Z?%e>z)K0`?4nrH~vlP7@HWUB5Ch$ITnzT06K(muO8YQXW7n$|WF^GLI3D!j?&B8vW3E5lVgrX!y<_y~jC(8GyD zXzbuQ#ca>0M|eoT&m)xA8fQ=DtuRmlx#Bm(%C~?ji~*7gAlL<5c%H2#C+*B(eO*A& zog`-PKHi3v)p=aGB$=?jCM{?Q$xAHI%oTGwqGzlHq)S)0>Y;<c1_K%py-Yp@6z)M<8qOWQ$*`EN99s zbO%4Yf>&DBV)gbNfM_xO;VDTiB!~%Sm$4Adl0Y+&@1;51SPJ_#ie()dzA!eDRa}C{ z83XCxr+^m=Wy`0CIAp%WeZq4tP<*T*YNA2|UaR8v-~1J>P4?r}#x}~mfDT@T!fxq@ zHZ)Zy+2tlmj4sq3aj!#^;UO-peG-}`XBR0|w3ZeLaoG6HPg7@SgY{%1^VRg=pZ#Up z9}-{68ZP4AvW$nx6lHl;=Qv$Z!fA=v!5NtW&$>7{O!KzdtL>$w_r41)(({_6wCSar zB;4$!6VBo7kH61(fDhGo)9&*x(;>k2$N&1jPdiw%uLudCKoxmzt#cFth;uEU75Mp? zfHShJab=olV+QaQZii_S&P+zM_v;ma+(e}rl=&#n!9<-VyUn;54`(AR*vJKIb-9+F zZCkw=@8>+VCWqS^+PAwr5i!ActwXA!caAT@D(gnYY_70SYil7IFv^ zzQCOC@HMH+{;WA9O4i*W0P^9+L}XOU!Y~kHD$GuW+oC~|BCb{gNYData(DoN7#qao z*}wr=Th{o}GE+obl6h#U>tTAga=-l-TeuZ$HDe&J*>7136V23&(Sy&+#rK9bfW&G7jGGA%ZL^3H?wPrv;Q8m?Q7QIz{wI5fuQZ{91wU=3hafX|>27+ftO zSsKHaNho#VG6YF{F$jRa5H`9Q6@BajhcH@zz_6KhWoya?epNEHNnUuI9^vV`)0pK8AxbTJj0kbnTJ_E z#|TFi0YSGft#U4i)(Szg8HZNX57$J@h{y4nrh0jjt-7L)9&^pH4;9?jRX~uy?gUL+ zcaj>8$6k7oAsXkTo+#*vxI3QXxb2nNVV^Ge$+6xy(YUppmbvtTgI(h+%+5{$X0&9n z2P}wI3I*^C``0DzM~mAC^j<(fT9DWb%p6&5ea^+sA;JX9*S@7;pl^j`LAMKgGD-cX zBI|Taj9C4QJv&CI`5(f8U5l<3aNhu(Dwfs+o1|WtN>=9-T4tX(?`g3-L+d%l1ien- z)>$RSsSQK2c32NrQ;EC$_EcVNFRT2H~sRrWb2cV(?iH{@cAB<2T)K<61ur~Bf=$) z4z?+NPAMdc;F|Qay|bVG`G5T{Sp(wcjt|m5{4f6jkjdOQ11EI6`QhLEZ91gu!r4A$ zWf0D0E}QgrPU6-$R_!twf-`_Y&tmiW@3OE8t#2q53?cdl3o7c3qqJ~uIUR4lN$WRe z0Lmv6+&%>4@3NmY)|R9*0TM#IGD!(0WWp3#s0XZ{t+IkIBh+P7Drm|_lM<|4(4?5Q z^3Pgs)^t<;2zRevD$ZJP>{?{_j@Iq~+(E;cvTJNqcwgRrNkACa1g)c_LI+LcjAx-p zX{fMDF%CU5UaMlOY>ilG{(Y%?nb=Bsx>=hR@YOPAzfnC+0uX;|UA9i0yj?{<4=>YG z6f%i_d+(4FjNei+CU1D1d|h{!>Zdw zaa6{s4CkPIpC^d)J)ap0Kld})z}QQlr%W=jtYy-P-Z9>bv9i_{;O!IcT^=If zk(ilAEWG6x;D0;=RLy(F(8OnN0fz$g90C!P^(i+H!rx3UHZF(jrNX8;J_$-Dt+v&{ zQlVzx`xbMxb=Ls*ibOa>OVTPLQ=9^@M63oI;pX_`0_D^`Bnkwq_t51T0)R1!t9>&E zb9BKCtk*qjk5YyhAk_ot!O>k;;lTE|)^X|XlUObpXM^<;*lE#n)10-nU^Mv}NI#9W zkIyHAm18=$`Kj zSS(^WgwPFa_qW>0CG(%b_2QxTb$uVybG{3%U_4Qf6XMbZD^rl{E};hxYm^P)-^Lnw zP)ZQ%G=doU%)L0kGA#BrDku^p5@8V_$l%@uEDm{HDi-^+>bGkws8<=O;C{k@ zLpT71e*xTrUW=Bhp=)(IdbJZRvpCkVho^LsSi{X~YqmmZjzfwYp7UoiMIA=Be4A`R z{tc3#`w3P74l%n$!*2RE@_e6VaP1CRbFNQWZ~1A+oHo-k}Wr3W$;rhdG6^E@q&j3Sd!Qa8D;Dj*Uex(buEqk znSiDi5OTU#7z*=;cAyu9?--?SxDEYTasTZAP$|^fJlke1p+|rUdLZY7^DP619<)+& z5@OGFt@1Hz4d;PC4X3b(Y85aU?r@Ha-A96My%F?KU@t(LX!ls35#BSH2jmOJZ{=B! z9M$m)0Jt$qeL%=CXaN`QP(dj{+?T|gSvzoPfy`Khgl)l`6zK?L=HXh_O$RO61`qRN zlppF4T;HLHY={#dEN4YN z%2cr8S+}zXkaX{iGuD0DV9jdqye6~(GLASsg`smCVXt-dm@xK?V%wH!@buWDd4 zLeqH@&?Z>Nk14i$ezKE(_4yy*Rb<~XAN%W?#BuA^-Siy5G=oB6EZpwqW?BFQS%rQc z5H*5=aGUY%eDR;tF=ILdz~5qzHfViRSiPS<{PTa6Y7oSuvg^Vq{6-lO+LyX zP319Bs)qv*LL2_jA5plgB;$g^=+LoXH)G`#guL_&8~6QRUfee15p!Zl`WRV&`LFe%oI?K6Z4lDI#?^>yi9%@8v22pfRSY2k8#Aa zLgqlrJkvS&U+{A7>r1-v$$Z2=1hveK=P0WLQd2WnaIwhw@9alekO4rIc>!zOn|t`% zduJ~&1Lwa*e3*$r%0bst0OcI)m{8w4;(Epj=u()3r;T;h^Q2+JTN!h%1xN~i-$uz1F>06j~7<_?6#)$5Umj;*H!TAua)TbnGi zmrDeQ!Qp(Q>*a#^KDZ#_@sN+f1GDv!56Vx)1j@Pm{_kli#f^{mac6M$1!rfN-pz)R zUCOLA8!>Nw14@id;BSf4=bAe1L`AM{qCmni8F}IwMo4`AvnX&BQW9 ztvDVAH_E_v5jI!G0&6mz=CI1_9Gj?jiUO#D2Z4z~I-bioW!;uExPZaO{8?=Kdy3-C zlGU|q*Mjw1>!Af}8PdIV&Y`yB!XBy37clJ->!eU`pq(9QK2|8WPH&&3CO*o0jz{5a z99IBe34*n9T_r>hv*uODqObfK{8$OvJ@j+e#TdRG0>m}Z%B3)jxHOj8gX5xkPX(=j ziIw8MJd5|jjLszr;_Nx3&a;*iy56|2pvss;h6w>m{Aw*&JlQwP1laz`I2_kbz^q1N zFl`-+g*E96c6M(nth-~dgMc(O;Q5h!;P}`>SQg{PCCa{N?K5`5?4$udmZVsMk)n)8 zpHypCDU{6i9ij+4_~c=F_3904P7l#}v|)ZX$->jhAQ19Cx{saz7=PQbgLlHEHjPl4 zWh^cBgCcZfsZl1OIty4pp#c=y9i+qFS=K~a>A*0~a7~$DamiuVnkJ~7l~(zjOJiR= zlm(d2mFq+2uK=lxyKRqK*2S35Q9x!6aIbrC5>S|vsq)+!uD>}-QrOq`owa3vHu!;i zxmS4}rX@*cs`jhJxQ1Xb_}uurqXT+!LOT(B&u1QSJAs%mphY$Z{nB(STyR5xi2~$` z7`qYT>833d3`iabe%X7LD!{9}&%M*Nnj6u2hJ_yH=DylEUTdBJql{>t(n@-K*zunw*AxOK5E5iFJlfG`{6RCbeuB#BX^N1&Ae;s&tlcousE3iuI}} zSu4@rq`JV$?2YaiJrssCHgK;VQ@*Cb-dXy`#06_sT7XqPssecmxNM@tN{^PfsIpd7 zcuIwP&0@8=fu$h_u$8AA)8@*!F5U9BhBI+NfnsSC_e%b3@Ve_Z25pRxMS6?{x5oap z40xQqo9=x0{ZyJ?4JER<^EmzL>))Vk(3=urt%XU!<2o9bb%y6afH^}3=n25mgs^eq zo-8vn3Qx3lW_^DHAc&=iu{)n*0D0%e?KJuD+o?mm+d1QPi7&CXb>|c+#)hmH8>vNX`h?kV~xiVXkp2h_VulJiL>GZ`f(%8{fx_AF$ z=oE0fa0^!;-Gx}^9P4hP%qVMfj5YAy?87LzA?cb`Xi1xMvb~FB$_m!pOTj<^%RNjp z*3w{jg|p#8D&PV-c`2m&_BUQ9tbe{2Ac(aV*mzKsb3xPmpZE67u(UxF5h}wwz89ba zWk8T-9W{BVOj)^*#cIh9;Hg^Doac!eC` zS;``@C#88I#taN&96Cv(GcPr$7}O#p37n-^RF%!()gTAgCvd_fC( z{Ot{RF|@Gi1?XZKY>6cxGSfcd{ai0z5le`RR?8<%PyJ@_4S9ngNTpUF&XoDYvBx6LK{Ye|*Uev@Ws1{NyoTV!MvbFwRTT zl5UK5@+MYDR`6?C(;9{T&PE_iVK8hv|8brFNr;B1X-7#W|RIXkUmx@&!@(Wo)U5APdG zq^2Qpc>UoCMShPC!fK^E3$o!eR}?_N|J&Hzev(6C0UT$ze*h1<2TKAU=5I|Hg@A6M zusra$o9=ruL+kLko)S|VU@O>kZ*(=fhSrAic=~cTjlJ@Ee*_8&JxNCwoHV5H9Rg0m zDkF1=FEiixj`RUS;Ct>FQzTgc%K__gDahh(7TdM#iSF(z4wY+U$qn5kucD_hA13aK6kylv!z+k;#t2=rg*_tW64R_gBk>i)L`Vv2{iE+N>w7 zK!V$a$Yg`jNgtTu=Lj$_v5sFpe@P6P!MG&maLqVtX=p(O5MqJjyO59x^vwGvl!#WNlI z7#6zSr(aMurAFd}+31Yn?O6&d98VhMyoQFz%3Gqy^WeVP<>J&}bm(&&z$q}30xkOL z8fdw8FIgUFmzrwCgIKqTe=>Grxd1v1yaHNqxfSm-FNCHvsTJ0YP)nUCY>Z*EpKqBm zD_vu6FqxOJn%*eQ0>kbN_Qa1NS=jYJ|r&5#P_YAFQ!4==H{ zW@8qxy6c)*z&c)mZpQgs%i%cy`vj}k71tVvt5T#8ssq~exTw%t62u;s)~CBbDLI8F zHYgZGT0Rn0so#or9vV1R84+$d(4~MUy0^e^^-M?+$}bbrMm@`{qrC6>CC%uX z%yD1@?B2_lp09Wye~046TI=Et%Nj9-o*G|1Ur^+jB}8N0tt`12ZR>WVe1?<=9*=x5Def3Mwy>Z^5{X!5cC5mHpHklUWC| zHCOoxpo{V(kQXaBj;!^Ve9ieAKBG)P$!DI|d#)?lvHM_?E2OPHUKcEY6l&CB0ecX{ z#Q+(YfJUb30;{d6J-b(ukd_KyTe!+bd5Azvcbsi%rN~oU_4n* zNQS~d0{QbCfrb_~U4FhFS}XUmvV5j}PvJ^1wH_Cf*T9%v#P9*LIU?gY5D@bimKk4} zubcj@3pZ#pw4e5=^TMs;nMVk}@KLkv{%3ZZOf#&Fd{-B$?O}A=3eHSOa0@iW^5po; zdFW=5_#D44tK^%YAl6lAS9o85q6>ADz)athAQfPOlM)PE>!*jEy$zrng?st8*^!fI zA(8RL!)-EeY?naBXIktxV#Y3ES`e&)K@FGliW>#yj<5@10n>KlyiZu+STEhU{#dv( z2!Z|6ilD2Lp#}i7ElcbJ-xkz`i&+tZFbl1Pdzwk=Go*?JYbeoTg5YszC%oT8__8Ev zef>7fZ6=78Cwsg5>4SIf zAf$196C z^Za=L${bmuRo5CC*yoz#V*;T8i)RS)*~K}Ej8gf2nzBDUXWv8!X&DewB}PcbCX+X` zz-QXWGZ$6>`p~-8FiJZ|s2q7-{j3KAdPX~@gG0pa-H$2EObiD9^=i12TD$4?hu=yg zC$F$hoy4kT4s@%;S!EyW!J8v{SNCE4 z@H{=;%Tn_PfVQzHb;NT$pU@%$1<+wBB12eVrJGZxtmf-EF~(=yvnfDSICHeJ);S0y zGkc(L*m$D}tioDAD_E`_x5~<4jan!Otr&5zDr*<@I6sgYG{1%YU0@ zAN>&8D^W0Y9|eNz;Yn}4_-Wew{6En{n>b{Ze#Z0c)fRk5>+N)$k#W{WtKTSY$%WNh z=}*4>Y3d&Dq|>d9;M0}rB%lk6BxONn-n)}-e&&n)9TE;^ z-)s;|20$jpvcTG3A^&VKWn~5})h3)pXbfvIiL!JKuagE_C^BYZj}n6_ueGiqO9>AF z91yyp`z&*VDdgFI`3XQ*5Yk3Lw3eLLO2Z0#L6soR^B#PczvUm&Venk`&ujgDPC=g? z2j27EaWptf&?EOXuFbP4@412>2*@o|Zpc%({Ge~+=R!_FkCwAB=FRwQL4@rT^`=<< z#J}}KDQi{Gl$RX2__yU#6b}I;`P^^XPbMTX+%h{3rA#;cA(pWerpzrB}p3IhnT zqLo4?@D@jSmXB-T+&U0!9A8!5XcXp@M`f(&E&Y)dbE;BQ9B zH7a0QD2=_-pukyI2=j`w?l@S?tQrD}8++{@IKOc`F-^;dLDvFV@~6Il$suubX?SwN5~1U)uY~r^tjw(R2+2;+FzM_SyXs0OoT9 zGa4+%+~ef}l_1=(fG{atjPlEfV*I$4gPG& zn9*DcOsdR=n@uNByEeIMj&@t3))>!{@#-2YM|y=A!bSe;+z-+&r=AZ9&dY8xZQCYT4;jB`IT znU4@C(u7?&?9e!XA0~YV1UZ@lL-8Z+BU!%ewA6e`FI3+C}3f=l!a$Xi;_TGS8nh(&konw`N96h zBb3}2v4JXU+2%WbXeR0?g1kj6Nkf;8V2g2%Qx(5M)Vw7?B3y&}3yg4v2G7u<7=xj9 zqkL-?i;z7>bzhwm&X7Zhj_R)SdukdGH!LxjW84oi*P<-w7ig;B2+m~umSFIE9E~{Z z6exqCBS097*1xO{A*Ek9E5?3#Jp{qIP{Ws1D3B3|2oSaF`myCVJpWuvkLDWIi8>uW z+PJQ@K(I{VR#JJ9N2u@^f0M8{)}h)V8FGE-*QYU z0SYB+U`D-3K(c#cTR)CXSP$(|a*7qQP8WxBm8^+HT1YLW#r1pX?%fa5`0Wp{1hOXV z(V}4V;B|WZPyaEswx6Uq{LWg*K?77Ws1uV)I09oZMsyr7Tu12k0MQ>l`haJa)31L0 zt8`gA1c0)B^tn78MPMzylM2i4rF%d8QN&*kd&k^M!M+sni3McMEHvNT`I4G>VPe=;zZfwCU% zHeGyzBi(tB5c|ivL);aBaT{fAj5tHxje-F$xCTIaiI6@42;@V4W@cnYgWR9z3;bg| zj8l-~m?RiMo4`q>p~Jt!!o&K9Rfy;4t+F|~%8JVO_4j;+MPzmLc|R4TFd9fZ%)##) zDukIjWIPXx>jY-m3z>xy5P-)s#X4D5MDXL^f)SN9g|^p}NxEoKLoeu-d&!5aIB|3R zHVY+?Wo+;|=N+=1=UmHfe>+$2qtb3@Q1p`Z%rW*ZR=OSvqigH4M^LooI1Gyy*hgl? zcjfy%EVI|{T8lJWD$8U)&xBq^6Fp2qv4D7+zNbwUEzXV`9+pBBGsoh0Tno><2CeL@ ztAG=Nfd$sNxE8LhYv+2p-yyqs%Wx1Apxh5`$7dsv_~3lBM8s>IJPJy&P?;7W z=rLnccaR+o)XYTY_VFumcyd^!x@3sjUL%N~#njwsg?<5*^LG)ABhmWE>M;h64x8f~ z6grA@0fvV{hJVH8utdR7onHp8_zh9H*rxSg*Nu~S+zbzqmM<-UZbVo;c!pY;L~sS# zcCPy3d#)SQqoCFuIp9YZ1f&R20B{7;Jm0GEnvru5q0jL^=3$7&2pAVv=0HU26|V8S zqi9%{+{?KrfJY%1I6B{lX!wp@EiC5jy%a>C9|-~BgaG8TGCfe0p=P(`Sl^uSNBl@5 zx}_q{BGwaAFquvx?1L^+$u$pss4%d2v37PrF!NAij(T7|lY{847FiF=l_)&R(4ZCs znF}||#>k%M5ms4xI0BeMDxkO2#Uww{wh0AY#%mD2 zae>n}Zlpuc(#q|$OD)9t#kF)!CZcnG{p@$R(y^QpH`BPny@U1XoLHUqF~QheGYPov zA&d_g{|y3k56PIiar<^;kscq{_#XEmey83eQtip`M0a;jw+0zow(z08`vTA{9t@bKw7XTP3MuA{{_`8S*lo}nO=SwqwEIPU0cNJH97T=qvKJJg}Qql=T^ zWs{r>lLu5XjLPCS}H@fkE@Dq|5meT>Ti5&vY`zUpNI*n8*9zR5h8)0&283ymZ00%^Lmq>I0(jz&0a?0QMAnv;(-iYNq4m~)(DxN&p$&fgXH6lRK8)fH5;4e#yT%OUh%xx^6hf2fSwu zWl_p3I2^Pojmd`ufATJ4Vm%MaAHhYqE%B^{wU2ActlI@%Diu8I)%mpmbjlST9@3Aw zXsHSS%TqHz=(FPCA+sFAl_5cn1D-DrQnqooI7eEnRP+R+^7zbp3Xoo>;_?1i^%E`T%M?$GOS-1xzL?F!dPo0N1pI%A*w?a?s-Z4wfl- zs6)=#&$|V0)#zgP5jp|JHG(`tP#7d%an2~x@DAgPHBLBpW1gL_$p(zbWe(a1j~s)D zwcvBt+n8B-l}exct#QY7@yC62KjbsglHaRh=xC!bI9J!ah|Kl{EAy@3L-No3xPm@r ze0>uK>xCRKkvTVGMY*}bm@sO-WQ~+7QqZd~D;s1_0hXd+I*V+4hl91YQ=dcZdV7x+ zs8qCTKTd9sP)FcE%;ykl8>Vp;ZYU-T0p*GR(e$tR-Ui+>)+u7xbR$-=1T-=88)$oJ zh76NjTUkjfE6Y?cpGTu^r-Pjx+N01;2qcP~nE$%JtzGg}{>eR4xH}?6pGHI)IivXwXuP z2L!_it(NOd9+BA_8)7_IxB$4Db;cB;sR#pQZEKY2FecLjcg4Ll{r8-s@f8k$wHL%M zAY=xBgCZ_vig=`Z#1&m22x^oqn8A89i+OtQ$(O`a60l4_^?*a2=e{zOJ$mGt;UwdX zx>)R;nrn+^86nCIpa5O=V9l(qnv&5b>17{Kw>6N>Pn$+HGF z(?*x)JYa#dVLeRQ?6m^))wXK`7DoBVkigzeE8Pvr?FcrvA%~emm!V2 zyX>(5yo^>qI)`TKGearDO$_+k*IlJBrFGU2=3e3Ko0V39ZgdxovL=1*Z&?Qcx@YXp zOeKB!4}VYVu?^g&o2lI1OCQYVDP@6MjV!^!4lhP_vC2mSDH( zc}#Fk0nY-5I(VN^U0>821;vnv~Gy(CQ*JUdy;QUGG!SqeoH;f)>f(JM53iCMvn53BW1miPIf; zAcQmna&6d%u+W^b2ZCMYm#wqxdJz1YwX{5r9+EuwIKZ7}T%D6MlhbY^#&D@rPOjY~ z31lYyKY#asrc)G$HfR6iAN*OGSXxbg_`6?HTdkfRu1uv9-IoAY3y`V|*?)SGVu@wW ziJqAupV_9)Lc;57>*<8}x1Bc|bn?+kNtW-eJ8ACGC#f^D1`j6zXz><8n^;$_8xS6G zK}o3!Yc|cZ`GlLC9TtF9 zaLHyx*Wu}$AIs;c>_Q^J^JMq&Ip&5L&lh{SLBcYXGA(kB$@j9)Gt8lE%+Heb(_%ne2@}zTiwj9m=MIy- zV+?_X3w2v5&;L{uwdZK*LMu&vB)^I8jr_IO{DrS+JMnPO_VgBCI0Q(>P;YVErs-P^dU zH?Ll%&wloEv}OpOp+~kRM5o3%0>c%CxLFfuIYx!$0eTe{)JDN02+LW@BotgkHdiL7 z#o~wn*>nDV%t29Ddw_KNX$n{HXjQt_NF1NmE@W&7`YQyH8M<1nS^`Ia?FsXCF19YY zz(;YV+Ysi-4y77%^*~Io&8B&@%QAD*WV?tqIE__fW{Do1#9df$P|KeI-C7)VV{55X zo3%)(mdJAsM*If>i0maC(Y$HzIpPRyO_GCQn`LBOuF%|hewDdt4Ka>~JF_l~jYB1c z$6;Y^9&EL|OvA%E!oC~uT*Mkz#eGntntL?I#?q+^TgGFJIt7S_dlv>8S{Gw?KctZ% zpoRtIF}OX^bTvzNs23HjdI0bh>QdPg&p9+3#52fTpt(r1hMA}hJ3igM+b>@rOt=T{ z%RCKompQ!lkP>%gkShZMi$w=5+3^F>(yD{MKF)Ee>ZL}a1E98DI{yJ zNzY~~663_PrqH~Wh}%3pI!VibvJ2=t$K3NJTJ<2rOIX7uDKPr=i`Rg_3PrT(?+6&5 zoL%4y632$Sx>TW}JoC6DC{P!bXUG8dwOjWCAi2JFPKdL``3UB~`fcxS0py51D zzS?i5|LuSNpVHvyO}b0Z)WwN03dYJC&cV-wA&|uJ?#GYlbG)zqnfBSU5>~A?O!$Hz zd#yrdW^3s){;hyAa0y75R_Jj46&OCtuDfM@C0HGuZ^G& znE(%EmU5;Dn?x7P)6-KwB1gdrX^+J+G2Uux4ZGVuZI{?jt0d1T7V(E+!j}{yF*d7F zZSmtc4d19l?yuuLRbC+{izFu+GPfhZ<+V+d_2Ll$M1C~^b@Ht1R@Ezs-1cSrq$;vW zK;Thc8q7V`%+noecv)-66OZ;0=&B4H=LtPBFZDJehjMt!a}*Jn8ZiaDRs|k-BTw;s z73cfu#mm>B5cBYJ9lq=zzE1Cc_-%y8_sBIpdHF|p0J+MzOYC)#u>3k;ZJGJpQMsqL zdmWnoK!mT{ zrF#0}$CNvJl^#@RTQuvSOh@Pp&Qopfru|?4fPoIU#WmTeH88#a?Ra%SMd_RGewAK) z4`?P_x?1az`-dKZ7mnr_J@=lb`PEyzjUTj)5u6NpD#C-px6<-5!Ud7#4~)0Y4)8QUx4*-$&<5DXjbGM~^rxBsKp zWOCU3&~c^_${w+nb}`^Xzsj6^j(=wNE}EQY-kB(G)d9ksc?58Y#kW5UURG{^%a|`N z!u-+nR00INT+!Ev;ifK0zaD3l2XtLs;nhHBRM!#ke21d3$g630ZV)Jd1BhGilQ||@ zXf7CX48e`?rGxPj9;OH@GQlM_y3hS2_Z1l1hO+(EdFo%Dlhl2eUgJT@v%m#o&Me^u z#6Kr*!cEfpj>D7*KbfTL#ES?WQMVp3(ejY_sl*EW-6EW;uN@JNQXPZ3aoI!c&P$Jm z6}2}AfG+Q6u2TR5Ajl?~kQG8^iCi=u&0*ztB)*sVYz6~NmLXf#99Dxn*8se=3E6z5d4B+nhrg)P}k)mWh)RpO&N5J-%dk znX?@Vn@}6qv?01PfWZ`om#78#UN>M}wJgF)VLWxN4e*QbnQ>Bw`xMog?Coek(^mwL zu60ls0Gstg@p&B2go>gS4jpbZ?gvdSql`u{97Bbesla8t(x#wikwSnR0==iliCc|G z;;jLkN36pg#Wa&4M-*}11bwcr0?-)naLQdpOmhLVN)2qM#%k)Wgkk+5CrA=Cu+}kGq3Cp_MzJy;WRG{djfP(IM+61{mJwnm8 z@u-q}Bpny2JTzpgIbb+KPuntifQtycDjr^lEofm0*;8G>3l-j2)*HSvMPY<|(OajW z^i`qx?=rl>LeggqZ$FB?0g%h#;qjcpF0^(Kg7_E81lRhS{K*8n%Ymyu`OKz8_j z#_kv_8mqX*$QKJVlxt13^v^z|ryNH&{P6qq9iCv4*2H^Tj0rl@ zJ=2r2b^l=iF18@R*&KRT;Z8um?s$Pw$>292(pyR8k^9$751jzc zM%D79`N=+0a3z2-jB9ZNAH}%BJIGkQR>0-Ue+4B`85%isnNILK5mIGbKG(KMf;{Dn zo+9EoV)C66St!L7aNe1?5P1$K_j@Zq5UAskD3-qbNejO`+xd9T&c zF(iL2p2r#p1_d|*IM@iCn>}0OaE`2Wt>h);w_O1AEb5taAB^Y;I`wp!^BbTIneKIt zN6=h{N9IKTyuWjTVOhrvT=HwIsqY1FoiGk_hRX;P<3G@8fq4CtQ5 zzMy)$@B@;fhyaa+eBd}66gczA7M=xSv3WwuS7|!SIK*EQgz|*3V3OkDK;QvFRF*s@ z!VHTFN#?(V7X+?K6_Uv#FAN6E<0YwK#Vye@sfc$#4;<7GN`*CqzaK@(!T0-~575oPapXWNM5umI{7yddz z+HT@AVl`f><)jrK&x1&8d9XzOl!PM@89E%IJPqgB&P30Z%@eh*#t5`%D(QP9Sr3VT zB7QJz0L{4Dt%@Ccj2YoS!LveQiel0dZ69IfuqRq~zO*o>j8PA+6~|TXGs2yHO~}Mt zkW~`&JyWozlDR{f!;qfQc>VTm%t25TV4d+sAp!U7LOZ<+!kz#;^F!>1WFC(v8+bNl zi%#qwS7z_JTReKh>r48Q5tfH0+y^}xf;KmQXsnrMWC~obiQpL?!E54P_%hd{&(AF$ zFujDEcrnTzXNG>^vh*H>AnnVH3egVF55tK2)}YhrXdyk}h^!%D$pX!Cwcr#y3xv`= ze@`HCwf};SR;(qV*9#QFHQ;9N%WpVal>!B>xw*Q*0K(&qC%HpEBzNTAqo>dTLJs~= z8Om=BPEjxiZH=l!y-q{=qTCH`(#`QvIz@n9U&E(VmyXXrq&l9Br|Zx#-jf{rXDC@O zLK}Y7Q?*Fdox;e^mJ~V|NlWZ&1@~QvQ0)xwWRqi_qUstZ>~n-yn^4jvf`18sdIdny zOW|4uZ?pJ%i_F8yfJ-@TF0ICMq-AsHD&!_11`yyw@(Hg)HYgNrVWP*>rqYUN&vtFV zYlf!r@y(iMfjYGPvoPme*WsO|C$5eZ=nzNQ4Zq1J_|U;^Dw-a*Ut6|Reqf4L5T8# zX19)cm;30E?qPJ-%mu?hurfISAdMM~X!Mz{IeR(IOVgNYHVOU1CqwxAPyX%yklOTo zdvU<&5b)3v4U8`Vf@5g1NUFi`%}a9a-laO8%0+lIkI^9bx1qS>yyeEXgk49Kbu-$&}M~RbI`;RZ9o=?($uMzEuvG*k(+ETr`@mqW$KkSk?~Dd=qmDZ%QFl} zWQPS(p6K(0{6E9q(t{}z<}M1dI=8Hm%9cG7lngK)m9M^Mk=tknW$}kOYrF+NqNw@X zbqwQ~HPw@+Viri4$EfE{#UI6B>OHRabMINj0pRaG&+t1$!h%9L{NL-Mij#i@bb<~6 zTmUcjTv;hC85!^lXErE6iC=asi$-X;UcKOVJPk!Km_;*qCeLz==GlcOk^6mod3}U7 zk+F^^!mivWyl7tM;CTs2u`zcb3N}%|MFYpio@%7RR1}@9`|$oW&v5HZzG6=1(HCoYJR%v6!7QRcRV>+_m^(##2n6(*Aw&j)tP0m3 z(Fz0sNSsj4gRlQf8qiW8z+`(pI@A-tJUu=kM0CS_P*4cWQxVN{gLAVLD^V@RW@~j36@O)VbN{;*he^MIhe%Xxgp%aqFT*a#oDrZ)kzHbOoZAO zTp}24c@mw%5!^G}wH`cfl5&~xne$`*l#ANIGSTf_?0Y>#!oBT@WqG~=SmKG}bg+)6 zCd5(|wM6K#06=Tvs^7e~N7zcBhLRv`HNk`E*?)qvBJN%D)kaX_9mHt?LI>g125jBo zMUw$NuwCV4&O#)cvT^I1TUeO@9YR4P#$>i&5x}5@YAcqZRx{oO(YgeZqus>N0WYkdO_IZ z!Tk^#dV70#zb2Qh$o;Co1lLrDT^xQ01?K4wy)FlAzxxp(tu}KQ%#C|`c)R@t-M13L zwT$Jz_vjg*bAZLY4_K$g1>+ZRXh{lIZvZ+bw9YVC44>QUZvlEOvet`)_6_ITH>>Jl zoXkUS>o^CUz&Pf*Oz<8EB3KFNjkY56AW~51fkE_87y?5ApxOw>2^EAhJlV=Bv6CJ^ znNe?$?KPTsl2-6GuOKj&2|wj&;(EcHPN`~LAY{-3O!es|uR#Dm6P5u~Ghf5*=49OB z0o&@)&~$T?XB5)?`#T{FqGiY4W~K;GY6=9CbeCv{;9gB!JM#1{e}0?Z{KG${A>Qm6 z9?2pCMX!6E@s!}3HO@G!;5q3q9)+czNERCahSyU;f^iW?$RUiDFyN*tZq}=Y*P_Up z+^gZuM2VZ8(76V*19&Fz_J9O<&qOSa1_!9`Ip$fC}uKmV!IfJ==C+muL_?uG}&*WuIMb&mIf);vsJgHG@@#~ba51FnNIAm|5+ddR&5b;xB} z0?02~(DD^OjLSWDBpOat#xBo{*gtts&%S*QSpn{W+`urAG!02A^9(z21^kIXhK@{i z2t}xDKzUm@rE<1qkomNRdLti_b+ZMC$4j|c%dBg5q7C0?{RLy5_ECme4Sj`&(e>lm z$_dA8F^@9QwKLb2$Bv2~+4@E>t32pIqXSxRoW6O7mjgM-{E7ggMb>nR z3?8Yhd9H<0Oo;oLYiME1GICikPuQHj*xlm%SHRJf+_MIK*N}wB0@sjyF!YF?Fq;7{ zVEk&RDHqSt5%cXOig1?GzxcQRQ)(6Kq#CfkJV)qV#Gsu5-ZnW;xBcSxX=#WTkT8C~ zLlFZv?hrNjj7)zIi~hL zn*b1Q7<5#4iTVI6Xs%SyRS6SnB{Q$cnVX}`N8}W}c>71(pIhnV=$H^3oxI8YP$<`6 z5W&-&Ro4S|*Ws)=JXI4I^2X!4-n|GxY~jEh!QX|5Y~4UM9tsa7N(i#g!f6ecm;^F| z79f^9109*PW$7wUWxnAp-I?D%UFSP1ef!(msFQEm@$^D+c}mkR+3|kW}0~5!n@{pSa=V}Rmpo8 zxPVZrr2x<~$F2p)P#9Jh*HEko_^F-N3HhNMhvW`*E>0=iZ^0xaM)Ta}F1a08qh-Rh zbM{E$)3S48Z5}+I(#4%?M+n3j9w3FgM@Z!%Xukbk(%Yk&-T@BHFMa;Sr|I83{{&v5uND965vu#~)f>Ddjr0i(Os`SS zExai=fVm3F@aX-^^zO|c5h@fBA>jNGphh(o`()_m4lr{~PT&pUWG%Z;6s(8&Xj{<4 z0>TK9#Qxw7Kp_wCY6wWK=u$6^Xu$g;yk>|X*)vEkc$gxL#0Dz}MbXafy#-w0B zrs;S4*LdH!*6%<+W58RVvoXybd9b^MMLAC^4bl@ByA|aTu3=rw!D*q4uh@fgXs#Q1 zgwPNBSi|EdJ$PD5Rqr-y`uN%7^wqCEM=o$o)Qg{Kz0-z90WuKWK89omkF%ah+$hW$ zeqoPmd}dESmxyOir>uPkCtgNSoKw7}^N^HtW~n_bJNYmOZ+}2(D0=(r9iHA~cO8pWfq1?UIsE zgTHKG z4@A`D6FM2({`x=B1={>WH=NJyIT)(+Spjn-D+`{m_CJz2`Td|4I6g&PZQr z5v{mQ&$-9zcZcv#j)4?cUl zM98cQE#_E%<+kU2o2Wju?F6)@45(79WYHdkmdQN|RtA7s8`%4@_)5cCc})Zt8j(T- z8-(sq&W2ExgOU1zj>!6%2?_*0cz5w08k%J3u<0-yf6)OpBDca@SFfa}XDr$t32dOH z9?>pknY_7dzB^vr8FH8@h;{-%m`)>*(aWO_(7?5d7otFhkg~&E!~2Y^HiV}K(RN05 zXfU_co9P|X;1T%vW9pBU+kQ`w>ze7!GTkMFLad2gqxVVBsQw^_SVhl7u+s?v)D55; zRj}wq<(m19Q7pySs;sp|Bl4f)8l&?qg0N4lU=e`C{ORuq&a%M8H5miQ<0Xi7m)SHu z;O3SKQhf;;JquRPw!qNl`6c;^`S2igEM8c2v^m^fYYw<`l8W@M`yD}~C)Q-bC6DWr z3G@VNe!v&*VD2M?{{^8ltE2=XvSf)6%pC-!GJpH(k9dsUa&y2Pp*2g>n>#xZJDws? zMAsd_h^6Fu6b!Q{yj9?Nl!%?sZSzrr@jO;V&x4_WHU!Xt&eap{+V~6D;dg! zNVzXR5{!yNceo5CjfDK+UvoTc@afS&DyuSk7JY1F1irTeaNL7V`ZjKEk@v{nsVMc9 zSt(_eYmUlW6#}Rl4<>~wBv=djV@y0#&l#eHL9maEdu~^E?pvc2a{0yG4*|Sa&YIbHEEJkT%z{goW-iJlI|Owxhden=*5}peSh9 zlRa0N2^zHSK&Zrd~fvc|2n*EVY#!%HE42ka|aJMN!tCt#PTQi0=hMMAc|NJa2`LzWSAE`f}rz= z7uU_gApxRFLQwmVLXZ-`T)|vp9{33WG^pSi@QqM+n_MIdlsrpzKzLH&YcbM{u!QT~ zFth?~R3HY$t|vZdW!gimM*EN&Xm69M&fpjF&zvilV~AYQ&I0}+8D3A0Yv@7h$ROql zP(Y??v=|c9D6v&Qbqm8rFCDUBmw8y+VnV$=6-UsiGHAe_UEtAv`~8dbzy8fX#tAB( z{t}ILLkOZE*#nQ*>!5`6*y*Jb#4V|)c)m2I$BfJ1X+aqPKm~+yoDdu3ALy0Ya={`V zQx%g8tbvsE&h&Q$Hz1FT4N3Z4z5I^1PP-F9oQl*we{J|*@h!dD6Kw@RYE{3-(Xo+| zCeP3Eaa0aX`rKuqIK5yfGuEieURI$;!!%azUScqH;okvZWB`p8klPcKa0~jrW6%EV zU;S4YtF&zag#GdNze@-257<8dBQna+F}I~L6EdH*b1wy`Cek}URwnq~>zedfTatDu zY|>b}LoPbkJ&LPLvNqLXjqv^wiG(c-xEpBF=tM<8$MRV2TP8uEG~Xl&s*QLN#HI z(-?BJ9JvEvmN9VGX<^bMs#E|RyIFe9tme*|+5*6!QMQoJq#NX=gu+YkujeOZC;tj8 zy3nY5dB@n*c>-kRT%=^gcc9M;c~2m~UZ6AeoJE_BP}Bf=^h$)~xLyZ_;A0K%zx)^A zh4s}4S9Uud92pKwnKO=05%+awkCv@J&;Ls@(vt`iXbqzY;`C=A6i?!l2 zev7&FC<-$}-!=1CP4BBgAJ2j&1^6nc3eFfE8bd&}>*#X~`|5~@VnF^5&n8zFkGYu_ zsMxQuUafT+GQsir8TS%0@Z#oTKlfZB>7d|r=SQ^#+w9@MhT^&?SOWWA_5045n4^v13Bl%e9q(}8B*ei)xnhtaqG>~vr)s+(LJ0k7zE8` z2|D$Dc@CNhxi%ZL-H29r2_d#e)ldm5s=U?=VLIdef{2{Kp7Hz=xE_GIlLNIB-dlnK|m(P$MHXS8T6`+A;=ltPTVCakJqh3 zLgxtZe1~Ejn#n6jZWxc+FUQ`-T}( zjE!fe=8Z*SFalvh^`>F5IeTs&K!sN{og}|RAY(P>B1z}wkV!XMWC2*>b}Tdm>R1$b zx8$pQ%sVjTVDn)K$+h-ns3$xCFi3>J6++&W>k5SKBHxGkNz^W%A9Se%HIM3!y|rrB zB9yR9`7D4Z%6x)V9naz$vX5Q}lgd}2WnJ4A$4afzNvs=t8pZE1=p621$7rD3T%_?jY(G}QzKobOpaxwgtd2Lh*?(^aPJ0>Z3p&IiH zZ=KdIbKy;BhWVjL?LB3r$U7i ze8>9hh}a z3>rbF9xK;3)}Hy=pG%|7eT7K*4)f>c?r+Eejny!$*l)%(;m3Y&u6=>9zUy!JlAFXD z>6!MWN00zpJP)c#12$m9dX-UjDwi4_EW?Gn02aM)`v<4#_WGQ2K500OjIY2TIe4)M zT~wg=1!#l?L{5{73~25^b7PKQ^So8%bsc`F0Ol8;hs$k z2w2x1A@{LNNb1IeCr{Ja1)e(gwL$d3&|is=|9EXTJ^A`sn&6eY#FMxG;UMii{)A($ z0Ec+Zmr&MiTJJ2vgyuC?k>AC8pQd#-FD4C?L*$BHP%w#IO5bH<@g1sJ>&pX?tFQU*QT~fF6k}+7`;Rj>pJ<* zhG!Qs+x?)T>5D#Oe{lJAwD(c%d15L79zOmV3JtkfNP_N6reF%moUt1?SIi5H() zl;Ie{L&Uy&tvbwPP#O5W(v&5m*B`WqUh<#clD|_WOM$Ut7Yr&}Jd0WYH$j$Ud0X&=bKri4{bdg2j)f9>(3fZ{ z6yt(N^Q11gJ_}yMIQ*V+!PFlc^?QPdSVDmko>*9B3NoufJB)7tm@}c&<18Gj$keKkI;|HLcz*#;Ltd{C zPxQ4v*Dgpp(4?Q)uJ^4at;8T>h^`G1|<5n3Bk3@wa}|% z1_$@KBfJ7f)8raL;%5m3ErXcHV;JZTf-z@H0BEOupa#`|gqN9z6?T(KcChu3y^xt? z#4F0*vHf|3(lvv^a8ywJb9R^Bhh1b&NeM+!6Gl za&*5u%Txeh6}0Ur{Ic{^K=lazY!yO8N?`SdWHRBDu_&kGs_Rk1|I%njFA!@`Wyuy40PwYh%Z?|j0VBr~aauLp4 zm$kOZENum~GBA{5i~~W+{cIH?F9;wZd6yMH0%d5p$lk@fmO8c7eJWFB4^ zGfs$?a>^VsJ!FD54Jp1*fcq0t!zp-?#teD&E*NzF#Fh;gs+F;fM$V z%W4vyq z^;9DNxVFBL+ArzLMj?~oty{Vpn7(p}zL=6;@WnUZrIRi>+cxLs*_a@y!D%5Iv?y6( z{6C&?lF38PKBe&P&B<|Ersv|AXG|w}A}gzaZ)FmZo%J2XEXRFS(ucRN)8^xcvCnag z6l1f6l&v}}`V!n2diw}avIKM;5ArZH35{@5=tj{epwB=8)D`b5?*#1L?+!3)M}r)Z zt!U?Agah8-%>XRYf?hgL->@0I9wT$WxN%&kEY5K+Y1n7UX#yJV95Pj!iog91FL__^ zVE8xkso}e5{0$JFTI|OCWQNmwJ)3k_l_WKw>P8OK+kE zpNwE5rpF3e48gVL=`T%-YFZ9U`kM=)+ z=FnlxOQ(Z-;q&~y2BndyX{79A`ocLVFMO8MH0RaA1sy$lg&`tIMiQ3AgH~bVSg}ybf(N%{41*3?ybg(-ar% z6wl9yB-{2|z!~?Q0B$?27K!9-){r$uXyha|i31{OxS>?}d8$B-92K9V=W@>4)=^p& z2uF~!z%wEjKubV%$l3-$K(OWo*?X;pfOSopO)B@uU1}pd2M7WWWVdYqs5U^U$>M?D zVSr$KzT+J9Ao!WeUr;Awx;KIYd;IB9C_yJnJg<%it32NPIX>-FYA2C!u-B6P;}Aqe?Q53Zoj z)(xKHiSZnN8*a1n{)~SOPc1@Y0&A^GbBMg}7-k+62@|re0v9dnHXh71a%G~CF*{#2 zbj?1w#^!tpQ_T0_PsYfA_})lwDC=RE)johuK$m=gJrt7bH6qlVS(tijpg-=UrzN~u z&_Is;Epy#B+ZzEId$ccccMt{9Hl)D0vTZ_|k5MRjy@m+*JMMEpKH?pg(ug@Kbi5)!akJR#6+GS#_jc3yj|3+&rW$f6YYD(!!lT|(?~iM`z$Pgi9jCNwZe0~G zkit+U@oeug#V?ce^V@eZ_7%XW!r7KV?td6>2u~%q`>AKmf)07W`CoYo0BflCQyFH@ z#ammIU;s>Kr@>csfJ1|V6y+da44Q*I!f4WR?&9Uwvt`o1pv`ec3mwKSAk?#9tE9{m z$g_Bc@>5ykx=i4yDj~^5Kv#{n9}A=_n1A@YpWooU1RNrRRR$UsdHBLK6I42T0`1`G zw`W+L->aq+K&LEsdUtRLfJ13;U0jRIF{E3=sMUL8C|()y4xsg5V;#dH^H!M7q7nCq z%F`C%tR>sr(AQH#)zcxq|IHuap>o;+_~!u}cTD7xD&*=8ecIms1g+4W0FO;+XDj6! zwAEoP>foK9C|0OJla0*>saI{JPoMr-+W(n*buUux^kv$i^~Z#sZ6nsQjojRR{8@Vb z2Vqzp_J(hlu# zmZ^T#Sz-Fi<;fxa>Nx%3a6i@IOY@7BA9{17dyf+IZ|TqD{mkJ~x$6Pd(2)ORA{Dzb zM){`Uq8tcC4&OvIFMkUbO~@CNW}t{Y=f27{0i3)W$5uf{0+7r*mhFMR=hag@%RDWO zUxA%t7y;<@>LF?&@()!_scODX3wkh-FsxbMx+L7UnRGqh8 zGK(E7xDgZy9?ePBu#QwIt~VFk96e>a&H(43(H+l#hs7`!n1q~XqCTIvKEG*BLhl+v zyk(IWUk-2>I+c56JZL_u>Uc^!&5uX^mVk8)LRuG!t_@ zv0(rasx++tu{>Tcn9Se;v$wI?=0fRh$^k}exKV8r0`+n&KnG%(-%+V}^ntn3ypb__ zvcU?ma!8YW92)yhu(SjZ5Yr;}%2QptyS+x}h{`aMW?P3p(GKG+;h|o7Z@8FE9$`(+ z-E8KQXA?^-{2G9)&NJt1upy8Fh%Lw2h2|ZNNQ}0^vkGBPIeSkpgwLO#l&7rY0=Wwk z6_aB<`Gh?P;p`Sdb++~pU}s0GPzeYzHV{7Yr6Jz2Yry(k1Kd>5?wuZ(2upJRIAub? zAqB%AeF!oDid)P=df4?c_auT4UR1$}grl%A53kDM++6@`4<&OoJdg0RV8VIWZ_31K zU?^W>E(kIWfmJ~te@jaWC@uLl`r#R3G|}8py5m|X?;T96dvKBaZUVy*9keD{=N+8_g|UPQ~`y^vV{Q%HHDsq2}Sd2n2TPZNc3eMvG&|o zI`$eve>3~svA%22iQY(g$%<NiY($IPprXG{06$Jq7E@Cnah^t%L zVhCOU3?320u}!>qgL z2#>H`uTWiz2hPLNowGv!-Oq0k7v{zrl>|{@hnDj{v>FCS^dXyC7 z)x)}fa6r@Djkup{T`FR%a*sBAH;6s)h$@vn!k4{r4mJX}*g-?Y{5?HgI3l@+xL=+? zZny{nZ9bDW71%j`dhC}u$Qy7utuKJH3sXsmOanaJ09V9zl+k9j#r@v8wjH{NZcO;7&pOvhm%o0+uKeT zghYYj$OaTQifJ^sPETk){>A5y(!cwwU#Cq%P8WFBmMC^HZNOe|cNhw`Fj+ztmIpT& zOE;`FyGcu?v-6v@31BO+xc~Hr7bxUedi?oU90g~;!g`vVzfO;6>yiKPJ~e2%evHCA zC%kk?()X{v`#KDi|KorE-&nU5n*V;w-gaqegB(JB4+(?1_8zT!1w*l!U?rRip6npF zn*b>JfZzEp`z;tyZX5oyzolTp>y!=3(EugTlf6hI+=woSbQiwwcMYkU+#fQIJ&(kD z_DF-(&x2f`8Ud1jD%S>hV8f_kZ!{9B>Rdi_N`lN7m!M1eihOVk@&KHtq+yOMR=7pr zH~26|&LbL$Jw#SXZ4Y7w)l{)1uKo!204Y~ zgW?#7lq1HV8WJb?3OUPs%+H=FKe?}i zhHgTZM@kB7s*!5Q-W+@NanZwuvyJm>f(gpd3G>9b7@I=R-8U}lDhnb+q+Cjvg2}-s zd6c-x^Ei1bUU5~%OR-7=zjc};7O=KTSa6Ug zo|IX7NghUNe8?g*S(v*=C0q|iSz1|#@E90B=nDir3kh+A!sNNcDk5Z|KsN5(BiT?i z9KEoZi%Q3hTgEa}sao-xo72bY4yA+gSf|1397KM{(JU4n!11Yx7j69hw|E(j)2p-N zG^P@!j*`%mvFb*!G4_=z_^(6{GBY=8J|QU*niM`+T{pO4gCItcy!kLT+q9RIn-_Up zY_=~ekxlFt=y-WF{^S=0yn;j9#h1uF^OOSCqK_Z%u}7XFVDGC{gy%X!TLRY;=NYSg zL4i!mkCusEBCW9k085Y+3DS&L@Dqg0qf82QAGNM6MzD*#N2o2`Q6$3t+{Y`_;r=FT zXAKX%*8Y81Kg?0DkfBQzZ-fAtXL!#LayCx1=hZoD?RrFh68mq3T^!U5v2rUIklFLO zV@5xPwPF}dU?)@bp}zr4Y&)6OHMuOOmoJpdC2%|&Ypes@0MRe>gSk6WWjcAtK== zlk@d*$~$_CHISk;kAgX#0k~tm45`{~K>$8+PZ1IdP(!PF^G13>7{BXthes+b)H`xS zc@|rUk^%(4w|bNKkajxIC+nb>QcI8J44S<~82;}2-=$rWm`{!fkuq-Sr_cVC5pXu} zwqp4DoLoibj@V?|$(`bUxA2}`a70UZfAz4LLSge+FjRm**E0+2V00m%G~i|kd1)bf zD>+#HZ^f08Qc+`g*u|EI+lnng43=VjPo4AVHpH>qC zPKdaqU4`q;ybMV!QC+@?M{@};wP7%g30uK*LlMI#!#N&Ka!?TRN91@N5z;F$))fqZ zHRQ(f!}|!>Oj(<6He?~suJu-<5Lj!Cqyckrzh{K;Jetg0%K(ego7=0SH7WwBR$qsHj?Y_!Hw`1xM3`%abWCtm&qc3*VEel zt#m}FcYJz)!SDt{ZzX;DFaC8pIi=`;^~m3xr}^nmoKt#C@6RpAGo?{= z$UfFbUJv7V0@gM0!RR-c!Yyh zxwqGuPNhr|k1UWo?6_+fK$@39k4E57h1s+IUFn&yhTto&Ap~gEQ9*$&u5qe@3j`C`xDy%6 z_fc9L;++F*n4hF2V8F?+rRdupSWG2aGRRz7a6%acp}@f-M))I?h>93e;%|WnCCAL6 zFegNYfYPH@fNAbvk~|wuQctS%Tj}Q2KXDAyDPASIpR@52g6?N_nePHzm=bU&Z6kei z{B7=-V~E6Ro@OkYHOB*IXoB%!Rp)tz6lWeCLt%|kbNs?G$A+mOBzTEI*Rb0yMN5dT zb;QvjGQ&n%QmlT=+3O@^iPLbPds8LJ*4!xxaUtB#J`=*_UULKmx`lP1Jp>-Pu;^oj zxTeS&at&8SIEJ7&@|C@Xp}8SZ*An?R0usDADsWJ; z-(YMNgn~_??E*i9C@!#W-~iUq1a%0IJyO|9d*-Ydwz2>q7?bgIv80`(-mC)Q9Oo5G zBdFoK3RTz26^oFtLWS_NR-%AE&l=ovEuxAGAOK!@yq>M;6$;v%I z>-9J}GKIf1<1tSH!N|*luSoxgNfAn{4YDBi+kKz14}t*cP~h+275vV@!V9J!U;*j_ z=;Ep3C%_Q`e2V8<5N`O(Rx@W5I4m+BliBrBKA@kMJqtZsRPXi(zV6&u1xIKe8Z0m; zy=LyM3drNDhOCj+x@+PX%n?(bD3ImJPz)>uqRUAzCo8WE(HLSj2ZZ%tLV7;p&~t7Q z-Y|rZLeo|_EDztdVjVi?o>M@^41;SGkf4EPc=3|f6ZUs>aa>cul=+2GTp);LJbnke zp7lrAQY3V8au5dLfLta$QW{N$%$|MnBt3kz!`@?5{N@MpQS60>(BuS&E!NF;K9>3y z2*nC!1zql)-hc1mewr6^$6O6by2iE)X`pOt@Ty}OLXTD6^9Vt`oqF92xl|w)4Xq12 zt{p&%Cjw01kvU=Rk;hX#xkv8W3NjB*EA#Rsnlb(HhO{3VoF8FamN4G*5+UB;=kOCi z(;6(O+Njj^8F}`0g}InJWxnbeVd6`x>QPdwL}DIMVOobF^6nWx$9y@#kRY>&!T1nk zcl|+=E(Epo!_V*21$mFx^e1EkQWH-uqvl=!9-g&`E|_4*6v)@i;ZB^}t_GksCG_6K zt}n0bk{kE0Q<*5t-7z6)JmT%{VJcny%;_C?A(7KPP;h%vN%ube6?uRh_U&mYR0Pgu z7^$4(Q0FMNUh6RR_TQz4cqPXGz8N%aweJevw^PnIzPuo76{TqYpJ#6Njt@|rG>rZd zPZ|15`7uK_kU?^_{0JSYX!WcMV)QWGfi|pc9S?8gTIIN> zlj!*9T8T^eHj>|=L&K-a2fefrDn-fYG1k-zJ?k@A`v{%k98mgueHbGQK+mnb9$`g! zjn9<7;YH$F&&UlqM4qArTJp131t8-ok^k!by$`ZdnHQ-k(4pQeTUcf1XbUcpM|jO@ zcw&`>nf#G01*eP!I*y~mG^#^3`-yzW_wZB+j3?w{cv?^wgJfg733$p1oCpc<9qESo zu}(S^>>)Zw?oX%|wnK%y6ta$M%)2*oCa^8Ij_N?J(eWhMGMX4CFR`vJkf{ltO&x2% z7zDzOQMn_27!SY>jMY5lxG(GP+NuLxae37}5Cppa0hC=!7E{wz`qn9A9$tuJc-=6s zFAe4B&B>S%VGM`0g0}2sJ5|tNUFoY2~K>Y0ubc=(tL!F)Cof~2N}^_}!bqM* zkS9c$XJH-a2q9#WrU~D-fC0mgMTp(Far87pA%uGwLNF$7EE%N1qO{pi-OUgUradB< zh2?Ju!Z47E!`Qaq%6M`Y&($)PrTJua=F@-~@2(LPx5ugV;tl<<$Wy~QFd@GT(G~?E z5KE5t2)spW1wA76Oma75BsYjbV5RGAW4U-wFM!HA@&#CCkK90w22(RVmYu&~YK6Jp z99_M7V2xKEG2$``7;aJqKaH;mm9kbe3Aj&rKOgi5q zRLrBfhPZktwJw5nhAUqM818YNqeAwqS-oy~2+F>7HGuU6=O9KnipSe@w#3*xsbU4I zXUe|tRVtGa=qBnienD)L#*rpbdn(5W1zVy;L7DFvb_?r*eQM*CsQ{9TteJUdu9Lk5 z4Gk*vBq)K5Yo{w%?}7kO?_?Lc(0iwH_nb^WGojpe3^Esg09HV$zlEh@SXS>II}`Wu zGZ|7>wu+Pk2IA!!yVl{A;$AY7YolOXV64j+3@TOuj^I+D5?*IV$(Y@{If_F;=-9*h z9A|(j<|0k{8TZR`OhnKlu4qGxHjCB6H%BO2P!Yhx`$4N38G1a;qw~4WOR$xN^0LOy zI@fQ1p#@utSaB9Ec;;$Y(Xd43^@B!Pj|CKvp}3ejb6n;c*L4|Xc8fw>2AC+wvQ!5Y zD%Wn|aTK_^J{`Oz=j3bJu0a{#Iuzg$*G7OXCQVG#cA_wfRhsx&}~} zHLy>ahiiLQtVWZ$jIOz7rAB{3p4B5QqQ}_H#jzLNw;Z>2hXwy|Z!d&>k9D?H%q4k# zTkFfw=III*!026p){n;n!yYK!Y%{V2c z4c(m_9;L&B_sBnw9jv5o>m(h$eaYz=93qeMEb-;fAKnY2RnPP6+T3F(j^CG{Rri!c z3k+HE;VAgBwaSl2r*V*dp0(G?e}vVeS|Gs^H0p13-xNm02~|Tn9sbS&o|50pWOG zz$iFUzsED+SAnv!RJj>|mazmd!eIIcm_F)OfuKgTz|P;f5Is*@7xbgg_n>>%fTiMo z>>W5oETcn;x8pm4I1PVuKRu=_PZ-tDrbH&ZH?0-%u6dN0kj-(k78-%*I-Nt#DKkou zB^VP;noFw_om=R3zT-Z9d_}b~4^V$(Lskc|Q0^thmAF8|Kpmn(N;y3X11qDOz*>Ai z3|+<^x}3S1lkPKh9t17v>XvXEgo|s75Eo9NsO1X!Liv((i5IC0fLSD7-*|MNf`fah z*4(0P$2wjZs+xB80<6WEdNC;kI3tgyhY-(zIJDj%8JF9UR7}hxq%oS-LG%!x z1C161D2^GqYJw?S9f+ts89^(nibdtQiYE5jR-<)vkcJ3e4;pW57YOz2rw_mTA^n74 z>%4!Hb~s~o8OwEsh3&A~wGtRjDtap-%4ohe2E z5HJ}FXtWab%sDp}F@T#X1dd08B@feU`q;tORD?CgYPsV&y zZhlY2CgUrh7V)+i=6BBn%pva#^EzJ>VC@Iy=^2984&YUzT#x90Iq1!iZanAFwKZ=- zFllAk^$iM%by42QH(6P^LMIb6^7INJfhpm@^UL#~$D1}DQH%$@K6(vY)Fqy;Tv2$| zEqLI5H7r+!l>n%U5YsEV#5e>#7XUn0w#4_C#4ke0=TGsn{=(zzl@)?=fk4gkMfLiH zkPm8QbO>xckj|SeU;u!9zae;Je*~9)&s9q-6zxw^})6LO)8ZncP2oU_{v*+pG{5Sty`ugk7()jIu`p5tC-vg9x z0OOoGqSun=gTIgmq>m6z4MT)6{B&|eW9U)Z-sB8n_{w7j=g?<|bu`bp4nJ#HmXL{c z&bIASS!XZ8LsA|d(R}#oyOSk>*RCTx9QT6Ex<1rJC|_ zNc2NsRweI`H{-e~j4K%OJ9);TtWn+zP9hJKhO}L1F~-CEV+|Oq)s7zdrtbIoMhyg7 zyb*JYloe>+Fsm;MHKOPUo#<--F!vdHpZtmL4a_IJSF~(#D*e zbC^P);(Ya@xy1r4a|^H0bI#TE4dCY*^H~%m@V?F^L8@|2VC31)Ugub14ZJ6N>U{;D zI(w8k5jkWJpaAF*J%tWaE`{e4es*pGQdNXUKbqrbvwy>wwk&i1)&Dwk1ldLhN2~$b zGVZMbVv#hFleM$w;$2wZuc|=L~r>UaFCi9cKrfbb5jrZitdniAUHDiVKT?WsSa6 zK^7U9(Q6DGqglHX1jqmThga#C1YLU>5r|)lkOnwPDJmvx{_mN)k8gY-Qc!2YakF z$=zII^0{7&4+jU~W$?%tt#Vz)dRlGYqK7~r>PZFeeU+-MHn|}=o>yT#wGyT%d%bEZ zS81?|$53W&%}CBisqGLtm&pX2<}3={JQQ2+i{XVRu7GIux)}!0I<;?94@IGsC3v${ zgv2!i#CXQ80M494L$7=cNXR^hYzOov!?HfSmxHznC0ZCj7ZdV;w0H$(GB(^EUO$;N zN4|rMtx`6BjxWagxyunsV?Ht@(^DseFd5^Wu)Tv`WUrH^?Df?pr_wPD%mISgO$;}& z-D+(6((w>-E~tK^s#+i>_$GTJ_{H&q7h_HlW<+4ywm{|R+PL=&h;?I6xQ}BpB%t8% z!FNaK3cBHY0`Mn0qWLIL&vu(sb-L$_vCq9n90$e9W?&Vs-55{4N?#BB9$vKu z0Hgu!E5yy^bN>t%wWu8Xa7u+E>t=Y(&9M3II;W`^(my08_nO?F8GKx>KaC=*g_S1Z z%H6brQS32b%DGcO%>Vr#|6OYHuZP3yH9r3N*Bm$x?XreX0O$9@@&1IDpUfQbRk?<-_Nn(39;hefasOv;c3G(0S))gmBRnf^heDgYVv7r`nTq ziPxZgbD|dDX+iq}%5bw$OQ(2hxA0_w{L=05e){&&qjb6dl6x?2VAq0fwdC_~~{NB5jPEb(IjkHF0RwGW<&@HVkw@k4EG2?x@SVtuV{?4I�h- zBTB|~!Ry>3+Uzh6m^0qe-ngn?K9^cxEIB_9eHJnC=><0JDF z(w2z#Bg(|T>Tr3_M0@!#FHb^~aW*s$6W9c}m~N6A%^%TRVpUPJG8mffdwx zptOtwUnoi<_Y?yzJncM(3m7+sH=d8k$y1&ocO2^t zCw`PU>B7{Gd{w!|_jsleN5|z_*+W%YS|Zoqv$HK`u!`M2ex`v%do7-89l|nLYZOOz zDMGlGHHat=_wgmoSRYVaK)9ervAl6^mCW`=zGH&OkjU5g-UPWm)qY)x0YV0Nqd|v8brO^rfg0AR+#d(bb!UZF$N``=(8Rt@#StJ~p017r2hIoX)l-OA#1eA*< z@r;X#9ddu*0op*wO4Tnd@m zxd>Xc21L&RPW=uFR}TuSkpT=gzRlXSaNYL+sx}|)%vzCO=H{8KS}x&@S;xc3y52$H zh8OF2y7cmiUD`0A)iZ`_3{%R?y6pvKWyipe;b(iWc^0DA9?Jq~8NtT5R66GTg>d4& zII?IfkO^%ouUw;Ryb(>V-JyD~h0wdClG76l48tiv6cmnML757L3>PWwo^B?GE9}{_qgMihM%=uB>mS zKm71h(CQom1{527qZha04@x_Zry{pYFr(q%9JTT%9BdvS2pU;ooPwhp#(s?#fvdBt^wW<& zr3r0_%6Qr4*T-p%qw2ogZA4l}o{-zl&PH0mpjCd=*xNC5a`oZo^tbPRWNoq7Suc(S5@j^$?zlCrW6TanB z_@oQiZP0XRdi;*uz!eH7-r(79q#HsbR=H2vYfq6V;Wf5Wdf|4O4qxw+XPn1t_c*}R z356YX@~gl6`b%=mZfH63Ds6Fk$>^}d9`&P$tDu*O^%zqWQDh%G5N838lsVhj=HaDX zbA1?VoZF0?J34%y{^Ec9U!njdGaQuf$RFf^Imvp&Z0R!Lv=x?+|I3%^1mqX;IlLG= zQ@$|K-69p`)JG8JJtO4Fzba-86kp1*Oy^|!MYeM9#-6+bPs?x0J$_;TSR-gO3t46| zFg!^-OWC314C5C*(>unpGfH(9Hw~G=KgbsznK=QtuO296sL#;L7vL4TFil{_zSKy$ z2ya}>8C}bwss93WaxW#6pw{s@_uwCd(#Mc+_thRERmUWD3nk9v#nJV4WRl z2Lq!7F!SEwNo20vi2E`T-U|;H(UK)X;sQ1MBDzMdv0z!i=lliNUhh4XpW#Jwvv_5P z$-_9j1a<0s_tMrK=GBJLgCnH}zZgBs@C%U(j6?6IXV06=A99&u`oQ#uK0#(;Y$i1P*6;|iPdA%R*%;*92;(I7RYd;o(TIV83qkN z#*|$&tv3lJez=ETG9*v5$IzFt2)NS2;d1KjxCQhW(nNknn*i6O2WXgbQB>VQFe+l# zF)LrTmmHu7#BPot!{8o_O(BRvGmJSvnD>r31%hX5i7EJ0Q09CrFK^InlaK~L)?RkR zKIq4gPZwC5+%!E{!{P+&b`ZL*zuqPLh!yzJdw2r~9tv!Eo?vzn-q^UenYNmn0n8&* z2s5>5fN8Nuhe|xpG}RMRpnL2C@*d9sb8Q5fd24#Ch6tHu+~1y|3$;XY_l!MKUgTi{ z59Xa996flvw7HJ6W+Ucg6GuZN3OEHvxV2q>nOOzM8cO7PintZ@MTIlOYwm!dQxm1b zBI9$tq%x^4G8sotpR_6C=niMPcXuu++>~gisWYQ(PbZw>4A1m>jq28P2`S#NIcv8Lak5d=NRl?j2O-faEFZ6g(P2NKneq!=j9E0qa!2SXg9VgGagf0`wzT6X@Hk%(=U^ z@{1uzg}%Z{B_d$6ctUV20A>iWUsjR&nL;QU@-i0J=A)O-^)tP}!Fb+dC@R)jaBEL3 zd1GgTzKtl~sbL28%v`+i8ltR*jKfO1<{Ejj%u$)~fpzP%Mk>la^q(E?gm(hkfQo}} zhU`}rxv`Ex;x;!mE&v5RtYtgLU!7l}Y;3)tl5^?+8F)1hIDY?*-i&yJ-2Y|49wERL zKy?L>;?ZEz9n+_;8HG&%SKE*H@~lexYV}#uE5fUKA6iM(9g+)IV^d(b^# z1n)?daN+_=MM3L1m4?*pPuYQ&y5x7=9R300>6$hQ04?}mPoTXKZM<9oKs8ABSfje8 zNnRI72O8E7fma=w+dn7z!1!+gwOzt=dP?=A>)GsKoY|<`+@vikZPzs-e&NeSfZqUj z+!+)SIw! z=H23;g-E^W@SHn6>;TL%G}A|yScJ8Pj`KV6gGuvHe7e0n<>*5C8X_aB0G2L>fSzS( z%tT;YJqT{yr|!v-56)#l_X>V>m`0#4jd3d< zy})ba(ExQ)i~+J_{rSOR_>VzfUKOS((ax z`98{kaSA@rf$U8NKr+KcAj$CJJ%6z+$IaCxI$oZ}DCVYm$92s33Qr;TQ1&=Jzx(l} z%<&n46Oa)ak)H3#j6*h=6YTs1FnXdatZ;4~(I#z~6R#6P*`RFJ`14=yC(YS%#IU4m z?)zMcu4C<;C-==Lo%bEIe{o&pb7Dm49kmKKK4Bb@!^vOfJ}x4|Bi|ImVhG>p$y0B; zrpjC8ogw0U9zD&y%>i}~eEB&8jd{iOj-7c&w2ryBUM`XHF94k1chMa0FLQ{HE@N@5 z-dB&V5i&#s>xbNa_}~7|ghm^<^GW2yLNjNCDU+N|Ap#*U0fq{)kKnpEKHy6X{$q!9 zCLYOmYz#{ufS}L9)bX-djWc4xHNfv0#B?_Yq(S6h3{DszKjh7u^x?z1bbGKLXIEAr z$_)Ss=Ub;$7^#Ma(5q%;noaC%Z()2{5aX;xwz^LFEkM|a3A^QWEa3tQH}oB!8sh3R zX@Ov3ae^)&5K)hT_Y^bN6_m?1TXEMUgH@c zJO+VyP<0*jZ#Z%ZqO3v00!l-J*L*j6jbRN`5u$yV$}(1ZH`2x(^d0q zGeS>~3E_ojdwY1s_JTpj2pmJhH=OV0@ivD)|8si#@&&>ZvV&puI=B|b=67f&x_@#2 zkacdXgvtkCXZXzXRt>93Y*t_fxF%tF0dG_5-AlqWKd@FPRQ5{mi`8=~H9ZpMIYqB7 z_T_)Gbh%EQ&wI=C#?(HN0gIASLd+kT$zyTKgsK+sGFV~R!Yg5KJ(c+_&ou-z zB>_BIgun}iPi{zXT}MF}wt0&JP_Q@`*FXTpfLKfgE<%L&1T<_5LfKJg7P{|ijXIyO z;{2O1u=h~FF9M1@=_C3S5=Ky==+Q*lhZ5c(EM@9~3HBNq-rIH`E38|T_rqrv2nfhb z^feh)&!!=7*Gxm^7CM?ha{}mXl&blE3L(SKCSg~|r7ELXR!HXE*}acNX;{dfZi2x~ z2nrG`)6WCm49{43b=@P>NyS-%9*;xPAP&y2IePD!{o-s;WQ)E0Lb!9hkKkkq;gtx_ z+OKJi0*tSewJ3C+u^|ho;?gST(8abb&kS1CQldRM}v~ax%RxQm&uR2JcFN2aVU^G zi}6Pr5hcP9#b{@MItsRp7p_cnrpbJ*4xIO(`za|FjB$%SeoNC|Jt;X(PM9yygAky1 z);&t|Gtxnr*UH*PY5-Ude)z|`)PWAbd-Q_q3rU@$QKg*XG8Q;b(2C6dcxfv z0N@DE14NeBR?{`I(Cz?w{%=~Bp|_S-w~;T%RA?y6#X`&iro!6}xR5Uw2pQLq858bh zrDcP#{S`$Ng*>Pcdhh^s0f_P>ij8=lx!@H%un=0%X&&Q2v4HC42l<58o(%z;2m@IE-vt3!sn*h2#5s}1$33*RtJaaFd**$q~W;Xen1># zhbJlsP7n|=_N?tq21uQmlM=jPNF#sB{H&n&+IZdT8xM$TP>~vALPkOF!ZQoqrW6-h z$ftKpBTJc-F)IdIt$8Y&QGMe}2@Rc&k{SurxHyuStdU%0- z!V$t-q2G*{%~@;L%Sc4zOESJVb`H62cr|vP&wZ|E#O9H&?vs_e#lprTzyx7UvT3zI zD!2yOMi%cp9qeC+RjJU_0v5qH2&a}fj;BYSNRM;tatw3`NP(hLgMKQ@cGtFq8bWPQ ziP+tLlYacCU#Ds7L*yewrjQK}d1f7@@CnShLM}-IA-0A~nzE*7h|B5+;IJx94+*OP zqk!Nc0`px|!n))|+=kwwaVb|B4$yn#f*F3XVh$Dv<{Rq<=6&xWAW&!O9cy6eX??n> z_FZbT@jqQ$K%{Ux*QvA?+58C}8jnJ!t29MOi=`arUpWOn;b{?%fb+=el6*ad&4w_NNreIC%N0|Cvy+n^B%zO7DZMn z>8VqAL6@wb*040P3<$8&%{-6!+ zZ9=1Z3@dtD*VcnyXM{BbiJruQR>3eOD+~TYRMTMiCe`T`HT>{C)viv{91a< zI-v1^;WVq=v1A`T0SF>*X93`jGH-^Sl$mTPeyH{av*9Zk`R2*rQpgV z_iRpWZgmvJut^qQ83yX~35n|QVb5FF6bk_Clu?HJ3-rrdq+x2EaPK8Lph6x-7r^-bcds?DMqCkUXuD|2xtM#2HX(f-0yMCQ^?mnbK|5bp~Df_mffq`PRHz)cM1DP$V zQp`}?1gvrb#L8y6qDY~P(I6qo!y5D^|I4L0jOYhXKI1fqE$EoMS!ApppR&e~GKcGJ z;i91VGzr5W;4%dia&1RR#$&6sldA0IfB`+YABt=D7bc!wCz@|R;3An6qx#4|E4ongtCgX0x2qhAE(9<}H1dcpkL6ff@=P7lU^ z-8xGT?r+B)t4&OY@mMRzt?m(A*$>dkxSvM<%mvP1IFUK>OqaIrvks6Xd*}6z&%HME z>yM#jJ+`r~_{gORZ}?IEPmrnh`zz*AKB7*&C0g9K?=KPj9M%p4NRbOL)gqNVCe*9M ztGLv9gy!rAR7Oe2tP2YyFZ12r`TNv+d6vd4T+o*X5o{)Y@Z=%j6=Xv;0XoG>V^nzf zxDnmZA#$1=O1y9oz?T9K#%48>jdHK}PK(6GWvm_+C@ri$NSiR56|@zU4V25zLgc9F znzvA5NVxdTKm9G^v3D1MinYiiOjPUwf@qBZGka_he}uIrlZO|Mzw0oV9zZ=a_Ha@h z6cHxB@l&kT1< z)Y72c?H#8{?lP5(zXT&G6!;kC;8U&@NT0(THk*}MBV@?lx(;4vXq{jA9!w3rD0;lJ z3P$Ts5bSpqj54G-jCRe}lJ?jZKIQu;T$s7fnDU%EYmti=uMsEHcEH^z?up zJg?IaTKuIk21_U_=-U8I7?vYlKJK!;#nP zeDc~Qz=y?!4br1WBVcg4$ioFWG9%s_h6~;{+Y>B89{`dk~*X+F!fV2C121cibV;6GIURs!uOJa{AAGW1g zCzNPI?o$l5`C1FKYnsNuRH+a>YBXpuvV)8W1PoEb9s&$BVusjF1*oBaW&v(P!(LQ{ zcTjAjfF&ap=B_FWHorb)*A%OB>E2I%o-Sd%S}&fbTrjxKhfxifapW;zG4sB6c*HrQ z6oRlOhKFr3EW2;7&@N%i0wF+i7fDNIjhSPf*QUs4%oRHTeMckUMRM2BeG$@i!?KxZ8B%@B**<4(2ROK|)>9qfDUKkodG-nho2oXZ5Ss*nRKfV~ z6FxpCw2!+RjHJRDP*Q!Ou0cv=QRj)lq(%dVm#YO}L`><0lUjI+WXn-P9@Yyl~&^k#5==^RT?O%#(UO1N<4_ zU|B;ST8JUQRQKwH)+qE5*r;>-di!iXm%2}o<9jpkhAV?U7I64?zs>XO^9(#T|I6;_ z=@Lcd-@!7D_>FOyBR>ku>&wocxqZ<-jQd1RIP+&H(o__b7@vcL5zwGxb>es-^maB* z03J5DxIiTwK#R7L*CzEQ%2T`us*J&Y4TWqLCHn!uu$yI#)VY zQJtzxg~Q0xuqMI6sSM47iKrsc81`8QwpP&USLE$7C~=97y8yL@oz^*UeM~co=VILI zczYEN(IthQmoE?)!H+pJ#I6M+2xFNtc`HJq@u(gmJI`E`@hiYv9*+bdak>Ry!yA*z zTcI+ma74HS!!j)eKf~5$09P$G6q{4!lqXckjpCs-Od3ZiSz8p0$W*zBm<;c!G3Lz} zw(K=$T(s*LofN;u_*rK0Z3*I;m$EtL|Q>^lN=N^i-yHWvtCGhfR5 zfDyqTESekLP2r}Mr4+0LkmFjXFjSU|`$5rsSVjcJHLzz_4RGRClWCLE72sOs8k^Wn z%>!0gdkYwB0_?e#f(c9A6?F4q%pDW(T_V)xa0yNntZ_W(sEAVWaV@Xua&*`L4U110 zQ)OUS4@JO6;SH>6wwa~HLn*`tHxVyIM^TjarwrE=1PGw({lR>|;#4QCDs8(r^~ zYim4g6C=!rBnp`jkedG|4mT;^O5>p7y?_b<*$`~Xh?)Ok4k$-R0$TXyvnojjfm|8# zfS1zvk&f7wVkm@8M zFl4(mRH6hH-KI;v^1a$7bF7kv8V-u-EiFqf!gmQ=XWCOgSs3 z%#D?i7FT&rvZ)y|YW9;HAZJ5PQMmQ+y>Kq8fE>fKHeId&GRjoB$u_BD282vJ1*H#Y z)G&{7cIOx@mG5VC>LCpx+jXHPD!q6(BYl@9tx)>s9YQaoQ+{D2RO4iin{tF z*ef5lNMZ-Y^>gHZUZ3s*CLYoH;#HdF_`Mo>aZEkNJnDS##$dS6syM*P7VI$~=9)RY zU=H?4(w}6lHZK5300UXXS&Zg3r$SiBafF<4TPipw(=FP52)N=*Q|9aB1f~c%rgdg8 zHnybko^|GF5`A)ac^>8s#*|QY4kIEE5fs`Nl3hvo8d>1|BdpB^s^iw*{Rigi204Dq z>Gjj^Qytx3Jlag1t>>Im%RK;w8W|0oErU!uQGTri_Bb;m9JU}iW>d6VG8q(EA?t2E zqaCsLxQ=HmSEo6Vhj4HM(0|h9NITxwBOHDL`&YjIqm-Y#154OWW8@iLw+?w88sZM9 znC$GOa&MP!dV;ZMLt|eYu)kr35a}VGzhE3~eqIp*AE3`{3frXFulb(NUr~^W?K8@t zF}-%|u*u^9{gVz|56GR=xl`Zj=)e6I_y~v$n+51XhO}ieC!t`twu(<38@W*YQ-`5L z1R_uFl2N4*!;ReA*JYjs7wXIm@Sy+DhhD==Nnn%!pb>WDncP>QE0K8%JiM>3 z&vBX9TfU}FQeGWj05-?M9#UPYGrg~1?(I0^{=6>R=IgRR-b>)#!oF8H?OlUyiZ3|2Fw%ypT~9Sp(ibX>9%Qa6u_IgW*to}$#Qm8=dOcNi3(({E_udr zoQ$R-e~czx(`l`s!G%4-465GH|z41`Azx%sS+leUa@ z3H*H=W>2tliV48X$te44$zKr_s%^!M+7 z04S?+98R`jzx?#G^zG($x^?$%T3x-#;SJ~M)yv1hd`QC%LFINMFQ5c%t+KiEJpFO~ znf4)K_8AQdC_-2?tcrOZ!RjzB&q9@1G;2?0Mv&0g05i7e9L=-Y#oFIxA!M(l+M5WH zn_A&&u!#h3ehXf#M2qSd*43g7Jd5Jl7)2dVyN-hwd zg~DkyP#{OvP=-|96=80aCXh-OClrz%WEoki!)zV_jLryg_za$1>AboXZTX?MvotfC z+PuES(OFi0YP3~a0z(GDb9hY@4Ud)Odsrohl4nf>wDz;}__r2vpy!?)ks_7OHPHT( z6R>Qjp`PXJR&C(oP!}=@8?^)=Y<>K%Sh;h2 zJo1ue>x8eF-ch>M7nrLuOxiiux!_(t>puCLO#s<8dtqn$b%ZqSljqS{R)AJXWmu)+ z@#e}3#QIfw`cHo)Pwf~W%K4N4^b&KlxV(^l@ZP_sF?K5p=k1a7%|3cgPMbMRBRSR@ zOE0r%F@lU!glkY%Et>J>(Gz~@Rhph4XtkL%TL3Y@c%V2tus1E3ztOoBT0VT3rU;vr zaE|gSP!zPewrzk!G=fHcWYA>e-4_C0b7n_aNB4+F!*S(kKus@+0EcDL4Mxd5vr5}4 z@F9%VC@h)|=+*94I(zy#;m36hBzc|W;}p!n10cm|F|4I2CCZb2&4BQG|8$S4N?KUt zVR`t>tV}oi+&nNHiyNy8fNzhN15_g??*Cm1H%@a7Bpod$> zF(2*`SlPj>(oH)93?dffZm_pwJVh**pQ#>Ypm(%Q;0_a{dSFXr|<;YZfh{H_)$ z0;YjjwBhLwZ4CYiI{5wf-%UraU#BhB@cw`Izof>@-E{Zfy>$M1J^g55GJW=oU!vnM zC?xpjoKt{0iWdE4-ZbGHW z4fgEgOHz4eVIi+!xB4)E^`HF5^vBO0GkzTbbjTR`V0#N@lH5T;le2`^Yj{lRJAvsj zoeKICc$EP4qz)2Y=%zVrSpmm9Dm~GE>IqaD^9s{NVMvL2*Qtx}Bp|lBNT`)-4$VE} z-!{xPL0-KTBm-B_z(#nEXU7a-Xz53Yt?{c!Bm~+q4EeR8Gp?a?sQmfN>hDYTvi}~z zLB6&>Ci+NyXaaw$-Hg2w4y!sAT`JHBJHo%C=z(a4;3kT&)Cn;*V+w#C2!m_%d9EsIzoyzBe?vKWy+$CqP9`~qxPom>V4`#zu9J?)ov+Lll8 znvQ{cQTxPxm%$R)2vmce;~6~l#b=5l1029`_OVsk6HiL=*?a6qiH+JU%s+di&E|`! z4bR9W7#>f+b2~QY#@Bp)@8`IT5=cg!r$clo_L4owKw~VjgU*M{s`DvB+5tp6*A=em zIEI{3q}nhfE8*aL2x$GR5e_YR2Iy?50s}{WX%{&kp+BFW_scPw8`viISNjmQ zU!Z0F*hOfqOq03cMc|8J0tL|tW>HW8K7__n&ysm(_P7VJW!a`G<>e|+J9KXog526a zrLipD9;?_vXgu`W_5o!o!j^A7Ob?dVPzrPhJo#z9e}*D^#Cr~2y&$3D49|oXC!&3x`vO+UGdXQ0%S()b|6}vZpr|mM z1A`3KQ2+oy07*naRE3!b$~H33V5rwtA0sTfXu?C>1&b-|`oJqqJ!NAn7CGis4)rxsyZxz08t?`mra4T zprQZ(_ow&eK^YwP;91Ui%~#`_XL7;EF?=eiGPx=jfXPeGjdhX%BTvbrG8`La;CT5( z55#8N(TG{O8Ftg-*El#*+^CtYS=5VPi^Gwl<;1YDzht$X2xo=EeZJy zF}&t476@Ti=jwp9dKg|kM{0P|kldyo&dfxCTux+UcJ)T;F{UH(ye1X_7li4`%=gOL z&0uCn$*Y>dP-yh*ygS`R>Hcw%J zIi}3Hlnu3ttTutsXK&|Zvtb;D-BAi;8tKtz>nKf}o4J`Z4=Zu&#xl!A%Lo9W+y3Lv zzDx&)7inkbh@OU1kND8q_Hek6wUDi}WtVFXR=9rl*F zMeaXOfpOpDSyR+PF!mLa;_GJ}hYh`W>KUzFh(UD{=tH_$CFnEH1X(E#VPWRD(vACz36vAvA?r?OKX$i0e z{JlcH<~Lh=={Z2T!vJlbJ^^!9N5?IZS6nBjRC47SCgBv19|>fR0Zeb6?VNS?M2%&p zbdj?{=Ve||03J28@!|zK8K;Evq`jZu4YdS&hJK`4^#`}tQk#5OQUK%lC>pGV(JXN2 ze4hNhuYUI{3IsTJl0qNT0!T><;7|mp_W+GOuIt|6T*kCW9`q$)*$(^Zi)S0rH*^G0 zJZkI|*)5YlEvr$Z#mxR$WK>3k;+%FM9%iHju;4k!0?)DSv;U9BX48IgG2Q*(-@$Tm=H`nhX^Ec`qnzzZ^=|Fv z17wr-DXfWx-#ukINEKjgYos#0BC#->C`6u`3?({i1b{C%F_b0PkumYh^*1G?MW+Cb zy85lg=LSRab779nn-fTsVE(jcIb>1BMOIDS7so3y4nuW4&fo>R0PKiZVH*r@1_Oih zY|gPEeOpFEUM!z+?M-p=$Sy zaW&aEPtviV<+yRk{2(X%S3>_gS4Eyv$6w&7J{K@q48`y?A#pvgq0S1R3UlFeS$tvp zk*J8}nqDiO9W6fwuo^PwS$m^wjs|(CObN#~GP*9fL*W{f_VGaF_A1-VD%Zl^HZ& zT%F-M`k?+^#-W!G@pqZRD!^W6Q5kSh*@JmRTK=<=4JzSg5+EjwV{z*naWpkI~0-$ zS`5tdZ3scH?)%Nnbn*N-3xN`0BRTON2ySRc6kOr=G85jReSqFcfw5oE82Fqd-IoBK zum1W;EQD0=4khn_hLK_y%{2;?$s%}ygP_)GCc%~YU_-ubcpJ!>_XI1+-^@9ZvL6E| zj+3L~f$q}!Cn#r)tzLbX^-xhL^|5HomHowht^q>n8Lh#*u~xE_=AzjT&a)f^J_=eE zCID0NoL5~876Uf^?&0W;+jBR)pUT5F9eO3^r=7ZH%$Ef$R=NqPW8?e0tcT#Kf>9|k z?od$tbbVa+bFN_rdXDLRnr_~|m8vigDlhXpYAY+S0LU(5+oo&%0ZHH2uyZO6S&+BRC^8Hso4v?fRFI8Z^~LD&KRrF3R##@z!-sdNFl(ld z|NMwtCW;SGiyD$j9wRwJ7j=||xvGPG#s-R*nWu|5Mu*+WGtd)|kn%O)aA#vL*f4vA z6{>ZB?`oP}T;n*YC8||taUcMRtm(5aK8ZZ8JnOl1<6by;MZ#d0jdkRd!UKTo+~SRt zU%ms7n@W4fu=&Um%^~@Q#RLq?D$4SzAXTtBF zeV{A{GbK>q8nQV)^8iaTK#m0w7wl1rEz=6JyG#?~CBm>B!bSs-mMu&sk(-tvfw7FU zHqMD5a&rh9fS)aL4gc|HKTnOVov0q49>T^Pzoh5dlhpb4^R!I3(z8`fG3fw8E3ka1 zjxa8Ut>WBgcGxryON0G=0TVHMV~y5DfEeaaBR(K6)MnOb`vA;3O^2|7o{D1;jd{OD zRXoMQ5Y#wMk&UglHn-Ew5(P7fs^QOmlWH3oC>vs&`^dq`6`+TCzZjoS*Z2NSD$Kr{ zj%Sur`}9>5D2-j>^mGZg^7jGPG-_aqhk2qNKCfH|J}oFCz-Yx)fNrPRi8k*l64(J@ zmpHB^hy%fqUplRl5E{}5eZDd39dv|tC=bAm{%PDxuxpjKGH8C6?VB8n(FW}jSBTH3 z(=^=bN1Y$#%bawpQVr?FXSnYe>#kjl{%#N|j!C((D1r=P= zYlq9t=MBqgW3*4QWwKF@L4e@druLu<-O8HSyx5SQjF7rr8_+}ELzgof{%2S(12f)- z`Q=^~d~_QCP{tJG41341UTed6P3BXkDdv>>+Loh(-5$jx$fAwWLsYx(oC*@{4#Bj9 zt+wB)``>>@n-?zPJ!Kzd<1I*1?ghX6g|W%LI%Z^!-%V+mq{;l@{x&khJn)+9dCom2 zh^R?MgtNw;_PT!F0*op0g?-K%drtfA-c;v1_FzZ>sOEr+L?g89dSk0Hn}tKkb^Nnm z{`5#Sv509UMyoIpVUzv4;5gD+_-R_3>%dU5xj6aapGh#~A(+HT23clN$DDXj{Rk{Z zo^S#Z7eK+ADhP5Vd_KHdlZgAy9zT8fYhd{`ib?>&Cz%Yv19?vO0hNWOMOMs|x^08E!$BhG&B`rXPRjS=cye7GM zhEDZ)TcD799`rgaoZhqHDHWd0SVzdo>(I7hiuEc31Oiw>Put1;3XtI`5|gx8$pa)# z0PB;U*NZ{(*;Ee7nv*hu2dVchh-15BUoeQe1&yPqb z2ZUe21Yf;;h78EeSTunb$M`gIZT{wYpOK@%9}U~tQ|{iK+pu*qhocxkLd^wO1rm7l z*sF$-xW+WCDz1hcc22(LW&boi-`Zx~kZZXx+56wGT_$ zN_C#^-pX`{QvwB!_70o(kyV#5fs1k;XK<)Aru}0 zDP-=D)7swwU~-r;8G5iyLk>PCQ5{*jTDXgEYBxn=Q*(t9*uy1w4hp zDAOX~qshb>B(OZoX#%iYMzBwxd`jxs4n5IeR>=WdAaTBjF6oj#YgjXi1{?#|>Fmq}son3>oCPQ3 zA`Q962p}TQXL1+?3xCE?zV=*{ktiU~Dp-#QvvONQYTALvPwDtv(jZnN7;T62oO$2X z37-=AQAXYL}(cK$c(*kGao~RG2P|3`yS_3jvi{ma^MWmKvVDyt zgrGj6Pyj>z96H|We)YD7S$!@jlEM#L%6pV8X=i1Xe{v7=IIDo*46J!ifs)Kw=qBdK z6c4_9OLsneVH%h!*HDd7NJB=T~(cN_P;3v z+~eAU5cGL4eCYT~QlLJ;%@ zMBEl4m=u`hV6i!wXe%%bwaLJFo~i@w69Ul~*tTJge6uO&W}D~P7UF~)t{oVTGY+o5 zV1W!nDp<7{4pm;cO=xDB+#(hn0Xe{kZXP#M8wC^L5yojktq0VuE-$AC58orKl}l5M z{TyR!W#%z)d-GOupyF%B^BRC~K|V-9!BOCNjshV+jb)4HvRcKekDLq7Un5scAiyS) zK~tdY|7H&|zx1^2?IQq+#7{O;2pnvM|5?I+&StW|l^a%t?uK+>&|NIHg3TNv%!;Z> z^frD~h)RT(WYySK%%81S0H17V4U~WP+)hI&o)(Zj3yY0_B9Uwam;i+R?I{Twl@`pz z846BT=n8`!c|(lLT%B=B&x1jAof(%2^?DONch60vVj_QKz^3+dw3tUg0iYNR1v^VJ z$Kz%!;{dZz7To{F&3n+`%Y;*C7lT5qP9DQ>o`kHCwUa*h@xMybFiW;S@hFlq#(Thg zb@8$p09bk&<%(CyF=;z(&WD8&x_RdvLKYl4ZYUS8-$^`?Nwyg*W1Zl9ACCyr<1}oR zV|~W(KzC&)F+OevH?Jl47v}5%qGR(hM`~O*vOSYDw0eV4UFIgDVW{rWsNfdOamjV; zsYKiP097hR`w6LJtN`@DWzRC5CMW0p_dY;z0Kk#88!M}5z)Klfx|x?;&AC z=ALzF!i2S8&pNPYN(JgR6`32Ngj9vv=HO;f?KRp_+*q2$7;--4#$o#8*Pk*!w4+&^ zPXE)te#m^vzFkEFXv^-m*0EU&YrH#*xZ-gKJ-A2 zo@>{z!Fz=3zxd=|XkCIb+WS6DcaF(BBCUby%n^)#1BS8(fVzT7vgx$*G6UeV%5;k` zu;aCeMV;lE+KxaVVatmy=^Q=2i|j${Wf~&)MHFlqX0ya`e3OK_rZ`8m#_#^89rM%- zogn-CVXpx07`pJ@X$C$^B|ad;{&!^@2^IDcVLqLYGjgBykU_(#>PP2l8qa?k?|e*M z$a~t#j&j0S1n*PiP1-icIoAFtjCt0{lQ(2uVjMh|Cr!9FE!>IRG3AcDW5cyebF?~9 zf3X(oDZj3H-ZHxC4ClCYF0-R2571-}Trr=)C~15d7o8qcmws)t^EXI&sB(rf z8R5?1hYdhRY%em7Q!WTnH@L7eL6Q-Id%>I`OhbtS zDqx2gWM9{Qz?bnmJ`HkU`wtl?EAa!=F*Y41L4<%d1cv|j+B(A77|g+6ejRf@K?s^jNC22t{_dO%enJ95*XT*2oG|9TW|?f(S%$k zdrvsJ=#*K~H^yoB+yd0=2#*Q!7o?gEAmPC9EEZ_mz#>k@Z=GM}UyZ6aiyM=8H;WCy*XHD z*Z_lCsX~BHF0c#q0snROOXq1F*=f*;YGjib!|3xtc zbE8pFkflux14m8+n^%U>Z4;Z6xs>U`Mly@cKjY8<*gs8=JPJ1eV|JnTwJE|2GEO#) z+`S-|10fVVyI%Gv8Uj3Q8(}Xy&tB~^cNR#n4;V+bW>vyHdP-JTjUot{!hz`{bZ5gt zSDiscLUf-zsY`NME>YHPt|M@gB{dIVJkND7!gI8gsMY6rZVsQmbvI6D=%9=mFkzOj zFV8JdNYG4QZmh#_*(w5oMHv~=^W0OGKfQSLh^I$zQ3jqOU=F2;pE5)mFRTWiLxJC3GRlnLGp*Xm;UJr5B(fKlzFL-s@owSvOXi^c(LzMR(NqC z$A<2++zjW*qsA=UkVVh|soBVvF)k7=tls9Md*N z@5}Z#w%%F0|A5cUk+-v#e)!>sFd@h;pR1#g3~BFdyr!q$6QXuFBQSX;hwsxE*g^$& zm!0VCZ&o9zTPD`ma?I^2O6~5_JQc2sdmY+? z(B{%b4LRXDb4|)jbO+Y@nx89#GX*!5S;EdV#eCkUVKqlgVNF8WOfY};&QgcyJqn5@ z0iGLQew7Aa{)Tg7>Dfi^!!9gL8QHDzxwQETkib17=LLhz+?dC8-5`9;T2^%aUXn-3 zSapUX2MeWl0r&)c)AYASp$i9QqlQO;;X}kJQo*l9$SD7bmfDHaDYk$I;=+KS1>F2rGa*09xBADDqgY3<$`&ajFRNDuLkpWt697G_}v@ z1>Y}VtKtM=y~0rxp!m#!Ds7wLV4VoTdRCR6?T<1kFK4hv*Vqur2K$`_)2>aE-O@fG zdl=5JDcQYzW*I|dnlOA7wpVaCv^ay$xi?r`_MZ_O!KAj0XJxEj(>%H^IgtXVV87xG z0uuE-pq1y6$&<@NSCj&-^by1oA8r406NYxTe@c&@O;^K=V|LvPhG_(4CIA~Ybf5wjm4H=!9$RHT zfeLh*g)+2I$1AZ!y2E=TnV5-`rQ2u?({KOud1_z)RP5EM#WcINkQUc&L|>jsK)^9Z z#6yNZ`SvB_p<2rG$Cw1fA;6T3P#geG=*|8wLlp5D#;=7S96fy$jMNMZ=J7EdLTC;C z&NRVrD40wfO7w#3M!p_wlI8Gvei|qjMMI{si)x*iq-+x#uJ9mwX;v;2EfA!Yt=hA4 z#OkU%*N};^W4WeE5NK$Vn}vY$M_v=UG4-*KDlCM8w+Q0mP22+=8*|?i=(r*HybIR? zbnL@sIHD2?2+0ie9tWu_z-(38Qmad2*O0Fsy;!xh|Fi;lUUXIuKZzhvSAv zHKct2kY4Htiixvk($Jn@6jaPEu0(aMEh%j7YPg8Zod89JZXPu{&?O>(3(`DoyOBa|&e_W2eiUA}d&CKI1p{$0#aCQr@1=2mye>r$Mg?*8r%I zA<5<#k2clQcL|5e4m8Q%ECQ4Yc#1!GpAaHzFhXcUP`$rRRUdMCzITvr-@cn}aiDVv z0Q&UlQ=S5ki)&6G|1&c)RFiT(B0$A9DRZ}l z*^cVf!oD2yZ9dGU3<6{4;;HfCY1)1D6i|Jge);t$$R!(;XY2yjj$sN_+@5w(qv}&2 zsthc10Jvv`y4Q+4v*Br5ZYXDEPCckmg*XJ{9vz;ifByBK0WA&I$RUc;I39IpiR68W@{jH@^%M-sZR^ljAio z%v*CO_Wr4-8h}P@daU!Ia;;$ytk8VAdxrA`i%%!~>pbZT^K}lkp9bjY@xwZkSLTvm zlRG;?56QjnzD{3!^SiV-I!N!8R5p0v7&yIK6}sTo+@VS4OQt(M446d`Aww!=j&8!p zShU2hV_%~DtR{tBtCWzt0(p=nm~(*PCF7oAf8}6gWz^vb*q4YL0HP1O&7Gqu@g&M} zHA4CrVk%0NwF>L3evnbVR1Rc7VQXbs&GqWBUpy*Kr7zR%sVjR#5@dipVIjJxBMa_X z_2;?ZULAq%3^kz-F=kd=SFy#+9Z6=U66==2lj`V`&OqFu^>s=Hi!4`*bU zetM^pR&U>6O#$IlPM0VOc=64*Fqh0P71UYl1e^a_VP{A>12c1Vm=>$tgS=x3t@pZZX+;Ug85^>-84lDrUM}KjN9YU0OkCiBNxv!5bcL- z$rST^Mt{I>kI=Wb?xho$(lOF5O7%L`-pw>uoI`ihVuaQ#)hWO)&uaS16(Bb^F(2fC zZCm#D*BJxAgTJYlLWdKb$%nCpLB|1zBM^B8dnpPYYCN}2uw#~4^C0pL>n&4qhT~R% zH5A+lQVsKsk&CE4i^kK)tC>jVQkiVu#aS^bU<69pwh~n!5rqkk13P6uKKH%$!_Nhn z;&)jn?;~(50F=R1jGy`BI_eB)LeYt(0E+K&oCC95cjK%i2aJ9GhP!|VU5vq zoMg`UOgFMx)YJfhj`o=skspaE><+r-Y(NVgWX}9}SyTqjJu+4_dh}uU9bTj+@svhQ z90QB?=DB!s)~=5{X8_HJRa}NWHN!JjIErm2o>ziHGQsnxl|zj(m@4#-g_r{Ow-HEz zp!&!*C6N~kuCQfTHm~ua7^m~%x^WTyPL3&ifV+fj6|u>2#yR@caCaVEEim>z%2~7x z@uPm4>qNl@Hqjaa`tVx*%gqtyU^qbW((Ni3k9@LPQRO255&+Z)x0LXp6#Uo#EQDqJ zEtK^k-)%Zc$shti(Bt;fy9K$x0(SbIeqvA``$V0&R>3yM~c&I0i;yz%T$4!8kR1w!4#BP>s0xit zjsk0aCK--rp#%vH;r37&Jncx+5Nwg8AhxlJvt z01}o^$S{@|{t{lQl(daMPs#hbJUU3N!+pXG0Fbi-)(Qb<-W;pdSJJy4pw2_Y?wEu% zOf~7}Opy#RNWuK^cZJb;8HYvDOS0ltIjo`NZ4BJTTULpvY^Mm{(HQ1L&A_1KY0VJ%FkHvbn}7*85Ke(|=P_$>O~!s@#VCf*aIUQ4Ir|mW zmhJ=gii=rw8KNFzJcRWT0JSj==2R^$pyZKqov;fSk|U0tQkI3w=Bsw0zuPE|t803V z2{4!w^ICWu<}G6-JXK?qL)vGNu>tv86EpM368UpSN3?G_iS{_(19rR5j_G*6N$%U% zFpv}#v3K;!uL&*mU~7AXpZg~Vv>VE$_Y2IQdx<4Lm1>1zFZIZ~a_>Mm5$Z%)BjfBE zm|@|F4gXMAd*;ghTYZ5!0n@I$U6``dIz{#;( zq9ELxo)Z~{6WKE7i$B5SQVq{%uL17LjQzAMLXv@%xjP_&AlS56X@M3Xh1pV?Uxn!= z7M#0)DP%u%&Nz@Br@M_4pnfv6>*)wOJ5|{;_M=R*#`1#ozEJm6VG(fxruh994v(`p zfN7e#3pi)ENtJJpb1BT#^^} z5;FB@Y?-{gMV=7@hn};kvua4TqC!iZDt#l{uTF8$okP|FNA{fan~n6G;M^b$$R>LL zee1m0Q_*m?t$Nfg&fS2|aKI|#LQT7GnG1DMjusKY#4)e3%IYf(ypasgOGd zEdCNC+IlAPKHXC>&!~OoT(oR^aosLnPrEQ5+C_Zc2g3sj_BF2!B zZR92ZIrd9Z#W~``nN4Od`dp@IO%Y|142XF~j-sj-Ti}Yep=+ zzSpteYJBm#e~W?`K()U|^i#P(76kn6fmna#5jj&<@K?E>VR=xKYsC2duW(LswSdhJ z>S3sm%-86k-0Va@&v7Dr7Lb!&ROP?JgPWsqMslJh@%AzD?8G5?IHR5iNM&CUpinGU zVdkL!0Bn~iM()ZjQKX<%6KSQxC-IDA(*!jg4EB(`1rrd>cQZC^IemV7f;@+tY30W4 zFmyXFpQTHfw*vRqD>Jrz!gJV);~d4ALrIuN;xj4elQ1mYl&Ud8pS)(Ty6C6HJ)I14 zW5fIRcf*oaT4_Y1teF{c!#XpDd!5j61j6NTQ<;FQLJj(v&ud`KSqmd9AYk`+(6p4b zttvDg?KfV4ph}@oVfkfJtjfdr5KP)Uvl@$j?~!!yW>Z<#Evssa8krA|hH|Y8O?bGxN<&5I!OBDOf$}hSIu?#shGK{k zsblyi$OkGGU?;j2`mE8Ch1U#UUen|?d$wF^93HWGsZ2#to>_%Ug4q~5&nAtshFw?` z&+6MJB|wGY8tOE;=iL0}keFAZ;hwbzBg)yi^{66YjDTlAC~S@fsR%Bg=aa~SZAm6D z!k#_qJT_>Fb96!u5^WkT00hdF$BzkiZ7MC)Kx|@@v=l(6{fu?q8Gg?FBi zR+rGYC!8Epuuvrg=lx{G)DebjG2_%O0&5zxpDbqNp(UrOuo>S!CaFo5ZJXxWI>=;rs zd?|aMV-J$acp#bLopeJWA_NUsJIP%dUi&y1*FKi5g9v z`6>*zMMx90l2P}Uk+}wWn#af*lT9u#>?mN=R1TSgHoE6E?H>%ulWmlikm=GJgXS8l z%S=0oR1+4-zKk`(!JdyRM`NU+&oy-L@)Xzf=#RBSal+K?_joQ(*Z?xSxEeXeDKlzw z@{tf8{vDF`K)EW_Z+haq4K6M7`k%kUVn5M*ca zBEW7FTDRDWJ%UU`?iJ6X(aqsh1{1?QbYw>X+!h>YXg%4*eP(Nm3Ff;^PNC0jUUFO; znU&>K9+b=IydQIqtwApY@Zmj1G1}xDo0r9t$9~8P56~sZe241_iZkHA9H=wU38r?q z4(>BUead|>QT+Su{ono!aweV-t;1&}A$+fT(ryK6BiGLk3yuwS%UIZ(*`@d!^ToRi zW5(aPhkK4&#!o-%H3BgGe?Q&#T*J-6g`9iSZInG9RMuZ&#`GmW>@zxaLyykAtvUpL zempuosNi^`^XZph+Gp?$>OA$NpwoZLBFb6{rexlHe#c=7K!beym6f|xj$`|HT_D@S zcFo`@XuA!oO8R)O0I}Yeb@SQOLD|e?&b>Crc~TqsY_@Ju4rQ%%4%7|a!*8EE*v&Wp zR&JGnxE~YgGjNeqc=z_LIIR7M1C2dpKyOoFt5aHwI)-j1_RWK~U<7tRp$&l2wi}ks z;$h#l>6P@(2Op-TwHp}u4%Kgjtmpw{PJnqGqGGGwYUDy}Y`la`usJNA72E3Ni{lu) z0_K@`hEFtLCjA!>l%>^&>HUv>l2-2CqVecyT7U8+J^K8U)FGs`HaivhYG+Ko$2{R{ zLn+g9hwGz;BfR6@uJE8$0S^p~-iXzJdZ{YNJl>}{Yvx%Q{xsz9R(LhEh(|d74NBbnQQ( zhuI#6hq)05aPM&*#3O@drsqk1o~IgdE&zBNhHD?-=-l5SpUTiRX*p@gdS5Z8O?q*i zozj?*kj?pCwl3H7YHL{uhN5vJA?X3oR50vp#>ybrV;Z@0=6QZ;HU02|pQJ^uzd&bx zi)!o-w?&V%G7f<)IWCa5$6x*xP)IKq>S<`q-KnCiXUT!` ze9<=RWz}(oin^#~MB42YXc3CJeV!U(n4p9Qs{^pIz`AjL&kR4oe}^ zazq}JAk)UedGZ&x_8asPJ4xF|oaA}21-nM|C9)<66lkbytst|idYY3&stb#$f(*?7 z)cS-1^#CW7eUv)>7_x-Zz!ujV7Wb^%F*?qJ0z+8}Fl(*o|S6u6FwGc z0p^OC1Lyu!N2q{xd~UAZh(^TjF_aE>XWT57`CbDQfS62%0rd*l^L{XJ{H_u;ls`+T z*S0{>8xfgrk+f^|r@CW|MxN#~+QZGP6VEywWBoKF_Lj2xl>vo(jl(WX5Pn>tz@vh) zEiy*WHwJeQBB#hnCC@l=jN7niizA^r01V}|$k=Oy4HqA*;iN85Gy;2$f&Neb`~RIr zaI7BUbe*gprmgi|@&m_U<)+gdA;KkcA?;&n3P}a_*>w@TpWE+^d21=Ruu=Cvd?zix zdnX;x*KLlyaz^66r(&2c_}!D2u%H{z$Q;`>&cX2NG~~MLi3lX$=h>$^_Ag#fWF zOAj#RYl6bEgGMVod9}}+dOX)CjfQEOiVWomJ-s?2r;`4FHbkE$509SL%pit}XGc&& zPIB0YD<;}pErFk5=p8x&vRuo#UvwHU(~bT>Iwz|Byc4YNpwnw(FU} zF(KTI5pBZmnszcfu^hVP90t5zMb9&~6Pn6T%o92#T9d?ftU`6Y-$F=FwUh( z^b}wkJ9A0yWtqHf!`(9@%UlaUgAEWYb}62~?B9 zQzsv(L$?AN0jelu0F9>!5uw2e_EZAToK{^My}$yo2`tD#C-52bcijPoYE7A96Ng?h zgzMf!e0c^*8tx+thr?)d^I*gr3wu*A;lc8(kb9Nraozl z`4#zvXyA1ni44s076v3>>QTWWNYW;lYv&r;SXsd3zBU!WV+o_pm<}DRvpL&sqJTZ; zh;_F0l{$8RYXc{p`>gMi%U>v7)#&m&ViO5u*xy*l7~r07&29!$wz#TdA=_TLqYhzyA5pAs8|*f*Zkw`Cv8* zwOZmFZ6Q)M6gPq=E>YYPF7pE23o5Gj5}x&$~eALKNVXQ2UvEs7+-(JqPyMgvdlx(|rr{$gg2HSstg zd(>sfUpiSI9b2Hgpt43m>+On&1xVh<-x1Pc6EHL8or(OR7DGuD z0NR&*I1a`)iUFR!vC6npdQD%YAseKzdqh>|a=P{259r*!hH=d$yDt%LXu)8#prlVQIA?%?m0LH`-J3Vq zJe&lvvy~3m94DNwd&%7E4IN{gCV7rqYgC}tNv=oW_W_540zhz)o?}!fswmJt>G7Zb z2!qD6uvQn0X?=5@uuGBC13rLtoTvHhZu;!gf1y%N27&Pg>j~qFl6IcWhZJa)WoMY@ z0neuKoq*LWz%WnQEuIZ~#r4gp^88*zhcS6RrL)12YvsnfyD(P#dQ} zg0WjsWZ3c;BaE)YAa>K2ub-3C$B}O+EERQBKKe`;B32zIf!=N;f_wUg9zTK^p4)&8 zju0acDn_6WugeYrdjPg56sos&b`Q7dzAMY8o7CYHO##TLZVtm@Gw@ZGiHmTTIt3VIixrY zmxQ_}*s~T)^mSK_e6h8ijE1 zG^4%e&$3e+*o(@lN%VdVkvR>qxqw{&kdZ8-4+1R1!0NXQm1NAd&W?RaUC-<@5EHz| zYp8RhK`v1)pTnem_l9yW2z5_I%Mf1DQL~`V9<3hvB-0b%i~T6@P-le9Fj${gAZFgD zz3RNStvqb=VYPP@GhM(4dQZkJL*$nZjn{Dx8Lg5Ok%fz5L!MWl8Br|WGeuF<_Qp$W z$visOes}09zS&l9zIYP%XfV7m&&;3W4g1XX-LqYEqxXt=^V&L5%%%FJKqm#qqE2wW zg2iG?f^YxRj%p(;*t1&KXP2>++;PF2yB_N!VAP3kLu%OM*+#$Gb+04CY&EV^uAVep z&35MR*nY5&bEqS${uS?t8N#uS`N>j=d?vliaDe$Nzr-`y7+6iG+{x}bCg)Pe-`89R zqan&-8Rw`lHjUCI>2-j>e7@5l!T1IJ*)R@xK-C#MG7?)C0d8xQ1y=}%o?=8_J^hvf zgQG}bwoAF8ok@f+C~R>CPh&5x5fZ9pXY^9Xjn9K_qdMT5) z0xl-#h(3$E~VU%@3$hz)&Rr&7@`lUi1w^DIU?9rAadX#qKGg8G3-> zl^d3Bo*cqf9nqEnmIv>3nvhQ(Z(d=aQ2FxvCG*&1Ghs6bhI~}Xh3UfL4PoL2$YBjOv%u@uR=&ZmjpKFa zn72Nb@mdfg;_3?sNWR4((0H~8x!t^d2XJl3;w&|3RdIpwFOY{dKq>SoTwt5R&JlhF z5bxsnm>{fr^ZQJJQHm{?cq^#gYy2VdO$~p_dXH9M?g68QLOd%~ zhexo*Qxl?xHAXpk;B_DG$N$WO0}=V?F@^}8qN}nVd%TamrIQd&lK{dwV6XY*z8l~S zSaH~cVU(eA?+P5t066o^Mw$0TXfl=WDPZ39si4aQe9DN&22Ivs`d~3F$`QoZ7(bJY zQFdKm7YxPI-shA#vHEd>edHRN8x#4UNa881Y!kWj4BvU2kP5PIH8K(yZ1Mu^RRz%0 zz{_vE+DXqo|DHWcUK``|fP2|mL64`PY;g$o8SMHsVZIaQ&~svSeka(+Q*#u$EYd%5 zb%q1k>7|Ek2#ha|`{|ocH|Uc{1}t(rg)>p4XPT{UhJ^ah(fL*-*4T$ly!<0VM^_`= zbYmHiN}7w+z(d}9%3k!yIMZu%TBg|NQ#j1({9_7W4rr}%P2!@*upN*FQO8m1a;*aM z)j^gG!F7p(6j@U%eC<+TyPgXB?Si&M*BJP%Hb4iqWN@-YPH#8;`@i`S!1H1Vg{4?tcJlIW1r|q2kh{rtQ}eUw~4 z|D6FSm?*~Q`P%C7D)*}azFSAzkw0zpLzYK9;rjzzF&~0bfoOCH;67ZM`QaJ)#gjrj z#Q-(U?~n0@n~!Ak2+?Oxa1oxxe`mT_1~piJ zu50Mid`=xYwV~j9+&v%&Wvu>N`H5CY>Qdy`)Fm4B8v$n_BzvrAHODG%qJF&@T*remchIIe{KmbWZK~xnKx0B^Wdj_K5fIC6!6BZNg z0`hnZxpSOQKq9FjN*~U!1|k`Y*7Y+iR^*Z}ASaAmKVdRz| zWk8pp5n*^M@Vt>fWMg6#P^c!1%m3m{kWUw4r5D4%+|*T!pZ9WH3WQ&8QL`H$5-_pw zCK~!#JB^zOamo8RR0`o0+>j%P0$sd81u&afCdT#!=Fpj|bjIfu^3fW-tpKGRHl(dp z^vo3Ga%Cm`=-xZ&?%g}-uYdV-Jk#mx&Gh@<{4(tl4sBA=YhHlDJw&(+@oJes>RQj3pJ55RUIN9cc zg%SzbwK6@r@K$p4lo}_;&ijmF%y^Ip{h7SILRi5PY5UQ6JeEgjsh}=A8jJZIVeRba z^_nv%PsyKg(FCeiquF=wE#>ZE*fzp7G?&L)J0g+XZEH@JEeA$1sDid&2{F`qXD5Vb zlp7Pa-88aWwu>;i9g$7<-uXeg@q-Uij%Ktn(pLcXKQW&tfYVXtp#&)QeAGNSekK=R zqli_|7W~YtEFlWTv?iE zA$ndaClxav5#8LKZ*Jwj_I_o+vMsg>tkH?78_2pl=>l)x zFj^EMU>JQi^VxKWWcJ;Q9#F0^F!#-{t@3~)n<+l`BF`is5IiY6w&Cz_>b%zm6xjKH ziKOZ>0Dxc)($BSczYbrI+$NKq4J~{6OvsWuoBc3g55=BmuKP4Gmc`>w$Qe!}c=3$N zV(>dJ$WURlxwp>8VBq1`wj!b7E}^-qDGB6?4G8fJV5y^09esANK>-MJqi!s7?`fON zG6(7?P0R5qKG$uo5M8T)N|zLf5Nu$11pxtfZnh4j@Q zzhb`ffHxuyjAwy9o&}ry0!YpCG;G^w0$PygC0P1CYhQBgkvrtv8yO*)9k$Q@aO#*D z=HZ%srE&CBgw6R<`w$=_z<4{BA?& z)5xzLV$l2A7Qmli#%K3G7AEu|LKZ&-gaQ=haR^uxtd*l0I0ILPW7ON`{^qC#cMYx$ znIfm|bAgm0IgNExQYt_ECy-ZJ($!FtBUBr0J(N|~ttX?-*JLWfZ z9ZraIIz|dk0C}E~35@k~9*u1JJm%sRwz=Wz?Yrq6!lzzzePca+|Kd9gjIna`5qBNu zq4kz>c+d8E>lh10;m3^uc{UH^-7&F-Cfs0S)fvk132(q?tPq}=sMq5x)r=oCIb30LPf<4RN?P{%JRjfp$df~D6yIS^iHT^CFPE&!AwHW$K*+73b^I5@{Uv2shH$qYT8W8x;DDo4jzFjs)< zdf5(4mPHMsZWq(ButWW5Y$CUxc{Pr~1(O*t@6G;6-4T#|^LeT)uHhW^(%L)klIJl; zY8l2MNq|~=25uK+~^3P zJ*P%_{{%+UgW2m#YlPAJ=`~=f&wDg5ZDtsWeVvC|?tX-NHyr5ym~B z99<^hs!gsacNdJQ>{OKONLU<{8skO)aY}e@09HV$zhoyA7e58vGF!^Dj)q@41OZl= zB)zsQu~?-lct&aSpLIJ) zEbKF@p;dP71#@I@=dBZmR-T2@8tY}r$+%2;GID(r1NY}foPLut=G z=OJ=cQTDzks*QQxF3;oo&a>X%J^7MCfU)!;xn(ox7uwH}^MzcX17ygRA^R3y@v}+} zqHQjYk0QROFL6umV-{cg6VBO;yg<6}YzmKy?6%VHYbg+3DeJPs5zgW2mY` zA$o!ao=Mp*!w6oJ2bDt}&e_LS?oXo&rvbm}qDYs}Krt6wD4nxEzbBV$5*A5Jx^M0r zpM|$c%PY0?Z-4$ynz>a57*SA0s!JIr`c|EOj9VvZ{i~hy-Q&Nc5yCyASEH#mguz4q zm$ zuyNW;@87?ZMmY0yhSo1XqWbo~{rSt3pT3tm1=2P~jqe7eJ|m+tPR^M&CR(B(J8u~<*&o;1e+PqtJrT4C<^^k3%+JiR zmb@;YD_mojMy#x6hLE@TKd$$-H=4l4uPk@V5TS)&830ok_`0Ye!q>=501_|6w?M~G zp+5ydD9#wu6@?&+Z20*J7$ z1XKb=PnNJk);w^Jkkf{nk6Z90BA(mR51O=J5xA%yeH8NWc*Yy0flTtTl4oUaL(1wLgMQukK4MO>$gZ4>yl8K=D-4NWr~ z#(|T=7(LV4{&D`>9JFk{1*N==ITEnGjpnJNn8dh$0N5C%Ofh+O@wzOYhna{udhrdftpFByQ{^jEsa0&LKhT*6K z7R>X}L$i0$1;)q(I_X|(TJ5!?^6Mc?R@IpBNW71}J|5$gu~dt9@22TlD0upHP0*X^ z^WXj|o$kJ1^Y+pv24xoSW*#rBkK&TP*4sEEXTd%?8eDsoiOQoJGQuPa5DBk5l%B5( zrNHQov&zgm59RAVvY*G02&|;~vvOily9tv9=W;ZLBqxBRbf+eI0Ri#^NhLs}IX)Gx zB|Bk`NH781D|+I&Xe@}EM&n}+9iod7&R9RgOIE;IZtVRvSl|O}8t5MnUBmtJAO0(P zZe65}oh`x=o%GTxz2VilfiV#2Y74OZcKbdia;aTYhbj8|6GaEPEyc`o3M$38;NWqt%|0wTYvtar34;ImcMQXr)#Xu^B| zZ08DbXSFass6Gshici@$F`83`P$B}@Hepi^)<*7VD;HJFw&ihOSSf5{WA~hl)Hpdz z1z1#1U9guK+KdpYKm-uUdPQ|CfG|!bV4NlPWr=w!V2A}i=V5$Y8!D@qdn@5gC8&^^ zU=`=(F^miQ!@M6Apv6ME3rEOr6DPplg)%34u_l7sc0y0z{2QC#8n%=%j}W5k&)?0x zvSLv=yF{1hxY&7Xz+B|nKkGC^{(Sva`p5Tfr&V4nBa2-?#xZkVMNT4wl7qEC?u&82 zdM~~7@4JlvO%@(l@p=wR=AIWU>$F&ybxJS0%icENo!l>EH3#ebkTYW^Fw8m!%7wXu zg60$0Nc+6m>PH9FhQR{FMf&;O!r`&n&StnVPyDVS(a5O(4q+$zIAiC1m=AOdtBdZq zj@B@L^V4_I5-i#A#ZlUPOzJ?DW9eo9Eya5JO$x&J*Qc&Xhhg4{qVo?E@ z3P$W#h@|1DlIFmra)h(y>vQR=SIDvhxH85W8|)bj9uddcW^wTuO!Bgg^UFMS@PnTgp}XxnfN$+6xfG+Ej49rC8fI^Dd(D{=HDYg~xEuF2 z3c|itH%9>ko_QPj6x`d!$`hV^CObW7yOn#R0X<^>vTyeNJ86(kMl)&0^cFzMnkoYU zB+-?9@RPdF_XteSnxvi4Qq6hx8GJs+VWp~tKC-OQ+!~n=9f3V@jG^y{40--rPYR+F~l_v)3Zhu zU3F>>WzRzL@Mrrk(>ITPL(7k&Eb*7J?lF`z_#VZ~{s92!4Or=FrJ2e_%2fj}AjhM` z@3sptw8~TB1yWVwgYeAop@lF{;iT%V7Gr>vpZ@TsT; zfG$uNSEfCOlJ{VA%smOPY)F)^%z$ku{LkKa66gL;K;)!K)tabkZ?I_pAf9D$%w0k} z-Yg`eU@#^mK6&&x*B~s!=Hk!k$?yLg!ZA#bfBg%L1IP2wdrU=PB?^DWTg4!$lJl)ni2-_u9tT}QgCTStFIJ?4Uis=^jZNLP6^P0og@qBk*ucyYd$LU~qC-N<9 zeAl*AMGj39W%2nRejmjHCJmoZ9MNMlSqL$X(k?QuToHqW+#+Wh_Y%xP0|%f5P%?kh zBcSGBATJ5$TCCuK}%x15Zr{Im-!sU zkWG=VW=KJNread&Ly7TOL*Kwj00PF1zcWDW1M|w?RA|b&9-#s&TX~I4#rdu{IDM25 zsFQ0lt1u$=E;G$QMcXEik0HI0KlK8V%=6nr(Bs=qI)`zRfk)qYpSNKJWn3VW!LGXPq~OtUA@ElevTM1I-hw#D&e4GWRROgZ577P>;^U&iTc$#|ik zY&V4H(LTdgJ19z?b$w$UhqId=+_=FSQt=7s8KRgkD5mHl5Bo5MRzx>3IOcMB43s?7 z0EJ-{VIP3I!8x+FIdMEHW7l#T!&c+_Ar;Y!fRSnBLI$gZCmww!0l^v<42lr-4 zn^+`|wap18yA+4PsIm6VLMz?;>2%t7yqz{b|1PbJ6C%V>j!-sw&kA0K$vo?$Q4B`m zfWUOXGqxJwEp=Q=|BH(|20i=9M#=2;DgWYkrpsPT14ackOBer z7{<)7c@v$`LYFKq-{ka@vGmQWmuZG@@#GYSkv-F3U&)}%FkdwQfM-<;MP;~UuLwY; zmz|N+@LX$;$D&n;WI!ERCis|MKmV5M*mBwbU+yI1Y+^uWT4~Q2O0)ZH&E4wK-ZQt+%eSLY>$hUK`u`)}i%Ne#uNl^&-H- z{pTqHrU+#DV@RYj!1|ba(W2_r4X?d(&-pX%?aHV(vAe?J$^a~+x#E>Or6;V8pu{2CJ>wOYhDXb`1E?I^7>3%ZsE60A9)3X?*U9Z2k? zYD8TG<^$oBQE)+}e|v0rmGKy0M;6nK9Sg{H4ObYRE-=q-=-AY3PD66j7?i6HVL9d~ zLb^T^MHvIC$8vzFu7@(CEbAEi0-u>eIp%G4%F{qZ;|(AH{1;Vs7)`6$@}P4W13|2t zNyA}lm^}lKD8V6x3dO~xRGnJ_i~*9_=rvw%!Cbv2f&FZ6Gwr^3 zgu&bkk5y2{g=zG8jh76i4PD@->QL<^6|bSbA`i!7sH*hv6S&#WOt#6*tk-A`&?Q9= zu5TyyaZ*T^2C;0&9&dDwT?bos zD5Hk%RAMdG%8I%UV~ff&HjecE^T%H+74aoY$VVwQfTdw0LVkoXRuTtL4IvAYp`5$?Wme+qjY)lVB)Mo zHfK_gsmOedBDaEG4X^U=URBwJ=A|or45xm*&%_f{Ko6l3_84zzmd6l$m`T}G3-!}TTYCF$!YKSkLGv+|X(1ki@o~&Ni zQ?I}{`I>-yGFUxCTiwI_Y=N!~~^J#5iB0c{4aoT=-nf5serv#Nd#oq0Z+XlXh3Vzcff~Bt40kAb3{GEsv zG}}0VumVdQ<90zq=PAIc$=D5;342(|NGb&CQ46PzX_PFu^EtO+w+>+qFUQbByk{D4 zI|AdD>BBN)61?9o2+X&&9{-IMgQfAjM+ z*4|`oWnl!HC73kNU#9g3I%O37f4ciBEpfhWj*#>uI{K&o@qbQxg=qloSUPz5965z8 zgrS?HGP}=sRSZ|K$?7@x2BN{*QS<@pcm^oEIH%CSP0IeXFyU>16k--^ijH=hP^v&s z?%X1W2rsf%|6iuw>__(V%rln5J6Y zr@HF*dw=iyEZ=8WA?u9`pac)t(APea@}v!iZGbFL2+t2XQoUs{OnAdWS9U{~0xXq3 zYJk0<5|nUdB=kd@u1(KQI-dT@V;jgy#S-;eoaFiS3yTS$BaU7=VxYxbnbnOvWWv+ zwdq=}QH#k!kjOD)pS`bZ(Wwx||LHdSYMU2jBt{}XUAJjW8mb!Go>7FBk<|juVG5x7 zpt%-2r`jN&4?)HONSyY964jbK?87K9&Fst|Tnz^b;#3lcr6ZmX#LdNxt#^qJ^oVfO zsh;B+j`HE7DFeYdiiGJbacv1HfDqS?nPd=&NYc?U;$)SH3SA;?I8RXl&u0vost{}_ zsYTeLf~YV+Ta%@gRDv<)Q1nAEY94U2yZMSi#`GeCDYE3MaVqf)=wk?g!)JXFuV#ZI ze$Ey;QJ@PZWh4k3fvZ(y7^xRx7K{jH^}ZR~*Nrt!1?H%E5-0&HiyZuG2v@hQ6zKCc zP$+(xpMx0~M5Owd;PAbEo?Iy&FBnFzMV7tp;W?0NxQ@9eC+;3!Hu35J8awtJ;yrXYWzoqCS84psduj3Jn}nOZEnf<}RP6o@_mW|Q1H6gEs`0=__251JvW035v3VXwz=^gisZNqW_Fwn~eDY8E1cG8@4 zim)aQJ9m(4AbXIKhDk@@>pq^{u#=6(O)W7@WGKk}FAF%>TdVX&07yf_gy+~BxD#xN z#{{_}FWYOg2JbZ3FTJkH7em5=-zxMwj8dBdSj^3gM5xkYqXv0f`=lc5?wz1G@B(1~ zADqbD7hRF!Ha*{}%P!5q2TXOy_9Oc!0^_QL(0kIxIo?$C$`zozg}3xg z^_Lrcj1hWZ8Cq4bv7tvC|) zN*ETLG%_lGg`s=q6cgH&P^~ZT%?|gbMH&Gc{>;xWrQ9qzjc4#H>kzao-nof^0H9a# zLc8|jj4&X*0rjXzAqVv1T6^}Kzw1pjtV#G0dd?x2^u+3=*I3m75sm^rr(VIfK!_-3 zyuZZ&T{)2o>d&FodQY2awlIX1q z4-n6+eiIOeyx{uYFV+cdhIh$}@OsEMuCF&Ogd8sms_Y3U=n>5L<#qIGX(ZcY20< z#pid8iXTIf0xF9syPy?4%JX^#g2XVcxVxYizv&?Mifd8mqA?JA4A?!# zLE(BWT#9o>9x1zRqoUC(op|PT?von>gFvUI&*aJ^Wt$VDWiMgtDY6M2)ByS%Lav>pb#9ys|(nT);e=cP2UKMWCLubX*oRdgf$JPw;z5-iJMB~h%K&A2!K_#_40Yz zA@^#Kgh@N4Ta7AGmB?xwzH{fK&j_t@O%7ngqa=!2|u(ZYt!nmTe}u#0Y=f**3+P}vV;CGz(ByO$J3dK zdH_986=W4pg7Y(LP4aK>>^Q5?mKLXUPB#x|f_a{LFrt#hwHC)w3|HwnMI&v(kNu<5 zX^?SPQ*$eXMn}@M8+Vu!TBI50PTGW+mTuibxILw_`vD<3V(y1K>BYytlM9rUfCkPD zf?`5kBpMe{N~TR;60SKVEYQt~#8W^&kUO{Eq%rPFy0*-0Lc%LMTQ8V3xRJhk@)Z^! zV1s9|ZNwnYU{gyQX^s-^DnOtyfE$`~25Xz=w$fG)(#-rqdib69)5(Adjg|Li9u zV?ROxSb7Zu@E$`Hm(4wBMQ#HOYSHmM!TCs|4LmYmf5lV=|6^aJJKt*(vtA#cmChQ$ zGP!S+2YxG2I8sIko#G`v!K>wYD)^*9h!JZH<;%WB3IQ}M z$W_{DouL|H4`4}zJe0-fOg|8i_$_SElPx$BH25X;`gxQ61#mvAe^YrV{G@NgJfjpU z*?J_r!F=BBEdvk)`HG}yn}A?JYN5cXTm+o#ldqFQ36D?Ar-hZ-G)>dA%_pBBH|&Gw zY*?-jO*#H#__VR$!AY?50x#@St!VD% z%8f~0vkecu$jFJbN2L48O#t8KLgJ~zTVVxhFW&hB`iwn0#Ii@;jl$D^@4dH}MQdL& zWCzc7cyydrx2Pu9U;*GcX~4bjKp0L5aC*WiPW(-Bz-9o8Ko@2|GX;Y)brOtyI%~+F z7$eBvPWaozL?M0ut()YTF*}spzB038_tsyfPyX{y)4`KHiaW?L0tD^gaXKIzIf#6I zxUiIJRJBjC7d?dWo9K&U+K61Rp1npc-MaOVdm}Sxu3qE34@-yXt-0|uh3D*oy*E`t z<4r|5fXuV!TM-%b>Y&VARy<)SY8lM1-onBILfAQSX1CHLfT4j$68}@m;IZxi!X_|w zo$b3{ZKaWgJ7GLbE-ho6vL^tj;aT!_d&kn}PoAcK{U;x#SBI;F@k@;V+rm>elL|8w zR~*o&93y{))Sq6A>neuQ`)|FIKHD#0B+S5noH5?r0ru~ZFdT=`_1tmpc?{n&wAuq; zGX-Vk#yy%`)7I$GBRncOGLLgHqil=*vRGLll8E z@DwUzID&Y;=i>cMxe%nOZ;>9{6Z+@(=*Ga`z4rxSPeEmThLZ`v#4`xA%r z$nS$*&M{1)YV%uz(0dE21cK&-2|5LS7T3rNz3g#-AJ!MGSonJp`Zm{4&!o{imKJ(b zXOq_vAiV~^-nYBwM4resRnCN0(&yn>A>;V={qC3hucO5Vy)jjT`vpAbtOR!IYbT}j zW1#COkvF*ndyMCiJ?)-d7XbQx!EG)_2Lt}*Ifz}9zIw(3Z1U5#X#<4``lRlnhXKcY zCS(P4CI9Q4>_B&>?-)62QZZ}3w?NVS^6A^;8zTJm%;oI1Z=V)j{@fYYEMW{iJv^x! zDH6^TDjCJfSy`M<>#x=Udfg;v-W9!&8g$23LHJ+_NWCidS<2$|Ao2n@0hn+|qH)Xn z{fP#_0w;$+P{+n{1b7hGT#$Jkj~CQNQvPk$hBAYAmKGP&um0=L(z|bcC+)7UruAn} z@C+~=;;Y|~%ulF+`;2iNnMQ&Y2$i`Pv=}aP`%W2p4Q~X%te1mT8Cr>^^)!Ie5s(#7 zPR6_?Vl5eAU;GXes;H|V^b44C_t6(TgC0JZ38L|LE6BvL&RJ!|a(E7hAe2#vTK`3s z@Ug__eWTY5uj{fktf%$C(u0va9IYDXfkqQW4~h#rr_!Y69ppiWbGx9+dWlWDyucMr z;lk|0n`r>ha7b^l4!!#hp|3Ka>yy9!RkZ(*h%WFFKKb}3Y3=LJ5MWr92opVmL!J@g zQ;95ctN=M$It`euRpF%hcOFG0!S~~lnVDUru`{hgSjXDSrzB4Q1c8Hx1@K)UOfZcg zwh*C@_Zs>%3^@c%nDf*R5Rx%$2lsl1UDdR5ct4Cbjq;(elA_=Ogg51rRVaG-;)p$^MfxS#~CIgd}-ZJy;_MEC;%~k0$;{!3WeB9%==Z z#3ns)(ODX#dR^ye8rq+sHB2A$b`JP5bjZ6nTL2yZ8B){IvoDv{Y8D{&TC$z+T$c!4 z9-m?oE0@_{_NYMEPdbHS@JgVRIQxr%k+ks6o%AO^{QFcX-(+7nQ#{}K(pviUZ+}Zs z0RW0Uu_(n!6{JiN%Iidxu2$&RoLTJqx6h=|Q=pqS8a@RCe4BoEy>$IA-hMZ2(!{k6 zjh0cYND``SPYIblXNt&H$`N9lAzZmkp42k=O}9CNe(3QE!EE0{dAUxtyWq79sMNaF zNIP%Pyp?COrHh3ddSjg%s+T~ibb+^6o-LyMM#g9v0{@=^e1^$?ymM`y@EyVyc~-_N zRc2~JsYO*S^k{W#cy0u4@TjmFt`4v1E2>J(`F-=+A_iV>TBh2(LWu9#H{0-s4WLbA zCY-~m4J$iHl!g)`!AXfGN-L-1QA0O8{3IP$9Q@&O9=% zgqNwKw+Sy}2RUA(kKD@gB!=W_@Ieb%61oZf)55q)mBVx&@8;z(t+T{cL(}O3&{x43 zFgSy^WvYLfmra!TNs1w<$}{IVm1_ZEnGMZfhX^&t zj9&AR5Cs68t_RM0Hu;-ta-Bxn1V)#vcMt+y; z%QJU9Zs-RLU}ue+QznoTs45c-nfg3>s{}Xv64&rqUIQra>z;RkCi&iLlhtGQs zJgZJ4qe6x_he;)~>NHw@q3e~+&TMUvr`ECK1Stg>Jc;2kW0S=hA-lM?q4rLGB<*2v z<=F=#DSGt$3ed)$N#EUdqbC4jPR@xcR@Q^4WlhinJYw4=_f@@RQ*8$gWZ+4|nXcXH zdV#)EuX-t*$+ci6bH1E67fErPUY)R4RQ$17hBghiI3!!)3LJ!hQHI`@hjbQaG`-~5 zc@~%pP=k_I`LPtX-0Cwta|o{xV+hM(Ebl5k-@45@QIO_m09H(1gagt5l5%`uL)2z?`R<+0TFIc@UaaX5cy49 z%VGmtSeSt57%vE=(Va#}J1K*s8cbnBP!s4JJAvm#oN` zs`cGNdNVCiWk~mBTNpr0lm*dTnNdIpk1Ts<&ff;z#*dDv%$lBu@tCj3YnK4*#_BN> zGj;<^X?x62(OMKp5&F>WZ=$VXI;aGm0Dv+EOP9H>Tb)1V3@9LM^>I#tMVSQh>cva9 zIeJJfnyUign;!>sL+FvWDCpt&dl0TNgoDJbTGZ0fF(j<0!t8V8_gVe;^@|s&!g@6T zY;z$~jtT_%oYs;a2uC3~%T_4TB0PZU7Rl@VctvDXJ)c@;Q_xf7VX^*5Ugf#s9JqH3 zc!pl|G|Si$uIs@*FRpwuN|VWxGxIJaId)&8FMqa*EnA72z^5%=EIolG)BImo-iB!N=RcgYZUO?cvRq< zgHauQl*s}nSWkw1ZQz-~5e`oQBH0HOzPVsph!*>dN$bJfkA4iW0>vIqT*=e-%MmnrLR3>z*V6dXefr_CZusbo0qd8C z`_M!=tv&vnT%~oq5bQ0_Hi29*uhDCuxl!WK3ClvyamLUD&%#Z44)&ZiT_Rh;Gs+rC zoMdf;9r?h)ldm~X1Rp|PP?BMQart++~D*qM6tl3Fpl?RXAJ-;h0$Ob<)|0f zb{6Ksva8;WeI@^tdt3;Dppz1c!ZYrLx%mhaUzADkBy?xk*zs?ZdCDP?d3AuyDPFm~ zJ-jaL>$&mD;06>ezh*Q{m6vTt;V{b=P-Swx-pyIk3-tUK9o5D298mnD$YN0#5QNT< zQN!fANw3nLGp1$D5}vUW9;plsTj*emh$7c1!=Ft&)m1!vle3dR+j^R!N$@qd(^Sn8X8+T>b20o(XRQsTk9QvNOo!0y4#MN{&uy@;q(ii9q0rYaEprGe~aM z3p~*M$Sr5>9`XCg^i3*QN7BZZA5(NyqL_+MG0ns)fb~9}r2;q_WKa755p`&FlxEb7 zV+p_&1q?(Q!t;Yi^sh^HpbeU0p_zA)96*Y7zFT(GScIcicI5huCzQY2pI#63nVJ&6Mrgu z1VjA7C!kVkg4bm*BTjlqBTB_Hxqju%2`NerQ8)L^5U+{$8uP)QaTRld>(O`d3J zY2mmry+i;SqJsZ(je5O8#<~J&A7Zuu&-)5utI)TVvaav-|EqUvr|>;=qCE9_ubbuz zbR2;tXw(=&1OlRvbL<0Zi?#8b$Sh^Ig@qAOU{Bq<;5qIh7z)oLUSFLLuE%ddxcTZ4 z*5o}tpKJFTf<%pX*D1(#-^IiYbW*0s9+2)5@8$U-V&gfoZUJqZNPu~30!JDm-*LiWLfSCp*TTz zbs9TiI4Two+!@_LJh5u`xV9Gsy~ z1i)Z-^iqcf_rRD_lglK_BUtgG_?aQVR2BF9DK7Zh?t0oh*<(!CMmpp3zOTSeB4@G8 zNPy&luxYTYyN4G9qi|tEB^pW0{3XsDVP+yVYly#TwV_DyD4IxUQhOaYeNDis#R0R} z!P-OZLs&Nv<~$(1LsL~O&j=U8v^n;u2nehpWc=#Ivq(&dX#)ZT6om19 zP9ux*j?rTL?%Y26PucfwhgJhs#@m<-eT9G@A`Ga&93&s759_W&9$W9!F?#{@;@*>U z3z3)9gvs}|x6|&kU*V43NOcri8+U5~u+X8xa)e@qQO@nv+G9dW*>r0JFCL8F!+rWV zEJLMQAXdM6hI$Z9YV}~P!fH|pq*W#1DHLKJA3#9xqAT6%FafVUFa)t) z4+dk)j0B(Aq>gd9lM)P zgn57>Xn~LhxF;8gbz9irwHqi*i8RiS|1&%&&GvH^-pljY2Z+N~c%q~d-@_S6XVNjB zRA|WPHu03U^hmo;Y#QKE0|gqF?)k67IXll4GK6=y8lUOOkdMgqGbAc4&2k(O6fdu*x;PzH>)yyE4M(rR5{?3Y+YaLNW@05Dsnj*>fM`KHw}q zLmm|hua8E}BmEQ>j4h-$nN4iy?}X&}$G`prPZm|`%<%K91+3BT1O?v9GZqnudCt&2 zf@B{e$mK$(8~|LPVU0e8Ws^O%g3jY3LWUx35+=oaA)O)g&#*!rThqkzd`?JCAujlX zJ!w+`{ZZBKG=O=g4vZ6)7$e+2!yBgYo(v+ZI)ZM%E>wliR8c^mafB#vj1lu^Pif4n zhp|}f2ZU%8DVK;<@?0?lTsp?1UZbVN;SOuy{9OAv-Zc$E^K%7SDq}&K3C+p~4I-Zd zIU53e8enc(g*2&xl!nX|mZtQo`u-RZMYXKT$;d?pKHH+n^%47CCY_~&Y<3isg;4gV z)RS_OcUrzC?^CYA% z5%$$PI*6ROKLh=uAU6q-9w4*7`<;jBaCe*U;{d%`JU^pMPT507Z|prvm6z*jif1{2 z=3XL)Mmeu(^8W^?7!|ZwL{TFJqyYeZ=@=`FtUCE=s9~%=iZTF90`YO^e*$?`!}FEn zGkIt&lgC>=H+r!AKJ=R0f)&Tj2IQ!B(d-vqocDjH222&{Z10c2MmY@N3 zAZZf3De{y6OD4Q`S`=GwM%`*r&KZ4KfaU}t4R}t&%W$W;pGH%z5OjvXJTK*l^se5E zd?$4pdN@2V{M{(UAmjSlxCo_J4Gkrdx=no|@Y3*+h73^$Eph)G;qSpC>fT_=C!Ug$aK(Juj!YZe@77{_~fT@ST_(zUj6GDLA6 z&n0Lzk6WS-V8}k`X>$F3DR=$s5T5H6d+E)?NLH>YNBO*t7jwxpfDHxPi%!p*pf^Ao zXC5Gq^|-$9s6lUjR%2DKmuX=d9zKrGYcazGa;ZP|(v>JPkq_KA`p5a$yrwKu2e{Wh z%hBOpIxE@87pK5HAewgxVtudoQh&xXA-naE@p7y!_AmC*XXaBr9$#KZTFK>#;c4?V z!_IZ|nrGm7x_^f9{ZI>sRKJcVaShi~@4AL)NrL>KFp?x;EY~6~YG?9kshk7i$Y@6p zIgpvSH_G3Q=Krz&0TC=zm!Gp3^%NO)((~c9JV-4y0S_97Djq7zs0wA~Qz(pKa)$QS znfJJ}AIb4&5WJ(01Vd1KbRZvspmP-(tCEM&@c|HMIbyNWV1b8`Nd!_EgrUqZfPff4 z^BX!+fEu2-WNn7nrt$8$nJR7;7tDEjd=S?#oZ81B>+v#ts=^3i>a+3POCqifehvFt zfh#y*A5l71^hr1XIovB(2z+6!{e!p&!jwEOtiuT`Gv}JZxiOP(`{*zo)4xr?U`rN- z`_j_&bnmS{C5$qcdPvl_QD_zSs2&s44ONQ7z&9t@lEdBVe{t|Bo9rp2R zn4~=j!4IJ*$JwABS{Jk~slEjeb{Ys?lA`~?2X6%!-dTS|ixkqqA$tCY@4 z6cb(sUEyq4C_1a1xg=+R6Oq?dX?kp3hnCeOiP4jIew-7j;JGAWytKC)9=>7D(eMDQ z9E>!C604)YVtNMmwVL@7K%)0X(BLyfr4PbMFIAmusetu(aS>?QybqWa#bN%6;9mwY zXUz`9qdzr@1$WtQ} zoW;-!VQu)GoTmc&QwPLbxjG1Q$Vc{;16oJ;Zvb8~gg1cq$C;!GU?-0p<1KPzUk2Je zWqa@cAZ z9(yDko1Wpi^o*1UZ&pZ>7nFOxaCn4JCm22T;NeP|oKX29!w}$g=5tnP{(4ESognQ9 z;8^7hoOE$XxLkRx!KtU&0t7u_3I-K|{Rs8!o}$DB*Ap0Rj_NZkqJXauI7usraC8j@|JGnHPATHr+}z>scw9l%Ruhx+K(5{};D~)^Z|l$tM8S;T2kF22 zi~m0T?B^e+kN@v~o7#BWui=K;WAqolx0r!SuE1IYW*JM?)QAjj|uqjLa2 zQ$Y4?t?jelL_gkqKi#6O&;j16Gxki-(*tkj35T7r_MNYOmo6z@nnl({c)D?#-X|QK zhwsW%o}MF<&XCz#CA{6V2D!x0tB``zN0@2{Pimg1iTiPhXXfTK{pBz!`!7n!>YL=B zVC<4By0O2vNqUg>8?Vy9@`H3_5gWOH5O~m$9+NBRu?MU^MYrZc zlTm{EM*w~LT#^W$ZD15}Jwv_wWC^H0NSvmx?2<1Diriu4RS~ri}%56c6#cx&k zMlYw`46HoP;&q4A$uCrgU|d)V|5C&7C$uoR;NESB)B*Rog{3(N8#gVk_A*snjycjzsW9>> zT%?r~K2Yoh0)ykgqL=ElCVT^jfuOxa(AXBCH&22n7M6`^UFZOX2Nx_Xx~22!``2%! zwd38CrC-)1$+V8h8D?{GSknEvX*mpY-=M+}YJD;Z>ydcHrFz25(UTCZj5Pq?v zgi#uwL&?|}(*}}tnsX{RZFsBKE@$WvZ-eefJz&i<4uro&UKPsFJT2XoJ$Ml(Q9M-+ z)g*KCHw0aAm{`)o5a%!orJwH|@<;@XHHdq1bS}*@NdAy8#cv*e1HmIC0W7oEZl{6U z_tKlwQ_=0+O1V7XL#xc3uT2QW=QJnAQs>0jqUs4fmdK%ciomP@4)kQ$(b?w5TF=d1 z7zahB;B-`wjeVn9muI`El@NG)sfF?@GK}6^wtrO(He#`H0Qy>19Nf= z-^v6bJ^+w*q_?Nk2+TS@LBmk;zDx{0XHSm_%h?Fo`)Q$O48d{_0YsFY^Vyn7=d9h( zJGK-C2aEzeB3iIw0e1+ceO%x&NK|@q|LEBlbmTe9s0c;Kxu0-fcPyMl4N!;11#)^4 zwSWvWvR!}}gbBa5*q;^(WbxXa^xzMFl&*jHQTkuM_oH-<0+LRKS#P1|B0c-vr|H?} zzfPY}g~6>0hSZ-L14Z zF_Z3Ize#J3uPFffCD(Sm*dapq3=n7A7jo`K$$c7~B}BCt$=&~)Wbn^^_W!5V7hl4Y zC;{@Dq7oLgXO5j-zye+#wlRd!FklI39_=GW0QXjd9s{gu@&UAADxohNBtm5aD?!le7gqTe#1R2B_fb^yC6_r$gp@nz%nfpRsgbo-j!_Z6~y9>J@nUBgprW%C(RH0tbi6~)^JFmP`KZ?07$ zF`{73oMV8bd#Aogn-TWqn&G9ECIFIps2B0BYRKwUJ0-8m=DmitA+s=WB1{6kIS(FpSASY5F(+`~Q|o z8>^%b(7$b(wnG#_Ra@os-h*3M_g92`Sp$G!Vv0~FXE}zR& z1c394$6us(@kXs+bexlWc;>nQ0WY?8*gNJ&BM)s-JU~eE<}@K;iVSw~%;v`z(=%p^ zJw=x^X!Vh&k6JWMhDQ4U_NjL>P0<$RaQkIiB*Z({DboUl)*ii7*dBh9GNo0tFIBtf zfhvIO5?OmSv5>B2ZZRT|qJw@6l%WYC(-vt6C~9~U*%Rcz?C3OwEAS90BbO8}L@!D9 zjzD<;=K=DntK_kqw+OY5&7{HQn}m(wEsVVm&!=~IWO_DTgQnLu2(=O^I<7cxbCl>5 zpaI_WOV5iDszDuGY{rp#_5U%^|EQD{a6{M}qytKkjxhZhUkuHm8{57}kN z&KzuHl3$9<@bdGS_a6T`d8Zoll$ z=H7U!U~NUST1yj6er)!o;w7cLl+*L;3ocVBG{y4RvTJ(Ib$)fa*MfL1D@N#1e zyEGuQI#$UQLN*LS7(gtShtZ0l;llW2nweNeNwZ)WutCyjWtWP_(q1YF%5XntQ5XeK zCpq{W6yHHa;x!Z`LQH{f$cw!QOVv$sQ+0`|DHI@0nAc6W8YoZxL$uI3M z0t!S0n1OI@ZIWSQ^|*8(8UbN}>kRT796M{kTIZnjirJ3AyrLY$nul51c=#@1>Kvf7 z&E}sXym}}{Z_#^cC_l?<08QKgVnCh0*XM*|zb=t`bi9>@XJ*s&AN*t7>J&2&vdv-P z9+Uskz}je2K97SJFXSGenK4;-Km?Xm6uf}#cy}H5Ic-n?R93YKdgir)d0@#ms};|` z1TltdMg1K%>2>n7dv(RL7WjM~;`bRPX#R<(6%YjBSfMK!+X}%@k_q{26!&7J5QpM5 zvAH1Lc9D0l$jjMc_x3|$GVIw(%mX@s3* zKF_dx+_i$cpW8l7Dvzo$S~KHIf${v`iaHn&n^?KxqrpQbDcyE(35 zH-2fZgJ6Lq(?0oAHR!b`N8&4ctEE*1G}yzeA7I=+Oh}9N7l(kdDn$cB@YMoBy9UVZ z;Q8&FKH>c&^cx<)TL1mVIzpf7N<3!EcV|_lKE!r3Z3Y8Ep zyl2D~8tlKtE6NDloj3`@+*6~=R;23P9wI#J_^2UsKnYn+jM7M)jj?BBM!x1A{Q%%h zgbc|E1(aRksmM|dVNRy9d>A^OVD_sGYwbGUplHWj7yGB>06UUzFGeLIFaTPULGI^- z5f4e%P&r1Wmox#r3+`<6Jov4Lq7S3MTs6U)RbUEX_5Rit- z5XRs1Fnopg=b(I%cJNdUFW$#Ud`va@80%(g2_e+Gw{PJEfZqstmNj5G;S;=y0(ObxxEcs4Uu+1qSy6H&rnyP!!!KN0=q6qVr|^ts$Iy2 z`=B5Ayy+Rf;OG6;f#6T|ot|C8zkbfJXz#a?HATH=s<#h{xD&%kaH$%4ef2)`2Lx>C&<5(=9GbZkg zp7GuqI7WH)H+C>iC>BB2w2=vjy#P(|LD$Fy0)Vm>^A7XK5MNV&3C7d~Vi_(TdQY#Q z{OL3MH=oHb_sA%aa--Y#Qv;q5GkmKq5LA|lRGLQAkD-Go;9I8>7y*#$_&$Ms0BN34 zfa*QHuAs$za9#0t&dDwJ-+%WjiZA&?yo^3m-b7&tFBoO;?Cpm->~kZRxu_FbxG;oa z=W|G%4^Zu+i=T(?gKnLH;< zLCLt*@Q~sv7BFXM?$=2Y5YA>Zb81mU9csDu)tv==K6(iiL4dEkMDz{Tb*!5-{_uK?{JyZfznM$TM^lDzr1^cN7&B71& zR;xl!gM}Rt4oA`G4X`hny`>s}MTylfL&#{FxTdWRyq8J3eRxz2jSfKg=BL>>)d>{} z0pt04bPa()7_=POrb#GykUWdIaowIu02HogXw*bjlek@5&?*nFAgj=q&hRYk8D}U% zsa&I~i0AB9GJ3uW0^7FWsbU>kz=B%^kt2}y*6{>Gi;mJ7zV~jLx`w5Xx1<;AQ4d~J zw*m}mcm%ruZY%&?j_`b*+&l#t;tS(|-eNlk5nk^hXXj{bEmfaCO1U~cgb2GRWPr^8 zRg?$&8$t>tvR}s1B3yOt=7aEXnhH<`K$?zl{oXq%!{4xj17x+?ACtNDqK*@$D6U*f zM=!s|drsK_UKztV!+^d)Xm5vR+Ev;pj8HMTh{D##rA00+JMY(uW`&BcJ0n%z%L^K) z+*C020HXS!zbwz^S%m?`yRFo85PT7EG=_plR|PL?Oz1hZeTF%J7w}dCx-mDY0=+p; zR1l?yc?QpmEj#A$5{>Y45Se0+Lv9O8&<~AP2llr^gkUs4y3mY@w8?$*H1F&kG7kj9 z5*o9TR+;7O$^!iYhuB;5*$|b{L$HMKFIV(5!b|K^oj9$}&ZFaXhtBc#kAU**PY3%D z8bM$KQo6JTXQ`BU?g*7)`0y*w1~6p50#NdAC&dgixx}qH1ngW2!uU^KGMzwADg+nB zA{{&{6|N(w6$<3^O7NcOFUkA7?ga8<2&2pXZq8H2TEW;Mua8jTC~~63_$<9J&QIM~ zeL~A1#&X@BO%Ly^q<8Kvrqw4*Dd3h>CKkMSc?i(P8w?olW&dmeC2t)Pc{phz7fIpR zAsjP8VOf#$)B~97CnWpU_tLXRzfU`aGV2tnSa6fUAU0Z3!P7D`I+&K{0IN)U$Z+kx z(S`KW&02bPHJrvD{wTFZ=XrL{jADQu?l%SvCei@%$ikTnAz&)&CRl$K;a*_x&xk^_ z=?PcC_?ftMGad2l%QxPnk0{`qQC>(q00*8=(p8#(5C?e|$$2~nRNCy?etrG$=^Q=c zh8d5gGBlOOYVm12*c}XH>FA8eNC{qEoS#YikH1VAnw0l7O6lR^T)G^m_vGuE(H02p{j!n}E5W!`XT^i6v zE);_TAHl)j&EwKDEWpy>e(h1~dcBm7fP5LlC2@U2gHD0)O9zMq>U+B@#6vDGK-%8l zmQ{i=-(!-zLdf(HJ{Vw(7m#QE-S>CBSyuK(|25ug^0 z*YS7X=W|=-X`Z&Zq%A(6Jd^VRM8@CcN6y7(2Wz;;Mm=0hy8|9@{aoLWrLx8y3~=Y# zc zYz6@4bAoDaa-d;=f zgS~ht^RiTOgMhW^`Q>zpH|c;LVu$2(9qeoZQp`QUQb6eUa^R>vlI%gL2%df(Fp~Wc zYC-%QvdSzh+t^UUQs#Gb1)4+`N%y37yJZd+br^tehsvFIir>ohHcL+JF8uee>7< zfdt6)sK~lPI9-yMUPtf>tT^AuUlVW|$`}U>jF2lKM_*lF0i$G0`d9e((yTa39)K04 zR$V%19NLSJXaOM)#qj*RfDk~&Kx`;L`o#%ayWTdy3VZIp*__q(DmFXZ*ac)m?0W7D z$Be@0V|aJmZ^NB?G~YFEP2b8fOk{#RMO)-2sg$@MXJb=Tui;!ulhWtM#|UH5F9_k< zL=l)TcaB9WHFoeUnWJb}D38a*?&*fmbb%k>4H2wK+zXC}_tj>r;cjL{^`H?UjP?0%*7(jFpF{j z<#T%2eU_g7{Kq6%<3;AV1jl-8oJahvq>18@L5^aWy|=g36+GC&+qR1mlHLdLa7+xs zhn!Q1oURH6m7t$P4jz<_GOP;*2}K~lvM8yKeNu4f0kVSBP=e#S2B5Pd>mTREBA3bY zbA+5r!-=-(F}#rhV4RYeZofN=gv=K#;$@k|Q#MWL?Ak2th-Sv1NyaCUY<@<#u8Isg zsaOh)J=>SRhGsQVVzj#7fk48$J95IuXEs79u?bBLgom|6H z3oVpuc*VnLfsZ>DytMI7vPYMIR6X0nc>jw06r!;SL>i1k==H*&L)n#ih7NgRwqH;_ z3GCdzHc9$r6tmtnwg+&?oJ|y3cxBkTuABpqDxg{GIkc@(*9$2KI0IZq0;Zy!-?&`y8x$M@48 zee`GPWNkbBm;diSGT!qf&2mls;=`QD5?;FwIdmFBHb>VRb%r9i=q1crL@D919jcPM z$Ua^n1Lhw7LHfz#)wK0|D~*$MWFdv2z3b#lwgH?oc&(kEo1u|&7JlDo4>4*Bxi|YT zExma=9TPrX{p}~|gCG7Qe%DxS_!6RU zL7vGXG@JrlNAJ4t)yt%gP@f2{q+@^D5~zk8=>je80WJ7`J%Tbq@GtN6%ZUYpOzSXB zCwf(~SUpPIil`XgYQpz1`b!-LR?xL(@5ji=LoW(eOt(0th(NEY1(t)*fV}}jUhy1V zpv?2c9-}V>bi6zE(qFtbXJ=s^%jX#co9v^=$mimB4RFDv=B1w#ko(+v&Z5l|>-YJF zi4;vuPp8u1KA)io4WlUlEf~W*@D9xg4LA3g%7bo+J@OiYIk1mwj)!Mio}V7t0BXoQ z0la%0w87tfpALp|CMR`^OgJCpZ}%(!{ye+IkPRFKx!Z655y|#+^yUQ{G45=xr?0l3 zqHyt=A>gcofu@|Gbe=bCc?khH(&)yvP~x zM=UR`e;$y}9UPP7E^$GyHbu4D{SeAa0bdA|BNy`M-(Mu9#5^cQJs;~DE5;2_Nx8s~t{($7OpbbqA+cmfYWIV=^aPvB^1P!C%W8T&QQ zOwxi%1f^kL8i9(r7Y+K>=pn*}2*ult!Ak&OCJ5#L0x-YL(5o(8JHX4R6UAb>GfyiB zynGV4)wY{Q2WbQ1jQl|est#eFQDx{v2fdOuL?6K8y0EwsjcIS=srvlmpAo(y`ImL{ zBDl^0v)dcwqyZH5wpBn8unS%h$Jf9_FvB38y+bO*c2LANa2A*iVGsVJZ@3t$^Bcqqkdsk34a~L6p>kMC+;|VGa&s9H6 z+6$dk0e{i7m+)GXoVGvvC;uYlZ#_tl*LN8D4Y1!wK}R()^jw?`56i*c4)lsJ!;6w5 z*G49_Fd|O^>-`Vz1t{+zd>W(+eDI?`O&h-@ObrkgWR}UpYeQFqBs%Nu{NVO2sv3`| zwmwb2`SHJ{d1;A0Rpf&rN>Js5T4id4I?tqFu?RuWW{jeej-kUVz-1^Gu0INGS`mAQ zf(r{);{e(@AarqtFmS)}cwppGxLLwFr3c;k_-KSNxAb@-IP2zc z!1sgDSrdb+O?wWG2vAN=4m4)I2pA69pv}e*%9+#$jZ5X2`H?pEReqc>+`Wvk|?!sI!le+5jDDtjI(D)S#6| zO({?Tst7JQ5B)vMlcOj~j93P)b|LTAK z3nmhbr2qJT{9mcGhUbVp$P)|?0r55VZ3sGT1KzIq?y!KXHh_kSCg;%c73)}7pegvq zI{S+Ekm8LR7018*-4hb~Nswo4`+&hA^5#Bx^KLrX+oPDQNr-j=nLzJ3_kcXh&)3h> z(jWYY?DJj>S_*mASMhKy$A0$#=<(U5A_}^7x{17icksf2YKZ#Six1Bv0H+2Ybtrbr zEZm^Q2nwFwe|uEFwm4hoOrDX`HcM!JYIG)TtZzVnGZ@XcQ;iDUv!fS`4@0RcyWtf_ zSvhVbQG9C5`y=Ze8JP)(8PNqE51jWE^if(wa3VD zK^mf>3lv=j{~=oVoHB)WHPE`iunz;}JgZmH5WmYs?jl=_W(hV-ZPDB0^9U?hAW;@% zMiwCLv$}*B9Tc*8J`~p4amRRlxk)khI+tUgH~|%G@=%G^N>py;Uv|ET-?Hy026*MN5gZ2~ikHjD^dKtEmZi`)fhxxs?(_;RyjA2c3W z1GL~|Bjr&TWBe3Y`DLTxGvu1GH7Z7Fb01(GxoD)uJW>7_=ND6ZzTMMkC&YC$>#uY5a9r>YSapff`eo;Oq?}T z;LWY7Y9XW)UY7{jyL9j#EbP!Ul#mb)(y!n~IH>FdbrQ)M9f-C+jm| z@q$X14g_1<+CreBa7uJKCnPmV1I**yuh^p|z20)^LvoDTyzhwDGBtz}hnPhH-=sfX zXND>fEHY@2>q^vEP7rMY1*5ld1S_|WOSlR{yxQ89qk};>+Lj`M!gdUO{o~ij8i&V!FzJOuuYqkZ^ zl}K#12bR60w1n$a5!Knt0wJNgo{))2fIR)}00>J9%P?_`ByaM|5auoRYWHZLd@QU9 zENcORiO_b_HssWakPS@4+UqhJ*FV7ej^YOY!Fv5s;hOx$ul{E~BI_o9MeEV$DDXZB zTV7Jx4f9;WJ=>u0cwK-8BidBi5Ycryt`||bhQ%v@fn&mGT3HrL^pd~HX{0e`zxE?c zN+_~dRQ0UkA3NK@r9U_uAyRW5Bi{jFP(D7SP0JR_M2i-B!~H#o^=@UEv;H{VwD$Cy zs6Hbn0Z+(UY8@YN7UrWdQDL%}=9h01`Z{K&BjNPNpD}ctepsi6>@|Xhb8n zWAywy$}dTTR*`@Js@W95j0Z^9s@<6v7&J@CbUQ`mkE&t-&V@0gKR}!31`b zbL86Om>CAsbNV{$9i9O8HJaA4$$9eAOrbEZC75L6u9Y`?SWo2 z3Oj&6#qk*)bDM439wtv%?i3oa63k}Jqkts6-%c^;XhcwH%~l}L8E_r~7C_H+rCsM9 zYt>uOVu6q6ftU&pVN7}8T+o97nhWrAfwej2Dk=&&6Ep?J4f};x&EfQp4D@DJyE^mn z%jd{H0EE?_o29d~Lp7cwy%vW1(}#D777&8o`pwVNlONNV1HiR_az7`8s~j#24+8vX z2LeCO;W-nmA|13N(*&M`sYzZ#S?v>o9U)17Rmbl5WZEJuaSrWYRriSuWYhJfMLeeC z>G|WY(lBz`G@Bazum38#&V(vZ9v}PM!G(Sk;wh7+IljeUu37CqsKfide@aYRK;Ho&~uN; z7W6u70ZPwMhJDDlTMyn%r3?De@%Q1`>**p#kpy5j57@7h7NZh1T&g0C3Qh%)#|Ld% zINgoDYQR4Hp{*)idqi4#U7 zbXb^2eGy6X8nSxFO_|V$msYq2bOTGWml}0?#Jc<1!GaZl#CRuQsgx+~x#zC^b$bt! z<^9s@9_#eEpljdL&2fVlFg##=?(gf}XXgif_^9rcX8BKm!bQ0bDLUv^eTuHpZ*`PL zy?yQE5p|pS5gO?Bm(<%6*X1{_ClIyLU*LB^8VDCA9qZdXI@m ze17#st;jW9#o`OW zy&W;64Wkz^N0rxF$T#;1xEj)>F z-WYvQdw@Rg}guCG!E<@U9NB}3t0fBjfOSX!jukX+|jcUg_?)hHI+6#F8+N|4fFc1(uN8Scis72Oy zj913qTQN8ukj5TU-hamXsa!$;*9g~?Hf?`u7!R)z*W#ojTxkwKL5~g0Tw#qq+eM&l z@>&oca|rfJ0V6<7!Dil`Ubcf{T*v%Ai1JZrtI!QeT>(TCB(Ig_C_qA{G*8Z9#ae(F z6#O99wMuXbtHs|ulrU{I@FuWvtZ807p1_}$Q=p)3`(28NcXYr(|9 z8W%i6V}_l*7(xiR%dPPG0)h{*)K?QAblSLkWUg%uC~c_t2WH(biC(z&1mtJOW9hIWP#7X6&B zeG6;QZw<><54uw!3}u@?S-@j7H911KkMn2$iyA8^G3RuildDk%7#ng_aYXw9Xy+U- zEU1|*^8Bn`8QvGd3%*963FN&Ws))w5++87YpkbcW}nLN+=PLG}yv!J(Op zt0Iwa%w=g{Y(+>IprDS5o@<<~GU*gv=|_lW0ZOOHB0<|A@~hCKy$1rB+e2G~VAxw5 zFl)UJPynE{tT$AE76uQ%RfEZJ&=ow|f_^1`_frtrt)^$62;FD!#=GHq^?h=OH+&rNDhQ2W9 znYAe+&A%C^5p9RY(j$bfhpE`)=3Sb=xSB>TIhz)A8dEiO@|i%?yvHQ4Y#fAoVt1++s0@cjrNw~X8wz+1eyyu#Fm>*;LoWy&7COmAQy#rQD{COsab8H@sC z%rPOD%OQ#s`eA{3YeIO2IIlK&oJJGOzY@g7 zcskyu8`P@Hs6K> z`q&}v<>y=v@AM1;yur_0)Aa{%E;!^aF0gw>ZUR?C7;$Z$6+Y54 zYiL<7o@>z{b)UK&5(Kt{tLwI}@jL{8fIQpvySb_dLTrOQE*a{wa58!0q zpFSQ?Pm%r6dZS#N&4xkLfqEB|e#X8Djs>IHo?`Q;L{nS3Wh2mz*sNm&vf&IU9w15b zFdn$U`MLDq?p+eNXK92x$u+GUL6D000Z7k2`)%C63NXrGmF86D0uRwMLYJG9SAiD| z?3HCPR@l@}*0J*G8%Dpi8s9^NdkhszURj9HLH1SwIZ?vOU3-Pu={d{uw;_rcaC}+B z>Nb=)mhQd596}P`cL-S=?roFXaYW-{zyQx2-!PqHhbHiTUSJW*tQ0OI@O3k@C*&at zp7qA5ER_v)_My&xU2=vE`t#VgX^&i!UcGsM@&fyO#j`kk+x|lYlu~hafqbO7Xz^m0 zae^@L-TU{`-~9M58PJSt8@?LARJSMt^9W0t`H@_|RodZqJW7BIee6wMSAckK3NWjY z%w@Zf$+)fxTgK33t(But$qCRDMuHOnmta6f;j#hxb&pF61+HnKMFV6p~s@RvQZRE)n!c7kz2wy+2QxTkq1N zYLuO3w>gszg(LMN@{3+R!7H~Ov(z?Wm(1636J3cm!u>%jmM*_Mb_QT3rlf}mX zyG7`^#Ef2z9~D9$`7XnFnOh@Xs|7$25MIS~vhb0)A!Wn97Beuv6VZc!Mn3S2wgA{G zF}9E2Ez{3Y0iWl(Luds*cSv3ntHfa8SrLdP|8@Zw%0@xV{r8a{3T%PCGUS{|4uY-$ z6igOcvRY6XHN@v92-#`C>IoG{Ic?#RL{Rf+Ez;0KV9rYq`)X>46K5pk4Cf`_JwU&E_}1Cmx7gRF^)N<#c03*o~N`1tR?`v!bTwJ~{d zqj<(FHaIvU_gIf6|LH}vm)i_LaFn(y9oDIlU`VJ1y-tx5VqT5%`hw3mW4LeOdin-{ z{r~>tQ_hM}d_*ed$tN2l?3u???xdUQ0erA`Oy%nw9yeNtVC1iEmlP;rH#kB2EYcOx~SA=6g!goP`30?oHm zsm3^{%q%nuA3-n5A)_6yfsqO|2Gz?M{7(Zq(?l;wtDLW16+P5Lq#wl~c6d|z7g*>S zG-M}_2Ejm2T(}Fe_^FN!uM_K+r!~wB6*}FL8iqjYnH$IR<{)_PA}hkI^; zpM&Zvcv4LH(%BVvVjto^uI)asQtqk5M$fYEf?W5?^HKlgPW1dB5QibO2`pHZrvvUU zsSp_@*|i@qVQ!3Unfx- zixcZ=bbcX?Ei$K-A?oG5qtrqHWT<-Ce)e@b+I+#lSXdsUZKrrWA$r+ZZdj}HD4=1i zzcYp@myUKg06a#No?Fcr%_~{|7=nbPwg3it=WZ=6K`^w9KxwXRZw4WX*lnFrGZE9a z3xKJ820RZ^`u^aJx5+mfPfwnInf~kAi|}F&0|u?ebWhBsk%$M`Kby(E+}I8V^uQVb zG8;J>wmV_n84^+LfwiWxLQtGT;7s7dsw1CmY$`2&??L*LcivBT?%l?+XBiOJ&db$! z7Hx{{)ztvUzy9UNRA1@IX{D#npQPXXr+*)fSjQzS&cikly)b>(s|RVJT^x($R|pjd zw@*ew`H1HX4OnR?u|otf8T%nI@-we;3}WswiOnPh7J58$5VWAcK6Qq$Bz{9lDqP4C z<)^UGDnp2&bhJRm@lL?o=?E|3kquV~ORD7UP!HE|J-XBr9+|uL?VxLv;vO_@s8%bB z@gp#2A&q|L1H1`K0O~aW3(52+yX0{pOt7j{vQhQ7f<<2HrvdXiCV`-x&?*973stMJ zRo_Fg0cZQBuFu(UwvaR@JH3FAcuK0$eW&Asd=cm?o+iwLKv9AH*x-W*f& zK?PZT04@be18;8aiX?css~}rR|St4^~>)qM;X~Mu8D)f!=2cJufaSL>T0Z z3cyV~>cb?5-@AS%6v*nvX4<84QA4C3kMAO_Ca%xVlZG%vD3qK^lg`zy}D=ZrhR~;H}Ja+fPx@d5h6e_o+~9pv1lQF&_5Ft%-^zH+y;W5OVfeJCClLK;b^3e8k)vb*yedrPaT8C=0pLXPaRlG(hi|9noX^@SF9{)@ z0`88P5sc>>ddvbGv&h^dUYnv=9gs%>(nc|7#ijpkuQh$`keI- zKrb?>?-5k!_0rpEZ!TMNbQ|O%qj>!_5U{9e0NRP_Pk33d6MPK63DPwv^bi~BGm_;f z!VEGlfFXP5m%s$><9v`g@UkfruFJw2C=sBdrwyHpurw-W%M658{CtJp zfYoA6VZ=k*;a%gI%z;~b_LK(pD>Q5d^z$r+C@<7&6s-6;SL6Fzc;7U><5}2q^Wp-` zknRvISX>|b>N+_98Ia<(?)!Cq^x}<)LP%t}GS%nI0>CuL{5-o6d>{MDd(<;gknt_> z6jD3)+q5T(^xSI;6&mP@$V=nik+;T`bW8kQFQ)j$`w3E`aLo1bC5V=O)T@ZO*mJ|I z?vw33(zS_O3G<>i)VXX9cjqumXZ6d#x*lvV)PsGJ?a z)8OWMRn8?gKCBYdt*EjTts6qHH3Kp#LNB^4hafN$#tRB)wMM)bp#vb%l5&`C88BF7 zAH9@gW3*r|Y|jBfK&t$kzu~%VAz&t#@22;D_y$6Z5l;x~ReBhi>tWxe5)Fa(Ki>$U z*$1#GnX`wdtp}o*mzW@0555xzj8d9qt|JJ!frWf{c)-lnR{=J2Ft_2rqLvNBHb@Rm z9-=g_OmI30nDzb7)?OhbeSQxO1&Uy|UtELdcJ|*@bgHIr|{}<`&AuO} zyz$PvY32T#Se`c#X7nFJC?4&sGt7C1qGwU z8wt{rX#P?5WCBKae&sQV$W_2%i~W+LAOpw~Vc{8~3?a?PVd!VSdV8#j<+(j9EPH|4 z5YHoHL5~6SZARGPY<=Gq`)gQMg{GIafW<8^lrol=j?>Z}tr;F;Rn=LdRa;tyfKBe-ruWt$3ULgu z(*&rIrW120Pj+6VAAj--l)*9J3X7b*_A`BeJ#zzV(jU)z0pICi)obLqIF;HJwzgfdELiw3LPyN}=n z&XLz=NJPJ}G#%|iYJ}6aj}OxpMJs*$%`^wcDK+48XHhzVp?X;fFSYq@=A|jX@)(6b z_}=%R&r{tS$QQh4N4pqp&9qZ0g(pdwVjiQpePaOPA)gH;=Ah_96eJ7eS_oc*sX3XZ z5m;3|z_pva_MB-jQDw^xkE5ukp_e}L7(NGtmhhUh`J5SNhwwf_fa>$Fa|Rc9`|L|7 zZ*<#8pgZ`R*Xo0p25oo24o={%b~f?==y=*D)vnEaM)+{eN?GL+>#tx)*`{NDlGXxD zz&SoTW~|jvdh5ml^njqoW2D!jNu}r^lQK*_8KqsuEY+$7#{1}XsSsXttxhJn0_>a< zP84hyDlW5MO@595aHjx=bpX_7U%p6toTDN9CBixVl5*VpL9PTaJ4iD}+ffyIf1?vaJ2SRk<2d!Adl!&y2n*~Z?D(=H$~96G@HsXQ~#Fc%K_t^9Yqk$_YH zUnlfCItoyMFVs0T&Q4GxC~M&T)bnMrL4+u+@P+0D+45i*w8$rxD`~}f8cirr^&6jo zz9Ch}z6%EQX#K563_g_(1asZ7n8+~htA~y^1#siBg6H-4nmf+Ny;bNi(=p7wvdOyq z7U58C6FFR5Q-k_%!H5@P9UAgg3h>+m0iIVANE${ikiulXn+CkH#kKV!kL%sCGMzGlK&=qa1IB3eXn|p|0<%$|x?ipZ?^9axiX!Jc6vbY!WtIifda-6;Q zizv+Fze3JMI}#M2;5()>aNP`H934~cxz7?`tVg-)`<+d#XVY#2>RG2eXxw`?0&l;h zNo8!7`NK|{%1OYs2)M-Z=*30I)5qpENo3Z_)k0F)IHo3&bUg^}ZRMYb;Dzn&rn@q{ zHf%&MF4sI(wL*>C$EH{nRL9b+nnQyUthY%RC)|=7nZ-J!3IoA-%7L1^Jb)0&s4%en zW)=wdUAsYd_{r$ml_!75nXEttn6I49)^-_Fgx3v$a7+{2_Bmt05YP$-&ODOMA=O|I zWA#7;z__)A%Pb{Xq$v;b)b1zdOs|?*7DYh=>tF>Ap}=fR{^zHVH83%5imKE zOF`vVRL?ay*bDACI=7f+7nh>=;OuZaJ^tm-3Co?}jhR5Wa?I@4_;5azd4_ZL19AzV zVy;sRfLEx%Tut+AL{2FG5}@}AhFG}wHi_Z$Y3kks%G_5NQ%0y5qO}3=7oYq*oo(%= z(#}>o+<3*#Q#b-ap~*>LXR9HcB@V=~TTQCK#;NKtj6A|Q+ep{K2t#S+;hErl!MAxu zf~fBNM%Kq`yZ{4fc>pN+yZfOyF{l_qi06p%YW^Jn1LBN)FHX$jgbePwZmt`2#x)Gj z8P@DzVJc{>(rclhY@uS4-gBx9FQF$J-ClA=Ll_p*_WQ!CHca)f-VjUHZF93nTaG44 z==t0C(!&QIr4FIG5&+>9e|P5XMV+Yt2Wv1QdqT6&Ucx`U0PZTym(QUI^BMG3kl#%H z79fjU$y2J|a`gFI`shzFI=Vutg*Va)s4IjEU8C7U6g>*)>qnmvenL=V6zy(2N2u2$ z#HqDplwL2TduIB9GT3u*K)0SW?QE@T&&m8wbIIf}jU8ECg`nZ&zS!hCLZs|4ONFO& z>354)T6pYaeg&ottbLyzP35><`~^8h1j!JfQ)xDi@)igw_yl`pEp`4Kz5lqEMIc3Z zFJ}G9pU{Wk`jjy6B|^|Jgv!+%7pvIx!pa-FUq1&}0cJSwB79*lzS6-lYn9%(F4q=N z39x)XE1az&R!}O?__`!}002M$Nkld3=2!_ngfXKq4GWdO z>uJ<_K7!s2ab5waE6}xrt2G$%DA^(*QFGne7#NNXGF4<652KZ@?qmt&d%|byctPy% zs3*{VUi&nB^~`L;!{djNA7BroT9q^P+RjkU@NG5qAfZDm&KCh#qjW3RV<@26#^ZIw zNU|n?gA~B%9hD{gJV9T(X~ILJ0d6Yk!5qMhvpT?&X`2m0uzC~EkpoLpBX~FAL(i6d zIt76HB`C^4&*voIpLd`${ySm{O&Pg32|(!q)HSadw??&6mHg0s3VUi0PY>X77I|c9 zNuuHObQck#D{{?Aid~*O{xq$;{{wiH zXJr!20MFfzN4o&HbsEPe4S|bz+s8P^D%UMA7H;Q~vpm_RZRcdlQh0R^K<@xLhZ)^? z-otq#LLH%1EuklPuA$;QxvKQoWRI+zPP7~;0A5e&G+=1oFl2=Apa}t(pjduVx!ZqE zoupz{?%7N{8qgvS1wSfu3B*DMLH-p3(AAF`veIc6+)xm}=cQFcXD!~R_sV=wJvB(# z_%FhX@P;WRa7Xa5d=dbIb-6~vn#vZvXf?qh_XysH76g?Jl9$Kza_O*8js$QJQSrKK z73}yqBoY8fZxPo@-XU%=;f@$ud z{9wPmx$AMSy81u|gsvxnsB0Fdux|DtJaX)@bi~bLKYUN*;fW!5j(7$zf}eA5r|3g_ zioS9ls?RPhxdy?v=V4?fXYAgDV*fFTS7#CMVZxs-#;u(lh^xkf0wDMRG6Rr?rvT-m z8&Hpo%GhSDc3tn~;0tWJNrL9jU9#{Cs8_+~AX#ol7=ml_UFAW~C`~Og3vZexrZ*Os z0d2Ta`JBK}0vRGyR>QltgZF6v>0=t;9)-fquxYj9BizFD;<9=R5ouqjaTuT%57!k7 zH5W+bR*A3yu3FaMOi#P&J9~8s7qXQwf>Kxwu+~SIlSY}`Q_sXE&u$Kj2Oa&!*n@FE zRF=@(5gv`s@||?=`#)d^Imxdk&mw4tDFCo=;2Et}Dp;oUlfd0SmgcYDBEgaT3S9Aq zPK&s42a4IWRi}3tN;^C&9Q6>^qV9OZJ~k#DouFL;Vd5KazL%Ci_+c8q`36-|a|r%s z`s#PTNsoT^vvj;g2p2DpAXe`h5{Q4BVA+cYfJAl#ugU(=*cj#Lz7+8`I7C|DrnoS+ zZwT_7=S5DUpiy|7vsHYCE;)*LU)fwhtOM-|L6_CUdLSBva>7!B3i(SYV#9KN+fdi= zp3QtK5WSvTy=f9PREn~Zne?_|i9vVfbsWK17toLv#^UwsG)KNg|G`-*Q}H&iH|x({ z0+4Cj0$^?6y*MDNy+i)acmK)%l-f(Ts4VSGpB}Q0yJcqIQs&RK2E)qDa2EFfe*@8l z{S5C;&5=KZWkRE}wm}$Z_i%@*M|!xiR~61<=Kfo0>Gm5i4zm5$9@b zyiAY(%fF|o;uD^pw_Q1!sdXZ1ZK1OdTWeWT^fRNK0X;rG! z+s2Eg#qX4mQw1=5V^~okEsv`J!y0FQ^;&wKR<{{G)uX9b-TWCnHqQGULs^fZa0Q;1 z6a(os)u>WY88X#?n1sF^ZR07Iv6h`1=)LUiC8$$n$oB)ohBj$6gZHG4$FL7TW6_0% zhaU7Zz^{hZ5Q=bw=TRY@^Q}i&Z)tIyzFyFa)sYG%!_w}z&8(f*n`3VUe&#D?0o)eE zm~ZGzKs{p?Bq8_0+g1hqAEWg3j(IH7o9PJ-kIxXc+r#^3(ZPQs>P?s9JhME%CxJ{P zkeNvA00DvkNPq-IQKFX8R@K#2y;WC7k7pb)7cgJ!JMbkqd=CzPaM&}p!!gsIo{rvJ zrK;|!Qk5i1C5k)61q89Li8T|6-M`;?i|wQ!0Esu>_gT(!&j0M2C^W|23=;1&K6()g z{2sO54oD7xk@8&YHOV9uGJ?PW%dDeCz+)d6pS-UZ&}VGl31HCz*DdhABJ*#-1Q#xA z_OY9U71v^=#uw*m16OCaT}PN#oju*>tjf+iu`o}QkWev^iWW@Nv4wXM?2oc>)X;KB zq67C9j9(n+NEiALT$};VoQj5)**HV2%|Y9R^kOGR)@UC|9xqq}gc=#94Z;O#(9h=P zLAZ%6uh2~`#B&l0>}6XdUVEh!zTM<=a|O}Yk)vK9avK!0t+P{L(69*0MsKWYnLr{w3y%Wv>G5Lw{R@N3@qHI7~-}V`5)hnMc$=g)XE&`R>RFwI9j! zRqR^ZF?(sL5`1YP!y3n#Qn>3X4c9B+INX4qan7J;NuVO&@S3o0qkx{XXZ}t4(%N!p zQBB4x7u14djX__B_Ixi3z{cnN?pFX5u5yh2Y>#Q( zyD@gGk%EzeYeV<|ABeR$f*zfx>mo06{N7uOmU6@2BUw$v8~64+3Xnh!(4?;b&nQ)n zO@6@uem^CXJy6ylu$Z0i$9?=wh^FPuP?c<7eAjz>S;#Ivmw_z)z{@epXSB#VC;79n zjK*Ti%ev?{eTaN_Zw_#QSD`W2!kT~rFpV($z&*U10FDQ-zRFA8rzV(rUqQWQE;vw( zL%MOFysuWpJnO~k!LQjz!KT;8l=vV2CFhddyuj$a5Vm3Ti!`-Gm zP!EM~hpxQ?!$+HKB~|{_3tT;$X&+Z(otj&1XkRS=$r6KZbJLjw4}qxzBAJXthTB{@ z3Sn^Q(A0F39SSt4fUlzAp~&-t0L@Y)n$op`oFYh$vEmF~8lfBq$q+EoWvnPq?>|6@ zcTnbJlFZB9^kQZfZJr{vV{fE2eD-TJ`X0FUQJVPpFVX>;;P5zR^X28xAQvCrO*3Eq zB2^AH(jbcL-;sh}rpPS-M9WHSG;>gRwK5(P33xNczeoI?F*}y0C3fF&c zwy$V^8^YS-P_uH8G$8Y|`Ev(FIqi%gk(ChiwuX^uK)RvfMJxfIvA3fK*peJtB@}GN z^Vk~8Imj$^51W~_j_bG^U|`b9I*Pz1R@`ulGI#eTv{>$?)vwN6tg7GNJCr{D=}*%; z@4bhh9VBD8p1%6sm+9HF$GC7vCwJ4$Ug%X9R0VUfXgHS_ho^a9X5Kl2D6iwT;T9J5$uKr8ce#G&kSa5zFRj7t-5 zq>umAzfSvr^Np9!(&F7O)0NAY(+C#6b_DLdFF#GUfAfn}z_L3@oa6<1ceVhorO{*7 zTCG$d!I24TjoC|Tox8eWxxWm^M&~j!L4sMv`>dy#yE0(A3JmsArwic=uLl5;G{o8} z6fE`Ajk0t|%q6C14!FWsJ(^?7VYCy53Y8rOdcyKnWG@63Z2(yI3Pxl$u)(x`Y7ctT zHHr6+&wI8D2&Rd2JKAU@4$Ym{mi@hXq?8F3`!ztoRamgauQ2WDFGmT@TMQ zj?eft0cw-$c2KICBv#0zw7Ny>Hb(*Puzuv^NI!+831(+5(u?lk9b&K6*rPoJoJl(- z5D=RY&$nAb6X0NtzH`>LOafLXt~A$h9i`8Bx)z$4Um|(OdXrCI&87#>Ub6R0lrbGJ zA9=lQ+6wDxDH;W0tFCg-rQiuGjol}-m}+@jURwv?WuR5wmMJh;@lY%d;6v*>856fQ zpY5|40`Ygmmq{Bwk9&N4<8ow*Ht`A+xYzy;f}ORtK3g58P=_z&v5@z6w_w31u?&82 zP7R7&6Jdkp&S!Q|PqNZpO^;WI?6Wvz}~O22zBPwOlM4g91Ct=ppQ5Fmo7AyHr` zwbka+jfnx;v{8Zy1Hc*dw!s{}+#tRZJKW$K?*jTsNP&Jv2t}#1W3i~SW&(WXwTC5k z8G8StKmBQ1{^lFXHqGLnCfkzpckSAAcoF((&A4_l18*Wre{n8#bG9}B>ect(O9$O! ztW!tYSX(9`2kRSvbA#}fn{Ql6b$CjH%;mitG(_@-$wfO+!_HP>loY`%cZR2?t`uhJY~)D9B%($>8eWw21D1VVxeH_#gIp6lo2F)D;?8`l7Y!4}HCGh^*&zvlc;Id2vzZ^G-# z@TOzJvw+pG59Wu3x0TN{QRo`%Yv-k5?!_60F2o;p$T2J3q$-s{nSnSY062SO2+S$p z{gbT?m5W*#E#VgBY1maIF-(sz1Oc#K3VsD2<^SNrkJG*T_tT>X4|P=YM+6RY+<;-~ z5-2Ok)#MS#Za|&|YR8oo{&sHI0^wRpK&&MCRVYcQoP(?=Cfd@8gMsmEk$rSb_E8s& zvU@2QIWm3?>sB*uVeR_z(fvps=tBs0u}D>(U7!wCQ6|e~$v6aT;55G2Y<^Q4GTTP0 zfQ3o-b?dND0TP&m)-!>on<4r)=(L?Uwr(tCt!$LY{0Q*);K4n{j;jaj*u=;PV6U8> zVdXi+lGAzdjdV`@#$NA5EMgbalj) zwKmzXEiTQ1z9~(?+Dg<`cfLrx+Ytx#72xr-1B(vY^W=?d>E^q* zN&7FPzx(+&>9gPd?`V@-jE_SM_!?sN1&B&Z&h^QuK<8VOs(_X`+?ekv*P_XX|8NU% zuwj%t2wr1>Sd<_KU125-406vhhsDg(0zK-?NU}FwgH~U<)zmmz080(_*=qF0fpl@* zIS0bub-Nkxy|qn(fQN)qZ*6FhI$UM!I|Z%k)?O+kZ&UvAk*~@1zK457rdH;R!`K>$pp69E`RrV`=*1AEzVS zC41cO{U1!CVG!$v8^q?*)>tZVP<9@_pc(svl%vV@IqvMkwZ(J++BvNuYzPd$c;o%l zH8w_H(>K$|#Bdt8&<|5B5_H{@))!u-uG)Hf{`B*-^X;eU!MA7Wi3cB=)M|2x`E_>U zHsdhq61V$=%vI3ps}12{VOO(cJtejc%L~#uty=dE2m^4P*$#jnZVcunz4mZUj5+;w zd5JRs00#h>dM{0*GlAJ??*(`hn5(t&4Uh!r;7%5)1#1u!NPGqYq92Q+g`>NmZ!L9Z zE(=x+qF#rt9Ea_TDNKw;j?+Tnzmu?b@po?(m^I7d|~388{Id?1w}DigoH^S zT>z(k=*^A|Dh6iVRv4_MSGG|mER$kcobfK=4Y|fPQCgd|cvrit2zle`ay<7S&0+FIYU zO%&k;#F;Fkl|c8ENjA}UnDQsV72qif#yuEvFnIRQ(klB1YhB5L`z|d=(7T;Yb}1>K zhHm_-S?D2*n0J$XutlL`R4{f^*2CVyBZI@~^2HG{zLz)~T3q0_fN!&LBO@L<7{5N5 z3Y?pv>Iq#HaLq8^ZO+Ed_9@{USc`CDGTpfT+4DKT2nzS=%jo>lPa?s_#t!9c8U*j7 zY;{xra-cUYT7Lzw_;3F7$7%kvzoTo%M(SZa#_3sjI1g|b>h4S5E-qu$-im#;f`FY< zEFyhE=!Ne0R@|*q#Nu?Yc1Ro&s5;Y2p7T%3^sKCor_SMt^paS;GXT6aaj?CNBDunoi9G87wau7$uzk?*v0~ll^a1T5W_t|a@$;HH3pGo zt~>-RJm8o;zKepbtNoqteG=}}iKH!*FgPvU!DYQT4YzSqT6*B3vmxJNqOcOv=H&@mw-|W zeuJj$InOtHrHsuiL@qyaj`v<$v%;E2$d9!n&p8)9Gh)VIFkUC4k-tcE;i~hD@T;K^ zCNFlvceE1f8kFCf&}Z#4U)G6p9nQ|MQgV%-^IFH_^;SO6f-6|_8QFTV{CrNJ6QD-9 zg;MY5l`G0>U4-7N#n^E;m@!~_R;nGeh?~4d$0zE(Vq4I1X)>L2_PMT|@qB`8UEl3k z+uhqJ>BHI@CV^OIyzC43A#-Jl2v9b8FPN?qgD>lGbgi8Wx8}ly z0Wu5jh=td;tUFGt!U+ICYs?Y5pk^$?sw1fJLgTW8_&G?ZD-_@yO#S1NFnvJG#wv}9 zpQJqq{0LWuHK2MCG-VE7g9T~_M6`o4GLy5K4fp0bZk|FeARIsQ*kF#LR$FEqvC|d` zHDw+%0+Yvj2(#5ysf%R`Cb+oqlFtJ&2wJX?`E~1~KMFU^>cR|30*y3BJj8M9VCo_m zyZ`MEQ_s}RbVAG&{1pq(Zd#drM%)E$R~EiYufF{?RmN!zgClm7!jJSVP93;)A3uD6 zprx-LR=1VKIbswLnyjH9N&>gYuJCecV=nbmL+da8{3G0Zmq?%QCDv{>{rc~Jo^~j| zur#-c+n_y7(V#jH;5j3pHL_CBe(aSvzc3qax=0WR2oHHmf$V|tDu-&vjLH6+f-^OI zEt(z7#ZCqTAbQ(hAvZ%)G^S502`4Z#xqS%RzLbYp1XNzkjJ4{tCSbJ~VYL~_&P!S_ zfkExTX1i)?XV8}MkCdgPPGB*WCNHO{E7#I{-~U0X5>SmF3d`3_dims0`se@jII}LY zcLsb9B203`1D)o3QVUEl*Ed4&aW8Iu+^XC&-`humOBRP%1k`O)n#H+mHQrgo^@0U( ze{mtzm*!HQf}iIA?5eGw5PW9oNi$=_Ti(2WEu9QsOL^-4l|ZtG93Hb{zxm}qrtiM_ zZECI~Kmh9(shrGgSU=oJ8nK80GyBW~YqybDI~4`aL6!3(m8!9a*`77pfuS_9LYll_ zyqmxYVt}4uhWuR(ImnjJ?GXHIYZw_^04oH8XHW~8F?WUyxCaU$tp^p(rWODJf`Z4R ztLveaBoke!#&PXIa{^)+s?0kW0E|Nmu$iZF(KEoJ*3DCdud!`Rinmb)7^Bui^iwDg z8fL7$2yS^t3*gnz3QOzQ@=3bS!!UIeSJ)S6t3vmLW9Dl0{Z`ig6m8qYC~KE#)!iqS z>*&NlcmNM$L#%{#&NHp18&*Df4UCTs4pyj;WqH`kEDp_evUOGsSEtba& z!mZpx7Z;hcR^oQnY5-8%;#lEx1DtVr+79RHfHmTHKr7U=;~B$*SM;*!d$i5kz-_Dr zhGheoXR$?QCr2i>{E>C=oV}hy7iC$xu=Moef|Mq<7~gFy{Z_G8xfML`Bd`Q{W!#V> zBLwbKj^oA@RRN9>mXr>$zw8>|Wg7q|a?|=$03>eX{*-qs#agt)HYpWCs_Y3?9!ul& zVfpW#xCVg5qEAwVS?nwS{GU@-a~I_e%R6gULn$2Y9mQJT#4^2@<_uzE-FFcztpK1h z<4lCBOb`nSF~ zXZ;W`yGKT$0H*`XT*Mtii^?{8eJd!V798|qh>?i|o~a0B;JfnDINSq5W8?uyn~;|w zumJ-Bo}dw3v^n@J#|ULRQ(_L;Z>@65R^M;u3=43)r}yGU9$DEKZMLxIm+{844f4d&X9jvF8z!yX@Rbv2_DH*ovuua zsXz-}i@f7HW8K%p5;J=}>lR0psJbc8=FyZ1nEuvpX?9#&fEY7whcWx}*Rf z&P~*#>r9A>t5?KV$a#@@Wdcr?KI>Vtf?!%L%PQD$gj=f&2f!& zLTz`J?}CA=p%x&THV&lNONlFC#i!P0m*~eV}?B3VJETAF5U@bmUVzAv3jD*MgG=@lJm{Fc*YvOft z3dED>pZCjtnvps!(`AtwFmr`u1Yr{^bIbynd?0gY)$zlVxyeB+fsXUN)%3ySGzU)8 zYGzFsz~a`#v{2h1E3tS1vs7EUFgBGshQ`w1)YViSp#U%!7m)gkSP8%V`~Q-5=4R3m zv2C4LU?!;5RmS~TWego0{;*D>k=GF%T3c*aBmwD`J7iuaL_h-p6yAE{Iwb=Dw*b{G z1ol_Ixt$i~NyNZ<)Csb`jQRcJcPPZkT3cRW1;)AoFy5d~o>ot-7oJaDS!VcnK--P& zi0#e}W_I{I-1^?NX4$_IcN%+ER?u^YRmwX=pHc|fSRl#8do%z-oclN>x{EYvN3AbF zlfvBR+qpxgDB$vdv+xK$-3YiHAiW*-p!e$4^w!(&q>*daQqSNxg61qe{PuQw_BVf( z>bQf?CaKjfSU9PSHq(hZi1W_%JWfxy(#5hZL;2EO{|pQpa-#Bp^3q7ZDy zSbxsZit%Zsi_GE-Ok6q2$^grZwV#UmGL{;q`9T8xyLqp|dmFu8j7@ne}t<{>{3Sx`0hgu=lEexgEW>Qzh6c=Bm*6kH6TAQ>)BLoGSZbu>3dXDKJ%>qJ}Ya|<>qUhqZ4WZ|+ zG;#@-E$&7NyI062?I7s?G9HGF8oZyX^>bunHc)~U7HeEZmYJ^PP%~x2w<&iwAg#hl(lP%n%$du{@ zDiPd-KWf<%3~_cmUnuJ*1i<%Df<)kKC_sV4xP7dR?eGo(!i&3K(E;Tk=)DM#qJ<1V z6TuEp7@L?3%lYc^LeLb054w~#Blc0CA)WHdz>{YH;R!vIm`9%C&pCjo@h0e0Y-op^ve;}P2hty&7AL;{QaLzj45%h)UxB>`%Aj$nG;GY-@Qsz{vh zoN@omVw8a*6=r`A7aa7UJFtNTQ+esNj)7yr*#fi(p8Vn|f-J5MOIsvLu}9Lrz)-7y zq5yKZg_%dCHq`A#>IQ!j?Gc#8Tft_Unc`W&f?A zsCCZk1(3R6wJw@;!cxTF1fi}=%#Z(t!3a5rY8~$xYUB7m{{A1PfB2gP;y;l!JZ~5I zbI8vn@(5VzuyzKh)g>ye@fq*c%DISnvSuc+G%5R|bykX!2$JNw|st2`+>h9?Qi7v5C-?i?$0h5=5TC*nlnR$A9vd#HwA6eyT5? zKOxsb6lu-97gF!YL>hnZPtz4@wzYR598YL-^@O@@fBWy#=He`_F#so< zFu^>lmoB8amro#w{WL`2a)~$u)fAGjWNeLc}Q9;kwg`rhne`lL7D}t|HbC z!cs$hedvwT@;!@M_Cx%gxU95*)X~hWlQn1GM#!azD{cDyTj}5Y^e<8$R@WB5$I9wP z`p3Wezthb9yJ*Dw0o2>Lw=S&~wEH6n@~HD->U;MGspsa$>B$-u{BgY(3Ak>@HKG~4 zfdwlE;9GwBAienF({!}95ZbxDSf$-o6a(vLMIbwKkombMumgmhf)*1r^KJtT^l)Dj zI971+Ot&7Sm)|}h9;T4imrRn_3U}x!>*ZRwE?SbbZfPFh1-M1;Rhn7r5@Heg0AY-+ohpt_-QTG6`5p>FagK?M)i7(7kt3q~P00i^d zP!okwT3=}^&Mt9J98f{5^Ec5V<nq%$N-LW!XZA7+Ro!i+CED9eW4I454@kAE~fk zRmOx06pE(Ws;(#9ot8fV8^IG$NKxWgGH9K-!mnIw*H_KhgpOW>ukFFi!WS&NYx`KP z>YUdGlM|>JFQ?$Lbcy)ijAu(Ct&)`BXN(Us_C+3cAU*SZGo-IjqNf*Y&n9t#GfPXX zB^E+T|EyR#gmv8I97`vNW2}C}{F$6UEj<&QsubdF!f2m7U8Zb^;1^!Wb2ji)RM}70 z{N?gStch90Rd|HivRd4l`L47^mxDoi_fB4#pnS_(y7TK_qzgyeX%L#Bq%383m=h4! z{mKLjYlekwg(Z55Wy^^+ppU-Mag?vNbaQ$-?LYn+3kblE@D5flta1nNz1yoalKG9)d>ESoGNt);bh>{5l|0(oS{1?U2atN))aG_7-n0uA=DswJ3XJ1l?fH*Q} zc$5O{$2|KCS7#Ze;ACR~^t_ygi7j2ANN?X8KTOAE?)U7sDZP5Zb0{FqOje1NejnFo z9YyW<081S_c9A`7A(j@WS}MY$&uYtYHY#>=K%Sg(CTjFL9vrz5pveN~7wGP=wLuIh zu2IjoYa>zW3N0Zc3xZ^5A-pHhmgH{?^J^wCw55MF5oj=A1KYA$*ZJ z>Q#sY4GaYx)aG;UetG6yw{zm?1W3_-jC;6Ff)dXK}?;RxUyynL~-*`R8sAnqN(5!*ob_8xFk64=Oi1TSoV!qfNSi-nAOEj6uu6$sa zyyAbZ3_l7WS*Pr+ST7UdTBF<4ZhH9Oet_@Dj)!;FiI+UY0(inTChYi9k<}~0Oc{&k z&EpHtgy$6HJ@6r4dKugolSf@!uTiG>b0o~=2(y4sI!9@!z!^IOM_KecZ`E;Qt8x&% zuEx-wIb5d03)go+^#?3`4W_jX)6bFR(}Px5#SGX&(CiI>(EtZ^eg^j=!Jo^|zYBtF zV!hDKW2O8mf~HfGAEsmrW=4!01BVA7Va+fh4g&5E*Y@_?Yc-!R?q~lYsHuV zn{>Bx;H;hI!FNAv+{fc0_-o^tf-_xM4Zx2`MZlmX7Va3tqgLRv43M@T>f1MO5~ER1 zckkSxFy;oethhfh3_Hxbqn9G6fA-&`>02blOkE=$g<5|IvA_P?pQX=#^V_t(yqbn7 zSzsHi6Ivsg^54W|w8&l`llnjP?hn%t8MpVBHqy$kewVI)@(0A^7>|Z)9M|>%jChQU zwom`p|2e(<=Cd@&eq4a)I}HTo(AYjm;V%Hy%_f8J)OZG68{Kg5sBi;zf%4i40ItKG zG{hd<`}99V?AA->qBi8=!I2c*H%$JBbaGdYTW%o~V2@Z&yfga(urL6VSm}KFpzd(?+mYNcv3D&45BZNo9&@m6^Gl-=^*R8>@W{+A5 z#@22!DaY0W{q_^%Rwd)O1kDJbmk0yc*~ma6`3;C{EUM7NF6UTRkiq%hhsP3oRH4xB zATgO(QxM{1&f1PKW^mEfU1*1E`>~|AL*q*u+?S9CEmeYfU6#6bEp5VJvA~nOfC9Hi z?1lw61--f+>|mm@=;ng+zg?<<=w;NQnJ5@rUpHrfxpk-=D5=#?882<(%JN--x7-IxfZ<-% zO;)b**+qqI%fq}mHkigo4Dd%tvNX#ys=xbkHaZJ*Aq*!6yQ9#zrB}2HF0JhX2Hi8X zbATJ-5Ykf6!n-)bhx8epy8dRI=VxF4b1EL!(+4PmUF}C$wWK!$A@jCwnjIqQDjk#x zdA7QhYG$jdc(dp4p=6%pQmm3B)W>}0UOr9l5at1Vppyb^>`KGwvsr-F8}FynE7#NK z#QM%ZpG_l^?;?#9?QH{MDMbF+9k-h#iOxL7+9*KaGCz`DZa zi(1PKw%?-^i$VTP_+Ay)uU3&Z+=3yLGa1Yt2>JmS(2M+8VXtCS1fYeX0Xh}@1bC7s zai#e+! zC{>;8xkiDA>tcQaI<0boGp%%5yf}W`PgM#2&loxKj7e_AnS-vqM$1zuzI>m$d5f52 zTBhw0qt1D(*KBN$b(V1nY_ixo&JqfT7B^Nu^b&m45rvYJJ(u=8LZ7t(?im;ZNMh8| zXhQrAXWa&k;9g;ZB*4)}bnOkLnOarF#GXQ%XIU>$@0ZABSw0Mfv*9@51#|0klp2 z@JFeHd(IxI-2=mD2)L#gf~7O+FJ4iS;~&$?qi-f zuxoEnIWr9F$1>Kd5KIyh8u2+y&WurJi-3diu`iv!kE3DetQY$VG3(wo_*)B#NhSi| z65kEnth2@k2z%Glwm&8i`1c%Z-r(HDy4qF3l!~)5C8Zo*>Bo^6ZPT4rLgbjMe~o!HRCLa4~SL7n&t!Fntwm!Y1v; zazUU0oUM0eX$j8;oQ5^xBQPXu_IWT(@X)Q%L$hpwnx=S76ri|6Wf++CvkV*PzA4yTVlszhDn^o%sOWOU}svdtIt3EzyLY z&y+Y{o(I9dt+uQu$AX5{&mptFx!KbYsi z(kzVQBwZqNvqDeMJnLFww8z9|HsFVA0>94qO{%Jg619!d|ok6j7T0;jjNae2_Tg()ST;w*qYJk>t<`U#af^>hPW% zkeSXhvgU*bVBsr5TO}0i4)$&b!ECsQG0~cqcPT5>-8&rgbwK8_w^^4(kj+#h2&D@<9=M3;&RDIR?EE*#(Y;o5no2mg^nA-#rIm zCF5+UaB7haFrnoX1Iq3eRxIwNWl{ypYwdNtjqEbn#c|0;y_djL7i^jM*Io8US(lYz z!cYVk5%Bu*oH3f9fdD&PC;!s&)&dW2M?tOgT=}qZbXw{BK$JaU4!RR{-Ks>&m;G|C zh1PD)4puS7$BsZVSlcyJ#9TcCjDdZMO#Ey)Vy;X)JPZOV*WDx(`1v1Jvu|_dSXQk}sw@4y%KlvrrnddvF*YS;c=wG__S}!sgPi26I#X37j z_uukbD#!(_+LT67U}Pr!CI{T&j*`zF0jjne>7~@i808fDhQ}dLEGEQiH3+<)fBFO~ z6>d=i_VEHxU&{C^7m>rx#u=k!2LYZC#I8#<;twEL+eOu}1WCjWQpRslN-Y>A4wQhz zawHr9V(eIS7PoPN@Zb>USxAMLR)UeSD`{x_N)&fh`(8(bdV25k^z6=O%%6HlZ@-r= zfBX~r;ftT=Q8R7h)!nrA_?xuz@_8!mQO|`ILbqTvomFEWAOr-LS-3?6 zyS-5%38g3|c91eFyg;nO0ULAW&Ffh4h_!&3Tf@vuwfR}H%N8GDty_f&6aU6D%M{Wq z;-1TdvRfg=B}PJ$HIDs!2cxhJyE?x(xi z&s(!gaOTk{B4h+OmZLE7Au;c>utqFOoiynh zZrp;IQwZMScmIH(bb4uyT~1f0Z*i#F5MD3SFaF_g)9j0xv^ckbC9sh$UwtE8`SAPc z6aiixnMh@T^9h2ri9p;oP75M0^;}}D3EJlX)Tq5?lFY{33DGreejXaRwU@M$Fv@)KTF$?yO=UQOZeU)*zkM`Fc;`0`B)xx6%+Qbpd zz1UE_6_>lQT>*MH{Px-8!;y^QGtCzBUX}|9Yc+J$u9cAlmI{_8HTrdgPMy?Hne_=w z?Zw(UF;Hlg>{;k}&{dPY7Ra**QOtD+>t{boiGtK5A6t(B|M^_jdvuF{+3}Kpff#@=rnI#Oc z1ppB=Y8})XAXxUMy8eAxD=7kap-ol=x63u(LSqAaQ1v>d0~C7M3?D6;ewJE9=3hIsjgSxcN&# zR|wu%BP?H7?l!QvHCTJCh|-B*PM0_}eA6gvUSuJLXhdJ?z$(IiTT}1zrByODv5-Pf zqXWIPBD)xT$IAT{re5Y6>H%Y|gudcTXF%CCv1hZfkMc{t)2wB(m1a0N%^vQH$y7d1kUH=j~cbB36}M<@fj zk#><6&^3Ig9VNbn&vuhQc1|YlK1`LPhV>4x&U&2h*H9ujYlQb;uV4*OqOdNWV6B(r zOjLjb;E#u?n-*PrctTO1xR>&#aKrgHo^LNsm9|i(JnIZT z02p*G;1FaA?4@?I;ssO!2zjIN*N$6DjtZwTN`id4&jLU0Oh+G>JGX#s$sdVdj4YAQOrCcha?5YInSrZ&Nrzm0{ca#|rf&?Gd zSYe;>68uUF?BV^{~* zi(9a_tO@b#+>}vpR$_fzW7!Jt!DIF0v1CCD=vrZkCyw0eq3qta zR%oKUdq%8A_2Q*8d7b{96W1^mQ_g_6qt)42is?S4m1nDfRa%4AfjLt)L)qV=ee=(`e&r2;N9H zZoZeM#>b-s!7*-N#@g6sWEdeRRIK3-DTVa6fqv;V6)5?BsQR5g75F`guqSKK$B){GKW15X&2VW zH*Q>us_(O_bZEH$2yKjdXfV>qNMeVH>OzaQpCrV?T(s7}GU7fl9RR0CCd}>-yY99F zK^DZ|nq<4iCaK|ShUYEsV;2mYeoKHB$2BX+SOEjLv+P``f~7EkJTxc>z{=WMMl=aq z;MH1wod{FT1y+t?dh7ahSmgEzs-9b33zvB-y;WCnmDSPQM~Gqj;}1SggM26I6DwcV zQfBtilk^||um2-0({J|lWIx(9$y9HC@WXWLhks0l8a0(zi!Ll`yTt$OH2?+mT58}{ z?IXD)2lL#e@8J3(nWAJKo6K-dJlH9jMtSac3Bl8iFe+mDR*PcEh+l%XaF3Z#AkaR9 zkWPs?G~P+UYz&ztOiqYzv^AB56}O22v+=uHk4YY#9G-IkgnK5y5kRP|qG1H6Qrs!@ zA=;qDu+XqCGAS)X##$KfAraSMl8gXm0Vu;QXdl@(2Ky)~&1Ypw?mcHk>t9%0`F;SC zGFy6qxF%f;3e!&9{=*1z`|BEus7XJvx!LCi*Vkb@HG;xhut1o7x4cRpQxqyrM$`@5 zgAR7EItv)J822!jX^J0r!UT;ywX}#<8>h7auv@6@QXH5%U3^Yq<80+9z3gopad2^- zd3J%ZSXaszPK86!H3jU;=mb;7pGeQHl}ulL*FqBfOx&Et#TT`x`|pi6CA*J|7;tjz{yb5pGQ00z$k^D-Xul(E=Bq`(}t7`omH zRCXqSf>tF#93SM&Ks#j4nwT*}9OURgcbu_xO3K`MJewAXp%r+xGv9u!ah66gDZ%b6 zyC^s7oQq>@V5zdzgr}2nzwtqPQUZ$*~@b7eZq;QC-{SBcjKmkSoDArXO1(^k453dFx zA=0Y=-t)W1v)fVnE%a(_(M(zQ6Ml$utK}WGqbp2T>oIfh1f;B=;ciDJXdzY!30E!m z%)l4Io%b||L{VFK+9~d8JyL>A>B@0wozq(5E6yeFN4y=fJU~g@gLB1Mv6g6#b0t)8 z{PIa(+=Bo8{!F8E8p_=1_U_fcY$f>Z5G6mP% zHF2)q*Jt?8_k1af!@|iM%KWfka;-)z`LD(zEtZ17sGZ9*+<({540*w^K+{kTU z);Df1H&cIYSL8#SbP4bxV>3QTB)g+~i34M*w*3`#nc02SY5)L007*naRG1MLlLcV- zH|w;Sr3S`!$+;&$gUqCl-hDq!Gq%Pi1y~<^O_ltmIQXNW8RR>Y<7ybUX|)?{NPBO8 zs-hM4z4bw=(@SrQBBZ;o9;DlU`yW%|#a(*-)zh0;!>;vGAB$Q-C77?-i)t-0wO-Qx zeKkPuo(UPh?gk5l-g@Ir3PB#FN3%0zrO|_x_!a}atpQ~*VS6zaSRb?Eblqq=_W;W6wF5YA0N;3nzm689-*9WrNv7tx3kI35*RgcU-V#04f9v z(fT)wWlD&~5q7XYT2>W&9awONNztDg9ZOTxPqJX_cQen@x8FS>fS1x49F%>_JK!?^ z=>50Tt@l4l16MB7ZU^h;4rLx7v^Mtb-sitcU;fk20rvE;q*j$oy92lN`~UjirjLH| z-=-X(`6U*HCI{D2nJvq~HdtJO-+`%mGJ60ACb)Q1btxAC^p|iGD)`rM(Yj_m zbl8xo+ALgmoG(j3^pF*17UMo9X=5ESw}UYdS`#>Qtm_cL&t>RYtx$$y3#gS1I)U(c zD{VwjJiKC>7?!JAa5xtxCukXEQ#gnyA)JY2;zG56;4?}eOi!gZs$-lPgd0MdZA9Rb z^~=9Q!1g|07Q9}7PIeea4f<#zh+4^J-r8OZU^RrG7MNMQS`Bj;E0T4(u2QaN0x)<$ zX$}El9VW4k1yy>`>Zp-^A3Bs-_Ye-1tJ~hWk5g;!G!g$G6v3BhpuFXx+q;^ zGkn!reL<38-MTn%5ow1T*|W-bPA&V=N%emcGN5@*0D^(}!SsaH(tbM@04>klG?<=a zZzt4af=cSwE)sHq0#|4Mzj*i*T2b?-_9KW#g?yZAM)# zZ@FzKd^w&uL>;PYBfaT@)>i<;8fRvX+83wL-wMKS3niuqtxr)VKtD0UbaMbm%U_$@#7I-_;{+x7TwY6it-QTZ88@ME9GD$47Xg3y3dOhIPfHYH z?YT(Gmda3bm76qz*PWr#+*g=Hft++THN=HI8ojA zt;nMmK&d5m^fqnbp`!60&qV3vJV<*@1YcP{PtQyCzSi=8aq+=i^yX}nSsxZ3g&Ow?V90d>CoN)4 zEa`=UE%uPr(2+ffouh5-SeB^Ecg)_XKnsrKe}X|li@fdt`7cOOvD1>*g*!fgD9?60 zuB&v}K~jj;S?gQ|2WtgzM29Qzy!Q&!R z7247jZ(X+NhsxYc;PBr5iX26*rqNI~^zt3?Q_P*G00L$|GYi z#MAWMuYQiJX+OQBP~+yy?>J1&^e*nVH~Z-2hv{EIpwC&jCZlsa1}~!n#5lW&Lt(JE z{U8SJH$#%c{ZBs&5OK_5t)Y#d0#dqh=a#sx!g~>W1(PhH**iDC)PziN*LBfkLj{J4 z!H7$PsBM6)h;Wl5j*j;v7Ih-yY;+SSgj`2^Md}(n(gJ`$!Q+Ks$4$g%Td>juVu#>6 zNjNZOLE$_xGMc6*uhPt!-k#Jrdpi3n-TwAop!rHSHK%$nrqN4prs1ipsps-!>Icjp zFdtNYIyaD+w=|d5i0`@oyW8m%47LqxY7wE)g%$G0pZp+Q#?{f@eF2c3OS5FxHkV%! zdxFIaYr|n}Gxbu|qp-e{W}iJscV0dsCI{_`6~0D8-iq^QE^UC3Jc77GEfxk+h_Vzg zW%k4RX$F%UFV@Sv+_zo=%UkjUUPJWOU=zSe>=Jni!L3$PVUJBH*)gb{uVag0a#m!( zQA>!q)vUJ8+UasVW*n3oF(?IQ7P0>(bhV-M%(1S{q&+TkGr(|gv~2d#{1k3GcC z1vujdHWJUQZz@Yz=(tDJZ(@DaaLsgfj!H5u>Y;H%7iz$2zGi$j;Oqx{?P5jK0&Ed% zHEgyQml&>b_T`+umM2)1v<4k|9pAYLYYVtwZKUf?+)x4+*V9bmRsdo^oxhYZ)97{Q z<^ZyL@ST@)FH;w1Y^;jgu8*2tJZB4mVl0_1`+fL}*?46bI3IyN04eNgfpe@N(Q45} zqQ?X@e`TO2T5Ns(be32vij~5&?Fnk1(JmC73q1tw^BJqg>q0V#Xq$E1<9ACbRAHdH zY>W-G=zR-esX}G@sUv8r2hh>kL-s4mu8Qr;R{%G|Fi;xeJM+*N z^3jzmX^S({1x@t<+`6g5W&z|oFF1qHOO;TNGx&=@whSk)gewv7!~5F+MuDX78xw27 zgzmipL?-{y&DSdT=lq6s7lq&maD8#~3Os)&jk4w@SYW@7G9nAdt!b4`CzTtN?|8bH zCa=B&?}Hvd)Y5qlAom`%*EUN-X`gtsd>iyk7Hoy_7S^Ap+IPQ6@08(Jf^E)tJ3`o? za=kVPk^pokSVQHvta21{f5tOM-~NNNPwee6XTJ$8FFt?FdSsRxGmV=7iViGvmbcm^ z_FBtT2T3Fb{u_U*>&d!>>nm$%3D7+8y`Nxp1}u=_X%Aan?H0NfzzQlPTV0wEpOPSY zbGHI64s6m2-Lo734tzh%B`j6k2B-kg!7}bA7?6BT9wd-pM*{>2xFS3R>);vIO^pvG z3#qg!M>9RK} zhI%04Z_Fv~$GsXD3%sZN!tcJq>c&A2ki?!S%MD|KKw}M?oG)L_Q4kmak?Z6I?pv%a zXF~V2ad@xq>wvOhWzUINTgD|gwUBTJN~7S_i+s<|$tN9__l(Tk-~X&K>$+(1);JRu zPWH&pWPlMq3h&K6%s2LeDRK`>)0phR9FXe*WY@@y_-c=Uo8syy4*AU&4=AZIM_kxu zxJPtb^vFoj64(;5BRTJk^-N*x2Y9eWN=4Fc5F`MjqTB_9D?!-q$NrBF(1Q{{4JIYn zY6W1yO>y}UFT*C=_M$>z%?ia?t1!$wpr;Qr{wY9d>&45ITX}+skXQ%?Uje=8*0EXr zF$61l>y9`^3vQ#tgo`0SJU!k@2QMF|mB;teJOJ{51?YtlywOK19a4y=XfWMNd`KJj z(DD^3p#n+Z#$h(LDGn=RG^l=+dP1ATVVNwk4Ldj0^qPa0jnFrM<6Mo z?6WpjH`ijM$(h(##t5;95?LGQCM-i{FY2BtLI7%&#^hK>NZ1e$SJ6gyaoyNF_zr+=i$fr|?xLea$H*kD%*Io7 zbTpl@7Y7n1dr`OmGe+omU>6@gNMHZ@muYVH8O)JDD=ip=wrVBq?z*|bf3TmdFKKs>{kg~b41x=?gC6}dJC&9t$P zu(`htTHFFc1R7VUlWE9$kg1fMj=*0CB*d zYv%fdyMXVeaW@%*K(2#>ZHtpZk}J&8G*h__=+{!J8&%-W$Jqmzy9}H$S|3X0p_WwT zv(!U{0i5C5LUV$a+KAUwBh;N`;I1y09;}GH2;&P_l=s+=U1FAY5fG;eHrzB@y5rc# z0}kd6XG`~Hw4QT+d1e{9GM2+o0prQ~03}`i%>C;C5G~&ad_N(G)gb%OUUA>_uyz%M zD$5dDKP$_eFu^l7W2lUoH15?BAJWk&LG-QAbDmgKh?T?!ib==-p-<;~XzUncm(g1@ z&?Eqt=-gNr&ci7}dY`$Qc_u*P7QvgK8vp z1+CjWuZxRJ7N}s|?%_)icz_U?$I4h!gJow1x&csD;s=;|-VPBMw%&5@3LH2sy$Vy< zY5My4D?k|uGrTu3)E}j2iqMzVE{CnkJS!!L;$lh6@w{Xog;_69ZXbz-1Jx3RZ1f_F1aMm@}0d z!MT~LI{>CO__r?Q616Nh32QikUiaq~Qa5!v@k*uzdZw*GmviSNhE0Dj<;Jh4ooB?b z5mvF!bC7@Re+w?pG2D^s>AZ)8EnK-KD->vblwWzA+WETJev-Oz$$CEA1N%h_N|Yf6 znU^VO!ymy ziEVg~pLZX!5<1?KbI!R@j_F#oeU|{!@7_N;32-Cte-kRCP&~uWzAh-P>-qJfduOeTKG)~Ri|@v_ zIZuIG5uVcAW&9;G!ZazYxPMF$R+z^r2ir`su=Fs_bIkmphd||~=lajWrY)Gf%?hdm zRMvJ6($kIgw8dOqBn6lC!bDMan^3TyV>L9xO+aQAq&wk;5*r&rUofjxEPHu~B~XZg z?*%sq2E$SZBW8*)6E$auQ6c8R@_q>Q3PMrB(^{f1+TlRgVZ;Ui+b!eT6t3@)OW_7s zUtf!ZZgJS98DbU@YWoQFa|p1HmR*z6HyA%r@Sat>`r*PjuFu1k42@duE>q5Bm z_diPyZ~vC8uO-#qbn6E{N>jJqP3@GLs8Fw|rBKC$Ob3OBFIbXB8Xy_saPeuhx>=ih zN;w(!lbEv;1dwHFE;8>H+USJ4hv&t?=PhM zZ1;d1E=aFcI~RzRvEp4|Z`Cl(!mi_zlQnTPgZ>x<48^sN|mTWlq`#$g6%@`8I^ zm-ZwyHcp1p7v&>_z=AU94-1<0&|u2|a4lu*3*+*3yU^#+8I>XB1<?}Ag$zabPWSfsSf;^8J4ph7X;D*OQk4xd{5ZjN6FjhMOpz%Bc8^w*8=swF3i7FO5lOv?dL(Vd7Q8^3n82~#~G^hi( zjiQFj6Hkd5^BsbGtq}bEP!9oXVa(hv_cK)Hgmo~BZ3$q~&OR@0G}4W?K1oklmo9kd z3eS8{0qu0SOo8WeLMDR z^w#%M7uLVS`T0};yf)}Y!FB*b@kB6=0+uz~DEgkfWs7TS(9Q-!8-V9UEjsq%l;_Ep ztys{G8*i0zMT3Aom52kBFORyyQvu!zZx<9;zD69NuobxQNB9W^4reGO-cPU*{ENE@ z9t3iZ!T0@)XENj%>**L=1Nn^e@TFBQi`&A2vNbs6im7O@Uww--4V*GPH@#mEE1^^P&eb@J}8Eb_BFDi(|TYW#7|&ddA1e)*ml z!g~k?T}NNKZJTy$LbhA?(V3~(J6=2C=$;i^*+C{^a-}aP#{;#l8P4HtU5ilSp&LFD z&(Ww7ac9Oi`UyVgUTgK&a;JN{9ph2N-Nl+RIM>L}aTE7JcdC}bZ2x}0Upy~>E}P*T zeU9UCO?)}Gh<$VZ{)scVe7;QDYNKAE;F0b@6!K;Af~`Fk z$&F~j1lwQ=2b90prlH1T)|!jEM$FUUs_D2OU?<7|3_6ZhH%z{r>q-QRoK*Gna>bxsBlcE(ujVBn&5 zVkNRwMP#yYs5*&bY2yIwFjkXdtkd@XM$_fEDA@q zPLg4a96r8l&5RmZyzgJ_i5Y?4T&pEVYr_dbI5Lr#JJW^`&rw6|wrUjGvjkn)T@BjJ#kovDqaGGF{#OAc) zK3Jjb!Sly2a5IsFKs~;V$G=OR2OH^RX_mEJBTkZZa{z|5wpvIlZ(>0-o-dy-)4y?$ zxSfT_nuLJ`qPFuuJ0Rt3mvtDR-41(o@xpMr zJGY1cCtFtHXPvcrMNdlUQ6gvVGHF8sw9NR@swC~YQ^rnqQ9yO7`!eHfOV3_eiwhSK z=R%iDmKn*OKudxO0rwGcM!Nkv5eydrIoB=@kabG=9WoL@W-uDGMqYVL8iGT0R{@eB zuZjC4EU19DHC%JH8af1E>RQvHWpd9Uu2Rpn)`bGh&w7fH>EyjoV5~`K+*%iVXoKzs z2fPTdSi#lnymXPBvTqy24O&8F5Dl)udL2;WWCH`R&Vqb3&TLQAz4Ij!NYqXjc9C1m-ZHW;f3GLD7A5nw2^ zU*;+J)Ospl8HSF7R#c-XWB)FY5ov~`3~v$3)(&Sm2h$QldQY&k{ zg*8j}Se0m|FZwQe<&J50c6;j5qf3<3l_(<*IR_5`63E6BC-C@q*%y@Svg ze44e>?QFs5oO=YV>h4^H9=ZUD*REVnGt0{q*`>8#09X_tKEF+=k^UlJne*^GS4}ej zlOsy}{NbPdC46UwD#i0iHv&oK1w&R>K$YwE&%xFq2~_buq)R-N!#$3_CYMbTdQei zaW)-dA)NT+FIi{8AOL!Y@TxEtxbG@-C;%1s{5DqDp0@6Y13SZN*iJU;2Hv7$${aPj zhB;Gk1GJ zWvv(ixEZ946$IK-`Bbrk+W->aWCT&_#;&3ysNd`6G?Vp^P=Ep!8-1sOA~Up&w__MV zi|5SKaxx(US$DRTc|ttQmpl4^D1V}|P@-!I>?pv4mKwpLAtz?HYq$t7%2{rO=8pFb zdx0Nw?%JSx2W!l>0Kxgyk)yf_HQWfgY{Qj(=Ygzqnf21ZV@zITeZ!Mf(u@IYVx`yW z=}Mk3S@~9P|D{w$&POaJ+)oRsiF6%trU8E_wZ_1!BnD446dUp{%oIGSARA+NMsk;ysU~v3ZU=%Zld%fO?1Yg)x-+@OKFzMBpEKq# z56@~~+0DW0c$@jV#I6gU<7^mnrs2TewJJlAMJ!8WZ`_;=pcC^c&nrkTjEC}A;=^#C zG7hc`Wl{^GwL`^MBFO0E<6@iPZ6h^?#ug$JUb!sI+~6}V2!8Je641v3&eN<8G8gM%8(^u34zUr$3vt{nuC6b z@H(ZZS?}aj`r{w}IjyPQA+EuCMSK=8(mgnkZcR_5mw(5E=I#R^X#7j9A^pHr0#duR zvgBD53NorScjF$kYW5!P4Nc!2EZAk3ob}jr3q5-NJPubUM5TK~A#QMWji1BZ&W3wX z0_nnFbzcfH%b3sIFb|6W%0TWGG;saSrfX{g-sF0NrT5(czMJ(XSB^kVw#zaHR({r( z?phc}(ZM=-!UEjN2)HiX(ONSgM1`~Q1eP4Io|$gL1FqkJxVHgz#!XoO)ntZVVgmfS{)TQ>8e`)wrBDX*BW8+B+hdA#e)4C3gf{yg@hz-12d4<=*#Mj`Zc?Y} zXqEbTq+_F5eDMXXyPiFhyU-`FOi7mU^xyrD|5H>RUsjrhA+;8t+g=IvY;nM*jMc%cCto{C2l4TS~s9V z!4}#eVjYM~k9XPDhXWWDtQD#7H6U;B@TEo&&yjKHz~{scku?v3A!acxGZUm#lvgBsmUfxhXb5X3s{)(Scf%N7gzyB zB>LR)oSd}Bd==KZS6Vpq5zhj1D|2lJ0K#$C8HGz_#wH zQe<0URVV9r^_Uu1RG)1EV2&vZBEu@c@S~_OW41F&k;2A@SX=}WN6@qJgI&<+1W77e zh9tlbR7yIrN(s2F)uj7XhAHS#+x4p*jnwR2ZW-Xz1yKaaVFVi^?7vyV41tmn#7lAp zWYYa4g2>#gZ{=Pa?Cn|V#%i(#O$pJBFI0ndJuR8BgFDYGZd<#lD(PnSI-eG;>o`xw z2-<7;t+5Mfe29wtSa>W>zC}43U8KA4fpf-V>|r~W{chH1(8K@)wawnU@B^N-kabB+ z&rEHDcr=O$GbfcV`Gi*T5`bDwdlQ9MP+sLMD!2@XFz!nbW(Mpa$q|lCnu22lPu&A( z^U+X%ui(aO%W z%O?flhLxN#4>RkmIM4!+JIAuWz`CB3BzN=1Wcq(U{Ta$bSDL^wxP6XNQO5pBjVA+RD>2vloF69R(%Y^IbU(k zxX(V}FcpBgFRis1_~P@7Ddq{y`o3e+i==e-%tW^W_)C-m;&-pzgCCWt?(g5Vo@p^W zBe6t&={dA+-jn;^r0XAkobe(75bNkmUYD(j*SoeTa1@2s?Ra34*;bCat9`KtW>;qx zML7XyM;YoXzPT%~D*m~q%4T`?A@QV;3NoJr48Ci4(EaY;+%xB#L!MI#j-PZ2Q>jVkcYHf8T z>av>HV8a4Kry9z?PC$~DHIX6m14{%hO%ED?&9W0N#43kWtCBYrf$HW%6cVtj7O0k; zZOplr118tf*PjNcA9qqf%N@Q#i>i;JR@ENajV_8C!N|h`(@lGxZtl5@20n^)XsHI% zM)2s$Jfc2TTM41Uq3T5=w_M0FhwBK9RL%V|8k(_VD+mBBPaC>DxVIA*Oj_qm+Auau zx0+0Zrw34CCIB(4DWH32S{~jr6_m{eTAWV^L#`3Nge)ZKsWA&(l|5 ze#YNN(hH$Dt1DCNPtjy1+I&owTo_h+B zUL(<VueJl zGfj7%^U)O|JwzW$_RTnvwe4-JMbxPStRWkC4orbP!KaaGeEtE&m)&P$DLcDqJH_4v z_4E8q7^nN9X4eD=Gn{zIFSNwT*alcv>z?gk-7z@7jq&SFv!Z`L4A>%|B|gvsT{aEA zWm8Ke3shK_DwdgWRe3|QXJIAGMt}x50rYWXSO?*H=3c8aPTQbG`3ydiQ+OKGvyhJzkMspqaVxK(|Q5%i1RS^mn^>(3Tg1S1jP-@jy z;24u=Vu3|g900UEH~ zox$V^hSvp&kD!-5tPoqfB*s8PeSk(|UgF= zXMId=zzGtqt=dt5Y;me|Rx_mEFh1y4lBfH+WzcU=?Z&vsMVI21$g%W z_}*_pr@lmFx=d)efKXbJj3(&D)S+f*;FY9KzRayQQ3{c43`SX229x z%pyKa%MEiu#f)coZ~2#eQzcZFvVT{YSR>NV8F{GIEeqklj-wMqgj*4M7ZCcp_j>pEx{c+LUaWGExpzIQ7JCO9#6APu z5pXRe$^@+|*mSV_F&6op7JZxFveDxZcU>)c^hmsdMU07%bYpRyYo7D`sqUaI=t*WVmZ! z2J=tTH@81c?U?DcOetvkVFtap#q0yxfkiG)(coSW5HV?xXUb!lQBymFU|@6U#l|M? zSuzIs+43Ixj@?`{OC7iFv>;lk#ONg6)6yl8vKtTqo1#Qu^N!s>?t_h+>`R3Yi;W+w~4!OxG`6P6@_l^1((= zKQqYDzjY3+Kc%N%{5swL%`el(fA-^a{eyQ>9&r86pZo-iE0zuJZyS=M1CmB?9knvP z+9MJd?%bh7z;lEH+5+IAURzBk05~i68}Ef0k}mV?_LE(}Ru9>4Bn3dM?FhoinqmIN z`c))sg(#fJ^k}Wl2Eh9Yrpe)3=aAV6LBQ?8c-WVSuaf~;x9kw}cfp`y)?S9Ufu^qZ zQrF_hnw?|O$lI^*$6Ix!Y&7uwuq4HPxe zEki&B{kk4z33~V~5}U$}LitEAx@8+>~h9Ouh@r1p3&4MZjvJ&&soeejL^=+uQ zz+(jR4$t1JRRkj&&;13%DJ_xgp=C)6qCh~mpovGiuyxnwU>df@8bm>JeA|?TFg{fa zf@5ii;HVZvc5X3%l*Pdwq?1qFO3birkR;XpliT`E-m=d3VRyG zeVIprg{zP_AyC=Gs(j3O>BnlNLRBZ}OE6>Gs1u%TtXO?!4!#Mn1XyekqrA3FHZMGT z^vavmP}`%85oZzQgPZ}vCthI%#lkdkGmVdvz_9uR`T+D%{5Hpam${MkXHhP;az>TTM@aa#r4_r6~K(XW%;u`*;sJQbYkiVjW{b z+)G|AUy5h*yWrR{I8Wt|ulUX!1vlI+`#kT<8zbwL=ej}y5nbNJTvn@8j9{xO1`rCQ zAt9a_4GT@JNZc>L7w6QqEJByjCl=mxLL9qgLrzYXv7Yfv48Q*}_Ly%5P-FrbX*gbE zhAAP1Y>=n#VSS7ma?B^PIpI6*i!8`-z?kBmATBNE@Z28CK43y%}D*O6Hkp?02-W4Lzn5#(*a9t30H`2kGJ zm(6Uwi9ICsZwp8a=rzc=3!xxz6C63v9^8A|e6~ph|0eFeBAH6H`jgb!KbBf(k#vei zWeJDyfg{X2(Vp8N*#GDM{9jVv@n#x8K-+-3ueT5LGp$W%oV>FMbJ2YR!`+53fk(mM zYs9;3D#&r8E+Qx$tubQ;^_y8lvQZ`g7`q41I#|Xy7!d4)Bye8_`nRBwTR2yulX`SH zHDH|k0711RO`Dc65X4yw*H3!Lm@-T_iz#r!O*OCLa(2yh_ql6)6n7Ife;^=(mDQ~C zn2=dGm=qa^%*{>Hr|-ci^Yc|KkY(m?dOWRr(=wV{```pwOT>n8IC8_6(+cj5i9fwb z2G~Hjv{s1S+vyuZN>o!98N5xbj@yg##Hehf=XbwIkH7p?^oGrm*|nQPi`}WG(Xl}s zqA)O)qzv)20hs|y6+@Uo z7m)ay*PQp9-|_GM=RZsLe*HnpAPjo2N)ACZZ5+-P1d~nF#bOr(DI2ThfjKAcf#VtG zf2}K>X#ZLb^Dt36Z77_j7kjxH4|G9hn};Y)0c$9&sd~j4z%cm*=Ehp`Q>1iz$k@-U zf!C~;HOHh)-KuA}sEq3fHzCJDz+9u`jO8Qjzp4pcmy^V=l~0$SAS-(EAs}_}W!Niw z#p(#`U_5ygiyXpQ0UXv4By2n1#x!v(vFhysx~T0$?32pPIJDYBP<$4_x`Bmqi^90J zTGCpi+vtLK2x_$gdA%9Meh=`@VXDF#(;%&siGdT?nUSjk;GDD&YGEt07aa9CWBGh% z#9?&6OthjIe`UW?*P{aMws6Ky*x!yk+1wrdyx;gEgd!lP6@Yh2xrHO*5wkJ`GJ^Zr z-+spFzRGy&^xxgZEo&=1m9wd#QPvYH9)eU5uEHbJG&w-kLu-PJ$uEVp7VINv!y04O zZc<~`8s{Ep(UV~vGgdQ0?O|wKp1?-h5qzr@m7zn6QdeN$2dr}gnz!el=T?qo&tDY; zSTpfTxU&ISgM`w)#uv4- ze%&Ze6XVo{V^1#-#z*j$CmY1W2@0XT5rEs(u>rgetQNp(VTUp;Sost-CaipOinGf# zS}ltRoHFkiBhGMuI8YTguVXPdBe-XH9*2m}1oQ9?7`|>b>l1F_vI}6qSvse$a}Azh z`59f^6|8UVWV0HsVdoSxbXTB*@yR)kKZz4$XzwmiE{U!$B|I59T)0Cc{Q!tkdQ88@ z>B;f*;_K8yBW~{zR_Y34L;z53XejMNXS*m0o7F~o@o)SM5^DgXaM#0qEMTmA9pRBf zvF#p$Z&EFw2AFAux1Jr-03IbkH*+Yp2%!K%`7Yj31cM(Lo@cAG-|yMlmB^%eHfRf)uQ_4>V&`W$ekldm&K6as*!$tP9>@0l|B!jKy=u zPQ$nP9MUnywV+jwJtzo>p%X30ei_?lrPyHAaE#J&rGwE!L&!(i0FcMilcZ=a50Lo|I|AqPe-Ymj*}1Sc_+% z0uU$73yi`d#Jro<8b0H>hC4;fGdx4fM+e4^=k8YD?R%S6L1RUsSAU15J+m=JpU%P< zHWYDPywB?kWOti*7X{$i2hW<}Chma%&U$d1`tUr-w^goO2gl|f2m*bcfVv$EsIi6@ zgw|+@vd{V{d*gk+%e{o;nn&Atb0XbC&$%9`3;@(5nVXYS061bLA3TXJP!{5j3!$Tc zU9k_`!+j=B3Z?{RO>ick(&)nPu?D!Zk^8*jtl1pnq2I~i+W2#1zK zfxXZ1x|lc5bv~}E>+ZcOxmxA*_3E`00H z|1^E;%{S9zH=gWtzGB=H6RaZfi`3mdoXAVz8W*6yGfU)i|xTa1LoHGn-bFUBK@3Er z0Fy(g>$==op{MKnZxgc!Ac8RJSW~Wk`?u2f|IXh|6%JF5IHoaz^+|(dQnmESTR%yk z{^bv-k@tik?SVASm@g3aR)8l`o*1yU(CiHuw!YCJ2h_GO#sci(%649wf^|dn;zEc+ z4Wre}E|{%xkByG=)~;ehH@OFj4v>}O`q*XXoK2$3>FzPlF3NqK4ZVkmO>v1OI)wO z3yNx`%uE|r1iuT>&j{-f@kT`eMv1dBHar?Gv*IqazK=T&rWILhoEyt;>>KL@kXGv- z#In)Om<@jKL6{%m!rLLsxx!kQG3{DKpHaqR?1eS$EDM1@C-p%Sa0}>Gt0Vi)AwM8y z>jdj-qtKUf{bR)MVD|%bS>naa;2PkeU|x+kNa7nW8THL6e0e&(5QRFOGnt|nus z%X_b37@i%0p)PG*u04Q!6-I-4%_`TLX)Ma8abGZIWgq;9p z*vACM0P2K|8-h<99oz6cosNRHolInB>cr}uvKKI_<@#@Smf)%#puL~*iHi|a5B zI31gvO7Gu)LXFBkssW6qo3C6-pMLN$rA{c&dkhGG$K0GHYZDqN^y2|Qc-Sscpiw6& z;hf|V-K&=w&;Fy8G)l>kQ9@jfQ9`mPI}PXqYD+sk${s^@1+566FC*gn2Pv?l<$Z`TnM$tPx(*)ioStbp$Z- zLjSI%tO-z>VDAPgTCE-^*k&o$7!_3YmG zo!<+L(T>cuyGvNBI8zbZ4Bs#dwgKSr8IG?&l9OE_qzPrp89>7MTJY9~N9;alIxvFk z054MRs|5Q$LEsr-TD~Wg0&Y{WbbhiRpD7S)t~&Sej0h!RJ;egfJ-{EI3-}$U*MlNr z?}G+BC;pu?5797VWE&w9&cXeVw?!*8Xsl|i0bhRZdYcjLdm>I7@5?T+r@RGZ&Vq3j z76*utXFg9x=wczP|Kq-lcH4_-Y&@5+g0>S6W@M-~Ywcz8nM_%`( z@iYgjFF#!%C=vk2Mx39J4&2v~{@y?R$7$pJx6-fw^ewX#7xDr(goV1e3W&JmOIjQD!c(0TyAJ#37E zNh_2wZN^&d!+cmN;`tzI{NsaIQ3fzATf(BkMz1oK-5oT&iMiB0e>2VEE-kRuZM5S# z0?a%_ORJ*M8pLS%6@>z{4iMN$-BdlVQKWZwd6De8zBK>pZQ7yDrqxwyL-~9JkRV|1 z%dgT3b=y`y{~|4X{!z+d*&2qqjrSs8kEsMstbm!7c>o#`F@%pza|IY$rHubF&R~ys z*<4)#tdNuVE#CkMv6g@+0D#19ky2ZsRS=?@+AWZq<07p%JqS68TZV1C5`31Y7+DD+80cu}YnWS+<&3mwx;U2UuAeV-Q#XrFyIy?37Yod$^%{N zJ^X7KiY{o&{*+7w(B}{kv}iXGa16Z~G9X~F+<~>oEE;@>``@C*j)}JuV+Jwt>;h-p zvm=0?9Y%;UH?4n`AF&*d>$tRS4MU%$MTUb7h}JsmeH~hgVDKhoRB;`eEFu^T4U+d- z0wa%QxI_?pqk9-rkvYy!-VDgQGlzU91SuucvTW2E9?mG7;6 zEtH&d=I1&ItO^1v#$tdw%fS~yz%_@+bg4!g85o?FAQcdq{t3cO@K<4-wVK)TPOEhu z*BBg?^TT1`I+JlMnPclWpKXswCoPjR(LvW^Hy|oYf(K6s5VZ$LQgfHF1@PlqOU!6# za({g8GDV}Ka4GLRqN2ZTwWbHU(rZ^{$b=;BP(a7rOek=^CNhA{0diPsp>vCpy1u(8 zo%W-xBcRL-wH=qW87(uS5-Clt(VMggP&t4OI*D@`p_uT^X<{#7$eZgMLE~!1X5e}- z6}}zt0Kua)s=J>d!-PAc1)nwlU{ol`ycY(pMRE)UO9isdU@vez*;=ZX!sH8I-j7L5y^!_g*{NWs-s2i#VkFgVkExN4d*?}Th#ohbC$7t** za>i>NqG12kJDcVQ$4hZbQ(wryOw6}je zYMm80dlpvTh8I94&?T|RocCo?{kH+()-a5^l!733Ra=_hs7Fz-J!0D64Xt=%1Wy*o z%>d-i;jxGCGbN^iQ$Ad)?n8ehQ~{ujm%K{ha4q-|$^u;oXv`$^blc{z!hW8xPtuOp zi!4G(WFzD)DhdJ?_r;PJDw&5^7*#S9neFUR27qO#kmW)IWD0ekp)d`%E&Qoz_0mFT zmZSL8QWBgYe(~ATfXOmilq^dWaap|Ml)Y`Y6bE01G>nXqjNo}hj!70AoHz$sT78y4 zLu-8=`PGz7kdAA^*;5w!E`Qcy$iZUVfLg}qePJ!~I-b)UThr}lQQr)_O$(oHPHC6R z*h@8KC-?vWKmbWZK~yX~&z`;Z^f4LCgUlZ=<^Y+y{8#`Xz!3<_7qVD-m60vP+&%|o z$MEjX{-GFy*9qbrk87b~6I})hR+=DH(u1a_Qz4}H+Uv&Xah!BhNIFnJ0;ZQ==eFkqiG3J}J7bagXbcn>kq zDBV6g=7t3@Dsl)KkUVHDA)8_^xK=q8wcl7LN9v%%YK!8cQY%mMxv^FJSsF6LNWZD; zBZko;cY)DCk5BL|*NOYgP_59h!@0>iLM9Hd0j+GFK*(B6#=>ak*K``S$ar>sbTZYk zrnTrMn3_+$6jQ9bSsd`lGTYQABaXlF(Yn}vnJGI1?#?oK-vr(QOZ8HAwjB{kWWJ>lYw zjetWy941ge_qdVNkQB;T5M5$c%lH~imlcGj*=8mr`12X}ftzA6q+S7t1JKoX!NW4# ztEoByQaR9hv_`ECUoEY~VKPWuAZt3mu2t)mX{8fnDOlYMgBK-I`%|`oAVvUggA}lA z;3BAU{|d29N0{q}U$}z{XD;>8y!jkWl>G$oF-}?#?X9gxg>|*B;SuU2(O`P}(F6A9 z3~LZYFV`r6LfX7BAzwbEJ0Z+5;_7=?c?oXp)7(MT?lleou)((fw?|IG2Rbiyp4JgV-y>S8c~- z>!<>yMpCYawr1mLnheQ#GQ~dm(SKphThkJUr-4PJmCUjsg!17&pa_i(&KXQc*EPHe z0#;+irNZy7aR46Z(?b+iAef1XEGD>^1#H|yQ=?K;G_86pW4O86VW<{_EkGkcc|Z$- z7y)e@q;Ulv41qp91v#GVt~U@Afaq9~Nh z9ChVvAC-ye1Bk-7_KD|_R(0ohLQlO2S598MQy2X?%>R_{%U7?bb`nYITvsM7{t%{N z@cceM=ekD#z%VP`T?cFlSkDB{tevjlLtK09jMXf?0R;GY3m}Cfa*UvGPqeNnC3K&d zIVo_Hn`r$ED>ZX$o&(MX<2N?dQVC^%D=XtKSbscg7Lqnz4vmmK6FFq5C8!(RL?_EI z1Y_k6txsmvY7KX-+(#2sYz%Mgt(^vHeCEGqj|+^=a_xogEIZ(S=w9}3Rpz-vHgi== zE1wTx>FVQKVQWjgFiHUPU>s~Lbtgek!F?FiG&e#kpDZdFUPsOug|IlKbroH<@YXhkL+ z0s@bFM$^obh~?vho0H*HYXY2oh2X>R%!Jw%59Zo^mzcSt0n{v$wejF6Y=-cyto z6cE?aNj1GR5h5nfW?_qr)FL`9tpLEQ=6WQi_1q0$l&!^Tt zt&~~|kG1|G?*v?Cp2`pXIZGBZ`YH6woVmJbU||gypJ2&YRE>*Te!B>1FaffMwLfJ~ z>L~AmO~kmmJt&m$56A7gS}w}H&Eq;}gvbCv9YyJt53U>M$PN$S zTYrr?ITnGI3R%Rr^Jwc03b>!sin7e-1_(x`2&{|+_sdWwpQY^3dTEP8tzIlC>=D_b zX2d&SVqVQR7=iKXQK+LVghw*bd5SrAac-ag-CNkE0J=U;Iy7uT;U-v=Cq!H?Gzj%^ zn_vwY!gJ6K`Cc?K?A(T(FSoj|=1>#_HAjgT{DmXcqW*~KwlUN)NlGY<#b;HEk8IWof@+{*D zU$G#Gg;bcklP9d0>H3f_i-Bnq%G$x@SV90Dvtf#%V|HF|w+GQRb+NVPXl>}?FqvKC z{5v6LE26iep*8X_FAkQy47+e4RES+@L1RRIv#?>qgSlZ9!*XLSJWZjU%x9RU%e`n@ z3Jd{{nq86kmH==SHel@Aucs-3cH00q4PC%8BbkEZ#9h|FTsR98&EW#x-B_fb>%WFc zouqGl=er@m-u?NH(hfp<_WH}|9%v9 z|7IW#tGL@XN^sgDgJbWF2MjYg1J!MsCw1SpP}Uzas8=v5fmvL119ws%?m-K_ni*H+ z8Eq8q{Kofxle$i1ih*ztiz!cG+bkfd9p<&QyqX^4o{+(fU%!*aZoi%;CMVOv2XCd) z#xi@Lmg0ey4)Oc*E^7sG)|$+my_J>g{xZ?Vqa+Yf@z5piTqJ{20FR1=*@q@3WO=TA zR>{RApqhLSGUO&vCahYo_HO3j8LhjjT0sHKW!$8M1F;qW4dyK(5dQXIy(w#$A}F9v(E+%Dx_vKyr!_)W$xjMb2W` zY~@U4*ee6TSxguxV1>QaI@Hll?JLIMx$<8v?7A#D=7ImFG$C4b&6zbiFer$23>$DHoL@#H-GA$;06p3lvh7L?BvY)Bk;tFwg zBhYsqQ(`OD|5~Z3ASj5W(GE)3cy5=si%~*H#?rxhcN!?qz5z?32(fN>fq>c77hM-= z5jy#ZHgk3fupj6cbU8LMk#@HzL9wtMNd$D9i#4~do;+=bwH&_rDt$yL`>{(5F|2NK zXb{Ujpd4$fUJLs}3!H81WnzFaWyo|q0=RFtj>78L#hSE{7;%H{2M<@D(z5Ci#k)7s z*u)6@1Yj#-sRWfkd?r&17YC3fsY^45+G6lRVtRZ#u5R_~1+)9Qp3f6E0`gxgB zEnTcrJ8P1o)KO_;54Sx672vLW^B5j`#JDIz5H%bJ@`MDy8;=l9)?zy)ZtqyHMVX?@ zgmdif0U)u;Y;Kc@58>EzS1ZoIz-T%sp|li6crMl*tie0fn#@p^WrMOnLwGYJn7+X|*Cpo} zXd~{_vv2&8u0+dyxbK3xys@*;u;0fh2!c}2n;@bU3vK{yb|eSATVKuYEEm*(3tHza z=k$Vbz1QjGi8w9a$$lL2K0%RZ!44D3B*BRMNw6m{GnTw`zRUZ)uKAhL)pKZks@H23 zv!)}=jk6{HVr@9D7tF=8>d#smJ(tQupX;~J^Y72yr~*;JS$xj?wa&>4qu4PZ%(c2+4 z2;`}ZfhRJS4ol{6{(Qb?-XsUlqzY`5cM`GTs;)>n3OuKM1P5*V=r7JMZH* z_5%-%c!Y73N>l~VYca{B{c>Z%3c%*Llm>U!dAVl#FCMR^8&|#$_@FQ!#To0z#V8L^ zfLU~vx6@Dl<^Pt7wB+iAx%Hqe4FMuL+XrJ1#u!1U#(fOViSdvd6>xRr=w zsR1%e9P|qgNGn=W;1rjW=ZMTh`;s7GKng#$2g@LtTWnZAb1;EJ*Um8}Vgi!L(jq_^ zp;;njeh2|#){ixOEDvDYq#1zL00&nBoSPaCW>UfRQsdC?W3Ak&EA%L^d-V#M@MyYo z=NqXPi{?IB?LMaVHvBs>)e8jvDW6ReNM3n#FMam@JL&0zyF9m^{{BDuhs65S(tr7n zf5x6R((nJ>KZ4obAc%XDgN=ob0-@y(f0^F@(GLi`ZpTH6RS=CvE$2cY14tYs>q%3% zjooeOBy%xP`B3*6bCF(}nRGBW7FhZ=DD{A1%LiyCi?s7GJ-?f$4nuP3+%)|PC0?#w{#cKVF8{27Nzgf2|V4~yTowjtcGl>i(=!dareXFsi z*19`nPeukPW57tuFhk<95jLC;Ktm^h;)vEhYL|O;7%_9QXlpy`)`_)5L3!o!Trk^T zyz@bNxUhlHpeSg*9|1;O3rdRa3i}h6h!Hhj2G)zFxeXYnE*@u9SD~%78dx}V-&eMG z0G=omoF$V3Tno>S_2Z23JjT@+YYP+U23*KIv>4l+LFQlOeR?YdXts`$=?bR#PsFJi zGoot;jt*1SI?ddyVaAi1E+qDeebLgcOH$XNR#EOi@Is5+FI^>@ln@X0dj~L}3sfL5Ls<^HR~WxJ2Kdo>uvgp-kfkNfzilB@weabZ zZErmeAzLfe(g5BB1#O9Q(2w=px=c?g5Wa)UR>9H5y2?A$?&V=x|5r#FD6;+?{LRuZ zsCsD;fKli7nOnEh?|=7q(jjN~?YDlKK6|(TIPIq=YB^{pP6E%czq6%`dI2TFH(y9^ z|M)LRAV3g6Gss|+2+V3El4al{#sZazl{Cb`nu~jYB{OhgZCKi%N%paQs3YB5T}{99 z2ji(5e!1}Aewx6$(l$JY)vJ(Rn!T2mKYb5tBVicZR3@O2{m2;QifEDbWa*1^eE_S_ zDf>b`(jLIHf#;(K8hU}4vOe~0b{s{HH9ABaYGc!3MTCQ(WLOvp11I(Vo5%A=Gk92L)uKu(6%!#=J&eF*;;<9Fx*+Sghm4->rf zQ8amd@d+{ufsb`kOHYG&7zb!OC|#2#-UzgG=NqFXQ1jgS8e^?>ohsa|Q|H+VPlC^2 zoSZP8fdavKbz*|HsGa6S{Ni(EzcGRV=t5SgC=q*!3$%nKvyUXX3oSy7sp*;pNJjLA*3LfC%1Om|fmi!o_T9Fyn<=(J;+)-9*iORe>|Y6c%g2U#dfJs9IM3@{Kx zs*wpOZ2Vno*k6%{F)@kF_zAau9UyWBDz&6Sg+)=2T5>>F*9nI!3lRk}hh{m>u^?t9 zvh^%1!Oa{TvlJ~b+Lj^HNP)=z6w-TJ#GUbuI&RE1f+Wo->tF)~4aRs}5Nl%4^sT9> zbU+5KHLw(TAKrb)#&OU9z!p*Mx_lM4$gR{#Q1ltXrFVP+0h!WMguTLb9zbKj_3q-+ zl+)t4zny;ipZ+8*(&=E71Jt9{v1cUx^d}@KV9oi1fAEhuFD+@6pys8YzeVxQzI3p* zoZk7r|2Z)$mGm01KsJSsmP>${K<^N;Rsk7hnOMWf&V&JIU9tRwa4DD%ODqs-g&wrZ z*}&VZz$$B+>7Sy;5*fE>h?&W`bnTUI)0dE-eq0rAKUzuyGcTt3%h%|%QAnlDVg%Vo zKStisNl^YD|4;vOD$*PE&;QN;l~%s~DD`8fyFxET-RF%9m>7Fttd!c7#4BA|C0<4h z^#zD9<=+?x52BgMGQ-fQ*ppsppo@br4zugy;LOfUQm(-e0q28748FOG(A!0FJlfu) zOarcL!AtO;aN>v6)n?`0f?t#p<4=rb)Z(T)&e~&UYgy#B0LWSgP+OwGdUhFp?;af> zh67N?XJ@ojs$pGejEv)M>QB?eHRZ9&8lzN25%^?jgP`dWG%&z7l+8KV<9Dr8yBk|D z?q-0i{VgrQqcy8QJVjd#O9b;!sWJKJ3}7My()BC|G;{#A5gHv6=S-;eV&P(5#^zbZ z!4^s3mf@MwlH6D!=-&}FA^6+}UF0%upJ;5S3|BfaOYNARv@#*%GSzGygW(M_BEMu> zCO|}uIL@4wbj!GOv{6fp^D_@UH2`tCWfgvT&b}o%wn@@~OFj#Kz>UV4dF}dZSU(bA z?H-w)o9S=9d57)_B$42{s}_&2wBfGjI$ha_G6MI<)6?}G-g(GgLg$QYijV`>KzcC_ z)r{Fatm$E4lm-~HB?J0Vey(4g28f{va~2PHv55?QChiR^$QGqYwht_E1jj=- znFXz8FMxFob(wCXj2*h>@Qp*x)ftw0nYR{>=?TgJabAA-|G7VdN2 zr&ZH4!Ab;JX(3@`h!P$T09zbV3Q7-03{@xPcB;i~EU$I=6YB_F*lRjOID6WUHL^vp z`pOIGgMa_SbnDvH$YQ^G>6P@sd%sE@D2uQB&fiQqln#OC2#HX8gk+qdWSx+;+{QRY zOg_QdTgMYHOhfV(GPIoM8nxRXhBO37yw;D$2DSPeesE4l1lv4mA?|?B53)Zem=n#W ztw5ivgasU7d23J_p#z1v#5|=f`(ih6hw9eSinz0}o^CMS<$X$t0Q&ZKHbZf4$3vi^ z>6y`GtWb}_vy5dQ@NK7=E)?&uXvEHntKf6c)j8*?lXKL;ct*P5_581e`)ti*7n4KI zbS_gz3G0e@x(s`wC8rFi9vB}8x|X(e2bz#@K}|E0e?RxA}xG9Gk+D@dzO&E zv9f5hVMT&~1B@A4j~}G#uYZ&GYHew4UYQFJ@WUW&HOJ0#{i|+C zAK;53_KS%g;gT9fK(|3KyX#ckhv@dvnh$y4wVT(|(l36T*5Ce7>Z|RMh7ZGsK=u(- zIvkD#Kxl!wP&5P&lKFTEO}2y4>R$AaYNmCakOdoaz!ch1LOf*Bo-&j;Oi|05d z1)%~+gnk6W4I#y*D)b5nacfu&n2m;Jnw2ng55N$2v5d&{KVMkFm5k4xH8dlzeFP>< zw&CFX^)!9s#gwJ%#4(5P1OelmcX1)GwP|ReA3_Ew5-+&<@L}3tX04YODE&bE0Km2j zroXkdO%KwEbmR6bSnvqEWeh8e4@?XUF(dA&!aXO2p42fjm!>Gn+&OwFmZ#OFa^PJ{k`|zcJTq4J*j}(dwp#s&0oI}Abt7CqqM*DHBF*dh|>}bL%^)L z2f%h(x+Zl=)ggZ4LM+2!fnm0ZFChfP%-;)GF=RqAM(4C_HnePaIc!Y$Qe!#oq zvAcx_PasuT%tjdgb3Jbwf0m$IM7y}jNex7*%bBY)bR>s3`sT$kpTtF|< zs7>E>qpO6p02p()(dxQ^Si`QV8!&z{qtU>}0d6@1v^i@BfTkL*@tt)93>smU!`?&X za&93`i8gfrmnv2OOH-U-Su$I*i+SljgeWPWVel~kiapW_Y0Mm3jv!(`1X40L#@55$ zX#Fu>%$jp*qFUyPfX)D9jK#RT0LrYXW3vX8=SIeL0#moFg$&ZUMuG*)6(EhZl))Jv z%J#>uM9n4F$TP&=cQStW1x~~NJzHp&tUVou*t@ngN-WtWTEiR@*Jhv3bpp>}yJ2~d z{_Iz2e4I%UcI7aJ=ivg`?Fi-;1n)?12bM6pJrLWa^>hr&ocwDO*N2+0G_pkNCz-t# zNdGp1wZkIG3fg0sZ>2^&?BM~uO92*)L064DYXCr@8?sIS`zdQ;!h>0|N328iqJ-B8 z__Zzt@M8~jBX>e~#u6D9yTduNwObh;rBW%sGSoxb@~fBbD7tFu<$C(X4}KI0746K? zxFA>!fEEh~fTy*U0BY7l_vr}0W)onsw}eFpYs;h-f4Vx%kW8`m^@k)nkZ6P@JBtPL z96&D}#1`=WNZvX*NWc2oj{wcYB*R;^?h2O2UY`mKIL;GiaOysE%mzl-o_Y%ni(}bBAqELlzO=8JAMOdFuz9vH)=$Xnl zu2cKB(g}}DRjk4MTp@n?u(%#pp;Q<`unP(iicmw9n7IO$unJ)2qDql%z*vc!>gA14 zptEF|M%f_Za%%vg9AK-5O#T-3GmBD*7R9sZcEh)K(J@P{GVenBLLsd$d^W)35(r2pq%JPz7LK|^sfd94#qkY`l- z(-`5HWY~fML6h#v95J;5BmZF8p(1<8FyibR2E?M_dq!5n6aA9sYjuYS@-806SJO2r z761S#2lx^^3Fx%anWcLOAcyJ2xpoX;x#C~lh5@>mlWt}$=l*W5P$jYN%}{yT3IbZ* zl{kX0V=Tu7z_@dZaWEF&S4DopaGPhC^L-3GfYzI1;#_k~c)kf?&3HYPdV316zylQ5uD3hg(wiE^pP$K3 z+&d3gfLiy0Z^55Gc#UWM%&q|Z*0d7xUc5a%!{6nPt~2D$nV~EXhL#+BTf>ihv4`+B z`MknGZ(AF@#S&6%cr)G-o(hf0-1K3qRYMFhXbuLSJ;nNC%bWXaYiVbDn@mHrXFmVV zKmH%F1pR4hK{K5Lob^)kXH6@Dqej>iA)+jrOh4MuI703!+Ep(WqytQi8!$>^in%D%$L_z9=3mf9*P;mLu!4R5_z~}<)CbJg#EmTXdRWSg&5Ss$1%f`~smu_y2di4RQ*p`8#_)feA?Clw#xBi7K& z(Rx_th7{zO+Re1HwU{7CD}$-zM-aHAC}@-Btd*x`rBy(tAR}&t4n*}R#|RvlHiR=c zeI?D_xSiU^CsWJt1lGm@00`D7tN>>oaITjS6@G>$Pvbhd_xM+|*?E#i1_z<5fwb_| z3hUUBmNtm5K`^uc_zMURjuLweI01+uau5t^zA$I#!1y+cC<{70u1q5imP}HsVbs=R zE{8Cruoy$$-7x7ngklfZbg=K;Fmu0l+1F^{!#ZeX1nz|k=a7A9VO$<$+fQ|45o@Cm zwcRlUDVD5o{j7YhMOB)xM23AfwZ!FMWG&FNi6|Fn(}&EXOnlkZ*S`Z@U}2ge*#~WE zd1-;T9pWXStvX83xl9l+#$Bq+VPPq5*Wy|$@ayvJm6srt`gp(YZHR*2={IoVSf32V#$SyyPD`3UyTJRk18>l>0DA;Rxw z1%|6spvs^84l9NiG6orwG{!Lso}tA?E1<&6*e+{)ytqI;p7SU_Kxz{RJ#2c_n0)>Tw^$KXcwtgwA%#-DA zE%fijCB?a0*+sB%ouEJ~Y1Ff13<_1_ye^;_!!w+tuG|ds&m+LvLm}ngVX{<7KDzth zE|xpu=?MS$4wkCo#tPRkP3Y4tv-O|xlY-024y9oLK1Hs*eEoJ>!9&mk5ZopXuZ8s> zv@(5t_gBP_9i&TBlp7(=tyQ{XO$t;Gury7U80b9XWte2_(uqNEfW-~g&prsS;ocHd}pKy$3!1?zFjo+yM{QJjxaqEhc5HZ@o;>&`2nSyrC=l_v`9 zBhFa?g*QW)og5h=JNG}p{XQJZQI3v5-3aTPncMJctzm++Ze-ow?ke}83_%n1j9|&5 z1D}8=DAYMo>@aK2cno={5qMv~1sxy=86v2&Fl`HU8a)$rWUBG3QCdoD&&_s4FuvUL zhpGe2Nj}U}n)8-Em9g9xK#unYD1mo4HZyed z#E$yRu-GHNFRbARZ7NHBpMY9#itgcff6y)pI>!Pr6EV-Ogh>jkQ}>2%+ueG2n;0)CVJ8%@{HpVs|n4t-Y;VNWeao7-rmYTByMO<#S zW&;aWFAH;hZUP2tb{Uo^w313`2+aho0PBJRv5IgM$Z74%KrvmI{aaZe3wp|!j{#oA zP0CzC+*(*F_>doO?WTS}+wC{sNbO_O6iZ|>Xr{(=9o7bdyKe#lzW3Mv0M`g^R)EzhsA`wY*LVNJ zzfDhxZ<|JNO%rsVX$66E03m6{k9LD8CJ2YM(W(({F*!J25%9^{*h5XAc7nCZYofge zM0@}vUWaH}2gcLw-}t>WdE-^G;QG=LLTGb$AMl6PSP@Wj7@+|k0psD2!Q49Wk)L1{ z_=|t{FKAiB=fvSu*0$66HXxOyscsOU&Rr%Hbi+-75CzcgLsPICLGT%JkLfvBkY&Gu+8oV{aWTcjG=*Eg>#Ar5eQSZ}$E9><4)-TuuM>K! zVJX`}=W6o=LPL1unfD;ir^E;)hz5z>tHBQ+5!2RuG?!=mZe;6Q)sGMghi>mV@WzI2)}DYQQioPm>|J z=f)uI_yEj?!YENYDSxxSic6OHsi5b1#vmSvS-dqzSi^?LU!Zt7NdyS${cYT|^Yf{W z^=TRxwR+KTUbb=49Ql5bUsyrx^tqU1$S%p2y zKBzI=lo}$8{|aLVz*rupVyUZCU}G|e=g*if6+7eST6vZVlp#F`2$W-tB7!=ec>&OC zAn(qQLxx(kGaqLrm@xiJiNS362J|4WO^7(AUScZ^=Jkk}2yo8*w#*V_a%p^-je0_i ztuek*t^!(jpbg$FF=BFx{KBY*r_3E1UgpsE8@JlH@pcRBF_)kIzZP8n_$d~nZ`5ZbN*G}<$Ie6A6(7eX@S!M zs&PwTdxoK;X_aH{tm*lAv$V+bzn5`uIydX;X%yo)1h0(y;mFct0>LwAH{XZ1sD;Hlf^oR%0=I2k zd#w#LKg`7g99o(9;0OisU|co8);1ZA=$h&J{1mY`xFsP7MXBxqv$(qF$V`L;R`>SN zY1`u0f$gxvpT6IqfjVt%H8Y=2WKBszl7liWFZ*TmedDG zoyAo%-amoWrX#JvSn&|Bal9LsagMmLL0qB_x3MsGkHY}x)3;xKh4G;A5_CM%G1Me* z=FkSAWf4Y}G$O znr8qicu1j@U~GuCwEQp~Z#_xtpTC=iDa>iC3x@g7b}qCWF-~jEX<0ETppPUM0f6x> zYIwFB+J`yVb5;#We&Cr?JL=>x6aYCbFy5+GTK13KgV6v26`Wjn1?D%0&_Bbvr+e4{ z^8?0cMSl+`4~>Im41(@t+q{&uoN_WW`CCF_ACsd($AkroKFd@O0FyFc-5R?TuGZ39 z;emFvIL6$c2k7<^FLZfkHUgCmLnv;nrnSX~X_KJkF(F8_SuNYBe`y@E}Y!gLWp(YmM^E9s=;RLbKW-prRNl zvu<`Gz*3BLMT@#|LQIcGt7=;wMqvP8f+kc}WMI;TmYVV5c|aNU4PhvZIPD-n-BA-p zrcqXOQ!G=}U#pwiaNhuJr8vu4P0N6!V8Dz|!4j?-#%ptV6|4g+4>B$502<~nNDQ1> zw!xatrtTEiBgn+wpCE@3UYTf{wfU?u#!zx&roUKmSjPo5B>unS(ykzLQc zS^qA;&I>oMq)kE>R=3DdeK&t{;XHE4GSSmy-rNrDr z`Jo<^NXtcRv!0t~9IAHlV1c?B!+Df1i$NY$`YLgvv+3=RzD)BkyqKUN57iTnP(T#!P&^hpYwrP!QA$-x@-Vj|_56D0ui*v363%}6 ztSZ^4VR6RDa6k<%U7_+Ovq$$xLNTLNAkhqwfO=7q&fwF=rFor-vvFpTfWcmCEz7d* zY%qAPsjTkr?E>P)nt%jax3(g$hi}2-`JG>53<9+it(nYE#m8r9xe~kqUHO@Rkwb{jEXKWQ zQM8~jjXPp&0uup}{}UKJcfXq1TW76ZivYEN8w);PU_^0U14J>7aIf-v#Ed;3AOE(& zC%n(Z0YP8OA z>#enmkN)ep!V1dR{fc-$cpZ;`Cy-oLL@`H32m>9mb7+N!1;B((+&3QU=sjI{s_ltn!wCPC(@M|%&m@fI4{KHx|~ zcd^Va-2(`@p;i>nw34pv=S)DW!Xb>sAaaA4YaBG)dG&MBXIV%cFJ`2kbG?=kzc{<$ z9?T;UjGt;lv*@7lbT4BzTdE5oXyC6xxW@B!XQ%;dakE)^^8lwbH`tdZ5WoW*VqFp~ ze7C{}rf|F5Cv)%=W_pS>?!0R}<*2s){lE1ZE`-svvapyQuWnU){2>Es6*SUe zK^9&^BWv)kQQV0Q4svyM0r%)mT7UE?-TUa>RNdN)Iz|11{izLY`lXj%O_y%n0XWc- z>8nrDFaG5JNqg(d(fz_C0=qiQ@-0Jd$C_Xuz7%UMHi^$e)vTH3(S%|cPt(E%k;@P{ zh=S2(KTKjJIDB|PVf+B#UI6L*3$)jHGo7Q|9;0bL*~UtNcGa5eXD_g<9cViwYjmAJ zm|AHpj%&R%DLd|AuO;Ya5ks@1Y$?T>dWe~CKYM%$7nJ)yMka8EzwfU+#YK;sf%mRr zc`5=BG87(F7)l48BegDjS7ym3wDgDos77p#e>7vT3!v^HvJA~3z?0~&Xzf&E4{RcS zNYci@{Ebuqz|YTK4Q5$I8$KeKzD7mt(&|#G7b!>$Ep!2-&xto;f}kc`Xn?_}h2}Yf zMvRltY_B~~i%UCFBy&JNY|%vSo?rLm8r0_@ps+1##e>QgzJJ4 zv1c-2v(;pf&NUJpIKxp2g&vXawaG#eW9oBwH@qorBBZKl(Uub#qz&F29T9Yy=tele za<+@Gk#U>B+H1@wAkvKSirORd(Q;*Va;=?O`UIn4E#-5CSRFvCb+3;s>>SLluXB*~ zCzu;ccOBn;g*`If$hEnQWp%)KCI+^NLiiAi(HPCc2O0wiUU&nhoD6)cgEgNbaijzO zQUb6&-rNqlH=FB-z0=($ntAKNrR{5z(C-RdB4RHSW z>zP`B1L;#~ZMNbW{4s-t&zfw4FRe_%4Xt*LOWA7Zi8m|g{g!S82Z9CH%QNfqn&3u2 zq#mdnT1yb`lz9`474|fwuAFn!gD4cSDp6nu8UTGUH|e{m3Hq3wg(@A z^x_%Pqd?CI$&)<;&%gXpjG*+)@Cy(H)2|(qm#@o6c@;}Gtb-HE!a&N+2%;)a=^Zhc^%xmL%el~WN z!2{rM4?88k-Gcz%ZXQ?0hy3iC`t8^QATtiXLQ&^_y=c-X)Wb6o)I8Vl4WFxmK7bN0 zJ=QR^RpANQH4dI-I81|8Kq%Nt#4en`kck=zP^bBE92ci?TR900SAh#!)V06>s;u%J z+Up)btj44*UfGK&Z~#!!29qvOcdHAa=jSFV6ak6?35z)D(lekqQfC>YaUc?flXo$# zVBK6#Sp{Q#W^r?Z1+JeYn7JQUUN^u?t*eZM&jPeH1VI^MwjPy2Z4lzWPMaDnaS9f- zFRfVmnvF5Bj#!P2I0fuv45sbsPd(!|iT%PNfu$-taWz~k`+&?AO!l^O8JnAddFc|O z(R(j}(7bbWs0RUoyAdGq=*wTH(#k4*-0r8X)m4CHo#5U}WRcCM>D#wyZF4EjOiiWb zFTY41{_xLI@!l6{7;W+r>&5JYXzT>x0pdu-zuhoivrrU*&BM%^u%W;zh*>Lx#fn)N zzAX4zh0z>Q%vC^wF+KH+&8OL$cT(pV2?>+)sTFIX0oUgqZVq3ImNl54C1p+^D2uad zkn_!Xn5Rw3euCHoFTb&>(E~)_5xqVvTWMz)<6J z_mYF;xMXGud}Fgb+S1lJL62JLpl7VL2mwID^dL0I&n;LG8^kb`D9wNgFYet&Q!YVY zMb5(oT3-gvt1xM%*vnL~pr9=abm|<0YA{3@n}E^cqa>2hYD9Cn-6M8UIICD?2BzlI z=w)KE28JnEItD!!$Sy0T^(UXC+WM2!c}(dDEV_9P+ZcjDfnioA zx3CPPLad30N1oY=OQudOFv0xf#GD_nenqnFuwt=>x{zzo9$OO68Ngza!-99h{mW}h zoExh9<2te|+oXG}?;zQg=d)JOaTOk;#lzXjSZl1GdtscJ`%`AWF3n#`6La%vaSgEe@NtxT z(M7BBp~9xc!!jR0Ue>&sQH{ibcVb}y_{jrJT=;Zh0bsxxfhL|{5!}IgKK|Ob;7{}E z-rdiM3GARq_uVwfvpZQ9^#*BBK~^S9_JkOrEOW4-z1gT4tftwK*;L@0;#ph|Pp|H< zz5p`92#Up(bW++(uid$l@-=`V8PCTAS)XCqlSU2ch;Ap$-OPUz(9Td=bP3zq?dv7@ zf*gtSr?vA$E0qEh9+Cqb8ULyf(lW~m5JQR;1Aa3MAFq79nl8HM7&)O0{j{&z=WGEq z3(&0{7V@l_F4=Nm< zM;FxN>l{gQFaBoQefvv1H~@My`4be1L)Oygx{i9E(5}+R)C6D*4+8Fci%d5QzVd7s zmD!XEb%Blwr-I!wC&oIJu|AswARsVHTmCLnQrY33_-_73nax{yw{&EGPZJ}|__TY2 z<)sQ(nM{}{$RC+wzyltlal&u^r#v+v9wviKVE%zg-Y5pwGi3@ly}2eY9Wr+wUWnMdL@!0*a{zNf==*5#v|9% zJ?o7=M+=r!BwPdIPyLf%_y0eJ=VJ*F2H(=^uB_0=phb{J@;nqiWJhGC4THq|Zkhqpfba;TQxL)= zi&)?eSsdFXDLlrB&FBT>I7{0htx-1n1h)#H!7>Fl9T%;R5;I|K zAQ61b3*fW~g58Jlo2@rEJBQnA9N}$WLj*yA?7^4cB;#^2T_C&`HZ~(QyOVT#iLwMy zoow!MyCX{3Lz)?$1*B=~oDQGSL8&*d}pb87B$wVhqu{6k^sr%gX@|WOT6y z1sIqmH+BFt$1qLl!|b~Pe=~8!&L09dw_Tc&p@I1$M!-F^SL__tnk>XnEN-I}6F5yV+Mm(6BO#)nW)mbqFmxWar6VAS^4Wk{?U;ETN#Te=`a8XYid${K*i%=z=$V3N2 z5IS))bz*_v0if-%Ux!9hM^g=<^uyNZz-e~ob(V<*E`sOA}m zz&iw`;EpvNK*1UYY}sxOQp1G;VItt;)Uta306+jqL_t*VK6x50?Mo<#m&S)_kd9Ia zKssk11z;PtMS>Yzo|sJiFtZWTA+x#jv_*#zE3-$E0E-lf0ssXqrnWaTvFS_=oHf&> zX}OLaNOOCO5D_dgPgf|`3wY}ppG$9i>s8hi`YdfQ#-6BS_F!!l1+_*7Je^{lfCdWebsGwit<^?& z_AwSr1%k;ogRJ2QLV5rI=e}lP+M6VYRaui3c$)jTh=uf}-~4Yd8+WC*-}+g4}9i~7Xak*^z8*5isB-}|A6)E005~#w{ynrC;h^=ucgBDHR3C=DANDlN>Uxrb__4hV$HHk$?)V%T3KBQa6NnVCW;%em{{l)jPemKiMV6q z-*iu()AdGSYBG{%%qZE37k(qP^bMw!^I_s}1$-!f5U=gX%sRCtS3CSlYmn|hWt3+u z$J~w4${sg)wnAI0pIPg+r81D;dT+*=X(8%hZM2qIi%z~PSc0Q-=IBw)-U<}LOTruc zn;-*mjsQm~VC|ZgB-V^M84Fv;4sxYmO|T;X5lBQF8NB6)acAL8D)oFw`^PcZ7FOK& zl2_uggBDe|0wh6OCS7>Bil;^t>C(`fcm({$EOnDJ%tXClZ<_I3Ky~DW>&;O^ylQ*_ z_;I~_$^?*h&c4@bxiV~qAqT%#apraL{%^2_^twqG$`&URS*;0IpM z1kD_1%KVrgf3MVNIIv z7IbNRaNV2Ng|(6y2}i>U2;g!*EKy|aw9iojK}&?iIJvCNT!qtFd?-@`EC^bO=EEF*ZQ$n6nmc zF14=)ht=iWg^~Dq24q^o-)`{U%-BR65F3))VAp`hT>t`Nia>N0lItz#Dp6>H*CZTU z>A?9(kfzovl65uJ!dd-h!)FH8&gT<-Lo|^l2pyp_fD32^G+KoDS!Q5>_ z_uidn4Tv9uYJFcP2Yhm3JlYyLmmFH({7WxUShSJeySJD!nBXU0{zl4E_oa+>n1$eS zHe-hn`&qaye)gaqqNx{PXfl*m7~t;Xhv`TE$Nx&g!&(SaGopq89^d%xZ>1Zre*=pS zZmHBqyvlV>;9O1ef9p+IuTt?HmcZvO#7S z)>hNa&yIOE0J1`xt#+QLOX_TAjY60E=~o}Um1=8?sUL!w%8=xdBldv>HZb1|T^pE7 z@N6MKnmjYciLo_}P)#1ACh;rl0o*KJ+67UXu%e5y#JU-8);!o)+<1;aL>e%AbC4L4 zaTuuOF|3tj<7u@Kfr%EvJ*-m3B}rQxNe-n4zZnbGj}=GYWz+D`v<+CD~FHgHu}u%&Kn^CNTXV665{wR}aL-w(LPj5e90^wE#I$fN_UHpXU7U~Szk zAEt+&ewr@0F9(n+lKDIU@Eo9wiEIA*zxTUQYinU?A>DiQIE|8de1)Wp(dkLb)_hI5 zhc!xDP^y7_-=<)&7gOUjkOvYslYI`kXutv z3)Bld|MtvuT6(%jSstvaoaJ4TmLTOcj@7k`y_=(7a$?3KtWg_#>{Z`Hn!59H8lSO%FnxsAmQz1_ z(t+?^gWilY>gjJ!Ys9P_a_$Td5kY&d4PQ8c(T4RC9@9hIYm}Q|4W?1>E?v5oK6>~h zeR6*VK}!M-E~-<)DcS)ChK|$`c#~uk8ea$Z22?fhB=n>{6c)?C)KE0G;GahgGLiw= z_L?j*rxNRUPgsLn#KWBf`0U-RqH8z=9>$+9EE!os}=tuzy1I#9xO57sI7GBZ8F8T^bh&DkcwpuzVU zVAO)M1;Anv_p|)EopK14_5rba9D;`mQVcHFigFBX9Rm>05Q;~Ial9^_X=izx)`M5E zZed|ZxVD=Z0k5$uhfvw=z5K_gc zFflV9@iC916 zf&9R8uGL+Jq0*+%KLuo!*|TutDt!Gb_bM|i7uJefAfCmV2!tTL_`6KR_nom{4QNIX zrokWp3+KWNSow}$0wOJMHel~0q(!ToabT@jG6is2`RrdD9ex;(g}AjuN-xgAuh=8r z?|hYWjc3+ouM4Xt?~bqo-engP0iDXDbB*s9yZ1f=o$ftfu8ZSz+(5gKpT-NSxI16> z!+8dfat-2X8c>=_wId6QAg_@Yw5(dd+2^^x{>?ES*Z0ykN(wE9kSu=Ak33i47NCuV zRf$E&#<;91VCgi!V$S^Qbyi?-KTKW<4+{@+zje`XVOTSqNGr6h#e9|_JlRtKT0o`0 ztQrL@LEW740ujP$0|Jy10#XggHb0_Bt!tWvyU;3hL$XC_80(g8m5j;B3gpm=E!kk{ zljnl+T$8H<E^h|G6=GTFTMuw z^rrmSOnUJ*ew+S)fQ$?vkvcSH2-^Ooo51z$<>jDNF0 z0gdEoG7YmCLl9km^|fffQb8DRt*)hae)d+Hn!lW`zdWDTu;4uT;GMMh^=D}wf4^M< za;%@kpw&jVnR~6t`i$%>wv$5ydRH*A{`>_H$DX*B9740*4!Orn*b}WyF#HBp%?AObTAYKtoyye!L^N)d%R#d6a<~+SIp?uvLvzYQ%a+kcry?` zOcbq#8Rglh4%$rhLQ7+~jI_m;i#yRzcN=D5yoU!MiY>Di&Ry+S?UZ3bBOE>ss%CD> zpA-R8x<_=AMp+mRZH4vJI;27Un4ih1N-Y4l48@VDVL+kKTY!H$C%~-sDHwVV$bRzq z2kDG)bRn#}D9JH1h`R+hV;;+<0mWr$P6p3-t@mV^mI4RSI&GZ6uo~L-&4AZ4z`5+o zH7Kxlqkz*qAh5xD8k<+*y=4V2o6^EOEH7a^Xdro>={a!?EMwt%wZZ_j^wgkvtz;#H z!rMDX0zzmJFW1642awFY&jaWE@M+*Y$2ypbDe)>)- z6BOS$G@P#e_8+1B9%8XtOAkMJkCt~>-(VeA$H`$-uljRIgA^1menVN)C zK^ZpVx5gRL4P_i}9>p|=)#;7j{_XUX*sbl=rDy^5(#&Z3%YXlGqIk9TkCE{y6jKxr z)>qKez-6eVfY-4;TIeZSjAgZf5G_E*y%`Gc66$hBvYGD9m8Inn*dzVJSXqh7f?jn$ z)^PvjIs1bsAvt7$dzl}mxH!of%PXtFpG_Rvq~n5}8v5C{b?By!Rq*=sR2qjT6ab2? zJOkY@Z6a&e1b>s4($?BS%2u!parWls_?_pRvre6~Nn_zrF5!`cl$;PpeMVQCE_n0W zb_srmOBkiqkcKLjIh6*LDf@ayg@4|ol|>7ThJZ?u#3jy37g@|<;e*FksRKAk=b8q5 zp{pPM!QNWG)N1+!H-;tH8iew=I<&x6Qkuqs+bCQhH3h5k(xe=%Ga>kSUw{KnyE3wi zjCU<>f-T3QWlX-=2Cdj6-U8*uL;0OW>1ev_+&3(`Sh}heY0PT^f(Gb2B+Xi=dti?h`{YMy3-oQ=Y6+Wo#yt8#u;M7~d}+jl?Yy z7|g77e&M|2ey8bJ{jY1I=SXX&W0H?wSkgw?<%PhzuuxjB(fO*lwh)@4aSOe|)qr0S zm(MeJC1Z8hcy2t~=QnvsC=|*tl@am~e7|c3sl|NtYUmMhOp%BI|K9{lusOVoYmg(+ z9ix{U6r54ym6xp%z^~wtIm)~YHkW|=A!IY7+FKATE>1i`cIF!}zuOO%HgPFHSTrT1PpR;$+Z=}&%;zI^YOX?tlg z4H9Vl`WxRP?fr6^xp6B!dHg87_sjQ)69Wj~>+WL3D*&ROy!}J6Pw86;L!YCzTplor za@^cJ{xud*0*ZNwO!$ij29_2A>X3sk_?21P4Ir9{v)0+Z;nY2IJzlhB+J!?u6P#$F)r|N*^aKOoycKcc8sbGH#3; zuze0U4APDj?a~0mWUaIl{uNl5%>#lDOHec~W=lQfaS)l0T8ed(`T@)B04)nUY7Mm+ zZ$Fy4F^N40dWFLd+PoINJ+yzQG{8unXW75c!_1EeXf=~2sPPb17>gxpj8n1cIL99A z1!HjS7&ogbHxJDnwu@!5z+T;?ZYaWK?s^of-asgBfBgxG2)o2D?y+ao@dIF;u|C!? zJi}@!s8i!K9?dTm1ZxG#G^1ZCZ&p}qMbrYP%fXmUn{r1&3c{xVZK{1&q30ch!XD?w z?9Vtuj1wN<;&C>f9eVEdGljY?NtuC7r(0k|o=f9n)U;oI^$ir14g&MV|paLCZQnST#fHhcCRz;t|X4h78ESBt)~Rm9A(rU)^4KnE7bOXFkE zkS(|>>1*EM_VnWt9L7p#J-$wqNy{6QIX_hr8*Fz~=%mvK0w(j8+45{+NG-lQI5^09LoZmf4gk5$-msil_RkD=C#PjVS78~@c+xeIN}W@oRi~ zzj8Ts0eCm0bLq?XKS-U>z__)+NHnTL69Z(lS8F?Iuni#yvp4HnkU* zYxdkS922a~U1+TXy0mh@BdXo^Ox&Q<$RItBA4g5gn>=R}K$OLzWIW#kSTY3~dTUx43y#%rtypmD)xSmTrjgu-npFP|k$vo)T4Ti#&N;fMoc{@&*~u*69QYr4M+Twx|$ z3Ge47fsWalAnWE(`3n~ZW(aGQL*-X3Nv4~b8(`ny&Zo6N_iOK@DP;0zOIj0XbYSx{UI-3JIv zBG4CMtoumDFLOM?Y_BE51J(_3=Xkb$Wz(#+mWX@LARgnTJVcJee|1ktP#*F;?gi;c z6fS6!xf`2uP$la~LCmsr;1Zgpc;&+#V&+J|n8dZBEBMo&f0!otd2@3Im+S)O`cjO; zD*8jix6;*1GpUCL?G~r}>Zd=4alVy`2z*Pe{O$kt?~yGvo8Eo@-Sj!`sqGE263K?@ zgBkTf+=qL(r_nI$EVf{|ll5#3tBmOW8KH298-O`0kh6uchCxgaH)1*}x?7j;0^r-= z5FS936;Qic@b%ZeORV0v(h1gNTI_`B>(&#LS#;BLU>z*! zJ+enxb7EO>SD`&KUJqz!y)c;#=xvWN3*uCE&Y0H)OwD+>(4x6VChdV|id<)`Df8n@ z2o&WXG8oUN-6$N}0D{O&wlD8~1qkuHa1Cn#8we$A_T~$zYh)-bqlG_s@&%X zY#0}eK+lZSO)N#XuH6Wbx3sexR)0u{F(I5-({2DMBo=E_M}T*cQC(<-M)4wS0OlT7 zvD{4FVqdapbYePn4Nq}xj`+UaRHgLIR7*8Yosdm;j$q8%4hy=&m3?Yl{^ z*f~!Fd6DjL@#Yw>S(6h4u1XVHa9y9^mC(a6%GVJt$9iEH>!h)g=jo7aQw-l+ zO-vYW#SDe0C#ELDE&0*MAEiiGAOyP=#7qEEXkhBDG5`b*mL^ zN87&W=sPlKDU98T=?P@8}?U;jC);NC+Sn~HVOc?8SLNV@WqXn%CxU@nAh|zk)*$tN_ ztK&UXs03dnEcs>wcf#A8qe@{bR+J8uBKf6X3T9=2iG#+Gc`suItu-ls^=~~1x(T&_ z2{KFq(8BNED$D_V0Q#0@DkA&Z;YZSpak|QlLz5HW_f^2H)+^`8tr>&XGvl@hXY&4B zY(RZek&qkZEon)h!f^pRCzLFCs74Thn!md~S%&G?_u-85i1@6}3h2e(u9q&%kb5q& z?{KZ>$SivmgeKyLr3cFc)v+(W?>&5u3Mp_P1EnHJ@4!7ZEXk>j-8|TIjC`}Ee#z&Y zNEp9RertLi*(d4P?otBd+$Y4csDXq}USNgB=m+x<4$Bqpk$u3O$d;$ax8RN85T}z3 zgmDB?S9>uF#*OebQE>!OIzY5O>&!V3ef%=`9kdl|K(#PQ4@(!%w>PW7d1^|nF#8+~ z)yDPp5KUIxk{R(FYlMYH$QUhWG)8GE;*EDA4;9zc^*}UtLrWeLWs3&xp_@Fi10K~$ep;qT7(iBMvcRzd| zjd4AVU%r&S`)~erxe*d5TW4iO)t@PqW;&|@gOY0MNBhYl0F#%7DwL1I%u_UlNHkHDseEK73*y26NQzTc&=eR@oa*SHQYx7WRZUV`+u4`hp%IO zquT*7j0R!4ho*`bP}+*M~ErH3EBnU2XeYsb=N*Nt-u z6KeS?v1d7+-H*#guvdifNkj)^n;sw-JuxPI?6ZF-Xp`>o0gS@R^d0QsIeS$EXm@b< z3@F!|1^e;Z?K^;vTAFzDIL)n0hqX(_rCVFAQ3liu&}f388M9PIFx4?P<6_LZ zv=DZlHN*I}0lAhye{KLwY_DYUj(ctBf$<|ISLl+!SP=|cCK~Q#*PXp^ z|D^}BXayCP*a^1_8veda8k{;YF%@&SyFepW7)x7>QbVhMcsSjsP1~oq`D-Xj)+Vdw z7SrJPZ2xGU%8sDKL2|9iw1nngW1#5x+A`;QTJGRcrIvCeJt`U5#4qFJHbKO27e3ZhB@eELqkw8|)mw zs*81>Skje^Rg#^^hy_IT5j(nqHSO=-`ygTgbvbtdRP2%K8k-@ni_iklC5rI&U^Uvu zMcG^$NS`k(rfrf?`fq=mY|D%EV8%tpc{(E4{*cVd^77;K$^hj_HWyQEWr9MqC@;8d zi_qr^0LZ;;BmPO((;`;u6$COYG&E*~RVP5X%)BoVM=OavSSA~?Z6Lk&((CEHx88(? zDR}}K}uxI+@n!}Z=m(eQbS`QFAY{^T+3uUOLp zuI6)#GQ-P{puYq5{Q!&i&KWLrfb29@teb=@Z0?cx1t_aE5FP=4UYedsJ*-Cud$PxV z>=24DPLcLw=t?VI7xZ+R-%J&(Y#k(AS-(+_OCOfK`_qrHT$Ta%O~kLZrZP!!J)9Sl z3KZlOgp+65c50^xBdz(m2lFU!hA-qR@G5}aKFVM-VHQ@NKjfiAnxszVv`V&V6@}bb zSq(BOj)J#iEE6mnf)3i}wantLs}hSi=f*P@9vR%|8U(@@IDqaDk!wEXUza) zL!0n7$2fr%VU4$laPVs^OwaBKF)QX^>FfjdH0eMv6J@&qL>g=a74DbxZsm0he@gJ(e1xc?e*)Uh5)wqDi+xqig7M# zwsFYaNK?)0n1DWTgTG^=c%X&%*qVs9GRR&;AA#!*zzyK13mpPCMhVhoQ6sL6Nk|O( z>yL3K8bqpV?9w3CAOPGM+Co@tJRmSctvwn-B~VSP$6VtC*t(=4`N z_6qeH&81H`tj)x{RUkYI>R#y^Nb>~f{_Q{g0m}y&m31um>JAx(H_|0E#u^N=Oq^RI z8j-D@HkW78;sh;^nExqU;>qWqqz7o5{lrN9`TzKDQa@UqZN==5Hu-cqouQ$>G(gl z!Pt)z2)`M3Oz-H8^!m4cnDUelC}WP;C7!8@06vF^-5d{PsX!|lVq0`~TN;F%159vO z02e25LCgkdIonyo4YQvHVA@R(m{3^IZo4soNGo8`jNen3ulrC%V_=s7NLpSqVm7-} zv$0y%AWSSeVQ_#g!px`AHt{_6t?k2QcXRv-?jRT?;I6#A0v(XVz`pJvC=pvLMm@&xmW7#I)p2E0>D45_8E)cd$rf99ZDEk4u z1as$XWOdYFR-8CXRj4Q#@Y_vnV=y(Z6b`fQiif^eT&UWjbQ5zet$#JgL>QAu;GO%1w`e#XhPmhR2On+53T z?y=D3aM2cKoN_Xn75;}++H3XpQaj9YDaYv{VGJ11um+8U0M+VcVhT5i(lPR!UO;1k zQXm!fZ<@{u))lyh^$KAmv>%8RoUp{+|-y4cUjKcpk}x(no9 zV*FbD*0AB6G0_>R<;E4OgwX6G0m+sC90Ebd0*KS;i#GHHo^k2gj zXm(>e10Au{JMRV6!!%jRH>m)+iQc)}N5U%q_PZkht zdSEzDQ1>nZ>8!Go9(?qEy7b+jqy;SD4S-FEi5IsbfIH}#n22(O^52BD+GHPjw%^Xz`w3z!v1P5% zCTJtax#gYku4(n=qOkUxn5Sjazg4JZzV0L3I_{%&Q5Ux0SQmH;ag>H67(eOWG0pf{ zqAZQ`v?_w*ckTTtT_}lNqu3A5z7}W|17;es&MUG#W`w*ius2>)LrEB>_?w9Z9ZEma zYS0e%^&WnfE#vvR%Hc)MiT!tf{L<3Lztfc41M7i z*mY60c4)q~0lUmf<`@8tL)*gZ%p?T2;7=yi2C=uWS>X7v+?Zm^uOM)n{&Q*UVkH9eC^w*ig3RBgnCOH?jlTV9*w$6i4C)@y12BFV12bH(X(Y& zb~JHHY;36O8QDMpraUTyc(fUf&R=(=fm27b>tpr%?klgY{>!nLDG{D>yz)x~*gl}!`7bdMTtPqB1j5y9eVEkB!4 zH?udoH+mVjO{4{s&5TDtommBm>A=cy065Hw`a%b+xq=$13*n{(&Vv_?=wYC?x@v~0 z4ij@UG&D*#4+;X~PTE|W;wM=%Sp3@2kWAi)kO}r(mxEws4>yG|GRB4J3iUvWKTlyU z+YqoWG-JnVfR}nYU=-2PO`a(u_*byL%6bajxXZ9u3|u815W%^#wVs}S{)e;=fIUQ@ zwg6f$61bh?;mthD!nvEPOTo}O(Tr3?jC(OgFAg4a)l^-jxT2P|0)LyBVA%v2g4Sl& z%x(t?rac%)nYn1CJ|g(rdSqsDIWNJRYfL@n*f+tNU(!biArBP$4z&c`sRSsfVjZ~D z`KuKl5$jaXF*t+)o)ZjCHDmS#Ac>GLhYVrdGU#^?3!Wiggc9`MVeE zve{==e@7gJ5b!*k@vCbGSsnt%tWGV5#)GLO7&lWOMym~L0=EfQ^7IC7S^$J`v<8(k z@pSG$p;;}SRorseD{duyKuM z3)kQ|J(S}i8WzoXD=jTeq>Gf=I7TV5v&y0FF8XS|{-d7($ce*S;`x;4sP5p7C-blI zh(reFb$|uSSVEJ3!mR_%cj9if%~>0kzAefe`MS#WtKaxWdh4C{s1SfK#8RR~`r*t} zs-qbB$mm#<6=($%w{%k)hPXmAo)s{5W3d)q4aK#S5R!?h*>ssPv|@>?0Qk=U#!D-k zC@~}KH{A*Vl`H@t8}NoDKllNAYI4#BZ2ktxQS)n-{~`91dv^?vq%Rh>(lptak=@L> z06vjagUb<3rG(4Yh=3dP70rP9flu zo^lG}jj@4(1cl_07IwyO=$A0r>W@+R(d^tb=_A|-_2Ek|e5MTcA*^#Kh zx=Z8n4zKB1k+#f8)=1(iY4MB?aFJsZpw*!$XW$zC#|@vYmDh&122YDh5637h$I^xS ziLrrK_<)PxMuZ^Y??8Y26fr75+>97K|3V5(!b1p?a8Y6M1rB)ldU>EikTEj}g*~5T zvZx*egCjsVe7e}oEX_ecGJK98t|y)jQ8S|e20mXmoK4w{6^pD)Hcjoqnq5}2J|Iry z1Zz<*mGNJAIEAT<;5O~Od=q!$e46;`6B@41Q^a(K zClf48Y}7XOuzW@JG#hxYDZhYdF%PKIr%Vs_U13r-rO`rL|Fqq zA+8jNMJ0Uzk9EnICSM#W?aQg^ms>g9K};rZ>|$U@viSZx4Q4l&Q48e~6y zlky|4rj~&#xJidn1p&2Atc|8_GZ!s<>ES!Tnx)Cu)-d&0EW6Rb*S`ERO@8|Obcm+k zfUwA80U1P^=CF(uU>e}NC~MISHE1ERCYOPtuB)z5vkDC|Hy#pWSJc#s>i^8tqP%7c z!(k$sz!<2g$;Rh4=zC}&`TzoUjCeaWpS4F*RCx!;^6YK`)Q7O}7(2BDxH-qfSh4H_ z!O8ZGv>u>VDvi!sBRw<%#?bV6wi(fV0~b=q;00cz5<4#L918k_^Q@b81l0!Uj6T>&oqtW@+Prxt<((br$BF8nw*+Q3lAnZUsxvHBP`j&L&JcycFISr zr0L0N>L60)hJ93lD`JsUh8YhR8Y**a<~6g&Evl`MTc!hm$r%FxbQL&(UOPfomrxxH zZ4Wn^v3=Vx_BFDj@u0ELFlyL+*>Qv-4@{h}cHzi0vG( zezZ92>Ai`y7W){=*)KfYr0P|!W#EGg zSo?FFg)L&cTCqHufMPqcH{SR`nw!Kj#lD^3)~WD}BFxBub~D4xJ|&i&P9wOi>1^=m z(PLVKt)#9#td{`8HI$_mI-A_0q{_;pdue-SA@#$zx;W!1E0Ajx;1*QvQsjFBo_o}Z z1s9k$IMkjVlij`yc+)~|92X|Qw25-Gh`VxvVx!jT?AGF@6&P2kwBxsoybNG(<@W}{ zGfMEpCajy)<;`@ATWXtcsvIQZ z&PamjfFG^n9{2n(30<-V;mh#s%CiIiu_Vqhe7AvQ5v7{vQ_FRB4mq=&QC<4{f>OYY zl_%`C*EJH)rHk%RBkkaJHy%Iog21>?0&N9+}IHfTQH=lsV{thla+l)@k4Jzr2R${?4HM))uo`?tOkcD-gr>EcnrsrFma$}j1@|DY8_VrE(Guvo3v%SuTH-Vm zd2iQ8aAX|5>o2fWZaY3k&fm_+f4+W=duZ{}I_8(x`t9=tagI}>aqnt?a{m_G3ed9a zSue-QjU7)|H@V2q=3wv@ymNthT)%8pxHs1;4*-bDi=ay%!|#sIamLQ_d*;3txO$G^ zKg?U@Kq*l_qQ4gat zzyT)j3{9E|!#y|<9UN%eTa~RJ1QYU0jZ~LV9bl(SOi(oLcJVok#-EtAXZe8W-^fD^ z5O*9FIXr^cqSzthwEb3LXe|BYC;yzqX-MzA^Lye=C`G^n%yeud?#qV8qoWtoh0B-H zTOa-*eMa4*Jc3RlGRUhPf_T}rb7QHZ+jgQ|+ zzxa2bVp&-a7aX4KIB1bc!}^_J7LnMbl?Dl(F%|)~MTmZQpf9vCcxkwY8kzqSTt(Al z2x_G{g$W&C9m(Nx{J}r_%T&C0E3&orsghkoFdWnXJZzUK*fol4N^n->*7;%}@(JK+ zcjA8f?fbt=_0?rS8gVIn!c4Y0v^ibTGRpz9F-tSpKhj|G9ke;?^=S#HV+@=G?uZtF zrV0E;z&a-YHo!$o?GXtKR$R9hTO^?%XxFw`5B`4bxtlZ~?;+)WJ8e%rBsL4s&LQhX zvp4of-#@Ap@j>PE$R_0U;5o*cu@57U_&p9)KQ8Yh=)zJpK`$^m>ps~(@XFQagF!u< zr1vU-y8#!%;H8`C^;cfPjEz|w%h1%_&(g;a@1gzU(qm375b!9)s8t%)vB<{30{n)y z1E?{3?GRAnAvL>H;3#Mn*wyj6R?;HBlL!~>x}q;9w66xFma2z@jEhplc zj227Kt+nr@OI^-dt0UHh&t|OH4j@=&LRb^)MJ!MmS8=38SXyD`5(1l?G(ry!leZZXKa<;`|A~bin@g5{vk~H-1XI z+DZDS-~BeNauzz;I-&m|ELn_i=wcclzmDg9C;i{Q`en*vVOIIl%BuCT6+!4Z)e^+v zgX6F^oGA;k8a!`JI@y@Px4zR(fk05Qnvq&MmvR^~@G)!J4pEW?c~IT33uaWS8iw(Lew`i6K}uwiw$w zW9lPbu+Y*T{f|$%zSIhzg9i3lgH?FQ8kXvYu95WT|HJ=`TdtCR_3!>y3Oy4O1^o^8 z5JQGJstJqMiJ8m_D=4J3i=cBn0eRM@nKeCyUuB!bsHC#z%GHziN-6z{W$64G$ zQRwdlL?P&(fBsJTjM&@@xG_susX8bF6c!k$a&?O^30%Iq`#ZY_Q_t8X)^8=^PRl3@ zv(RCY{n=v=Zry%9eemCYkp}YcY`PE>iGged=-7&{itjiwa&d>&ZEIm--h8gIy_lwE zmnfQy8o}7>P;^>5hDjC~PLHQ26FuuwJBcIT{L`PN^Ur=m*8u2>byE4!x~~gS?v*Eu zqO0AW$MA*w_tQ3GYcMQ;_+5N;chmT2?GEWgvM2VJVHo%pL-UPjnG5 zM#EsDa4WQ5gE=t700(w2Ib|O_i`I*4LP3?07~F4vUl>dV_ zp8`}Gh{+TT?UP(7HpxTBCXkt38rl6QOgf;nnuI$YS*oI6rxlXl1+Wf8jG_{pfi14* zMgI#h>WXtsW`ShftgA*WL7z&od!u6Q!r76CoANmUBeIBf4qn0YEyUf4jMplvY>9Dz z0ep5OA20z#9OGv};Vst4GaggkZ(ON8bnA#!Mey<{@9#LR0D=(=Plde1{U$0ZloTzR zEmYyKw3*;6C!lgt`lM}`Y!hi0&4~#TAw&%vY)I9zJFW^fok)#8Tpp7m4MrC6Y-p~Tp<^cj110JA6o z1P!dbGp;uzCj)A9bU1zX#YgGC{{64mX!TQD6L`WXG`-hDWL7&bVI?UcM7t0~`uA6GBUw0j4J~O0Mxyw()XF{tCwM!h zKV^x)^4I_6zb0OV7?NXlxDtn|*;ab^)~l)}(_?4weN&tzhqZ3JxUL0|-%~v`?)XX6#V_ z5?$C0X|%0B4v$2BNZeHs#@mVuwiQOQF+Yt4J%?Ksfy5rRAr!U76tOg(LePr{s~rlk zPLQd~-FYxBDE7^ABwCnkD1LW#h69Zz18r#T)c0jlTgU4ZhP_Tgg@vhGp_6cxGgeC<7>sKyp4|YvpS9(T zWAe-FQLR=o6~APH1we{cDVc>`5p;tha=so9bqtB+Wc9Id5(=PFfIHG=*_P>;UzjJ$ zn7V8bqFKlG8*U>eQIJ-~bzSBDYL>RJQ;WQP^LYeDV|x4Dw^3qN0WmG<#;rR9x&v$g z?9c!3j{^)pW=-#Z`dJjm?&qGT1kDx^PzE3$6SGlaP6GQ9`->MovNg3_TA#?IqCo_! zpjSp);0&0tZBYK&Dxiw9zXNE8l`(&qH%e6*cb-C^2@8n9*mmBKAM|we@Ql{<*29@p zgO7|}x{)@{i|Ny~-PAIAGxdx%rW;t`n_36c`YQB=Rd(Rg7{;y@EN_!(poYtB;eP6_ z?WJJ?L-YWaTOD$r+6C5|*9zuoMb7QM9lSTDqj8vk%?{0ASp$OtwTU z+cwI=IRVRB#`ic^`QhR8oge)?z4!KeY2n_x#8(n4#du|=Wq^~`aXYdosO+BNzAsQr z+rRNv65tQ1!cSGljU~38o=IQ(jIjj}FW5%(M4Pd7GMO7#C%ZFfxhOzObtpV$a9h+_ zD|`<$AhYkp8dZZXCnlzFx6p88`J1=X>hzP~8%2a&9$wqSxV0Z? z^*AU)w_5uVHr*;BD1AqSE3C{f5NnD>1#5dZ`#VK^DH1MiVol#B(W-a+rF7@jH`3Gj zC#erFL4_jVj~CbDj0?h}?iq5T8E{{Rkg67lLu@!qum0IzkP+RG7Tg)t$TdK%DoVG1GBQzmmYoUGngO&-k5vW-;K_Ty%GNUvOID4t3$p?C@ ztx>5B)}074tCofe@6WQW!%0hUUtvO%CF4|k5PpE5@IX%__7 z>eTtkiIFLA;!vnz_RJN)1TYp*nmN2ncC^{3yR3^=`3UTnsrtnphUbEF^}(D0002M$ zNklD4~;j`se}m)}~|A5bx19Z|wCyLOUBsvyZ+?AAR~^`ps|F5ZY)Z3QeA? z5c7UH4oaes;>e{@+6!HUDbQbZa}|w{1zVj=y|j>NJHh-;mWSmuE^@dO@JE>P6*y5F z2*S&OkR#~=;t#6z?Aoi=~_I*2P=N4t4_QoBW+!oIZR^1T z5iOt~BEhV#a1GA&5T9U0&}ux6@V*hSR%AUIiNhhV6JtKX*M`k7htcRYnvmd>g zR+bh+8{zoUiU^=ZKS!BDfw>&QbRB0A24D#r8;N^g_fHKtz9bB`LAtiK4~ibZ1C11V zE7CX8LuGc<4A1$q<@j=`-(1Stm(yfeE>v0chS3yTmM=A~0LDHZbo4EGX7Q8$c_x%#X^277&@SHe9WU z$DBViJlAOS3!o$?v0FW;)q~g+-X_yGp;dbZ;3yU&<%6)uZBct}pYdB%+U_)GSjTEu%!{Mf z)AxSxlXOODmIX=>Y3c08ijw30RP}*&pbgH4OxwPd5CN7)EUpM)%g`ujBCd}2LojlE z6e35!Xw?=pNj%b$ZOBZ}+QdS;?f4l-Gj!I){+}440&VD~Ufo)S=IvO5pk$3aGxb;z z_DR^1XKVGo@cg$@j%M%+(=%z}llRk4Z(d1TOY^Kb`~Z60fp+9+t=vbet8UL4@rH;d zEMdD*7N(S=`v4jK)2k@#fWs1CadwOHOgCR+F9pMx&r^B%Lhe{kFj(qq)1Un%?R{EXadNLj6z94!EMBKK1jEh%F0t* zuP8mWwoo4A9hL=Apq9uiW_bWPnNUALIXcAJaZLFXSa;^0a=isO{4}9Kka8e6mrr7i zD;u}VUOfBq95W2Q6x{45KP~F3$tKFouxA2`9P4GK`qWG1miTm7Rvb)BtL$ zvvXOIWB9B5M$nnL=j1)a+rj$Y8LPy&Ea#Dx&hdA^8~jBuBF_u( z#yabIZ6a|&;3LQla0-|);i3Q-6-en0eYOuiLjWJ(l*M#hhB}GgKz6~u4)7%Kl<*vB z6cW4%Y_i37k-oN?v5wiiwBUm=*h8&;hCNt7xE*DN0W&7qm4H}Ync)n_BgnE;RMhn2 zUV0#noi)KC_%t6BsJfp%-?>?cIdi89@v^-NpvAoff*M*(&Wf11uiwvNG2w@-yVr(v z<9d}NKf`Y^wE&>J?o6Xh3qO5NqlV{Qxgdb|K6)~9kvYmJ47AhNs%EGb;HP|$7rOYG zZf*78Cb314wrpmerbC5+=r_-28`+5xj6)6fkiRX1U@Mw>2*MtT)@Ks{uoPj@r|1lw_)vPM4OkKc-3Fyl^dANvxAq+R8ne&PeZODJn3X(%tP_HFr$70SmR*n2F&Sou zyDPYNN7Ijg_GgiuvbeaE7G~$uIsn6(N%kHzV4Mx&FcZ7MxMgYrCtY$?>cAagYCp$? zE)jZ2WZ3OI)6ZEBM1YF-3Z}+C%qxH!>#qC3!)s8bt`DsP&0Jf@;WOCV!imr?ZXk*c zbKN=}Ee>JGrRH|DJbKb{!1mbNJwT%hq2*|7{}SeK4vH~k76(<(4GmvNw{Bb`Ua^cI zU#4#+y;>0=_Fo3i7 z?6db_y-eSe-sy`&FQD!C$VP#i7u?~hd2!1S=(hG2~E}Yt=Rq4+- zGtr3!dfS4D?~%DX1;D~N5^awv#4TErsS^QY(tr)1Eip4k{8Km9Mi>P(DY1}r5{Qm+ zb%L^0>c7Am)}@Om7Ij)km~k%(M2Y9_VTIpaT*T5(hUDDc)O@g!N}S_1+R!Ef=S{vcYj{U@9eF!g7>DVH`?FayKK#ady)a|M} z|AW8s76D1h5h=ANLi$j=sN&?6u@lqVs~ff&(7x~}w)*ptV_d8Q0t&-S2~0!boTnB=4Q zJ+atB-B?1Uaa}|9|1B{lYfK(ual3Z?o9R>H=gzw>r9R5nOnv!r>a43!3>!B%AaLsd zdd0)A^VO5+l2XFjUZdc!Wu*4d%zH1~Oe=*R&Ie&6dCo1Cq&h4*MU?8=_Daxm5s=u{ zLC6kZ@CXa4u}@nIQz$T;Ph7kT*$|qXaU?8YLw=oEccSnu`zgRPxoL8Gnl23Re(2A# zApud5MyWuQp#5H%q=TNV>mcDj#yBnypQ zzQlO3lJPu+u2z2mi%MIxMPi%%k}nAokdVQXtp8?`h2W*cJ_|_XCCXaQe7H7=ON5g| z?6#$`EDfT~WfkkLM6ESV&>~@-f*_U&9rnF0U#?*N|!(f-CUap1yBD{(Z8ROSn zfJWeD@>}_cJjZJUA$~<{8uyi^&6GEJWj)=ih%4mx2m=ER>h9Oul)cB9z!dDR>Zw?-?%3GC)?`)WiA|FT7SKl>*T69 zKfxkP%agMAAf&MLR#}_uGub`8zvK0OT5`SKJ-67fH&EsX9^-l37Z)mv&wttfYz}|x zYI0*;R4o{4ZHEXD+b2n=T~3VaEL3_7!CW9ce2I$g6$WD#sv7tXT4GJ$BSG3ei#fB{l%}I>hJqY{&ilrAFx6=3(%aQVyR0tSQZ1pMQMi$d)bEcp*+Ssk z9IWNdC93Ef*ozg6h0@iePh8|`&&6{zfu5n!bYc8Pm~mTp&c^-s(&GFK7PcJiUUFDl z2BCtOqqb_lsQ;ta zT6=Jp*fw1Wwe-rZ>uHF7omVbiP74bpW#9r?0Mwpfsp=)`X^>i79u_-S6tI*WV+zn6 zZ6oegP`tvzcsILRsCnr2DzQIBwIZ~SGsbEU%VFFjx(NmDGQI+P>*3Mja?anH`S*z- zDzScD0FP}n+d2STFM;tpn7((3+gTzW3Y-W-wcN)M>w|Xf`mph#hp=|RXjyI8OX?jM zV2pb)*6;uLkBLLvOP_u87M%ys?iha_8p%OC1%#8d6U5Azk?sIQ%C}%iQK)PotPNHU zS4uFHjHcrN$iw*T87@?ha4XDMAk+;lX&qJ(Ie_6BB<(0OrUreozXCfCa1~mxe`pt$ zW*eSQudPHd{vMy#Ihe&y@RRGbF=U6%y@l=p0-iY!cNN+#09wBF!ixZiopkTvqjYWj z8p;Jh-p&C!n+LFO?ryO3%oE1YfhDTdvjpvN8=e=uG_tFj5!kkVYC-TAZ@0X%0XQRm z5bKAQqlZr(QIvWpz4D_!!FqI!Y-dWZ5Ic5ChVJ&R%TTV zbP;2S;C5ZrRs==$>_;a6u?-sAgid5i^#DZo(cY{Z1gGPyB20i{y12cz0Z+@!(NZF# zH?GDQ2Lb*&tlbxPAEyDzZt5$x^!KJV?tMUvYKy>)WT3MU z3|PsjyLLj+=3DQ7|pvwE^jKm5DDPsh{K zsR-X|Aj{M^CHbZ6A|DdqMz*?dGc0hpj1_=}lSI}hi*8Kn5$ENYbK8N1rNViB zJU17vXRZDPKHrYZv4{qgKLs4%9q8{S)(*hc#oil3v`OYc8T$BaY9@6MTWdDzCN7>A zUj8=Btb#H~&A0+3S~yF)fUtTj;~QA%+9(<gKK!hg zm`r8{4}gEbkIPt!_lVbh>E&-Bct1(i_31P=g!O}>=n5{|7FN0cF2KE}asjyFB&TI8 zY9p74y=)|Q_KbqZN35~Dmw4jC?bS3&$(mzgYURrYt=A#Ak8$UFPgXEoq0J~AW*ZAs z1YoYW-^vn&A^)RvaFRI}dV!8ue~?Icd3?{N^;$bDpj)lzS}g#0vFiB@_C>d+d)b7~ z*k<(J_(VXnf zM|e^_ih32#PFRSzU$m%V4AP%T7?%C;e)2nciVGaBT*FM_PYf)K9bo{dbo~4;DB)eL zWi1r=)l#TT*HfpW8$iz`WWa)7?i`=V#_bjXYi}=YeeObsi&Iazi1WBG z&w2~sSOxf>a|nP$_*M>!sdY(x)*-RF#|RticE+GRck)Q~J}hQF$2oEad~VeFgr^+G z`HPvy7~K>1H2@v!k^#QV4eb7CN!JP%ppH-YK0Co@G7Bb0Hmr7;Wsuk7Sz1O7d4Qj? z76MRwSNzRE<~ISEU;fleDh`W2uHKo~jDwRtD^$PWs(2RMw!jwXZAR& z1`}2x;uD@@ywo|HXHb4WTJh%UMrZ&X9lhzs&F2XCCP{%{&j%m8Nrmv$bc=xIANAs{ zg6TC8ysbMI+!q;<0^3v4+HDT+7>%8gNoZ|6REV(gyF6!tn2!o;V7!xYV-iY0VhGZ# zCCewYQQ5!0rzc$F!r| z)4~_OBcrYkmk?IS_CmUXaJM;ld3z)M=MO)?r3o=|utu;XcGD2seuI`=@Zf4WGe~?L zK(mW5mm}G{-Z(7*b_-z3fcqk!wSJu-{VMLken3eT_uTyaY`7oI5H`k2!2zbm!bw0b zbfZVt0;?vVQ~+#!r0z?HpFDhu)#VUosnw4I!20a7X1YnW07-HTn|;Kx3ym4aFfw+P zx^vCR`dfFtLCF-x_5QnWMe*YT`ydmu$Y!L-0~`)upeHTWI(nKlq3^oPqSC zAO9>px_=i7C~Jz_XrE_T7}9eGCF0t|{mfbzH)CjkT}QME37m=mN!vPk@Rt!73X3k3 z4T8`j{^*pSw|??3Qa_oxwwyX5i20QGq{kl+xApKIK;Sfe^XkRaf0ep@lo;7reTq`F zlrG~YJ4c8e0svZKU9`+_9uTU!23L0g!T?HvkY~^M%5(OzwU5|6=%p1`?Y&9da|o&~ zT3qQ0-XoYBgDryRG)+vdq>EUy7I4uG5R+OOrd|}qczZ|20h^RRVgI$f_L7X!*FFSI zp~L}V=VrbnkzqSM{mp;FTDY9X3Bqqh0JM@Y(9b?KL3caMZ((y2_kK&X$*KZ0x;T?I zla?yV%xj)950-!Eq!ya%y2TkfX5WY~pk;OV+XXq%T<<~Qdk;6Ez{Il}G=jj8 zM*ZEGLc=Fcd2fX|7>@@`NuvO&KF-1!3flpoybWc~KB`>uMr+e-p2eYfaADTW&Gi33%Lz^n{+k`n3 zu$mmwtb29;G|lW0JNNQ)m@FsK3Fq9Fcpp4^n1243|5JK8L2ub79}>4nW_+zZ&Es}v zHh|~Tw22kv99r)(A%Z>K0w~v^xYh&O+d0Eq@ReR@+wcb19Uy8m4GdpRGsI2}j0~q~ zcvh8+-tPLV0FQH&;N2*mTOvNu9@z>^Jv^@69*Ux%!2o(|kje?{dQ$4@OQ)<`V<#m- zP+D`WqYc*UbA(BtU}SBVFc2^f>nSRf+vs?+M?7gUtjx#?0L)n(026-Ubv{qIq_wF{ z%Poqt@lc1XjXkFeJVT%*ZwU(?VB{QG>F=_r%n~NJ5cHV@2!G@p8K0#U&bV8P3G42Q zOiVzDGN^b8K;{}1+79^GA(m7DW&t|%2{#FuY1-RoYbP^7#kwBgde1)}LC`1h+iITk;Jbg%{?K-=X*-(~giqx`(Y5 znI{faz$1WizOIRYNq)$UxQn!JoE^rweVaXmC@*o0j1nWWmh+>}irqkN| zbh?IHvcH=QSqQ!xLFEB=7ZvzgR#KAW;`-X-JlZa23et-9Jn%b%X;V<2hxau)%dyzxMfpq zmcIPr6Jj56ze5AM-=zcZQ*y1ir?DzoLkumeFne9T01{n2g8y}_Sy9r2ClL#d3mL{u znVgQXaWb1Pr>5?{R8PzsrkphO`8^6(lX$|jZol*rMO?3?+VcIh^yn_7Sc++=3&BQB zwKMkJc0ckMYck59!{XsOv5zcG6rI%)t(EJ5v+daxJZ)3R7_(Oz8M(-|kgdfTwm)YZ z2}k=_qK&H=qK~hoBYMY&(lKt+rVbjVYdK?$_IP#+S(YPx<1kqi0<`BG0=SHV)45Mw zI?CldSHA-wqmCGHqP8=Wq3=>J@dOJLkUG6``$a5&SdsyZW}wS6tWs|sv@!y+oo7R= z6rUw_k$oz2rZ&i`@9rOE9<*`-)LgiJlOD7R z;05adI<4rt@T)3-Kw6OMu^(Wm2fT;Ch8EasV?R9ydNb@DX=#=sXHr!_(5HYaU=6d? z;W2#DdT@gGJo-cQmp311K2{Z-17oKRO^qT9y{sWyYq1DqXViPvp)$o};q zXt&m8I1TJkYj3Ksz9-Dja29)}Hn2{|0P~LSPC9_#b>NI!UTBk&OP2TPB&+=hLqL&4 z8H4}Zh=a6_mevvFl_10-wkO2b1%T#BQ81T%Z$O~q2#o|TTb%75QzREfB>GMQN)7b5z*%?{)j7xW?_mwBbf6%03l=cKr0oJ^Y1{L1ZMIOA0=gSk1FenTTV+E%`^C{zy zuLgMK{LAxHDtM4f6<@BEU?=XywH7_L2(WU;HPJKRxU|*@e)Oj~u9&dA8m!|U9I)72S`FV zfB-DksyW|yH(j2(&>|C*!{K2w9?I+<8M^I=%*3+>l&(AD4vfKZ5j0)z;KOc*DN3lO zAs?a{@8VY3#lLujChdkFU}5V6Sk=+aXb0w0!xCp37fuXXQg^y^aWGBN$C3JzG%hFp zqO+ISpfO4yP)(j-T;;qQ?_p^`sNT7BDP6!S*ns9ABQ44AUt4_w0we*-hrYT=A*g)(`jx_l~6~VVd5H<(C71z}+d;02+ zew^-n>(%tpKfIYv$gZnyZ=_%R^!1 zCd2?=T__U1he`rAkk%*?Ew>);8f(W8eKQzw3rtX?v&lhTVSnlrs?_n?focEYpAZ0m zCo9wo_L^W)`^0&)Vzuc4xEW;L2;r|R&ZLdXPI|Nj@MApid@NmxE}5p|f$Smrtdc&~ z7-boBV=0V!-ZRNTjqU`AYB0GB^Ni=}-gTO0{&}#ExN*$OLj(wfUj&$SzFz0i(}HM{ zgTU1r*C&L#`XRafi4k$}iMWYN2xo3(`m=F3e9a zka;+XTl#>yc{kFv=dM#kmDsSy_tF-nXohLuaYomU0}8sD{G#jK`b8>evMsZVjhT%E zBVUJ(O(>{9H%BniBj&q{<;C(SC9HxY!&k6gbfhOw9+T+NMGGb}UQsTPP7&Kyqv)=6 z#18qphxTxJz-SqT(5@D3WQ#Ylo*>BJ4ZYCsk(R;7pQc{GQp3(nYO@p)E;qISp-y`! zc*iU;cX|xQ>1%2))dj#tH|MeiYt;clNQ*!N`_{|#0`L_89ktC#tN~z-jHPR@yqfwh zT%~(OD~TPu=>R&nv`&eP-)5@b-~ET*r1RNF6rnv$L+z9$!aZdMriYD_5VWmIZ}}P( z919ey^jgCaQ4u`Sbu+fba1fJ#S^+8$6TF0h^SNfMD(ip-6IrCC7WkD7x@}zVOGmh$ zTTup#t7_w17{4gZ?_k-Q**r*dCs@aEoz@RLM^<7Hu(6R=rY4X&tjjmwK!Jw;VO2X5 z_`@R$(4plx&Zso)98LhFy~929IUP!G|L%9hVRZpENu~k#Oc6JXWamt!^di9f{@g@* zFh7U#jhl@yhC2B67IDQFX=uMUw*db$JJ0YIlt$>Jh6~q_lsri^hq%g3Zc=F5?#*zM zQ~3*tSmjN|!8&adLtBIARuHnf@U%!Z04(y(AATx$E5KhAwE3`{!T(sV|1LOLQ{7e& zF=vOkO0Kh@YaK$~wrmPu6|0DIgMMo$O16KDzd7gfGBjZc%*qaZjyFr^r0Pg!d4XoS+<1c{$#T zjC=Oxgfk(}b?gF5L6vh0%RAOb!4FHpJprkV!#@Neu4UYl{S>h3eXx7O35rc5qan2A zkxf|T{E+Kl$(%?ogkF^yf-n^n*DqKw)75?K{FOmx(49sl4J-AmhY3#N7U|t{4;hA4 zP>W?GCb*=^9me*|LKt9;dz`^P#TbguWh29nW%~*`NT!InKlR8HxjQ3d=Zj z0D15kC5`*6#hg8bzB5|mrp$w3#h3SIsM*%B{MN`5Fgq^!xhE~wITN1QXJSp1v%ces zvp(=(!LH|DtEl(XI-WC*rjb)98a7Q{)1au&KyaTcD)$C8HY!qoS*;RgdMzMLd>6US z3@)_OOxNf^;??SQ)`ZuR@l4U=wJ{W8Xbod z42U&}P2%%@*#LT>hJpSw=uVshFzIM7{HS-$8Ey!1sHHJ|zEU`LIGfQ=C=dctW!c3;j0>nj| zM{o=b4ii*fAr1;{p1ZfhB-G3;lW;5pMZn&?@eQp2QqvOi(!=0I3P*%B%ezIC@`8pj|Vmq&)eKVYMb+3B9<#$uJ9T zkeI73wEZ&s6F?@IMivC`=d*>;{o%KaR>sR-%N#f#x?fG+(2{IYPe*4z+H8rkAe(7p zbDB6eEGjUouz(|s3M8QH0LXOmMH~ck3=ja*Yd{O{qpnw17ul8o25Yz-8l&SH@Lq0s zBsyTUVA+`2tftGizJcJw-G;DMhuB|Si55d+&(U(KhvxFo)O-KsKcuNHea{J& zbV3%Iv0m6QLT%LqB>TP&QWC*>bt+|L=fe9Z+;@kwSVNc`Fb1=k%TRb8qE_--NXFz=auyL|NehZ zqX?95VA(H0gJavqhZN2Sr4I_Y2U4H33bQFtgsI7)uE0B7sQ4W!8;? z(!xBA+f-ROhjtg_AC4LKS398AAng-)+c7khgN7AC?L}*d`Pvz=Yz4sPPFi6dHjc2~ z4O~c?vhyE~XX68vvh;GCo=h?jEjR6@YVqn=-Sd5g^Iy^@1em+6n-B+@s@#YiXHyx&mzy z&kAr*_RWIX#ctyL)={PzQw891P8pm5JRhEW1+DA4kKsT;Ybz(ud(`IAKtXnB56TZr zB&^n+U&}bCsD{Oca~C~iku4m(INmCIib7+fPith-Q-iG01BSKd-4fA8}`t&Z?HVb*wqmDN?!R2<1oIif$9dj?lctCb11zg zkN3e)jskme0vK+j@3-wG!`eldh3x=M@X4XD*fk~OxSJkh06WOTolv|a&|!Q`Hb4*K z54SF3*5auJ&-II8N!NN`kT2fL^-=bR>o4w>{S&;8_l!3U016EV3U3?2Czv?5 z6n<({GWo^d14!~7OT}|}Fq2Vk37eEHK0`OGZ-*O{pT6flA0Z=MN|PSgj<{FUy(5NF zE8(;C2n#6hyH5U-xj>aOTGo8gsWJu)UcB#he%9;4Ma{hgkAg+%Bg=?(j_x5e-G+O@ z1=+<^?WVc#TI0lpBI!d8h#SPQpj<^a2@8~Rz^dcMobz1;6zot?Q}^8w0@2nxS(Rx~ zJdB>WDiP_1Xxf-1;#fwW`(|pp@p76ys83ruv8ZOhVAA$$gq&f- z#7P09zVpU+)2lDPOieU2N)G%Y8t&}$Gy+7e;{-5>rH%QUL?H4nV@W76W=omu5W^AK zXN<>nY=(%hz*IY7;M=VE|M74CZQ5SnOnsR6yLqpRU~yfBM^M}@Li5SSQUId^_MyVQ zl{wG?sn9MYNDiVY;Fel2i})&WfoAhqPQ(CiTeC=go=vS{692~6)>pHJ@-PQWO6Z1@ zHY8lpzgm&>VHI$nZKh{E0?JHEPheQ8By|pC9-&g|Bz^|rGl!*g3Rjl3$J(jbZpP*0 z>Igj;NS!+Hh@X#V{jE#WRGa^upI1%pY$>V?4$HSIAQ)Zx*Ef0X5#X8nW{LMuK;^h2% zTAW?QS`2OBZsH%3d`LWiX0#*->PrYu!CzJbkvn23878L8c0VSTJbJVgYh6Kb)Cdm$ z;m`gR){GXCTBg&dAAOWYuNTwUE8hvFt7KCsuLRCqwI$9eW&cF!SsVyzCqkP7F`s&Ei9qAzH5~J*iEwwbGRw<=_h~jmsnxx z>B^ZlzNmrCYsar|BG3AlAK>aE#%J|m+MoI=^%9-dgkUi#qYEIo0oeFta)B|hFT_6f zklj57xH=>;!33N^LMo(@vfUow8(l?Ohw<7TLTGkzG5V5&kd;r$bDdAM5MsK0HjMO$?fo?BLdYs8FO=ag963-#N}c^AgQXY z)SKRXG?{L_`qQ)wqDNeFmgxWjofhHUTezi{v4)z=f^!)jVWsiDtIjz&uB+^w7;00Zw>7)=hvypfL`WT!D~EY(2ci*v|^jYAfL`<_yaR z1ai9QjS<$eqsQnRdRUxz0@%Dv92sutPCgIsGVZDheQMd#C3;v!p<;ZNBszd5&$&m1 zWD3J4{9GoUiEr$^CWr z`s@}!nY5z-2XS%25YW(#AvfMb-YUTInPp^0JF-+>*F$WkyrZK7w=Z#VN0h0u5ONWt zhF092MY;?@chB5;-X7K%UyjxL@pe2{K%@c3nx+9l90xRK_Ns0>-xCaSi@1rORW`_D ztw`Z`l_mCMjd{p~xmx$KpB2o7HH(SL>*5*AD*%(D`WiTCnTm133$?%HI z1Xsp}2?|*e$Q(JwSX>{q`D}iS8Q!cs%zz#W5b~;lWCFjU&@Mk+13~2hW72?PTRjaN z&PVI)f1fLR6&bE7WNs8+L08OQAR24QTDg|7evHkHV`*Zo!lleOqnuE@lzqlR$nI_2 zQyS&l=QtHz=7MLFnw+0BFIaUAwc_RMKdB4HO;K1YphSN*RQE<&H@e9H9VS4&naLcX zy_o%(AwjGkT<_?tF{K7$W%7&&AU@uUUUf{oz5gODTL!Vd5v^;bBMWJMC^usk!Fo__c?d}RakFoy&nv%I+)V5AA{e!RyRIEdnqf|#=l z=|$W$&BR9NBJIGMpbN#oMP0JSw+ULZtXtFV2|QwcHZg|$w6}+gL+xS$3)?h6!&r$t znxXZ>DjXQ_7qMnkcxMAIj9f|=M@J(ied^vlTI)=um6f#!?6)S^9_t`ZHd|A`Bara< zW-PV<;8t;0x~~E&!QI&CNa|qS`v4(PS)Xjy0mgL+fZd7^Fs?=g#d=C*tRmPf!}?)K z0usn#>{c!3gy6P8Lj=5b!ssszQJ9#RHsdeMFmoK5i7l20X|aD9fs~c}vCK>y52y7B6tAi%p(tcXWE$C6>}4npGfpZt{b2pwUO ze){Hb(&m$o(;b@lo-9mYU9aGtCMgBM+r%7Cq4{1En>GZ1ZHo4>1Z=auRfL@mUMI9E0<4uFaCC)ci)TZ(6$oaRP8_bo*qvLH z^%-W)xYM!HMfd^P&!?N~X_M^Z_1OpMZ0<=~1T^(wf$4^EYpvX*6i7RQYlW`&4=C!3 z*D&G)`vBiK)Y1~MfNktqk$QHUTz^7BltMjMXvGSgPm^@vI4$1&JiV#{ihqA&X;RZe&`~`A-B__5Tyib&2g^NZ;TN2- zmGEZWO7J3n)s3p#ML}hL$4E;@yD;}Yf_=9~%tR+Rj`^?9lj?+~M?TE>|K_}VmHmzlCCEEnvf zo=gl_Fd&WTE&0r-)H%Kli&qk^ZTy^jLn?4#vK z_9)hC;qGH(hA~#QIXgnRuyls^-Ndsnc6l5brvPSj08^$FbI8NC%ND6F)4%q-K>O?+cM-xVGy`ynqOyQ0(z>FFI9T{h=c47nel z%RmJ50|Rt zQGN(&HQ;DTkN7;UlkUCN_XU0ONckm$ihXlzZissl_wZWL4{Ob1Sr^{nT0sl~jjNy@ z?yL2VE7@mUjGPG-=tg89>^u~l%*(}CxsPK~VA)B=FXU-!~J|V9&Z>pil2_b=OGChCvucE zGWWpo@Uiel2r>nEnVe+~{GGEdkq3cTDL$Ea(VBM^&g9VqlbTr6fp?FPdd85&M=8^pY@$sK5Qhg9U&$`TnL zFtWkHei|i{S%l!7otjR&04?37oQ*)qTEO-sP&P9H=V1oAJA7t6j9QDIzVl#3AlAU2vuT5gO1F zYs1=?(Dv6B7t=re@~_iAf$H_FMGpbiGCtj=^)NdvL|3}Ih+h(15R-{z=v+_>)3kuD z2^caE2QI)e^*(DC#vs}AJfOXc?8ZH;%a$oI)=$uE+=dAf#%aF$+h5_vtT0;61q|XD z0BmMmRB)tuwW7$NX8w8VmGv^FRe-ANx6fLd&DDxhp@nI3dMZG$$_mrr2(c3EiEdL5 zZ=He@nnOp&-2~9@ACBIeYOaoRdU_`2U@f&m-vv5RTus+s`ZmI{HQjsgIBntXtH<3u z{@NdLKLExGb>EuVbBI6V5T@!$~hUL;O z8XZ`}wB8;fR4^A|;UO0Rp1BUalo{&@%-B*n#;OXw>*%I%4j4Z#&;?-VVrn0|!s~!c zf~o7<2Sw3o8i002#$bVx1EaUD(c+2oPZ^7-a*riu2rz!Ov67a*!rJ=qF7a*4X&AYrY;NKzUnc(VySHCW%>?Mb{mcI=ja?W?H?LkzTkxV)>f(Lx&I^Ez zQxuAOoNvGawBWhh2cQdZ6}nc%H!?}Z*VALXsYyf?#=MWPRxHFBfv)l>nwAU75ro$UU|qFs?i(xCg85~jP_JmI4#_(_oV}-T1AO5tYMG)ud{CLfZYn< zU#ovFGc4k+|i$w;41???P-ZKw#k@wXJ@erVWorbL>6Mk8Ny<`$LC5UO4#uJ z;x}GP%lF@pgqEf{x+B1kEudS^9L@okM_7q3bQI}h%=y8Bl!GSZGnN`!Tiak>lu81~ z?O@Hcj8O%k=6N^Sih~(~UpIX&4`641byca9>o(IR)q&OjnEg`fwQ$}Ij#r3VQ_Uos zoN}^07Op=m%(`L^b^$r;Z4_q4ny4_=Iu|O)CmArN)ID|)?gUkgfCTK8NtWF3u=-mV*d$w{*u5Av>cmYLU z#S_*Mp0A}1x^v)Otzckz|J~1_X{=yaXDv}8G!Pr9fW{Y}3D+J^*JHro`3(aHJW~*o zWtOt$oDIDL;sb6H^lhy>0Tj~)04kArD@2PxiqCTP1eWqxv4BOYy*^rmdB3l}B8kfh z$@}=>BYbxJ!M(Xryy*Pm8rDS+8NbJSoIS301%2HTRRY8V=;4t4)%sb2e!NQnH18dJ zSH8`Qd@0`?yz#tY@$y9%if0CJQu`82@q+uTalre9r7V8q9mmI)_wqYj&T8h4jW?go zi92K$#^-!n9)-pn*YJ68g7*?+23X*F3nBYl@8jM?^1|2GI6moCLjafZ3!ihV3lNCz zKbP8gfIQy{Ed$H5AV!6bH)A1Ws%-EPgj0h#^r4w_V5!rMQHA4gQzL9VO$R!SeWfO0_qU$?X5+>Lm z6GblTCEMC6*yTsH-yQd zsN*$Tuw1)-D~(;dfidGCeg47Q1SV6*2(9&y1O=Og*AOZ>vWtzwK;9u}$d05e%Q*~p zYnM_N0A5YdjT~OBOs;i2o97GSE0~*ow(E2X5VwWa_{A#(wy$AfSr35b5$&J^O6KAw zo`ZE!8!mcyn70hg>`ApoIBZy9JxsgdNEuxMAJG$z7aQ&R|`!?zR`^2KD ziRl)s4|)J##=W2wp25%}cJY+?c6RlqL9Dy0)Zp5{TCfkjRfBF=O@hcT-F_|Ic;koR zikg_7B1Z2?^jf_1(u-J7 zfPV4v7yy|zU0AmFaLtyOR~JAC?IM5&2M5b3)>zhZ6I!ouR?dj26Vw|V9qIlEGA&&# zLl@AndaJ3_~n$pfViitR3I3dkLYgwtv9qM|wu0L&g@FZl2As_@m$$XKJXz zt9PC!4vFI4ghQ<3%2yu>p#T6C(@8`@RP!FI&}RX_C&S!FDY0MTKJ#rrklAu*WoZtJ zTz?v&G|hwe-%6dea(d%J580P%X#+RkIc{%-t~C$WaIcy9+K5p0Tx(^QIjgB_u@*q= z15P&>k7ZO;>LMN!MWmj1wq_DFz);aodY8h<4Y<|{&~X=iQni5Cekgo2c9=GJj+OGIwaWid8TNf951btlnk%!se@Ml-KACC2QMI1KzcG zu^tPYg6ZMH3R+8%b4(`%__+b?W|QwLsR0t!0~0u%cJX}HdXLZDXOA06tT`tM%^I23 zVZK?tQsu6F^3{S6H--fn*)+cd8}2 zo@*Fv7Hp63)5U7BcT0#ANrcKQKk$2GQ)Za~@L(A-IikQgwRY9B-Zhd?#4TpoHj@k{ zr!qmO4mqI(=aiUet!V-et#Ho(+a4faMN-$5;#hzc#g6&CfGhWa%wIF09ft-HLlES@ zen)H_dm>oT;wAA(e}X=}Nz#l8brU??+K(rUA+uaj%$Uv-nSI9WPyE|_PQ!`(xDKPh z&Mxjnt)5JPrDm|0DmLn{=Q3WKkKMRy^+!^HiOQ`(}!Zz}I!v!t9qE*8gMgJP7eg z_6h#z9F=p~TJd+BE&lW+coG-HJp@Uvr7lmyfh=2HwpLhvkkNv3Xqi5E!W-Du) zxL>YM$P^x)tr^#UJtt*lfi=!z`TRVbD+bAWGbY6cY!?Lwt-}Lj@zMex?uO}k01wa* z6oAo$8lV*YFbiy3&`f-W;QbP#b`(K0Ua*n7(-WG=iC}MPzx8 z-MC+ks_kebXy98kU%&tPXW@Hp;5mmI%YZX7&v28K&_qlD*79}dh3AQPp@$;D>zA>% z^>I)-7(+dTU!aUYBNH>0&9wb82Wp?W{Qo_j*>l@h700g@OR^T*k}ON!oy3VloTi}* zGo9f9W|)EYg(nyuc;|oP4Q803lc7w9HbYC8rb_}%n#8-~C0-)Qk}O;1^SN#*giu@h zE%%;#&i8!Jb`p&@E5s((0BmJOA`S5l>j24B!XkCpjR8ds>)Trt;-Cl+P=|zPP{;cz_d06}{>t@d}zir6eYkP>yS!@F`>_z63hXMOmgdK?G{uszT3=6at zxhuJ_Lq1e}b^$wI4@=9p!q)2Z@Y%h;#1;f2D5ErMX}+Z(m>b997eOak9WVwkQ-F&E z_YhcEjKX69+~_Yx_%*hVK*{$|1ZV91F<#9UK(7U$KO;6a#QJ8)W!zh9lF!%=cb0$3 zK=NjI`QjT!&d|EW{ba49@(1w^0FN>to(*Sn<^YKmMoL9*oBbU{j?Z|`Q{H!DcNdvB z2~`q^C-L%E)>kObK%sNR7|pPq15hPBA& z1OvuChnaFU{OWgqi2c0&dhZ^JJ00%+?0qH^FesZ)rpdhzKjPNFtVxb_4*VZCXaq zz<~Z=C`Wb%^~l=t+{lg2i=kzEX^pX`O%na9SC>PEbO@V@-v8uNWSAVC`MEH&bd$bt zjP2sEo&csSim39wN6heD`Ql@kmnWeBV^{*PEd$)HlhfK|{Y>ijjN76l1vAnhG#sY{ zj4{0as1v?B*opFMNPmp{yEbP|?4CR7LlPP53lbvaTa(TbXqnn z&Mk(-=tTJJ&wmWJCTpC5et7xtlTahqwaj_8$mk`-9nmxAjJS8lfNVn`)9g(P!`bJv zo(a#iX$qcN&kQ0Tji+PH%x#p_=>TxX2_@>)r^#V-t;YdXHpI0s$R1cA4|Fm4$J|fJ zd~JIU_QkY`b6DL23NP@2;xpuJyMnA9V^FRwEMUBMDERpX`Raw+*RO?raxjyygHFzv zrCIh7&UKS2#1!YSk4`&)xj6?M|A)`*^6xpoSeo@p^DGAd7Yhfhf-S?kaWDbJ)ejd!BNf+a_yp z&RH`yT;MysLIoxo2NOHf#JL|bRNbXm3-CPDA%$m`=C~8rF<^i(h=OE^mQaB0SFF2( z$8%{4XmCneyvt|>W)t?g&(A&2DP;8mcCi8@8~cZ$BTSu0BCwah%624Fg0n|OTXC=- zk`qQulmQtnK|l;;p@THW7D@|d zZCd*LQ1Io14IOG7M*+();8mjzd?AMvu!rLk8-cU;1HipegQ(4sX`mf{0KZi2r_AeS zPH_(qZVs99!(~J$(XQ%f`xuD?HVL-ci~In5bvCta9P!BF*`7#e+#G2!ggNe)VMM-$ zj6`OP-}@cc?h?=mh%c`@g#9^b!3d*2Tq}E#n%^m$iRVcwh=);!#W!U?B2mHPN&v)B zWqjNIY`%h|k&Y3Y5N*~a$hmwzeiWTJb@ULqfamo0G8FBD`(2%J9yG353=8|%=rcjO zfK?kBr$(^$I`roPMV>7QDV5@nIegw)P%6MQ+-~X-euo!|&Y16&zybBSxHJEsHABWt zdCp97kv=d0oBQav>0$Th>WG5qQQ>~PRp6f)fV z^l^Cdc$24vF=3-@XJLB)`dNfD3E@)k+tA>vgm(%6s|E^oe}m8v3>w`%@Rq=|(a@1U zf?!>ln+pz@&vSrpz)ICn5F;#Bo?Q~FM<$Bzy+&d0z_|Ih1G63PB<*iin2Xrt`JQj= zg(d=Wz_f!khAMa2)I_d?XKCnzP$uD(Q9S!N4h7{iNzb)eLZyPAr%#{6cQxGBg`Kg| zRTpMM_ z=WID5plbupFPX2)&)F)aK!xqj{tj#nN+in&9-4g?U}S!N_ua^rw)x$&&+mtoZ@-Kl z^aS9{JQzcAdO3PGycVn#fR&ZXvz8^<3ZI+zKF6@ja5~&rhV29;YaY-U>`)>LNTWIE zDIvwVg=OYuz7w9Uta7b0_O(Ht7{-j4btRt+PyhKhWZ;yMO56n_Y6Tgu8@*JHRGiFE zrg%i7in?*5V=p?JHanj(hCNZ%Iqo$^N`U!>yS_JaXh_oJ`4~>)-3;T{joWvVF%GPk+|{pdtv(O&G6a7XBexA@XjxP8!9uC+~)|x z>%^0OSSRH8#yf9@0->?>uOEcR|NbPTVE^V}W)pzaf*}R?jBBvuv#{BYU0PpVC*(jP zI{RK?9X0M6hkZhLtK@v{@oanaLu+G{hGEpqA!NHF_%S4&p<$@ZZIU)Woiw3S#v$2W zXYC1EoM-5=qg*}Y-6lsv;|z|ixn|kFScmr z&AC&S3!Ie^98_~!PdKAWy2jWc=FZn$Y|N0eIUQCXJ%9=Bgjw`Kfi^4`fWc>*>+vk8 zpKK1=WsPLYOj&52kVb(!EIVW6wy6gGV~&)}+knu&lLYK%W=}1dH|GeCSQ(E#IG|7= zO^(yr%8O9v`MUHL<071E^dP#d3y8o6p~`p!7N6^3+-)V&CiicGoW(9W@C;*JC&ze< z>;B{8j{qr0VWtdoRjB}CpEF5glp+y&?;*E)Mkoe+4;`GyltOudN>TRAzJu+Z*Hqm$ z!aj`V-1IzcGxh*n=YZ6)aBJyCn5~Uc974Mx3SZL5eDa*)l0%#VfawV%y^Ky+P%vO0 z)KLe7#Er`AZf~+?t$23bE5`|%KNXu=XgO^(?v^b%; zJo#xe$a5LS3fKgYK<}`Itv)FRv@xkH|0EjA2o5->v09RQlxq~Mkd7gvX-Wb|E3zqu zHcd~+m#tD&=iqFk_cW@u|M;OYGcqwKg(_gT4GXY!zigkjO(w-_*!!FnL6A%k zDRdE>;YUS%BQq;&X_aW2A{!Z%sOx#An2e4bC|feC1NVjRC29ZTl!IYxlYqrGD6*}n zMD&mjCQdarZ$Oe@HCbaVZP|s9N4J1Q0FI=1;H2A*!+QW1(5aU;R{0dbL_5S9Q-;o)F74>Hplc+Z%nu+)HKyYvLk?Uq|pnDeIk9UtLG;y5- zezI5o3;@j+jErb~{k4UMM&@E(n$N;Md6&qv@x0nY-$SO=h=Vqs*NfNlgWfv=Y}Qp< zbVScaLALMjJ^6(H^Z$S1zJss4j%;QZOyfFuKR@c8>agit$m02J?_povcrSi#&ZN)o zyQHV7_8dh4+J$|9;PLwyho7FAWpvB4P@X2?6EAOp;nB-)U!gJ)&_PI|*D|sBB5WR# zCzB+Zv(1v!D`7H!hXRFNn(OWX+A?_mC}b2*mIUVn3e56n6<^Q$-9k%E5da z)-Eco5^PouaL^$ca}Ez=t~4GdIpm{kW(xF;Nbq?OGXJp|CT>rxthxYD*6~Pk7|t9W zA@vgW$#2`jb7|3dG3KCT0YnNFg6~M4S%!Q&!C;FV4vPgG+?-{j9L(M0z&RdloP%qb z{x~Cu%$HGEtN@c1*DKLjsfe*V+!Hmx8pe?y2p+*Gj}77iIM&$ko^F_%5j`^g?g{ev ztWA#N&hQ-tHBzjArBPU)afb4GT|Gj3m$`u+CK(00_3IVcn!*s$DTk{rRI^9tA#gu{a@c=Or(tRnA$^zMi!!cV>& z0%NF=|3)S3b1Be**kQm%YK&0S)B;ss*I{h4VTF+$s?A62KaGbOf}J22?CR_kdk1Bmy`;45JYE6>6p=xhPCYbk&U?WS zA|qB2vN*^h1sfhja@`<+!z*MugF!8kD`Vohjhv;s5nYmMQ82DEhcZjv6M-JS!}=2X znYpS4GF_^nJ|qv{hT6vbn)K_40A z;D+}<_(eEg{}w>{cM1Tu2tN`EEt5lq^yL{lRpSiaXiUWCI)q)`;cR&h8k2-HIg=J* zv~gw*Ve@tXRXrx+)d=3O+3SKgYX zAb^KtuguGGxbv%xjc|&A&yy06gIPR1-V8&G?^-AVYRQEv;=GOE#%l25{wfFD&&uCA ztY(>aXrphQ0(M^Dq@-#y>dOHjC_Q0H=7K6mn?Dd%OyG=G|WcCe~=CyU2uv>;;wQJgYga z^VhBc0)~;>GZNitj7vdGadrlWm7)_uXU}&J!cne@6HBvj_P-B+NV1npH*Zs!yb`ME znNZ<*9kG??I*GGfn7az&%6ZF@-&le<9UsebW*XtuGx8wGJx&ptbb!0v@n>*0_Rg|| zX<^Wpu7u7b+Sjm`P~~BCsv4T)Kj#7O*B9m(Gk6G_HqMOb=@<#|{JDQq^YxHmI>ZG; zL$y*4E!L)s{_fEVr%n?8)RlQER@HxiK`Ki%f~hV}KVjaYNz(w%Ny51`TBx*(G_&IO zW4y}%2X+)9LB<}pJfv9bRL>|11y$%2p9`f6`2f=iY)q3ITC31j2sZhGdD0VA@(^K_ znE;0VGnbS#Ij8$SDLNxucR*xf23xwlu?EO1hDMD@f$fpFzu?tL5w@#6!#2t`bzxl2 zI0PLWiWJV&8EiBn^Zm2ngisA``;P(+XqJ3LL51Tg&yW+zff3$m5@tOIDKR%yqaWgn z(2)6N_8NXKw&r^qk973WixYLnuhFT2Xo75_U@S6I-|ORU2WJSKW=?F1CUaS;;Z1XK z0!Kc{ck+E$QbsA_EMytymL5!E?}uYQ#tgbZ(!#M`Mg*J=5+|OF;QTS+Bnd&%z*$HP zj6eZ?IiElzYz*&=BR+vcoWU9Iv5&1Ldjn{-orlpRfuL-ZY!_c3Js|_DT42m2RovK~ zb9&*`I1}d(Qm4pv`l81B)1GUJ0vtonoPTqw6a2C5R2~plWWVwjzoC~6@fvzJ&(i4? zbvpo{4^mWW52G(HIt|$I67PxbBifWJRdEin_B=D@nLJdYg%o3@&+8gB@pe4ZMyIOi z_sWRTHXTHWf!-(uSIBFR+0RsxnP!pWnrZB%BfDb%LiEabt->>bd2m`l=P^?HKa1hv UP9}RbDF6Tf07*qoM6N<$g1&=`WdHyG literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/0-pipeline.png b/website/www/site/static/images/blog/scaling-streaming-workload/0-pipeline.png new file mode 100644 index 0000000000000000000000000000000000000000..3eb0f3dfced4e259aac8fc10809c2e7dc304fbe7 GIT binary patch literal 41826 zcmeFZWmr|;);_%1Y+=(S(yf5fAf3_(NQZPtN_UrZE1)0}q9~0t2uO!CNOwthH}Bm1 z&N|}3lWn%_`FupZ4GBS~4X6!UJHZtn$WqOS6)w_23K_@(?H$yv21^3 zaJ#I+F=GJJqY6F}*;^!&!sVbWd=wOEENa2by04JdB1oxJWI!ym)LU%J1@4U$o(40v zN6JS-b3!Rqf=obvlSDigJHevXWk8b?qKHUKRr`qPirjfDlKNejTn!QzRzlR3tGm%oW$9i$rYI&%}uDMf5+H&TKijXJZGaP~xW(9$PPe|aG6#POU$nQcSsNgR?_?66r{`)LaKqm6PKSMtu z8a`8#kdp&{)l8kt%vS7TCg~K+PN5e zu-G}%{nN?6`;jzrHg&RcaIvztqek>=Y+~=~B1}t*xY2)p{u!s4ht>bw$qj3?e7_Ox*)%EA8I6dfRD7SjNmu4m>hM>SF*!_)OqE6V&nZJbzuAXHqI9 zWVN+~t#BTnr9J zqEU8-Uls3o6<}i^(NM#J18As`GO2=aaiQXwFYw|aIe!ARszL(5F+Su13$&+(ZQMsf z3lO02aMeXz51P**PXGVbJn->m`P_yZ2`=Bw*;2Q1UgUjfMPMUG2YJHJSXszdJK4P2 zXK5F47sbdTqk1hVT#9veOE?K8{oI&0R*S+#HKGxup$c8y$6v2f>ZYUqRyi>w$*BIi z(zGkMiEmgCn~T-H{(SZIY83p{3~hL^L1iU(o^bKucekA8x!idOs+eb9>0HNyvI72+ zna7v!(HYHr4)j;-(Xh)c_~NiaFjl5h%TiGMT<#iqd8t&(E1{j?vA?PyW5auY zsuO4Vg%2<>bkyv;)a4YXP`N*E<+y51RjOL~uO4C#JAAVkdX<_OcST?^kiaJ!t@bD> zmC2uxcbKZHD#btDE%>UY%l2V#!NRD|Yk$>R2|;l%ld4s827bT5y<18K+?4%PYijm7 zLkt_~hN)sJZZ4D0nB?Z%Q#;izK4w4Apo0;uxlI;wjG^w9b~mTGiaF}r|IYI>Wnlx@@C`t`5gnSk*+rZnvlOu z8rk&JxaQ|M-cGq*2icGA=jwvdl|#V#c@Xej216!bLNqb1spC&4=&COwFO$g7XYl&! zuwQdFnyYW^y`i7yX~S)jABXqvM<1&5i+*+q9}P(CT<$iw7dPDe93i>2o~kT-J@?D^ zO7msk?b#B}>Df|TZHoEF-xq@0<$vC1r>~&@t@zATRNY)y)+bRq;OwXRi$ zRXBoK2dBq}W{b^NFU5AhRJi@@E!mB;@t}Thc8%V%T6AkRitF@ui*ULeZU3I9C&yum zX)J#hh1)pIfr7x}%U5%Pi{W$yu?;sDOr+e_>fe5O%%GAxG`{*^*!kx6>QJe?gVI~l zU_eSuNw}iI_}BSLlyqEr_!1a8DTPxf&0q_TsGh7NOW)ydh73uAap~1GcY~-d>moyz zCqf}OtC#lW=Fak>ws{(IxqcP!z#mf#zpKeI&eh~p)3>^`w2kkbF2#xd>=(K0q3~M! z%%fHuEVw~tSKYLcZ71^mlhbpzKhM2)Z4VYFru{hy?D$HpD zIcPxkiGSqdZ|ts}b%%YM*aBQ7rjvA8sTe{$|yublJqR$EYx&ETl5Kbh=i zsh$riGVO?^7rMfn%RAeynww^=-CFc8pR%oTwXfTkPRKL&k+XSZUcW1ks`YQFIxnNxepQo+_++6+L)~XwwW@9|HU%53ORl<^%c8w6*1vDx z=sEZP9o4lLKJDS^*Pfa4yDX%rby;7`c|9maRB^#*x;0*c{S2Pwcd_1Ae1E!|t-;2Y zf3Fca&G2zGQ*qtU(4jBkpY(pw+v&jv!(zC^*rZQ83u{(Bj%w{)uB6&CV;%Q31|r+q zyw`J*Th)>uYrzCBew1II&WRpe9Iky!wTl!|tMoDGx3?nFa|q1kJ(>01T`TQojS}Lj zJgP=t+bZjiZI32NINzJk{hn@pF5tRxUu89n%8%Hy-7-U-t^Vnx*JdHh43|ZI2iI+m zVk*zK$}C^5exWjbZ*vmQc2duU`<^sLp4aFMI+mY9XxSXHd{1Y1!8KJ|B8GnVOs@B? z9vwvU9zbclc4}4&SSNQv#EzOV*j^p13OxQMOyzs@Hp#z_e^uL2?FGNt)6If1()SGM z9$DGvgi+ap?icGR4lYA-3@=5k@(rI!o?o=J9JOIgSr%4|aw-t-)~u!r9yHTa zG~pKLA;1F>CF4Qa)WAOp3sz~=rWTcm>TKbrB7SzdGsDu`|KJPTuQS?ve|yl` zmH(zs)z--n@ue00QtA8)TlC6-NstwCcGw~6e|zmN4N*vDZ%VH}o%LqH5l6WIrlpwd zQ4yiv;$SQuh>OdvaJFw#j6| zLx`n4_1?5oH_P@=aRBM@pZxBELgP;d#~9Bd_s%x*iwg?I47^$1SkXWsfy8j6;LHOu z)U8nhB_X_X4YA-pWvxC{=%WNGz^j6MKF`R+FCF>YE#I1&`{AgLwI# z+@%t7diDo}M}Yzz{gP=JXj6SO z?{8A=f7Rp$C@Wg`<&oxxSO^d@1cY|y)38O{fbHar?UW&!MTtp(M<% zBN|$+5@Fb`e%bMdRf}3geOzAMab02kU*bfpPwT{z)#Z<=9zyc`G4sEn)-7N(%^M%p z5t4@=BDI|Ku=``SoWu25179cIx3t?28R7ZX6kYec)Z$HBM zE1QY)zkc6HlSc%1>k?{S2t1>TNf+^XcaZ5FHtTcPtH6ngMX%00SZq-5MWPl`W8%{$ z(xj<CtF>`1@2ha1fP7Z2!|3{^55ed`KpB9voT#g__F1_cZs=-_%YqTw*Qwl-+umrUaH< zqDm7)2Ta+N#T-sOROIuhh(7%3>prX^bDm>r+p`u)lS~L@LMFy2vxP4))mkOndUFTX z0)M=uLFlui@kBGu@IV-?UF8O$0OzhqxD@ZPSNn2`6{KHlQNCeGn#=G?>{)@!JovJ(t*^lP|2 zDNBN(xXt=PruxzZbu(mQ$=BguoM|T+P2a_GIl!l z@Q}CQ$!7~yisL{r4zCwSS555XCsz#hbn(ID7uVHz#1xlcoe&x0h!t;=cm@BW5dL}l zR)#{rrE@Fc)Ee#2;TM8g>-<8_eifVx7nf$CH!o)MUv3H6Ay*&v38p@MOdR;;*GEhP zeOyn%P~o_J&RE6DJ_=v%& z$g&;BUL%t>Zg~A9OH|nVq=)!pl$FvuEb1H9R(yG8yyNYuv5l2_%t=^!bllwISE~lw z=Ig!3n;=e#tBWtQc%W-jvIb_SZ1qe4I8E9H^zxfC+s5|{cChw%Dh8W}e)lZTub$8z z2u3jT&)Gk^@saOeTACXweSYpkhR!9I$wo6E4jX!K3{S%KKA!lPqF4Z3^jh=~$Rji# z$@Zu{tKI$OY>+&@L1)mh{OJ$Tu$bI7*lsfvxSI=jQX6T;pEUC`K{X{EuYFIPS?W(v zoV=CZB%ynlRL$F(t2^&|9Ufzxw0b;@q4}F2GNP_VIIMo5Fe7L7_p zFwvlfeUu4kT&R!fntHxF-;}n6baov#YvMCsSxrcrh0<_ ziU!9`Y^}BbVZ_-=bF#z49Iq%o`e1G`W_<5~Z4`zYl12;|ga%Fqsi8dlEE#1b0K00K zuef&Q8TsdKkb^3~%E%x+XGL+^RRU5$# z4EJJd2Bpa+bekk`=?+FgWNqKKUt0h9-HV#)@=OCg+bC-|((M;qw7_@N@tR>;kJj~4 zFlaW$c4~LibK4rYUl0CFg8w zrq1kx{wVD$JR!=1*%!SfD-d(APvW}YiZ??A(@ZOzRqqNZ1W`cyOoGf^$~8-+&h`FpV}fH8?BDQUY6k1`DDbV zH(gjYOKQg}V1jCNZ~tPmSkUA&mXy0|D1t$O*+9?yPvv}A!(hwL9BUZx;9h6LXFB2m z0h>8)b`95VQQDfaV(u48nI7bMcn?wSGjsf*A#e-7kij)S+11XnM5MOEg%*qLim^)0 z>*vjjIIZ;_lgJl&|J|1ngZrFsmFeh?>Aa3I=|u-7g?n2PeNf+g-b?jbM1r<6o@b#E z8Gw_w<1;|Eikzm66V9TbryJcu{0a^AC7KCS9*&KD&>Dz}TtvdXALVIK5CWUZ3^K}p zLZb|a=Dd8tw*dRoO#mH|87Ym6TQn2&o~*RhmE~aRD+Ad^^5l2--PuZv2x4?qqzd9u zb;8VWqg>M14Lb#$FO(X*dAAkm)s?Mnh4=SONXu^AwkP>%WA}qT;&iWF9IdP6(hn;L zTV2~*-c#lh<#7EvikI{75LJFy)DI)Ko@?IpViNoF@u+U`sBuyLjT~fn#38>-JktSo zak^`7oukM*GweoRf3cCD68|pbL)IrNX|Z1cLqY=9skoMAu%GU`J7?k61i;9xo%KeT zzzPW#D+81A6yq)D-O>W*&;lmOwT_=7GTH%VXerX&xxn`{>~_<9hvPNpE2+BA^ya|s zsEEEjlK){TyIAb(bbYKi!y9ZX&4pQU9_O!k`{$e9aG~fKFt+p}UtUyAn3Fl*D!Vj! zWzypgpl8yDs-(%%LHV@7_SQ>oe}dR>o!;~Z>-tZfP^u*&^yXR`! zm@SAhQ#Q2>Jln`i3o9qBQ@r;RKor=#(BGi;{Y52FM*03yM=h1#Sra1PeeaqCHgr*w zRXUm2r>Ks4!Mq;!)e9(9V6toV-DgG)<~4&x8iAuX9@}Trq&|D71Hu%pzwtL9sQ9QG z8A4<*UKbK-;dgzkrCW$=6IvDa{iAhW{8!WGcmwFv*~nkaHAJB{8Ou_6D{EO%tWLGt zmAWQ8=Bpx<0X45E*^L4aX$Uh_(@Ud8zSzs<1f`Ec-NIKW6ppQUF++Qg%X>JxU+u8e z9}TToFglwKMa0R>$MUZbM=7^ISph?+ydTtI17d5p43|l4ns9AJY8PBh5L*94W0W7( zvPO)dLgGv42r`s_Ucte%J=)PC%~jLH9g(D(mU8v+eF0loThExNmv;`x6Jyln_o1;u z2ePYg>Z@N`q;dOA5p(ZkAiuos}G#E=ctia%E0 z)86OC8gaC8h-s7$J!dl8F}1+WrUC|h6!%s~`wJY{DDO_6;I0df@R72dw?{jP1#yB5 z4$hk}A7VL~dC%Z>u&r+HF{3+DT1>nB@wcrQ(ZnXgRTA0{CKGVUsmYZ78?3*y_9d-YT_3N{NSJ# z%aKpCgWboWo>@}UD3s6dKg@%ch{s4ljNJnwR_Qd+;Q=JNixZP>8Z$BD*KBDckl zsGz_(EmK|JP8M?k_W`!1Stx7pyJXS+Hh28CZsRsQO^W?X0VM`7OJAM!hM(N|m1^W zOy*+ykJiy>;>o}(o*rkEf~%7U>T$r(ZQ(aJ!0DfgmoXWp(NQBQuZlukMfKsq#88F| zAEz>`7DLguZQ(=^U$7Ied|XI#pXDh2{25|`ii--FyH&X#GPS}6C&J^4y96m&Pm+Y} zGRuMtmG5v?#6p>@dMaYqSgFp_(ufrM%mI6Mb-Wafbt~e3wI3!C0$ad=yW>oENFDyV z+(~_AzFU9F)1Pi%fBe3Uj&JKn3v8%!$w%RG<;awT2zhJKV(& z#qfvcn;nj1J0DZ_cgy-kP_T9=S*z!vho^5Y!|Y~$rnKqqn_+OkP-%tl1l>}q z3zF~8DZOG(7JUj3n^#vCbw@+;^pce`Zo2XTWs7zTpw)N z^9>*m!$+5r3YHwh-CAdDk4;)qbTZw88A;hKn*S=p{upNkB0I^X=h z$>(_qP$o`B_EbZk{kE9%vKIJmy2q9z?))xQ3wv;pC~i^K{$ZRN@6~Mr8)XFZ@}H6m z;H5GzI+TTTxDb7yqNs}~gVc-J0VD8b_>!PWE4*sbI=_a0_0u1`u;jC5R0@}P67L9> z=y-Th+U3di=B&pOR@KSmYe7x)RxEn#fSeOtWubscvOk)eDmuRDRixB}IeZxnH`U$d zZYMBbf6E7z5OqQJ8+_sVz&WQ%2VYnSDJUToUcZu5M&-a&fX=9HB2oP9Va3t{S{oWj(wJSwDQL7K-@ zp38V;B-5^A`uuO|)cx&8-sk~?S{9c+4H1i>3uvlV$`Ik6m3L*Mf==E`#G%N(i(IXV zfsvC^>bWW+HsW|3aOAVbcz2-Of-uOv`MlS6II|C;HwhW5y8}`LARUUTrB8qejh7|F zro;|Ee<{zxn~#xs3zpkRV1V8;^$xz=Bynm6XdC*>!{X2U7+T_rbO2md#3>fg!hr+0 zM(CbdD;< zeMJQiaTFb*LyKsc!w6auhUg~Uy@H`P;L*L>#aZwWbE%G?oFbbGyZ%Wto`DAxHvRnm zE^2xb30g`mXqw%pF#pT>A?!3~EQuBb=D4zJ0Ybu_Z4JUW*;b39OTJBeR zd4R8Bq)#`Hdz&MDZO`XSzj}55iz1o7QQkrv)*C_|r2>R=@UMwK_9KKzd9!AUOh?a}y z>P_Oc3oc`He7)LtYwc!(#rgcDuK-+HJ_U_%Vs}a6_PEJ=ZY#|VStzxugz8_cAWx-5 zv35*)SB3&EGVCKv;XpO+PyC0ipH8k?X-QG8*AC6)V7L9zA6{#`r7-^xFPccUfpacD z6YFt)K8d%1(;+_fHs4>i>!A>1L$VwBR8RHOv4;1Rbcu$HlrB0F+K`V2wK*&-LdrAq!ugILFm2-jqSr{;IJ=3>BlRHj4D zwYyuZky2vP^WfagJ8E{XX=3-m>krj~yD=pPC`DY2vEBuO``YiGtA~RX=?NT6Cq^8; ze2f^+63$;kmst|ftjcbR zqTC!?PDxkVu|0pONNd8r%z2j#3hRRrC{x19p^P|1Pq>i+*}G+C668k`*jPTx&HykY z^B&yt-T-v($Z`SBei?SaappmKu0oWQYT&akb;qmO|2qFBr;*`v5RqLEGBL#=^!j~3 zx75tmd)I>GvtU4m^IUu?-D45$biyLTY-dvvo`?=6LzO1i>3}xCj%wqlQ_Rqr4B0prz^K3T84xdvh;?Q zy8Y?GGa!=LeXnhT=02wHZZ_mf3)4aPKY6_2!+SKiZkG>9+?RSXn%^DVSC_-!)CZKz z4%kxr68C_!uLWBtm-CDc15?3|Ac^pS_xFhO25d{(9_$RtP}vr)LJj;4_o#MDFM#@D zlbSb+8`9IB81!r}NDu7(DDt{e`CpY}>w{Qm^h0PUN>w8i zLpv3L2)To_WSk#rwcPnO#i`L`zA+FR)2aoKT{|G66qKI>fkR=x@8wCw2nKI1Dh*Eu zxifKW)vVX+Z_F1}AZOI^Mg*9rfWR^v%4v}Qz&gsa_grq{B`n{$h&tDl^ub?s`h{~iW4 z9w7MPLFay9i`6s%*T}0Ubd#W`tqVksS|=n5GJYul?Z?znJ(53cZXUAK9V^n`7V;?s z6z=;ujQF(}-r4Xvhj1Pc%XMNDgkQZbs`+N-yOI07fcZkQ{>bzV7SE6JDAu2h2SE=H z#A$xc2VoRY8NjtC5sX=XAxH$@T3ihRQth>8Jw?j~%m`ou*KGI$00%lFJ&TJdBg z0U&Sm6u_WT3+`gNP2F6_Yao=1@&f@Yoc#Su^OUu`{G!M@>TvU9W0cz65b5Q6vi}gt zIa28pGJp~(t%!Ktwcq^P;k(}k6Vag7#!|8(O)tjj*4Mf*q=dQ@9h$*Q6(Y32A?o;D z83Rl;D!1J>dI;b9RHKit3oA5T+6c8I0VVCa?0Wn*!;=&_AcD2Dla>T&5Ae~FfQ3_8 zw%8tgU0nabw)8iBd$}cD&wcYNfDFpiZk5U}YrzLaZ?6*YAY+F9^{#FCDa><#t9)?1 z-$6C_^p66;Hb^4gYVl8WY@UCSz~*l~phpX6dV{JvWt2Rxn0nnan@+)D63j6VR^=@0 zPomljlJk-9q!8uLG2kV;-A~Qesd{eHEO0%XV z>=pwV_uj4F{>1h}&n!t}ixg>IOB$~k{UFqVX>;;*am9((lGjng?KRua%bi-f2B(sz zlWQ@%gCM@9RAC^d{Tc*01Ne18gOKhE^yqXiM38(L>@64cxu$aY(*EhG4ga5P=m8z` z6H(~e@x*1E4#O7wd0tZa5B;YftQps9-cKJY3=ZSx_$Xg{7#lso5Jh(VdheiT#?#zl zgG&8=ztCayVVS@DEQrrPS-jW@M5SbxC^W?4>z{7^lH#l`;HSBr?ss9;u|vxS`()AB zS4_RtQbg15z}at>k!dHcANjpo^(UCXO;COJ&aJb}#>3PrU;t$kwKEiJrP)j`{IWl|B{7qV)o^$#rsn zL&Emnpn)Zz-78MA3?vAp{Z$@$O7BX?kp2ZI;_Ib_dnpUj4IqbMTh$`AE2>%HB-F7C zBh(IMn_MQTPm1mei!+QEulHF}3;mW@(Ru6l*{o1H1Ouru;DndhXpjBkj7O;t6?2NK z<%$DSE7Hi1JrMx1llork*?sv8Y6|!161ns2UwA~B$oZl3I-TGhXD5Os8`5z|`l|3#BM z@4Z=Sf&K|2F=mS=hDS2#1T2mBr(IoWoI(4VfQt!fkTVUBZzFveC%-?Y#xV9 z7~_WGnO3*uc!8B(3rO8LPuKV)GRjaDlTRXRnQuOgrA=+!g7r}H%3jH#;5qvfZ|?Y7 zKmIWWr5wZ|2|NMCX>FSMN1r%m56~nu#5Ac-{*rKc>))dy+Yd zoLBoo$(X)9$Hk;F^<&}mZz=wC@hC~(5Ptv5s$>HdW=qq&Zg^nzaCnx*F4*zEX&1j| z|Kd=+kfPj<5|B>$hW^3{Gm+TvI)l**2w9kGi74GDT+XI3rH%Ss6zAgh2~9aFp`HC6 zg$gbU2Ez9}H58A4Th<}9{o*4sj^YMvqO*U4g~i91_*l@I87I`Te^^3wdCr!dj;a50 z!go34SGI991;8pNaB}|SzsW){oE@jy5`*<+I)sJj}tI zqIaG46D~f_rTo0jo1~0eU(l+Hl=X1RtG+Y-fYH#3>QeR+_6phZ0j(J>v=`*~g!Pw? zKIjektdMpE&a~tYSm($bj}R6@HeT0_OmWi88-9GE_IsW6K}&It7Xk$F>UhCIgvdI} z;UIW>(DXg*PX9Q9NXCz2({xx7QLZH($W~c`lyL53lX*m-LJaSQDz7Nc~-*t*!TWw|3-@euw}gFK&~U|j=m+k zEqzzx+<1&^hsP+mLx%d@v09>&UQc3vbGD(*3_wZHH_qI*?IjrATZbd5`UXSXx*CqI zZ#nB{wR1I5hB0$LftHq=NI^hD;M2Hyv-)*2cJXg80RJ9a#{4 z|I%4+Eicb^>$4l<20ej#=ItG3H;?(hJyv$ZBsoODS9=tJKmVPtuR9O(~K zOQo1YQ-d)%?w`h9e^T|HJyHvN*cFPoHFL|z3=ZUBm({Ghh8$SCS>#U3u`R?Z@gwL}z zeGvV9HoR}Z%f;_V^Q@7WN|<>mGDyTBrh(SyX4+#Z<^crdR`5ouivA|Nj_zk?!#xU5 z3zrn9h}?iVY9FHg=+gjetJ#@~dy}7)U?pffY>kZ=O=}>C>`TOZ5nen`Q5qtDn3Tw5 zPp{VX0+?((TV~?5a$XI;)%9Bo)5tjrpFtQcHOAY#(g^Q;BL=>hL%&t&s}T<^l)=nrZ+gCCsZ=|>e;?OLR!qfU%@@)VhBMd^ULBH==5uWRVYlV! z$3uSF$c4S_P%^t8%#KK6xDQ}NeG_#VzEQ&=^KkJnBR-&P`-Jv%x8mu6BfO`EtZx#& z2&ECi^%y1ynsS({wdd)u1{aG|7zm?x~9yYDN`8mEN*fYN5>5F^PneW6|RhMyX*@DGe z{+gI)fg(b=jhQSI#rchoGC^&qO7s0URm$=`7vi|Rd`r$2 zm-BdE9Wp(058mDft?eBQeXy_+$y+HA6WsU?3=}T)%M@Q(H%ZbrYafVY9|VNHrhRw*BPMp9 zjku2Eof?WGIy*bTW2|O}7<>P43-&q&yb}TwO63g0N7@R(0f$Li2J3F)#$i3C=LhDT z>tBr!z67CUeUBSJg2x}F9AWSwW1PFVwrWkEX0U%b%@Aqa3@AXi$wS?NhVCQ#Uf%sE z;(N}oJ}A|?Z8I;^^(1*?F#ix^?1~M3^VN=6q&Av*H+`ZhxIV49i{XN3mE9p&vdC+D zLC(+v(gxPL*IrD@JZHu|C;F?yW7!_gMw{1H>F6uC`g-0i3 z8rVoDM2~$up4Mjf*v%L(i%Yqv4yhqtsiPU!@+yjPi{vjpR3ipbVm^`n8Y5 zl>xJZjK+551!{zm)RsYVycQmb+M)rq`On~pV5<^>@UvQW8$H1#a%}16<;*bT=3wQw z(>VF)w1J+&4cqkSW<&HQM_#UxAzs1fCHVV!aq(^D;}gf`_}6H*%Or07jnNTK((3Fx zZMgv3t&AZg>aMmhuhfsOH8Z*J5)2$!8ZoQ9*heu%q?c{^R!%RzvW0hAORCeM{J_aF ztu3i*8gnzp>+Q++FJPOEH9E%=uQ|{`@db?ha}9HAO+wmuO<@b3#Pq?G*y&qcQi_R} z;#5I>BrW+C@64B9eZUrUhX25aQ}Djc_h5ao6~4bfw#^@1vlSXh>Al7AXS1v})Jec? zi}>6fgPG+fgu^q2dz5{NpwSi|4}W`!XW1?Co#OC;@|fxNnDpu3<|&*NY^WgEwix zoy9^06x&l(ys7B@p3jIqkw_zdm=@K=aAxn(0^EiqQ0>e#ca%Q+7$i#RvkkuGyT2}Z zyn+iOw|0Jit7h+fvGmb4)v;L{vy3^hGy|ufTg$$)gG1}-4-ZPWNnHBUN0Rr8O(v!E z_tUiXI_M%xgbAM!h+Dj){O&P(c3qzS{kUP!Lvc;2R2DwW1SR(LJdZS5cNDKkkem=W z-V&O4+oH^6%*Ob?x=w#DP4}gsx{$`e?bqQJF#ZN8-U72-Joae|M>_BHpArv%B1d*8 zC=D}0*)=iW{-r%Sp{0qZhrNQm_+y#FatH6CBjDXEUzf1a~QuoJao8A2(&kwyTw6tf(XjTc% zN7(D+87ARp@LA-Xink~Cf))-dS#@P5cxr(^{3x0kUu5cv=v#j&dsGvSWPaPz<>;_t zC|eq?ZiDyN58qQM1)CQ>YvD)l3F2GqP*0Heq~u2^1c^4`rJZJ31sc`~S~^Drbfdtv z(BMcu@V~S^D8d{1!t-`Pgu(vaXESrqr?AJtI%wbB&NLDEp(12a z4hRy*@DR| zZ18%{FlY|&((#DKhLOK0d6|+VO#q3q_3?aP$_QRC`(m&4w2=gWN-~oI_geaz(IGUO z*BmMFi;PdGK;ZeeMf1@Z1lrT?tInKSf*&`A>TNZ|bhL2R9TtugFxD>yARu=>TPMkS zaXph{o)eVe$3UlNpWstt*BJ0-&PS8MC#WdHEt*1*XMeCdlZGnqqxd}tN9@a&)Y@xY ztkeQ7YXe^?HA&{%x_%4BI$vKLmkEieeBBy^jDDMWNgv;K>Wt&mD_nre3Ry6l6M71!6UY74!(bff#WoKGfq% zaZ!B$!h1%Hf+yvZcw+V94_TwX;WktEl*$kCZoy__$_Yb|1&FCjV0j@wUd}&_OB17! zeNnfa)5s2?d-@XXM_1#iC=;p;N`apED-KPCar_@4nYUr#sqt*@bm~~sltiF9ND3lu zUZtUUM&BZJR)8ABZAk$@iuhP5canquFZ-8{hThY#;ey)U$VseEx(l^{nK%?2h_mYnM9hR#1m*a5|o&Y;|kRstDt-!iYWJx zlX?KEU-nt4C-7 z<+0I|{|NfHfC-tQXt&+vE<*_XSVu;B`yW9c33!W~$OVgcIgb_K&!`LOzPSA(=!*s| zQ;;$Wkk?(AW;;dQAUEL`<}dXMh(pH{ZvFK?=dh9MDS6* z3g?^ut{M|egSXC!*%gIUD>lwR?H*8@;LMvN?PhsEv4Rh|>l_yS8ExcM*mGui$E?KY#Ad5A&ko5 zg(X)k-GY$(bwNp})cN8W00on4&4Sv*_$q1v+}UHwFxc?Zb#vv{^@*um>2m~W1avK*2l@4AP?t^Lv>53K( zSRmRy^4?|~84BZvem}H;hr#)m=`bzwB6v8$urD)~+g?O90z@F9&fY zz6C`Qb_U#zyqKR40^#4!#!ch+7qRD4{Xr$igIjmt?8+ONT3PAmv5+PO-L@>nX>$Uc z&bO+9jy2TajFe)ESQeXZ^o1h*x}jjjv+hQNmm(Fk}#%QXvGLO5*Q*Xs!fx#QC32zQa|4)bo~U_gPO1E?R(_ru%_`y=0nw?V&fwUpnXhg$ z7ui;%>G*ovPYzOk0?)b0lQIGbrIYPcB}?^nh0qeqH+v%#x2#bauu6_Q#8?3P?{&JP zuZ$4knem=ZI};^Vk#Y_4UTDJ!BVW>(Jyz-lc4|rD=k>D>&vNcp{nV=j#e*!dz4sNP zI!yNIIsaSL+0}RK z^XuLWd!;jL54W6t);N2_s&e?csJUbw;$NK6k^cC^B@8=9$IT94O(_k4aMu% zd?syc=wXZ&0DwyLdl>PQCoeYn)Yi-C>4N!M-|I|_fGx5XJ(A7Cwek=jIN7Gcs66F4i}>`MtJ`56$^quKK*aK-K*b$b^NNam3t zlqa=X8eM!`uiAn9)lHIP{NQc_WMxss&y|p%ay^DzJ@+vc3ea@0S{rlbb75llnlv?% zWT@qikq^gGbCRf{#I1*|GjS82e~|eV@H{O0`EwfRgvdqPl^UOH3H|2-V+Lh>8glL4 zi05Hob6WqQE~gO?Q0MZ^KYrZPwTGi4nTD^nV%#xS^v0v3>&37WFeRPX&(CE!pmL#e z_j$Dxqi6y?vW{6pin9vfk}L@R)9h@0{fUIwk}4a5>2RxRi86ZFCZD z=XdmA#uXq1w)`>;+a1J-$58L89-(&PMr;>hIzCIzQBh#K6T2Fhb*^CKL+WK zIvP~Y-a{V#RMZ0>(rfya9DMhc9LWu}1Rs|WF;R{lkE;O7{#YuY8r$(ByPx=#>M9y= zaSsEs5GAVWRL}`Kk#W3-U2&kXpHCuiW5CtD1cS78vmD{L%cA_i(ZSLe^}B;!gtIJ@ z$dA2?6c7P^JQY>(#6hMx!Z%NR z*66qs0G;C{bNZL6G7trVQc*uAsu5>1NwFnWX@@Ar3lKOwrmZS{j{yecM6=nwJ?yXk zWr!@aXIYQTpXne&t|PQbl)H<7m9+23)AW4u?5YF1>+RSFd9VB3O6OQ;=Vje2JUa@S1d6rZp3Jjhq~8Nd41Ouofv2$0EXP!|<2^7FiO#qtRN|q_~kAS`f)oX={iA^?Mx#-b;&({Mod?5JHy({hh zt7>H_lOsx=aKKRU<{+wxE_d2zTeca%8v_mE`<-JAU)_Qa5+*6^Wf-f*-5wr;6UzRK z4a0Ym7&3dAoQ#aToMSMtjH|@#EC<8vtz5UvBBGAc+#J3pwER;Z(i;{x1WM;~+C%r1g4!enl_THpLi z7{qZ)t4S^Qm%eYSs~n29ttm)7b_Iq;~lf`Gakxk;~eC6`Q z;&fcbRe!pv&f;668^iaNfh+&ttOw{=j4Wm1$d*`t-s1CK)N_n}{7+%{x6BA@#Q#6w z4hN>sgl!GvCyF&|Hw$h8E7wFbjjk89MHD}SA5z=}$d4q%o@Bmz_EPT|(Jb#Y7V*#T zy%)8tlWkaYuan6Hm(Qg`e$WzM&ql@Utc%|n{1b!#h47u!dxHOzg%iF^E4_=vKLV}H z0PNktoY4u+U(>z)rw8u(<<3&-&j8OC;|+8YGl;5i0r#D0SoHS{zsYM*+*j zBiX+*=!|K>7VPl_BK{q~z%xDcrdP7mU1|TCfY5BRR1B+5RH->5c{~q5N6QSc8{Ebw3aPFhCw`28+fBgsRGz}i(SUUCA&}O zY8tBP_)kAi(B6PjboXBFv8P?`=YK!&{f+yuFgQ=>(ps&R#4JTY6lqJ3eIHP4zv*fRQ4vGD*&;+-Pzv4 zH*^v~w*s>nps1M!Rn@zGXA7vclsq=c`d?TlfsA4XaI5y9CPYPs;_m|k596wFlc<-e zfNsy4vK{6^XtKX;fC#S}i=1x&!A!><&?LJwqM4@^&!hx7eNy0PyFdE<-?E$GXTZ?B zzD!T}W|)7pBF+AA5yY{W5T$A8dtA3(lm6%01!1%3ON~WT@Z%TG_YZy%Syv_t7|Ec6tOAjlvKiBC|mx7-y8lY_g7L(gDyf$)yJfYGV zQHUi7$_A%F5s9T$IY$wSgzX_PenHqXTu2zXQ#i-9_$0-oVJDvMGr;@K0PT{gN2moT z9v3DfvO)3bQj8NmNd^+7CGu#I)atMS0u45(nx_4q{HniO+ZO*1ZEqP?RTsVe!e#@T zk_|}LMjE6frAtsm1tp|Qy1ToPloli;L_$SEx=T7lX=wrJkcKlC&;OkFT<KZGRKIw?eq3KN^xIl;HS{sr>;!7TIDI0mX;xm7Bx3C6Y73SZet4e$s0?z>=6;@p|F^(3L}A zZZqWXmmzz{xs!FM!2DDSHc>6e`DLyUhcpa-#$82dX;g3VN&JDjuDqGqI4CZ@Ztv;o zr)WHciXZ^Ms#j+3uTfQ<0XO27ddKyZu%pcP{w%$_{YWCT{x+lMMUP>j4Z3;fR{#U+ z=Z(e<|D)R75vk>iq7b!uVXhE&*Yr)0g-kF3H)#hQRCKR6p_$p?AEJk@N*eldFH^b&@B;Up`ncK%qfX=bN z=X{U#j{UEX{TaT;7O5-5(b-y2r5yn2_AHx(ZUeyLy5hxLH(q(F_(Sq{v!?!w(^6pB z!Qef}xPhAn1>)Ob_>_xRm&XQuj~n~<<+Nys5T(jUHJK+5%+&4|eL{s$wG|=X$Lem~ z(h)TP6I)reRfA~s9!}!Zm#QPF;Fi{^61OF(zdR98y1RrKO&3Q0uwM$LItzC68dOpm z=z-;oa1Tz8(AL?l#LF;;0`qHhVz>9a(>{=d(&4SGL&q*&Iip!W#pls|FL#7Gq7p!m z^cut#pS*UOjQEtU(qiWFo8xyLtq1yvZ_g*;7{J# zd(c0#y<#OCnPsG){~_D*oB3{nf#rm=k7tG~%Rn;*atlnK5hxH}&)v}{O|8`|W9_^U zj@n{0?Uw44G_XGM)Gc+Nu(${gG^?BClHGA3*&g6q; znMudz@?S)#o(!47?V=*LZ%0Zy*dzV%X^(MOGY!QaIM64&9{%;`{quYN8!gCp9T$(} z8k5Tw7bTNkpC+cL={LFk;$On%QLRJ96QJ~#RQbzVd!iU9HL)AmfMJa+$`_!z@lAXm zBgu?TqU|Y5BcFaGkgA0YO^<*f!k1CGxs5T_DxvO1?A+*(be5=!h%pJT48(M z%X$;m@IhgXjr2I}O(c_xupn2-eK_tQc`-QuDSb_&#(i~#OMkPMMgO{ckNW&-k7W7x zo2t7fGp#H8fZX`IyAP;SZ_x%TS&y@hZxWvJaNF)#Uf_G*~((F+UcCqwQb6*$w=>5+J#Mu4a!Uh~C)nfzf$Wg{o{T==tEU^lI1vwY@B?*^%$Gg3|GJ^2{QNM`dS|=sg49#dF2P-#-1Bp8f;;M0j{zZ}2ew;+yA!!5t5dpDBl&O- zo=RUIKiEA~7kGwyB*bCb`0jIlQGRDQA8NRqmv7pOU_D{lzctxrn#6O%bo8G7jyOl> zFqdEtcKMo-M=D)$=W8`TuN>>fIM^Oq)2Jo%nO}E2aG>UutqM(js`0QiB!WTukhe{D zv6VzNMqzqSHmW$v3FUJ_awk!pq}8=YE<{jPZ^4N=O03ZzB1Es?d zhy4vi27p`ie9NDR$H&tYwqeBBYg6Q`d+C zWek@tO8LU*KwlWni}K_BRf||ylCTnr^6C@}!brxhv;X8+mic+k;}mh{EJWuIxxPm$ z5i`D7#uKP+bwH=YMTZuhUW2w5|GMT)AuU9n;Ry740RcIUYWN*UFGCMc&kXuPSpG}8Tkrr4W zXg5PEPJLx|0QaE;RNVST#d*wv5HJ5IZwoOI{+&w3{x|8T1i~|h$7$<~0){p4u)x;x zpok$0E|bmI6fFp*pla;4ZYh4VH2RUcv`~^%@?Ld&j}*odLPT!Y23!X#h>gx5z-7M? zQ0ghzUCRFH1=zTpG-shG{8n3}JR}3a!N8y6wcBWolPV#DBSVI9qcd)RjuKLh+dsm2 zA4tP9BYViquEjHW?4M88vBJL+rPZi2o`3rE7Z*K^i0GS!o~!!n-Q*vhusZKqSC&W= zFNjQX^^Sq##h{T#|34zExetw@cp>+{_I=Q+cOzrNTJ3E_sxaS<5Ho|7x9kSHt7=ey z45ajP@{w|p@IXc;r*Z1u<(__OSF-5a-v$B~w=?7<7cbh>Fh%cP9e1@6|8N5o-u`7S5&nsoNx^{rUJWf3-GOCbebd29m@lJB2 zwT7J47(eg5j4dELC*4LhCQkVGoPUT)vc-Mkcv$Uo%>|yduZ- z-R)BO`@!5I!Te2iNw}V zsjk2IDVN4RAm*YTlS%LneIehz5eb)es`-3^FtM=Gkh;A%iI|NKzTV$#|7O&_702V) z9oT-?W)FmNFz%<4$5vD0*t8AO^6n{%y$~N6xcI47Zx&AE%iIk3dBBsK)qlJN+`oEC zeD4i;U^p#n3b<{B>dQBKbm5`DR$JAi`x*OhYkGQ+`JNX|C z8W3s^FsOZ4|Bv?A#JmIE)Y|0elz}?r{^S4gn1bLXu1(;jeiwSBTj)VQ{eS-M72Hnm zqNLjGVt&3D?H58s;qjEwSgo^pcb|cul=sNx@_^ya<(SWJ<-vnJTt~jv*ZMVvY)Na> zJm#2-1)MZ>3uxRK7QaDSAi-DXmn`ZC+KrqsC>dbvf%k54!zJVnGo@2fV8gtJ@cOrD z>8j%QIFc*hgO@98x91x(J#I8u5k~Z6-cyX28&CtQ8r{Q2i{&nUW3g~tYu#9 zx+cDM&}#L@g+Dhcd|Js1C;r6XK=$YI!d$DcV#Wb2I^^ zNhDSDctMF3m4OTEmG%&M@UX*tbEbQs9m))FzjLgO{%ZWKW7<2V||FK$DPNJZka zEa%aOj5L7CFm&5!CiV*XgGSHq-6Je|B?5dI2HIIUQ*6J@{%pv;1@bLH-{%=fce0V& z!gM~nG9C87AP^lCNXW6L?2B@T8tHhxAWldK3a zC#9SSmW5=IU)p#!sDGy`s(i?`lU0|${Z(3=lNh!FX_jXYzYI@Xe*+P)Tl4%qnC3{4 zs8ygte;6*pgr|e(_XEeablT!Cx*f&n(^%C|20=R-4x2*hBA$|jAcvL~DaA04fbN)? zD4rnajm58rdTRrX^%j2#iphV-thHH(Mv#7~AiV6^XgS{&WE9fT!!R;w5n~TqWil?1 zJLHG`&bcx=XX5~q+vdU1r!NDkQk+OQ{D>$qP#v3(NJWN-oS`WA&Bm|pY_EO+e<*%o@>~MEoq>{W5xw$uE6q^q*dNbW4o*Mn z)d}L}Q&RigS85ZZlE!IIopJ|mF~c8JBN;d-rllV*q>3|QQK zzD)tE8oXGydVHSNaltN}O6VW76@uDbCV|EziW-bb^~swO$8D%gve>FRb`zAWKu$NA zYsEG(uKnLQH8x*a=-?KcT5)DwJBn2}9~l zXYS6I-R`yxj9_#CkZ+nj-HbSV5wjFZ;VpEMusS)KI{?Mm2k*lXMXtI@pgtj?8&O55 zNr2T^T`Ht%I&gcoo)s(o`?lko9bcLfBKLsdH?0ZNa)c7gVC1pZ#{mQr13m>30t1i# z=-4-|xJ}y%hf`Ze+_*Hn=HCUmImo4Kgrief&8a;0P*8L2gGNEw?f zkkox0Y$JTg3@?V1{Ty`Sg1&cPkD87Xzc3*1vXpUt1o7PXGju+qP)d`MKk1#%`%{pr ze~&?Smkjay@?<^I8@mebixbEJJtJJ~Cr|96rNNhM<1!;bw;A5acKxZnx` z>>Y(a5&UGEb(jFSGpGQe=RjI?u#nGbA`g4i%bw3b%+CUbeo^7bbDa!SeBZGorRu3P z(2agY-RtYRfOZp|3k5U(RT*V~OGBSX(xg3eN;wDV&AHg9!13=HI5z{C?2|$zwBFSs z@`H9RWO1@DGHk20awkhT==X3O`1IHo23#VUQ`iLR_klqeWD)#5r*tQ16;u}AqqPv} zlszx^ViTF-nI_ckjLwF>a@Uj4-u7E;P+*LUNJu> z>)m&PysCk)tsdkR>T?>#(;}Njx@C)0`f=GGdxl7tL*MOn+WsH;9H30KOMD{qJ^iU{ zD!1N@wsl#u@G~x^QN^>jaV@z0KyX@x1V+eJ@t;S$QuW-l@G|#7#?kbS&B*vITG5vQ zDL}hhtl&LYsUwoA^D5!N|AlW*I*84J&BsT-dl75-RZw09oMXXj#IoZ5Cc{K zSJ*Pj^Ciwnq4G_v!4ft6cI}tLY@F0TkGwgq8M_6AUq?QC0PpeIZaT}l1D?=x0g9^8 zB*^+{kEBa@W3UbHF&bR@YdOxb>up#v^e!5W_~qgIAI7m^`oCcug+OI}_dMlOsB|Y( zz+@#u-uo?k5X$KO_BOe%MA|Mx3k zm&uTEM1B)SU<0FroXQ+=79D~aOp_&vd~2c_mDkf>M}qmksP$FRhT?LAr-vDaIu^x zm2tbD-`5EPNA~TOq<_GGjua*PL3RUm@Dm04U7p?7PkH?rECr24OxNA@NKAws_vj1!5uFXGeBj4*zt7gG_DN=r~MFW+rLPPp4 zl^D#A=kK+cO-;qD@CL$&EDPy}i#GWTvk#rp)MS%sUwailf#qZ=OdktA`V;+DvHqF% zb$CD&*LM-+K7dnP?tSbe>milrinWTB%su-qB+_;15Uw-Cd8H)B!g;y8A4 zybq#Nb;R>Agel>8ssFR7-eBdu=J>0iCqRzgVugxS!7-i2d$Yfug|xT@E@>56yx`)w zu6#WIl}ZFmLYg4a3?$&Wz%*b6bEXpB$l)!dbNRw_;PdqA`}chX#P>2{XOw-HRnZKm_Z&6VC*HMaV|hdDhQ^u9Cg< zN&goG3=z0V-c^ZPR4&6U=MGpJG|2M4ED)_e`0FEC6g81vH<%m|dAZIEDl0&W0K5xj zq0;?r_7xJCdIXwkr@SnR99(^E{D?1uN+{yhXSC((kGR<7K~GZtU$FSsH~G^ba0Atl zHtIJ3=|d9m@&9fKWU3$oCN>A6y7&el&Bp=XNgyU{>OWH#6gZL;uA}~+sSC99zg3X4 z{m;||368`KDl5KO&A~nghI*2hTm7G@3siiESfHClS^%u{@r5HQIyXoDFGoNWJVg2} z7#$D)Z`41pDf5OX=mfy28neMhj2`niAt0Ik9JB*?QTRr^6xkCe{Q!>kWDR0HQUo&Z zCpsrIk9PrgfEXz{v9%8ho^21>Nz|hsrILQff1qXX7hy?$ zT%UNR2uwtv{}$LNHu%rKE1}Ib*faSb4-vZVQ%9VWu5Jl-e;-|0F{`8F^3J=~usE

      G-={_ucw>#g-d4+K0 zsm-taV&6T2fl9{ZA$4wI=8v>p7bKu0Nwh;hd)HNLafNk(9QkK1pmOjPpFovwt?tRL!FX{eEKDE6A$-v!YLsg7L zCQgNnPG~aX`l@&Td_J6;7*-Y{y#hz$cmKg3$TD#8SYj+@$gq!ZY(UW2mMLUyDgSDv zH`}_9Q=H{edSm67=!#GjH&8lm9QKZ(oZjoO{WgY#cP z!{23|4Bb3vE65C=ibCF12ErqF?}K5ce9Ov9;Rcq#K&TBObWtz!x?>#4XZ<6oFXt${ zjps!%Hfy@;Na8Tb@@$~-fzB|D5Bo@jHW@8ddH()9e0n|Y*uVwj*ZNMK+Zyp3u#>zQ zMP#UWT+-qH51Wv0FNvc4b~+V~U2aTFkg&u!X$}oEqm;Nh*3@EFgQ;bAgJ(B5(*}8< zh5`Ej%^{>9)FZ|i5WtYm-2upW^&Y%E8ha*H3sCMEh{hQBc?@*QI{N!k=bIJVz%Ro3 zupexmmlUEY7NByp!tsIL1%Lsz%e(0ST6vra6^J(Y_<9Nsvc@fVDob}$V%zont8VGYT6bcAdVmd5?9DAI=K|H%$`P=`x z@f=VPv%o#rsKUyu4frfZ@m+#U!bsVdHr$^f;OWe6fxGSI2Yg4S`#pi}MET?ah$@mT zSD!!TS^*HkykW=X`SG`NUMsV|pbT5`ce3IGgy8U>F0s;zxw!;m5ja8a7<@0Izp9?=3#W5rN-*vtg#5#3QjzIZFa7i74qZEv%i2}8hGCEC z0eE}{O3DcR_Ho7z)&{1N(SfZ3;rNus118ncXdb|)dK%#`RT-@cG|hLdh@Yz|cmA1YVSD2XF(0 zNtIb$Fg)$LY27O)&p~y=dL6iDR4knuVgQXJZU0jT;AmfqL&72wa+?!7f)q`;2IRTj z1~k9f&CUQd{~vvi16+#swK@4ED`_ zmoq?6-uAKm<2dk@Sr`;q!p|V$KcEwF$T{@~C4%yt-X1ePmqc-%FHjD_^i z;<8`ZIs3MKDj$}`n>2aMU7KH~qEa;rTvqMCEOi0)p(YJIQCTKKIMGcpaVjL6VhGGM ze~_iCf)%Yr6n#y(GknI0c5O_~?-e3FL6r1^eYDMOM7KCM|4*-{{gmYgK{=i?d$Xec z4IsrK>|sgctT0y-K(-l!H0C!-`<_16oN|-ysZtvBHvS2+ms}m=3Jg4o6wgsuN`e{oXj3=G~n2#c_7JoD1%ELOEs4hOCKKdF5tW|CxK_!70 zpHen}D|a*OJB!P5(F5KT*&@A(y*8mf*~oVSt!^r7{D~QH=e2t#knlUc2+H zF;UA8zJX~uQthb^AeidEk!3}JD)ya6rwe`}WK*ownZ_2~pMrRc1p%Qj8e4QHh&0@M z>oPCuum2;b_IYR3G|F%nv$0NI|E!YR2Z|B7=8Bqe?*eF*e$bjNlx%sGg?_J4M z+uor+_;b2?783uez-c8}^X*hIS-O-+M2NKYop>rz73-j!I@UWaLaNnq?>`xb4b|f> zU7q6<`Rx-~W#3;-V>C#4KQ{3(1f8b+3~U@;9v*l>2=prCK;ap^!O;zN2KHS8Keq)q z;rPFyf=6#SNVY@Dzni4SbI1a{I9xI229HjHJSz8%O5WU1sUJZgjT&#y>-tZ&1p(iu zU(~q~ZY%f|)Oi&vCkCy1FJKVE=!> zhDw(F6_7T{WK=w}J9thzEl(&iAMnOr?umi@_#FlnFObcOFj9KOGb6BT#(6%!*XRt| zD|9+3i+7q|f;@()J}*5;kvqTd^7BL(NYw=N1_Efn78yjX<&YRj?l&FzzOT3 z{<=rV2ZOx;PG1#V3TUPWHZ%9No4l4>^G|77YXR1pk;9q)uEwzINa5pe8Mb`86(U$w z0JtMIx6`!__KEPxrNsN|j9Mo3`S-rXnJZr`hDft52bYtuo&e>akbxL*W)S`@)$l_e z&7CH1AvIeZ{GqsH+LAf5g9a^XNfBxjji>q%q7jtR(-C3;tWRk=S4xESCI1|T8g3^A zCRXW&AdQ(rn%~d2CIL5!zj$zZ8S~wrk;f__Eag~>A>A*|f?DtoL8UY!QDlD~zbOP| zOx!-am!`D)R)uvB2jM_+EKSwB$YlhZfcK^fFN2gTBJdhSJtu^C&LRqjoZ`l^HH`D( zB64_Id;ZU)$r0g}aJRt8Rso}bt-_LY!bbCuAych?3m3ef{MVp7!lXJQE^Tj#n~K!| zDe1KwXB%AX2{vS!VfGDjPzx80zE^}7 z?p3rX+d{qnMYg?MABlFl|Hc4HOuqo}qjB2&bl6EfvLCVP3i8X{U?7ABb~C=%!;c*? z97bTs^1LrJ3PsD+Zi{X4O`~fHaHwr@<(56+eXQ6_8(|H{<{)(Wt!skOqY2gUgCZpN z!nioe#iLm~F*}c7jy{f9*e0tFQEAd4mHGdn2$e`1tf9h*2go3h;I21Pq_U!-kUD12 zeHy@H+CJ~d`NX6D%cv1^*nELMLaLXEuqG7w_JqA{Th@i2%|O2obd3Og4gMpxBU`l{ z!js=JLO~ zJ_CW$pGrKoznWpMQ31QV5%wBACd(~1uXSBQSl<$0k($m?@(nbnk|F=25paVJQ=5arOp zhW;0Fkc}*2g;QN0{QdX`$M?Zj+dpNqqP6D9$>1jXu;9HnV1K3^RHV<#5dEIFAm;-s zt$H6o5UrG7zmw+g=M4@v;?L{O!C~wSh%%H;^8wM2n^Y->N*~fIVOBoS0(g8I6Pgr- zo$~Q3X4A8MoQv;D5ZcG@(n#YIy#$2=^XiXb4e(f*2f*~B$kU{Vs423H;=w((A<8^s_ zQuQx~1x@UAXuRCSdlcxg=(!b`JOy#7KSag#U#`DLL1CB{!@H$SmM7!)%`uGxPHC+KxrK&0t= z^O8&g^E&=bpbG#0d@iRjrbXl_-r#rG7l;?PwOxLqk6*vp_=oGczAkPECx$NJ{05q$ zdFW=^`b=#_<4ot(nG;eh!C|B&qj@v2`@Z%+JJeYR+4uSV+zGcY)Hw7~9axClKE$Ec zu_7Vuq3z2^zlaHqDJb<1+ROeM#MCb?K3ke@j=T3D6y^Px!-}CWDF1Zc$WnpFC*P8u z(etE5C6ZWvrP$|vPxl9yG8ccFK<68L&PPNB?19MzTG~PE1!krnlQjM`H~;LF*mge^*=SRsZy@cdA|nK9Ho@F6U2siMacW6l@uW`Qf+{7>3`t zx#P5`C?s2sAla{5%GZ9Pt|<53dB3#TrcFuHu^&JNtt@hgzu2{$b>F$L_F32|!w<>J z^=HC`h6cFxevXgPZ;9BwCaI5|gp=)nWMJH&L#N64~0kRG&x5PGiGeS#sYez?yrWo#j zRTP0@MWHkE81Y*_BNSU$N-ACdHM~Bqz$2HzD${SA+K|zy8M0fHzU#cTw#NIxrY16$ zWN>wYG{)e)QpYN%51lHD$l_cOZO7*712oWLwW)xzJpdg;2^Jt)K25ngsLW`q5&WeK z6HAs-*03T&^h7`ES0sU3O}1edmZXddrsmIiDVvEeZ|d##@BPh<&EMHBWzolehZiW$ zpt44Z7LC;Gui**P!VdblkjH}TzXF@)%EvQ2*sF1dC@BWY#E=(4X?LP|M=E$~7G2NOYnooBo6txO_h=JTt)psVyhm0G*b-9M47zKX@WbeREnNO zNyvvty65c*+#B*DPkx9YL_%CgYiT&DkSrvwv>RoWBDj^7AO*%!+A8>|t1T3pAA2JV z59fkj=7bgIz47q!^c~n2i`{%fMoWcUkLnOgzI{{Sop=U0!xmnUBXktSo20)!vHTVN-ch-Gjp|zR zguV?&pF%HOyrzA=5m6U7_lrm@Bb^|ChtX-HwG{QpgAdahu@HV4di97{=YmKYVX~3H z6M*Th*Nq%K9=jQ|D!+@p5|zvEfqrz zvZf~Jp>sy-CVE>g&B%A;-J&^OE5?xWQ(k^Xr->+1&i-5QJjF)L1Tq<}ReOiEk|#*E zQZ#AXKqY2mf1Ijhn4ZFCxGuv#o)W9!a(>(pm31TvGY}0!F+7&<{!?DhG+f4Orre-b z9X44k*KUBLD?KuNS`*3I;MiY!pU7i zEPGuq{`x?qOVQq_4b6A=b#26L%s^RAu@UT{bpGy(lRT>yBt=9HXVGnnNd7y7qPF;T z6tqT8ym-;+kD4!%S|8q4XfJXGQMS-qMG9Ue=C>4Qej1%QR>pAZ2$jTWLktpT+7exn zdr_vZ-m2}iz(HqJ8xq6*JFwhQ1ZR3%qXIqO=vTPleCxT&cx9B~+9CR5wQ~Oeb{V%f z9m>|dN$e~%DiTB!to<9vnnU$8$GwbzhtIr`#fP*QVH&uUT`!xz%g+2=``Ea%k)2rnYX2P=-C48}%Q<9Sv&SL0VrkVJOR;I&m+a9srPw#?QTN$rM>c4 zCMifD-hDxzdVBL>?DD8@Y zWPvQtvNl%--&x20MHG_g-c7s){%ri8(@&OL7qlbt$@VwcK#=8!9F`2YI8=WkX!;|z z{#9lU5ehRdjV6+=D?;kzR+|Zn+((q9$JMuxBb4ffVmlN^Td8m2prgoqqjI2WuG zX_a7MB2t_oWc!xV8^5}ys zRrEb0`8YI0N71{qp0!hk=%x;4gd<*1&OBvw);+@ukPS9rpQ0Q>@*+U{r0%{y=?}q` z`9Q_b&sW8@)yZkjqsP9y|H6WBl;vEwqI*t$8+?8BD76v0gS$x0k4;e#grr{~#s0tZ zq5{E?S|07n9xj8X85=C1$f{CF;wlT#2VFU&|Es9?pL*edVHyuj88#u9Hm2IYHD;(&Bp{0s)%rJu z@E&5YCTMLCj6b-kLjM2!JLv=D%L+5ix|T>ZGy8;ZbodQ&rZlfA?U0Q-S5Q1{8~cG2(?CSY_Rd*&@SuYvWzi@>&^K z*f#E}151X$UPP%@ec|Q;X!P9o$#sMjJkd{SYq8*o-j|ccn%{O!gG(PffQ~#U_`Q%8 zTFz~@l7U+o?rilxABci`v0k(G2|895_3>)b2y~{L~@JMo2A30KY4 z+n~u20c!!IU;d~k|O)^RDS%U^Skh|#J{lFyUAm+gK3 z=)6r`gjGok`#$2oFFfZK@MF>tCqNXP^Z9-#9yv%X-#N1Iu41-5f_EgFhc0{Yo88en*b{#y{P~+5g;vRM%1D5sI?SO zAJ3mr&%dg@(khxZ(<%zt7#4*Gp*^HNa*iYPQWF{tsVpmbH^&6Gl0%)IuLzfn_}fl& zzIcSzF=CN;&@$nel}z&h9sT%ZmldxqAV%;kxavD06ZXLJH%iu+t0jvce%$CPVj7`J zma;9@!6%H&*UEcgSVqWg%mt#@g{D~99CbwgrM{rZI^7vD+H89nZ>{07&&i82y+ldI z2w|a%(>6XtkVEu`@me5+fh~L^@BL}dgHpX#h2{A5v0898n)K2E5Jtmrd2)cvzR!i*b0#fc`- z{<+KAAP1teUi zu>#)vkOVhZcalW9$d}FbaA@br4}LH+y~47c?BRhez}qWuS%L_c!QP`xO%XlPn4S{9 z<5AwV2-U1qD%zT)Tkqr(I2HTt;sIi%opm#bWG-q1Q>x#cL%}n9<=>zuBbH+Fs2p6W zU$iqzU#UwV^`Hc2G2JDU7<{0Q7{i&u2iP2&@UlbhOg9Roy z<8$Q2OoG{#uZka{zul1Zi&gXb;bkK1IicMnv_mv;(*YqEbMgpQyOZJJqe5BEw&;KZ zl)MP}5giP%EAr@)L@|%=4>)0~YQZuZy2&P=QGoQr=QXSh@$!X7zaCvm@e8P$5UD87 zcLA)}DF_Y!3c!O;Xtx;QZ}NsR1xkj;Umc|sK{%oS*|X#nemD{imJz~&M zmJJO&V9Ygn!CZ5MAt#`QBskS;y`!W@H5f4*o)K-m^-t`1@oTz{5x$`~59%GCr4pSG zfkFVBGWq#XVZQX};o#qj^APGj?O{hp`acxLHj7f!v9vF)wNhAgU38bnx<(V4mc~d& z3zKx%eYI;F6pvJzcc-m8cb0l=32#XqDAO{d-5aOHEQ%gr6}#WacRkEhzjJM4?h$5} zIOz2Bx8t7BJJ1S<-%Rp|UeeZ6lu2H@HhJ_za^GvUI0D){zxDlk>)ss`j2SI4@U|&a z+}Ec+sWKn^!za5VUX^goa&=WY+Vh)OOZ1J4Ze`4pr!Kck>_y*LXSzSvI=gT`+v2(b zD)|+Xdj_v`?;98fTXP@%a3%?B>%avbQN>2 z>r%Bi&AH#{c|I(+d8Fo5i!+Dfe@gbQiv(+tnXjybd)b%zYAB>NCQO*wT&WZbv*!%% zk4<%ty}y>UFd=)SFn{--($l{oL?W-lK4rFjv?50YmR<>0;#W#b-4ncdA*w*lzm9SW zDsPf%FzB_MGpP3BW{YuzLNPi>Sg_l+BVQh!k8gj~XB< zpRZ1Eg&kYJ3vJL>AE>x@vZvN%8&rP1%fKI`9ye#;_KX}{kpMRO0NUH&5~U|HsOWvw zW3}f;YThZ&^X;G%7zVVMpk;mi8H<+ZUqlrgC&hUA<=wGe1jm}a|9M6|==V1RdeyAW zUH$u34-8{hpZNd^!tWT2Vs~_lwc%VlAPU`ZYEBpSyZmcV53wf%^pXu5)>r4huj`84^gE4N?uxKaId4dg4AJS0GZP&1T#l?1oaZmld3wn{le%x+v&QlHa4Xc5jGHyuB)_w3UV z(!Th8eRce`?)gq22DUfi@{b*#Hqxfb2`E zY(jlBwwgyW2<)y@dak1YY*_m55&w3V;uR`6rBDoUCiSi!U@o(x1&?jv{imRs{MT&Y zt!byDF90yyKCHdn-vp*C+w|8bij(HRTce2G-k#;{J=3W1Qvg@E$%WxN0&uAMe8Aq% zL|wbeD*621w`RQ;M_n=0e;Qh#eo7hVznZUBn$Bl;>Hvx}3Hf0Nu4ukr1$)V1pXj5o z7?jF)(A}s@;j5<41vqrvwss|DFs^GBLSKM0!`7#UeI>wYo!3T^u-27)s_O-vUmq7z z;o@&UHaL4zumk$-9^|QqpTX)b_WQVID=o)#fde<>$g+Cz@aKOvllHzPzDSX4@U)eV z7T$ppIs18zpXfpeNhW=AJ)z%jupPM4io+d`Zn09(4GJJI6OBr)rz<@Emac&bO$!9} z4!_TSegiA6gt_Y;quYxipR0pxo^eLMd;V8G-+*CF#{5AAEIj^f(7lP|+2V!G_l-@( zb2}jL`qw{nYxChdOa8+h|Lf^@9*dFNRiAy!KK=v^lw3;4El=<2H%$00GtxWeDa3re zcVB9^1!GpY`SeM<dec{-a&G1U5nGUU!cX7;W53}T5eCbe)U0yUEO!O z&pO`*KYf%^BlyjvPk*%=o(F;+$$huzHQX zHP7wU1}2un=d80IA_TAkt)%w7(nS9SZaB&7o|Sg&Xj~{(XPucTZPge1jMsK>sLplpb^rA%ZQiN}WdXS=$i!{WBgK zg9ivlPgi{Q-Lmd-0cHEvH)zuF_;lhEaMm{1>Maw1yQBF^uEQ8!Jt8FgebOboDUN82 z$7+?Of~yOnf8p~<^f=XuPW~G+K~%GQYwCv{=t|_2!ed~U?ZESd*Je{<{EC{oZ@T}_ zbM#X>GC_Lu!?V&$#WE5NctK;bxlyeJ^SFw3lhX0aQzuBc)Y}!eNy`xroO<{LPod_% zb4Q=tle3`COoMSgPr_;KnpOorm%mU`D3Wf7|Dw)U8ZBcER8$W)hL7_&aOtJaxFrto z`@5wPgC%12;a_WuKXB(C=b3#2emNOa7*)s(E;asSwg&INWny|Zd3l7(mkV4B3dfQcv;VM#;I9LI+zP!M zmJRNaxU3N!H#JEgs1AOiqqFo{14Xhu*1t2a#u?Ro z2+5)?B+T6jdj#2inNd9Dr0$}$K^G6lky~p^yq&`TK3wsdPBO;$*3pm9y32#?YXO~r z>@pFlF$aUI{nXja`_6C1Kb?!-rS05~XUbwY97WW>*8EI47A@7h6e~6Z#94nDtiF35 zWlFT&O3e4#dyr#n(lWfM?zL?~fBy1oPu=Gw*?@jTh{BQmY_jF^uz|N~pQ1=%dPqYu zG!sQM7r#2DaxK|BU740wymMZp(U+c-zLKD|-8PWNl#^Es(#kRVTcu5c+UK8!x^_L6 zkq1IS9f&?r4~6|wiYOBNGvG<&q~C4JX#an-cIDwvu5El~n#PbcqOmum>@>2KWyV&N zLRk}`n2=ERjC~|qI;m_a>Bw5LWXY0Ukz>ogB}>RYcEfznJLg>2cmDmZ>+|pXyfg23 zxBLFxza>xk6r2h;bf1k0n!%pbyDQl#oAwP_z+vD+h)SH7TJ0+jlE?)Ijg z0yi2pD`nFH*K({sCbBdR9`EQe@SinWh_a@oF`K&{5I?azt9b3sSu&bD=BEZoPO~Viw8UiI{?RKx4~p76(BM}F$E`Fp-yb}RLRsg%m*A%=*(GIJ zU+JF@Q&+pMze=mY^Y_p)=Gmv2Lvocia=gBso(`Pr7#vwIo(7I#;yj^H$X1slbuNRr z>sSON9AEO-P7SM}uI8~tR#bnGKnmkgjq1~VvJ*Glj_-m57mOMpv9ELU!6rqB)AjlA zde2{W>KQTq#|%GyLh`{Hf2vQH??xV#Yu;w}nh3L;=vH1o-HHmT;rawfi|-!gnB~dz z!!%Q5-FE))Sgl<2Z|ht+QUW(b&5Q1;jBe=+Wp@lLPaK&Ps~RikMhj*&|8>Xi}OO;qWaZ&RF z%-YuKSb-Zrx;AghGoNrCkt*P4bq9+$s2S8jd^IBCB@{KQqu!$ayXoQ?Mll-(SyN+% zGtV&qGq4S6#Gh(0Kru9^Bw0FQoG1)b3^L7}{nu_eLC`ks5%%gpB4JgCh8-i-ELyr; zC`8B3R0df%Q%osM&KHD@9kej*tf@lyYz#D>U5c$EiO`mQa4_MtH;Wxn>_w=P8385E za*<|VJDLsy^5SCF@>Wfu+wZiZ6f*M}AZHdI=0RX+M< zgn*zk+IsHVT%EHCTF8PaIjyy?ClOcIyq_>>M_4F2cK&(AWcR&=`>F8pqAa6~B;oR< zrN>%q5fya5@>LZ#=`=2am+H+}^!?CL;gk4p`Mj4~E0m1fO}RoqL{7(){38T`n%HK3 z{A5VA4}I%-c*Wxlw9^(^EH+NzbDiSKd)!=N)vT5vxKLVQ_r!`!CT|?8p62!bUfJ0T7ZFoF8=caDdDr`t!8KwL{t*J0G^?3>yeY=4`B4!@ zv?k(Shb@1jpdaAc_8RLOaO87(F8^#0dH9fZYG2W6k?v$0H)yzceq&5@(5a*{AWFY@ z(jfX8&a;I;Nwu?YOCF_7JrD8#lEZM+AsOKu>%rs zd3c}ccYY-Kp^i@OuqiuV7xx;6QDQ39+3;F{T?F-y&lVRgk-B6e z(L8N$Sx@&)TTtxQZAl85JeF2BWI7)QbI4gRy&0xHkA|Ghz8l_bdgPgVal1P&z&^Rj z>7<5u;iDij+`cm0m?+d9g=WPOhL+inaDe74vOn+doO+u#;B+{19y>_yB`DT zdyl#mO78`k2lxVpQ2evIkW+B-gMws*>pmGS#pI`Jw6()}2EW=~EwT}5Zczpw+{QG+ zTOv;bUqjMgD!Eakoxb|uh7=SPNOvke8Vu@TZ{TaAOq=h5I&*1nQQL^H&N0-ZIUQw4}&xr@uSdcK!kfkFpCZ!lonp5WlEmr32N85d6&FXObW^LOJ?;DVZLVE5c6j=*n$xyZ8w z%p1!DXI|>9cqs|Ykch#H(;cLP^2r~fpa|WvTjvt$XTbDNQKU(Y`_;ynH4A)ewaY*y z&5J!XY;D7qglVG#UgjqN*>|@lDYPV%ZLGO2P9>$v04GrjGv|t+Gb4CbKZURyVQ~7p zCNe|_eJM!}HVZYqW&MS>;wd_`-51oC(ETjJ9oJ?<)4}-{L5_2m+J$t5hDgVlfXJk;ydi(+JmjV`t4nS zyJd05slCmPel2_rP?@MDrq3mz1Iy0h6bHso(7fBm5GVMnPn zPR6fvEJPk1C;$3kbM!&GieKY6dTChsalTpy06yHKR8ttzgcF!_Fle+_%z0e_WxP>1 zbuLpON#)UEi82`aM;gz$FAqJz^j8L{7ol_qq>9N;wW+a|MgQ2hrE4)JzvctdTY26@ zG2OQ9sTO@43nRUmJT6D+sG|vy7%?LEjafz?UJ%2~7;m+v;wa8Ev++}w#baIfXl2FS) zf7sqHa9Y@XbuQB!g9HkHxpRXLNw3Q{UgHGv1IO?C9CNqjC!K@8CW?|=ufXKf*c|Qg z!nUZD;6voN_VYCcgF_+Mb14p$N-{qe7&u>YS2W)#I%BJ%F|KxCriYU^AkXcZ=mYC&{hcY?^wh(h|&tF z{peK`HG{DWD}Shf@Kr_Oz2j8TgM~3^vUIz_LJw*9y&~f zoB!jsR8yo|LE||I0E9sb?^bOy`vbws?4_VJliqBGJa0iLLXxt(%AkTOC_+9_#kKq( zVfxG6_olZvgE!eGVocaen7ewjkDEK08EG%vlez&)+lvGVTbkyLL z?-UQUk!R&cAfWn+#z32DHLL5 zzj1VcvE%?N{M`H|y(wY<#EG#L!Fvi$@fD8T( z;WS!h)&il|P>jd`h!L3r>W=m*Q0^4Dj^<2)YNzN7xX$nF?K7l8%dDYgC5zqnpdN-F ze5N8Dp1B1rJBhGB4OID$K7GyWO&--AM<^vP|nCR zHe!5(7W51kdm-_U)jv4cl`gaFEJo||8myG()IS0k2tk-H&}d+5MzY(OF*!E`D|uNm zSQr$82`2EnXsgeh7^wpeVqoRLFrzSrm}Mmt9CX0;1qDVBfIJi`^Z+EQFrF@35Tp11 c{_|+W{<~=|xy^L*FH>1C?O?CcM3S7bV_%JQUW3+-9rqBB0bU|C0$B)4N6IuNH+)!9YYK- zd>7vL{XFmMdwjp&`~CO*@o*f&T+HliUwiMh_FCs!>)eECsw)uOrn-%WhDM;ID65Tz zhP#M{hQWc04Sdqui>-@>hA&|&BcrJ#Bg3HS>h#L?wG|qgVpxI!j-gJ^y$qw+4^lQ+ zDtIk;Oiu#i?>>w`R)fzzDPit)k4m>;l@G;+d$B-b>}Dq^l=| zjDpZxZrJV&p35#6?Ti9%g$$EP?`JKJjo$kDI&_Ipy5P>?)WO{?>E_T(3bKOosRtP( zCHwDw7DPN#y#>i&PCQRQf;oEBUvM3pSszZ__E)WW$S;9sal7N*tv4Ntp?Zv-{e97Q z3??Hv`(MftPKB~ojGpl@M6cbE!B|BaN;e7#>6@ER!XB zVsRJrfl?WtZF(8Io^Llwq3tKVzI9rVFX5M!Ch;Y_tl<4n=ip(bdj4mcEGW+2ich3) z9B$W`6H8ua?!wIwoJ9#uTCNq5dUWSDlPr3+T^#Wy>BPKHqmSfBjcJI{S0V!$a|KVP zoR=n*l2gCdQ;I!{9oxA^jjxmGeLmZ^pT8o2UT$Y1WKMj_iLR$ejNa0AJg*p=ytPgF z6K!Ks=`*o?d$h{I{x0~zjET&=h1J1J8QQOsmjouyY}t;|?9##Mt>3#3&&TqHY}!6D zePq1%^5w6wl>Ml-f~t4M-7)Ug?A8=$xqB4nD_d%j+buyG)&A@EF3|&rdcJ=BnzPVQ z9*4Gl8d!H7vzJDI*(wr`E(CeYO-%7!*m(NxqRvx-d0S)~WilppJ)QL;VRMiqE`~Y7 z?T+A=5AozP@lG2kRWYmD{4_Co+g!`3KI5de>6epkqvJ?n3Sp}UyDgH`V>kx-I-ec~ zQhcIs#>^zfVr0~P8?wR77RqV?6=KHkpul^|^udtjzLMOdyNpcIuNd!qc$7=?4EkEy z?!(o0;?>8l=$lfz3?_0HOKRy>uKgLKqzhio=AB+H_-PwUs~LAVRONA!F1s}Agc(TRHI>tQAD!%#u9Oc zcQCHSDJyO%HYu?y87PV85-Im8vndZMe^usGewxRWcRSBSnLWOgOI#1Bs?Tl|+wxuO zC7lnp&wU^2hA_XBSs9e-$R|S`KaPX=2fcp1&y{J{9;`{MMO9W-rc@TVMpjDsWE?(U z^Y2|xXo{CCwke)0D%WKzRMbr@-qsY(b5&R)NKmfHM(XGkp8v@EK?T1Lmx1HxeJsx5 zGD;{AF{sh5F+~)^Ri0FS%%jqAh<%>*{N1#4O`n}V{G|uCN1}*=e>6A`NsKUpV^Tei z8llpnYM@ei>}hK}?47SPT0M$Cx@vpR-f>(kXG*Qz=GWtUQ7T4KGxDM;TO&+%&+IE| zDyw^|Pi$)J5%%_WO?K>d8WW{IBueLsf;SB}%ROA(6E|sw8;XvUCKQv3`X)?18x{EK zowm&5ibKR(O^fDDTd2XA$D2rL{E$v0mtFWJpd zOy*sx>=4h8iV%K+I;z_Hgp~KG-%*A|{=AeT^yA;pqpx!x>YM{ z6yi|3gFVCTS(u(`l5dhXXXX22e$Mm->}#tfi+S$4;?j3`OdX9bO)}vv!Wu%8Xpv~* zXt`*DXg-B!k;?aLEOfUDozxxGh1F-$cB6jM&}YF*z@#HgReineA%iakuxpfbTBxdX~BEAKg{D}_-eM!VH#n!=e zN0%FGy9s+9ZCiX!ID4=6Oy-BW453TVDS`Z%?*8{HRhklM94#LgvtoTA0#m-emqu(% zJxjlQ?-Bp1E(hadF?YEQo?7D6zEbnk_dah22<`C<@Su0h=usTsc%RaYJibRuE#SE} zc@dKs^X+>G$KB58=l2l|(;{HI;}J>=Rd-dZ{wN_q1o8n)8c%bO0%9zKJ|61cdCH><7#90QOBdIdm zY?_U@3$o)*96sBcs))ohhey_jUv;0_X~Ro@@TGC6Ke1+rO4m>LIb`KB;cT{l=~Lp} z;zu7Kd~MsP0ro*w;b%rB+;2$6F%#XeEwYtn0}yuC@GM zbi{VItKRYGL>8!KjxttKB!r3-fYnUIYs_eP=QG}~s`cCVPrRW&bRePb;fpQBSa{R> zhG>kWfHtiZWFpk!5Hc6p$)9l#n*t?3n%R>+CD!Nu^YWQVRZ3M|zx1!%0ox3Fi8^n` zq2>=hqU$=ptcR{!z84iXZ|uCuy`|PE-+2_?1i?GctCg*hHFmYnI{l^=u`l;VZYRRz zUYXJ5mv{qkMSs!f?Cn01-Pj0NG|FSAELdeMsg}>ptky-?rNQ}pfINWPGtoYtBGIdi zrl~T3++T0AxUPh~tW|UAd4(;3Rn$wh_36->sI3P-dCX3&{C^>-6VMZUOYlqX8MG|4 z%{Y7HTz569$*1-T_)kBW^z&8uo`|$7fro$T_;SlA^R>$CPbBq9-A7=n^wmP<7-vTs z_&u#vq7>8yc-u|->_(m|IqdD?HI;4S&5jwaYt~tn4bE5kzsP*KaCB94;Y5ckIqdhr zrF{3pT2^s;b$cE{@hTFD3h56S>{{)LW;R#&nA)65Nbe!mxRShC(YrF;os|A5jY9(D zUAe#7#yL{dI(EOHDV1GtL6m>DACZ`xcv7`wJ(PJz!Ye;v3Vhf+bZ$w}%uCboy?)ZU z=Stf@)o5mEVJ<*&c6Z(sGPUBmLO#T;Mr$deZj5LGBcCtUxJ^t=@XdubkGSwJ{n{}i zb@Z73H5cH?zCKvebjwR}A=N9>8Fn0m3=V%ELR4~Her4%Y;JbcM+E!Yt-~!$CaBDIX z8}z-*D>%bLD0S+@6+9VA%cR{R*ljuVyFALE$fn4=P)7YYtAghZ^vOWZug8B%mcrH( z&c+wFSe=d+w!OEIwLT2s#igcMugry{fZ^-%Bjy#$t%6()4M~lQr^w2alAVcsr=^8m zqF;oB;X)SC+d62$WyoY&Ep9}`otQED-n!jRC5 zbvi{oldnTpFtof%zX$$moI^lB5pHFu^h!+)jSINOMZ-d;Lc0ZAp#v`obn1Uy%cFCk zVgCLc0}buHEgII}ZPbDHo1YKB>*k)n-Z5iC&~SjiNPw63Cyaku<1T)}{O1~j1NaV2 zT1Q4n33%7BbhWZ_bhB}Cr+;gr27G|$toY0g4ecK5%?n*gn`Iy9f5KMR(A`i?Rm9TC zf!F+{lZ6$pw}bOdKWJj!BEY4CmAg5Ex5H~kHxX}frr#|@fa{x&`Is1fH*vQYXEIdN zWRP)kwPFz9eaQQeN#ZsG1B00B%U2@Wvhsi54*Vs~WaIAcEW*d<<>kfe#n0>HYRv}{ z78d4v_=xY(BOagykDHI9ySX=yqZ{*Io&3{}td*OktF5!Ut&=0eO~2+AP9E;!OiVWq z`q$52pG5BdJpH*l-i&8H%ow%%5+pUK)f0BZ(3L*mJk zM`FL*|Br9}{lx!rr{TZveDp~0zuo#@zWVc4JvS>?87BweneGz*me=3U{jXpCeWMuP z&D8(PQv9{fzdr>QTH>}C-@k+=aeJHIZ5@y!jjgN(5Z|H!T6Xh;P6qq}H09>~<`vlg z>>d9jG&CtRC0S`*Z}jasoMbV%l!yCF^x+m#Z*bp}5VGDUB7etE(vQOwp7xShH6(_>USvj$A?=naQqM>8_A20e9UvT0H@w--${9hXW?kq@K3L~02=>OK~ zP1`nH3=(JgwkodwO~2@PLc9N~b{NdKif#ANhl>A?llo`kEQ|Fx|EpDqWMDwI#u6(b z`CqKq-4B5)bpM-n3=iJ~Vx$$Bpu1rHqaWNzE=_3N4>M@vmDWUN6;_(vCw9pp=s63# z?8p`RKYDz}rb%?TI<)u0#6qM#HF#dWMcH=OI@frn@nJTJ`c_EOA%`yIrgED%bZA` z4$FAlGu`XM%EN(Z6fU-Wi|Gw?*ac@ zg-_=Hn6oUN?Ci>*=-z`BI=6(k%&1D)IVQryKblc$y^vVuQzL8u7970KUKz8jai|v8 zmreMG4=c~Ty~>~&d6uehJSOk{<%=eZKL&?)$)`LCk`xOUUH8KjI-oB-~dKC&rA66kC7$^wHbGs zeWZKNjweLCA4UQN!PT%J%PyVd2Cu=!aN<9f3oFqk$jGUsW;>EhNyvvKO(<(h`5Q6h zS+)}qj>aF->whCPagZeNe)1WHI#!S?hI+q0T~I!`zG2otMUl!Mjqe&-+;VK%sYSWZ z3Zt__L-?uL6D;|b`Kq#PssGSYK)%Bsmw8c21$aU}$Eh<;uojxDS5sC=4sNgEf65mt z(J*M^16fOXqFKPXF6}LA0fn}n9zhVY{H%|}w?xSK&y^0rFzGg%ruq4qeNlFiU72N} zpOI{Ep+-;k zMU>jY7&I^CNCp8R#h{QAuzWrJXL&mnMSrf}UBm2Kr7#rIy{>7Fho^4V)#^|!mo-1>L{xP7bm46-sAY?6_P30Wu8KH*%=?f9ddSh5mX{qwU$N6sz8% zfk}LGx?Zm-akfAo_pt}@2yclGgKuL3O>aKj-6B%{<#2=!BS7qs*gui zG?GK?;*a`nhML3iJn0V=ro;&oUsbi=cfe?Qz96AQUt;XAE zN^Q2@=(>NJ<1W)rC1)x|LU=+Boj=(BG>?zTHn`AO&8%>|0Bu& zsOx|BS&!@UYHq03yWhUPYlF?@ zk1*>+2*xts_1mo*@K&kVX3PCQ@YF-y#a`|*RrXB3!?sfmkTQx1OPM;T?tPB!pH2wIYei@oO2^ zH%b^f+H%dh{s*7kOOaw(kiBc88K&i2JM}76+9_(M@~`2aB)VAKV3ddX7|Y6*VDt2Q zzmZomJNiLV*uF`p8QZQE(tyM{U-088RX2^BYU4_@=qM)rS=(_oMT?Dm_feN%E9h#+ z38G?O*5Wzse>!JfO~B1#9HheAz>@BNGPQ=jTiM5JJLfTB;*4`q0rh`i5^%n@=F)Of zQ>LcE*K+(uR31n6MY0r@qUkkqBB@8P9sVN;w(C)*qCn^K(!XNj-xw;H3yq8&HYxvg z^9J=;Z%ZnFQQtpXfojyvgYBQ&f2kRz^W7e?n`WggwU{>=G`# zp7!)u%0bY8`G zSyXj;>>6cuxlFOiZFa8Qh=iAnMG4LuvVHvwJl7V8K~%U)>zAu%U^8>O`q)|-`)}EE zkA@^6)5u`1&&`Luo4-<>)Se*(-X|r{Xz`j$!QL6hvCq+`-q&&4lJl*9Y4s?9S zbI*4{bk2QDZ4(6bQAGlBtXLe}8CBs^BZ8c@_pN{tC1&Z-63C5PUXg@0ez!8t`>pe< z21QE5zQV4r_C($dENUyLUbO7_J2g(U>?U{uBCftX>>GN)9pOaolGo`qkPiAwn^bPJ zY5KoulS@?Q)wcEKemjwdb()KOc$Z1bED$uQU2IorfCubN&5_;<#S(1F)qE|Ab0)HXxAXe$$J9Px~LNg|?%Kg0EDYEq;92Y-A z9G#uS3AJ~P;uBwGBvO(*@6o`Xb0kglUnb2|9P*t%>a|j!_0YW^k)mmi2Vq3F#)AbC z!gPTAan2E)O9%1hq$QL`@OX&`BDB+dG??S$-e1R1`uG>cy zosI$fBUJ)W00O4+*HplR$IyIIrBs5Z@MBj^RKE3)(ESxK1a&ez{7j)A89=He6j207 zIyR6Qy#Og38>lnBpVlS|+(|0_>JXl68Lb2Y>&0t8kYk&|Lq3D50#!CYi4C4VPMQw; z9qtDCP~3$yY>@ylRf9dzK*oLaf)t&S=b*!ZSE&w(sFONa!j|~;`H)5#c)X<7^ciSJ z{dgGpOAwY~dKjA&-y@{OnIF=)ksX135=ecDl~y3O#<{6thSYxU*swObQ@1eR`+N;@ zTd~cB{cf^JP!X89TTuX1qZ+h7=yCVl;*~r+L>P4W{ca!>=FlI z`3E1dwTx}47@Ypz9#w1%C|3$AgM*}|li}?Yth86^U+ly;l{A#sGa&&D>X!_ukisuc ztH3hpfPpQSZc&}a0fO#k`0hR{O9Xa3Q^hAX2J`~#W!4~KtYu$cLlQN=%yrpR^QL6G zE5*q$7Mgee#9S{9aQ`I_&i^KlX1M-~;`$Hu+I1(j>VM7jf#dJUOpg@^KU4Ej6tJ_- zN^3J+-{Ty5lW2GTaV`TB554IZ!35f>N;QD7DOdm`5(mCpt=87C+qgCAzw4Gsmb0^3 zdDzQckm}UhKx4aJVs>R`Dt3I3pX$h^uo8xtPQ2Npo4`(%dG0Z;*Qe!>Zvr#O9WP$(B8&)<+t11 zsttS6PN?%C2ydDHY*Rx={OgDQmcPgKG13tg~9L>QNO@7xSmBh&rm4=oP{(^RXGVx)|vN*0C=-5C|CzS^I0* zBb7l3pPh!4&a%ZW#ZFZIYmu_g-|Kdl67pAo{fiBqeOk}lAK1NBhykjc%dQ`*C0}Z% z+lZkBGcT&wT~(Rn2{MZNJuwD392)i;fz(Cc(O|dVW?oW0;1`zG`jWeTZR9`)ouZB@ zFB%rEb_MgCgyh-l>k0Web2=2{Z6}~qI-y&44;uEvY!_w{>mg1JYl)A#COzluuFelj z=-PGhgwFW|#zR@h9R@m&U?@b{X0+(8;#g;sd<898Q|gXe#E^-f*TCvU75G7pACW;F zKgCc%IL>&S6X0r=!zXe6%5an?V=XUqA2j)0Qa=uufWpqaP-)jjFq_B$J5CZMupDNE z!DA9|=Wu*WcGgdBD&ML`=)2~MmectrgW2zE3(u_6$2T-HFJ{+ICeDG(kc4eIlTVJ{ zHJ*m|x7^diHx|<+^Ms0=vZePHuCEq+I}1gu60);&c=_@hdrMTomw=s` z@a^O_9pNS+Kz0L0e(Sh)A>d+uEy+xxD8NH{gk!G-aPoWNXJ3hS$pt+c0%Bf__znlt z{SaLaWd_UTls<1UQ0*D#CV}?Cn3Qs0u;%R;sNd>aq)rP*tOjxhc>sloRl_4pkFzW1 zfdrUX(O^5>N%>)m43T)XUs-Zr{ppVai5df}@?OgbdrcNyMC(D6lj`%jMuy#M9d+S6 zgU8#xr+`lPc|;I-_?U!`M1jr4OwPX&1q?@6@=#6rCo!QH7~oIMUkrM+@;f800h7Nw zA$~gR#y%c4v&qSzvl-yzO9}5mFSixK=f-T;b>}n%_l%sP-&vPH7FZ2 z>kHVj=ku5@gDzJq0Ya-EWWb&7uk(~pugSiaA>lG+?-g4428C84O=A>)`cCzk{+zMN ziR$(5XkGU4I|+XqgYx8U-v2gV+O#`uIk($__@yP^9^M2{1LwxzCR$6+nN z^1@}-g&^Z{mHkxWv|&iC#;##UAoCrvE8s9L^B8p`d0jPYcEUxH1@7E$ZOJw_EtA+p z_^hW-%mBZ9uTPw=fA)2ovaBM)mr(ELCC8&Wti>gc&YM^kF806^O}%z5^xcS1k@jAW zfc9EpgTAWsPu)zKqmq$KVfoi7eIAz2c!TGM5aC+Ru224**pHB6uwWvl4x?1+P|o(h z=g_UyX90!HuF#!wT&tlzycf$QUh9@b9iU)KP!V_`-c}kIUUcQT{$#>NfLJ6CLI!y{ZRt{As&scebQir=zq^0esT`sm_k~_JgFaFH6+pWI+G$@uh*fgK_T@t zhyKb2BI?51`90stNTfS`IU~$LHqE^8k+R>*cu)VCv*%Z0Oai2>#4`OZdG(N=0O+5u z=%y3hxuRa+z5!Ex$ri1UYlPW#&1v(xc-iTe*>z)eALyeUb$tNM(VBr|^d<8CkhlCb zDq?TZZBV$NS@3Y3jc#Q=%D2Q+UF@I(b=q!QkggADx<)EUBDG+q9o-+};a(cEf(Q4( z+m`e*&@Y7Hn9&y0pc(*Cdp+IeT@JvZb=UN9lBE$gTR6kl>jRKW2FiBoJX8kf^lN2)YuxN6&`lEO*2^`%Q&PD@&tm8h1a;&q zJo)eedO03vyhR&>6>Rh6^knV>zUq#OCr&l)R@J>J)sy);wZ z3_znDCwtBI$4=~T9#mXe$R;%xUMEw(i#gl>IMJM+S^ojbE-Ambqmzl$g|xuqoI^WO zjlG5~S!z`iI!FoTa3k@b3nq4=h$Juf;f{UlI;@Kw)~-YO z_F1nqfXbFw{K%TW zXY_hx!7M-$DAdVbcp2Vjm3XAw6!0VYX~&$vtaE$TI_)}rm{4aS3#P^uALLwRYKl#C zoW5OhInn35oKX2_O>jRKb^Xl_Lml+AXV~|NNZj}eTjz$_m0NW5vvy?`KdV%8&<|we z_+CbRo!h9+yn07Ru@%{j1=-1?qK8gRD9Qx~KJTx0kn493YV3qjXDb(bas_qLtZ@7b zy3i}R&%7kf%hzTI?HbB8-}=_M20xxs0NtL*{e`gZCO-fQ8&-!I`KC4zi|&B3`E2k#bF2%2NotxLL!q%yXh{% zyR;wkT0TypL@?plW}`o`d7A=&&Qi!Fu_AI2M{@E4fJF1$`Hk?-gO_oNX`vZHT0kTi zZu2uxei3+ z&3o3Va;W4iMKKHkghJG3>gqECx~+07RFuR&Bvu6rz#_?6x=*lEjvRh3@x#b>r{-eM zz}KVGGhew)crg$Yxz)+R6%6k4U&N%}mi zi*;uKh_e#52r`^<~7FZkyp{NLHmdmi78d|@1kbp3|M z9*?Kt!q=s!MEsx^u`KZ=X~Wi#Kpy}R#L1C#krgoblW4jy#G|;Or}Of&_QL0OUuu4G z`F`$nQF!r$r0OtK+2xXWxFo6NWV!_6NbDTt%OvsMWqW+c1xpm0Rr_k00+#%tnv9uI zZt)Q$NtUSRH3bwzE?!qhe0?*+!1qySg@zu?)T?x89Q99q#f|ZZ5Cvj){5q+>8VoBUke6Q%WSb)pfwQe zx8X>XZ_ko4OOxqDorZ5Fvsb5Dc8r6=ax-kK*BoR=d#ojnQYzIb((|(|)zrnpqKOeP z-XW}uvNIA;cbc8AH>MC_k~Bk=$8!l}HahuQC^t&0^v(ui}4yFHha<-?$n z)1~yR`FAITp|5B_c^l270x>_xX}^ABx*$-&xYKRSzMB$SBx}4JmwEZpDbxRi(=z1z z%9|!WkUuj(XHI7vK<_8@3d<5v>QM(Nq0#!njud+nlI+5CWG<=jjlOaJUxUU6S1~J@ z{WdWxh0V@*G{++lYE`Bh;+Fz7JSx{FrQF02O5JL`3NGtNs@a497cf&4C174>F zk}lZ$%!RTl7;}jz%SQ@Z*o^NDBRO8W#Og()1Q4B8{Mrnw9~JvL!}Ax;)kMvhUFI1K z_WtO^?Qn1c-ITNP3(eGW!=%*4NTModrSAXGu`R|*yNQObJS;h;z+o@L6dS+KlmTVc zHr|u_dqfxJ5>y8uJJlayFXU=uWh9(LW7V+};MK;&V#P@t;<2Y@kg}ewDb=>Hl}Wpj zFWA`TOu}55_ALbpEZcDj;o)489Y)`3O02_*Vx!HqHP^RZ?!Kn&Uu2M$}bChYAbvjlFJKNrHeC2M&$ zr??*`;?h#QE&{h^3=tfW+0? z$Jz*OhY5vBnadZZeB;C%7Gz;|_u~Ue9XdA_iMz?ui{a@$hsd1jwPhhUQDG6UbO`F) zr|PO(D4OOd)bc(nI=h8&@N_dM&%5U8o{lXM*SKsv61Ov_>+jfV5(`xp0WPCO z-5$l!rMmRD3Z^;PMTAZK5#w>S#4hh18;>L8ubd;0caTqrEvnbb7=y?1QT83svW5ze?~LysoAPn}Vhm?S&o%{2LdOPI%wy7v$fX#ZTh1!qTxF|cKE8V z8w}j`B2(c_m}k!?Z`IF2c1FDUn}a(UlzG;MVI$72Em|bb@Gxc{NX*07I{KW&A8|OY zEtH#2CQ6`@qINgH|Edu5ZbO#rbW!)g zW=Q(FY~LjAj+T|6<3($*^8u%m1*oB!dYf;rPR6KNMrQL(u-t?X_eB@Mm8+R@w%6%qd0t zH+8*3qP-X_EgoXDftgT7>GuhCqtfd8p^Y)KadCmmG>(m%`R1vVjed-PUz2e&EMj}p zajfg0;eCm1_5Dt@FK9%bTq_kU@eNtVHutPpmXnV z>HIF*9$RlQm-d!C3L$o=5;+?;#0id3(<_#aG{>V0{#n-8?Kj=7Y#itJK<5?$NS$w0 znjSwWW}8ZL>AHV~xG98~H*u~j4_y=m3@Thb(DkK~dyy9r>q|8r2eq)-_frQ75dHB@ z(M2uUj=vR*(@R9b=~Zwo6tSA>J@=B@&HcIOKa{X!x1b^nsS#|G*LlC`9F8Q;7Rn#Z zSvB5~?er8)54Hb<#yGedWlcA5f_Z+efm-%;SxIb}*fjTK^#C+9ND8X)od2ck(62S) zIv!(f(U@Y+Du1`Pn^cL$LD>`P-BC{B60$)_b`ct58j_$`{JmQXx)Vu4A9fWrT1x*U z|KYY@NneT1J7DbD%-Z!G-?fE*>6i#gmZ|gWYP6+j!2JB$=}Pc~bo@On$tuneB>p_Rouz*me9*BQ2;W+=^GT;1&7>*>-V(q_i^a4heetDJPe1 z3aw!L<{~+iMDpgTGs4gM!l%`1S|Xf#;3T13{I?q@_r~flV>8|QyMk0wx%T`r$N8sn zMV}&ENsgiLk{gVkw@klnTQjboTI2uV2LbsJQ-K9C>2Nh~&6VaTcV-h8L9~_x6A+coAi9wVw%oP1>?Y z%<6~m5L+(!`OOLHliUsA*w*7bzC5NFcJdrBeu;p)&xoRu9AC`g71LxK-9e}V{^!)8 znq+lu@~9;|k8rF!UDcuZaI7@<5jhGqZ$20YNxU{F)9t)+V=88TKSXe~j!mpf_^mDI z@-{W4LPtJLEHe?=q-`fID+NeGwn zd?p^fqvizG2_G_@HE@~j{H}~CNS-+kY$}&M@}R09goU|rj^AZxD0fk^s0g_ZYhLz} zaVhE6y%zJ3((9WV54l@u(jG-ng)UZC$j>jWTB_6u;fv223KiRwI&r1P>n?^C&bkEi znHcl^Opa-|_h6{h1a`S$R&wcVS0z6Ua_P`RALAiEXt!th`69QynjckW6Vo}zO%kAp z@>|pOFOJN27P*CoNB0+!w8G76bcoM5Bvs_fI#CiI#W*p-IZL%>A#!EihsZAQ^#$xY zHJqhEm&GbC+v^2H=OBeQq?gx>LcB&BH23_v1sQi1x!~MRP;@go4%MqV>-I8Og%3D( z@0~ysbT5<1X2y&)quYNvw_{y{@o$b2kk1dop%Xf;US6o}RJ1rS^(=8=o~P6hYM-N+ia1CVPXC}Th9hIdZ+tp5#_Cd zS;G!~rczL$t|b?cND@JAID^VCbe{Hkt z+~~0K6_}4ad+G>_qM{GmiTVq`Ukv3@qBlf3bY4+FYR2D<#mV3YUmYLIiWIc4UXHgc)uC!tzTiugJSRriOfDU6w`s zC`gyJ;IfHUI7=N_MZWPo;EaF`yL)H_BGBWCUF%&Zw&oOJ%@6e;*9poioTz?$zI%UM zU|ELvD5CNjby1K_x-i^66L7JkGlEy;C%LTCS*?B5NNkTE$I*!^bVcdiA-z&X(ftf6 zdTLNA7Iv`<=oPo*5V29;!hN*|7W`_k_uNfsRR#R#*Nyd~X(n`{f6v>vTz8^klnL>@$yoj3L3x{wkeNUl-<4G(3!_|9iE#2H(EO~XI>=xnuP*`2F7%) zCUJ#)LRXrv*`VWEk`|dGZn@JdoK88o4^P6e-kBccT*tRqC&m*y7q`@46C}}y8)q9# zQQ3bUf$>gdqYUW*aKjQNA~)Kto8+?LJ5KBz-ZlUBs(Zh;UAf=KM6#Bx?1)2gJVRQJ zWVn1TzORW6y+DNiRCny5-nuVzuf*5_T)?(&`bCFDeQmt2TTAlJO(gCMve5FNKkyq< z6Wk&H+GDEv~&iHCiFlB?wp4tBY#@X2^fCWlHgkO`r^PD>Whflsb9W$ zyhT%R>(=@r+0L84i$xFFF+faKk?#B(d|+#y+EG2C&z&@oFkcRNcqqh3!8k{$NL^ev z6P~yE{m!_jLrfsQZQV#lxPWzwiA14;7A-JK<{l4q$*W_f8HY*+XrRu*=}j)3Qz4hK zhR4AD4Hr~gemuPxP+pVBK++3d5z2&a5*=v*ZjiD@ZK*oCgqsc$rbpH*7YQvQ`zl2vqfS;+Wtp%}1 z!3EvmOe$2tCGT9bPUA1{RDbKnJ6-?doxZDQj#nmQN{wUNU%r^HCTnp*5OK+-0J)=D7Tl5x&VNT7@x1uK&~#S^EXipCt;4e69LKo zBP*p4)lLqjs8Yx*bEl2anz{b_;tacti$Y@u1~1U9@j^Tx`z7lifg`>l9rYmGR1lMI zLRIokMa3Cim@n%anRB;PNfl7@Zh2OFgw|_hH~qw;LH7JnE_oiVJy^Xyk#`$ruL$sP z#p1pT4*4fk-s(O}0A`yy?z{N7)eT_gn8-MSR59hYl&uo`zoU^h<2(4utzlN6@j(l2 zlUsrJ+%8#nv)&~Jl2!n<%g{=byE+~e%)!UY-Md77i@e9+uLM*;J8CJin)oxbl3iy@tyC|1FZ#Ogq8F#c+YN;k{h}T^ z0x3WX!977uK*r49^0_xgN^tp*qTLj+B^y}sJe;$s_w*H*_9kjQ z)aD#$KdV?P#7~l7a#CRLF{5q2~^s`y37d=$&Q|C zG(@GeKI=D|Ur>1bW6SCXR$B5$_DmU2CfNVzYQltVadnWy#ka4VSiLp3728>x=bawqgX7h1TKZKyM&taj0c=2&W zOcQeuXHx#X@iPllY2Qrp-i=#o#W?xy(mO&vdz zQJnRRmM0(Rr-prAxBaWU3~$J-Xc%0P6;K`cVR;#$G)+(qxQK0YwI~nnkq=HU7z&T^ za;#qp$P0XkxA#GYaFgB=Un#wB z-gj@#%ydUCWdYhviE)7fitcR}$91qP2MAU=?pu8B0~3-7;Z_w3p)I@n<7JIs&i&>+ z>le$DdlvjMa{T{w#H9br?7V5`=7>F4rIBgbNu03<#SVTVetqOwJ8f6;i0y7proM`LTM_gH$(Y=_fJ*fW ze1;A{I_VN663?9xXS@It3FY#jgV=f5P~o4v`94shD|Ir03hI?fd$8;v{hzB50yGH; z0~3dDN6Cp23mOP8kXl?~z-caITf79g6Lgv@nGuJIPhFzhEnZ&iHaXee&5>|`U7f5Q z6)gwJcRy*47;_a7!pve9qd8hQ$x2dThDl(=zqL#JFSa1wAOR_VB6 z6iK5#NLvAh=EoO-wi{-c+H-LX*O4Yh!8h7H3EDXOwJE^R*&FVUpcmI4)H+4`^9o9(4zRWI-j}HnI(ahkbmKMe20C zp2Zyzc1~Zwnw;R4juU1O8@P2`1?wkM0~3Aw9ka7%R-!gx%ahsC5;hB3!c-F@ai#M*k9&Lm zod}?iWppMtXGb(-2^Fv16Wyp^Nw%b1^dqn+JOLLGqKnzo!;xM}0L!mdFx>3Q``4C7 z>b&x}ZEkT+rq#MRbR6+(OKj&~CxBk;VgYLiGDda`f=Al$>JHuNI< z`M)bX2U)WRyd=`t9@vqC&!^Z2B}uA^Swp@I(caE!F^dvj1>Y3;kr!bOKS-T_jq%a1 zly?KBBJw8*v&clB7wlqHa)sib{{=9I^6CXRU?T1;pWy4_3o0>ph2lnktF2HerLZ{t zmc+$2|Nd-o4!6Rb0C0d$aBxi10SJ*>sGXW!NEF=hOV|oJHl)W#e^_L(FNSjKl@+i5 z!@q4H|9v3s!Oe*xdre8I#*xYI1gStuzRBx{mO=t^Vh65p=2-_j7?kTwNG5kf?Q!2o zbUTy(Amy&5 z&#C%c^Ygh?`bHReRIxE{({`olXVMvH42V!<gan2h&#ewykPkv`-1ag&C+{%EJI)X-JuVQ`0DA3pT*>iBtV3;T2_n9sG9SfK`cPG zjCla!fo#hS8wsdX<>u8(jBD=*Q+881#)OiFJ&Hjim9bfuQj2`4L%UN*^*Uk&x*D$- z9^U22venU&zBl3*a0h@$yD5~3qV(3si(y!mxUV{!w+^OmNC$nVD-y-g0*~R=<`X7f zJ9~oi#o>$gSo=&;#4V-ERWZ;Kl*wK6j@LBObsG)$a_l?(-GBn^aq-w}rvMX&HsAPO z?4@ux;>b%$9o5tQkOSQs;ME=UuW#8&ab)g&>V|^^2+9>(i;Dlnsxzd)xCthFG1O

      AF86`J0bnWWH&hG z$6ZN315vuQ+D^bt?BKm8-WiM%m@b4xB~Sk*T0=OXlTz-rKwZ=jGVG(85J)PSyH5Pj z;>za8;D*Fw*PWJ}!7gurcOAMTE4F;YBe`z2xV~8>)thbdJD;klrW|_mo7VE^K3Azn z2d?XD&;sO!jZ;!5b>93jl-y5#kd6g!$^5O4#jAsnQ_L0>{G4hqUyGTX*;L~9ePOOH{V^Prc>X$_ zyVER-9S5}j$0Ju4%p|)2F}wNE2xlw3rI`=BX!2TJbcd`&l|zlE_!OLaw){5T!ije~#~I4GA>sWtO!n7I@=uEiDN;|(=DZ$6$UFY{l7!o}>ieL~$Uicxnz@LD z&wflQqYrSj&Aj!CZ%kPR)-0pC8)aC^b>o~Wy8%@3u&15U;M7FzN5q=T>ll1pzL-fZ zlQPt$IgziS(@K=!?nDt$1KmsngKewA)>ik{q&zAEC}on)PfgE6d`gTka&~0Hc>8<&Wr|z8h9WRt{>g ziwHL2Vo|&jO8lXOIZhr23Jzn5r1W8?cN6=iuHo~HWfMPq;6?*4!UkD=;^P(nANIa7 zs>&^D7jUD5N{a#lO1CHy0OQo1Fjk#3|L?t1q| z1@-&J@ZY_6oF9h)2U&Z)bItk8XU3ZEuqQ11-l+r*a_+#Bx0q;#fQ5|L;R|3Ig{)O2 z&urY#!y{r!W<<2TzfyMM?8F%^|Fvs{qDPSTrWrfx{fXYEKKc&oqCm%%7gLHF0I3&M z_VpjJdVN`J?9-Yvl!Q4qAC5FO-smOirYYxb_a813YSUVFTwGl9gIatyRwcS&$ei6c z677o>l>MG<{6^DL6QCuzgvcX8ogEjKUshjqQ0jrv-X)U~6C97?Eu|0OPH6XiaPjl) zP4A5_4lMRECPnL`+ZEDgt)fG86h&WhqTdbE6aemDw!u9parCA7u@yjmRzZ%)lSs<) zkUiiQCUo|yrl8ZhQ0TN0)qN8#=S=^X7Q1VeB4SwxhD1NFiI@cmo^_!i?iB&%TQBoo zo}7uJR)&fX-Cj=0Np@aNmAv|y;ZSdKVGl_PXcqqJ!t+GmAVO&P9%m z*j)XTHt|u9xjV-wmHILpIAMGaIS7?b%2mAdW z@0|F&-qH90Rawf9*RL5rIg!}s=N0G{=S$flay|0y&G5H0&U6(dQL&cYeA(XmqR1Qq zz4-1eZELqv%CCCKa50wsR=l@6nJt4l<+cE+p~+ZiiLchg|079J`w_J=^<~N^;7kaDO=Lw z#9pEnYg4XmsOb9FWhDd3Qutn;?vm_`QCCyM1sB!Tt%-zX96-q-A`m^ZO4Io za5as1zVn54A%%vIX@xL(KhfvGLRpUIPhiM2&yxxtsaBZZ@l123Bhm5YT~?4EzLC0D zyi7$3<*tbjyb09{3v3Pcf-9`6Z8b%E9oA;Gzg^dL`FYtfvd=V0uiO>$zCEK<ruCnLTYs~Nw4dLCC&B0N}rE39bA4{cWo11A)>r!b~H?rP8RWOkBf^)l@lzd z+$~7dyO%09Rq3QyXN>fUCN>5-FMVCV_>29rRJl8-#Q24+YMLbB@V36T-pIQvk@)(3 zQQi`peQ7+nW`9R)lZ`k>F_1LW>##X%&}&5^`tUE04AT`I&AfH5vSPWUNL1W?_`V|V z0l^91^-QqOhTL752AkA}UV(ZwQwm49z(-zHQi)pXgVKX9*N*$%SCGMe?seNuiO5m* zdei;p%ZE9b-OMIZU%z85J~DWLKRBpZTCe!S?waO_mZ^lZSKe>=aV1hnC&f5#WJXe# zn|JxzXucfBjPNR}ywxd9WGi`H_m1kvJ9Iy2+EY%8g=(fGJt=>L`>n0-VP0k1h&kV> zPp>XWcXD!y#ez0on%#H)E;^O{%-6T>q-kfm5ANWPokhHQAfxxa-S8t%m)nr0W7ISUTuA=W7; z?NXPGroDBx^t^Usu6xc!dYr0%c&ePYau$51vW$Gkt{$(Z6-o3aT7xR#zaQ}zpp`+J&FQeKEN(PCcJ6whP?{LK*2L9y4Q#ieoZ0NX( zPi81J0K2dg*9qP$FR5#iazeMwSncZ?#x?0C5>E=Vmd9P)>n6O(j-`a{`e`6OSWcat zbj0ovboC}9*^G`!T2~I1=#5VrcOeP1LDsSm^Sugg(6q6SBzbaJkR>r(zMRLBg;>?= zMiAv2?p>9{{EvY-WT)@a-DZ5Z2%5P0c8>@%ii6ukjO5M6sM~r7Y-Xr}@(SF{0Tq-9 z9m$gLK8a%>QSMU+U%J-~+M1eoHs@kidaN2H$0&tY&#ouFIQw?`!xFesBTRPfwR|ey zJN&N0()4{kQ&9s2dzmdptRvL615Nk1r6xelhi_^7fnyvBx*im0AW0DGB({};eK-89 zH^Unp4F^1=%SrDUcbjH&5S$Zp@$B&aQ0Bo0`mX3R%Lu4JU_9q${BP0gUg^@bb=bxSLh6qn&9Hj`1IR`RQCU0mdzX%?CfQF>6h^a#2r zbZPX3eho{Js*ck#DevwA*Y0^8_cq(&HkJ(NVqs?E<;{?eRp-@w9f6q*Pp0S>fDL9^ zIzZvn21sdj-@CL?apaZWDJ5bxOS-)anHdZa3MztIOa$+t6Vj!|)_W2ukzQRGHK;Hs zju>Z9;UG$0pEeSSRMoZ{nbd1U_sl?iNQUraJ3gCOu_ryUTp5J2r!bAUd#^Wy!dI~Z zl^4OE2to~1+fi*pSRgd)ZGr0R+H0T4SEksXF>#-@kg1K5&M6~R7MxDIrky!JL+5!9 zlrhn z2(Cd~sL_^(dh<5`r&f!$s+*t|38Jzee|aa35sL?JkHooRAW~p@UVcJ&azg$SQczVk zMfFtoD(ZIEscwx5Pofum2p~3Yh-b4TM(u#iQ;Zoj5_^yFP)P|9Qn29{Uoc-@t|9V_dHHeti@Q4@$nGhN<8)0s8o2JCVyu7eq1xAfHKEmFw1wW$PkL=b_XGBj(>AEEPvKqK(19^}i1M?9rt=9DWty|>gBSXD8f;n`-xr(*G@Pp$ z1oaWTedA|F`k_nv<)1Dljdem3_ay%|7v`obR+7u~&_sw$F<;@}z@e54F*}*K;%nL6 zW?;6$!ukYgh_T<8xD%GJ+@T>Nyg)=C-`M@Y9X0PRS#Z+(8iC+hEyrqxi8mKSqEw5< zgvEP%h>M+3H1N*}gSxncYoLoHqR(dXNQ*qUa%?@KqLu`Plu%DxA<$%+$r_uTD5W&x zpEwFq?EQ&xeHJo#TV64;_FN~nDpfTS)o7e=s)I=`M~ck9)xn^#c)}Ea8&ZcaL+f^F zvuJ6A#s20oa93^T2}ujMB=9s74Bk&TB4Yd5`udGso8n!kE~*2I_v^M zVLwS+esZ0Kx7Opg-IFSX>=&(jxW!k+3+$Hf`^%6!E#D8GKpQ@H9hC*SDYexc>ck%9uD=DYqeD%O^lzHu;-DM|NoIc+TwgtZ_l z7q+z@!8aaoD>hs?G8+@UU1ub7DFa+<7d~zK{mg{8$d2TtZ}sLfyE|3la;f^I|LdVi z41DD)VsV<8>X+!=9kXiQbsW#Mt|~o}U?k*}ze6-z*jO}@RvELkHesBl*D$a7D!bh( zX4Nfof{`-GreLPHwmeMB*=}QJ)#XL>cN&coo|kIl-=9!Y)v{|-nE(Zzo8Xo+b=0hL zc78(4_SRVWPEokCj!$gHj{Q`^+n9~vOfcJGtUl5FM2r0PJV8fdl4-UqG+acn&#*N= z?&QlcH+(KBHElZ;5Ob+`$H$5%`BEUetbKBV0Sy5XhbAb}-#EUo`9lwSP69W|nS5UB^Z~2%X$}as3)Cg~;VKsK%uC(?K zoejLaR0_sZRj5(f-vkxt;^4k@zw88GK6{BvsS-u&ih)l^;t0&=zgEhmnqSFA1*6ni ze+do2gq-j0XCD}53+reeeyHuVc6-7~hu;C@KOT1I=&V%Pe{wMi)!xnP03vxzAi(al zDcCQWM-jDrs!y#Y&G;$NBkD?=dHq(^ft66SoO^*$bEB zvqPr*hFs9FQkh>aa3J0-38J+CBaITY9`(je=Pc#EI@Amj3olme=Zhx1Vz$|%oYjf+ z?Mfn~KN1LZWmH;NW^+LcjbGswJCsW}PNLoJc9(v(R_zGrN;u4bX<9tA*fNvdtLpc>rGNZ|FdAO!%t(`d zq|4CoDgC6R(Z%@=cUL|4gj#VCW1;Pwa7#mDe~QqoY4l3nQnrSk#c=SXUE%ZI@J z28LNufk>QII~|&DR4%F9NCHhlO_-Zuas9(&oH^fLEp^fPvx*mYTCMgj3|cCKJE6k~ z6CfcOsclzPI|W~dWjaFbD6uiS$|-B)vXimR@`h3QkdW=MZXYKqQl=~MPL9ztoV5t3INS3m9T+b#| z4}6?!i3$Fj^cvO+eZvGUz!fcQ6R;l=c9?mfcD^naH2s-*(uWzHP!cN~j6K9Svp8@~ z`g#0@Ms%~q3pOA5G5!1gBXs=EC|!e*rfXmEUP<#4LS+p`KYa|i_M(V$NIP=MT!?{mYz07-SUGke(`rY zTR7D_Cs|2jBV%6|$Jf}_Y(hWKnth5g%(aEd3^WPJ-9OLv3&ntbmz*Uz$*>cY0`Z%g z8y;%f>(O8nGm+5?@l1&cUN_7q8T`+0O@1bjmUpTNo-$w6*^N8NF~*`{u2x~;sSd*>3|g{?C(!Q7nqN)a>fOpx77 zGy;ATnhGA(u(p&`b31U+%LUYEmyk0yGE?ufwe45)`)+VPcMX1MI0x#8P5cZ4bwa-n4IY=nq@5o7bMe78HK3$%&cNc;UW03Npj9qz$eeIC{8lY>D)W!>x^5 z+;3Y`=A3LcI$X3IbfXH+bxlANaDqvskbP#YXQriSv%vOKRr!s%u;LAsu3gv1V4_~H z@;fltY07u!D@2h4|CZMT6d6FA^Jad3!nSsv%LW)bm3aE2i(HpMuy0F3*bc4J7DabaO&NZVmL zn)ctOAy!(T$x z8y<9#<5m1rVw>K&WE&?u`*)nh)H zN?-wbtA(>|B9yrNPF}YhfCWonHN4R}OZnTt|6o{=&BSU8=8;0Z=7kM6o1!68;~`(_ zW*x`g<&LE6;-RPsz>y}oyqUmkA*bdzuP4Yu+B?nG69moy$3+ISh%Fd#d{$E3$Taz6f@%P{5CgL15wK>T=mP9-*WrRLHyK<4Q>wu*%mAp9_P=@C{|y!YZ*uxUmVoi!c=-Qj;Q#+K=zrJ0 zP*BWFul^Ip23N^DT*lHnL6Yp}u@;jfgndqMA2(sX>ToD{C;&@Pm!u1u;D~{|-#mOn zid{G0R7wq&$She=daIL!$t^X-)m*Qljq&4n%Ghu`{!;9dzrP#d5|=_~vB5V|i*&uw zUlsB_7I+1biXnl?l!KvvkfON4$FNfu!q|~ma;kMg2T}y!FKL+-$S_z2VC1?!=;no= ziKg)axXMn@Q>Bt>ukas@oj(aD3_BHz78^N$<#Uj~VPZv*C)entTj25E;$Hl1BN&KS zxC%zRD~YBn_kkFk79P8ZrLr!4LJpIO%`p16jnER3CgZu1TDak#`@PI+tsTqiQvHc# zz@WVDkMgV=mI`J7mYfA<#qa3TfUi8lb&DSx`R0Y)ZyRxiE5x#5D3;)ImYn=eR!w$@ z)GY@5B-dcF3%|dJgAnJxLpvzR|A!9^;Y)1G7oFM9wQE8?#MMz*DDpLLXI*d2BCg?c z={r?O<)WYdlQr>P)m1OByO4K}hO?N=`~<gV2 zOJ0AeTzRx4V2wQT`AB`6#Z~1(id0~F9{*%|H}4}RLd~ON{FEx!Hf#r^_*p*85~zXg z2>jc2vPw#8Ma&9kT1}5!E1?GV_GxvCBOCbDx@=qGmu3Ey8-2UhyrHw8Gj932&81J)vHxn{$l&_RN-Cz3s ziUYS}PV4=`WCsXiRi*xIDUP%{(`4#7)2wVJ|Mrans-An0H<`Z0{7*;1fz8RuWa?s` z&tXM|?5KRaVx3m_d~76kRvcwbzW*j0xdBJ(wg$17uJw+y^*i9dpQd0YOr|&4s{lDJ z<$pUXoH{SQj6)lZ#k*g2mK%$8M3CCJWAz<+4eqsE8A;|_$i7FDofaLT=4Q#Z=SYBB z8_v;er?`@Yo)`ZuVL~W@_660~Pag1_2nVv8?EO^7Y`VU6T&*80o@ld3+~ut3_u}7t z{C1_gubPBaxkp&cszH-5>fl-7mxyeQ$4Neq{WyARRW%~q#WU2r3EX?FNImpbAYhihiSz4jp;Y81t0K3GfYA4kn96A7!d$=;Zfb>g%YFjKf%yI!_TvIMxU*P(cFwL z&XyOXC&G;XDhpP)tMuc7wRujLDMmj=fq@xYF94dKiSz#{cFtSujT;6TTLVN2C6D;-BP7%aL}o zNI>LN=62!C(xxpYLIoF6uOYR6F+8{we4VlUj;$C6offf)VjpAi|0K;AJWWlxu5Y#TjgPuR14Vp>sYqZZ$n#ne&wHlP=r*WX#I@0h*>hE;3NNJ?j<%@-XoEV!B@KbyLZeQ zXmCu>R^swcR?bs_g#%gc+k);z>g*&O*7)TLx*p4Sq)!$F{EO21ADb z#`B+6>e{nZ2ES*5GZs05F4N>#k(`ic)A|=f9nS^W(^!llEU*nC!P47^Rzg^t#hsNIl`PtY^)j315 z-`WZ`mt!92S^DmAFyh}*80fWab}rt^(DEtP6e->dzrMRK?nvmeqVg4M&Ij@iCCVin zNsa_KtsrpG_)|Z}LrXGk7Y(i3E^9io^|4294Hdp!G0EwUKHekk^rd1b4Gi^p6Wa{!@+<}oJGw%0GA6ikwu4!9_@j}#*2 zU?Jv3bo1`isJw&XUpDx(WT}P^5cU)+Cnq5b24u+1Rrl9WhGo{p26^VEV8J)e^aIb8 zL=@+Ou>q_Ud zVOdp*SgY1qv>QDTz)`nvyEwX8M z;;KUVH#Uuz1p7-+B{kS5y!l!V@l(uZe*QLF<6nM+_$v%b2|3K}9>T!b-MwHvx%6W$ z!F2BtBE0yR}3s1+FxIE`YIsR%EVI^4-ba+VPWkNg5@p7#?64raz^+`+ga864A~x3OdP3 z&)|ObDt^)a0!F38T|@UvM9_Ef>ai(zKD4e*vyardSDg|7}lHtrBWVKr#2^(4FGn5`{h#E1PdPfH;-LWQ3Y%V9Fxr(+F zsqNQ@?7;kokx(_AQ0l4GfWGD8pu){LmeYcdj)ftB6!C|}cQ`J(0aLziGq zIjY2_1#lCoCS|Ci_RZ+?tNC|cbL=u#(g>9XAHxN1 zt;PAAzQEetX=Ro(Z@5aStuN*qLIYkEmwU=BEOtU4k-HjToP;lR z8`ss-Lbe`v-Rd(1&4S-)<$p|B1Fr<+{Oot~jE>{{)9K8im(%+i+e5R*is%|I zuOfvu#)q1Fn$L>#TPq37^;9>1xaQ6~rhCl}rQ}&>{-)$9#5jkGy|9a{d8xD_BEoX~ zmmfwAcL?WKMA!~#e_fx8m@*5H=E~Q6)cufLz_N^J)=;+U#VCEJOoXG~g{*o1k;JL@ zp?99&stETTH5Rxx{ULqbusGCnNWQk-)@sX>sa9ln`H9aX;yaHf@-uAiQPZ7BBf|ib zYzO@Ytq<6n>ve?g^c5}@_8r|16v!?GxH-czml5wbKEuZ6;9jG0Uld=X@U@pPs%_-u z>F-1fK^K(f1L@pr&GLCNbGy`ZoEYfDeFMpL>~AqfJ#L5CISvK!V0{-2Ku=J@5k~Yb)qgxN4xA`ciO)exdeJObFL(I z^xIwo({6NXPDGlIYQG-I>KOK9iV0e8IC@FAT}S89*ByDg{tf$@-0jxQIgWs+&1vRZ zj%=juSgWOuAeb^ly_;09vq6~-MmSxPyb@V0Mrk&n!V>EL5erT~5O0~&#QX!>P$vBn zer`QcRgZvW8gGcvX=|P-y>@oHi%N|dbL`4kdw-H|WB|F5u~lrw?TwbkoD7Ox&<{3T zHssHt{KWJQ4uoE+PydEqt?_WBT0Ntci0fv(NivZ!9naM9oo9-STqb%_GPY(?YZtp- z%8SulRE>5ve5Pq#s&i+^mpA0}22WMQ{WoXSv@d7Lc48hnQoXg>7wixv8Xbxhr7m=8 zEmumciQQfa)ri`R0B=0&FjMqrbXc1Hat++fB@kGAQ@sGWpt0sk&=A}xIUv9khGX6O zbqIOOCb2eC&^+v&R^D)ffOVB_ZbNwPn37SAVPRvgTyMd2dTq^Gi`)aFF=M6N5EHcK zF&uhgxX*97!tY`89RI?fBJkcxNxVbSsj)XqJgVtnRgd#5A&f!+|1>fS?(&eZ4l}fzV@zo<=(B=4y&5!GDpU!hc|ca z0<64WA7*M=`hfgK-#aeDo3~}xRFr=In$9ifr(^~_Yo4=w*>Zv-^qmX-E*7g`Fv=IJ z<6rZOsT>C|%?9fa5nQGpVv0Ufyj9*fQeefHUzs>BqJc?EY|+fGewSoFK*nEpy=C6q z9K1gB-S8A=h^tlAz)J{HI&_B0ZnjTGyU;ei!AI!_@^AVj55HyR^kk2XV~Ex6{D9>`%FuA2I888qqli3j zr)``jR>Q|`6Aq>*3(a?H_1S%lookDtJvD8wmORE4jbAyOk$g;4j6|oMm1l`L=EpTmQdG>p@#e_@)#FR zg3A`~*~jYErZ~(Squ7+VOa&i*m=&3Q5jS!~WDTJfQi%sMDc6yCEsEura&m-dADI9K zM@$!cg+t3NI|NU)NjfLx{Zvq1N_4R`@y&Pn-xzaO@}@@e&5J{wR!=h@5y+^swl+lC ztp$pYgtS!TUuZlkoXK4^INNf*6o#Y|KWbxCJb&DFdfuf50K{a2v(#^6zE z%NNr3(mX{R8pzFILi7pM3p(xO8*&k$K@v6x@ik{glUc?U>xFhHSZ` zW8n2XT6xx^&kdX-Ejvts2&S%K$q{`ZxV#oyMa_GVj^HS340O4RJim{0DSv*Y`;nE6 z=|%?GotB4c?5b~4yIsIL8C4v?+%plnV8$3a;7w^MmnH#jQ=KWCv>$DFUCj|OobGK^ zoBT!GugY{)Rm@kQD;HZOSktCnC4EKx{)k~o__tdFFIPeqWfXvA`q?PwvV+zwLr`DTx8{wxc8#4vtm6GIg7ZZ(v?`7t|3SWWH=QpJe(i;clAg*RV2k~<2;IOIar>%#%&VZ6H>N*GJ?HG`FD0r~;k zG$Nrc2q%4E$g_GdSFe(^Q~TvinEVQUE_OsE;RLZ+L7u&T=wVmDq@;33yQUKEcW_#M zO?#ORT5ctd7`0RGo*%2{yR%zN#STUt<{~fe@a%HhY!6hNS?HR2Un^|WJJA5Q`s~YB z3*qD`lFHiA$ZcM=%U~e;fWY!F{lKt?uNvSwj6)dv^&DQ}1wW^HWx(7&@evokEhZoVyO^btApXq-;RW&0v7zuoyz;WL;?=SBR)AW!m3 z&#MC9-*+autP9^NQ<$@mC2G*vj^94T=ltrzlx$Qm`-Llc52w_>r@b#;7kkvL&vnZ% z+|ROdCOa=XVFAxM{chlN-}~aFn=Ve_mh;%NhQt;^QRCF6*sje%t|U=BMpqAf*dQ4Q zL+9<8zTHb8wwNuPa}KwyxYe8?PLOLdS$_TX=D5Cptz;7Wb`WWHENhKH!DrR2^31b! zuKKKEcKi1X$3T3Ac`@_>CJ9~vCt1)GaNAwl`@-9KvM{+oFf@&c;kE5^EALceo8Fna zpbMfKbDdmjD_v9OAt_hux>~d+tD5nN>(py@%ENS|Vv6#mvfpXl0TWF*yU5+C ztYhC~soL1t^a-7M^wna0da?0NqjP?KrO;}ROzy;}hJn)9niXRV0~l%Z#(|pH!prh` z;b#>0OwWIE=m3TxykKtJ_=99*587KxfzUJ0yf2=Ac2v+2Ovx$Jb7AU_)r z+PFP!)mZC6Cq8D*{(brt{thdpdFIRfG2hONk>n4R)t{c?_l}*uWF2sfZ=8XoB8HL8 zX6m_M1Pn=a-2jc~qV3m_I0u%;fk5plcS4?m11^8p+EW@w+A$rAK%td2kwtb80+Fp} z1qWOh^!PlS8-8Kr>yR9nU@D)S&IHANS!e%7iv#fkY)J%+OG%2BS2V)~ZQcdH$OyEa zYR}R=#Vf|9%Z34Zq%*h6I`-+n^?M-ENG|Beb$Kn1UQdT9PV`9B$hmw;>`3aTxpV zCU|PJigw(n_CZCQ=Y#&XY&xvq38gUq3+uRU9NiQw=`NTFwkvq|K)1w8;r$FefF6_B zJgAkj6&taF4SZc!v86LEP*GvqX9v{MKoK6SKk4wuFLP#wp~se-ZoVRTfZG=f?QI`q zHZkru!Q_u1L~WbJeCH4UpTiKi7Yq;kxqSzRY*`8jv?r(Xa^4;bMw0M;6O&Jt**k!>pP@c*A7d@0fz+aXqzA(fN zwu*=#&vhruO9^&F`gA$`@h1KDKe!|jGbzoPc}AAPYr=#$ zT)J#K#6^dtrT-%H*a~d7QU)#ou}BbJl$rMWjFyvSH48G~@{w^{S^y$}l7HzyEbe~! zD_lhAnuxMp9h7a$dyG$~@BtOXmj5M)RfxkHfrTx7RLmQf;Y$dzlE_Kh5{Rrm{fo-OeR1y+`!Y7InUj!}XXh0RQ<<@-=pS;M ztYAgU<4tc>`2m?f!{(U~BX`jhjS(lJ{IZxv>k40n^hQQQ>`#t_pdcQI)VkPuOZj5x zo~HaBw0J1QD#(b+%MZ#vsY2o*o}j`IUjM()%!{-N%vGYUDTgolu_nr)voYW`D-;KC znFyEE%X2)%(_0i)XG|y&i;||2l01Jd}tmGk?<%e2NG_Erg3?*x`Vxx~}#Ny?R z14oqTT5;7Hx6e>2JJ2=`j~yo-KriB+XUNP*Ry9N)dUYM)tCt5> zyc(;(F6w<>NHnbC+3YOc?ip8j9!iRBe>wPobBM&}<;|@H%ySyr^nLVif|KP8(R;0) z4tVu#|H=b6AyNfrCZ3&fgRh{dE5qNOoe=XF@5L~zZA>jspSw?AgA@oV~%i}oo*HE_m@GrKDg99^qHKCQwgPHW0(Q1wD9Uhhi1QTC*cZ2_{4cvPxF_SQTMYvHMd*8cvGA<7p=ea*^Mcu_VGs?Q944I zf-e~h4DokO1n{wpQKt6rPg4spMqDkWAt>8C+}>)uXdt$zXKvM}?~pMRA~pR*f6C#J zWsqIoSX3|*4sC!YCW>?fb&s_k$da*=G<7@hUF53SY$oyK3k?VIH3pu-h5O;3naltI zgr(7|FRp=%L*t69XT*&vaSuP=s)B`YQM{p3&_F*;Yde4J$NlOPsA9^>KtBqNR# z>lR4bxNu)&_(Na;xQJx1wgHuvRK4Gjd)q(|XWgSAwcDo=UFZWzTB*RgxSvTP2C?Y- zNV9JfVBus!7}~rOx+ops8a!8eVZT;X<-lLM`im2f@V&#U_xsF|C}d1cB;k)%%!yNU znXq330PzMNrq4~`0^+X0o@e=1?Z$USuyCG`f<5Eft!r`zJUJ1OmxlfBEis9$x#Wbi!!eW}aB=CvgCD7TA=mbiSA;U|-2m}0kiX70Ccp3{^UUo-SU4dpXbcTM{pt38AuBw@+&-(QwD_vv z#ZgZ_mAMnFu5ekDX}t>g!<8okz3G3qKp~n|89YFQ33~++50J{IQV$C4Uz9Kj{xaw2 z{0n{u*ZXGzkLF7pZvGfE)YzBZE)&rJU0%#D$2-9|1@%9 zHidEO!JV4(2_JMGKZFr_062hh?fmxs_Ya(4F3_S7w#o-Mv9H<#BFWRp3rmk`xa2kifvL|(KkYd8lqRjU#nECWu#vE}I zR-hpMhjm0o;p59gz13{02raMpi5{~abVff%NeV}qdw9ryrynikgh-Z7D zS0<8Mka6HKlr!@dl z02a(YP70F_p_u(*!Hna2vo8_LQzCfmGqe{@d3gn{vM>CR@GKdqu0J1V_o162$Xu~G zh360=1Vuj<{@f=HLsK6|qC_5Uxx^V5Ve$j?iQjY7RdCt|VE1HPha)Mb&ZT<8D^T9| z)Suorgp~QYEr2$u^baJU@Uh3Uq>db|-xm!75zzP3rzs_a=O2bx{;(E(840n)RoWZ- zAP^JQMa~=oE!Jfd2Ln~6zt^bXimY-Vog*jNRY+GgJY4d z_CE1d5=#z~ly{$|xx)D|U1%!=uLhe_Q0DDv<#`S?042JU`R+tiY&hN75`-YisUI7bSMf+EhJbdN@@Hm6BR_xlA|1JY4W=8|jG(;GJ z7*OaY^*g%30qs*iA+5A{Hv7&_Jmve8pXc&FJ$M?(_deD7>V<)og<3Ne{(F1^!a@gJ zK5Y`o0u=CK!fL=%0pAL0E&~S&D{I=otmimxUc*3ip*WGsA4Rp+Soqf`l%^4kz2l&E z-Rm?g0?7@rfTrmz#q#{b`z0y5Rn0fa{m?oJl!T8S0~n_ejAExJ|2P~9q9(W(=TkG5 z5p%aN3niV7r*LnYXTV9qX|d2CbatFd4)M$Fzz<`)dGblY7i13~!IBlDy@Og<=8uJ+ z;00I&H~GC(2ZYlYz0!H%H+T0JPVeBtg3XttZ#kZM%`SEPDhA>SD&fEz{D+3qBF;(j z)ma3`9*<)M>j~_w#}{%Jkc^)${0=@)v?WmruECX|JHeK;KqqBTItkufbuf8^U76(ynfEOc zjPf`h1`)R|;On5HK5hGGUot!30j6PL^%_R(I#?aWdjfxdRKmxuhb1G_pI3qK9{c@h zud3IRNH-x9l!v85?^7@Dh1bt#2GU>#=1<&qTKO<0Oz(dv8gT*;z`W_vNEZ6X!y{Y|ive%# z6>fGGW0Vy7?5pVYBqfe09oq*Lsmx8S={G^Pr8>lT(JBkHNQjr>Vc{4^prjh``@#x| zuE25z3A}8RCbc>`!;j|#Lwx|e^C-Xw*wb{ybNqL)_URKD&bcXBY+EW|`y}87z!G-d77C?3AotbvN6N?%>o!=8ACQ%oAHXA^ zgagsuqsPuhKtJ#4P{}S#qUbg=^wU!)M}VjEc1;zeb-UA!>{AV1tlK1JOK`(H0|1ZOe@L%P&uZ@WBso44%h?FaEEtdVstM!=1AnfKKQ#u^C9rNr3ujpCL6$Vb1TR z%AOq|ll|qgtB?S1!>24MfG>ImLC&)_!rwqV;1ojFg6^vYv6upfh_UA)^96 z9~T2_f)uABiHEuk^Z-TqCa6GK7LEe3w%=hloDkYm&k&jOZx)+k(FSIoMQ%aTcO+#L?9f{ zlKeV#p#ePHk7I!ogWNf5kat^G5!)(*XaLGeV@G}3 zpJAfE2|O0e=C}%QS!F0==KZ}dDPGDoRxC9yxB^lvh-yGlf`%WG)R!|wv6fijfv8pd_CAc9(vajG?E$IfKa71 z=r>ep&BVa;jhMa63Kdvd_ekMIK2j*~{fW=m*d8im2Djz|z^esO6=YgJ~ZR}pxBj}ifh!MVcQ&>Ky$ z?@;`)9<;I#jE8fyyzIq#g@@lK=-=f-U^b)8uiy$pV&KUC^1`o3XumODl5RjHCln&l z!fYM2jcxy>0-dDanfk{~+md`zYZ+m0As7K#2$iJ6ev@VhDNeBU=c9ZKxSgKR@dTVj zpT8oH#9iQu_bdROiY-wj`Ea$FANWQOR8nQ}x1l zjR-6*f$6f~tD0tfZ%(v5SPBaJvQRmX$$g4>PYf`&2lcHNC%xKNr>T}%1Zc1XFthfSBF5gpPlT1wdPUS4hTz=?a;8J^Juo~m# zsUHdk?2;IS$SS>(`h=)l45$_1?i1ZKw045$p-XH>vNbk}G`1x9>)aL#r6W3IKo5oJ zMZ*u0iypD2sO2R0ltHpkM(dY35)RY``alm98-d!n={^MbfV)N_=|+C%QCMu%YU8#U zztdb$=0(|v6a!obX%=EL)*GWY(V}48tGO-|eLMPV?>M3i1OC;MVPe4EyeC_cD9T0q z1RRjGhx8cP8B8Ys#~@`76tnF_69KC%hqUEt)pAFrG`;Re%3`T+4|kQ5>fDsRCR1oU zbesZt3dj3ltIl#SzEx0gGod*^3E;%Qps3=^V)r|ycEI{Rh}C)mctA8k&-Ie+d$HTE zL!q@qC`#h?=%oso?Ydt!EM;jtR};6jO*CS+tV!P^%2Bhv2WcSf3fb%3P0x&d*%$>- zBAKlWJ5zgO_#`AcOH_s~;+LaBRKeV6K>~Ws0Y7WBBlD#!0>!8YUz1^+}8+zcm#9%P@EDs^~6?7@GdIK{}Zv4>0@c6a>|vj zKY2;8;(|f{l7&a39J@^`>}q!SD;n0(5g7VR{_e&>C3)+En(2vZK9)(%2!I3^LF#=+ z&+E!Br|5$%#XxW2&7vXfGhvE>6GQ+{CaF zY;e}!CHuZ|KMh@4gFixxs(SXttC3&Y)<%R=f$6!cl0=gO!Jh&YEhRK%Ntx1NCWRbF z7bkpRONTr_LZC3Nd-;$jXB}+teA!A2AO3L~oHzyo=Yf?5zYM50hF&!pZSy`{KkOYK z!64Dyd$-t?lInvYK0{7__(hy9Hn1NFNUgn%hxQp~G~7XwuSbv7|23=pmgiZ=b8iap zx1#$xTUE$|YK4f<0RccPG<~cj83h%JpN8pxReFi9}0{2DHTC5oj?Up!iEr+Y8}8KEB0qt*)a{Hvq4iRbRlEy3C8@u%+dKG| z4u$Mh4swM*fy&EITkr@r$$%pKTvF6W7(9V~l?5f1^;mxz9>}PZ){L)4B+{G%y`kf? zri0uJSUdD2hNZW2Qk zdrutFBxTp1eCHy2tmol8*oBRU8X$%MXGS8)=M^|u9%B+TLJ&Mjh*VU~7p`Swp)n)CF@0kmFZnyDqaO&KwU=yB^|#deVHdQ$SP?mjucbd|NMUymG4ue3 z{-7wNOCTq547HPs%#Z}7eo2smh?)1ZyK4WY>lHB`ev7*+yMp6 zNoATJ#+n@D8m2qM@Oqs6OP;OH*Vl?!Z?xOJ-GeRkL&JmV0@N`Fxw5`wL48wie`&uP zwp9K1QjSF8o(C$)>X3g>Y2n8|j(azE_6`ImaN2S#Fg;mgwrG?3mQz-n!Q*50l2%sU zb&(=EkGhP-PYD`^SdTZIDS!JuDiL*KM@dlMHDV8J;k?)dqG9*4E=%L`UeVu686c!! z9>vP9$ooG)d*S$*euiUVzxYOnhzk17x?U-xZiqO8$?@&6-z3W4iL!o=nUJr&q3<7B z&$szjhRQ|VtRiwqUg*}&`gHa9XLPjRGJ!-qK@i^`WiXoC=1+YkUdFwA+ zW|Kg*TAIbjKFVYjSV*U+&k9fa!56@m=-%^W$++JHd2fG{%PeYp`2&P4BqrPU8{2Ko5xUo2)vP+zQ2=97 zka1m;<_;e=&4#KQEI$LTud%pS`y!cKr;N+dzRNOR@$kPnB05F zuDo@v1Q(n$A_Sk%CFSk$k3du^KXu9#8T*{=+DjQke1lsH z=pN$@i+OxrFzPc(RqkmIbdE)b5sCXf*RnAVxIyQhZQNL$w9oCg~`)dL)N~m~t=iwUyBboE(b?0`@FEv&0%6@TDe+EPH z@~Cv8Lu8e_U~fk)-u-t$Ssm)H>z#SaAWs$Y97Z+BHSZ_>4&)#burIIajWlC_^>&pX_&@BubySpZ z+cruVpp>9=h!Uc7Hz**|3Ifv7Al=Pi(cO)-*rDm_*Z4mk=C2d} zpa$$%o6{7_l54vCKmK2`=|;69$nbffBspf67CB?@+ox@UMQSwc16hfNoVt*kPw%7Zv5U)o-(9=Yj)1<&WckNecFEgNjki>`Lkdix5=t8T6Kp9@aILGa z88*1P)`edrJFpPG5#+I>5AMHD1@KGb4y@41z~@4G&w(%aoRt41YnxyQdDTm0Swt?s za4$OZKKUo}e${4(*YT*0YWiIw59nT()<%FND@JJL->aH{HXXacxOyjOZx@vf!+e9w z0RKxY9lL9|3`->CDLqj+M(g6^TkFm@froojYl^%a_!>s9l{H`cIcP+F4ZC-j6Y}U* zN%`nKkeqA4G=NRjB>$7E;R=QVR2uikRCk>=IDQEBA&3HR`Zk8vx`ycM z4~3NEfcKZ9b$!VItP)Q2Xe_XZachR=(&$Be2*8P?e`H7Uf9qEu)$NjG#cGNQ`ziz@ zGfXx0#p%ET{`6m07?JP)lrqF~TW|3{2HrToqP@qE&%)9s_^jtABoMtDo;xN-1MeR# zx*Pfmc;vvZpSOLwuK3jdL7F1{ev${u_z6^$bDAR`g;w?xetd^PZvL+;Obp_`&7V&q zX@&GeQD{T>pUd%|tA+FKy5Umt^j~SnfB3!^fBQfc*iZfqH~w1`6kJmrzy42(38bJW*9DLLKc@d5_a(ylOVRs9@C}XZ{j0XQ{#TAZy3V*C{*Uec z=L_in6kWK}nD_s!*K~bfjE`g25ciM&!c+db`M;mwA;6PCE1~H{{EYtdz7HckuMq`7 zVte_(=GONgn~q2NDlWH!U9~L#nFUbP2WUZGm*w1QqV+iD0%SL)CTKY{aK3E;Ah{mB z*+)842b{gwrvj>QT_|;1@X$1CInQbWR4zwI_+66$zR5@z-%GV=+eRDH&IcO5p7oL3 z_@KUomU3Uf7GvnHHUo}eN&;+Cp@b5M*Xdrqd5w0!xu4PLVxIR|^Qp*K-#y`IIgV}w zHJ|uyAp7sL^@jsOh7+(<06|phv)&j{F=KyL{i?bF77^FoCq*SSg$V0;7vNT!{j(C; zTUr-uF)DL5I>Vs~^k&DCI2k1H&sa!N zcy7MGSSHkDaaXiA?y~|5twP^;tn@IOQb74^fwQ`!fE@Wr;`?g*roJ3+dgz|u&8tRKsn5Szr(5{l3qkOj|9BX0eR`IZ8 zSUK#n>{NT+G=z;mp>-E!t_2*6b0kS&?m=DPLEJ6axSW?_dUT9^f-Etp zASYnN!J2xtKjsc>rU&{!<(Xat@Yz;4wkUbltC+$aULtZ+)0fHrjrk-W7)OFJB7Mt; zXURa{cU}DgPUF^_2cZjhv)tTSWEAr(=GZB%3~#hqp)l`hKAYzNv{Fnz{qds=1S>xw zfWzmb36XGv`UB+cwcaA+29toU_wt4EkDs0%P+al)Cl=_t$a&S@BQv3?HW3gtU)@Lq`72BOqlZoBQ5#C%NidkV*3K z16@eEv)Kac>wlH`^A~38x2TSzH#SI<1VAmjRiokDuKnNnL3nr4Rm`g?#2sTroMM@{ z70KNS5lAX!VjC(Y0$-6EWrfPFvQu%LR@^4Nd`r52LvIv9URxb1X1o<4GS!Yg4KlWUXoP@zt$^_Ase6KxG_#k~Z1Impc8+4C4Kqlue&i zR%Ep}f@k3ln?4rg_fNtYIw5f8!qH(jT_aD(7G0mqoRats-i6D7r(3NAX8zle^|aCb z=pOTo9op>5wMf6!Eq_p`aq5ND=zBfHAhQ*@hNo8^HdG$yoZMWcKWOVwWs;$BbxjFFKJ(_T-hg#@p*;Qg4=I+qqF}{sh z&K&oU8oyjmFsKp6jM6ZV4+#Vl4{oLEC{!a2i)?hdl?0nNZ&sIRZ>WeIvuJND%hI~^ zg=V;?X;_w>a3}O`9yZsfGga%Ur8qas_SIDqM;j5+oEVbW@RCfKg^RDnT zZRRwmQWVhy$rB(A7AmP-bTHoml+ZKV{Z(tHsA(vY51e;dnP+@5(g(Jn;k1v`dD)#4 zV;!kkIu>)A&T~o(a$)9DV6@+okK~zW^BS8np3iVO@LN?D?3v$Zc;cQCEWSM$J8{@s z1a_c{kT_iXWSTDl3vFDK`(aOqa@{+**(W9eyXl5{9zd;Kq1*nMr{%Df&Uealu=%W7 zSsdClA&%cz?+X%N`u6FmY`A!XOJ50UY`fn#b{5r1W>BRpc%^t&7o$RxFspANV~_xd zD4Se%B=&|b5&cQ^kCd?z8*L(*Z>@zDgs>-Vr=I}>Hh#s8FS1+;*RS)>Rzo~s19Pq@ zdGwHp)rEAC$yT9S>YpT6NE8*Kyg7F@(#*tW*wjtEsgt8XWGZhvUJt^pr!Wx zogfnL{!DI&<9)tXV&Kmvxv%D(r|v!0kw76}sq=R^TGnhA%^GRu2Xz++x|05gZH5v9 zTO!8EgM58Q1wVBuKh+Ctt$M(K9GUpSwtVA!7jUSWDCY?S4 z^$Ayy&ga-+!53^@&c*EoNuB5hV7>`~Mek>m=g+{%6fXiA<2|+jJpE)m*aG|j`d}RD zxn~$aI5@Yqw z)vd+LMrmkR_`bpzi@SZ7fM<~ZLfvrp3tU0Urs-ih(c#I6r7SrgPu)d5*TLD`ps&Ic z&Y{~Y{RA8FPG<1lXZb2jg0sieJjv-UXKWY8Y68=;Bzm9vnjt$m@#8s_FBAY#>s?Y- z`-n+tTDwDHd){f;!K*=#*z#Vl*mjDq5@Ogy9_k8YwreIc20>1#mNq|{Tt5hPB|XhC zM_>LZY|%&sEqB^r0Q&xeeQ}QZSceJ!?|A~iGjICx=1PxJmw--ugvQ*i<)o%D#DVnw ziV)Txx(#xqCL^e|iqB9%@FQmiePAmQ2v;EaPC_uuhv6{chfn0!5{Gb}6Fk^V!TNbq z)}1Y_6>7kTjPTeh>Hx;{7~nyP#9m9L&?qevX8xDNGb}r=Thu<-g`kdM=^Iy%YX?kw za}ydtk`oe%l?(=w-Tsik92BpRLbn_zrfhepCNNG!PNwgHLffI4YQt`Dkt2q1*V&ZQ z!5mC=)gGR9Swm~TL|0Jd$-k1lA}tm0UGO;5n*_hc3TLw2 z>gW4v1{rB9+|HrGz$1gB|-JLLU&n4s2Gx_v7@&`VC0quc}o2ZLwM6P$YF=IM&U=vSXf{C;25 zqH;v6b@fU3aUMiBzo)-}6M_&fejF+Kx2JXL0vJ$-!Ax?wX@O+oAW??_0G%Jfh_AP0 z-0@o*Ix483TEX>g2cH#2rUTpbC4y0GwX2E|SY2S{nx#B$L<5K#u$)HdQlvA^Z`JgW z6mIMo^ptWh`Qk^&K3)7^s&mb-!+m9czq7r#_gKnht_yg4oy7LG^?Roi*n|{wg<8!E zSv|Ks`|#4fnt+>cH^^zjch8_2X_RB2zRx%Bk~M(dj_DXZ&_0d`Oa8`rmgaV6&$V{T z=00!xb94G(8FSHLV*(MQFYq~3MuJ~9&)gzqne3lWY66U&MS+8xFvZ$uAs$xqpEM{~=|x%%dK(D3vv z-Z!eKse($V1~Sw2n&p%vSw%mi&Xgk)${17*bgK#;`!_qbi*dqQsH|V~t~{L}0fjK;+iR zo9^W;b7e<6w>-H!@I6$5#=*o};g}%X6z9|Wg-nkK<}}CTM$}g_gWlPFu|uN)*5F0C>*_i)LIKaD)g*}*-=B_fceRablsR<5|zxdI6YxQhg>;)G2r zkZKu?P;IeUd8OW52MprKgYkP@I@BS8C+<4eBB}Yau?)SwI{up)BbJCqu5H@3J1r!= zpcx}lpfbP`D;#2<#p831deHFTlg6&ZXof5Hspi$*OxB5sY{gA^pTNV(IbMt$wF^tf ziOuY;n(bY>g{;{Kk!1$Axm4IBY<@tv<(N=n!RAv_#X|N%jVyXPQ2lQ3v-zvDIk#4~ zK=^lHBRRAmZmF~le*!hX-zvtk>rc;CnO{rObKQYL^s@%#+TMIUB8w5I5T=Swu+I(h ziFi;~^0tf#P9WU2XWp!AKdck7MgE)E9s&#ngX?ztWXjlyOz2~UPgv!*G;ScbcKJMK zS(G-9kevHzvl)Kb;1V61#^#jJjlI^Bn%2gXVXlI^xPs}S7ukCbq{VLvCY8Ii{@OLT z>9P=hnFTh!)XuMMfRO?jNK{L=V0RLP+8sEFeR(L zv1wLjDwWV=)-K_42Z`;y7T(PgACF$kRUH@CM9#jlp8<{Cn>OD* zg+S6*(^SqZ>2_d7a$l3Z)plJbQ!w*~1o8AIz4igSu*vSCp1;hexm-h*6}TKU?$DN? zM-F=70E?FTKNgMB91T8s!C$5YDH-sT)@^TVD7`=72<2&g*)Xw6Y5u$#oH`vJF0_B+ z>}(C2P;7VKEcIsGKYME=)=vt-sAbH*Q_XTh@xx9E0Zbx8xpQjOUWrGahsfFG)D{C) z68j9;$I!TM8lLIhd_d}$W8NP-8eIqMQriv#G^_mK#h+CJny_D{003dW?Nl6oRd2xp z?nmCH)vZ$W;#u1KD6H@qxomd0(kU~TEpiHnjZiQg6ZR>jN?dLp_aH&SDa!U9hq(#! zuGC(Lpmse`F#WZ&90=sb$bpV)Ni`+CY{~&=M~E$edgdCKqPFPhU94}BacI4a@j%qx zJeoL(f01mTskOU6tk^cZ2h52p8&?<;avGW5KXK+K1yrjJ6un4rvp_|%{r4c28I{|# zb5oZ`&%oq*J6~dy9U^ROEiIkqNK}xs^KR`j33YH=IGko5MrGHMtE6?+mg_P!ZGDIB z4a3r$&tUUD-!Wl|aEr6^DaeOu4VjXk!MJ}3%$K(fLf?X7c3CE!cjygtO zKwM&}H5VlKcr9a0tcksmN^Eo~yI@l@JB)9;!zOo!4VCEXrvJO~brQh4z6_wk&)N~l z4`K7vp<~Og^iO1sfrw9v?e~BrOdZKhNtj-5ONF);=3zcm@)kTm+|^lhH?BeP?l;(u zatTfmd&W(--K17GX(=D2CnLl-ZUI_rX&~T+~ zNqwFbhJCzMi|A|efXz<}I-}4tw?LW;o`8m+nK6Wkf0I=(LiF<8!r`R}5^+(oaOFjZ zSbPh!x$|JUY_J97ItT}<)CVl>8`?xqPUyP~P^VauK%o6FUcNI>le4PS{rk)4;BiKm zUaNa=?taT%j`Zl+e2Kf-$UGg}0h`e{t5Th0NvC;yfeo8UX5q)-;Wo`*2Pc(@RWfCE za-b&(EU|t(MIKxIZ}p?U`hNbQlL!R*CkN$YRMWuh(*EYKG>G{9re^u_qOFpGDud=D z;h6q-h$8pZC85PQMq}+};@GU*men_ui#)g*U4DPa@Csq_4HLg4cPqZ1^0AwWDsJwR zJlO4+tGiw<@iw*lt0KtI@m}qs=ePwUCbO(z6JXdkpN*jb9@Z!E;n14yB)61M2NGq% zn_Yr;JKs8ewT|FX!Wut&8S&seb(wK$(Q$T9NT5p#&G02gbI)V*Zcy*>u5biVH6)e1 z_R63aC12qH$^ql{jJpBC@Em%YAcW3w z?n=kHCFyv3TmcxcWNK+^8^xSz<#T-4Ju>8GPQmlRWZ3D3n;^E*AGm6Ng;&zoyF%CO=uF&W0p7Xkm9^IQ>|35kf+Zyxd=>kLqVTA z4jG)1L2F)cyZ>V<+j(U_y_0^ziJ$qYyK*PT?1!n_5eATZ{{*5&pGTyRe2xEnZ3AXt_-r zi%T*XDMcXu>egOk1Zu1}fM}?^>`K|kblznmstDY~dVm8ZHF8|r%rnbvwXS*OYnwW&(o=UxGN0AB*APN2v=TUH=tVfE0w9Z$v>v}(=J+*!;C669 z5PX&g)iDkmlk$TNdk(I^-}drN>NB_Q>t(>N$j5YmDV?c{GT8G2KwME4PM@VUzw}s* zY?m?xCIu%vhvUZr?V-AZNm{R6hqJ3_WJ?P$xUFK&v5yNf!j)qo6MS<4gz`B%`-*IG z-JKe~nry#d%|<=d0_vXe$Ja2@1{Q!|{)W6ld_4~lAXTITlIpQbA`sVJ+;Ln4fT@np zc{1t~2GEbRU}?4Y4J)eK+oM^wvYuIDo`=k7i%IwBo3jdrnSw%Cm?KR#SV{q2N8` zwuC|9{c2f4qt=b!lv(OqGa;8gn-aF9=bV?ar2CN&wC8wB#Vg-|4s-}pT0b|Ah*H_+hmg}1nZkYWj7 zFsxTF&$EbB(84sj&o`h4+>6OY7>~vQPJ*w6AQ45`Ap9Jzb0UdwUfERC9l0lHmjcBwYHns-v zbX+SOJruQCrE&lgv5ijdQXN6cEABvLn>)84H|PMgL|QcxuhHqwk#bvf+9200SWxu< z@T?ON=L)&a9f$umCWt)H=W0OU=yJ!dPVTmtS20JCNF~>lHYw1S1mLWAZ}@y|#X^IG z6^U-GiykD!<|DePDX{3PprLChqh+3!L8_Q{c()lnc=!@p{fB)z0N5uEw$hdqsbZRugE_;Kgixk{%nRun+Nd1Sc8um}8{ z!8CGvj;VBG{7e@(3q9P~AzH(Qseuk9C?PZhdI+=u+J&)tfHpXWyfIezq;ZFVC@$^j zQ-Cy8B9_wR3Vd7r43p-U-ugY4IDz?Bp_T5bqplBDI4pPG%*0%ek5g0lhjj3t{bKaY z8w`0TEag|)|32B{pJsL`y2sDL@xQ(Ie=%_X?|Se5IXdUhW50gb0ES_5`}M!6(f&5~ zRnd|$qdT(yB|G?UgabfCbj>9CvWxLQ7fN#bnrOWIFMi}d>-TrLiB$g(Rw8L6x&HYk zf4By~&0jTNFEaK2F7iK*=Kqpl@xP1w{{t4y|F=bc*Rh@gsIDRJ3?_wYA^)YE{6`Zs z*7AqSV`lvMJOrQuUUN$SZQp-qyk_%YquuwTrxK;RymVvbTaziDG zT|`mbdhoN-NB~OJ@r9)NgBUdxvGMuQRuE{~ z{wMz3SfjtQ!>9kuLP5`mEB8>H=wV(C2FqJNTmDi+wRF;w%6^ zA?7Z7UE5-8^iBvFM?AY;L!Ien2f9$zaqB$Ns zJ~`P}IVk=tTQ(xT5+Ynw=(4dzlfpq`$6l8ogt4o#2IL|m$fzf0|M#=FFW8M5cL_| z2>k0Ha`-1ZMhj=k@ASf3T^^kBs>zPajo2y00hDi<5YiyCDAdwziO2^U@93@1Oe&}` za}`}h!b%N4GAhLXnkqM5UkTxGHGqCnnzqOeTT8YojWR$UCA8}u{dyenYpMP7yh-Dd z^3{-pQ3sCDYA8<*zz1uem&wpn&6utdSLvIbj?KjPrzIG`LyIPVTH|1h4@Bb0z)R*_s>e0#P&aSXq|_iXH|BA zt48Fr9H6QO-giJ6@=VVqQU2e23eks_81KxM0eN!cW?@mOV~p|zCGn9PXW-MTRv>6Z#~GbNq+eK;1JOFw=;0rn^8~q+-9MW@C8Ww z{lYuNtG-r0Q{oh7ODg*;R3T&r-t^6$C(&=RU)J+1Y*Hn%sO1GF+;Cd(fc5}&?7if| zxl0SNDsgR&=99D*jkEpv*)*hf>#$P~V?rSE-iHXW(@0>0vpM@bW%)iin&juq&&kIg z+wr#NLWuNl3N!L|*v{c$)Wo^MA!#KuE>?#^^w8b0_B;$+!DSa0|FEdXvJE^DsleV@ zCd@Hexyc*!Gv%JXxjAyc&X+_XO@VP~(@W{fF^y?qb5{YGs|~Dy_{nkro$HP>e6O|- zngQAe$HN)h9ZBrjn~zm3t*xC`KkL}mn=W`z_${(XSMPr4tl zu!P6n3ZFlaZjH;LV{-YRj@o^HN#Br}`TUrQ8FdCi0j}Xl?zRjgU0c?O# zTlY9FXcNCO!h%H7OU-fQ6|541&ZW*YUR|#9`VeZD0QlP}zyVC((GKG&Z9bjZx7l|w zW&|MZP;t*^=i)w>b>%&CGV;Z5DK+N+54v97O&h~E@lI;LZZ(`gNOLCL@TY(M)lj*y ztOw$-TLxv@g_g}fBm!UafS;jr&u5k1LW?a&28EU%>`yAehide}%;H1cBO=`@( z8`jiykU1ogI3!W0^_&#lep8fP9mYl|Lf_DDE)GlTHIb^SFw0^l$^>lU7c1l#9b8rq7WPdO*z+XNgYyZ&(& znRwD?u(5;dH+1l|brfnHjJgE(@pjo?)OCxMuEgmF zCTwNaAbyaw4D~nshD#t-`>;Y< z33#Q%Y{TLjl0D<1-#lL8H@3uU@#9QFHqdsyZ=oVRMc;k@iu!`fGq&0})jWFJ-upq> zig|uVQ^pZ&qwa<4wXIisN4P5PFVxdDTaMyfkX!~&IJ(7~mpCL)7lWF+bg;zkx4SXV zGevibD|-c*1>NeEqggTwfBvLH{yx$g^wdcDp~yaelGtG2wO73k%RnKH0w0GzQOk+q z7VAi`3{2yea7z|hUx3HgHsvaYrEVwB0-@@e8Q5pfN&(VZ+R;+y8eLG@fFu;zEh`K! zh#Ibz=R<9z*s+85CVfl9#dpuf!M-llU?+%B{)#6MII3-p`}r0cG8uVVPfG0K3d*&{ z*W0iOS=Cg&7KZDpAT(VVXddqEm+C-fVy*FQ1S(FTfX8^4T-UQ*O3C zU%w(s-ayHDHTi1ei?3(wQsx^AKPwd2>lw^A6kAWL z!RmcIpC`DPGA}B`De)V~_UxP!@8jlJe@HNP{?2ttbmrdsMR_(({4hx6Z38?rfL7b` zrbVme=5Kt{@RRKH9_u#)o{R7W&5{iNwt{_Vnesj1M_yo-;wMnRoy;cF`2?*M58G))?zp^J0nxG@D;Qj==H7M; z9r|?iB~LmXDt=baIK!PRFe89!ubp?*Nw2l^wQh8%A)OiG|PO({jd(q>9 zQ<#^ei-`6Z<~{txjCNPmqGRL3xSjs*eEZ0g1|jo_^>y!b;FEAJmMK%VR_A0+D}so=ki!ahG-o3))#fbvrek9O4C!L$o2h1(4zrS)6_v4U{8R?6E zr3r)9@aFB4FRK3P^)z_XHO1VN5_F65W#O-%3%U|n;6{$J;@uN5-K#6B5trsZ+ z>V=De_2e~1&ECQHSSfQoq4V~GhDfOtcZZ+0%qvl-I3ur#i2`XTawv%}RF$ZAeHhBArSL7tDo&e@k z>qA$l*3Iv~U26Q!NboLXfMuiL)uF_o7fz@^R6=-9RLjMH9lI#DJwmGSk%Wg~RK6+Y zTi~=r6}~0WRf@syz2W>}6ZPR$?AXc{utdzqhRzw$@ms;e%Mk?mOz0R-JT-G;W5mFJ zLK_UX#T1~=D>44>%LJ*I5LrOnTx?|<7 zC<@NW_b(sP;YV2@n)ht%iZpk)m)2f>6llE|a6z6sq8mso2vfPwl-O8MQn-PXKVG*io`|Hfw~$JA=%P?hiKyWdJIGe*=HW@>w#ZOp znsHrwZX6g+Kj5mT-y*))JwY$9^qaawUsXfwceYKTewIG>TarDe=9={&ILRb^7k-xg zw`?P8`lkw2PEuZ`RL5_Ancw{KcEYpwWBLWZQTM8X#OQ2ch66Tww z%5Drd48r{iOCxA=D9s+_=78|@#-IkX2Z@t#!$^7581O8uKmR(DUd`O!3{S0+_5+>yitaIj;bNkE5^MQEJK$T#U4)qw?kch3`FrQUFPoI27S?ER$%LLyDpu8q`Lq`;$_C4GKU zQ?F@@VbT{|gfd#>zJ*CdWJVqf&+M(|?}<*+PBMg*i8|O0efD4XAbfzwW$5|*tqs$Q z(U%mK)GxCPe|Ssxn`I&HQRGvYK3Syo&ox^S#&gJH84IuPN|`k{q56q^vO+sXy3Qo+ z&kSTJcT*mX*-#(o@&+pg@(;r@E!&pUJH+eFlDorBxYWv`r;Vq~XXex}k7H zd95!DB!>(%DRpQSKMxlLX0|4?>~rk2v?K)4IqVxV+3+K@IQE0^NdBzEzhcXXA50Hh zd6Q}OIVgzC$ywRb)p1Ea(D1?L``COSd9wv%d(6aoiTv{3Gs1U=fcY^(RUe@;4&Yt>&DvN2jrTrlO`PM#fRgb3=C7Ksj<91)VF64}LQrAhz>7-$m zyFsTC=CE5UbYt8$0`1 z8n-{U6|dyPXTp!?{@55oEo<>hByTld-$cX{mgAaj`Fk#(%OzldWd?P193b}(Hl}wD zL?yd^D%j%DJ3g-q=1Qx|<@6`2W+WyoE^n?`jS`hCWIOzYX`Uq-)z4bR71 z;H~ZO_25!YZ#tylSWSWI7XOU3kS(6%4OsKI^o}4q1|5=thLgplU*oZRfc<}^I<=<~ zSz6C?sl5T;7G1(D5&IIj+i`*DQSs$=RZpr3bnfu{l4XwxZmydRKSA-FZ^yJQPem$+25DOJksp!}D9}fdh zW{IIJJN)>94iY9*HY{YVDVbigM}tBK{XB!k+igPr?n45GBcWdRy1HC)%h#W$2GcrG zN7TTVbSK^DunBH;*4%$%Pt>UR2Rgt6RQXs=xRSG-ewEX*ZbdsBheqM_;j_e2&xa?L z)^){sPv61h7jDtZzu-AUP;dM$r8imt>K+Z#m30y5{1k5uK|Q8y^P0#pKonGPWHSq9 z5?=Epz*g2|m;_+#^8NQ`380IFlRV=mDH~!N(F*JZE*bg2 zTD4`8=w`}Wb?SFl3}MePQZi$&5-XCAqhVO z=Io`W5lyQ!2j7q0vj%;^x&Kq+kjTH_OJtAFhz8Hn>&2R}u7)H+o|}e}6rb)2h|*;_ z*t2r7%lwEXvh{kAda6`E{N>Ok`*5Idjh#Mxfq6H>5R%-@?b~h7&zx%iLNC_B{qXl( z9DkqR&q%egkC`Sj#A5H>L745Cv=9d)>OP?KfV+Oy-y zNxv{!;>vabgqVdH-StPtX{AIJJ?;62_wFzDpH9_RQ0)T}Y+% zvZz&iJY=WjSv7^d#h6UhLq$IaTc}!TwV=w8!>`6uo=NXeCtZ}c;~VR7B&P$7Q$LqF zO(6A{>5O)D+Qw1&x}nHDo3PCM$yKfPv!ZD3vK zpg+{|L1CMbs^EJ9EBADdXU}~l=cth*gQA)GVj7Ms*ppQY&N->gwgg#ya~$^#p0wy6 ziVgs{Ut?=_9pTNAU!whiF`8j?D8sAYL z?%EzRc9~>hg;Ux$AfYMUpPSe&SxQguRc%D`BD{I3!#1PzF_!XM zte*xtMz6r`W?Lg`M<*Iv!Oow;_6|cQ6dmY7L@%7(+*+*T#~0CibN}vjLR~TH#}hmC z1;HT*b6QVKe@EgvZ8feNs%7@>qJvrdjf6E(Jcnw^ua8m8wEPV(^j#_JAwtkp&-tmY z3KotpkMHGv#li9LrQP=+2a@NV$Fu&HydRFbmi-mQ%y-$J2d$nT=Swv6kFVjac-fk4 z38xnoCQ;e{ux>jwI1JWwE~OOH%cAbf!vQgLe72Z^O4;(cRgB-v<#LG-Hdkm5B7nm=Ydl$@_3J)ei&X zDZZTa^S8mq0a3NjJ`tm>PObEC%59P5jN-?U_^u(L&U<1PqhO5Ke4LZJ-KAq@Z9P;b z7pILawKA*QGmFAdJVkS3gV>LE$(e4Tq_Fk15x(HOl-3~@|BQ- z@V~@M&q@d#m-b4if7FBeK7MD{;9---_Hgq=e`KvHm#S7rf8$tpuhHDRcH^E5((;mx8@|;4z*xoXd{bJF@72YQ* z!10>azkS)TI;o}+GvIeFec34LO z`+MmG(?iHXF5_HR08V-Ffkc`$NSG~TsC*o7;oIpv3>5^!`8XUjdARO8)*~`xda>qu z=i#{FR$D+vqbP!KXV}j!*=_SVITU9&n{x-hi1UYa%#Qtd zeY&iCp@1li><2kdD5WVixA*;SH~^73RMY-UEqUfSICbR5l2vi9Gr-A3E7&XOo_+iq z$E1HSI;VR8N-*L#>eEqEhKk0Q^n=00S{8|UdGtWK^K>dh2KY=W@okBAOA`)b^|200}L?%&gD=xVXq zh**?3mXdgDa|Sf(ocd=t=eKXAmb)yz9_GcKNYHcD5@WdEuSr2T1wljZ_!9o#+lBG= z;+V6xIqB}@41_*-BD=jhbH zXrKOR#Ru^h3h0-$CG8z#Sg_}!rh%?UIs{6QpD!F-EpXF;Jmu^2W0}c@ zv(M%#DlC9K*nPbR(O6Rdp9lsydR+Se9l zFSs+NaQPvN3f!?TXl% zJ89}Hxm!4ahTxyl(Xc))DTMBGE=f-(hL5vpxfLsmor-gRU4M&BaPV}9VC(hiG9f5& zn7x8wCP^< z4b51zQOH{>T}a{FAyKydH68k0z=O}--T0h<7U3?rDSx%Q;3 zhbOjs=e9BZp4Z*wz4_og64-aP)oUuLY#iZgu`I*dj_`MHUJ@%^+!onz$1l!N1x#H^ zs&nXeJ>J$F);Ej+lkchWu%L?^Pp^l(8dHo!_om6pAnA3R>5l+5D%)vi+t2KJ*^Zk{ zB&9hemQ>6d;Tg3az9du~Mh-KTvl29O!v_Q<8O>nEK=0~IPp{OxM@#n4H{GYdbRRiEX zYh*FaLwaAYDX-9Jjsrdf#uim*bh0mAdAu5#d5_>NjjcB%+GY91$3=K|zr0=179B9) zB3WY*KYIWE&%G%DfDXjmmCR^WZLI?int5v!R%JlK$VBjGU`bKZj-&_wEK6KIasqEoO!N}(rnbiA|P+eBS1ZKK>z30jW8%5!V5 zt+Z-^ZSdH5MpCLz5VP0Dr=r0oQpgIqYULbUn0I*bXr!Wr6rpv=L#=9#Sf$!xM)IJ? zTUfbs9Jq|+Ve{}S`bWwZxsBW{pAg)2PLQ0=3%g~)TQnK1-n#V;QQxp>#vf5ocTsM! z|Ikb=7`DrvwuMv^lGUFcx;V1wsge&HrAUqg3Dk2Vj>oVn)ZGpE%2>;@~r0}8Mex)nML z+%>D}=YI5%fP+UVaqaBOaLNeLmK4iAT;7R%y|6qNR2EC%5;(kBObQO;B}IBo#4ykY zSLBs;l5%-YUI5}B4UumvRm2Syy?S?$MIYnax%}>Uiu*~HI<|^Y73Sx)cFR9uMTP+j zdMToyGuPP21R#J4DprXiv6vcm?>qLE%wL6gGr{4t=X0% z>>>@Ah0~n2M0&ko;2^O*uMGknOZm`Xq{N`&6%Kr`%5dwCy?>T`<4ODW3SBaeR5(87 z4Ojc5#FKm(Zv|vIAmePClT9GdWZ$=83>^%n!QEGN_Uavs$RT*tha)X3dSU88WDJjI zi5BVh%#Mc4*u<5~QA3*T*u&CgeNqH%;ooL&pA41O5c@x*E%r>Q?bLGVlV(s^cH0-`PmyLvQjY@*eBANdOC8OHbiwj+9|Q3MJxxHmqGJyz zY1p&m*i&Yk9riiB#QbP1sQ|TBq_~~0xDYieLsDh@d1U}nkggN;I-WByQ=plArD@|? zROi-aJ^>n@@ZOAx{VP#7HPVIn-6|v-PGuu^=ch)VyCP;4%V`i2LxM$WK-5Gx$J&zc zKwugm7$w!()#Wa{6V%)Co%bN1qjt%*cLKKeaZ<}biR>kLfKb!{?+%r{yiF3tZ(d8RwdjH=}LJ|6%;taZFU1IcOHBY?L-#0WNpkxaae# z%{aXD-HX)xPBX{x@s(AWPR)uy3x|WVdG(jwCY9Aw9AUM7uV3u?t}jUK^LuZh(!#y< ziK-X&@LO@C_Jzl4x_oGmChizBGLC@TagMKj$n$kH)7reI&6I_wf0C z3F<@}I0z{TMSo#RIRHW)IZ^=nAx4@bLm24zD8abcvRslwP$`!!NlU`hz-p+(lZtSc zH%&Gt10E;54J?C^W_3!FY+(S^N@682pb8|);d^XlBi3;5793)wI%paFAY}d4NxWpc zO9T0o{OU0&4v4qhp~(B;To<*3!=(oDV1^tWbf@YTRy9tY+Kp*ue!BbT2LlA z##s+NL2+(b?cl`C(KlDkv zCvXIsIFn5mkY9fd zi|*IqyX0%*6JkII=Vxuj7O7y`TcMEy<0%wTlF1>-&t6 zVnl&q5hO$Ry!!gfuB3@mB}d-JVQAui?0V3@+DyU%h0U1bB$r}@Jl@(E+6c%WGL0lz z#v!l*z<{A__CT%_uWRLuw0k{!U(k8IaOMRA-+s~@b%Uj?aak=ZTwoC%c=z(=O1ofg z(xUytygQEs3aF~g7h66xJJlO5UKXP;)u(*Ktv50E!(fR$VzYYGT-SAf^DZDaT%B+D zQ8pEm-1a9ClWdEz;H|e)ILB)c$j_Ssb)LPdld*cAs?4(cvYdr1H!fee|7LlEr$<3Z z28`=?jtVWT3ah=aLbuA7Q`=>Kp}nx=jsS3YV+C~=I#ck~!=-!xAB0rr#> zP}5m`(#c&Hlx6C+jNgu}xI>!qtX8XEaO?*14#hC#tV#49L5n7)oXe%3LGiZv^lkP? zv__b96w+P}NhhWmb5tYkygKAC=wA-ZDK{jA+msHoYlI&lI`TzOp*fR!($&vXE*k`B z*YoRhE98dNFpZTjJt{8fmFkg5w!E?3^KlDLqZ96}eNWr#^%F@AgEU=g{j$O-b!Wyk zp{2e~aik)@LcJS)Ncz56D@uK5%uqw6+W;bAskuo`$>YVE&PqzoXlc}xfov5%1earp}3M-0B6<7(Rv>Dm|a4hBYv?CX|nXT9L-}u-!y_Jw%X+U_9!tS z<#2JmJn94_UWLQ3xYEI$k|+8!EjBq#i@lMxfFQVSK1d+fC8-M&6W?9jztx+(je`|7 z^LAZp-C_8(u^ZOS63OS@M(SL+dVQ2{h<7{n6I8-&E5r zr6BzN&<{mgD#R;dF08w_KYf`4(q6A2nFcq*d>ZP z7^qOxGMzqG?-02{OGgi18U3>9_1)Lq8{xQ=Ll>Cnny) z{q0+!c2=A?IUaKF;1J@~?2in?il$f|qSNAyn|o${D2};A=iU(R#f3{0B|@sDPgQbh z_^nM}E;&DEmf2~ZDDw=WVp6v`>JAIur8Rm{ej_*<^C;YBUN63m>$>sMW*|IqJy_bv zZpqk+`6+O=(J)@WQZX_&3W*fLH@GVFemi`ZrfPhSQVk~COS}y~0wggn2Q8}h*Dgm+OR^nWi zw14@wd$JUM=wQJk++^BnwD_w29d?~my%4{hdm5?U?f1uJFl1HY2N;?98Xo0YXNozD zeEj|n0+ryb&VCxqnKt|J%2rXrzzAdCTTPbXTMfWHbULXICPb$O{1c_1AN-%kGlr78 zm{*hV49#N=PlCELT9hcG;+HU5ZWqUF)Ff->@bi#nQMS;}iM+`5T%oU;kSpnBfSla# z`5mTEf$HJ5ObbOP0D#Z!O3JnFyCx#RIlnVT6&5HnAFcDuI zC8^1)XinGmpKt`!eNf^|HC1aD=vSvE{9OO%-sRiF*LKlzn+8%cVbg^dGiSAK6Uv7`s@n_U_IJ zO_VX_RgB-?{(1%WfprC{v;<$b=GA;@l*0Wfuw>L)9?EnD71{)|PhX<*2Xst%D({#0 zyWb2DCIELR= zbv;zkdvkR9Lw3>ES2AG2AV&{^li|@|1=!)5K#OP~#PtzoLTxcBt_KT0>4e>K-|iNv zl~T}mSf`O7&Fpwst;Rl&0bEI&!TF>(ddKKTkKOGL+OZOlDO0^k8WEmDjYdH~iws%w zdyLIeKJKh2BWA_m4>T$qdB3na;Hl05XWD|p5#QpbkCmhFVm8$y_!A0LXs3{A%!kN; z%)vE&67~lhuVe9^YEjOWj8#iG8+DK^@`Hi?Oc=C#?Gy|Hz#1{%Xq{7%Bu2^N_rp&Gxl2ZlE&!=%=pP~ZBZr{^^r@OO>>N2d-DP2pj>XU8)N_=$+ z0Qr@GbUC%9@xZlENwQ+!g8t)6&rfr*t6)yXv3;r!z9-?P7v}=7l?4bVkMhMR{U9cn z9ZA<>T~Zd>sh%QVy|6pL84udP&z10uP1lajjWD|Grk$th2rHk;n@G;Tb=sq-=1 z2~QU^=vd^jrUWc*`quSdEtW;&n9DBeV(tQxnmIrNnv2}NDApw~wHY)A9GTIMs?~DQ zW#vEG>Cdt#+vbGP?`lpT49$5G%C){YJge${{(R0E;$J*xlXg0k_rakQLj%OFW>7gH z)PTu5P-|xg){n8kKYNJr6pU|w#Q*kPOc(u61Mfk>A+5HFH|TQi2@`<7dOUS}N6~c}^fT33*dQ^rt`F!_bUB zb4oVXCALJ+Ha~>ipv(cq)M{>JE+`LY$yq|WtRYKK>{BaZ&Ov4Ne;R~XJ7XyyfEYil z_S7Cwfc48+fwxQ2XZ%?=rY_I#uJyB-jo_(dafWLKhtAODoCddh!p_W#$3mF?{=+|? z@zw6s&FYK+P_X02ISr%|8P1RdA$+&dKP~^cng9bwHcz>&*C4kQwf~Eu`K#>A zD+emHqSJEYKVJr$<_8ACAwE$W_SF$T|m|gPE z79;=J;2wt1Pk7EAt%Lq2WcOP$L=XcQ&4^-8qZ#y{4csm`0J?XODA-1elKsp4oOaZm z@-ZmSVdVQQsQhiHcEBt=6+In^&K`5lJPQC$bkHtu{;wn8{Obt+I>P^#h%jM#iNZ6{ zalN{<3>oj}2TFeIz^ZBD7B-zO})nGbO3Df;+nXYwgg zB#cpi>o~XpmO-yaX}XfNS$(@iapjucTLSmB>fNb1;0Wa%@U+?MHo}W9PL6i!IH^py!Pl_3?}`$vG8_80m-qECa-#m}64T!`oQO6;6)|4mA_XeW}!)R24T<^{ca znk#tKHNe@up}CmpABLMAB6}W9dft>gsj-Xi{y9pg83TZ?%R8a6bl9lA=X(?5J1>c4 z&8GXf6m`v}V~CQP59qx1`4h_jt>b`@gh$Dhtlp9b6YmW+x(_4Y?u6xIi(xHWV&HaR zOAFvydljKt5aXQ#571EUA+Y4|ss&T~&NpQBrrUQ501{W#Xc?~g{)zcIyG9E-7KQvT zxBYbO$je!d(uKLLvqpLadcQIAysaw)u?D#8Ml3Mv=xA?U*=IpH#q|$9o??7AFEB3> z(0EtDGhe!BY9r6t{cpP8#kPtzjC$-Db2xo+tb%pRYm$BoH!d)$rAO(@XOUG_o!sgT zts{dttI3FSo*pBuo3#RUChIUkBweGqz(GWu0G{T81g!u^NoxH7T0Z5VOEAK?WZ7|~ zb>C-IV-L)c4+xM1}{}E*TB_@mFmD)cafqpqV=H3!(QYHAMwd?)X>$U6O4S8qe z-(IubGCmgS?k}BLJSXu2DCL-Y*aG`pCvy99yAuZRevPrkPA$CBb5p1x>6^vxoZh>> z(Ajj1JEW}0k>!|?&$#BawZ32FBs_g4B>gGy`ISEEvJe_w5!e(+3l}=6A+!GcDPoVR zf5dg$w!@bByfm!@;+|G=y;lDrz$8}|{fGf{A`f{a;uqIjk>?C8`ejoR4B4dgeaE8J z-T}?`wY-&rYrm9hfBE@GbUonb>91T@4q{#*K3~n=3+vxFgJAwT4dc-WRYK?29}Z;R znxA7L!iFJV1w~S}ri$5RBa$C+R)GGt!vE`Ah`Auve{Jv=*9JllySv+^*vlJFRnD9A uzZ(AASO0y?zk~L#r~SKb{?EY%ae`@kHQPoPv$GNncu7mhiRX&yzWyJBi)hXO literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/1-default-throughput.png b/website/www/site/static/images/blog/scaling-streaming-workload/1-default-throughput.png new file mode 100644 index 0000000000000000000000000000000000000000..3946cd9c49b6c3582c9c355530736faa65e345d5 GIT binary patch literal 177003 zcmeFYXIN9s)<29Oq99z6kV4Yb%<1y~sv z7}&KR+%sZeUo}4UJ0=eZ8GrJscSr9=u60XR$Eu z=CZ5l6~9a5>z7G}DVIiNpsVql%f{N&VvOjX62kd$nLt z+wcU(%`ayo{^C$F{1%^>$WVEf6agT3G2WZ9_j~6?^E+(kVHZ(LVq>_#?jrz5%3#|P zV<1YFKgi@{c=R*F5Xq}8TdJ;mmg!aSdoJf6J}F!UeOusk{m>h1zj+uwn%oxhImJ+H ze#sCp@1)K?7XCWx6)^G~gM!Mpwx%ZI;By6cJ(xW2ozo@pyQ-}`JJgr+v`Fsml6sIrfn4u1cr3YV@4Ss>4b!W;DF;6Dl=T}A z^UH4!UYm*q{z4|QnNP+`{CcLzfqs3pYItAzQsNSu2IC^d;!gAJ+h$MfKW8Y_+Ijfp zzYs3!VA+<#YAhXeS~Ut;9zK_z7g}HGYz|{2zt;w*n1XZ^m8ZkxpW<=eW$k zc9K!$UB~BB`7dr(_)>3qTT?Gh6)GU_G&=l6dJp6^oR;CiVh_B*O|A>>5$ zBk=VXhZ9fCFe)$n(Jql5_Z5Q~ePm9vC==W7DeB%{mp%`SiJuq6C|CMEX;rS&={*Li zeG7N~^zO~Wn;*oI{OiPYpPrL_$ES1l+SKn;jeyMrt&VX~Gv~}mC`TE#1+;J`JBs-F zdDNhGqnzc#zm6n+R(-jMXOZ%KPj--bE>*Z1!!i$kC@hIpX*}_~!!}#=+9nIArPH{jVD@xmvf%kR;!_xap}c@#a`W#!#2mCL_+I~znQN*{W?0{Q(Qy2pkjc)}rt$j+9H_zOMpV@2T(_^aX2z|)-v%|Mipp+$}!>sb+`Uw_w zrrW0sqWtDBG%|XbeZl)hw3P{d{-l_g$*Y)^E7x9MeVTOp%GqzcXYO8pXYm)m zw&tyKVwdkYiLt%Al`CYKZ(J$D-5S@I_j~=eHXfUs;BE~CD#q% ziw=I3Mx~4;J^0FN!C4hgvMfZ&=f!`^XC|t42n)RxpSKB>1hd$`tmqB;O$ufUXD4(~ zx4chEcD{RTNW5Hm278+SmFueucbT84F5FzW(V7N1O?Z9pE6331l@o;T&z|!5T=O~O z!{h_l;;M?8)EBz+{sHAL=m$=Tx8KB;l64-eJ!sJu*EZJ%<#Ou$)VZcJpwp=%p>sFy zavp1*t&aG|a!HUWMbAv!`UB#}qh}%^r$YEc1e@N3rq5^`=?#6cFb=)G_3_5f(4SU1 z!b>-nz)K0$)z#_M1-|js>LIXgtEI4?xRjQUYQ-+alSP##*9sq)q!zC~RL=9&T4GPp zsryPXHZD9E%^MZK^W!z}ET*4}b0n=(3RKMNjOy%2#duwX>d$!s#-1OnvaO=0?$jaO z!tl=mqykb^w89c0d6e@cYdn*He8P~xBY`FXUHL#)o5A3G?cv(tv%`z7Jnmj&z#O!G zx686TPlB$s`t*HO-L;|1ZkFy$YsoHk?j(12w-z^XH^YhYQE>TeQPir%YGr`0 zf9k5xU{leK_QZpG=iuWPjOKe2KhB>eiF*$dT zdwoE5Kp81aus8wUZ<67bu~vW2I>xhpZtRr+3Gi&#_y|c~XuP3>< z2Ma&&?;C_ngg$XFbJTIzCaNT!Pt;6gPXuUL#_RCcJvCV?^fvG^P&Sy(+)Nl35>{+> z@gG^47ziA&!Rg@EWK7Z?r!~nW$W+*D*dCM%+IqJP&Ga-&`@08H@tX@WffarPQYG;L zsc^gT`{2`ia;Y~5pcjBlKpaG5=Wu0dGldY)u?FyFA$SmMF{mDkq=h83V*YgRuUKr& z_kxkM%?x}Cx?u!0nuImq*!wXz1IM+`9!rd*rhlL559Ocdy2No?hzMKy{<&X;`!#dS z$<$NzEUu@no-E^p@OkiLp0^R45&g|`OK#ERx|lrBUvove{(J_q{K;Ny$g6%1_cP{a zlGyA;kFIyg+!Y#<=MffE3|yL|zE6GM^&{r`x$lWq{G>}$DiF8bA-<=2{(6qTk|&3% zYMxqSU3`+Uf!`5voUiS9A^@G&?tB=`EKI#ER&{MF%;>REh!L*w5!NZG<)g>D8xOqn z#Pa^i_b}w=6O|7q?ZeJ^UDUZYlj#ETp?qUYK6`CRPX%o0dCPgb)8ww35x#sBka^ue z!TC}`mRZU;%F$=yxjpePq$C&-DjKeQ?AmMy38B=Sg~zAxBhtguxzocKQy(MdP0d`! z^Go;8kI<^sEPfw+$;*;U`O6~b-B@d)rH$hY^x(r{)x0UMRSF>bwxBwDAsGxTc2WPA z`>%W*oUltmO^dCR8 zw5>_6Y4~+#Ik(>x<_>NM_CmG33sJ=xFFT`-5kHCwTURz-<}&MlzyE#bZA-+NgS`5C zb@yz1-Lv<)^y7${FEuyfY$-%s8{^XU zdyZkt6v31eDbSL$B|PScxsK`Q0XfG#&HDEi$TjxRv1FmVGN@wuXD? z&Wuw8u??T;Qw3QcF)KDR)Fc<^tec>v-!Ic`i*y^Z()J{5o@uFAKQl98fqU5CSTTU9 z4torLHn&47qI!#*RvQd`=To`)ZYjICyS6)z^neymIg0reGtjfxlX&Hc*5{1Y3=Yu% zU^Dj9YSmBdRBu|=m(1&6+2Cs8Vu!>~QQHWAK}&|X(wwT?<}Xs}r&Myyf-?%v1`f(k zK|{7%Q3npZtujJQKN=^W_tA{PGOVW;=4QjyW;QW)5ooM0mK!CdFYKUVU_)wwP^{+b z{3g&7fZ5lrLq2i~%Ny1gy#g@Hv*Cf_xPg)u<{-7Xj3D^)_T5NI)Z5q?&XVIwnuB)% z6t`90QC_d*le8J&*J2MGfF9-*?4KcNe>YApP(WqEh1b|O5!<1MJ22j_yl|?{(dd2+ zKCd6C5pi%lHm+9Qj!W4en_s)?y*sxayhf=HxdfSCXqgFu&!vSA9#`&M!8)uJvlA6Zzf?bDNyY92{?NKTTXWW{Bd}H)NS7aWEA2F$~Th#NVHfX6ISpJdcGLXe1 z@$hqM#;GIy_i*Ki*HoOy15?{h6s?^huR8Qbn7#yxzM**QXrb+-ug@S!FP~;Oc|w4J znO-_UKfxyi|FeAm#B~Oye>`VoV2E{PIQg%44Cwbi$2ylg};Tqo{EFFr_7UQ-cKE6f<2%A;llt7R-qR?9sQqN3ikBy@>2;0UH-=# zD)jQ7$AHV1{_%>xJLs~7{=-We-oB2P6lHG8+`J5Cy>#gk(D#{>iqXCM|01XV0$q0T z_kXSe00adE$pp#Cc>6j7WR;bb0XJ^}Zrzfmzai}x;^qG&SlY|)%0HR>2cLV6eh$8_ z&;4D!y)OOX`{b#2fIsN+eItIJ`S4&=g|N1QY2Lk@o0Ayuu0{)XXofP=z zsmepwU`G$jd#;{znbF$-%gQR=0{(;Ge^vcglmCs>;=hr~$>03%r2nnzf0CN|Ir?gN zd(s>A2mjZA{fqd&SN@9-2>A2u|4kPE6!bry(j^UM1p@wa)WEFo9*e-~11aQs&(MT^ zr>ohYgHe-yx&F^Py?jEQwo)cSR}ggu?R$4jf={f^vb1uI9rSGR3O*)WzCPK~b1z}l z?K5IeCNKfvzCJstP}PG`s1kG_C{(#~ao>1-{mZ4d!~VwvzkT?Vz|Bp}Y)f=g+T2#u zw9`Q5!N5AE1qJdmXn4t+bLj;G#}l-@g&T4`C;{yzhiy+lKOm~^1o~M{)>(pq>xVF$^XsvPgAFX$}E4|rurA;Y5&_q zAtkTO1b(@modqDyjO z^Wpr#QkwnLgnfkV(lv{-f8U$@z(`A{^zX}=egpY(qe8(Pf{MS|sGBW=m6py~V@|LT zr;KpQY6j+^;=lRaEZxUlug;MBmF6Z_hH^Bzq2zVfodH#Es zPfud<6%&7+|2t``f7xM+7gfW=aEgk=^mqgsqKcUe6VL}}XE^ih0S*qYtl5D(ub=qa zag@IJa7EdFWv~)qhkz(}>d|RYlP$(+RVdkBA?c{c8h<~6g9<0_CkpY)czZhm{*V}| z(MJ9I!w47ze@%^hru{ne-?zs7u&prjksqUkv&a1+72@GOHmU64b&K|J+UFq-{Jp=? zC3S|#BK7&4ByhFYY_sKDTUbVQgDDHl?KNwpvvDF*%H_Y4>`l|t@~X~%5i=zXt^{8A zyRnN5J$){C8GCz>^?zyaKV0A-x-w~fUxhWw z^?cFhMTZ=Y<@K`I|1ge&E+?WhB{_jabyHI*2Gf21sKH{hr(*O8@tF0@xnKjaV8yuq z;SY3tJoj+Y(sMA~6Mh#}H*O_Gl}|daeo35hiza-0Tej{f`(=lmxcFcF|0%bBS*wFmPsO@=95TllA0G~@7qPgSG1w9 zlreLA&`f##l-ojhs1R&1;`GOJd`b%}f-0e{2|}Quuuy0bh2m!L)lR!0<m>h?fSS4p*@BJcp+m(ry^*UZzXFv=ATx}0ClD0!LSM;KiHfsNt+90aT;H6v z@>Dm44VZ*jWP;*TDm;*C1MX0-sg}5nfVQ9l8`OZ?%X7@IJV>HxjaQo~-rIKE^iHMC z)8{s8Jj0|qUWRe2y^mIHkM20;;M0vb`Y~R49mdm}k~rhK#FMZw9R-&rl-ltO41zCe zcB9S`!kzto$!Zlkj9WS@M4Pl9kt#O&#NnkSn4%|ipDrA2X62e!S<)-y7oQdU`;Bq8 zmw8>%T*NWLv$FFPxgvrlb@+TdWU9oxqIkT&`|cI3#9AbveZ~nM7C7x%kmV7>@AN~X zMkP$pDZ^7y5Co~E;mzBNS8H@}!=SyHi)4G;xcrFT$f~qe9TT8!|7+OcR&RX=Bike@ zoLpn(SuZioFXyCpIujO7akX~N3g}WELJveIp3HL9yNjy5zqZ*6xJL?K_3)2lVdr<( zvbjANla%>*wgBFfx-v0B>7VOv+da2C7Suc0;LzTHc7hGafPLQ#Vrzz`D;nCo+h~}? zw*9Dw{Sr`shaBmb;kyZZ$Ie(#dsPYH;D%5^chI`^J!&5g>yegx>Xisg3EmvvKY2yX zK$G2bJIXf_XJ1a6+>BV(oet?f@T3MTyZcHNf>}SZ-cTLZs4@9+t1bq%`uLVgfw$`K z4`nqaCr!3+zEcIGG%~s25$WSppm}il2SY6kA#3(kN6mMwhoX<&D-eOU)?o|Lyz2#O zG}pqYjv7TBM%1>QmTBKJx4s8?;S$y9|tv!?B z7M#MWDj0CJ!yyU8BNN~qe5I%|LEWzX2L1v$%aHFeoI}((4lp@la5QpRksy=|2ofQY z?x}7Lb8e9K+NvCR)B9v@`&Z4_{w`Pa3jLAMc{os$#nad5PZQ8m{Tv!z;cs^#Sn+5jP%5}6 zf67U`Vyrfs8|iYbHDJQFVKM(+9-u`Mf5Le4!XOR?9Wb+WOsN!#bbee&iLjL*hT^h< z)?Z&qJWXwz4V*^P71pdyBk;~xT^Fm=9YY7>3F2^L9g1;6INMM|2HHf-8$M+ zpPv#1lS5jf!l_L(^0X&5L5X8e6?^P|X1%1M4R8N3Uv7i?@kW*cag_Sx_qR9ozu!s) z7VEynihh^J22MpZt+?D)kxU=*Ws3J5K)%bI6O`@8Y#{U>3`LsM)( zaey)jQ^FN&fRHqHLg|t=OL050RLc?s-X&PY@t#WCm=V=Z-!r9EY7A}1V9{T{Q>mv| z=&>8F&~gOiGnS?*4dO%Gl)%}BG@1P=JgCTbwJ}UxG~>rr)_xq zHE?!sm7LxHogDbA2A4~JxSZF}Uf%-adx90`>Z-P)IU`YSBT&wsy3i*3LzjA!1gGP% zZOk%f=w&fma@z_c$Rld*m#40@yu*<12Iyf2WH0%OM@Q^jOtTZ*QKh1!z_W!MI__Tn zJKKF~ZNW0GKLk9K7{kBP-g&6#@*{{m?{}rUO4Vs1oV?s2o6TDub2YekI&U75mnWWJ z*gD}9)^&B0wfq=@6o0a3FypuIOlb_dCz^D&;VQM?Wx7 zn|9MzGXs;I>s`LWtwn^EHAF+Ph*!O_uspRrhe}KZu6m^6==Ryz=idy%rV*jw&FLd+ zz++K%VfW*3vMzKFODJ{TbM;t_cCIZ8wdoJR-KbCSOut$6n3FPDlXT?#kzH7Jz#qtC z|NK4`i(0O9VCe&d4 zy^UE?%HpPb$Rn5qJ&Zc-N~_nniC8u`4)t#&Ow?XWEAqMS6P-yBGU+=7+uD&u==shU zwsc(VZR;|MhJEZ|d)6+$TqGWKH^cMju)T7`T(sTf^kw9k>!%?T-Mt5)%VfU4U;(p_ zTL`OwyOm@zi@=)SF!!D5gZe;BBLib>(>LUrxNdinJcPhwkA)v z%r9#{fp-~1KPxsC!(cM@suCtX2@UoH=*+fl9r39yY;$hJJ-XP1WS#hZblnBycdWUg zrTZ(h)JlR_I_9LxSr8k1yX>8Q^4d4w@e^@V?QrU_NQlp~_nK!%WI%4(@Q~Jm6nA7d zAimbP{P6n8cGlbQa8k)lk4{cCy|zmRoA}ub8MoshUe4_=0eZoZDc5E+XZTHjAlJ(h z5#biq0V|nT?^Xk9_ONXSOa<`TEpX}GW*6(;{rxi5AEy2u zy$Q!+9zg=$sKm8?9M*>{$;yLdhV@bh*$rEPKPk|MP2G_1Px2rGu$5lh=@I3e`tN-R zE)0CNSH1hB3w#X5f7nJ=op!JiBQk}T*21-*4?%S)`yqT$iR*d8tQa0!|DkigzB{@p zyq$i%mcqL8+dYe>*ix}ddy|UR-mYHu5tat*_+RI?k82PHUGt#sQbee?8Rr+;2d+Kws69 z%G71f3go^OFQ6d#DPHDl=n4yjOVt+MiLk? z4i!d^)SLUVz0-R-P1FUT!o}i?DM9QCo?2KqCUmM=yTeT8+4^h!v&>^X_7K9@7|_H7 zpg{`Ha_rG=_cm+JMc7zBoiBoL-`K3tgrB<+v5|a-ww4Z9r$mI6r>!4q@YHbgz=f8J z&GR9@B%h6*)K;_g;-L;`5O6aKGlDF!4=vZmE(E(B90~5Sg8aE(weNH`1b&Wv#R{pb zn8>X3e_G;*Y8jfIM}!8fC!+JuW3<(9aFvv1r&E*4f=AeqAlFTn)UXxMT=HeoD<>^i(X=ZO8*PtXDm#u-*)Z z2Bca82q+L3Uls|#mC87YX6>qzkaw=O< z!y>7BRcvhS$20<4xTSY+n==Z=3}0%PN4V;Y2Q8(T^Q2lleM2OaHDDz7&(1Qp+ceJG zk9Sk(??=Rq?*NLKMU5+o8FPpE^Cuoz+(#a*XsJmCKQ|)+sG7(E(fzc8XV%_r2Tqi{ zuT<5|Mh6SJN0%xE0n<8qGjIHfnllsFO>5!2_Mtu&zuNU;RciH`TX%(Om(_SIQY?;iZ^!P?O;KR=fMc3%Dv(fPBaT{2L}6g9*ShyG(298nAP~&`#68;_rXF*J}(rq4%yOVGph%T~;nKd^e5rT9e&X`^Y8J zO6=;Q#}~3%#SfECMQid{=FVSquIP!hDy%3O+)OC?}ypWB%Zk#e!Ubya9^l{ z!h{~x!bF6^J3EL|Mdsn7%XRA)0qwK^m+7{MkcRmM=OLsBco&eU$9AeVan)b@|U`auAx4|e2f*5T&pBC5PGbhuI9oaYW&A_A% zpe+uEw<>9BN2_&Ruis;_BYhiThDoO@yzv{yUh9i+jUb>R5EwC!CR|102Yk{(&(Y@x zEL#&r@cpg2a9}MeV7A*T0z4Z2%NxCFW2{W2XcO2t(cRenc5@azHv;7-JPz42(M%}| z?@fS5)nVLbt#ktg>&oZ&Q&F2aB54#_-AvmL1*%-JNmfdDPfAIP90)#(yWfC7la!hf zqZ0y=PPq7!Fa@Dr2%}-N0yxGY17N>}Oxf#TyW?1|>`afIR3|d>GOPFq&r;%veUg-~ z>sTZ=0%m8nhX`^wt@t6uC@7I`8J^7=MHj|XubmV_vvRTac+yQD1f7=L zYX~BlRA=6`QZ@YCf*IsosH~GW43KK;5nP;aH z{H^h%{Y=AmCbbFs>XFF<0SMghYP2=8DH=OUkLXn+-NUky&|isUakT?&G^sX=-ucN0 zG~mccV2vKN&|+m>so982;(pmntb=tJ-|R`s&jJgjsX8-<0?`wfIgcE~xycmMrmbx(xo&=1R<) zi5Y95OW&huzLax+Njxvf|Hj~sGG^hDwv!;ncj8s32seV$GQX!eK)zACWq1N6?J?W8m%31rc(xJzX+~+O5 z=C~e|$V9)XvVqR!s?$o>uBq|bk8IrIMC%GCTQQ=lux)_DK$JNft9-kDbiR)&>#H%v z=q+%#OE86Qb~fJ~*mTe;lN3%%97m_O z)+8o)L2K5${h>t_3iJ$InNPW!6zL>vOZOw>w1f>2nbmC%OHEy z^Dx6{O6vssqwTao}=aC9$cDETr*Kl`Vy6fAj?=RO2rlsWL7`g>?5Y z=JG9tStlrjE;Gv2(oVBBiqzwo0s7jS3Ow!;&uc&HxTlds*V%8m9MNJaZy}hTjiT5J zh&#ox;mPqR+pV^IhY!*!idzAslg6B2(!h?`sk?^OB+1b=d=Za^9*Ji>91n-A_mF5G zzAqm5S0-~P)ZpwX3S-!`tyde8#F1)n{o^Og>COiARljZ_Us0RKLdw5rgA_SIyMv2| z#_A=Q_vo(<=cn-wTroEd?FVVq@RyKFy)Pc&su=ez=z;2Y>uCcLW7&g1ft5oY-uBGK z`I<5DX(P@xad_x04=oUa1Ww#&T=^O|ToOZN9UG3AA9VgspWdHa(?-{n%PXu&__Dlb zbKGtv}jBnpK&%a9LCvss}a{< z_kUys=DdH2vfz5UHtE`~n{uB8hPN%+5Dq-*c;<&+Eyn4PW)j$hNfX-088UHEEp*~S za1YUFA+$~t_x|PMx{W7QWuhk+MDv7*O~{dp+~r-37#=0$)w){_B6k3HO>^i|@M z$B<(x0=mZ$aQ)+U6>l7mHjA==&KDE!Yb1*}&mVTp7KZcbUOz?*)v`RgdmHuIO-Ny+ zo2IJVp{C73wxXYWB1Vx7qmDJI8@0w3_n$mCEA61+-&8R^adk1T)r;<1<%Tm$puei$ z@=_B#{O&lC?L-C|w&&}v=tg&`>|quCOK$4pjO$m1X^k`Vc(&DUHTYrOF~m0ec@Zq7 z`4_OfeJ-5JT_!48Xr7ZgbYNaChMMzRSBbMgT7Nihzaa1 zy1`M|%!3?Z{7@im6v~5!9}{XQTVSoeIy>ED@}vBV$yyrQIa7O344A`p7>p3t;f@~k z5tYBE(e%}U8|sDeAJM+bp+y1KTQ(G&mWQ)+!>V1zMsxuh&R#az4)n?y2F#-YBU`%8 z6y1QsXmDFSGQQ5hSLtsrZGB^IGlaFF5Zlma+C+rGt!=lKEe3GikUHIZ2Vq$v;Ur>fkUy+*q8MK?{U zO>^m<$Df4OqynJPu?tp)AT`t9?^rTKC{AOYkRnJ+mmeC{yRV$yHg-!`8ci92_Y0TF zC9RD)n0Q4z8;ge^J(F++B9XrBs;gmY3OtM`Az>2T*I^bAm@YHXj!iMQ4KN+LyxLJ? z4bD%KzA8eFy5E! zD`rpebs|aXr}@tp=~^LE{0)b8uO5c50>{=2dUewn%_W;nY>}?NESN!gZz0cmusO&Y zhes>C&}oT}08_3nZl!2$bDOG4q)bb#W$yb(=e#W?DtRL#vz-&X|2QLX<&FPf-a94R zQS8Ax4URL%!<||X{@EnZl027Y(Xbs|Lbt}n8hmk85J=CS1H@xu7wWq8ofp9WxWtyy zeSbYHgZ(evI#rqtCF}!x#|ZLIvP@eLPS4^l<|U~e-9D3~8=Z%^F0Rj9tB1VKkq<8BpZIVIS(}kB(UNEa}ai z%6+&)PxDBy#)<@Q#u2TgLBO!>%*sT_Wiz)ZJ7gjr=jjjg(wzT{uEb#q$b4 ziFFg*H93m(ajDqFRH+y&^?*I;TBo6t02j~|vX%ptJBLbRM1uNqeU!(nZ>L(AD>wxA z*bPpZNo!&oe~R0Ui$K2}ky86Q-K9wZb&fUg>YlbghLv%>9TyxG*KZ=j8EoCaWkM?xNEx!lUeOBPWzlHiD&19mPvdoG#QPP3YQ%Z zPFgkco-e9W#q#!&A@Y*92dh&U4fU-|%5$M7i*atvxlx%$AYp8^?6-%iT=%|xDCm$= zF;$7+%Izh(bh9hPw>;W5yGU_~2#^SBzWy<;w|wTL)#_5so*;j)6Eq^fA%#P=8gzLC z`Tb0bih#v8-JvkWP5-(+8krHLN@$ha8pYjj`ilIL18r9FYgQ_Ol$uG(T9i4TMd=VO zlD6YLf7wAqleCJEQ&@|3f$3V42Ye{B(UT%_gPlfnr)=ZM?W~;pDWbTt4;6OS4hae` zBV#T|m=D7vKJORX{IGSSLH10O!@oO>Vl3Jzc{(6#g-D~OuRC_6KdbE3P3N)(bfui| zL-S%OZ|dAnaxV1JO4^P`U%2*!18oF?HrhN`_cf;Y2x;x$f8CQt2OfB=iFGusllwIK zO;~|-Z4fod-4JB=u-nhZtvPYTUux9jt#c!50e=zW(`rQ_N7U6~6@NAIlySw7QQlOC z@!GgsPQW%-@X^|;_JW_v3ll*Hx)BQLQ;VrRV=;Eo0yhzGHRj&yjn~l>+b|}WFz+{4$T~_SpR9%#P%O)Ec!^aMk+1M0CA8dGu#}F)sAHpdi)5EI zMTfg3CI;6q8YU$3wLfECjV1G1$!jk0_epkR z?YCFWnqYMs%&^bF5)jEBH-dn>txkjE)z+SMrqyheJA4xWrFc!7Ms#FBk4%E6@*%Ms z>(sq5cYL(BShwrAfwvT9>i>gXaLL-g)Mh6`ES-2bt!?;CVZz3bo^7QtK9abRv=P_9 z#KNI>^d%P+@2eS5tmIW|Jh~i#+xEuTq(Tf{-4-V8Z@W7%z?rPv^4n^d|~N%g*WFK;MW?`dq6Snc&oun1|>v z7Rwoz;63NIO!jWeYVF@$n#*&V9siVGTw%nCT6bJ!wXRF-SHIttIRyaOs-~L4>A7E7f1^SzoMe8 zZR0VfH#t`Khe)y_8}*{koV=o2z00rB+XoLJ`MZ2hQZC~sHPPGccO1I526Hu5#RT2X z0J*y=T8?wE#daO&TTeiNw3+Sq3e(;~pPTxW(51{qv|Z{Zkbr^Fsp??=!v=yc^pPo~rP!U-mqn~7dG@PxWtP-tm1)hc$U^;dx zyE$~H-nZG*n5b@_3(kj5xOTxUwl4%rs=w(CsOeXX>v#7ndE@chcMMak^}K!}3;#xg z{TqZUxUc|oHOl&)sccPTaOi7O^X<+FkuPU9XIHPzSlSgyh7SN8JyR% zTb7^bBz+Oetu;;dT05hvI;GKSNyYqxjM%r1w7>2JETAVwJdHUoBg*ZkNQQe@$m-H| zu>Hl-^+oJu$-?O|@$)}p6j@ar9{c?WAyBCy+jh0Q!EI~VXBY{rCH3h79E zvvO{qV8tWQc5forrot_u%`AAFo%6bO;l?2q4fs&}GdjL*s<4ZXH8rEp26=tdRFLbT zd(4)<#6&6`2UDyxAm2WhPT+16kyDm8BMeOXS$Gy*rCWfsRR&Ysbac2(A&dlkXb`*2 zdbeR`%b4`p$vX)bH0Y=e5VuMs5^(a2+3{WuQUUQMoao3>N)|u;F zx4?=HEy%zz=FqQ#s&oKU_vQgFWIAx#b9&a4^U5#hV+De&!;xdd;x;|4{`TPHjRgh5 zf~$zgRb+_a{T<>JAX#^+UtB`lE-t8592R1jPaKRA+5v2D<$Y*yi!&^A2yZGl_)Dkn z*3tLD(9A*NWN)oJf&Ei!Y8w_Yt>?LgjOo{?S)n(cA5;y7DEe>k6ldp6A=|y+ZQsXO)1v9o?;I z&q9k<#sSPq?V}1naXEC+iuC!3Ep62g&0TRt$9)e|C8xXzjwz6_U*ZJW_FuPwxU~I@ zP&esY+* zqvH|q{?hb<%?_q5k4NIbrjBq?xo77a^jt=gs+9@c@WC@MM^(rRS8!_XDnb~ZK~E%j z{CFpJbX9ie>5#TmJ9cd&_`g(vd{Tj zaxsuOq1!z@p?e70lQHFWViskpcEq2pghm_44y5PgVyQI)@>$YD4FwGg*3`iDJ;mPA z5%iCti)9%A$T72&#UB5)<8nVTDQW{#&{mo5oH-uey7-G+!tGq6cH}Ye5Os4diD>|W z`h#keBW3PVJmd_W|3DA^fEeVcegzvXuccdlTzQ`ak%Oug$}p^=f2K>f(;i!Y668_xjROza2EasWtUaG=fz83A+u$@|#Ixq& z=f_{CVZsi;yA4tr(&FavxB*Qd-P%>%E;(m=#-$ehw+ z3c@3n1i6YbB<>tvnlsF*so6iM`wF*L*@K07`rpPNaI0Pg)Pov2b|X39cNRSn@C?wZ ztO9JsH|BX{V?e)kXUxRDpv=OK%p}3B2HQ|yh;kWjM>H3XH`*IftI7YCxC)hEj(hgP>)H`qF_^;@L(`wj;d3@aNwShsnwNMYpp{?P!bBxUhU)<4)z?TRHgo=ECNI3&BTptg##c%}>_MEZ9A4;n@0z0^&yn$VBU%YIq3NDsLGf}oB!yd$Le zpXImTOQSA7sPZaLXgt~sof#S*9pSqj| zl%LB|w~XY{Jax603{A@`4?jwQt51lA2_|mHi`X20Yp7ehzcURhFrWx;miwj3Mbo{n znTDL8Rp*K}Y658g@JBAq4+TbSEcLE5l>B;B%}VHZ#mx}4Pb&wAlh|PucuVcG4YdatDfJeLxJzXp!K&7 zX>SevtfSQrZN{C;kAibAXq;!xrrCLt=M2^Q)KLS<=4pdXbc{HIlt^vHXgVtQ1I#hcW%Qz+{nXgbBYVaX zg!yQV2}05|)I(!O*F)FzJj6H^zfg2e?C2c1Pv4^gL6&T4gZUNz+kdGngd!=-Kc{$!N83m_c^&0B~H<6ArF{Ua^!XDXD4dT?v7ve__`Dq3ANXcMt#w^QZPZG=ys z{F4lFmscf@9*9Cou!y6BPyGd_-Zo~-yH>8JntHtinsNDXy z>9CsCh?wh)0nB^M0R@<42c6_rR4l$8FYp`0Ip*ixBa*Ua+L0@g4omn98I<_TvqJB% z?Hok0I4HaRJT$>+!Z-=#gN-;1Vk{|AYH1VEx>D!aBZ85t8Ze5}SI98xpl5J72=7%V zJVwly%=HmVsRx1UoXctSL?vOV@bMucsa+0UC_Xs@#jtL8shxt!{xT(kp6v&(Er29& z=?)a%3~7kn+FA0BZf{sylhbL0UL23hU=*P3^Erq(;I@5I;NIFYOKkKP+Y^G08q8MnhT*GuX)Bp1I;QsjWA7`YqD=d^my%M^MO08iQIJL{X^;h^r8`tQm4=~4 zMHCQJlxFDe&H)4kae$#?fT5%r8b%r7y#^n5mwkM`y&v9l-u=SiD09zs{qy(Fd$!7g z{u2FH!P2fC-uoFCJ(Xa<~@)IvvTiW^Bk>uoILm1`-~v&t$TU;=*-6& zqkS+HNa@g%-nhQuBA$$R?Z#Hh?a^UEcu8mNaN)w4?E?*)f$Nx?kGciVc}CV6`9ev~ zlSOrQf*HKz3;JA9WjUCdc0<{yoyetEStcJBRq|>|;q@XQlxO(7_%bhrYOvlH_Z+uy z_&qvovT#-;Q54nO+p17&a^{zLIXX)mKFQ6=M1~O1YYzGkrI#5h}8UFNsgi>rtKugqTua{T(!zHJ> zHzzgERu&pECh~i!Eo`bLLW*xRVn9#TMU6Z6Rni%jM69-2$!YfQH@0e@#+Dv#l4se2 zKI2nrxQpTn-%M8PGa!?W_1|Y_r`Iio=oxYoybl+3;3!a|?Gr>~2i2+>otJ1WWpOwU zyQ9|Utq+lp)3+H44I7BTZugkVm6(Ron6Ph!UayDk-(pA%M=S`|DxXyBMLc4Jc5)`& z)0QFRoM}F19`gKp*BY@BkJ!VD6jyNGJC6m|+UwD@j2v3y>)$+-#qT+ZSO<+-)N%oh@k7knN1_vJ{R=G%I>i|V(^t?HI8Ay=rvMT4a9 z;i|5lw$0A6g0{c|3ZAb{-i&T-O_C~K)K%7tSDQ8%yx$p%i5q&#TiWP;Lg%vD#9=p1 zM!1;^%@de>;CXWzk!_5AnrVHOO8cz{zmiAxRFoYIn<~b0W2(Q1r{$5*>=Nj%SlDJrH9z75<9^M9kdcvmtJZt586 zPM=v-==bumd-)U67bL44(VWxL&T$r7e|T#|IY)f_cCkT0JiQ{_y9^G(u9_r;(vpbg zv_b5Bk}^;Aw)!Zsi(KAtI4Sxmzs-0sM}gj&6ug67qSNlf)3&1ca2EsTCi?ue;gwXo zy8iab_rav)_vu$<_1ufgr1PIk6j=0RX!xPB4sSB+&w_3nF54h92kHXVCnZ1Fd9G!u zsgoEH?3TV^jy2Q9{ItJ4n?V}PS6eprnS$FAU+!qy-yBF3F7`f{rX zvGc25F=aQsBWt%2JrTxT`_2f9M76>wt*%e2=A_*`0v`flAw)~^VO!ocrUhxN-C1Qu z)!SYSP%%pDTUq0{uXnLx!YS5I(pr5owgs?WT*|YXAC(-ejCsn{`jtOx*~052GRadr zYac)2y*(WpzJY2Vtx!s=T|gqg(HgA??thOx@KNPSG-0U}Rc$ulek~#I+R&Z;Ukw&w zq%hQ}jtu&SGPd@$n^P1NIsBKZ>(?NKAZF~SN#LR|t+RTlCubV#TUYvob>+%M_;i>k zg@;P89j`2e%Uh^TX0@rz-HiU3)@evp+@)CGAN_;B)}v!#&k zPVtiuRi{?CIl;|Y@)d~B|b)hGNl(v z5JlH=d5y8|zsO~&^ow!as0kO+Uwc{A?$gAn#Wtb*Xz9>#Nf&YR}KfoO`Mb8;NK zSQAf@#45JB0_@u`^_sMCCA3oY1a&aYxs+DyTdLOAefbrjgl_B0*jx7K6OHbSbq%+uEh4VCxtACE?>`rcX(!xDC=7WEe&HVx&!Q zlM(SWKJA@rFp?=f*zn zo+KY`4dMGeEQ<=w8|@rOwrF#ZdcJ-ck{)i7xaxQl!$OOv89Nely-P0si)TN>SYM$x zEeFTvcQj1x!05i=v{Ap#27@R2T_vn)Zz?h87L5bAt6^M=H_fqPOsYa~j{$$J%NZ+@^dT8#M)o z19TC%+u7+uiV~Ub-#oqVi&C_*(**WXh&+sQ%}r81>_M`q@g~cNYJ&w)d?>kZl_w?MR*J}6()dh2CQ9eINOffaHoUMq?Hd?K(a-U| zESHKz-KKjj)=xToyLCiEd~)fgmRc+5n?L7}*=O}&M|@I7=$V^qsBce>JN7#Fqke7O z^6$$CN44FjkAP8Iaz99`$?NI&uNbka;{x?oE2b&G3fm|DAETgeFv93Zh-PMynf-Y}*?_~=j+U-TQ3wdc?Wuf7p^taNhL0O-K5mLZw{`vJ2?wG*~ zR=$sUbYVUlZD%gFpj{12;A30TLhj2$Pr?Og7xmowwL@~cu9q3;9I}%Z{%V=Yiw(?Q;ha$s>3lot)^&-W{y*zk{Ieyk=({QL$p|x$_$}e` z)_-!_yy%&;U$ZQ6>5s+B-3NAzZo=-PLC#V0xlv{43y+1zEQ!<{l;*luYFA+=Sj?3}hrk#vnrt!I2YmW1lTnd-mZN8H}VbFSQXR*U%$~3z}bFti(PH_?#sd1P)EMij zGgRaqt#wdfGoToD0M;5Dzs% ziJFPkW?0gWC~w1MG3A;x ztY7)e2U5n9M3Z)2r`{I&=4E{T*xa*L-y;(1nXi%E+Qq{xq+*taEefj)f;b4Kll$^3 zRg;F#PLIR}y>p1n-XU-JL(*q+|Ll7wDOXW>&Uys#9GbTb!y zZ0vZnU_^xF^*(1$3Un#|;K>=E#sDGgr5hywg?VV z+$T|vH9DH3+X?%vaFBB%O8WV5EFwRPHWSXO>p2X05~J3q<=A~%+?0E5`JRQaejBUV z%~fq%^d+Q;?{L;oA(52!&XxEJWoyRXbS}&~Ji=c7degB)L!_e@RvUbH`Q}i}vZScXFukfsq zy$9pysC4CRv7^-@)~}^>Pm1TB8lwASC?mFDgD>Bp^=N3d47zev&D`JW%Zd)9et1R{ z_mB|T=|`Lic2`*?su?_#5cQsCdpG90t)Vrv5as*a(?)V8-a z$aR$A%G+1;9C(m5O!~A?c9JdKFX>SxsIAu1`)9_%q-w5F#HWp`6A_nVsNiOx>n!(s zx}c5{&$qf|nfzo{X{+flqlzdih=AHF@rx+FNjET-pQKG_%(ay#Xl57bq)VQa<7V&V ze@R=dG;Hu)LFL=qIr6|Oj97}t#D!)=PoZhh+RV30h})*bC-wHcEZA=PWO zA_!-I_-7BMgj4CQvW(=7gCk?o)LuNe)&JVe#*ScUGUd*^EMj1&11s^@Qc>_etrvNk!+*9Q}-((J3N zVyCCqt{vLFkgDeCPG@J`;!JOH0UKnb?1;bkns~GzXB7!FIQT~>e>3cUE@z+mAM*1* zNZinqug1bqiW9@%m|L!6UI``S*lAIz2F6MV%S|QSyj&{tX%SJ5ykRf#M!m!+b;Y&6 zQBy)fU1feUd;PaW??tv=&#-f#aU#ImcIoY^-j+ne_JCoB6+yV#;}LRV{`w*{w7EpJ z=DaW(%~PCU7G*tk?tB-|20I;}=y_l00MGg`qDo}?!cvvmD1M_%f^m}8TGO1fm1Is? z10UHt%E10w%?2J7l|TlVW^eb(9h^6qtVMwlTq`ipmxn`;Ua?{#9(jU&)2 z32)UgcI!Xmt@_fvsAHG7Pf3?=a1}IFZ)<-0w(qSZKr8Ezy_-yJ62+^f>ku_Y%k^7DtebX{M^LCqy`#ly+r8BNPfMFWAS&UwY-q<_rKnM^zWztrP|*q@UOJ| z)W5$kkos3k{w^bbQRZK1`Ts~-l9VV;UgrNSvA335U;2gZYOmnsuFDf8Pmaa6h>G~B|M&P z8Ey_2vsK$I0Yi%wT=hc_DmD~3h9eE_EqguJE+S_98=4AgCtKDYEpb(r!FNW*>-%Oa z4hw2W@X^uch5hY2hK|D)P5Y90j^T(~ID8rG0w{*8)Qy3?BNaL4 z(0*0{nBqaSpbVBl_KLaD_tDaW2MF#`SI^t`n}|o;u;Q3zu;=fSSY3RBE)VD#9<<*Y zG#!}?VtBBW2iGy;8ihR`7T%Pn zc8ae0xQ}!QT`NrI*z;U78*vOuc$?V+owM;5?wyVIJw5UeYaFsTY8%33KclUqI6(V- z=5trvR>jC_(fV`?ij6hFy5XRoCt`3qL5Ukay0Ts|?%OJ94`B5xHIH3GLZqrc+3O!ArKSanehljL!$qrtSH#tg0Ygs#U%KBQBa8X-YRY# ziBJ=_2x9Pgo!@}2Ne;W{$zA;M?aoRq5huNQ#wFM7^1)%yitZ4?Q(pxKqofv5y&}b( z%v6%?115GIt^$(>PpQQ1Srffu^TF1ap~P^pl#Znl`-X>P-b(NR%SqKOVyG3Xh#o)9 ze_t>fMJ)Gf_lw|=XM|v_u@Y!|{z!V@R=B-&lNUj=&17r^KV@HUvN^&fWTIO(L_{TF zo*D06gQ?nguRwI}H`~=>sw+=j}TBdcWe`<9I9@{dYEC)#2`>w_IwZB@J-|5(n@b288# z?lw@NpL1Cx$(l%WjQ~1#?xR)jul8NOC*d#d{QQPFBJi}t@(bR23ohtf8-rH;-bSuF z&Os>1Tg$4hPb_!gcQH4888vCV4v9UzMBI|uINv5^-&F*A}U)@}muviy= zTQqKY4$jhW>VEnYB(P2T_=QbE4ZH6#Wz#zYsq*1PuY!D1QX*7^`%jBZ1iYo*-`k1$t*XN5$9ynVC?HAdQkN+7?njz%IICSKi65&YK0yxW z5&svftr?e`>~X7=YCWBG<5$+&ac5I}e?^6tQu2hzQ-bI|xmR>E_h*)`nBfN#FzHdI zy)#y;v*efwW>3yO0K;R9ikI~Q^J$S*LiOAaM5*t$Iw17w~6^8LLoCgADm zic>;Eji)6Z9WJN1H@##XVGb9!^sikbH1w;sh-lpWI2~RJrm$}-E0=XYzvNw2V_LGZ zBf4ElPF-Ia=X=6i<(`(qM2yz!>Qs|F9Huthp``n{thj>YX7Sz1A=^0Kg3K1|9vB42 zcreU%-kghfpLQEOXT0M--GQd;KOFJS<+@cqhrI%|@(Vx3CG!j4|Cy|h&NB=wl-(JE zFvi&xR4+$MGTM}Oh}dPRBNJUWvUi@TahENypr3X>{ZQH|Jh)OnXCgXXriq!8G!!Vc zD19Y3Hd;?7ErX<&)0V7y3Qc&A>&eO4Y*s|hn_;*@F)oKe$USb3XY^cR*!(n}X$(Jwg z1GJzWYA*lq8yME4=NSc5Oa2Q2AbG_hrW)J}vy#S9&^-8EnrOA3&Qk}qb$4~UIvIPgJXWhVukH2ey>3iPh1g1TD#oM1IJG0E zCe62E*s@eqEKQ558(6uIf_nbqqFe7sFLCpCUj4n`v2#pO}8MQhF%8vomtYLGc(nb|@9+_nbDE zpX)GlmeGY!S(1QLMj{jWy zXsnpTM+_r0b$a@$`5P>J*J6=QXtQpUr$`b5k8-H~ZYkR;!&g;UzOq~K@pmQEI1d~^ zqeJ~p|Md{lp@qzNt+G6ZtkyJdT}xK3p6eKjvy${8eHAE6Q9~!V8>Z+fRf$98INMfZ z-2p21j&ZqB4`|!hlBIY=ld)2R=&WdVAK6XcHyxIXv$rK(Zt!n=tSBJ3(2EqaJ{SqA;@a@1z0ouVu5g@In)3G_Hr$|NXXzgDpo}7 zf{OWA2hGp(Fq>wXO4jZya$qnXCx#dx7>C^b-9h)BL$z7zDF5AID@>LoBemtdahLEh zd;G$(NtU>gvR>7P9%BdLU!5Ow6(Z1X-*HnRTr1O04+ON%vT_AOGhw5?g_6TvYLexA zht6`*^nRK26_p9N)pLJ@I zUz4XZbcJalNAZg-9VP7c}efS+*Ff zMVh(n5{t2eDXVAnkfxUv0|s^|^&rN`zEsgDlO2!R$&9$*mPsu&#FfO;#Y#K}aa`3O zf4~3ziRqN@&>R1X3nrc)xU1(2E82UMg65E7xXZP`pxY-(VjK+ZD{MSf4@07ISF2lB z7+yQl!gy#r(LPHK?%kiFhBTD8ELQ6_!;B&QG!bjp@dqH7e6JmMIL?n1}tFV8?|bQJfnYImP1>xFkEIL zrk6;j^sfNyuisDyQDk1;6sqkgcSxe5rK|IzB(eRsyO0tvN=-+eRSmAT6 zykO$f?YP+nvBcgK7U;B$ual2*Vf0;oEV$~kIczBr)4d1x9^OCnYvkGVYpEFXw!=fd z2)R@1=oo*LmE@)A2x5ogg_erZ4a9J6VV$MWxOZ+oSa#vQGHitcGm;k~0Uf|P?@_^V zTRY7pI!H{*7dvvXC<9xYr7^quD%)tc{Rj{+NO;6q(vA2h`Zmoh)o}X5UkMhExq}&& zfo%XiKDeXZgf5AhXJIOo{BaBHxz9vQ_K^Nfw^Ktn>xyjb)jM|4v|zBu>b7Fp+5bjc zQcQSDY@3dhI2`Si!0vU)@O4y8h5MPyGrFaLZf>!KObnEiv7{7P+t}iVb@24_rn>82 zyX&-^#eCX{diNBFbWV#--2gigJ@s^wCwkCCTl|cJeBuZ%P%-7$vl-t9iPrIxQ6)=n zcl5wa$+XBiK6QMG`Z`)7yh6Lf_qwpPDgAR1#bIOKLYU zYBe{M#b=T7&vtu|bKfPPReZ{qj8 zaQ|<<+Dk?E2C13Wmc3iwiQoBeSC|2YR(&**?9o61}>%o(!ch$88lmA|O-e-)q|gu`(Gg{3)%w+|fn z;=1{375Gpr^fo9Big)bnVGSWyN;##6Q%Z-dHK6t6TZtL5i>c z=h`3Y5ULr_=n@|gA9l=>G4YB(Yj#hhXJ&-1Tnm#s3INTX0j*?yHpP2f0-!I?77-z1 z$jRF`;xDQeJo%0A%GILuGA#>>k@x{_(+aJHg$0=kpXL^ooV+|&2}!Ux;2pn9#^zq) z;a=1OtM;zNr@F$0V}q`yoDX(gJ7W0czkl}%4rB0^+g};0-L7-%_`W%3w1r@xd_W|*lXjiB_1H=eB>XOLdGhx!9tbF+~ zWN~qEtFl2d^TNSqimk0}*?5V??og`)Sf98tDLr_xquAKkco*k#*wv7~RAAa*LPBCA)YfMG z_qp$^k}g%V=E#9D5Ac3U9Bypozkop1`_{sCo2L~O73tLZ zyuH0$cdF;I?MDP@44~!%%fsGahi+MJy6pH~%b?qkmE?4Fr*&Al+U_trFjiCK#-r^#=??BD$%s_djO5Ci`2)(MMx%uLoAci%c+3M=;w7Qf#nSPV5NW+!J z&74Hj)OY%AyFJxbqKR>KFxMt>k+mf7dSHG*evwiIMz1|`HxIYPFC2b(RQ~g4kaM$b z`3==7YWfa`)70VYSFbkJ(Qkwk!>&j^b3L9|AfqR!HB(R6rh1GEY20eFzavY-!?y8#IO(a({3$}@89L6`zk2HMqWY{RKy=MqACeW$f{|z9B~(%A z*d4vudb(!#E|D5UXa2stk7`V5=@yQ3_0pq+h69Ygu~&cp?vzdDVjdr|3-FG|wbRSA zyXDoT?a!z%_cIC4lBg#h&1pPNvgl@0KJ?c4a><<~i7`Ti+u9?^?^z6K; zrYgC~S?=MwkVK@rn4OtOd&X13p`#@@i{}%lT~fk;7n`r*E}7M zC()E;p^*cE0+nSV#^Eo+QVeifdn`#vvQUji?!4ZiDl5w>-^KJOqve>MPzFDo*wQ_qR8^yj<(qR+LbDYMmNj_ zR+w|F`*PMmV&XySRNh(_F>GhU8Rw;Bm&@S)6-yLZKBBbJ@Mt$iWF{>lFY$eCeEP;p zOp**{Qdn!^dta}AkH8?Pr1Qi3Vr{&K0tC@6sQU&!6V=E5`?}k5?Iq zj~RbKbZAzVp9&cBgJd6+4;neZqeLh6%>1M$Xxmd|Mvj3zC@=LRQkeUo-T;Fok+R1b zvVedXZE(*`{oH%qG`a^`QAR$+$SHBg!>^eF2-Jnmb$Ny!>#+QhYR%SsVRgpCdHL;T zMjxkpNS^%`p>5RRK_=;HgRZd7Cs%4ull4?a+gyR#9+Qd##HGSPKSUIy6TY+paUB%L zE>?_>rnDZerGXdkG@Oja-_vU*TEYjD^b&ljU-5u_P6dGU-Yx={d!-VB77<<-dCVV(}*2p&` z|6=F(E6MI(yQe#zRItLJ7Opq2k~~39NEUj}tPBd8YsXT7R_RLa9bmT}=d#yg2fJGt zoH{b>yEl;s{S;T^{gyA;jrVP1$uzU0z>$IS}TeCc1xeduwaqN-iPXILSHkqPF}*;eyGd`Y`d` zECN0S4?SEY>`@O}Cpuy00OsrKyZel0k0mp0{sV5MJCv4BOxJtuirx8~?S=?7kq?_D zMUq$6PVW4(f-H3r|b}fXlIY#)R;a2WeM7g1{+N8DJZ-=Eix~)KV_v0xq}R1 z^i{6ho2~R-eA)W$_3MdvOXRn*kZZ$ys^Pn%Z$SP2y&_BiVRS2vfn2Or*hn;g0pW9; zZDgU!Ny)`+s0#%ijUP(+dQQ~Mf){>13zsrLeLUP}aG3CFP4H+#7jBS_;|e6!*TYXW zl}DAaUcbIT&f$5vqqwl4SjV(>YYBXv{QOdsk3*LmPjTdvL!`m)2LS>PZwawlHJ(%x zU8_6%FfRNlq@ZEvf{5A70Sq$kg>%x@yT_Z;=#nKa2Fdfijcl0zJ>T>Y9ngq<5>lU$ zH9t=f7`psKR`T#ux%a+*VeU9esQ-YSlk?r!01!tMTO@y0g!JwS=-h2kHvKOYqp4^n zkC{+lZd2gPe?r@>`32f}l~1kQ%6b!Mqp{;kl)z_DLo zIlC+#_IBB;Qvx=5t7(e}+DI1a{(Nrt)TQlo9)H;&zi-VoQ~QMC8R>^!3)g*dqT?<( zDT&y07yj6X5D3Jr54>IY>`h}eiGwVw>MCCc$jdsomY?(_;r>o!T*639iY&`o*|-=M zVXEz>{l4J)cVxWcMyCHHe}Y%bLgzX)4X{JKV1-1fe`WA#jCY78fACV{z24O%-S+(*)>6;rH5^FVgYEwt19To8q2 z){}SUFk=HH7)1RxJ$&N}N6q%2ke!11o6Q63F!(HLX{)anIlouvCkr$yzB<9Gde_TT zWROBr%@vKM@st4Nq}^?5c@^L1DkEgvd&?pZs{)?oj_&I&QTZ zkzMbI^@^hTjNZ*P^+wgUp=vkn?c^4LO(KcrX3z^*2$HP1a|qrbKw8$}^5C@uFeg3G zWc_3DtW*`bCn;=f0PWntm9Hw~QgW*~zjlk1fOzj3uhGQ!J;}p##=ZGgl>aBcn)Tf! zF|ML0Ykw?G@}8V!2EHrBq63yMc{o!z(1W?K{XIf$AgWUwqTH%uP|DZfeGq6{8cdw< zX!NE*jeR!S2p7rV{p~)NX@&BjDdZKvq6v053NbsorLx(y2#cUXm(5ML)iDom)XmsN zm-VTu7DzPCrLUl-9$unhm~VmY%}^VZol9KhG_8zUV|=_-EHc;5N2+DKy1F_Dx|ULd zzt|?Y4A2_5bxEtv?C6?8I)<%W59+)C`^$6dgPJZtt!{+=?r^-2)+CjbUhW{(xNurZ ze}Lfxk@AHzWUm|b2-RwB{P#ynYj&Q;^$mBgl<~8R$l+b$76#EEeB<+_j$%VaCAZ<3 zaUEmh1Q%n`bDSlXAtn}>rG*0L6^^wswQq|g7Zwsc#@!-IM!?RoL2E)amoTr;7cU-B zTpyO6A!oh38Ux=ApF7NEJorW-?s*9R62sSec||PJk*k+RvRcF4f7H}HdKbQGv|@zZ z_v4jhT{}7Z&%P?)4*&z*2PCbU z*+dwOpvn>ji1=7oT5xBOOWl|463g=I?$FTC7#JAk>6gA08zfI>-BCt%=p?5v=Gf}9 z6_pa#7ZsV^W4gTTd-YfDD4MLVU+5&Nv{#pmj%6(U5*ZUOIpGciZ2a47rE8k@_dU(T zy#@tqg36O8Kta#!A_?ke#PmZBl?kA}Brp)R5snkPo+Xf^+#6(<(lXS{Mbdly@wr2B zi?8EdjTm(99i5}dKd>DvK)-&)`2AR&+Jvyz!{!8#$ALZq8KIMz-Dy0>rTh%#NgDf- zJnL#nnDFd;2)5hiPK_m$<50jseiA{O{0>NsnX{F#wd2RKUeFkxB##3=cj`xp)V}$z&JNKVsoO1DELA1bk0>>r)CF zd&P;~*Gx$eh_Qh$TaU7Wmq>X>_wGW$xnG92e- z$nh@zO~%*uGJ*>LiE`R8N;OmBDFRgoFQg7543RlTegfhamI->#mKLO$K5poSFeUz( zX84y=<9?fZc#M+2LD$KvD1X1+@-eI4ImqS1cf@(Bnj%PKX5p0FQ`L!6KmNE5B>3P` zo<}VCOX_%w31vuNID-Kg>}_6j^!!VQle$MdC4&G7En<8iay;#(TmV9*(B%c~TJ7|) zg9rM!uVIuP@__hF+Tg9YM)Hw<$FYp`)d`af21)TdugXQmxi-u5#*Dkg#RWdOaH{=> zi!TAq?!LQy;vY0?CuC_XcWi?*p7Bp*lO;`vEz7|d_!FE&nB{`(2yRj^?_K3gFYCv8xE1U^~ zzX1L?xAsLxf9?YKLA57Fjr5)sTK%@hvI#v_Enb#hy9NXG@S2td8mQH4D(((Ev7X!m zcPAqaWI2BK$!im*d6p`4ZbRqV(_+3P&`!HXSYRgkQCs*nxo977fi^}WgTI)8|Ls{h zxIk)#il&3?t1pYO@AyRxh1JDX8+^I!JT8blh_ur4hja;p!;IXY`pij$nt$XLQpj|35~t= zibhKCsfVd$g^hgT*Y$7S>}GDbDSvd-|G~Mn1|V7S=Jt@r$b%!A*wnVbC+R1np}9_0TI1- zt1kQ>X26la6l!H^lqOMcn?4XVC_2dKs9#$A*^Ek$0nbStp|f za`T#&o$epG3|Ig;2lc}TTJ`Bo)Sd-;?^4OG=Qz)CXKHfVQNfGTKj95*5e29LEPd|c z{tx4hXduP(5g6F+|FFJag)6v2u|GJ_=8Ix`567jPc$3qWKx-Gof3Wcix6I~|q1pqo zu%KpK@ah;02W0oJ zpl`jg=Y?bBl`@MbtS3Z1N2^-QT6hy8`)IQb3`=^ed+$b-Q+tNFnBB(DkdV0mV)8PY z)f{hWQjQFWc~47CpB|ZGke{2S37EVwXKC$j8{f+rS*|uP|Hu@4Gm{UvN3)0h0_}14 zkWIMK=pt{Eu9-Y`S}f|k1&4&A(`51+ZaX2Gam;T5;sO>LxR#D1B0kXtmcz64dE{|Y zT;Vb!{Bi(e5E$eafm-=%nQK~hrzUu0ohCq#Jk4OAY{Vw_pqo4H_8J=`zqkxXnEMJdZ z21qcPaOKghl1yET8BwjVzLcRnt$e*`mRdb^5C2Mo8iROE`Pw|7@rLn(N9IKi6m#yb z9rqecy!Q#w&^bt4Wo~twryI zc5qa;v4Fd8V{Ztsc94se{$3g%r&FmV8qRDyfbKzM%tq?W#c#bc|MYR0%a4B@J`sF$ zkYqfp2~L%ZJLXrROu+j_^oojX?>={SlilLrQbYHdas|gm_{v^xp@A!PQvt3D&ran3XAgazjmyI{s8nXs^OtUw zipd)Q3M4kVGzEnnd8jM+D&B_7UD9KZf=*ELLd{JfK{>fg35|S$9NHY`Grpe`TN+vF zAFH&ov@+Gbia<%G2c}zVyNI$7aJ_<#P8 zxD+qn*yqjyoU>GPkHtsaEA>(^H60tDTosVO%g6?W9=NoEHg zRm5N-a@?*Gn>0Rq?LMKeZIWu6`?ZGNv%mnB3M@18X)zomr3_v%ve0-pr+m)b;zWJv z=ux93(>%bfPaW@CpE-7_Gbq9gZ*L$t74Q19lHuCx!?Xs)yej%H^>NGbKh);!2KU^z z&Rvn2&QNvxy^|J5B-W_enCYc0hT;5?XV)~q!wD+MgyV3A#iS9)wS&Vze4rP5y8@m1 z5uDhJ;$rBsc9A4R5^x@&{fs=ovUl{}BFF1=z+^#MEdD7yy0%7(5>BYGk^J2-@44HN z<*N}PL`Ne0KVZ2#-i;>o*a8ccYo;CsY|E<~Yh|k@<0geG_2M{>NZl28I{vzpMex#@(LbhPa*>mkQbeH+_Z*=xAf7QDF2W#m3 z^A!sWeZG2*-HMK#OzP})x!ZH4dA@&yUVes`Y6ZMh3opBZWeI;%y0!2CNJTomXJlVr zE)RFlk#l*T-?cJ!RkQ9hr`>8vFq3Sh5F0a_NgJ(=VyE(E{8T+pKT4caOd2R|MjD=Y zm)%8z&J;KizWJaRCypcHWM%+tbHCUD=XF(d&8S>VKATTNUZA;`-g5<2 zZa9z*N&+I|6lHj$63q`TtB^X^a*TFT-^hUx?ak>jan!YTa?-Y(yi7BDjg8}`T_3$5 z0tG-$d4X%xbHf406?y;~w))YOe0SkBJh#a%AV_v>E^q|5`AxQg9~mU3AjNXPyl^??aB@ zKXE3AfF6ZW<{$6RzN6Yme|yIO%&f5G>)^^5xH!M)nzZEI>{u&1D_5E~MqB{wFcu^8 z6wnjpo>q4lyS>J?yRDszgq23o+r+&!`! z=N4Uk8+`uE7I>Jnnmh~=1bt2|{v_OGsQJ$}n zPkbm|f!Od0BrxiFyuW1)kI+Q(pUBR<-_H5d+Vpr5snzydM$TDBmS<*O<6vPy`6tYN zIiiO&D{!H%GIyT-O$`D4yUy_{wZVyy*S{}3Ea+N^%yKH!^)Rj6)bX0Z2NCS`4VttH z>l@rA=BOqKn4>fd&v{KCmoB!UeDLG>zddfVJ1t?mat=2NHFf`Jpg2EsgXiJEGh*0Z z7Q$iSJk4AN6@^%beQ)&q4bYJA4Zb>h$q3( zdCExKk=Zyy<_~DO#`w_tIKKR545IV_`+P00W^RDSY-Jx8Y-p68(6X*1LiXjE(-g!D zExK1*dEq0OF^0(U@Yoo%A0upil)zggZqyFeBA~nrA z>nsb5}^w4c`yh7t)jizPR!i}F~J9Ci$5~<770Qy3RyhQPUQ3f0dSyDnL zP*%y#ZZTT~o%^GG<1&ahZ6e0L0x+^&fc}G?=^i zYDn;>2ADMwvoNojJJpk4lHh*USVb&H=ZPW-q4VJPS`iq~hXH8rh_V=e4V3g-TkQDsjKBFw>?07T+mg&ShO$R zk81th6D1^sKrLN@^bIg%>)6eD`1ohudOIZmPv*ps<_F0Y_=Xgtqy58k|opj`z=kr)k=?Qjp6D$Xz?$=QM z>sA@UbHp1~y-9LMP&YuT5nx>+)y`8tgCXeJxksc7(5(2C(_%!A8`drhcn3CqB#Ul1 z1hR`VF?pzM?WC>sML8cJw7id;NR?3T2C#uY+AK0@cJJbx(nviR1A}g&2hH(zTMmMJpo(s6>q(T=!NI` zKGcufHK-9opi1Xv*aWYr0wvrReRsed(5QuXX|~L&{y(WjAPEov^BIy5&OJh)I*>Oi8p+z6ZqnfqR46_%re$;)?{0nw&4iL!iKW_E&t1Ax zXc^GG$)cP$Vv<`zXQ$Tj!jHbFM9M2$KR7HDU`V408%8tGPl8mDX2yNLTQ66VH&*th z(dAmo<)`3r&nf^yunuR;{}UuXxnouWGM6<@_mXO$yXEJfJl!2yUz=9&PZv zF?P?AhST=_)I-)=!?&4U-$D3+#V;E^W4=oydR z2niPAVF@o@jix`O+x?VEsU%_+m)Ubrt0cJ?9Mgm$p!E+?Pt7szhhUkF#Mq;jTNgcf&W0hPFR)+7&u2s~bYnY?Ugr zZVMR$pd1{U2(o6`eJgjS@zCYWbG&uCo#l#m6F(A`WsT;YCGqVuTTu5|KQiLffbXJ1 zC8g^9MWMg}#O8AfNb0z%9R!!&@gA9> zGdlSD2*kVone8XTXLyuDU#k1ggars*b?dY*cXboSn#b5mEmYGIh5?Kem%CJ!ZJSxc zyjxy$!}ODIHC}GRXfb&z060o`l|j4)DhG3G3k@?o-+;7+p?_j81EE?FRPxA}f@Jr} zk>vD;e@cNFvGUOUaW1Z#>e_|~BA03G>Vwq|`4?yJIXgaY{Ec9fFR3I?Lwg%EjBEzH zC;6h(u>pE4v%6O6vqYKbF`NJid^Q;{d8o2upPlUjVpWSwHg-035VHio4Abzo4aP+t zk@gH3Gq6#-bjme8olm>&3Hd>^bQj?iFUykym;)?5~G&bjQ&V>Hw@;%dwVf_4${SF;i!U! zSN}7-1k~;TEa>IZIq&TGG?d@hg+$pRIQqMLGWz;r+EE*E{1@cR#tCPw8Z$f;`?~ts zKDExjeoFD~ee{g57w2R;ByujNc}G`Kr)16>=l#Ia3xz-#(xuJPYpa&;yr|NX{n+tH zm33`2>S`h$N+xpv&tCIT9`VyyLKQANS{=7pP1LW>?z8CyrC!lGe$!eMq+N5aIK6{H z4Sk5}?{ibzPtmN+XDLN-C{@NFya7-Hmi39h*)O1SO?5-5@R9B1moNE|F#v zQo<(g+W3C&_uYH{JLh@Mb69)*<{Y!em@8%0$$1!=|LQ_>IgEZAO)fUp$c=#9sprVD zG(e_IQhX&R@=|G857?LZTTs5-oAUZe@gHXur^`dv)0-21nQK^SeRRQrEWJl}?R#3< z?cIc_Qw7YF!>kY#k%tfoY}eM&KNMCetTkVwF=H`7pX3tPdS<3*8<33nrr4HJoxx*? z3!rb!QVT-=cxiy%j+;HO{8U}&Y4vG%iISC!m$PG2PC0qeDQQq)cLy!;>R^0eJV1uYzh%j!Ek15c2L)ADCy8+iB8p-_IZy#6ar^yIYy zPSN-tPUIQEK@IRy_TnOSSF5P^+MJ`oyRSjx;DslxLzu@tb1WY>KuQo$yTkS1U%&QM z2E0^L;fJUrW`fM25mBCVyzo z^Q$eq6k_QQ_MiifM-`n5KI-a2ZZS000ai_mtFBz39Al8Xk$HkDxn@3S!}_)<00>Xu zZid+gHDP-T!yJzdekZL9AYDNenYi&J9pAPxktGUdLtyc+Jr`d2oL=@bJGoM7uXR{v zY0f2Y``2D@BFEWv-GbE58J%Su?eATk(eL3zGagXctvotg806bMlc`BAOj&F6cP*dw z*mbcCB^?~fQ){M7-auY98S=7MvPPNyd)Wl21v-&_TlfznKsQA#nT&#CiO{lapPr-nG#^F{U!Lrah&%@`{9f2ZQix}crKzGMn& z_tX7om~1_2m;f+6bBJ?HA?1yCjfO*=`*52tyE=6&;|4wVa}=Hd$e57U^2`X>o%@2? zMJ1>jG~98NG?s0Q2LTifRC@s5Z!mo7H&$owpF|)3AmfzKOAz-G*#ZsC0DpcO?+Nw) zMnwh_OOF}Pv_-aw{US{{#a7cAwK_Vhf?unrS~_RiQSx8;?KYCA|CclJJR+VwTeN}koUy}3IPI1UG;=xfC*E(( zmp5+JB&5f$7$js8FZTbly#H7CC0cbkCOP%Q!y;M9SVRG_aUqsYZpI^k7nJqzzZ?K_ zQg+)K>>w%TUb`NNo4q9)@%aIbCkEykUQ&P{%m65R`q>N3EM_TdHe7aG6%~_dJFIzM zIoB9rYDzC-Ty0R5o!+?it>a(~#7hGhLD2|DwPX06*(Lma3&>>D)`Q~KCjl*cO=7K2KY0m^t`3#iD&;NTG zCS9d=2Riks&|a|gSlLX5Yjs>g^3#EuJokNFB1BnRtI-FJavx8>MEY4shMzFb`*qb~ zKhu;(3RFq#+S;$hGZ#Xe=+L>cj+$n}9yZeA7cAN%pCDR_(H1i6ANgxV0nX;m42)fS zA5MnDOUjE>WhCN?#zoVgWg@eZY~ z_#Z+CymzB6;!QOTK4;JTD29?7UH`F#e+^FhLSi+U)XJ4 z+NREpQ;$@jX^}kU;v2vcW~ny$gxF)2s#~JmGJ>AMa3HiWcPD!dHTz$I@^Fy)?F}5= zk`3r^krlQH!4@j=B0CNM2QK3@S_v@#@ov7NQJ8nQ+eUD}VG@jwPzPDq>*W>1Zlw1V zI&_DBs-&@Ge1c17prTyP)bS${;4n8u7up`LwoLqn#79rYb5Zwk*wPd?FXAUIp1I>w2(Uq&n( zZBb{0V3W)sJX4v1Z!Gt*iu&c6MeS*-ZG0FsK!kdGkQC(pt{HY3XdAmjfJV zI&l`bl@xHRx2OegXCO|FDw3Yb^hB%2RTI+C_8TI@%Dx32&(BSueol-_+`XROaNX~c z*04SM@eP1x5$6pWT&x{Bjp%K-f zXE5!lug^us#Hg5vhn=3`kifV1Rc)100q6(6GO5*B2L5FJC((-eW zigy$SR&>;Ot_Byos}<8~K3a32hZS*IjY_>`jD>@JL?JM`}R| zv@DQH=OWDEx@2WsBfO9}J29E>tu>t@ z;1uHuL4_Sgqi!H45S_?&t*@Cx)GczIKH}E+>$P1Xm2uLU=a(*W8ms$D)I(?(mkUL` z64^p-9V%Dq%(>3fSsRj5HTur3+!-f+GON9k^E(}P;}monDems^OrV1Ufo=aKHpv(C zg#tlVZ!Fs+S72b8K51E@g1GX7uWcD3KgMF}|33h#OPCa8jqSCFS-M|H)JejcjUE5! z_FR4EgP@WaZyeH~t}vParl$l1P>=p`28?>b$e}ZaLAw<*y<>(AMOoHR_2LXu#vm%J zJ-$RR7Cc`}qa6RakPk8iKYRC28>BQrxWcb^2FCD(;UWPL3Xdl&-bv5OL;`ZlMFPMY zNR5lV{0cC6B?bsu*!6hq6>wevr{UN_XvA(lMEDalNevK%L&16U!8SjScIIZ2PgNkd zY<@e^(q(Y6y9lR}SfOX=VL&Bf0ReEzSidl4p)~|mnPaNmLz+H%sW8}3np2r!DG({y z-4G*E>b+{gPdb18uSmBI$_|!p9*HORps38AeLe$!245va3Bbe!OJ4Lf^mHr!(NdJ; zJL$(>K_<0+oLWvKXs|MIzP#(oBN7&cDg5CGJ=Npze5UOr$r3g{$q@_-ZGN^mx|pMqNgxK#SeD0BA5& zYb@aloI4XYSIWjW-}y7tNjUWK<{Yo;1pI!kayebs(RlAZ-MDfc7F8(pyMDNFZjl`D zimUxkHPq)17Sm9A@#3P9-(`FzjT?9dkx3zG^F8fVht+{iX23Vq-tE6;u_H5987ra= z{eak;{$~mG4zgJ2WFx>1qF0y?IYUGwoxkcB+^()ztSOb;cPv&()-3x5=rz9!AxSkC z8+($*6Aw9a)DS;qe1kiru>WEUYjbd@NVo-qHgzrHc^nckYT4>uoheR@Hr9wF&H`(c z@{f4SfzE8S9G@v7;nyniKKeb4+aXWe+f-r|=(6{XReOULc#43rE=PK{(z&Lj zwDfoe<}Z^Ru(~rD-Dd+e%e+FkHvQ_)WPR#fptW0XQ0MMG+hP;HPWo`OV5dS;LL%6; z3=h>TUNuu<&!tL(FQH7fJzge5uCT>rZ5m4phL&#$8}W8tDBfl zdKAY-a}1bRm|yEH+mLdyRyd3%eV8LQ3nP<0+Rz!$Sj|wX;$Q0fo|H>KRvehJwE+4} z`$<|Qr4UU`U={C%k!!)9vt`abhMDr(Hhu~GL4SU#2~cJY^}X)Bq3 zhbZ%JsQ<0P6EY0 zV$C0_)$9e-S*Vl-^KH8EyA@?Ue>l6hhXErbQ|bV;cD7pnDPAQX;J8{fu>hC|Cz3zl z179rzG0OGiQ4ci7B8Xk*_g;$70hQyvif8^QuHE|NWshsdz`+1vJu{3DlM@VgR_~Jf zDQe#w%ljRml}0$s3vBpFQ}R2GGqm4iJmWbfAuca2pGYNAp8R7m;D(eDPBpAeE^T|Z z-?X+Jl@FY3Df_u}xGTR{)8OOf-7aRKakn%}?Klcujh(6qxM@~XQ}-KgYB?-OJDDa@ ze(`bxY!ZA*cW}ASR#9a6svKWzm*@Ta_p?^ZHvF5f#I7TukZbR&i}keH+J@;d83^+h zkb_m+)JV+W4Bxyz^0%@ogaTC++s6fa!{%2<>%$YT#C(dZ?yJIns4rvj2p4wHjUjha z(P^Oj4#gRyQkV0H5eU{W!kPAxg|&p^JtFcdoJyEg&XbQY4x$EG?n z`3$i&DaQO*#`HGiq7TA?|HQc@>emPt_Q-659vti7yn2FJH3&_fmb%SXzD2FVaPpNK z@cJBnDcrTZBUm2(mBbhmy}l#!`jXV}>d5YR`Ra_-YM82UIu@ea)O>QQQLM6*Sak1o z*sIUjKx!20?pxPgUmnwD=w>AQ?Fjh*)mx_Tucu5*@P&#o(do8KzoksW6Lmk6lg?*B z8>y=?>alg6>$Ff`HJ~S||Ks8+a#khTXBH#YIfLOg?kB=B(Q>TP7b}MApR27@%Qd-O5A@3vvwDCGh*(C@c&VKFaUm}9 ztE1>sI*uC5VyY|^{lBF+`>c}}?zq|As2o{nJGXDzCBIPEhx-Xsv}|vq@CZj>3&YLY z9fqFq8GCdz)|I{H(vePnu8W8?aygVnD%w8BX7n0gU1(jM%vha7^wKyd!Hok}xlX>( zUT5tGoC!^q!D5AaHMIACYIC*d@DQ!KpJjOBz7U%?2K530b4(0rr=jSg9^D?wc9H_S zycs5u&7YQ^8lb=KdvnI}8JmQBX$6~9HTv7*!*}3P294UB+S`c2Ia6yv zx+OBJy=;q3nj#OEKB@uv4N(f-9^MZ-Fb-M95WA>?-fXl#q$aZCkTWh3aIG%4*h#B> z(|xb!+wjlw_9i)?w%Ykn=d}GxfpbDN6Oqn&H7gzBW0Dkcvz^ReY<)=Y_lGA!fJbJ( zdE?8=UeWX3LO1vczs>{EtcLc)+b-iDcm?AhB)+U z-)3*<2+#;b(>qAG*- z&3|$jY1!y{!v^bxQ*71Zg?9(<#1C)`fRIf?>+4W)Wt+*%?844=#8r0_06^>OOYbH` z%-qSm7<4N3%il;P(L*V}nbh;GD~kY2S~Co&Yjus=v9p~y#{2q(3%h;ElUH_JgT@A) zt+0+GJ5s}}v;g-g8vkQetM%WP-mVi%OBH5iJ401T^K%G?>k}tf%nQI)_A>H6w}8x5 z>X+1=QgO_vl@w-r%OC|;T>c()T6^tX0+y~nNHW}q8=-5br)j6ruqClS?=1B@H?ewY z3nxmop0HRAFWM!{dH;|~I( z9YW-aNBRA;f}PMS`Z`|MChw?M=)qb*2`aZB(h8_!g@!_DW!UiMyx-({FZiWhEeixD zQdnvZ9Yh>T9Bd{bl%9_Z)a4bTCwE&!CSDW$S}kiW`^8-2Jx>`_Nmv;`QoCn6_Fa+K zc4uGBM+u~LgPOh?Eni#pZLXG}#FVrGTg!4gNA0veJ&m5%qf~2L)R*_K%keZpy=enq zNF^qAAm;3MjmgaT^xVc_{X1r8skmNhz>SgL$}x}TSBJE^-Ze?qis>&Eh8ONu=Vwz~ z=d*O93S~_2o0_CrvA0Vi!_h#_{Aa9x=tte@FDl{gF?Jh%&4mRXVR(n@0%Yf8tpZxX z1@$Ny7Cgdjz5cGZu}GSJqggJnHw~1xKp$@7*ygB%+ ze%U8FY;-RmS5x=80w!AcWf&j9VZ}8fDf$6Aj>t1{~Yh!=8eoPJ)F!_Rl%F6FmVts*vs3Wlnu=zQP3G ziS4$v_U`UOj*PUj=)cEuP6a?5;$FW~SI_ovX0dQEj~wvyMOk_2sP67&-d&(JoYgVC z3vC5OVF<~Pr~lUBsq21m@fYcuQKGs_A_eAh1OnU0Zf-k0lNdDQX-Es_gY65;fjTw? zPt+#&`-g7Lj-{ZDv$h*aWJX@5o7nE=-*_IzGdzkR_V=Zs-x_gpcmz z_i>e$%Cudpzl-=wI)oID*{DYN8K}g}Cex3-rSds}K|>hW=w z!){4UFlg%rX{X0r_Z3(TG5_lGHL_dxh<0W&NKCNXuHO*Mr2_1wG+cbPadECThPxm~t*THR7L?@H!@a zn#*3K&Ga(B!a5Y(p5}Z1?(-KG+5k84cQY+^$u*&V|D&r_N~4!Jb@77ewjc)ZUn$f( z)>*2Ge(@^()mYJSl(clI)ehM_yBOkVp9B?+0#Ie10ENFNCy#27RlcUp)o)nldT5fe z$a(1C4z2Hg^ToGbvGw`VsS*k&7}_U$g5;VH0CVRg?`2dQ8D z56MH^8q;gvl^p+W)G3qG**R#Q9eb=TlV_@D-yr_194#Xk7dV`7%xV~we|hH*gl^4co}(nZMb z;=mD)6}(u-esABfScHv1F4E>lj@oUXZke#Mdl2ZM7j=F7r>NUc*jK*l|LrfW*rHr& z@$GPNf)uQ&D~shGwODB-Y>@@=yY?{7_iw3^qRDSzX`66 zS;U@4Y#WyL!L*0l#W84oE59^e6atn9D(r*S-4oZgV+#(Wmgy&1ZimvO+tazRE-{(! zK%cl-U;AozW4J9h!tazvDTp4jq}J?nIi~fO+-gXrzTIPgu^dL@T}$j$u-Vp(!3n$R zj#u7lqICS&>@Y0%<{Bf&>hpc>1*9}f;w6}V;3fx4GgQ&qel7Id({Le+B<=+=wkK+% z3kwy`?F8&1>+<~cg6nSV!6;?(^E}C3V{>tvJt;%v9Cd#BvF_w`FvK#N$m?D==18~- z>A&W2f4MGn%#^{sqaT&na(zIgbX2=4OZ;qs+RF#_hgXbkR~~*i;;g6jSu+1r{64{% zq{%~6u{sGcNYby-U-^RTn=oC7dVxLy(fv|CKwS!>JRxQZR1s+}klJCURI*?gs;saf8< z-v|?A?$W&nO{vnTEEUyTE zm~lFfu3}080-t{_-eNEQ8RPtT1eo$8$^*!Fk#d)f6<_28Y{8FDAqTFF_EPklqmkg`LikneGyT$zBZ=6|zZJgsGN>f#9u;wNGp zY@48Nh`0P+d^ImJFa``h$Fm;W(hAgKl#yKM!5SS#)qZ0`?H;U2bFiW0*0VLwcE3iJ zUEY1CCmeN2p9A#W-Q9=v$I7^?i;85mbrC+Z#3KvF4h{|%U;|BV{gQX@k%yMn*EYX% z{<0m;vlf1^-2*}GxwQyhK42}{ub}*b2;2BO0tjpI2Fh`9apu|q##KxHg79&e&?5&w zzmOP1*Wg_V`Bzl_pLeW(?DcAjHOt(5J>_u^$j{H$?+s9-)e-Ry`#lfG8g~uYh(c64KXD*RKda)#|ab(n(Ai zLk{?Df~h?miyTi|Xnw@Ugtx9(IH}JCs$UpXAX3k0HbnbG;s^1cZVPRn{Ib)3II>t; zrW@J+Wco8A;m%6z31Mr zKkUUb>z6K44uC9kKWzpOSdSN(kMMfn6(gPONjW6|J`)vx02a@ z4dXK%dmsXXv+TDR{AxMHf4bX&?7&+4lR%E}KCa$HH6sONx}*X_A1lLRFyw<}Plo9u%S8!TyF?t}`d#rs6f+DG?~p9Q&dD*#!<&Paxd`t)$|zSAsbTAxa`-Ul5c_k-Q9l2q0%3ExvG3qWL=x#;Lol@$ zTj)ucZ>Mu6Wj3qOS?o7uDr9RS9v4K^uA9Z3q+Ab$p54EY7n zDxa=XM)7VSN0^JS`ks>E>&bPO6TKTYqefXkC9*BI8$*&Qs5Y+=4QGQt{4C#-HUhS9 z19(o}I9*I1ghu;-r;U<53&^G#79k+;5y)`fGJ3eRP_Rip7&taq>!*Hyo;Qt8YgrJYwNAIBHQ!RMg48j^ATu2S>01&1^rNV}cdR2y6# z15W33kQIp7&y%%_Jv}S5Xw}NP1~0S9e(vq{vcX!izf&v9Lq!xYXxm1QMrNxyW^4V^ zK!j=3nil9yt)loxc7;9t4@X)AOhA58^5`MifT4*U8mv0z`ypgRJl7(45vg#S05bzm zgYZ-Epod7;c3Sjsw0?(%^?s3)hO}Z0g^tAh&5qp-eIvCmqqBn2Jw3K3FQxmH2jU>#>mZqfRf~2x zFv=1?4ckm>n=eQq5PAf#ySLG3<;X61L0;h5b;1hOi(9PkQ`}U&N>h2PG5!g^e(B1r2^^K#Tg65DosMN{s z4#Gk{>?mk1OoMUrjpwGn{09ZRS!{Sg=^pvY9@E*LJe-{8SKV9PY**|#V=@66U$XZ3Y)f!OPo?Jso=DECu|8%}b761d7D4Gh>b913odom-#Z$Xz4y0X^1?m>c2gQSVt)o-NJ9cajRFssJKC@KTzEWW=KP zul8YwB(9X)-TUkmI~=dvp^(X>X0k}nl^l@U$isfT{US2b?nR~1_w=>>;Hz6#1B8~~t$?Bt8qh>Bu?OQvGuKsi@Xx?E2tH#Y zkNA82kM?wzy>k6C7>j&|?|ld;b8GwMi@KuO+{e37P`_i% z6TW(oYGfBu3li4T9P;^wk)8fH6@?-h6SKax3glQ=hN6mS|Fa9A|3~iIk$0*B1Sqtq z%m=<(A5PT1IdVK6p1Qt^BLTF0PC+m8J!nIQ^pRHd4mlbBduP=5wjp}A%kM_1W*0_7 zOY}B%xv%APWBTdbSZ0&@#_S#kc{rO6uP&IF6UgPC~Y49MfPGg0Pbgx4HjM;C~I+ z%g31<+P4%SG8wvbG-dF{6>RslvZ6EN{65Aut?J_N-lN9723&Y1CEP!10~413eE2qO z5{Dvttr=q@&tZ|j-BAfGZ-)bS{vvu*9^Hc+o$Et)l=;B!!?NO%u_g|SR2l(8WNGon z(*LKnwP$R61qU~vi;?`gWnR%BKhD#bV^rkF5eTXf(JSPy_w=5n<^;Ud>;j?PTA}T0 zc7hsVk%mbkC#Rzn!|_cph{SIEeC8$9f%$n~7c3rhaBuy}|NIH7MS&IvOchWQ1YXBC z1{B(aR^0l2xTd{LOx;G-*^;6w9`I%pYm_6LkI3HfiZGTTLpT7kb3b-)avlI!fCeS1>9|L4tBZ_EBH(sK(rB#MFwbehj*9N0# z%bxFcZ&s+Rinq4F3YSfRbG}$BzB%0HDrlQB=QXPD* zK5)F>Tqxhw5KsPXW6fiUG&KtwPB~;;Q%y%uxMh8_UiaGaiF?qcgK#fTp@J zHnU6@x{i1^1Cs`f|Px1(dS{6J5X3>8kD-}qb+I6{zO@NiWL9JHCWwsojGnXb`XO!o13O|ct zG5Hg8@_0eDdEtjZnQ=Z!xs(5HK~E*U;tGev8g9I#xf;XF1cT;>+&3nFU`V-b)nj~p zR3KQc!Y}kl4kwvd&@W}z_;on?;}5rK_WxaXvcgFKTzc@2D5NDEM+lyh0a(=&fMRQb z!QmP{#ZFAK*CHN3%RR^Q^H-;z*wK}UK_@-rW|)-Lw{xp6xa&1Bm7Z{MhD!vQCD%)^ z;~JY(al7*E5jS#!?VoDa*5=07gd<~h7avVQnH`}K=E3uDFOwLFd2Dqs*udMc5|35qAub3SP~1BHrJKe1vhPiX^3lLVvb9PTV$AV!kdw|G2Cy|#Rnmux!JAwCAr7dCWf>Uy@mST z#l9c=)mW2}U($K$2|JDeE$^2dm-SAo7ita_6`t4!S4-Il8@;3Bk9fky-s>M;LSog5 zMabIbtkD2aaz6!OhG?}<^R~Gz^LUv$D@?7E{^-8w*hF@h65^<2)S;weX>D*fIxR6L zG{Iam(O&KMF(zOa=VTd8I$l`O=&n0577XW}8|~A8nyeDOWFyl{?Um{$+=@eLVa3eG zBjh*T>N#}c1uXx^gaWIyq*Z3Eqon?qR|2QT zy?UB=%axPG3UvWiRhW$@IzPy5{KiVkJk~>F@>&eygt|ECZWP7&kqk3#S@>Petx&p*Tf}vAnJf4)mG->i%E#%#(PwG=tj>?i!D!U7>Bo>peZ7O@ zLLNEQoU^HZE|Y4Wtm=a&s`NeHP&wS+b2mZH8=Nr%?_d&OEVsAE3Ll9=Dn7Ud)(l_V zgw=&vD6V;L4?bN&`)zqbZV@;$-uIY+@Ww;dB?LDr93z~&wI#56R*kBpLh6lW_T-)1 zwf%%5^nsyQ-=!;BN5b6BDBinJ6)ou#J-hi*7F!s7hoMr*3o5qsUOHBvL3qg{hv^CR z?3!0eXyJv6l)LScp>}qD`?7EQ13u_8RPbMa6vpIUNf9>d;jlm_Ly@6lB9PVU&XMgK zUJGNIr_AU~713AsAlcA+1M^s!g*!LAYH1YC?^h468(U`XM$>#;#}d{k=E-`0NOLmb zZ`s~&r9K$KYl-fHW!*1addNMo@2jQVPXp1p4~J=2S(h2gz6V&6oy3p%X(>s`ZQnIK zl?%VK_JxL$yn82E@S~%G*tp?-{3jI4NGWv9JaDgRnabMC%1s zheuKHO7qVnKXpRl5fuCc;gPn+wHp&e&=U1#n~BgQzdDMQl-EnJ5!24zr?*#Wi@Zu( zhW@KJ$gAw=z+_}tDSi<5y!38Q*j*GtCl(}eG~#_kOx{J%@THkTdu>c>QX6^IFIgM?VO`(coz-7TcClj^ip8l@ol0WCSh zK6jB4tV>h0q$;kk4B^2d`FDe|&(J>~ysk!BUH@kjgP4qIZ&cZTDA_d;E5)j-doMQn z-+x}^YahVZe>N^IqWr96=sAPv77yahOCOrzzU8-Pj17Sn*GkXWn-+Mo*7j>j*H&L2 zLGjLoo}^!WWv9t<8;6 zy?d;0DKR2`-yX}kwu*ntIOeBk9iy)lV;1re7!dS>@COI}Hv)jg&7VizXw)h`SsiXu|^5EWCqI~F6KZ>NNT#`O$UUq;-Y z>mQO)HnYAz_)5efxBrtfsPS(|ssA?)p*jU~VD88pbOCl@2`ltMK#4mNNdBmMHTF&s#UV z!J!Je&u>)cKD@-uXfF<{p?n%pC?%*X`bPEH@BIYKd)S(A8mgBY z6Z-V|p?DFe8rGp!m9@mxGywR8$WiEpF=%jqohVSt*5_wuSKY%CSd(+oVeHjl63(YE zaJO|T;+(=A{ww;35STgA-F^gtU-9Fq(!BgF=El*WBp&2(uftmFzp41edE*qTb|z$0 z(Fqm%Yp&Vo8%a9Ha4QqCD!V2vSQ#a3-`7eb%nT(GY9_*-)6a`jpdk6cHJDY>h8nTW z#`i!>_YaW4Y0!rx=quF|nH#vR*t0xif4NeOPJ#Q&a}{y_-du*V;!4nHek1&q>xSTIKF7mm;WojyF!R!2%|()AbY zu{4uVNHH7JOk?G4S8J6GVX#r)Zt&&1gpBI;xPv?csM=ae$+~nktg}GMx4H(x?e;aG zHTK|Pxb#AIybhnh6>gNze$21QQfj?!sMi@oC8^-o_D_WAs^I2a6!MP+Y0lY35DuPWe*$EVE4?T+%OGLdTa`SAHnLWyqNvF>Vj|=@D==bFafikT@VsLY2-$ zqf5PH#ymMV?9=_l*4W5MT``No0}2VH7&)5v5|C^e%yTXKXADU13k@Uwr2G?G&Aa3c zyi;O_zgTdKKZ0nFb77&lM&=PLp2lkgh5mWon~^NKCHM>pFU@So#VAENfnx42(l0okU5THe7sK8j}e}t-l!Q|3raR!@uS8qBa303 z5VXE3%dD4qZEqh&7U8!kKhDI3-hyz)G(a+&-NTj1f1&#UX3@D&v82pXb~P**oKa{k z3jpuqWR_y#eD4t@CSfMg6@tuKu1}dZ^t|ViEUqb+3Cq#~iQ&>y;N5ncA-I)})VUCN z#%#Z%u4P30;oW2$` z*ROfhJySes-naAwQt<$q(96^OV!+D4oa7Na(V{2SS+z2m<{wlL1E_#yI`stG-32?P zUl`O;!o?c1J0H2O7N9=&;d?;{t#97z=E&i;RftkCG{$RF&g+tOr9}qpLONu?Cb!uG zksFu`mQ=?(5b5S0>7a^qd`sI+=+4&mcF}2;Vm?UdgE<+dcY=<|EJk{er`lxyPfvV^ zwm8-^wu;EAY`Np(`?lfIkhIgBs%q9JN1pwT_tYFm+yURpP}Z~Jkr-P%^=Bp>V8?El zeF#)SXcP^5KOmy-q?m#`|6r$n^4A6!LITgN*Gbn-It-3Vi0a4?gz0dfB2*2_mir zqs(<6+$URFur%!6R|@`WODT62{$hO$*X_iOGQS?aH1vLPKAiM{n9S>gQDZ&@sjimd z3-1yvA(61qporMCByse22UK3%?=p?;U!^kPA<}GB8$PC};lm#oY2tn1=QJ(!h9DCuLqNSL@~4?fxkvg zUsTtq!QYuEz@HGxA>jL$}tBkx%J;dyq++vVy*5`-HS#|MHl z9l@tZ8M8ADvN_+>WlR`QrMwo~CZbCD9qmFl^l6;FAM|7=FbWr+l~A-a6{D7WHjONJ z)6#6PD)CO{7PW*LN84Y1KA^ZgE^Z`1-*74t$A`DZS{#D(x1P$!DlZ@{Ka~0zUI_4h zxJa7C3t`iQS}nDpVY-F}58Ct6mW0fuhZ7f0(M4)PDNS%7(!AOGLz?6-)IO~U^(tWo z74l-6>+rt4K`GE$SJgB|vC(lmx99)vblUBx*h@s~+ZCZuJ&4}FI>X~aOFHZkO!fI} zN-MjzMTd+mky#^Q)PCJ15%8h5H%Q=R_<^^eguaQWZiSQhM1(ItCOC)O zpIikb4Lu^YAK8YcB{yw*yE~GvtnO>Alc;hTz5A#x)K2!v`XBb8n1PJ0u&OsgzxOu5 zQ)3YS9XEm-#aJcTq4oligSo;KLXT}0%ReKxfm`ryq*R#g=;$q|qGgZhnT4@hr%5h+ z)v)-FMaF01GjJ29?vj&yc&*%>@aB6}lHhpqB%|N8l7OQReMoeaQR~mU^>oi%Mu&Dg z*n@sOVaNh^{ZhNa{f(Q;1BcT=9NXHX2Xb&U*Y*?ay49JE|{ku;P8~%SK|r_PCT)I$wbHY6_#aypx`! z3uQm#W)9=$WWboyRFa6IY`xw=mGKn^ z0Ewl4AN{)b51d+VV=Ac(H+x(tOIS^_<@!(l7kla+-{W>$Kn|Z;jCLXvpiM&|#aXE! zW}n~lkyZH_R};u-zOL~NwGK+k|`#<-Xe`llwn>|AJ#g$1C(hp6SJk-qtL_OkSy;Iq;ZeC*#(Hfnur zgj(<>S;cEzQMcY+=i$3uT=W-&S&=FMC0ks54Vx1R+B3x>Kkpn0{=5(*5{jQyI?J ze$zBnQ_a-HV^FPrf5W3hY)yg2N7wsL21kQeX?(`)30l{{KNv+jkbe7WFNmA^tG&9bIHlt9a^h z*sA<0{aF{b2bN?KD4?ymB<-J9ywv_fI7)|Pw1 z_w#Jdl5X?IFoP!I5H5Cwzu4kVZhUdAz2e;T_QVlkIbl7wT6{ge#4=qhQub&i-$ zrUE~a!cn&2#}CpEK?1AeSnwxdU0Q8T z?hV}0l|n1@A`H7!m?^IKDJYp#Aa&Gl_M~j16$kI}vd3j-eEPkgXP1e+CcNNSSMuI4 z%bRZvz4a}fWL!v9!UmIoEwLqEY;k2B(@QJwjYlg*a-ITtcYr3q&+G>voe+VB)c6f>v@)+cVNnaT$3JciLumHohsX-&-L$ z=axpObH4O@l(=EIGATOFfA=}pXl{JdK;G%)_b?lDY}eT2*?8FxLD@ELZVRZRQ@-KS zw%5ZdQvQ!~cz((fo~KwoTLJY3k%WH<7uDvh`q2@*qFmx+&*dR03A(Ldg&DfXLsU1X z7Xii8x$*IjiSXe9HCpV^91^ML_V;Aie+-Lnyw|!q|DYToqT=qqlh>4?)ppfw<*NkW zg7T%?+x#YF@V7ohy@j{$$gZkc3{L&~+X5u-RflZ2^aXI4@_OEyoQZi{-pORTrB5Ha z2D>VsYImD%Wr?#64Y8`rC5!jgc#l|O7Mz}{F#fX2d=mLYRhA3w;zDMGpBp`XCd7>8 zeoj**u3>%eQ@3m8$N$q4JP1;P=l&UrP+tniw0iK;zhsJru#s(Mz-ujE>$8?(QSp4( z0^eHtbNVjl-wWzQC{6t7VoR(!k4V7YpqN4DP(`orX}jYLG?5oavO_O@5n=%f#$ij< zKA9@U(G5upFWkMjSbGKtwSo~ZzvJ9u0U18>L-I4u)BXF9v?zW`T{=X(pXjAniBoQ# zIoOb!YyCDl$trTx{2dcTz-VOZM}D(A2e-eTdU3$G|xdB#vwm$ZQQ z8BYFb!lPgMc=waJ&``-qT90AL%PA>m>aJn*rc9$H4-zj9%Ah@-P-ZW;HK770)_sqK>>n2<2+^A!N?JbeuL0YaAG(z)$nr zci^8Pw|@IvJs@rX0#-hP9GsNlOq=}5S5q~?JN?rpf$TWvOriYWCp4Sozj!LK7qJ-e+CDKe~L1{_A2O~ zsP-0L|xk#2K~HQ6cDAn7XBZu-YTrh zzX=x>MCn*`Nq0+kcT0CjceiwRgS2#qbR*qe(vpkrM&Nt!|L*VF*FIY(>)=;2bH~i1 z)=-)b*a}4f1Vq{>eSGM}pwB+#LKzLH7z6P(jXxdY3F!3J<6+G2LZAcPcUM(4SSm{B zsDsPRI@r}+}R`qlL z$MXHZF_D?Xpx|$oQnxr~0uN7cUAd_kQpWqGg=0BGwPC4cuDSOcMak4WL(nu@g_FBd zLEzO9{DP4>W&z|f4^UeJt>CGj=5r*}(r#r5Nuh|>(L54BwO8b!P-ft5X4-GoFleeH-P+*~I^!+wSJo3|eiZdx?MF<$#9eUH*jK&n0FPdJl$P4Bx$Ew`SPs?OhaP z7Dg+aeZavgl|T9W={PXL?7zfD_I9R@8qHt+OXVDpmM`)oH2in#2|+HBh-UiPnqeaq zlKj_-l>6PqxrLY?OddWTBuckKDS1d=k&qZUvf#V#Oc6BGo1UyzTt*F}kg1aA&Sg)d zLwkgT#VCW)g!o zI;Y3NA!{)sJztwWu~_l18Nud-)5G#ye*Th^8{Ug{fiKr$9ST#Cpo2L*BE4*OftjGK zfN#I8)t(L8$KJyKJ5mD_0MD`x3EH##Tc0EbR9N%HL&w}= zFCvmU61GsRECv)}98(tI)5f%a02}~JB)wlnL{!P)WUjv5{Osw2?vF{I);F^RleX;b zKN9ukovuGMcPfR@Di>O+;_Z=^di8c2R{iMg=Q%6(w3&5@j$l2(HQLT&f)00+@qm(i zUZa4S1U{8HG%_LFvIGJCKFU~J4MnUuB)_z7FQt%tZv7Y>*v!Qq0FX-t05T{051BKu zza5R&w`Q0Bj^-1jCkaLT+jDF4GDTWEL$WXxYuSC^oKg%B_9Cv>`*nzLBNY0PSPvkl zE3;{*JS2CQe4hFDC@@*C^3dp?{-9DYsc0DHb=MfZ+OUm1Uoj$o{c8PDfKb!TU@=gN zj51lWcj4wnv<@Hk0jIP_wbT5ERm`RP%b#Q!jr1%LRflj>ojyjiJe<$`6sjmhH+K&* zC)l`nB6<$QBf4o2pOzD&kCV2q`p>*STo+?@%qYKpHLfyHWqD&8FxavI1{F}Sn%@fg z)t{Rw|B)j}fL8Ne6c@8}^H>>KyqBF;304v}NRZ690$L)0QKgz+N0E(jq}DXgL7)D2 zJJkYdT>A}(8k-Kwlu4}Ug&T41Y0w(&KGE#%mmYQ|0;01{`@$E2QmCVHUb`Cp(G$lq zJurHe=dB}j$fLv`l*Qk}%J@|@4qFEkqJpT|Z`rM!s^>7vM8n|riGP)5N$M_l;l;#I zI>nz91Lrdb<%;!A4>8}a8_P422DP+Ad9y;ZnhvLnIU!Gj-A5+7e@i0j29}ie2jb5x z%l{-hY=BZDoTy62_@L}oMg|i-?i!cnOmNfgKOse5FRBeW2;?+k380+Cu zVbZ9(1yVvD$x&jqzD}k$uQDzx_f(`{uIO@=+U_uAuk6}C1S_jgUN*B`ze^xNIf z7trvTAlZ#<=sy3p9WTvF`r|b^I&EKr1ZasqgO~qh>?l76KuntDH0q}Rr#S-hEmMSH zJnjTPA9MJ6w?joibzXfd`b}~Rl~a<X)$Y9(Ogg3BcN8Dl*ab>)yu z$m+|O@H9?DNmR8q99ZlMnQ;%&p>elA0v%2qJ}*6N?xuV1_YUD*zmY83EtpB&9qiMoNk6`L(*JNbUoCe0zmx5dGe)}VoYV1)>HKG4x^^-DMiG0Z%vv0?9fT_oy1 z@YUaJTz5MbSPFDL7jIn9ho-}dxqjq{kyI>&EcE^=7vQt`Fu6r5;Jto!op7zvmHLk& ze-i5dU;f5?%ilb#t4#mpZ)6A(0+rr81JPdBYuS)G9;{URvvnBBwFC4ZFo5-Ucxu0V z*W9=I2qIil_xjAnTUkj;J~-^wc;9HZ81-mSsP3Vp@dV@oJ&#gV7RIu%Nf8s>!Ua}< zIk5^#0P|+V`_)p~RqF^hJYh03&pI6M2qo<$Z69A%TV?!D#9Ey{pH=imO4(3GR8Jld z1GuF_S9Jq@5ybK)1cAfVU+%7hgYWszker0 zXgcj%OsMwVzP1pAk-S^8w1wwND1PUG_3yN?{oetv7v~+iUjKjT0|kh(j^tnd6*ZD) z?>-h|)!X0ZOSnCi@W@Fkn1L7;H${etGBEh@Rn3$F3)umvBwof0*m+q_0UU54jk-wn?$+&)OLn2=~ zC|3lgaVoaDeF;y2N~TdPBgH2-Cz`oJp8U%Q5dM}H{1wqH$8+)qj{m)5*f)(B_G5U) z&_A41OJsoFOY(nmp&-1O!sujAc#$+HL=C2VseB;L6e_4XTSt~`@83BC>Q$q2{9=Vi z^@2*qe9vZ6IDS1DPZJq$wT?w^-XQQ%tFP_58cs92X9&=@20`$jynb*ZN`RjfZS+Lv zH{oPgFsGcnP*A4ZjCE9W;wBF(?FIL|mEa&+1$3HcVlwzTa5o>;7P}(6BHwrN+Xf%= zs@V2R$b*B$_#fr(?L+>(W!3nEA4(q{TkcZ!%Wt(`&V3)=X%SX{sS2Zg9Ud9$z00zz z7BiPqF;g7&_Jvdy)Euhx1+Lxw|DnWfKsP9M7zohr`M3HR(*IV^Z*TW5Um0G*o<8@h zT|*9Vy-#ITCMFEtSD|Kmk^pylraeq0n+;=^iuS6tkC_V%;fr3ign_LOgqx`CO5+j7 zo54iY`vdN!%=Akl-g4E7&F?}x{fgAW5jJe0%kMUjlUZZ_mO9sc!xZSLm@H|aqwYVC z2(jV}xF(uE$lf;vq>@ehL30&uF#9E20b&9x|sc&wF`XVRCF63S5G44?mf6-P8DI$HkH_1mP&;HLXqVbk% zD=z*l{40#&>S3k=LqQ0Ul4;EBTpyTe0bhk5dF~~Gv06bHPqgG|Nr4#=z`kISpA>!# z3rU1))z%?wI$aktO;Kf`)B0Z1N9PO1>bk?wId1lQBP=VsVvl)j!+h+-zrBoXswPc1 zo6DaddX?GlOz8m;^|Rw@*a?8^k*oF1`_=Ux@jrrtw;l|+R64OHU(Y5-`~7AOA`UeY zvmg{0)TB|J>lE+DB^4%Dd3~-JeMV!iD_o8T%(9;mJjHNP=9|Nm^`Y<(x2pwwkC~2h9q0-*{=p$+6s{)3~Bjx>A}O7Fy=o zR^<gks{g?8&aZ_&mU;$z0&P3r>OBa z76U}legNrMVQ;NZ`wpvC+3Sb@jtlkM8S-EI@`geLA6JZ5^mV=XG6o%cG1XqxO(uzA z^blMcgRBdVvQ&PkKc+mZ6mQ05kNcjhb+McloZFX?0d zMBC}lAeB~7_v99*3xDb~)TYXF>YYGVM}bIZaGdlxR`-?BPJ}^C3X7}mWRI!(IgdDA zF{U#RDsQ@dK-$J`y^RqiNJD!5GTBa;6?n9{D(dR<_$j9wNQO`+jp@DGr`P^yNZ+of zbz02B4$hvp%|#2qEl!M^M*6P`J~qB(yQ%gC_hdW`Pq_&?@V?I-jN3;WPnN&LSq^Ye zF!hFE$jE^Tm}0laA^#t-$V32n*o%bo-r5unC6+)E@G6-tW?+y||iqm2k7jM!R-FJ!;)4D@k& z)_smFhZb1FC4&^&kD45oi4HV`*;6w=4&4K8zqGMq`KB4(N6`VCv2{7#7+pwhT zNTYEa6gMPruJ+cgJns0B#u| zxSBaep*1D^i?t%820;LDUXv2k^{7KDx3J)^%hP=b27G&@OLJL{Q-P(tp?e|cfY7Sp z)Vj$8Pf}!(;|c>XiH^L8LGL=S_ge&FVkVGwc9tCu`-Bkm)21%n_RdD&aT=v>K9EY7 zXOMAQ@M{pGUe_uv#+80yg)-;u`2&!sHVpilFvAQq)|d+UYLr?l?&_sU2eho2A;{(w?o5N+guET!6YH)dp%cp zc_)D5v37=VX8vgMZI9G<40xtA1KRh$^k3N7m0dnsu_gYue!mOgea_w+_0j()0H4EP zho2YN-!|gi)zlsatyfUJE&Gouv$*O~$(~r!D9|W|Ubx#|6W$&Ml-`&2fPKaX<{k$V zHuQENMa2C)k3(PoKh6JoMV|qRRNUN)Q?$53LLixDs_Wr&LtNHRNtYSHhn7olK9#s1 zI&zBb`khm}iC)gVbFQ@q;RghX;y!<=r#ISS(b~D^ z6en_cgF2kY4`crkKnRe&=aWXww+8)-IilE@2fa%-reZX?pqPm9MV!E~ASI8^OC{RS z20Y`;gj}4N8goli{QkB}hk{Nd4*k>V^eT0S=@G@gA+@jx0}Uxi))^-Ec_?b!OJzMH zQTJQfSR-3U+y)a9Nyf^-u2qfM^cDU;2n)x4M}=@nBAf4fdTG)8-sw()we&@_U&j&6 zUWXW8DAuojd)eP;VNsYC{cPP48ZHx2c7NHD*oZMkGTfNbG!bhqDE{_vKb!FyMX2B* zw{FaK(Tp|H>VeA##2WF~PlUrT1(3kcq8=*8H^+e=NRkOMoSiOV^Eh`bB0+VHI#i9{ zeA6(X5S=-Dja^<7{le8)g}Q7*snz6AI)k19tt)FgeTL>oLk2sljSzRWiQ=AYv4>&anP1A z&;`z0i`AC%?EWqfiqhMC*ygA*^zg(i8%c~w9AVXjR&Zm7bh?7yZ|3{MLdX4k&d*={ zkyVk(7p!19VM%=KugDBJjZ>mqf4W+zez8mFty{v-7H|?Uu98EO7O!SHFx5ksIVxX| zzZxkLdfQ`qm5%;_&%f6HG*an(pK+qb92j7X%GZx@ z(R1AQwwGB^#T+U=BdBeFUtX%u8`NKHdrfaVQ#Nwzl^ulm0cKTT7G^TiyJcT+Jj6&j z{x~UXHR?KvkzlRRbu8v?6RgX){WnM;Ew!S+tY(tiFyCjLlr1Cix9we$%FA^vEhde^ z#74_n{vg;0>g$_Z?Pqtq)xY}HV2~uv9PoniyidbO<{C*@NaOnrEzNH|?OJ*{mw`2} zl9XRDW~J3fGie|Rph8jWc8>>&kAtC`_P+t+DMW|BQLhZLUXRQ(;|C})a&gF0gCEJ) zzg{?ApRbyi7jbG;yN47!yrs7DJ_w<3oK_BkgxtAn>x36@Eb~=lrBPX@p^Qoz2aNDQ z>6^D)*@u$`LGl$tnU#C-(d4%ofYq$ENSIN`bZ@W0s&79$w>#v^6f$V6 zB^6j6QK^y!F^o;S*^p^d-_3Vp`C%`{*pfO28D#wP&g|C%sVa3^a#scy7Y)eus7}dF^%9y z4NPsr+{_!f$`9*U9rZnBE1mN-t`xl9Ojih`aVl@gPNla$8nE~ItB4AcGiEU`3Xwmx zwAY?~V;H7wk&{QO{3BCu2m)n0kt98Nf_0l^tC|DWCy@Gz~ zlL#x82^#5po#W;cGX^eVu45h^?XK9tfzP-!>X3q6;+%3tR_rW}J(|IGK`yeqZY07i zI!Z1)b2;5Z6S4dcXXnFruiYQ1cHnz`53c{*igI&@{PhhSdaneO3%5YIfO7CK{P^Lz z{?@Wh0n4kVy>89jBNU{vwUg2We5b-2m1(##mEgKsBF;`Y%bcpfiv`Q`@Ko9jDn!fIT#b4N?%IC7yViy2{1$2~{=t*Edc z>VuR?ITRHrK_zVLr)gz5UMzeas%F6MK3P=P^>cL(sme_}%8#E)z8^`sOV|9XEJ!jY z?_{@@{h-&jkEDf`6kHxvx*=b7M4skUc`W&Xy`A7#4O`GP70V|ErG}H8>(b(*|y>h-`9p0(f96ht4oaPRVmanNdh@ftU;FnZtYXUMC z28kdkvA_W~iC~6w`{AwmkW85otys+T#IR`CwN$%Ygn29|y^T#cP?&@NW(M_1|nVQ8A zh{}1u@_OH~&9Edqikw(UE9h@jOhh&~X1qAbLl@9Ev>=J^I8BE%IB|OIXk3A!{Jy-g z-&CaCCvWk!naF@F&xz$y2@(a#u!Xma$_*0}m5AyyIr~Qh~ryZrCP?48p~vtoVe?*wgeG zgtKg(tVoVdZ>ggMTrrf2SRO-ffGa%N%aCLYfU(l=xfpo=TZR!w%xAej`CD4c_qrv(l9X@U!byFg-C!)OkVN4TqiH^kJuu*K=EpJ$UPl-(OU9a?qgob7y(5fu3uMX`5T1eA8@gFj&#^>bT|333{6ub9;jT{yf+ml{w3 z@mWb-imKuR`R{4V3G^FexC0!y5H4rj&M8gla%ydITwfR2^u$^c}eTYvl)UL^A zY9!M!8XDbtlN&!KGVRBkid8bY8d$n(iF{5bTnH%TrNpGPuY2P6JVSu*O_o(jNwqBd z1WC4@6E4T!+OA0?SP-gWpx1gZp4*%y-I>^6*UiZO3RfXQKpe$htm3m$>GMnShU;6) zCxO4iu@_rP(3w3jmA)p<<@2_q7oHqD^<%Q%PjG}c-BC;>kmy=s!g>mu3a3AF!h(Lp zb9W8?sfS4=D|GtcR2eJ&Tx!P?{Ho22yxBJmp8C8`9(eOqGWryRqi$3R3wZFvyasgv zPBelyiMFJ2CHNVUjcSapZ)XY}Et|%Cq&@qZj&*YTAoGZ7e7rwVfxifpJ#xxx#RNqV zg_u{{yjrcA!y^!gmaIp9e^7RF{Uyu0X-cER>1y%OzYiW@dEWLq0STWY11O%@Nl+s8 zGynC^-46r`S2$EvJvMT$9YK*+sx(!)uXaA?r8#gdPKx9f`;f04g-IRB?%fa?)>{t` z{n{Eszoer`+VMohlsvJ(+yO`BF@%|&4gfI~w7KMFhXPD5w=Zt_(%A+najSlc?Lz7O zv81G+*xA~xHS4=Z`Y&3>1`gG=hB1|Ei(4KeaV~_VtFecd;>V3= zfo#1IBO@fuw(c0a3hhsz-|)}9v?6Ma@7X)=gU^Usn17sDR?&f{-L+w$yRNLmL?=?K8t9ZxG#6L(9Wfo0{y=lIa>$DP z#Z$8#hD*t!7pU`0HmCz|J}=2+&)*KyJs@s$=9&(ftZln)e{#{oa(&dbZ}TD|T9Uef zdenneeU)D+qU4LDbGpAtXmdY2DgL6PQ8W8Ta^wYd_@Sr21<__h*RbC%G8Ka6qQ#I> z4-iTrrQS+>DHfGgMF3gDgb0#}f_rM=|y_VBn4~G#$*C^YTiD=cz%ZEHW!yDnmOgx{h>Cs|eRry%TXJliZ1oYN1XLiVy z0IHq(-?0eSh*o7E&N*mhw#DUkCjuMYCbr~a{`xG{Y}AfQIW#d#@%){9t369FpR=vDS!T=)78b>hHI`w+g#S7J`IU{mXPoR+~79J5sq1ZQIN0Svca$&Vk{Mk(B1Px2=wuH_?i;yleKWC(8N+{R~U}i=RF%B&N z6PQneYtlOyHBFGL+37BY8V&bas;{?Rwmj3xNcdj=Vx0ge*Q>X)orD)^|FPg`rmmGj ziMhi3!HxRNmMPfTy}@>s7V!F=KAa{ZeN)=0-F?$FhNFSa9p>{{^%OUdAVq0+`4%zfGw7ST*yt+hjYjp3My&+=!8M_$zSU#7fsNJS1gBX3JuL2 zrApYxB%RXOOZceA)`-z_%9dGDMAb=P6>uheP93qva1PQnr{z`LMkWKv%*-{*crCI; zR$zlGlid^33;wXA$%Zv0#Ga;B=h%q64!yg)6y@J`5OS~5dq%WhI$RBx+7%T2}S}q#Eu6905XSu{gat%GhT(rs4KrES!iB5q+3nN&Q*i9`0i0T@|kIkhm4wF%eEkI)-4XCY$}3y4~CB=pw$uZ371j8Ot!2LI^bzO8ICSHH1)aB1?^ry%q zA3{u4#~ygzBmU}jVGbn*Wxt=0iYJ{8c)n>UDX$c+n7<@W@6Jd-he#$XWayKTlO{vK zp_9TeBWj-8kg+Yst6XejGod_@p%A;lVBi89H0Dqf`6Zmv|6Z z_;W0P@$Y0|=WgXhNe=mq6Paq9ih%dWu>BNrEoMam!s=2o&gyIW&}2K;I@h7euHz@u zXa0?@00mz+>pIKl#;(VwWS`OYx3XY3`0akO>8)zL6|6sA>{AWmEI*@9x9nW*m5#c6 z7e!)cOe@(FO}r6hPBzqcj#5q4o=tDec4O(q#PPt9!ZY9C0%yWZyL~mBE-mF#&$O!1 zX8z2<{IQ%#aanf@agey7cqKE2wLjXe+;UPBXeqG2jl^31Nd{vrV;W+rR?luD{G?!O zj&j<^#QShA_O66R!Q0$RPV6Cum5D?tF@hDIORQi`8ZH|EF9~G}y^s)KRfcY~149ob?@WeoPipX+|tfy^X z+rvw1tw`tb-u*u{9DvrlSI_O9^OiJe1RF^L*5=7!e|^Jg4E7Al=}!Wqqcp ziipo~^iUq92|$mObf2fvMQNYeU7)7M*gBJ2F%x6iP@_c)U#ZPO0KTlcV~}nC6xBkA zn34$?-S)P!V=++p0P~{~Edv%Nme>Rv@h|fCDRdW2yls$8qZuM>k?7W~=9Q|X9$gWyI)T1D5qE9mP`55C<=q8`4TMX| z#mewH-)uw;*L+%6EshT!JW8*B9C~goeEh-A7%rm~#^F(##nj^qu3lfbbkh=@n9ykw z8{KF@(?<}6%xEt<9YZ8)YDFnhMay=#e(wBLU`A$s*%>Y$q+CUHv& zeM^GABz0oa!*X3BMSh%h&Gpwh7h;>icc#v03a2#y%4QxduolNNK(PwhjAlM#GpHCC zsC$5I-XXms6YxE3*8H&Wv81}|PL%&phVFjc7q`E|e`p{`HtJ=?iz8uCh2za-z{LON zNk@rK|P5&CIrca*k@8WJpoy zTbbpgB;o3T#C~xn&!N@nn+uk+^zWMndvdQqd-5$hVpevPfN_buoC8&g1{T*c zSRCT!P!5p}U`aoB=9VCn-o056LdntVxdd9u8s;|v$h@u`m7mN`!K+H-kq!Dy3{WsF zA|KP+0444Z|qnE_VxrnEIPtYGdOVs&L72)z;B~yA!Iiv;cB8?&_ z)+&;AsP4NW$y}_;HL#0e)#gC|wtcphH@#^@n=uYRy-O8t1=W0!nlwhRQJ+r-FHacC z1sOI9ghgv54u~eqD~)H2^%Z&2D`J-`m;6xet#z_n2XrWPF9xe1ZJkFv%IyNBbxZ08=xbHNU+r=;x8#=nZJ$P*x({_ zpj0p0_~iIWfrEs|;Yd3@sNpwc>TKXuw$Xmp?c{9d#Za^r+4OG}Ly^`50lmHA0#adW zCUTJElav_Qc&G@>jL#;zr4j3m_n#nadg;qAq1Z;)py6cLS2Z-{O!K8D&y9C%4ENyK<*~ zGfPp+N!1H8Z|psJ=%FumM6fxRZ&10c~70` zZD9dBO50QS{k5cyEjq zgufv2Q7(E`ci4nzLgw|hTbqo~QY&@D%WA@XCTa_BC`D>LlN+GYHhiFrV4>r0po|B; zMyi+a{qVQ!Nhxaf^?+~J$%m7<>7Zg|n2Y@JSgXO;(>5RNjE_$er^m)>tZnUKW4xiJ zYIAqXiaMD~E zU~uakz0Af0f1(qo2Dxb#KB#KJb}Vhr;d;1NOACgAuNNxTO$G+ia(&_|mAkLQK+H;l zNW%5z0}l8)+O~PWaHzlxwwHDrcXR2RJ!1?$f6YPU_WlIsQ$87&B%4e?++MG$qb)ga z)2~7Qhe334Fohwz7n6sX-spy=rgD&o z!UH&8?t{+PHE1cAcx|t>71zj&0&1!QoU_!AKTmsI2fyCl5lOy~|2{moEQE_Z<{3q}F@Gmtmq9M}rmhC;W z+iQ1{D%r@m8;YGgG)GTb zWWLua+A8!6(1%T~)_&E$d!q21c5V8^%)6H9jndNgI@aX9w_d!N-LbuJodfu6<+6lW$lBlx ztS;&nL44;ATlM!kw6s*8cO&h$Fp@BKh-a<46IOyV^S%*sbt#8YwCXE;na^Z4&YH_q zi*F=ZYw!y;!P=vg_{>;kN0ZjbD$6)&U}3A-o@s9tWLB%ks1@3Hk%ORFsI2y>O0e4> zUsi4+zsXJV&%sR3N@V1fqUM3i#Ts=^zYRVlzv_3GlHB8@rmOWoii#4h6g<69Y*A}F zGV!!%$cpu&F*B<=ybn7@)I#nd)6WH~V%`>nW$!Ca5rUEIjGz+Uzg25SnazMBWrRpj zVCcpacdkV;e8Rx+C}FjLyU8-9ZgA7(XHjd3qI(UUgyll=mss|!gz^RpAjP2hU)kW55&$1@WHXMeS%?Fyy;!4Y0F)gzIU9-IO!X)DlJiUax7Eo#%I+2G( zaI$oA=Fs{bGbf~c^lB_fY^T!;fIo=8vEA$#B`){)?i6`z{TB+6o?e2~G+J!q{Vt38 zu>7O!Vk8WRVVaZ(c2ZlJ)XxZme5-#|Ss_!@LF&tO@@2e(7!iIlWjW6_HjV6EP}QBT+wwHe3@I`qm&@Esj_^;9 zp{B_?FQ1Ze3DoRU9-tk8#JftzQsnOag&=Xy5+T=5WHcKzsp?aNx+m@RXqXHh7QM^# zOAochA?|DtSo$k*6Y14=rj|_>tjoAQSONkIAu}_L$ZNlT*8qaiiC&d(oxi> zUMG&mLyD>!=wnvQOb!nw7dqeF`$Zh4KoibtFoTPd@9TNPqgr0Z#g;HjF1#?9r#be#!1yM|tC}N;CH=97FN9!E&+7z^0t^nyV?++U zQ_VY@hhE;y^s;tBzd>$g&eU|V;ee-j7&{= z5!O9yQO$BIQp>UtA6&|1r*U_@|9$Ao!9v(1Z-$t$Mv%gZR0Brjw=VV!myRv6@9@qW zNgqWC7SmG_S>W}A6KA}K;%@4lQx|whxJr}MlOH7IwHo2nFIs+nSpBy2q_llr>Z!t7 zRebrwyBSJCkepXFA*_l}%5!3zj&$^^MEcN$M?tAjxok&Fz~Y29wtpotk$w!yl1ED= z{DJVM-0t|MjF0YkFY~4hFTwEDyeeM1^lg zNm(T_1XzE=%DL=S^*u~stj~r_nW8r#e{vwWPrqHpZ1K~0MOe?$9ezommjPUPe)+Nm zxd+x6JDJz>BeY%*eH$mx-s<`T=#v^lr_s}j0$TjE)M-Z>1#)btYU_lsh2|o$h|rv_ znlJu905B_k*~vpK?oZwc`O*?oIEpQt6Jy!!EwN_LD!E~<&w%7O7wKu1CP|b&IhO?? z33vF;b=V7cB}v!M{*X)E5B9&udn$%xE)SjX%eN;Nqav>Nu4r8Ao1}rkVOOVX#Mc{m zq4eW$uR=ukfDJ9wY&W z2(>WRfF1I>$Wk_Hobprn_BvMBp%1Se9Ta3KuUjP6!>bLR-VPo-tSusGT#g8!|@Vn zHg3NjCveu){R7wL0} z1vL2zt@7|OE%`M#!_P1~dDF-%^(s#%W-gC}zRtCKXLc!zQM7yTSrnxAeIVh(>Vxmx zE9%pc*}Mt6ll}m;Ba>IY*viBv5PN;iS`u^o=+!B`P%Qi!Jp&znRk+DRRXH-7xx8_E zf6)BG&Mdu=*z(C=CENE__OrZtIHMK=00ZJs-wp(|cPQ&OEDXv)eu`+YDD1e)xume z_!AnCq0_fQ*zxEeI?2SPxM*~njHH6ZwXrr+4~F`+aTWel4o?0`rxgB-5lA}nWs#4w z0wf(o3zB9B@waE{MRlEAC(PBq%-XSR3x-oy(GrfHkFQ2G;^ z#3P$JHEQM6P2YJ}e>Oa~-qoFEb>YaT-&C@97H&VQ~j+gro_HaG*?@VGDBA zTrTJPPBPxz?=lD;xHyog5*|q^qcf3#Ey-Mxg=X9{KWmSTXOtn6^OPS}=s;b=JKk6F zc3tk8e+Ao#ep-6)ers!Y%b#2X@)WU^7XSgw z2##KT00tWpsH6}-BdmsH1^NATkoBf$a`euPv)8=N-E{2mkv7Z#cy5mkS)~}+2GOTL z-rs?(>9I~??z`V-W*cU8+nsbdE!ymT|02oe*>nWzHVZf?*hq_BJ;~hWjqHt!+&J&f zb{ZXYcf_r>Ce9hMW*e6vlYJ(QHcCm!qI&6M z?up7LlF#1&_Qf&=Pvxo0iG%OEVsD9?6oY2qdcV5|BkQ-!_{Xf=vAW$pdzF8l=N~?l zck#>ru1*+qK>R(k*u?y)%nQS-jJAI2_C!&9B-GW@aK`D?zvy^w^|wp}adMWWm#RtD zF?YP&S|BNQw#IdEi#DG{jdHfZ#mA!Nf+>Jmk_hUO@DfM@uUg|CcS;R@Q2I6NNy3%7 zMjNoI?=PbhiRmDtDkZ;M+)9SbUN=7JlD2RJ&*!IrBxhCbrRC+FK`sQ=?q&gz{_r6< zFpwfEEhtW*5N>~Qf+yuFy_OwluLT7r=0Q){aXe(uD;^Ro**thf`GgK{7+f?jDtZl2 z6bl-gx!Dmiiyp!|8Y7FlUb-+!Q28j2Y)i?X(C(QerDAP%roQCEgG>5^!j84^Y#3Q6Oxdyq;n@}s{r#+GJ zA;Mip!bgX7{+)qw5b~<*W$G` zT-qFb7s}U@o+P*nrw*S@fIp98nIFn|f4baM>f%>TX%Y?^nR4z4(E0hkrPpdtoCBJP z?K7>0Tepe;nScl=hc7_(5VH?_b%MnE{bKL@%`tyQn0~V>r;_a49{?3CmU{&Bm6^eP zZPW;QbUdBVAc8L`K-5)kL&De&_Vto0&lFzCaUb)F4W~T7p1Q&ZT%9$*V|Cxq7toaC~LQ^EqiCaU_U{ zyEvpnwAhDG%XEtrg(kjQxtfq$i^eQ4KD*7z)q)Z9{W=Q&=!9FCob4bpH%xAc!t+8# zY+ZflQBe7zjTX~*KQI>*CpFzNYNJ(#1!v;>*O{ekbmZ^p$sAJW-ur0rE01f(E4JayB)tVoS#l?sK)joR5k!9<0 z;1@k2T7)SrrMmbjK%T?P5fMXSR6Ig=oP$~YSUSg2OrKd9gJ-cBX$c8T&iTa z0&}Ft!{lmyT@Y_q7xz} zX8ywwwnAH_JD=jE51Hq&fL83#mg2}JPSJ16vgK6NLl@rSO=Z$CA%Bu+CdfgaT?V(U z)T25BsD$*!e2j!hLNlg-9$>k_`rUK3DbO}#AO&ti#O0T@L;QfybJWOWUc=NDyq$V- zHXty7&appLp=NBxr8{fkObcI?Gc<+8X~a(DqVqE}F?~ThvQA60w@_~TjurAXuP+?| zn9R(m11i-MJtU&wVf&8u5~~-`PNCF$9lg336&b@p@av~hDC#~p_7ZeP;Elu1Fsl4T?h7*>kWJ?P^L0nke^{pub z*^5P|4|qZulpnxoYM>v*}brX_;)ar*u}fah5xAmclp1x>5;6e-A$i);xWTm$Qt3S6W6?kTWoMgiYFT zk_s9_JAl2grIgvZt0<$pfpAY(z-g zkop^@uzbn1%11d`wR`_}vjow1vEjx34tzEtp|%V~fra^;Svy=os?Lk8H6uGT>11x% z`riz9ey!~sLOFVDR(& z86@4xvQ$da)p2zVd9h`He2fI2j(hj#=j~Ttft|jw(_&JO-Neng2F)b;fwi#=Amrf^A zuU4>1aXwZx22M!F)0awpPmNUf`(r^kVcWGtVQaIzwize{!(|uR7=yUjEr|O6W9_Y@ z+W5Bk(F(L^akt`5iWf~=3dJc>+zJJX1$Qf6ik0FPpt!pRDemqP+$CrT7Wk##d+&Sq zt@rMKZ>_Ud$l}b*$!z=V&)KtQtWgqDH;T0UEJ~E+8T(y+{yuly?;>5{5^8wJtjJhe zPj82N>!hW+4VJn;EGRcbMrbhc_7{F&;T>%MKI*4k($TU753QqX0es+p?U*|2tvr7o z&NDm?m0*p_d^h}sH~HgBM?%6=9c{w^LV_~tGXFrz5s+IxW3R1+i$$6wq2~mD>B5ik zMxD+fto1(Pn&hU};f3@zFD+%t2}s^Z%c0CYZQ13q ztnyw_s{&aO?MV2`T3ki@qOauEhJG~TJLT6jp*5q1FLU!%WmTGZHslPIl|9Zk`p4yk zOJjo_4aD;X8QQidq)f(J);o%>^axqh_O_j~ejB!dwi=q zVT-N(IMTq6>~tHvdguI=@z@Xk|U zjoB{C1ns1n`c#;eSSla&s!+e>Y89(*r1Jh(!!fmID2JeK&gSNddXFHhL8pYX-Yb0rlj9s^r?&e zY*VqPa)P^)k#k$kA*#Z?pzn0v$DS6-paRNg-Gbo>>eIn&^RGo zQ{4>HZss1_y`G9`lKN`(8Ygtv-_@aYk zMWd&DkxbB}xl9Q_yXe|+dxmvt+)Wh)0l4A%!Cg`B3KN9@@=C9Q@d2YFm}f@b%fXLd zNYoIVHY)rCORw8izjE_@lxQx^$zY`0pVEgrus`CCTdP@K^pVIvh3&Or4SkIdCx;5E zL(O2pWz=F4p^Odb>3UdP`kp!{=|l52-Bi+75_uGHWWA0aq5kvNd++5&{-DB29it>M zf(-aOmk-+pR$0e!(gZ1@{Aiadp5Bl7nk5Kw@|j*gCAdkYFZq>76nh|&zV}*9 zZIF-AH?)vhshJ!@V}eemw9nQ&=GnW13jsfqsS>?}+u)QGx>jRN#3`c?iCXL>=6jn7 zI2X3T?2G`m&30%ZdzTPr?f|}-73eK23c<}Eibly zWly&Blx}v-^!?V%+GbW|m)zUYT~c ztZO|bc+`#+lQttjaBpd-bvfRu7Vm7m5abX0+zcc9lJs#^S~rEGZA>&$XJ6He`1y>?Iy*+auFh-SfDLcV|9*j2UyQjt`qKNZucd zP0K|!t8<+EEX=4tjyvuX`@^Y&85VD-_O*H`wFX!$j2S$|wqN^R;`dzx`g6yc5mjXs zInQ4=$kX1Gz-4UHV3~?3oXrcRwT1CtiSL`4e5&ebUlEA%ywiuXy}x?Gak&0pK3vaNCH z{@woIPY|o$vtOi-<($=?GtFP0hcVnWtQ3nbMn%!Eo!*bJ=59x>!Iwf79bm>g1$7Yn z&Cd|9*p^SjuiKDw)1|%3Um2bO{Q95NGQ`x??R?7txaI_ul6hg z6W*8Cu7s)hxJhug-?}MfvN)kzV=E`U|DHK?F4x6MqGB#eI+4h`qkEd%^K;$F!<^va zl@%FYhM9DL_kscKt082r7GQlj@G)0-Xfu}z|3_#+pk* z>eQQJ8uxU7W)>l@Y6i3w5vVqiH(3(Ir#hYm7GHij|L{2MuDM6iGPc4PlZNp#v805on z(#g|Is?^cP-ke{gpLbwe@YcKv|B)ho$TK1pBMW>5=sQ^Ic%pRFpOf$ZEF$%9IUwp0 zbs{i3u&Z$XtsJUF=B{|d;H~+Caq`FAxfws|*t3tM@!r{Vf>pBU(fTaY8;4ytRCgn% zF7fR?(Lj2tNDeh{XdW?(VJysLMAFnJNnIIA-{)OHw{aHQaH!6uA@@F^7MJS-$upH< ztxuUXB~eegox%W51bkPTUI$v^&Csc}-L~srED>LmveJfdOA5T!CE8>SMpKy1=l3BQ zOs0*zO4GY}m=_~jZGhcG`PDWZ-&TDQapdYv$PHQuvpKYs{hBZJD3x)JM^SGfOG(}M zt0}91nJyO7$0U4=*;hdcj7j*v6v+t*{f-uB=D= zmT%g8b&srS^Os?_lA~7=EgOE`_YY9yv72A0R<{auyIFjx=gqHPfOkSX>1SI7BF!T0 zn5r}tGT(#;p{l*b$B8yKlM_g`v*ZbEvns&>KjyH z)?TkFon+<@MqVgo8R~o*U^)IkXzf&JbT@2kFi_cSH0Y(fdaLu<<*{Rt)7X?IaCF!} zFPub+^moCl=jge)W4y0je4dtHn}oT4X!cE1^WmNY&!n~&X!k@BpG=hO2=~b=5v0fDuYRDZ<7qM)aW#T;P2)?X* z%6S=sCi@MIQ0ZHM#c#~&y9UBn*y*$+&tYMSegQ^?Z0{<#Hm!@}KIZA?nvwvrug!o9 z%Fzi^S9IKkT<;POtZ+tSc|w1{ZM|`EUHyI2%-0VhLPde z_C14vE9^uhRU#C-(~i9I@Yx8iiHy$o{nxhe0m9#S*PfwrlyO(P^I?o9m`u>33uYH^S*PHomvt% z#0MXfRPxUo%}Ed8*8Cy-(HlJD@A@T1|=*WI2@7xITg)e>Z{ zB%GLK+%aO*`Mkq}v+G(TG@7S?5&M_Pvy5Se;k%-NM_cd`VUn-XbTTc%;Xgf*+1Qgn z!od}DT60oIZUXUN$YplqDk=C|^NiPs2yd3}s9xpgV1n{b;`&w19H((u2eIXZ1v4f3 z{%HZ;8d`Axkr>mf6p!baKObTKeDm@#7TEv&RlONzQum0@Egdeqv;1?(0{up}l9c57 ztGM%R^sfx08ET<6;oMPMO&f&`>T(H_3fpr%KfSGDcfIAGU$kmizmx^oljm|UK*I-D!LFS_vX*zZ(Zog!0}@(%-u+GEdkMk&%#Bg8S`F`!c$2R`!Dx ziHN5Ty(HmMf%t8t;{pk9C)Da8B_#19Hm} zO-flOb`qVqi7fTVAV%e*6S=@}xL|yFveCv_%1?Xld=?q{{=&Q^j=JkiqQ>N8w$Wkf zPacIaV+GpXHD#K-l8rGarGA9jmG}sK^y=)3r31@LZ0~$s^6z6Fm41%^UMT@h<>frd z(k&LmYbM$}F1#b%U(%lQDR*18E0++Mg-??h4#>{BXQsqo@Wxp;>Fe1{22?H)B|WmP z+!>m!IVl{9@+R$-zne1ysEYqV%lcOks+zLNoSSV zT3IfK8PnY}`jX1*d$&28_1le*Zx0QISdTh@a>ik37Z!Qft{xKc^;5=Z>LDlWGhmp4 zUBT?vXDOV6kwG$_IIYNMCvhTsnch5M!himRSbXDAj5Jfow+5+J_g$^sL&>Uw+nc(j zHS~A+-dQtca>M{4eo0NXPkR16>saNt^y^?7QJ#Wh_tTsgJx@um0hF`OPLqq8o5t#~ zfVYWE`|?5fCEcSj%i%KSZq?Hknqg0CsftuJ4FdQFTB^^5dsU4z!#=s-MnElT3(Q1k zNIBORBn8&q4GYF=*XL1`&X$5L;(vFKz!m^)_fp*}Mg3YE(O zdYlRpY3C_I2qt_g|37qs&Xx?`BQsoA_Z?mLFEqke(lz(`6l@UZ&Z#6x89fxi6=@WaGr@9?D#u5iFB1=QZDTL^? zWob#FgUXYB#+M!WK|e9PeMU&lO`9L}l@A3n38cj3$WXkPiYLxSF&L46TcKy-4gv9UHd2*PlTgf$OYwS+>uhwyL_(A#_uZgYjy8CYp)8RYq_eY zgjBWol;vj(j}vfaFJu>p`dj>Yj3b~Fm@S)7PNM~|-|5Q{h|I^2n%{H#Ib4PzjeJ^jyuqd?8Or7q)BhH*N5i>Pk_M`{wE({LcF&=U2ckn3fn@<<}c| z0t{lw?gb}%PeOGHTH;NMKQZzC9nIn-gUyt73vpSe%C#i|{ z>s^>NB|3pr!nPeSgX2Ic4;g#L$NVT9VooWp%;6sjzI)bszOo{?kHhRd(0x>FUa=$5 zs+cQh`;wE<|Iw2$;&-*53P@kMOcR=_+l9$8oJTZW;D6)J_ul>Spv}bx+IhbJyMwM>DNRuq}3qVwTfmLU6?^e`6E2ZUt;S@>Ej( z$Jf6u13!SQq?gu(e`q+S?+59*Cmr2etqd52%zieJw}q*%vYgjJRJJ?hL89mH}maR zXWM!tyb#8>Vk-n!+s{Icy4%%ql}X# zP=sYRU1*Ab4^Q~@MIg-OFHU@f`CcwF-{MiU_k-nz<&SjLcblevqXlZ%2Yo}9(^q+?{s@OSj29synX5yhD|oIu>Q8O`Z{+C+*V`Q0kLQ{eZR%u75vwjNdBhn(|We3?p}M~ z5}_Pn;tHDC1vxoAUG_ZKc7}OQ+{`R3CDk-(t2(2lkpb2((;=rbdZ70Fd*li!aTNmW zr->q{^3N$>vG*d%+>hV;Gto4O~r59SG>&+ z*m_!9{<|g8@B{Xv`3p3Z5b(U4u))Nl+~U`B0)#Ig|MwUqz+XELMuysNzE9j*NGQVn zy=sY!@wyL2US0SgHjogv4LVq}>D$1qlqIQlr^l3f8JVjT8z+9-PIz6~?{<5|s_ozg zL(W01?C9lj!#G(UH`By6BB_f1sMCLb4C=QHsu&OOSKmd-#*%Jsn&-)>lq^Q7_^yMK zC=n1XQHR-ONTunmm;(X~A&sm3&{#vK$#Vp}Ye!U8#n5AsKC0To!~*la@Mb_CB~R_? z{p&RU-@{l3vx~zWEa64!2N$O6m?_mey&hez&jiplZ74V2-yA6F)_jX-Q9(%`aaE1r zB^(QIE3sSU6$*^ji+HpX-2JSJVR)^%qVJ7%2}K$iDHVRWKj96 zOA&eg7iH|-_!B-Qlq|xqH?*fjwbAe(h`<_kFtmU1-+w1z+Dj%`o@f)v?s%O32{h{9 z2YQjxA^Sor4}9tmcuZHafgPPI_bN-*2)EUzeyxjs;O9(l+GCx*p_#n=IYP(AYg~!U zP9v+!c=GnKy&+n+$!+Za$G7uG_L4)L?Z7g8)OE&zuf9clUzj$C@AZ@|t_0Ry3Q_Of zNzi)TIsCO}0PyNjRaMn%GTA>W-vaXK(D7tnIzHLE^CuL+mCVAlH~dPtl#2TJXW6UG zV@wUgM61oOCRZz2{&-}`*#ppC)4j$EAdp3PoQ6*3pPqUFALmb6IveRP&XyZRKYVnM z7#ti3pkEWALIDxWjdqJ!Q`Vm_I<`upDhtdsr#5)tqeqV-<=(&5AiJbS*tejf--kZ% ztO(#x1h(cm>_HIqQ=;6-aL#E-3Z8OKPfX#JQ9szcyLTjIWJI?hWb;Nxr?BODdJCQ$ zivExlXnyl#4RdbmDYH1{@@Lc%Kj*cZDt$YD7rVNw`0czyF|aoUr3@_XEy<{lFP`tc@pG&f^ZM69kXdns&c`-EDZoX5g$J=luR#9FHgir6>!&JRS`Z2@MH;Zg|v> z7wG0em^FG9X>S5{FnTMR?KmHku-SGm)<$s}INMIu3a-kN{qL{LtMYiZ?F;ZI4g+T& z;(c^o**5_e*q;Xq?-?_nh z*)bY;ACwAe+FP1!6SAA=Rx8|uH6ms~zr$D@6#%t4x3g{cTlXicQgV>OaHk>OP%4RY zMUr|$AGjc5wwKn)v{!Ox+54PZ7zdoO$lbW`qIyM7NjZatv&{epU-g?Y;Ou;5M_38ItEt_5D_b$Hra ztx!EGnVQ<2kDdJXNqPzo*z4W{F`)p!dnWKY)vBm)$)mU}a%gqu=4KAoaKLEbG%}~V7a9+51#%TDr(EAHk#2QeN z+R>{6lht`r`8J-*Kv8J^3kl-kVyCSEPx9&|q;BfhiW8BFxI;Jcr1zN4dklGN^x)c1 zKQy5bT<;9S6rd^}mdyAOBr|~D@m2zj|9UeP(oJ@2?h3vU_PujQq^>MyKyM}zTQ^A! zzqZHKwzkYfy|zUkLd~D%)_v>yCsJ97EHaM1NYB}@>FU_;AvJz7ZhAQ70eS2PTQ*ur zbTMn5N)?pFNZQU~1ZbB%AzX zqk=KYnX|MZK@$*T)Ttc zUb`)2TXk}N;Je1ta*+@FF$i!nq88-X8=MBV%^=tC@S5_Q|7_?LgUkUh%OIC2I@HW` z1&);osAnQZ{g%pMt-Z+Y`<2Fgu;;+=UiY4K+bM2+`wg&foq0>(N$34@>bRy)t#w!H z;Sl8I{GihL7QkY*paNEXd_RA^zwc6CZy%!j1#=F z1xwD)a47fWO}ZoE5j(1AN%hIA*kd$H>^XeulDG z>*&uMs5c!tsRE?&(z}C18`H+gGO14lDOF>swxHn7?;* z56X5Vkr$I(N>HcS^tB_f_=hW2fW+!IAE;ZRHp6uk-rYT3)Y&O}2G8=ts*mt_1b@S^ zJ4DZ>H94DCdvGdtMc?Vj1?Fqyn=>RXS-xV~dlb54m4mb9rR^FV&sWcK_T);btzjl| zfL+S(pwsCj`q^299Zyq%*mc)NXi(NH&lJaj$BJhwoH*l_&ixP6b&*EWK;lU9<_g$q z>LgmR>=W(P`1s2 z2y~fsQ1Cs5kTA$(yM+H^OV1|Xcx&c>#%(AFZ{<8%2SHv9W(hhyG_HMPmr^Y**k_3( ztw3#y@5l||K)~gq8=dfjw-I1P6Z$*Cq)#rUNxIrh_Hd7+b=W@2PV5kZFa*J;HA@?O zh8$QMgqPj3*He(0tqto~gNvhQmH${ApJci-+ha^H_xB_jG!+hR;*czl_5S=NxbV^{ zX_B9Beb0VzaJ{#w`wQVguIewdQ6l$fZoM>aO@(=B6Gv6&%Aj_1(v$3_*2DYz5iwYl znBN{x>!zA;dk|5h(};LyxLvhY0}%>ocZ5UDfVVo{9(~8yqrmb!Jt<>#MNmkTQi+6Xdifl{#B(JHt*A(UWS|WC8YI!B*ma%l5*N`=P?` zPL36#XV)?uU6V56W4qkw6YNsrg@m^?#%tZTiv7_dR+9i}X5!Btlp+GF>M`a3ZD;aA zRojg_;KB9Fi%I4FB@Aq0+J>{Uj3$b_jMlr7^gqVwa3TBStBY@Y&)Z{P1eMk=E)O44 zVI~@SXJxU`T`ynGKfFO~#;SOV@k1K7Q#zxgd^=j;GpWgJ$PLD5`xT_J?iHIDVcOIg zUd)$U8aKy`a~kSG@RhoXD7^^f5@oUX!^++E>TYo~@P>`5+hg&l9)ei*Td}R{X91;_ z)(^cS7Lvz5@b#VE>KFLXwD6^xd%$iSFJLE97<#BI2^XlZBMU<;`OKa{7BLyMFBiOw z5bePGwQaB$E#ebU_Y4mYIgY$tc{mHWB7+8#r5)rU2egK6ewwyi<7S@`qyv#F`Jl%c z!b-d7haVnpl2R|6GMr46B}9ayYkT%#5ZHPvb;K(2Pd+-O2!<0z0&9kQz-=B~V>l)i zGw01ngOsD_2B_(p)NO)2-+c5HRt+86C}Y%3CFB^D7L`#3^sb%A?Zup(QT-1-Qia!Z|TZBF)lahWO12QTD@mp9fg zqaydJCkOVQs71SDp}V!4r~ao8RHb?XY9R$6)gH;s9F$qj^D$li)bZB?Rkx6p8@dg0 z*Dv9>SEDQk6O3xuG;q|mB54Lt@IArdCwn+MVT2*t9(;s|zIzJVazyNtnrTQWmn^dVA49Dab#CDd>`NbBGYZT6W-N&P`xUUCX&yKHmjM%s#>jSy5$`ah(U*<-UQ%!^s zfQFOb6VIq@Q(CB+SNtB3^q)NrPg$#O7hx4n{aW|~2;}$HiHX5-ZC$JRErkuMoNCXz zTO=Lr(A^^9TcCVbWpbdOv$iWV9MiRM{pt_h{ZAW3{1!_vUz;Iih~S^|t7egLc1z?zs6Dv{$!y%$DxGpESWyTpr(Y zV6JBq?@go~V_sn$QloNb?~@MqOY(W@3UDdu1scr{FZtXgTc{~-8HydGIaTkRO+;oi zfuV?aawArx(e{YPj?qtpYUg(0pL}FSo;v2E6Vl+8scNY6(KT65ystxNWUdq*@b~72 zX^Ug;0n2<=qtE3&mz>Ua%9!XbvNlbW|Y0Hxh$2qbeUW z@UFMG21c%**|-k7scCG|n_=I7S)_3t=-7}xIB~qb3!$tUHx8^eX;KY>!LR$yVZkdJkz%c-DN zGfNR8GJ0e_&dMW?h%FM2OSMtg$y(2W8&m`nf0n}zOlcza?R~iF1;hPJr z>aB63&!I|2&;3(#-Bl_Z6{$Ry7*z)Nufk~tl=+MF$iw_i$D5=%2Y!xOtjK9C*l!FE z>@k3ZuPE$vmafM))=ULQsS5Q9a0kV!E3cX8dAMU%GC2T{X6)3)9ux^L%M58 z*nFEjXWlHB9N4NWCn{4p$y}xhHk}8JZ+03upDvGzQg#T$u(~s_aP0+`eni{(RG3uO zc-ce&;HWm=)qZL4lXPfmBO@%_I~<;*8~JI8Z+vAUO;G->=JHTN8MrZ&lbCAP_$dL9 zWW2HmVG62XA-xT(J@3*~O*_&Y; zofz#q5oe18+Ik=qPOd z$eDj(pI7@&dw-*xwHZcPuM4U)18a~3d$AO%Ko0*$D_YECi$DZ7 zioKH8m(#QQsWeEojuY2Hl)ezOPXas=uH|UDQQlO>WL3Kcq-sa;3za-bs@v&#)Cwy2 z7M5{Ep<>d^iK`|nJ@gLdzoMmI4$uAv+~{h!_~oGTO`EqkII|ckU6g#~pO9aLu=-q6 z&PUo_Sz}C|&{=ju_)@0t@o!dtS{2avcB$%Y#DGiZv^#;sk8B0F&_j|v?@Wn@NcC(@ zqcy21c;gA@D>QQ6d=App*4FV4$TTA+V}dV?)!F5eTP~phFCpSb^@|RPhYJ}GXZF`Io_Asc(U*p|&#B@v zxNllWElbMo=fA!rvMW^$$m|w7E35R2uwvm(_ZjiX-B6~+pXfwzKd*VNylUcsj-pZA zmwDO8quE-xTfw>5>uCsOOz`ZAy?O#qhiY5T<|!+92#k&X zrtR9;HIhKr9Cd2L+r^ct9lKVu%t{l>!KDE)A+?aCRXGZ)-^x}s9As~t zqTmo6-62EF=QRJ9+cEhsRY)2wZ$7gUTDb3uRbkhW5GEBz1#w~m@zN-K*S=#%VrfCFdp88c9?M+sfS`(`Z}$ zvacAY;tR36zN?r+;mG$`Oq@3F6s9SQBShbSGZsO;;G8d>&=ehu@im1~(Bj2qXitK8 zChPe7IFt+MM3{4si=Uy5j)B}q`Wi@(&PI7y!P&ez;D8vvy0kY}I6Jj>+3)%;v^=~0 zD%xdqcG^QyZ4xk})@o|!pjMHeIxK!mSk6ZX%9F>Np$N}$^SNq*_nY)WDHU4iDjOZ+ zGLku{lN)-&goy_xVvUN6yu)Q!Ed$zgNqRMWP2V}aPFbR6<&kasaDPP((Qv3~=1jw~ zCdIcRFYP_gq2r%n^LPlvlULMZ1!VaXQOe+t`}aV2MTujI>2Nyun$-dcR$a9- z(zg1$s#du{HELRv|y{a@`So>&Id z5Kx}6cKl)clv+4e9AI&^oVF7WuWUJS+t=IijcGz(VZqhxQj*b5waohq?bMV5i}bXK znLdRW=W0Tqh?TdU$}|o|9jyJw5xo?e$U`}jBAotWJ^OU=ux9@|QI!)L zlpj$z$zhE5~|8t*Vn+a4({5`U$6qk7gUaIpUCDkHtfpV*2?bYb72bVO4A$` zdfQRu4Q8qMxkB9X9%uKrbPc8|uX**nQ`1LP#9>KswM*6>_Z|-8N0qb7pkcuod(>{+ z?a!U)*AsRT4#zFNvnP|BuCKd(b7KuNz@0`!EUxEIGrZTq7gWpMo5D)%hs~2wpukZf z!7Ayj`EO>?P}hDT7D~bEO<0uz4+n5`9U*)jsaw{#kA>)Cj5epTyXef5Ze8Ej)c5!# zT)Z;JS!=0E@zI{o_0S2{M@M($rtLFt+Xk%zZQ%NmFO~Z04z>x-7|wdUT~^u?CN)*B z?DrAB9{)WbaGw2P^l(J%mJ4w`iU-Z?5=u+5t<$N7c5o9v+?_}v;#S*IBKZWUC)KMTL_SuZ1%J~@;faeDH$NfI&M9j4Jngeq8 zH=%)1tz3neN)^w|g%Du_YK-gF!k~VW$E|x*%=0QjB~6hZYfTUCwSN`x%}r^+I%vyJ7VS zs0CN(FkC6zI8|#VbgA>a*-oM+)~$7c(i-CDeHDEulnO9(zHjtB@3*X48U8n2(HG9V zV-*iCWToPk0(;EZy{S4cn`hB%P=Ky!7|m2Cr~Oyh&g%92CxrW=dYRoUSXbkI-2q$k z>{vW)z!hGe;%-~2f{jVN;CWs&-V{uylX5g6dCC;qAngO~#2I!KQi7)!&gU{GXm`-K zU8uV1xrFnDWwF9%xc9lGye%M$X7v))N4?XrM|-eGF@j;U$3Q__oWO0G3J#wUp7<8Kpo#}5YG zrb1YqGR2h%tP^sefam9`W;^wK zGB$})j+!r^7iZR%W`0T(G*>+?qL}+)5^X2drd2v@yz8PVv`6Oi+*iSs&oQ|k6Hj|) zvb$(eC*{YHy5}wW0k&&p7rK@!Efm_!+I?4h7otkpkWRTaI6=I$XnO3!NcP-!VVAmt zBoAB0f-f>%+<#c*ojPss+PUArBTqzURp(4LlTrKEAPZT$leEIVS1i8YJFjl3YLX?3 z<5yl0j=8Df=p?3&r*Eo8)LlH;#qXtJm$C`Uj||F>SZnEUDeccDKOMPHiaenXXz|Wk zs5h+B*xKv}=ixc>8>64cuwMVBT3@GW%KvYg>x{8TSBDNbZ9~z4VL$W2+ylq_>L<_K zqe=4&0rh`n77S5q?JMwz6Pp~I&n%P*Cc*qQXEN?Wy7ieh43TRp+U2Whu0(r3hkumt zSlX301;>zkmRJN?Fq&a(Jv%51tp=iI^F4{9kQM2(i^my`x+;FC4-eCt?W z)jnZfw&mT5G6=^e)>Dbv!y?BX6`VCpL?)~ze=fKR&1{3+P??|hT#VZf*IIeHt(Cl%HL>T$0K zQR=hhhvDnZDe!E~^^`d=0iSxGcX(U4PLu!Ot~)P=kBL+qh}sNLT&f?2%ThHTybYyJ z@nCJ|&oks8xtt#{BRYkJi?``vn&h!9mDtDCH(G+U2UfRH%^$;pL);L zcHqDSdWS5xdhx$$8A&)pfJL@h-}Bb}kapCs68pt$n~Wnv~v5O9C!t zb~gzH3(I&edMj8ta+Twz)QtR?GQg3UDeE0mh88}3t!HMJZ3yQfHD-uOz!670}uH3tDSj3 zn_Xf}rojX(FtH`tmPmI&b8{^rm1Rj!B;g`jH|1y!p3I}9m{m(Ad7WkBR83yK%P`gq z>SmpjlCQq!*`3dX=+7rfLmRvwj%oYO4(v_8jj1t4PB*BUV9!muvqZ1zVa$fcu8PQ} zImRku4tx|kkrTvvW}82%8?!RUwqzd{eLW*fL5K}*=uQdH1pO3pw;{)zLV{9){PtdV}CyzD&?;<>dC+7>Y7Qw2h9iYgZO6*RVeggCG2c> zu9B_(ZKNh0fo;~9;Ndv>Xz`m_E;{oZ30sQW?4u9&Lz%>7^AVA#EUuJL{8K%-B3Uik zDC#_V*%{8pmH~`=LC&A|x}UY+gjnq+4+|-jZms+kSl`E+cGJkDZc^lZG!2chfhWZ(Q_M9c$a}D+ia-AKO1Yh$NjkM=jTdjHQM5)9Q?TCG6Z>w#aP7FOBIbTvTJt-1zkSHyHO3p;@w*x^}AM>7sCWUWh3vF z>t*r=L6A{=8J^0!AJ?vRuW>D-a^-m1&Wdu*XV^Y}fX|^Y?<(C|75hX`Q87(x3%Y647qrsbZ6efa*aksbC${%vZeYpG$+BlYHQS;Wtz51 z^zCv)c6&XIxH<)2-#d$RIn>5=_WVN68=#{}MlfDSF(Z_^} zD`q)pjK=5d^GOa&MyyIF7I|P)#EdVYfQ;M#OUU6$wC}SMy(Iw0wa2MgPO3!MVDVnj z9_JC;%4s2zgJwsS09ImoX1(9rY{0Y1?^)xrDHw@w3s~E-&I6s>`wqu`@@V%)6Hpg2 zo1?%Qih7wPrwHj&STL^nNr4HL`F-7l9>C|}XoWN{zNNX#?i`>+g*6nNCMT$gGN$i8 ztp4FmczRs0MKfi|*gtnXqpuhNiH|>SaDA)clj&GoTB`1Xen0 zmSqiSoTELqq5#B;&p3@X>bW1@kWi$DbES5@BC1WHZ5G1Jlho)>i$BY0TmE99j{RSL z^hZva3!%R&HrS_9K}<~TdZ_!T?$!{trDf>4qWy&cF>J@PFQX(mAc5M?yFEf8Aa1C4 z*CXwbaHz5vC`bO~hD=2TLi*>rpSknS~l{G#Q? zs`!Rx@v`cJ746Ci1XNUo9^nOA_314JBo{iGGGG)HVKRf z`k^FXuf^jAr#i1odCs!c1YIjR;ldRiWP2#BBe3nViHkb%;bP(pw2-w?*U7WOvtmy` z1j}ivL13TH=-ENwz76yWKN@soqFoaW998EFQF-3DWuoiOou~&0#YDTHss|uc5|fH% zl573%echJmXuOSG`?V;cXPmf)2|gUeMmb@e^-JPHdgqjFb5*u)6W82Jk3c9iH%(xG>$;Hi2oQzT zzpV#g16)xgy#;7on#{#0&HIcme)0%I}otXd6tIz70Z^M;z|N zgwpIM75H+7F)7;6ua__ewfColbRCX#T5IJ+yARQ>Q!mb7MT~8#T+T`=VkoN|$NHxo;6v%!NB=cl?m9 zcHigJo6EkpM~8&(`kK#24x%J(*gh=fD{^8J7wdCZopcyxiWMHFqueF2FAZ0Huc23B z<7RpqJeJ?Pd)0dWs6mwws9s7CWB}wFxW@gk6o@COeY*P&n`ifh$nKw#bybIrYC2T3 zRF5P~%XZun?mT^WPii=}QuD8KYTWQN9#-)sPW5*go;2^2iB;=4WOqeJ6pOXJ{U+%$ zL;>5@13CEWs&2!GjqZLgdY=z3;mzoXkpvy;q7afsHk!W~yw-2E$M1|T^mYB}6YMx- z+^WbolUXR@V(Y+YP+yI8<-%mQpOrQP`h$u&a#MJA#XZ(8NPk`OYBjLREmm1HdC@GN zhPuOj02PuqF~KZLilXy7P;O!U9hP+Gi6I44NZ-`^RPVl*o0y&fKJ4*~;S@ks4}F-q zeub)upfw!XcSEJDC=Y)|SzM&op!h}KVe1qB+UZ0%pyYdo=O8|<@-+7ypBgNPNM+y| zXAmtHiu3`^rn?mt0RBT{GDJiUoLdE0pcrtJ?&8Mqp}e-tK4~fOUs(0MQXI9_$j4Ns zmotGjdxoYma69Sem;l+D(c@g`o>Ue~w+kJU6Vo{CYC#>tA@!1Vvi~o-qId24(j$rR zX8DGyMxsX5Mdeus0-h@K{W|DQG<3|kKUNpMEvf#b5wWmcNUcohzPF{d#97|FE?r#? zVb7-~SBLET+tw+aHFFnU2zGblNb?>q+6{yBSJhs7N zRNNc_NTqK*tmziVt+(J*hmbww*K-P*0EnU;fw6{gwlD#ZHO;=mM)PrY-TC_$%Z4uD z%kobP>)+zjgN?l4pM(e80*1vWBS0yZd2%|#Wrd*s!`@qlRn=|%!-}AEN=r+3BW$EY z=@1D40qK@*K~mUs<0eEv1VkF8q`SMjyBq#<-{(Bfdv3kY|NZ)Yc&}@JSo>OYtvTly zV~#QAZ;mlmRA_dg?H8N=(f%|~H$l}jAA^rH5yZlWU3xKz74-C-#S26 z-;?@p!Dga}_uAc*(hds=%7qMxcD2>71=mm!ORHV?Gc#(6G1ceBjsJ41#6=_hAqiZ*)~Mmq#!&oF{SRsL zouS|Rbq}i&z;vHZ_0??|;4ieM25Uv(^>Y3{%!EgLu19FM3$a%vQ=0dnG8G35+P4bOI} zwO*)7O~juazl2k9xIgDtC>Y=!FhDYF6jZ-I+Py<20{5iFW#A(+3j^mTnEU4{31ZvPqk1L&6zJOd*WL52O2_x^0xwzWFBx-vwQviALkY>GDM!$P2!}PuhPF-$o!yt9Ez^K7esG z(x6>o^OqT!KReEYcf7HtHV5V*_znyZGZ+y`?2mS$h=2!5G#Qz2it)hTXEb#b`v>{n zA(z5sr2KVUCg|5o611a`au%uE;Ob-<+}liePJ6nkfBSQO^*H_aXNM#}W8u;_FA*o? zUKceJFfY|hULTUK0)gUE6EdbN;rc4T=FNv%*r|0yTz_vg_cz^NvtbTdFqNWRd;T4ssqW7m-Y#rWT(LK<$Fas1f}(eMOTKRQg+ zUhe3W*DZz%{lrrg$hfL{sW`#*5&I>ZbOiZye7gJ`zXxOKq+Hd#f%uA;%*N-x5PMn6#~O4!_nUusPp)-A>K%X^ z?fhm#=8?VRI8<}%O>E9I)-zRU|DQ!g5&9Dtj#cx%%)eh@?7PTI)=y1kkxBE-t7cGUYrca2y zI;vVzQbU@l@M8*Ht-U zIyk4qw)T9b-HS`sr?NFzc`DWL17q~2AAso0EraAbIXk&{&i3mcP{gu>vpc_M^Kr&2 z*x&NEQ%wuElt~pQYUoUBb}HuaJqDhkqzV%k)SRmpU_+?z!Vl1&Wi=`8$nBvsuD|;G zqEDZOGd0D|_D63r%wy+=>3J45Bk?kE@{r@nW?|h7m0z*}BE!yWem;2*8Ci_ep)Yi< zfrj)p&3R-4Y<0kk@0j7hi?K98zGpxsUsO+SVC(qCVoR@J_82vKyIh~S`lpDfs9%n% z_$ann+*~_ajM&7Gv-fHUY$|3B1&R3YJoTLiX5FDuwxXB9(x^pTkCnm=lZwNtaKt@O zCDQ6AX|n68lfT5okO@&4F=`5OGDzOZomM2r5yDYbFtbXSdv;-J8_wZr{|@e^ z-IY)+^&|AOk`s%5Wh0SCoNgPrIf8sVTzF{7^N!Bhp)?$U_r+q%zlTUWvwMEi^)rP? z&0d8^9CLk0V+U1P;3_6GuxfT&Yhsvf`#Kf(=iIdpt1E$*F-oe`v-Z@#D>qSQxSxX5 z9OXbmOF056CKts-t`;;VooehO_qb{cZBN*Bci9^HVYId4af!v`HgTJzDRIh5w zx)RM(2oY5I!Olazx7+91hCZ(>@(U;hi`hI*QfqC16kENN=x`@VmK7&w>lg!_{B`jfl1J z{IPa4u4|94x!GS`U#5(ZhsGm|%D?AtlvN)1NZiz8_tkYOwftCMlFGFuQ;5Cx%&>J$ zox6xqyO6_bXUHM{Wu#TbwR^)%uQRRdiNlu5cvZ@r7S9++Kq(>>@%A(k?nw{*X>h_K zQ86nBEc)Lp<-gs!@p-3;;I(noqQMy^@?7)5iN*D;VM%dBDQ7b*B;^?zo{^_EL36|v z)AM*F{@M*@Cf&TFUDEF+cAXzh{-vmeW{L=>W9P<0izYI6OFNkIDkp90YW5}-xY{4@ zfY9Cct=1$H1DWdPKA3xIjPBnh7I+W-z#d3{aa!RXmh9E|0IT+bL_6eW#lJnHP%^EE zN~7YGY@!Jl_++?b>M8=UO_*z6>v4^>ShL-jn&EURd@;I#|9H~6?#1r#<_@sY$3!ES zf4cvU_&{Ok;_LF>OTiT$w+q@d`3n8XBP11GRM{SDXCjxZWmk{HEN%N1cBEnX($OQ; zs0vZ30etWqEl6(|k}#)*r!uPzJe~@bLc-s45zuvOl0eG|acU%^sXag9A=IIDUeJSh z!4v`;js>lgtIoSQXXat%3t>;ooFI$DG26Hh6qzkTtFOc}5iT}4=BL5kK)hC~PNFAq zAs&o;9#^_+aj4uI+J&LIfg46^1xyw=;Wi2B3zH6sQSWR)0K!OZ#> z6|FiVup59KPf)PODs0XaSLovrDnd3t%`2EEt|C^-jF8fpv=*KzZTqI9xz+aY${7u| z^71cBr&H;euLztBr>8$B#Sk~9OM}yOq#v2~mFZb-7F5K~X`w~$0I&2vW$EAE%r>%& zgjn&gVNMNYCb_~^k>#U9zO#_Y&J2N&??j3rtz?#_FYzEoV%(miLl52Y_;5JcYmQW_ z+tGI$gu{uPGiM!og<&YE^5eD@<`wU>3NN?4XZ7r7Ykm7OWh$2A==LbYbAvm#F=eAT z+7wOkIO=22`?#vqgBeB7GG#($n8S?c_Azbu4?)tz$4g+H0jQQ3=Kb4P|Lr!qzqu;n zvn)6<&&-$v=h0gq*7Kp3>p#xa3amhCJf8g921-5l(Uc; zkmr@&pwkfSt7|4s#alhoap;cJ?mTI`TygMnGTRz_ZT9B%Ubb-zM5{>GmhidXsD2e!lB9}P3JE_HZdQ~7hcv{=C zVEHRbYA%PrP1CA@PK}wXD}60Ko~jYQ&T-W=f(fu)8#V6JWGJaMbG67xAXG2!jf!D8r#g{ZjT56B-tT)EkK zKU`ZHfD^0`<7^oB1dnr!#})y_Ph!;6sa4X~v1VpqIgP={PA6HN7xe!UA<}zfd0-=< z1&i2m#s-2(_K+C$!`~z(SacGutVt(o?Gv>ku96HSv$ucRj4se*a^c`SG5p@^AkS?& zltnBl-P8tj>e#f`>9q4WYjGR3&A4#_>+Q^k9V>WzW zvjdrb>FWchixqp6R4=^Y>Dy1h3lYIQzW2kEOu~8s&f+y>}3AtAI^}S+7=A%8w)FuKt@_K?}DMh?$STi`I<5 z!YMBH`!dB?I#K2alw5_ev6?mQ>I$TNxBno zcN`oboi0xdV*H_X;hU#zYGxPUq+Y=HBHkhEqb4#zH$n_D) zJh4Q-?cc}XYd{4^rOD9={8C?XI^Zz`(Y!3A`x~v{?CSFZlyKJn$k+t_!K483O`C(n zY5r`t3fi5xA@ps61uOV8{bFL&P3$;{U#@1PlrK1#_86LUPUs zAb1N7E38iuUs1tvJzq>XWhJP^>yy_lrvIlF!2fQ}{^J5UaqdSGu3RMm6jA=As6}C* z!x0qukw5#BOot(YI3PhBi1EL;z5#~-yiMH2|E|OSIxCovzEkfI{$#}toUeLJUPU&- z`QO(GpUvB?jp;wd=4bdFzntt67u{0EU!W}+92|Kd73lX*l7j`=dzW3p{1?P0iU;1w zcULLmpF}$k1#)|JnA|gC@#QbYRfv29n=@yhdRdg8@FxQLZ^HNz;1LPCJBI37K^?Mh z%a}d!VyhW(hnycYHN3tVv;AX#{~u=~8}xl_VnsKR>Q4{@T=uN0X?7! z;RKWzJBoyN9t$LjC9^N&9|jTzflE2X15{ric`k4zKfRyR_orAM!4XGcf(&;uAYcZ7 z^A2NCof3bnhPyaz;2w7xbmuQo#sQ-ZRgMPF{DzzYT$K*o3qmQahMPKgtpN`{i(^B$ znEd@W4KL%s-;&(G0tDz8V79ZW6$8s}jlf?ofKr*x(%-!cx-x@fiF9{2?Ej%(GdS*B z@R_j!$lyg3Xy+Z~R8#T?Wkg%yDyc{%^Bve$IHG`Oba7Qsv;S=8gALkI?B`biW|>id zcH2hG3IDWuY``2hrL!NvWk1~ybd37w0;r6?&ldcpf%^$<2MN32dKx~d)7Ffc{}8f& zubuQDxDL)}8-n|HVaT9k8b96Ce_92}pf4Um-M>`BCqO^~T3U}kyQStp2lp`Vy>SDB z>Kg>tCyqxO%m41}Kl=X($FFTdIRjpTc?`k_QA(uZ`e(Zfc)zvb-Zj-gve4j$;I8D8 zf0}Rs{7~Y>4(WAp?*=Huy84)N*2fF5@*XxG;%? z9|wX-Zo_j;YCM;LG5W(P7=n8hx|MIV;QA{Nu<;3E2pc-T@2H+f_q5ZfuFck15!}bJ zKK6zeR{&xDIXVUb_*3ZIU40sLOO?OBk_Qer(+nN`1-_z#i97+pTr{G0iVpKjdMv8? zi0eoiRuzS|ANOs)os8bRb{KVsVF z_ENuMMBDO50D<4Y-0h8t&-L-BOP68f`cnjG%dKqfIi~SU9>wKf4n}r_k8FTB^rh2p0ChSq{I|_ zOU2s)KOD#LTHO3r)gyd7q-L{{Wa^w^T|OQhnDr9kb2c9|4cyla0zFrQijA*%m~N~) zvu>-f9kBCu098rCYcr_)zv{2W+OB~aRtW+4y7-q(^?nOH3`gPq6}!)QD5%3ZSpGz4xgSEGYUdcnT6c9|GpN81wXHbfpXwD@3=5o^ zG6*7@N~pV94ornv6}L|TKcB({q$Q7*ZJ|;#auafhfGQZ%Sl)Z@1FvjHCHSZ8-g|(o zS#By90D6)xV7RrxPXer8%U-01W#!ZA`#!7c0=*@kyMc+BHfS?ny8(7g^NzlXS@vSD zyLwx0PHhf51)%^*(R#cjx6fJw*B83^XSYT)ji6l>@NLGw;#Sj{d-ri%OIv7L498)wNS%&DH}4@i#oKF;m|X*kQTPh-eA?8|vF2?G zXA2e;LOIF<`D(6U%0Jra)20fl4?1{pL=L;ScTzoe>;V$4uW{mim@?b7ky+L;4>0<^ z>9?J%;qFXi{6m%muk6GsE;jP2K-|;M>^e7sHitICH17-Z)AgLRzi^R)(t+nYRkPPN zGc?Pa)u+4Or&TNv{i_ASX~<#MBH!5Sd;F8|Yz2RM(=;zWEs6iaq&ARNj9xwM5xH_i>(ptL8EA?PQsHmooSv2&(@js&!3wwgwV4^vHQ1#ex#ihU=0#p5t?=! zjJvs76$y%n?e%)k!fYvWbGCR;=B`+ctaUPqws#EC9%eHijBsR$Qd~SOniblqT#V!i z%Y?T04Bsg`&$dlZm>on*#joWBLI$wdq$<6=d^_%A8g{5PSWL^`qwcrUe;{F3j7_73 zb_D;j!-F3QKX;k3Z;kYHsBGkXb)!OlreG}RE&A+ao6LPJ<3-I0ESpv&Gvf-TebdqJ z!W37%mw0(KR%9f1*dt_npe4fa=_p6!n(=hJnl!f2o5#H!&Fh&Y)9T8!nxNgpua(~@sQXxaoj1d9At}%dx>SB}SP~Gu0~H82A>$6@nd%k~Z+5x!EUzK#U? zRR#xxk&qO6>)tr_{1Qg#x{_6-yyt&D{8C?cZMad9I5lQY3ps_QEKyVhIn~kcJnDoN zk*l_BuP}bj97gS!u6CoUc-9|eOob=LK74mw1}FY%t+$I^grvw3==gZ=>rMTH9^uoe zXnjn&Dn-&ATc1lET~7BggCOHx)!k{66i}8(8EXP4`%c&`@~5HuRJEi>pSSM{?TGcEL=eeXsB-OF`AnPuiKg z6*%6zKR3U{o=SXEDO^mlAlr~2A3(xk0EaoIcV(x|XbFUbUL09zyvyge(OyqWGkL+r z@MS9~dl+N4&l<)r1DZKpE#xQ(u7BPVN{_}PsiV?#hCTn1X83T1DQ%OKDRV6uFR#nz zI8R?8aNK>Z0Jo<1>E+|?^V}V+i@m}4m{Zap0hMBMij7FXDbD-qnghSWPJD3aAr&hH@ydYJ(2}@rq zUv58ueI7{)I3pyjd!+QF+8food7A_4SWfiC)M6`BUVgN2%2$u1Gcy*|d-j0>YwTya zm&<#5RxGaBsYRA6^qKZ*E87*bY%ZETPcOF^nej)7+?1x&*bef;kW1k^0*!$!?23b? z9>s^kI*g(O^|KbY`QKm1N}C1d&68c^9vT8qh+t8DDE`=AIru5C$z@6-RFB@ zzedm6k&{B`lD{+!IU>q)f1Gp_98gHj*f$@%M|@43toRUu^TZs&)zDlxO>nUBe^2G7>JX5&vj(c@r)dhM=y&89cCA0cT zb+#teJY4C;50?}nqO$musbBZxr%p%w;A`wV&fkEg{&4AyvA!kflo`9f8ME@!dLeV! zUcR+Z<7DnB%*OxyRbOE)3^uAp!7Xx#sB~4i_9?W(z$B@{B9Iu{KL}Hi7{`nkxN-$_ zf(<*vxI#h-M^xnMD;0`V7U1^?k)}qzT=2vUGJkzisf(6*7x2fB1CcjY4`XEK*>e20Sn*d2O}FgKnoG@N}|pG9z!Y{Gv&+WFT)N80&_^YBQp) z1J9oFkH3krdj4>8@?7e;WH(%3(PK&I0(dPt;o<|uFUA@n^(JW=9bM=c1)%acm(RFcdgQCyCYxFtk2Fx%(RMFBaNDy5&R22D zu;KF*M_L- z>`>T(*v|K8{a=4RX~Q+@F~{ zwBwsWHt{4k=64Q_YOK2Rk}Tc^2<+`Uo@c3B{C&*2-sj6nE=zSMuOPTEX`WN;)~i4Q z+B%h&@fyJoI-W~^$(fmzW{%3y;Ff=}1UvHp=TaRS+pu7tF}Tjvf~br{n_NpjeUadM z2I6Rvn+c6kcb|{T>QdUeg;3AJkTCB8+--q(P*e-^owT=rAfw(V>cTEw{*Zf!w%HEU zT-1^r2vfFP%9?v!azbxbj90I z)vWR!kSgf3$hY25^&EY?Xrd=w)?FmVP7TD4uWUx+cJcL+C| zInZYO$$SU~RDNM#%U_0xglBG>WWm<#fJDFBKS&x1b!^gSQq@RW;9vJ^Zd1EY)zt8O ztZ^0F&ZwwVV}ddHJ_;I!>tPDb&7|!DO)B-npjnb=@70|%jdCp9a5?dD07PG`TWFC6Z^z0xODVJ59XehE`*xROk|k zGIN;rA2%e?ri%h`7ObtTCa4h%_3m+cw@4pMcr?jO{$l7FtQa#4J@7C}E9>MKPSk=# z%B;pIg+2dpXeLO&Dla8}&%Gjjg>zkO zrGy_JDXm_^ou7y%w;Ds!oZ!@L0?NfT>k6OzS-&s*CS8g7d+AY4wl@vZDq{lar%|Zq zz1}A!s+G0lV1t@cVC?r!R16v^doOu23^~Kj@Qt1s_9Lr|HL%U&-Td)cRY_t{nsILm zN9GaE(|G9If}qdCmY{Q~3iBauu24N$yhDaIbNP7@OMdDFgVm!!C3dAJpM5Eks}Z=Z z!WP&wrLo1aTZz0HWgmT8G=0Tc2|$8kk111K(rgpTk9}&ELV*qub)dRmY92i#SOOyP z3O7PM_KPy-^#IeGu<{USfy1-R;3)3TI17=|%qfm@CYsL{&$Z2S%t$0ZH+kKx;RkX; zm87@Fc^pVSx*5_MNFh@RE4zKb$jtikYwjt{S|X}$(?bj&R?M?3TZUUw!=u;&sA_a*yc8`u-R(;gZXn9uv6?yBg%5wDv1UMVEOHVG1qI$8&ci*D(dX82q! zkkVXbaB})zIx6>5W}s&n5PV}MDrD7As5a_gr@^$3IeYyCfkZ(`s~SNb;YK5lQr z#J8kLd*b)OQr^!S2d;8kSi&B35Rbgb{5Vf41}U*?-OP$I9WYydN<+|YsRk85GxbEj zHkMp%u5G+y04;XRgz_<6lJ+7X=Rk_Hxz->0w1-*~8ZCt3BxNujOh?_{_A-|}-2Uaq zk(3S)1a)NS^zF=PEumgd56;8wCx|aLm}1l9ZA%$*rdB=LxyuF;QBoao{V}{Ya2Ok1 zObiT5^HFLT>mAx@6S}szGJSF0a8O+>>4YV=!iItiIKnrd>E!NhXxjn|-ObDfQ7!xs zN|nD%taBNigkjQFw?*2Uw;1iBO)N8={iUc!Tkx$WNhA6Wr_MbeumuhD9GtL~x5dKP zCYtOE!Pql2TA*=avvlZp?jSW2s|vT{p!9X7Jx2Ggc)qt(V_*Wy>RS40lCQ>Cwqwq` zK{b3eQgZ`8HBi{M@-wUoU+f{CLj0^!G}yowT~$$`^DZp2LhBYA)e@t#Rf*Ksk0_Rl zZiIv6jtU(gvJxK#4eyKhsS=w~=sENK9H08cMMGum7lQCD_RT`<{OJu%t*|BRMsHO8 z?)Z_PCUFsvFnj)xc|RmSMkM7QOzO|W)Lm!y+q&uB$AI_ej%1loN9U+@^626a<*7j43; z=zs`wKes@}&+`miBmozJ2U2D87lFa-Db8{i&tIHL;DDrg+aJsEEIZF=3?@pWhBeIT z3Lj&XR7(iB_mUvmugb2`i)(g#MpP6-;+k+n>k_e|???!}ssZ3^0TmiC!2O$j7P0 z2QENF9A~990^M9)(;N*9A9+VnP*YP8Q9T;w;zQsnlk?DQiL`AoMzVQnWU~K@VZ@b( za^I519J|0MD&lR@W((GSXcWgQth#`q%cMHwp?zsZ|Iv;6_A^yK^!*TS*YZO#)|;fg zTS!BOFLpp=7%FC{z?Gepe5ngm3#3pvqE1v5)7V>KyIC(rf~=+j9uq%$gVwS957oO< zwd@;mMDf^j(}o~{`W~L7<^d%iU2L>cAQ{S7$*Ij_>*kv^7r1mLvAa%Gbjx1GW>`tQ zTDrtIl*?Ao@x~u9Fy!A7%Cdejq@1P=U@h)b;;iT14cbj3z7l%LB*)N8Ea4;Nb`xP` z3Q2~~AImHqO9QCC2DBq8-B*d!pIjUJ4&kqQv@W}Nt?;@OQ!}qBX}bD^^t4-P!f5Dv zE25HPKE4`jpO1~Z%PZVJ!T4pHoYn#>U%{84CYueHTXt zYy9Jqq(L@sgM?8pNqN2lA!+t$Q7y4gMw?*+MrN{VplgMB?7YXwbD5y5ua3vHNuXh zJ#)fPB8t{@H>)ynYz$1HTd`DRiLg&<6RdSKUP~dEWqhnm(Jk^1fh(&Kl=wh!vYIcANqZKVGC(6O+6C6(xE)F-O?W70@5x*2Avq@(1SN_Hp1?zktv~qS z;ZE9vphnlvyv(l2cT9-|4OPDOtHPxYpN31dCHtJy&mLf>?IJpM$>ephV!@eW?zh*c zrfRC@cP87ANM2{4kX|s><5j7jxYH~_2pQa~3!ZBi$__($$Rp6z_p23)Nx;p| zp#%i<&r`kPQwZq585CM1>tx-Uj|znB)*l&M$z5ypmKN~09+nM+pQ|uqOn5bj-b;>> zF{n-w%FSd=?O98S)l7B|H23rLxd@cLAvIijwBJK9YucfX2Pv}MG#qy{k%lemO+7Qp zd}`g49fC0il_^Zx;{p}?)gy~7p{S`m53T%37j!o~(D)f;3+jx0nW$Zpb%m(cBH1JP zqX-X!_K<>H9d)9lfxm<-!?zOR5_|XTec#p9VK4pcv~(;@L-PD2A5Qdcyh=hD3Hu3) z45H9gRz8!VSz^%<(s?e*0i>4#V{{0)8_4fIjLWU-yUt^Bs||OHhlS@df~H?HSn&;a zC1r>W4XrHBb+$=4Du|FIpDGwd7bhK=z&<~cP}{6B;x}jrGb86u?S=vODm!t3ZL@Vb zN!!mf3TQr42F;1!Y-L0$bg)mbH=@ZcA2&&1`8a==yNBDmWXqy}XnUFMDzl-Pk&E9t z#u{BN&hIpn0x%x(WYVIg+44X(1R5vf_*e6jC0ZCGPdZhNOePpzi&5zEC(9e-I$XIs3ZNZV%ZagJ)U}8q4?G&ZrS|pHgz9vjkNEp z+wOXo(90EiYvOpS5|Q}kPk5sfOhYIIhiy|B5#vZ)o_n+=IJ*ex3tA(@O6bGq_}6 zx3dW;GSA3gmm^saLWUM>n$t8|do6;aF`?3U*eI^vcXfrW-1rsq*Z#BW1&7rtq5c`< zU}>iXhQ<*x`YiHN7Gy;87e{hv(y&i)P%2d$@$bjhxs5j9xQ9s><+_?s6p`LdITpzy zww!A=R4OywB43YidPHn*aZ{Xa$>JVfI%l3as+M`Vt=m8YHOOY1U@#@A*l&?8Td14h z7xgS^NeQ`ZVS}_F7!~Y>Um*6XWiJC|t~5f~I&JsWe9N1)zC@OU8?Jd<{8i&8k%bhk z$0@FOjmVPibwe*>m*;Y-FE`|O|K$y1(>?E;P$(9BBLpe6XzH{P9k(k}<4o-6hTOPYB`59X; ziMTg`ma(M@=F}Km!h*&YD5+tu7u-fH#LY@@I_=qo$vCV)9iUYWa~c0&sMq8+(Y(1( zBc6%$#PG$68TK;Vpm|R?nq0jpy-%ZL28p7W{P_xsoLZj8L_^UH+??|JR2VSr-R|qk zyKRDIVcMB70lfOGy9=`S(0q)HHY>OWvb65;A952Y6=V>EJM!K(IL>^-Nu`SnAT><9 z9S-lR{ocQ`AKWR8q%_k%@X%&(-WlZgyh&M)oMGu6Vs;&{3-YnZ#PDbZa-!48UxpI& zLq}ncCAmd@ue5u}j#OpVA4U4RVFN9yIOl9Uwk%3&@aXq>A(>xUEXIRku4!H;a5KSuEr9M#!l0$ zt=>ZM)ra*}x6Jv1l4ws_o-U%MI>=p|qJjI*zkk*?lix#6qwR|4*V2U@yt9l=F<7N? za{pGLz31=CI*sQ{tu>a!>sOb1kv?8zM01Sop&&tjVm~b zz@dLLVt?;b{Fu62DVGYREws1VrA)Vuc)ES4d5qE%cg7}lz-DlB!AZ8!g^aPD8~q-X z)89@9i-o-3M5vPb-H?$*7MVky1__VHe3SyUCSH5?V&5T24vmgJgWdx+#~>He3is^F zrc50PkB^(AB#*gX0W~lG^cTe^<&Gg-NTd0F7?IRxF+{8YFNaa4vtVjI-1pTCeA^Ep z3XkStha(PQjWMTH!E?`+r?QP#3u2y|K9~;?>asx5^AEl9H^s^90+KNDCH+$65d`OYS)PicPrOe zQ+d%~e(fgjD4xKY*tvG6rn|0DA_myI=hm4cC80O^sdMcv@4N(27tJ#vp>4msZVWSO zH69&$ek`w~8}tFjWrP57s2bqpu7ZU%8kK!GT-!ZFmtWa>acn}b^bmS&Y3vnBhu9*0?%cny#)AdCc`1jwl+Eq#Aq6!`+pd z7|W?fo~(Z4CVDrQA3qoJ0&m)CNkAwj8~qdEy07_O{5PU`f)worw3EXo@|_oQhdc~c zpC5>Oa%TfEK$@xb9ZJ?YoBcA^PKQR|6RctaO0|O2s+>ph@Q9rJQ3%%p6ymr3hOrSH zVJ6DFe%_2gb1*~V9-H0{tz+wm^cx;SRwK(68{h`uYsjeXJ=SAY(4vXsRb-txfACz0D>NnFoG?4BZyHpFFe7O! zfYftOsRtiS3eqpG=ehaiEL(=HJWnzKP}#)u^9;)xczl+TAN}zoXF>Xi5TvK7&~A;_ zLq;8%H}WvrS1Yujuv~wVzt2*cTcA^K5ni}J&Q`gn$S|>$&G@B1&66Jq^sW=qWw9}i zKj+wuHC|zv<;T<2&}#O#SVFq?mM8(Ta{4&=e&)tQcIV?d~y+WM$i0;qunB zQ}fxFVH-vk2SaK~xrPgLVr*5QmU0{|oz!ur+N!n0HAv}(7 zF|SZgiTrh+q7px%mfM48tTKY5B`QjeaWH>Qi?w?(t$oIY=rqzZin|6Z|xESm?E- z;zC9E5WX$ky?t-KGiB;eN@LQcnc6myHH|{Su}pOlQCiZD z{erL4mfAp1zTL7$`|U6RH}*rYM%bHqw~?e`-s~;1aj!Vud9ykHQ)h6-UR;--WjZ)9 zMwK!>#VY@(k-XLh7h+&HVV+~W>u1+Vk~O0Frr*-J5o5Mw+K)c0K)k5_KB>C=-JE$- z(ktu{yPxbB2Agrlk8K6SV^0nG-;y+_+lh;n%1-#I04HTIj2PC?Q`~~>i?m-a%@&Xu zV0HJPTAKYPxu}EAUaXf5W2bM86xAI4BUaAbGxw5Re}yrYLdu)m&l8STTT3A-bR5Y- z$i8lLQfb;vGK0}v6T|^4$W{SR1?%>cJJl;w+=vKsY zYEXRK!4|3`T@v8N2&3Q&5G0=u>}z*m#Ob_Rk_`${=%=ZPDr|*Ge^%SH%47*k82e6U zO00z!+6W2&8nnoIb<&^+8P}qW>>o)9qOo9VP4ab3y2aZ zA8sa-gsH!fGY$t+_a2vkuuS&mjEkr*ji6cp|kZ68@186xAK3*7qpZJMI++zV8Ov!tbZN8Q~(z(@%^EH*A zhF3p0CX*zTsKu$A9f?T+C*b<`%bIO)>t%hrHr@OlV%2@4v5hMe*dPiNWN}N$o!iIwJ1mp z33uoNQZh8|15;AQkh-OeTrHe@OUuFSBAKb8O}?S$8BQlD^OQi@s@e z`FQl@eB7%UX;QU9{$?rPLN1r^43aE6b?gd(DECxnd_p>EX{NSUa5%N%O;a~ErnoglZ<|pd-kMM_*?U~= zR8w#dS!gM=^!&>MO?vue%FdvsmW>0(`j}$m)3eMO>^dwPEukpBw`L!<2EwbpW8HR% zgvT!0=TP`@o~mXlNZdZL{Y;Kql>wrB>q}Y(--V>Y7=Vas-7TJ3vm6oWY+`i$a+M=( ztDT5lpI=n(5LU~hre6Aqv!(P2bC_L_AHm&Yjyf$V2I-{9u*Y1dtkuCdM{ff+LyHXj zl5FrGDEt`4sHe*5iys366v~#alLl~-osm;3d);4$xpLLFjSc>Sjv}HpHXgklX(*W{ zbPLDHD_4ve*7LX@uhn%Ns|*C(o+Mx#cg$3eO}F&tK|Z)@Vk9yMSw z0S?W9h?k6|KnMSv^YA-FE9}9(@-A4-@&hWVoJF4|l3zeN`Cpr}*bb&sH-v@+2U9bu zF?S!L&oa#>gkbuIhEq#JoAD20DWs5;5OJg%wI1BR=|gEycRDakQ4LzdAweVbBm4C7 z%zn=vE6=2}>-#fmgb4(bI$+kcVMnK?^L~hHBl>*WW8( zSpC2~yr)8v5s&d$HBg17+__BHCqgsJI5|tAt!PKzc)F0j(G z_n~=R;kRv;6LA3-$kiaWdi)+dw%Q8|^S$+K=ZpDH6#9?K`&GjtAY2iALh7v^`rF#O z8=iO7xm|02IqAJ~taa%**OORfKH^PlFPZbep9bYLKH4%E?<|j(Y{#5hg}CZaZ`g$hJ!w@K)1zzdWtd6^6dx%Uz*Q~;F^JV zb@T@qqeCrq+A^89v`JM)(_@a{OFC5*|#3A zfchMh)GCQw47g=f8xGFPUDZu{x3HY2eZx7D5PrEU1>8n>{QUpbC2hLkc5B=<%JJE)F z3bQm^Ag-Jf^@#krtEzfsgTEqelWoV)Vp)$nI$mw??%Blo@`eSw%%VhYS}_l3ep|$t zq1+m(_(Iq)g+}&5*TQ(>>qeA~v+!M!%$9xOOJ7Va@3CX{p+G`W3kFq1^}}R7yyLqQ z7T!Odc=N`EH%e+9GQSGfsfT9@i;k~LwZBP9TD^0OG~1tWJ}*S8GeXyPXj}PBjZMWy zx!hp~Y2aryn<`drrovbds!!%fF-mDhRvp2hkf??BO*uo)L+nNr0jYUtUz}dNK`;yPKF`Cig zpZr-$_n~huGNr)9v>kVSlu<=WEW_~j?$Gx9ZX8c`>F_qA{Tax;bvV=P%sIi^<3@$e zdQe-YS%X0P?NjBa!S8@p2UUtCQP3;yJej(IGE9-5OerBbdZQK?N$!F8wio-^-XxWW zYJHD{YrTIa9-=wFoEn8E?5&b}Y;mrRhC6z@mmaEpVM0{cIp@uJl!zU5j7!fiP12uu{7!hHWR8iq4gqK^=O>8V2-(JtXC_?*QgQ`hH zdGU_g4>bbyL&)^bZMDNSUtLL0@_0!V`1;do*NlgWMs;4@3xce8whnrN*Et0t8L1vY zEUrCX4+Zbgy+%-@dtL9)f%AQULewEnni@Wi`J| z`S?gK<>SL4i|&@_dVQNFeaE7FZ*PxHy_50;`F4wXUIYYKcrQ%;xYsb%1! zjG-6%i&Xif?xc|yi}ypymXED)_dP5hThb$jOUJ$lSXa(e@Z9VSQI#?vGRQE>$|(w^ zDrm<*gi4sS_EIMkE1Kn75Y7BSwwl4gFORze(1^Z@J(M-LblMI3#+F*HbjXYsH#5D-uaO2ZljYHRr_L&~A5$juD*g^;15kF4fu%9cK+ zlYl$nnHV?PKmVj`81PyTI2`rIu7j3Vb(mrYEp7k0#b=#Za)#nVk_rSwb{5{(Oz+=W z&DoF;YIKhp32rw+08;<^=@SRyY@Jo@?+~b1&=K$ix<$vGP9*LP#|+yj%$qINK%e<3 zD4gh3n1y!^LVc>{|M!>w^W_0{=@KssU$Z$pN!nIdTYe;4EL-;{mQVf;Nsay*pUO4CwiBP4$Zbb0ce|-#!A8G8RZvClPhj-jZAV(O4 z=+^q*LVx`RG*|xXGtgKD2)>x0>IB@)rgy4gfM84x%Fdu+UGONe1q;a4ibrjKPQFu( zej!l23NopaOdw|kZ_3$yx)lGeoSX)iOn6h`OrwAOr%oldz~IHZgqn~6xqdEq*N_OGq}`Q^hWDrm=FSq33rhuMjiUVS z|8M#7z{^x|m;!%x4}4cZ{x_-rcMXn~kw@u|_fSi#|E)v*XIvTBJhB3DOxZV_H@sF3 z7R}N|Dj%KK2vP|5ka(Pq|I4WQyK6stA*^8GT%P{u<<7Yc@AG1(b}2?}YQ8%jz+l1L zi$0kS!MN-KDslr?k!GHXNHBW;p(S{UpnD!h=pYXOISQaI3WyaXtpB2#PT*P9%4$CR z_n#xckNDVb=Kq-*c$f#&dmZQVtTZ-@w?EuMWoC_f-%T*l8K~U4AkwxMkh6g6)iy!t z<-hd)Q&4tPmaAcSR!G3A+21qw?EIH}yCguqH9a$WAU#vKbi2qWF z{|DN4Q&eHv|M&v)so3n&ht=g*gpXMI*6&QHW$FBaGhUma&SN;rF7LO61^229WPMPdz0CTQ$++uLe zGr0PP#W2`}TO7g1FfjhSd8xrj8l-%`(AaF)99e?rN?sXxSIHKVsrR<_xh}=L=IL95!}+GpW_4Iu$nvBae{O`_q*F!p}*%)XA{*; zZ{06OGm1rEH7gOF56I41Cf-BIDbx;c_x$lSSGk9LDkm19jk`Q{m z{gBS_VyExRb(4)Y$5O6Su3O*ghc11=eCG&m-AUc*tq`b62|#-?NeWB^EQ>OGYCyXb zy)<-AloJ@M7B??CJ#+um_Mtfydw7m;Q!$D}j0narkN(OTZPCl&u4zRJ(RG^6y{I2_ zEM~Wewcj8mVK+P1yPTP1S^#mL6+9HjBi&aiJHd5rrLu+rYa109a2IL+!*N2u13*u> z)afop^+`QFJEqwlC8II{5 zAD2aXSe1cz5u0kK7qvqk#XK81F<*E2w<}ekr}c}CZY3KJCYXY5AU0q1 zl9O9d7}ltNy{Z28oHIx;mNl01I+?q1L*D;j#Fy-9X&H2dh$a1w>tsD%X=lT_;4Urc z&0b(N2e<*s|?SpvOwKbECJ8uD4 z{%HT?>rmu}e&Z15h82=RPqI#o+B5m>Yk=hHxdQ-^`Kc?s2@ebTn>ZrqYx3nhdssNpAw4Ub7#N@dj8tEsxX{LLAy?A?5?b1%P@Ftg* zxJx*vEL3Nef^xt6b9|+=zVD3Sx6^o~y&Uv>;`%n+;x}^}3eNAu7dIS20RCHd2J+G* z@vJ}poq|6JI_Ak{E!@ZHzU~Y?>OJMbRB(@Byai)dIX~j-990QRiwiiJ!G+VuAyPsI zQ4%U-`{3-3u?e)5>aW@*pC=*KAoM}ahP`M4J#>oBrYvrd7-q#;;GGxU_tg=taP)#B~7 zetv)YbaS`h4m8a!pqp+B-bv1IkX+Tv3k)LldHpIf8O-dy-MA9>wJ~3>=F9n#t&H;M zj5@Eb4R7hZ&hl(6C?o#}z0YbxB|`TbeKT5S-Q%z7+rP7wb$$fMTKKS4;;5Go%>jI3 zXGX=TxMFD4lY!`V{*=i+sjrAG)C*{U$1W%Vycn&w8@ zbMH+#R=`p>*3~BK=hm3!on>1q#`$Ahzu;)lHOC>SPyQ&+oH@27k@9RFi7Otn1iLl=Zs2u+ekZ{Q>evi(S04(o)>4QZ zxvf15KS>DGaFS^?@)urXE;bEpID}KWP35H-vw4@wI)552#!K|x4x%AYImYnuLDKib zAPy^Ea6a>cv4dJ?lI~Hm9eE1? zaeg?cp&t9~Y@<;B`rX9ii?G;63xXq$%kAo2_ceNb&EuW=h2!G4yRis6U)M*--j}3K zH!s5-zd3gr%WunUqMd)+(L(ogklFuk5AL9rlveNbQT$%4E4?`VfkjX~Xx?gHzOPw3 z@xn<4F%5Uv9tXihI4IDuB=!3+Ed*Ljx7rugqQF~W(z)v}(KlRzyWj?*>MnTXrCO9N zFR*#iLiSVzdNuE-0k^)Mn<(+pd#S2g3{v0gEt*m{ro)ZbbD2V-+!J#fa1Jp^_Q$SH zs+)CL4UU4O1SmXmE{YW}!Z!d_JUb1s6TX2#gttPd)?lCBM#i#d*IAU4xv6U&HIp&8 zOhj$$I(I-EThKvxjGev$6qU)^kzJI!wW8#m%a|J$a7!&UX-s}HC9zCpE2no6A-#tvBXNd)e!3M8yg3PdhE1Ik2$jVU#?QM9x7}u!saD|FC5s2Fi`-l2; zqS3%S2gUO3^gvtX=SwtFqDNBBsZ z^z8Q@A4si-#7hJ^{rpL)*25smh(CE^W>{FasHs;Qie8(smo(11h7i6L)(QNz@-6f* zR^jUnMgB_7RSdPlBTEz&9Ni@Efqq9M^{rk8vy@)>-gV;$-aN6S2-<{d=iY6ty31QgJo zT5T^t)KuRYX^~8#4%MBRI*o#GQuD`T6^(Yk!MoJa8HvS^;q(*D7k zTPUTC{n+VyemneYH!Fpt`OUQanC98RAgOoZk9Zsd?8F}6@28~#*W^+-0)2uV6_K5q z#qjZ4G4!WfVxakPM#<<)F#oQ*l3KoCG0OKF>LxCC?Y(_1Fxeh$_^^j`qNGl+$gQlb zJ4k2n`TKH)$SBbp;Dd%tH9Op4#um&-oah;0%iK4W&5vY#kBBIb$u52d@>PRab^QHQ z@(BUu0{8C6MUXhLRyY9v2t{-6wD>-jwiMNKT283$vSPW+&Ms7mA!|{Fu<8seay0b% zitvSPR%D>|(f6Vi^Ov=@m_K(*TmYUeim&ATrI4v0+&6Drz@?-q$M{~A=EJ0E8bEZ9 z7Dll2&BE6V=xBZpeIJV-t_2PO4>sek8A)i{dgM^}UgNlyrzkC%8;Q}U{n8{&RBr6X z6*FGGkKrN`xQ!YFF?&Cr`zSrX;vW7p`mh_em3?&Q8nJwVq5|kHBi{#Rm)0kByJgiR zlogJ$)+oMhQV_%wju)Dw{WrpXNtcZy6rNoy*}egf`@*NHyaSd&`|iGT`jJPyd0XM9 z8(!()H~+x`dWf~AI2Rb%<;W3>mJ@m)l5BJF`gH$AGyQhQ+CZ7WS9;1mvC_H^B7JK~ zj+yS54}El<=f7WEe-c9v)+-WGriJD{i%j2JRCas zSUl*lW1oS}?(rowD5a(`u&u@Ot4wx@*Y(!1{b?_o^gJX03#oaZ$g$?&iMn9ErOt)4 z*2He|BcjoFX-m_#x?{&K=b5yga3Iy`T0uMCV#1!7VH$V>$i|L)ll6XWv+0aw8^`bA zxMd~n)RSrjagrA{tLdAoU~!i|t{8+rcFmyKI2>*>rdjV~oiD(k6zN=IJ?fEK@RpQV zwUMk(Bysg%v0Lb*%!cn%kMcUz2cl-W@F=h31JrbL?En^nuP%MLZm=x0EQsCZEACw8 zg$zhVj}Nu!V@FB3+pURjTX_}W8okq(5lVJc6)}%5mloHg)jvK*#y=aWUa;0)G)A`2 z(+>QpW<%+8u&}=tueX+2Fs`q|*&`^2V^~m~73@QO{q&ru?E9ej{kZbiyE(w3lcBlI zqNER#%qTkM$=_^OlNQ_eH8WF#Wvo5%u2vhlW}7WlwWifJsA}zpzpnFIQ}KJSZ1KuK%5EyyfOK~ViZQauj|)<^N=8( z?#HLRAAx?jtdR=B8N4V=&yZX@9z@Ad#Y;JwnxqiA}QkEP|Dp#dC2FeErhU=@B5C zO--ZH#-&ArUq+E;RhPbD8!Fd7ZCN8t8$4QSnFO}loT3L;%c-5oi z`gzAak3*xGl}yIHKUNM0w^!=P>RDesLp~fSrQ2@z34X41GuXm(nY$i_PPv>!Ba?Tm zprV}_zC_q3f29V&3DCJ7;`beEceFV_U3&rFqgnC5p@38xkGQ$QU(rM!?REGNiZ`_^ z$z7;5dcVJ9M|?G-VSAv0%RvHtc5!8KB=kOmmhSFM=7SzN%z>g`o-@aVlY51W^z6e-_v?xvhseT1Wu*aBy|D(IQ}f(mda*#AjS$VL%2x9J$UT|eM^{xEs!Lgb-+ccCc@jf%kjnZ>8{bVgys+L=VjiJZjSlidednr#vi5k z2ZVo!k3eZiEWkbWl^hGJSj}Dv)~E5MVeZMVt{t$h3#;u$-8WC>V+C4$HUqZf>TMPd z;;V}g4d*%+&%A}JuwEmNtum8LiAZs9@?uhX1nibQNPSG5O2L?!b`TmtYouR<9Tb>2 zfy7v!y(C?usU^*49H3Q{Eqw^(l!OuIktnt$U<*vIM<4ZBNkM;qtt__e=N|iYp^Km? zS7 zY9*`DeWw~V(<p2O@MrlfhX=zVl0ZHpZ!-_g?R!gWcnG z=08G(#T=h|(!VQJj>%m#V5r^7@avw8;AgrAUq3hya!Wilra8@S?C+0|G$P+ocItNB zlecwMaj(1}M*fj#^;9zluIJ=cfku~)q-gXL8OgTPPl$`z<{WETpp`-ny8B@m4rF%7>Oi7+!>xmFHSk~B+aNJ+FYunk^o{b zQVJ8 zOuiv&m_w_*J*i8CqvoKhD;8zY*(bdcNCd0gW@TJSf9Y}%aydp6({H7NZ+s2{lu8KP zK$SmH|B|^ee3St*{os!CYO_~&f#e0up=S+mwzk8gFex1OA(i;fSn2%7wxj@Gh&_ee zJLKaCal5NFHfcgLgf7u!7a1mtPq|$Ox1z`-Ig?+npB}pjn>ynNZ1hMd#k40b-_uB? z#rIFETnWy%^T`*O9U@JyB|K&Q1{Ok}9iy%$=sk@gp6vl>uHSYoOX?yU{HF?eJiv}b zQyUuM<4%$fbm-=&*xbYVIoXNbv|Byw2P$)QH1elbV&YnBk*#QJgbTT_=VX%YMO@Re zq^d;G5RLF`H&mIP92-f|xOL6tnLbqIfR6SA0}}bUUH*{bR1vYLg`kh&9usK2S(Xgpk@i*mRW2co17VqfAn_1r!mIq5er;{p6C=a|5 zN!y6?Lf#e|uU%hEhymN%teux%#C!PMUbxs<+tazF30{!!Qwi2%A7wv%ei_`RRNIZB z1=A@gU+PUy?oPNYTa{yuqILSMsgTMJJ=zD`l{a6dj7;$oo~D90ZE5geA{U_ugm^zi z!^4pjn&N^zd$(i6I*a!5D{#svopMGLymU>UJybMP$_vP)u*1ew3qX)>9yJW6zTQmI z_<&C5@*5T|JCr2<0{bDBq>73da->6RhS|k|9F7^UKM$u?^edO_*H8J?IgqyGN-Do# zs>2{ZG&zD`2kh*U_L3w7X5E~G>$y)h7?kz0!iK~6%*6W4cQve`tya(1G)Hi#2ALF) zrHwwQs!O6%x==*?5Z(x*y^^dMOIoVsS?ahpPwo{G3~c#q$6Jl|h+TY@%uJ$Byp{cN zZ{1MK8`!4~O>)PzozI3+1kq$0@4H%`U$w*d;_zz+sVOjaN0sCk*RECl40HYX)ZHUY znNL_ma31Pa5k}qGQ5=JP@LAw?&7aLTe!i@N8dKlNE;s!7ZK-WaKTFmO&%+_nEeiia z^{cL=ev33m;O0)v7qvs;BEg1iv~VZXhc%1%!wXI*5kBMTdXnKX)kPugbqv*1wK|pj zsSJCqWv{%G_&zH-@h))mKhHsM`dVQ=tYY+i!nU*B2h|-kt$+^iE1D4&jjyhV_i5+T zA2_?^GM`)PmEZ}a7&8e*>7M&Kh|SNK5QzUg-0yuZK1z6>m!CJ3e8&4yj+p15hq4e? zUa2O(Ic~NP+bwC0WP5nM?piK;si5kk+Tb3Ohj0?lAF)~m!k&d+cE)d7Rd&~l<*I^X z%vbSs-PRUGw+yFu3m?G`eG^B#{W_p}YsX5phOH{%v$`8+6#Zqy6pX6{e>A8*CF@bB z!1~uaBw(SJnrvd9Hg5KrxriNG6gDxg2LA}l((hq=j`zGUG+ePx((}y1Qac+pHJHML zxU$ldC)Y)Q5kJ~^>>@n)CCun?+h-6ZwX>}r%E7K%j0K6v3&iVGU`X>Bdw$Yu9q-v! zsw&wW!}PSs5Q~4+RZZQ@X@jIKkOpTZafso=I6GAAWU98;x{Hb1HMg_p5%gAb$W@JV z<>jD4l=UDI#*nx5@ZR7NwD_>(BEMA<`heJFfm!I1lf4y9` zI;cwuQEG^6n`MEH_US9VctQ3fXCocJ|HaKU;fU5xX7mf5_f%mmcWw-`LF3)5sB>&K z=UpUPzOeholb`VqB2NC8g4SB=@=nrwjGsH|c~&oJlBqE0CeRGUwhA9T>+y{ zn{^7)jFroKQ)g!Rs5ge^?A1*P_Dg%AkZp8W_=86RktuwoQI}RRG3=EO{?LdU=AYf$X6o6`cVOjiRCPB~}toP5wBlT95d5rgW*G3480}{)tb)7A#IC4*HWlbh^bq*g5L#kcrEN2uP^o z+g8nb_KFE8v%1&5&25>cCU9*#^AxoDLf2j{^D_Crl($TIFybH}E>bbE;b)xn`8DOs zWIfXtLtPPp=!_vw9fO{8o<5Lekyj}V36D93)4+_rLcAblS-Bx~_WE(U671*3z*&jH z9x)ZnDg#SG@k&v*({nJ^3uS>9o1YyVG9=hV7A60ZzA94%IJh|PJeEj&=gU#v;tg9X zG|Exi=>fM!#G$%6qyH!lCc{hzvu{ku1G7u<_JtJ`!;4UD>?3-Yuc@uy<`$f4WclLs zpT2pKj)BeHxM@-)5VXxVo>Xu~Fq%uC6cN3%2057xLxSp`_Rx<_LtA*}*@L4z6hC0 zAZ@&Orlw6RXkSXat&4iDyR8n~RaFwMuIp}!OiR~`nIfN>EN|j|vm0)#qc22U{`%3O zh5T{)j8TZ_Ybx0#GP6QD)Z}w%QnSwweV@~LVSqQyjA-S&MlgB(=)tq<+24zzn})Dz z*8AXE5-pa!J4t8xiADbo;&}~XPu(BlYBvV&pnT1)HRxGE0`Q;C9!ZeXn?3Lr0l|jS zvn(NHAv>xcg*uM9Z^@Y2S`WP7)acwf((S3Z%zy)}FJLa8T1~a~={_VteB;x6^ z5{WcdEGS#M_>f}~e)aNKY&-=u zUuN@Op&Hl5I(;@4SVy|d;&*iO?;`xxfATh&IDaXa0P~^ZcFaQTPN%OW@s!AzT==Fj zK}@`{1`TDF^t_1~9{VUwN%o&M5q5fZWp+72EJ!Z21%KxqnfM9a5M!=wOp~(k;H>kL z^m77A>#uV`Zsc1VE-s35ql8cuQwP_cFAL$O50@>7+6mDQFpixD#l0h6u35{^tw-0f zCDXVpz|6R}7DC)s*(+ka-;(q@1@g~jUK0Cm9h}lUR3zpBu88ZMRwV0krG(F`7hP+J zM8Lh(X!yk^X(V}e(b>6*rD-PeH>`UJI~D|sX0b|f$l+5p-P&M2#D!-sJ5%uGd&B+v zA&oCI?UYXOF+wwaVwyG5i60_gkwMl`fyb1j03$Z(r~pmm$CQEiYyFjois$~4{IY78 z$;QZN_qC7=MSgv*P{wb*5Zt*Y-0pY0Sd|1boOOp>?5+bNKNHU6J!99Fhc>;tR<)nl zIhs-v0LCn@Ga|J6A-E+C9Inh)w&-cjC`Q;ZC2N?GzHhld?`-F`a6(5}=1kDa?u7MJ zKel_bn738m(2~-{?oaf=6H~Ri7r6T>rmS` zR!au&Ql$4PGBoaXcO=az@r3!TNAT}VeP|RdGY&-ufN=BAuXQpG^~$9r@b!!6@Rsy zvQ4K<5A}MXFepDjfs(FdU??e4r<%&5sI|AsavvRc z`F%(wrM$0rIM>KyH%Ftad*bW07R^JO{(FifNH8MRJ@K6Gp)2^+<<=8i=b zn~2VSVRaGo6{V>OX4-qiT+Fn?=0Bh~-gdscSYn*nfk&cx^71Kcc0dQ1c#S1TS@OW9 z4L?rmR62+@ImAOrzmrQeA4ZZP+CHf45Qkqt5FCVwfg8fkQqZi5&Jp*v041hQly^wa zx|}|we$V8M=JScPL4Jq zpgh_)FcsjB?S>}BZca$Szih>h7OP z!Es`EmEnA7{^7{I!K9iWd}+?xX>Aqz4*NRB74i@v(m$<8QyW%zbA}Ud=AMK?#+GtN zq$L>mW}uPM@n$_o2P&z$?Aw|aC{_#@>b}0Ydc)5p*+XScsmpRafE(Yfx&+b|AFmY} z+E?LnbIwepT%Ru;4tPy(&g_1?=4j+ocdepj9C_z4o8Rp_K{D`Xj-C-AU+!Sb)>YqkGb zYX8Y+F6%J%Syk1~>l#0`=I&|kje7Upmu-7;I5~+*Sn6u;Up3BA+ZgAEIQr7Hl|`cRm85_xG;=9 z<)!F3k_DP~5{48IF$}9+`xBv1vbA@}AE^r%JgpC2;d#Kj^L>_Tp56SbnDE{qAgXAY zOKQr+b>AJ@(Phg-HB5^$79; z#+7EYo;PxD>;e}I9l-ffYUZb8_oL#$``)n9^I#-X zf$vJw50-F&88Fk>`W2hm%*+kc>j}=Sb$s!9dfC#n5N5|(peiW`MUuaGU`UkB<2@c@ zWMP!;YYJ`K$+9AWHflzuEEtO16;*2<5oe$cJnNrm_fv6mDofH zT`U)})V@insYpZ#sd*$il7ZQ;<+ZX$2g*I3$%JWOBVFMpRCf%%EjBLMJCO-xIBxa4 zNu&)BaXp!asJGqgi*bw`V(UK|*@c=_>~w=I&`&xp0kf3)hJMGeZZ?Zw-Vi(Eh?MyqG^ozhh&4Jgvh;kWK@k-$J)XIqHCYFN7br*M& zQgG>5WSQsr%w$sl`?yHjRbt8yZNZ~vFKclEqza?PL{ivO3`LQg%!Mq5msew>be6m= zugrdyyZx;5nU0OZ4RjthG^O{|;Kp2x!4W_%2qiGFCTiXj0->SQ`$tS$(4u@QMQ=$G{G#7+QTp(z02rUY``Y;*Jrs{!#=v1^&HV<^BMR)DAF<)c5I5(xW9{Z z5@xma^KYZV(d2@q%#8)D$X~>aFIM#WALJa8I89}0-{w>2#Z(yyiq~b`wmyC` zP}QM3VwhTlg{(?L=@NfKyb-zfC&jku!({VV$uI}-YEw-Yh^x1`WiCnXGNS)xx+Bhx zQ6a08JE!;QV_y*co3L8Tf;gu_9{UPY?x8@7gR)@V5<+c+nco!Ko8? zhu#=KOPMc3Gm_G%+n3Oq57CX(V=O4!_B`k4eC-Zlb{{2?$SvT}7jbtFh}8MY_tQlC zb~L(|PRGVbv^wmA42Ko>?R7?Yq+7AQj`BRd}wMwdf!2HUP!=f&7C_kf&duN|$`xm<#X&3g+O6JS` z>9GNtlaBRe5iU=|dn#r>*~E9dJw(bbXDWhE%~=+}J57W%SvfV+b^^(}>O(ocp9^DZL4~6xUh{E6Z&KrC6mLi> zN!8BWIq!o@syd$?TFRXe`#o=5AjxL%ds#PRac&G`Ci*O9@?8p~;c9&Hsv@;7dCuZ` zS#}OzJ%j?M+N!NVX!~`9>5rNFtP{htrX{DusBI^8oxhne=Eyp`LA4F z@T1lICz>{x@;EY5ob4(nly9Ctm%JZy)P}`j z<~Bh%@+^C}EZcADl~`n>wK6upx;)r(<;<1X8QoY}idTJ;xgt(Ui;L`_Wrx#Wmj8o? zS@$(0>LB`6XY>FMf2KtjDs|Xbt6oRt(hBtt9qf&Wihj{kJ>BvOrt)aMPh%sYc?fAQ z%ye>Odq$nl7bN#qd8>kg^4pflIv5qABiJVjxJ|N_f+xOBr1=N1%qu6 zzcfvb)Knr2+UE>o!TmX5j#zFqcFZ9)cKnT1xQ0QKHQkOy-K8;nNrWEIpHXay)^EPt zzHrIeIn{RRF0a!E>qDPJ{H;gJSifHGt`0oL+l(ZL%)#zJ9vKFp z5j{g&G_OI!L;Uxu1XBFE2`EZX~YK^VkWs=qO*(!r52 z@oL@(ec9f@S?C@lg-XGK@RY$qw(revgc72xWA0j6r#?tW*W0fjn--E~ zHFoUonNCInd5&i)RZSyNp9M3)FBs-z) zN;|XD$3xHK;}IlU$s9$ep+p@axVxxWPRSmokTM!%xsYl*i2bGhL`!PSs`n9(l0W>NnYQ%n-q1I@{58>hnf*U5>A3jHhD^e~5nO z(RTjG!V#1rl{?VkWzmSulG-}aZicEj-8^aYY5k^b!2D(FOvgrB$qKmkr344%ZFZk3 zX&!zKq1^72{(gPIDAs#_DGSH0@Of-STc~o5CT6PWDE_wf%tpbFYlW*fWMPy8xjUy9 zdNvrUaT{wN4U~#uo5yD(3)G|q&NR$4le0&ih=s%iMHk|0BF1mgkKXC8kA%TO)O&mF z0-)neOc+#Ot&KaJ&*Uz$BVRB2pUV8=@G!Ay#z^+UU6P(?w;P>2r)IH98}?bh%8x1w zFd#-Si3z0M){D`1w{=JCUBh}9-fnl~=}+m<(nqK*!OLED(KxqTOapN&cInbbqI|QX zq?+oo94x_m_=N^SgJu@In9PeXD9euj(tbD74x=n9HBLFU!X(6w(017ACuHYA(~JA> zuZqL|q%PGS1n+xg+c2FzWFQEmfxHEK4xRh_e4n4Jd480rjjeI5*J0C%WnWO6sq0HmnDx};D3GStrN>pO=8!qhda zzISSP74I?l{(SbCen4R6?zzstNZafC?9gAFtZ6N5Onc8dGGUVw~14zZif?6qA>ua~H8zLq<;e+K5oyl9(F zqX58&g1;Pcrc&EW<^H0d?Fm2=tR2L5WeL2`dG}}XVL)#Bn&g>8ZHCG`C7{mY(?R|t z)b}3%pp_uHr4n(-E^;SJfboO>t~0pBnl)!>-)UR1mphl=Ky%8y?dm_hm~pjAT{!(^ zo-ESQa_&WqFEtN$i<~W-z<|dV)R6 zUdc$79_mAV>06N@eEDq_$N!;XJ_4K`A%OF9zs}CL$sfh$58~#77-pZUP*3;*`M|!1 zQ)$BWgw9g~d&_5`#5`@8FNklJ=#(O$9l52`iU+@9#op|B^*<2 zc@Ms#cMsWONH5LV4>3-Z=r3BMqFd;0O(xCP+RWE^D@5CD2a){~qw|Kp&57?N>;F}2 zIM4d=SL7)9$3L~um9AwF-SAocUhvN!J^RU=mJCh`)OFkcfJsF#z~xQ1dk5C=*7zd| z62K5?x{Esl#u_<^4=8Rqc!+~xWzWWli)tSj6D$gz+Cxj`v` z>4dv-vT#HyTD;O1V8<+6pz7TL6L|lUugEP(<7BA+uab*^spGaB|+NE7SwdV++IEMNwxu z|E`=Y{J0WJyxbQc;w)c58iImT*@t zJ^{$Bg}VjL0XYs(r-*=T>$~hOi+2=!h+&0QbpW|wc)rU$&Oh%~#4Hk|Hyw(7*#qR7 zP#Dsd3hoW5b4*v??UKL$!=jA{0uyabAN;&^r@Xe+xtT3eN#-IatCm_tuFc+U`WNN> z&+t7`G|=%ex&=Fg0Y}ZybKBpDLvgLA+;@5bG(97N-iVO)zTqNJu6{Dm4~O`Wld`*J z00FMzEm2#afr`udfw}}6RQ<*9|6^eMgxhnc)T>x{5@SH$HKn?m|3#l#a6n(CA~^5^ zJ5c%tv@esY10T!X#yC^}`q|L^`5HdHt>NRA1ui*Um}KSnQ}5<$4=nMzj(Ch9+)lLvxxx3VQ4KlMyu|cfy5Sg9nt-VnEx6-hOMK)cWMCYWEKQkk1zz)4Pe~{OmOY&Gq1?M zt4(-uKu#iDGq4ZHDT9<1LMrF~O*`?xv#O7)QtJeI-1#2stz9hh98pEhH<&g+)+LQ zCHS}>OLVLOa#V2mKIio;$(_>O$2Nz*-k-eo1h%3D@fk?m>f_44I-#%tMWU9U+X0Hi z@C;NV6fYY+^6sG2f!n;vVaO0rs(!fB2==FsPJHgjeS8GeN-=8hFZ34)-b5@mocdU#OA_qMUYu*n@AccAT5(0bO<5@87%)+yeY%XEjU^@N0V{{s+8;GteLeCsQXE zh#uS1>NxdZjtKr9gbINzzze!I+zSns$zQ=b$FcG2r#im|tbu7zh^5|Pvaw@dd`e5@ zBXv?@)H1`U!N$xiU zb{RE~X4|^SE98Av-D-|nq+MZce_u}+Er(Q}*J0uH4_x`J~TkNp*lIv z0p0KKK4YNth6BM1qxOW9LXRpQ4j~3L!!4Rf1}NakqyEUq_?C+F<04! zzMOfY@3+<&kB%sorqdcx&YOTc5^s21WQqJyS~xI?hu`=|;_?8NadVWvc`mxJ-Iqz| zn&>-q1ITtYp}KxkCKW5d`HEfcfUrik`+%4-&2c(87~^Rh?i;X}r*1;o1+iTBm$4FR zIw~7jM|Tt~6#7TUU=aYvCES8TjBwZFAq;s!6|a20W#cU3k*K4Ny+*qYHuaky9XqcqqNjugCJOhmFxKWPnIs{_KlR#{@7|d^ms?dVAS?n?jcm z31S&`!s9zHgKZ0^{T?YHyRV*vfh%y{HB=w4`75P$6AC9w*MR?2VzK0#{4jnELT@j| zeew+wlS*h-3c~(Qs{M&5NFR>A7B<+!0ls&cuMne4{XTa+YUj;9qbkwgR8Hs>RK5-< z4J04<S}v~aQu-@N4!$!OIEE_>r)j^* zZ|$cU3a_`Z1`9}ezH-m8BIFo2A#$2(I~z%8a)8$M=@u_e=ASh5a*aF6j5hZ2^8C^~?E}m`!c={9{4)M4)$&q>jKLIMKuA2o9vrv#{7XCuI>4a+3T8d!OY=gd~2YWxf zqqlLJoyF6?IY`Lg zh%2kP4uc3}Z#Ee5k}$~0RgYs(-kgF+rcIxR6GIprCNMVv3BUuwWQHrQZ~9)H{)`wCW{tbow*T!8{=-{9_RweORWWKR7xTw}bHk zHLuuu)KS3G*yPkVi9(~hr<#}o`=>mII&v!xO!n@Vrjx(VKi~?C)}f1j8sy!GaY$p2 zVs+o*+ozi)KxqKHv}!P{#u*^4Y7_{L7HJyE_Rc>S@uc^}831dXp_}QlLbRws46aT1 z8i7bJcmlJT)L!m@U1Q*ig^0oTV8&&~=xa{`t*(02o0N}#fq1c2*JF)(z{5aEhaF-^ zD(}@^`;&`Ut+$vN(;zr-=xNhwV<(!;d*3trp&`_;N?P9x5X2r&&gN4su>AU|fQcSg zN0N>Zx-g%A*U%RB{@wG-q#<`R&X%l%q?y$tugm8N8U5dKlh9&UXq#HM_of)j?kbKF|sr5 z1ch4}wC{M5DsB5d!!Jpym0f+Zvr?VAC)4gxF&4Xh*jpQAr!Q986=w3HX`a>f^WS*D z(Xf3<_*Jz8mH-?_RIAwi=m0EF0}pVUCpl<2ho`kyL@h^h4eWZ^)SHl0Ig>YyMcm1V ziU+e1{5u+G4K~0f)3{6}2pQazyxnt--tYQ?8DN)--^^@!6^gq_oOL+2o}Jn{*U#cA zx-VltQA2PCdPSoL?%t_U&}7X^Q>Tr&vf#U6GGbz}o&sFw9Wu|KZUI2~l%%id&vfkP3zjB6OA5SOjeSLn2M z=xswC#<;snuS&Xdn&=*Qe>E-rUrfDaSkwRe{;z^Cn$e?6x|zfn4Wk=jyrmljq(e%& z%aH=3OF~Meq(izxBu2NSbo}P$_!(pT@8KIn=Mpi9%}UoHO~(>i6_N2nbX zt%d|{3)jEe*(r{gaO~k$=ZY&wh;f~8dH!C(X6bOmTB8t*!3un{Hmvcdcvo($v_A9v zR$xx0I8*HQ9j&l4rOvT}zWllRnfM8>Y$7;33Q4*|)yfno=-^ADSh{#Hu~#hh*eizI z`smdvIpkZdRk_r^t#hoOLoo!#B(g^WQXY4Km9|EFjOyH z--ywR`TgMk?={r$WR5C8S`?YFgim%lWt zs9W@{ucqtx(W}h|j{;h=Vsw^JzZ|w(EblJ$mw>zfF+7#n@ zhAd32iw@{&7-iTchO(eL{_;82k_nVog{-sAcY@nZZs;?+pb668N7~Tnq~eIc;vUj4de7# z>;HG*_gc}PWem39-)#&kg@EpD*QJRq`>(aI^p9*n=PGB?Tov0y5G!NHhi$_%&j{nZPv6JioJ zG4%a;W**T&)kd%M<06E)Mq#rDodqz=AwGNMKy*5L6_*N3AA}C8P~vhHz~%bGBR-+a zdsTyvd4@i{+o*X1rm~jxwQ;7R@{>DDR>4$jI}Z5#1aNB66|@`u^C1S6=Mvf)TOn!w zZ+L5%_>T-^R%>UFgA%U{vcIc=hN;p>4d2Z>w;aa!q2J-m$A;$OyWxYpK5#$lK?QV@i}NMZHWIW zZ>&6fXJd5DAE5NW8W)2n<`ChWJ)yAvYBf24q%Up(=dJV)23WqJ!8EN7TYV?62b=TwzQa{D-wAHu?V;jT_0ASgKtm zUZP!94?2)A4CfOX*N?*QMcF?eL*r_Qs{IWD``fb3e6YROB{rTfv;3L5+P<+*g70f+r8HCT!sAAMG5h}-eI%Hg`i1vWSxY5yFBxZL=HPTj^GmfJ!!b?V z3C@}|T-vWY>d`-u%TlwQzrek4`M~0LKC#7|V01bn%hcodMLsq``r#@>QqWMy)DY`bfV9?{r|iUkGN>qD;CE~1TB zVODEfcxG2}uO5U1gBz_b;Czqab9Q@$&&O_d7~koMC`=g~v%X8NHbr~i$gvsxT^b5* zhCu{HN~bJ`Mxa5yO&we7x+2M!=U?sTq>qD%DsM1%@0*YQu{O9Sl0!q4zdOZaYTcOs zo10WLZ|mjW=>L!Ygr>t#0zNh>HeLKHo>Eapb3xs@O-7>K2s;4AZ#VOu6~>>lng4Jx zGn1G9?Ef56!-p3eO`f78Dgb)3*0{uPF~&zH2rH^e7O3K7G;sV zXx$|c0W%b@S~vf@a*eowpE)W8x#r6O|06UFv0de8W~aS8k#wdS@ur^k$QC#t3e89t z;R8O-3nL`;1;7V8E<*y;s^PNT|6#3ul=l~K+)Fsl5RP&?@Rje@p&P!U?ZR4(rVD|u zPS|V92iq_J%BjNSs0RVW<(<;tsKWf@ZrJ%3^Z~IxHe&}l1PnnkNrU=85uRvo$Ahek z#V!^(1V+yP(8CC*Z;hE^(37{hPw;LnyaBaOsZ9HU)6n1DyJ`pZS5*Ssu*0~ezR=XO z0uU6Jr`n+v@x-528RFeUEQLjjm3Ghd*;5$)`Lz>-^WrKInlG@fY5Ymy>R4?w2^+u? z7#A_hnb_Ci(c98M~qwA&ZvHUD2vu=>9py=#HpqN^Fdz%IN&%-Ax$hxrHksMlfn6CN7yMp zJ70r_@O(JKiS&UR*3(i=_35)@2?vR^;`cDaO<;i57oFo7$>_8w;+nx;qDx}`SF=S& zC9Zt|r#Kd0)V$}@x4hAD2IA7@>|_0>sVlmLokYDBtzQ~reI=IPoi*_V?U;1M!(UNG zWd9aS7{K$~WOWD!BdAZMyL67Siu1En0QhfBF9f;&*S z_&D6V#0Dve6UPz`e4Cx#7o^kpJU@ih{hqgB(|L@?$ZJ{i2d*plTr75XU`Z>;`-OYO zsyn84*%Lv-y(Czo&Q6_4Q93q zc-7xqHSz3*g*s)Zi14GeU_pX22FD7@N^zy_JsZ)}_R<53AHA@!=YJ!L7Xg9g6MOA} z=?zmvmIk5-Ns*M>kCtVW_U?MLOwAMv*LX^M^--oyjAC z#HK9*rNR&&c=@&3v+A?fr*DQEqBkSIQ$iuMd0_&nX+ZTwi&Nr5Y5Ys8+w3;59H&ik zJkAEXt2eBZS-*nryC&AS;vq>?sD4=$WowpWYWF;S5Fn|#QnyOLO@BcQrg&MGz%g*3 zMO_*vBQ&WTdh*un_P?3=yOYEq;{$a6Y$<}%O9!NFG2UTo%M{pQ3hRcy?>xm6bkv|tbWb8}^HkDog-%#gQf=9>NpX&3qW^PPmpgMeULI`MD`f3pcPT%tOSO}FLDtL73te1$HJ4XG$Z$?N8FelXh3_7@mn!Y8 z*zY_aEP9Uw3^Vht)=UCJ)72X!yvhnWy6aE*>AWGUjRP%IlTVxnlSaXfa(CL_Ioe@q45ch!0RkT`J2-$@ZCT{HwBKoSG-1u1`dNaEtnUbNaMuR7|rN zA!ssIYD3Gt^Ty;phyl9JNxl1{V#Z(bH(d3KYxC=HDP;F_Y#O@MV`Z10Mis-{9gnv}K zg>G9upd?m{?>1GbneoHIk7_E1#sm{;m3&jZEntgX`0|Z_T=jDzi#3zwb?-&iFIJzH z)w%_%p^?&ilX0iGV@V=p6scpD>09rR=^vDu*_>5wTd{@blj#NL2kz}R_;&UFVjbzW zbp($&H1SI99?e&Dib`$Yab50B3jTRHiC-{`;du?P*|7qiHz|}bk6vt5d zkbHr%L0qum%szCy&5|5=NF1gJTEpcNio$JFq|XabkqE~yquMFS=?YNKzJ}$<1jrx@ zHNM~eX&GvM^Mb5i-e#1OsHsN^w9wL!^V6(`_3@h^`d7NrCicSJr&lp%AZV${NHEu# zCA!N=sps^3jC4?nf|=TehWq#bK#+7H;-o?x74FZj@rI02&I>boI`DVJGJ^*wD%A^z z?Tma|=REVaFz%@rSw%@y+IL|Yz6uMbcJW(qX2n8VTZz44bUDK>6 zdw+>_r}|8JVINOB&J9j&O27P6lLbRy)|fh!9&#Rf$l0fCgaN23Lesra8`6+JQ4vjq zMaX1S@llaCq^-4kn~mjmS73@mEgn*Shs4FcgJA$_BWlJ~Vsb?OAT*SI-=WboCp^EoA6SVp-H>aMDXFYVa!jUM9#qGi@TrxGw%!*ttozRl+r3Ct`pZ%Hrvbum%LmX$c&fGs{ib2SM-u6A>Q_=%#VjS}i13#3+R1(FpNcf^7jho2U zU)&fTxcQn{vi>Qy0R~RfXNkAFnQf}-nUppVuY+d`V}y(2;cP#T8dFPF&^871rlT77 zvZi5C04{i`o;7MAi|!Tt@JZ1zDI|xwdw)4t4T#bnebQ&7dEynb^9k5nW~8hKc%N0^ zAHELL9rRnY^~No*%xSZ6XoSS5nf?js`Z~+b` z`5evmG`Enu#H4|5PF#5B+0`j3mHO5-p_$8rEzODTtfzIUee^0pnFFE+ou4iw|F(Vk z9}nG16&u~H45l;q;J7^)u0^nqgdoW(!GmxwVmq&&#ot2goXau2lv#@=6;cMG<#{A5 z&-q254jC+#C;pJP&-}_cPDJShM}$@{IedgzUOEAEPp0zoYc<2;;OW%7CV6L(OP0ko z$Npq@Xqo**t2(zhvY!F){d3ASFK)i4-;Z(B?D9+Sn)_(BZHR@VYNh1}`jNW=%dIh+GsEIyW5^n) zIC5H=B+reR=}0j@=`;+=Sm@32n=0M&f&4Z@f~%Z=+--g5ey+BPs7Q z87OlKFAEJSu&8^6JzmuH*57tk>h&3C68B?^&1ffSh;g+F9|+44%ZDhKcnLJb}T zqhxc_j26(l<5d37;0D=^Rg+Q4d*ta8iwvWz;bsjD9zAsHeCJJsuYUyS^9FDse($pP z_QCOAVuN^zhk#1w*^#tVx!5D!e#~gQ!BNmWo6rAsKAYz)$7XJm8hnE??EWz#J^kR9 ztQ`X;C3E$2K)P7_1++;vH|?qo_7yZ>skfV*ue)x#^dJoW1H=Og%+%U$ByDc#T8?5IGcwQ=Ty9R6?Mv!h4^6$H4K^bH6tPx{Vz`A&tPFk(2+y72uW_n?p~A!`9k zIVfh=uNxP6e}(NxRGp9p)Mx%_RqyTRNrO33&X0jb3Gp4gcmne-K3;ix~PUgy$6-j05%?+*vnOtjEm zrIP6SaAC8iKN-BhhpnB9hlGtap|~s9_NnT0Y@nt%=V;#xA5~(eAIV;NEGMxmKt1;{ z7ybyQ7}doJ;rXhD9y@$yoc8GW<#%nDkjtD9UvJ%^vbKMX>FnAy95V7KyTI>1nYC-?k2a|Fg_}T}&%AbW9 z5agm>v+C66u<@PhK?W?WD2&0oosz-qX7Hed(;ft;;!4XwZhh`E?5a_k;!YBgr}QB_v|uDh@?h4=WD-*3SYVhxZD2_R583!_{vk_aQVoe}&U z=MTFZH!Y513jn^Go`_0ud_6tXLHRUt^GlSJ7P{qtW;Qy7W#~j`t3!vI(S1iV{IBFt z&UdEpYFjH1*f-JW6PnxQ8S6fn8R57ub3csyl>Jxco|52_m=%P{_(Rw=xH;9sAI;J> zK&WQi56FJIr=FCyab2KS%C_b=TZneJH47=lR2wwBsn~G6i+Z&7fCKOG2t~SFu84%? z{}oP{_#2jxJgi~(Av+hO z1si){)HQ2>^wxFrXpp~B&Yf|8+BVvu4-+iaoDU7|iPQ&lejy=YUEttFullZ{Os!8$ zYT2c0SqD^+=yrMBqaO&e7{+T?V4~55fAo;IDIwi44V`|ri9UHRao>lf-WM>QnL`(aRhNrq=1qwmG1UK*WH@B9ar>MQH zLsSgb!wGxDs!5@~hOJYX-(0!_2E|TwM>&pmOijc*f@pmSTgW3aY_Cilct)lcyQi-W zFE-aarm`qUH{#Z;asu8f&0QI*V}BObpFM8H^h2`T7p)a{W%T7P>MrcOLJ!9(&#QyC z8sc6_CP}5Ab(cNY8fhpBD7fDwH!#`9)EfC_{l&8<8`7=LYg<$ty8cO^jC%i#6zh5-$sI$?4zyaJ!lYPGo@NT;pPgvcDoxLH-B7Vbtj-yUQb2EG z^S_0Z626eDv7VtW_=F!q=a7Pf4ah>z%u}3SZ35IPf4xQ=vWlHY`g3)&5}*M}c5grb zgL5~(JWma92NLai*Hxv*_4NBhgoG*d$kLfA&=qvY>{tm|(G<$Y%+H}N(Y4@IzT&}` zfnoiHli_J=iFkV|vM%ZAAJa4qc5V@}bT_@OEnqC|9kY`;TLD(PFF~*A(fT1u*6K)d zI8d|H;(vVLGmfP~3s3Z%zkct7pxn=$@541Ar5Aeng)?u-Vu30bF`eVUJRMc}WX*Fl z?r-G{!j5{;quM@ zzF*Q&ob6nJdtssWLW3KWJhCn#P;=l^dk)DP$wW;#gMg11f$EGz7 zSR4hq1=KL}tg_VoBHUDtTlWVN};T?LdmPc<#WofnJtN@VY3a*8B8| zyDXuov7P$0^dp^}&qIE=q9=6NVC&rKCh!bf;&Iy@L$BnZ{wX9o>fIo5HJ5x(ty!l# ztuJ2$Qwb>05UDvubt&Mv*vra}a(XY!@6&fjvTk^^^WU~J8J%w~SQ)A?$`$%T?#-;X zN#>H4Noi(P3SCrn%8#Zx^<9$Wu)A)*jBExNua8h+e89bKr-EbCEDrL&&F)IkusBF} zokx#C7(FHfh;UhfW9BJal;Q!?AjxaO3Wb#XkwF>3!xwbcB#N}ZUU}Pi5@3vN_$uUO zuhx)GV$~01yb{^}U(;+Fc3ec8_fFAB?)?Ol>BNgk0f)@yBv7BK)*!>iVRl&p-Jo4# zr{lT~XmWqy>)Q&U_#17{7|byasW%-@f{=Qn|Jif|KQLT7R8)QYe8aKWjT$=}m3Ws2 zLQaf;vr`6AdQ)7R+(HNBTXWOM%O(9w1M^li9&u~ag5jX9o^O;+3!?A)=QjUk#>lSS zwsv3eJzSq15s`fw`qO6UT8vE#9dGGw1FJr*i#LNE(xqJ)oBp~dx!)?iZM zuUczZfq9)NpU@=-FMH*FW$DGPAK4Odpyj-81UPaj11-oX6wnrx7sO31CzhMnlh~0l zcgykWC9h}OJwQmgSt?sgzv(0ueZd1BnXh;@fgXW(?_F7h_mC;XF6P0=l!@=GS*a9( z8kZh3E#=x}-d5QT5$q042^9xU?l0~(v>pV+C-e$5AxX&#{7<UnD{9d}R}rJd=*20v zH#iP?RAr6BJ~KB1$6RKJ636&bvd9Xx6^?O*y6+lYyioz$s17TUd@tkx*@dC0!%ln6 zSFN_FH(SEa+outYOoPC{N%^IY4)ocI!xokc{Ul7{(Wpdqj1iIPj6OYXn?tPsT5U0e zHuAJ<{1!QA@9z|lIBIqOx|j+cW(Xnn4f}?@a;<*dCuDqqCidF!cbi6>4>}62p-Vnz z<=dvZrjc93`+v){2n~tl`9Ayb6%bI%%RX0zp?qsxt2Zzxx*|uFEe;m zn`y)I+f~{Tb%{6eb$AF;=ylGE<6%t254|hHHP&g??=vPEF}3dUss^Zruh7%(Q7w~h zT*XnF@jRtVSR&Ch`9Y4VZ(shm>G<$l6|@M^ibc{CyHNc~qLu~f_%M|N# zEtt=B;mI&b?6eCmt6GP^Nsl(~?Hl1b&>?L&Dmo+lN!Y-MS1g?t^&47_a2Z9rls=MY zs+u()rDvEn{c_-Wb}!R(^EK@QPk7vb6yiNa`y z2%(S!{u}}jj)aDT;n=K+%9w{x)Y~vT$W9ZKY@MyosKM|~S4aE7F@|O>G6CX1W=nNS zXT!Y9fs4E%HBHgl5Ec6_m;!8+hp@&g$uIN=bPY_JoW2JCH~N~=OET6h>a9ARnq^j8vy_dhne1PBp`)Vc3bsKXN6ZFZW|FiC?UDvfo9Z z(KA3_`>az!2eXqOyHy&G(b9YTV)bFE-G2#*^+h68MK(aOc@@tdj(zkrY=zT~lE0w{z7cboG_WUiOx%l5q*a&mmC=kkITz zM2n{AIJBoc5N~3Xwz`|^=HxFHF3DxF7@7)3N4D`EVlU7&xs{JPKejkwOPY_?KJ(Y=%A_vow5U2kJHR? z8|0i>W{xu*Qic~f=h4q@$-E-k)S}H?d;C&b{fGqmqvTnzUL+Pv&K#sLfqxM_^#ot% z+D(o&F0ZQ=CV4RDH&u75{#Eb+tB$r@**P^?b8K-3rmyBcTK%9%1uw*Q; zUmv-?g70S}Ma*kUY2X3;|M9}PEhHD{6u;`HB)52mltRutLj>;3<5a?le^r4rQrL^i zA*&xEIA}(MZ#0osGErAHX|iCCD3x~b2ZypLro@PV+If2hzn#X?96&K>lEj(uYFsdR zRmve{rSNCLt?};0v3hlVXMioRhBl>JfFpEIq4_8Oh}CgXy$1USaE2RV2eE~-*w4sU zhnvS2|4{_Ud(40(sw_H&)|ys8LKCz`QQI7&_5vX*`Vk+~e=;}}CWBSmI*!8P&tGjn z?uhJS0Ta2eiM{vcN#7NT|08p`Sc!U?m>~6_69XGbq?&X0|58F~{(E*3CvT`L`)TE8 zAymGUcPl>wR-UUTo8a=1os(r(%xdi4F#gSxkOn1c#f=cUv7!<~q*bCQ7TE$7Ap^A3 zc(3d{ZR@NdPO)|PtD;M+KrQ;=MkAzDBs{N^uJU-o6uEp<@e|a64wvZQhc|x6`Lr&A zJ+q5RL^7a4C>pHz5DP01bnXyKk37kOjBVtIz>a#(hgwcr4s@=$gSIeDk#t{CDmQW; zWL&D=4%KcW(<=4Y*0J#mFrPg?p|op#CHi=&<;1_tR0RvpQToYnfTtXwNW+x#qnsQ% z{d8E}DzWq1?Jx8{gy)p@`7JqlX^(J})X={ht#%4scpc>N6(-ErSjc+o@V>k9x$e($IE)0e9WoJ&0o zzL(^*<8+^d7>->xFJksp38RB(U2d9DXYcqF$GTrPHTQnHSFJt6sY24t$rq9_{=Rb6 z|1%+L&cLj>)z>p2-RAx&8xqEzUGv`R+_Zq)%O1lUvgL8?a6F_vAVi{R7du+9H~iEZ*wvQGanGa2bQz|Di?-2e=3s(q&T&6Nyr0)+GB~e0 z>^Hbz7qjB2LVw~fP*m1cs%C@&HDqC!gPG5v(gl!a8#M_i%$I4MWGZB01y&j^B%@SH z6$-4l8auRejC$xGuZ?YpdF;t1v}IHBNoD(~g0)!cz+Da;gEd4%?2`2VwCbfM!cL8e z^D-=$SVG`sdyg(6M9fF0``j!LOD7^ApoYPFKncnndhY=N5)}RhXJikF(tlY;&2VCXKfZbS#bHTPFX4`l(9bM!iz ziMbn-AdMxVg0Yu@5V4TI5-2Cuy@x-9@)Eqs%y#LJ6d5VCisp!0;`t)iW7kLg=O-~c zpvB`OP%ERHj-Wn&7l2r?C~qyYv;4%w-^K1_8_klYGMQOAC6m!hBEdSHLeunHZM83d}8WL(%y6M&=%wfqD42@qYSL002OnoLliF+L;9j1CG zQ<~f?oyH&-ulvv?gTXbA$Fhdi&B*Mk2w^{TW00<~6XwQPi<7z*uUq>L9@Ehxj-)ze zgTiA}l+tBG?5_x_Ro^-ImnSd|hv|f{9}=!1T{;q;K7s5-88=9j@w0-b1p905h`S{O zC^43q-u>(d_EtdvN*ae|r4CMhs>2ub7E6s4JQyqS-}iL_NBH_i!K2r_vl^I8oUf()thhIxKP z{bCpp31JrzdKwl@-N3>zWb~d_9^{3#IJCdHJe9!s`i3Ey9Zs#8mEjist>wnARV$|> zShhjho(K_SLq)}UKbfGS#mc^-&942;QkuO?m^iYN@07*8}xC-0k8MQ_+dH~qxgcOR}-IE z=vo(}X2TFZWwzkj|M>1m9rfq;-gWyFbh227dR&3DNdv4>GA_dm=BOr9WWu2&*9Dj2 zU)q+Dsct!cD$*^HP9<^vqRNu`(RM88$K~N!1M>d&t@@ahU9LOw^8F9wc6NQ_%htJt zw_pAmt$dyJT<|jS%YS4YXUaqUdRj*$ov8lqu>miocUH>b#qq!qW$WEc)>vzIqGlbp zU-~?_UnY(gn#qQi!{&Cw0x8rMx5-D#MLZX_2!9O4FKE#t_9l)>%-^{p7{F z@ zlCxAHT&+PB+G|b7>0)19(%D~kO{z1@F-2rkm|Eae`9>2HfqdEz&UZ`Ot*`shH~OKB zUF+89*|7V_DGk63xU~;H%em@~_DA^{9<%;wT>)Y=Aupj_PjXBL>1bD@$JLtFmjGJG zU3}#9CaqxT+1HBw5$F5d{nP6G7U{Y3fLV(vJ{9bCzr%v-M#4^n?2}XO{n0b1U`F}{ z>6*e`N;L|{DlD(zhrKsb9ZN_ltwrsk(_8lYt1J*V{*I;2l4no+X?USj{vi$JTfV?t zv<+iKGUlr|;oDcpS4j4~+2%ld{++w^U!_&Ow-w8okRcUUaRKFZUz5^evraiTbtB2PbX=GUzic? znD`0HkFOF5sAy@EN8KZLLN6%0;0}taPZa;)uThp;o7Yvi%b*NP!cFc|4=Nij=;5!-*^{Y>H;V};87JFV@(^J)AFe{nOj>+xN-3ZYalmx2RC57k zKMo>+JHADXS}W=d97moKJ6?*T@xMEMmFbjIFkEW1nEd`#dP z!wO+>tv<_5;5e}f_Qf)3OzUHfe>cdzCQnNTQy?FF;i=DS?nbcrtb%GS$Gt|eqj;I= zk1IHEHR^G?0$`UyM;l57pE>hqa@NLXsnKS)a`LHQmg~mvjB)D~-+MktMBubG3NbL` zO_KnBjOO-a&7qZ>s(>kitK?;^V1@Q6p_m zgIk$Q@O_Ah2GK2dRQ#r~+II8jbBM4~3UDXdSaw)mREYfdq-IVX#bMtTeZF ze>&!fgz9bJ1$Ji$#XM1LC4OE``je9OB(GA39mTw$7XgerCWa{*pRe|a6{j^cHcKs_sg{Qg+{ywg>0B@5}?~n`A}V$ za4*ecxt~wEc3{oySQ=qe{vPK>mvE)4#q2Zbux;{doA_-Rsn5GLY4k^* zP_B}|HU@Y)5n>hr@}I__XTu^?d#^Lj2-gBW6jq`rZD<~SY(YgbGAId$ZQs06wt#6G zKrNiYEv`|(KE0W+1fbRMU6;Qz=^Vv=Qf|USQmas`uCP&Te6py!<@4@TYFixDWYjm) zbwAFie-~Ru-6n1tR4sqp6LgAo67CIOpsUFTk)NP6kL!&n#UK%-It7pC1$6*?Ph&u+^}-yJS>itbDaJS%O=a)er|$IX4+cd!!}ue zOM(}UP1e)k>UG*NEQ^v6Q!{y!A-`xLwcmOB3EV!j4?3a^AeFR`Z!WCK)vHr;k4irkBRCf^j}j4LhN40;;s0u8D!B zz-fE*7^d0dQP0H`!@-6~^Hz?C8(|Jo4a=K03(VCoWP6yMrkz zkcgCbm;!flDKTXsihu&l5bW*U`N=18USi-n__+s7?C0oQ?r7fk4jwEwxrABqW3a*djTy9vq!@AL+M;6KGncx{@& zt~N>e6vxNuN z!a*3AP*Rr)lR*wSfaTL&X-&|kQTVGt`y3*9UoA=b?F8>91s29g!5$doBXq3j>34O| z3*$i|xnpJ_t&LNNC801OS%FaxfN~Dh{vJP*t-!nhb_{5>SY{gIyBcrv|>1U zE@ii|8kM>3zh(S=boF{mT*0Flnv@PuXeGVU{E=aVUH)#TwJQU{5vn;Xw!`hdID|6h z{UiK5)mzcTkgeKS?N_3Guj?{Klo>P&ifCO&$t}r<5Bl*0I-Uy8n(yNZAF?m27~XG4 zcxTq26#XLK$9UN;BZjVO55BgZMT?@RtuE6o4eTurtCEPiPcF9!n~U&VT%=WU_94-c zY@4Qdwz?rw;RJz01rnb=H#N*>kr#b$j2+S*AZ>BN72nobnRXzh|cGG)C;-( zV}3pp4cZK6&f5VY(os_ooMDzclz3v&_MEEu9Ol^9uJN;C0OQuA%7HuFTX0aVpKDEd z#q-vulG~^H6AWg%`+>#rM@^#u=d!v2p1pd=<|Y{Qz}XBq}SB^gq?v$*G_jtBGA`WIou?xgEnFm;lifLuv~ zQ*N&{T5HSX8Dq&!(0gUL=V@eKEUIIg5rzVOV!oN!*~3M`-vUL$2qF6(1EdZ01DdYVn$LgOc&8EhEB`hG^u91sT&s)wD+`jF*R~#>=W1EQh^@wK_L-Jw6E>>KT#m3$`SFkhBY2>WK5+tb%Q)Th);EPAypL6at0Txl!@6wtp_zEJ`Qpyr77wY{&^8}>Q|V!qc6X9WHCJx z?$WlvU(7M+818HlKn*ZsbR-k;5de$|H{Te)5rJ} z9YO;}X2?E?>PM@)kGT{rpZ=-O(Sjo)$`*bJcNC?#?AWB2of$9_6*Vj*Ym0} zjlk4`d=X32FFUempbo~bn5cB3am^chLlBIykBbTZ6CL}PgPCngz*Rnk-2!P;!l78U z@!zCU|F<@2OOovtLT=W(FQwDx(aBM<>bv>fxf(Xdu@jDA+bB>a~JondVCk_*sy zHGDkxt3Wys!AvFfw_MAKw_pH|ha{s}-8j=UemsJO7M@%Rh*FpBwdn1BZx>h1;)r6_ z_u_34>!kfpzeLSNZg&WRU zvTrw!SzvkZaqgV393=seK)x0XtR|`4l;=F3Gc`4qx1<)H)kchcHNn%Wd}lHALEY(v znu$ELrlAkw$yJIHC%-<#G3<2rJMGF)mU-5qBGc(Edgk=bvI^BCYSc6lX71U^CL6_} z{#Xbnxv=DDzlFeIUn6F0O>f1NO!9qVV^3B zl2Ef2u%Yr!d{NTCLe-TVPfz@um?H{f_7&WK8Dh!V>#BfTg2((*fbPe)o#Ij38!=0g zE>|z9{+O3|KiNkCK5!wo>irsAd3@HfpAv&a59kc?H9SEYJdL$zvAD;tW&pYXUsbqC zez=uVZ`%XY-96jQ!ZwW`DaecZnGM|z$@hIX`bU{^x`^p>3{iXSry*{-wU3t=B3 zx8B8F#y0w%=3jtBk6w1ABFp#7ZY`^_ueJhY=sG0W4DJERCU_@{-Rw5O&GDU7i$WoG zvv1j!|JE^H$%_|g7K|41<#uuUXvzk(bm<>2R^JJTkR)~d0V=QHj+oPpYQ)aUdM4f# z$!9BOistZyWdjJa2!9_ZXgyiQ-q$GMW1j3`QfF#rT4RjJ28(_}8J%^1O3+{?ZmOwM zKt*qw=AtqQ-o6p6yGZn$-N}aS(2!tfY{edfhh_{stZ?dl6#9YdjlxMt73tdEnw`-{-Dy zzi85r{}x_pDO{Cv8o@#Rws|)6l>el6kUF?%|2WrSA71H3u;QC$|VvZs7Vf+3Wsh7ZO zp(;s(#lLj~0!s=Yy3bR9Ae~P4UqUe}t`-xb#R0#BC_9DGw=BL?ZInWpb$QZ4%(q+G zx|HXCUu-s+K0K3d4zb&NRU#wAII~?gRX41-w_t;*wZ~62;s!LgGpVn3$3^Zlw@5s2 z`TWr|1?k}^n@iwu_&D3OWoN*1OuSxIiBim4cL8;X@jY5Jo~*{dWCW(yv~}Yqj?Srg zS^<#9Rx(cn>nRZ3WceV$dZf@iyN@lww?7`8SD~qVIXmhWv?SOyJiS2*FcG?4clgsj zOZy!3cG7^qTVp;gwxaJuxWn0mGP+;>CHrsG^Z8g&;#WQ(=?(4M$T~J_r>Bi&D3xrB zF6v4kS&M~$B7<6A5m9pSTY@1;b-La+sqtMI&l0IVJbzcx-EUSBZP}}`e38mjZH3E~ zR>8ymVSa*K35TH;VV6Z$wR=||>$Au+QNi8dqLqtk0??E^+I{}hj5((MH~y97WQLZQ zF0u~4i>FiA_veq>|L^%1Nl+vgF}M;~@D#k&I@dks60a85%tIc8?P z@FAfO)orMFW6^X=yY(q5C7M0gq!SU$vZpH2Yta-XMLBf&o}yC-olU9?Sm7bn@o@RL9SYf`?62FRtA?~D;}yGA6`$HP zYhJV@U}V7sT{0Nl2mC+w-ZCu8uInF`kWf;TkOmcz6r_eykdhXq8waE%hHjCN4rvBZ z1Y@K_x$j}kT&}Bb zTBrqIPHuGGewKzJW_5QlA(|ByiD!fV2X(> zG^*mfVKRR!(WAyJ(rEtf@Y`YHrO^&1+N}pu>O>unlQc=cO>74V*-S&L@#o!YK}Ean zv-C5~i&vNnzRL-45h>|z`Qjdvza_y#Hry*P78nV^ zk=9ljdN0M>fc;TSN;rsZcpe{seUp1@U&V+m2-99;=4OHzZdhqq%UDDLxerYX7HSk+ zW#S=wYmI-CZYO;a)S$%~q|~hl7G}M{Us{UXbTk#J@= z^LfhWRvnKi^MpJ2kk?X88}69k`KLAbBjiR6wJ&9W#xYC=a(W$iIYZ&}9nu?uuVpQU zEWHDksi7q*;CeO=oxlz}x$kGJ*nE=={xOvViKX7-jEhFoPHq6@C->KY+og7KQujs( zeugljiTv?A3-+GAn!P%k(dr8=E40X(bWZOcU7NZQuiI0D?t{UvgEeAug9e;Kw?J*0 zqLLe{9Vs(|6$EftdQWl4YjMS>cXU@6gk-^GxTl?yiw-;#l0h8l#`DIkqSH)Hj%;xX+r@y?MpvhoyK zNS=+#L8^hX5J(eTpctIm!=;%@E+wMsRT;q6R;b(f@s2;$1Dh59f;Qc>?5eWu1rOZk z2;$^enkl+AW+v7LgOu`g%VVMwvn0X8{2Os3BA<1++#I>+9rJ1$i0yhmdMR}Rc(CcI zO|K|kG9g9zE&=k5N{wgazmwOD^0!hvPbDf6JdOK=paOI9!&20Bz>ccEon=7WSCzcj z7rapQ92+2MX_XlT)83^cBYn>D+zLTDPpxJ=#zXsTRv4M1N>B@%lH2a`_R^KLzS}*+ zJ)O$=rFgAGq9N`!U3oa__7Wjnzv^vr*o@AZDt&-(XnUIy{qv@=1My%Y-fanE;~)>Z zzFVCx&z0r^THH9r=Bdz*MJ8ikdAwKYtR%nJ6%gf5v!-RL$fhk!zMnUcsR=0+gIROA zz5ENQ!|uYu??#*7?YU)uCwP7+bR{o&68cGO;8ay~qeOG&4ew3s*Ld%!7o@!C6~EDS zW;8uNhAso0-qcT^F#%3UCurO01X0ry0@}V(~!tgfOcNw$SepyOiWWwtQefHCUx5+ zgM{K}##T$8(&1R^zdztt^iUgCi+b!v{&V0|VoGi>s;ckP)s>MYDU)X7|6|01X7 z!RDmLSK6M>b$7(5?w8gLv%78DPo?kyhVX-%LZPNnwR^ze*#y=@aFG$uy^z>hW?iw@ zl=SjsS`b5j98gpG&3&P*d=wmP2vbY}W-iC^vAjQb-6sze2+5u$_W0zS<#XNg@Up=^ z)0ZtAG@Iwkr+jIem zqi#TuUL;Pz(Toc>sGWMu1gxwNS8NEv{t;(Pb%VQPE+?kcOA8@ z_`YVOkD>2lQHBbjdtt6#Mhmrtc_t|jHB>NgUuBvqob+m86=om4NTxCuL_y=(yFx4w zb?vuTwgupw6QSs(?Pq?+u zc;dM8UT^Csj9ePz&s6xjTYUrscHaX26sNs(oFDGJ0!{22aO;+;_+^6l^16$S@yW&2 z8U8!5#Oc~?PJQeNir0THg>AVbJ|V9By{f+|>AB?Gk}Uz3I3~ z$MK>pQF}!)Vh@y}MCyfcNvg<58Y>#Ew`Ql|SI#~sAD_7=V+gxjr3@N}$tApP_mGfq!8*sMA!^*{A{0Csb$Chh}m{&u=<0@%b zcszfRev8^4&_+P!I&t$lk{jsa7R13lO+r4}ZDM|u^XHWX8Z;s2X6qL{~Xi>$d~fW+L?6bIZ;Ap^uv z#E(7zTgr$pp!F=HEl2IIu?{mpPnXL()wdtH^{_6uVA_aL{9Z~J2mVD}e*sHjT6Jsx^5bH<&ec=Ea8Q*2fsm~`Yhh6;c;Nh$TL z3_Br9Ap7KKu#ea4UkvlZ)x7sExl=#Ky9XdM%6Evpj4XF$J^IOB0b*1~ViCZP6HI z*_W85F~yw%pTGs8yw}YDSnE#=LosVM0>M54$e^j1fIK}a9@Y{a@KWlc?OX85CFY<~ zCng{R+JtPGpOy)Eu01gNi(M`WeyDWORM;0e{RSuJY>$UnglW9sO*YOZWkBycJ_4~t zc2!G5;2&j#H>bS#*L3R{$bO?r^M=wKT4_9^Z^iLqr19~?H89zCv4Gj|g$bbzFqk1l zZ@*K1@o+PSbJ6t-*+BGhU)lq9P=K}GYc{VTOa!ZnjfL9x0?<>F)jJXP zHW(C(CmF}hY${Sk%3uDH9C_oPglSjj%l06{zhr4mtIosebTzlid|7iD^(01PJ1wXx z>%~*-Gie|(zPtVLe+l!ygzm2 zCF)SZG~ot*LPN^)F4UpTY9(6M*{EkStr#+pmMQYlpE^=}@5nga7A%RTw27cIrI_oYA0A+fIYv6a7o9{@$A!kUoU*@n7EG z4Zv9~za_5!Jw+IJN}o#S0;AnGz<_bF134Q1zCb?|(joMM#G+sD<0^z_u~@%Qg&1{aca#CJiGj7$uCS zk(B}ZrGQbwWLsI?e-d^033ODugjfaUO%P+NG_r~1{`t_of@@$>*O}ZyF~iXSHdxf} z#Q&4)^2P(;7a`?|ZSascu$mS2B32%MXC{V7<6Mljc0Qy5#?AO9#{YWufcpAB*5+39 zQ}9brUgc-?}v??7_{zzguqF2V2rT%CdoKvlV9RCpGU z*-1=W)Hgi+pTz&&Ztz0DZ2Mx?zzhFit1%4l@n`X#3Wi>wjU{w{qvqeuvJlMDLW9&X zJ3}mJtC*+8V0MPT_vP;mmbM~a#JUpjN*QC}Zd*2i7yjcREJ99r#E3w;x2z!$2W*UJL=(zB`(GeFRL))jqbx#tuox zp#Pw_on7LBwohuN&uTux4R|mAlba8K^C99+NsZxA8%DU?@HRd6krHR?OR#*;$$Ni} z19*-DmKctHx7{w&u33o|6vpTe=z}m+!-~#2e?N60JNKu%ij)WoZT@(d2QR;+vc6skd8q5 z_JVzngQM=v$3%H8#7vuAjX1lRwHaOg#b4t(La}udQfrCJohTt!fOVWgJ28WIlkM>8 za_cH3<8n2_F8^xD{&Fcuhp2Lg0FlCD);vRzas0d~JPE*QFLq=6xTTL;)uwxcxe}!n zy&t#-J;1(KS1W=>6sP!}B>GxP>}QddF6-oDvid+J6QpKb; zo&5V5S0@?WJXgCs-}1u*T7Ma0 z%j>%@=R0`bBzfM%{wy|auzoS#db4)Z5<0I|FqTo|F3UgvuZ zWocE*{n^?}-}2hvKe+(>;J4^JAFsR6+pEYps|Y@dI$0Foa#yXQJi`=8d?;}NSfjd^ z)Fx&=YitLARfWo{7rCfveKL5?0fP2B?J0MOHP&KL z=1Q!L>m6{5%8|Q88vz)nkQnOcvM=P-B=kh9X6>~q;r1U@AII^srRHJp$zd)5wbL#O ziDKJSr!8f0%mb*i6~sn*LO(Zjf|JoF)gvw3sKfk_Wh&ntDxuE969xbwDmdu>m$Qi( ztaR8XvAzQr#pNdWRiikXgQp7T(s};Y@h@wm#mr0E;iLJpLutuu+y0ufKuOZ1|ZR=_SjZu?9W^6>G(WbI1C@V_y2W(+hM0i z{FyK6A+RYTO7Qnfuv`tNSLhd=+KGH$HL=Kz|5d|B8*le~dzIfAcX%es);;K<1_n`o zy!!mB(dpNQW1nSL9}7?7VM^=bDfGv8?%EIFz>z#M9B}3JNt8W!M(!1<&*zdXc{Jff zIjy<`+5?n>a5^|X<>Oh6$Uz$WL^&!`E}FasHMefBU9UqUVjFgN;uOnMA4_m zR2Q$4yFj@CQp5Y{?sS?SWwzVz|v*CT;{-#0N?v z>Ar2@duV|_NF#$Q&Pgh>EoS!wZ;OY1P%v%ltaeMZI>$!do%#?_-Ce;5_6N!8qikPx zaMMz?;FQ#5Rd+ik*VeWFa2*v<;hQSa>8F+2!{WwG|Eq{j)6plW$Bw?B23$p{w(MBZQraM!py`Rjm5GtfJ7wb&k@)sJb&blGqTR}EvHx7st9J}^)wuTSCQ9n?Rc zquMg;KcnsJV?6d}e9I+KN%_Vs@q~r$NtDOn7nM-CRX4%F4^1!V6r1y!ApWZfKYzdG zA}k+mjB{nj1i7S_wMXAw4*7!SgTL$LLw#5UDPK6U9gp}|ta41ckKJT8k z`ADwN#2Gnur=j@dd&cSa)83dZ3%I+vC;SL|D~|7BZyCqs4C!TNshC{Q%;au-uGr$X z1mZ3RfSB=e^Q_T3Fzmx31$;Yj3Bb|YPeXI*JZEqiN6zPXE=|WNxv@>HO*%x^9ilw- zMXSOkdO=l$wTAl^5&)3A92LQ-6Yz!p8GWY`KIchc+Bzhm=HYvz4U@4~fo(->dlPZ+ z{cP4%mfC}U$Ko+K;a-qXwTZPtYJv9453a5>Zq^dV4Wwn1@UCs$Y@cO=R?%NX-xzC$ z=khad36uDjuqLY)PzHl)ZF=RqmGX3l8!2N&#h10pt8bosztnS`SSvX8q0=&a8~nu- zI@Us-@3S%N^GiJVL!4a|L03`qNL~^Hk01~!88WL*5Av+Y=f9PNtfS0YbaM9TL=qV# zq?rpvGqj^WwKg`~$$R%K@lG>*^1%;hj0x6AJ7b7fjtot_2xwi$f7aZ0Mc74XvDdAy@y$S`}O5(C77hTBnE>ztPS=*=DqK!`Eou7{y($QO`~|c3CW8AN&1Rqj=Eir@gSALH8IEPvnGpgif^iX z6O|v3(6+aHUWsq-f0{cReywS_cQD&D6a_cR@%MgVoQ{+F!}X4s=*;ptC(7s7Ti26L zqs#c7aj^!9{A3nw&gSvmqOQS4kv`qi{o4U-^d4~5%S!sUx3)3%aihZY!u^GZAShBj zdfyKEYn*kK!;PV#aFFym^5l0~zeGkKw?V@$Ei|sy<&?jXDspvVkgFIMNd@YeYzXLU zaSEy`@cBGm={d0gU{$$k88E*xj-$ft_Ro0;h4Cr=!Oe@w>5Iujv4O45l*3%NGt6GS z;f`04po7ghXEtU4jS9d?_?@(92nu7AnG`O(DrB&&;~8L3Ov1!LC63bms(bh5dxTD; z|KmIk&GlmLy);}+7T|^7DqFSpXfvvt^OJ1uJipkXuglR|2p`e$b=(kfty1)~KTlzK zZFVzyub&o2(y_k%_2VJ?_vV9v)KS$tsZLrTt+iVMF^Z&PNy(O8@!~1Ocb3X0tDL_$ zeKxrW8Q?bqF`{n zQU+IbvmqdD_hkah*QiHRApT z(eIp~Hv6a#o3esMjUXL1PMc+Fe7898X{HE^ivEv9}BEU2A+rygXG)IZBHzY!a-!A4wcUB>H+ZLclx;_KjCRlrEU&sxs zm>c$lce12!E6O!Oo8#-6{T&|}@XtPFcv=021Y~JjNTs&e6 z>1uw*Id`<1LadWy72oFCZOOHgxMAPZTwjkD`bkClZCQD&+rh5d(@eS>>JZtGa)a`3 zGs&lLH%fYGI+xMqeiZQ7&eq zp|wK9gW?8wfx|}(Gj0<_mR;WC$L_aqkrJCgmP@ww5IGTQXDufs--Gd}(dfN&NW29* z%rX*rrZgSmvK(zTbEB@xHeztNx!jJbFE>R%TUyw?4OFyJwc)zA!)O-G+?HuC$YDw$ zhQ|vI(KZ`Fr}p}BycUFTotJ+89XgiA^`8!zg2X3}*i0h&NE#R?|6p$Nn%pIGR&>I? zXnWap{L=GV&oP^vHCgogmS~`8>9&_zSD8g$7(_+%#hXB`4s+&J%eU7Q#Iqdcd-imu zrCl+KKWUf0M;VJLg9gCeIfbjZ(fa*arax{%1I3y@ zZh(?(&(Rj;>QDaK9mv_6Lex#D2X?EDZ?|E{J-yuA=4J;{+dA5<0jZxf{he zdGLd8Bg|2=%Gs|Xxaj>%n=!%M(++JTIoCcuX=z9T5doVsuW;*TIYJ<1|8mIa;x+v* zjam>Tm(#PTlM|WU@LI6YIs=AiBZ#sHKOW|uBVMM-{4CulitGQwCdio@84NDK4s3sI zCp~$mx{;&__vhVxzeup+7aw6ac0_R0r}O19RJuL;OQT3myqSB06Oy}l`-NN)ZPWyk zO|w`*&RcAnid-zzVsssGu&h|OujpibR`JJq=p;a98B{hL`UtI_-vp;Jh9$*ei0_>S zio*pF{-?3IL~7aMF{*I#9y6Lmvyju@&Y2?922D%e21(l>of(L($kzA_T zkc)-EG%H`6(BQ8U7`cdsQD4srD#m-ue?I&1k==Z5z_MyrR{~L_>7oJBYkm^maBVCz z3^FE5L!|F4cW|7JMH5)doQOqit~8X?iG6|Gi?7YO+A$`ns(cxmp~?I?EEoQo`ig}byP3dCLcFn7T08P-@1 z>s!kc7u>;x3>y2*l}-b8lh|*B#M_8Tv)-+I&!p3 z(-}_nyoQ;Go;Q*?6ZT7(kd;RO5>@*1mjan+70F{<-@Rcr0;Uy*TI-hBM4siher-bm zY%I@RUqwZ}X#AOoWBfuMR~x}sd9>r$g-0_{_YuTgdL{MCYkRL-Ih3hLT9fu=p;xbr@6kX|F>dKgl`BP0#d4m5U>$5#;Z3*m7nG7$NZz(#zs< z8WPeA7J`JCUI(FLFpClnw|<)YlI*Oo(Qqj2Zuj6P{kC0_1LtCAwaWLbu%wV}Z#zvn zSHtAob#lhLC#BbA8lMp?cxg?vpLlVzxUuw`eE;YxhHnro-z;)jNvT9Gz0&L~dV1XS z%6D^~UY&ya*4Rr$3-2!LSl35hY&D#e) z^T(4DTXbhj1!v+-Wh@WD;}BrEy{ASe0pK!uzjP4$d((=7BkTO}IANdA2H z^Foeyd9uZM;dHb|pmZjbTAqRAf_cT6p0X>O7R%pNOf#uaWfLvNW?vwj$0Z38p<`gP z+oV{`0ldqxBt4H?s?o8ogRleR`X3uq=g7+3l7Y$%iXLzeb9HV4=V5VkB(xWxyjOPuEHWm## z`Wjm%2Ug3!{)2$o1Ovw(v47s2yG+N?bz;A;oxVA=2(N5 za@%j7zCYzg6&{>b*TS(x&%^4IutrKI9XHgjBK8R5E~#zn&SMU#r_}7>?n}uvTOonZ zymVs2j|g$R41{Dwv(dSAs+fEkAw1vq)M79)e3Zf4-Wk z@d6dEC0I49E{rR~(btgAIdet+Km=vL%9T76`yjN|`Qu0@UgL}zW8j!8C95}O7*mVa3-@(U?w;rDi0pQg(@@BMU zZ()1Z05EN0<-#4XyL$%e-1No?C)AGac1itXu)Pj3lz(=p0PmT;0qM&`k_wRGy~c2P zikCqaN+uSf>Z-;uN4&b~q;%EcN2TfOP@oJSviZ|VBzBL{QB}fs$-*b6rra=!3hUwJ zfpSgTtsuzfe4L=V!t>9FAr_6s1(3m$;Jn~_NmjiomNPHwv06O@8_F5@n5CCS~`R1h70{sZnwX{%LF5nUZTECbgWQ_ zy?3O5I)y}noP$`Zy9&mGB>ppzc<@@J&dSr@lNGPN0J$@9uWQq|UG40;FNkpaF4|^? zkyRyctcvL`o*LSEw>ael6*KHsLOOOBkG5=W=^QDpw$|G?M%vine?s547OD8B`syQjzcYP z{)qZB8``zVz3Zc`{IXN+(4}-;XIOyK&dc+0TsT`EsEPOs-CKdqJ17flcMOZ@K7$g%R*p?L8xwp*!```HF zSXZZ=mH+RC9DrO|c6dcHjg(aFZ^DhFvK04YKDp42%sRXcc5n~X_?G$qg zzaL;c^lKm(Uix|<$!oF|gs)~KC36ijlH;uAGTq~gPK6dGSr)Dg?(f&+f%;F}6J@_x z+eC%YSgiH^E^IbyG1@~G1xa6#R*7E)3f8hCHz5h>9Z#3dK;<{qTvj*Pf^4G{NTKN4 z*jh7j4yKLJ2}We_sePJil9&^xbb@5UJrgn^?>l3G#i%FU#;5`KVF|A1!rt$VXjb^i zd@~3=($_%cOV6zD?FoNg9yDcJiQ;(5o$B3292F$dNM@HoukJQXSHYUvnabnkVA3Q$nv(`R(HF((SOpw8WrAHG{s<3 z^_i)TUp&9oV#RZkAX&?$@r@zP0)+F%z%h%>-M1kbh~yFaXOVSY=ESXeY8poT`;!)L zJsOJPYmk!XZDI}l@7bQrIUrUQY34+}8K4frp>8Ol9O*t-n6qCnr1DtcXV<_gwMXq~ zB&iP9tsanIft2i&Tk!6q(sZpfhdh!-Sv51K|L|z?1GG;@oi)-!H0^@|VMLmD4@yHO zor!d=oDvj+izz==4ufrG6eqCZl!yM2Isblza;~*uEzzfLTM#?b6yas;E#J|kR{Ktw z%S{5UgPiMq!EL>HYYGR1D?gK5S|k=S&w1wh0B>vFV`HrnF+6SU@A$eQPC#LPxpzGj zMgT_xBhKoFqx^*I9 zZ2aEm;Dg#OLeT3VJi-k zP#3kE-W_!MyicbsN9EGtdOLJp(%cP^`^hrEVAevjEmasv@v8`a<1>S`cK4~7?GIs2 z2L7QWOPcE2?z00)xdSouBd$8zO03CvEz#p8m`L5^ER9x=CzcD_p(7eXsN|{SJsWC% z4qp+u6bt~)U)BVYKw8xKW9i{ksG>#xX ze6>UarJZ5d-rY&N*?D(qvRg`<;`Wuh<|uXG5i>3RZa9lqc6Ai9TM@3(c~gHa@|X0s zz6Hk5UuHS*78HL^Ly!7hYq2408D2Mj{C!4*)2&F84f=jjs#2poE|G@r4X?=Se8(MN!6mUYnq{b{ zmtN{P`=MSzPa;xKyseE=>`xr$q_iQ7zF3x^JYRgI#^sZX{4l9+hCaCS@r*zEX=7aS zhko=-EsFi=;mNePpo`lm^QRt*jipfKz2`rj9|ob05luhpEbvtjqE%Hzjs`B0i`TWQ z42rhBTKVC9wM^*Q8Y?>Ez;5TWGaL;y0$zFEPswDvw!6=h z8JgRz%juv^DYf0Ib5xcVkM7dXZH9Q}{mDhD$tlr7Qmz7)V`6{wjVd=xB;# zu$yDZnC!V!Av4arcfND}2NNbF=iaTGR*Voi>tT*nS#70cTK)=M-!4jJM5hhe0j)l5L*9KqC%|jB7=r-N+uUY9pzf;0?rN&VYdGE80h?9Es#y2Px(X@5temQMX;0L3&7TQPrN@*S2vOUYgD82a} z%5xHJ_oi`I=q>~pm`q~>tHw@DC;G+#Y zK#*yNO#5h>DAyY7M2*t}EX=E6XJ2*ry)|p2%U+K&PyCGgD`Kyi!K-M%BjcNhbGU%m z9WgDW8^p~|jc5w&^q*Hh9f z$my*TshBPQ3*JD0q+#ix1a!Jy~Q5dD?3kS-2U1vG-M{jPLj%IY`~u4K~Ki}SVgdxdLX zid*9|vWUO@rXm+kuhbcE@(Rj4U@!m9z7g2|l6rwB!O5G2NFzkU?o5!Yk+j((Ph_KH zU<|oAXp`5_XEG1LWt1o^S#$cgivWNyoZzcJ0+A}n((7EybKf++>seF)-%?wOf&p7& zSbbTxWZf~$ttkiF;HT`l$-tfSuk$a5UxN==tAw7otrBuH^`G;&Zz7&432Uz$isV!{ zF`$K?r~DoaMHoPk;P$pca*GSMz{9&wIhW(a5mm1;BMqhSCssK>u!Z|CI`z2bZWdQR z?J2~SAC_^>6UuBCIMA&r{d)OhVmt9G4LD^%%p1Fw9j+slpmHmw9_$;j+DgNeDTF-k zams^^Vi{s)&8dPqH9MM|F+A>n6b~hiMmLf?+miZKyk5dhMwi#V(ya;N43m@^2@Bop zaU?1Kmt`3|D!F4-w%l?!<5yC#IOs-&~rFiT%2t(fz z)45BKcU~B3ve92_oQ~At9_AV+NnXrdR>7RN#XI(xO(@j&M9NS45A!`E0+y|=`;phL zcPYzEmAW)}uz&K($SetY3&T_`KX~7gmWZXnL<%BmOKv$q#N6|mepxT3tB@}1;2`pZ$nFY6( zsNq&#o+8cp?+sh~@u`8*NN~Z#BPf}=Oi3<=iGpJw>E#tKbV*gx+J)*g#2pDN-gv;N zLD3NLjMbd=o}|e1o!=;Cw?_Lrb|Goa4)WiA?kx%jSmYGBx-U%j@P`knZo~@3+nc{$ z@NN|-LGu0XO+WoBhD({kWXG_xSoU1UDq#NPCs&geJ=1-O13W8&D%fPFXI-8rxuiA7 z2wkYu)_~ry_TH)-qx)GVsxjS^`0UklBsBq3hegw)F_uYUc+eEi&z!=g5?he>9DH4M zs)a^h?sh(Y^EV^2^QSfS#Y4*@A%aM^=~jsMky`G|QSUnb=4{1-t+#BC3zVpvKb=~( zLzQ2S;g>$UDSN}Rn4e%!AWz=tGjdmAYb3kSe#S-#Q>@5=P`%8s|ED3}f zR+ADd(R)W{>{ys6Tm_Mp@q%w0=DSrKZl-r2Uuiu*o>Z6635ikGEVsl`aq^HsqP)4S zzxMW-Xz2+eLiC=Oe4eHM`J&NeOiUw~3}<&v%J`?8fMvFY|J%GQ)=yM4b))JMM<|{F z0}3%q`GrMQkC7cZ>+ukpV{wf=%+^Sce6ac@ zYY7~-xi{DuM-lI8U#ySD)2QFTgwb3kwZgapd zLb{{6l*RpD3ntLp>{B)bR%Ro|LOkbz>{JM*?`Eg&X87YGOp7ha2PUi%f4_;lvxe3G zdYRBVtFg|d7*ux4>A`7Sc9)xopbmbgLmKFyD*mp4zso~^wK|4-uEU(jfuKe|7^-<; z*Kx9G0WsUlDZRg0JwU)}r?JXXz0g?{z;?uU{qf>8i4wv&Mnt<=kNpeqg^D)aa<0AP z&>bYfDvVdx5e015s)#Hb7>MGTnVNL8MwY&1OelPlf< zZcg@^AC*59IV=bkN2o}hiNiQx>chWcTk{yl_w^c_w72hN6j69~kb2_?s)xgP+k#Pj zOPA|jJ!kL8<>+O^{p+MDv%0zz(|cg&kBi^Hp<)RcHjeS0+JYgjcHxiQt_}zjEIW?~ z;iX?CVK-sE!(TJx87o|*(QQu3jrZ~aEYKJV8rE@(2bos0vKLbs2`w|hbew4I4XWbKj@l6pvv`hI6o z1RW?0-?tCGt9yVE(qR+v=W_h%5zWa#ENH@(-9YQ&2h8-{P`WuXlircA0x4k$>D-qS z`~1&|$gZJJuRo4n-7HN~5V`9}dTJk)G0}Q@DKSo{`y&aWd~^Sw1C;8>-5V-Zt{}So-%Z%K_s> z8ShxHK>UNL-dL4lxIllE&E2Z-veoi{W#Cs!h8t%VR4K`)mur8_l}Lse zc#~dm;~Bao7GvpW%+o*f{$u>;;x;NHZ+yV}Xn}gm;q(U$*`f{;0uz(|o|bzld5c19 zM@aG2j?bo(B~BjrV#mD@C-LZg)qc$PuqxEIzzmpg)q$STgriNei~Qz}1~zknUBjLi z?kf;Xndh80X8l6WVzbon-6Mahi3`1yPPk9plE;x}{dYFc$m+e<`8JoUp;4AoCu>@Q z(}$Hp2l|D{($K*T`KKV#o;B*6;jGP}$#dwZ;n3~J-`I%@bDl&l%jQ?lr1X>7bbQm- zK;WhH>m-2-f$Pv$h3v#2o{+BOlGq?|9=i5MBS%W8MfWp%#2VdH6_W3C__lRRj<$7a zxZ?a*cKXm9B?Ie^y}j`&=dA!Nf3g#wN3<;d#^Tal3S%0LUu^}UeL^j_)_*sUOsUeh6o0Zv-A;gsWyn0Y`Y1C{ zIK@gR6z(<^Hj>)JFFE@fb;k~!)1PJ8QQyTrOcfTNV3`~=g)+<4FId7szZll;AXOEcIu-g#V?RUL>fq2Rp94HoL9s8f`l$6TJcB8JLZ3*|( zTsmw;p->IKoN#>}_wq!0R{tVy-Erp4Fug(o`;Yex#;Li`rUC>v045xZ8lohn=*)&A zbNBl=BR14MHb<$=rlr|bp%g|4U@XeHV`b&BIe?+n$4V);;qosE)YUkIzdf@c`n-Ct z0Fx96KJ8M!#}(vyY6q!q2=tuO#_b4FQcji{HzwDoJdv2w0;ltMO9x_B1k^)MBS(or zWQ&wgKGU+}RbCE)9!NnSv8gQATgNv)e%b+9a*<5QZ2 zPU)GL#OS|e^l2x9H#%0T)xlq%g4EN2fdAX`{|xfqtiOM;?7vxmeY4eZD7ys3Z|LVnm_2U2U{-m)r zE>fuOq`SMJ;hEDShf#&d^huA9ht$Epz^4A^$oMxw7_rA*30n!r{6oK6=J2QEoZA_n zIxNG0d1yO*;Ly*)085m#a~b|GIGz7`x3yB*iUlk<@vid{@ur)!6kD8K?vj3R~K9M+~B*UAb>I$(|uW8 zFmr-ISa5F>o%(-v(alNw{MIL}^MGe3!5x9bOvvXvvTHI++`muL^Yv;Ta<5li3v@_} zoc4Kc;0E}!z`b#j%eSpp_5KoqRXgxNCRc`ypF#Jve+Hevy7t=KfM~A|Puj}&zK%T$ z+)|*pZ`w->*=dt!y_BYRxFSs;+`THzbZFW|oV8-B^`r(E@`E7w&_dphG7Ty}+JmhQ?P99FY$^G7YHUie6^i+REkRKYUfp)!`Gd=mR`MVNsUu&aGB z;MGK&3QU0tRu&)l06Q`cJw_WZ0_QCI+JgGgqY0J(2TPLpFCS~xwdLXWh+_;u;OXk; Jvd$@?2>=4ufkyxU literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-ps.png b/website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-ps.png new file mode 100644 index 0000000000000000000000000000000000000000..61ce97e20f8a567538e66779bacd82ca8c186091 GIT binary patch literal 95852 zcma%i1yoes-Zze*fRYkQBNBpu%FwAaNOvnCLwC0*N(@L#$I#s!Dvfk^3rI=~Lk#d8 z#OL1o-g(#hUe+4s%raF6J%Z8;2f4 zNi?*(LS~|(iqfK@G>Y~%Z_O-C(9om;qcv`8s&tVg>qLAOfuzY|HDW#F^o_#TM*AgR zLJ)zL)_Nb4+1Xg>TV<-@4e~nX^obj#bU`1)OA#0Z#3p)ACZf3?r*1awH+FGz_+0QF zq;RbDvEKBZdK@Yh<)3%+GfNL?M*#Xj2YvQ{w89PCr>S>+9^rB;w1$L+qLti*`>^j? zgT$r`96p<$J0P1$aHxdC?w}Fk*pab^CEVGgMf=TMB$bGVrqrFF@PkzPS%HY`-J9=S z!--5g?V^dYyY__QmgL8HLF@Kyw8)caG?djUUuG)}(OrrKH+ zSN!akmN%`$@MYl-%!JpCk#ZbXnvpGC_uKa?JA;%{X%wPb;`V&9mVIJpD7rr5v1qD2 zZ%Zp3;%sS{?Mo(PYh%VoU=2Tn(Tu%6_V;Bh|9C&sSmx!um7^S5?l268TaGl&OLm|Z zTCVpznqTfcNLv^l!uV$SX%Y8D_MQF7J^alV4WCj-iLy&3pCl9J|NbzR9jx^V!zY<8 z<|O*movBOyHS?i?>HZ|P*Q?5>970%z*sUZO?^>lksnI(3{0P0HF%j~($5Te9<6Syn%EgiCaDRaNx-se~r{dw|DyM{swY0El6-1Ir_ zX+}#wIq957F+RgZMswjlY%U`-BFvYq9QsxkC5N_hu{2 z&EJ0h?;ga5KO!gi&V@_#PMCJdzhTksMwyZjlP_}ojrts)uj3@d&r*WR6=cVTd7Jll zvly2w&lW3zKtRX>^&D@B{hJ2f5}A%mLE%V+QtrWQEky{Dgl4xi1Qso! zvU|UG+@DPAiOw0C>=}zveiKIG=xLcT93+}2xhFMscI=-FXZAzB#y^awB{4Q$9gh1A z{hD3=LAN8^(Uj4Y6fI+y^kjKcE@Z3GZ@t26jRbkax4-M#w{Nid+LB1LErf6NW%zC) z&dnyis2e;!?^y^)J9%}d@E25G;>?+ye!c(YCV35&=>l$ppD-rq4GngyzwuKni4-hE z?fq9b%fEUmg1W!jmyi|QPWY-`a&PO#ZIPQix8(gD76@xV*1jILh(lk}RO*JCDFo=W zw5sm|*6A2N(Hn;G(A{k%#d`Vhv*x1*(&EqYX&=6LOMB<@^9%~DFv}O_pU*o9R>1Z* zeu?bR=t;5flbyUb%)!q4K$fPy&J^YMe9#t5mGtRts>u9&0n!LKr(MIv~R{Qt#5~3%R_lz7$lnX%l?HeTwu1Upjk~FE02pE^ZiJ|;o zDJg;&7j!+Dg>DR+2 zRJ}B#w1%``2A)i}41-Lc%r}|GGA}b9W@2aR$uLG0F$=1lzEWq@iD>LpGNy97<@Ugh zyf)A?ZbtOt)nKZoiYL=v)RS({ZfzOLpHF@Y{e+g4mBp21+lQ2ixFzpv|Mco!i>{9n z&V%Gl{C z3wAcs9dON(9;z6+JG5d(VqrZh0GpI+fvkc_pt3q5(-Qo$n}ZL{wJb_2%PP7njv$p5 za0?6bdUHl|h4G>hp`zJb|6iKFN}TK+V}4N#)aD*Yk4weo{utLQ(8=~tLp07|3i=2( z>F3VrHdaiQi5KVUgoTc$H?5Q{_Ey%O-7I2WUNUoc5Z}MQ>apW!NtI>#MKMs3nd5@55);( zm(&W8c~EJnx|w4mZ!OO&Kb^P(9iyP+s)INVuaEaR_vx<5tZlNX#=efNWrMO6>u&3v z6p`!M)DOFd*K${HuD@L~Y%y$-D{H!==i_pDTlR45PQph0Wc6_WNZ7K*HllNWCS|R8 z_VV#?OkCScujhjWA{tyCir>jU+v0oqh(BQjpvT;*x@~rg9=#CH{k|nhB7rXX4D}Mp zbB+~NCR(t7qxd>&6+yy}qBn?-Ztr_>EwD7O!tNMQUod@Vdr2_}CZQzfa{f7S79JD+ zy)%Fbzb#bz0i0%v&)xiR@V?%D)?=t*+#V4d?_1tv-o9at*<&B~d_CzLmPE#6al6v%5zlq(rt-)`kTfS?fC$%^4 zrCFVVyW45`-IS2%2aR#wal~<6pqST<3u@|+v8)2bq|zjR*=>gi`=eFnpINI^1m@V~ zA6QjFvR|bP(Uy@$f8xt_m(vrhG@xLcOa8bb*JIH${*HRzijcgEJ)#IS|E~KT-Y{V{ zWn$4YJ)TCZfZ32Xj%1Qsq!(QGQo52R1%5b=A+_|%Yj2T;_%0XxTl}5)cCa2vu@15@ zO2fUhC$}Jd>j&XZL~wKHh0}JizwB^q6}y8$l^w5Lt?fxKu{Vozj71b_j7u>^eVI40 zm)ftq>U_rHCdEbVQZpPAsIlDI)Tc`5=94j21B8j!>M42jjcAYjyZIyCq|?VY!tPfH50a7=gM9Q zI@62uv);lpJ9GLzlgswY#QiLCltz5=y6}4UQ|*OHhw;gA_SsJjgLWK?tJ^yFtexgo zXT6;n*ZT77Fd#vpheJ!ezv7*%VG8ZGqk4G0a z>1__@w_G<*tK4YZ7Z&ShTvFy^y$3E!4(OJRHnTGn6oeJdUY?d6fczr&dtx-v- z>3!=zZ$`^3^L*kZ_qZQOD1uBhrQgcQp)mv3m}uxX$j~r=s~f;e=mz<}t|e|Tq22uR zJqQi$qZu0d-+SbN_p86p!0T$x_500;0JPh{Cqm%mnhN^oZp?wyoBv#caDg$j7b>FC z(!jflk-dqDwFAV)ktTE39{2*wR!YkO4UL5U>UBd}`O$A+{c$r@O-D_+SA0e`R%~yK zZ46D=T&--c)v3HTy%FKesqq2tL%5Q=}2K zu{WXNVtdN=^q~+o4GoQez42QwTKIn*IAp)((G<1sst5YJ~k6+f(*`tqn{SxcZh)(ahDvQcKLt3ZNPA3?a_vTmpaY zM;ZC|6aQnT=D%k?f6nzjyDD-cB{eA9#4*s1ezcYT@EQ|FS%8dUwZb*V?9Nt7&!b*xtoM=WR;8i?I+QF3ya( z>OM9Ilg1bA|MHKX_)7qKe1yK6JoycDd=a$&|34Z+8ovPY8~^_s|E!fd0(vwMb-M4c z|9{=lqXApI|KsK{qqKd|Sp1KuBhi0(f7L=<=ef8 zoW3oax_6Uvr`E$U8c2+PxQU^1piYkFL55xNENo*_*hcdFd?LtWhiY~&vUt{>iU5UU ze%AM-+SO+)5ILu({W@BgG+1D^r^~{p{Z61ACl-YOFIrT=pvBPjZTC4$*__yM>U4+9 zvsEnt6V9Dpj(<3(L%qF_(6*HLa?qz}xn{hO?`?$wxn^#K{YPsT_rpuakQ+B|i7}>0 z{x2dp5DYT#J~1IW7=Q(VxGqw3NE#ok3%448zLn{qFk~IuPrlVan-3(X{d+Z-z3J(& z9CuQj^ORXVC{E^op?o2s;*3#u)MKR?n4UBknCS$cz(di1!*?R}OYE$&snWM>qHr>V zBTU^FonfIUJQ^tQy|V;q)#E-bBA1vDF*JluhG938 z!d#p>csQ(J5Ma^9`|!UhClZLkppZ=A^k@jP;X(>bO9;{35)W%NkntS0+xYJ-M7^SM zL}t=sfw3oY)|Fs=Os=qwxVQ1S#<9V};wYOa!+w>>^K9c)#v?LrLg*9iXfaV_W`PUI z8WhC}$kP}jGfjYXmuk?-dd~|*217Xd*8BVEoS}-HJ`v^vA2Tx$l-z6epkFvlA|=kc?w9*u0^k24K{$7EM8n-*^geC5+fMx< zE1GVgX5|6&LKd01DBF7LzezzeuneMIX0N>mA*xYQ+7C&t@Aii08*JM-=Y^ph4-?Gr z^v(D8H7enRreH#-%-kcgC3dLZ?J-o&En&Y8YOZ+XC-H*f{Bhvh{gtlOVFh7Mhkij9 zyW(4>0jO(dU^XD;Jl?LxnQ{SldPNB=XW5#IgK+j@iEK$w6ieh6`pY(hcaH`<_q!P1 z&bh6z&WbO+M^V|-8PIOSL1>)QnAUWJ2*9@)1exi7A526G16WHtUIcbh%?W~ncy8Xt zeX?1()PTBN2I%qv(Y%ZvZwQ3vFPl?JU(u!eqoD72UbC=AY_YZk zqK^7sZyD@Lt4WDMWc*ITKXSn!oA?vO$|`dCDejRHLb{z9?V3>NaiKvTb7D-h9f*5U~2DDCx@$?EPF zpb{C=_|6}lkF@IJM>&ro+uzeVQQK9ajN&U^pVpTYs)u@7g}p39c1L0D*6cN-4i<1;X-NoDf7=_%K8y z=lBj5SkU2)q7sU6Tz9^vRNZ;EiZIG3B=&mlcHKLQ+bFca25i`s&M(?f%HyMqZU7eh z%3k!{`D%_Lj=my3*Mf1#*nfsHsz~xzlT^xy(`r*3 zlu@RS*F^qVqw_eXHU(?4;{!+Fj`^8c2i zN{mm-esjm<2go101+^Vi*_ih^1V@)kv}Tyw7T zPLM{+M?CIGMg^{5wz`~2qcFkvykQ_3d@FRe#ZmtI`<;YF#9V!@SBVk=JA_JbYQ}lm zN(sYd-pi%@cq!OL&!o6^mFxg{&@Tz+TYP`7$M?p~x}s23Our{LFnHvJPK=ic-xv=H_L@$}gJfmI_*(u(2aXfbET9C94J-{wh z5YF4$d)w9XTy&D5A-L6je4Af(y%F`>eBh23Uv<6Uqi?q+O2gkzZ2(C*4l`zjns1G= zPA%{KYf5Jl&|D~c<^Dadbyt3@f%hcxC%ezH<&H4!dkn8WMQDO=BeN2$&ELkGWqF4^ zJLut%eDN7Rrqe&;vD-07f`E=;U?!u|)V zX^fup;AzlzMGu#8@SaMsLo(cFG%=4UUf5R9^ zHrigS`Su@wpO@s&NXBiQ$a^$l)C=XFgN*RV#uR&PTfFW3OuKr#9LCMj$78g=eawjk zeukm&99=54%lFMQ5HC8>isF(XOHc$;+*mGg#MaHfGT{DV;b_{1TV_jv<$lw`MVLH5P(ss-gOtlJHlEAmtJIgrl2hC2m0MF5#hd`CjViXHROABrbG`6=Nui&+Q)rt5X%fiKJ z{IaHCoBHSQVjoU2yJdeop--D!Kc19-0l3Fb7P$I@y*VV-VqKell4&AbJt$cO_ET7b2`B`Kb58sAvAXS9LpJHd=(I5}jKgE{RyCAK`RPbp?1f*N_ zPbfy~*M3P6F0yR#o5Iih1|cL*Fy4*R^Vk+P2vY_4B^+l}HJKGR8}*iaNhY&)B$4S_ ziYZBHE}p{)0W8|;K&cK<~a?a#>VYW)qhQWstI6` zvXld^QX`JhnqQ^>Wt(b&D1v*#%f^)c+xpoIeYY`X`q+_7jS?Xa#y`eJBvs(r%X}7D_6hnQJ8Zya zQDk8g(|ueZ=plI>bNboe`D2nJ51Q%@`uJhVPE)-b`NdRFu!==2d*fk*mU(VwNR}fR z81mU|8OnWX{F^&o6jy0_t=j~vQ8JbyGR0Ho>>rzO91Z{58=FrRzqW-x9zre(^>(F;~dL2PKH zZay{XCgMQMIE?EGih-bgI*l@K6YbjLJLEkMU|~W*MU6Xp8sN;yUha8{#u45VM7B=$ zU`OCRMhO@R+$E(6py;5}akoa(&^m2MXqpnt5f7|QcK+lMqIk2|%X8bpK(5X!hz;6czweKPTo8mp6#=?c2V7ArTdea ziy*_0+?Dt&3>ic@Rcc7CO+2mnqq1Lr%JuRbQ9MEZ?3A3-TvbnP9*8QIt*G4B(}boc zR)W{h0Z&rg?_uqi#FbYz>pZPSqWRhnqL;v=c+z}%)*M7R%AnoGrJDZJIghDYI;Deq z-m~0(C0tV7GF?PBMVv${qinlsT0Pn4;zVfk?2aX@&VHwMP5sIT1T{W}ZCkw)GbwE& z$)|9cu_K;lI?yPdcwETLWyf?{E={txH&dpjp;u{suFY<_FV#?Nn&QRfFe-R7L8onV z@-ud~ysw+3n4-~TL`hP7ce2)#h1|O4P1~*1^txYU6Yq1n=P!<`g{DvMI9`TP@Rk{p zuSTD&r3g=rc{9N_4JAqQI%}HuM^qHlR5hlhLiH>`a&5S7`fkX*4i)apHg<1ZIX3@#bUqlky(I?fzSu%=>)nss5+BEi1T|MpgST z>T{;L+m3y87j@JDxRd~XDNdYWr~Hk-MprL~!HHmz;vCgeDtJswgy3>_(zD#|Y<9T^D9~QdSVEIs`4L3}u6lZf~!%KhOX~Kg{LwDBdrCu9uY>M0EG5?^cUWyw+Z)xY;;PP~Q(7Z0pNw7j*y=ytPvfr82Wz{T< zYk)wbh2dLk3U75$Ol`zblTLr=%VqUtf`3~mbWl(r6Ic5EKtp=-l-%tv)lq^M&!;WR z#V;8y+qs*RNPkja{Qh`6|2sR;US^alR^Q!hX2)~8a-126V{#_Us$o@GQxT`t4O3D< zaTBc(c~+6^TfZjS*aGm&Tbk zT>Xr%FlRQ$X1vsAfg1318M4@=Lev-EC9dV(KMs zt0S_pwR)H!5-H%=J6=SY=au^>Ogx)$<_bK@b^62;jB7UCF}!J{xopd*lJ;fCS{mmJgISl(6z0EG&wG0~+SK$zshoB<=44_3Tf~3i zU9;cK5^2*>oC25?SHZT9jmH-*VLRkr0b{xjy*yR%B)5Bv^1n>^-KNqcu2-4a{oR5W zwu93aJ_{c;?PMp_b5ugk0VZ~%Sq-kcyInCXPmG`}oQN`X2+|P-LLs%yQ+ChbhWVBY z@9Fayul`IyLJf1o=)yk@o$GQ5Jg-Zwo9b(6qgMK@xb0ab7m-&`Iz-q;3UYlf-EXp%(U5(w|2YNxh$EDs4d&`lL?R>jxSP}XYeDVV?r zzN2^-h1q7UiFwVKjy%uedshQ99NB7W4Cx9V{`4RnBH=oTBtp_q;frIQy|1Gkg3Pwg z=kW24=@W&O)Yb=Aix8@q(=Xy?6s#t0aUR#?1f*W65jEtcBSO z>&aD8Ly;)U-$Yh#)E@H7!IJfOy!nC{12I%JTxR2akU%^={G>>9YEQ+*k~EY`kZ(>N z>dCoM9tz8NhLJ@)U>m4wwpmo(#}}8PK+k*PBxhY5a7iFZL4eT|>14fHzOcLe{SJG9 z8;&aJgA8H}-$O0iZ?ndYaBZjY#t1bc5&LhjifthGKy=Hn6cU}7(%1gn`;n(nAH#AS z>pW*2XMl$xIAf3aa+j*^r&(B+)4A8dzmg4+RLv{@ZxYsAo2m#WFg5!CFjJ=k9 z7ya^hmrp+n0YkoAejg8;8EtKnM`4YqdTHZN=KR7c?bXssyvp8TKHl0YU2PyTl|bUY z#1r@}lES=V^`q-)zF;2DM?P#`e~Eh#u1Vuah)!F~2<;Ci61y{pFuo?j%&i>Z=*lr| z7NKO1dBHp(^UJr<0(;>C+ouiBFxqBgwBE+`9E_c?gtv!cN4RfXp6~j=`kwbaK4J1{ zW1ZbEz?A@mTrl^tc|akxli%0n2>~t!N?^C^^UxP^!Kc$k;I~M)|)~-z!UngLP`ul8EnThd7+JZt5`kf!2TC|oXv^er@!_*HY#G&E9A!zzM~aQm!5i=Il;zxr;`;sr$Zp8S z4g6Ap2)kh);JNCSsuJ;NM;z1M3&A(4?V*%9N=C9i=BZ|s7*A))D#BeO;LgQ5aU!sQ zvW>Fu*et3esp^6wXRX3`(7Z^BZ(_xq*g^anQ09#GGLWKz@VVoNZ11)XEH$S?vTVk2 zSZ?nK8$E9^C!nmLsAegs8+al`^DZK9$Qy_nC;(Tz=SHC&be>C)1hd_4tt7mzp!ygKJf;6l3MvBonlU$A zYP@k&sPBfIWi}Gc(J#OuqdC-=6COmgWw-fDNR9-4c;GKbHY(vK4yPH}Ls}G2`E?ET zn10(9Ma0V{YXdoNI+amgH3%kCU{6Pudx70a{{sjpNK+C(S#^wuysES3{iW}S%>otK zo00ZvL*Wt+l@|uRVsk#Gmk%3hgXHv^S9mMWoV1lgeEQA>5gvrOJC$P!MbkSDKmRdD zz9RhSBQeSzD?soWd~vut{5Gx(LiMt(_C8tE>5 z;^TTG`MpafRcU$-3NAQWD6F1~7jUd77O+~hD5_;Bst!#PL!-}uyjOJs?Jqyz6NakJ zNZDb5`y`SK9eat&2M{3=Id)t@;g6-oHm#dZR*BoR!G<4;zHv=k-WykWmS=1!>C_E) zk?ErTdnyA?&M+*Ntc*VHG9~30KKrikCnofEU4w-%4~b{QZS+MzHrV}{Ry6%fnukG{ z85TI{FyBex1Ph#3ze6yay%Cku|DI2#wkBy#!CPKeK+2BX=g{c#OA$v*ldvvw*hhG~ z_Hq`sydgAJ3VsLOKmegrYr@Y`wJmu%;?4V~!!A%mxzoFkowGf@H?C2U%cTw?;-}S1 zjoUIu29=(@Z5u%D4y-V}=98XJpWI!%pD(KLiNy&sx#e$_RpijdXNf%;qMm_IhZZRk zxh%TM_?>A$*+vQ0&G~f59eY$Jm3P7S=8_dBkwqEPXymz7+{sug=CD=tCa`zl&wd$$DHDA>Q?}DB0|W9F9RgI%Oej^Jj|1T>ra(+b$hP zk{0$(BVa4G`&qAZ4Tu!&)-pv5N41p-z(3i1)yHlesBlFrk@lz1lDbeeisdM%g>>-|{3r=$O!@h1d#o zvP$C77ou?G>>OiLifS()ui5xD)wPl6wA+ZNn{5xv`xxw&!{$Z^AzGHPI53y7Za&Y- z%^40#k&2+8<+1U6q);zp^Xd(7cWD1*AOG%_!$zI_#kxe75(FvFxsd_(Mji{DDe8W6 zLWVm%_sfd(_F_)ZTwol4$mF@qxiOYXWIi_^$#Ff5GDaQ(aX8c~+BV|J#=`oJu#;3? z^|3VFX-JkA0%G81b&f7fpFy8(I`j;=dm~Af4htwt)PIRKj18h^49-Gx2y~ZuE+bF3 zrUO0`OcZ!qZ*Wh;yqc%upJMwG`zYeNnmXEt(>N5CT~rEoTptLiauP){`_XEZ5kkOq zff4$SuW;BCNpuIBz*B+bZx|-gR7<{&@>-oTX^@^T?2M_vI*Bh6hrJ!g-N~3WRgTvc z8ml5Q6957YD5cA2D}|Aprs`9BEA~R&Rv}> zY*RuFalm50&Z>3deEDk3?eSiBVfLJ~nRuYO^#&C@;~+SpNfxSup<5P|h3C*AeaG3v zY7L;;N?gIdKQ)qO&Sz+Rz8&L;2!B7AP)zr!3Y}CY4r4w%8oAQ93{=vr8iE^AFzBt# znok?DF`hfpz-flpq`FwV6?mb_%R9oObon(0!=6*#d(E;$Qr%>s$g2!I+h3MpUjkvfR? zQ>zX7GR-;CV5Hs2s;pwrY^H8U(vzO{(X^Mfu}0v>o#J(css`n;iPr(SmL zbIi_6L5i5nKAu9j>bFw_1UWjfo^LEhhty2gAMX>@sW47HabTUq&@}QK?v|mYNtfvEf zo5pl@<9aU8c@H2k=5Q%QfYZY+!0*dhy3pfg*&D%gDTMkX!>svCw4(5#PHrt}&8abZ zrQFhO3H`xN-krpe;C%Hxe|#-0GPCKkop=Sl{Pp;nx)JPdn~Fh+LHCWxWyY=<*JbG5 z5m0HBAA6Ht`+e39+3yAzx%pv9zi|9E_@Oido*Bf-MINZe7rQ~^pqWkpYu(PxsVd}b z`1~cPBfB`<_4aXhOQ>i8_e@g!ErBer^*tFW@=rfLt3y&Pej>xpy zD~XYP(i@!XF&6G~+TwvvgRma)zW_Ym1~N4_cUH)v^4USvPh_|OSWi}c5L2i~Nai~v zgU6+ft}JNj?#2Yn2$B(9ke-355oT=3sNgBq@ng8CpIoJr&%mcvcvStln-pc>N_u#b zZcoSjm>?qimo&qG<52%7`{tlRYA$?olnK0ZAtgV%9YHsgWvA_P4u@?6QH-*7W19k5 zE_^n!Lw&qleLu`V!-qLrf)FxNs$zYGs%c)^&D(Jahi^+UVnV==|`-c4Ez- zMZzm1%bO3q*-u(SZH+9AW+u?dVD5E>ks2XZsSl=)IjTPTmO}9m>Z35ZnWxTtsgTLe za5`ubAG5a)V%CM*nQ36Xk}_{%y;4s3Y1JFEtxoPecP$0WejEp2m^|T=(N+$bR=x>a zHT15Lhnd*!^lRt?0ax3Nx*gqZN7Z(&xA|kGSQ72>HpZ49sM~_y?Yub$cFT9+r3Vir z*{%Aq56V-{dm7c?IS!w5DVs(o7Nwv``pF=fGS6sfzky#>~8@8h?$ zEQ=~_99a4Z7b+cQen9>yR9$1%8z{4rdmT9%GY?`WiQ^Z*A`{hcT`gIQ`c3i z!BxC=EcZW80?nroN)W#{N5AQ}&=DdpA-Gqr;m1raE?WYx7|Y^s$M@j^r8i!{EHcje zkN850imT3miVegvBB4}Tt^KGk*d?>Sg3_GN-O5Ae0l%IWQhz^CXqnpcw_hn1xIOe8 zQS&Tzv#Tkk`_j$_3EbjcYH~*?_n^aUHk`=og6W@j9#G}})0r6_YPQ;zx*fMmx`)Hh z8GMDjf^ij3&72N53nl<1)?Jm=<6sJ1{<+W@!b&->go-ui6^(MBN+DtMIACGQN81*t z&?nG^@;r1`*mkSz7z&y%QsdVyjjAxrUf>K;$dSu(4Kn0ob|!j#FyqiK-cQU?B1|LM zZzV~8)zFjdA1x{xt$VI_zz~_+&-IsH~*Z&j8A>+-nwho3?l6voVrP z3v|wM4Ok*GYYWQ-B$dN*qNHL}Io7J~S5~`oRG<)td50-+TPlNGyz!RQ)77T49a+8J zP+W>iQ+U?Woucy?ZlUz^pH|~*fsfaZizWy@!6ssMgwH46=P{{UNaBPp2RV!B8Abr1rBaf3*tDQeu8W<--PYWP}skU|L?UouNZ47$^(A zMXIFewbiRyM#L)>bVbXu)wWLjDu%^6lQ9}a&Ze&gl~2fjkSJy2kB%91hsU@#dF)Ry z3uSys>8AZiVQ9(N(6eDxzg^YtoE|c#;n^+!gQz%P6ZT4V3lnU2SJzI_>qxcVc!qaw z!Xy1`2Z+l}iB8F(Es|ru%a8UYe7JtK`mnes13)^Qoz`iy_zZ1&3v@DWu@Zi7 z!LW-Qa=t8g>tn&t86ivG62p6UnU5sT;u|&GP0-1jt5v6LTQ=UL>+2o#avON+C5rDq z;{`z_&Y{U51UOx<(9hmUR4MjVuCL` zM=GL0V}p@4GiG1+l1G+Los0#+wuO zn(k&z$?UZFBvAkjC*;)oOzn?3#!!)A?B@8})-4IA7x1K0k2y;`+HLOb`X-$0PTGIk z@_tpA5HTqzIPxfaP6===?jq5=7wH~SbSCTbV?h=pD#%=q2$jqc{(ZmsqDm~V7jFJh z)Kp@ZvzzwxcWNL(@T+kus@upDIx95wBllx)+ltoEuU-9w4~S{iJrz|KsJTx5@v~@r zx`FkHDt|>)Qn}11HLwg2BV-TskyG3Kbuj}uZCr1dFF4YQLuO)~_Esb7HQt5n`>W#@L2J%)QCRx06=PphS$ za^{csV1iHC91#6%^;^mgm+mSl{MAiD>_sC+X-;QnRL*Cct#o#o@;MH#u6u;ttRl$F z)69O-!`mEJK|#p;NQKAfHt z+&cquId1ma#Q4YFAZPMY2ti$gmr*SuT(Pt)&L5jt6$^A-e0sa`}yZ#tZp`Ly)3><3%^M7R*l zIgZkV!I~SWl3d{Z2Er^o6>5T5k>x2m?=>p^4V6_PAOM*ed2Vhq#yp!W63H#Jl;Tmo z?Oq-B))4!utX*T4;DiU6GU0z|-WD1~=wzw#Y5{14hwKUq`$-I5X79|g-hXQZrqRriE8>6<;8%Y<%9lXY3-%iHdFys+!m z&WuARqqnX-pCP(|SdN>*O;iawF!nkKuwO_96Gik3cF*iNLGomsB2+X}Z% zr-9Ggh+`YlMPylmleidUKqW0y9Zn;7yWYN?l>MVGZe+a;ns-PW^PIL8GK5RXkc>O@ z$}dKFvUqRb_K2Vx0L}H!bHQVNs*-<EubluWPqzN9F|!?|*4OkBu&HI3P+)Uz5X~Qm&pJZxD5uz5tywaOmY^iQg3} zoS^l9IYsfTkzQUn!S0;wp7oa#LSCMnbl{#h3_iN=lL3>c9z7+x)+&6F)J~90)F6bz zTF4q6#B6~@{Pi)=?cby+`3kx-;$-nS{A*=y1YOSR`Jy`A#~iAuW^XYc07b35nKAf6 zm+}x+!(!f2|I>V)TLpWgqGRY-VyU3}# z>fXYuHp{Duxw97doF;#jv5+Nk6&}N$<}QaC=zdqT&(QOs6zrEt{iQCfap`^!0u&#~ zecm35id`2D&w}b7pLDhn;Q@W7m5uYJqP2R);zA$!E~U0eVoL@;iA#3-E`J%qEuig| zXxIYuWFG4`Qr#)s>L9IW^~)NZz_-gh-akP98?@VF;`_t}kzOH}ulo4OCzpy<2PKanGqzIXj~}u4wWkQJ4au?>x)l@vsmhM-=k~r}T?gvkCbPBO(6F=s=du-`v^2uC zL$T$#)HT~tEXpuYGshs4Smtoq-f@^R0@T}vp!^@*?-e7xfXa(Zr@hHleuGFD{cP~& z197?&&=#|+)|w6T?x*1NwIFWx(_zVpnUSmqq;>R5{VivU1}esU9-kq%B?e?%2cQ=yo*=8F|sg)FC*2(tc3+8 z>-uh$as=}lyFmVA^OKO~*pW1m3tKXjA`R=wv z=Oxc$@}kh$t9!$XWHjh;4^V&|6wJ7edId{{FXcWVVS`kK%yOi2BLmIrR8x zn`Xg_Ck*$t9Ycdo04R$=@Ytkn3l7W0p;V<|#1vQI$Qh8OI4`vZ?qP}oU>#9y&LxxH zM*|QeZbG%P7WD?B-L7h&J=Uo}XUcM;E83kEd7OS$oi7#GlN5cwtGJI?L%18DuqT^eg%-5r?jrs0R3gS7Cm4Ag*lbg!-vyHes|H{Umb6buX-IPcJWvCsKt zfv76bs3jX$66LEeySzB-_;Icuvg0Laj`{1VV+tL1eQddoy@0fgHB^EfdT$U zviib421%ZC>$de}kk#VIwa(Ii0|2L302HLk5qkVp9sZ)))hCJO5LUgZnCjEq41hA! zxL?dsTNeYNMh#n6xFGUo?Vz^mN8{<<%5iXn+!SV+weVG?ljz>SI?US+3*1E@2A(am zD9@pcopG6~N)$dYI1tK=1Y2${j=35t1V_jz>3OjW` zn;6-fmu#W!;v!5xzL{a5Mz5j|xV7S@Q}b@JOPv*o{XyQhZJoFRzrS{hz}T+h)jD*1M653crVklhc2A}c1{ukO+B!{GRnO+BD;a)&82p5=vA6?3?KE$AH7dS~*nE>s^}?G_ z2Iyy24G#?NP$`RS|BN*lTdK)viWib~Qt{{8p3v5U8o`Zak46q6OTZEG+JhzAJ2UVJ zDLjH^J4$ikp1C0Z@eD_ zbP}yiW^nZ+&XA(|TV@W!XQJy_IEX|f8&I9d@peO2cyUKbxZgP$etH56<6SF|^r^Sz zG2Qt0udwjd0jP5J78W?(eZ0~H=*G@5S%*8(pXa6wODOc7wG>Ju+g@Ib9|9G}ZLR81 zsz^F~03Y$(+W;=j0(gqoGiEH4Ck#Nj{7mWYcgnNRqPV}xHwCPC=^1NX1G1>pU&@z3 z-c$PUDka*peJSc!}7*NKFU@o(h7o`-EkPy+!Bga`BLd>Y>{mc?0qpxG~6Bkvi7YUY#*pnpR z_czIFj*yKS0a}w0R*p9H9wSKx)eHmc$%7?hMadg4VCYXVVBp;vq592&a{0ri9Xg+i zl!lZKfJ@*IDv}6N{;hh02jl+8rgRF9M)8V^AIP^)^92W-Vt8G>&ZbY`peIOJF7YTW zRif;8o_@z6V|&&mPqfS(j7FdCfQpmb?63-bnk-~aiuz;gb7Ah!U7^dG0} zh)8tb$hG5_R?)~mhf#lnFJH10HxO3lpA!&AL14zA#$ABaPgwzYu8__2aN;5W<`xt| zc-T@oZ`WDL3g}~Lo_Wq}?Tr18wzGi|DWi^1sQ0?MrAQKeRBHP6a32iNeQ!(`Iz@;_ z<;gh!BUy7P7(C=6n-#kV+Ge6E1rT7ZYnjU~79i-W{Z`5Sip_&|B4h@ra=ag)1XG;! zxb`y;sg5IJ9sqZT`b6o?7auZ#xGasqmA zibKxzq)9mHsS6M}i3iG@dXJROjL7j_$n$S!$mPj2l3X;-NqLrTy!JD|*TBBpv-6MN z^1FKtv48%RT(4jufIPaDGpmfsqBpE?xlY1*U(!K?5hr#tf_x}6ZP6ooBn>eHND?&^ ze<^hND10WH@XZjR&d5#iKu&9ZLFU@8K80&!tDJ>7;lCJbYPOM&pcUIl6X2(MR{(Gi9mq1r~>s_Dvm4mAkzYMwmqhs^Y$0y?#L@SO6 zh`mSSn9vMD3_wgAg?A`SyNKp;KWJCLEO1G%<`Mq4v8|95N#Y!F6Wf+Me;k(@LjMOV ze+eDQ8zj>G_^!e^M5hYq_#21TO$PxR-%C>$1cOS2pXcyF9tB4*f%iY;ms9Cw(^tQe z<}xdd)5Zexh4Ou@QId)J1VblnUNQUem?1sfO85V9_TKSShHv~hr=yILEm4%6ME1;S zD4R$st0H@5l|Mb$ipXa{s z>$=|8xW{!p$6AkXH9sqrM02UTbJ&)iQ&%}}kC5}{b2&t(tI_gEH+%Ks`}BRmtflr- zC0DB7Y*I>L4~UgthVFE6Ht~>s3k@xHSYZ7rsnMTP5WwowZ2G=abM`^#a^lzP-F(GO zs`O)tId0jrJmT`@k)j~`a>FoTeDg@oN7^lkwB=H8r8q5w=vt0END^yfWs$6w+dCfVD`Bo?gR3$4S^%fB3cD=q_x%)AYI~^hMLYQScEH9b zDd1naEvpV6IzM^9ax~U5+w9U2fTN?8Abjrfa}~1&#}D-43C|`ow$Ri% zCWS$@Jb5C#M?vt9%u>2hDkt-*bc-2tEk*+z33f$=4627G8(AhEof!caywi(bqJWWf zbQd)i6$)a0-^P@#rlV)?dp#rT?FZI97uKE0N;-y!v>82r%_7%_=8fV?clo~p1yJoZbetakLwOBZ#AkjFb zD*=`Cn~&HAT&SF-b=UkG@W?Y$>w4|RO^w4m9wyIMp1Ep4h9;w&Q(s>s;HuEtmHvI? zx6P8dDn4j-iKupkrbcS13-6R(O8PtQcUV<))KzBkc*7}|&G*Zle@2#&(rrl!QxiEa z`{Zwh;(1RRaUSI_w7z20WysVh=MsWFN>O>y9`XICIag;$i$b}@Z;N$~0{{0)Z#;{% zgRUe*>veY&1d5&v{E5{Y|sK z>V8sE%DS(xhlEj2L$uDfQQNu=TzioT;4X5Q9Yr4Xdqd=yhrJC1dHQ?KAUe!kE<3kx zk|>Efl#cO!?ZHXo$m@%e%^s;Y9+esd9OlWL8uVk$-uxQ{Zl{0edE;0o@a7y?BAr#E zuWo)mPpj&caEweVX;qww$3n48_h|?Hzxw9js-tN}u9W0X1@< zq<{>K`6x4?I~$=Tsex}(pW6i;8ouPU?04UY*{o*yk(q0v{I4cTRx7Z}xbW!!>k(=B zklbJqBW4&9X~Q=Z{Cu$+H^z`@r^e?w9GK%rc^4WzbGdQhgWr@%PL%{n5-f*^>`CTsGHxT?Xi_ z4yzc^GV0!^W0p88CfM@aBbSg?i|Hl-2UG4wX;lhuQ?~{ie^qU?tpt=bIbZhgs&+;` z9q(UUe^9>d@wP(7cU8IMW!J@$onb3lw#g91 zOKV6UUGMiU{~3(Y^rCw6HeFsz`OP_ zA@!>9EbKOUqM)t#sbkoT?gHT)u+xZcA>}cDbNzMXv5C4YaC!Kd-^`tR;QUAPSG;43 zV$Jdm>q)&AB1TtigfGm=H~Vp8S<-YVUh8jA z!rjx8XTcWQVJ2x2m&B!j5=`^$1OCGD<+Kt4xLvXID+BMoP=41_O?rIXvFSW?+!Tzx zlNIprgPj_wquKy;SL#z`D!3RYU*?+K2cnS9^{t%yzm3N zotfc0N)J*_G|15)6|9{H_4++X8;Ao7J2AB|mmz=dpuPv?nIL6H&S;E8z*qS%hv;VZ z_OyF{v#x9Q(BG|!<7wYW_}P9sl@x97ETa3H4nBNiD;+}w6SWiZ0k?Hx#Kl;Xf- z1c@AamWt`fU--)65OmY>ZNck|V$p^TaM2o+WO$pp-X75#TT)Kj%HZ^rJGSmt>z7_| zCCOxUZm^l|CfLjodcOuGG-O!(^w9IF%K)cyUk@?~ag=-nNnGk8)@z-niPqU=f9uS= zpqqwF!X_#gTZ=pNsjmm!RjPVXM2+9k)`pVeI2AD&%L5Sk-XNPKSTKo$siRw=R8zyS zJYw@T>N`lRu!6EWyw3i>2YX;wFP-MwBuPrMX~ZS1R2uA6j(a765<1Aw6g<2pQ*S?r zfjqO&!;wK0^+MlIn+$(@X7ZueIk1EG3vKYXH|B5kp9QAcb=j28^t{gYx<=Dx;5I|A z3htf8m3<66{r+h)>81Y7QSshWGP|j`j>SM(G8lP4=>%%C)T0Xr!7O+ZDuKH`vLxjS zQU~heY(Fw3RE&yrgtA;Y=5&qD>Z-Rd!Qlbw_=#2R2QR;Bc2Hf(1*eQ1UCHleeX%tU ze(64;y@)YFfSq^2FHa5$#}=-Eoh6Y(uWc7_P_vZ>_7&p51zx_RuTqWZr<>3fTQjy- zxTH2grlYyY-N5HpUElI3Dmm%Yn}Vb~`?L%ah8vaBB~#g>RkZ3NU^~tsCxKy!kO-{= z&RFhEj$m(hm}H*)0ja$vMH#p((mdAy&Ima!u0*b@4GizH`!6Y+r#+VWjvqm~$Sr8z z!T}we#(?t)dU(`Cm_kDhsIy@W&;Z!BMQHH@wd&Xr;gO}Zs0te&vB%Bi=)UZj`N4e|&N4X8C zd7katk5AJClrh>PKzsldy!XKFI0mE7}Y#7$VX zLjBtD(tBF-ufd~68gM!9Emk9Y8ke@;Pj_u^C5RGboNP-P+X#1$2#A)#rR|Sp^Iq9o z)8r9Ow(ppITldN~6<2O!{rS5Sd#22$!4D_F!{NYx$+iq9+m)9?^?PMojg#%_gJbc*H%dz2ck=mY9_OdYFs+1vUZE>d8=mU7o`?~IdhxA(^noiEtSsSPB2 zg8R`@HF(dYFv9S=j-#&kukGO@4+fqwN!d+_y_xxD0OydcNj-CbdG`>A5U6>;bDpql zpxN6o&cKb7B}!QCX(eTYVc{KEeP%B`+&G(D+l^M*%PDcV`b%rDPMp-9Nujvm!WR=& zKZm3D@KHknE;K;yQfjBUr?u%ZY?D>>4vOu;IWe*Y=CWd|3DX`}L|nxwZl7wr@SavA z>_xb~joeM!J1VY%%i=%->cw7ONaJ|iIHme_uWYwq+2-%ux_@a8lX=o{vMs+O!M~@~ z>;^pLCjunQ1c~m>ZUWYzWVbxJ|I}Jw$?9hBI9EVB{wViFuBE zn9sYT;r?oRi__aF`ZR;Rtv2wxT^^j0$M-Oq4%+Ppg{D6D3B}{e2K>{52(B2B596bs zdzegwJ`LmIebvpq^-%8ND*ISHpc&g+dgK#~3Nq)~UuXBsX)zi`l&mwmhW;K2j2qyJ zi>$hp-&l!(?(`?V;i#XxpP|ds~l!(lF5Z(yQ;4b^kB{CuUS3jEm0E zGC~as0s=;Guj;K(o9?BS7*|T=Ve>E1z0^kFsC8o2xVeX(aT6T1YnfW&ds@wK=3=HS zfQ90}s6{^_lUZ799R)=!#6jj+AyjCN+}SxeQ4L#hBG{0kkl@L1DuSU$Xr-#0ZB>(g z3ySEOYd%O@VRds4av7{zkEWky6!e>j~?IF#Y z`iOnkj|OncoZy<(Al}_(RrG0PXa%~2_vho7!9t}Ws}NX`#^1c}&sm@8b0F^6gI|pY zj|j3Eg+~pB`r;=um&ojnBEWbpwto(`+eD;ZK(FI*^3W(O)?~vBe~pN7 zBYDXx-MBe-dWvnlBd8_q*6`e7=HL$fJC)jQEpMu~17)P3x@G^Xj1=k& z8f4qYdMq>F?eiS#H>n%0^%&x{NMm)Jdo|kWC1V*PF|CpJGg5a^vJa$E+E7a0O?MgX zG;N46jknph{i(f?_2tnhuDskI*LT>J=)ZxuHSNR6iL{1Kek>+x%x8PDifblCLreR^ z_9fl%$=Uew(*%>?(EY-zVj-l{^J_vw0AJSa#*Lo>ud9E^?v%Owtxaoi?Z5ezt4qc5 z_~w*hL!dygxR1_FNvTF~QNpUGeD0CU_^XN-@qX9uY9s~8on~KVVuH;DEAQUM!=H8WaMs)e%oe@b`IW|VVl`zeE-yGCx*yq_qX+Mttm3THL-?_Ycw$NLNKc6nDL|m?=gQ~Pn zJ`?Y8qyS%(@TtESrPnjIb?iZBZ`sze?r5Q|jhIqM^P+Oa$GHm&KE2Mu9ozHjptj?W zhA2aK>KU(E&L|OSs}FA_w}OPhPo7LO%AugVXpv+;*MOPZi=>cqjWR;()WVag%=IjJ zKaS68C5>!uujs_6pWA9{T#!wQS)K4|$Zu6Oy?ju*IxBbGqPb9hQypu061j#`-Fv!~M4E;~#FSH-OYzhU&}D$wat!tR6$j!SxwmyiD3}IRx9t1YhXj#j@i< zzgRrWf7(+NHS)}M1Lx?9j=r^XzqO9T&c4I39R^;#^8@(XxN)_}0B*edLF=-+3hSwA_tY7~0* z!}eMsum>K8C7Obm#oP6qQf8*}2N1pPO1@dT*g{;sW%?4l|HagkWbqIG@{S6u^xB5s z_;}ZEzd_&Wca@vYv8ncAwIeRVzjaK6*njN2xapo-UEiPhu!Y5SmD;&)hJR+W+i&rN z#9BmRjNA5KMu+X<3L(DJ6?!ta>eu`A1tzse#EO>J9IJ6<;b1^1$sKIUahu2s$ey&{gwS4`sGkNMW#@K2G z-M0PCo<==XUyiPg_&FQL8aK~tj(2?DJTAO3U9MBM zDH-?9q4Mxyp*ZW3jJ4V(_SYi-0{|0kRwcDH^M$VWe^Fm-0BH%mJ)TqP56ROr>B;9# zVxulZ4@*4azYwfLq37B;G8)jo)wemxICkk+^tDbX+pbx>$Jk+EZ7jTA|HDqFn!fBs zcY}Zgma~iFs|7Dr^8LjOfAQwo7O!0v-muEkF8>;cGpF;}U)`M3E&r`F+&~I!;7(@O zH2Bx`r=^MB=wZ*Z4w0MZSopG!Qc!v}|6*C$Gh{6Fmzi@N z@g?dr!%vf@=`=%SZMJ15xrD)af6WGeOSlGc{7KFxDlguaV?T-s(bNpsmTfVx4S9mf zAshCuklCuw8~y6XV&UBFq?0#$PD`3Lg|q3~y~HiQXy77WW%I@__m~5g@H#ykNW~o-jEVstW(Ff^G_QQ@fERd8(S{- z)mO!9PvknU>YJR^_3&~Np2^==uj*COrvl3D%&)V0Z02@$WtR@qD8~kRdKl!#BV(h} z?x};dazF&CRmkd}4KfWWa2YtIbU0oR0sOP^V$$f~mCGxVeQ$EgO%{9e{)+1efsNes zn;YGhG1>=m?%I@YQ94Xc$bHbV7BD|kY-XG|)AI$vKN?w6|H1wNF4eOi8~I+9UMh}$Ph{!QJ5PGI!}yzE1uDDU;HviZ@tQ8wnv zwb`m4^_1NC7_4(yKgJ5PybVK zWd-@PVzEy&sfKj11O;xWG)`4V83-LX?rugG2hIcUHSD~V`gOWw{N!l3`nj`E7*+hc zv9$g(1;6o??6s1$z+?VTkVi8LQnA&dT4Sfa80>OR=?u`dVR53gP}FTSuU2ZbVZsA+ z@kqNasNyFC1pPyf1_$b#jhOm$Z`Q$VM+3_b>^M-qD0duSxr7ljm6uDf?!Ie z`8_xGWIBJECYPPy;?Tz#fRXsZ7@Eo6l!O3Rk{p$0Kd!u#?5MExnDB#ZCazheur?QQxOL{07WyQ^1XI`|FkwCK!LVf-rvpuk)jds!Go4}oR8M-lJOfjz~B@= zAr1XPegfX#J7IHJ4e_6WFK8_}5AQM9fXvXnUqpafq2Ts^w?aKbA@%?6`4-p^gLc;` z&QO-_&W!LED9H-mJ+#93k0rqc1aormu1L|BUC`qH`hPn67~hXRYgBVh+h}g!lXi{^ zHmK8}gjY;;_B(cwyO9V~_p)ty5Zccvm4q!BB!b(tNY#venG#)9j2Jt z5TDwu(nwPhugt&H#)+M=}XI&y05E z?Dz&8r`ePXwLzN)|9k5MgMq;4dPkBUIZ1Uy_K)1?d@gHeh5G9P9evmK@p%kCLza={ zb>3JQrt+m)60Ey(FabtJB&kLW5RaP271!-6|IGBy6j0toY6qGmIS|JfG=}QG#q|rS zhi(`Ssl1osJ0tpr>HP@8q#(sa`D67I0#^ukAnu74J~tm6Lh1re^j^z?%B0iXg0b|U zD+5$1l5`+nIrcYO(=o(YsB5pD)Z|Rbsk{{ipMy!wZSvYrUto8Fd8=kVDH4JZpjK?# z&uPONAR#=0i@0iqF+Xl4z>8`;3s~^Rbq7ae7cb_SDW!~q)kVLufo~~H{;4N&`}8HT zL0(@uclz$31!@YgB){>wEX0Cx-06crKYiQo=>x_R%-69Y1#^-V$@pukeoQ#cx=yMn z)uCki+r9UFCN5}r_l&nm^>fDVd_|zFEpsU`s01xJC#)E+Y-rVX4=ylTK$cRRKkTW2 ztl721IW22=L=>Oh{YZ($M)V4W=|2W68epTQ>;`0)crsyx$(I`u&g}tL3IR;b`NR)E zr>3CL!*bu(C>2ClGqJBQ4c7nye9rkU7UgJiZiZ2bT;2Nd%W&} zsq+KZ(G(RGevysA%w(PG!`W5|N&mZYPq2ry(;H9n{*k>pkiCam7Bp)*+zH?7e#v-N zg2JP8A=&%J_^>tE1Cu7@Nmc@C`as3z9k)`_9_=cnHMoe@vfr2zQrtyvoI=^nXZ%-D zWz+*wL0_#0dVV(9@>sjv>;5xiPi0%ud{CzyGHT8S@*%q{#a^;*YDFY{-TKP%aP(Jd z9Hw`=CNji#{m{Lb=jWYc7)VNARs=z_ziXUcIuBV*tGF?xS@7I%

      9aaCiT&o!mJ>}8(weBk`+}kRg5NG&N3TKbldN+o!-%q zlGL}{U+3P~rWzyS>&}iweVQKXR*WI8dh>7-kHGMK+RR2LtBPkA59 zA#@lbG;||$ZKEhO>8k1GZ$OZ{GcuNBNdBn5$#8>hblO)B8i9pGv#!(=x@RiC37=CU zd+CP-*`qi=t=&S-<2nFj6_>Ig=|$kaC!pDLE}mm|T4uxuiC4aza)?2s-shkYOh)W; z&KQYscf+LSbeVI@Uj8C)&d9#G=`TR_UbdzPV7^j+{UQZ33Zh)i(wKdWXptyH$5?#IYlKN|U9(jFXVE1k z08HRQhxbuPX?k(Nfw-Rj?u}cvrUhUFUAJlOBV;*HpH&X)XVF|2#X^myPbFSxSEj6B zZR4SE11ojh7brCQ)Z%x)K!JJ##F2k1V(WyUw)f+h3$KwpR9$y2{BA~@<7o2p-Ddf<$~#RkM~Pc-CUsXQn@76JJr zIv69G?{5BcWdW50$U91J;F4EyMn!fdXBTl%>bhfWH0@r-A8) zpqur#H}Y0+hjUwQKEk`s6!#ap9ak^?KNl2O1(!hw+PJX0It@11x+rg)lfC_Y?X~Zl z)1sv}oiaMtvny$kuDFXF$4jZx_a0Is;f*1#Uz|o?cj<&+=Ymv-=KWDHs$z+YF}f2T z?1~3xU7#|$*biNd9UoHpk5+Ufd0V*17gAwsPDv+Rb~7$g9KlHD_9*MMM-hUtnm1gG{;K6NzwNsWeUMlW~OD+01`E;+KMkYibe zdt0rO`Nz6r6oG|P{hWJKhn&Y8M1YCWXN`8kZ^?=C(~=2i$ecc142OT56}b`~yLeUb zUZoJnZg6?SxF6j9=lcd25->$*ks)=esb4{uqrq|Uuie~n+y@%`Y!I3M#JehBih{SC zp^Gq!|MSEQZ3D%-Z*`uyE^%Y3Vt^0ha}woj5Uf)~g~yBU`U)iuU@k5( z#%V#fvMg{R-oAtnyH8N1>j2Fx*UaiTgeXWgyPF(iQ*@#2RTK`vBc+c@b{A#&N??v} zI69#ZYBhDkY+PLSI6t$Cm6A)KyC^-<%xAnBmI0}slz*69*LS^sIj*+t_aWpY?n46G zmX)zseC~vnmj*vel9?vjJ`%`oK2k!C zGob?%1bx|(1D@t*vYAJ4@jSVpVP^JI!1^$Ea93DnKUZb7Ma#`sIfra{W!V9`&voq6;^$VLX2fu7BRUAVgNvYCvyf;%mT3VWbzmHsEyT1Jz zCH(Ew>Sy=J>26^`?qKC=-{PHKG5TzE+$FvcK{u>N4jFA5P5u$O*e>%)r_J(&iQ}k zX)Kc08vH)&jVvLrBDbS6D|znbLm2ox;&0S`{=y6zHWbDWm9;64PCZcafiMo~A|G4j zgihU937O-mM&o{lJSPi>5Blp}=cGlOJI36f5~1%R?~+K6i&STYCgJHNy{3!W)|^=q zb_EQh7WdcvR)&lKAjf_%>SlzxDxiKAk1zXJXx+z1gIFahSo2~Y4tLXQLpXA>iq;P={g zy3St2qXRbpMh<;rEfAsVwSA5t7eJ&B34;!0h|Gp8-+|BBzFXE`J9D>s1m(m#3mYha zVEL)|DzAczsZMtmabUyG7XrdxMv~b70K(DJ@AC88#5Q7LVhTG9t|_ze)f@XpP3IOa zYF4SxnFeS$9z>YKa&OVIO*)C^8jlW;Q1*vBz(+Q1c%GSjHhP31U(iquT(c-vJ8Cuk zS;g~0b&TF;Ve~rO(WF~LG!tP~I{-o3Z{42*jyN%3k=VADc~%_H40H<;DUouO@&4=c zkAV)lNNnxc6JSU^bO(=gerZ`ej_dd+in52 z<){A!Y{SID;;=9{d|4IWQCPG$5qB<$q*%O7HIr}5Vtd_F(l$ky!n8M4dIVTMp%1G# z$Y&}CNN!WcS{SXIsVXZ%dkqI$B@?Z?U#l}vyOfx>T5&$QLQX_^{zT{tBv^&>JrnLj z(1L5%z`UA$g{?q;bf;mwK77Rzg2yW+25?NIy>b+9-g4i+K{H#C)(@!g>4DsE`#>DjqGExY1Km1jRyS0&ZW^ zsVM@~Xk^1irN_xpj4((nIjLPu-n#(d-}Vo#BakH&uw9CCBM1y53Az}~Du{vXQWV@p zR0$Xlul9?)z%J2dVnTTv>5vnGfa)_?vGclTu!Iq)egt|UKNEX`7NGnQwim9?cO^Ms zp#xo90N5PKS{thS^v-ky^{e1r#mr;r!bIlLF|O`Hm-7IA_N3Uk)#S!DDiVYo>%GZY zK!5f)ASmbx>>ge&IRZ)u^INC-Mj_mJr6bbhy>)~&{ri`;2Ga5Z~rqEL<#yStPxKGN1F(&XrR7v>y|~_bbY-__f+afjcD`9 zx;Sz4C~Qh%-Be>BABJUIr8E%NgVUQpEzw*QShbIJ*N&0Kx1;U=5dV%K7xvZHq`c&f zl7|(6zMSn~96{t7#B0j0;wdkKKI+J^eVlSrJR(8pLkXdJ$d1lLvxGd@7px~>=f#_q zeU<>NAYDwx+V`T~ghbG31{t;wOi3$fQGa4GW~8q{skar(n@#toq(rCGg#xv`w;d;(GkmXvN=M z!sNj2(O_ktT+n#=B7Qi*<7lZdf#1|l5O%KtRR)ocR>Rb%bb@4QXw!|nTQelWUzqef z=gkfLVMj`(07D?WO^Ep*p0AbB$jRu6ApaI~@T!J6>`_t#D15l_SXYq8s1d7u7^>gw zF+SB(11Xm=EWp|`#b|;hi{Q~n0|Ba78gNyp86?)>7nEL8d5R|;*m5UQJ37-E`w^tHiN{X_!;8>92qqGH-zgSMXGYdQ(aR|q(wXksM|L4CgwQ6l0|CJkd^q6zANGeB;R74T{e~271Q@lc zV@|CpZw)sR2dkD`evF zdfSOcJI5?k?>f6`O|6w!k7+;9ynHSnD+H5q&RljpW{A*#k=Geb+WS1s%dgg}s6XCh zteeiW`I-MRdIj)Yunem~Caa>kkl!ZQR?=%aY! zb0VU-%ltW(umjAPmp8H?y!Mm+NWNfRAqNz|MlUHFISCN4bE7ctVZqlK(Ci1%#y`Vc zBzIpM)cL%1cn6bHp3~#3HfJs3arNiWMQjXqOYb-rU?%*$pba`DC8h90eSLj?xxs8T zn{UkYvU2A|MH5-=S>Spw?rJ{&qCc*vsCHTkQI?kt*cEMmIFlI8fe|VK5yh7^`du zLln>);quC$_pasO4coe-_kuOUGjNIfk89zSbzF&y=?91P$wvO(zQXd$uq8UG&yjk8 zK-4+(d4eSp{^;ar0?Gaq*2ywC-J2#XUNjD%&;N~*s_$TF0ZvcI?K)-tet^65rLugz znW6eyz0(ZyokkEABsZ3)+NaL;aCwI5iWlrJo@DL6kVl?WBUb}Ocoy!>(MRzDK)E2c zkP^KC!sNmuZ^MNF{cMF;G;l4~NuDD-MUPysHOv$|Y^0)$;U$T=`m>bY>ay&g$*q3y z9WU}(&yE^FCG=>W_1NR7!h{d&?>w_#!M5NDV<0bqsSyHAe3cW|;%k+JI}PoV*Xc7x zD5M~iPM*M$oAGt&4g-O!No0$4hL@$^mrtDudtv<7Hmr6u)+4a~T|b-FCImm&);09h@J6MtIADd@*Ad*gJ1$2PFJz*@0_e2)H8reNz6Yy&oq^&Q zJHc{Z#b|9mACd?*mqEzCFfO4tj$QV%`@L#%oFsOBvqGyn56i&JhoVP>39Pw~u-q5F zroYo6Rx{0PN@v=oPzKpTepvep%JftM@NmD4LKm!k2k7;_GBbhSmZOYE@MD3~Qe^vn2^aKb`&1%5KWG?V*ss!; zkRE)2ABHWqvJ0#)xsEQ~pp0h}3MfeldtQGdGB+&zobz;~F2KJ`TW&Vcv?7~smy0u& zX_L6n-{H^^Ui?_7*ND!yu_ldM=ex%vmlJ<8n|58o5+)3f*ihtI4=qKo;=+9JMyT2* zW|qf+IO=^p@YVYc_rHzY&-JE_GO=i?bFQlH6}rZ~xh-|>XM*njuCIP?;7BZEbm?{* zLE>bj#?>*KQ5$Z~R%Xo4D!eUz&piS|1XE)@kH{#E&{e#7LLWzvi^$rYnrp}SGLgiP zpSGwvLAk&$@?bWek~KE=sl+KiHF>Nn39ln(22mkA2tbdzWsZb zb&80HbW-us`D=rw;2{wmY}D1-V_8=ZOe2hpK57k&%h%WBa)&gdYi)d9$`-0wQ|6IV zY2WD6tXUoLGbCG)>)CGT?^T;_SM4>PESAOk_a9T$Sp;KBPBX3@TdVeL>%N?vI1^HW z1T78ubSR{NmZARee<$Khk2=4= z#!E+?PsE%dP-lx?A{XxLTQjUOu4X!nrTN`0EJ1mgkDhQUc#Y`fU|WmTvq#t$N3#ARv*9 ziU))^P%P9&E=frs<2j;JV6)TcM;*G|VVjYBYU`fLw(`f%DuE4h5DiUfaK!h6WbP~; z58&N)`XgcjQ;q{M6aMYu=;2x3dTcoV6h@kVu|)`B^{5e0Iav`f@qw+?{I-&C!Zj<% z1fb8u$o9#4^n(zbP?;5?o{_|;Xey?5c7%7*DqMv@HDtj|OPH@AVr9CI(Y4n(qxJlI z!_~Yi&Y?ZiSKAD$PxJE^ghcHK3k!$ujK@IQaun=w0C}4?$s1=9|Dm}S^A zbs0)=5q4IZ{Jb|uBw8@OG4><(JWgkXQYqYG+t7ChDX3E+^Wduys*e9M2xf~#)Z zzEDkFKMDbNuhq)j9jF41)5;3^6TBDXljAE(5N8cX6Npp+z3H~knC{&|#9v*URt7U& zhZk{^4#Ck-yo`Omat@)|V7MPh;{%WS2?1u$lo`d_?g-nm`O8JK+Xp^}tIHjGxv{8^;n4egFQQyFAKzwII|}q{n%R|IY84;A8zC z>?PpknBCMJ0Xz`UG$bYTreIyE%pyOlB`R7u@pecbG>WLhid|+|8$?iSH3l=K?bm`1 zf4jObU=E@$`ce8Oh?CN;a9!sBda9jNu>b_c~ zvHZv5jGo)pp~1P({)6C4ieEn$DxV>%nCqkXtSnGLv+O?WHYmNb&3fnFJ;}`$c}8CW z&%p%`kGKYZpN&@l!q65Vcd0mGGBvy!zC$X1S|f>X=^?_H&J=W z$>$ZuBr(&uU?Dx_V!x&_#dpPEZeg%H)ScUX^w-|b_OfYA{~ad&rv>jd4|2;%(tMo%v2z3}K4P~NV)>=6?*7B^PdUB|jd9Zjr7I_4Pk-`sqmM127%Ll1_Q zCRuH^78~CFZJ4z7Jg$bdslq+3zXD98Za=sf1n(FT ztLc-Al3$HVlE9m9}oOZ_v%^(YAavxV6s zpb`tm(ILmAM`zp~^sO65N-mD3Eg1B_&+$68GAfW?G0R}xFtN4m${!KsK;?}B_pJ&n zb+kv4vMbKpd|VQ{`Eby8WSl)TjQ_*0o|TyGph45RU)+h$wbZ1p7WP?uuasJH@eH&s z43`WExm!QJk*qq^j&_l}DusSGB%sRTa{|NLOF8@)nm#i~!udmWM^vv6O?Ov%5&yFWB zO`q9!;^v7kGeoG|+3QadoeCXiy0jbR#;nY{od+woWj4Ph-EVXmQ%+M$FmC&1iCB=` zS`@zD#q_1#1u24#t~*D4eUpS(wKH`9ffox~m?|QL+$2KnmT63F>d1sBcB&B<*|LZz z5wZ8PPzHo2iO~5}>3*ooH%~lQ4E)-3nDiQ`xWXP9smSWdBd#Z{Ddnt}ef!ROyyxoG zuirq45$v~f5rLvF{4k{x5;nqecIwMv1Aot*ys69Nd9 z7iN?Q@V%4XYlg$2^hXPrVkwzC)f_!SE@HevD{C9eIJQE+9JAB%g~={%JK4%9f^{+=7eoT@XslVjo%|&I|$^AK?f(^slO{ceu7XQ2bH1G@q#eq}}J~ksCO<{rt=4{YW+{VFWN_Z`}V* znEq{T%VCoJH>C;rm9?jvZR(n$@gR3@Y_=bifRmx#CzACQI^BBTsM zslJ5nm>)qia@jPL(Lv7XVG5}le*P77ufIq_Q|vrJg+-{d`_q>fV7mceadl9o>95ME zSo@vV2lPg$*}S!m3*5g5nicq9f?7arAPNFORpH)lwJDezny(`L3fgGqqb{^b9ssjs zlSX1?SsQY=SqqOQAtg)`jjD5M%tu*i;|R5x5>=u!=hM6ZPTr{JWRK}9&!fVPb1*Zz z2;D~3(bqrb4CB2Vu&R9aLOeuN*dfH7YC|COFJ9xEc%d_aLXG}h((-3s$8<@N5XX1@ zO%{QMRpf+0tttD+8KMGbo-2YzUf%S6F@gRSRjr9;^q?cOD=jJC_9O4qzR2Wr z_&Kvt(GWAQlEz-w@j3R?IX1)j&`nIWig4)wI+Av+@`to z-p3K zVLw){JSN@Jr9@v_q#MccDX6r!M0K-T`LPEywN5ZK$l!@>`5WM;22joZ)m__hUC%aL zn`&->{ z^^4O_R|5;Ua9bQOBsOM-K#}|vuS2;C; z1Um|Nk#>^-adnU(t%?M$h6O2-D=K-&2gh6@gf=z6Som;CROX}MlIYbyiv4lW%!O3o znIDv<*C)*UDB}DOq!gE0*(~0I6`q9XCxsKx(=_U^LRXdUKL*JKHj7V)9>q+upjVZh zHj9pe38HS1KBo5q%ENH@6)W@_c2+NmKF9oud%7Q|+nHRjCj3l&MT94xO@k3}Dy&%q z1PTjMHbVy&q&$bpew%O)unN~By$1#10V{!n0?y;;VKe7l>AWRli_Z|fT8AUJDhlSo ze1RlCo(F7h$ez4!5$s{}&o2o6=KOpP?hYdY>u*F&q8@&-D2x??)U@t$4GH`Ux(cYG zH*gB`+2;@7sHRdNCa$=lo@?VY+tOt`(VX(aWv0Ejp=_&hA%wB?K%(uag#r8&L`{D*$E5`@ZPT>vc})sR<3LbfI$B`COwEY&y{ST zs-8{c05tncu(JDq!(79Mt*A@@S;dzw{uLZQO!ALFPaT1s6xo+s5oCC&4kK$+V-2*K zxm03D`C^+d#FTtEm7h@_O{8X_s{9p_NCYsZP{8)S9%X{Z44Xz6nERuLcRil3DrYrZ z{0%+W77R<%N%#akWLO~HP;sFz2v@hgHi$IKB&l|cI#0yiyPWa>YE#1bstg7;Ry=O0 zNV&Z2xS~}x8>ze$15(VmK?B+jfEK7bZFVouF`R7BRAi6&WAFHBQh$F=pk?<5+S!wf=*?#K`xMNVKO%~b|x+u5WPNl3s2Le5{B+mPy%nN?wFW% z!P_fi?b`Z3syo?HMBHv-XKqU>HJ2*mcyxTXT-DO4mC=K zmZ8(^J8*0KUKWt?q+*2Cva0%X{ysb>WDuU`DdRph3i;fwp&1mV1TLWJ6L~%{B=AKX zEcy!b@vuW1Q_ThPnxFQU&=i{1$$C*`7(hKF1F_kP3L6e^{iD17Ck7aYn`+o&6@$ox zyEOowMP;#(R@fC76EgMFc+Db0Uq)`;RU$P#dPiv!+_v=>KY_L38nifDQYAx&7YV8W zNqYa9a@Z6B;`?Cr8@jFE1u+$xEsBMVN-H2q+l&@w37a0=(PK+-CA=%akP<&)ktec; z+yY<_5&9A}Y=ExXpp5XHr_&M-DLEljMRCtWgJ(e8!)5X$RTB4878P6NGkWD#b==T} zu%N^Fk-*QBjqjUTS~w)Lx~{pXXxjAf>8Qe0Fw% z9KygUSa@2#SBnVo%q5!JwW*h4`4=zKdXXJFOAkp%0SRj<%^lC9IMylbC(e0z*;g1W zitq4G8b^m-zYeYusl6a7hP(nw2?qcIkb(08@+AhvHf_=&A5B07YNoT~!8S2|E# zQdPq9@2THr2!LLfMpUh#r-zi_UW;zUJL4_B1_h!|g7$3cEe%yuCLls{9aj=aggEVu zuq6A7i61ws^_w+qmBO8>l*aPAxMBSGZ)@qT~r**<)uQ7 zCH#b`#E2uU$m8s!zH@IsS1WDdgG2?2C1KY_ddgv}FgC&B+h8#0!f*vqG_f%R})pELi*c?KG?^LQ|TKxbP%Q3h~rb)3l|;`RS^%zP%O` z)cr4b)?9^LX?Ji8>7tt3&zZ?fKd&OXSOi5nSTlJV*m@}5r0BcIiZhcFT$RsNCkh41 zamK6-020~-Sjs=?gk#mD0@0w#u{&DO>JbJs*o4Ba1&WT~e^TTc2-ccsf6Jj8UWk+P zYi`)CEf%KW3E1xD{a@^^0tK;cSpcg`O7;8Z6`xh*MKKJP z-XSnLq^}Y;qQr26onjj}bjgew$$pBcmXj(STz{4jMJS$yXrKXp1q!zwOYd*MxjEyf z-r^?mWv4FIaPpC~**bqI9ZY?p>z ztJ|}~>!F_0s&07c%>kpxB|yFWg$+1{gPGW%kj(`S5*1$xG~LC5S%HW6(8+7z0H9p# zm(^?e5Q8emDV^DhtW;Dk5_&%|u=R=YEFY@^OC}iz2>xB< zVa*qQk%ukalXpw_z!I^Rqbxb+* zHqf3$T7ZbfZix_Cj%}JNZ<@Fu%9e28-Lh5;#y%HzAL03#iT)#m+{~9N<2@yrQ^g>> zCXE96YPY5-1$C8-(=XoRC~2I2b+Qcvd(9{1SpAZvEC&M|;A`3s4bY4;39j8O@Xo0M z8L{mEs#JcCjk6|&nTiJu`lVrehP8P6akC~KdliVAH4-F+z93>xDf6!~)U>%>0=XK@ zCD>m^m=V3l>Fy&^Dhbk)U+hVZ!Qn`}RcBluu>hVhgQuN;Hk@u zAgkKR0p=a&oYn!&y9_qr^UqCi<{s!4DO)^Q;OUX}M%UIDnF;Ht>K@#b^$gHN?D)a z_%w_M;dyxHg|j#2e9CYUC84|j5$HV)@HX5GlDKiL9Yfek4}gQ))FUp9SZ<;2k|AYR zA3$dI+!6rm>!~7ALZ8@uOK3~)pM6VIIjPySRZkU@kmud)`(vc0llO=x@j(|x6BmeC zP-kQP1mtpP8o{dOM4@#m&gWpMod642$RbE>`WRq|CqlmF!q^L|0vlr9M)x1QLPz8L zR_gw=n^?wZoCAxh5#itD0L#wEno!n6og4ojZC@Eyb@R0=C7>cDARs6p-E2SsN$G9` z1(ez}N=k!>bR)27X{4kZRJu0ZrF0|RaOQ{4^FQx<-s@cF)AVa~D^8EQH#T_6(8t645 z)$eN)ApbY?po`iP&;IA$YT)<9A!Z2_baGme+?1V$F76rscZEjtr#BF^j_%#b`bAvBW7n1BmXB zK>+!qCUgJ}j~(0p&5bYVIIIWk5gcYZ>5p!}kmo)Atl%?bs4AGCa3F5zWDb4rFIq3J zRtU%(O1^;_KLN`W2cJ1XKld;-ag~_rj{L=ggPs=JnBh(34k>f^O;jw?f!7kJf4#mj z;kOT@^?lp1F!URplAXZEiAbZT-Tn0f1qV%*iZspuI=N`+@3&0N1^0q0U1~E3VSVpA zzNMI;y3NXzI;@g|oiTbp`h7=^TSGQ_eNOSw+Ogc!N#8XJtxe_SMaLYIl>#XOmttz% z>()8Zr-m|rZh>UN7o6vVo6&P~*iZu8&X0o5t$lRk^dTKe!}dp+*b=H6-j50X1j1jW zF$QR6^ehti^Ixi3yLRAS>*Q}d5fQ?>TC&%Ht+zjRG*eu^$!-_IKSbgHY(f+|r2EQD zrj&}1-sFI;0^API6(mR>e?LmPLzct_E|fQV-5dhnr$#yqS%9`@hl*-Oh39Zi%lcpy z6EZ>>gd!qQ_O}N-t}i?XIum|3@DrbI+_(gyYEJ&UdV_6hZalDhEb#rLoZQpFA8*V$ zQU6fP1Z%j1{U|x$i;Ai0G$iBPmr}WZZ|f}r5fL>!T>5LJe58mzMNMioS``2@I&WT; zNHB%RiE^mjBO(P$xDWR1)#RmmC+Pznu>FX~;5eg9vIiZg!7li{v&ev`N6=68xH(DS zgR}{fcQ**u4+oxN4aJKL0Xui(w|jj_Qo%E0Q>OV=jqCe}N8l|h!L_yYDDK_hYoN%| z{N6cePwz0$PWzaGTs52+QV;`~Q+z2P;()(7yD~c}Sg(+6?Gw4rdBznE%fEnyZm$>9LegZg`#uk}LT*sH!K+>Sbb^Z4; z+Yu`u#{*?bEK>q2A+m>d(BG@}Hl+lPf()e@_T@|FFLX%iks?Oiq>z1P!Uh$KOzqWs zkbI5_7_>Nir0E(eGK|5$xr0Og{zfEM$T5o;l2{8CD91_q@8RVn0+BGm+dhL2*nNvXEyD56nP)Fw#mM|?)InLX?>|^Z>LYJkbQnv$Iv(w{ti;8z zsWC!-T@rkfTkPjG@;B>b{weYU_~l1Dk!MkO&->;7`iL9oYtW%gN2pzUwSB@qzqblo z`o+fwWIuu*5`{wU93ZOT4jAL=0HrdD1vhwOB01Sl$!lBSH!KST6_14ei*ya=JXky zt|9n!rua9}z@q{ujsG*-{NE)Lh=L_kcZ**`md#Nj#RwC;?&d-3GAINW;X?G1+aA#|r%vfnK9XrG-S81;8D0I3Epiymn$=e7> zk~h$!K_bdz&BPSHI#P|J)>!l98?=qoTc{8aU0dUNwB>yEW}74R9`Bn@D0EgP12-)5 z9W@C$6BST?*_?McKzHt;&eJKxdXOk~^Jp150g^!bW+@5^RyPf}$QS~DW%>*#LVzEI zDF!6AF+zqi@u(!TPk!5D2@z0%0o0`&Zup^OqVwZG7>TM3*c4NbY_@?IkO3zrdJkCa zEdpgK`D(MD8D?BSr>Bhl;UFi6yp2kmJ{gZc+TVKxT5xhlyv|mIPO{?>RMbmaD+hI> za|2q^tPDu~K44$#DB}<0^Fd(V)FF8=*$p7#@PRzK?}*uf6+T1tHTi(-dSQgQ10H=R zBA-DTig*Mjd>-j;%tZ72ck+=Vc^fdx9Nn0y6&Dm+ir8Etr+Al)Tzx-W=PfbFKZw|Xt+ z1;}>0kq@9jJEO&Cpo{Ali!;l59Bp)6wA1?g-Cy&nb0GqO$Rn^2Z_}qJ-e0l*xM!Px zgo-Rf{x{|jWZoI~vD8JNbHY$qBDlUISl(bIc&Vz7A$@v^*O#kD5AywZNHf0EQN{n~ z;(-N0diA0TB?KJaqeob#Z$W1(Dd=VwXBq)r=SKCA;|;k#B=N!{m)m3Eg9z)jpOARS z@D~vNO7G>BQU|zU8V@MCII${Kw#eo5gGL31~mdY zE+8K&{MRFh@b9WD=$$~t&ojH1BL9XJV$?uxkH6i!E-hL@w2m^0ZZ5(K9tJ307I_c( ze{k`Vt-#F&Kl1@g7yUe_4n#n}kao^XSYHQnc)dfRw6vnBI*>@eS_zN-4Np&>NqXZ}@8r<3kz%WRYWznw@J*x%uu^2kYw=?+tb;X^<{};35LI4iJWdO{Nh`x?a zR#|DGS_Cj)&|QzSXN00T5mIo#(I`Rr*LPX@3F;k&VH}?8+6M!)RMBUNA4N2v(m>0oH&Jz=RtSLKOS2A4Q0UZuGR@=+}R*+5ZNb zEGO4Z1#mcA3gAn#dX$!*(*bi9j-vJqT~$oP(D^e|O}|cY*dB7`+};^@M*Ux4`4!** z17>c53PA4T)W)p71rLsme-a_}!^&tysyIep0_SU||HvAc>c9jEQ(RyRG=MEAF?T-I z#Raw?7t?qRa)s~VA-NUw+F=ozU196c9fK~Vmq;8q?(6s~oDsmRUG&OKok)}KofJTrJ0j>hK&S#+(U!WRu2G|oQ-tX~q%HK-5#%7M0*5%7sT(9N& zd(9v{Y7GE|Q!iX^RSeiX9Ws1}@G5zZn-w?EaY8fcgGSV$T2yV(l2@;=acV9o-N+BG zHTgZimdKwc-aGScSM~Fj*HjhZ4T_u5Kk4MJb1yJ{z$=Se5!@hJNI=*;h2A1A*vHs# z?wIU%b>;+JlUmQiEU55=AXEDLDEx2gkP7RdlGucwW;i7x?$^uXQ~!2Q!+CjLc}M8!-S<4ei(77@eY8 zqjiH>3Q;x-e*+EfuG&Ts z60+6eOt}`wM7}I{m`OLsrp07M0k_#eM#gDpPliKocq&=z;)I;LvO6aOn91ZekR}n7 zI@r+WdA9lFWG$~KLm`3plQv^r8O|DJr!(k;$<~1zJ)Eyy>Nc6W&3KS2CNr5UGgGv#k|6rulrlmFJpAfXG+4i>}m?8wi?c?Y0ADW>y(?*x}WYxfazJIl%1{x z8Eix{!K5(BWS2EYgn*NEVVwktt~!`7=(O6OF17WdoIJ&CH=yo#E{Mx|>M7sxFCXI& zo)L!HcG->04@op(&q`kFG8a~*CBFTMjj=Ztb(###l@9sUNvX!!N0(q};MS_tm-aK@ z>hh>@7tlkyz)bfy?65;&Fo4&ndNHPpV$Zq`%mTLsowVQMCb$z_mRXD{#^^d~ua1|( zX*>s9aD~s`MrmD~=9eNPLEqu-N|vJ^?2YHDpKu>q-!+?MYdp6-GOJ()K0Qn9?W>#K z1X$u1?%5ydB0*?@89q3?YCO;77~m$7&7`5k!Z1 zTpS1`w*LMxG)ZSwSrJ4Vd(P6%d2L*B=%Elr1(!kBzF#8k=UoLr|Et6&9g6b&QUj)O z@leiCsH<9SOQ4!KBa>FuMikk^+0n=XnnOYzbt{Db%L0M#u^z#-%oBD&8 zm2NJ6^7h>9V0 zns(wX1Wd@2#;Y@7wQQ>t&2O_FXC1OL&35g_oXJ5=$ay{qyJjaQeo`|G1hH4f`BwOhkl`kz(OB}wsHK`Zuzm$o?r zHCy#Zdd|?iwS3yPhn{ZMj*oOrq$Qp~bIRcg+3yuP~0Q)Q02N%`UnC1%VUsuk)bxAZ=hue-7zP7^6rx} zg>dn)Po@xy`7JpP!&^t^d$+XKF21i7=qG4x^S%$iW{d%cYbp++b;W2jS*GvrSHAdl z2jB1fot@$l38rl*@Y}ES`K#6A0pGC%Riwv$)wk9crTK$t&l>h7 zQP2!~_fE^nQ->uIBE7K;ImXpURZ+QHs|3%Y=#`L!7wS5(Z2jq_y$@{LsY;Oi>tl3r zqh^Cyr1-vp&+0ye*9BuZ+S8>ZSl4v0_+`o&ur?geeeyl^MpmW4ilgIM*oji2E!Yq0 zBWR>Lbu0>Kr5?N!&(+}V%%(=XY5GJe8h6XAE{$3@Z8d;CnWIaO!e!-4^?FhD5}EVG z!Kg71&}T3-lE>-Qll)(jvAqKx+V4UH)HqQ>dR$wGwA#Ta5LRoI7;t;KY#Cj32W0lp zGr_YaK;k|zfx#k?GoWnduYS^o-xu8*%=}21$(x~ZJ9_2fYonqVUJ<{kge4W5NUm>b?*bI4Y~a+vq5fKQ(mQ_m6YJx zu#5C{^m4=777Ez|%td5;Nthn=kE_@t(!jF5B>fq% z?iG0$QiSXKlNPfa)<#uSdv6AmQMik zlVHr}K{D@l02or({B1wMyH)RX?2e9Qam&I+X)mweG-~{2#U!#0Q==*aH7hlvlzeqV zRAA$q&ol6!u}nXRXuN%ZqJYs?g^o%~;J?9q4C-Lq;>_W;8JRg4JMXtuewlh7&a&iq6109Q)GX9`Fnzg)q2E zp>PhJh%qO1aXw6#n20Oq#&h^`Uq%-)=Z07d4ez*di26#}a<)9eK(U}~K-T0YQ9{v_ zmhOaf>{C@vCC0QM+pbCj%%smPtQCEk9`+q6K6m-0N5I{cY!L#6*rhG@HJY!UGHehF6`Oq zN_t5bTdjYfw0d)DLuTpKaV9n)UAsR%O*bQvXD+UVDrdm0GTsSOnsLvoiyqD+VfUw_ zq%l*likxtHczr*=!|G#ZEL(jiAixfbqIxV*R|t#+hU0yxMwHwMv|b) zP${ohnJT-L5>UPi3eX(DECh+wPYC;=txHz<%c_qC-CGr)D~v*d;(kAPEta5yR1`N{ z#_j%o;7fBx%i;=y5W{u8Yl<_Vazi0LxTUKT7DYUz$Q2f9aF*qq2xQ=wO}>q9O$%3- z+gD_W{#&N|ACZA+u-7ZLO2X%Vd=U5xAe`i+CZ!mv^tl3FVswRtDZ#1&VZSycqb~m1 z7uTh5CT`@i z(Rxv%o}Tj}QK@A_V6o8opXW4pR8_)Hznq!|5wqI1e65m%=d0k$Y|qr&!TP2gC#}*@ z>4>ULC+j*`FZ&Uu#5$%_2qE_xd}=ejB(ogtQj?jkV4HPIiM3k(`H&}{nNo}8vI@YQ z*H)YI2bEru@R1HJ=6n-aO$)ovMbo4R6zFBT|ws4kuX;2DBm86+Ql-4 zXgM*$`}QHisI0=VAYbDZx57@(M1SUEkmcWk!u_-rm;;p4dx;o=aGwnJ8w(v3JEz_7 zUtsf@2mJ?1^o!jXTb2M+gzBjVGkyqBw`r1Ie=ucU=2NLI8VHyI$<|K@ZJ-ZCp(P@C z;Ca4dvUH!K%m(B(qWRR`0^4;k<>)sU+(`mXF<@LS(%(4W%Kn$OFRUgb+*R{Fb80`r z+(istr39Nmw)w$AU4edW@xV+mhw7r?#YB6=ADZd;Uxw*U zbRc;R;hFum8toaSkDa3ebtpa0-<67wqhLPxC8#?qc+tfbxH}sHs$tD zgJGI|6&E2a7lY#|%stB~`ZopS z-kGVY1SeHXqhZe=QteN;-Ek)td+kYibQ^A?C{XTxbp#a)->@Jm<#3UEvre~o{iBTW zzoEvKulrhJSy79r^EA0&Ax7$-|PP zTl%u#3I^@#C;6wVMG1iZN)o&-%Iz=B$eYh1JnZfE0KM(C!iT znId#5Ttym*QOT0bo=afW)P=E;WE!t^voZ~>juxfhQ&gQPuT~%v)8T$fne2%iytGv4!j{-I>q;-VP__sqPbz67a4 z0+JqNUzt11wGW2v+hR%+wH>3Yk$qq!I=xI=C`HSmb}Uxuhli|k62d9&hlmGYns~{K z%X$aqL$^o#u3={vChH63|0%sO_(JlLH6w9e0`fsrVZHi*PNQ8)@~%H$I}x-gy*68q zlySUc^_?|BAKIqngI86yTiQ)_Fl3_bgtGGpV%+)9ca##tUW%`|A1o|D$7qER5mkA9 ztem$rpxE&7{y}-MmU#M)01&1SC1OjzG~N98J-6X7jrThFmS*>M!DUz_5UEiQM#g+W z+i~QWCGgm;KU!GLPC+I}v?eptr6PE(bgXZZ>wYr%;!DhB5DU^U86H^Q*y(f;@Z+(; z6aphy*kuog>2%pvKHO&GkW8)3p8#?una8w36^3ZI6@1 z5RRWkx|t)3P6dKlAL4&E`H`!HOsHW9luHA-9G9o7x-v2g$VIYgWpoK(WKLSwDkQrf zYD?vmeBLN-qYNoo9Wbs5j>`g5#iy{8ri5Gp%G8N^o274sUw1?vZlzj0uvNP~vrHWYaIE!?%rLRqHB4M`V#hp%I-)eP1^j3^dWYcSK;pBt5vkai%$| z#=joSHknyEG3K$toa+Ld&R4iJHsI1IZ*L2TZnm%OWMCQL1se@NUwv{N+Cdm;PV)CJ zeiRTo5q)*Jd{^6Gr^OD#<80b#Ag@=iWQok8ohVi}QUEDw4>CCMw&Zw2F%e&~`12Uk zk~AC>_JGzGdiWvw%y`|$D!>0B+W}_8nqaC?%Z z6a+AIe^m)0HMP9h8r^QZ(o^JHyhk%o{paWHy^q*iZkd+r!Hi7FaolE+`M;mZ=RIv@ zwSOD?4!}WiJkcOc_Uxy1uUkdHlOuppoA$RTdtk$q&9+@WbMc>mAu)z6Pb27Fj8_(! zYwn7K1g%eEW+=bI+P$$JKS~$rEs&&(sDL>_YDU1{rFH4&Qt+&*s?^+zL0V@Re0JQN z;9>HFhOP%N>Q;1zl>&LNaB1l}Xr!+@NYV&af|WkuKLT-XhsZLCGE*ukmythypDCTi zW(&bOzIYoKNi1jlb z<)*hh<9^5gepdKM8QZ&{a#n?(r95MbV{Jk*_GsKy81A^vxceurKTX(?T0#!on6mTS zdNI_)SVkf}Yqmi%{R?rJi>d^vzsc7xFKaF8smah`<%V)H|72~dSVMLqGA-o%lfHrY zyi;$V(+KTY%mdWsaf-)jZin4>6wHMZV8_N#E&$#)#PTGaM3X7C-#Ej*IvyBvQB8gK zgIDAmtw3POFK1fW+6;R??|ntRMIw?iVxY{_aqJD|xLI}* znQ`sWjO${A6lDY$#P|Yq;Yg|5TO7)~7&MOOwgN$PgViY*@&eyQXJw7nat;vqd<(*; z!00?ikYr>bMByOnwi=Qps>t~+F^+-#YD0pFPW5*{pryLyyaIo1k^1fKOawyAnJr^A{$Y)~yvv%q$AEF{JA zMB?;n(asvBhd#y4Acjz_KP1q@-Z>itjHKT275Y)`jm00MGPN(5!P8w*RWW09lU}Qw zji2bcJSTg-3KrQ&>|_}h(dmG)Tt@`0m3SsK;*s|q)L3Z+xy8SSe_f9tA~>GW-gQ6H zb2r*kc=Gv6cFIIfw6~8^_-JcL)TB+lcFbzumwxBFLs04bbT_Ml3u|aMmN8S^!j}{a zF8f7e_L(M!L~2IER4&e63!4gB0;iG+d(`0vt)!js*3LCj8D{&uO!KwDalm8~cLt5w6mzm10>8ZU6&TO#zUxq?0LcA?8p2uBED|&n@zO zEgFDW77M;Q%tI2V!(3TPySXIozm!JZlZ7}#41Tn+uLk@{>zp?UlspWAY}ZtO__OI| zq+t^YLYdvn*9DMlMLEs7KOu(oshbydMA4JK31*YYkmGiSn-$O96D;bwhtYE<>$M z(;g9+>)`_g?TuSm|U zV#U_`Pethl`heu?pX)Z#S14%4@Z~mZv@n4Rl_TVR$E;w7nB|7u-7X_A;mpE58SMa& znX(${a(n!|bj4(pG)}wjr5MQZ8$BGs?}`2xB>%5w7=?=nVsLA|{jGPi6}LL@iM?hh z0K`>5B9v4h8Aa5WOi*K z&eCPLuBsBihMy^x0>W{TyU+fJxXLEG?aI((yzsm@$OIg&>Uj@x$BvZ4%}MeI*#usz z^O8l1L2Wn1r@zq=oQ9vE8bQUFLD19*0N8W@RCw2Hx~3eTAvg7Bmh4)I940Hs3nYzx z6Tki!a0|L6r_P=Bfl^_XZ=GLw5gok@r{-L#IyaisRDD0Ki=d!-DV|SMF7r@TfABEx zSi+aJQys3}pD{;}#3|V|GjxK{58zYd&~+XrN)<}$9|6gAbAim_Ep*d04pGl>1-VCs zlKLfLtp*V8WkN*q;~CqcMB6#a^6*C`2DAtD5V)!`269|hzlR=G^^dp9!@ZV(Vu!B| zW+xb@pJ!pQJx(p|@*eMmQl05qr^?!HktiCKTAY8y59-a}arKqL_txnRd~vQA^y{~+ z{}lVQsW}civKOGX2zu_P?O>Fj)sTZ=wll<-A?yIPb}GY%j)xdjqv7Xs+gFyQk^KGE zKTs< zzIJsqH?*^wW>HZa1KXZGm85jtSD{f}nf08>Cvlo`_N&{U3qo3w?sZL7+Zln-@3V1Y z<95S^ru{fQ5pT}h-`~-$J~OMrGpbLP669xo_=NRRup7?@6f3o_oNYhRj&+K$Rqb!e zN5DefO8R1gYQs)jN_eH?PG+ngHPE`^jm9gKfpn>`@8&}G6Rs!5vGIGG6Xj8wp}!t8 zur3WjPGU*15uiY}*-c_+wLfn$cMed*#Mpq*#|K}lOpEFdk%Z?Ej#}<8C#V&tUo&;4C^UHQSxq7te zK!Fg~_;5qmNKooUdGE}LoQ0Wa<&0xq-5Lu@X2W#uVCTR{uG+=Qq)h|o1|}{gOBd_% z6(#HARD_^w1wJb4x9_M8a#l4Yt-e)P)X=p}`DuqJw! z5)mG&#jCqnsj{6L?2=^V(p>GmQjLGG?nq!4RPJp0;6D)=)b&nL!0fGrR%ptfng>tb zhHsP^Dv@%()_X9h+jj$wO&_8%;}}GkZhw)}ndG;53aTH3To&luEhx1Z+#b}Vlr=%H z#}MuBZxtLX2s9?*4&yh7!=zB@R+VF2WFfLs?cZ9Fr-LW+O)Us4MyrL~|8dKd_nirv z*=%9`GXS)tQR6Dq?Du~nw_13XvlU&*tG!L(2Ce4VR}D(>yW~cgZckHNcuW z@A9em3OfDvCGkgw+crl?$&Rw=ZvK`m;`iw~7!5PGpW61i#To4_P=9pkYTFAuy@{d< zRS4ToI&3^A)p%G#(*ge3d`-0hmoHJ@W7U}EMll9^{r*779$RfYlde{=Uw0l+Y|gi~ z`Y9#7EK~u26fq_8t>?o)-72f&4KkP|>4A5k000|YH%HYPVqCYY+x*j`v+0Ni?bl}% z9H))7z&yHbEqzbgSU%X`iaZY}ec8_N_3Sm_&US$=VdgPXgNNn=i;285?+N?IV?RW) zJwE_|>{Iu%7Nh=gcYES=_rq~1d*akob!n0GgS;EElV7Rt+njc2Q=5C0vP306@ zCw17!R{}AA!A8w`p>r%vYDbew6a_lBFDG>>W4B+aFLurg)xqhbP{zcF(toW#6oY$- zm|&|jrJjI4X>{dDVQJDYbd>MBBmbn?=Zw07`GT)_YK z^Q){)nX_*(8X)l;Yu~tkcQpEjho)NMJz|j?!nbdzWj7tX`k6byHvZ%MUuYpPVGZks z{ik5sm>1MaPv;A|?$%Y9mxmvz%be->vF)XI{aOir%=BW1j^ccT_Pu)9TTnwvpmnZu zHm|pt%$y_DMPSuG0N#*d|^xiv)Pi8s>5u z)%QJF?x|8uW{()2AgMw9a~85K?K~pkuK1SpRhy}E`FU}H?UAm4dG60=+tmW>$@@Ws z^W(-xddRTv{2lFiMgBMMmoGtOL_Z%2tRX*sc=?RRdg#yrQo^22?|rj|&X zY`>a9lxSR360`OjzQ{GK{n4Z+-ZDEs&zP1sf zxdJo8Va{_F%_V6o!#|~_3JDn~nN_z<3qfrPMzPMAok(ppSrIbw4S%Sx!G5cHxf4LG z^q}XJ8ak_gL#HaH?fSSXk@5Kk%k4}U5=_zl0NNM9U_QOw2w2kQlF6q;mkPg{4`5%| zCQYPR==cl=4G9?_Cv{(ZZ;$?7FCf%C#=Nz+#AtKnG%!V?O-PPs9Sx6r&4_wx>V+w<_KZmvJ(xFKN$ zAmp}1YBw22+_@ZMD-;UwqAQH~qMJ*HH%jjG^$Yx)-Ldw~5K7V5ol>k+_&3>e7M*l@ zGm%v=Xc%9Y&MZBIHEI7RKd(^IX$hUJsBxQw^j`GZmql{|%a)ohg-=CgE)pS@h>$NWy@!g;Vi!J2+4 z5*Ek#cNjkNeB$q`&d|5KXrl4IhKCN&^NxST#pB==}KA8%d|K1D!@o6O(93PV=YIodYApro|x@2Dmag||JiVVJPk9NED*7T z1&+BAKgv1FDsgbd1(s>rqS9u^W92pXC(N@>k!CH&mGR4;|Hq#Gj}0SP0bmPX@2eOh z_|=;9&z|zTBPLpZ!T;H?ghlB84WFpH=ng0{bfUj7S2i)~*jt+31V8UYGb&nKt5x~9V@sh^>Z{Gk&&>{dS zxfLu;^4}Ig3>NW|TCN)Ua3@5$gIXQ{dMKdW-DTbV7s31|f|6zjasFyKtKh$I@U_n# z2AssNWKliny?#jehxYpsg+9xJf7>fS9|gM(2U9Ffx!|_^P{X2ses{xr0GlIK zouAnj`g>SWz_oL%sydDU-Y)$*rO4HPNjjmmcSFm(oBU_fX|@BDeV9QFIU~Pfjj4f$ zmz%piN=cA)ujfe$G&g-0z+jzSMRjWlquakanTuE`VqfxZ4*@`h)~5cV!tMCC)GolO zIsm1NmO{WW8q8)F(cA;#ybWX0qBc4yKkl3<*UQ*sHYgCXP#TP2M8%qb(gJm z7J!IFx;mAsHsgGR1gSc^P2*lDA4vO|J9k5s!b_xX2zMp-1RcB(_KW8>Q)PLny}VNj zsPfffAV~53)mascLK^|nf^a@NlQY#p*K7|zn~5^B8Rz;UWU}BknP8Eps0?PyayPk zneU{Ts)5J+oADrPh3yCC+wJpV&lmvyl>6ZQAF2lDi@TH~MMWF+@RX_C+RHQHYL^9R z#F^G3Js&9@zpNG#eX*N3E9~nPQM+ZN(&{ z?@0YY4|oJVix0`IU)VB(>a>%y!A8{X#vXh7*>)p)(t{nQ2&?ma{K#&KoS|M1z?vj= zxCq%jtljREdgy6+ZUIro8vGtS&a>Z)Z^IYvYKrJclDz;yt#Is1o5trt?>*CUa^45D zkiHMsfXEw!)}D-qz6Bv(tNnn z>5r{ddP1iwG=`&HY$uhuy{xJIx3{V`iC4ev4^&%he!3~@v_32)|E-VDA4G-N+u%$} zSqCsTo}W^%>GS`n!`!NEBXxT@mfTrC2j{zon9gbL-M2I8!=~?qS~+&H)+X9-)s?_3 z#%p&$!8~5%3ekiN_q5!vtiluS%6DDl*9%B-2dx4@LqidoXDe&|CB%U3)T;D;nsh9l zZj*wl=U_B6+;{SvmFau0ZIxlUiV{l5Ov3(;%5s|f8ltkaGMKBNs6?_na8n;81i7E% zlcbQe-2^a9yPLgB_f~qi2(Pqv?=gG3sLjQ~1Uc4>?ws$88$< z=A@UC`$wEZOGm4^Y;CTVuo&U|y`C+qFX>0r^79RhhtAwGkEU(6G8Y%hFDg@pPfsqA zX!b|;hof!$8()ZEyA~`sw@gdSv0IZ_V&a3-_HRV*UwDt_Eqfk(EQhBMdi~2fE8mY$(My$Vlkp6+efp(dod4?lZp zUF!atQflMPZlVLy+1r_}?($Q|phb~8*@K9WHVqb%uPk==1?CuL_qKe@G2tBFkG4U{ z$85}PeUpr>?wH_+ws?l}iHBIzrqg-$Z*UEG?nGD9yk)BHVXr?AthW9W9u@34aa&2~ zz~0VnAI)&%%O~62B{o}LJ+~8MwUNHx@M_%UiX?sdC;Onh6SZ7_B+SvK@4U%V>iKrI zsb{^Xawev0d*=OoPi6Os&)rQej+8U9iY|KPf|ONvrET)jkQyPl#ev4MRzwJy4-j%B z0&VydF+)F8#8z`W@Bmi7yzVX^wmssiWVbyU1&&H(iTCS4shT4B#J3KUm)y^e&TsQM zx`f#^zijQbu3BB1dUwv^N9{-sYQWk)m%g79e;{vOc+C6jk$D{d#@x%Ey<(Ped3a{F zrk)-AP=PO0nXc%5f!z9Alc){>A*)mS*>m2gdL&}_JEi$Pf6>gt?!BPoveEPNy*dv* zpPZ1wezeo%KHk;kx`vyJd-ZWXw1hb$*BpT{DfjXhB^7PGfe~|V)#%*TsYlh{qDHJBe>QiBNf+tlC+8w4LG+Pi9IYHN?41x$V}f5yj1xzBxShLRGc= z9~E@iD;-^3uAM!i01BtGGgWx#?0YQtY?G`~Q)3<8tUqVc|Gpwy2Uu7FrP|g44RCU` z!l8f4-W0hX+KgG|S;+_Y3i4*1`0rJl8C%rq2oJ_3hbm3&q|^p}XzC55rhe{=t{#=v zJ4;612mhqgVu;sQbw^bRr3chg&8%6F@~}eU1)DX@mqmg{W@EOK6mFOcdv<5rxurQ+ z&oRhrlA_CIQLeb$_LnTKKA>|W+2zTTJr1isJ(+QXjS9l)@a(3a%z7k%hNv|~i@6+a zS+=_q7W?IKUXpu>u9v=9LKbl5)ORs(^D(p7X|lV|j+J{x7<6q{7nk_Yei1(|Gqb0{ z?4$;|qL6f2S{!wta<)dF`lGj#OW`Y*);hc`<-KiFYv?yopVO_3A15H0o<9E7mf~iv z_#195Ha<}}(mND;_vy+n$-ewwPAjBMcFTWC9|VqB9pf+zmqe&t5roG5kf-RgzXXft z&#`#;>U<&&xO6P_}{b%_e zMf+zDQk1%3v2N~Zr*EH# zi9I;eOW|RDv7gJL@j88uWV(=h74AP0o1wnxKDx`g2&+x3v;K5g$`61#&||JDpv9(k zVwq@;eW%py!Atdm6nszc4tRbv%lrb^oO6Jo5Cd)45?aN*s9dkMYsy)-;eNN!kNa#z zXj&wZ9ak#L@#_JMB5Ihkm_*Mlm*L5IJ_9wEI`oVR1hduodZ=VL$aMu{KMJQ@CJr^( zSxG&CuWzDyDlPl8O(UbZS2BEdR}s*r20tZ79HAIsaL4L;WFi#A&JIo%I<^c#ls4)nDRG!eTsC&-yTN(w z;tg8@^&R~F^+3&CLo|9~$mSQ0a2{fuKufvYqoQl8o@~D>U!RQQ`2D;c2+|ArLQ+mt zI$Mj)&*fXtm-XJhX#!#A6xIlH5i)_VN1L9{>8^0s7yoI=F;udPsZvkPj|8A{f~M;s zMp7?{DR7_xNM)O#62AO(b~#rph0jhim1F9b<*go(^yMkk+mo6Xq1I=a)DkxF^o1vx zZ$|Vtj|lso?Y7h4fY&8v_jxVjj&?tswNbj}S8wsaD}LtFuN$74Z>of*Xs7JnT{wNm zx&RLoIyG%#tKAsj0MmRWV|7xnCwoENX7t+`r7QO@k`Q^kGc+vIz=U=W;(G)Qri?C) zY)&`H7OxIQ^`F!M_7jd;mb}^9nDckZlO9YH3`D;a5}y|TK%mx`>e<|^hl5(@5l&Rn zvg^gh;F;E!#?*&T8A;$WtvBH(no@mgbv-kV>LuF^+a9N@**GAWmq`^FnYo%u+4kMa zl#E^&Q(o^=SJZRTx2*RCu1SBYd9G5oXD{$j(p~zZ?GmU~O2~X)+O=vu$iIKZt3D4u zod%O_&#fN#5o^xFw+@9l4`m1iMn*Ytg^VidQOV}qtPQR&8+T0p{2P6@lS~wNpL7^H8D$z z>{Srn8`O5(AC-)6IJy4wG8iAI8UvxofcNP-KWb3FwpTl9;PBt^={A_qu6+UMR- z0Q^o^_X~$CSM57JhMt>6gUPGiJZsEYrqS|MxvzxxEJ8H;*GO=$qvVo$-P^*Y=njuw zuWveW58ABVxqFPDZL(lJm&Z&!aBZGBqwM=O9`o{v{E3CL;yKIAB~gV}>$EFT!VCQ1 zv#RM4Z_$fqLpQz%tfpq&b$jvKn@FGvgF9ZTtHOC|w6?O%oulB#Wp&v4;dbL$uaLpb zUx}yFusdU-)u4pZ!#bF8q~TOj8oxq-&+p@1fr}E7p!{_5^YM65BQjxlMXT-8o{7eB z{~fVB;ar<>_3mDXM@W-O%5556|MJ_frN>kKs{Er`h&B2Ku#u3plG~z8=wGE&e?l$D6^mKM+i(2+>?Zh5W@~sh%{Oq-S2qGtfIv4Dr zpCDR#pIfp`tDKvX8}61s;b?x}ab!G8S{McAoe=xx#+KF`9H70(U{U=iYc6|ezZ5FR zzKXWw@JCgI_RzWSNf=FY#FwT|T@{m-H1s%p4JygmDbAXS z`mODB;uv6_XTuK!cKw?&m5Kzr5A)_*uM_6N+70T9Fg{sFkO1gZn$td+ zeeb??8pKwuE*a(_vQ@L^=nM$Ji}`lQ&(CrBUK~!sw_j26nk%J38vg?#L?PLU7roE6 zB78~op2lWANNMi19aF7Vxh<^50bV>s+!5Ggd1vx1(!(A1v-{zM+MiPO8hMGNuOq#R z{EOVqpls4|lJpI{J`_g^EL&=O);lP?{8JhzuUZ+D=*_OC)fP(eyl4*T16 z(@-AkNb-Be>r^HxWnJ+;iOt>)m6H=*R*Hg+(djhMxz8bB@Cs+^es@-_uF;HH%xu0* zC~r=hXEj~}&CN5+)XdgTS)6Y2wqRR!QX`$rt!9Hs2Q!Vke2so!EPsA;%l4A5G^QQr z$<(8~f{OS=m$lr(xBa7+7iU3x-|3MexnPP(y6L5k$y*b?@Y?N$m&E#R?;vWqxwieV zvBn(yl*N2wYpOB+i4EHxUqvFoWr`c;?OO{WpsLuAN|;>0G5^)O7NQdHM6C;h~)0{>_ui*fB$b72}dXpFMjxX*0ICwd=SGY6z&ZI8F zqOiKvt>RuuEzKuL#-R5_mSNV@*2I>pRqjr9;=wcVi;GedKG+G*N(`be{2(tY+~!Yn&0!Ma+( zzw!RPfHSSly+FsfW4XC*y`!6>KYPpioAuYMYs58PK!J=5$Tlo2X+5%fX{0FZ5}yiF zMv_Z~*w2x$wjfx=k+r|XNH^SOCySR4vxJ*Zs9_cxOoh$8_Rn$VzL1&5l4)j$uo5*bH*aY;~W%altV}<`1U97He9p!TPGyDY^AWi^RvjV?wzips?M^-c_ks9v(upbB**cB-+ySrc+a{; z3^|4t^{YLWMsrg2MSjCQ%1WH`L71eU`O32?#6&V&fp;?P_r!W(??nwMsOl-59MtFt zyBCr?|eEStj*Zl6of8R@c~mx74`BlP5H4mj8GI+=n-#gpwzRf-mxD z69G}EW~#osp|^bcl#jz@f92D(<77|b=(x$04E%k+wn1V2;l#6q`t#i`oO*|w2{ZNN z_A)d{4lFBrM#55crLooP{Z_h^vg^A@%giHk!3Hpehi3&|tzfP%6>W&5aNn=H?=46& zl84Ng$5KZ?qMIJBN6qA}^Ke=PGlF8kn(+8kQ=vG7Gx5tUkHp^j> zk9)5-$fRjDhPG#ekvK%BJWJoLE+21r8W|f5>(oBpb6aMoQlvCJc3CK5IGZ5Th-*I2 z{03Zh>(3CIjs5b5JoK|h^e^7?fv^g_Jdu&-GaeIjH-ey3gdn2sifaOPMRWH~OQ{!6 z2Ych{k}Z_Ed%n06rb|@|0H!{P<-xWXN1vOpH@^J)%f$9p*fUDQo?n_;#@&AQN?uT=OV3qD<+8rX7oEn%WY_nu zg`KRl&h4@3l^^obI^WlPRmJ*ERNAMzjs=Al}JIY5)|t4kw!h zav9ZcQ)I{G%X#Q-gj_u$pb`7s;PcjSyi@I?{D)jTZONOxUxPu*s{4AXd4950ooU?s zvYt=g?y!R=V%7upNVX22w`MT@TQBKFjyY{;&TaAII@!%ud_JKQ-=v5-VBY)P%oKZ4 z;$zLQAJ`eO!XJ(8B>68?dJ)tmO|yPo6Ui(C7aw6!tVf>{w7_(|ltYz{G=>zD1iy)u zSf2j$0tFD67Sc3(=`GvN&yxfnkXbH}aZJEGmAw#cswNRlmw0v{QOC3PuQ!$AQgmp> zxVG+m=t(Vq^^=Ncz(9_X8;wv^q}JxaWlOSgKHM_0SD=@hrFy|VZo6+GMHu#$t7uUz z&}(ji-PEK56y|@;_^Jlo8m114&%u4qx#rtf|M7iAGszmbA(##l5C&4HT9Sxtv_){&VL^c@=xliD2_|SS|@~7yE&rRN1De{*QFgW*QRP(}#lZ zFNc)qnM|BZeY056_REjYl`ADUCtyuihg@#*Im5Lc?Dp&2svGKE#|(@EJHGCS!pFU| zwm(v|rL1bxeU&4c8P-|9de*X=#V95Hapkki>=hNk=N1jF8-4TH{`Jcqb=%WhDE=jB z&JX{ED|pMA#m8W?L?_Pb^tu-xO@sOHEt6b_@pCnhQ?s@&jexskf=S#osCg&)4zqF` z8_6VL8(F=7)C+A@PEzvkFv8Y#%IOW+5}*U+t@5vn+(YCQ;c>ht2cU$|bEI+axZVNP zC?UX7cjpZ+=W0@?Xm5n%Dz{O7ubZ9miyt}9t!ytQPsUC7l91mb=>Pva; zE7#0ncTr8CM#G#*wc+mp*gVKBmHE=KVo33QQ4MGuR@mN}D$nz1-gF!Q%y7gZs`>uy zKzvirujJPTGN17xoO1tLLD-$+4tA;6=2*X)5>t=%@DjAkZwDY^^&WbrW#L`A_F>-{*%m z7i`v*J7v9V1L9%7b2SGD@BoXMh)(SDsbk~G_?pDxx-Cd2ib@ai20YXvbCQHFDjCAC z35#CfvF~QFP<%+xH|};F65e*t+H&aSss3t2WOL7U$})BO6p&y)t*7}9oQF5v#OFw^ z1IYxj+NRpat%s`KbeEF8i=jAbGhbx_%hf7)eak@;4>m7dUeitIJ(thcF2`wE$VrPZ z!gn_~JOxMKIs>Lm(~J3}-LJO%Cz>aC+vd*cmAhQy!AegoYp}=cd-42uDj76BC=Y?c}BN<2P~K`{f9=XrG@&hgST<&`bF+&tMzhLw%gMUH@i<=U%)p(U1w*t@f8-lWyOcB^N4m$lE~^{D2XjbWWj(ahc_n+R=q ztk)Nxy_%lK-gJi!H*mwTI&YQ%TQuvdN?_k2wns-IdIUAzfjMuyN%{lso9%JYq{{#; zYc*~K|Od<|kCmsISGrQx&4^cgeT+b&SggB@6cOPTuC-) zD)B6P)@?obdHa|__ko;7{Aw!W;A#@oF<(a7%0BYOxG4Ril7k~eCq~n+w*%H7|EbaZ zO}fN&yCHSZ_>>L*?v=-Y7d9#7>Fbd6i)Y;{zub|1LDf?W)R>a(zCNe9GVEXNuqoi6 z0)*;fQN4b5lGujqQ_Vj8$APv?Fa-BGk8aim46T8+5p%#ZOdFU{{o5tMGR3ABsL zD{u*KN*5Eg7BDK|5bE{);ofEvBe9QzdV_YLx8?CsqS;|_5AF|MzU05CSq8;midqn* zDc}r0;xyoqlVTCFpArixz=#9ge6PFn)}6{Q5(l_(Yh{f6izW+*WVbCUh%~tFxeWX1 zWk0lNxn%^Jo%-$XV%dx9Lw&7Tu>Al}t?yRlsb*Z2Kd?n{IvS#)k3ST2z1%+rG9w>> zw#rlQ{*a2pk?r^&X%yxnt6I;;PTVK~t8IbkCcSr-HQ! z-C41cY%EGK{ODf*2*+pemWYO!L2|a%E5hnVA{7lTS?l4awp-N7YUWW3rrX)H7zl1i zyEF$OX-R*75|z?1nfDy&KRh1M>~_opHLhaG;=rdvai>R*l>&s@segMR6pC7z?>%VH zei(W?phChgwj{`wZ!E}RYM_s*vwGC_82KWuOxG<={Pkv175&}fcQlV4uoDf12H18! z@bClZhO0cG?Hb|uHsVlGDSNdKlzhGijs-iK1LY1U7Tt2mqT~vO$8(->q}0jLUxO24 z?uZ2)K_J*`K@aOuXo_-rEKx)soYFvQ1Kb%UDYYwf*>%DZj8d{$iS(7 zep^re$I@*n{ec~UwZ_~`yg;c7R_k2&4S-Kv54l&8I^x}C-7sj~Ry5a|>=lB%=Y8{L z0fV9pbVzrDZdZ%Fj!S6ilKkCyTDNx6PQLikv?I1ahMQo2Tl8Y{NAQ~*EiFz?IhD-s z&S5=j0%qzoLLa#9-gWQheja6-{rqQ8W~Q>P%34(PJp*eEPKDN+#1uBI?jv*f7*Z1C zz)~>ZPLDK?0slrQYzkI>X6#UW0(F3wpLqw4dk;OnVEA=1!pUt^mVKvd$VG- z#z9XO%fgwrDrg`zsN7hk$$Af`kS;WhNf3s;ni_k7KDa?VRx{S8$eJ;9t(_z0y?_Wb zw}0^CG>PrOUI7_ZcDjA=53*x_KFrHq+i7#ZZ!f-zg$5yvPBRmDuda!=fHU?0-6?L< z0p;o=a`a`+X(2b^6;rt>cIzpMFWmMvs?(9DUlwvaU`W~N`)A(@(~sLZ2Yu2fS!3I5 z_*yn+%V7`T_AzZ5GAps9WK>BZ5tP&4WSdBgr&UIl;`EU;?%srA0EPs-lfFo@J zn{ukwr*Co0`-86vALkkz9YhnM#c8yo7d@g*jkBt^; z%1Jnmy`|L+t2{iWpU4btRsdAPhcG{QYoJ!^%t^csgK8O1@sC5FC9TRHw%CX!N;Ij7 zb^(me@YFW6Hy(++4xwC^1S&l=aW;R{M2Vttr@r%^L2Ptk*Hnsshw;rdjczK;j_?yLY9#GH0^GkV;#Pisf%QhBugL($MtXBhVe zr~NB+3i<@0>bcViimzV1PFI%`%}uGAr=}x4X^bie2JhKV!!d3jt}n-{n=0eEh~2Bp zZWYk>(9`q;UH17eAeuOl-1srmt0?IcllGk0AwKBkaMFArcha|czNN2%PR+<&m+FhA zO?8(SW?oa)In7UFcdRo5-(Bi>T@p<2Oe}ylQj-{4t)X~7_W`#v)oH2e`_cD|jTZIy zaN^+$H^>Fm+89uC;T)3!ni;*<+}baYJGN3>C)tD>Rj2 zddFPTqzau!$J)VrT$3u7gvjwQk3pB(h)%88OAndmlVnawC;v0#S-R?}0?p+!K;}Mg zj&O*8IM-VH(SM{23mRI6sRqFBe%;rOJ+~g50QB$ z=N>U_fyu-#i*WW7XqO?!&A7*w*Xbqc4plUrZh^LN^VO!_o2IQB?>+lpfBId5Jff}3 z%ItCzuwL^NDeTq4{O6$~iPK9^m6_*E7WEsm=984Y%$=dRHE!>mu8P;hc29isn!?F1 z?1DpEXCs3a48ExRMCQDCi^#{s-UN0e9}_u`59~C zKFb&I&?}+PtBF&DVmCGjK@FGDoL{rIrcJRud11jR-wE-BMq5bXlQ}D?TbQbfaAy3d z9KBhqtdy&9-!Cwqk+u6J6@SL?($gW&%$ti6sU+p)$TK#Y4m8W3d~A%4zpDAzaQ)$N zwT8&1&V}iELg=Y2yum@VQ#~g=WIy5x>b75SsVqF?W?Sdh)qlQPEK*}ejD6)by((%w z=+fO4%lC}(@K1zB1Jws687a3J_TT2fd@^auYgXKI(0O(hn-RkHDtsb<7nCDwPOquXhLUm;@GHd|^%Ju>k|t~g-K z!T1V(G2q=iA$BDJcXFz4)3BC;ag*-Y$~-%)Y9!(yG&zL6=4$JVb|*@KDI}>g{Cw9i z)tObhqkZmrP+Un^rZjbxK@5UUKAlshpzo0)iAlGT-8tkj zJvcKg7e8O>W6Yi|1m7?-i-H;EGm%sQui0OHEjY>$!6<#mIAC$sgxE6~X4I9@8tLymU&%MW@hl61*m2Y!oG zFJ)HlhE^Q=o(6B}q`#fpb%HwQ1)Aw&2{n#0lxkE4?7p)-A9j)0aA4B7rzFAqeb(HD z<5D`(@=j|4(4VTs*6-?F@WAzver5 zTYL-tC`nyvA;i9$v$Tf=pU~^7n;-`o)0-kla&)8@J1gIkY<-X;vbWZ>*!3q`O1=za zEqbj5T>#G{DX}d6K2Cj47fM!8>Cu$5n$zX(Kc|0P+}S9!2h(!d(wY`t7u1=3(k6Uu zK+KiF#J9`$^z#h=Z-h!?`?UF%{FNE|po`SVwKn?(9EuBk(r+DvD&O<>I;{NQq~^xL zQ!$06FJ28U%@Ym7O6UZOE{4OPLz%%y5zV01AHSNNtF5TFuMRYWrXyZRIb`kkM&WT} zP(K`Wo-#`!{29gl`2g0zz#z}lMgREaxTcO|<88Yqy@b!xkE2ms$=~jau9@3!uZnFyo*%*`6q@-xaGmkH~W;ND+i3q^hz!6X+97K zOFyJKIQ6ENENnj5Im+P+wx&@$q=Sr;7Ee!pv&6oL z7R>y3lZYh*%8X{D-u#R`w;aOR&XViebo85snodg^sIqfm$Z@A-|;BJgW$V9 zj@=QNe$H-8KgJlr;o*fog&w2sFArIX7CgS&KE5q-1P3mC^bhtUFoH3KDwv*e2Ce60 z-IiO@!xF-c6M^1E=)zv-F=0Z?VXd^X$N1oSPwH;BxLu+|S`5KN-k@KlP&tAUL8^s2 ztqvra?{i67i}Tkh!j+fzlA6Kw>JiH(>^5VpMp)0_6jgU))1epmXvRorfp;L#awKHqVQ2d&jX@ReDDs!y8L?#!by`@e5) zh5MwU`*sB0zhs*MeZ~G?T0TI4^MT#ZkjWnNTu|7l^52UzM@~;n@8VGI<)}}fota%T zD4KXj-)mTE$RZiE3&zX$sHR&9%z+y3^1WofE+PlvCWu$bkG=niS66-@b12I?f1tux zy^*dn4Ro3qhjo0FXm4Pft;$+IGMrmDq(IPW#B6@4@ai3?J*`Z~!mG}@&(Dk2FTt_G zud4p>t7Z@&x?q=2T#EA8~6n!^dsdpX0G#KDGjH%Pct)dMr{xs%PuzUr6t z*KP*JE_0E&kH%{jCI>1f1#EaE#}9!}==gX_Ef82bDALDEQ=*@i^SVNW1i-%^ zznSV%AN0vBbPJY@gwY>oR18cLYXIoTR0;dEJjQNqMVfwxYu(H?^<}5uH4jmZ-Or1- z3yOO`;}aWf!aNUFtFtF)J*?|itLm{O*)KkrOZ{nt2fFmMevX&O$j+14wSTW1E3y<7 zpA@g`>ni9}J?F)$0iEYW$@AD}UCsIe9iqfQ-;2EC81Dualm?T1_b1l(zI31v#o4@{ z!)^G)1cVrGjzWn87*|(ghCxy63;p$eq^4>T4u~PuK7n4n<@h9@2VLcpjH}k~wdvn! zoPk#uS*Aa$shF}!$pK|cexAxv?U6%^E+CbHj8|Z?2fgnq@ZcN7z2)OF!Dut>)6tJ* z`yAFTTqB>JyD)J24=YcZDEb{|F_^{JepI>qtt9zr9ISLaPE1}-x9Vy(U(i9cJx#y1LQ?~pQC%X#`T^h51?;DA^!X-&*4s?Js18|d7#lo@mJ!HvEhmL8l9 za4fbt@O~UGziBi~%pawlshOG+2P|Tc?>2#!T1_tez}Es0b?^w#n=Om_IFB;9+#p}ZV?qBjF2 z7jFH+SeGwO<>P83IWfLXTzd4o5ohoblKBDX_T=apm{po$IO2T`3sPL19zd^)$D*+Z7^W)dNeHR~jom6Y%a^b; z+q(H?RW`G0SK?)8XQN0o);%j5H09cASM&AYfw;8e?BQG^(>U6Etmj-piaJy7;>{^2Oc5&BqNp)_hzT z`vhJB8&h{;q^{3BuP@f~Y$djh-h`^l#)YhjFI;F`ZYW88ArWTkv%js5{sodHWevww zAIjU2+ews&syJh!5ZTO*dwVOYn=l+;z?@E?6gW5VqL74)u(U)cB5eh9SU7YSYp zk_SYorb}b01ZSA*VhIg3MPyR;8phtJagl;7IAUKRRXz4~jA(neZgLc-gmqL&QkZ z-AkWv!;VWwdjAntkFsyUSz%NclE{rUFwAEbhPPbPq{NgVb0xUPZvqovE{eQR{l8uL zcY|77G4Bi#biqddvseH5t>GXh#a760$*WWuOd%JawtH_4UNzj&K3Ezab!#P8fZDK0 z0ME;-^#xN1C9yl=53cc2_kF(Pk$AlWSFJN?a(a{fjaN$H8kQHUNt&@f?JT);1HGa5 z2i~)%qFQ!*_zkTS^vGVyJt47d!m~RRc)-cmq0=x0Ie4R~=8KojnI?NAz`bd*a5)|y zYKN&{etnoUQZ?h8f(n`y$LgKsEM)FOZv9C{ci|y`j-=@^k)g-#FMdiwa=uE_NRA_N zkqSGF*0>PfW$3zva6IY#PMjvYUna|-xYF|S)zjPan!Y_c%*JAXTA>Xc&OLT&kYy^t zJ5V8yY*5qf>CH;;lJ^<81HK-JMA14~7%qN$QGYpfnVxWw2)aH>PxAZObh)JPikk#_>CvftHVc7gJgSF6cqCs$7RNAL1SnjoplYya_a|CX!aO4_=(E zuuIiGd~vKmgyOHg5B!J(**nw^q#$32L>5CnY3ei=rAdQ*i1`2S0~6U(>2ea2F)`)8 zu91a2CPjV-HpIX2XxQ@t?!28c$~bQ;qKs=vV$W{@y{qDS1uCmgfaH?h)J?Z$P}#2A z{b46ic3Ty3qizq}R5dq-w4lVsel#^;g{^TP{Q5G)JyN=w=*=hkazJ!F8|(>B>772r z{8>|{%otFitNeL%Z{4yb{fbJch|i5&6?+aIbfyX*x=3YhSzNmav;5a(+BWShQBU$i z1s?u}35d1bwtM9^GAtfNA@Z1BHD8tYPJUZMW^t}j|FHjHk4BD0YTWrZSaKr zG!{epXzRv(?a=RdZaU#N=dm|Wq%OKO-+JRV$d6)J%l)nZZY)s_+%mtWxu969kU-%} z)`?lb8%hKBw7$wa?>oCw%S-tThLQ9(2kZ&X-TJNk0>$GpWS_01PU89xzpV_38s6Hy ze5eGoWhIPRM9xQXqw>1KqZ$2+%+@b5nNgI661}R#)`ZG_^5`xqVqy0CGe0?6gp5b1 znqO*@vbymFc8SjqX%3T7!p*s7B5{c=Cs5A#Gfp5W%#gdbD=>UDbL$T%WNJzQgVj^! zIXi<3Y+@Giuy<{ql403%<;%6l&B;|D?LK}**}&*kUF~yWsoa=*Cf z_s{$PeIL1f&~MvO1dyA^{C`dk4Ua&>f(Zzr2VC%MhOqF#@dwbRj`~ms0WWOW=Yb_> zY@QTy(;k!x9HQ;0T9gLSU9iuMc-ds5#PdOboa(<A#E+vTzqhmgxc2LuY+PPfb3uwb2oLFf_ z&NL?=j~kC*Qc|c0`i<9*=q_fyL#10!cgx4N%j)7yicf+sdd;Eju3>%|o0asV(SD3x zR7w&gO&!RbF6YO#L)Ta9Qc2s<{%nDa;cUn|xRx`O6VMbORNHsl#&BISOKBv)+we{S z+HRHn9S5?A#<{od#$6Rj=Ve$2V$*s*f#@1HxQ(b}!+7{$%(7d0doRgZ_eLH#Qg;XC zqiYgS;m%*CJM=kh5 z*jxIeL=cNHd=#1B`5_VROGB70wAr@%44Sx6=uf=c2p`{X$l~9Rjl@7_CTgdnWw`-5 zWHmlrmL!yLzNqK>?b_zgaJBGNZx;e0A08iF)D!x{50UW@e`H&T!0LKOLW?v%Mw8F> zuX6Nt#SQ+SArY5?{Dc_jPI5?5;#T&=(_aO@+vrbA%HFBIiNAR&)iBb6ok$QxPAo&T zNkTD{#`UvAR(V)ix=p72X%q#h!mTb@GGo$5eHjp0e#bm6eFsw&`{*F|aH9G8kpu_> zxR+A*?S$-_Ik0yAz(Cj~x?6)_oj`A;o~F{@F06(g5QVcdg==6!yc_}JU{yT06@zTS zz}P-#IJg*OQmqn~m=jxXd^!1w9;06wWk&kjBI~k?-JLI;^`zzMpj)vuZ-cZNr9OD& z4!TH?3=US;G{4KDA4Bx2S*LF=n=YU-_P2SIMUBpupaRBm))mzji0^j0m(wkl$u(F3 zckKR$H`aePR2gp0yHs!9G)zV$30#hKNuHM4LyPm>E(RKL{}7m5a-BKL3r(1MsMiLQ zLt5|2-Hj0ZbVoHcb`rtMPvwq83^b!wvK=(loMP&du(Q5-h)D zJ1lU%InI;1Tg;97l8l@RDCxPk9y=ZSOIRGC&73*0XCG7DxST>k!=1ci5WkK zIprWpB-)&2z!JGa4f*CC2sw z{in*+aj5H4y7`z}>%M~5VIJDcU-+s|( zkwcR+oi;I0(+NoH-JD4GV|4FEk|s69uIJV?+3__{()sU{ZW0Hdx82cH()y6R1q(z! zPy;R|Xgf3{g7_@YZ8U2Ok9AA(he={fo4*<#8cZw}D2L8yk;XIto5eeCM$*UF^6N-1 z&pN3laxo{WNj5#dTClZ?;c5}5BaHg28H#>Z1SFdz{`7NQGH;Fp(fcw}(>e0Ne>S?Y zBHr!4Oaj<$3vFBw&BZ5{(Ee96CygGhAg!YS8`dC=I1wSg5OSg(wPB=-eSrH&j4)p~ zrw}ej6+%fU&wo}o(YO^Uew|Bzt4lRV6w6M>a^k&CcT(06P|6rT(Q zHTjU}zeY2hhIeP%XzzJ9XFKAiZ}gf9`Xp^qa1bw1U=l`UZM8;d*r%N1xIp3-wZ3zD z=1U95p|9O?U(nq#`d)YpJbRv#^1l!l|w`oDq%Bfg< z?skNCIPz?v+2l^_`=}5n0p<@0j{8)jD^7TPD%{=?{8+IX$hTCohG7#wpy<8+63f)E z4c4uA5vGh^%cCM&4_BXGe`pCO+4r@TrD=WkKNs_}74Y?IHE58=zfC(^atMY>+^krl zIy%{BA$0SL9)n>N43T!xXPXP!WE*$q)_@9*2I`PRduO5YSv*Z4BsQjyPxIy&hDP&+ zQUIpV=yGJpkoD7Z4Tv zxKB73uUJCySQd{4!XHI-eEiGofFJplKwj8K>ZAEznf!%x3tG;_ph}_^_}=%?0ja{^ za(bdE5i~26d7_k9|Ma48v}2I4*ZFu-kB0R*-nM&24Q`p$r|BLrI2`x9f&Big%3skl z_|b3{{4MUi)9TZ|zx##?_*+Fj5R~h*7{~3a3bFZ-llY~w&X*al?-5$M_gWnmr1|e% z0Z>CXBe0?Z)4#O;-bM>%Jdq5ckNwZSflnwr0WO)@(X#;JfLifKqiIk6#VP*J63df6 zl@=W8xBBz<#pSO?t+>f`ni=@CG|5wG)Yq^$29KbY(DC@&W}_C50hO=C7TWyxrm3PH z!8(|x2p-WwfLh;MzuU;DUWLHKUB z>KPWO+`Y43mu%xTct+z>ftde!#%oRRj8+#bhyRK{LQ%`XFGh~q1}aW8EFqQ-dmFOP zvMuYWoFZ3DfBiYs(hs3>9=nwJw|_$i7B;AB*!P zwesJ)6&(j25nVo4SP35C3m%~cO)<=TEeF2I>ygB9`M*C4){7stUUIOCzhA>}5$uZb zBBA363aOF<>(yZY0AGw4yoMn0y?*Auzcp&TFu;0Qg}g}ouV-+ep25LmF9Du$p#h!| z;3Cs2)r~q}E90e(e}5LWUMRyhWhF5#h{_gruc{|4a!$RQ|Le$7f%PT)_f-{@z!NtN ziQ-05gIy})`0D#4rWcR`&8Sywo`DVj?x@!y#F~mz%meVsB1eIQh?+0 z5OtfX3xj8NZ=O^mcb!vuCCZ$0Pc-S&jStdP^X?{@zi`Iin2KcPSB8mCt46#~A%|h0 z!~G`ehk+S8s{mB0ync{`(8o_Qt62OYS@b;w@EzCPX{%U?VZbY^aeZK`{_jm?VFtHf zp3zqP9%O*lw-Y|haD8^lnVI}TRQ9Q$dAJZN02)C-mLS>2$%zeEVHwN!L2TEh-uGuQl&n7 zBSQ~*W%Hi2jI02>S1&aKsg~%VD7Nh`F=P6jhn?YxU?T(b^p>BN2dEkpiU9s*CcLi~-W*F#-w@ z@Crbp^?EP97iJEq9mB*y`(e+zmQB^N=WSA)$h?|mFfzwbF%xQ#)+U(MrinuHN5ELL zPiZ$->v@15SY%a&!nZF^ii774L~u}HRm@q9zWy^oiqp9CtLy+&Z-@I{q}0_PdocP* z4V5-eG)Vl%&MmWobI=gb^~ry1TdkPg@bC-aa5mp=1qr4Qn0eUe2k$q*)DF(GT0cE74<+k(Q!`i$RXRA4+YG)r3mzd) z+Yk3dL7U=LfNPxx3*7Ut2V`5Cs?l+JBxiV~d=7ygRfJ+(?pP+2qP{1qR1?S>f>)x!>k~X}S z#)AxZ-&84uOdljiMh-JjETJntv4eJUa#>Qg!N!w97j?K#m-rgy9Ke_pP^mr&gMXZs z-t7w{BDEJX<3P9|h8om_mtrfy`25?oPl3yA)EKj5j!94MQ;9vdSk)VFDDO(mn6wMv zE`83JytC(#j>wiH620HJ_p(^^OghKuKcxU8cCa(!ABlroK{@R^Tus<#|Lx|e9QzF2 zf-K1(|E(TryTFnvIJOBUo)9k2E5)j}_-}8OxB5_b7#@`+HU8`iG6ul$Fd#d9CIWL> z4$Cf3AbH;x++vGc=YYSIn08u`mq6Kx3V{@84?*p!%;(!5s4oCR5~x{FAdnwfr-jA& zB94>@i;2y9Mn$F%XMpEDs}Yt&3BB~i>!7Lxgoj+~L?gCTj{(t5D+YvNn$fHZ4y~&+ zYG@d0(_SA*tc$`p=W8rx@5|DxU-X-x3zDKXEV*aP?Vk-x{%6DHq~<(r7udle_s`aY zH>xdcs(~~U#wEyrQSJkqEGVO@I zNHLaqTNDEAX-KUqc)>so?_hm-{7*( z8L=F=R$RR><~Eh9kz<*;f4i42Ro^t|hlbXyROr8Zg6TyPEu;`9+QeCiCYn-(Bxjpqq6WjEo{GL4{GD_;Jt;;D8!C zhWVsj07a>@<8Q=)b}9|1R_IBL!MN-qh6w=z+B^w=p;uYq^Q=y-9a^7 zk9UpyG~t~wljKh>toS;3BmgjN6iEVwv`HB|us?>ATpy@E7>O9ipaOomG50dls8hAIWRxXW4 z01}{E&H0*>>8gVOU|+T0B%I7J{vVH{B^;8F2tv92U{omQY(UkHP^%0tC@yn6$y;Fm z+Hzk4ZpW9rKk)1zf(;k}yKtM5TexbOA!`Z60oHZk6b^&oTB#@~c8T*#&buN2;)s$D zE(S*6`Ko!f#l4i^mx*C5+1EW9iVdpp=`ZT~adQoZXPQI0O47$LQ4Yc{zU=wvCsx(B zXY3FeJ&ZKlXRj>coJwh`FB`>V4gf64^{$|6SzK6|?HCDQUu6WmIXIIR`=40#vl;p_ z+2EqE)AtXAa)mz^9an=X3IU6^F<~sDaA(}{JEZI5xfbH4gM$=wfUhkQ$@59Z=>Od+Zc0+Ckb;q-^X ztHJTQ1u*u(uTwB>sE3%iutDEWS!w!eap<$K)Jo-{SaM;c#Exa*mGoFo{-XY==U@+U z`1^yYWWJbjFzIx58cf|X_7(j!9tIN3ah>$b(Qz;^Q!IxFRVOp?_5F0%-LtGsF+h$u zv~cY5@D$&FqZyF7anPsmqD*|h(4pR| znyRu_4QYCav-IoZym2n*!CQT==*b?3H&tcP9r~v+Lp@=;g<$laRt}^%5X`eZz`2n1 zwMSf8EL^Vg1shG?^fq6qTBc)R7kW1xr50|GADb1$0j@>W=pWSWa{$#HXY0rB%`_#R z%mo8jyQ+*NBf(6m3OFYJ%0oFHuKFrRJ%k1_d)8lSKa{wl>{h0!$5hx8%hMw7#sY<# z?K>ir6B-m93ENWmEGl$Is&!ImBLNu|JK7#o z0Kq>w3~*mQ&D+P45?!-2@6UzfOGfFE`JBeq?^KQQBth2yD zt$Ne{=7hbUwDuYJiL(?J+jsr@!J@CbNIOOED=y4O?t8 z#JW1XU+$%Kc)y~IcxLpjfvMI`Xa#9i#*4)p9yj2>V=1bv9p@*T$``W}8u<;f65+>f zB5+#}vT$~#Tq_9<$OJfMhJ}ObNApQGyTp%#ByG63=u0aIM#>5*|BBfxmC)&g!ajDW z7HS4==@d4Y>UBY=Z5d%=E@b9U@E?WTS09n(Xf6=KugN>hiulw^r1WYT>2=~GutJ#}X!s%`)w*q3f$WSu$R z&j<<4p|8WU1EWHJ3T!a=>Q;kXz)Tztm=4gqGJ+)~pz&;!OP5Y5h{=EAI;23E@N4Ud zPBF$39WNESY*N%jMaOrcC%^a5RGUg1Z-e#|9Yh1F@HC@UmKS+?9UHh~wfk$^Z;Ir> zC`Km>{b+&Pa!z7kNpBI>c6W~p_IMe4q}TKV+hKtkDB&EYP?YVHI0>x^AFZD_^sWWg zKhy9_RggJ}$GFixZ@%Y7!?R@=JK$hc+>pU{KG_5jrI#rax=u`Yq%HL47#4mI^EInm zeyC$H?d4^dS$`|gd9i3l++e{J{#K*{EB&01?;ZJxwEuxN`9VH1GO*ZxZhK3)2fp*F z9Jh$s>}B)^F6e^-zE2;A7$-k%%aFKSJCO%;^w^%!%iL9nlC$#V@!O4qvO|EI!^)lj4N+l!W>V zE=e3hfO12gbtK7yqR@skkL81n?UzFq+4oLP=K6g|CrA`SUN8{YIMynD!B)!E8=xbj z`T$~4v|q_;D8MKR?_H0Dpib$X!3yIj^9a(xtIb?-6?(x^`ZW<)WrM-tKxoNp_z7`q z!leguCdCpHjs5M4;1!)pTMLnQ$VrwZ|G?f1LAW9n?RM9B|8*`}U2d7@^kX|Z4ukkN z?w#IMlN_xY-x^oag(17F(J<(6ta`=BBe9WdAJ`l)$+?r}1weTmgwgTc%xbTeeW2KN z4-q@QnY6ajDTrl8CO&eV!379`@M8K>VKgYsbxE$+K7=w7iLEHnAL zQEv=Eh z7*FK18II)}3vQ8#>Zo+)m7GykNGH>?Ov-~~L)y@rBee(b1MF@}%oJuXpMQ21{{FkV zj!s|t2xPeR@x(JiRRdZ$;dPc>+j+17T_U!$))M=ljF*C$mADYAzWZME>Zlt?lVp@807i$H9Oky z!{}Sfs+jc5LQT9I~yA+4pLz-55d9Ir4Pxxjdq^8b{}w ziJGsWuqQ8~j`?GmOGAPzN!NrP=DqGBcvtB&YC(aC5H$#-!0#L(*=7`5T4it}LcW>` zc6}SL4Y&qH2QQNSF9wiI?N5E4{fZ+x6N3R@6 z@AtmVRJSD*_fw}X7uk3HjoSw=pXRE-t&IdbcB_w_Dv~M7KBB9p2=3e&eSV{q%I}=| z0d7w9q{ilergBNDqzBI+O`c5KoN2rML5G8V8>CdFiz3W#t*+~_eSzb3s&wN(8xi=6(#2?KZ68J&hrH=>`|x%v8} zIC>XX-0Eek{oIICW!j`;$!mdcc=Zfv->p6|g(0uZH(3e57_m|3sY11|$sgVDS7?p{ zVI?a!>`1y&H0wO*CgJ;QFz7=_OuuIY5Nh~DGt&8gcCTsy2O(Omc52| z18;pKuTpoMoa{}3%(O@p8DIwgn;JNxAH)>md_kM}p`0xc6L`nPkXR_*VHR!T6$HZg zq{Tv;nBcV2;sLSd*r2j>*kg=Cd9yj8Fvj9QqZsl$-i$wJACUx~#xWE>?Q1`!J1>%K zI9xO(K4=}e@A!t2phw7&>UBs2+WBR)Mh#WC>XlM^ zWt2Y1wyPCLr%g=j6{&q|qxV}Ibg2-ePGcCW#2@9&YR&tJe`!g$(nYxC@Mci2&=jvr zjksXqk4F2GU{CnrviwRkaTZ-ADSDRGSfH356+XmjLqCcoG#e+vKB3k)_av6;?ea;| z;&1<|YRk{RW;7&9cId4oz;xp6$Q~C0NFND{#!1>JRc&PIpQ#_+<=XeVYHk39F83?B z+^wPrbL5r*B@_AT;`RJ}iYP?y_hQtw*+5Wj_}CR-4|hm>v_1tvD<#0dmzf>@xhacMbkKn2-5Q7xI{nV?a+v$_u_z~Np-Tt6f=>|gmX9Dm1y4bT&SP1 ztCOs2O1M${Af~reV=2{-P7Fzpx+m&$P3=cSaUwnp-;g=aZXkfN$J{4*wZS4lG^FBR(%PJqXLb-{A=^#@$&BahpKiRjU{56r9m-U@RIj>yU}CIe@epw z3h3>yOHK5ToGvA!QvIIRZ}W4PMq=5r4UEa*+L4%9+$+RkyWJ&%k3Q%ZRf-kPc(w3u zupj%bTmB3Pe#ROUk*NqJpY=`8{!@32OLp=RDi>*KJcGTHRLDWsp~g^<`ajJ9Js#q_ zcMm?xJNyT|>qaH+Wiz(Mn-O={)p}jq#SP8la6$pjmu)rhFgmP~uK?x1px8eA_O~Io zI@^@=CPSA*bQ_}-zsTZ>?|&g}OczD? z24A|Vv|CxEh>Rt5QOy>g)jnyXaFNR{h!)Rx^ShwO*il#U7#1XtINB$Hrs|RS8r140 zl~Y&jMST-l0K_~)f&{m;$t9yV*H^$jNqOBJx|_%ZZz_GM0uB? zh;REk1I z%~fw)q+N9KyTjR})_vuR|EImLjLNd<+Lja)Bt=3}r37hFx|K${BqgL#z>5ZHNdajP z0ZHjbkOl!s=}zgEu5V88z8|04>;Beyf4u)*ey~_v*EutL&Y3+kdmqO!)54C8&Y3P9 z)=wwK0&eK4is%gD*ImVIqW3^<5(Uxyvad0@P-^fLjlm8<+O^0#cr17!m2Pe0&O!px zIw8+@<%yO|I+*O_gZCzcica#^yA&_$W!ba6eY34Z=_R--S3l^(g>D=rA`!o7HP-S&(Y({rX@+E#UX-_dg2M}%#O5UE zb`e^2iC_s{iC}wYgw4H^Xg8p%x=Wv^i~Cez(v{eJBID%z#6FU5vbSpJVZzm!a>aO` zpDZCQFYA7w5qn?SbM1iVUgja@&RefINK52I9HpF|J)}mK;X#@br#&?ay{zR}p98hryP^1{L#lrJ=HN?L{w`J-h7*xtYg8>Np;c|t{oMCq zk+;Ywpc^0QeDrGe6Fsh(tk2ipBGZiF@xjgq*!0g6Cl=x)9u+Fu&bC>n55cxpB5rB; z1Dbf7y`dX)EfNpuu9{Fct(RX1nNVjd)+^( z#vpn(Eo}Ei_6ja{NLmt-Zj@R7h&Hmrf34Vt*EZO74<`G8=_lTXNJF{3pbi9MXC3J3DWppIGI1M}GPF@uQZeWjui2d*2w^ z==bo=&3OMZTa-T|r+#daXJ$BfBXIe>Y+t$(FZn&UnQ?bXwW!UW$M1i<8s$%y<=+cg zdN|EZX7$x)NNfzT&2q}{T^W=u!0G5F$WP@OFT;-P6V{ZBM}Fa<1r2Cx%LY7og@tEQ zrf-({&@m(^P{UEBExFF4a)g*rDsyVk^q%X%$WtW7nT8(O6w+4!TJpUCqhm~bgEf6W zMZu-dVfq8UyHr}sQE#me!Yi znhml}juFgb94z=~lPj`md6nH-m z(Jmb{uM48EpA0i@T2up6qck{PWXLj#67|aPrjOYYMgZ#F#{QV3P9oMO%D7^gvfikN zsfQm7=g@qjj6ki)B~<8>dQFcsL;VA({S6=fQ9T2c6+h^J+jipFVh?Vhe)cFg!1r}* zgCq}A<1@h(f1EtF9s(z9vp(U`W-1w5?tw6xWrLkkXtFm#$U+{oxp~rE|9*-D~8 zD+J9N7LBF_Gt@Iz)=J^BU!1IU0>C=sG)hJ?5!KuXz8>+}hGGxh#Nkxy@xdlZ^KhnV zyDc4&O0d~s9)QJt9$?%f8Xe}FM8JxwyS>yvfF#4=nWg!_4&sC?T~*>ZlbOgGa826| z(C42{S53;S0K|auGNwjb)$C@8yV;5n=5L(i4Lk48sZHD%#h$gnyH1IW*2%{3M0vff zJ9i<{RxlVbhRfEKi2#Jz2_z*oWI^`&9C&|1P%#z3tsy;bU^vW4az{zvyFDFk!lrO~ zeUChj!=^xT)otUGms#N0%dlV4wR*>6SR_q;>GbXjQatsWkeHU2gI6-OGI6HzMIIgr zW7p}}R?cwRT;+4fWp0moVBy z^jf^Ia&Z5kQ$_bZiAz-fY>$^o= zwv1>OdggBA$JX&6xftXC2vWfYt9Xo?aZF*p0pM6sQ+?TzvZu!J>*A-0O+T^dJMuxP zH?>KY+e^JI!l*bjV-)O<$FeXs%{=yKA_}i`BUL{6{vKTE-IuQL<(A%zx;g+UjjRfD zY1J%GlOSt!KiU}g97ILoU!z~-M>Be?alLCJ=@HRjFin!o2jCJ{IK~$>cTNcPRZsm{ zr->u)_%X+9yq2g^oru=AMUeiaR0$FkeT8Oc7PwuWBwAauTycx1N;`_Z9k6fOR!BEd zGb6#NkvBf@=!6CU<_X3>sFqy4g;e*I*`mfSY6Hrd6d`$MSWJPlzpbCogZ!|oz0AE+ zI9bHgT1l5<=&x5Zp+e(N_^!qyF#h;JT9@r8+on1GYytE?$+!{42RB1I6!u}srs~<9 zL^9suNGa_t2UW)wiu>IzYh8eBU*}!a$#*2xatINZ?9(P3mb2SVu@&=19hI0l?>q_S zH^ddjXtk9)!fsr>G1{J>8E?c0@z-d1aVG77ctqo_chwe@EpLACB?Os0zXAboAye{pcd!@{ z;h7uhZs+0b=jaX(Os^uR4NXy<0K^fNBZti&9+(_kz&1RPJceL3ypW?e;=22g%D46^ zrX-u~mRLPYWgkUZbfDZ*^5T&fOcX7HJ5I&C)gEv!Q9X;`5#mn^p~ zxi+k(R&*ASal2f)rW|>{86jXYQDErQ2MC^|VzZ;ge&_*J(#@OI`T1*_2)r4sR}p4t_se+8~soy!cqLr;Loi(YoJaP zG548SBE zw9shJENXF#<+m>;6+Vx6c)rV}LWqa_EVG%LSBz72Hk+v+z#9&Gff9hA{XhYo`0~3H zn%>3(fGdyZ>9oSI{zL1?F4y;C(Ta#X&zLzRfG zfwMl}hmFUkPn4!gmN7E>(d5u#0(hs3fnu-<21*^4oqv?TV1^b1){x|(ek>N1k?@i- z(XSw%wAVs>Sa7x1WBie~?n8hL&%bhbNhlOxcY>E>(UpQ}t{R}HWkzvS%1LMZM01_= zLjB@YkzLYdBN0&)P&A9D3m`ZQM~?aK^oi&u77)_$8?^GjtapB9gh?swYn=HyW@v*; zAsY>Apsamn%FT5nzE1Emsp9ksNb4yi0rnA0JeICg1~F#CM8G~X!6G%TXzKDbIhg=a zi1`)Rdy39JuuB`R*#z`<7UNF)7{!ict-3{!hjq!gVLL;aO9zVB=)m!4jpu3BSuj{t z;>+iA1S?yc-jo4t#Mj!Vx?^ovOJ=R50D#QN`kbEaNUy&YBrM11S{ob_!D#{(@`0Z% zESYSQWI743Ws9BnV@z}Co^P=B;HzHox0~Gl{{FN`WJeN@xgK#L{`PQzm6xF)?ggmU>qMw&j3({E@-ZQ!tNC z$JJQ<)Ys^2a6(+9^9a#8PlX*NLfelgCTOq1vWp9&!ZU0)w1uDx4GVZ74xa>hkTjH(~g4;x6TzA;1oh zAMPay?v7t;8MUnc@^gm#tnBG>^b)wdhMKz17WbzPTWY0K<~Tbz`FM{E6L|HGuI6d@}eW|RqQEn2y}Jix2`}zVINz7orlf$0eq2&e@2ZPDZy}> z&6FlPo;IQjkQ&Ll_cwxA?vUBQYCd@b#e(KnK)HBJr&gX?`U8+TR5EXGBbde`%j9Yo zn*8X$W1WR~JtCeGQ|n2L5YNW;hN5<6rnpW|_#>vH4K$&L`QC$ZV3ckPDG7!;ag`wacvQlyiUU|g5!QY+ zQlW$?P0p4ao&^b&T#S3dp9co`PHHam$#ryIbtwD!=|@|=1;(`OQb4BQGMVY5okKnh z?A(+F6WxL#Mp2~mr=2(Mg=E-*qS}6@L8k7euTLs%`jJLw4%6>ViSS^Mr3f<&q%5R4 zFK{;q(&FfLnez1PRRj(p!0u&S4!B|+oXIrhSw8$C0w#9VDdS|&TwJfrU zgYAycsUW{ktr?2pC9jZ0mSB6SXdL!$3)o7SlAm+?nPR~;H2RYpaZ|)pYkC)6(Bx0!6=rJ!JB$c&G%9%QxJt+2}kn#HuJc$ z&}`ivTG(wcFR%4-E-AmC#qI=goTk}av|-7$grA93DShe-=iwUbs-Fdz2$=WX4~rik z=FL!^8W)(KCv?|WAX0i80aLm3!B6=Jlep*xY>(j&-uYOzxLwB(h-O5VTl&izgYKve z^Id_A*H8U%RYfiJ4GvYh7jFJGG?-=XWN)cwPI=|hL;A&2N-pXw&KZnL1 zZ6>FwtNH`sjC;>uIpF5Fz}?zgK@XB{(sVd!*P(Yy40>}*Tg4$7d6PpBx?!`-8Qi{S z*6(W*o?`L@`@Gs!shqyvC%n4O;vTpBforW{a1f-Gt6x||@Gct;1b)n18jw1=?Q)Ll zcFzY8UN+@Qf2NnH6+|}k9bnNX*81XmoM0IGs1SPM zV4Q9iQD^3G`LCYXWXC^(Y1B$H_4~ECh$L;*C69cZX>Ms->pMLuSeb{^VUb!D!SB?mD8S zrY%k1#!OT>t;qp^7I{&5xP_35eS4*RK0zUZaXRLQI+g6hH| zS34y+CpZBJ@I}DPj)66^i`Wsx?TpO5_=CLyGl|e7%K{CM6ybt1_y^i35g}coV(1lLxgK>Rw5Lf0#^Nd7T z_h+qQe$f5dr3R(v^pcSQBVL)(iJ}>R&edqG-;u9kvXfNdZ*q<#y7jHh_lCCCO^}2p z?@x}H7^S(!w?EnALwsvCo@MzBU&D!YYRwC`si74B-rNj7Q%J!=!rQ|+ZC0#5&og8& zGuYuS3>m^MT_Qse7ZFsvY|EOe)VBHx9YYcvz`fg>e_o`7Zz3e*W{K@jy;zQmdD+1I zbY7J#+7!{_igBKjr?2Pywu#1ePVKDE11)FQxZ4QriME7_;5;1uzB9q3Wj@=*F z)146!P$DnQXkPvqd+0epM_sYA%`n=M^R-Md&uIGgxLc+>daw2SI&U?sL3-V~Go+Qc zKDU8SdE8W}pM0RR%h>%vRVF~xr0dkqPCx^HLX|HFz<7*nw&~$QL>^zz)XXEj~(xI$(x))q3oc~`Xs-bA7RQ9-b>y9 zOT6;rDi#NfNN}_+@ujZ>n6Af zgT)h+`;Y>bmn=7btNZ>fvhjXZ{lJn*1AIi{H+W12y|GwrN1j=-gHLF7`2piDPT4^# z=rvt}tK32J3#k-H*>adk_VyZe;s;Es?`dq-mJ%cQZTG1Mv)+;&lY( zj1G!LWlT@DCrv<`J5i&zN&nN`n~RG#HWqdTZ4|M5m8gOTsjitEx?(>E$l&cz@8!>Q z6uVbs^$9OS`S>J^(GhIS7oF2zh>viP`jMO?XOaoN+Olai@PcD<6~{4eEChC=Ht)Ox zUUDKESexR97@BT@`Jv;b(v97cH|Nakvh74RK?9ZoV^UP$GWb^SWPGghOt31w5#wb7 zMW-A%2NDSNgp~o15bRL+=%7#2vd1WB!#?DJ_5IFgMf>C^ProlEo_d3Jt0htFZRte3;k; zx$i=dXAWo88)*m{up9&#{9)T?uk|<`Z#QRZEzj_EKT~;TMn(PlrEJ~ueY;l7`-6$j zd`&mYek{AsCvT{BocObl^my;t?(5fk*(wM;!`l;_*5hwp(;%JKI|G>h?|l}-;iX6C z=zSe_(31h)$>qIhngA7^4X&iG;HBu`cOskK=ZFV~>e1EbMYKis^$jD9A#=6^)i6vh{qfTcdv(md1hp?Plqr-<4(6?1b z>g8aoO!E9Qy77Y!Nl9OP>=i}w2L~Oj^44($J=G8PMhYSW56iy>*Hp|YZ6cl^uyjBY zIVGGS;`h;>_y{gt!z$aDR;)sW$u%Fz!#JtVOlpKN$s+3*4Su2%-e(jS83xSWHU%yA zI<(g;GqYi`7JGLMul6BX!^*E&qia+uvAn!3qELosam7P(M*6^JxY^7PW{r7m^i`BD zQuUxuR|;8FtsezL19OAu<9c4|%6J>~q@eqg24#oxjcpXKi^xwF9_=XD9=9{FVmJ{p zWWKHqLN~e0%p*eTIMg4>N5!q#A#b_%)>i=v4GpWMo;9%Ho?Kejm!3){KQX4e!BojG z*DE{e(#ZS8L0ZH$+wsdAr|VtF=%xOFAPwEvKi#ha_+St`mAcsAN2+E$nlT}UUKgt; zYyKVCLp9ik=w}G+6t;w7U}<<(fWofnH7qG+#6;9UJ=_`3Y<3gYZCLOa_NZyRCclDa zJhf8ao){S^`3jpKW#&9l!!-_H=k!=N?=7p&rm$C1xEQgRVN|hZiYd_<37V2`I{`*Y zE%Yb;#{SdNc*>O8-A0k#uZ8r9lrw^>5p35p$+>N3`_`SGsYVQvWYyhTS-^R1kaj1| zx48_hq%f0_j0JU9fyc#X9|cFk(`AHni?A07^SFl~5CwzUoKBTrXs0ddv3gKWzuA9BrC_IV5j9_p4l@^8L zd!Aukr}KaA&>H+$Z6RdKEIMH(m};8BcAgI?Q+D#vIj%33Ga0BveB`$xHFEC@XzV=G zl;6FncxdYKHj2fCL#Rkr`M7kQ=;ueXUg<~V&Ni(~?tKdiUpj39u^bs+td|3A3HgNU zN{4&lVVxTu%T)fw^7k=599nznl>Iu%b8AFQxcx)KxcAtZR9pA4&riF&_!bg0QhiN(tC zB{(SQS`{CFG>eNnw)Boy$gSK_Fty!d%E`94-~D9Qi|6gVIdpzH*-!K({;6AvdvoYM zbh3z2_R*W+mt(SgTK9BU`Mmcq^8F@T7x5`T2;Dk%S7bjQYyEV z>Zwb6)+4)_^O>&ebEBKp@%Pg(9rB%3I_suOgQuCdq})g_*vGmC8-y$7(%;TyWX^yn41r|)+GmY`v}q2LM8`4lQkEf@;orEb| zV15y)8oS4Uh@q@>q*Xf1@cVbZs?Ht&QL)lhJ04}($!JUs7v+8 zpJecc4M$Gd@c37S*>JYX&WnZ2jJgT=sx(FhKPZGeA2)$wXdhS!9BOsuY?_q*M@kcd-FaRw~$W zbVgq5TBY!(mf>t?*m8TMub#);(WHWkG`e>+f@^$rD0oF9JEG z#}JP{P4T8+JKz&ULYV@(qxm*0Pr+Q5ovb#l-aghs>K$-JNW2}80$Q@D{+;2N__n4D zBiqj-cBGvDa+C+mCd} z*V+RzO577!1_!^fu?_t$8f2J(z#9mSCrUuoa1>`!y$7W6Y?_FS@HM)@3648AvbLYE zSJI#45ok74DTs2CLcIibkWXVB4FB#s@WHQ^5)eXf1BB}Gg7j>EksSYe7WxWdR883m z6F?$Z;GVO2e--;5J2Q}*3~HVTu)Qm+hV!^;CHUC&i9jLjD4ZkmqDg;03mT!*@ayqu zZ&vQV+VopFKNx>Cf8?HS?QcEu*H-{Feui}t1fPo{N_c*6_h05;|Mk3*zk?Q>@(FJK zGQTWG&+D&3f3uM=q-Yyx=Yx7=dBhZKYu(_(5WdHe5o`nMf5 zqu3YEj~1mF{Zl#yDjxxqkMIbPz*iWQ|BLo~VW_+N#UM6J{K0(TF@Jw|(4_s<{_kp^ z|1;$N{*HU2_%HJd`PEF}^nR1Th57o%42AH_*ZO6C<;pbzbvWHC4q%Ir`Qm2gZSc(3 z{AIpoQR+Or@&$jH|9a(~vILy+5q_EP*2E0C@jp7)nChZ_>fyEX9oT<=L5Jm6^YvDw z`_(Gqc{pqe&u@iU#&KdQc%2{9M-r$(1L!YkhCqWZ%9 z!%z-*&6m2Uo-GxXO#{y0`0CeoH2VKr4zRsnU7{{0W^^si~t2Dp>ve+^B;o~>cHq4jseXu zWpMT6mabkmxPunlGhsn|>mRZUkRQzxKnwVO%8HAC85e!hV3rPaWY-tLzRuKIEuh=D z9TY>+uEPQ6C4~sEA@q-nb=8ITL-J4RhS3IZ_J~B-J)gi5DanCQHuK-!=s))75uyi- zoU#_Vd5fWQ))88-RV?S&K*9vEmR}w|fmL=v+Kc}AtDg$tnC0(Q1JTGDR%obk&INTr ztW^?_?|H@6!1yDB^L0*zoC?RXye+M0<1qnV~pH z0QX=JH3E*KKtQ*-Z3WUaX@Jr9Y7qeoCIE5UTDPQO?}_HNq^-BkC6lSnvzq@fR}}-< zu>r3<0K*Je+ zgM5*Hw4T3dOd*s^auN2@2^U8e`YAuuS6qZqgFZgQ)Azk(j$)@PApMa)oL+#308RJR zqfO>Qa92+xx)6g#;5-ra)%@U(vAzl?0R>JzNx^#7W=gGCA_CG5CXUu+KT|^?A^y0NMEd|l|Czc zePu?~b`ij0Td>)UAJ#=AofycyPKCD~gra}@cw+Djk@(U;;DcjBYQ=Bfbw^ULL~t+D zpO%6bx$SoeF?vJ5qd6KRruTS+ktcfgZF(sVse@OgXJU2s)?H#>Tau~neylx%& zv8BJK$Z~X!*%glay3H(ssH*{$_fX}SY|*|@_ajFu#;shFl1<(cX=}ar@C?5V7Uw1J z=zUU+R?QPW65I4)^=U`5nd6;_{E2LMc7RQz?TEy25aV373%S8T^R#=adQHXK)YH$z z1M6@G9R!j|Cox&7Hu`+c)=~0g&n&We8_ocsaMwleU1i*vsQ;lY%MXyUJD$Xrne4~E zL`!~u$2CsW%ofS*iDpIB7!n;ea4MA|b{{IL{szLTJpvkJ@cQb{7xb$J+QfXUfB1pl z>;7X>F_=JjAlq#X`~OKbDD1m`jFA6NHSVI);K-U};c^%5N(5)PL4_r{HSztcop5Z- zO9zQ`wcd8iIXElpzqSs)Yw+^FGyrbpf1kK3;SH~Us|Jzy=#Q?4W5QoG)+9k#_$Qpz z@8F!9{@R56@do4s^vVCN0q`2D+6#{U3`;*sx8RJye{J0r;}xX;$@l$ZKonPW>{wX~ zo>Tv~Y7hdQ|J6=7HtIY9V8X)m*ojtn+qFM>>Q@Q>H(CDICyVm~3b NE-E9EBc${C{{c_d_Tc~k literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-throughput.png b/website/www/site/static/images/blog/scaling-streaming-workload/2-skeys-throughput.png new file mode 100644 index 0000000000000000000000000000000000000000..4f722ab79efccf4537da41d55d8f30e2acbebad0 GIT binary patch literal 199363 zcmeFYXIN9wmN1Mc(xj*e2+}Now1`L%NDxp_kRrVWr1#zf1nDY56r>9=Aieh*A|OOW zdM8vV0Rn`UPy=7Q_s-0HXPp1vk2%kC&e_>z?X%Y2YwgwYR!c*LfsTWYf`WqK`LoB` z6ckiS6ciVrG#AM=*Y-5sC@3z09F&!{o+~T!YI(ZZIyl=9HGZz**1O0j75h||ui1b_ z#uKIQe!;cuAiJa`LQWsc1qUT>v-i>?qDZKFf2jO3A=%@?*=sS7+&o$36-urSX zVdCtaBv9v3MJ4R|#HbGDBd zMxbvpBIHAvDI{gTw=^{9_%r7zIiGiV%&bPZ?x3(@eWEVmVwlk0#ooD7^W&ZN7hcUz z?Wx-#xeFo5Q#ZOkFbNy#-S0qD3`({)PxoeC5$h0Q-lrY9naw*IaS;AmpemX@$68hC z>LM|pPZ~z$bv>xj9bL8?~gH}i~&dGtFI+srt{<~MLQmb~%_ zk<2SaTalxAF-9+_LNae99VQ+@1iLg|3GJEN?M%=Gzo@w<0iv~}`_4)g_Wju#JwD%G z-Es5=HZ{Vxx?VzJ+;Ox^%UfvsdnGu7YpWI}!`Z zONXt-^*4-8+(pN)wR}ju_qDx`lTEyum0RK7vlid=>*guUdHsjHKbhVn_I2FN`fMKw zbnWb;{kUVx+_Ze{``zW=Zx7v-e-gI6UqZc`p-TiV6k z*CM?3c&C-{v&CDjug`zyJ--zi9(FzL<898%ZBmRZVG4Y|!<**=&Q-ny3BEoVv(TTB zeeFGAAL{&6%Ae9hjE4H*7V5E-n#{WBW%;Q0b38K-%RMccAC{}4&wvWwp>`=B-fG>8 z=Yx6I@Tplci+^BOy~IEH`(iy{Ggjr>D389~=g>gLl7%MVJS`%8>y3MO|MPkYBdyz~ zcYjuVxw=Lz?D>&sCFU-izZyk72h!paT9B#-A5oc507P+uquQh+x3VGnO{n*2-M13MUL;BM1K7&yE?8jFunhox`D= zKKpNJH`Dl7t*w`bQnzBi;J%s3oJ#&!609=DVqA%SZp= z{?{8uFlR-_52PQL7aw?@TYa?23w{RJ;W&)2%%>}iX4B&vA0!xNf`?m@^F$0#TcXm7)E96a49+4EVO=ZMO?Ttsh=#Zrv=W zU7)!h;Sg~}iONE5{@%Pub25Ym|K@QP<3QTVIedq;C94O&2d&3>55P7{MfkY-4c?E> zj&29WGu?gnoo^{Y^%?G2({q962G4jNn{DM6kAzYnOd3W z`M&Z|K<19|Qt(e~V$&yuLi@t;f^uE{{AapJh3i@mb39d+7!p-$vW|3g@(+h|hB?-* zuPLul>!lTD3z;P5$r{vX*O(Cs*VH5{({ea;T;h!p#*veXHQkQEYt}x(K1s4F!EunB zqss)7weuVgVh1>0ax`$LJ@9of?f1`pK3F|?X>iej)yZ{4K6^sF-G2E2Ypj~dBg|7d zHQc~WMIxdVw1c;y?x{i4KM`QVpcF?hUA36_g0S-k0y?U_c-@>*yk?#UKGDLbV-Wiv*ZdV zLb5IJg5TrSH|K9{a%4sMMpZ;fFw}9>UT0*#&Kb%6_WkJf4DmIIQE{Sl?|n0yptkz4 z2;1LIH}4tQcn&T;1gH0Y#rBU4nyip=qjC|sSB6A$H*(=U#XVPkz57K>1K!I^k@z*# zV~{GHzHgFZ_`W8^=B1OzSiEyr`LAkIs|m-$nV*wFU5*I7@DT|opi`|6!l#4~%vl%< z@6ken-MEMm_RY^o2j_xwrfmX;XQs_wp;}w4Zd-iCKAZos_Ob4U?u`^i zD#jW{aGY%1<+vwt3~>MzqxY)UYb0E^jF7)m-_dXRSR(0O zZ{QXA^YT~-_wLEc(q!U}OKs>=Yb}nm)Tbl$6>r)f04mj)j*|=Ehd$Qb$^k?6o&kQ5TXf)>1oMymO(1 z3BvBo`uVac=M>Lx*838Rx`KQUF-T=Gh3-1>;zY@j=Dyl3^7VJF|7JNI9 zF-&jFSwjG={EG4YpYxMs_$v6fgSB622WVsKUoP0fnm##y5P9bMf-mQGuCwNKcAf`N z!a*jj>s3|$sn7O6kE8GO377bnUdV!sT<+WLwCXB3YOj?J13n9CNZRqnX6Pr5_Stxh zxtnjD1QhurgLt41&m0;xApu8Km*DRcuOm~Tsn=42DU)6y=k)aLM{|q!Ctgm-RZ@G! zdlHw0mU5T5FFVpzMT+XY&wBwM7dY3NEG*XL4kunf{AYWjVLL?-J+wo|FA@UDyr`EKvg6C-d{YE|7Y#pSO( z4w+7%I)B%`<_`gKSe<3NzBA;Hg8b%{jj*p&>K#uzcHcFH&>rU0KCXFe>gj~oZ&Q!i zdJ^_zBL>V`W^z*U$pBLEtDrb*z5B{$d<-h?)MukCTx}@17T{%G>+#T|!Tqr38dTUf z$>|eYl3&@4rb_6wV7=ACx*~zH7Oi>X3I_(8SZj6c4C0@Z#b9XS@Qy({@%LS+JXR+}0<;#{apBeP0( zZOAm=)IfuTubo<~ih7S&JGk3%!1%cfev`JTY@K#$$PlYlXH(WYQyKgUZauq8DmZrI zrcvk*`ruK%`C$oB*k0Y9LwH7de{>r4E2_71u`}+Lg-Tj_b2=lBk9^}o%4)^Wg-LXB z#+T26AaVc7t;KJ52MSt-uIDwS3rNq(No@WiB&8$~tLE+c;PfEB+{6jUPIKR(63*$RAFj!#w=uF!E@nSpRhWYm8^y-p$T$iJXhm2HZca6e$zbQ zAu+$aVRF^gXJ&aC>MMZlEo!3jQ9#v~AWYeni5ylHfgngt$#O`~xAM(Vz$pIC3oQ#;3ELQ&*Y28jf#-|ZtC6z^;dpWCXdQwWjEG!z%kaZpf^OXtWp=p5&N zmY<#zq&WYNdP)k4Xa|Z5|LUVbe*d{XkncZZ{`o#1A4Ne;{&j_X`+uSQkKR;DU(WwW z`2v*OMxm&q{QNojtz+eBW8>;&@8+#trH><5(7Hb}@}i(%z4PZj_gwq-78(AygRY^s zq52D1D>oN03u`w^8!>+u_djqbiJ9!2SE8znxVYp!`F{-fA5H%?<-ZLz{I{VJ52XHm=)X1n_n~@THlE6EF62qQLI1U2{~G-7 zjsF@b5BP)oztQ5Kg#Je@nbIISdBA_J8i?-V!W$j(Lf&wAtf@Y*X*ZFFFG4CO{&Wq`cxsee`j9&)=@$8onk&Jv1 zGqkUGKT#m|XXGv@G2PyC{Fcjp?^Sr~_#EZR8EE0G*dQ=oBk#A#G${)9(X~cS?sz)U z87M6`ls(rWNW=S@g7SYp9tKC;D^eE-*i$_#cnPW(r07 zWQC3!yng{r2FD!X$?(6-o6N>bv$2zxa{lrE1p8;IFs;)6G2K5uVP%K%0s?%>;`RD} zqVorZN6bGZ|B5RGCD`mWZ*@1^l9BoEuxXSUF8&Qq=0{(;>6s&Z4|RT>yZLw6ih^T* zryq@XKxkoj|MmhJGhSikGF9(Eoc5bFLTIh*$oy|~(-r8?R$)rFO>=OLx^7#ZYMa)T z3x^)9Nqtsw(z_M`-R@u`62{Cm5&RXLe_a@6d!JAv+tiNb&tAP#H=Z^ruGqnYZD*m! zF7{i+RlSZ9eezZ6avRkR4=Fgp{{G#E%qcP+S+Un8I-kP0xj2Ta>@?H;W&*qh>fb5a zuhSNamF)hKfT>GQdpFoWi2cjvAQLk5gziPkz1FAi*)KVe)_lzUrfB9@y zc1cIwdUX&JQmOkQf+yspVm0%01{o;9Z{JcTU>yCoVUNV9J^_U6hCeI+ye?Pui9t!> zVD7vy3*Bg$IjYy0=lJPg_h=|9tN$5kzeEMbJzD2Bs{wpRMvo(3iFvXB8=Z0MZ&>vY zQU-Kgr&TgJx}UZE*X_G#@|yR-EH--duT**c===p5CC=B+uKjf|?+3SYVf{gHp1-3> zLCL_(OXEUuQ}2Ia$`&dbrQKktJ=fpa`N1PH`<#2D8~QgIQcxbU^3wF|A0mAI{%!qa zrtZJUt4;NH;QxpC|E>7{vvK}!tN%ab_)nhu|0j=ozxx)vKW(QRb&2*Eip}tATd<4~ zW#t|-_3Fujw= zk-N!0C(E3Kxe3w#E;RzKXwewEP^7ukwe{DG8Vfh2z%M`I6`pM?bX+R(w#TLZz3m+Ixme_n>daI7^?q#( zb5t%>56ivo!k6bN)^I{VxBc0WGt1)p3N>t_l7*S~o7@gNmS$#U%Qwdp(s`?w0IP%4 z?t{+-CJwe2O3YDEsaM}#4+>X2xXca;st3EbQq9{18-cB2CF95ZE&pgG)Ct(kK(Pn} z9YAo}WR85+OGPdO9}{ab{AL;)-#e8dM=vsQ-xyU`%g;WEVKmclAtbNkPQO4;}G znmFVnuFpsF)RMKA92=NMC{`SOBa_nD>3|9kcw?=2?9sX40Nt(ywjX_{G<$;? zQo-~Y9N;{4z!RUHX`9AfOYFl|hjR9~RrNkluly`9Lc4gEM(>C8)n&x5Z^g;9 z#Q}-qo9^R!TnpsIapQC@51sbv1I`0nzUkCP1vUQkaac})X5S+Zf_#TX|-1le+gaYlhGo*N37HbX**~)k#lTo({r*|%v#@jl}g(Wwyj{mp4iCS&EN|H?+VVe15HAd@m=w zKSW_ahs$d|uvQCyeSI)H$Nf2&CTf&pzjw#*3DEZEwCd5`kS6A*aM_K~TQm%p z^v=qJsSs{0hq+vk6~z2_9mZ7-<6PBK==-BIzcI-i8jvnC;5OE@qpA}eAVjt<67Ejv z4wOu)iH;EI(X;}eUq$mk{SC26Cax8+(lavhZXIW<`jc$tK?A5q>Q=JEUXc=xh&9p{ z*~prp6E5!8uH#3WCppNgGb|yvd%+0P5E7wwMBikB%ri~^)QM-;zWYpE)ki*id8xL3 z!GUJhGpqCid z>PXCp_NhE)n++?PAYfa|C#9@KfBQ}^0Xbj>FtUx%6wI6kx;ug8fQg$`j}Yx|(ew!E zpEi(brwKZ=t_K^b6ff9V8E!J=nK{%`?jhRJdO`~H!nLSe>6B~=oSO80E~mS3x8YrC zCpt@8j@I-YeDyYl-4K$YuR(5mOsCDso~ZY_O$O${K0~7T?EwGZXF~vS-3Lkb4NC0B z8oA=#3#e>elNkue8GjBN4Ucu(_niyx$@^TKG;_ZU{21u6*oBNr+@N*y5Gl)zPe_hZLK zhHAlqAe^MZHnYqIfo9`PPb#Pt7A4GH-g+Xnm+iI@nx_)!<@fy?l(_bx0x?z^CG9_i z$$1pmahlMnFk@8(n%57Yn#7u(S;#b7`I*Z+d_f>0SIuuZcR?j4rJQ=$;?%K_H?X_1 z`|CrKLfyX{O-fd=Khdl`tm^l?N$25*%yd@G2HO$+R|$4d#?j*uQ{NpMo(PL;jMD{f zO?C5VD~unGYMnH{u-=l_QeFo-7=>skBofU1pJINZH$MCIT4KhN-p*cIKgWY1&YwXY z+EwDDGhiKTp9fC2h8{YnKCydObO%Nj30!C=W%Q(*epBSAw{ zW_FKQ#M8HIwbGktnoxKg5TYqEn3uOv4aFcnn|bDumuns^S<{sx$Ea7jr2`Kan&5o^ z1?7%C*Dy`gtegkud@1`Wj#goBXU9uf=y32Ds(FbKUt0xgL?J@|L zv9+l?BNq@En6}PMau;Zy{v{;ocy3vz%x9@!!^4!^! zJ)Rb}l2%atNi)8}HZ9c!s!DVI-QitDi(aXaVBn2C8x_Dc%AcN;bZ=&Mv^+uZ=b(%4 z^S}>(Z9;$VGS?%wOFOS*cvBDff?9ryfqJukj#;7VfSLi_4?sCeJOp{Y5BZf5)NTSt zDVbgYf2qz=f8><#hM-YDjpZ@FZFQ9%Sj&BWaMQ@gmy;OZKU0 zq(jz-)LCUL5kf%+JKcbAqXw(eYEy1_Ffp}9diK-xJ>*NEk{`yU?q1gi{<(M?$vDsc z3o?zJ4-nd-3GkK4flGrDJ-R1M8`_;cmy{&C2H#8c+7D?e1Q&n>x`$^1mQ(W_;nvC_ zJ7j%2dgiT1eAS*)PUaX#1oU{L&cYlrJEpCQOimlP^#CfRx8`A>c07a`=7<=z%HNMD zy?W?$#wBW(;ghscHw&%pSE8TjM<|oJtufq+tFq`=PP;}Zf!o<_de3(|0EjM>tws_R z&Q5j>YR)X$qav+_nD3pZX1oNKs}K!<6th^`Dr-%H%Cb~klEo6&5C-#gq4i)*ZZhrCe67 z7nE;*lb8{YjK-aMiZR^?jLr;&2aC7Hp$uKw_A_l;|7Q+pQ)ITNSM7?5u^c zsI8gOqJZ&yv$t!aa^3F}PLW2dqX(WYZx!>`^ycdKS%a14PHHrjeEJgL7`^?V!sPYh zaMr3NHu#O5Lbg1}xKPmW_ZLMXaxuX;i-Vy}_fE-nrSako*((u_q;s5mbU?_p52(XU zY2WzhXu8)v#mPKF3Kb)bc4iAMTV}1Yek$vOedlMdyR`W!L}(e7bs3E%msT0dazqr? zZG_rgq8xlID^^x031`kGFGGj3pXMR0s8BHt({WR>l)z70 zQHuh{5W(V*=&cSPZF4i~8dWvKCIz3a6L#cL5U!#*zdv zvOaEVXx=*`Q83r~q*Z`MsJL;huxs1jP;cBiL7nb22Z(goIe~?oSmg(1$4b$4yEbok zhV5_N^?${(HcAXRXy>7u$rhgACq0fDF!f0ku7u(K*uEAuFojbYquRMJdVchMvI9Y& zo2*jqc88Lf{{*qx0X5cMf!>TM4|2H*6~u1d+9gab(4r$pD|`_lXQy3(Z6T>zhSndS(1Ah?=8MW&geDDgc*x3s!>eWBz@>%M9MyXl4U081^bI97BsTpqC^j1+WHqJD$2mcd}igH|gWAjy1(T z9ZT#JoZURis8DM5RRuzr(;jrS|N72bT4Yo42?8w5LrDmZ(64y4g*?}i!G8Ao>H2l7 zTH{2=w?hLm|M4tNsJI0~A4wnvHsrE?hICL{9E&mrm)vvQ`!3~U>eDAMPv|*2DGXs` zIRmc)pS3$Vooq+8m20B!V7Rwoh)6CNB-?#f2QJY4`zkV*2!hk~c# z-d+t6EGFe8w>kNNq$=&@NLyGG6Kz?Bf7h{#S+|(~X>6G?=5(*Fj*j~sD{|e!elUs& zlo}+Dhdyvtvln0#+ z&1#H}BxLu7NQUZo5sa8tXhH6+Xw)GtIX5ZHE9D zGlypgBogabY?%*1g~TZhW%S*9iSezr-X9p)FtH%5i?3m|&!D1yBAz{`TzZAQsv+pa zqNcVg))hcdzaHAW`2GUoTCX2aF<jd`f6|}aU1u}OtY*AJHvVZ0(aNb0s$570P9MZ0$#z=!h6a-cRvg1 zvE+aMBHa{1Y9{NT(OQrCs^(wH(&)KYl4Y8J%g@oA;O67aCYLnk^W0SDsBgGGn%D@= zN@xCHEv#k8zF#u~!P%H=K%pnw=mY}GXyJj|gLs>Yxb^IP5sfu_{9BlL!Z3E>vc4%e zw2x74{X;Q)x(upMv5vG<8_h1S8kX`%Ek*7DC|FiH$9@8jB9yvXG#)N}vL>nU1V12) zht;t-xy}09Q^I;>v#QtIZUd7bK(XDosqcuFL$sK4oWS^odROAhbNWPi2KIYj){iF5 z#;j@uhe=(UP;Fni4eYn`OnZQH9p?I0z_X(<6dm1W)kchh@C_KknM_8?WlwH8hALK& zhRYy7RpZh-%Jy-6upHU`bl>qZ&4j*3H9s3OfFbsyL{`|OR)ekK!jH!QsguHIBz%Z6 z&~xy(S$6+BfU;gy%A zEl&YuEM{x|Lr)O)qr6-Zj??S7Rd^LH7q=Wx@#v^qy#TN#E*Ohj+Ma~0q&~M8;ipC# z2f6E=vV`}0xgC@*lGWbklhWU>?v}3CEk(b6u4pQG=b}jyT&CWhpsHwp$#iT2yOecL zM1RgoElNu8O;MfTmfqGr+<>1te5nm>>}+Xxuoz>W8kTj#IEO%Nlh)ButZo74_Sqarjzq&nsfdieW@2VXp%WY-D+y@0VBm zw(M?mcM^LEh!I7Xs6lXuuHlA(5I#EoGA>Pr^H)`!tNNvH9j zfQ+OdvJ;fwNAPxm?y!#ZrY10Kq-n@)mI0k-mar|Ek>EVg@RrPrDUg!_X*iP}Gi14t zEz8WHua@1l(61s+CuF!?+7(|1;_n^`)s$WTWfrnJrVE>8WFZ#uYT=H?BO}S)`G*te zK{WtZChp}}1EnBgOa5RkJOQWKS_09rjp=>jTm%bcR{{ll>|>(o)+%mS%uW0~58=G|U3H;znUXSK86 z%J6H_Q|`(;X4-(&dL;J+kVI-0pT|w-`M#mZ0{2&#nyewJ8dMjQuYg*iKm+MAt+*d+ea+QU=i6r*c%Y!?g&V!>5c7ot{<75#Enf zY6F88RHHK_95Oh0(oX2N<1Qe?hjs1ET?Tx; zKwZ~n>sBF+j<8Pu_cgLPWv$vsWzLsw5FUAzN0i$VkML@wyM&m+I6DQBIV#n8U?U9~xX==j4a;d?p;RIhSzqpNDsa+t`V3jv5oI+|xhC6cksXc11<+pp2lX+XT6>z; z^)qh;G7a}CLH0f;5U_=Wytcjx$df82rc5Z96MnCU@E%3R* zS^%3!VGHC^@aLhg!4S<)LUefx;`aH`o=>ns#JxYZfLdv;`N8+bz3$h}FS#0lX1EoO zlix92<}@{~WE*v3Cs`;MWPdlyTIz#Z7cQ1$+sw+nYUv1-{l3F#1E^=|p;oTeR zLksv6ovMZm0vUX|4+q#F)r(34lZ0X!l@>~f=e6rj z(7PWtL(^TZY-Ihy@GWMA>E1W4U z7dbv;R}dJ28Rw10mne`XOr0_Urb@Z9J<+#{Y==!;LqzZ(%B9ndulvD%A{P*%?xQ$b zrEA{y>-E}qxGc>et2s}KzJ~X!BE49&SGvEn4er&yf0B5}v?k7mdQj-6+c?Y_u}1H` z&QSR34yRUUu)~WMe>pN`bn7~j>CXo1o)ob;HhvA)n&vPm{Ti5*8%_B`hq=%UB5U55 zQ?ETBiZ|A@8Wf@s(i;w!$mE$`=89?iHZG@W5nPbs zKS9oxSv(4!3#Fc{R;C|3m(n)t{gP=loKaTr@tRi8b1V)4)i)*MwbyD0rMjGp=l zMz3Axn1!uYSPkKGI@rvct;RhGrMMgIFD))|WbH2|T!@LS-h?fG9_J+FD zc!IVRuLvd6NsPqEahnu;s6g^on<;0^)}LvkJ`IF6?+vQWr;Yk>p#d6<_Xv*{z7&eZ z@|g$)jTNqHl}y$6kkhOLj@4iZCisdqBv-VQ);IIg{4uWw4++CMiGuXHu(qZh`G~gWaWz-3&R}-7;2zL>zquT5u9fi-#{F zleqRJobC;yk4?MLoO%=x0uo+*9rl2D;8dn!Um_Jirv%uZsg~7T>eP4fX69&Di|yl3 zyU`>8bhQr>1#$bFe)UioX_84YG?284=rkLeG*Efh`8H9r83-bN0b-B3LY-CCaE>@z z@_vJ76QH)X3fZ`lvZ4tuc^dDiN5tTXANxUeH#CvQw`ocY-07UnHJLdC)PzQU?UlS~ zbOHU5iLOSXdc%QYL{FXimj(G_1}AOB+qZ{uW_We2!aD(5hx3QV8x z^Xh7jd6Pw84&-U4k5xUwSMbZ5JDQ62-i6*I!B;TEU=*h-J5%;zHvg5u?X{#W@0W&b zec*<-O@lZ)xb%IR9wMtcUYIM)I#4TjEaqzD2#57|lgVJ;E$^DHUGkxYacvUa9>zXx z$ift42R80|HSQNd>h*=h-AnA=_p9Qs66j)Ge$BM;^r;kJ78VTKIOJ+{{n^VVwD}6- z>dcfQzl-+|$$T+~8bNO~sJ1{{VkJjuE^@?wHGLj(>d;o=Iqz4nfdlHROR@I7478%f z@tCa97QIdXc@4Z@s0mjH5Z;!NJ4#_t9BTE+Y25KW(WwNka?>BF1lC%knsIid0pAI2 z*8pE+@9D$1p#US>uoUN}AmTprx~YHnBA=5G6=Kp1dFR?iV_CN z83FH*u5*VcX~X*n>)|e!AAMP%ObpXP8Z?>k(YS<4%oOwLF5K$`Sf+KzfD37rB&95; zTBa_DugWesFVmq0|ExW*&{9s7K%s^gz8F+<5y8=Ze^U(&7aO^VaPZ>_6bYbJ8i`{! zc57VCtq5@89!ijM5YZwCiR}x-xl9xplx0B%72mnG&)TND(9!Asj3pUXSDf9;3G0^} zGnIIVpOE?*61VVHe~l}TaP3HldGITUXiz(k_8EW(7Ulb%3e>ba<^o5 z-RLwqEhApJ-~5Bev1hx*_r!FvRpC55o}_Sg*ym9Q{)3Lj%jV;| zzk~1z36v$Vj6pf~h#0)8lp5kIf6U7d*{O|I4geaxgM62tRMBbAp}(XP_ETG8n}ZXx5$BUWqByYnase)vi zW=RDud&j7QdUYb(wQa*mdL^9XFm!lGlGMP0`TE5aon^s3N&ke@Yeo;Ms}?Z6SA*Gkgw*G; zrmYA%WfDaBVd!Sy+V0mv&v`2q7hQ`>^xZwhr9LdgLI%20$%TN%Pflv0WdC`r#4O%x z4HH?0)?iE+;e0@Rpl?FnfR2}Xyvqx58_pVK)mO~nK`}(Vstb*K;xZI4 zGvQ`8<}`sBkh>C5Sc*HTg^Xmo8)0Qx!DRxbCuaWY_*mxw4>ZmQgF8Z0U$Y;^=70i+ z;L{KdxBl805ydYO7xqAXEeZ>L}+>MRWP499|Hb$Bvl=_5?k}FV?JI#b?-XbENFkNW!ac1xy5* z-Z>_iEF=(b3!j`_Wy;xjeV&myT~`W2|xnf#8Ml;pWfCOCvsg0etA0R#XwdEs3rH%FmXNwX_tCO$Pm--Q^vgAtx1X24 z?{NNrXS*|*^XMn6M&ISN;sA-k!^a(N9~sMVUQglDLZH%Y(Z-dm`GA76RrV99E%%ZD z{G4{tCsX~6!xcbr z z3L2ih-1yY{Wc4?)sXb7{%n4F5V@39NfO`V5QPcha*Yv>6hM}hQtzG|MT;s^YpE}zX zbibmN3mTsv1@=ULrp4g~0z(XtCDe%J1*AR_YG5pWYNCQ(wCafJL-+?xz>#^-1Lsk@ zL=c={SoocwicHG0;ekrc#5K+VIgjS+P8&wsSxvs=5mNmioZ^mI$kR>35KR{6(}ujN zT}E)5E!-yU>*~Kk*Q-ud8>=n{oh{e*kv-zL2U3oEexvOM z$bEb_W^YLSw61s6fT601`)tZK)5Gz2J6-|T^Wjt+kS1SY>NT*Mey@z)^E+9Xn=W65b4 z9#YfQIuj3c_GvbzAUy#tb?T64_1Rxs zoDHI{Yj7030Sjs;i@_inJ2hhZz@KanlV2O%HA@J)rxUcv%{;w*ka=U!ch8kRa${lB^?3i89Bz`S-RJ(w>yF=+a+9;?VW6`d218WxM!8v; z_2FaGDbv=%u%uj!xw-HGA;$?{ko+`ycRDb0b;BA3W&P}SWSxGn*k2VrDNz>3s9**e z*8Yb!nN?R zj-Vl*d=2xcAWFEQPdb`>V1pFj2LQ_Yr6Nd9hHK+0zs`8+qsQke_Om45h{pUzeOe{+5R%Vs=9hUxmeC`(#TTgJ zAFPq_pw*Pal5HvPmHzc^zGiY-RxZ#x!vW597t|f($?d}!bfu^=U0RN2#HVgCKB48( zk(;pSF$_ueo3{r!Pxga&EL&^I0k$prizLbwa)xBWs-bEk`7DJnsBP$Uvy2M)>&=~Q zwKGMJG)FhCCf}Pd-&>`R)SP66t}XyY)p0)6E3$)F1QS<$b2LGXpHz7_2?EoDAA>F^ z=a(koL~Cz>)*!)$OUcS#3SUWT=44yshP|Ll6XKed;>wDmYwS9!H2+G-RQQvWeDU)t zU~K91CEUg-GGq}3KL`iA@1lu&-Ps!neFC?mzifFdNG?~1LKR?~iS#2l$WdlrJNVNz3q7eRB_AjK#Fs6TZZKifSkPnMede{F#n_0m+F(j!8*W&%wFJ z?@q;ZxQ$3fYS0Pm>Ed5A@JAaJs-dxUX3?)--~@-YQyPwKkuLK>=Y=9e>>zK5Kjp0S zp4d07$@m{rZ3r>BpApstk0YZ$p#u#Iw8erh0gldOEw8q>5yP~o((@qCHWCyfr!P&0 zh#zljOC-Cfl_=&58N7kEb>K1Nj4oQNuMksp$nRs0`hk<$cp-N; z`hzUeLM2?w1(y-O@vaF}Bgf6@U=9VYCfms7CMIQe5GM*XX~Kq%-}Gf~+zhrYkAayA zi7i*qg&s5%_nDHkpWlhPL-&k;j^4R@_>il}Ez$gG2hn%2ViFZ1t8J$;+H;DjzlEuE z-Dl|!l=9si*eyLdb;e)18c{3oDST{~BjQM8J$D8o^v7K{c(sHj6%x1F-3(D(>$YgD zi-BYnvgN*mW!OWDTnmJh@ZH)#rE6l}QJ;gguVJiUKvmbW$N(Cpkt%g$mF~$X{`U8g zzO`xoTagXh%BtN%HmZ9x8?z|X;R`GxIWH_n`w7r#so?j52|uu1wZsH>KeD9Jj3ug& zI;X9lY-B$S*t}ebyMOxB6x(NKtjY6qr+J3=L9ojq*_efGSW|DvH{eQ}exVx~oC*XM zUY|Sv({*5-S9dIAohoS5$S!sEH+Yzpz@e{c&Et5p*_WQlS%UFmkcdv?;`|cne(np6$;v$x;8Ae?ULC6ufhYmHJoGC zA)dlW?|1<3&7-uUnman_(is3;92WLg{p9^d_6{*QI`}&PQz5z&<(rm;2@vM?ehm*VXv{kV zE;Zrl5dvQ!w;XvYk{x{I2+Ujx^U?j`y9!n$RT6vkk`*p)&O!sS z>~iho<>H;S1TvZm>2dVT!eHL4Z{}mVI{xycA#z=Mf_cHG_ao<2b*7iJ@;^167>kA< zFpt27IlanyMi!!(60MSZ5X2>9N^1&Pr51~^YFy;B9CfBAt`{e}89DFW9&(+JxKw3L zS+>JAFM-?ww&eixw_PqL=Dy7?7>#C4w~=)g7Rkx=0S?|Fp_1uH(eBf?{et+=xVlYG zUsOU+&1}z2S^KDlgjKl8q+{LZ5UYDEqwWguh~aZ}M`QQkDk*S2S?5}9;YXb52G8@r z<<;xcg!gkb8?aRK&3kJLPsVQwLh_3^n8e*0SC`kyiaggxECQ-9ETswH@du$;LXIPaVuB6>KoOzcm}HrXv|x z_x5}=GtxV$KIgzSrx&h{zj3rR&39KO1DvCXgXkF&1Yy1tT&9Cv)045}>mX;#Hz zE{(x)e#cCV5N>&RpzG55eDgDPTu($0IIC!D=`Ess7Y_tx zsN$Gh6T9;a7P9*w*H3m7QkX9K4Jd~3$+cJuxAk6(4-ZzvkFuH9GaTs%OHS6SnIuqC z*NNgBVR#ids>^NekqLe} zQhZ4+DNd z5Iq5$+EATA97HlpUgnVNC~wXOJE`)B$b0!>l6eQ33{HKc8= zhQat5qf4p!4kEIhm+VQ6h6Lt7bbCzlht@W*EO0r1bvX0V5~$Qcl^KamiFpi%4RGf)Pi}{7`g#4v<*H5|gX=u;-#S#1Ld5mXfkN2g% z-JU89Kjvks)0{}p3>3678pu`zZb)25QE4pC`s1*e1tA!D32{25y;2E_HiW)D0Ir6t zR#YqZ-&*N!^F~*ylme0+AsnaCYa_<0we4#c@nr2+o>M~~FWbmHPTm@j!dds#>vN0- zT30l-`mL?yutj=1i{iCY5iEJ>p+fE02wk%`Sj?ff&m@QZ+j?a^O2Kia6BEP65Kyea&N?ce9!Hmwz{Gzo;TaRSK6%3404nCOQt!RJB&v+3SS&H3Y!bC)>Ov|CKCo*~wsY2+x5X{=U#LiSd)2In6?uNQwyC4c(49h@%IS*zqtTv zifGsA@2c%&zta?f9ZxWxKo#a3;qeX;12mU9dpg~yY2OReZ3G(PJ=B9fSyfe~QYcTJ zC*%c$vNn94g^OY${L^RQP|Tuc^g{n~ z?r{|MXlzeSF`xDVEY+bN!&f4l;R^Dqw=7=)2HX(v%GDjM_Ts@6bxYgGn~mP0Ns|y< z)fr#d;X{pD*qQ9>EFK+ab?Ke_l(NgqaF}3aT!m!jd{<-;DW@S<`7Ek%bf6@sF+C?f zXY-IBj$WoY!p;a+$`7~(J!qt!+e6G7>k`U$^cqnS57NCyLjonwdQ(c&!Zi#(|JdJ)3Yb1X4&=l z-gh0^aKSyJ(V3Ma#hI9U6rdEg9>+}@;OYl*wSzU6C%tl0)TN#Js&p!H>BiY=nkQ3- zpPS7$n6&R1H8ssK$@(sONEAi+IFJoyZM|%LM{!lyCW_7T$$<4)0FhUyOSz?6^2w_& zN*+24pja8Mli@Q#Y;*eRsLmm5@)Fh!i42XbkA0d*CKl0T1nH}CZi&cA77D@}qPOvj zmTP!uzer9Yf{^LPCx@)7m~)Q}Ua<-?7=&g0)Ha~NU>ysh5>$LW)}R;(6C4zA&YRVw z)V*6f=y<~YZbD2EpE^%M&n;!cG1rEP#IF@&XrI5%3&s*K(+?k^P;R7G;H5MCVCjQF_#R5jB)5Mj;s1fd51jZ-0Mn*b#)jojt?U&M>A4h`oG(vY2ueE$Dj^dM2jo1wOzKtk#~( zj1**88FBZ_7$G{dTLiB2VT!!j@XR?J=94A&S!fMsJW-F!h)_|}uFA!j4k>Uv%u#&J zZPJ%+07oPDD9w#HU~Zg_9JpZua?fU9dAZ4Tri*N!e z)(zA{J~ted=sohN*Dg}6{;pC}oHHYBR?sYYxD-CB1{~KQx&+f0shs_;YB+0VA#SCu z`uoA76VuNv)mX=YGCFD0jQm_cZ9;}@?v>TJyLq!Q9wz3dHfxqB_O{!h<))6gm2?9H z6k=`Jz4xj_Yf{x14j&e*4I4w{I+SM$jMxiPWQ96?vn3KR4XXc-juUIWZ7sS#FKw-%{o^zA7 z-x{g67}`e6+TLH?l|@tZAgh^3w9s)M91Tc)2IHu^60fPzI5fPTj&(8}5!`=et>f26 z2H9ivv6Qm9^VD*XjW7nU&)_-2p82ImMzHjJ*Zks0FUzqOXUkVU!D5pH{?^9t#%iX= z#xgsZ=+gZB9OO!a#xWcB9@FhUmEyeXe|qxPshe}OQcF%kbqvT&Z`|C)Og70j!g<$h zj{~v*vPODu^6a7|V<#ff^uOpGtn_D(f90f9kO+wwoSB{L^}}Sj|Bxf^WtCX;Ytu%u z*9k$pOXP7XmihZ7!KwUU^a`QQhKYG3yU$LQ4UZ%8o%U$DDTP_To@`a(OQ{EBD)jR$L9(N$?T_#8 zmug~z+EWh@CDr^pO}F)Kc@>p>Idu#h#!qy4ON>yP5XWm!VE}x`=1O*O^a0t(=}})v zcQMENcSbVD!UG-`ul9G!yLpFMcC;>Aj3WfE9eOU}C5EktM+%nJReY}yswDg`SX~Rp zTTl8xTsE^!JBfMgn$bIpa2M8z!0t!el)0|zNynmAQs?sq;#rS*&8&mm8sn3m)5En3 zg;)v1FR?%j}%nBjf*B5c#tykzI$Lzm+(ra#QxHrCEiULQm3a^iUjcxGo&?t@bJvYN}ddqkofSjB2MDJ~up zVxG@LOLF>^bqOxt?Y=z@cvq7u=l~B7ep0^}YW0*|=M$}3xmH6&XO&^(fO|sn71A=? zQzlKtZ3A1Z@#{{RUkj^6J*~${y=m6Mk!@sm8*6c1JBH9@kFOPp-v~5robEi*?NU@n ztdU|*XVmiQ%M2nu_vPWm+nc-oeH6aM_XAZ)>N($*%5Z8gZdaO-PVyndhX?P#LYcl_ zCDXgBZ=M>BS|6qXd?J^32d*g6Q+)(AB2FzfGmI7D#NfmPerKDR8AQ|3a)PFUJ$g=6 zFt4o;z)J<5er3tl$vd+ttX4D8erFLT9E*HbWORLXDbtA3ue$}rsEuQ_$~ARRJnU%p zDIZz@M}N*n!#3Ulb3xX&zWygMw<>LoStc1P;YoG+kwHf6Pq_`O6@H&tDjXQ2K*dQ8+I@!_X} zI~jYs=ys4u`S52aCkv6ut)ZVN`W_&!jTdRP>Gz*s0Obu&(qj2p=F3n~IK0Z_hz$kv zlvhj<2q7!2*>@&yXfapFPZ)@!`3ODBMufI1YW%C_iMCu`^4~BK6|u-OyqmMH9qwh= zv_|!{(<^iHUz?Gh&g-M9Yc1qTEU5q_SyS||H_%cFi{z@8N=4oU_^5nJf zXU>KP_}V<6;U9A4(M5v7XjsHUcOh%f)3uS(iVz%?0U?in3{>TsM-ONs6B{i6)IsHz zHPa(3z_r;{a;u`j(WoMtyO!;P;+a5AKq$ti7I{b?i(3>=;S{>b3(D|};*ZPIr*H{y zQe9{C9Zo6X3PN)W>2TZeevN2z)0TK$FVmz|?JTMaoze1=o{1YyARIhe`mm2T0%C<9 zM6E_5^b3k%W*||%Owx@CL8QYqPauL}5h-w#Tk@z{JpDHE_>-8_@1#xoDgY5u`WNIf z+G|boqT$)3JZwNEbD4y}n&UxgKTX9?W+&KN;k%-B3zo07f8g8Q|C?b$SR3G!a@;L@ z`hP4F!i?xEPNiF1gsO}wv|C{(frTsU%)Bjfd-?j07s{Zf$=zn|s zf5z<}JLdmw)s?~b*&K4=$B1*i@WptoktN_ZL%De)4{&z?Ks3$kj760nySdg2=T51v zy@uS5HcW*8YLXp$SdW9Dk_=}ob6-y6;ri%kcCr=hKw7L9Gr%ve2dJDFBU~FTS*cRJ z@};PZ?Ip96_zy;=o3FXjv*FLsCdG(!&9>|JBf=l>jhr29SvFo@T3Z${bJdvUq*Ui_ z-J=p8yNz?%nd&kxc@ET4TkznpcvrNpRZo<7t5l|X-GAm)FK|#2)B-DM+zn)I7@Myi z2oWk{t{LmXf6;0;Cq^`>qNYpYHD%VxY|%UI9Lied`tZq4B$vVWccmBg_E%P&EENte zSLanDhfclL@!EC@Cp_B5$xN9k zMNBYLIg-Nu#c<0vHre+!zD6T+cngGfDTuTk6q3z*cwTlkTwl3s87{_d3@OQ$lL@#% zNo788_utv74uT(FomY>&37xoryNv^sE*6=MR})trJHFLD6Xpb*D=~w2#(}Gx`tYUX z-m_!fjyvwwJ&{2;HC3K)QO`?tYu?Ha)(F?l!1ZET9ci_bFQt0b)7ZD8#8xQ|J~gHo z*>?dnJ`IyiJ8p&c(m3{NI6S$=zJUWZG5dIT*5qzy)hn>3AC2h3VpJ&sY z;%hD=w5-eXa#DF&tWG8T`r?EF9{$U0eT12jC$}pr;MZ4ZPG@;JV;x=vwH<1i5|gR! zoBBZgL!YeRuFLspuNQF~8r#k@vAiB z)GASIfiAD3L6yx&((Cxs5XsOq=qC&!VviYsytIvU}QT zW{@LwIeb)Ce{JzWXAVG|nQUDfZpzeWf8}MQ{!mmJwUFp)|Iwtypv*<}p}@l9_61o! zoJ#8kQX=F@zHL|?9Y|%n`-Zf0MBTIMBQH=gy6yd>t9r&(I9Hm|23CKLaIaqQ@fCxM z4R1|uf9@0n*d~*0TOU}?1rU`Vj%Z98B^t&ZWY>Djf|Jgbp5OkBUVoF5CQ+tqbq9Ix z1y9>-4s!uCYme)=H+HBtgK5*s_j~y!fYrc2`eL z#A({0*X>;FH=COmS2P@^dVAF!%wvX*4C827R(Osr!=0zeAm8po*SHS7bsU0kI6$pn zyS&M3*VgT$Trg7apcJQFd8bYr#s;fex#Ne-&%{k?DGqw-w#t;`$PLiQU3`if5xhY3 zk)?6ZjJ)E2xNiMO%L0g{)x_&~jbrnl$R|J1^E=eG5lNmEMe&$7f-PQGD#>AYMupp7 zK`A(vY2PmW8yg~r#OxV~b-8^n-;7t~#h%`LI&j;i^XhPiDw{0+^6coaJ?QE|>-rFi zWLug|6lv*90}bu{v+nIt41EX02GY{o_f(dH-8C3+_DG(!w_I3ixhFH%m%YWcYkN=p z-S0vo`IhLEth=OH;8Q(<=JF;=QzucK#igArjm=lCUtFC!EuF^zTdbx6hJ9N~rm49o zD1M*jZDrkf88L0UtUXe*?dNoq=s$6=HpHC^ck4E8BX@TggS!QpStx0kJfe>^hF_wc z+FF$FidWAFx(68+?Fjkxsd7tK-UICIe{!g%(<2+Fcov)Pr91RnF2$eU-2e{Ba_Sys zLDBKLwppZ}Kj!q}P7Yvh4gHUkgHn@lgj~`DuA4)vXOPgw14#<4(Y6|`omBYF#uhf` zB(5x%Q$6FDYc|;0T~gb~JU=m)e{a@scsw6AA$ow?UgjqRP?imnNr+kk57V zAwuYI-DoMT6?733Jh_n<+ZZw;|KdAM>YJwQd$rZFl9+V_m^tTq!$-AATi3$~5H9=7 zPDeb;<0W3Fp-)Wv#NuC>shx}^N|5fVoo`x?yam{JV4Pmn<3?3l*03+B?pLUj*G>MG z{)eL#{@7!z8nl8uJF_=cr@aB_NgN7F^_)6D#Ryc#vD;B^;1j#7(6H)7dnEJLc~mG% z={G{{N>LG$+NTiOGww9+iQ`-Dm*LNtNVm+!0pXjkqQ`J34CLitl>>B9-gNlQdQ1&t zAFGpU9F-b5b>PNYSzsvF?#p{nu6yzM|FxLuTA0AaA%x9Bw^NVnH64$1ofZ0Z%f$M4 zm>1(oilAv9bIr7sajIkWX4Udmbyj3uB8%;hk3k zOfz1~q;+ky(n;;cuX};}0g+0045qkLGDek~M|=Tf$!glRreh6MHLXBkPUZb$G1%B` zgm2y5QqtQ{n?330ro>CG1`DJpPzC!?Go^>z(xh{#2mq)^jwR-y=3;{LK z1-mO?%Y8_rNcuM>;64{$OwT|kAjdRWd>c@kYO@M|$7f?>*@+ki%j|Hwi7sy;})jcs<8B&rnteLh`>c9U%_jGfgmia+H=& zu@|tZwa*SN>K9wrErF|jchAq>4COwljcR2{p&h>MJoVYOx}ZiHZpI2ffoM6r7i&Iq zy(SkiLG*$*|C40%MoUdTH{IaG2;L>`!Ohj5F7lGKaW8)p-s`2uFF(y&{4q6=-1EW~ z^kn#4NvtO1`jpkX$~e_^q(^9bqfb<^Q%>MA?8*KG!hKUu>x4U{?CT`L`MAj;xv2VX zC=mG{owmtLIgZ3EyjGMB1Rw25vz7-CY0?$WyrnqEiq}QoOa%5pjMvC zwp!!1rNA6;-Dt)nwFXtQxhain0>qrDn%V?aWPYq~*bSXpUY(0!Z0%71k8 za^1Qz55N|}{Y*PT7d!r&{5H?d`)c}`4+;wF?Ghxf#;>oQy=1HU-i$f(1BY5b^slj>7WRo(6@ zyA?{y<|azfpb5&LtKl0q?vc$oy}5zQ_;NhYsXiW~Gy$f;Ybn$W&4$smqbfBWPk=zG zG;_1E3c!`0WjsDiSRbn4j0F(se6OB*Q}GFHy(RKuGD7gkrEV#|)6!SabAD~k zcBO86+_o*vgg4be;k(<3E)LdVR{ZiQkBCWXFSg~sY9!3ZNZfn$cH>k1RsgrT$*b(Xt*-DAV7R=wvnqh#EZehNBE`fK(tBi4kxn2;hf8R*|+vESehX7mmKjZd4 ztNu^FmHzMUEM3Ppe&)VK=do@%!fcG}=Wl%T0pC^d<_*n1x16syCJ-^9ylYv>boles z9|Jg0=B#G6|9C7Lcm)erN&@N60P-iGDW&`T|KTJP?2y%FJlw~|tCOv`7o9p%f4XN_ z5x_fNBXIbT`yVm+zFOoUGVgrmQEzlA;OfwR69RD9zFUuk!4uj&9Bsk>K%GBc0=YS3 zTIkYE@7k{bA03+Z8vBOT{Y}9`|X+q=c zSpDFBl6x;FQI%qaDdWw+ghMgF(CU@o%<+E)hzo#uSD!ujbb0fPIXduVn0SYwEPVTP+%Se*F{@$vqag-Doh%D;#m>W@r@7 zKnApykA~x)uw>=z=4CtA5AC5hog)CD?>mW`;QS#+M~qjV`3+Do_AUgQG7t*xgatIycMhOD7B_rg8a0xak3?BjZM|71mev)V_Yf;lcl|Q-PAf z!!OB!4#;-p&5aBQ=+FCr^uQNdisV1?!oWKcR?(vdGH69i zj8pC%X%z|Q0k=f3)U;!ZZkLRQ}eQ&a}i`sSMxzR=CI^u+jDF5i;zp0WH}o;-k~?WfplUl>_?QcfF73toe?qqlYVa99Gkp zDQ+PHAH<_ultWqu3alWXTfaXp!9HJaRpCY8M%5@;4BwAp14G~(pj<936bDKU-h2#Z z&06tcG@K<1y>BhQ+DQcafRoTErtbVbhM$A+enf$cJ87Ve+dHZbTC`l;li4X(qTXpE zNB_=m^pqTuD-;2?Kn8x|J4=0wl7qXA0DklB?+*xqd=}-*XDz?AH&NTU`n_m5_xmvJ zPB~^X512`NKp?JGZU=-F_g+j$_!f$AQzN=C=x}@2*YVeo{`&VdE|`_KVb%hDcSo7g30%g#HqY*hyt3>+Nii64j`o(>yhPaUtf>hxFQ9*GU@#wTA#?k)Gq0i`x- zDeI1Fv9>cEF)T8Ro$;AKB1^Jy(Z*w;6>FY%`*S9;SRCuN4eL8O3h14^l`l@e=X}fe z9pyU>KWe11((UJt>g%UQ*`J}@84xsgZ;Etxu8N)ZWKc#z!POH;DG~}KvMH-G-n4(7 z1_&UouMVvaDi^9h&DYFggZGuN#6mG}E&<%wLn zVK1IQ*Dv2JJXUd%Q^Zv%R!-cCs12Q{vfQMo_IhH@#r&%Bxxh?=w$|EUfdmZpqnUl$ zeE?tz{V1CQKV}&&RL~}U=2e)dXD*qxBBUtOb? z1~5D}Pj(pXi~EoZgRpjxpr?N>&=)yjqKFU7!9*BJ`se+r&BiO2GScs!NT$EuRzo>B zWg<(51(~#cCiF674Fi-MWQClUm&frVR*ilxXj8fDjaa_Du_dqT9<}G;As2&sDFt)V zHRUvL7Zalycn4RWdnCePr;jpUIqy+y*xPhcKBs}x#;cE%{%HLRGGz8XoL0@$0NH;p z-N)zXAtCwV?SaphckyU*gB7fD6$I6aU**@cXvWd$e2DX`{9d1<=5q4=-rCUVgZ(HL zXv5(V3WI7T-QimQbViSp<)plYMc8SV_m#b@Yg+NEsw5T?$Yye;gd!Ym>)CeamveTb z0nL^p?;;(MTKna=PNc>lF!7G)TkJpg9tt0lr(``yqUWp<7!6n;pd;pXjny$npve z$r{*YYgqZYcqqaY1t@}723Rsl`~k#57hpKA1zo_U+J{$r{cLXDt!SUzP0G1H|H#Z5 zTBzL`$`f@THjt9SaC9%u*~3oZGtRCJyO`W=$cYca&OG0O2O-?-D_4A>^5BloS1x{eezfhZ8Iks$SN8#1LK6Q` zu1+|wPU9S=n|_zN+rj(RgEdCcf$8lX2cmohIBXN{5O8}18QOP?m?;6{#_;_00lT5R z0?dN}x4wtexg5bKKXT42Pa$&71t@PQJ+a&K{)G>Z(|b)5YUbPiLM)-mx){SaheM`f+6GX(E{q^7Gq5rvc`~=j24Nv=RJ-x4~2Rx z{BgYaT;9R>2dBSi#cW>v&^2G(nQKu_7e>mLr;JJxKtM5wSh^xny6#YNybi<&^!vl` zLLVV%ux*5><*+~>hlkmGf0&j?Yf4uxmS!L5!vZb7kBG)ljtn@q>$!`niR$}~wn=)L zMCgp0)9ira8xDCKB@Q8{##EU3aMT$|ndsr>P~PAWY+6aV$c*6Z^%;lbdVgE8QeG6z zYRfqJirBF2Ikwy&y8h`UtSXI8K>;eSlPzdJ@(^Y-hg_t7^?o&_183LnqDMO|D`%oc zNqX0{9lpi0#w4E0PU+A+(Q9(?Rg%M{PuL8xVT(qx%&czkl}Y=hxsZm~>E2={DWCH| z`WMmFj#x(JVhat}8=Hk4r1~G;%_n7RNusD;9>|nKk!WbyzJE@i3u!k>AEOwQXuiyn z15t{?uf)8}%+(m>dW%Oqq4b^AopnFy+vSNWR&01dbVQ5$II9GvxUaAc9=PQDR1A0XRVUeZ2d&B0{Y^-hmUwqL%`Or``fCTkPvp{g zL~Si67`MCLmo&yDxPm*}2d0!DevrP#-y99p`{?ix8Q>V8nID8|C zd4NTJgdtE5h zQZ`oFU}UgRzmG4gLMUz0U^>ycR_MT61?#?S)@Or4IkFimNt|6SshQ$m2T7fk<OP(KRpoWsmtkhE`K%?jt7GN!>bQx^V|++%eCtwJ#+Cl< zzfr*rJn${VbyyssakdyK1V9<{H>}H^DeOO;Ha&!4wKj%Z4KFC_ z)1e@1P*d3R<#&({l|TelFflU{ax_pv+Jq^-qE7*g)EbXVEl#D}tS8RbP8&Sk`lADF z+NHl(Pkk1Np7D%S^}YyR9p=#+FnGh-VQUS~!==z5@nkDzP7(4>RR>LfUX>>1Ig<%` zF74*p&}O&+?F+|rD%c?NG7A*+-oTaCycd%a4s4!}rmMKe`_K|El%~2X;ImjEa z)o{*IaVjky)(^boVjzxL%jDWuHwMJ-HKk<~cV@__I^aM>k8d0rpOCjf+0mY8HX5Qj zZ}qZEyW@Pgki!R?0_tX33jBl4l#lv<67mI8q=&{cC-XmNPUdjR8*!0&9tXoWTr8 z`_|)kq?8VeYmufQlrH5>&m24Wu<@z4mYHj&Am)NPk`+rHFZRR75=$D77z7Ny;#XPK zb6mw~h1{2nkE7onAVR~+rcHX58Gi@6l#p}Y68mXft96bT=91z#o8d!#hz3ae^9fZf z$7)LO0{v1kuhFEvQFpo&0V9kCa0B-`b`{PK$zQlfMLqng-QYAR<4TH4XZ!KfF5csQ zoWSl;toI2u9kC3Gw_8TI^-cL{O)-F%VVb0e;cuY)pF{xQfR3!8fy??jyY+}YpikRX zR;dx1_i1$WX)%vum^tOQGBn?ZloaSqE`o?g_we#LkK`{L4hJrdVXjG3Eq>x66x8V7 z2Tqo{lZYk#bzZimFAQq0zAbxsHIr79;dreh7oGEx=IEiN@z?30nox1^)3zi*5x{vW zJmsfZKNGsLB@g$suhdZ{tq5AYx{bENPNktJLUp_DD`+ummkDaeFDsk-RWgh_@!sSY z>2ac-Yk7g~o>Q2%Xwj$@fMi$gFKIHmCBaE_d6~4h#xLIMS~%t8L|Zj(~)ib>`-z z_owX!x%AF>R;9ySIx5vw{KeY*u>E}@Fyzw1=QY`HZ0{p%a_`+q@RYe4E;Tx-fPVce z#uEjlkRoD#?hIs<^M|zQnEU*Vp-ggkLR1UJdZ|k%D^0h+AOYmYi!(-neG8lrCf(Gf z-jvN-KHs{Hm^JCGC5}Kc6Ui?H+>f))4#}QGgW7NFT26|$=!C01n07z*h$zm^CiZ(n z(viX!Ca6=@OnGNHt#$7-u9fW!ShoJK;~6Ct^#J*X@0O%OUVg2O0;+}u%U@-B1qW4P zk7K}RtqEn7UM)mJGcTOJrlyz+Y31+>sWIzH0Y%~kX6r)BJ$XN*9@cTZ!W6Yx9iVY5 zRwL(iSb5^feSSONMdG!~o+wDAB}C{q<^aIQyL}YGqKmID?=KXqeMkYA)Y<&pg!=uT zzjEYURqToOLH#1t!~k7-@ILN!=Gg!#7~=2P*Ut*x(MJYALT#$2@`bd96U&m+hu$D^ zv*)`lL3y1UiLM(bwluHHk7=#zl|8;QmJB%0c*+gt$f%G-d3auYbvEgqkXt=%faew6 z%TeOeI6`!SUji35b66(PGsu~>WCsyxA`0SM?u2^Fo+e+BBEysy1t_6gW3|%bo{Jp> znoq1FI_~sd&T{8$U)$#=6pWRknTHaJ&*vaT1gGhc^CUG)U3gHfwQH1-Ucz2GI)6Oy4h@nvXtDNl zj<)46KV}Aa!;=Ts1QSJA7GT zR|Lz3b1e%J^1-;@=!;dP86MmdSr@#n(PTl)>o#rY|Olnu;9Ja9xV}Dg9qUeH%&ewRi1xb(3Z~!F;T8E!@sA{Tp3!?M9$~* z!R1tt_Kn-zX@l7Ve)lFWZ?*`SvwJHg<^^zK%hNh0a~N_7jPDx_4_8V2(J1%g$$Uaj zHXnm>`ndSUz!UvdFA57VDZ@8(kb%DsOGgQS#IPe?;$c0~O%uyhwaiX&kD_N3LNREe=Kl$A4~pix zd7P$P<_%}dV_DVW#Otp#VKg&oB{Ra?FTLWgxKLyfH^eZp8dtFytT}kMT~{wVMRl`) zON#~N50@+4lsSp@zhW_~R_4N5y*Hrp4Tu_D?}@{GtF@z-L~9eLrVXFCoT2ir;zz(y zLKQuULHX?`Crhnlvclwssh3pZa1j8+WU^&ROWRgmAJ+JDoP|IAdfESsP$Q_3S9v%` z_jp+!lySw+TZ)Bpb9(;up`Zq>=-6ox_!1tayfCcJ;@+9>+TFI+^n*%bE89dxqd0t- zVSpCP@>&H3`YhYYM&M4|@$I8-LZmf5@SEArky9|J-BZqY$OEWg-|3Zf04IkPh!n0UeYTckF$j!{}4GBc=GDOh%|NL!s$n*I_GCcx%psk;U-ZO zgW`%=p;9wS-ikM{{L`ZmxQVDbJt%{0KFsXbR5yTNLDjNf!Jq9@%;F)RDb8)YjRB6L z3Jzx61WeV@7=hb4O6LQ(>@SvFw0(&AJWA!x0>y=!q=eIgz>p#b8|E+40*%_H_1@>Rd9UstR{5mLn7P$qa&;3%_8&a8*la7nuu64Yc%x}BY?~fOx#Hn($V^ai2^Gq(VF#B!!QiCfA7VLijn;17T8jy>gW`@Vvq8v7Gb$waT@%^QHE7a80T5_N>g?j*TmYd= zYrjS4dka9LwmS`bn?9=>6_22r@n3ZT{ih`>lHF%q{+Iw?Cx!O?bj;$*~KaIC=E6}x37n<`g0pCoPA{Tw- z%|m_^PrjiuHa%xC6HiGG#P?PQ3(tFo7coYQtBMAcu*CzU5V=={#k+eaKXT8zx^~A+ zM{A&cnx*&p0{V1Dqg5=l2t{1+C=dRIfmo2-W}m-61Q`&l9P)pnK=9jtpb#V7d;HbP z8I{Ks+x-|FrJ5CA`#G4HWzQ zg@xW>7N3dUOt7+%RtTHR)IMjfQ@hQC?L3F$)za-n!?s<~nh%ABMiD_orN?cA%wu@K zRCzn+RVa@@!vXTCgKj+3wiU*~f0;xLDqt^|Pwn9UF-cUUdk+tKni!RH=HU8z+Yu*) zZK&ZMo`eLsWYR(c&(f|==F-^?_Med`cE7dJjQLEf{XxF{VJ~XMK(*N|Z()j16bGbK zf%p4iCIx1iueNlm;(zGymuNLU71g$1o|ea%#rdl<1IR6i$zvZIlHYsWFV=_UWvSYH zk3f5V0hqjAXr9+yEk0-6R)nX{X_pIb3K09%RxR-7+ircFQ&IB)boFBx`QCV(gDu4S z{?6M(9>D@cE`rvd5PGZky{3Z7BFF6U*#~k&IT?ZWQ^`Y9G}B=PMVDapT7>8Vi2nt-Xrk*qkW6 zZ93D15upfDN#~x8!ieN9oxCHr4}CLyHSFwdRC-a(3!~l;Nzf&Gebj@&II%0qb|MPs z+2ID%_NGVhXF#v-Y>j=f`)w+|6CxGd&rYjb%SvFr4LQ+Esbb6kZNz`T6f zuo7eYWt@FfH#B81@tg-23ly+zAnP2bGTYGuzV3RC$KvE zeJdaA4%H0k!_vz#>F?lNYhc`#I2*A+{Y&`&?JB_sC>wDSQ~-%>p!S%(c_O-ilk~h8 z3jA~yb*=H}9a|DyBr^UoS=^df)LXYGAKC}(@otMLAp~L^A99KmR@J;zV&hI@1w*Kn;7RopG7tw-dtPm4l;<5wB@TF`CI{vsz zGtK7vH_HKX0et;!FHrv<$4i?upNjjjFffL1muGRYAmt+iBgjgsQ&l%Z0qb-|1P{v! zw&>l(qWPrlyu%E?HJ5cLzZ8lJZEKnki(no$jVv8z1}Ck|$FcmUYcRnPFgzgXdP9Tz zhvml3mn(f`E~YX(h%H4 zA4H%uJ!yXaOH+jayD`*z2W9E^5PVoc20yTJXZ%5Pr~ThwK;uT(LNGfF&EoIoltqr_ z^G0}&fuXr1meOmb&5ay_sziWuE&;>L8n9$VMPVf#z6Lbe`o@=Izj+w|V|uOEu60 z7w@zZW5T*a*~-b*Kp*@5#H5B=he(8n#U!WhV@yV};AvR1_(gg|N#y5b;wqP^jiwOo zg5f=vU_9QV1!sC|Um?2p#Y$~ibzOF_StHtxBvjj3Hs7>}SE`2FOv$~A#)YVc5j%&k z!N`Zb4-3Fa1sA>f=S&l#psF5}55Iu3OVTZyNu$X@Q8{M|52jz9mVG? z1xEtPhD%u7J(4<3}dTF}_0ER>(&-iBDz1M0r04r%82` zw6=6CVU=mSo}H(E1eyk`b|j>k54olv>$cu=|0T%}fM4KPZo5VJhqlY1ugt57Bm1-H zG%*g@i~i^ubh^Q%R7|Q{{I*Gf9@S#n(Vx>j%9oF+=kKRy#VuzoHHzh*W8n=(_2%Lg zg#k`8qES8Ft&kz)DdC&c@y5^d95iGy`=DWS2Q~ayo=1)b2dPpIybhbbZPU1!`07V;`TQOz1&|1J>(c*SKK1yPQdyd8_=MkKp3e; zKCe4-K%(mVcE9fG^el;me~m0*R2>?YW=QC4ESP{Z zM?s3@sd4e+v(5bC4lb($!g*PnQ0$QBQ8>z3T~2QqVD@8wLQQ6`bOOg<`eIl+Sq2Q8 z#QWv}7O8;Fv_F`E9qF$n-@pY-thS%aFYNmh*CU7dxFpvy6u)rYdRz~wcs8brqgAuu zcU3R-;GNypBdiaJ)iJ61{zlYLsSkY$p>gtGb}rXkjcp-M8#Ksucke6`bK%fM7e7)H z>nD$nGW|B?Oq%L^sO{+TJyP}qdWQ7+Xy((%R(k1$XI<%Ld{6SD2v@FkoqV^Q8m2e!Or3 zmbG^6Tjoj`Tcjdj{7z#YxpSp|*TEVpyOMYlJb|9!_Cem$Lpue}jgMQFs?TOm!Lg}x ze#VqEl`^jDdB6FxcV>OtGo_+jx{2fI<@{n{!o0;c{3FPAy?Mr3@4Vgd00IbBC@-Wc z!=K|<>2Uk(J!F>L4Gtal1Z7}|e!qI}{B`=dJ|*T|QsCTTlL&17c!dm#KQN~tlJoc2%+m8 z2E4H(-e{zv(eA@dm|Goh>vBYic!$tL{|s$#&E#V0BZeucw>pRsFzCp@VEcs01jc>^ z0DcPH3bz!)wpcJ#1!;KrOf>5Q2=WLj6-6B}$#c2YQ`zzzzc*Rij=F(Nt%9H`!b;!CA zsO2@qUFUPm>k9n@(aMeD!>hu1hO&JjveCmlv`CXqrTP9~Uiwqy&8g^@dkwjMCwK=y z>Bi862BA1*h-Tg_a2RhQ1whPnMnL&*Lkpfo0U}L=3mDJ8aU0;)tY^P{u+vkXxiG=W zD0&aVW7~Cc?v^Wc0~%7cLKNE`8SDkWP4}n=3KbYCm;-Pu*;@-pTFG5gKr#<35xfG2BUHY>SHuOmsj5{ zIjp8RhK+BKU%otaDy?=VmG-KQq){nYf_0J&tPo@U1|81)=oJv!F9p2V&>akx-ysX( zQe5zRF(Q?t$^w~@tO2LXAhldlg3*#yzjrRjPu&qNi91JTk@DiUBD(G-3FG%!L$84_ z(ptUVn+Zkz{cNZ7HuNftjGp=bkF~!Js&eiAfN{lO11co~(w%}J-E8UHG>CxGpn`OR z$VNJ(8#dht2uLH{n@#}*Vbk6DUYy5s&hwk!GxME!=bdrsk(m?HJFlE_y5A;4Ibf;5X&JX-7a&k%=k-t(`EuD&#F#g}j6{&*U@cz-q>^VTKk z9mZmy>fL?+9w@DUiy9vTKCsF7lyNdu{SuJyC z51i)BjId_1HNnf}c>L-N{o*+PvYMmHZjnl2r;R%QUkvNPe@^R;)D30JcEqPI@+7t?By3bg)z5}5Zv z*H7dW%?3bATzOzyM%0!(S%3F@4b4p!uVi5?L`#i*#tXWZ(dTsv&d9v4L}{lH&UrR= z$zS1|#Slc9&}H)?y7@rV+&f=bu~z#dXfCYZeM{CFX4NMky-(M?PEcdwp3@t$TaM?N znyo=o^G(p3?_uja`U{oVxOwKKCyA`&h<-TiWTzcjgKk$RmiFs+M*djw(IcwM*unNK zO6z!+pt2lBYaA(pyXQB(!|6eD&-9~6{*R<`jr{4*`saPho>Gz2L4Dzv8& zy@L3o9ve&wrS)D}8Slunf1rq^cr5YIkZQl)s#KjN-}2ELWPC@aa?*D0_<^w+MDIl- zxkJ8M%R}fSmVjr=z5L1r>tPx$hgQzV0?@V0~PoFalJ*bP5?Js?hkE2Cw_v2h|0IK1XNL}k$i|Y(b}*n^5gix=(86ZIaqV2 zc&5Px1C9+xnXhPyI!Rw|$)1!#388vA%&ZP4J^F-TPtuFr-ud6J+}@yaRHUGc!iP_g z!~f|GIli7hPbM7>91rfnVP3>4TT{O**x!_DdVYnpI?0@V5QcpCU#WW4Wp=i$$wfZ!YlLHg2R6*+eYtWlbYHsZ+vX)o8B z8}RdbD<6)ENr8>4SgpfRzF=5BwKN1jf7;(>bY+sl?fDm*Lw%QwCujODm-(|f*J{o! zYIyN7*CawbG^BRpl?DXyKpna!u~d^&SLjO)hf#ThKM;Hkz5m5sAve)#I_YLT0WOe- z(fROS|G4Ug#Gh`ovpNlS&n0j)SSqMYIW4hhH0r&C)xXxi?p7{#T9Cfll7|jw-_nNp z5H;qb9L2ttx4hOd{J+dIeQ&o4eV9vL@EF^;1QxIMB%dJ1wRYHW+Sc~V%SEqjI^`$t zKlDJC|AyCV=NY|!n97QN*#qofw7C%H(|SC75&ujG*%@1_jCSn%t|4I+(&{OuP zRiojunON``oY{0r?{~5%uJi`3xXZ=6_^$Tk{u0g&H#wF@wVgZQCP*SiI2jnNH_Me4 zKCVV9hipg>=KW&kVPp?+G8tZgQ@@U2Y2{>F2r=1lgfi-WH)KSn#!DMUsd#_=sdO<}iWQC}q~FYT)U8zAXjH;;%lFdDs2;rH#(@%4|GZoXIg;}ihY;XR z09qpkX(Rn7)ClxEN=j(%rUo5!c{E7ULTS*E%S4x4+x6-`rf zEOsM9QjDJbb018vySTN*m{Xf(m;x}RyHsddbyMs+ecDp3=Tf!8WQxl zgl=orDOwwf@0{>sJk+c~)GP(uJIJSIzZ&!%@YdK<%M;dB5BkFv1PGvSHU$h#LzUkO z%3&bRDVH*1(OLZpjLPl(~44g3FDxrauLk>s;sL|`kx33 zWfkX)e9{2A=~(t-6x&og#%LQK1|r;|jY_jg*Dz#;lnI9*0}hkw-P&%ep?SC=wu9*) z30dUt(*>DSL*(M?D}7n;Ye$5iEhv@_B|7;15JR+30uTd~v*R1h?*GvvuK({0{=g>X z|1FEXPsHiiv&NO_W4o|{&zLX?Dn=_8Qb@gRVk)0ITV-2@4=vik+t9WF`4rsl!~)Ph z$#J19){Jr?crCaL>P_FO39VtCZ*5=l&FJu?CUQ>@0<*H(*!Z0F;KtnCp;_05UCdG6 z1B+1^I%E8uMtKDrOR|{=ZOFy<Gn;G(wzlOCf>VbTf!o-6%ke3f+(Esj3z2lE7JVG zkbBzspetp4scV72C!GAJw>KyZFc99`o3}S8GmI#IqJy9R^gCtK?SFs%ZTg!#Tnjw3 z?J3T*w(bLV(iEJD$WX0Ze|J+8KG^L-=vQtdAqKg8y|Qoe(B(f>Poo6<{0FaJeU^p8 zS}{huzB#TxDJctHk^x#zFI+Z>OD3knt>sNWH)e5c1d{xaXv;x(av!2M`7}_9d(#3F zF?v~-d~Mz4PcXx6>*k9KhGUQm&)OXhy|q<6jkjZ9D;uY`KJ&l5>kYdOA3u~P^Y|yd z3}Hg6A>sfaXw=CJYVKGMl{CN4)D&;eL-Sdlv%RWhM(hcGe+gB6 z_mL_6wU6?rCa_$sH_pGN4;wp(a92`nFu8x4o%<@095&1MsgkTv4u3p{?hY`a`@|vP z9rXt^KC^&73EU_35nBdy^l^<*Sjs=fBOG#bEkd{UI86Za5Hnfp_rq)WruT?SiamoO zNUUZ2XK7ZXtY9T(kgN7bfL!s-f}QA8@W$KIuPx8&L=OGJ2tGs-e@d0136JV5nimX+ zpr)*H!8rqRjf7Nf!?SV3eYh7Fkxn{w&|4}Hf}K=fSSa@CyPRsw{d%n0e7LNTH0qtG zgZ1+hG+K2wbQZzji=ut234LsfZ{Lqokr>8R;+7mN5q zdUr0lm%iM+Z5s$qXcBdp<39*RtO@;^M9t8s;L$w^6f|awHKnpQXu-1r4T=064w4jI zQ-TfGM&kF7FHDQGeJx$Z7MWP!KQbG8(ld+mN^v8v5g<$MUuM7=_R+@Jv1!hLH_ z7VCH#G<69}rW`E9e>{e&Skr?$DE>aV@&I?o-^+Xr?`UL4LRueVdAlW#rZov9)jVK!6xz)Z!=6 z|FD!PZvz{E84b%xN8%AK)J5l<5J|4ZW!%M82NizCZ_nZX9N(D1==@I^G>K#I9l+-> zNR$r;38%bM2W$h3yr0C>-a0|f1*Hd(3(YJ2WhX*YxVTUqR%>F7zz3o0o3KVuggUY! zK0A1zQt34Kz4kFjQ4Jc94Lr4%bm)z%>GD96#F^%zv4l5LE0o_`8$a&o$K8r!ooqmJ=in=!@2~cJ_6_&)sk3RVK>dO!r<388iEZW6t0T5Vb(a&|37%gPaT~J z-l_Xf4J89e-3%dH=XDf&61mB~eaC>_?4vHo{jYj5UhY)NWFT=LoltwuN|!V-+)9)Rr0i{+paN9tcyxIWh`eZDM`%EV_3K3x4MB`|6X#U@rA2*IUmdgl1!``69#o6M&~R+@gxcISHAP<&T66c;<} zQJtH9Ji_WZWRxk3EQpse)#drykvq@VNGJ7JhhCDi&DL<{wDwFJ_>$Ed;1|;|#tCg{ z@dT%^3@Q3K*(11gKkN+) zotK9}q2v6us^bO@xog-H%YRxKm~f$^{D?t9H5*UeD(enw=m2l{4S$dki#Jkqo(zzj$Xx0=2>a)v8M!5?wAL4P+*^*LaQN4g1-4X6FZ<4zm?7;lFK?7lRg@ zD*+Li`rzA2mE5aLO}`e`bwPoGz2Ed;f}fTZ{Um*|k9iv{^&0w6Jm3FY4FSmWEygmm zb>f$g263%~Y==R$q{;gwv^v^oULBr3>dC24f**kdmgB|wT(_^Zbs%1k;GUsu@*G($ z3WvnMWMyWR%7Xd+V~0bLjqaGs(l(@Gn%4x{(j{m?`U7+ zVIlsg?~2|H$l0$E+mO+KI~}Fx6F-Aa)=ET4Z+FO~6H?+3T|?A%zVCAXVx;$pADm4C zdDHdWVf{oUScv{QsF8ttIQfZ5xx7B221b*TYehmqi`_}|#W}<5mYMjq(eod~#Mp6n zkqFqj3`TXK+zq;-x>O_aD8%L4T+*N;qN8z=p84w8;HA#gU(rJt2*+Ox5;D6nkOa!-K$)J%5IX6CCsoyhep$@v&LkJeX=)c+t-2fqoT@1J64*}PmHq~sE>Kwxv zt>L&--0wD(dXBdd4nz?gpyzf!2H*VsDBhF|X+d-mFI$x$ixRS=6zX9r76+KonX(*i z1p0~;RExDMy&{t#}F;uU~D0AyibvU5E;R^yv=p< zLO79>P{}E#4>fwZycNhkwP$Gt;|Jz*3}A76csxS4|Ape0icQR4jAIyO2^ZGupmY`y zEcyu7X+&MH2CC`fmz4WIKy|}9Q_AF=RvpRgv1)gYum0OXQ_@*);GJW5@G+4M{wY0e z1IwGfoL|*sZ%due(C_3HycHE~f~s-KboM<>#1Z&n>B(GS7L;IiP21Qw+E#Mt=0I`K zo@rUGu$_I~Cg4*8q~PaUVCs*c4oQ)?WB61s%Px#_j+G`BEC>j;=y5SpS`_ z^L~pzAjt}D0i|Vi42USaU#4Y_FtADwW$7ueXJ8ke1HsZR~c*nuH@Fk!2~5LTk`zC6&LHQR59zGuFE@qRb!vjmzRIma|x zbT!g;x{NP(8XiS)CSaZZSc$F)VqUDt3@pbHD3QVNRp7w~B)M%i&`M_)gQorkUzF02 z3725m^pvRKXr{%NPs-ZSNALG_vg{YzZ1kniBTV&=x4iYv#yG!wTA92efUd;|uc^Aj zmS_V^gg$fVquAn?i=ep>D+IIoS@#=-B}f=XklQKg39=+)6lPOa7O*A_{+^xh%xSxd zdXPSg5*TQ1UN^LXG#BE+70SbtCV_rV8CS+egY57S=lta z)2Z}6(&eFLNX7!L#OzHJ9H%?gh}4H5 z@d)x8?v!1}z<=T6eUzJS3UW-r*GI_z!RCKG;|;t4NeHei!2HOq^8kS`k70{4@*T{L z6GxNCkFA&lB`_^Gtn_;!t;7Qf31cd*294*}viE8=t(>Jnt+yKP3+brvy5#*<&d>Fa zm#fMh3KPm%FOH$xKO&2MNI$oDEvr*lM zH38JSfSBf>OEF)L-gQ(}1GOH4M zNm3Z7;eibH6~ZTEfR4mDsw(E>oJqv(CE?TS1$I6EyIJ9W1I;QP6@+{UL0A;lpNz+D zCCp-9px-i}H6NO8=K%GuAB1lb_Ka>|{qsCxO&P#lWR+z$WPn>GhcgTlMQfJ8bjasQ zl^nWzSw`Yv3xoZq7A7?~y?{bIHa z$5?Kq3EK&iZCE_rtl>)puVe&hTCq8)nmzNAcRf*_B|EXiENb&rQ}w8qvlJ3Qn4u|2 z%|Ka`zAE<@MgS_8(L-`2#{l0{=9Ayrw6y&6;U`?ysZCU=<+6TA9Fk0~8SS_!%Tvdd z=r*GkdDb)ypu{M-mymLk)>gu(t(UWOAfi~y!kW3xc`L}NNyHrXl2JjYhVhB>Go|@!#{uDPFc^{Z=o|dipGY3_ zT3+yuYYU#2x`dT5`QoQ9s#FZ{KT+vjd_dQNPGA)qy=#{Ezkv@3-LygLMh+s;t;}B?LrZ9+=B*Ii6_>Up6PX zF(D6A$?)hy=xO9Y zK^SQfQz84;EB;M;vzC4PIz4OZB3i+&6d$XTPWy(UY@Q(DGt8)HZZoN|(A8+q_bAM{ zHGt>|c{LtGsbklC{3p8bh9I0(%l2c3t*sn~;lsSFc2#L9w>$$O>;pbT9!(4w4!_c) ze#^Rp?I_+EOK#b}uU8_zzXk|9#wJjGGr&bX1ZK8TPZ(Rp#@3Jju*!Md>F@S(!z{q>PLw9fAfiuIs z1dK>+Mm^Zco2lgR>4(tq8Awiu3-##U>_h0{#p0)r9{p#G^OBaXF%9>jr8C4zv&&G*o~w#;QRS{pG>yi^>Q<|WHR zvOfSWWjbEH7%TU-YbJ8yY-jk(p0kh-(jfvV_9mt~dbby$tgPHC7mYi%hiM?tVhaK- z*Az>7hU~f7#>tw)&Rvr;cPh9*A`M8-!#Zbc>iFg3Xg?-9lNU5!T|}zsBY0tMmguL_(_IyXF8Yx> z%>NQ?jxMM$TKlyGEfetYT#`7x>oVtY@myTy0gu3aG*2l2-%{^GdjE}Ud9UMyvv2P% z3fJz4JLrNh+_FNtggwLPLf$ggEznqfWYO)GX7-c8gG!uyEb&Po(wI3Q8I9^RZksSh zhY%qyg1-gDzjGz2yM3l=j0k@SZF}BJ*eoMuC(*36?M#tWD(T`YH-V;?#x|Q>h0DGd zdC$mHF11b=b-sn?=4r_^`$Ysr z^l#F$b^HO*g*l7l$|1_LiW>drN>y7mS&Tr^{%HZYgRtVmgZ}~q4?ut-Eaf4zTDL|k zmo*HvMm?}y$~wKc&{iB}#=R;a{`dc|x z!|*p4tB*R`FG|nvTXM3e0i~13-uYR@bXm0Jl|uyPh<}!d+pyeo z2!W+{HPV8LlVLWbiIhqEKH7JHw9U6em|Xwi2{_HS_ZKV47G8vxOUN3DzKo`m18s7L za>Vx;uEn>TdJjwSE!Ts?**o2ly3kyPwTeUWt!Hi#vcZ}<y?{jhArb2M{Xj6J zW@KkCc0VdIO@irIy{#B@W$XF80f))LX!P61%zS>mOtov!&IZcJDJd>c`@gSJSLij< z){_gqh7r2CFB0S~{VHu&Yh znNa}Xlekk$3E`o=k`g-ZXl zmr{+W?pt?V}nmdGiq>7-9j1_ zyajd-bkx&aSyvoF$VB+`>}pVnyr_CE*287!^0zqYNQK`u%u(`s7W` zkJsiB1BleM+`o#uo?jh(0zq)rq)eh(1v`v%S@QEIZ_eB`s3SCO;k!1*Y_4CYJLJ`W z<=UM;eSqMU!6(^7mKWDL^43o;L3B&K@^Q%4SlDB_Y`SmZiz5~CpVUw;7wYxkl#UjI zr>{IpV8e9j5mbU3kfRsitSYrTFVTJs2QSuFNtQFiZgeZ?TjpF9`I-%Jhkd<7HC9_3 zG`ED)LHFJ6Xmu~Yfs2oAe0MCGM_I{_Rm!m&+K2pU+b19##*P7%QeTGkc_~D286~tQtC}GKY>@H?BQoR)a zY#le;DfG?G80yO?G$nRWMZ^Djd*@Qm5p=2*g!m{?;Eng4aD2D}+U%cyKXK8h#X4gh z1*y~{z8F`DA7WnYwOQxQPkzj27A%1Q|0WPdbJ$5}Y#pFlgR@%1hXAdCP{-)Gvj`YC zTO|(?_uci71lcsxz?VbTwI0x2M>&hDyqfFbeH#VIS^Jw^O1Cw9gGD!4!^YbxF%E18 zE2c*C_IN3BZ*!wt{ZJU*lPIwC>3gPhgt<4C6uC2S)V~BV3!YO)aaDK~gz-!7m;O3Z zX!pfOmuyGc`nRy8{B+fIFGRXoV<`_+`Ke;7M!w96X1dtifGV?H0@o<|%L5@lmxbE~ zoT&#Q%C1)E1bhbL;6ri$R3_Xzl#FEayqoyucHITPbwxAt`yexr!C+xZ@)0u!(^iik zF9hrVXZj38mEbpS;!;W!MjWU8aCoHcEO;_O@Z8wtIhol}ShALRMja||B_~p2)RFuR zUhPQnJH4pg7@4jO*S(&S11u+iI@SVgX^rX%s2Pvc*b^sro$HH;Ln4;LR*PhzWXIHM zywi#Ji6u)(v%F^4&4@KnP~w>%l80ijgPsK=GLzc6-_643XqSJ4#eqDM}A;HE>O;*?{V1AeDzpu;I811% zPxo+}Mc~QpmX_SnvBD?$-zx`-MbH*J}@=wuNf-;)S0$kuKp z{ z`fy#88{KPfhrIov7FbXpeBTezDc4L}+)C&%dX^0aDb#ufSOmKZ8vJuh+*tSSgIQj? zm1oEf59J?5n26bXqm|7Y#25Z8hlX|`VoK-7VE@tEtl*M5{02N@JsP+wI0LiS z;CdRa@}D8Azly{0hY!i92wJ%ag^{;XBf-+S5;JbXrk|6wavfIKcufh2*6(d+i{(`@ z7vYTsexG8+%eF4SX?g@Fi zRV}d{)D35<5a3E53tyycW?j{}`s#zA-mSX<@m->PZ#{k|R4cr!E7o&q+;`b3`u()# zeJ&QlAhkO31%(_|#^6(qlzjE+`1u>AIc=c?@czk`^JEyKuB2@Psbj#Q02jyknOh@@@GpneR6U3_m$ zhrC-7zo$_np8eiVUG~pI{bD)$r-gAXWkbPJ_oLkS|ALxHj4LzJn`t~nED6L5z8U1t zo)ut3eq%c+{Z0s722iEj{Sr6r_hGmC!Vk%aOao1T;uDL~>m5V_K{ zH21Mo*u|izmj{)|>Oyx)+FxPBUke3U6zYbI2HU<*$bI|+0i`bklZob)+rM7_E#m@t z4I6FOgq?IeEBba~Aq zAsfHvptQ@fnYOZPnmXz-WevLAVi2V%126*awIVl*8rR*(a7AIM`n6c-XUsxI)53ZH zfA0f3^k!H;+^T|fn$?%oeUa>px#C&1``RrWHmJ-yBorZhkrAg0gAoB*pL0#?AM!r_ z$8o-Ka|?iIH@q_izUQz?>vu7ER1gyByZ^2~(u1E^;JUdoz{Yi`@dd zJ=SMveikA11zD2n2|34N1EX@*MpIP6To*jPlw*v8*KB(o5l^DHrTurOyD9&nQGHH_ z$DnD7g7wq1$0RGWv0^A_r(IZRpW*4UA?58Xsu9Qp@n^sGNLR7xfkbR zEX+|`EC9bL0)K^RLsY{k+P;WZ^3X~qzU|na=ZmIymkz(4b3`x5RGCXb6U%EZc3hQu z1(ztPgGH`SIdfJu9#qRd;cbvS+y->plJQed9Tt45z!JBqvln#?dIV6(ePdnwEt4I* zXc4>S%yWXh1ZZx~b8hW9JtZCn5|C*76<>e8yomKnWhukiM-F?Rqh8wyUbBwnmg#Yw z9J7tNe&)rnu|F~~Wg5npID4}dcbw;T&JQ;lTID0it?A{4YKz?PYgk5{)#3)MrwffU$TK*pD^@errOuVRYCIy9Fnt-m6qQQz`o>2O4gEJxY@3V7z=bZn_nbDR4+ zHGI{sY4~H>vj)(wu@E$y>rzhKLNRPA5hHX6yt2rr0c%5N8x}j#@SMM!^vW}&kI&5Ot*Dt-5#@P&gQ#z zshiGhXLZ`*g=nVo*>E`2zwsdmZz0z& zWPe0o+iRtbl>EaL54*wfZ~61Q#jQjebUKk-TBe=swi@&A`Uv5YURWQHUAkfof{Q$k zzT(g?G;I|AwOwMpIKduRdd_Ww@~NtEZGP(-jI(3>hePYb zuT=0UT)V9VwwN`$-)lkG!;SO#1K;QmhUw%Y9_=5!kt^RFIczmtEb7r20f~ora;ulK z@!K>GEBc6({ZG**SAUWK=F-0)G)Cta`8p@}%J7BS*y3}jC)Pa3XR4nnUN{yruUb&4 zug4%^R`8af?e@2eB5;^5ZI4$S_tyHo2#(>7kr$zJ9mI2&lCQH!@^w?(2@Bu-68Zx9 za)a-8^soI5r@6uOJ9c?*<2fIY@p+tb6ZxYAcHZ?#NDx#BnD+VtbLUG+p&Z-oy1ETzBOVX}=}g zc)lA{BeGHNlxG^Uy83O$NHNboFM`XdpVgswJiZuj0ZRV-w+Gl7vTq!;VMR#6wch_s{i?;O-YVDm zAyB@$@zj)qK5yC&3>$QVSR(tZP8Sh7K(z?{2@y=}GkJ#&d}ywIRja{Qqbw|CqZ%>QR6D zNI&z1o^$cK<0{d3b1_k|FOl~mi`L-C(U8zU5eT0C-M;HdO%(=0oY+-i`IthiTu& zW3O2cP1J96o}L`d3!7UjI#lVZ8#H|>g$sSQ`=zX4n4)BH1QiP=7VLhHGT&ly8kMjG zjxW)wDAz+Efd-C{dd`a#Q=78NyEOjhudT0hb4TKfnyzd)0X<5DuqeGwZyi>TJX$)V zO~zT3GuSv~AGg1$BV=)~P2KD0Fl5uR;27d6(cQT>QLEIX1&DjfHP?TB>)`6a!(&g0 z?AZ}w%}D;Tg>q%y8mP82 zP3Y-tyx4Seby75Eb=_+8kQfBR%(5>|f+~`xE8MkYo+1#rlD;nSg{HNOZ1S$#`ug z7=#o6KJuv2cSFIvSI3?X*PAb?AC)<)WNaF7y;`(zm{G$BQa7@?QR4h9xrJSGh zs)1a+>Id=T$!|H2oy@#w?ZqAO>1{4*Zu|~7De-!w_$?-_CXL|@KGGCdu=)c~FXrx4 z=Nz3#^m?kDAE@Wv6&zF`gz|sw0Ok!2bkyH;D~v(ZI$0+j`CQBM>!2uB^P#54-?CvHIGRl^WD7FyLR&+B zW~Q?CO|!l1KxEgydTXjkQ1AQ&`+i&+*SHAN0JQXES7Vl9QXtmlZJ&C`1(QU_MO4B%sclTu{t?|Eu|2rs;ZH zl4^_X^+7fBUWe=%XxUG(r`4Hbh~y;zKAjE}^T1pG2g8^aPkohkPj%e1Bl zA1B*rhHgmEzY#uF{jKPrq}F_K|2q7XAD6Sh`C|HTvHJ345yM(W>upv4XA<_Kw7NT& zFhhPU#Ff8-V*;b>>bS6Dc_M&X`Dv@n{-!z}(c{MgXAQN?1<*KkcdZ6(*6qti= z#yq7K7?je#``lICGYzxDU*fU;PEBLFV)){$i@7_Nonm96I`Zy)ayhWP7GZ1BGepp$ zdzkrJl>H?2sb^u0smMAC{r`6>+%ecD65c9e5P&g6erR#~w|uI505VTP_SAW~S@Jr~ z`~(+jLcE$qy}069!4(#er2UBf&{Ym+Sd#J*@5y8noss+5)csrsdwGkoA3%IJ?jCEC z2`DfZ2@koYXWZS%q7Gl@CU`K^z_5vUU{EAiMan-Q3|-voiM3NsrD5l_dfq1Lr4vGr z{7lBZnGOOj?@<0CzfWtUXW+6gOLCS($LY2le;?TGpNHIX4*;A{);}s9hk46q)=Up_ zAd!35L&1LM4Vba+_PnI&%9VKPi(?6bW!FUioOh6b1`tR=WR0q5_wBnk_4E7X#|(Jp z9m`^A*omt!5lNIQA-r?x-nUMt>4_{yjyVW2?6Rm{rHI&lnxFcrH5*T3;Pr&;_9Fs( z&tvaL>|8=hA@??^;i2RwHM{Led%uJd1WfQoi#2-O9s7Trd=e8(HSLd+DiFCOEYdxy zmWP^8^Atso=cwB+t?NuRUfsC(KbM2b$1-OXdst6gMGS^%K1t2I$)4$pAI@{`f=xA& zDve(G$1eFmfsaEat^mhjrgg8a^5iE}rWY2;Ey1IcMLPUS*NzYAUQ=wnjVlirOyaD# zVPtvz<)V;ak587tkI!F4b4o7>FOA{9$0iPCkfkaIpVA7_Zagns>eR z4j>c*OkgAsd$|#GADmJ9hM;8RBnIz2YhQDh%ebddW^9CSGscgMbij6fZl$)GFr z+P|1R4kB=@3)17PtLYRDm|nLNTL?} z^Z|DuH~f4wX4w6l$LsRkR52*TIj=hSN_e!P$fa)%8|VsWknbXYpvkrx+HRY8Z_j3| zc^3cfPE9%NJ!GX4hX+7&zx5A5?I?wPkE4Coy+*%^8s-!mECLJe+#|9tJR zLq_Wt?5T9-nFEU^XV&x))O9b`qT+%(O)RO8H;8f}hAoZgbW?j|j#L65GyS`Q0U!J=dg79xp zDCq@#h4pyM>Hg~LC;6K;#DY{`U;CWy7k+kMS)aTb$S083h{!VVx(KNeIU{vgJ$`tw zd|9XqHBTI~u|Jw|isyOBX&6k-b6RWC^JvQJCt*H`8*d23QA~W6=VV|s*NhBi+vGeJ z0dr=G5w?m_zk03aS3D;p!%fxM64R`k_CU;(_i<={@6&$wkHY_5?$uG(+Wsk_7HQ3R zr6&2j*Mz$?M|~SQLsrdfM?90!r=utcu!GK?XG@2DIK4##6qt_bzKC?7mX~q8Iy3b0%7t zQ#Mqw7KS4Hl<)eIEW<=lE#PV=hKzhj6i72mJBZZ&VWEY+PrPP(VWMFV9xqadR5PW@ zyQWgyQhLl*#kbo)be<$auQ^3%CECi!GWF1q9wlC(+AZMQEQWoz%VtoNEH$@^a ztiv3%MXW$QNZ6z5<25vExST53hh?A-S9HlQSXXC3iu6l^E3e{Y*dIJrQ0B6hV!uxs zUc3&LPzPu2O!pP;mvN;C|NkwUrG1Bt=k{UNC|r)g0lTf2HE8{-cf~z-8ABDLTyv(EKe)=}#SxmcKM7G%E^$y<5|K7p7jfGYdPI=d5 zuP5ZIcRkHbT=HXPFOJ3HC8?4Dz?$e#&HNUVA z3ISw7EFDy@SvaMfuIf9&V_At!_rfpfoF#J--^m$!$~Ju1BSd{V6VZZphc@H35-+St zBxMqAZYCDvnWcyOi`;n=n~jP8BGZknmTTc zq_MH$N+2&3%;{=Q*_lc+zC3CZM7vIK6v3oAi9}JjDm1TaEy6aeg)4jvTkHBN#m^YRpOWn*D;`tvtcCC-h zQW5>h0z(vD=d#L`mWsDd8()#2M;oPjg3AS_8ZM+yH_vX@xSw!pgavXMMk}c#McH^> zqJwk>Hcs!~!m=0bs}PnBRGkaUW~?aZHhQ8laT@Dd)N$I|L{{0+a_;HL^=?&-S`=q( zrQx_`P5mC5h62DeakM=+CCQ>Px(gJYh0G;$NGss*Zz4b0t}B3vkF}nW1C!>mdwk`k zJxC`Om&Xm$8gUseJ^%n&(&Ylw42-BB)r4>LW8TNg{_4Z z4OJk06+uoBWmMITeWR053PP_0Uruyg5_j|tu&VRg|$hw^}Pi*9uJ zsME@ifE17H0_H6yDs>@)Gm@^*iE_IQv$ZG`Dt}RarE=OzX-0HfGXZ)U4v z+4Ry;_iE#Y{mMnt0-e4c`9$Rlp?_vZ5Tl^_3E<&yjXqC>h?P`Pu>8_;lxH%W2 z{h@P3ah}$cP6-54Y`Bf;k{PeFs;X07`R^qT--?06sxxUw@__L>iB|1?(#9MMwS;e; zQOwj(#?;&Sd}&^u^}_-`rW{I=fU38cX-g~m;9DezKCeQ12R^WZ^z|U!n}Sb&y?CL0 z=Y;gyjSZbiLam)N6JwsK;=5beq7alqC;GE1{K#NV2_k&W zTP^0it%#s+j8AsCyf)vKTRFi8axF(0sng*ntn4SEj_Z6oyB_+7SIv7KqnytAvvcQN ze5RS{uvvBL>g3Yvu;JFr{CW4Q3>r!CES6Vd zt}36?Wlk)(zK6al#5D?_E0r*U85NpCxdw@@m}C5DRE}DonW@!62_frq6g_g*uTT0D#ygZirwp1dor6+S}aMPHOCse6{a zWyDG_bMQJ74{pA!_DuVf-FZ;sDm6nR_xB0;k=0JNX7L=A{BZ&f1DiVN6VENnHy?Ly zP}{GO_xwJ6jRE zDNhR|m{CkCD}^stsZezF94#Vw0kEk;MOEeBUAyxyBHMl_e9YX8df?Z^#zJ^|;-i?G zVZW-2@%ga-677#(tgDiHXukB&$+fSbf;IabJY#ryulOPiCe?r?7kuib$2&E2`4?r4j5am-9*U(`At z*s66~nYY}%ui$=ehl_B&EWl!mAbw`nMchb4pOboMC&Vd@M;&G3T&!7?uwLCkUN){@ zvKDH~Eu>$d8hB%9m9zD5{<>%vgnu+e!2 zLstFEHB9q$Zl+8OHar;t{f6R-)6$^hu|>!pX$eNAdvw9T%CaL{HWT(CE)F7jt&e?y z!}CX8Rdk-fsSGK#x}ZWjnSKPXEJpKF&Rm6Gh3K@d@2J08#fnH#Km6WuS{EHv?SP9| zk90Nil`tQ2X_bVp@;|(NpcOKeP&UB&|M2zIe^GW_zfuCyA>9l^NOyM)U4pa-NO!k% zDcwCZigc%dba!`m=MZPO-{-mC^TYX^e_>sFt^KWaZTO1dU3cZ6CLP8H?Xo8Z{^=`S zE!~B4S^G=AUgT2;bZV>ldMx0+yEuuTw;$FwjqR5EOVj0t>xot?txXO3pLLOVerZ)w z4<*j~OWH)7b2JR*{cV9JbR>cuw)9BLc*E)PiwV2#dRm2DIN-#RT5oM3STdM@imC#O z)DQ-*7=#zEM+d{pkdIUroSozzen1>3jG~S||%$ z2{>y({j%rVoYA_Q5J!#cs9zfI3`^*Aa^q2DGy?$l-WhoLQu^_34X}>|W5p(TN(6~x zgVPcmu~q?@Dp9|p?$zFwtrE=1F-Gf4=WE4=T0JvJdx!>=dAfl!n)1W#8C9P6hg7j2 zkfOwPkM#?~bR6flS;PNYsRgKvZi5sgV;BeUu?8At2a0!}6UTz-{fWgvX2of&?V1}zO#z00q`6ixqAFIzLCwVE7152IgCzEJxt+;=No zvVuEhQ^NMOx$3FLxwsDhCq%g2#+mL`d#bHJl3_$({dnm~#B3icTkyXA{>^kVjd$?@Idf4L?mIZYZUa3LGv8_L6L5-!O-yZYecNaA2J zgTw?ft#4_C9-R1gN4?D4qv}bU5+Z^W>w+tOm3dO4i}RHAy2r)nY@qGO@6o@-`nk9O zEUfvv-k^DC`Y~G{PB&abhB?vIsaAfjy5jx=qqlHYb+jl`+jY`{sbEhu$yEDyAC;{^ z;V<=QgOU^8nqB-=rM)sRsy2&N3QglNgL+C-S4+P+@@*0o3&Xgvh@_vZg&#^Gh1{ea z=c;5>3CI4AVLy_H@^!oe`IQHHn2rkuzbnI8x9++x%bl0;?O-lQvG&M!lSVwD+s!BO z=P#GZkh#{7XnE;>Rwg#GVBuVREa$;z*jy@dIG-i+C+COFLV5zLyXUok==PNI|1o`}wcjVGlJf&&aSn+v( zP3@Ge@Nv)%dvbgX6_#ATv&{-6GS<(-atN^BFb#ifq*+grpmM^j4l#=@N-^cP-Gb=B zwC68Wtaowu56I>HSM!&0J#Cd`y7UJ4jzg8J!NXDsZxNVT-M$%btv?m)>u$p8jywuY zT>7PKFkh3h{SckSinnG)emyWfMHo~>IIT6{H$n=XNkh}j)RVvaoF;CDWwgg-o*{oCrI`!_ z9w@cRcId)Q8h_cSl!;@3J+q<(&>Kp?;bWwVqo^b_;1*RMwA16R_gOA8S(_HsN%@7D zQ9h#{h@+sd^Ff4D8&(<+@jB~8#h-pNj4>fSOWUNt`7r5Kr=e}H%MK~?!??ZGZZ;-J zAM`dyD9;vf3M+MgEJI~?w#eq#+BmnUMmLfy9;w#gJ`0e$M}gLbf*X3GexM~{>_IQa z|4giJ1{6-T+aR1>)(>AwF(U@A?%e|9Rpif+zWpYIU#5b|knf1(dBo`48up*3yLyQ9z-zW!#aBB8#ibwI)QCGPR zOFZiT^-2@?!xg1|b8M0UQ?-1KI?)!|K8Ko&eT|mK)4p-T7hn9c`-&z|p%r9GEj&Xw z|8D_B_o+X}OMXfyOKqW3SufaUiOFtUkOX<)Cg>4GL&LG7-fmNRsYj6R08BKMmZEDs~a6? zec70mEE`m2*jV4Ipq=lS7ntxK z&Ee#I72TuqT}gK7Dlw%xE8A<|ROsIK6Sdlp))7ccHBd<3Z+d=r12s`H{mk8L`Xotr z@$N%3P~2``OA30HwAVO8)Z7!pktVhbNa$rg$476LQu~=sZNw%4#n;(yj#X_IUgM>6 z;~Ua6@a;QLP*nDPkb-vAvVelxaokpwigA<=x~apkn9gzLL=J`6OF|U}|K9Vt*n4X) z@GXy!QsI$Hp$R19Tn!8p^xR3ULjO9;3aq&e&Hs0np~7;HhfDFI&vE=!EtZM~Exn+~ z@-V^V6{MPzz04b|~-n;TWM%(=02Y=9;M)_6*XE!ho%J%_@be{C%Eg=bA_k6BAvh|wH&o#NEEu@=tyQ39UHr+Yu)LrA29 zsz(0C6%t*wm=|?>&3?tEIiTl5g5)fxb0vw8I2QcrIqy<8^Poz20~Ex!9I7PkT3E7G z$0a`Y{BjF3VS5{Ba_6^#Wcg(g1%a@sb@kryaLk<6e+gbf+|!s{cFyp1cF{zBqZOC0 zv+(Zt2YJ2XeHBlnIe4saV!87DU?-B=jfr1I`$=IYaB{^opKG=T{j{F-_{&!)pv}nH zKXJD9)p2?CCbF~!NbRT^yDLtRss~vUfeml)o@4vY=+0+w;7dZoSG)*-qKY3yOxrRO zl~8<`!T!Ul)_~}!6&Qo0NENIVRoG{xano%*VQ@c$>$4STgh2$!(TTuPvos>xbn&WS zG)l(a6@!%KTcWQBTs#P=I7`V&i-Zpw z<)YuuGzv-<1IOr8m8ojPGHDCcH=|&%9?C7slO0m7W<3sDps3;#qU){pICipoXOK>VXex9+KfcaFdVa!+Ux zrNsW=fTjHB%{KmfBlr(QzoCJs;KXGBY-NKsG8L0HQbh)$MN{CvCwYf&($Rm{3uB-x z5XUN<4HS_BydQX0$=m65+z5J+tv@E0UzR3MJz@y#0^GRZ3ww&4ToAnMfnFGKe#AL2 zi-&ENT=T)fI^mdN(~ZH^RWf<>4Wqk9`FZt``QIK5$OcuhNXL5Khv55c4aWDTnb`cp zfd6i@8Qi`S;P~B>47-iRi6(d1U9pCd`Yu2|Xvm(lj$TxJFXlIjsY+taQ@0;9NlW{D z{`d{a^u_VoE1%SrHR~j~v|2HU>IYrFppXG5XFfl*4&$B0$$a)Go2z(hSAGRvgzw06 z{3u)RYG=jGOnW#6>6yH`sB9p&-d!yxBtOdAd5@@L>G2HFiM>x{(SXi3yNYyvAdacc z?-W`$jn_-zH^Gum*?%a~8dkA067;MbjwTbPhZG}eY*KcI(9<46o9o0@y=FadBpFJei~s{iRoH&%m{g5{XP$!d=O`}Z zn#^*ml!m`O6C9K1(KwOz8Vf#z z0mZ$d&8dq>CObyNWpj2Hjj-7xsNWAb*q_W0PSgzm)!THNPgrgZT>Mlt@xg>G-y4u3 zL-{9HEcJHvJ;}E9@@Njmfe9p2=>$-;e#wp=zjETO!D=TJTaNV+veM2lm6DfGopeBx zk~SA|A5Kq-&Qb2!_LN2<$n}(d-4`P@aq*4&t8Y{lzFdiA;W57x&YM$)LA|}7p?*79 zcVNo3@G)|oNXWzM3I3<#6f*Ev0R(YkX${(Lsxo{RSDT6ht_>vBm*cpwo|WAQf(D0D zt8mSv)G`XCwqIVSW#wla2L^%N0w}3NA{5Bgj67O=X7X4sG22Qw|IL0Blr$R!&NQ16 z3d1B>=C6x>_jQGaQoNEPdXvpSe$x_}$E3JdT?!N2#=ZG3wmM%$J64d zP8bT(E%#xxzbf1{={es`xU5&^8SLJi#O}Ob@V1YgC$g%e&GMh?89%8VH9Z^pxI^?} zuyox+)5BH1L`d_ppdxJIh!rLvg zKT2i6zq4k~TDD76WfOw(HpDo+8XP&~Y;n1%J%pMvPS)-5T4RCz1$rh2x7B7V(+92L zpmb3PV9s152JofDY^ptF?pGSF2x=KN?RGdS`Ew+d^SaJULpSX8|^PDes zbZ$wRYEiId>^QgN=A?{5VeLsWs*Z)`wSK+}8$-s=Vw^3=2wu0J?r#Y@XZfC!eFkYZ z84WLopV$V4sR%`@6J<-SSzC&}`KFs!jp5(^YbG>ALS^4jMb`~h{ z2Poe72N5$3XMra{pk2d_mclD-L8#*ht@=&spD zu`%7-AcXOU5wKQ`Tm)=Kz8e(G?c8%a)Im-7St3I6aaWk4^`_XS>^wSXf178?I?Mmh z@@-|!71k8nuInmDk8G36SLXmbeD7QXOOkFtv|mD3XRg3ge?Cmb>sj2#BgZaBQJK&Y zm3;eH;Yc)SMh}7?Z=2BTu0`%3G>I@JY2HQ(q$F1~Z`^T?5hecR!}M~R-dOl4{-;YVN_zPD@w%Pgo9Ijd6Cc67rKLJ`P^@2O>PF8v*(Lc>sLv zlLKmu22M($&ODs3SN>O8vwF<8c8^nnZ^xTQEkw}r8T4Hy z&hyG#malh)e?jvidZ#8@Pi=*P6k;MPHmxn*J8F|1d>;*$C0JYO%Y#PO&wzljN^@*gggHP)}H(7-Ij6WfK@-nfHa6TU5={TgI^^>xeA- zHLE01@M;@2I$6r{0C6eIRLr*5T-tbrROOIJpi=P{&v|`$I9RxYW|ogOWV~b!O!k{e z!|#=FC>-#M!vZ%u@ek52>uOO7?54wa7Nk;|!c>l}yCo$(^|QH>nIHMGjw=5)bn_n8 zO$QN!bmnsOW;Ut$JkFa7d7(srCbstn$wmnTF>5;>mBv=KekbvFen&(5hMH_plVDHi zHS{-;$6v4^H{_6LMa@8@X=0d0Th2YcOzf4uw7cmNZke)|ubK42ePp5is)6W3C|D8j z`YyeCRS*jfSg)=cZQ>cP~? zWY!Y1f@K9>wW$PgISWhvK#6r;3;lFEh1fEJJ1xTJY@UD(o5!Yl!I_QyKgb5qBkq|I zeBDLi06pSR3_|ycK~(O1cK#QGkRX|VL5ci)%$CJ9cF^KEeiW`kmGGz#2g7C)pu+OF z!7KMJzEsYD0ZY!E@(Brf**vf<5qb&?0h#h1q+WXG}A}!rr^&-{kSEHvE^;9aQa#MDCUnV=&;{69 zr>f(XIc|0h&QNp13P27UTkje{0_4sY=@GvHiYNJfF9&^=wNMf?ts$vcyaaNGtKfq| z-dLNp&0f+fzMuMaounQ|e7?`RAH=i5$?a&gdIwqK6SlO@S!pD@S4w_D-LCXQIscwt zw}M+kBTNr2r>S)zNRD`v-@V1N<)zBDz7Ze1^a{LsdaW%kB2?$aLo~~d{9hgm3nR<8 z^#ySh7{tle8DT-C!4hDN%ouXJdxT_v9u>nK-WyE5Vu73zm4_Y=Y<*aB=mCc(7;Qr2 zqg}5#R%jc?;{usW#^@4dWi`2;M`0X$(kj*RMu_P1y8u73tDewrWh3GJ%H~}JQzD_B}6`dTpW&)Y36a5 zzj(Edmp%BgmD`M0PHsWQxjEzd3(Nitqq>lOee5wSy&?9jr=WdcB>Z&c;LL#7kIA?x zEi#9e|B>=k(wC;)W%Qj+CL|fc#{BdhiZU96nL2vgWOFB#qK}7MrzOo3d{hc0EN+-N zgwX)GwY%9o0#r(8;)o*w^o<4*|l9Pmnw(6 zOMB&U2|WkeGPfEQ4o%iw%n-VvaPOc%94-ZlKyvb5DKLm9nkD1kl3zr=!NZ}&;N!0h zHDOLF-ci`6*5^qd zAm9u$Ed_?qb26+gB%{CXO`*y3sf^zzl~B;VPE5~!ZyFNXC4n+KVp!Z~KvNoLbuU&I z?qTU-BDQ;z_W~hCe~}ukzaBi<86Z5=3Uj-H7}9rF=cOYvy>cdJ)Y4ifZ}VwCo8vzRhw_wx>%C$v}&$p7I^&AM(sBk&-Hx|Ll(x+?9_$ z$^BMx@3zATdzWV>zWuyq^m@qd?cY(v#)0R<2d1sZ$QI2Zaw=Ag@wFODcE4#|cU*>T zd3x_4kISS3cCT&{J-PI^fhGeJHF$6*GGr@;X>DB0F?%~HodUC1QVUI;5VO|-e*XF; z&e?AH{sr4PotS?RJ(*Wx8;jSe`lW&v7+PI3-u-f>8J??kH~UxvkD8x4LfSmv0*mz! zZ{f!A$$rK}GO^?Lmffry@=D@v0K+fYyxIDw_$n-LLTN-5j-~t~fGMwMq}E<3&LG2A zL5+XFSP;aFkM+ACzQ)8Q&T)r>aq%OLEdWiFJ>zniE}k@e?BYwtXjSvXJ1Yr%BQUaa ze&Pp*K0a*AbEjxqx}stCIb1X!ldrQv8~~v9PO)$Rt97z|v<1a%Avf8PM7`=$;UsJ) zvmsR3ArI%k2LCy`MkpU#HuSbZ0$C4z=J}vwLjMFMqW;VDd$nq%G=gr46u)tU z{k%REp-_z;U#xFyEP}AB#!w9NAY`j&$o20%I)^0slPVd*$?JR&fd>x*MW)rZsxn-q zwQnF)GE6`D6w+#SKEm&W6%ARfZ*-eJMb2sAGBwuhY`ThKk|KAK>Hs}o=-KUg<+qF)S{?JCI^&(Olu^;#V^e8&2 zsk%X}70*Y^jq2al8{#Y*l*3n_s^j6PYUby?J3~(4U!~d}#YK9;ef)pL0oNK%# zUCd8tSOeX)A8e8{$7*{GfzYRNs*$r$&i{H}!PW@wvkEDqH#}6@V#MfCh}%S;&xe^~ zgWvD7WnFjAO9@tfH_F{(6=aYx4eYvPfd=`3>N`$_Gf)+Ms znDK@^y~#;{`>s=$M7{40%6cPzNKN^CT2OW9TnYZ6eskS(+>>6MaMX=P5-T<=BuA8R zzh4<)m(uUk&WZjc8njW-uVTZ(DaU~8Fn=01L>jh>0X_aXQ1gPS=8OPB@-I_Tv4y7? z=PbP)5kFmQ1Q68a^d%*=7FULk1?R1>-t6lE(D6SuG8b1R9z!04fBvrFqzkL}Wzpog z5%ZTyfkMv2ZA$9SikU6ceDPVA$uyoVKOz(N`-zGo01V8>%*d>JgeK zq{W_)&o=Sw)Z6U{f@J33AJwR5vqHb%1JcAhYWQ>XP}@9lJ_T`MPeDOf2o3vQUglN| zdBGv#^ox#&O`;YRuE)X?)~IMuo7=3$eK)>E5wDwu#RrsM+(Q(?|!6N*jZTe=V%TW8_?(E2+JRv#ejcD&JzgVb@;1Q<`$!Q{$-C(p+Q`RJz?`et9 zdDD!9Ap2WiWSuKj*ek2y;P)j;GcY#d)@V!f(^B!SQ9jIuxtgi83Wwt(wJ86p(Fg zQ@B8^UwcWMGf)r58&Jh0TB>a{V=$fPk*6nY>{G6)O^v@J2l8in$sPVW9ZU~OBf}sU zzP94CwcY%Nlv$yryKg^GBKGvj0Q-Pw5irZ_%^Iyhd>xlj!>$-I{`I#l^|EgU&JiE( zjaYSwN?vK-sB22i*F75cjc^{tkIrqN1I?ofBjbSWvG#lS9d?J#in8jcY|gSwvR&tA z;GReEI$1fqTHmWYiIX^y)6frG|8==R^4;vNY}|!l>|Wc$@BBo}eh$Tyyt9jr<-+wo zQ1X8S9q?<*h_BzKdi|@Jk-<{n<^hg2@^kTnKzP=k<6ehj($eqAP;6iI^KhG06&Bz? zGev}Y@2NxzUFU-4eh~c7nZVan%TTC*=KoE!tLHYE?Dzm59*9X+-Nch<1L0xYm#O7J z4yF?T1DH1OlK$d#?xrV19zem>@DAIGl%o*GgG0$}eT{ll#0^1tC9v^_8HG4k|IIj= zA!wDKC4CJb{yM?3#SNPml%gwneWXS+>Fhq~v*B;#ESnT}pRN?uW;FX4yu~8=M@&=5 zU45=zlxpwdLMBblRMC&F;Gf%Xgg%&@E+VL9n>x!LGekVU$x0}Egle3uYS-BS0P9X)+rcsque&<;Ra}d+Pt|CyHU^n3&Gx z(kSidVX|B6<%WAWsdfd|MIzOPleDHp#9^AGKJra+gB2%YoZ*yIDu1`y>whNagoaG{T8(@+`dOQU z5~bVCNBlzg^w39Fq4MagYQ}6+H08Ila-gcS2{w3Mu&23VW`EC05vd8fPC_(Yw=Gw~ zC09&2+|Ulg@27WouHrLbxYdM^{vilSN;}HOTv9Yh*ua$5qa6C~Yq;Y~vScy0saWqv z2dFQHA2o2o3X%_8aMErHS$&R{xntCZ9;R%(y-8n(l$Sq_g(g9p%s$!VV}S2upE zInxcx(;EEv3VC<<(JO6WfBJcnv^y_UUbrUe4=;wG$$*a2P_|{zq7yNa@~OX}3hkw~ z!<(+%sr4fy9|gLv5joA7=x}(%)(G=_WPy&i|EEo|gf)hed=V3_r%?eY!*FbyG{Xk# zA;qGCiOQ33tHVk3U3+FNAcs{*6`2k8W=Kr^lC^iz*{v*Qg4(n24t#PG@bciP4Q=6! zd96BTecJ2AQS4&O5K;=<85?2QkhZh0Yey!Bwb3-LDfu8KVvfw)d=K@lq4deD(ounj z7WdW z%>_i$`E}O)CuO(z_|-Jb6jJx*OVy6IN=vIB3$>0oKyq8eX2u^wXPpc+8ugt_HQZ+f zZYFoEtolpjCv@H-g;q{C_>pS6FKYvdxN{1fc^OSNYhxK77>?q!i~i$+UPvgVDIqP2AMq>I+e|LHTT*FM|WSHJw% zw_m{;iyUb{!P26v(STi9B;P9I<+qEesLKP$25(S^ZBRK~RMN4hKE8Qw#5*bF#y@L7 z+<%)mVod%lx!@fO6xWZ6i^FCd`jA|G^5IAJ{U`1k9VXi6pFYJJ#e+(Fb-wK}>!(AU z^S%4oNU~}uvj746-xzT@FZH=?-cz4zy)~~8c7dYChT z_Ofrq*j*0MIPwo?qzhBZ9P$PYRc8}`g-vxy+)LLoEQD0%9Z?I8Zr5cn_Zq6EXoIs^ zepdvMR3lvzUQo*+C@wn;u7o+W_hTT$^qS!GfZFJOlP`Si<8b|83bdb)eulqaG*a8# z9OW>&wi7G+OX@%>xUS7t@gNU_<$108Lc6KdI=t2O97ufQb4eBI=gB-7H!VTl_K9C% zanE_B0GVzk9($=~R;PSMw-O4)`^r4z1r)vrqZ-PY3CSFKFzOP{&mbk+aA@wvGFg9W zv*y7^jyN}c!RwS9s(2#qSFfIwEN&%ypH#J4MEl==0Yo1*l8G2E98+In%qbttc&WOU zK@4$PmQN57Zp7n@`wmy>Z?H1)v00{9VeplnQ)P0hsG%QqU?rgK@js&1S*r|%)-@+F zLF-+2Gp}TPYq7=IQM3P(08=X_R5`imjTNmNe3EE#V@mqu*ZM8g7)h2LjYxkG6I_AO z1eFTp;|+WWmNGz+qD;>d<38`clH;cCx4}}??C&1tkMuoU6m>z&62f@^xRxxxvE~*R zR~!vJ+>=KtNr4U>-#NP;s)1so^ZW8&Sz^cz7aQw2$4YhHc2hX} zw2Rei+_Eea5G2n0A0;O#E@Ts2ut3qcSTv*VzePm?To2>0O);2DC5D?7zQY(-vf9bA z&(L!-_f(+yt@X*kTM70P4mf6P>G3i58&a#0vD^Jm1I4_6q5FiO`jv*Yuh?N@03{b* zyN}YN=hhr+{pX_Ac_W@AlzLOA4e03rjGfUo4KPWcukXrf8gXF_>}HdG z(o%ZzOa2Pjrix*Qy-R-s6F2R;IEUgc`{&Q(IiKIk;$JkU1j&tW0KFJh80I-G3X7WK zrdvhy?5hQ;(WRG62Ji@XRHlmD$ew%4jL$+(q>;VwWs+0Dj$d1x(;+h#S;(~jglMJK zp`lN;^-qxrH%&Z+KHd(SLA}_uago+Lkjh3bGLP()1(maE=)FHEa8*-&6K~o;p*qEE z`76NG+L7_6U{f1E3821kIE6M(vk1bsVnJnaJIo7>1&}wck*B5;@*5f)x;jAf#<81@ zDStUR+2ZA!jSR>y?*tn!rgs?~CHFQ5F=B6ZO1UQ1UnyY;^DbhPmSB*6 zf4=2;Snq4^S^5n7utQCgmXSTg9WmsmJQFi)3l%)WmnfO;-P1Q4G*6mh*?RVnLqd4E ziSf8N&r4(zSK$J&SbTJ$6Uz6VYj$s;XeD2uPm@GP>bWCR>#u8u$t;FTSBE0rlfJ3J zo!_e%B=(x<5&L`{{;wEB;fJ&rjG(62)F4XI%G#T2|8V5te#E(CXW$Kq_HbI4=$uT_ zL-L9;;UpA;1?{evPa^GcSJ6x#Aqci?ax2aw3YmI;z?t8MribJ3x`c~QJ^;kQOxy+;T#_a6_{pO$tzU6DsuA6UeobRvGCvLXXG%l$^OeA`2K)B*1D~%ZIrL zl_=w^)Rt!INTr$Yn`K5;#{_R?Ws=J#id3~FDl9ivT13k9h}cYQ3g-5H$Umq|O^77o z+GXO0MkBK}uRpC7iUQ`MlKFpBtU5jk*dLCa0uo94O$9xV{qeW1$5~8batQ3uZtM~t ztVn}u7ggi~sZ_A{p;4FK;8!1Ezh~8%13Y%a8z&chaHRlS;+LUZgUgf6_Fd$a4+Oua zK!Q7_iw^!Z$XX~x>Cu4%b#KxjGTlIRl)2LfrbxgmU ze)#y2;bgGiWkL3()fG+ft=T1kNb~)Ns@o*f#0YrPVH@^Hr#~`Y-ZVakE!F|css+Cp zOBYzDq(4kD+{^<_JGqHP%cjg-ioKn6Jp*Nvahyn_SffWgZKZgx<_Vp45O+vxdhZ&S z-68;k@t9ze)%l~KCK4bxgV%Y!t4D`FAM(PYd{>_#_#$DkWZQ!L&7lHnvL%A`5zVY& z0VT--MOW&ILz6pnVPi$v*>qjB?Vn+d@;1{Fo=jTTd-(zwrs;NTMj#+HNDd=0De#B) ziP9y~<$&StZ>pC~)Th5bkkM z4Wxy3>*!qGu$3JArgS`5Xs3-pmgdqeWPkoTkPl6k>5#40qn6z|rqbZ~2&;JLb-1$E zXR$Q@M@WB!K7!a8kT@KyF!dKpmRnA0AH$(=v^r@EzVZDI9rF5hXd9GiAS3UrkFHYw z;jW~arh^HLsj;d*CFa;(E3vRhuhpfMKZTTkufgDlx})ub#c#A+4NL01GnagOnt?@h z*Y3%!t=%t+AX~e|?!PYAqr6*aNA^nX$L)>>Js}en+lxRZS=R|^DzZJ{NJu5xZr`HF zgRH0PoZ++YU8+0aLr=l9KF?5^$w}8USpR2Dnzl3%vdOf%JlU@r0NsHV4j2$?m5*fd z1+o++eIa)}L!%tDa7l4c)c-!l{7|cird>}GwX+$0nf4kov#2##?#9R(Jtk9kQpso! z_-hZ~bWnDTKFu9T!>u_EI%s>Ur`GNNart5ySA|R@xYe>rWTkrFb#o$MrmO^V^drLr z+Y@jLnyO=|jW^0QNpF`NDmcFXJC(@tJvlwgXGxL8)cni-mHbz`;69HYwu?MK$@Pwfh>9HgfI|KOsN4hRHr?j#>%?@)89}E zQX1~864b&)Hv602vd_KMuif^zKi~5+#RR7X(G}_lzU#iB@qbHoRhV#4=DPRgUG@Va z_U1vdbpONfQ;=n$p|64RNy*|T&EqOvLgg(Y+8fit?&?VRZh^UT-LZ;VWS6-!G zLo*&*y1QA61DJN)Z4Q@{#!H$)vzHDjU4Wgu!OVQ9nph@su00;q02Jr4NDW#uZPeSX z;3tQSg`rYim0uqaKx4Ckm+&A?!AL7|+bf50Eazvd z?}(YZ9r<3i6_DlK(U%ROd;ns`GY;WhWcFOga4WEia$KOG%Lumocyy$>zh$(DlMdzE zY0}K@a?c`a^WMD1R^@r!S$L9|o??DePX<{vzje&F_SZc2MFcd_c=@hKdFbEJ&ApT; z3!~^ar6=p^{%fTQ}1dh zckTZB!G~?Iul )Ft>F6qB#=rgwDEvaHmC0H*m-W%U#=8~b)BKo`wG1IUxngW#v z)tVjWw2JimVHJ+t$3?nOkV1AlM#Jppk5m0!Gc}*)mDL>+OmO|p31vIqHaj&|+J}zA z0eKPQOfIMaq3pgn{Ac7W-pY$$+W4=rZgplO7`Ip0oB&JidXJlY3V8}Vi`ptFEqGo? z&u}@lT~=)6MUtRJlyE(^is&9 z>IlHok_HsNWI~I$@?Xwu_vuLS`_l@ftJNshj1_q~#3&g4>qp(7(y7*UpfA;BJ{=51lY1w01ag4FK3-c56y!m0 zr-bR)U21;=$1I;b%^L4BPLVota!OF3wgW=7ZvV1Aar)<@>@*WcP?-g`hQLKQm6!`( z6jYv_@VE|H!_tW0sV&1_>Y_(Px0beEZK}3j)iku60eNATeKd}`GFRn_xq-CB>X7wa_(< z|K?5XH0BV;A_MxHv`KPXH$}5+1&tsA@(!I3`T;|Hi*o}=kH^CwwG1dld~J(Ri?m;j zYeA2Nk3jfAY0wufqDDdJis5X2{y=@}oj)bby1mW8uFrO5-3NsW^TRrUnSsv35%yb+ z$WWg#>nInM(CK)*foTpUMs*D@IK-#zE84;o`?C8I1z0K1&o5EC9J)_F0jpOQC5>2@ zc`KO4zDDrHHXq{GueGDqvF8^+YLxk6sUp6M3yKQYzZ01BNIW?rP4;GvoBNv-pMu&) zmng*Vc9MlYN=v*Q)i+i$HnHQTj)E?bLFK>K%-5sQzWT-gB}-J`voUwl>lW%{5@5@T zwC$V%-}Tp5ND*J|s`b7Y!Ng4ShF(2^;&e zh&YWc!ziOB08T@adO2Dsk)Rkhi|>9z@g2TEe{4`aE@v{SfCno_Y5w>83A@icLu#Yo z6RO3U5PnrdFN{UgJzRRe9o6&f@9VE?@V!z2AvWle0B!d28fUg+c1Py1AP&La=7qvs zhFlPl^_EK7C#bZ_`HkXmyrq(){p|P;G9E;Ef&6N13SC;2otFv zb}(W^-1JDUWQ3A)GmU=LdR6yy^dPXU5zhTit)i?3uATvYlv3t7|TGHXC<5$hN?BNvBMbT3~`hl!nUh_xuqF~|IzyBjjoN|r6 zqeL5EUrUF+Jvz8Gpq0%Lebku2aKU6ar~6h>zIVcy%2FcO3HyuD&_0{hZoJa`%j@dj zMeyt34$fEZ{daI9i2eamA47PxEpySZ<$R(<^ohv*vf>HG_UH3I^hZ}apAetZ8*PQ4+oy z*Gt+>lg3MFs-BR`caPHPO|c+h_GgQ?ED{sn{nlJP?z3^w=<|->Q^Dr1mKip%QHb4I z#aifxY*Z9Nt0YNF5tt;q$(4#;_6gMode$ z%iIpZ%KA5-F*jvWn+M*40Ll`c+n~*@CW%otqa$HsvYqv8pdUR#B;Qt7Xx1o&c=;i_ zyl4HY;aZ3OXh98zr_av}PqK=BWqplVJF*LJ_k&VV8uI-6<&TTW&@I*|n5lx8_C|j4 zEvbo*w_<|5V$u5zBQv9zZ$oWpb>#y$e7O4!g8HvNFV|=kpJ3ql`XwJdWS??R#b9uc z`T990ers_78Bu3g4LR6eADoN!k(2p%zBX>B(V_ayzm407ML`Hcj-;EQXXJRQL_;p% zC-owr(h#ZdM_~fNCO*wuGXv1F7qRw!3Ha6R1nv_aa9k?|MY2-iu3K|Kx#MZsy~ZVf9ah?3j@7PpUmQQmadA2B#W77kRZ*1 zdM4(Z;N1jOdNcRgd8PJOBN^0pM)Io>8!Q`Yvl-U}EQy1D(m|Z-^gpb;ld9OzrAN^q zW?qYLu~k#>Hq3ElvOGs)P7y%yY+v78)a=KA@H$TiLyyLSvRsKenTj&Ho$#h&K=2aZ-Gi?{{i>Msph;7VExn?y!3gaO*Lv zF4+EX`Zd24S*4}I@01<(7qJ(KQHG_(7^QwxJXjy4emviAHLCHvEXQOI<8f!bYq#k% zX8)-vWiimIG?6+>(au{E6ue-pRAh0>x3f9=s(0MKhF|X4uIT^uHKA&iy@MEXq+Xw- z97%3Wy>+_Dz=Fk54XOov`sayAx|k%iEFYe;Gol-ZtYbKXqW%Y5)CRe@3Iq6k?y<;r zL{|ansFY93Q%DTw(>M167WfGtfsJa`kQ}4PW4hcB)3;bnwwGd#7rHpGp|p{BC)jhy zQoLU4*6}%M30<9+-P~{zAe9}fY#Ya`^j|=PgGwZ?E(5!P0e!7Or5}Dt4+8U&b~9V= zw9oq)wC~#!b7{oH0!HS?^{hcc0;2==qry_1QO5f6Yb*a@$M#C<|%5bu|&qTJWeI=iW z>!XVYiXZZ1;$ga`ymX=e>|5Ma3@!3e5zG^k!~}CO^UubpHQC-n6?2O&kMX`xGNdpS z>-ApWC9d#nXJ%pDf@-$koeMt-wDA!asl_CtRX6W*o&EGh!Yc z8~oo5u>dEra*;o^y_!$w<)k!}jwDnyC+$%&P!5zpuQO4V3|jNL`DTMtfcLVRC=y+O zjbY9S8vD;e(g(UPMm`#5a#YSAFOMBP1{~snmINa-`q%qQa9afr)jw|CiL(jamVhi# zzUrRVhqiHJ06Gng*{r8tRE%>K{c_b~6dXKkNhi%Aw%O`-p=y${%M})`n7|@m8Rj20 zyfVUj)pnvmn!0{hGo`0}#r5AZJ9I*X9|b*b zb~sE=qQ~gv<8s@(G9F;3Tzy(7X@7KCk&cgAnR|!Lyn*0n)x}+Wp^vK@LADQ9T~GYG zpVbC?rbutQk_U$ZjsN28`J#80nXLn59}vD>*&f`0{P*?<{|=2<#3{g|o z`NJB&HJo@fALicDpRl5%F1L7ZUe)ATfpWkK$u6!mK$@T6ko;?)apuuBs>0U`Q`@H|RHM^kzky2pdhp&6%~nG(;W&J(WM)XXu6wU3XE+xoiIBtQYsg@6^M;660s@3fJ;TA~iZ+_gp9- zRu=h760Mnz*i+?H;g;Dq2VK^u1sE{(ecfQmv&$5#yn;mD1M46&)wmE6=_hG|^o`MeuVIoTkMcD}(EX5b! zzZn!C9P1Nfro;O!aEvZ+1n=hP%!W^VG}}GQjCLYuH+it zZ=x1|*UEo2B}OBE|3zm*SoaE22kyk~DOU3SA!6~gyyD?@<%(R^agBiW z=GA}wMm4A4e`a96H-l0Ih0;`Q1LIUDkU|u|*9Kr2M1JQT2Re`ip}rWF`p{}@uOlQ} z8s&i`k6|g+T5zyEQ^wh)`YJvw!NO<>PK~K+xo<#^_wnz8q-9w;Wj%kWlMMZJ82NiI z2Zp&8>2Gx`@>YwryjGS$2{G<+SP+XsA(05YZhw(6pDc~>g;}}nbT8KTdLbo{=H`FH zdNtRa1ufmKBRxTCf8@ViWc^VUe(Uy&R|m5g&EWvA@?orDmUY!XO1LL5q>ovDu33L9 z!!Ahrb7?Pg8&8`E*dNy7jM{|+O)8xXYG6WMn-|8#Z#`Mn6c_Q7jHZ)DP^H zhM5Z9hbA*cRb%q{d`S?d!@@?OF#a2TfmtC<8UWL~cJe8@&+s<}q~p{nt!&je_paPv zWnNuwo&S02s<7vZvucj#nvbxX6ij$k%BUvj)3Yr%xTZ%N)&(r*8k?C*Rg=G&l+fc_ zxhO)keW{2;m_bP(K6JOh_o#)WVXp^0M~z;4#a143Zbb9^T@bGld3K8Rbdyp7`PT<% zJ^rU7;+`P=5!YO)mhzRyVm)U6a&QW<+6|QowpnSOh-j@FVk#7YU5bj_qb6xUM}?+K(yk1B4&kYxu#b=al>#enU}j zgxYnH2-@>YK5Lk`v&zPty_YqodOAZ^bI3r5+G5Q%cMuV~;uf6SGhmq3pq61fc$D4X zjn?9q5-#cy0)Cb}9@|fvg(93hO7F?1|4`R{J?c_V83ihxRZ3mh_rfsh(y){9xmuG* zhefs@+{}ePt5w$1Ro#TvUda1bx2C_>_2@-(xE!CnHoNj~Q4&KNRvM|1NBjBAg&xkP zcGQl=xv_~C_9emXtm5fIFZy!NFojbKc_+8&+Mtq1#c1T zfqe$xRIbo5X6o+{>oMQGoU0GdDB}W?&cxV9O!JOBl0`*6B;(cjgx>2+#G5$(FE@r zB|0oOO8d-$j=4X_FOHSO<9+9*F~YzzOk!qlg|DRlH;pAhuwy+X=c{vX{hZNoLAmk$ z110Nw+|&ClLZpv;C*0dqQ_~SPj6B^p+)D>X-|iYw66!}J7w=>O#LAPir?wGiHifMI zHhX%_kl};mHSilJfOsyXJv|m!oIyoOPuY`RiW8Ghb>}ZEePxpT?RtMYuJ8S`Wx8aM zp9b(goY^g;QEVqGb|9`U5pO0ir3M5V=n%OafPj*mjpy%2=%@rdv9EmV>5~?X6~`e| zHOkD<&hml(h>@y&nApD02_moiiFI>K?+R2QU6!T2A#sLWuN#yecV7m-@rZy=a!sE% zL86?Ht#ds6P{FRVI)He9f6=8ZBC??1YPV!tG6dA5qb=O4W#jm6#d44PSMO<+#Nalr zK5tcS@AD=^HDRaXHC8^9OOV%U&yZRQ!(+HYBg&0MurUz` zy?Ip+bHnm&z->YbZc5#h6e<+RE8s_r*K_J81M4~)YBgnZqNh8tCR=7pZyua0>*Ai# zwgp-i^>*l&Cd=^|wLdZyVyqQQr?>uUsS4;3E?$5)cHQM)$FO|96zQ;ihMtNpT9l&JXI+3@F5OEjv*k~l{Q$5 z|J<0cItO4+?6KrYgoHTFBvLMAt>*lGIXvdle(iaRz1>if+LLP5`8rnt%Ql$EjdWb) zBzxBM$FPj>qx61ShD9fcm_CoVS^luqr<@jQ7*mpQmC2Ea?XoIHAL*WQGa(RKgxmXz zWYhD6BU|VM`0_QKU0_=M_(w`pDa|=c7s;o86d!B7Bl>}5c60$Z@Js&@R7E1=NME$q z?WGWVb%etH*SW>Zo6W+_rco=PgZ%X2;oV!lhT6DEH@ma>d7U6cUnj?6J;(m}+ z2HgK2zJ};kG!U)d75(q*u%PS(TcyQ%$-uNt0p|wFQpGU9$izeFBvUWVF6r?+MpB)-BG` zL%*S4ea(%1Q+i9>~=3@O{p@5=X-zMIk|-1l?kySL6p=Ss)CuB8!1 z_FfZJ?rk*^ek#*;8#`8)3a~g^b{a^ocpepsQG0* z+mWE+M06`-`v&Ep>ezk6m_qa}%GYhWw1265<5U3Z?$xAn*ZKR$|4=2ZmiFK^N(YN`C%g}aq%@50c4%_?24A){ zRM_~1eIL1jVO^!X3ZHeUKIHn{$^AapZzKsuZ1$2}p7TMnnMi?_#z|(K9^7F;vkD|p z49@xwqMO^~*Z^Ti#(e%l(yvV@(Ca!hVrPbtFTiU-zz_c=`eF>Q(`1=g_cw3=Z;zFM5wpCB(9(a15&Jv%k{Mt+NfC=J$spfrb~ zTJr*>!jm5dM&GYU!m(knrzg7E>OwDA9+15)Abr%znbNnT;t5D0O_>|>RQGN(^h4yU z?*HmAPewi1X3NEO^Xf_UW-VxX&ueSzs2}P>Ac>Lk&ouNB=^Pecp!~vn60j_gTQk&G&Y@7{wZi`rSowau@-$Rb(Y27m6=;*{FHf6+i|FrUK z5`aJ6=FYG5H$;+D2naC_A?|zCe;0XT)nOW|!s7o-fFdIpW(o)?ztMb{@|uPBy9}?# zw7`ToJk9rzY)FSqcrq<(?9FgaaUBwPPpNWjq|QkUk+EWoC8~&GxGPKL5!5r)EYuiY zYaJG90RQBB>fs;b?i&lT3#`*$flL{Twa8*B^X0TsRZ1yYQ}&~e*SOUk&5ki|q}8DFBAM83n>< z^(Vhs0(@&&N0LqBZ@z9toCMacwiddtJM|S-Ai*0~4O`k8nLEH8n8AC#&Z%$o&SQS* zzyYV*TvS(k(nRQwvpwNVIp{npn_tj9kBARwf`?pp6k{zH$bwDcyP4a0lms(!D08f> zn-DL-Lyyi1MYq$sYe~T6cY=jz{OhqoKGO^X>-3@6!Hs^_Y)#cRs{^t*`m1RrC<^^4~GW9s0WDx9Kjbj6^7*?^pgw z5#W5=3$rE@pOxtS_^2 zaH7_5blYv&>iew37;dQv78OqsgXtN_UspG4 zlE%e@Z5Ag*h|_raLf!46>KuA%8W8#WaJ!W;*ii`e(`%vpzV$NQq|xY?&41!mSOq4q zI+Pr?zPO{z9z#-oY`WRNnEqLp41EYWzFc;0aDR&sI}yQo20L&n1XX~<>KwAn0-I1? zF6}_hUnpx|j(Yv%t+pqdUt%5wEwfLi57-z|0fSz!nczok=M~+#$E^OA*GZ-2h$)50 zhG(Qgdu#=<4I z|J59jZ-iZ5RqzK2lK&wy#U)AqNPbi0mLWNWr3WRpFE}>YCwIo!p+TZzgvN^my;nAz zit8PZGKnbBK}@kGs`|9Ja`${BDLj%%ZF1|Ws8Y}MTJ5Vn8q>8la0htoY5p)k52i3s z7-W5W8bl?8!698Bf*u!ftD+u3U8E{fcY=2=APsxs33D2X+Ks&=%70Nep8mLS+7vT- z#1cOjt-bJv^Xr?zrj=4pYpFgJO0dU?vi7snjPd#P+r3vn8)4&>iKtGUYfF~!L}bpL z7^xpu^8TB!g(CJJX1rZ1Q3Vu#+V0;}Sbl&%J#o$4bL_?UG6J{=RF)~A zdV$?bjfVyQzDajk>%nXJostxQPxKd|5ttek5#qi~|J%X94)#s97^2!hzp@9a6lM)F z2+R-kBQ6q<)+Wj%0>X<;TY_BM*-sz&qpRTUU(igqo4#OL*1*D^NP0aooYA6gj|-|{ zdq=|Iq04B{k{UT{{%@C#X({nH6(OI?PDk7;=YoqJP&r3fE5qLZ8lTQF~xzR?^+^FMY#!OXo>vBWfqt!yHaCz$$=Z$ohfKXI;r9)Q_T2p&TH}mO*VePrOYdg5+`9|F zb?%RuN9jXuk}nwI=OKWsQbGa3OAF<%r|SOW5b$CA$nut!OfM9ctqH;GoZ zK7uOUjJlQ|Xu1Jh?4#!+goA~4czxr{{Gn`gmQLiu_ma{SILB1lT%1Yt%^l2Dm;`lF zvay~`{_zNfi!7mGbi=3PRoYVhLiUCFm*nm97H4lLQeYu^R@cP8B@}qRi{k58?@^F) z2xDv&0q&V&Rm1Eig=S%{xbsKF8$)23N|p6OCoqpnGTsm52e9}A4a6gPY~KG?#u&=Y z0>vSWey7WVQObC~Qd`*OSp~`hS0eAErwyQfX~D&A<5X7|6yQ^a~s{Ln)%gQ;@im z1|Y07_p>z#QkJKWYI_n*M)s5@zuPe(*)97&2i1AR220qJfI4k&BlpsVRLLiT#13;8 z?qy31EU&|I3g*$+Oc!bYcorjf>FLb{iO`H;lYl>;0KZ*k_MYS2il~;>OpX!If>Kt2 z=m`)b+;>yN3ccQdLJVFq|CZkaNKn#(jm{d{?$cD@pI=_#P?}g^_uhKU`w=Fu9mqmU zIxlIt4u2m-buvy^JZL7+wjBr__TvDQm)7Fq{tr>J_1(-ls24+zFTY*?{An{9 z3^0VI<;CELCFT41@CJX;*{*DIUwmzMYH6rQcBwyHJzMxr!!tn+=~E1R)8$y#bzcor ztDxe6T`DbGm_f48AXJ+Dx!3Fy-fHzjT^S{A!E|3oCHwTMYY9H7{v${53iA0=J>V<4||GX{1v-g~vp4JG3I3fUD1 z!)Jt;V6u-ut3a;$S3%aLR%}%C7CKH9X*DbrKXMjUV20;1$M|(k<+b88g7 z5jUtvAiC-|XTk8g62?+R&YuVS#i{6Vk9u~1VeQ>+i%EeUZVHV0LrahLRpRDUAd8vN z>M`){fY9AW1zLo=sIocn|F#Iv{omkTNHL7Tf3yxf2>S8tJ4?K^2kfK7DLJ_9>MicQ zZ*qN<`4lq6>qYw@@C+g2OyHIQ8?`BE{TJYtUg1+w-O|(@)388fY? zMG*_bD2|c1LrL{F*~@B_am|ukH37zq55tNJUCV4Kv_~&z0p%~APd~l@dn>RX8T@CI zz!WW!H%h0Hv(t}bo$Nl)CM|3*w1wYAcpO`rEP!cI&|m;9<;M=$v{SE?S_P< zT_#+1C2e`Lcg3m=+ysbN7r0-z`(>GbpkAA6W>&dZ!~_ zZ~}11&^5?XvNxj3s0R)@$H*dq{SY7Zd6G%n_?J2BsChn;1-e)lCgw9MK85f_&s?f7 z^%%v~V8LKX(s#gdNkF#NUa@2U7hbtgXzNha70ns+tyHU0f#B+jSw(GpDBlRKa$=v; zGv?vxajp!48_k=dN2f;#$E-vIaS*S{1d>fc>eaZK3%%5__F@I$r zxzn_GInp7c_MSHST=NMBx{QI(e-+a=mG2K!{C#UKzsCy}brq$P#z|ONzGnWmSBM`C z2H`CkMUP3w%$>(`EDRMaNySul+_VaNd(Rab^LHhrzIAH>O5ZSYlhg)zfL6AwVI#Dq zn>N!{Y_-%8hKOTI5p%HPFq^6pbVo3MNW>?SZQ)TR%qSRsNoAib<{k^GNq!gP&c9b? zz%4iFrEBZE>;H9<8wBny5cxQg4MM8qqnF|$M6`&(7aIwJalQ)5MdQ)iw+{rJQ=XI^ z9mn^~W}rS+n77v_XEuWbDq3%Di3q19FA!*e8?NNn-;{qZI2u?OsgfQ_AYbw0Wt z3-pn2?h8{GcZz3FfkPTPMA}!Kg&dxCe18>&-El=Ma@Os-9Fn~Qaa>u9~slQ z_mQkmVc)kNI#Y$-esWY2uO7aTx7HDVv18sG*FZ~`Kf&2t`K1wh9g{&E9FA|ThGmW0)S#Ax<(j?Gb{;);trG{a(zA4N^FY{@~Jt%lo@1^89&~n28~OzPTnN zy&bBZM~$zg+ybm-`032#Sa(1F*mIGXeGXYOTJ`W$jkx zYTv?OXsK`@$cFy_b5u58%zNGn` zQ8r`gH;AUM|29o=;27HxZJKC@OvE_@WG=g#buw~!AstArC}YsVMWUkuS(XylITav& zg|f%;y{;*I;3Jv(w-9!d6qeHZ!M=*eGafcD)8V1yFhPAG0hvB5=f6PDyzI=p!bi||1id! zn$fUloD68xQpyu<{N{Q%)}&7<@~>8|la~#!@TZb<7)*jtspaCl+z>X)01z%+k&F9C z>!cW1Zrl`_+|O&qej;X(S75yexjN z(bgr_d1?e;Jj^k$(l#6JXIfrj%Lm*%6y;HiddCHtzxLhMTl-sJsCop?1@gW7Obmr} z#tG3K_!;EVaptoW`J^rN{Ni0{!M5MprW}}wYOp~Lwx+duLiT%KW4^SlLqqbL;xF`T zS4}83_C!v5uUIR zcs7B`$}Ln06w4?$Sa;5lZjMn_JrV1FXdHDbKZwtD$mb1oo0i1$5inCE8( zsE9jCo|ohPN_Z7JX)c!dZDpCS))!Gb8M>WVOllmPH#x z8XkV^PzpVd*|$CcD29N|#sY9Lu~7X6QPa_kWIz>QD_TdF3I^sr2^%}2|Kd7wa}oAY zPeAn@$F5A|iNsr)ZyE3^J->Rwp#zgjYL_)KZHac%&-}YH?44JpEjd;p1!O&5YpOsF zNFgA#^K!1Vwb@}?;81(c#@e5j8;EO$WuHN`oF7%2Ppaj{Tul{%ihEmzMg;x81yNb1 zcG%=|yMXaydfxx(9>AA*@w?nSeb>(xx;lPijWGg|_U_jR2AS$AetD1|#tud(S3hDz z%=kF8M4TD$b#^O-{(Ji7W`jd#&=oSeOo+4-QSckBD1PZDw9hXRo6ofh`ZDE}FVr!S zJ_HToWB}qrky_f_3V7v>q1MfMsLqy}$GKw~&dOrS8Ibj^MkrD@s0VD^q6X=w@tMwi zIEDct4bY)VL_3_0oohVNbu6@2OJm&U*MJYW6Yv2qzs_K78t*jzm&6gE5-q@+(m1jA z&>tcSUJX?{jsP(T_QwYa88Tr3?~dmOsR;Cgm`5EVN-{4-f$Beb(GJ@O{YO^p70wH1 zZ;b>5`7Gr^HLD(7P2Os9g~c*h!JB3k{EN_||kiS_EZ=vGsrNgCw#e zd*%8ht2?XwP!F}7?Zkvs!W+hjnYj(*fUJ!ZE{}Y!GlY2o$c%E2MtrLS;;^89V=p;a zUJqbu_Bva0q~emjR2L7}rm~9zGjtFs*b>kr$2G+rXLz>fa&d9C)};W?D=0a3){%U$ zYKx>vqnXp10eIj?#uQoFXtzYBeCfiMf=i|b*CfM}K*+~)HO63``@bxJ4^#(kxF#fH zCy(M)5J^n#0sb|VG!DD_8zM2YsExJ&wm|6s6^v;rkqe%jnh(me=NR$L?O*5awF}SK zNo1$S=MW!sG%~rz!_Wtnykp2JlHc{hHPmAs&TMVv9(B9IECZFobX0`T{}x`~iTFkk zqnS6LjDmvl-W7zskrHEcHBU5?lqQ7^CjLF<8aa2c4*l3kOAo~dapt{AgbUH9jD2S5 z4{9@BXZFs^?`fVz`e9{M!sMZ`xiFy2fMBYaP17x8OMpfyCkqTTv5?Lki(A?G$CRe# zOdSoM40)A7botXcoXSjD{F1d=o=ghymtB<$gBjc5^ua6<{yB~(8OpFzf&Q`<$dhb~ zV+N{P*jLfk=bgD+ecn2gKed&2m=xR7J;{HkqU&%OtSK0L?VD5B}bgc;VT{* zWW~0X@h@J?-ahJ4`fNf({}1a=DgE1s7lSS)0@5<#=#Yx`3`VbqHUZnZ#%oqgm=0(4 zV+Z}&Q` z7@aH!RAd7^CL#iP=LG3cL%H{iD)Tal8w`7@;9oFk1$rA)*uaqu2h_$Kp-Ta+YNa*I zeU<9Q$c3id~`-cBub~(^_fzmTips!{Kpri3YfDUiTGSK|~VHjj3M$={B=SNY| zL)m5Fa2Zed!f=(Wh)fASEv=~>IhdCzsmBW~8iPiFBJi5WQcb1eXPTA=tel7{i^n~GWQEgCZp@xv;^kgS(%hVu%G70L7>R=nNM2uq8_a8T0f*5SPTMK^ zOURaN+y%BR4%v6$D+eQ3FFAgso%ZzeJn8-u5o?D0sm(;2H0m79jCb6GM2&)5H5c@UHQIkbEN*T4kWBB_oS_H%v$(xHFqC1YK-A|aAxG`bK zC_@K~;v~Bc_1$5>ueNv^j^|fY_6HP>LHa+$5E^P` z?>YX}-`iN4ce_-MPkYf3j>&9f=;@{RpRs-x4UZpnVrD1#wM4SUe<3BXcY1MevRarB z)_XWKa@%aAz?;q#DFE*X(2n5)Z|U6iM@$)C%1#*%cpMqOFit^{CmuFSXM*hCuC@O0 zT(FlL&XWl^403B%)74}a)-=14V92%iEgI~H;BEsHs*e(PG?q-}Ulckw2q@AK{BMk6 zDHk32K^x&Bfnd5n>M@mx7JLoK={ZaZmaA^LBiyA@b*?LtL$b*rYf(Tv-d~M5T<|<$ zP>MLf#aS_*yGoEd{RYS)%UKs^G8OmK zT?)BD@LU5j)Ba7lxR=*|HU1=dF0ODB8t|(F;8}VB9Jif91K-OZU@#DrDD?p|Zeu3- zzUx8-tvFl$y-d@W$+a5Tm@C0>qPnmNO=NBOZKmTuJ-*}jeha=&y!GEO_%xrRv7uc_2f{U(2zvi_4EBP9{*>EB(D6!RqTcUq5u9Lt1t zb=6x~IB=E!r2*gKO#4QIY9%9RDT0BctT_fNqgG@^YM!zVQ%1)M`)-PA)sQRo@AK@c z!1k)_GdkL*VE>gCIr{SgOvfMi9Y)lT z#8$-~qUR%fBsU-z{+&G1}2{>`h z%Lt)1&(3U2O-#Bdo_38ZYI-Ik`x=?(a|dO@P)j7u19hKjvGeTgSk}qIsfG`G51sgf z>0DIlFt}wZk`xNy8Dfy{=bh8s@HwCNX>Jc&z62I0rE`jVBx%C|7f27wi1=f}th}>Z zMu}93bNdS$J&H6#;@j_u^TRx|MR_=DY)i$8(Bq-%#8DRhU7;O`nXlewSQ&#uz0+k&oALo zef$eGpVP$2&mtQ&tP$Tlz+hcy87%H^uwZjdmTdZs-5vM*C)zRJNciU>L9_VO{-qOf zX=a2s2aJSZziiG$6zNk>(!24fh)c(Gt zqzDL5{=(a6r1H>-zAwM(DbSOY*H5ol?Dr6oW*7|klVsh(vvSJK?{nQAm+r#5ApW>S z%thSj?iVdtWPfI1N+xFnEF`zNA3jEMC+V^ee^|anRO|2QwJi~-H=Q}ytZH25S&A6` z5-3hyySYc5t1}`UMcVbut<8a&c~31(zo_psWx%Jhn439)kYs|Zzn+(7&Z+nd`6HJe zQ?hBTFr^CC)TT)YQDLoESt?_^S||QGY6?lOI26*y)@VZNTJAeBs_amkp5`UYgqIQ|1(0}Z3kM=*%rurHgk^3ABImfy`GhM|YhD}Ir zYFTgFKuMESdC@iz2`3Scfcyd0cLBMir(^-}P?_EKZJJ^p|)dB+fVOI4`C z(3|Jou*poVR@s`Tb=8&-P3o4h3~tl==HO5|1?2lEc?=;+1ojgfL5;5uiIetqrb{@W>!JuI4}{vz^I3a`|;^7KK@)30u_WV&Uc@CzvQH1MUgN-cKN>~Ln?!oOEQ;fRz*)>A@y}V_77o>zxMd^cPL8A zxfUeB6Mh}+AS45w$WJVI?ZRT@wJX)kklCbBI>zVq@4c$E7?AnZB#+Ns5qtv53$jl| z+Y(cbkyu&myUdNvmEkc>FUzGtUu4fp7Y(MF+Fd5Mz(G)yaZM~3kXE58{&vz`uW?+} z6A~R{XX@ympV}yVhXu@k#^$IKTwgS0_&em=s6lqqH!2kD>V5BV@|#~KB5U~*svH9T)@|y zalwiP<^f38s1WP^dV2eb_wEZ5J{;#7YZ8Jc;HR&?*`H`q+nF+dbn}ZHKnTP+qm=WV zN|d=?_J!ak`a61OD!WE`yk&4%%pZF`))rlBV8e=c8PcC>f<$7*GyDh1zD|3@R;n;M zm%WHy1XN(YmEH5Z77NVN^qk||QgfiNR+?_^`)2)l0*-Rzw(+|u*-E!Sj;+F)(B<#W zoX<4Rf|wVs(XB*C|DW)}hsqTG&AfT68s4)Wqiz$@|Lm$s0e0(g)u}xb7kUxVZ{1hg zzi@vWrNpq@jN97Ej?HDgbTF?-!}uL#Ix0xP_I>RXmaO^Dwk7X7d^i%u?}Jgm!>3v) zYR{ZLdhpANX57$J5G#pi5&G%KPj4k$p~GPME$3hEVhSvFzZW6jPuKFv10PG&5VJy% z@_BnRLUOvo{T34bx%#eP!+0I}M}$%i*g9KDpMWve8VCD)+wPnd2tIO72zaV<&5k)CBwMDmC2GVd zq}C#z_FK@Uqk%fT>d4lUFKq+N+Ydm}KYSDS>E9^9K==!5$Rb8#fl%Q7xN{bI#u{Zo zvl+!4EbSQLJ(6FqUz2G`_TboI=6xNDpqt(%rsNF{Ls-eAfW5ECI_9mNU!9a36Rw;W zP3OXEhAPrMXO_AYtIVqVN?-dhhRL#Xi7v%Q2v3c|5%^oJzRR+^Pq?nws$YFrhfe_zB7 zcjlzHu7vJy9YjR~c1wvJSjQ+Mk*JTCH_Ot-KBwatC|RD*!Zp63X@rtE$WiNtuSaP<_w zX!%4sU$R*N?=u%g1490h5b~9xn=aUM>Xs;y@CBS}u+t8dXRc^ywwc%IcmPVk}-oHtD)E&Dm< zStivAfLqlo7j)Sfwp3NQCpn@>a}9fg8h&o)Q|>zUQ4o$LS8^vxk~DrXmxeG3l^?}K z$H1Fjo77>b?7f1-7XpGclLJw=EWj+q;y8)JM6uf-s&dvkQ>_o>`mGe>Ei3cNAx5Sd ztLD^LpV9ikWH#`se6bEYB6FK_*M7`^h$(h7BxMt`UAUuAqYjOSX`66|2z0X-t z(&a`7HG)L`U5N(B{BHmXx|)e&bT*lUk+Kq0)rcy*UI~|T{`lj^BX|X)uY!q{!`84O zl?x#^kJd!$lSb4Rx0!CyW{zJietxd{(+!>$i=&0Dl)m*^&%YzPuUipqxQc-P!Pm_Pa0 zh|^!*+v{~e27~ImwTFeKamn_8QtgW`P`1J44p9CZOUuBiWPUmi9PkUIr_LRp5hy|A zH{jQwNVrCsn>GJ?1c!Z=j0q_+pLEasxGtItn}}|bi4gTvgg9NIG)0q^Zz&cSy{Mu& zWl$(AjYUgT8hU^IqSNlkXB@!RFiJ%U}dR|H6$f(p4*W{SqaU4L|JAyuR%K<2dp#aiD|p z{aM(p$uo7nMG%_Puv-@Q} z__3f~3#P!=flEQUNSMTz-YgDU_4tDW!&_U7SQmbn>1H_khG({d<8M~yB(&TvBPr5O zw2aW;v`cY=EEr|FgQ0PAAIFa%TNV0haoZ#WVH0&tL&DDi^hI>COs1FVxsEb3e$Lo! zA{ULxxk7A--h@iM&yPV0&*Mi*&Ii&-iTbS?#zTUG+T z)(@AFwG^L7%ax9SNB2yG!CJ&>Wq|w?QJ*w1h6zL%9cp8?#{EZs62R>DJFx^}GP)4x zEh0tL6c*J(;KzS6KRjx2@!`^B1wuN%`0%x4tCE`wkbvt~Fj0356fCs96}?qdE_)KM`p#++^5rMr z?7o}j4-`H;ZfV(*zSbpe-9RQ{q1MjS0ZhcuowT?%&ygSl!HE`3W8s=x1IbTb%Xmr| zPgl?>=)*tRnC-G(-2eK=bSRI=E;+J)FBZK=tKYLAPL0o}TlxGT{2Aw2N{McO4u42~ z`zdWxu~I-T&LVCaZf2ab=BGHGn8SF^jY-GazNW7OubU+7>2dC|M%e#3jDcDNwBj$@QyrD z@HW@VeGc<;UhjpyVak_{9+KHp+enPN4QAwI^s%Uh4@csdcOkaL+o|t&cJR%$RhGa! zBt=%9-wwTI&!68-&po<@I1si)*pF;UcsS=h1UNqL9ywZ);4}rn-j5WHB4Z>DZ5q+s zkVxN{D|Vp&T^{HP=%@@sKyKio86}M8QR+}`milv7GqImG-cL4kU|WG#7vSBmBPGjm zi5>m~NE?ZPK~9lhL~qH!qd(*L{uPZr$bFO@{c52jFMVh3Hi5fxBWr^Ype48XmX4JPm*mKly z*gQC^l=L$#7IH0N)rH3&v)P6%@n6iYI46EA7-sjbi;4_?ReggFNxC?l^fv}Fj)dE- z!9nHHi(f!et4V2RBx1gMnch}Ji(UNB&jZ-ma7&o-0B(P)&$G{AdC^thAk8Ll$Oi*2L-6A=yw`?!uS0i0-GYJa(%oohLVg-%FXti1WwSI~JxY%RL{SMt^- z2J!m7&zMv)>`~^zShcGo&gMrQH*}$jqD9Zws@Ez_K^N`+LHT@@@*?-8G<)QM>xRG(teAp+@w=KCaA0o-bI_otgl zMm++Ii)Xh7TsgnOC73PKC2i2vHR@-ga9kwPl|K*L#YZ!TMoEF%b9>ICCb^5}M)~8q z#)^lUK@BxoGffSdHSgnz)KEYg`5E{bx8XKBBCY!P`4~Y*l08^Zwpq zo(@;kV?q+BfkNlwkXZIkR%)Wb^e)>t>3ZHd<)8js-+ZtaDM&I#wrn>3kRcTFccou< z^vmBWzD1|&ht)Gl|J_g?lsm_l+%QL`a(@aoEFtnqi2{ZjPJA0{rBz!R-2T|@$ZLU@ z52jpWZhR;8ae{~G=0O-|(cDcRQrQ-N;@-i$r3NmiLm19Mir1FdYiF!6^v%m#>jLR7 zb#z+GXz1aBv_6gAfm&r*-liGkLO&TehQSwG@-d&)#kXRvz55%Ag{56LZbwR3oJX-o zE2%6}oSuxbWJ>2qm2sgT$rv%UbYP`y4!D>W-4%!sn61jnVP!h6{}g4bU!cmsLUJa` zEe*eH2lE_a+#$B@W)qMh0`@B9Tu?=MFzDn*R!D+|phtUbd))xJ+0&ZMt1M&GK&Nua z_;rr{Ydx^GaRkR1PE)5vZ1?QGmt0@U!1(^50ujwfBvT6U2UoeC6%+g_QZHWm&vnw) zj+c@Wg8rco*xtUAPbQY>6J{vK+f5K#r0l7F;{V6hS2#qmhW*kkozl`JNJ)1sDY4Qa zp&%(ONVjx{q`*qIG}7HjhjfSJ(#;*t@qFLC_fMFa_kH45BeG=%%fSjub({+CO_Qyt zRa|&?t;JUb>noq<4X9BoC%3GRAQ9PhTpX;W>z@$80tcQr>`Z&E94WiY!nT>cj-^Y| zW0u~Uy+qu`PHjr_DxiOfJIIk&+xAjX*!^gpwdV6AP4w5t=4{4d4_Tj*xGV)tz4ms^ z*3P24R)2z532A-tt&nVLh1F4%&2?O^&}s9AH$J=NFfNmel3eqc_*#^4^zLQKDSXzQ z?l>o41ml#D2!S9%HsbzgNLE3*irzLR3Y8B$34vu7o%<)uniaiJNW*5PO&}_{GV%1T zA)gejX-m5)L{A7Ljsb0Y^Ky|LL?@=T?_XnRx;Of%%B#O~iXz-;oc6>kK(D((C3ckc zi({E-NRL92R1PEEi)w!YImcr`2F?uvFWEn1@7H#Rx#s9*P3%ufrDdAW)3jvV=f*N}n86VhD!(@KqshXMvynIdV6A7uiS)SPcSriR zgZsjd38CR(*@HA}i8O7a$q)#up1szeZZ}@jefq>|7}iu)&!uoRwE{TsCY)$Q;T3-` zHV5&SPV<+x1+A`!;)W6R_tZ%(>~0Slxb0UJ z@pRmSFZB**>E`6bmELRRXJE=?P7}IO@S=CFk|(rYLZ&6wK@;FxGd}j83!V{kmQKK2 zW*7{B0RFx{z%X_m9lh|k{o^7@MZrC;?kDWf#peN1u($RK#i7SRm~WR(*h< z5{sS4cgwfORC_x! z2hg@upRmn+CMM6jfL}kWo+V$eOx~i&(0=;R9!~=!2n!rWJROmQl7B7q<(gHYw;VRj z{^AijYcTrIUYg3s+#5`wZ;OZ4E(7ZeRjmoQFFK6jQ4o5>=S!(jZ86MY0Vny*@cNzU%FoZ!j{vFccq} zlx~fIh|&AltW8lQi$S*D3SBYsmi%>FwgU(A({Bw?V@v008JNv_XYb^NwK4JC-F*nC zS3}A5;vg$rBE8fKKTmSomc|Ulzw@+0=E>^tE-|^!ok{`nEBlBlO4GN5bgMS;lDD@V z8|C}6)OSICAhJ!M|B!7yr(Z++xBZxUBU$*7VB#sJJ_iS`t$mPd)cWCQO@N%00onx5 zh^qVex(m7#ejgisNFJ1O_bS8I&k`MLdFZ&5_y-cXP2(A@ZUxB zo-Htw;NdrFUuupip^S-mA5KX@npM?X{aDlcEQpvb-_4wv!8~CLO=qSBw>~CVyq_W1 zOXg67~aAM-D-JCMJm>5x2G98N{0)Z8vtjT7ashW);0%xqq+ZJ*?WqNqLxFRv&xebeIk zxagS&(Pb`vkj6@4O~~P&E?ovmcSYZ#DPOx*$_6?fC~@OYQo*MJPU*jMr1Eo4EO&m^!uQC$zppf$7>nuSixB}*l+<3@g1v7I9pmvUlj=ZyC?%2pwN_u_27 zGVH}n+(dlRF8ZaDAHlt>Q+RYYB!0=A3yq*j#ajxts9t|+cHkuun3#do{dv4(??@pZ zK5WDE?eMslr+-m&^Uuiyq&HvU% zG=bd!-EniE+wPiz4qInSnST+4mizj4VX%)$D$s0WiQqTzt)EDOS0V^^y^fq+Uo$}6 zpg+$ucX6yA>M8Bv!^lyT{s@;w2{C@h2ptV>y`jW6!S*Qk@TVWm<7e9TmO4Nol@hnC z{}Kn<^oz59(|ViK3UxSgEHcy+h{+t~8Sy_3G?x^pz9F-pP-Twe+iM(xWA%%>@)wQa zE3gXO#$h6ekcU$w9$G5pB8|XINFoX)jYN{Ub~8ejzQZNbx@bC*;uRa(Sn;aKk*E0N zSyeagKM$vE>@sA-`L2sv4li=|w+s7L^*BQBP5%Ls_wkb|z$nSIhD^nEncaWV}%gT5ASy%BvVCTDBQU>QuH&Uq450`|na~ z&Dg70KhJ>oIXBaq!=VoBh~iuKelw*}#D=o6qFYCBRncJ-RKiDsW9V4J1+#UJs>!?_ z*!!~;h}aY&NHXi|P{&QhV{mzf{i5^CjYeu*9K!1sT@>-VV>${^cg!O9kU@__Te$PR zTpzQHR(tps-P*Ugi?v?D39S0qwxzm=b2T=lAO)7IZ2>|M5XjpI>|6I1RXFA|f*Y59QO-|mRZ$=^y45#) zXE>=71SQ``AN@5UMmx$Ep#qgfSc>Pmx^Lg;F?kqz*Mmg31@Y*F!=)Ql3E%3E-CV4%==15oAB1#m_`%ZJOEYo>sgBaRlB*#CYkr} z+5x@`w2PT8VYpu;Aqgbfyx-KiD_E?WEuN2Kf5tw(T<56r;fn6&O|3kxty#{r$vWj1 zvEoY$v1~Wtb9!d{ti~f@G?h>}&zVRAF+)uU_EH1@meA-0h8c|Lz*xgLB)jiZD7Nm) zSsRWBM^9Ye!tc2=lNZ@ygEqW+NdmYI(fxudT3p|w8)sXa>=f2@n1a&vB;a_NGAXa9 zuC^y%6zD(^2-(PRif2l`RGE)RwId)K|M9S6eNPy!`1*AKW%OlG^X-M?@5&EZt`+Zp zIIJ@Y09h)SlsveUAMbosBMC&0I%UFJ0^iGoOTB+DGrK#sCGzQtljkCbd`&?$H9x%? zjDN8nZDxJx>UaBiMUm}jsC-obdhq$Las4={}OxW7|NIhd>U)2n$nAA7u9W=go& zy+h7yL&kY?dpA2~KlAOg#fR~BeY;i+BUjVMyLdNT0s`qI?mNSfQOxBp<8c3W8^4GM zUwzY1JUpRw@pw35Gg0rexTNZ<4b%Q_a;hE8MsC@{@jjd#1U?ocI-fBa*}c^bI;dt1 z#C7wd4)CEnbcDqEX-+v3%Qf+Z=yD#w*xR4MalC${I(`>`f8DOEeh!~CYBXo=HvaaU zKAj(e=G?l;!GHv;`>7Q`pVnju!e2k6&zL;gF77 zg!cDmp6wW4tE&!P&d0>nn6q9=2CH(lfGqQG2(|aHH76{7%AqgSk}#o^?ipYte@Abv zqVz>y7$&1IKs5%)taEPCK<^Kk2*p{% zj_=%uRhc6(RQ&Pb!!Fa~vNXz`y_yW+oRABX`^+t9^5961LnY0er0E|$7o&UOIdtQZ zKVIMNa=Og+V2@i0@!6Se<+mTj)v2?69zi08Nw(xv#{!4ek!0lcoXF?Arh(6@tLl$X zTmY4K0%o-MG@_96$MUjof=(z6;?wGu!77;-WoEK1R_A0_BQfvnJ-*GaeA(``BFpNO z6GL70dPEfAwuj@fn*%3G=6v{aviWYi%%BZTC2_^aCvPMjd15IOdvh=`z@YJST3+Xd z#@1+iKo$(~^7Q>AEG~DstsX|O@*sQJUL}p1-?$8mWrzG!W9anQ#W&Bw04hz^FZ?(E zRT?S;@LS`-bgSi9oaUgU3#s*wh6f|R7iBw6ap((?t!I9|h{*-4H-wZWnU@}5R5b6qH0HJWT z44qM;B8C>L5C>ZaFIcB#amD{;hQ!7CK=jULr%*P}@e6%t((y5xM1=AjEt^K2f6vSL z{`geoGnDhAVTEsyYTsGKbES!VY#VJS<|LcN#r{i0%$YGFuS)53XN@HJ(Ju^5j%Tnj zl$!TMsv9q-`!kWXKJ*Rjy88U-hde~oNDKT~7IlHu+_ZFg%dt(#dxD`{!O2)DhH}X1 z9iK-oepglFAkNx)K9=|i+4H>19EW3g>CYfpNWuN8`gG(-;(S6*2UYr>_aSqJB&&CfMh!0#MgjA#WnG}ohVb)H4H z_m|>Xd)d)1sM0aS_2JA#$i6G5iXnB2=w(q2!=@4*toqeR9y`E0cYU3s*^SyJr!YfR zxSP}TG!eN8=5F*TN?{b6IcKCZL^ay~3!{Gmjdrk;jE9Cmyt-U5e+6K~ zsd8*-#HQnk(2EtSh&-PCdaVb(?5vyof<1BAJvg`ZH52mR@^&>NB3C@r0be@>eEcQx zoxKc~^&VovY73H9i?Rl}If2aTpGf6cr*YYTf;z@NOV1MMnckH6z=MCRBHJ~5LK|I%%tsl8fkc-=%IlvuC$F00w zjgCH&IC{N*D<6|Jn@>UR$m)BPQJ%ye2w&Vx^z6z|z$-^+zwyYfFKt3tj{N(EV3ZtQ4LP3Ug^9PBmx$%k_M3dWkFq;OQr0O9GqNu<6Doc2 zR(@E&n$_ZIx9Rr9vI-fUSN+myyUESCa#1SR@?F`*!Vx>Z!X68s z{}V(tyjNxIlsVX)iKwN3oM9Bz46=6&8fq4-bqN_Y&fwr_c~fNJ%?ERnpjz8m44_)Q zn#E#3Q(T`xCk+#mej$Nxr5Ddlr@q85g*DvhYENKab!y zvz=Ok+mr;oDdOg;wdYc@!?^g?dB6pRuB^_lV?ILS2||_+%!0CNsKiBIO>;MGh>cDv{58ilQ67;ynP|8=kv@ zDdGQeZ@?R-=0x=S&eH1YR;uWa!4^Rh-R{btTH|Qnxt^BqU4*PCL96k& zp5|>DPn4@J4h0y3`@prYa~yc7E;iXw2M=5r2|N`JqLCkS_|$)ULd$0C|Al@=9Ti8i z5EA!SIPDwUF!^Q#&j_W1f|gsbf19_pji} zBa6dTCsWPeo1=%K<+7M~P-Zr<&sVjuHstenK}Ol@PsmSHqqk;0Y;(Ixq=^$Hnc@!f3tRK;0OwuY@q_A-71S>Q_BWX z=^V52@aF4Q|8sKhFAu8Xh&a4wtFH1HkZpFNUZp!fD*vXmVV*WL?jEf zT6l1(hx|-_wH4*=+;jF^1AiRz5hTnAQ{bPB=%de#v-0vqQcKzNMK-eC+&6aKSj<1X z+axEv4>oM^BqgKm_pqT!K5fVBOXDxs8mEYEo&1>L15}!+GMQUHTqw;S=sM+v5#N8H zM;jG_e)L^ABHWNSGVyDknVBfqwM`4u2>NjMIqn>$iP3AIg z9H;|Cnwv}s#Qx4(9u73=zg=^2ZaD{TtrsS~&3Om1PWpm1{MhK3_|8rNG>Xr#H%^jsr@c zXA+O~l3i-fz4gV+}#0aB++t3-_Lg~EnlnxjeW1n~|md*nO~Z;bco zj5_SRGnv8fW>30VL?7#Y!B|Z38AaDQskcqWVlxe{8Rog7BfP);rH%8p4Jv0&JvYPSpFpi>eZpH#}D+k%DOmT6pv;qd0Tej zU4IPS&$$&~x7;0A>)hRH7rDNQpNWT_{Tj%;MP=45cq!ssT>iT}Yvb2l=D5ep>vN8BcFRLj4g&@>en0P)f1i1S74OUbF@Tn;hUjfA~!^obKs%EqlCa$E4AcQmzW| zVD$+B-DhjDTktKBW6ddwW#s+lHdy@spT3AuHaJ8$CL^jl>iwd3Ag$KZa;IkcF8 zPC)FTpnF2lu$!_r3VZmP&^`7LnFE77YNRugv$OEmS)r(U$m=Z#2nNh$C1yTH5*&2n zb$+gC5fd8yqRA$Bbi-|!$O1)kfI%E&uXKPb?j6R{)J;5dDzwR^L=Y7YhNF?xsvq&5 z$N$}Sbn#C^TD2m@!4Ftpfwpa+C{0zz3&@D~2kIm9} zNBt_nI)Kc}ILq-Wqje@7cfF~k#C~WQ&Fsir%)8`QMayecIQ)fDmIV@`_o8IC16Z_# z_kN5c-}05lM>556F8P{JGxirT${G*#O8bwLJ*dpwJ3kS>Ex^5C*Dl zd13BN+xfyHqDr2cDX*sF-=3QOab3x3r<}NQ=DZEqg`5PDd-6!dg1+%?(>>$aO-y2x z944!Sn-V=}u7iF|c~u(%;iiIGQi^AK-HKw(*V*9rH>@uL zKF^+2yB2fuvM+UmE3%hbne+nzEl>7YLf?yjDc_h~+!Zr57eqOGPR@dmH(jo;G^J0w z5u&xnmDXNx;3Nck3(L987uoI(SB#d}u4;>St{AmwT@o9yApo#WQh)i1;41tcd!Pfa>0s$qS6$~^%}D|#)?^LMXp#fA{*WBHgy ze20demElHD{+%$f=M8Do&77(0NvPrIul3LIcIKjtXE;2T*v|%%w8cbT{Bd7wsOt>e zoJGVm6dh51GPdE=|24*G- zl#I=D2Q5)Xy9E6DMi~^(^yo)aswe&Etx9?$DHHjd%KHv&x>WxOD?;wVu5dqXEq^hr z8KqE7iGfE}DoW@0ALHVV)JH|fH~9iD3~3>zBHkBOac-assm zTlC8xXU$XsPrd}EWRAXJHL7L6QH`hsw1}jx62zb0aMr*6Jlspx9|Yb}pEwNkOicfY zx^jLaycN!-WL~@S{nY|Vw8q!{M;TvxtY@K}q2J{RW<6n1R8&3K3jR6b58IV=c_ciF z?g*4B?mI^Ivv{I|DjP!It6N}1nW_cvL;*oxjXcbCxi6ST^zvBD3$ESd_kyHvJoN6T zRn4cy^~5cluT;PpsEJID%DCKMB3#ZkNs88%0jHL{qLqHbdDCXhQ=YKWAnsX)w_5y~ zHlGoML^k7@q1#Q=54mA8RV#ga5zq#X&ZCnxmK%5Q4Tc+v?VPO{y?8DRH9um4X7p-g z_=Db55bB`OHC!HEljImQY|s$SODHM1TzzU6p`uEEbRF8(cOTjrOk~5bYdts83bbnv_JM`g&^pTrDkGmFSFg88NB4 z;!`P4`(Qemi>1p~v5<>+0p)mDwxuOf)i&_L3$fjH8@pV77E$yrz!B zy${4rF+?qT#(_XG$v95WTmx1OGHg*p8s#sTl81SgP4sH9=wJBZtxzEi_3#aIzf{gJ z_v6B1?EdMrz0z*RmIa2bg$)fRvL-FBLAXy>pweM&-~-i#Q4P=IQS*1_e|R}JhXJGG zZcdV+AApJYi>Y1&)l!OotB(Ec!GZH`ZcI9T%?upqeFl5$dRkgXQ}G^I0^gtYXZx&_|3OnvVmYgz{bp1;qfPbxw(w?mq+rZs zb^aNvmua@gSOYOFkDiC@2L9g-=nPa}(UvCq$ek?eL$gEIW0I9K(^#%Xg8!lqjr6Ur zTGv&*J^>uBlmj;{HcZHh>`v&EHB#iR#gX9ZV=q?|ve7R$)dR%)Rj|droN|`dmsG#X zs|7jQ>dgfTvfm+O?@5>~4fC}%h4ft4(AY+I$&3o#>6?`ZfBmJp)4n>TtUyL3iK%i1 zW*(4D{h6G;(YBsH?}>m$SNbk!H(>F~A7ijXMmuR<321UlG4cL~)zuTn6 z@_J^BIdLE-wwj}Rkufp|RmxvUU0rV6W(dNiRad8A47QVpKw!cqdv zfZ4@t)vSCIdj7zHGr&P@AutwdY4XhxU z??F2u=D|fC7DQs68%~gF3laJnE41$LX93+tUZq3b!Y&p#&6JNbIAZvDsu#Pc=Ax$7 zdpTbrHCE4Nf4rlk_W=E}$}l;5RG7<`1@Batyco^%l9PDM1-QJwR2k=rdNS ziuW=#O(e(;nPheehK_*JaG|8oB(UsXR9-0*4v)l4+_#j6tfoN4 zMt75d8Ev@YlC3ZHmpJ3&SY-G>{4i4yivXS?7=a0!D>g&q(;5@ueZY4&5!=Fhe5mCN z-=En(nOkgn?Eo8`1u3M`B$~u6hH$I=gm1%;tvu~!W}E)O2H*?Gug;moq4RRY0ZYSF zbu3Y;4ZAhLl7NVr4T6u)54bXN;@RGs!NfZ5j{^Hp8+t>(L?{VBJia@ZXMT1^5F+`R zU9oJj8bzj*4l?*ghAM1lXH_5>HBteXn5243`HSEbLeOSVp3n9bOGd%R*>lrb7uepP zzWxv={w4q}li)!N``G+z>!&vpoQxo!%LTqNOp!Z?j0?XI_PJ@nZ&Z}#>a()$y_0sy z=%{8Vv7SgVwG#(=J?O=cnt1m=0DjRkgmY&mS&2{AK5;P>mwP-=T#jfJzZRsI^O?V( zoOWohzirw3@%aPyhOWGV!W+vfx4sU=&}Z+OdhINr^#aaVk0f|d*ZcmKd*jourK8RI z#5;4>cuhHc!eHi2sF#vb1p4Tw^+>eQG8=2<0O#vE)|H=J>&CxINbNh%XVe{Q9`B=@ zIb9Z9+b)i=6sA{4u-|OL_QpJ4f-QJ1qxu;Cr^?%hw{g-+nctbm(Yz=qj}I9Q%s8ik zYI}X*eH>HnqH#t@?ImzaE_Y;eeII=Jt5abZv!;oe>L38XY)w91jY4-rAn3AP!)I3= z5k}kLpytBI3Il~>)y4P6qvRK%kOUjABvJ@_j|tg`{2Ty)u^v#dBg%%iUWWVl$P@6b z&N(3!s2O4t9u$fM*Jb0?V^As!nWw*$vflj`Oy%hU^9yRmWVKGwfKK0xgk zQ*OTwm|OZia)(9tQ2%-{oUK~n6!Bv>Jta5HoY)8DN`sNfjx{+&^Q`pOU4m@FI>p9jf8`RoglYqa~9s$_S| zbw_G^gB}T3hzt%VJxaCqfQ;4}j>;n$h1j6pM!;V3dFyyXh)|OFA&taoRET+w64WH)-huchBqVKRQ&@{h2*vOvbnF zcku&6&7foRnBVCu^gf(Nee&NWOaLz{mKLR0>)DY-{)g z4Ow&#N7a#6^BB_V)oM7Lrlclhs{+Qn__ zcAe@v`N~5+@xjHuJxn$t@qH`^uz)0s{p=+;mVaw~AXjDR0k^lfL(_ET7w2kF?K;8n z$-aWHPexy^rF5+uPo;0H}Qb`Jq4TlIcb1>o`)3G3&~cFLaBYwL|*V7_zp> z*?&XDJQgU0hL~WL4hUBG(1PQ2|w^2R(Fg~n*hH# zP7^C+i~luUDU9)Z$VlCPo5m@d$i-Xdt7E5f56r-?)sm$s@~&i64k zv_ok4;0R;Ge8y6Ge!%PB_O9znpF;~=M>cTxzZj7R!jjb?9l zK7bWIA5}3e`&oMnt=GAd`E`T*Z!}u=n*TYBDm+nsdgklMckU;wgk0m1aXfkv+}?^HhP&gL76AJjsqV~_#u9}k901CQ>%;y*y^L`%3^G{wA=u++b_Ogn1jM$a!3 zA-9%wb6QS@qWJw4toOVem4ie!BA=(8=|rVX^hQy*=xBZMEj#w}A^ksW?|{lkYMTHS(G*ei)Ho z9+Mv5J6)LDpzG>`^rPQmhB3Ms2GQL+8$#w#em^3en^$Koy!_SA5v#U2G;~XG`@n(k zgJz>sB}&nG-D>iBn~!$kzWsVO?vJNj@*9-OTUOjBTUPkK7B}a@7&&LN=+Z3=TwoCI zMV~z;&OBo({Tt=;ZpJUuM?hmQ0ceGl=1Ka=t7q}6u13!g*5-X$`1MN2O{?ZNe=b_l z`ovVPf|+HbMyBp9WdfJ(<%Kih6l4wu5PwJg()u35XYujyh{k-#DDTZ0SH;+ZOQWb- z#w{orFa&-iw+wIKP_THN^V4`nb*JPUPyq3}bSPUpS|K>%C$OJfQW7xci-LYY;xc@ft?NgI zwxF)Bb&U%p;mRN~bjOb$*ch9)^RDAQmQY9t;wTFd=<;x1PF)aLLN`+h8LbhXX)jWl z0G1(UR2*4wI8W8Zd-v>%op~wXY$fj#de7i(#DbVY^y9`3ld!{*;B_g3FHF$6DvZpuIVkV z3PC58i!YN*lbS`5j#Fk%Yntt$;FlZBvCB+V7NP&fX&61}n=zsn{-ap-LH0I~GaMoF zhiw`r>1zbeBLP-c!UQXAQqpGvJHM` zG!hI8-+GVF4EUcv-FI1Egn>^20aLznm3Y(Pq<83OwKaKF&YUx5f`-eGomhKF$qnih zI*ejx-eikuGAgeK@D7~d3@D`PhrjgyHMqR7*5^PZO-%^wzE(pHCpm{D2oM`jqp5lM zz?RQGme1&Sx9*X%7DK_#&2Jykk4KW$r(o%7&c+7avc@vx2{Ux5R-|@Pq{C;s+=C?^DAdRg2e4-y2#8{BAYUN{ow6+@74s}1lT_)=I)&Ir=7mOqa-s-OXk4-^ zuko3esF$Zb3awNK2rz8LHf)^o9DJlm7Z}AY!LJ5^kbS~9#En)>81<`x7bm2N4T#Y# z8<1Zg*Pq{kJ_J}oqnrsT?coOQt0k_V^$79_*NBv`R%>I#1d{FkRB6uXqKS7!k4ezT zO?32m$5`yGl9n64Hx}yatzsHjae$;imT%@Q_07UBwO{hcrh#gwS$;R-qN&7qfQv=H z3^^Ds^2N*C+oa15U}VtnTU%ljc+2p4S`t9COD1N<)&7z)&k@qkHjpT9mgrlPz%hR+ zQJk%|$cJqKN>&B|p03B}RNX=6Zi`)$kl%9q1A$vG5|UM5s!jUP$_FUpW;u7Of7ETW zq%RAD9Z~`^KkF8Z)fSf8uXo-X3bt6cdmOt9c#r!GJ)w)(>aA=xA8Ac*pK5`oKhYj1 zdIC?S{^_R&NhbPU=@!MBh^{zmT_pIrEzwvxkbbq*qrLL02nwIA{r3lMT!Ojy8bSs- z7%`u`vejd1KNpfA1D6iqRY?GI?e5NsGf2S%U0q{&bOcSTmKz9?#ukayk`Dyrzo;6l zhTz6Xy{sz!j{(Vw~L;DQAXQ#t8>$@^6p|s5@EZm zU3NVP8{g-W`~XIWTR239ZlXD)3K^H;oBAWQ4wburyuGlOZj!1u0?2+DRP9z@g-VyL z#K<1y0}cN+{SO8g9-@&*&HyZrLHoUDV4;qsDhC5t=2fyXAtJqObG@|&30IB5cY$$4 zI}KAllCheCS;x?Qy$W00+a{Hl$*r9=VM^FnawsWAjMOsiAC`J;BT|?x;zFoFfmnck zxf2E0ZU4POxHcewJl5KE24 zvY;Hf4|JsE!PUx%Z4i`Gl<0Uv#f-*l6u-BK49Lj*-`=kC`V?ppyMcc}-07vvON3DX zp5y-WcbracrXoM_BzT^v{-I1D1mk0-3+16nC5IU-P-l9n@!iFqEa~bW*=; z#l{8Pn70@S;Da>`ffj1AnlqdFCevoK!-3&-u5$5GNU@k=1qHp-NVSnE33Dh1!q8m! zhhybs$`b;`9CkdY)?FYz^!0<}KLdPI#v!7p&S#sB?wIwLVQ|AhC8B(a&$NI3cD z{Q5YTHOWp(JSOr0IIqf5JJg?>d59TtMz&rur~OL z6}PH6e|^vPA#gJn>HJMh`_B)~9a1>|h*B9vEr6GloF>aa)yGbXLB?LpkA+XRmQQcu zA4XqQxJl4XVxuV>d`G5uj4kDVA-5p*+gqMH`ZE97zy}Z^bdedNc8IFT`aO_iwj_?m zb4(&!WYV^@@>(@=QYMm@b*@rAFavfb2~`tkDS%{`I>(=|)MNBBn!$L&Ort?a!k3|- zVUQ_arZqC-x8UIA%NvxQXe_}&p5XepselMt~jo$B~HM238-f1+=}@h75` z4n-teNyi^MC~g>?3XCqU9QKTPvAeWFR_O4 zmG*mvifLc%&w~k=q!29){~$6zeMZe9_gE@pz3-fULY5@RH?zO2%zjv3?a;pU1l(N@ zugAJ2eu>EO&?eln=L*Guo>y8)P{(d~6G{5YQ!#<_^hcuXC5IIf5mXI)gIh^Ui-4bY zf|L0EY17w#+ca*{kvhl}KbmyMEwJ~D3o-h`U^2*F96IYiL`WZc3ChIz^nBUi-yZEH zz}JjFd$eLdor{Y(y~+aI&kL$0Q#P9 z8##+Y@XMn?SPyjic9hY{x>%{E494+*+T}^R0ARmfeX@<=e0rS5w=uI*)^+%Je@23f z0RPT|+KPg}Erw)m>dN9%TfdWll`qHi~SNs=(FP3r_|&SKk~7MwsUk zzMJ%BpV&HW3?7;{lZ6vPT&vu-giTq>hm5sNnShUPDFthT;G)VGTklGFB1$)7H2fu| zWuIn=#m8~?e`bkFdIYe=pW(GP+=W|F45^{jfMGc`2b7-DlPwMj9ctrI35CS1cMlpq zx?AYEzrI)b;kffcvxbyQe&adgiQGw{)F&T{u`Voqz3P`P&6~5WuwWB|G((kR479 zwLYwU_x$y(bY*^D z2DAwr$#*He`cvo(v|K+&U0kiNgM;jM0hISb=P7I`i+-HPslh(`F6aLJ43NH9j`K7( zIju0>z9+JqE!BPa9`CD=4yq|uAX8|&cFLBIw|vr1D^pND?X^@EGm+RKkZzALFg2<-Lt^7*ITujx{gR+;4-sc)%p9Yy#iXB1d0^!K`|$FX^Z}1Ajcs4s;gZ# z>kgQkx(2kKc|^+{BmH$T9Pv2*YA(YPsMmGEh~BNylsT5V9TzI!^C0j!Pd4y<*Odq> zJ{ziHrxFj^G4)Y*EJ~+r zLBZNWRHV1xW;{qb(nV9Hyw(f#qsm}kv?nPdj@BkyGOL8S4P*+N?QXjFfZ=-UEdtDj zS_lgNk$U%qJT+-N4`oxLe;FzJK;INy5%z(n%&Ldyv9oEdAN**gi$6Z7pIYWe(~k^F zy^H7*YmNcJ5__7Sg^o6)ZC!&S9t$TdHNTPSN{}Td_ZjhgY96{vkH=pS$>@X7;&F}S zZi@xcm@pOY%YrMO(Vi*Y%Cz;%q9>Sw`y2&JDff8aj;S)&v}a~QI^ymEXz?1d4On@- z3zVw7f3T$G3^sv~qyte-7eWZX^Dudre?xgp5>mWIk>RHY?Di|#!*8odR|y9{}*FHf*)U-L{*Sd?qug(hdNJvWnqei77`x@{?VVYHZ2hMf$D4l zo2`=7g&}|r$&_kQa*1h&ufiiTys@XvH-k-b((=}avfx3}1!PMMmr~HvY7aDY&%s=l znrp*|g&Q#S+K&!zeh~Jp~wFN6F>27|x*M)%;Md{VaKNwa0 zwWp|S^DT>J;}vSnESfGSDX>H0rjbs)%xL(!wFk5q6emot(Y=yO#}Qn<8fJ6lyroSBi}aS7F$z=I?j%&$e2PJ;1l0SsmEtsl zhC^$XqkT)L@DGw*VVMxe{MfOfwZgHpAp!5+a(f0E)O3zm6yEZY!x{RF6Jyf5Y>dPg z9{&jepPnWC32#;3pwIkUYl5VF(<)CbVDdlS&^vksaK;JTV9E@SMv1@VjHFjh%}X{? zy@I_`sKO{0WRQxa?OAT1>n|TsdbQc({w=lJ#Bkwk1A1BHFiJlO{9U$RD6tDfJg)XP zrXnUx?Y6(>MkN|=)?gP^bIM>$5Vi^g4%z1fcw`Y~NkAjVTOXE3q@@Vp4bMk$IQap10Y8xnd0v6iMdmwDnaZ9O z@kcZLJOdX1Bf&f6xho{PSJuUh5#5MZ7>T|2f%tf-|Hi45E7C9Lfdk`wp7Kb;b(>^{ z7)X7m%;9sThtj=8LV1?bXO&1V@aQ!yKUm>&cuUeP>Z`T>cK~CqpCI1T%Zh5q(}nYU z84WjsqFnEYaPxLCn{!n%?#>TvYB7}AtxVTHlJS^ghGM(Lkx<>{Pa{>4LDwG-b*Lp| zIN7(QTRqDX`7_%Wn7~8ACHcb)R)jx7^-7=NB&ICw8l86nok4jDkhypcjV#~XJ(s80 zW}zG(-hI4Zgai=_wFRP<`n};gyM*bunllK)elNw!&;L9oN>KVC+*71E_ z9nm}f#K{l=>a|~NHjol|RZ2&}b3Ld9Wh;*dS|t6J9yG_GkV)#?FK#QL2oVS(#(Iu^ zFps6--{Nm1j2hW{46e`~`3mn#wj1w>}B8|tV0D62v?a8L%&v2S*EU^ed7ACjMj zYBHA)z{)$&5B#Tv8ha=Ni>`FETRXh1k^-%WPX(6%FO2`axD`ST%cO>-`@e14^beIr zd1%oqWLR+B=Fm6&HXz+?)sm3ea-g}D6>Z@wTZoy$G_!2mz$rC%a<@A1v0%@l<*3(# zq|_EJl=lC+_aI~-B`^p<^M(4qPe5&jyE2!h1*D~+*(G9bK1hXLRY`d!PYssu(e)7j zAKJb;F3N6eTM#7_X;8XBDd}zykd~50=`QIQkOt|J2I-d0VN|-iyQF96p$5L2=Xl<8 zp67l4{r;Mp`3*CBuf5i_Vy}H|L}{93HZ3*uHAL48=)JbIe-TUgMkmLFe0cG(1NEot z?T;fZm+OGs_VD?h-WN-DWu1A|O=uWCc=|ZwJ-zl6`C`bWHL2(;!CsBX(6Bt*2t-M^x} zd$R@(IRIu2qL6>`FaJ7*1~l{xtVMpNU6laal<)nJ)eU)DkaM-mPMlTAQ2(Zn(2YFc_J_fGru|iapcDv(=8u2OnB@U>@SMFT>Z9`XgTR zD3<(bSGhOsr@=tw>qS|+l-?g}@j?T5+OXmRMHq~UC;dj@XX-(Yk+e*0JA5zRrUtyn zj;5_-kW|*%$6$Yir|tuU_{`3I+aHK=fL!e^9lx?~qJL2CO+%fyMo^~c?SMYQGAJ&I zIXRZPW)Bg7JNw>zSor&W$D)<+(c5E`nk`%L6L$cX-{0QzU&Q;6%G{rw?hn4r4ZSaO zn@su{b;Pu4qaktl@Qz*Nd7*`jZ%K2BTLK68an?SWilwBrhtICDbM86#EVm+x ziBE6E)3mIfF@<^DID#lSBeU@GLXT0OdauWY^Fq$ zLIq6fjf`k0{vTIECFUn9%Y4<@Z>@n@^x>->jP;`S#KC2VXnXh9-R%sOB$KVs6FJbn zDnAh90+FhUCtTT{GP}RGab>7D<@R35f8$7nXWk>xQnK4*JoHE^u$hcD zOuj!1DE39{oa*Gf5swc=Q)I0rThh6uQmB|Dvf5}Y!^x4-3F#AMtP5%A1W=+NlBKq{ z+iiy_Q8#43B+De;=eHC<#WH9Ez?zrdUh_ivwAINQSrrM{^FPA}Ua+45_>76#_b%j5 zZ~(AYQzxW#<_3QVN_9-t7P3VC=4UbIy{+J(s96Xwm=V1ViVdDl3}B*!Y>B3XjQcQo z*(fh6p5Wtz2SR^|UW=_t&T44du?{H?N#OB7w(t zL$Sy%W-dV*83mHyk7<76IQfa5!Y0u!LXntnY7l9R-*eDJHPx{Fn3T z;X?bG(y4~x#1T>0$&af&D6br1_zrq_WB+Z{DCj9!i;a+tU_Du*86$%v|7xsH_jSzH zgouVnbasWt_7-sK-cXu(F}`ddIfP?wL@&Kooc%)pCJiFt`3*~joMj+^RZx<47M1|i zF#pxKbC1KNb+v}DUvV`!DuxY>kXl(=}HMGdE|eSNz4Y2$hAm8(}X;APQ# zxQo%exz=5PR>;C&9BI{Q)~}>OGgTpGYY?Eu>|mqnmiSJYKBmcoad53ERElzqTUd2R znF+t{BR`J7c^B+M2Zft|rp=c$pJ=Mpep83_uhAhukH*ynKo_Ha6VU8mo$wQfax6Yn z1yQs1JFdy&_^Hpp77>q>2)W5N6o2*4fB`g&xJSeKJt>DjpaKHQ-hBbIIodZpc7t^Q zJJJep@FSnTc*MH{jjzR)g)|M$L;t9y`R=mG(voj*P0n4(VOvgHVf^E7z9OK|sn`-{9n_`` z!%?IG&O^BhX!Uf9GGe{yn3r>DY1%qqTzZA+x^aj8`0lu_WKH2Mf1^9=l1#m>8?EJU zFdk5BVO|;>>Y+A`eIaBC8sq25$|Oj6gdqY6L>XiTzdHDHV*)k`2RQ&CX00N4VSlEM ze4k`^#N_e2)qf7s2d?+=R*jm=4l>Am!rf0a@^(qEAjFpA)BNbcYu8J6M(=(6(_70* zO+xag6|V`E40YQ>7QRace@rX*LSBHAGtMf4Oayn&U2-}4~KWL)*`GIA?)sr}>K%(i#lMxim zFFc3OX;fANd4CBeA>`XQMm$GUCdBA2cm?xZPNOs%PVZMsSC5Fa(Rtq_aWz~?VG#HmnF0A=_x?=#CFtF z3KAI+p8d1W@K^Kj|H8ueIAVsWjgB?$VsZN&r_*3}n*+V4uzRff@~N@dH#Q9BwF6MG^gC|<(WV3`!-P-Pdm z|EQp^>OcMn4f%sKP)GXyizV?-+j}4}(%AQnxooKt!1BkgP6{#mHb1=$9$G#Q3#SL| zU#K+t+*zkaE4&f}=)gtha&8qiV;OY8meD-e>! zz0Z6cUrvA=%cAX7I6tJvd(@A*PJ5p?2@k3s|%i?2>CL>T- zy2~aQUO);V+EBMF&d}+?CI=V-b&C3ly&vjD5R{Tl}<}M72LmPS}|A#tjQfkZGG4t zq3u6#lL&SipOy$(3ND>>k_0Y!`2Y|Gikvw)dyHA-qipk@P9$mQWYdzfXV$X9m9&ec z4I!6{Lw-6rv2%Pc`p^|P%SR@1C$jGLzIwug~ZBoOy~M`8W|E9tyP1b^U==qMv z8as#p_v_m0!>08zYjvx9m*~Tv52OFG08AmhajM*askS{N`bTdKi+8QW7P(~+(21EK zRq^!K0Y$>bmK0-Z_p-VX1mGu~|3!M1eHhJKx{isp zyDmCq;}6J&zC#J+<9ZT@51d!FuO#`NS+Ob?94|~5 zQm|*Hb;{_FBsi>@P%}tI)=D%KBn483SXtw&pXnsWQj+uv87K!j-DPhZyY`eIO_%^m z9EH6iRb&w$@L7f9kKO&H14b`^l!Zl-4*d_be!L}1N} z8c!9QW3}5DtO+ErbhoUJg5DxR8<6UNrh4YGM>K058lPX^tV%)u+Kt?GDbb{rsJzVf z7x+!w^Z{G0q$dx}0saE;JYz~pX8#$p*u;J0F!3n}?GLVFKSee(JECdf4tXn$_Y=sn z!Cy|Uhfd{#-%;YzP~-PY9H?+58@e50=pBCTpI)ev;%qNye%*XiW4p9 zmf8;2Z53grdBKutvgxHn?-U16H=M4`^7SOb^7@AASZSnroMr)~%zmUmtT~Ct_;Qwn zG3;>5Hpy802Riaa`fIN@d@q%c|I4rd%zI7)xEpp>r~7WNZ`eq*%?_nW@2N2I@nWeX z{g#-?(c2^Mp2T(3iANLlJbG9-U3EY*VR(HLnBI+%B?T3Z6J|`^o*% zOU#~C`O>|^eInt@n;Q2x0(iR0Mn{eTQ^MA&>MXy1-+18qS1ui_DSnjl${{rkdWMC3 zoSfJZbxfI-{HQ}3ztr_c@L03#1yeHdzoVFYMyei(`%EZEKsWUjnh7mR%;^Gf8W7jx z#(pEu%FN34-kP%-adAXO(syI@YwsEmU~D3=LPA$ z*84!rNf!{XM6?*?*myXK?3cAE_Tr|wh$%)f&9Usx`e!c$(ND$W2wl(4Fo`j2Rk&Af z;WEm>gf=?FzKq0$P#ro6c6O_fp{4JbQxvC!1B06Y%~CEvYsJ(AXX_4%&jGSGfXcII zKX#&-wn@#N51Ac~KqF$)M`d5uRF`6$w%fE#{7VhC8PsJ}7<%N$|C426qwL?u9Kx-% zb-=(;!u;^Y;%h~#tR^00aNar71sTYzvk`Chf6>X6a*CT`u3waOPI}$DFU6kHN%BaJ zx?H4#S{>jP;e>_>wXc^?M*Q5=jaZ{n>2P=Xcr;wl{_=2t`U%%x8h&9;!RXB>NQtAx zH*YBxsj~2mPN!9Syk~$EhSy}7M?eh7xENC}jQ}`0uW1!`flB9X0=p0GnnUsv{;N3o z*-sPzr?Zf#1w0_-&&K&-G2M7cv*S)DWdMZI{$Z-kj{RE2NdRFNwPtxjmTHe!3vJ=! z;K2adRq+F7d26D6Nx%lRg&xcG)rpcDz}ntY;aSaz+Vhdy7~C(U(qu9sk^FnSd!eoJOEC z!)YhCPqgT3C&}4&Jz#|YqE!3y^6kC?lM9KKk`SkCr}En<)cA;t@CHuvcox_pq!$HS zd@{n5%9U>64EGz!Y*q9dmEP6@AMxX7Mu|I5V>yZ$sEJI!e|}NGInNjN#S&X>q3ol- zWdNN_*hXzhak$Ico3l0cOW#i81!SE2g6Vzn10Ue{t?XEW6e-TQ*o9L|umN{!5J!O2 zcw?lTB|^WsZl`0;ukNpmm!z}9w5DpsIgxR~C7d}DWR<|07PZo*3@N}4h+Pn<^+h0> zhDr~`f9nn~Enpr1-UaV*;Jn8URYF0I%}UR!k@wWGWXH{I>}-G*pA!w=l6b(U2Qz=j zlb8RDZ1$56!7xyW~=`>Llq_K7pU2( z$P_n6izboH$|fWJSyNH9^bLlzI)RmBKLxrtW%g(64-6O{TyMww@&W}uR#iFLybdla zjvRz`S;MYa&)c>UzoERDWM;pIz-~~mvl~d}vm4~p(=Giy%obU3x)`!?% z^6Pv#>*EC?)CqoA@F-;N2 z;wH+%W;duPS=lj{ef#xvPHMZ{dN@`&gMZsZsx5I4|13!&ut;%y%1y-QcUx)-A&jgt zOd4#?F;Oj{jL8~bFxdUBfvLr@!EuJT&~*^!ugUy-UPYM5p(Aq_N>b?l6WF0NqV_tV z47|#Uofq6-VG9-C9e!-;*9VKm}Far%L;hCT! zXZg5dN5qA)g$s ztW>DT(Eeql!eIDM30Q5#%H6}H`%!U?$3 z{llVkXK8nxFS-z=b_o!$X>RM;3P@d;gi9fi7$y-^q!s9U^~Gn*;f-DShz@?v2kjhP z*E=y{^46c?CZ8wVHBPo8r>A(z&4f-d|3~}o7YrYmq7EtV(&rzI^hV>&?bRe2`#v^# zGHu^&a@wh)wqI#_X1?dZ?)9`ViF*2M*oC7vxVk&r$!^-$@)999bQN&fQPgHwMyfx< zJ9yxGgvviVE99=@snly$S#mn@4t;0mv08^*!dGT>C&E#mZ6SXQ^Vd9Q4iJM!q*|i z1I@L}no>n@CZ}irH530iYCp8{a+q{2He;BdB6q#z!ra&L;ji|#UBAmN^f;XKrr7LV z^VQzJmXjT>@6wyvsvgZfpPX`!9pB6x;!R&)8Uhcg-tE5-)?zQ&+_0Fkh#}QrFQK!W zDw>L9lBugm`k1$sR#VV_%g>nsU$zzM=BGa@bX)mVhUTlQ)2!S$*opJYZWCAe6A^rQ z$jViBR&)y)zJOA<$VkVESX8`%n(-eU(i|@gi>z{Km)D8cop|xO^+p9WVK#k}DvA`& z!misnbdf-x`|}opC6)x1oC)+`SOL!^ zr3^Q@=d!`J-$yNUU)WHdsd0%HfH0dve6QG}^GJIR@g=;6cJ zN3&o`RU6}JfuQi-*S#LCIi1bSSMM$Kgkc7&HsN({jdJI{jbbHrYOR<`)nhbEzhZBloEIZy@pW zL!(YGL_)Fp>Q}K?yqUrx%Ib;VDXZKM)xGJ`p4FdC%PVJS@GayGz2Y8x2l1?$s2sgZ z1xJ_ow(U9CJtWbO9ziL7v^hx0jiV*|yQpqmxGOO;^^x_9@AM(yg9N9Fk9>>)n)6SA z8##PlL`%?Z(J5k9d*52(G*<7N;y!M9=GG6TX@ceWKuHli;)syN@y3cvm%>_dq3QB5 zVW)c1HmAe|M9(A?{5ESQ>L@5&C4y5S^mcYbD+9PP;4PNUqN%n;wHd&&*4I-Ox)(%*rRko0?N1{ovWo24dud zm$~xdjKS3jd6qp)HB4H=Ms%k$X|4>bI+>)p`)l3Vp4e*^Ety|6B|qEx+j5D;pvZa>)*Uti=HQAPUjtYD4IPlsCRmZAEDn4Hsk6jTzM*U@ z?o~R=eiFH}-Fgqe)SCjg^cZQflnGPc611^*gT=*{8y2Z~=EUy&6lKVuQ}|tnXjUAY zqg=_p!KUfm?;E)5j{9jRJ6P+ZK$F&It1YZb`xtp5B@#TDDk5F2MOdp&m?J)kF*|I> zev(H?B9Krr#_fN!`d-2P`2bNlGOD5?Yfxg>-xex)Oz%Kni7EQJ+TQ|Q^B+<&ui^kX zVWk~c46CEk7TB;SB|=UKX=oL@4FdeRNWvJ{l%;!4aKT#G zvhAd#`X;H)cAi#lSU!WW()Rr*`}X_ViDSDX9`d``!O$D2#^C=l%m*lO^nPfHsf-8W z*;2?K_CUL*ZY=a-?Hr&w~DuV3w+r-+^2XAdaCmPxytM{L)MYDvQrCgOo)SfA2 z>^~lH_>{yv_*R+4F|Tln6HwYt6W@}Ru`;Et=dr!T2U|?^I|>F?K6qb~_^l)qvl3sS zmy6oUU)Q$n>qO@h0(5)R=L=qR$7A)Mo{G0}jVsBMzUj(&&%YbRU{+9q_x^nNbF+`@ zt0!h{i3nINt-gxej1S?bV}pX;!N-#qFyD_rhPY+GAG94m?aonj`{SJCn?U?N|Fe}z008DqY?8ddt4tYuer+P=rA3aaCvg&d%gS~)jPg;V&k@% zGmn_pwy?IT!N-v;Z6;$9wfN}7bix5&zjRw3iLY;v{|D#)vWPr%N{M*-w_eyixGF!3 zrEB=gYhm-y^tR(SEsNnoB{vgmMA-p|GmaR`7=_`!9yk~aoC4f3t!yj*C>7)N)>j!% z&-@IG775b~-gBq)w5JpGwhB|qborU*mLDhki{AHI8Z(|+2w}wL2A!@VgtxtTM1+;& zAm@0Am?ely`EECZo1WmeO9EH+-Qg9nrFXt%Gc^)CGOyG5xO(is#DSI&|Fje6@C2s&cO#;`Xd-`nJrZE@5=CX`+l4V zq8}Gz$glY7S~D{W>^9H9P25jf8eDpQwOKl8*n@>%^;g+Vu@+nRly7byh&?JY^z}3i zo8sR22G0!-qf30Nx!~ndEmYp7A?M! z{H9m2*}HU$X#%;NDt@=MruEFRY;O^p5u4Q~bH^2{#T-TWKVR^m;bqAVzm{0YvOrM? z7daILk0l~R-?EB$^QEB+wEr|3zEOzD^rhDOhaifC03WcKGHxt^@{86yVdd#wvqxTc z@0Crgf#t1HvzqN&{bc7;WmPO|ny_p8p;T5ETkAYHCy0_n)0@gZUch!1Xtb~44#MCNTP;)NiZx9KfJSmbCBP4JFS7mf3tHhvaL;Yv5X>X6C zTYCA}R(gYiz|Kef$ptEvqP+L@cJ@f(GU;kgx(L-j zc&u}5oga5aEOeKqvL>b2Qn*#ND}CEwP@8Tqa-=}+#bY!Hx!CY3l>WzhVKW9jpsvOD zY)#`sszlKk-5K6ckZ^i-zQrR<+{49plts+X%y1R5IU(zI&PR(%V%`_~TQ39n)eUvm z@}Z(Jo>zRyk>quD^F{kMY02?fs<05@2pZ98+r9%@imAtNst5iXXq4jd#&Yt1iza~V zSRBYrFn3R1$gZ@ZlH+Dv`3|atl%*x;W^sQ1*rAx1PS002w}Kj}bl@L<(%m{KI04mz zWz>kOdnU;{M4@-M`QMx(t>PlTq;`P)W}S6BOu!oj(p(%wH1dCC4u3t?-6!IL%T^0- zO04~t;8k)C;j&$bFi@-JTUVmvswat>vPIIq9g#k+o#}Nx08$88 zIqlv6zw!07ba>`}iY@lJU`rPp{U4D04uE8Nx}Ixk95h|5iE1$oPgjt8(mujLT=jtA zD6UVAg(X);*j^tG>=-xovetc`?NomkFuF~eAHQWcjph6Q?*+wc94gB{{)JjjFcuI|a5<_KvO0+>x`3*y+;mwD zZGZlgLb_SK-8l{eOOnFQL;<5*r2Is!xwUFo2nshYDb;^!Huo_0g$rr9+}D(rmq%hMr41d$s6AA3ZL+ixh0r<-SzS zH1;8E+o2i*Dhaj|zk3>+h%{bX8tW;Rjy$@pjg_Tzt&p$YQ);MJ!lHH>fBGy7l> zv{Ck`1X;cChgV@`nu45!86En-T1ml*f|JmkW`f__l1i)N<7nk>ljF)%N4m~>y@QLU z7n}#f!z@`F3LITUjc*TRy7Q^kBhF9CF){ObdawVnP?Ry$zy$Gqz8v9)D+W!oKPpGP zn)qhDdGNA{Y*owGw~F(WF=Z{8;_7o)3ayDH9ZF%OcnxRy%=3MLu-rbT{WLM0k;e$s(K{ zNePs-zU9gX?N^Fvvh1s+?3(goiD}7^qisz-_GHczylifIb0D1v_VEi0gg#?~6XIMx ztQ3DrcI4gm5$iup;1n200epT!%7iC!B8u=7VCrTCIc}i3zD%jm^e9_ps)ofUH^KL5 z--;2uswo?!jZ$^YS9G9fH?Su$K-vTw?t9F+&t8lXYE^kNof+sfY3s>3uDHeG%gy=! zebOrF6ZMj^A8XJ8!Y>9PGtIF83gDq6TiZi8OiFwD(bA9{^?Ktac{y3o4x-rRy^g6gsX$Ld zVhF=8IFU*pC=&t~Rcte#d;xA{JN47E%aCR!h4bk08tlM?bUM;&eIlK9b zE#pi_)fSX7YKc3F2M_M<))_I(|4nYTpYdqHMjCO4ntzJ=4nx*mobD+t3eN4@Tv<07 zvgBsHX8D+yW=hPlv3JzKC8Cch=@#{qpCL&v+<8BS?ClF}?avBd+X!E0PVafR!aD}i z;R7ib18!#YMebW13pK+cBvDbs+^^P|LxSUfq=0;;ckGXOVXie0=jgQS#p{I|3L$OY z&|#1@68zHMn^%wXfH-&0pkMC)(zp>O6mpMK_J&SP-IhsB{m#f)yR=k`NQ%2uU7vE7 zg>>fus1pB9V5;-Ywjnb38>iCixS$a%^5&XC=bme<+d(qd`HOx7q*`XTwph>KQmtFV=8~WQB9X*(51+|nE)M&V0-Qq^$;QVfU5T4 z&iROrvm0f?Di(G9xzq)zZ`%dh-`&Z5a4ZH#OZV;4hL+U~J6^+D8gxG%bp}hBY-&t9 zz}n!M*8rQ+GyZP>TUtNtIsZ?4L{t-j%2LD6OfWwGQQ$xI`@e8HyA&mgZ}+*|rUt|% zgv{x??)LSQXJP$)rR6(IK5+fO5h{}__yY-xv@z%pNnh2oOL-VX2eDLYXpm;pY!FF& z^v?kAukog!oG%(D53EO$3LR*fH`WaE4<9rP8`$;I5mGos6m%)GcM>g_ChKu+kfL72 z|Mm4Ogc9@kC`x7r9SPSB)P{Vkx2E^sC-7xsvWv>jJL;~LY z&AoLo3PHh^xaK_wy+orN!6Il!`!PUs`W)%lCz=jzbcYe?7$oC}_dXhj@(%v~Gj>K4 zF6EsYKCk~7n!j%^e9!iKtk+R~@7b zDs;rE6A6m>EDr1$$1n|b)-`;q2|0@lM zonbqe58Q2Bk+)pec2Z*@xiD)<-}wNR5mnhTG-p@J!c!`OXLywbNdK;oN_nIk33ro5J2(pjo*!^5&F>jV`M}UO!RUC558?2-2}fo!s3PNIXB7C zrvxUD`UAq3G5U5ai;&D-eieL6C9mL>^5C*9SuLcoE)5vqV&D-Z89nCCi{B+ww*`2Z zSA2fw6$^ZXNG*tuDliU}azq+g{i%MS@`yf-SR0Jk6B&wxym3b^i}G!R8+fS5VOSxj z6uX7_OHpEwF#sRob3(T914Q*OrV$1CfLg3`gthu&z8_c&aJoQ8`693g2lQX=z<@d} z_ZLep%qap7cE2u`Bkn))^;HKtRf5n1e^%V=kK5GVpP_(0`b>}DIbdM}N+L#$9aI5m ziyNje68r(6K3KHKs@r_PN4jJ2j2;5>CGqQ^!?qwodKK&%+qrB&4~%GYg39iIUhLMM z_rnd$6d1ID64~069q{0a*L7z=3w)$gok|HUXXAfE$RFoz)PX04Ms_4O@bLdKyTAVJ z|M$P!8skC)d|SO%{81NA$CY7iXM^@8!Rp283lw(Evp(lFSuPPQ&7s%bqPMHJp4Vh4 z(`IBhSJM#a8z0l(scLMkhu>OYx7Um7(W0vcJ}#|kAA;FhwBJNJ$+Lm;(mut>SygPx zz;NBydKnf02a^2aXK_JlpO4Fw_F{sXlB)CVAn)=g_=1-qAJ54EXVa{CTKRA98{?ac z=>-Iq5cV^l18YM}4_c97H?&XjKgdAWX?g$NSMlbM}sYYHSO|If8ycTbt=5<;n zv2Wdhf??xMLg3qzJGh787N~RZaFcC1{VF=TTd|FWig(r|zUdL9LF^*)3Ep$1d22C-~#5_U_=`Fh&UX3L?t>5CasSPstTj2i}G)OT3wH=-A44YFHr z=SmmVl!Xs?=02SaSl2wWtK0l-w^%n_lG(hyCch|Gn7*sd-e3Gt^SRTwqG0*FWjEOZ zoan~jLLatl4|0oafi~~$>EjL^)UMA^N!Y9^3cLCfcM8I)(}1(c`U_Zu292vT;PW|a zXfui3r-sGzLQ1~jO_F7K=C-f#58|7nZ!eYI+9N?|(;}yb?RNE*!@qyuLA?Q&p@n6c zU{JZtkm(&Rc=tJ?S}DjFcUVi?gO1m;lx0Ecj=-Ci{GhcTx_a|(_rGu*jo3to#>s>5 zhV0vB3+w{Urh36icr?j=eMQrTphYs!LgPEE%PKbOsd=MZO8ThtP1Pi?{iAN5TSzZ3 zG|{HvD~=t(<&^&N+BFQ9psRcLLhidU$_)Ee)yt5z@UnF?S-0MEaBAKTM(%!?b9nt- z@|o)sAJEU;ri+M%6_{9S;#!pYj${G_>qrr z+uo31Gn>y*#A@qOJ6*B?z4!Ek9|Vvlz7D3n-?}~-b<-l(L26G1L?MF5L?LGST~#2GqD^~-Y!~2?swge45SfJM?D%mNn zaY2R!2hM$#g)Pp*f}-wK=Q`HSuw^O_)teY2TbR)7Y z`LQyfAJa(lcA`AB2>7DcFjWJW?s;kDVXkQs$(pjaSQlFhw7Mp3Z+;mg13#1qItzzy-g|iqr<|BVU2vd>%HS2g0GQiI+5iCE1rJhPOI+rKl91Gy1NV z1WLIJOcmd~2b*TiuA9?blC*zSNj|#{YWY34+)`YSK4x1pXu&(|*0wIw3E4IPtz)_! zz$%^1(>+|V7nOu=`Pe#q%Db{oyNe66R)_fEt*3`E%Y|8cDq#%QE&E*+}0`V zoKT?<`|o|Ww4T*Pvo;ZS2Rt5 z&(7xoMV)p|vu}3fqj-`d;LrwJ8Q^PwI_Y%~QTA0+*BIcZ`}CA*qUY&U;bQ}y#g9yI zg5_A@y)WZ$g>N>s3^t<#JoT_QyQ609LMf_MmfdSue)$dPE#KBRm|T1T#tLr_rKOE*w-Qh_uF~#P4;`*n`Qa%xZNSJ_y!-0{#XZsFPQJ)-7Rl+!O!Y| zXCkzraVG-^GKLS^Ro&!nhSzgilDElcX1fY#8X1r-;!-VT&|2rvRwn4@qUT;Y^oSwz zlk+uFkYhLLrpP+WyY->(b~Coq8H-MG!Y1Ui&U?HPMXE2#l*2C5Z29Mkz1wz=*B1!1^fn7p%&bJmchW>eAyCvZa`fcrC8sA1Lvz-e_O|sICcH_O*Buf? zj}dImE3E^rlg`7Uu&KM9!UMXX5NxXn)V-Rb#W6I@!Ik);+dihs#{49`U-=RiDb>q? zDw9uMwGV2vibw2z^@29>p5mBxmhq%9U%kAJ#LbGtX4F0Je=UCC^uo?mTa?13W0kwc za0kv4N31t)iwvd?6kr=LX*SRFDEJ2P{p8!{wHXq@qRqd(74rMdvfD;Gw>-jqcEP$; zmZ*aNBBtG`vgtb0tDW1rqQ+n|l6R@671|N6j5%#ncUGUWwH}alvn5JkuaSBQ--uV% zNV*wSekDWdqu?!X=mS@?V1kI;{p#bHk3?5UP%>z=yLh3*x0B$o%)gak2<~=^VOoQ1 z2Dbb9ATJXHFvGrBndnmHmS25T z2vGBAfuitAyt1(V<>024`uXDBkMITyEMHQWfhx%x6@CuSw1Za>Y-j7?S-ZNzFcS6E znG)+V{Szg1wnM}E8LPdON?2o8Na-?L%UYj&f!CbQto>(`fE!sgN(hZMQ%$}?^G%Y#)xEDHO4hb;mnO};+ns+h1 zU$CPP+ct1r{kW!ZSCRHGfJx~jC9_*Rr;v&F?nslZN0fvdjsV@{Rs-R{BOE%P zTWcZEWpPw+(_q7mdV=@nq}GAT#e}^+6FW~ofy?7h(JsM*_IMJAT6kYLQ>Est#W+iK zYXzw=v=O`BW%mSSd?1Z~W*WuDkACRdL!u~9VbcfEsYI>f4QVq+&sk~~*4rwdT%T`; zXI&dNEB{hYV#1Z^j;5f1POk*;<4JIfQ9%#_p{^xYFU8*m5D>pVR=dKHC^dP!F3)KS zISxPeWnP2TcN-*1dtX+1Qo0iJ=~*UDO17zm=t3{>M9BvFN|F=p>E4$ZDCx&ym+WB) z--Kc(?UM5EJuS6c@>)i;KafuP;c~(PdJ5N`jm9QE#O#1HEO`p%`RKTsZ{i`1K4>|H zK;vtE7S@W4G#zw(!Soe9*>}Eru@r4KC%0N~cgr+>&Q3Mack9?qM&YH6Xj|<{3wAwR zl_4cqi@?^A=3Ikt&Dr6Ew$?|Bj~P`jnjZQfGZ$uwSI+y0*q+=IZHetlw#L`# zEp#~VHbWK*pGn@YtEo%51KrJ6PAc^bN0(ow{ILSs)w?2*?<;GTlyM}~#At5ZfJI$y ze7!-S{UaqlKdNJbxnZ2OC1_7kZsIHo20#G_0Yi>n!Uvv-?!LWS9SR|Gc7W^~A>n z*P?duTj8+v_4dNaFa*177bw||5?D(DzT>a;oS1tJ5n82{E_@aE?i`i$rk`@eW;W~G zYcs$0i-wkA>8_VB*x%Ql0MF1f9mrTo@OlJj4hSVo3o0J#Gq=^PSKT!G=BO3Mp2lw2 zZ15%W9@+@<_m)ver6l&c+X5wToinmPcyqGq4VwtOCoutN~ZV3lsyr=y7m7Li) z{59S&MkYgnD%#$C;c~v#_-wBAUp_A%W`86sL(5^$`FUBf6WK=tSYijky(LOmDD;$x z)C8*R*`#v2cxMS@V_g3T@Fl0H@U3J3G=Z&@cd-)Qk&d2Y4>?gX9hk}ZeAXgBSQWn+ z_c(ImL$XHZlGcI|F5|0f7-+>m25#!3yJ7EY<+8rRt(B$8uJ>a z5Ee*`^cE2L6yKNDa+ss$g9mHl-+}t?*DA5IMw_)WX#t~st(d1~P*x00aR;jl=6OJf z(_?*Iw3rQlQzJJ~--9R7pM;PGrhVfwb!X|mtgifCZ<6raJ{I4_^aKBI)uIRRKKZ&z z-UX8y6<4b!iovpX1`|KtwW?zvmLB8!$$JKiRifqKh00ESE*7o{wbb0ei1PSQzN5ch z(|Y>3Q?$?u19#GG}UMV%qn1pc&m6N%`E-_PT=c{1YA(-Y-BF99NUbZ!~ z2lSBk^Z8)R*xK!@S@TQ-uwfdRp2wzryBW|z=#_PSNflD1{Vs9H$mi&D*a=O;hxWSa zG&&E}E{_j%&NX=lUHjX-Z?0Ne7GOphNiTq4pmpA;J#z78Ib+x|_H1I-Ob$qM8`6Lz zyR^{IV^d>G@ZoC1QUs@xXyJO&?7(W@~9kyF|(xvYqz3U2POG|UW?10h|G_xZ=XG|R3R3;gC8 z>>HZ%WyYM4(M4rMuv=BlU|bvDAeRyeb?S-r(p97-xr>1XW=Q?4CsV@|h*zx|le5<2 z+mfm1kCodMUZk5rI-+1q?oR=Aa_39VZoCc4jlFJWGt9j!&sGX4Y%pxY%y{HE6rx4W zzmk9l&#AKZzVbld=}M&y_i~RbVem@b0RUlBLtD2#VWzWeA6k3-mZ_P{ZKiGa2Lh2hh9vp9zq~6J?Vz^tUyX$|L*o(r#AZ}GuWanI zf*v$1JE|%12eH8*cyo)s^>@bGtO zL=eyJv?{d3vp1PcbIiTWcP3PmU*ZY;%HFL(NbGJT3y6564+m<72Zff0Q6tJekC~5u zdLG$krf4})m+ES#X|`5ueNu2|wOcV{UE@EIq7lgK?puBHmK(HZ-{|;tt!Q+0CPlhP zd&{z1<=a_fs`ifQlONQGb*lEue zR1y}Y9hmCUFw}o1@9V66CSW_l5TT)n(@9+$ZfXR<*M2nlkmkwudXy-;O*-wK$tL$E zTTQTCK}JHbTET%aVGd7?dHi5Kn)UhI<3ybyImZbXr}u#cKtACHc9ka&6TEZPdf)dd zQ#{cWXYTE+Jz%Tao{q)GyQeYwXtB78V>Q!znFR>yOzj6TzX=)QygjZsa)WZ zwq19I8|GGxYghe{*iyx@_HDkz_CqWY{F{csG-eSZeJ9ICGAgc!wG~q92JDi^9|?Dp z?6MgJWP5te^-22X5*AY}AExuNs<2C(b!ym`V%{8S`{rwuws_J67N7uf0Bl%6CLo}+ zvPr3@UB_PAnz~C#ZDulM(nu^x@Ot6o7g&3<1wF?D_9h8rX+EPH9b9}>{~` zSW_^wpb@v5?9#cj-6x&aG-l{ISuBi|baA?=-4U|?b#w;0Dh^8Oc%&U< zaVm5xKdnBdSf7;TlD+$QB!nUD;HjbK>}JjgWaV2#h)_8X%ex>WlE$wyt@D{)SWTL! zvZAH% ze?6Yn&aZnY3O3>k94Q64DWXYcIB$AvBHYOY14k0Bj1YG9`@90U8VSj@+8I+41G=JG z-&&YcYliw$m0W^sT}^!VocRJ(E_1ob%UdB!kcSw`Sx#sGfM}VIWUarL^Xqyo zOnKjH_mx(`K|Dr|Z=X{SEQG?*0e8Ib&($-2^i zrDe+a&6ww5zRj#CsYKDcxN6Y%W9J&>d7D|o4Tzc5tb*(nSh|*PS^lr6l^C~E-LfQZ zK=;dSUung&t@5L<1Q~AtCGN^~@OAmEt!)r!oa$R{ZPGdRMR2W1*-c}3_dEBxa;3q% zjrn^L)X?05hk4K`Q6B-a(Mn+=CCfZb!Qzs3(rD_vyaJ-4wHR*IB&P8fJ-8Bs6rH+} z|Bt=*3~Q?0)<&%eDo7Df1OX9|jzZ{NDFOi`p@iNcbcE20N>J%45IRT=fdCSESEMMt zh7JPKLX#4DIg|I@Ywf*V>-%@E^W!_~H_SPo`ONW*a*z8SDRHcrN@rk{x20{C4S)ki z%xUPOE5t5^Xk|KG!jX^evA2`IrObQ9T4zzSUZtU+(-Ycm;+Q%C>56zVWAjlg@G`tGa|3IiMX}#ezz{hSFsYwInuU-6TO2%NI1SFeZrCqKh zu3oq2@R);zfH1zl(=}|;j#hp?X~Y0^0>%Vy{R4+mFDjeDQ0C40D8-q$#+nus;7&_T zQq1B6LG>!;0u4gkNV_Kx-qz?7Ccp1y>EUr-!@5Qu-x;VzK4nyb`#tZ<3Hd0sJBHs^QSJv+Yj4in|28sHUTg(f$qq?TlEW`w2>BGVx9{ zu;pN>*q=F(k$8YqH1+FY_#1AX64GZyKMo5fZp;$zBNl!-wCL(%WjC*>qr zKvA-l2%2@LV1?c!8KR9n)~{CdJ|zyFR+N6zF>t=K8I4Ndd|@4ofqyG5fzjphyxi1sHS zcgz#n=Ld>%pi0@NH-`4_e8~PN#`D**@%9z(l-V4ajcg<=(N>5NTwoC0-X324PLwJd*r$xBIx2=zn^1Z-w7HQ&aCK|5O1fKMR-P3+$pL#Az45_L8e{f z-}V<-NOn7zFh{?LI$&yX-ZJ|OHVmGNbE{bq!aH#u`OU@Zjwdc-*Gt{DDl?;aYU`Q# zf8&#Y8_6ui^p6*HH+*s2>n9)2x#N86Zf{hin9SuCu*gKIQ<=7|t4V5rk7s9DgQs|X z;nX89?gqEdVloQtjwP$NcE7%5t{&7iXzm0JRZK|jjxX@lE}HK$ok~r@2iJPhtI5?R z8b!4J)u5RfqgeohEc6Lme=pjA`f%8BiDqEsp5H2BsKnkcw(qAu=O}10ET-$m>q`O^ zN7FtcuexP0+@t=3B9VFC{>`+yF7-iFlNC0p0WC}~zRNkKU#3^9yWZe9w71nI*54`F zua+>Bb!FCQ79DRM>N2Vk=6Qw5PkAqO?OsRW2ItqW2S(7HLqz=qU(G6&hq9E@_;YoM z1HPIg6(^On3TWg)Rj0^AQ>RdWc$cT*1ck)fO6~8ZfTPUpiHxw59&fWBDc*Ro0MFI# zYKVq(FIZ%?^hs^8(QKUl3$lv!NnZ-FGz80p*l?lP%Dc?*LETfnBT3T+|M8hHUWR(0 zD<)afawqHj@pcdx*z+m7<<;>gH)K{K#R#M#Q?*-24*;SEI)JHO<`rKLmaBo*Rp&nW zoe7|!HFNY>!wC9S`J0YGCKo?5kwdZVW!#rNk*JxGlr}b@6 zTZ;eu(xB-usQ%110YhII;D@B+_C*bAXHapX(JZ3YVZ^znMMJQ2U=63r_fa=E64wCW zLH93(ww;tt<<~c&GAqdE>dLGhBzRGK0trQB82>)lmIhNG4YaR<-!JYy__9AzatV0E zMXfT$>t$cc!O=tSs!vLND-^Xt4cX|@=ilOsKg>l3o28993bbntthFE35-QTvCje`k z@tWW7U2X+ZMu*a6z@5Ri9^!Ha5@lQTB+zz7sqR;6oqT`}A9sFXu?JDVVC+HrJ>JJD z3vGG|mM~;n;Hov&EAn;?1ni=R#Q(fi$*e$^j=O|AO3^R9gH`0=j+FE<)O*r?)aTwj z{NnpjsLb2;dr(wvdA@5k$z!_oM;l)cjiEdCPwvmb2B%(lPWWpO{Xf?x6FI{)+!7C* zlHQDz*%y=quy=-e)JdOR8vsXJ1lMdoZXZ;Gx7>relZ?^AF7HP=f0t<6oflHja@1{a z-@%konE)M2Bb{T18~{y@g&&?RRz^mYZpNntW|Oo#O%v9`z<6;t6Tp#`$CP?DdE^m= zY9lg+bm>>qa3(ihU}xKp4Onk4s?|y%+RoKWSKJX4E?GfOB`Ks@)G3||>!Gk4U?}qi zTJsb3MRoH^eGO|lyE@3#7Yt;9^CLDB9H;wD{*L&fYB>5>tL8O>ZQxngeL>{4Yrrh$ zKYH$2d{H)OzLQex1wjqR1v< zeZ-=vEL$RTR2|}kf+?Q$6N`WQ@V%5RqJCeNJo*@D+P*$)q_6*3|9DGnP1U5$*+O_s zoiMoYz?kyklb5)L>!7tI?F;-3N1?v+RW=h@(P|8%f)KEn)kJRJdO_IRXrl)`+;r(| z9(I0l{c%74;5Ys^y+z|3YmCGHCO`i7A4$W#E|8|PNr!mL{yToqUzeTr?Eg>Q^Z#7_ zfAQ9BH+X-Meb4>}mGz&-^J?>wVg!Ts$-M9+{J^AQRz$A32D`8Vy~LZ06?M7(?^uIp z*V=~5Rs^@y>Ls2Gt?nU=NINe!x&EJQ@6#zzH0%U2Y+m|ztg^p&VCT;xdoIW(|8H&6 z|5)-rxjp|!8@@>?OUiq$3I3l&{AVZrVP^lA?Zn-YOLQiF8>qX#bCR6wmI9pq&2jm3 zO6s@nJ7xc{&vV>Wsj_flNaW^iKA@DfeEsTyz}HJOrQ1ibD%shGRk#23XumpP>v1wG zft?;knR=RsQ;ssNAxGIR=~Kz0lV7Op0`(p7do(5azd2&!3ZG!CgJ5)};LCmJKM8Nm7-TnTcdIQ3s3vng^RR&hE+IC%&ZrxH& zsowJ#F}lNm&1|nf=#0N8K!3 z4WK~woN?94X#`kHb6=h;W=4&37uC*~pBJRPQ*zJ#UT#X#8Bt5HTM{xGm6E@ka203+ zCGql>51S`cWV=2g?QCk2`*UNJNpk}oUPq}%+!kzMa_e~XL;LiwYO{^IhP$vb!^gB& zfxdv@y!H*@z00hnbT^aO2R9dY_4#GMIbS>>mB9jYI0BO^o_TaTvSke z!ixubcp5pxgbQin}=qnYk~4f4;G8sSJ&&kB3&74*naM)1G7s3|G zl{CFC$YE6U2A;GRHA6=Z_){*S`T=&+gDa5PW%C(JewtfA!QRb5)r*y%UFf8iLj;Hh_h3KI zw#iLr=PTu2xGmw~K=~LU?Xjs5n=IX#-JRa19vjZ}ei$Q9bA@T>!We7;1iojV?x_7P zG+OZ{1YK$9F=9oU!Fem-50Rbi#q$MZc_#^C#A!=fh8iqYzS8tkkT5G#kD|Yj7WI8F_fK z&YIRWnkgQt!Nz%-Kb?*~$0z9&;ll_c@jH)J4cgiS*ZQ9=N-kqd%=zE=#gXRKqlq`h z*pWN>X(bbl)+Hx5p53Cb1jxjlO^sgz?g=CV%NlFyxi|e;n-yw3ymNtT9BK%dwl*i` zaTqtpL=or3jnXSjkYLNCN3>E=h0ZP6H54l2YTvpyyZ5Y9W^euZxvw*yCpMU0YtM|5 z9)+{9yJ326YlIE)JiT4hq%My+D0DT^S(rT>{d+}!9sky)rd_`aD9;)xka2&2U<&is z+{GJLUP)f4|5>=lW87;awZQ$-C568F`jWNR%!69 zkg-Rjt|4f552Vc%z`E=*8kS0H@@O!5=uwrB-#&kXdl7Tp9cua%UB8a??SNqWE`t(0 za_L&kPjpwi^KoQat3mK+f#09hSJ@=eEPoLH13WTjhj}niu%b_edvJTU_EC6B-rmv{ zXrY!$oj4A#kPO_M4p%FRQo{~|4^qX6g0uG6^>}F|{%tXOJ=0ezY=JxB5r+Bp9U$Mj ziB=o`GZ6vKzBaJmb&fP%Vev&f?*dBS4r8cc*~Jez`0*gA131LK)71ln$Gh|4G{rKv z(Qg@5cpuk&zJ)h>=DFEF;2JTS?YdR;;x`0Bzip$==5-(#7rB=Z2;ajf*odhg5a zQ_ef?h0kU^mtDR8FfOtqyZ;E%KwJsELGh;zjVj3slA#mr@)FNE_#Dl}ncUq8{u~Ww z)`kduEsYFUFBfufM!Vk!x&c{)sEjb+$!2YV3duZWT+5%Hgt243h4| zWQ3wSf$lzC{Y0F)Z_k>4aYHrVoRzCe6#5?e(x6@~ku6L;mhWFzugWq(JDJnk{^^17 zso*MGOR-j4_PDUmLHu33Efs4-Z7^_ELdDP$-Su<^cHx*UqUu`^nLa2La2%!yG+WVN+kM8^K`qa&Z|B}_thk?j^4`ITaN&pwxg=Te5HI7(U)Vm699NC$RsAZqrngEGq+z5{!0@5Ob_g z+Id7Y@bC6&NjFL(!u|2nbn>*j|J)<2(Dbk-Pz^sxaJh8BkyR6DioaZP4;G@#yr~$^CEH_Dfm&OMUR)gy?3S zRAD1&3iee4yAVO5lw(RyDYNbIx3kAb31N*F-O3Gy0go@b`aeF>OU2@u3$+hNt92fn zzEIGKFE%;jDo-%Fbxf;Etd{P%e4ElXYwDkXuxe#G*R#ppP zjm`?BVEV#x0bcbT44jtqlVXs@A5k-f_5IFnpJo|VArHSroc;&>w1L8B&CeuEG)Te6 zYUtSoqK$8H_Xt*VWNn$(Xb%gRP^mO?*UtG@)*`BXY_RhwbmIz9HA^{3T!v&dFQ~nVh7XWjU7~+1gL0AaJh5BtQs|S; zu@2!?{??4_1CNaXpM4vDypO!?Xj%k0lJNIZ{z@`uh30&v&--4{V-rfFvhhgOeit?{~F~*a#6sWO*e2V?6g9wcl`RXcR47pr4xC` z**BlPZ>N?oaOw1g(x^$WBJ<>b=#!2jHMJ@7{5ErX(A@&o9r32{QFT_RS&jr%&c^2{ zh}t8CT%iJ7$?U)S>O<08elZ~M;V*qUNqE}AGi1x$y;iq^Yj2#SB_eyUqQl3qG*^?l z-aYrc?vcNCuw>8G;UQ2tr>lwf*<2E5MXC$+K2^6zddQQ~o0Xm+A!ayW0k`vlV4JO96P zzfWwJ`pSew#$CSkD79ZRY_GrsCNj(@J3t%w)(#V>)TLOV)43WCmj?zkQu?jG6rXiG zx9SEK9CUIWc8N1V+z(8}uR> zuRdk7JAee|M8rUETTK|;tUr1mIR7i^Y3FdZt8(C286D~fD#`0~(DX0I)RR=tq*Gp( z8JC-8?XTe920K;vmx93)OQf@|yE@fwC`zo%Cpk-3G*PQU1D91%X$0nWjpp9ugRD&v z#17*o+Q7fd<62VZLE7UPPhntN9rzem*)AO4L+(&%W&9zA{tLBWy6|_Zqz)CMrIIC$ zF#TO7a|U(TR8!gic{pCmHc;DEd>+A`9=8?uZzz(k3b{7k@c=y4x5%3}{&c-(xBv2v z?Ior7%t!a%cXMgv7hP5uWV|@P(fpC;{Vgk-WCbYU5^y;+6>0%i=r!Sn)04xhdRwLH zA!FZ8u;?^$4Uk(<3?+OKn{e;10?7K3yT-g?xz&`I7lbN3!nSS57%&lC37zBr#ag~1c6Nla-U0)ec5hsYIS}c>Ip-N z)5=}r;HF^wbD+RKg9-y%k7N2HepJpWLL$J3UViJG>;Ik*^*;o@i-CZbJWhKf`ERA( zXaA@v3pj&wl<0rwM$g>Es}l~uKc2$B)BF?n{^v^m%Y$78$N<;Qxib7alR3NSKa2Q> ztN;JUPVnw^8vbJzz*#K+|IR){&C`B1)0?;hn!GySqm9;yR_4HQ{6eyAM##8BN#{Ju z*at2$8!lK5;*tDw($U{my(kGp&n`iZCK6KROHx2NJIePm=lKVgY9ii%?M;s9-oFi+ z2WJ%PG>)Ij5hsB3a_O$mVtS$Eq)5X)xB>tU5$p~Zg8xfw``>}(>}1&90G_tv(mX^* zwo*1^%v3>A_OP968-RSv)BYvR{re8#=g#_-EL2J}Y%A5vg`g+|0Qn9b-Cbi&p>VF8 zo#H$-_L{!zISNMy|NJXqpNoNbaM?D#lAWTgTrD3PCl?u~oV-!TQAY>>%HEpRyS^Qr z0R6%#R$FIj(AI;;c*w{z@=vYyW0Jh75|Us!F6unwal7BY`!Uc>Tk3}H%e?Y(M$j$S1JMChNI0h zq!@l{Uhm_ zZLMa#kZh(T?HHRBaIiFy`#e+LPkOhidoX$7$|VNY-#~`?rtBR$)ras|*aa`OUbmYX zZR$1UzgxAC7B~q8YDT9nu?6=q9^5+vm#>#goLt}D^#OG1QBv|Je)9=7kBgRqgW7p3 zlKT(KP7ao9+`7cj=?|s$eg>@pc)_)ywaVGZHDF)-CJtAQQ>+u=bW%bu{f?5{R6f(=Ra-BkaxQXJvPR7UNc(yu&>?5_)DRxAT;n#` z4?WzeXjlbQZ#%3DV&`7Pe>rQ=t4r_ya#6O@)sgr$?$(V+ax70c1e6f4IwF9hxB#Tp zqOkVQgde8?3BdLR1&&H?8B86Z`kfgsvGhn>+Exb$(y{8q7L9<-=xQp7)1$44XboxD zZeoIUBBz^JX_WNA!_u%5pqiElVINi2wR%kgeURq2Ssj3vz?@H%Ihww)hB9z+H94B0 zpSW<1=jQ-j|E~3T&3gTv(QE($jxumOmBn@?K!70065R&Ybv0Ca<5b)&NDc z8i?Q;KNxfpHX53@pl#d@G!{U+O+$13%py+75MRHJ_(!}O-CJq1LC-$vA~kh?xYpCa zdYPTP$c|P&cyjNE)Joroa;{mysj{EkxofiJAM&kl*F$WovT!fs!BIhJp9=YvS?-k3 zqNjPg<${sKd&IbgqilvN{78I_`*4?I{r>!*K!nlJQh^pXFiydLuApL)p zD=h=xG=KA!On~3C|0M3Q;rivmM*AB7SM(lF4C?^6K39n81Yk-VgX0Eu3<{PG-vYA= zBLirf>=A4yOo+5V&9*pF5ukoCeqL{dGV~f_6&o#?YR+XS3%3 zcf;xNI=Asl(xDbBz2x1#b*_fvbK8`sVw#wvS>{txM?RU#^Ax1+$S+KuG#XX_*Q#UN4Y2h> zub?Q_jql>0HkX%8PJOIilRd~iKmb!Btc>*Gx@lW0CEbN@3IfblW&|Mkov~|eu~SF$ zA{&l~2Hz)0%1J}FPVHTBb@@-QzlNfGdUr@~`EL1kwuf@Z!Pb`_x^=(QHv(To2cDFy z)!mTBNPnl}?)X{09a#5M+$tU3J&O5lxw^SG8}-(dBiW@J$vQr(jorhKMoxg9|tf|HcV0@{q$WK*Q^IaRU7K$Nj;V zia3Q_tVP{9#?--U=>QAmK#N#aQtwbThgt(B1CXHtaKP__$JzWCZg(xObSxXx(dHI| z6?^v`e7Bh6+T9=T zXrKcBJULvyF;{rE?^?Q32sfopep+ zwJS1va*G%5`mPkO0WOPT?!9$}?YBuDj%(VmCwWrr=#gi3NnC}Kx9<1fD>d?0=f7J8 z!s~tWesX@{3ZH$O=C!SwspG=Kji^9F|C^mU-fLA0<3K>WL-2-uI5^K$F`%WY4Uj)G zUg*=M18`L7^4wg`=1lj-BzupbxJBygF>Vz{__!wrQ(g?aRn-oOe zdMfkEofqU+STrrE2*F|R-&$J5&Y6exNalO+cj*>X_`R+BXyY0?cgn&zW*S6(!Ga%2 z(Kx#qZglVGmAyRTD-&Z5J*1DEAR{-9^mw=7)M%sKHrU<@Jepf_po&7{_@Yv*Q@v+n zf1(L_iYfG`dEM#j#lFYHA#TD{ZXK*!qk&PsvW5S$&PLE`>{(ZL=K(O4`yiPv2kTjq zOdrpk$eOi4B;kR@4KP9u4dClnytu3Bue_SR-@`4=2AO>;s7=#w-(fx^&AXL7*`pt! z5?^Yw0`=07^HPo1B-DT%jgOr-v*-ay2o;)1sS<4$df3An)-br zN>9K#{3}yo+pX=?gn?JkN^W?^enKUaU=}oe9eJ-WB zU0{Qb)yyME)2pBNHw-F9n**Jk;{D z2M_GYGHW?t4#mx{PXmNi=h?67PW!6%b!@?|)M zWJ^XF%xNZ)5-R;`W>NNRJFNlyeQ_VW(zvyl5Q+jp4CICozCY4PxlNQPfnxkX`1pWoi)QDnkIQN5z~JHTqR46!m)d*Jy;jSH3T?ruJb+ST9)=v1!WGiqfi2BN zFyy^TZ;4>gx;mzja7{iUtYidi`g9q5Ghv)~R?oZNuZz__k~OwVHKjY1@sJ zBlC6I25%@SxAO!O=*zE)DfZeK^lRPH`t0Guh}T>Jb}G%z!1EXsG0~Lihua*0ZaonW zW-s`fLt(X&2sW6=Zz(t$a0~rlA|Sn2B$*Lun+mbf+gFTuo*kkXe;oORHe36{7R3Dm zL?*@m-Q{AQD=|^qo=e$rcp{f^+y;;d78)mAP1mzkU&2eXEZiXuMXp<~Tht%OBm5u5^^0696r8)1{T;FCsr~2Ax_mq@( z%b4JO{{Y^}TNme}bOgnQHZA#(+fQ)VhXUn>Bo>qdLPJ{)mLAd4{M0(%ovN^F=g1R_ zQK5k)@3Yfz`P?WOcdA*X+v7|Pcoy^CB=(Bd2i%H;so4SMD{Dg=wL0SErKfWyGM!y~ zw>Ov2SKd}))ILD&YVS|j-sS!FXe}P-lxs)6D@tW8D=)tS7v^1&&Ms-w9^}ZoSo$Vw zDmUY!{?agUXGx@j)UVVj3r~*vo)c-TgVx^NTN-;+G>H&{CFJ{k)GK%_J9RuT(X93l<8ni-r_^y`e<_;Z~u7+;DjK`?O*~W3O7)ZG+7gXq+HOhK&-p@sw z_@Ei+{W;q!)T&m{4R73c+M|&+L%!Jyy>8Zn&%EUqyCl{bGm|RP%RppUY)@`37 zghwClWy8egeah4wggdc*A0MXggl>M=%-;+mxj1+dHuQQI5}q$PI7w+SDA-bI>}|2K zhm;k(guAZ5Kr0{1477MTzHf5y-?gJzl$KeQ-lCQPNhRwVls!j#J`KoQ{V;b^rp(Me zqOEQK!<_C0%u%rM$y-|oa?^F7JR`3OUS z;|gd{)SdS>Is^(O9%6MOvoHW#w`E1ECwdn(RYU#M`XwX7HjAqs!@0k<YPr+=140VK&a7P}8? zMT-jS4+u{tne4yVpKHqLUkYgMHXQN%vKFUlw&8VFgqqd0UbWd??DZIVuhE`KZN)A; z>eig*?On>#su--PpmQ%`=H6F!)$8`cyuZ+!NHWn)iNcH92CuzPK;hPJx7W1$Of6Qy ziEv+tq1~}!qWoYYas*Ad(J=2q8MC5ygpm^hr-Y|ur&zyhTKDryd-??{-Z^@Cxv<6w zY&1XQ8y1usGi{J#s3Z084c+dZ6#h}2#L~?kN!9^^#uoQUINC)7@@Axl_3OAHKRzVX zpagC`7d=Yt>doK=8Qnt+f6F|asK{TsX<{)IMnV(LoJMIIESo962n-*g+RP^@>cHHH z(M}uo_8*l)=1j;XO=QsUR=wInk|nf1ZUU)DZC)v zMQq#T*h{{x@V9iU(Sc5`fJd-JWf&M22U50pJ zgBMbeP8=X~%RhO347I}gS~8hJv}Pz?c45x@eY#r#tk@n(Iw(cJevc}>fMO3Kld!AJH7jbQO&|sFKnou#2YpZT^rG=o1e}oHUOR4UQ|cVetF~#DIDMBgG6f z_DY$~N3y%oU||@kfq8zP{sxLYlT7TYNGAk;UMHFTC7j*}ly}n;wC8RZxKv#RPdsfn z-93G6>c2l1cjfMkwy-1nQ!cd#q^ep^iCx0QI%d2}4AE%C5^g1r?9L0z-%5nNuq~t9t)?-cjr`ozW`o;n+nDkSr^}5{#@#cE?bO2->+O|f?DXV z*GG=U_|J_Db84szYkEG`(WiylSNyFjf>S+wrI?mPGmXIU<= z#q-YKN2ZmSFdOoq^HHPVqL|j#IyL>K*Cl$G+5z9}$9gaL9T#PTww;(=o2b&q@t}Z<5d_mH5{NlH zI)%Woj~*doV0j;lfzd4qmf=H|&(}@iHWs@>Q=N6HrrRU?1$E5-+5FmPC?*^x2K|g7037q2usx(^sO6(7s?dyzI$^ zOm%I!JzXij1dDi1qX|*vKuG^e!zO5B_|`ic8cZ)&$MQx|U(3Cy{H~ z7gg~o^D{!at(>%L?nViv+GrJWtzga%EQrKB`)+EH(g9e%E4Wi*>%+{0Fjz_)tqv-h zw2-14*@a!wVo>fmK0~5PoSYZ+c%WG`@sjUpF;WH&6PT{*kXtZFu8@}*9i;fTx&2c@ zGVF2-)gX?B&e(5~VN!oyPCes2*=vSZx))gy#qU^JsI>^kBZ|(1@T|fPU-%Mwp_lpw z1_Hyl5t^DV-l0xkW%`bbN4>Mu83sg?uSn-^A3y6+Pa%=GnL`p`+3ehfBq4(n46_S? zob6<>PFq@Mn7XTD=~&0m-Eb*(u5G?oM-(_EE#5kf@l`l@x{sRVeNpR>Z*4TnbHf0PfHxZz0D`C!VMJq6F-6GLMXf4IKUTBlas}9liCVG?dH^ zh4M?aD`GblVf9vRR%4&lyW@!g!-||sTL4X}#kLYpI%t|UlOyv;WXK+2J|a}I`D|Y~ zAghZYKNmYN5*|0)CtUYtiqiJVdHXnepmsZaoa8H_*OHlF4 zXIRjH-IW0vp_0^*gGUV5?iYl&!;@6_yCusGTOwr|4sSk9!tVUUmta!c^sNhw{JQ{c zC;1Lsv+ZPiMJFvZ-`c4ekt0>)&t+uE!!oI_{}2Hf^#F6gjWZem1-gMOeD!3&;#rde zJ0mzsiIi#bx_RtJE?Y60r_Xqra=6UEn8E2P9@Gmv9^?#|U zMHG-)g}~{gFmpNkHYq=c^Hdt&&Ez-K_UcIGH*sj==Pi%XzW1dUsw^VK9vEh}sMY?6 zBumGsbi6;2-u;Zx!%c=aw+%68;@xXB~o&2NVmt%0eMg4 z$TN4f6ug@@EY;)ptf!>4)2;{`(k{g`RsuFC{VPc9{wy?j|Qs4}~bQfN5Cqt5z90@b^=?^uu+(Qrcz#63$) z(034S{DRW$t;rg3!IYG}n`>x{NU5_I=U}_qiTXTZiJR0JQ&|u)P501V868|GEq~h> zK~8(KJQ(BV9mwFKVN=k&f%lAEtu|?Iu&lxnl~5*(cMO4!3X}Sw4)TZM`oz&YrAElJ zCY};$TSf}`#oyMH`7CPRp_k2>Xh7dBY?AIbzYYhTTRGYDIV|id6Q``0@Gux&QLqow zZBMKB|0rYd?)X~C^ktuKxbO@VNc9WCf&D6yE%b* zJ*(2N`UHDL(lShMF(^;;y9t>MwR67wO<3UJ$Pch!nI15FKDs3CdqYTot|r6h({+=5lN>`-rM)WN zwnzCM;`xG}89AEL&TmWFYQwg%vJtghCSL52NE@p`Y@*<<_)K~%r187K?Glz5YDtxM z3oa{VPpNwG4qhj6=yXtuEACeXFsezQ%?^qWqvm#`iypm7um=$a=1UM>b8kn<7)o`; z90e^LezH5Zj%dkJBL_~P+7qg?0B!NX&ngS zxc`V;&FT-E)U$M0@RZWYr%MktsYpiF?-fmKy!)2N@wlAfMn207r~&&q3hmaKbpOY_ z#$-e#ZYCH0iF!({qs?wOKLebit+KK0HKuvLY=E0^?z=OH0;^TtX$qaTY35khb2#4@ zPHCq~)>*3)Xw~VtGTAu(}1)Lq&Vx& zud=1nr-N!J$H!*}DVCL4zC@L3Cn-iMqQ#?x|nMVTi?88TzI1oOiDiU8n*J zkK8w83wwntRL%wFBZ0$(=!nwn8Q(E-MI^Ywh%GHPR}t2MP3#fn5YKeSQCM(~4|dhZ zpopJVzP&zfQ+HoJ!Y;`pt(B9m$AR~!`)EQlBn+pL%OEeZNS zeI*c`Pz^8AZkEA!e)vLlDy2TEsQ4j-Pt~$?PL!t}_hsOdW^MS};Pme4oAMyd%-i>O zIG)SDev9%AxuP}m+ZUtmz}^|!hW0VPB5#|_wm13g0+3n}XK05WZLEOCu0IPB(%`r_ z&yZUz*Ik+RoS#twovq`r&&0jF&G}XPnA|`_JoQ7iZ-L){7OQs`s#tkUWR>8C=f1-+U}0oBpiHOo1?jK51_dR&e}@->X6M2E@~*QBDf7I)1+T#aBTT9l zcSp9p$nx~BD7AKE^o!CW&H=rXH3zQh9GCA2i0q>nuk_$=%g4?A!AVr|j{P6K&O0UL z6)xn{=cJTG;%;%*Z@BOO7C@Xldpam}5O#TGn=Tvk=;LS0F7e3;94TNWvltl#_@v6{ zsGI$SLUuIjmO#Kk%$N5Z<(TK*fL7=&xhhWs;cmziZ?^9bs#`jEDR$sV>V6wxCZ$in zfhgL94xAg%)8_5S*RN3F>vG#DT~WN}D4$#TjOBwuX}v1rJKi4ihBvBbh(GbSprLhRf`39PTQlfAmvqY|2B;} z4u-j*oBnQA3J+-Pw(v1Rr~MiQud?^-z!lcO1UcNpSjzO>7M@sksC5-E(FAZJ8!_wd zVpPq#LC+|$U#e@)ovNV}damQ&`jYu@yX^)4ng z2B530*y*I!><=%*>U4vUKCM>IoFEEBDUUinq(`+nPm;C597L-4BCAl>2fy|EuYfhO z`{>u1+LO1IT-h8AWVl}r6D5)iCYhP9I+SU!qguLz55jE7TNa4^QBpgk-MAk4YlF#l zx@jfGBk$?0!A(au3>10nFLK&XH7(bocWsWHvQEBNNBs?Lp6d%pxyZ~@2G_nAB%7l zw`wYk;C&AL@*Oj6nwfI;v8m(-drLwgko;K$c=iprt$!w&2$%SrFG%N%r!mPVl9lnL zKfUi(J{g6lZF8$ux^8t$?WqO+zWhcZmmwIXSo}^eX5Ok_$NaN+t^RMx_K}&*nXJen zF<944;(AWAz5d|LQW7KHVOh!_H+ z6afq}E!3nsy0lqPgS2cx>*QVgl^TsSU~D1S->FHQb*D(#_TeE#ll=Qh?c=UP!F zQ>?f}$_g=J`K^eC&h5``J8(FS2ByO8z`-g)lh--pb3Z$0D6$9VU8h!s1*L1F?0qay zB_1KzEsGP^6Or?5^;7t#Lt6Z>!jZ>3#LOW=6ArEel^&##pp@@7{JO4AW&4!WAD{w# zV<{_)7_)X|1kHl2EDaS!w47Rr&r%0%8P$6|%ZOO=IEIGB$!`=AJWbF2lLo{mkILoT z$)PQUKxyY=^~5;dwRU^L`epX9&KQ4&ZO}N7BQxCiyP7+%eEJxGCHfhl&-ysc`aU*5 zrsuRNsYliPid6yt`cUzrwhWN!(VQP-QBUFOiq+BQgl54YvxvF^jvUcp_ckvY<}Jas zbGd9Ob38nVFEd?X1-xhTXXe#wV)x>m{2=#{cKS$!-@vEmO2UASkBKv-{~!vcPEG~Z z4Qb}!r83=x*Wq|ndv0N%TEqULfgi;FLbEmsHhSsV&^ThqNc<+?Y&NR3 zd~6)QXYUQCQH0z(GGVVgDrpwq~hrESCJ@Y z&FZE?2YtQU+2oA4aX@!SJsAbhNyyQc!B7aD2RWb9aJ(`H>dbo=R}9a3GLc8gwaiLx znyrqA8=PJGqVeG@uZb<{jA<{YOTz}1BtNw;9{d!Z`opm3T6!pYuEKg$!m2&!ABH8Q zmtXok>GG9V>eQd5KaxalC67K5&VT6q2@VjF$h<{`l{J&(=OQ-Cc~@iKm>shl^>m7@`)qWD-YOB zLCk*%Ny$ZmD-XYXGs14Hg?mY;Nwjkd+4C5l?`IKF0d#fq6tE}jF~mZ!zU zHu%lj2qW~dGPxQDXXU1eWmlOs@|*xv;LCL=1P+GIbz_uR=2`Yv8~Qo8$w93`l+N)k zpaF!b3Q5(?cl>i#a6b3#1-b}4;FRtcR{JQZtT{i0GmJQlId{#oTbN_=|`xblhU$yS+;i69IR@YVek+i_&iC?jFZnUZb2-mAVXMDlCm)2d3kvF zOe_PfUJW}FB}BAfTWmR&+}kX#Fm~~NG5)C?ufLW3OKx=n8(ct3a>fU|pru=WqI^8P zYkKwzE1$PTxZM5+ z8hEj2T2+VU>_cE?Z3DrDpfNQov+v#Us8vtkiFES|U?^(gWTtw#co;QLfg83oxV03x@rx`^@3Vmv4_i=1x$`+ zz<8p~CwQCwPp=U|h?F+fYvUUf6z>1LA=S=MmLbs$To?>@1TsX zl(#4@L|jd0VEH`qBF_e@*hb3cw3T=IVMiM}pEQg)9EjAK_;gqR&N^UAc~SI=QKBef z3YTnskcCLS{vgzv3;wblHK7ILJpbG(`!-GPc=zK}pi>B>kfvr_sXq8Ujn)+n$1yKW zSgk`927CJhXEzZ2fkx<+B)}`515ikr)uJJRzv|4jGm4O9L>DI?bqDmBCBoEwvZ)hj z;)%gn^uFP_7!Y`&wp>h?hnh|#Yz^QAV2z_WyfDSQfdgV6C^Io+!Z`Ho0iPTw^Vyw;M9GF28zaSg7P8DD1zq%Y<0#>vEK5 zlq|18%;P}e?opQnLdhyKvE8K5fW$PV7KDX4+N z3+=ZygCE%nJ0jB7K2$O;um!W5iI+E)32b9N%pLEH@xL>6)hCEq4&KpN9U`qdLg}Rv z{fhq|_TDx%YR@f8Y4V_{MktaqPo+_q*SBt-0o!YtH9+X2ZIZT2>d09QID--gk*?lK0;V zO-?Fkp8mk)>M48eUAMECCRpHKnIcfGP{w}5DElTTag2A&{V_^3u7Md-O*Yu2%*^j` z3!QU~Yu>XYRiGYO9Q@__%}JJ^!AR)qYV^yzeW`T|a=JWd62CmHGM_2R(`psNpsa4$ z%7GoQEnP8OOL3N)c2vL3eS>2dbNemDszWfdWp6FYM(1cq`Wdn3WVFUiwnN#ndT+WR zZ@bvnMSTFH5@jrUGw~HiArh3mdql9GtqTPogm0kAm-M(6*lTvD$BmY?X*h^!yH8>O z1Veon6cElM|aqDL9{h&?r81jVm z^qaZ0^}(@3L%9#t1U1ys6-Q6ryRK=;CU=SS{`^5oHGcHiX_O-7MvjNBU=HgM+hv|( z8o~pKYUEP5Ba5QDmzA^U{yRGomsp<@{wn#KM>vA&+tK$$ixODzSaw7z)77l`CH`Eu z6XY-$kKa40xq(4EiP8q6!_CL4lMg3U^OnbkdT$G~C}-uw`g+aYxrO}tr-+FB4`>Ma zY91`9A}U02^m03!Sb%Mtmu-YkBrIdQY+bq`{L4X1=|OsQA5ZA;aD0P1O6!?I(g8Wg zs-y`=f$6xY2v-|);pZ7n`T>o3OVg>6TDI}gTjM^SK$7}?!qPsTq@=DQqh>Q+FErM} zWVzMsWZpSEJMB|!C+DUCXgAI(i**vAKgn)pk!)91-cOfqnpKvIB*KrRcX~o%`ZTQg zMUva0qTNvM^cCTT_)rb2x}zi=n&vY!<=!eI4+ib}qV?JbX$Q9qIV#HS^9b3K2|tUc z9NaP&_kdC}M&GPoSa-`#OLKKu_+m6C3pb8&kg>DsU9V-B-8ER#Zs2^;$uV1$L&P=Pj!e=;k$`0m>Z~8BP zy-WTY!1l?=@blTtcdM{#j4k=muo-8{uGU0XvP|rwQj3yI25+K)Rd-R#|js2{8``MAkwBwY;c!<(N1s2e|T})#) z-^%i6^CiE?7DHge?8aBCMfvAQg48>F@PAA4e`{ZSri7fT-R~=K-?@ms)~y}DwRseN ziKK{M4_iMM7R~nip%5@`7PLZGk1H0bQ}86z4PiQ5NAO~Cn8sW5ftW^X`xcTi;Svnk zBZ_qBk-s%|KI0;(uzFvx#i9jR`x|>?mylNV<&{6(DTgUPHzBCr_&}5Sbqdl*=~}M!z4TD9|7+ z#q(j?B+dihbCvKo-ygud%=q)NfBvdweuH~DGz?f>uKdtIKq|Q5?GGx{+`d5}zlls< z&safuPm?og?U6SS=HXyYm+0 zUHzbD zI%;H|37jWeH)D`j+=<~@1CmN%FGog4=&g}2{hNC7`y=}Wc>I7=Mq!+LH-plO^^t6# zDb_%l8_A$BK0PGM76##$TMyv*@9!Z+(mun$+aAAqk4N}dcmCrF7I62e4-XXomNlmh z#(Yxto2cf$M(n=`K;Gp*bABF(EN3Y6NT>neM1UXIo)vMMz1drqc-0=wrE&h zeDguU6XH1cg=oKa>NTRGu~X~b>fxhW$tC;BK=f?y$!Z|JL;mcLRepJetBa$fv7^zo zl~Dh7kG+bDb52e5hKkM`>1Vg@e)K|pNOcoK3i+{75dS44IeUnmrR?-)BDeVdb0oc_`kvU(~$mu zYcLuL(E`7{2e{^`JwW!ii$Dn2r|S)<`rrMWsv9uR z5FP)Ozx#>1YhW2oqI3@bhtdA~c|SHT-g^nrx_HWo{&!bA!v*V%;#o@UpT6}U9|=;y zGFybD@%`CJ{Kpk8z#Sal*f0INRaSTk-jlEatJnMQt{4!1MqP^jF`MqU6Z(()Kcw<* zqyLOoeQIce^Dj5`-&at8p%U^EWcfR9!7L2kvyDyM?r(nLCdMJ&OUTyEWz)Yg7%AO7 z3-Osd=+!~0 z2x!d6Z3+|+rB!MdA7qojy#IP)dQyOe|B5FC!5f|^;AVDy!~EQvK@}*B7>sy}W$|IJ z;;-yh)f!kFrnVO;*BM~Sn)3tl@EDydl6N-jLwPap5HuzA*XEoBjNP7J6WbvM6^D1; zD_+A#BC6<)`&dcJ1Ch(<|FY|UnDfsYU&*S$?k$-i+92&5i9=mA^vfL^mcx3!w4ln`t z55Yghx}b&X%rLf6NXk<*yY&#n0)V!rYHr}FNY)^JSo>>2` zQuqE$xYu*X^ZI-k1Mc97Vj$ zB(6nw8t4h`sZapK!-h?>lVkB@e@dP3k5?&7fqKdss~_;18)S9Px>SZ>^@o}E<2AEoWg>4YoW_d#&!*T8HP`_4IUQU1t%A+-5J1>HJfFK37Fu}{ zq4wZ9e^;p9^lls>H77Rzv{S?R%uUZ~`H;qYDZ9R+mpoh3<2HNl>a06*BAsvK&uYNT zb31aLWpZ?~MJ|*L%PoC9?yS!E0v`@4+0T=AleTj6Ax7deulJd6}K<)m}Js;P(^I4w)_boHo z{2b3izd`0}%InjPBY1cHD%xx5;rjD!0YO+>Rph#P`(A0bkcIP>#MOR)$VJ+;iW-A< ziTT)(gk@<2)(L;Z@JRVNH4TRgxuWj`H`O z*pt!-=nHM4vptU1KV-FwOYu_x^Y7%*;v=4_AB93n4O5#t6O{&Pyr~z<^yn%zlcVYd zb`n^UiuHb*jaC7y2b zAbH2lFxm!i)6H@43^^p$MSU%PT^=l)D&qtUL}9Bb)G60ptBbf9B*< z>czXY6F`BNMN(|e!ZI1s8+l{jobJOKG`U5=sUYee+lA_Z7*35nhLEm3RsQOz29fV~TjAIg z@2)p!lh%lcH`n5bET3_ldOpQNwiURkeNw#+Socs6KjsC)X}9olB#D-_bJK`g6J1>r%cJNhf}Yj>w&dp z;(F}*FydAch78t3zO)ipcN+!%KXL3LAxM)k| zH?&?&pXDE<9M6ZfOy*uCI`VH+o(i*!lQDKH-)Rqi-0;gk-RjJn&Ah$ZI!tIZR;#2O z(i`T()z_35k zFmam}=VIsj$|@*?*Gxsjw;p)Bd_{|)Im**UbOLQPh!zKqLtPO8L7J)`C;1pb6$eLW z*k#levJSMKc@fTE$r3t-Dai28h_0~*<6_nG${3nDJl5OI^dRJ=5%pe8KMf~tC9a9V zBB9xk&CC+TNu@1zY&P_x=MTf4=B{=o_=V?1hbJ>1&xRzybr)BL-%iz+ zN_s2sf#6h3?cEQVd3J!9Jl0g|5pj&8cfb=OY}`6A+TL~RfGQ-DP6)|^)EL&H?2)C8 zE^cVN&B{Kno{Tyc)mXb0st~5r{VK2d6(mnL^pxL>no}b+2KKCcgOo4q(?esk?~5=~ zJm_+PTWXf7z^&oW1f2K9BqI~9s$M)0qi(VH$f6x&Pp5t^p6zJH=sQG2n}T%r4);;~ z2IzSGMF`$N!myUhT4bQgD+SPAv&G%rl7dL#!!^R+G3Z#>vvPZ~X|P?e4S!8f?ku&Q z-EkK)nQ7nD5dPLT-*A220NbsJZKu{=?2pYjGNf%-Rg;A$2+XW2r>V3qQz z@i~Y}=$p%V2^oFERo<2mh#eemrNQr-Er!|RLOaE~WW5HT<(4{A+k;+Pv{^&EjXEjZ zPjp!l1uykK5*AQ!O5#!?1_E#(2R{@&Y>f{A#awdl05pCmJT<+OhNPRw(%u&3Q4}R! zW^DJVRZ=i^PU=+7M_)z?fm~6s@e0r1b_bi6Rbzsn1?w0@E9iwuiULRHuz~F4!Rz31 z-R{n(XU4rL$jwOhf<3B-gtxGp^cNN$`GMF{UbBq59(NJt@;aNLJ**b~q6r(;4{#uy`9pjR9ZmJLYd99w6wzP2g&lu~QoMy2RZf(T-BsC&W!9x0@M zYWBFC4i`%7i?BWuR-kpCctP>|G^d+|IyrVup#sUA5j&4^Sf6TM?}BW;wAD&+oZFyN zlxJH#kdVwnpz&oFK(GE;%ghwQ;1 z1ZUtUolg>5bN1skqdp9z4QEK#c)bm+Xy|)RL(=|8+6vmO;ZJ_SO@03+nrIaJ8D<)@ z$J~Bwg(l5u8-#A5!Xh+ke2LiKFXll$0p~tzh%CWoxO1VV(AsAo^@C`q3nsRlUD9FI zvtFrHz<~T^I4)1)dl!O@*6fEeTx1sT?18Cs3+e33)W>s_$Atmd{h#t(lx(>XdgAd} z=})oog9^BsVor`4@LdLUNB{OqMAlQQZd0|$W*E~bB=YbG3j{E4a(%*oyA$^2l_ zaa&3F^);g4I+!Zft$tH$hs3|5F3IT)`BnZ>&`$3+O>_G;e|4LB1zdByUuG)fq$PM_ zK_WwA=ICXzO;#V6*PK-~++D!+tW<^S8xq8-CM`K{IS2W@VA98D za|$m)!;Q7jY?(6_K8QyYH)V)1e9_IVQC5@`Jq$C2ThQ2RW7TCW&0THHp~>|y*uNS=Ed84wxw;rud=CW>bMj$9oXmb-SUr|R}YXgS` zvma*98|Uo}nT0J04$IrvM(tCv=6WwY<141+7h~?nUg!w_`pdHGvvlQG`gUL`5`nG^ zj$CaYsKagMd|O;N(KOc}ZmR-wjMx)aZcC(a51Kqq1k|W`^r)TL_azB;hEGFQ*$!{m zp;SfczXDraZyUdtM$Ln0xIj`~|P zecaU1#8N#ZSbO`@iY~9**Nb;@O=ZX|VhbTjkZ(yKq}d8-W4J6+Run2bWNh(*H=^Kb z66B*l!~?sBPiv?!{Y<4v=6K!m@b&bX6t#WrhvYK6;GgYM+O-2_-o*+}hC4Q4&u0A! zE-de)b&%P~@r669gb?7rh0sFi%a-7xr>cD1CppA1%(gq0(1w!5k z>`OTZRv&#eIu(7r;5M5qfD_Y?Cx)-GuK-W>grXnNjqjlpaY;Z75}vYYFN)5;82L8S zA#%0%bYt>z3PxK2^{R<$x$%|G1?CvQFZWSS>}=i zF4olfl3$JaAbsNw>RMJkVvx*Q{BUXXMVqZJksjCQa}7r+!RNUIwws%mtiJ;%1YV$r z@*GJT^UW8ex+9eBWW63(Rs2AeD)=UfK7*Mug>~ailu0Zi!hw7;I+JpxAP=m~FkChD;*{47gur^%L z)tC}|s#wL|iMxZW|KQ+?7|_1nk8qx`FB&9pHCuZ)@B$?JLVBXZ3tkT-jQ}6fbq6eU z1~wD2Me-QusVr$~irwaU(Y7Zu1lRJ^(Ms%N=oa~-uUNaTn_pRku#^Xi!|z3U?IsK_ z;RiXNS?5#WhjH2@t(RJC12R6__(DLLM)SQ>d~RZX3#}B-)Z5%`Z?s>`GjM z^R=VRk0?b{)_ifir8eSQKY&tuO)I5{$*+0BoJsBu$V03>QlZEc$}H5n}>f=_I;C|spID@G2Hfx;R4TUOW!)d zuGDO;rZwRN8Rhs36dI6Aljb=N&Z=z+8<(0&E5lwetX5wYT5cs^5U)pIe^I3@>m7^@ z-Az2lnokuXRUONdr?SNCH%NTs4aFDZLGi3lgH^fF!VJ_}RUIwyaepBaDtaGIIL&Uq zH=Wb4NY&n7R~|-hfF3`$nKhlt$-H!&_V{9^;;KYo9E%z$ITw4Oy-8UT%%(Uf1%1b4 zNX-$MEvUr@hpBWa_bbmjGZ@~qj4P{+SL%vQVN`@kRnf3@9LTc39moU5afMWL_8}Tw zH1Y_bUY1&n7r*~iy^F@6OW&mHgLK!2;Q^V7`_)Em=+?3++vg&h~gXlzhAB`GZ(eI!28>oXc=T6zN?? zMN>xYO5EbM&8IsP`rujJ&7j76wb3@^5O6T(iMYyQzL8n`Kq-5$-NJ;h2&LEFdt9%j2Bd?g>~$#}qeK{d#TrF{}32tSfz*G2{f!_d`}7gB%DA2F*!#`U5151ar69t4A@q+-oHnA*}I3&95YQd)dV})R5 z?&QiC zE5iNBUTNbHpMxqNPQlrWDDWdzL0G+=PW6Qn6r5Y}22!w@Fnb*I6G&XT^n z?rG-sFA=Yx&-mcp$I&k)hf=<5$<**#ITOROm{GH8t4zASSnf%Jybl!DcSz?8C9*3| zcCG*Vu6?P~SarX&NYKT6FTM?H)fN@p~} z*b6<;SDY>g2UQ;?Oztka#(6?ThfHNF2W5d3Epc`ZXOgF$#SrydyLaM>)}so zO4<@F5b4Be%g+XVTJN@?jXa|hN}3dWY+=LNHJ?s-xyEkwL`Fa&s*67#pJ3Ceyh>Gt zVe$KAYux}hSBfiR^Y;0>iXCf^b})3Z8wzMM*m@r_AUy^#Y)!Q;a@n#~wMR>=RIC9B z0*cMqyJNs@p1v*k6x#tZ5t~sGym_ti-87ucojw|K#^%j$L4;fgf%Aoya>4FHFoVjE za^5zpHvB6;xS-8Qs%OOP*ltar^}Zv*lF6p@Abys@;?rrXOtk6M zp$%_4NkUnO*01=>VR&3yds(mP(I3uom6sqbeNvUpxd~Io<%huAJo%1&)1nkJA@RcX z2x7-3dgtQx5$XW4r@AvP$*2ej$ms=gy~#<0p_V_gyopmS0nR*}X4It~qRWx>`(n=o zd)_9|HBAKPkct2Lo{eP%H0R*X%0vbn>B)|h-;xZ(DYg9r37FBsL3eBK+J}?A13ypc zvN~wL7USStB`Wj27fqjd1{umdyPJs8c6eZ3rqWE?V~(xvI3}^YiOAupP{X8-*2s{@#ZjKrmx3W03)Kyoxr2^Q)MQ06H zV-`eG&I7S^`r_={WF-zP(prnEMxl0x*9bCMr8?xH3aP1TxD&vjEUY(HCa9`YnKH+T zoR+V6{F&$S)b@gfn~c6u%|saQ_DnQGtw%}2t!u#xd8G|?kd#_%_2CKNYb4_2;B;S=cV4v&72fZ=tK2=38*Y4S3VY~d{*@^5+X_tJOyrmN8tWf4pDUv7r2&DsO=M-~k33WppPw1S}zYiB&_ks<4M z6QqU8jCRVV=Qn!}lT_v@%@GhCF^nq%ulfFzlqYm|C;NA8>p)aD&-dl?eiK-$HslDa zV8!Trr0bYjL>-z0c6q?Hjc~eETlGO;W?4wD>ZY2{5ww1lE>A{#f5=c)!J*9Q?bNDf z9-a)1!CYWDcE*?fSlAo&sMLDO!;}5H3Wh#eako2-C zebN{3(vz7icyr%Eup{Hp&`8PrOh(Hq9pzq77~g;TlZr?pE>xzRPeKHlAc$`GX^{eR z;A}>A2Py9k9wqlkAZT=^;NnU#<<^9aqracQFX1Y7`zRrq!B}u}0-i^PTVw&)RH}Km_Y@MR1u`@QGwS-v;_sP_XmvkD%7d(H^sa-l@>`yK*{^}lhHKoP+f-h182mF zihtQg-#a1b7V+&iGeFih-%*_ zw6TF@x)Gy^x0q>9xN3=YTKymaH}0}pVZClPRI%_K)1QMvW7nds z?-e4A!`|$NztEc<`OXs5Rvjbpc8kUeYO{;j-94ul-V3^GIg4i+=OmFB6e1G)#DxFx zYHgz{-`pc#{i5!&UqdtYWu1uX@w$9y^@YFPFhA}l-x!>FdPQB_fl4kvyBNqlgk(+yQOIt ze31e7u9Vh34f&?g>&9Led;nF>Tws)FHB4(E@@WZ*;WlowznyX!~Ew(blZJV4HkTK1V3w z3;*|ud_jef!eV#gLr@Tj47^=eH%KlwF8%bSPK=a(vb?BNg(G~8&BnqGuVKAnhv%|{ zX!lW_9)pq&j0feh``yKEQuNX6u9I4|^}_4GMzxy!OnQ^a_~ijJtZ?gjyGS#TM19)X^_Wm-w}#i{8?S zQGF0gS}N&{!Yq22JGI*~a>1>FnO3W*4eA-#OJ5cuos22kN8R)|`~+Ls#{A!@%kYjl zi()5{TrS~*$Jsw`xk+82-T%@5ZsIN#D|PXR z7$a@m6KtO&GFn#QqAwwl{aI^-ZrP#O=4taK`S1@Sb`#bG(OGAODgx1}-eB64-pNUrAT{Ue%qw&( zPHQ28$!T$me3npdPl`|fEa~gkLqchNqx895GljgoC4~(9?E;eqDfoUBw(xS=_hc;a zqZt@MjHLx&f=2WMK!hIy`yx$}hh=6=a z1fgSen;j@t(fPAzl5(WyyUuKm!GPP61<)YYrlLI$D&PDStGd7*nYhr4-pBPyY?&?z zXabnrI)xcrs0>3I0?(5RGZr%&C!H5yb4A_MbDJ;|@^+w#E8_M|mw003JvC>Rgn`fx zBlgRBQ0W_##e+PMm^DID>z}xPoC@LXQq88 zuzL3Hu~e(LI2E(llr7WfKMiiw_ z=e+p~9zn$>chaz4`ik)51&;1`oYVv~*?Iax6p5zO1NSnR^3xY4xaapls@44GnHLAj zO;8eSiCXpLztL3NI|>c94R&j#hZ^jrb-y4gi<^XxFa0YZ{v%_EMcPdC;rJFBI+0$h z`Qu5f(##DMCK=IwG~2sev%j&!i$tlNbHbL}(9ZZl4{LOkrPCl-6^F=_Oi8#+XQZKj^@ zEhHzVxkBXLoj>uqclV;s^zKW@3}-jMi3l<2J$=@Lv>LjhW_7REv!Rw{Rd_s!VhkQd zm9+piQn`FP1h0c(30V(auxSnCR@o_a1CG4{lsxddx^EZxeZwf@ZFfQ)<;HA-;B~WQ z{qR4P_Q1C>HR03=)>Ot!JKZ`LhqRHfp{hvuO_7;V-6d$2whIEmOc z3h*!%O)&<6m7KBqLa+1FBU|pL6yCMa?kWA4Kn>~$p9LLf9LA&CK|++Vw}{)ET`yQ} z%VeG$kp5(!+mK-TUWXbBrgO;qHd#&u>N;o#qRXm{wVH(G8IVP#Nf#@U8DI6 zfT8g)(_lex_BLHU3PJEJuJ>9xmJAW(E~Q^T7m=a&Rzh8Q0htMXzC!98-%ZO3S1s39 zckDwUg*{8ru6<|+Ut z-r8b%q!tUS%>cjpRoo$lM9GE32Hnm|)f%6I-zFxto0ncQ< z6CRTxSVwEvl)c+eW3|Ke+Oz7Z0(2%Nx-B!BRHJU(H-OnWQxN+h98H4cfeL}EGC`$s zIgSGKke7AmG9zi9Gd245kEn9y_9N{Q3W{)n?aFAs(rDNRsVcov3TaiPB#6`wn2pvDRwaQ%k96h*kMsJ3e!#uQS@h})cDKEt z#xP0IO;3!O28SN?`&@^Wgq(5Uet>z%5xCr&>oA=gM4mR<^QsHXWP` zTUlF@4t`|uYW7G7&PAtkH`L=_1NA4wT%#DkglYxqPafWl=hJ6IfM+~KXK zQ0nLG1K~gyrYr7JEeH`3RgG;jH;b`QcK;G?9%lgHf_9p`iMB)GLdQRg*Y;f4b(P7+ zh^N=W{mPLESc0ar*os=i>MaYan!ytDtG|G{{`5tTN3)+uznZae18sz$?aJFURUah6 zPJ7UD^Pp$2^uOW!|9G2=PhMhQotmfbKEo1P&-gS1dXexsh~B?x4+z`Rpr1kr&D!)V z{-eJHKFcY9VxFy|_CMGE@51rFG0DsjFxIyj89x2(75}@%zenqTWAXpnvG9EG+DrAL zEl2#HJ=LGqQ=IHA4bVwKGmGi4;YeZ5$miE2r1!BmzM=++;G=(5^Zw+W{_!zXQn${p zJ^!`;RmJSOh7)xu=Esh|@@2@Y|A(Xh?Nt9qt@xc%`X9C8pNx?I=S*U{kna>ARN@r) z*E8%vPn|5gej%stXMzAYp9-+?iKX4FH5p9Wd%??pn4+7u7s_`GO(?6XEGcR%{W@`*fxB$(oxRZeh@MctR-(*98O6W&zo4^H*?Ynq?=4G_?fe#Ya`CrCMM zrDy#y1`36>bPeN(j@%`U&BFo}XCzW;$XMijq7?MV&5{(}kKByGT8jCOkVj z72iS=cMkePQf8%mcE`~U_CpQxQ%Ud+fPzUOF?_JS`T;-I=zN#aoY)4?oCvXw#KSxl zIdGmw;%x!ITRYm)_ETa8=xGIwYyHhBAiCNZiwpfe@&q#RaO<-V&SRQ$mu&h)PXJMe)t|Pp z#g<~D`TW5Wh~5mdv`;Su&Cq;#L^`N;Guhl3o9;ORII_m(13?iaR0neh0Hs$mJ&>Iq zf;~56XO>aFRv;1MS)iQ5xCJ^$eM0AiGXVhjTow}jd9Q6h_4+Jz{VA-f=lQtHpgE)# z{Gb>|D#sBJKKYyvuzyjQ17!xaseTZn<;9RTi%QHO~m zo4h3ab1VRUOblV6xvXVKwSHTSq*=eg|bop-^7I;holytGtA5B98LJ+ zO!5k(;6hc_@%T~QmewFLIY9(#tc9d8Iz85y`6pYUjBa;}(9?PM8#ykEr&}t|i7&mN z-pgxw0e1(WE*U%JvYCf|wm^2~6lSJr@KNB}2TVETuuOY`-+5(+Q2sImNrUr%Ol#{`@yZ_YJCsUjhg9pqUiz?znx`6VA?C&@(fExxq8Bs{Ife)NQO4w+5ERRHM6Dyhvc4;KLrS#sg5yrRxRE$WXxP83K;VV~>l zLHW~$`P*!~&MO?Le4XSmhT|@}>_pkZRU?_KZ+;4InJQ{lte?baG#qN8#WdF+US#FGHJJ8%#%gU107^ zp?WXON>kznt%n2!-_>A*2Pd-l#T0-mbT}{~r*fJK>t_ua0NsXIw}4~?T|AO{(JeLq zl~V-HsK}UJlYY^2&X=o>D{`?L)idoS<5WGIJcOhgyh>rF{78z!7?QNk>Qp7${N8VH z&^lV^1Z^Am4zVK6P`kuQ`hpK2yQ$sweYLE>-J*y{QxPp%oiu$8G+iQJ{Y2&oQ6lgrG((||^l`fYZNzlrm1dz2Lhxit~%u28&MAPx` zqV*$taIw9;_XnS=X7(r_@Y?7f>@auifoXoqdUT+J+Wt+E)qPzS=E{$m*7n7KWu6?Q zmKiaBGGdOlP1>_Gze8(7V%iNwFPG^>q+0`A2Ye`AZ)a!G=VGzBPIPnQp1a;bWkAWz zcGjM|zB-9`fMS7?>xKhzgsmsmpEDyx>lV=}1Y6c#ujUeCjH$_Jaf0ODrb-Dh_rq-Pa0 zV{EZ2RYazR9~q^YgJx-M`;qYsr6)f{J}4R$BEifG6V`^#T62U(ypglL+*7%=^hoa# ziwNV!b|AEvE9aJ{AX0!f@-}%1>$AWHC@x#+g$Id`N;ig>DKgKeSB2O032Rh~Ec4De z@Qq7SWrgG52Vl=*hkKXk32!*AeGs4-K^P?i^@ z(deUxfhZ!S5d3;wh?7j;J)ov$B7xucxV{i#_`0MQPPj`Ia>Fo#B?JXUqG@F!%m7~| zg}eH^$5@zyjNBbe>P*5+zvtC~52oh)tv1T(_pt13tb%?q6Zfv2O*yKRj@sNNst~sB zS1k;l1eUUA@bx{PX=|O84is68cX~OyT|Vftipx6Jmvh%zeM;AG#S)hGmQ=CHVZ;cb!jiwct;Ltz0c@A!gz>2zqe7S3X&7M|Bi6U}c!;9xp3r*)~han2kYduT8*xhhe|gvdXt{RaYK zZRE5*O5Nu`i^c@<<=~-zpG+K9Zb@)~B7{z;)*1h`v#DXufYhRO?>R+8&xE?Z4V}?1Y>q*54RqCkkyebc_We|mcNq#B^mz`5aWmxoLm4;@aKB} zz3Ye8TV^^ng6KMz8!9j=Jra9+O}ytdWZnMD4oYwfmr(>ecWINd%IQwD)N|hV)Du?t(@FMK?Cf5iMVNJoi8>@!chhcN@*K#53T7 zT;nCesQeAi*i(*#sX6<(8LxJ0qTm(#Skbmx+O8!J2=?w8l|Qr)nge?10JqXv`jH3C z6N9I~tomUfPu;paO4besd+6t2*V7hC?F&#O?<-~vL)6GZ=q{n$9Usvb_ZfILw4QA6 z_e<-Nf%t3b<>U=E4k6*M17!9{$OF|u_0R?MWxc|meR>AS>ShQ0D6w9vuBp|dm&U^J|I zgd%ua5CO#&h5*$idPRB0%^OW0cA)QdoB`XS&!?8N$tzODS9Yv4O>d-Bd*K67Syl^o zSJ5LJ+Rqo%cr7uUZELJyRAXP3e<>>BS~!kVrMR?l3I*pW;D`u!Vs_$I@({fU*t@>yzs9-(e~V=} zu*LNe%rt4N>$sF5ll&ZO1RtuHGAjMfAivBEjn!do@wTJa89~nA7wGz~UCS2>TCOff? zVO<|vUB6{+fEpc0PQac!P|9$=}LXMpLz~DYxu+A*!N*@TAbxx(co*rFku{Qk#t)3PU--x zys!K1?^_NmbWA?siP#*ZEV#n(s)x#7PK%Fhjli+op5AETXcg+OiP#?m2~RUwFDLWi zFecwu*DrCY*vtKfv3NX&!{Pj6)K+!{;JDCrR*SDQwLWH$>!%Qhb?BH$$1K0e&EQ>j z2G)7?m;>X^+}BmL#41N6&xpHX>)A~{4fWM_BQ+nK*D;sB9r)CK(_Y79cMU?JEao5iS0(qIaC$=yNi_8jwhI$-R$EQQoA z*{}1qpC#NRppG`?5V2ow7SUIAv+@|~6n*zad|a@@DhsZAQ&m9XeG6MiZp3DD@T2@Z z5j|2%O;&K=8xLN)2L$gK*!YX(JEWkYUUuZ;zHcZR@VA_mIJHjGP=W&qY{pWd$v=nG zaP%64JDe7~%qUne62gs6UAr}Aiz?np7UKC1jTuM9khomOska5o%V52ON`&* z)NAErM*UolX3CLLKG$oDEVfr zHfd2tJl*|CGKU8?`zqslr&^5N^~Dym#`tWh4(?*p01?#M)Oag7gg^9Dfjk6{Fg#om zP&}UwysVMl*hU~uh4b-9&WMi`;!AOomOX+Fm3hW;H(c-Ah!ms~Xm=Z9QCVY?_aG)u z6{-dnEbMaJA8iBTD=tfHAg)bAxnMZwNNTU(^J=1Kab{zxioE>r@I!n_vL~93sDl;V zkrjp0tI4ADarUJcX`f2{7Ad|cn3l`ZG+@P+*Xh&@{jvcEucyd**`P$%rW+3J7ac&v zX>*O>&+mOA)oIrd^=%KU{yF)aIXWKt>bC}r3M(p-Yo-TcQX_Nez5_&u zMpRa>p@(%$CITFRL`T-_;d7Ib8b zmxX!gAK|mzA1hlRK}F?ndh)&tnu~6xvO_X=pYgqC@6O-?l^nm!380^%L-F^XvOIF6 z$r@^YiXj4d1R89=UzdB5$8A;3DQ9KJlB5pFY}!NCJXIe&arejn8JFK363w+e|3lp$ z>Jxj@Lwi38hLtVC)An?q$h27iOTxJZRv zTB|2xstXR(a||b_3$)v|A|v77xRSaof8H6(`r2=6 z-k+w~GEz@a))yiWe&K9B;XP7;4y=r+h#5d#(v+;C+ECI_J{fEFv}3o-OTj>e7)k41 z_Ei5I$x-U-c|@h9iqTfIqWdnIe8=dbJTE37bxo?IN6NkuI``7Jf^T-tH{i`<8;O|- zKnS(K9PKARdbFox1X+tFF2M{;?ay*}t>GdBQrL0|fUm%bbze0i%9uzgv>bg)$f#wM z!&rDx`?nTAyv~kNc>FLt{1L&DPIf@n(>n;CS?y#*t6r>+!P56 zjKs0&D7M|@R00@|PL7!UM zjJbfCR@wW#5T~X-dos(5=1u?=nR4V3!kLx63__Qu)&J7|A)Zhojwt8G=W~!bY8n1C zCem_apU_uoXp}j1_ueA&;zCBaKbH+Vc?$pcb?5x06f4Wp`zMVu+icqraYJL=*n+6e zXm6V3yZ#S*ZxvQ$+x2^^C?O3uAt_9eF6l{0OPA6iAYIZ80+TN325FF%ZlxQfOS+Mq zAPwuB_w(HETHm|gb$mN(V{N=U9B>Hpn$GLI&N0q0#_#{PPjmmod$o&yBd$C7LAOT) zjFq2Bz8jiF@k*tU*K*+*|9&V98?D_k;Bs1iagJ_o>dt>oN4}PCG#YW006()wufdu7 zbGMoi3T@>hZX_IeOg?Oqu3aDqJpnJE-1a;hj*$Vh51ep14u3{(w)ox`mhZYuk2>77 ze|gd@s*xwyewdE~HP@!gK%4u3;mJvUI80YnYns2av;>Z`>yNXAwzVOg$W%q>jdw{u zMaah;n+KjPtT&jD8-=S7TNUWa{tA#-lOaa}Y{m<0CZEm|LxZ=XZP^F#h^IbzqT5>2 zdEgVT_R>gTywkv9oOAt%AWtR#L|iQr)I3D9zQ_-w8JN4WlyAFRbOdeO;ricVAJV$F zv_ii`x1QnsoxdNCFtp-eCFj3cgWhe=Jja54_J6km3Le96^e)4Ru)R~-Rb9p7Rs*6Y zC3z)#adfunYWSGQ8;}_@=DH@HFkQ`+)E)5;lVs^);zaUD3Fmb9R*Y`;iaGLDoLV@9 z^9I(xbvXfZ=GK_p;;YL`H4og_vXB<0T(fh6xlDp#(JoBy%-(QGDL`($+9fE7m@v$% zarMVTSyx{VZhE$GL)HkJkl!f;U#c z)`~c21)bAm8W4mAYc#!)O+UX9SmKk?N)a`?5R=0Un}5TDfrHs}AW>3#G%u$J>)Qg} z;(9luePz>)V2pU~41)?w@(LJ`7*=|p1ir_$8M~q_+j1yKcV@7w`z9QpGi9iyx# z0M(DiCNW0@3u#QX38z-g-KR_8Ttjpk%hGC>-b87 z*6z$hML}4docLwEqfrE|CQy=(m3HcFpsPu2*fsWt!k1AEUNfqLo@Kf9mo~o`eq4Emj_X?!8InT}QBu`M77Khxm!e#JS+NJhgXC0cYpPFBgN}(ESJ@qr4*SEUCSvhm! zUEP?c5trosNuT3@SEh*CrC&1iG?qIkKp{>w)yZ1N|9Su0xTHE}YvA8_kte*D=J zaPk7HLEOrk} zG;Z|3%8Ac3nPDtKqHWRl!#_v%=NTj+1_%ELstY^JGmliGM((Xkbo6_3Vf^Sw*R4OA zutYiA) z7i@x$jy=ycVKIIV!+fTF{Cp~1_EeQ=s0pVvIbNbO8s3IKf+y*w!m5u&J97{5WjiLO zBd>-_c(24JMozhKwAi%_q+BDTfpU1%ni>O4Qny~s$-H?)+65>(GzA%ttzze(%{f;( z5(d(i#4c`t?J7={n3nD108^=uT@52klx-v8tfTX19Ov~O+q4RXE~}+dC`_gaXos** z+I68Dk=O-F&=Hczt%V9pGUjA0abu=eS7<(9O}D#q?VE};9|~qSpr0i~CM2jaaworQ z%&%N^A)~6++bY3o+xO59f|QV$qej?Q{xN569|A%etADzARG(^246#&KIGrZi9>sEMKwp_pwZ1!mOYZh$Ik%DkWQJt z>OTI$grw7hU;xlfk=EMUHZirMyd&hKVo$DXpW#Gv3)4LJoy64$!GSWo+?{hZ*)xtb zmXib?cxa#~AYJ_Ly zTb$pxsq-iN2ThL{=gd4LG{1QW_zw;C{*?X_1!U_evz`bG??AKm=>7aW)!IM&by%_* z7U?~YPM!agI?f<|%<*TD$-XrABJo*I3ik@}WjKA2MkSTi$(Y2AXm1Gl1enHDop!DK z-EzK&JMnD<>^E`v=7w4WbUoY5ciTz<#r!a9)wSa~HNq@h)7ualmYjs#Fd7VtXY_w&fXCyE&8P_g=Spb6~uI5xo>&FTF>(`aDWbK0k8D{m)U42{KzT za|54HnbFg?Y_?_Q+Cj&NuHklh#d0<_ww)Qy@mJu4>Hhp=Mm=-cc1qN2 z!6kO$S9JN619QNr^xpT$1=jTdu2)62Zik~(;lR(1SM$Ea8QkmUJqTGs-FNxxG}#sf`almSgO;wG?NN8wf|yKSZBb4VfQnl#kWD+Zi?|5_qh|Mcf*zikGZy4+E%0{ zrn0xSCI0lzQ4@IT`Fv~eMAyiN90p1l$fhDHTLrC`xqJTdD9b7%Hw|)W4{pU%D;(Mo zq>h|fkA7GwRHs7aSh_a`xCOTN&Lm{>$gMqI>fb|X@{+o0LEkPuSwR^aF6}{Kgp~fc z5sYO!N6)_QQD+D&t3!S2PEv8UR2`NRSv>2iH&73xJLCvVv6zckNdk z`AhiS2x*_;=Qy3m zSK(ch( zUzCrCG>UyR1YP#a=1UpnL+*L|*xG<@w}T35TH6JDdCT$`jeA5vOY~nXRw1jA%Y}O_ z>YiIIpUHJ6p1|3%Q|c0LcPmn7SGR~RmaOv41TvYfw>kudw`uYB&CjJuh%_AflTUiC z`?ERnA^W1&$H-?yIY~U_=jZuSASuA7hud|Hy5We>$NW<*h{KW5%Fc=m&~SjnCXLBR znbTleccSQ(={704pk<*#b$S}PbRkJV;73z7g7cWi!r)lrIw-=d9Rc}02;p?H{)T7% zzR~cabbRUWr+K$q+j@oY*xAc4LfpO4`P~AYs2LMn>CkWS!`Mrhq*0aSS`ri^f9-qB zS#JBUP3}5PXQtsUTFXj%a!co_5ho3)si|LJ3_1K$Ps>|Hj{PQejkUNvvlrAeHGGe~ z#w5sAUY7dQ-yyoJAyotI)T@Q?x8&e7tlSUpXMfA zE+(iotb*pZOF=TZG=VNhf|oJr@qgON-RyilE&q9=v#$2S3thgWl)cGlIHh(>W&mT@Ku zh3oQ+(}!yLCT{15?*;D?0j(u7k2yTlx>JAn6uHSQYRapqmnU=3`-SI}-{Ee@)ZH!6 z^aI5pSbvKmMmiC09CdO$+p{j=;tP=ZK911nsWK0d4xnV+FQae$Js38s+(fKt>0vM? z9ZGJs<2e~A(FAFA-)}z>GkRV!WnzrmPhy5zm0N7-t}96JdDy7lj)XT>AtH*)zrMBh zG`xO|V~WBeR4?bTRN4Msu<}9k?-f5Tt5mTMIXOCE)|HGe2iS3d)__e5tf^3Stxr`B z_KwX7>|@0;wSj|W1f2w<7u=_WQ;D4e1GPf|59Sx_(v+=M zH}r}Myd;U)QGGMxQM2U6BRv9~;^xQ*)nMAwBH%Du(jsxUBAfnmD_g+;$rq}B=A`lS z+o+Ja6?Zmn6JIMRaMf^oS3U}Hdg4}Q!4p#PPCx2_%)$iHM8QX!`NyVU6`Z}!%_|rB zvBU}LY;V^z&71|)R9&Q6cSpeXtFDToVBhv0jWXRV!OYBo$)Md!e6D>`)C^8e={+;^ zd2fcPtZb{`Qcirq@hGIJ;kRsBlnbS0$5}(!Fh3H{@$W$5_UE$WPM|ng8?f4tGi+{y zA-XV8#GYD^;WsCj2&S`rC-H+#Gaq)b^jy=hjZH804aA{$Bx``aneRhbu9$KwB2i}+AH!K zBRWX;(2_@<>aSxiXxHyJV*WppH*5t+#F$JZ@8yYda@hCFg(%4@%V@xC^b;~;5i%<| zRc+GBmvj+~!_$_nQ=*g+T{APG*U!0mR!HtrR6k{Nw4ti5;To`jsft8Tw)AhcIPXQy zl3t1nSyIJypAO$^F(#flRkFP-nNyyk`U2OP8rQ^S(vH27wmT|FpdLA zit(VWi+0Z0_deTML0AZdlteh^#H<<3@Uf~92gVsS za}w_AD)*h)7rN=uP|sF)TDOBeXmom_aSU^ma>J;(ISO6Ow*-s_stxPkA8S^vw3lCw;Wo60*DK$U25Y%ZOf12h+4D-B}kxqjH50`m@@=?-q>M z0KhEKTnHZ**BX7bz{boOTC_)uYVbODG44f~wYM``)S?p8UVIrO>TN-Xh^$ zf%hwQx(n4R>s|!M21A;4vW1j6clrAcpvNxxF+H;wRF0M>C-P!J zF{A)Dz2Y}^bc*rk-G_(q&5XM>i^;Rfbx(?NjMUyuhu108XUyb+T*OAD>vm+28-yi% z5$TN*SRdV;W*1K+PUt6sJ=jmpa%O&?s;WssiNc0+~`W`@8-<{8R5f({o{BxsT5i~=C zFgjj?nob}Nkt(w}&V(UPIM6>@m0ZAh7_fBK@}oL^jyVQLrg7;w(glM@epQIdR$0#~ z|1Uy8pb+CEAX}M5EO#_e6TS$)kw~V}dmhX~E^00yb|KPf)Gwe`uM?IuR$Uw6Du(l& z@)6RAUfc#quNn*%oO0aLJOZG|(QIQ1$>V0ZHo->Lem5`96oUt?`Te;yyKK1=Ljzy! z{E(qg<}BIZ3#=HcnR^1Hmx$tZJyquQA0`;D8*N%CdM4 z$Q=C;g044T@17(0j@IA5P^!!Y@s%kwWF&ueRt3E$PVJ<^`drSvH&owxhrw-)gm&hr zHmB-zP1g=ukf21|<|6^qtGEbp0_wxR*=_3u70=96NeH8xy3%=%{UaEPN5YHjbh6{4$jFEaEfM#R zu?Us%ki|z-fx|hdGSLWFaW(UuCrIjY>GhD`ZpzCF=*^WZJ&U+?eRve(EvaYKbRZyc zSK)57Yu!wCA^!IJZi(I+o@`~l7~Aq(($zps^Yh$VWYi_?QfaaRaq9A zf*skvziyo;ONb2)Dj|BBbCKf^{y=bp6m(7USP;*C4jGDFO%T;%L4<&;tY?Yh)RtO%Oof^XcKBCCR7)*UIwUNrla3ba zwKmXX&aK-45)w2E|2kMrbWNaPuyo({(WVaTyG*2Eh;Z`Xg^8bBj#Oh*miHj z8RpCgc|)10A*Wg(>77jxQo}btHBed}b$CnxGVEh~!WG?iqw0u4+Ej3qR=QnahX?w` zyWU5uBTvk{RYW9YnXjv*rQ{=`-i`(uJF8+iO8xrl%u3y>&kEN>ZPH&=XW%*ZzHtJt zIrK_zus?@fV*cw8`}UCi8>j0uD1K#)_45{HjL1@ukMYJ%<$>njk5t87xw%w~tIFSM z4$cHSnmbK?mfNfy)i!&$OJz+5S|&QnQnbD-kKoiW8_7rTh^D~}+(dgO^algOZttBQ zx58th;vOJoUW?Jqo~mytSJGr{7Jd0g-&<{G1oC+YAxotBL3Z|47SD`;F|#q_NF1}0 z3ZbL>?KH(V0;B$M7ga(LRYJ(F>fV*03$g0Ka^AwI^aQ?Uh3@pJ_`2pgUGvF6LoTC> z*4o=GQ=2z=3qNHlh9g?yn5Fq||D=DCJvhxY>XVYuv<}WDiwU{Ui zeprDRXMt<3QW<3gHUD_iKfa_&C?;jXjE$Acq>^Ath3-Vp^FivWTyZneraxlaUNKyi zMJJVXgWfh;0g7Cu))?-Dp4!wEM6g*W3LT*k_|6SHynEZV7gRx^l?Mu3kjnlQ@^Lt7>2kjCkJBg7w2MRc2|)`wsCUW5%m ze$gwc&%M#quWJ^#>{C=g#X_hUtJZ30*EtebJ|;|mor=?l>ekP}uA)~Zut6VY4&0M< zU@ur@jle$)#-jWUIlMP~M7pBOZnxR;@fQfOWu)#93W@hq?dv6Lr135z{~DXXy=t!T zde%3xH$@;j9Q<%^5KO3Mgh&z_nq@erRIE-SXtE@tM_~-GVf41iTrny~uG40-yI76t}@$bp-5=t<|er9Al2ti$!ILEhSDhnDw z*DWv=+|%5{t+4~+=&ID;q3&)!Z{m0!%en=v(Ez=(&4@M?JznoVJ0Ar8mxKa_qxRIF zf-gObN&g?T4!fI$Zcw+(DgEqEvN;= zu)pW<|DkN^sT{r!dn1PT2TMgOk&Ru};tvX2snCDOAwJgsh5Px}UtuClbC3u&B?q1w z((RpB=D@29`XHZd{%vBQ{P>p{Pt4_iDd?XJ=)SuDqq|1M3W6HEpyu}f-!A$OR3ut> z{$EYy|NZ;_{pbJv(fuD5$N%HY=B_vxsWXt(j^iH<@&EMS@TJF_0TK@R#8>MgK?rNMD_}?DZZb-$VsF6B@*uD1QX=xMiRO|f@0fql;_VeEkJpXJu z|Lwr@@7?ylm&E`5C1Eo|c5@UYEQgRBhr$Or#sT=jKk1nPz$?Xht19_9<3!ytf^yfc zqHg2me=h(3eO9Z$=@06$+0BdmP<#}@*6%(tR=8>zST>gk-%#=7ty+-wPl@=_8APuKsofYEV3+raeL+IPQRfQUCUu|45Xmag^G$ zr}`gnB{qas7`w=hb?swBj?2&riR7ie1(Vl$$g$n$@4z_ji3@li!ayQ_OMBCl0$Tv+ z2Pq=#S;pi6z2}Iie#N?8?GzNa0BM(wB`^Iy|5*Nv%!{VogQi{DB4DEms)nJA2(K5$ z)Q`Ww&x(Lw5o-kIE6q4=8i(RS0JdaJ%_Vmn6_NO$e|Rc>1Ag^-3{gXisr~W(GmsKV zjkj<48DSGwJ7XAjw7Et67&PWsXLW-<}h)M_Vd6qF>35^)>L1Pl8Zo4tUEKjZ zg!gB@`=Aj;6~mqGiihX;3pQ3!vwqBCwQ~d@`90HSocib=r11QIB83fKpIs*M!~7A@ z z;Hw9PXsDdJ7my(axA$f_PE{c}teQoJ`oJ#AEQ&;^l<-6#IyfH3^2AL zHq992sne|B?j8nS_K9xCd_Wu;Ivk?w%0?D`YTEBp-??@|DA?7!9ZWxdMwy&q3IkeQ zTY<+WRE}LMcYqRm26~+o{(!r(B395<(A0Rg&NZY?c2J1Uz2phUQzsOaawEOgIe49L z^yin$(h8TGSU=4rLTM~;>#Fqw04QtRUgsRf6lLo7!K(o?iaG88ur*A0Ou*=hUE59U zcLeD+#?se*IT2mmf}djD$CMD69?3L;F_2)I(7a8U_+Y=HIeGs%wm`hS7x3a)Onr9s z(#h-BAHgusf%Z%A7WsOS%1fs=eP=`nUnS~}yDTp_jF7Jbqq?!9i{1&BG_=A3 z6P=W4`oOV(KW~0vlRHm9Q|Qx323!R2^3)H`h`->mN zn2`~a1!s>XJI2~rUy6r+`T6OGxLoS61j)6a4L^#0_*B^~Fe@s_Rmqs=OlW5N8+ivIa-EbLIOf)^O>wG+#HE;3-gC)#J zD^f2?W4~MD)&pvCU`EVEKW@%o`C#XV;?0ODmd{m+k3(^$sn3agYU@7mBXcV%y$O?` z%SnMuGc0!?`~}_C(Pi-^e;ib48{ziz3mzO`-ydOb@mfBCXO*Jydj6WC5*7f+XKJ~> zwv+Yl@Z_yLYH7hu3y1G|Pi@JMhkF%^?EOoydA_@pPj+UQ=XZGbhp0CTBex5K#h4U2 z{HmoG3H=dE7ZFRm?+Ik|I{a3Jnz#tFWSxF^d6&z!6NZhJ1B6SzxdoGsObf*^a(>EO zkSWi%3k5@Ap_97W$$hIriKuRPb9pC7Oq;}{HSlx>_H%`6i=tURY!usb8Oh;d4)3pN zzn|JLJnxb(3Vi)Mt{=UG&iklO7jAMTwvUN@Kt{yV+=i#wO!lUbbt zp|dv6FFs9n{3@}H2{8|v_gR=3QJVQOh zm8b#FrgANHNClcsVz1r9MvX>-CAL*G!#BL}Ww*ABX5IU19`+&GDc(yL->Oy%_^|k1 zXLG!2XLmfO`PA|K*P4GFGwiNSxTU4Y!m`4eM(!n(_N5D`&;zOD~>v4HPqZFLksZL5G#5UyBS94anxo0*wornFGXkzfb2U#?SEIDVGZ`b#m zGyVIsNyh?&w1Ix}F#f&h1NQKnv8l%`fgd7p`Z0&7ml@h7JstNA&X*|zMn@#tNvPtA zrSX#ky-8I&Y^h=1JuaR-X~QW5+go}cju_LsP4Ca9)=N5AMyK;3%a@@A?ugU^Wk~RfhEVv001a&F)j`?@h$-4_PTqH8F`O4+_d_v` zT;)sGEla6~E3L{Ju5}n}HBR#7g6F`RbHsP&+IoLTQ)y!zvCKzCv>9Rf;ROpqQ&KCl z#MU?pyDJ!Si}^UAl-B!;j1<8GJvyo8T>4=TO28wAU(X875#46)`E9Kga8gqG3yk9} zS~A{Gfj%AD9wtL}UL@KSeaZP)z8Bft#3&MhOKO6PnJV^)Oq6FGCdFnT666|@Ph*uh zbhul+EkhNOjZ}UI6}C<}Z|26v96%&5a1t+YqifBTf@+_i`zmlmsF8@gT@FXC-JzTI z>n)_Wv>78d56d4R;Er%+5|E6<;vTUXXWJYvIUdi?A4CA1OV&?^CrdSGYx2-Gqt&6F zCFWusW}MW?%9bS0ddMtS*&g+y_!5`F0B_&PXQ^yoQnuxcMZ}A%KoVvxa(?OdH_0g* z$$Ph!$IcGvr zm53^;p&tF`P5Fh$n)8QaISp5q%*Cf+!9R(?a);1ZQ?S%EpUvRDh+y z(R>uZn#SDLS7m*53Ym8JMZ(SG1ie}97o8)X z`YLzpi>Rud+J{q5v#c3r-!-3ZJQ}%aYP)U{n|Hf@o$?%8DX;`DHb|B?^rB zAjhFK5M^&l`OkZ_==$49{oBb7YcQ|9d-5&~ZZ+kU>koVV>Alh~8uLi^&!^&A0aTYE z0-}0ZIu@yaktgJ;FNIO@p`}jz95EgVc zB^l}NZ#nn2*Rr4(P$M#P)V39(zS4s}#VmjaBYdgItzVLS+w5F4WI9moNjnN-Re&=-`9&3^mupOTXN>r+Y z>x^A$^=NG`_y_E&Xc(Gn&3_T_alCC1rE_B^ERx-DS`4@&H=t=7H`#Jwg^eLAx{W6G zS@y4L9gLOnZO*;2cOae{BkLI0MrY$)Oz=v+eiTQr^v8P1`3tj^5E(W6Jy74#dM{uD zq;!Sv=2!g7@3Dy#qwlMx@8&zIY`tz-G-~l)!tJ#$>(o&5vZOZrd8eBg2tUd*@OZ2G zM!_Bew)A+7L}Pl)jhYdl8xm@1x z@THrfrRGh=JCO@*nS1MBoyDds)(O>|UQ8q9OSl-Qv>V0f@d&)6D-rxvLV&fn%hkkQ)Odqz)mtza^?cKRB@B?e%&Tj2znHZAxo6OUaIxkgCQomxh2%lW=nl3 zaz>u%xGv47pF}2%j(s1ZoMHGU5tgVYB)MhJY#eT;F)h3!(QGLg$3eijG z9f(kA%XmN5ATV{A*Ad_S&Q^DdJ2pw;sW>|H)Ul#)hL1uPlj-bhhdmwn8NwFa0VM+0 zlS%$nm%oe==hZCXK&yrL814oI^oQI#5C_i<1vPX@%G3A@ke=ZNT^HV%P1H#=nKD8c z%*X-=3Db1^;~6jw8QzpUn9G+{U)i5-FCzbogbZ;|HyIFx!ot7jvnxu3=AjE{)`dz6 zOe0KP+IBC_44)k)u^AYP_bRJ1w{t)Vf}HG93;$S=NuT%Wp*^lcbZ7 z5F>Q7u##6p>Sf&v0r7efc7_ir)cAzg^RdUCH*HVwrVJD7F_JUhOTifKvEx;}5~s>S znka*ZGX7CDr|n^mZ-0SQYRho{yv(J=GQ&hX=QpB0`EKDnfgLp|ME~TG>RzxqbRu{r zd9}~sy59Ij`kH62u!e8iTFlG7aSiWiwY#0nJ1~|GVwk#Wb8zCK;a)zNs?zNj3*Rh| zd)>R4{)Ta#xzqAM0%M=qhlSpUfe{+Wsnw^?<+YUpVo^dUEZQC?!EU`lgYEe%_6MEp zOdbe*BmFLj<&|2Z{pV^Kp&+CYV?mXE?i<7BSy4F%$sTn-dP}`%mj+}T+=Kc=Y&EIL z(jBIWy-QsQ!Hmlx@bo$eetM%lzL96nB1$)6Ly9zXsHpk9TBw9*T-hIg`uP#$LOH(z zSsS_g&0fm|Zsl4W&Kx&Z^Eb}in+WugRFF+Q1D;x;r#+w4q3*y&&`Co`>bKaZy^(3l z>!ms>7~ac%UyV1^{dxGqe9m1$n=+%ogOTFyGA#8V)9M?OP=mjr(-v7 z_b^ZJRYm)nP?4$?@rat$4_W#Y1=ciz?qZ;4Ec4G`JbP2r7%D=_`7!quf8<(!qj)f= z?~YBx8U<`DCALHR>eLS<4B7qzf@%Ok z-aTOXSJ2|uMdy|#M77^it*$T`d+Ts}--ocAxARVYXiQ&9_Z6q#`F5*RM1UPq9ruDeo_bmp9A;eU6NNO12qH(?E%f`y+L&fC;VmNw-;5E|(5Yz;UXi+93{+A*SXN+lzwA zPx01=)Iy`re?OVX<$}Js4Q4dwA{E1Z3AJs>vBOiFG_;7o30;;&X@DNc-fbtw6^0E( zbz+dT?kt>&(%B^rKVJzTT4fhyr~b*VK0aZlg`NEoYVAbAHX_&BJV6~-!LpHsU~D~t zo~OJj)QZv7x|`UlOV{7xJzS%mF2NN|HcY7QiNT(Xu645|InJRWCw{&x;6@X9F2DHR z-Sp5xswA1+T)dsO}BAE5QQJzgQ;#zt~^_QIy=Zt+UClOF2cSkjDk)0j5 z9xeJ!?iS|YiDUj197?)Lf=rKE?dZcKjajElDp4hRA+5zZSRpflcr$PCC%+~=gpAD= z?Rawinm8%F^6IDl%dVVgsUg#tV_D3t$0JIo-?=19@=TFHv}n)FQrxU$KS&_;gj;Q` znbhAkWRC&1_(E8sq9gQUbsW0?U&mGDnGjfe9F28?(Cru|E{ZFQsJ>I;M3Kwv@cY+( zS3%hOUl-X&yFJ*5wNX;q<1f~pwHSShy2c@JC-Mz^n7rh{s`~>6nJpA@yRwXs7ZSg)+4h*P}_FQXG!MV%AK41DQ-TvtaTD+iC zM$ zIu9R5YAs6PN+pOs9lgZ$>%hd55)6x+M!hPNh|xS`)Sl~~BdFevol=o5Z%iYUFQSb) z*;nx}v+CzWMArAo$ybl`wLS^RnW09JC^{12YEw_F|H3sfd&BI645GTMPKBPVX-nPe zfCse+enuk}5m0L<4&hzssa|Uq({+h*aO=raK65PFri&QECj8*$ygyNjD+6iU(WQ+N~SfCT&s?YQySOe`N`5 zK>SG^N!npYRaS2@%M6)2NkSp&1D_m2`+EA2a(oF$c2UzWX0c!ADeDg;@3K;xXq;19 zz*5unugcVD-=XTJ2)V>^CSLi|XF^b>VO9ELetRp4nDXw$)u>s)!@(M7{uP12vA!>F zwoT<10&!&c`0aDgo2Sh_B8By7G;Km#civaR#90HEd#q8jT8ur~Bi|&Q)a6H|ZShlo zfAeM4lR(tyEY9AZ^L0#PawMl)U?UUL-@)Ur9|xdKA}He7`6JmL)92bZr{=P#?uA@K zH;3h|r+K>8K+?R8X{SY^D+%8&5M=m@WV-?lRnLg!!p3#AAn$B20%Miz&u)w#5 z58f?vkR+{MsXB?mlvabjB^`B*IZj~pHnoEGLW;jXm(!Sezo)`N?m`q!X!MWXN%|Yv zJ?y9h_zd(BE?faEO9OwbEc!_OF?0ys!)7WT}P*LSa`ebynA$paZ$!Vwv3m9(j#mD+vmL;7fI&h#cT^CmIyL3#b344pdm z1gYkEFpb-zeMv;c8F0RxIxO_W_3)KMB3DL+_4s$GOeXu5>RL=p^^99k_d!8nDmHu3 z-3W_(=k)du%=QCK>>end$jOQvk$Gt>w>3J}Rv12OaMVhG^)AGA^AoH(C~7zYmT}yM%zlMc>!>`U43>=1 z-1gWbf%;B6!K`Rm>+}akFt_!Jn5RVI{`Fj<+bSraR8C?$8U5!d3@&>Rhtu!HNml6& z@iHbcT=YG|@IY@tb!ee`+o80UQB-b+d&^RTsw=}-`PraigUn_8B&aBaHbPD*1BRp*A(Nxndf*>88ylDQXnXx4IkeGgX7_Or2W%k8CYqaOPQz{ zG?bpS*Jv^{{wAeXmMM#&YGEnJ|9*Otwk#sJ<^{}NzNB%M>02XeR$Qv%pG2XWvX#4P z`rsatpu{Oe00_<#%Emf|Ugee*TOwz!Hg-M`GM9~D65Vqf!|+#+OSk-$0?m4wh5L$Q zW_-lZPYStzeAuwj`^4gaGT<=HvA^kL(CCuG|2Rhz8FDXJoAJ88RaKNuQ9PUy^OU17 zrZMzK!!F&SEB6Mif4WA5Vapp*R_d2tnXN7Ane10XOxRaqx#U+QsHMcE6)RLds6CZ6 zwtE~SAh6YajI)z&yIr=riefs)A)0Cs?um+jMju9B-rIILs{WphZv!`ww)!$bC&Anh z-W?6eFid-$KFCz6y1Ki~N8#}eF@8sOk4sVqdS#)WOwv4(4nzD;e_b^=pJ3e;d7nMG zvvyA$emsu3J+mR!!5y!8R*xeUc@R9(6)OL6SQH5%h3{iTTXt+9l$fioiBavC)& zu>`mA#cJo*=>E&39NZ3mHQWOhe!8A^(X}9_kKZ2fsW!T=ld3cqT^dJNf|B z8eRq)RbJG1sx9apdlIzu;a7j&NrdHBoX`N}eyLW-m;vU5wx2yKd9ctB#dRBzEgdCs zQWm_*QC}C1$+)S*GuIrz210?yfYgDueF1DVHNSyP?;C0hCRLn0&cMi3mJuwEf|}HL zr8gRRxrKAcdT7EG784B_d5IlUpvVW4%#W&a6dyx>^w*S-oeML}WIR>*>c=XYTIW_w zu_&E?c0Fq0f;NwQf77UBh=#^qi4f~?i31$}JlEe-4mGb-{9;C_xY4myN?_ON_An3Y zHqU8Sw}zvzz{q;&8kpBR|E4dK#dz-|gDRm@BHM$*l0RcqiH_dhvOYC|_6{}TO8}SW zi7r8;iXAs^0KLTmc_o6d{nQ@gX#}45V{iJi@7&966V83+ASAE>wXpU$7^;EfHInwAg zrcjN*%0lHUBYDk1XqP~ur+~XYQ?J3pD&j|#w4ZwB=# zar%9yM~2YN?4Dk&8F_z$X&AvyMtN&ITe8bSe;aC6PPe;c18#rC;GXW3g~7h`)DP$% z1Nl^5Ng<9~R7P%?JcDj0E7WUNq=iT{Ir4SaNsjPUj@txeNgF+@6W`z~<}ay7lUQu- z15R&y?OY~B`rwLQMj@oK*{%Ls{P7_7k>=<85SbU2Ry?H>TwNiG`ufJCGa-WL4XLJ8 z1{@olmwqJg`1MF;Ld>R+*NS+DF6G1m@OLnRW(My)EjnJEuan}w93UCp32S1c;&cl> z7wiE^@G{C_Z=70x(oe^qu$-q^w-25YCMAas^PjRzSsh7EC~AR_onWv)3JB~F#geRK zXE~vZ6W03^K8j+Te$J*&+kNrLkJikvMX;c#U>S2vMJL5=5c`DyoX`hJ0=+2#TfEMr z!!7UDqg>yR6ivmS&F0DvpYOOujK)NKu29+4g%xRKoKC-`st#?NP4lZ!TKl$1HI-yn zxe2*x!Eu-NUTr9N%wHxAy_hS%)w*l32lW?qqC`qr@+;ZBi6#rv2`N#YUnORx2e9PrZWb{ z*C*AR5?^$O$+)vkzbBL}*FlvZV%eUcD_Mz5t~viaSj9kzUQG%mY;6fHxTK1$f1Tdn zl9^pS;o9O1U(Qrbh%%orByb+A6avy&B?n*qMVk$ddrV)vpZ;hfB`FEr)6(G~5FIyr z9ywxHGLAw^;&{vXEJF+u0{5go+PK7xd?O5z5?XDwh}Q-U!wg*sdJ<;7;p+8W1*g{u z`Q78@o5?i}8kqx!(Fj`HHSed(AXRO?vU=AyPHOWHEPPYf1J_fMPEeIPL9RRUL)V_) zlm`6&T6pGRE@am0?>aupp>8x#0jj=$;Rio>x4pMydU}u8xm>5TsH(4I7qdzRQ}$ZShH5PLV0pDKs0otb z{@QStd6YZU+kiNVrDs?KE%Q{R963}ge`w!4H-2mb2=3DtkryA*KTNd_S6Zy-i}eqN z#f9XKa78|Rm@#jAq;Z(Xw3TA((!NdvdANo z|D+zRd%S>+Ei0I!qVGn}i=nnir!)Z_;9bepw$0d&ZG96L3|f-!r5?*S67Oq}`dkhx zoXh2f4?M>oc|YC|-2|!vE)nq_wm+`I+AhKz>WiO@_99k_D;t^{yHE~E|AmsAK?wvp)-z4P34SS5wZ1fM51M3tVR#dR)1KZapQL@ z96nP{LJkdePeTGdWq&ZTYbsda5}07fwINBn2RP6xvenL1#AR?Zmte|D40a z%p|Y)(`N&{T$hQO&g_qQC-eJA)4ysrupqT~X{MS^99GLGayt*z$Zi;N_jNtuJd{h> zzs7c;aZ(?c5&X2+*AmnMZfT6 zraT+!ctxj@};_H3x@}tZe_}@ zxnNUjyAzXD2CWn?7m8hN6^MwNm*yjA&6V0~>|+&%9H^#h!Wag%sEsV$*aeUV1yYPNljU<_q3e1*97|gwL7^(1%sl01K7yj%YYH>}jj! zh8z^~ZG?s~by@^nFGyx(Rybj=Eb#o|+FQXMxzGgAP0RNU4WjTLPx((?_u(TlSKLMDIqZ~ogcs#^kBw@@gu?& z*m9>EYVFGYhX%kg%7=1cM_*aRzS>%54QN}_8wFmVZW>#&Pr*;AN51OXT3u5_|J;TkN4FNtzV2+$((_sWsHM-? z0h*3TU8&jml2MT?L=uCbzkfDdEdB^H6YLa@*vcY^#7;@UwWk#DE(l{vE-OOW0DVm6 zLNpfC4sOv!SL2M_P63+Iu@Eh$JO6uksp1`7rE3TK5-ahqp zXvxs8o(HU9!WLU5_Pg@A^AvA zN9CDkx(i2Fr&;w9!5k=qxCI1|H%5TiiESL~#a_+1g_5>Z0HI75f_%1hbsFi#nk9D5 z`P3T&{$6;S4AbiGNd9Vj0RPC452c58x~8Q@`mM-*_j)amkP#yfa;HA+vg+eB0x9`w zXhC%WRbig!j5dD2yWnYVjgXKlc`%}mdkzv zXBg;Vp}gMXd{rkT*!5K&DkZEG;@b^2Wl5h&Xz%@UeYTAwg6_z9%!?byhNygZ0|OK@ zLBcA#wYfj3U~K#U+1W`ia zX;!Cmm`Yf=8$aZ4c@z*I-J-%tE!b6@87aiU>P$f=G3>9GQM;7GC|od%`L6e#al$bT zN_|J2V(n39!$Xk!SQ}L-eg}p3gLiOUz_36*`B6@Rn<<|gL1+;9kH@7uy_J7zj>Yj} zkH9eEAOnLg=#=1~_(wg25jz|GwIc{wk3Sg4tKhSJq&)DcjUW#t zwN95|PBigahV3M4OHA!AGelNV9y>JCws8U(hN7LEq#1 zhq8$Fd{=A4?L!e?n$CH9+8k zyBaIUzzh@jb_l@kM=+d3P{O(95#(>?+Cui*%HpMe?AHFwnPkvl(%`a#b72v&pZ=NR zBr}xyEF9Q+9d_AfgZZ8ISZAgFw;vOok=pdA;Xd!%5w)qVx)Iq9eSTLITW|PwprD8MxT_ihP5d{__>i&}V)qh@ zTTBw^Pv-wnr;M2EBGS1vYoN6*y<8)wDG-LL77VD*I!vdH+!{DVMnAaO8t@D+XklbYO5wA`3;ZO0Ff%o zXOQf`vD&*DbNM@>_T2o;$jeY&4R&EG;Mb z;!;RgzJ-OE9Axq4-Rnx8^A1*Gh}%9yPDi&J4#{r2!?(7xRU~&e`_H>7X@_tYaK4Vq z#+4lqbgSFks1Z#;Od8pmp=MJ-t(8ovJ0|5X{HC+0&RLVeZZ3^rD~oc54jY$4sy0`u zn(^Aah4Lzn&itde;%YiA_`W#akL)h+5oYvZKV;dAa|J-lD@^J(an$cG zdcd7;Cqh}m3@ugnbVnRfdqt)g!;)Md0y@fHoh7C~9`59PyY)_72Lr5T3|H(_ApGoX zP^G-0H28^U4^6k3yEENz-q27=Nk7U=z~7L?WhyejJTK5$xwCi8mmO|vj)gfxcV|5@ zV4{nA;-TuT!-}FXC`))jVRo>;Gu>KVZ%u4UNs8QT;h+*KgSA~(DkJiP)_%ndRe^lF ztOs5x;i5X{C`Gvv56WxB(h94L%KY(D_L4%756i__-~71e#m~DSLr*3|^F+yeIvS@x zR#AS(LHeOAxZ67h>(3N(#`s`aDQyHRgS4y+U!}^hEL-lOL1WYiffmX#SvV-u1*;9t z&dfFw@pT^1C=^B*cGtMkD|bbUPgTb-*EaN4!xz}X@)MDSh2+53@^LUmxN}61*XR@P zMw}{dDaa_(UK;E{c+$3FDRPN9$7gcP`&wp#yxAE{+Q^CZ{dRd9dGO-!$9*Nj(A`ny zoc-wmTJoNSmd_Ic*?rp6UN0MR5*$BK=vbGUzUrCbf+2-(Tip~nyD@nZ?aGFOvRo4x z!R(PfH0>rh0Z&E&wzK~3dBqNFhg0qtE5~?Hp)?99%wlm#G|qhkh9XGt`OaanhSWd z-3OZHCk}5kJYlSH2_^0iQeUEnH(v$S!;Mvq$&5I{?i;989!M6**9V=$1vFdXhsI-o z10ci6jZMX{l1Nf?;BXTSXaYP)IUzuV1-1DJHkP=_n|uD)G3f+y;L9=`5qyuToCqx3 z+<;6fz?gF->Dws}?s_W$NkQ0Qzd};U%RPWrPgMQ*$hMmQb)|H5Wyl?=1qSM?lRebZ zmenO|3Vh-HF;0tNhOqI@7*QjWP}z@}PKqS$5+E1n5EZ4v@Qb!L2?4rXytKa(3`nfi zXRq14Abp*s2lQ=WXVpj@R;9RC7d#}4hNCMYOAE7i(zE9icFQOPSCgn;8(N!R3~@da z-tFS|<)JzU>`aIO?Ax~gS3r0gU?jh00=kaByx1KN1av1r*$N~k-FpRKhgvbX$&qv| zG?D;F6)?03bGmI0o%+1BY7Ni>!)k>9p#qpfhu=D#Jl9?F=TMqSJS;k}hV;lHnN2(g zeLUTn{%`nnM?LmC9(^0do0E0%+eQfWpLeA)6Zt>>ko5l|f@w zUqMzdr2ZGz`qxkH0i4J2sM>}PXYq{iE>+FH zZcJc2I-A{Oshwm?yz6z)0j^LL90U}a&L%Is zHCO6^nfMNEU2AHb5^u2EB@uZo@wV4NLO0)>KB(*6tDBFM3e1I0Ka3Ys z`6G-1P00acjnG*v(}kl=+DK8m#7j$F?QRxr+Z zvaR!%foFE4`1&|nCF*dE=g0$1&wwP+_6B0Dh|dE)%wia?oxjWJFQwti`=49?Z^MKI z7)`1c)0pvM54; z&MhmJinazb(6SpbZ-^z*{r{eZ&K4o0KQ9U&&L0?fhNHD+cmc9bpI^LuAtt)qV1n2NudLsV9J!?iE9=QVpB%{{dBPS^fY3 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-autoscale.png b/website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-autoscale.png new file mode 100644 index 0000000000000000000000000000000000000000..ac04649f35f9087d0e4ad90fb37ed80b12fca0d5 GIT binary patch literal 94179 zcmeFZc{tSV`!_CGDwWl3kKyEwVEt*~xA&gi^`YV&5y2UDnLllBMiqXUM)U z!&t`lTw}QJ`*YuSpWkyF&++@`dA^Qg%=W&vb9tSw^SrM2ywFfnBs)QO0uK+5OzGB* zyLfm+C_KDlizGzg6V-~i0zAA^(pK{F8cOo=EE-M@53Ov?@$hcFh|nR{)%;GAXz)5* z)-pwzw49Vp`0-l`eY_ckJj&O2Dc{bKaJfCWQ(csN{}_E4*Sp?h`RqYY74kO-DXGnk zczPp5Ig{rqQ03o6g?#s}ttJUgweTK)+|L<$>uo^B@o?@Y+WJ6(w)%_dZA$9L$OV#5 z`CcFwRsR+o8j6>93hm3kWRHKNAL1Ntz3aSFNkhgY6Lu2s44ESxe^}hf6;`}uuAE!( zr}6Imh*NK*RT9jSRX%n6iN~w677dOOXVbs0NXMvp^PC)|!F#JE#^6YVm!ZR=&Oh={ zmaNO?dBPJ(KMFiy@oyDhzG!+-q+ho^Zg+!18GYVLX8OU7s*s)To0{+E>QF@uL3fi` z)Zf;`toWvl`9=*gd=Ed(t$XiEZAyNJa1COpCGiYjEf>WGX(!uzmhLB;0gpKfo}Ej5 zaQiyd`1(gy(J(@1A{Q37QAhk5R*@%Sy3thUQ${*F2&-+M|00)5KZ)vFp_r3HJWrxM zo1WLllXxa``Du6hOFb1r-$eGvtq6?guRNo1hN|At&H0k{l-)IQ^zukI% zkJYWIG4!NPZ!l-m0|oNF=NI!kHhEb>Cr--ak7IP@%EZL9O(C&yqJ_q`&S{SsKUEQ< zgvR72wyO0@8T57@@%B1ibw#D+}^z(?;TF!Uz zmQSVZ>sm=)p&nAeC#k=2PL96VdL;h?z2b7m#|0s&h7fp3MZ+6L3?XO3J4t{q9$pno zl=hg1#d67JI$(dCz54O-Z!E``{Q~^Y$G*BiPx)1Z{H(tW>u3Pt7xdWYJJOdQ@AQ~z zBgG%P^jZ4Z-W2h`cjO}>zP4O>Lqu6@o|jTGFnEL+c`eV$6mc!@cKyDT%r_s4=M-1oW`<^2)oeW^3)`7bLd&m1^G@&goiTXA z_)R~*vzKuGTsPiyuTmDJO--os%JQNo&w!CU^1k`X19`@3nH@4CJ*!Kr@zx2R2^9_X zsIAV_R?Dhbwpdo02M;DYW0pgz(hHs%*1vMG;IN>@`>;g2H8!UjJYViNT?m_^**W&O z^?P-7_50y3d2jIMHy)SlzgmhXJ6<9F_L!LO6K+b{hHHlX6eF6~$&glt>EGNDNJP;tgnMAtr<($g#f+}wf{2x_Ld30ICaaG!RRP)D(Wsi#ysRcNX zoGHb(f9&;WW9>0*GBe_M5+wmEtJagiY4%IcFWwImV?Xtcmh?JXxbB7XN(xsfSlQ$r zvYrgT@_|7w%vR1ie7Ave{HoKj8QDb^qg(tax~(VoKc2{VN|&NNefh25mG(zhnG&8q zOxA3_UU<$ZC`Tc)eaKSEldk*;QdXj!b{daJz+8n`sFp#1sP5~#yt6!vRA$b3ceC&_ z>t^#Vc_ieTCg?J+Q*w-6jSrQmVq^&77%}wbk|u`u=hnNAqCHOfkS%@PUU4Acstz|% zUuMfA9V0pa#OleJ>x8BfzXX2qAfh};mY&~uN8TPgeQc@r!F?LXOOB+D#~t}s&gKX7 zsxq*=x`nym_4+ht&^OkJH@9!i-GVD|DCsCkeK>vl$L&kETW(k1=DdA9l`Zu|s?lwZ zw>ex=_b@8j90sq;8}2+{f)YW`L+QV~@QxXj-&1K%*46aByz-XkhxZTt+l&)D6VekQ zpFe+&`JC<){8<*7h|-^c{g{e?zm>_b%;^1;r*-M$Ev?9md5vqSPKpy`5x0xpVKg;A zZv9OCNjH0bR(_WFUTnsDE`x}4ah;;OMaJliS!LnRv8i;LcCYnQ^q=<26*XGJW*@k6 zyGDvD!a_Y$F_dV7+2eFqL)z)?(0!p(zUpRW*yfR@)KS=Rs$<-W#>T!&@_nCbjpgK3 znh<3J*@2r9%5&{()_OMiMV||Q6s}tq*`RG~tl`!i*6KYuKc#bqJ_XF^&g8i|xkSz| zw0-%ss?>8U>QiHnQIuh%ORjpY}XwOA}9NNI%FtN^?bMTbZ!l)*_-4$lw*2Mk=MWlS{_0C|uiN53MvAfKUs{3PHMtADJ5_pT>|q%Et!+5Z zEqfK#)C+00>gUffU-dz6CX(7y-M%y!Zz<)7`F8TnsY??o;?jC{S1eG~TGy@b&gT5& zkH4%YY{3$epdHcOYVO$c2(r8b&GabuX7;(ZZ&jx53B?qgN(zoRUmoKVLmdOdk2EPC zxuSU=N7JQq**R5g++f-vwQ z6Q7N%HrX`w_%oyI&d`77f1QIr?El06bmy6L#`qjTBe8P3z@gw;p+p)Y+C34fr0)rR zl8r9iJ9)lEy&3mI1lG^f>+Pl75Q(wQoDItU zmQ84oWUD;bjiDbai3P6GSnNB*I@tb2$jw4IL{YVwug0j+x?Nw%ZfTJeo;y!E*r_|E zQDUCkg8U3KNqR86y8CI{fr&(>%{$yNZ!vr#C8MUWCKY{aHyE=Q*c8}OH(nRYZmJj? zhlnF*c9kp}i=N5~NSz6HJ^YVBVXZxp$&hzQ;I1bTa38BR%bYyhodciM? z)})it?r9Nyo+w1?mKiOAkKs#0Y44-&yLVx61_Qr_hkRrP7m>!kePd2z)UDjAjAr6$ zhG@7aMt`Kpxu>s(f9N@)-BIY*;Ld++CF!E9=CqrC6depk0lPUTd@X4lBvMTfQFd5<|vF_gt~i{OwoG7Ow+ z(^GZJc}v44No*Us6B`*vw5R$i>6-8J?J1^P_l&ArcPsHyKYR1QR5@EgLJ?%HtMpJ+ z6^{$NCcz^(Mu$fTUL6Au>0|VNyuNwtGT!ln&++l_o>}1$9A2Xap0R)7;DK#}dp`a; z5RVwVI|Cjb$@oWC6QPoiAH6=d2)@IU)09_I0?(RePUhzJ&Xx`?G<|{CmZXnv={e)! z(Okqnjw#)}unhX|vC`6Y(N$FuH*>J#GkxH2-<;3G?h&>hJV_66@Y2rQ#gxUv&eq;p z+(U}(;0kf@8v8Ln8_U5ZE;dqZx~dv1@(xbsEFydYd;)CJCsiN*Z2jl@L##Y3$EaGhT6NBdhpsiv*S8B>gR^J zvze3CBNr?Z~P7azom&+^?y225-P&~lLTfWvIW$@N# zev=!r^<%9wX|M1UiiTJJ>ipTI(y>sN`TY7cY8^wt1;lELwWtb5#IGSgVkX_tIPIeN zki4}q=6#PwwDxa}8BIGK$Fiir?|BCU1~m#wa_LaAddBGXv2qU&{V0oN>G5&$Tg1%X zk3Ko>tkkEMASQT^U+BbWjfKL$6M1x%ldfjarIG%7_p(c*YYO4CVYc4=or&FT!w=)^ ze!wf~PNqRe$s9c7J*QKibzF+G9p(E-(``lwC1!Ndz$`zLF|^+bVBTRVUoXnF%a5wo zV_-!pLogl1@yjt5iG3bB4-MQbIdG8icb&*SmQ^Icur+MAJ(GS*er-IQsP~LA&#RpB z_5Q*-I-9GJs#D*ZVuTu595S{L_E_3X;LJ*25FB@A162~kN;^6`w7aJMg;|J>N2n{o?&`$`wf67&A1Vvrs9 zU$6d&dt=BQQZ^PU(ZQ9ObtQEbv4QkzT+lIc?c%iC&-LBSL{P1es(XAb-AxK*;SZI$ zdiCnXN{J;zL1B`xO?N45=|KR=xc*Z|s;w4yesS0c|5wT1VtpGuz7+HLLS7B|%=jC1 zHRs7DB~he@nfS^#s?N&&?aC<3%JZ+yqQeODi)?EWvmf7sqT7&dtM{akq8)s-MIwE8 zchTKyCN)aUcW*VRQN+Ohi+R}|X0CCDp;36Y*{M8KbB<_;1<2;o+sG0Mv9yj|fl3*R75{qJ^(Ora+NAag|@TV_ev=3 zR*P$gpz;dgSGAsKLf@EbfT&IHELNu>t{e$2 z9c`X-**(#%1LKaAY~)gFZSpn6twK7+&JI-9SaJ}r+mgIVPb+6U z7OrrvXWGJ^z7ZwRj~UD<<>nU^y7U`gBe3S9aCbq^Q>5G>zZ@hnL^nlw%$SFN@hW}U zTn?PBF^+?Vqv53n&-L`pg7MeNk@eh*X*N9Y&C!?77;{RubK84H@4Pi1p{^c$N^30U zvevG6nmVlBHf#SIhqR#WKs4v#H+rtcb?$+)?m=V=72AvQGMM(TK(=JeN|>*t)?jUS zZ$+tu-M97Hp#b?>cF9H&$~0>CB{FLF9x~vy_vCBGl%DM^4DV&<@8-Z4#qV#sSF9$4 zp!q$jPyX10_-swc>lixLwnbI$Or(gV3xo+c?d@!_7dt41)3($ag~>bmVBPH5J0i#V zjG>;2^&-mpPztAAbdSTD%&gWRp_t3!mnm`>IupX^IW%Y7qZ^f*u0{+sd+N}hK1-2e zylBCs6M(RciwYC5w>KCnvHBewOP0p3q4LM9*<6?E^B4I`Q)*1$vE#>S z*NhTCyI_1pKkrVgjM^8s2ioL(NjpXTUY)XzS2xe(Mgfqw{pA++{u{c~)dsGYna=l% zrH{T!=(;!IGN`bTWg(SmOC>k7P%wOt;$AV!!*iQHM$ZNBGP4Y6~nw8ya3r5 zr9CaRITFwucRIo82WN=g>d>pMn+qpUY%aD;dm!Fi#cXs$RTM(e8Y>M=H;3ObzaMhc zm2@g-ztmFDDTlQDWth`*qpnM09p5C&E_NUkui)QFF7UdKPi0$)ppHE{>t}tE5kWo|_}EVB+hlhkGFEOildvnEyt7Kg@7ey6Zfbajg`Wu;vygrzr#sa% zJGhuXtfz@u^mm?srny|1m$jK!ITZ+yjh)&(EtM@^Ne2RR<-(6Qom%!g?aczibSCjd z899Bw_^EWjt``%{i65pVS@+q{SaR|T^>Dt(J5`=ufV`dOE#hAVEj1{Ea5Vf}q;~9r zCV>7=rR@B>^AWcD+t9h$Wud4? zBsReZGh-EJSIjwOW;ht5o#^$x`W#j5*d@&s%cjq@SYOccrZsHr_F5>?;ab#HZUwQV zlMg=M@p;IS8;hh`PkgLEPkc6a?KB4$qI|##B4K=sdzm?0b7#>(DIQu@Twm+zFs-mf zXR9RGI_K1TB7U#FM!LBSD(HT#`)2BBX!Y5#bMK668AB;`OS06Ft#gol%o<=V$??4Y z2rbE2;oR7DM@J4*OjoXtaO&7M*B#*8hiW#mHDx{iodg?pR&mu0SAmE zyKFXN2E?Ju@<}AVvCl|q*v`H*t=wI=dE(jSmOz4H@)+?e1hGF-?(T||%S^m;%O^L} zZ)*d{_2RV!2VeLp1Pz~IbkI@__rML7dy=;cPZ5~O5xjALHX3v^LiSKv-ZIGO6eO|C zX0;2TW0BjO6*_!U8(&&^)6+3MOFkR$D|y^Z`!V(vTa#DBUczAe+r#Prq-8oalQ2KE zyf=_}LsQAT*VAT5P{ymVJs(El_>E>a!*wJV3*$pg1o7+61uYS()NXJ^cD~>3YG!dj zEZY_C)qS3%;KwgNep)i`-fA6+M1<@&A?vYhGvO?(W@Nft;@1-@x-eFTIf-SFA(zh2 zLs&X9yHVyk+8(>I>r{JDV~=z33H|O9F6B(DQSNE!32FH(8F`#Oi0= zH*4KzC!}fIEhi`6he9^WdW8GkFzvBl!rq*xWku$#KHqv_FJmh&^xH-75_+OS5!ofk zvK5j>whzk6{N*mMAi)gwK$U01ZUz5Rl)>EL`kH7OBJQY&3=u^}ck7ykpi1G3Wj2@} zIu%#4PB#lNZ#5`(q)VeeIjsQRHn;Ybw&IcQn^7(<=2a+`Z(xy*a~;Asn)4aZJ^NlW z*0KE96_4g~_V9&b6lGGE-K8r=-X)NJyONeobJhC9ABbn6RHhs{A+&rmtZPO zMreJWKeYW57QO*JC?Fk%YK2GHl??lAQ+j11ceTA0kv6*Zhm zJ1k0DAK$RNTyr{|ElVXm-syH9!h^?ROagxlK8IWm6n)#czu~(tT03O(=6t9$W_m4g z1Cc)Xpb$>5T9LRCZ1F*v@5_UqtMlFN;|k>un`7_!DE^`!-mG&OeZeug7<$ef2;zlL ziP7R3NVD{idbF!=hKjfBD-h~tzYXvY@zJex%yXuLL&boAzjf!6+WM}}g!Wouve|9W zS|RySm!Ouk)npNv&Br9yr@K+U1as&un{8SJbNf7`wb<}-_iWftTZVL!{tP&i2n-3G z67D}p{IcLviC%$0RiAfO*GnqP^AXU|pPK4+w!Y)96h*XkmTnK2K4Zj^u~9LFY2m9N ztxB04vRU%%fzxe+Y|kc6lzYl-piA2z+g2xC0`<;$ORXA+-TtFT;q=Gdu%}&1%=2sR zZtePsERTKjUqD+}q> zn3`hFMLrvXOA-r(oq4X%$u{}Zc7>c3FZubIM0`f6l5~spe0R~$cw2>PH|m6$a<>3; zv6%*_-jGQ*M)EZyOh*#5q?@~czZ~7o9Y#6jgznNgZ4hO_hG=Bf07(B!)xlHq3m=m* zW_x(QqAWwu2u5HbsU~z9o0ywb0vwN89j)8k+g@T0st=qrqa7yg)s5mPLt8eB?fC+; z5XESFw+?F%d~KFr*@cFPI5H)Wqu$AMVJsSF&WR?$p}*B|BEA&$csV-T!R`sK<3|6D zXQeNBiP?Z|?9B>Dl2MF4l#i&xE;9s4wAg={h)iJygi4;0rf$dzI-o6wEJ)g&zZsVc zTd?~mlX(JPD@)%Cv|L;)lUVOHduCrYlZFzY zyt-J>Djnhs>uWR?nP1M|-zk5RzW@h(H$|vL?7gJV;DoLud0~4=udg?U$!c3O%ThHl zGlM?h3iPl+$9hKPRRPQdzwfoRy^YFBN1rdog+3o~tmti7X61q*}c^Hj(mRAinp z`*>`@?DF4{)c=sl=Q}-M54RHs5%R9%l#%hzbF`+$Tp+|@*+JdsY0cH0ne-&&=3e2= zn){<1GFwXh=21o+GJ7jwb)%z>Z)T8QOLk55?lZ7CgW=ox2=U+zDccpP#|kEqd=;Cc zP6Y-I<-*iqGIsOt42EAtNdV5g!&(lgjhcc-3@sVGA|z?~3C9PJx@H*~6F@(aTzFPA z^Q{S6v{?gnFzZ2iYJT<*k#^}~dWYp9k1q{EBhWV}zLCe{Wid`JiD=)gq z6!lgC=$)3D;8f=+zeOL3dN2kIhiAa05k0>F=I1QAS! zz^bN#n15XpuAQ|HdXFV32jp{lddjUkP z4yvV9+kMe>H2#v)GouxlC+A|`%U@N;PpjlN>&jEEhj>O!>JI##+`8z*GYc6-lwg^8c;m_0!zw%o&2QS+$qA)|rFD*=ah36!I%TDK*B15O(J$K;I5XH!Hw zSu_S2;@p7$qSU^#he=*(j=$#o?M_g=f7)pKy9ty$`LzqR-i(1O6>*QUq$lC)_icaY z!UUe`V0K=@onuNTc0gjHTCW2?1>Syvj)$o?F=RQ%6bV?t;^9 zAiaI`WNVVlO#NH6&&qGCrTYZbAB%Dthy1QGYDY)_E#eIcEz%tE8s0WO@W-!DPtwCf zmM#Rt(WpES!qTx|I^j+E5rEnjWtiDatRZ%eF>REVx$Erzh6v*M9Xm@gPG-`caOHRPZcwn@1)@Ob zzG6D_Tb+=cN`B1ZfcJ9o?Qg`16~Qb&OGBn_$XfXC*jNQ!f^$K;TBG(()f4JZU8 zKOwT=I{S1ptiKmv6&~2oBFdXqi~0S^llSob5tyD^r~iupi8N%xMIS4;|M)K}N7GV_ zcU9i>y!AWKqF@nt42C+!Q>OX5ssiZ#VpC#n3*uJ>(h1L-`z-u#IH2Etmpc^;R7MK5IOi|rf+pJm2r48u<+X%<*WS@ zFNSy4-Dk3#Ta>1feC1+=EG7!S*U_hnR$A4A0A1kx#*h9aiYRAX?RWb$z$}IRvW@fY zJU3H_JMRibG35ssA=kUvJiN@zwoH?@Y{k((J3Byxe4UtIjhX2nj7pDlsNgIgr5^aO zyD=cz94+|1eSNA@j2(tb4!{E6X5se&5w#Men?czQ*n1J5%ZaP-_-4@w3jA%B61S-Q z5po&D&KG~{{wjAynDiU=D`>_ z;8jr8Wiq+nBUjt?b! z+Xfwa7D;b%Dt`rH$zYnQ2vQ8}%fjj0kwfbnmFYc8d1jt{%Eqsy_wvJn#Zc1QFqQ=0 zYQhlNN9n%4UiziWi&q+K>~|KHyhUprhbxz)tp;?;KSN!ITTh}e72ni;AF%Q^@tGx- zsb-oW2@#RT5Bo;jqv_MSbRgLt-jn7tU}s3Rkq2&w`EduLGM7`JhN*cNU^;Dr2Ng+mbDt{VTVueq?b(~J5`zCC=)FIe%`#tmi1=#xY*7SYv zdfyKv)W=4X$$;f%mahI6%at2>L(Kd-=gVY&;l9sy8oE1fptlC-|82!igcRP2nq=k&wN$e|@ zrFro0?J4cQ$${?$7WPPQr!5$l+x5FImEUA=UIIlR+OuKZJ1r$)%9+_^)}DRZ2I=Wp zD{ozPN^T-wMKNc88J6}+*Kyd+I4~BYa(4Cbo~5X$e;SO78lZ~TY!tBvN&Jv!qgz;( z*qL`d#OLCjJK7a}IJ5J1vet@?kgr4rYzthb<&qfu(4p=NkvXNKL2I3wKJlOk6JcDw zJJ%6#T3V}V|1tm>Gu`d?p?kN*SG4zAkS zT7AZ7i9|q;1ab^tleTodY_2oxn97K2MRC+)7lC-hpD=WT%aBi%3svEnO zDgVSr#|Y*sKH=*d9tk#_W;6VvX~xNJ#3Wq!J55R62b60UVrFr`7>1~xhud%Wl&5$T zOo~?QbzKEJ#5XNkws*IzDo@YL%zG`8@6S(lsHZBiD#@rkjtMs~#nazhN z!xu0)g`IJ3JW>0j!H_g{@7d%v2Yzo;@lDMn%fj+aeqUAi(h717F(t*%=rdg%G7)f< zwSoSFwS>;!iH0mQ;Afc9Qne5)6j`%AS~g8m1BH_Gbh4x`rk)Gy1F6@X6XR(N-JqBy z2ZpFc_(tN|(^lqJiA5_n%^?nmnY6V;DP$B4(hK4_HV26l8MLf%X@BagvhNHIP>h9$ z+*b5nUoQN4S2@Ct*?YNq4dS&Fx97c5i^azXk>Q=%nY1{2wPDG{0fo4Wo%!|N*Rc4? zdrEwWRL(fPmND!f7-^zUKmhIhYMp*S(2`k~L7gcT?B}?MPyE!>RGBiJh|p1D-fPG7_N((jyq?fCYcO>oYOx2<8 z=@{|CEcbVVcI^jsMJ%!L>)%jd?GvDG>L=$4v7A5I4c{FJwI~B)b%I)ccsDo&QTT(4 z`}8&wVv@XmW6lWFy=oJ@TUEGo#NbeL-qwIarPMC^wQ?sYnnrZ%!qBF3M2O5|q`?5)h` zSpDx#cwqY%2m)!&*|RVI))sW@scr(Q$~EFr4oHOQ>cgW8$}h1r%oPRA@c(1vs*l*U z>G2%mg{Llns+!tP*T4G#o?ksMsxZCiLm^<~!G+AOE;N5%ub&^F{$x8B@o>z4Eul)5 z0IFl?dQW_JO2*+V|D;(oyhVJ%W9Yu5`;aUB>cn+l%6}wuYj3d%_>o$0r~m>2jCjL# zB^yFwm$ot*uf|NZ80pkl*sN_JRa|xUAB&PzCm}(3zEcrb!c9WudNO|i!kgH*V%7c( zQ%D+qInh(nzpwCL8ZG%F+84hzhP4f1+9eiH-UKimL#X+8;r;}#uLvbWrPzO@qGdcl z&`!q~5^<(y373TsLczfc0yj$J#E89Q`Q*a;asdZP;kkb#SZE56GQB7Fl*7%rP(9hf zA2CV6_Z7Y|UpDvlL+r-IrnocjLymO%hu|5hDaKvj4pig5@FOPyXQ@QnM|geRjZ5oX z2wVhhSEt!U8)Fprt&tW_{He+da?6*HOuuj*yzB`Bgv|!II zz5;Y%@&t6q4J8R+_c=#Gw&%quwR=E?2;d#AQp|AjQ*Bu~K-PUtD# zkQw;s{Zkjm*I+;YCOj^Oj!5l7gH6|^0JN@1BK`^2LJ2?hHj z9&P#VX@oj5S0@(iKt%M*ClE1q*jjtK+fASO!W9V1GV>ssF91LPOEf3&RK;4&Cux)u zn>$){Jg3~48?kq2rMs-aHp9flPvY-#fmc00#QE{jz;?PyL_BXEI*KZGW-E&y=)*qH zjVcV}f32#O>h^%;AKEjE0JWcWxTx1wpH)n1WXwHY}&|1TlxyIMekSWca}$c0viL&^%8JsaL&>%ksOs_I_M#1SoGt~7b^bZRmIPvf!Ifam2C7ymmijutTJi^b6N6?ScybUORg7~ZedOQ2e_Ry<-YA5siBigcw zTm)sbZ_&=x(`C4CD!^;|dUWgaj|?fV6lan&IzPbzRkU%F)012HYoKV;$GW#z?W6#I zr0S(7FD16s_P1K;!)D^=*V~ath&KHJJ2`)Z+N=c4F?`koBXTzSbDI+IF&I9KdYrJ) z($ttIOqM_J%Uq6uPnfaF)Kb^WJ zAm)KM-6Rwt+iK*Y*Lql_nI^_Qs^=d61fxdp6kHu$Skt<2pM;AkaToT&Vv z8%+c4k)xahM=4{m@w)8ux%zPCPv_l{6V;3!f1J^szIfQJmK5w_8}nheA2SPB+A;wr zGR}_oIYjzw0wT5s8&_215%PN)0jqM6GtSk~+9zF0Jn*304eO8f1Q;dYrRrTSe1`}J zc!(kKp_u%bXbwyp@T;G$_)=U4dYNm#J}crU4MHzki8X=f(ENbY2wzq?(wL_@Zn&i` zjWE)+GTd-I+!^}>p1>t~2GE^eI5K1M0U*wY7Z;BPOr*pOD3scH2Oq|c8!)azoDyFO z+(I$;{qxLK+<*dspy4To%p-2$N_)^yXCr8a*>8`^(ueWPI2sa?Kh%tGYr%&j#)1*8?Te4d_|NOOhP*%CQ4tW`#x>NK zaQ{qT%>f!3DY-nyHPlrBBi_gFkH@`bX;a1Ne<&-^jOv`q0ni{#jEP%fWsLxfasBkf z=WtaEV9iC!9X?0WF`~eU^(mX!1;1mo2ht(iB>c5T>VgI734AfhLp}Z{p0c!E!ws2G z;C=z0fQj;;<(Df*{bJSDCJi8Xn`hz0(JUQtO&Iz8j-NT_ftzI&NUIGMTL){5EWZkj~IIR)ZgGvA+ zP7tn7hUCC>0ZXwtW+r$j!2;lliM$D+N3M6*8XFy^G605k;#^kkV3i*Mv-sl%13w?& zKxldDgu`GZkcY!oqV>CvMhVI;;mlH>lOY<<@49UG)RjZgU{t{@9jINUaSl`^OM2D! zF<%4yN0fvfei2TRUGI3r$cuxZn?FYFeGYX@o#2kqt#97dQL*RD{bOuNC|m26aoBDjH_y3BItg}s;Ja9jca`7AC?;28t13ni3)~4pqC247pBc35Fm9f6 zI7m3ehnS9;%l}|gz;$G!hQA+bp_~ur@*bDt>hQE6OczL?DI4w@kEMVF> z$B%S=`fz!_?$3(*G2xbnwq$jUlT89RiTMz&7J##QOYoNN>>nmu&Vd{50AaD%iydx` zy2}^8oEHqYfSEaDI|6sX`Z~UyIU2C_aDBKX;j9d;YL!5OGp>VpDU40Kl1a*rf@gFF z&UrrF?g#Yc(}{ynm&RrFADhd;fNX|b&3|&_q(V6R@=S9Pzn$!W3vDn>eU9sg9trGA zmJG;3__9CWksqevpwc1ZbuqiFNe0fC4@UjV?AU?+*j0Isl9jb!+~g8D6GG2S)8oj| z@Z0Ve6w^w6TdPy2=Qa?~)xhSsKN=4*DFboFqcX&YpIBUPy(jlrHXX&b-$-3lahfZ-D<*IZ|LJ3vR`Wxluf>5`4s*?V<3P&K@5}+f zisE1#M_@7)^j!R_s<2y_#cOIf5MRp%?HrWnz8$uc!vV7>E^b`ZfHB6F@65YYMcFIS zLLAlKzoBnV=!zDROJ*i`*c2@|Yh`JXF)pjp`x?@Gpo_+#zIc%vgbheU=l4VF3b>@Pi)GQ4 zUG5bGv+NYP3Q)1VLW53b+QB%B5Yn}$Rqs+1W>OTw2Cc-Mzo&Md36nNnRgaQEr5MD2 zV98}lr8%la__>i9u5Qew6=oLc=sVvYaJ7Cimd!JH;uomfuIZ^0YIkdXms7byGHuFCwfuUU6K&nGhsGQ6kfN+R+ z(!_L!IV4cD4;06}0cTHGGb-#aG|bJR#IGG%<}jf?EU$MK0UPPPUFUT0A)v)w+u0jk z-4BQK*&F5}5o18&CY}Ts-iDVFE9-kr)Nj6G1}mJ0VnjLf_iT6?<|<>w57l7um;l>f zP!)0LANN&d6Y7@_dqV8&y0eA-5%%9D_<&2MDJBp5Bcx;u2AhXR9b>Zz#suRay%y!# znY_$+03Y$r_HiC6dY8LxHlSA9V*Yr&_7xm!^y|jYMmm(|7K;?%Zfz};Pfqeje3!gi z(AQ!FZDtD)=5L_S@{}+*TxRRRv}VD5O8>IV?zJe{ol`0>^gpTx3k6D>b*v(P(3 zpWzF}z8MnwZj)Cg=HDsEchwrj_|%?EdRyE#|H8`AUvfC2C!(WkGT9%||Mt4#5jA*2 zYl*JDpDDFfl^+p2abWn#MY5S221}XQ%~^m6&Z1>r6^I`ieC#bvJJ3>q0$=g$uw%*h zPVHNo(5#0t+$t;RlX`VT)TGU@bKt8X(o+LdhvdC_Hx#!6%g+W7+jr=niAZ2onZDf`-> zS=;1a!!yZR1a2HiME8MX$%YUB!#IJm*7~3b2PuKCs=m>FcIf!jy90TtD2&8bLPR5) zJNmh^uca=eIln*250_Zjjp0snA(2DA?ZrG~mopxI+wtPaO)rlGM95{8Ib&(CpV~>_ z9n`umWoO2i)x$+A+8-DcmhBlV)VC^dpKm0khz^pKA%r^aX>KJ-4u^?GaHm%mQ;wr` zq!P$*Aw$3fUt0Hkp1*_<)`+Hqu)6b8(D@-aDMtu3L#bGqXU1}8Z4aY)Q&!pWQVx{;sD1}7djm9RhgXFG%**LTg<}dp~(eUD<-5#$ga5& zdI{7yI>!4Crda#u6e|g#rpqCk_P2*IE7??|%eR_I1<^3tSg5@Qr8%cp6r&>q~#g6^jjzo=dFaaEFG6A>txa`j^#A9aj$INAZ9X z4Nxx-b_Z1wtXyT8NjFVW!-8hCuy2!#aZ^Tf$FSt?3R}7EmWaPjdWrK(lMk|k3A8l| zVMLyBXOAH-se2*t94>Up@>7nJxojmI(fBza)*1APSsARD*Zgr}-Ob?mhXsJ~OsoOm zefrWDbjS@fP99^<`uXmZ_(5G4#@9Bxt8H?p^6N=dKje*ek8dXX))&i*X4j74FPw>< zpSueWl`z`gsvJfpYU^3eIBk^d?D`cpKU9y;k?(T|1L4oJ@KKrSp(Bv^ec!_$3!v`7 z89+1w=dEu_;um=dpbND2uDet0N0pxh3vdUH@BDV+Q7O?TO9)KL&yD2mEs8KaJXRj; zZvg~vl>H1fb8%wclWJF5KuSikgVf9m7vVDmktfzZ>cAPie z>m*2Ao*EeKZ{6xj3PA9L5gyzR3u=DYE?dl8??bwJ?&QWLP%M?3fo0@A^gOEFT9T-& z)1IqjxjK!@dHqPSo`9|pH~ktae0IDjsjLf~6sEO%t)UUsOWc7?;s4LNy-wz(n+*C? z|AcX-XdIQFmONqt%>`I1vV2+Pk*jQX?G~~7T^J+vxGP1)GxCcKb1C;{xDMG_vK~$n zDSB-i3A2ju z&@tCSqu0A|W;x^i?(Uewa?NoTfj)@Ij=umN?Nd$c_K;sOPAmOxv$e=ySfE`n`vYi! zFgM1hSg^ZkZJd59#!=#3VQ!4|bC&Q6+wL+&r%Tuy*$LjLVe4Fi~9cmYa z!*cPnA7-tFPBVqyg3p_+kEuuT+h%QC>?b=xb+9%Q6+dm}bC?l3mExQ%tWA_-ZQ{QU znX|u*?qu^hETyC5dI}k;)UZOqdXE$~Q#sP7%e~|-;y`JSIcvR}Ql%^oT9LK-y+BqH z|C_QRsKX`%-_0%_Xw;HKOGY3bFwerKyNz$>{-}}n+g)|nHtWCPPJpbatqf$`EEQrB zwM90K6V`d*KjDy8LH-2WX*`LtXc+V$R>p)O`1z5>@RA~yT(9+Vr*VBE=t_VXV}P%T z`kysbSITI?7Bk_v?JqM#+#eE3jEbQ7hv)Q@m69=jrmhhj`&z=Yg1a$B%uL;{U}1Vr zy@;3@>SdWA)!f{2DwbssPhol>t3F5I2Zrb*r$&Ouajoqp|&q#kn#!aa- zLg-L-081`>FI3G$gCJw65$pqh!k8qbvpD3=TdE&8VA!KNOcxcn zI`Q%wwZmxlml*|vNz$t~cw*zo|M$dk4P~Q%RE+%Hdvi#^*6_`xs=xGiXw0}zsvdv~ z-?VoWGlS<{4TY%#L;shniDW^ZjHlohR0A%o5o`L7UJm2dL0ea>02;6k^!tmA&;}VLp-x zu$F69m9TZST;%$_p@j!Yj7H%-7KEYPa=-$^X0rkLbs#xaBF ziLKc(21ISyxePgjtVVgBuq}qRi3uojh54k4GevB9!M((#V&BCuP(efkRCX=lTNdMrkHd-djjBArAstr87XEpAS27Vk<@iJuFMc5j3gu1y;y%ON zs5<$~yE!N{IMZ&`YQmh7psu*N+Q@hq0c)Dt)sdI2Ut-+<7~uy}UQkCUGk6(QiR!XK z-_ubV-o)gyi+>(db5JcAbaKYbaLT9^JZz@RNBVyPvP+wM$c?1{e)W#8mEhpq;>z%Co2&}Vb2N<#D4U83iy9p5 z^ZYq`7d@8ixz-esa4J*PQ`ZnLMXpu7m?3#^Kc0Cdx<_FfQQ6)ja_VWie^ zb!GLlrNL0rSWl~rwO+PGkuemS1O3_i$CUQ}d2%t7;Xh6;hKP7i9!x1YQV#qkAw;^U z;Jihqj8R2i`~5L=q&lS6p}Y?9AG=Nmvi3t#KG(PyNK)(cDDwDc1W&6X#haS4Nuee` zRh}>r!KCA*gCZp_(2mcv?diw7vm_G+wCl|nWgKYD>XUr-KkBT8N8Q`av~%J$gBj8b zj+=M6fgih>4hFGFsO3H5L@*>VhJ0r{y(7+jP_W#q#686F%-p*W76*dn`@GZo;Ttak zPCN9i2<7%=Oq980>pFgs=t$#+wB-!-o&HNC!x@GwF#((GcX{$zlvp<}rCjs-U&iE? z_Zi&V6_O*Lk(g)9Jqy~p7!>}n{oN)xqJFXIECyM>5 zS)7=?WLiEQg$T3EB>RP6xAqFpM5r|IO9vO0;>IcJ&TEz##q@<^q7t=RGT@gHYM9>W zl{3AAc|0>RM$SWMw(l%iGRreX3GK1TCfDF^m76>e8~zo_v;K=o|Ekc!)>C-nt_xSm z=Jd1mxVx28`@g>a=P}E9Gwd5yA=Jw+yxShyQPN>RlrtMVaK&zj$)0F46fFK?1U|e^=YamB zkV?NFd^P01x4M zNHY%3jxQJe@7?x&>|49BE@M$Xjl3+e(_Q z%gtI|OqcK(@5)uwh5I=AgqZEjUsE;~;qA;l?LRn_rD-<{^*Jpd-^sk2N^RL6s0JrO zPT%6{$@tdJ?MAZ87&ug@?+iQa8on$)5IqzesOxBR$#Y%dISm5X2z@LLz=Wt9HMqH! z7JLR_ghQKQ;<(PU6+POJd7Jd@;iL7Oo-aGAL+Eih;i2)(Q39R$yto-}9oMYrPOcN} zh?T_5CZ^6MF9Eav;-Kp?sTS{_5OlFl^72pp2Y;W|TP_k;UeiZ_NRK<<+l--8Q+X+} zJ^P!}m2mB1p&y|M6+Y?N(mpsK>vi&1mya{c4viEX@BXN-;Q`^SMEZ)gpZ29KeQ+T zxx2XCIop{h{$Yec1+hiXRps&XS)t0~X}G?^;4@iQm}~}hm-ZR#Zi_zoh!vRl;ilxW zr9opfczUzdY}Ulr1X3j&1CN$p4BXo}vXVGsz-B|~kg)+*;RJ{5(C#IV5ueTTdGPA= z%r=^C_q^3-6xW(C51!?o{n7?HWSRvJ@NZzZ*3${d*K4;QJtiws6)|~y8)N*qgVII9 z#Cxxao>#@=+zy-`>@i&K06p8wTRD%I6|L~P^3_x_lRkBFXfat$_3=k0#BZ{Y0llD> z^N7<|1-o~=Z^A2gUrEp0#_ZL7YWhb}#2iMR7@WKryNgCfD~l#93X zo$Uz&@4bnhrawNpyA&46_n5^=hmqob(eC#YM=eyiDr=nb60T9NxbI|9FWi3%(p7`{ zTENSqC_QglZ7V7(e&6!aUdtuF)C%lg0Ji?%;K;F25h?c&BG1*$Jhpa8#%Hp8N0OrRdnbTE6(`ilOG z#7)l^5RClI_)zw=JArqVl@7})t9MR^0Zq#rxP-3FPSCsY^2KQfRqrY-;gS_{@0>Oh znwBb0Y0t!GuKc;OX%F9coWFarU(q!I!s8KljgatrJG?3MT6AeieI7I0^+aTjp5|Sk+X{)UaUF}! zZoVrJvX!QdS%0{{b44(FE8S+ev?fohq;fi9N59fTtyOSG%%;uIr7WR|18(W7AWfeA zl}z#t+%UzWPO>l|R$af6W~wf4nux6@g*vhC_+0CW#vQ?h4Hu6BSZ|s>xh!JqI~NKa z{5dGhX@bX%3MsA>yDetwoDT&~Lr&!RD-qPXai7UB_l>{B@Kpod9 zjjXAuv_h+8a)`Q~Ip2`0{R+!%j5CRKy00McP^Rg5)7KYvYc&7OfG_;b{Er}5l2^pF z0k}y=3w#s{sP1@Qg;!ENz4lPU>_Js3iX=`WUxnm^=fiPjRv-cl;#sG(E_|iN?4VP^ zHX8S0wkc1XDEcpR2@_xW1Nl0xh$Ws+DK9>oaK&EC0l!W2>5)gfe){6pwug)joj4-e ztNbI@g@Jsr;KmVnEiLClKMqZVyy&o8mP-K2Tvzs^8g$@% z*%N83QTlo$WEzZ3jr`-}1H=q{fohQ<_d59dF8b;NrTe^AG^<8r-mmDjPLSUGGnbg< zx_Qe-Wm}eaW~|ZM6;`!)5Mg%BRbSb}*|V2EpF-*1%$j|B@FQ4a?5%XEY!muY8$On; z2`8R*m(Z5%*7V-%xbUI{-qLFEywrt{8E9@LO}R^WOLiZ&O$*|^Vo2&ur7o6xK_-jP zldapqX4FLRLqh7fdNN-e2V0qNL0v3Y-R*SMwh~$JMEU6!H`d{Srs*x+<>xB}b2pSu zRc{X(7P>ovs>Xw2-MPlsqp87XA&u*+zNm#rzpuFS{P(9k)$;NT$`houD)=QU60D9-5Gvk# zneWugigv^qN)|-l!57l)U(;yAfnu{-k;IJPoh0hGKCh<=jLy;9=P+4-tyu69H4boH zynct1r=5z&MY}tb7pgt=@VM+)6;pUD9vFQw@u{#Q$XRb+)j8mDza@Q9>cAaJMRw3v zXBl#FUbl(@p3vi1&#QpL&RVg7WE;AA<-J-hcrE4Z{<+1^z5Lzu(Oz(*=R`2T_frR^ z!XR8cyU%uyy@|KTk^EzeTiy^zRrWBS$;ThsSj+5wz`6B{R`t@BR3?1tkkD?y1p&$S zznMvML?N&n{T^ggwlc`7Ed*vSSiv}3#^fFe;GAqp# zhb4`Q^>1?0(*+usI3niPDGz1VH1nSRe7M6X6fMF(CSzJPvKX|jYNjGex~ik+{Gvrg zx;e-jBr+%)#$Cq~p|}VZKo92XLcn!Zd#Z7U`VrTm|2cdl`ci;wp35mxjxS{44HCwboE}YNWdUIs)oH3mm3NeZyZ63%Rzp9 z#G8DpNR#Ty^{K-T@ue{M4isfluYF*ln}MQ5TlQYv9^vStQ(FWGcwpci%fT{m*~ts@E)HWbA3 zbC(_v&fc<;0Q^^p=TCCv10YKj0?Jca@(3uP?x#}k7x;hYjR^!BZpsUzfGrAKCr zSt!R@fqGKDk#InvQpbZ{BTqyDu2#{svoX7!);MjYCo-l6QPYgwahMjKVLKcgkufp0 z;@^1BgF<^}!RvU^;7t_Bi>!lTWihtfcbfjq_;$%P{VAZYCA0bY|B+KU{z z2Efj6zP0(2Co)icD)xcdc@Oy{cmJ}VXe(3gZ=KR;Q~YCHZrW-JnUzc6X2~(Y^B)0ei^rYebe1KL~^munIp`Cha^A@i? z8bdi^8MVu|FpyaLyQ9YV&%bZb29n|Dk73+^AcEVW>VN!||J{r%eicT4!BaxL)-Q&~ z2%!#&fxDqD>`Bo435rfD{^-$dRJQAm^(>9kr#qd`Hu{T%x2oD40m50VIZsWKRDwC6 zx0J@{)5;2WjEq$5|C3VuM>Z20{m@Hz@N%)br6iUYr)p@Esxv8=U-zTzIzFW1`WRuB z>XyPgf7S2syC-~m2^|b}T$~^q%as(ZZbE5ARJZ+(paS4lS6f~en|j3nd|;+=!LhOg zn1hk(Nd2z@xrk1#rL|U6+)*{FhuEmQY0NYw05H-0?ayGG zyY`g9$T)}T;fz}4^KN_X$SVoT5nX+YaA52TRNCf0=OtBv+Vc!H;RyiHHyI+!8wR~I zXKQLEPLTdoJ+d49c3165R;l0ko8ucXoMW`jxZ%ngm04p?msV$BY)VZt_!t&hT{~vn z_N(4&y$Jfu|FmK2%RDqB(A!&6gV&Xt-1lbaBW`HAwZ*~7+ zz%-vlh4B7KH{{<)*yiE%5oKy5a2xSNoaahL`VmU~eI=gX&BVKq3Q&e*R@6UrAoMET zzf`GT`&X@~bG8Yz8y?w@iudOO1kTR*{GxNna4nR02XIJq4HWD8`L&Bz^KTF>>swzb z7=lB_;f|(y`*z+Y-q`^4qpi)c@5Fi`VQ%gdJ^`1Zkai$^%L@9W#`VSQ8)LiCSGEawBs%|WN#RnLq%`jo>4hB+UUBrc zx&Q?n3NtD>R|!ps0wbNiLD*B@E*P0{8~yT|!1uJXlM&;A3JXCkCP-M`NgsahmQA`@ z-k8`r!fVVe7?#FWGZG(TX574bFwF!ajU#;AeKfkRux!`CKzB6Xt7vjo{h=E8>rLeGUfKGV^dV&X`tl$* zuLI!yV9)V@r4kEmIi~gJ{cCWrSZmqYpKI~0*=}XLz>OlC#Vx7J`R#^CI*}dzovyzC zzE;`n=FqJKOg`%tC`O0VNGkMoRnG$I$PeB zr3>z>f|wv~X7dg?U$-_{dCdLYwCc*zIL{}EelXJ(V0CPK5+~2b&WY&QjM9N4OzJuZ$wd6 zmXfKIyXrVXTv>7~NI|D@*uIK498}$n!*0*jQcM5C@}(z8Ndz5U zuJYQgUgRGtxdn}AKy)!Nh;)6X{JTYSO zO`$t_j96RXAlR84eo9o}ju3TH-7TqXr2SN4@SH%3d2tmV!8Y;$ibN!7j-eY z$+0`t6|iZ=7H-Cn?U(wx--9&7o;BwBuDq201MpQ8Htd{&Bw-4AiO}y+Aede$sS78E>gQPgYkCJJ2<0x8LkJ4aB{iJnp~wYt-rR zd#pXih4#q@TC=P1ydyAOf{0wcTlaN>rOuS~Kp1$D(#34qm~6{^yz`7~>*|gFpWkbk zB0d90)H??N52mX6yxUXLYY-FowM=KqZM&!XA|zt`GD~f7j-@BRJ*F*AxDV3;z|_-c z_0b}#+`@vRdCS<4rFf^^l^*&CAUEM13p&5)GpqUYv744H=b-zr>sT1HLkSLXcWJ5U zwJ#+-eEv57ay{f0J1*P6n4m6buTr89ccXKaT{cbB1SF*k8k=^)hPc_4q1khqG1)PV1X9fBsvMWecA*V}dC%vZ?Ss0eL1qj3v_;00c0OQ^V=p z@(f$Ki@J$G8yy|S2J^wBTX4EyKBE>FK~3q-fvfAYRonBQ45$IBnwG8X z7H1dGO5@PMD))6`Kv-~_$F!rMbDt8`p&K*1f^9U&V95p)|9y znD#Un9Y0@h6WbXcS)nI7nkABe^$EV*7e#(m7DR8Rn`P;YL zSM)JwZ5EJ)B0#IObizH<0NoA6E3Q|sHveX6j7Bo!@}QH|Yi}7+BW}q_(cx#6rObe(16G!#TTY0KMkQ5=TQ5)w*Iwt%n!U=_Eo=pn)@2Ul_3hx2^bSEv|29Y z)~zsH4rH~`D1O`fphTlO%U=TNY*iguKt%;|Yz&&s0~o=_&K<@pL6Kv}19*sVs+R&KsuU7e=dof9bEC?$z*==h z(ftk_c?RXSAYs)Iy6nO&1a{hK#sowQ(?jFdH6$*q(Xox**j(5gkv(RRiRFyFGy}ui zVZM^$&hIg)%$of)S8^!FIJn{ck2~mLRbEy*Y`*$imUOx<@vS9Kb=QkDH%2Q`7bp0T z=#WF{X2U8&;0+BcT694~c@$*P!E|%JPS{J0@1i?iH=4v0iMX3%*?TVO^(S$^rC6OE zK6m^sf(pG31uM7TAal6(k(b;mkdsK+$l!GA=m-(D%!i(OAwy_tiyC3y1K57o4HR_a z764XQAt=2lqQ<~~eV2^6l-Ah!?N^=-VCPhKj@?>Mt*rA{pw;<}u3>f@-dpk#~70$Z8X|E152i~c-JVQuMDWtvP6zf*GsvOdODexkBp_n$HWi{lc)1^CX z0wCJ(cRY4mD)Od#9Guox45j83`>BOH5B15h*GO#Datn#^B0RA`=`$<83rSS_p45@i zP_$a{;S37KDMdDK2^+Ue-3?mgJvtd$4VGj>*BiO$UO#v|c8rz^Ml89XJfvnKY`WRw zrXWlr_j1~u>9+Sj+jcr&H=6OCXFnqX$0(c z-=;wMt<3bSTU3M}=r7h{O-hmyS>m4NlzkFosnUZU5FchVPko+dvTYzzV2UJcy7xFq)NS4|tHs&IkAl}!3@pS(B@q|~ z30{sxWtQqHAO!z}1-}~4797dZ`)VG;rlNawh`PyS2*oa?d8iGK=YXb3+kV|yUA_qz zjK!tunF&fVk9zV}i3Xeo+Gsu?hH%lBoxPVe8Y{-P8g+O#KA6m!o(YG?J-PaY>d-P^l^+l2DkNv@7)xkRsvNsl=+$XMDEkcE;hS2bPn*)SkAu18)uN!QkTS{^( zXx=KpAS^~yi$$GJK9=-oBN0w)D!&b(&pluca?OSY%BjuGZRD63YV^!-w2FAICZ;SYoRM-XoemUW@|0O@jb)h9l zsQ(V_A$Q+o?fO1UqWrng@ABEt_ga%lr8tfA)r9I!hV)mOR)c)MYgfIS7IK@Jnkv)P zsbm;+kqUaJe0Lv~^iYCqtI@w$y-ZUfCLJY@gA-epM;54V&S$;p{z$X?m_~|E=Qe4}I9mPTFjq(vu{F?9LS*{zzp#7>`90*Pl!(c|YwPJQ#Tu*q3 zdr?G*&uFAGqv{J`N6XWpJKvP7HvO>0ROlCy^Dp)@;6~6r!BanFvU7e2&B^kcN{vtK zex@>ley_kGlTmO*55(Q1Qi4aRQWNEz37LL2J*hzhaT4I(G`cN@OaT9+Qb;_G|(qI)3)=IUrxyY zI*YA=^U~D8VX}N&kpG!R7v{z}MxgcT@2GhFE6#2NlZe~OQgiF7^~C1k@#qtGg~eY~ z@y$QdM20~7$=85VhWP3CXZISl#vCLc@xF`>EQLxTs7V zH~lOd-LVcGXSm*~jk3X=z=+IWKcn?jo0PUuVARB)B+-=O7lPjf;eDYs2+WlwyWSi^ z^A_&(Yn-L>p)Chn^Y71s@0pRAZ>5~Z_dT0sQNkg;Hb@!^vh4%$ZkYBT_Tb-9M!tbg zX)%A&z-OQQO2?je=ASTdeHX8*<(i!^%*exN3qr_)m}TcFHwk#Rc%!<8Q|$pZIjp9A z{YS;!yQ(bFTl=g1_Q9uH3A$_BFqMtB8lIVS2|^*T=P1bM{)v791^PuI2oQ=jRt8O6 z$=>Bt@=KhpTpa}6ON&aAO?;sKM#Aj{VL!{L(E|B$pF0$<9=&qz3if3BuDZT*{M;II zJ#V|P5p5ybUeT4=CNC#aM?HfCR%zx5OrN9= zWi7h33s+U$t*y$}|iN`pl{ zC4(mC=&F=QW-qg6ipe&g(n=T$apgbV9QDMX{1A<580RoI4cZrO2;5#H9sibY8r*;K zXL`{wh7Xc#5Z$3+(w;548i=ei=oE1NItJIWnTRSpu+_-7*0AVJ91978{lBPEM8S%qG{z=Ifga2YA0zy|kVl zPF1f=8`ZKNm%DJo3rUyg7^^b8*6#ag9uubc_B+{xh6^*ZKc+%QJr<=*ci|6F4gX8a z9p)P?dB7J$w`&s`npazQvg>zgEd(Iqr24|=X(hKjS|rdWLsmK?g~ zuD`GY6%5oTg0U8uO#jWwBnj8*qs#Tt&m(b(cNV!m( zfvF+(Van@s{RXf3`k!cF!DkQ(e$<4#Y>>R5AzbE>?yiW-=!1#9OhKoT)?Zg>wW@Wd z(a35lwBLkME8R##(v*x<-lNotjE;-rtW0D{#Pfn!R@LQ~1oQqZ+}5!zngfEUA&sDC zzXUB2MbD$sOnO;H)%)>=v%Pt{>{&l|%F&YKouvkRCl%XuM)zHbXCYkkA*bS%GG?_J zF3190uf>*FRHO-DaHTYZ`}9$wo2qWo(b=FlCkexIQ!>nk4onJn>H+?D`$Ijm0PEx zq^3`LpQ&jW>{U3}dlEG#8p)vNx`(fR7s}#se^d;zQ(%Oio`n+mv~HMcYM6S}p0l$D z)Du;-hCS4o@V$#E3Q7r%7+{j?&bQWg-h|}QBaHP=9$FJHLZJx(2fS{h)HQ}j)@n3-{%kNyjZBGUM~pf-EDM1B zTT~61T&YFeg2fY#Jeu-2TTOp`nSA?}ZCK*gOjYxK!gHE0n??8#hh?;g^A8eB4$n1a zr-Q=;Ymae-MDR7+X_j}D@il{IJdvVwn|Iv=8*5g&iXq+4z>Zn>n(9&*ri>5~Nq_J1E4Q{!Wy+D-Uzd$tYUNW{>WGV^0Km9~rb*|NcaJcd8XluFr^+ zJi^WZ3G>rj38QC(E)2{U2}c`U~ibgOFhn}zz#of3(@X%c*t z^!s&7>ncQ)bt|ng+V&j*tlwU0m$^Cl0eS4un17Q~W}XK~2^Q^TPipzJ@}T#L0C$q! z+o(+~Ya-DA)Cu37nEh-W46>E$+Z$C4c)Ii5+wAnyZeV!t6yO}olAaPgpgl>rH!Py$>kJ>s>)6=S+T@r)tx|(BNB`8Tl*uz zPE&kf+!YL>B1Fto{imKhFa^hLRyQ1%hW|nhThZKL%E8i<-DFQOqwqgn3^u3gZFN>I zpJ&iJCxLI872#oFZQk;grFqd9lMys*>(5ndgSt#+pm@%|7zfmhONo{vyM@S=)@4PnD8XUc4AZ@WmGr0-#rmG%k4s+=ajTG;;%Tk%1XRU=#(-s?%V z&S~WINq{B;=i2z8cnOwJb*37LDFk#XeD z(31R~d8Y9ld7s&R^57gJRP}5g*r~trkDI@X0Sjji-NwkOYoXQ*lj}^dF3xqtPMJEH z+{W7=wKBSZr8<--10^oCC~gqeCa%uaCge1OFl7Dwg0Btz96>akG&x1; zucZoH%4dRco$!=^L=~c^NsV>yN|YWc#|pi3 zaL5hJGLL!tQNlrp5Zf1lJG-}HZ?onQxvpss@$-t~6)NXGNU~lH;WhNIEYTXu(_iyX zLVASmyk2K#FSmx3ygkmiAlON9!aRxvCy+3Tre`4gq~6;yAD;)KXzqbjzV1j1_}<<2 z*(v`l?t9gJR-4|JRykDP|q8o&ll1yTlO zi@xd|f^3nc*`qs~s{12awXo+Wl0Jd1X(N3`qiy)M-BtdGIMDG)@RbpWG}E}R`&X2O z?C6tOGrOW+l0=9YQXVdl%)$k1eqBL;;S;ozKE5NXX*c@Gl^IvkmS&Z0^2z!_0H{fs z`@wdi#Z!8Hph$u+^tHTh?0o6F%Np zd@`)V>rM#0X=!VoHt<@G$>k6+@UIQKd=$6g5r& zZzN3nDBJC+!$vEJVu_WXKThPw-OU40!Wdg6$7d%sU-A*K`!Bq$f8WU(Xulr-ax}Py z)DFy5I5q+*N!pVI9{n*Wpxya3>jOrb<$kNr^qZb3*5E7%%36Vucg8=eS#u3awtH8q zg-@``>W}__BF@RviWbZ`L4zkvrA#rJ5oz~3t$Gphi=h`Wy9Og*yyluPfAwDplW~4y z;)p);`30o|qSvzH`R5;Cb*^Ai4(&n1?E-1PXekWSVzW9Gzggb#ym7sn0b~Y6!0oKk zmRx-EpR9@5f?Nm{J;e$+5P`XG%N#Wcx(v&O&i|!ncES6`sBC?WQ{WK?+#i!r6*~5a;zb7)dT2a*(u|b98*x~8 zeR*cI8g{-R7mA{4Wf?9=V8Y+{^(EcJc8lxh05izqd@VEwt2-Pc|9wqJd9nUwT1y{! zk@?fSUj%i%WJ}-NP8rtbCPDbJp?_+@H_7+Sl!L#~Gtxi0vc4xrk5c^dX#@s?Gn@Fn(V#0RwcyRCyT|zXR$K~qJ!=bE!-ix6Z?h!nWWol~E3gAJ^~bBi zgsbAGiYofPa-;yyhM>P%=>X^HxJi3c6af^~Y$hJ8SDpTu$GXg2k1AL--`xr!~Fq3?mqkok} zDE77X)Fa7Sm|@ikCl~YQN1<>{5w|45Gc%Vrv8W8@D$6tW+G4dDV>S}O23s{j=+L8l z%5DxwQEtRY?SC+@uh+BTyDx-GD(y=O^P&Cz#wPZNNGrL;+5jM5acQ@@V-*NWOL)Q>T*- z=fyq#`+e3MfUJrsmZN#$K1Ih3Ur_LoJh_8N;lFYat1JU{vC-9mBl^O$TY(ScsaIoc zLl)h=_++g?ypkA_w8L=EIgEyEK^PV;aDo@M#oG1B+mbWwD5?9QDC&qQ`PBVq1OL9Z zGQWP+-!?AuOU1kZH7<*{zd}S;NmXe76A!Ipt@p}ZCP0UcCPOs%wPAD1> zi1`Nf!#ruVq4tBv)3pcYt{6p%-l~uTiYJd;*0*7<2$IVTJh})?E(6BHR{j|`wBBpW zd&Ce&_79KRP9{f~T`d@tPBVdBU?Z=NzGnQzT_#rzhHC1X;a4{oqJDR}&8l!8zM3$=YyA67d zbE@~HY(M#|{E%Vxunyi)x+izcRwwi^alEqrL$f=w1f_5>@HqL$-3J-t%`Oe49)rd1 zpHc$6>nzhqC(pV1`lz?mhM;aTweM*#VJYGFj)!>|Zc3uM+d2jCem!r5?yxXtSk4+- zX^=9$x=+2D;`Up-!8Sbg2eZ!;eo@1#86#mw`}W_Vgw7O+%p5Xuw8cFe+eqgWd4hbt z@fs7Sv&s=)!_0b{QO(z&4)p0_NeY)^n#yby7qO_p!Eeta)x+mq{gn76flPw)HT9cF z&Hd}NZ@WAJgrk(K+=eq~2ba`ACk_a5>=qqDgZAq_-ZN(47@tre)E<2!Bj;^(xO)djuM>jZh7*zD*vu8PKTM@o#7NX6 z_zHtxZkBN#iXZpg(CptT1T~sDQASCreS6JFD{C^+RK^C857;!Vf0Pu-nbp@<9R65J zmHw#lX1nvUB@t;}vH)8_FW^B|P|A5`uVBQh0p=<<61vz~x}h-Ijg|7x)fGX**?Lkz zld2msf9zTyw7kKtVDEbZgh{M~EC`YZLv0w|td$^aR=|eMYLh3N^z9U{Y?T~sC7JH< zuv~2pQ)Sr|OP5$JikosOWbPN$cjZ&LUq%M}BEeU3=pHP_Csmc)0UAul7i{e~8#K4K z&^aR2jw=62V6YCnUIcwl9qeUY~*1zk+1m$>Dp1ltX2f(s0S z)`NZ=30QjndY*1(OF5(YFq`!38D!&ap@)uV_T!mp)>N$X3lXoct=kb)YPyWac_9|L z6q2qT7y_*hKX+u2(-zSWe6k58W=80Aj5&p}s^4bCwC!9}5{ob+^yA&rhR&XXr)y=O z?7uGg>~a@s1y2G&s+>_{0F*C3=b2No87< zkj6vm_u;qD`^5?yN54bUxcfLzt`rD{u(Rhqq^mKhHt+U`GpW(B$qy2rRLNn6w@bGK z$<6$dl$FYygk3Q?zVCL&p7~~ABb6DRto7YfIvzDB)m5)NDhd{9xRZ2NZr}fu z?&8~gn|Evmx76y%?xN&ls(!X6m*U{uJWx5zPztIE68F_VVvWLwqWk*RxQ`!~n^vp* zncu})W3kqThYMY6(Dz$oylPud_iY5WlHcT(l`9N-cAN1s_9Iz4i+>bB8>|=+S zE3m(<-_?DVIy9&T#Omt|uz3R+Cf;iQhqvi0@Zt;QW7D%n>)(G2XA8j{wYRRHF7|gj zv9_Nx{9m_+buYHF+9}6Lx1VO(jY=MN+JUh4Fq6LItrV#e&#k8@5rgj^`W9JJ&0}nU zgz(Z@%8GU+1pf);yxr#4AEB?;i|84lb?jDC^vJ_0K`SGcNFdy`ym;#VpSIp%*IAdt zY+=!LusL;<9OLoNKr@B|Vnu{X#Xa;KZ%NF!%M>V$a)84UzTUPSP&IHRl>r;ve$A;H z&MOUeE7hBjmCEoqnn-bM^J?1E)vv1A6D37H7%jF2SH|}HUJ2Hp6P3S3GiYHC+L=sZ zp4@1lOxk?+DH+L+4rdG!#(A~broGhGM4hJCrp0hDeZwK3&V~QlE15R=>12e!57No| zkBP_J)r^=A3T3;EMu8C(j|KFCYvv!(2!X!#9z1z%$$(O&JIF&li>WtgZQ5(`YnZ+qyJ)sH+Z=KhxQB6@R1wLaohP5 z*|H)12K3Osk&G!R*pv~9jTfm&uPWkNAdn-OO#ApJvqKp=f2TP zZ%J){@4S9OCnm->=8N}q4n0YHXFTAp%^K@bweN~@N~Fo>jEt|_-xRK!9AUA+W`kHE}L~-*Ux|<6dn$G!$$wU65{H3(jzeT+$x~P zcNNDzc5!#6He_OeM#wtjCPkX;-B2F62jAboHw9HGEe;#uwwf*bKl$sl=P#exd@zT$ zo_Y}Ks-=*ZN!MWCRQI*3tkR_Dl;AIRUXJwHavLsQ*64o$(A=*5LHkd#g$Q~z1@F{K z9&n`U1AU(d;VyRL0H$QfT3KHVueST5M4DVZ`)8lP_XY>KY~+I>bD_GupiPo;a2U4O z)JDEjn#_||BzW{H01xIu!kU zwQCJn`IgV4VS5(^0L+#CBdk=d$w4l3Qi1r>B<;vXymX#~OPln{>Ac;kAE1^w)hD|> zw~jbxt$aw*N{D^DudNozGK{CM){?B-t)ZD8XPdLutv}1E$})gtMWSn_CCao8Ef!h( z!e9@=n~zSE2jWgVgFsQr&H-v>a(}XO%o?rBrOA4FY0)|3RmsGdU){b~xpY@)E)mPF z@jQ+3^Fg3C^bn7(6yy9@Bh~*;L_FO>-7ndu-4a^<3unh5fjoXqpFP+sio=Y^I$A5p zjc=4aIV-li+dGcf!5Ja?0UR&#=|6d*92k|P42hNYd?r<-({;7Od1<`4C}cRQlZ7Jp}zx2Up|dR~TTf6p5?aX!Rcd3{9F>&-`&f{&IPh!&X78a(3wuN+zi^SieM zqEGkc1?y~(CDEz0e)KhF&>9n$)q)V9cG!cev!3`|{Pp^hgo=eZY?G7H=5#6IY?Y9p z^r}tR370+l_qW9s825)M;GiF~+uNfo>d+JRMq|d@ubkR4|!?a zZX5rE9GMYqBI#ZKn^31R?VII%{hKRLr+@gGCN&sDI@a`XC|$ouCAIIsq>cF<%h@=g z6k8Dl#!|>v^Krf6IVX$H_8kA)hZ+GI2t5{%c9Ub-$Z9{(O zGm;ym%j8|iXDb0|od|~9f)J+*$uMq&N#CTWkXK*P2SrE$J8lK#)Q)fgTw6AEer%Eq zSmxm2i;6d4YTtT0S(j&njr5XSKl-ab35m@$$H&$6VbV(-95g3@N*hVC2);|a)3q?9 zOJkg1+h&}g&Uu``fz*XKlacl;-H>Km{cp|R=71-5m2>i;V3)|ij6?in;Dp%6umsbO zBV87o=eZhsr8J3rc(`2d;k4qVyfaL`n#%}=j8(lFe!DJbD6RgvQRq0z)9m7rPk3+r zOU>d69VR;11Edpr!~ZLBe6LUK#z?xg#3)rsFv2x`TKoDEgp%GoZMc0$;H!X1z=_bI zl99!x+-0*1Vl&-@43m4dw+!!K<2)|W$g{t$hLCizG$*xt?YcTfDML}CS$>*d=2nxc zqDh8oGix+TUGiCjcqoq?H|low91LEHDygZT?yZbwnN=~``VKGO)M^eQYLX&rzW{Z- z;f6IrEk)fsGs*c`?NJYSLvoZTA$^=?g~=w;jkcJWuhY^wRIG-%np~#N51#zt&62ud zVqm3Q<5ssG%F}`tc#p5u%fye|i>0IK4yg3h#QMjr^ zo&{bvRT);h3@c+3lA(#c&C;W}(!I9n^TNk}U1kck^ng{%dMFvEH#U0B`O&`Fn56X+*MK}$qQRT!u#&*BPmDuf*kxECRwwv=k` zYRSOgq@2*PH!m_@j82sZBaH~9H2LwuGTpk%cx5mIIeAA(ROs^wbqMu-1HX+56A0Y> z%;{OXIx#=oSK)hw?G>xr+_wqrP;Z={W3u6Y>fXy(=D89(gs0umq%uoUUrLK!FxcxS zt!2Ww+{C1Qopbq*!-KL3C{^u+k9sKCcj>eAbdk5(S0A9WzkUL$D4^~=0fmSJpC{?t zIamp|CxCHgE>AL3$)Ci@>dM~s+!tN-lO3#ADf0-Fbaa2IRB1txHE9&lWC1W+0PKqy1+xUfTDE75+{{12o77_zo3(y#!BjqmtZB6+KdRf8LCz(Rd)-%jfVpgTk7dE$ z#Q%g39-B6zRjHRf&8fF=8iuc74L{vyfF)9=^Joodlf$wHi6(x*lpWgXY86(#!RbWl zI&Qi~L%Rw3h5TFcr1y_LhZD%6#M)K$8N?%-E!oU`W5$ zn!Cr7i6}iLs8f*}vTFjnj^&>i?EE|wZLJ$Gwe#_iXNN^ei=dik#mrQ&H28`%M2Iic zSmNgl+g;CismiiS`?XK)diL4&8~itY7&)da1>s{QS(7Bu5|&O|;UDyZP8%QjG0NQp z&#F>r0-TeF77Nf}LyeC^C|cI7IqjF3B2IpBSh#o+mCX?|zi{ z0H@3?@WpNY(&|s*k2R;FOyx2o_*974kF@m6Xek33wzHX~EQ0;X@}kzWy!z8T%Zu1A z{~U>+3em41ezfw|s?QoV zQ%)j5k)eGQZ1TGA==7HTwy78l9rLQ*D-Cw_o`31z5;)oLUZC=xWnGEBGWNeR$9^`U z&O3SPqy+CDO%50Fp&|a%A9&lct+No>@>3l{5|!~!`|SXS9J3hjOVvhYWBHsZPc`1F z;xL9QX+HIAvDCNK4&B%WRzHS_1U9TB!#yh6ufJ#J_%AD}-aFp?`7+&WCLvF~HI5pv zD7A41*C@>YBagSC8b?QB=jR)T+UH+^yOb@;M!4nB$S%B*l^M2nk4Ip{1-_aXRV#&V zbjgul!X?Uwvu2)E$1W0}Yf;m2rwa4~bV-J+syoeMyeU5RVS9gTp&}yjsZ-bbS5fso z?F!a+HLW_D>H|_~4a+Sl%BtLnO3fQ{qqo;UXPNVEyz&?ap|Ata`_d9RMtx7Q= z8xSLWtG%LP^Oy$q!yEV{k# zF%)B;d=Si4+q|LB?tLUIG%#nIYh8btQOus@Tr?QfB+6eH{?sLZ!g*H`^6_qx)3m9t z4$Qk`oi-R;>$t74PK-R%dWBdhwRS|oaZ?zs%3#}%=QR7M^%`E8U9dy9KLVa&{eY)oofRF!YhUImN z!{st(+-M{|IS6~^lMi$42be?$`|IxxBYiO&XA4sM!Cf%W=~C}j$~4txK$`fUFioN=CAX;QGMljH5UDE3MOjcBl8paUz@O(- z-H6TQQpZa08UCkeiit}&nx=6MNy)}YEco{<9D}*6R3NHVmUR$JwiXumPOS zlNr=@j|4RjnwKS1C}~k@YmI6_i-hVqm+>qlhlk7Q!{c0IfSUPe%=e-}n}ktjv1fs0 z;|0p9NK$3V>NF!#g0RhnAWApKbh1VF6Z%Q>d1@x(nMsW@Aw_!Ia;2IuYTC#JYn(@D zj88=(fDhIO^!ae0QIeU@vXHskb^s4oW{tfi^15rWZH`G7wuLhn7?;4q-V(&|y7=AA zDZu&bKn!X6c?rm&au@_PGaXhE8;s0@nbMmXG|uthjxY{%Y@>A|7bK#8%WvE)T8V*x z9#S@H$9z+v+aZ1k^oWUVpt+=pv&pZf<5R%;*9c}a__Oz=Ld>mwW zi5LPIiF<%7PNCT6WPPhjE!q&3`LWyW1PD&I(X*3d_J7*ee%|g%A`SoN+ zF4tcw+3;iCfdcWR<(^fWJX{>XfO6#bn5Go>$>FpAz(z~M6bQG~j*g#8vK}DUT^k`v zQc@7)5IHYz8Xf{fb+{KPem~1&*qQIxk_4QzOm$gu9W{mRmk0c>cUbb@iW%3aB@cD^ zOny~F@P2lpCwI#|T72=nH`}7=4|rT&zs0J!6@O^%X=X1hQUEh+YQFx+h-<6wQmK9@SDC7Z(03~PL<+u;~$!HH+hVw)(szbk-TD;@G7 z2w?8LtXXkPGFWB!fYaw+G9pY`jEkSa)6=Il?g z5cFpB>So+3&?5nzsOMy>Ew_hF2wIw}lo9jEOdsm}_lyaDEwyi-fc>IYHRb|vN-kxv z;L7suTpnexeAA5R)er+TTCZ#$Z@z`=8oZ>l2XQ$!ZZr(VxpYPIw68m0nCRDt8-8BZCTLT6J7?1nI7fTKH@U5 zn!N;3e%q|{?%D`dMtTBu3H9ppouc81?r1?3JEKO2Hy)z_7ju@tX*47^w%@_Y8ZF1G z)Sn%OE#FDYyA4;QTVZ8TU?6}3-gdlC+LXU2ERsAgVL9lsD)J`skGBiY=k2VP5ww_4 zN5C^>e)FB0#N_r_4)J((p!>yFwB1%^iUu?ZH!Zm^6~5xdP;G`~_7P8JHCj(RnA3R~ zoc0Gy;bY)T99|FJ(*|Dp2Rce98%mv;JG86Q~ zbVTOctM>2Ry{L$Am8pKXg*&;r=Xzz|Fh%@nWfxOhRN6gr~4A$7T!l)v^xGI4 z5I4FEqd_OtVJRdtnw?f+P27773#gyP92)S+Za8jjB5Q>p3Whc2QTjRaKm;fJY?QO~Q1GZQdnhzRUK0>ui; zkO5h2OKo6zumgYUc-Hr2dcR~@ZESjvcCx!{%J75B94*d^NbSUbmhre3Gw1G})S?e>kd5@?JIvN7zZ#vq%I1Trtpd|A)1CnO9R_g{~k@9^v+G zE-JO^r*$C{frw&8snc9b@mu|@XGHfwjJ*=y6EVy?y1*UcApv#%{ClD~yyPR7f0+@G z`*ts2Rbs$!gv-p@I^b6A@WY#X8aTsm7t{}uahs+TxPX1aV~&ZAN;Gg?!{^_At4b?^;3#_mZ)TK zi7nBtMOOk3i1re;jEs!4LcO>69r*?PBO*(OlO*GR zWbC9U84@fpf;zYDlI+zfxHjlY<~Bdp=TUv>#&BDGE>*nsvNuqoAcTn0Bq=g5JU{$B zPl>{}3zZkZdf2#57TET&&CC1h9#aE1PSg4gs;eQCN>mcL%XAH(CiIf9$~`4`Xy#69 zK!n;gzK}uOF{3jC3f?hV+OQyhrk=0_baT|axlXAcb~jmQOz&fmP;o?h^O^iq11 zGL7AB2^Bl8^Ld?PH&kKD-ZWlN-T<>+Knt*?YF&chVv{itJr0{*lRI~$;h_3p%ZN8f z3mC}tVv5cg`~w~mlJ8roLfcr!;l}>!hcrQ7#dihpwIBfEqDt95i0%t_Zv?uimeVg8${nE*rLNI~F?TbLBdwb2qaJB7R zy)gtS@kS!!{K>0=Q?A=%x}V+@p8Z|efnj9HPcw_Ru#Iyi{LfqqJZ$*z_&XqCmXGKr zWIGrF-j=t_f`T(FL--4~v2*<5&c1HvV#b1dW8msHve9rb@mHrz)@_Oma4(>08P*LN zVyPTWyBKqXNy5gBV?_f>PVLjD8x_KaZ>8T1q!PC`r#c_|jN9EvLtJ!8 z%cds0SDZVa5|Eim5`*8Tvj6^WKp(pS3}@BfoWqOnHL|+i0RzDH<nVzWlAcmR7kE)bYvE+eE42mPJeXCDo$pe(^2oTy~Yz>w!_MP zWnnY<@##VHd42%P&yNZ?q*>ct+ZN6ER6}1a42kXMYKni8Tjo}MjAk2hv?3&SXM{@# zeo|>%ZIiBHY)|2RIuFA-+cT01%#M!kBaUtH=To$2zZgK5be?cjOb!TyTn4d(vEL*> zSIZ6r;kUoMO|%WmpX8MVGDORk34H;@**zn?IBFl4R7ch-QJ-K#PjVux!fGrJS&6O^ z3}4I_X-7ZCB4Tr?dESWM(NnsnPZ!n7UdaWp*PY0OEKbtY{J=zx)r~(AfHi4Eu{d zh7d%Eo+H5nbR%qtBXAI082Cl;+))2pm4-l?_rUtXUKp(T;{HaCF*3JrZ|ZAwBjemQ zW8ksmV#D~pYR*`z!H}hp(?r)+zXhhi4i6lfb}7^)OnpBCQWotKtvQN_BB3dy9N*94 zm=2XB;fsgwVsO0Xc}RAYw;9baHzDLtf$2r#<8{8#?jf}}zh3{KUkpY4Z>w+P>MUnh~A6U z9RBv|i*^|CXR8BX9Kdz}tGL3mG!gn#5RiXUPcUCcvl%H*)u1CAT~^wJ$fNUn7(ceE zb9@rd6$H7;n@oS}?F=A|>l0tZY`&%Xe|`4Qw|W6h$N=Yhk9C7=69sR7}nET#E(qK?vlg}Q0(cK}5`Y;O;l zy@5+qGO!C)L;DS31 zP#4$(3O^sN|NS-FLLIF;EowRK7|7RJ_!0dTtR=)a&&;b6MM8;gscBD~AX1|;&(|LE z7{f4DM!C#UA(YVlsFOgdCh)xBSZuu+2{?V=EXi?-pFV3Y2GZ3 zKOC~A>fU}eo$&hgIv3;qvNODE>zYQVF=VXzETXuM8bP&I@CF6F>=UO=KBxpit;6)c zF*EZen&&^Nu|{pdW=yn|KSess)NQBM$m*1b*lm8|W^T<~CGGij?QC?9NG|kRYD&A+ zJ`f!yv&h#@W#8Ib?*@x@q2Gh(eCyy>hN*sGm_MyH!%0o-P@5)Fq|WVlH2!9Qh`&Y= z$JwRvOjm7d?M6eoawa5Vtyh+0n6x@M;|`?;2hS22yeAxKZ^WvAj}uNTnBs^ zT4cXQ=`OgoPrbFJ8@=W8-PbR1YXExS&t?>i`*`~lVL}iw_qQ1@H}CJCB}We@RgC7k z6j%-^wllYljVJAKA;4z}R5DX}Oc5`7zrQKTB4SdV6~aDwoGC0jlp;%O0z{)va~5q8 zG=_Uo7#dS`I~8Yz797Y1mg!e^Wch~eQv#zPH?8 z@7d_`i7u5H2`^}z89NvWOw!iT76D{Z)>Qty2k32r=XCBlN{j~q+rwfFJILIk^L-%l{Jxj3XapgOmaJD(T#gv`z7ci08jd;9iHdSeaFg_YgwozEw@Sl>loC!-uQ*-&_-q zBBgt1{~;_gl!C0H+i}~AGOXrBXL&MULPa~3S~E&F43e>6U+Z3Ogr~JFGthS zFAfCFb(*t)SF+1CEVDDRd7{(R`kYwkeAA*+TbHJkUHuV0B-a@yq%1D26XQ{^$AZo> z?}kSqBp|&0ag0Ig?lhUn!;ahTA9h{@JZX!|3S^^i1aI%)sA%C+@E?dY^H=U07FST? zCEo#)XPZW=%-Hoy39OT-zLdtsuy4-eZE&37(kamed-PIy7eKM>;$>6DbLdtlI2NTu zKC|@;!OYa6`z9U9IJ)@egFxA;%TTz)(AV{#mtdk(!ExVD;4qSu z=f!)ci^UAciO?Liz5mO_{U2LAfeur;gSWCAkxu02M5m0YH!K0x8ny?^ z(lB1kruBS5xQgohs!qPa-r9$HR z&llZjkJ%sQAv622gOcRlpiisrqc1NmH9x*7(KVn61jNP7#h*%lN2ReMXu9~xCTHGE zc~fqQp1jF08Q+sjuBy58!cDS+OF-f*e4nAc`x*R5h=V)PyCPx8&QSKdY`G#)4e@wY zcO;$jARwns7-(9H>l9+{T zH0%A2KNCGa-eXX3cmuqT#O(DF7a-)%UFGOp`1N1_}&(l}|)O zJS;aK&49dRwZ5}dLRjC4jljg;%bO^E{)7BK1I_=sl9Wl&AU=Bti7t#A5G9BGf^sCsIm&A!FS!>ozAxb-tUb3tkoq zXp>oyj5L^FWVqcIP9{`J-=t7UFQ}hF_?^J!X4Y6B@axI#Oo)_J&HC7J)9&{Y$Y2`y z?$t!VK2yxtZ0L}-0o�M7?6|uRO6WE?)^`{AFIt49JX`+YsX()4-mtd|l@s=Sa}A zW2}3(G;i$I{d`%+>*4Xqf${@m0ARQ0Ohyik?mD8g(yev)Ej#*;Mt--5?Fi-GWW3lc zeF3K=Hr*PJR8-~||1rjkaEX&HVweeF(_A*bL&Qk<#Y z*TJ+-fn4%lQO|o)Yz@TdthAg?sBLNRdPWt$s^OzbEc#=WTgHH_(tKOAtwEKTPJL=@ z_`hQL|9Zv$`2j+3<^GYymkq;m!vrzOh-_iJ$hQ%oI~UP)mS94SQH#+shE4Vj{&46Y zi{Z=aLzz$>JW9fr`cU)f6q6hpq)LrJKwUHcI>FF^nYb#U%C5vov`|8zZ){@Vqi{vY zl&lA&BGiYyq6HbnL{3M+TcS)k|Q_8{LxcHi*^-x zHHyR4ApHgX-8kd^#MkHmFRk*Guy4Yz1mu@(a|3Jw&Sj9k!!TdCM9toDqOe65rpIn^ zks(;R{B5{U6phS0X#w*nH{`t-y(Zy+q!F-GbDK0qKZ((Ezx>l!^S@U^|LYrk@!_B3 zRag=|W%Vd}{RO2wH6xnUha}&9Iepg{#i{!I=LlE}ZgCp!sQh>K5AqU(boXm`JxV6| zHBt~!3lhTWc)-b|E-zED*-A}Z!N~c~SY92Tko`p0GY;J$lh1Xw+=-F8s`H3!F+q8`29z&%?cM&s7Z^p~M{$?LwX9rQdT@q`!6$ORn`QdC|*2l?vi?l$BJd8FM_dSE0w@pq37#GU{ku|$iqBF*M% z>~%a2YG)z_w4FPnx!Y!g7tRkBz-zm(7Lfd$O)r2W3cpW!egSXED{fsGpz$Dm-B4RbqnSK0=S^n<`W)kSpeC?pb&3?5Pjk74k z*e1p7nvgib4l_ToM~DI4nNI+p7iZus4Qw=RLjg|2$gwN^s`L{NPiXhE$M#6t7|sAe zndl)^=^P1>EWh9@dhS{gP{q_6;GF0fO3|KCMR*>}|8~FzM)*)RC+pqQQ>@HNZ=T{) z+SM}t&ZR9>JE`4WWRl#zbQ?-j@l!*I)*3i z*h#x-M(=pfc7InJd#B2=)0xuuLML@gH@|WN$Z+8DW zqSYDtBy?n0Q}NrxFd#cXqtqC{MNEB`z1$4k8NP(?cm92a%7^}lxg+%$;IB*t4`#O` zuYhi=L=Cd#W}_#TALIOh-&8af*onz8BaHkY=%qlYl_;xo>^U$fuBWf9RC_#?&FoB^ z8*>S12JV5@ce1?%IceNmK{v;-Jg7`b6!AC3*I-H?9BT4!A}G zNxNuvK<5X#_f8oX*)J2H=}pdFJ}ONfPM1K~dO${4QsP?Vjjw)PK89f5K2iR08WFjs zxx_RRzW%*eyHe;AO^^{*|BkFZPOY75jI7G!(LAR<8`x6AvRqmO@wF7T8Ds6&y**7k5^ z%zgkzd`tcnia~+7zTE0ZG4yVNnc8okSLd_zBR{OTKGcl-xv1CwFT`z*$(pDX0rG^j z$Q??L&321mxg|*VPW++O3~64-jPI;jVDV%pnE$VbBDye&LO6q zOM7D^ljdCro%53eaZG{OQE1I^Lm@EL`w^XY$>;A4u#3S-%BgS0V`>0skR6+nBpOJ; zatjRR`*Np1q_7Pke|#YH_N3SOQ-Kmg{PhbyD;N_kuQwJJX@xY*1>gb~J!jq)D#ntt zRpn9cKF8x17zPP$L#GXF4Uz_OmDODU)HyaC|Il(lHpGL4okv=r9@($T)J1RpJ@8LM z;2{IKC?z>GWuZuuW?^M)v4G&OH?X4M3(+H(gjQb;P}uGp991sY0y|1!sN4JBk|QFV zm_Gd!eb1OeTf*7$dgP_H&x(3g`(&0ug(re89fEjQ4kDX`>0q=+a>ma)uGrYRB(E5p zM0+&JuVmuv$$C|{{-k;Mn$#OoU(r$~>xR1q8s2A%dT@$H)Me;d!UaIuS==W|@zl=! zRe(33foznA-D@D*$FGhO7B|SZOz%0%|M!yYFtXm=_P%dld_NJG36(oRr#&i?vG7wIe^4S%|s>i}DkKG9RK zSO*&l_+$POy^6XllY@o&`mVgRUr?kpYc8LNFcfJOEv#>Gn6#%8BJhL;Qx+m48%DSK zly_K=)u7P+b^&nN`n5a=&}+D4lg#;qFwF@q7GHt_zrT71z|$MxbncgFIp0jlGiQ8$ z^He5!EC_Z3{P!6SzrTko8;rDGu1BZcB20c7^P+Jx>71}_rvkQ8b3(yEQalV`*^gWE z%Yiq}es2NrjMMD3?lf1`9Ye<3aa~IxV3)wuf~Bd3PnuYiVSjbG5=yJ*CZ9u}cvlBp zE`grZqg3Mb+|*OXj%+soI8A?>;21jC z@!2<+#HYpYbl6W8&b82`rgSj`H{3QT)qw&MAdAAD!P8U)%oke9%0jL`i^!qwISt*> zr1kcqia2-&qUg>FOzBR+w>!0!FYit#!OQy|@fqIoHwk;8BGxy+(PA_KKMh1W0?=R@ zVsn3crQRNRR|ESAE_jZ3KBN4yh&iKG9)}o>m_Dg4oz^gNu}F_c4GvWeyOvEp2EKF6 zEGQyK__hW5q^zaUW7|*yt|@j%7z^3Osm?P|m+3RV#if7z6!%AlePG5pqRl?J0Bqdw zgGsag-I9Zf#_o@|&0f$>Y;@xiD&NH6)4@uB+nnlb{ptMz5bNX@d~-A`vogGi87g{kcP0Y=Ht7je(4Xc zpD?e0ILAo3*cRO~Lj%gQz*$B1Tt{hIk1?CjSsBct(!KKLk-ybG0!R)BS32OgtF_pet#<>f@QE|Zft7Xep~ zBOEB;#8m8HjpnJNbR=_|>wPeAIe{xuqVvVOj|mhUd>=ER?WVnG$!8=DBimxoAr{z3 zQPu!cZ%@@Q9Te(Sm^?+5OxY&VD|95eQ=38^r%NkmMpO=QaOtS%d_&P=M_6)vI+C2D z@a&KIF>_2p$S8%JGMq8YJlEkYzM4h{k}zGKFKvX!VN;!>IoBB|=KNu^fh4iRmG+>| z@lr1SNB3*=NY=+-L)eJlVPo%WKsRSp3V_A2K9j#-(d3NkXbGs=Oc2Im|9(Greh5y| zO?cWwQ2Y~JGWiPV-mp-H@R;R-Whu`ETa*4-zU-R3bn1`i8=II6m>^aZV=8NL6AW5E zT5kQw*$HtZN0N~gxAYEtXI->dE{^r1XbtB#{ za&{x@tm#YZgWX_c*9QUdor2_2DVc7Wqs*J_Cc2`ZR8=a1e`?|c%l}@<%k|!S-QfYl zaukk|RY(>e6GjF@+dz3Yr0Thp55ukHJxwlXf4c|NjU#e3LohZNqncQv<#7#NsI;P)pQ2$X_G||3FxTO9aF%qKd47^5ldk~S{I(?w zG$ThIsJaTAfty-|%N`D!+$Jp|1&MlX{#Ohpk~1**rl6YfF*9}`zSMr~b4cubq7rrGJ?dG|!=0m~Q|O`TQK&lz9DljbF=wO6oXe7p6e&6p(KHctNBnptswn}^ z4FDd$EBcYPM~e9Z=p|SkwqBToYKpS}+=ukE{7;~tRBgh4T~3)u%8+lj8f$c#I@kOV z&x0KK7)#_AH-^ezKmX4{gw_O|7OC4>FIJA&tl+i<{(|&9SK|NretrhQf{Ex8m|*W( zZ@NPCUxf&)WwAyqU=hEoreA>R(P2Q1O+K5HTk!;DPL$_ z-rwDnu_=$`GWM8aE^+riCGtOZ+meGtbOf~j=%@N=pdBwaf5L1S|mKC{PG4pMW zTsRFp?_>MPGYu@B_MWR8_$LyefBU9e#U&3&RjNeN$JZv{7Q5tFCvW%_H{_i~J$duK z$T;}*H?7YbGj>P6y2Y^C|Tdffu#P!{Ay&1U;o=7O^1oXOrL4q)YREyLd{Mu(ZMviBDBk(h6-iW@U~( zz9EcX-9VS(*7zUmauw7^hEA(`T7-_xh_&(o_Kwb4fZBYYSQG1c9qe6d3j+kj6q_WL zrGc`-+4<6&vUZmk=Yd%$nE2rd&ODF05}BK_lD~SzrpK~TTj3;MYj2ir{W3CD09k~? z{!m<$!}pgs6<&J(S1sNg?{j7Ru+|pzdwKGvNTNFv`CRY3*gI}6X@tWUw;b_Hvm*C0 zjG;j3Sw9pRl{* zUYelAQqy)x0qL&|K-W8q1Zl2H@n7#ss<-U`{b`Pw&i-cL z^v`%_Hh`+k*X~1sDku}E)nC-*`Xk|Y`XRWU>mZSNLJ>oF3lMQlkzLVC-x57v>rp6a z`DAPO|q!kuZg7$4GPnZj?!lH0R4CM>N)VAby+CBdpbHLMrlPPKuA^d#{V zIch`6offHeS;VWHz+8u%EZq&qHsi=NO-dL4f!{ueVratC@s{^$#^v!1!1;D7vK^~- zZ;WlSpE(^*hU(JbX)&dpF&#>*saBR8zXFJ zaeX?B6SMiT^{3%33|W&gDU|#lQ`I)5=p|9KtYYu~nB4=O-pSmik~*dOyd{C7ovQ%J zQS})97f)~+H}G^1^L!2O-l{?))wERi8Kq6V7zaN86 zE<#^N!laazug-bFCbYb1m;Hur@1-fgcl_;?xo8zGPC3R#IFnrl&!J-RY&@8C z9LW|vpS=_PpA=g{AbN_H4j;C5SU?|9*oCN67fqyP?5JeEn`cP@Ztt>$ps@P^ZW0T-lg?iI#JUSbiHsC*{>WL6s=>NCjl&UfGLfY^;`0e}^LzdOa&S+Qm}`~xU*ytJW!-`BLvG0+{X zG}iOVb`6KU1wftP*G8bRsiqq`$K8Hw0ko!V01IR%T1;A!+lfXG+iJUU`UMulS_f-t z-#>X;zCp&``ydhWe{Bz1ARKFo@TMp?ko=qrJ1X}@r|sai)par@4j&8k7J>K}l2%2zeg&gQKa)Chkd*p(h^P}S9A{@c4d>Nz;CsO0c& zw#o9pA+mKPX*emKosn-)qDO=-eQ%9{xjma6_!;h5#O`^HX_5?rOcvOB7UU}P z^GVEIZ&drb66Hh{Q|zJ6L&m~sgndp_JGk#Zc2WrI-wW{>Z6hln6tEXSgSFrp1@Xq? zKXAroBx+ikRJ)*Xh5;z46qgdmpYgoqY})967O(6J#k&SN9^t3tKo(f+*$SWYynx5d zTNUDQyKSJr)SNyAbQo**x5reE-9GRbcmgtpg#ZCU6TbsNkClK617z5( zN4^6fMnQ<*X6Q$O3ebQY0t8XG=73(m_J!E{Z^NNd%mX&VS=M^ZG$O7)jotg1fQ%6r z0X%YnA6vw7gw~O89RX|urudh>5Tz>IsHy^QWY{LQf1GG;>_Y-0PbP$q4DzF85wVCcrCDX(AIi^7L5mySHMoI|DN4=jP znihZ%6Vc5Sozim|;DKPFWuNis6#%~s7pNq2^4<;$ugtybp>7 z5HS4)O|&AlWcOHD{))v1Oq5)P^w!(_BMLCv{5H|i6aTc6mX|r@z;T?(00FzcY7D;V6L2vmTh{hLXj2dHdTozfIlxH6*_TuV1x`q zsd^^%E0)X25|g}3ef*%wpGBX%4F3omK_DV+$N`RsZElRSMYv>9)InCaV`g_AbDI+vAq07LAs(i_9i)q1M!=w9Q0g*2>LQbfIDu;4s;K0o?r z{EPFa9+bbPvht53J&5Z~?-R(+F)w^7F{=gEvo9SPk=LKOpqk?U0ZIPV+gm+m_}OgG z4^~AHae!E3(gm^`KwPj0j>Jyh8`Un59v0b0TG7}oP5-5fv{CQIBoU`KuuvE8fnx`R zz2WevQCoyPwQuRgbH^*+w^=ksdiU*F$!0RGa$S{AT$oOu@pl_nKGtEn_-5pCmu#Xi zSqY$#zunh-1>mM0r%!F&5T)l%y^i#eOU9?K=)D}ZJdfWVzIJs$m|tV@;jfrMo)t2K zWD+DmbYg7X1#IR3&7!u35%GHq*%Z12`@gy#4&7272#Lx2 zaRsm7FZW6MOCx9W3D%damh_1-8bG%nj~H+T69PtldH68b6EP^}^^G|g8A4f}nqY?V z3|YCkba5C|72sXqea^wI;hYt6_ zDTC+msd9Dms%;{JpZAR7huQ@(R500@)P?D9TH6$m;5@g}2q z-5%ojA^L>CS!LA)=oWn9E_Nn+xrSCn(cLb9)ROraAkfF@hVq<9Z@fiMpEH>4ZH^1( z6`zVb&?ue{=ai76^}me<=Lbk4ogxHK?RpJfB-QdnFom3I8ABAH#;iGdgfx343y|ir zbJ8a3ow3``!#-gj*(?h-dmU8CTy}@E%K4;QSY%v;>77#T231HuMRLV&x1t#g`2AD0 z7e=hX15|QVqxu8gL&8V0%Ld86A1||YvIdGCf7{@54@sN}w|^j;jkV zS+OkWmyI%VAf}svaa-EnCeJ+^^#BVoLNH&)i73DSrEArXe88N6Cj?$-3NTLl2chZlTUn?qF(~!xaX1ssT{WbMl~`SeT5?# zu7#X6LY~^>&Wg-0_hu?+suhh@yGE7 zqO`C+Y*+SD?^)71f3tEb@iNYH85P;UB~>PTRwCFUur}&K)Nqo3-}9K=$4wGwC42n6jd4n!^t(Kk;D1rkSa|i4k@yiI=wCm7e9&%>2BTLJ$&N7DUTML1{B1t+ zlCLox#(z+Mwr+u=(mUaM>D(C@Xo|dH#}?a5SLRye}CX>XE-s>wt zHd?1n>>UMr<9pXyGvf(HRY!AM^bw<8`$3neF~j)>MCZgF!Log>@6$U%;K?D_n96-6 z|0GqtUm45#wB&v8mZ2g?b{R<5-CuvROi`?@ctD5*8qOQ*kX^xFZX017QvS#CeUUd^ z4ACrSnf~&pa48$MXtJ*4+gqL{pi>l(lz8Vxl!T=`&`YCHum+E$?Pg>n_A7qFfmnME zGwUU=Zo#|!6`?=}_fT}T>JW&)w+f* z<>Nr>YQAH|LQjllyo@9+hvf6+!9>D~7}0D6;yMc$lH0@31~t9h7cpr3e<+JMmpn)U zGCik`WY_mR@SZE4#vuY5`B$=ao+)l|KF4D$Q-cK7<)nf79`9g<#e^e=eGKvM2~$+w zl`F|Zu5su>+>IY#?K@{F(|OK>WE7YJ^W;z;*%R7F1g4Y#@vQ(Od_5ZpLQd&uTH8A> zrRO*F$T2PYO$6l|OvYenkkM$rDowj#z>Dggcd8yXtdVkbefQWc5kuqes@gYy+Qk#r zEK&~I&G6H$xeQh&$EO%|L6@Y?Ut5FSwnk&T5((-dm8DUNwq!l>`MN#$hQC~kg&>nB zkRP9qy3lJGx*fjO8QIK2hsGW-K0Q1AW&_(&XzlY0hU4wcED_xt^nD6JWtRcaQb1VIKYI8X>E4@tseY)}!&lM0PqCtMSTx=XnXW@0@5 z09q#Vu{Qg=#AKN%b<|ievACm1&1xLNfq_8l;2Cr@0^(t^59A&6z1u2xDZ-|U`kzu4ClTO)r-l<`{rk8F(^82igq zsQ05Q?PzM4&pV0ZpO15dt=xH2T8byFDv+mW2hShtg7uX+lEi5JF8xTXim z;n{xVswo1@U4$r}>u2K>mBbrjYhqm}w0MmPYkHnKB0B-P3c-=Zn-`CH4qYTZhtztS zMr|k-`w?w!$Z`9GqJZ)4!MjBFT1;q|tNc%^ArQSoF|555^;W3Q~j4Q*TfbieRy{9X9Rx z&W_Y*I7bKn{g%Mbdx=N)M_rc(#Niy0)l%3zCse=iPGU#DavsK`sj~-v^-PL0i|n@4 z7J7&BOWPQ=NFa`i`cZTLLt&uz$+NQv%^H?VkHQ19ix3x`f#*oa9EAt~8Kg9cL8 zbEg%2v$um?B+^{|#Q%NRlr*q*@W!%!z`|PC#R8fcR&W`lE{mgx9)=AyT_oauB33Ys z;&Fd{Ppj>1lI^oBlgRN>N})ryTm}tg{Oqna1M(=&5!1|cnGkt_&h>)4pdV7ed*#iZ zixWR21Jr62KjmqhZ6@%!iI`cTyE#QiCySjFA$q$qz3A-pjCr(Jq3{Xi5<}8+yBcc6PBcaD;w++sY9obc{7#XCfnK)+W zLw(j$i8wbZ=TZ1_$5aHAUDoKtnBv$PoN zk!$bjcz@^BomyQE6ow9ezlHD{JCC^@Hje zi)4w0D&J?Ui9&{H?Uk6>De^@0Iq>DW{WGOe`$cemcHktA;i(Zi;K*X{RZ0mRL%(~~ zD3yGdhO75l=8X)#L#96uMB>GjPj*@TfQtzf2 zhE`cy11qmi`O8Sh0m)c*|C~81t(&`!mTe;cl(16}wf1x2bB#N)y0)((S@@@v{KCK9 zWzwT6OXQ*_WB^D5j)jRos~n-8)+Q>QT=5#?3Ri%PlxR*aH=6&P!XMI{Z!1)b2uOuskaf5}&aA zmUxq2{>tRlR@nwpPu+BRV-GQ?;H!dP_A>+({Qfi*8a%T@%4j~a=Dys`kK%YL_ zl$ODO_W=21bl?~V>CG&W(B;Xbd}_{mPI>){oLOA9MNon2<@Gk+>CQZ-bC#hzSKHia zr2kf<*c`sqDG=@8-h50Lvn(l^a}AravP9Z`UR;nIRUBQNkhU- z&=3%Ia?q}S$>Gn_;H|w|AKUeqpDi?2j2B+J^i=sBMA$N(J0dc() zDbLB&i7#=z&TFS%amPyM#JfQW7{n^=5oNzZA<{}snh;zpMmEqT&nnwJ?%&5w#PrHF z*i5x(V&KFKrC9&xY(i8O^qKK}8l#AD}~$6V;;IbS!E?7Lt)ou3%AG+C%h+{i@~&NbB_s;q|z zTfDegRK{wvlc$_6&#K4Z6D-_k8 z>~BYXNr}QGwiB~bRq&M#KTa>OPX1>C36^6a6{_JZTJpU-6l_Z>V#k@MMmhtU+wh+Nb$2!mbCOGd?m?nUuRkg< zTDnMj-y}~Ij~r8=l8*XWC_3(y9Qf7b;qh^e2slfYJkQ%GU z4)iy4j|`NcFsO>>G6OXwVs@8XTjJt+IWhyOSFa4(*sh*QKl~}^yL=_Vk#}&cmn6Y- z{V|shy>+fytEJL!Hh=$YvU^Vx16S6(xMnSX*~M*5?b<&sH+|KrWV2jn|E&sbA50#pPngsijfjvv?qu0fz#?O-adgm_=jt!zqc5;gg#BRdFhy5KdYZD1cguOhB`e#tpu?Y5h(0db=w zQPZfie2R2#I})+#yuL`cd4x9O7OO|~dhaT-&&1a|)VcA>x0M3)m4daB1r-&6@^`VQ z-}y}imw%nuw4S?iO>oEvKnk|!ni0Gg7p(#_qW7^Dzl%r&_mb?1bhLYrXz3AoY5LZ> zGSdnD7XRa!K(|GJW3s7l5V3K_pP(TQ>H#XFeFGM`)9)(!qCxYEuQ@Ddu(A) z?lC8v6W3(a%eq>@i<(V~ko>mVe@sMAa*}bzj7lMzWr)A0bmrFwVIiq9F+yDU>J8Whl@znsS)FY7n!39s}3&4r_-TCLMb|%AlqTi9>H!~y7tWZaPpv}*UT7LE& zNfq)^KOrDXvDtznk1>=Mota&te^Z8p+41S3^7#ovko0O&=lD0gzhK{^UT4$Xi=vn9 zG>K9h(78Ntj@yvtTw45QLSNu)Tu6LyI&+=84?Xx1=?WRD6U0rrW=gnP+Y;(K!k^D- zKCqqM7N!}O#Kz@KGxOP5b=5m(G37tKskIVkWidQ2&%sfOQlgJ5YD)-AT)jMP6tCWG z1<8|<>>kHd1N!PCxG9}pS-9iDq4A1$&h_TP$SC7S(BEXI!7UX;$u#}zW26o|`-;@a z*xrVN5<8GN1x5#6rv{?nGFMaE0y?_qlg`XK)Gt@)JHGcuwsCLIxSO=}jX*&8^01LE zAeTow`n1JIj4<*x>vDN^es*CK-C*M)B1a@=K=T1>Y?2RDIF?_-<~6RAP^S$Xl+ol~ z4CT7R!%f{-<-zc0vCK?FDiOA?(fr(;SsBBjJ{)`CbJIXs5{1GHZSPD7Z`CH;)Z?2D zw{c459!&ONmM!6B4egNh*@e&%`;Vv0N_>9m$Fql%KOlKt+c`sZBB{no{vk$sgGZ<< zYOt_F25)Q$2T;^T^NYN&$5mus4fG}Pg_AW&EGw78qg@J+H~bR%8|0>zh_5F&IB9cQ z_zG1wCS&*W`G0s!=~?s1*1oze`TXG}F)oA+=S(~AqI9s1g{sBn&fYp34Y#%Pb)JH< z8tz?63@8*=(Im#}lC^EYD`YyC;kMZy&^wX7x4@d{NALRTzHL_i^%u>`MsG+s`k!+r zH7l2v1XRs_ZQM5KFV$@w1F;yjsMFma(TKwFqf2iTpD-X#{5A8Gn|y{be16uZZt9`W zMAX+ugZ{d)A$4z#VcBimuYZGsBN-%jLEgAZ2WQ#x>};1Ee}R9X|Jve(F*S0PGp|lU zach^TB8Yzal7W=Q`~wDt5mpRBWm^ICc(uSn@ejsLTtB^HY0uKV$&}Ezh$gquQSZ$W zqM7Ec0JO>ZS;n|tK?tV`&2BKs zwtH`pQ2cgA!ya(Cu)pxcF8f8FoY`GHeXv9@br5Ovoy_+`n z;mCwe%B&{}c01`Y`j_%@)O=XVi{^j%@0z9lYZid3VcG|)j-)Tx7!Sj}xJR;i-qM^2 zd*su8)3*5yMO*FV^Pqd;pvrNK$-1We#;dok@Men;2qkC`&XI9%&xGD9inwFg(#+*o zy=ZRM%YK#BG|H@=LW-ie>G)fh*dw)$XxKKe2Bf*`DLDQ&R(%&fX*1 zK{E*To9-K8`E&MxxnKm-SFEq7oe1IefBs&{1smX!1qYwSB;)y$jE*Gt2{MycA+A^r zE_h19#AT(FERsFEB1h{Ny|-V_A0=Xw-8v=Gp7)-~f@Vyom&tPEa%(+DZEbR?^r~B@yqqgx24y6 zK9gHii!NAw6wbMvROrKJZ}s~dlWi``lUCuH1@cY%UXuG+R%+4OE>ZY9Yk&~NnLRG5y2}Y0nIlG zw)l)`%nP4sNduQ3TKgUeU~=CVCz?dee0HD-bD?M>zsx^9(|ER}qTEpDP2c%fsuPnZ zT?%ezl7-2X(9`vh{8?GOXMbB9F*nq*zv!YdOW;1czwT#*~A5gq*AEA1Zx5m+>#p z3Ppj(P4dfl^Y~V1Ny)TYr@plsl6h$-ct#_fDMlB*ZEOD7*uX`s2kefB$6=$dE$s`_}Z z)GdAwjwpjkq3LR9h&h>=RdDOY42R3$c^8c>?G@UC=z})TxAhg5P`+UKKPg6L;*8AE zk}sa_KxAWVP5X@8+IoWnWRoIik~|-Ey^6LuN1!!&uD?BvK|XA>py;rz!Bj0>n(=&i z_@O+5cK7$uXIV7Sqp`Q8=8IZ~#C-ae-%0@uv@gL$Z|TxViLAU-E!%jxhjysFa=#OE zGrOjC2}^DqvFm=Z5PcuXqW-0qVo{1VC1e5ZexXpz`l78p#RXsf$&e`wO72%4|lyMlGB+Rc9hFUvxv)r-?Vo>+)T_OdBtD~ygmE|+%xUE=q7JIg|d6{ zK_b)5e`223u}{^`GboMNIG=X%EASVV{(7>!HJ(OC3 zS9uz&nvdB&yu(G5O=D{Bo*EWEo?s+yC_9bBt(M!Id8@g8XO#J-S5i#*Jx{sP&)rms zHf>^$oVkXsOU58qekY84cz_cd*?r1gx9t9*-rYPY!^lqa)5PWIo*`{kgGs)AwU*>h zg6mN}20w>xl?vJmy%5puzrEQx{&hP?sJ}bn+=Dp8(weW%IaZ#$x5>QA>^7N+Yiy%e zzN?Cad%h2oS}3~^x>V0RN8ma0_$<$_A7jxp+|smff`o5S-3{ooGx-Br_lnQJN#>Hw}0t-kpO=fvku=968D zjqD>%%({E#AE?Oh0lS=pqJc$jgKTbqtr+GCmDHOcyxj>dN#%b<4FXHjexY-gwm`6I z-CJVyH3pkuvZsH3G(9RUuaCx`RVAAG6w74FMfqfd`|Q4^m#$@oMs-f*pAcrDNzq9 z{N6t|`CBs5dQpjOWhgB%E*Oj338F0!zu8RO+U41H^gXkFas4Tv5M&fT4{GZvvVk7* zTuoQ*#jETTZ_R1NLhvQVI zR~XBlxIDB!Z93wdx?ozery&8x=z`1OJ@Ux=DYYj;;{n$X)=RPynfe@|LK>80 zyz4X8NRjjo-jqnKYgT;A47KRbX6xF20Tj7X#M16!EgDQ|NJF3PpQ8_wk~r^)sE-RD z{cL5>(UY8m7voM33i%hoO5j+fh{BPG%Dpk;4;8W}))LRZ@$G!06`OV>g~OiTO3>Dp z)+)(u`DRVcAQxDd4sgB{^2=tVN5N#5bI9iZv}}7ifyBUb==+w!EUKPZ ze$N*c+zkGsA{VU6leNAdw}?m2*F-M5Ls@jcZ84-y(UST`17 zVChQ5MND>nBco3an#DCPG(m6reqHY=s@&Rx3PD>_+uN1FG_M4!*Sa0+D8;uFUI?jC zs%JoGZpU+*RNSXLd;+z)-oeuOnbyk!q+viX7;HSX%nIeXq|D+H32fY=?TIWvnzlQA;`W~+x^y=$r%ZcBs-`K zp5}0@tI+8>hx|HJ-ri4h(9O3S{{OMxsfh82!6BLir{t#Rl%b zavZRX-4;ucQkTFHk8uS67d{$KM_@v*QFm|uGG~>7?--j{Y(JxtSwep7>i{?7tp6Fy z6)}j%fwsBTz29$ghTNc-ES-9(5(NEHh$?7$+BU}pb4I{bGG2L$R{V~Ng_^F5)te89 ztYs3dFAz}d;`KNK>y*F(zK`hBF3Q586j@E-XP<9c42o?D6!&PFLAj_W?UV)$W_~^@ zeT)jKgl0P{J+nYML*{(|-uog{mK-l*XjiDG+Q9(Pp<+Pe!JX)=dAJ8u6pB>4UD_cl zF+w*k#K(SJgoAyi1pKzUhtb=T+p36}Bekb$yt;&i&T*lfsQ= z+fNIhQSp4=X`wv&2=1I#ly&K__SyMeghMPFZ;k>v3$bRuyR0&TgK z;iah;&H25CPuhf_tPDtSsJTkpPTjcVQn!4u&x}M^6y%LX)=S#8{&~v&L`46}8jpx2ZRJa!8m1mJqhgS zl6fWUDaXkao+BvUv)TqwD7n`Qluw;DkqxsSqz{w}s4Ocwnq(7f)U|X!% zKDrg^t2AR0^bvctG!;EJ z139ZcQ0*Iy2b~$uAf1jO&C*1Dx1>eIs8sBcKKefWHJ_hS)YZt0*AG8ALPI0Hx7E_y z`M%s_@t3wji`h&D9UgDo_gsQpdds@b&7@lNV4en4?c{5<{WT!{^>a1Fw}o8yhj_EJ z1}~7J2*{y-vi?Ohz0C{E((A*1r(Pk)2y^uk`g2dD4%RYcw=3*^H&%wJyjYyfftG!t?ED0a9%${h6;G>HT6Q-+|`%+ z{y4#3F~nf(Zr1-WGrPdKH#PHma~iwx4ZfRCN$##q*Bzt&686Lc-k)KdqN(DU?V+&) zdP08wygh!jK`N+;y-y17fh$Jyykzt@p>TQL$GUwN;Q)hsaA3bu3*BW7I5(dm`B7MZaIjp) z;m~Z4?lwz-gA=LBZb^m@?xfD9M_`2fp=T{aNBHs74JzB4x;N!rIEruxbd&K~OMU9ePEkH_cw?q{L}E>wNY!jt9Iu=HuI#TkMn zunxEpT%W)gKZRK$b+Pa;pK-jf9pCm>h6bAzT5R;4oU^G{Cs5Vdg^l4J$>_OUK|opS zKQjmaOtb$L0A2^kS*bO<21P2zx&$a2XJrhkm}VFEuH?B#;X_1MZjmAF$xvA&Iv+`> zY(1-?cO+3@3O7P1&^_GbX1-n#$?Vf;B3ZH`f1U9?DP8>b=&L>eAA^-rimHvt--nw08b!1XVtr zIzKk3vG~}>SeNhGhE7Th2%4*~fyTjoLP1EV;zx3ScyA1F+ftaBka>pyka1nm`sT{W zNcZIt_*Sjl9t5mj$P(24rXbeR@5huV`pDVzm6jF*b!24U@lKZ;e*yXbqDlXN#EqN; zY%x!FYEs|x$J!PQ5};YSUPqc(28b_)?*OhMD9>PgPR^Gn*AJjh+C;~FN*Y#TK;H*i z=pArfGnBhyY*iJfu1-bXW>}HfgVZo<_Qx0rs^sSxV-FJSr|~$N1zPp<$dfer)Z2Ls zCSG0J1#PbbL-`6w6KYCEVF?$9ekFr##fpOrad|oFlXf$w-NpT_=>!LlA4~Fvs^Ko3 zOM?m4sn4++Lk`JY#r^M*!@M*r=4Mz-w@$*`?Q`X~S+=Bn`Squ9u=>tU z(V{EV>1j0dbh#SXPmim8fjCG~^!aAyZZxg%m}C65WG!k{_)v*8r1-1L( z!Dg*WuOIg*e6tzA(t8r)Pmg)QnVQ*HAEkaqZ9?slnl9hNm0)*Tpz`-+^@l3sHZ7c( z$-Ee*By$3tCTBSQ#I{l2S=w`BjV~SfR8q2ti$wl60caV0c-t1tOlTrO|9B^zMXu}) z6DrUiy7Drd)(z`N^c2EY>&%1i97gUvZ!+!UPWGB`(dL~>^x65ffCwf*n4KX@ejZ03 zcy=zr{M1mu&3DrFdItN3ImmtWt=j?Wd^t~a1!_0?ARoVcrF!|MTwa}Rlw+>D%KMn0 zqzl#1X{3a`ie=lKbWs}sREOjP=w?ibzgh6exH?;-Dcs|UZN<)fYV7mBz@h&NVnwe$ z|Dm@02rikI9p&K@|GoXmq_{u`6Xzdzjj#WHcCB4?0lk(t2(H}G)QV~(Sry=o zlZc#WllE|q84IYg^BvC1i`J_kyWvz?JG5h>HwcoJQB?0B3At2U^t0UOZ;tC;H-Ho# z+Rd+MruuDk67`x?Q^=v(PL}D-E0idTWzDPztQDe}ee(8=BT9?(@**v?9bXk2?Z&{+ zr7eUI<}Dn0RB35bVr)_(`zW%n_i8BHes#8w%tF~@fuy~CO=pKLY)!2CnUW_zeVHL}FOT?j@Zytsi@ZKsUIa+Xq)+@AoZaw7%EU&O_qsQ}Q z-v20V|9Ym6(pYo*T~CViK4@F~M87OhvHi-GMwBYYPLpMs_-G^c9DS0<+(-Ywr+lrD z5%qR}LojsaBX5gO*W0QiqTN>=Oh6l$L638LclTp^G9*|nsgO%@+NFUeeC*RsM(lj- z^=eRU1d_r0144o3?m5IcrPtr`TNlmhVU2^_z#liGPlMKEv*WKKA~HO4vu*VQkHIC@ zmoqO_1TNm1d4^Jb@RQ%!3 zi5X7e{S~h5aDBB5k0H#dJWgIZXY-NgyS8jCFbD(rcKmf%sA7$TI&z7ITX#iN3OnEB zpYvEGA}WzF?qvYC_0r-7yw%CgpnF+EFh(v^-;EG6f6k6UIowj4R+3}EB?rK1>iFtC zX#C2h3P(Qb3Dpznf_?&7E7C5`?Liw>8??P8Z zw^E_#?~~FVYt7x=VtL`py&zkkDXe5?UX5C*IzI``wR%11ZQHNdD+Ez6`o~A;YhQvE z5qj!h7A7$sP0vI;T;KE8a~9q6+0Y~H^UvOmdytEFO~Wv`1okOLt2EoDw)cMg+d2m$ zSE$;2{hDCVplV?Ch6gTsjzuvJrqeKx@se6Xju3nUG4D9mG3K&?NZV>XnJ3q*4&*ng zyYI8d3{)7!n?CL9%rmPC!RnSMH}67HAZN16yQVEWqj>h)EjwtVS%2&p zmTImq;8B1x>U)jTlYnL%y64##Xv88JTDO1Z6D z)s1PV1XYR0AG<=yM89>r(KdwS#8&&DBh3L_VK!?EkqD;fjqsIl1Utfkg53UZgSlKA@fC!Dh$J0>X!Lu3 z{;gBm#mefrS22+~qL=CZ;a>hf{NFDIer)LF^NoLGhA3|MIV4kh*#G>w|5?ocOlH*o zvsM00EBzZs{m;4k|23-I-)VK z3e7YU3VwMQ_o{YDe85ev`13<14<`ZClL^*rnY#l)OtiTK^DiW_3cg)RFX**QZVaz5 zwJ*DMVM^36b}rz*iuXUitneogtgODP&r>>2z1^i4Dg4w85bk~%^z0f$p*Ba{>OiEU zc@SrIVzc6=$Qr&+kb+5!Z{){(iHYZYvEztKUmz@qj2&Rek(~W>Mm|LD8M=8q4-u&& z+_Dm;BMvlqU;~56u=E@_4xSSrQxK>hg3RvYer7D*QN{|agU`f~OTQMe=`G}3LgkUy~a)_C?o1|mft zk!x6O@TlB!3s6v<&+a?}%;MA)Fen|tJVn8Tvw-*quP?rF#FC9{AiIN9npJG9D>=!i zPdtGZ)su&tzK0AkVCQG3h1$$Zww}P;6R{#Z+z2vq*vwwvI43qj4mtD zsWkIqQ>ZvPFZls~j{FUKBWZ^1a7#H3NQ=-F%voWMVUVDJ$=9~VWqS_UmjN$%K%YDMg03|Qt;A$N5W39{l{yf#p_ftP zh#SDut;v;r2HmAx$@&XnEV~!9jn~lk!-~9$-rwIGs{?)hH1IL`Pwx;_>OF{WkbXTK zXA!|?GCaOj^ei$6CRc1keh82=vM>ID{l=TitiRd}*9hy?Q8=_X^B2N(WTU%*ozK^1 z2y8tldIGuP9!3pfhR{ug1I57U1pEshci=?o2=fzQm}RNZkX<24tUa_u2C8fyV<;(> zByG68FXDZOw{Ej|EqyV=l5tZ6yBgv@jsTj*eag%^>AUki84E^)#0xZBB!NX1t5Bd#y$pX3%!e>&M zv0n&$n=XbtD+&5USh<6Y+2;-6_HbP<|I5NaBSRYFN;!=Fk_uDN2@(~?Z;c1B&pE3(C z;gsuchdGeqS;f8jN~zH8Wf4? zi5UZ~N3;xD1*6VlnNuln)M@iIy9)xt`8fjwaj$GhfxF;3V=EO9Y}C6ktH3=bBiY^% zf|Jnw=E$r!VihDub6P*btOvw)>EG5dB%7_nPpJhOvAjv7!IapLs~5qE$kp@EpSk!q7497!LVy}F3P37m)b7ta2y~Adp(R8{oWCF6 zIj&ww=pvP#0j~ZIeIQbRLb0~?U_Do04VV8T*F=_~k2*s4NoX-%_f@E)7pQutU|V54 z-)e{nPT|^u&Ia-XmY&1yQx9sUIKlCwYKhk|Dbkm>op6ELH@Vz2cmo=5Z{F2?HU({XWAsUfv{BDA z#Zr}`mon0-`YR!^Nab&h=tB6}H(rL>xIr-drTzPImgOOMLKA-GZW5lr`gE_|IuzI? ztbN;mTVMwg*lK|%vnwt(zBSz`p;@dO#`}vYm82;56odrsOJHzJkm>oU37OsBlNs(n zC6ss8p?Rhd1sXw1;UATpC>;_-I*{KL)ZYKn`0y>%oKyjXbxmEEEm7!FWnNAY4+*2- z1)Om-3`z~|e8k-K=v#L6hrLHRi>4*vS{FKmGdq^cOpc8E+g0UCxb35^xX&0b&z&Nb zKl`N#)&0I_GTb>ph*)?7Oo3nHO-~0Ri6eo|EnTt>@mdPTip;?rLN_YKszd5YO^wLO zddFabt7k^xerS*zow4@sNiXWTJf123qE7kIrWaOxB;)~=RixXHzXGH;`s)Jk1l%T8 z*W2WHy3`_iLqZGtd7?Umr>WRKuGLK@NGIH?(TzO!?gB08yu0;8`SUH-MVwdc3htrB zc7zY6Y3k_Fc+x0Nc0?}ox%odW!vBpO{VtI@_Y#uXJ{C-6nX~bS50oT00!HEVFj0HM zi$ffNdR+LhEp9J-)UYLfhMCnFr4T2-bt@35_$j+&@SDN`i)+6bfBhHOdEI^4oMXU0 z(otvU=?7GSPW9#Z72+Eafix$b0dA18a# za9LcEd&eKZ-H;lIp1D6a!az=|zXlJAJbi~65zLV4aA~ixz=LgXNJp*Bz+OO zUSI~m0q7)SvMKSvDEk& z*~I8U@b%mY3K%<2{P5!Bmis+~{Deldc!cc3o(~);x%>!>?a};bJkes75`$6Ct_kZM zRK>{RD#$JL0=0XOsp4B+hmo#iWdxI4H1+`9pc3t%CapRtx)EEu*U%Tznk+hYQ+|9} zp?iAOrDvv{QC#v<^`{J5LTeJ^g5;;agHq3EX0yojz#Ifhb2)YF5;JE#2X0lDJKSD= z1~Gk)b1FY*2#&*Vobd9}SA&ZPs-s8MD{qB~F2v}XlXd8aMk-(HC91;B3k0_J%#*DgT$(E&O@1nIXO`)a__fyoV zPmwjnGvdc_HagFI!l(b;BnobsDdjIzAZtaQ2C)nM*hhkUJX#pz99?x1@WxJ zOLi9wFQo`CC#f_@w!iIJ|K^jEK=6c&f4mAvIxn;N5%iAy;W+UK1?a7#p)W1t+*_uB zDamYaC!EYxUHpx|+gg;WZ#B5@SGGWxHI~J>^BN9XZ^+$esJR{!Qdv$JG{^inp7SuE ziXcHMv11>eel6^f9#F;8tOgJ(zCTKkL3R_sKP|g?`NQ7kv?QP-xyQrB?>AFZ>3n_d z82l_a;H+_AK#Zbc6#&T*13_PzU|+-9k9o6!uVk~r(QXD6Nrs}(WXEnB3-7}~;O+Q! zRE{5-n{`z4EC=Uk;7f5^Dl+T(0w6Latwh`LENJBpJ;AMyxYIYx) z&=%fnzSmB4logR=8@SG&)?n)m%@fXxt!1(`g#IWT3=?u&-u;jk5mJfY7zL}z$Wu|9 zhSMvrizZzL|89j?RQ^~Ys3O2EPTcFkE1qu>8j0QtM6=+F^B)Vn=C0^6M+$3=V9yXS1978s}Ks3zRx0y<Bi);cPnic>1+c$y8%5PYhCBx&l9&i)$JEyos12~Od=|E6I7fC9we*E#_W-PKLaQ;E z_nMiI?vDKcI``tT8k8mOP&iS&5F8Y4Vy0b+;pJ{TfHWvjr3R{F{*6plR8fMDpQwv6 zSN;;HJb3&^#M1?+gq%u8E06jS=1m!Tum7kSMU4Hb!fm6P5oq@@&ik~+40XKGc?kj9 zKlL%Y7TfNWIfd<#X0Rs}0yTM}IZye?0px>hEnVt$cj%P#tQjRo?NWeki zKnlM;lU#b6cLLl{WC5QDY{4<}9thk?YJ0WIjZA)>o&hVZp$@Q6R%eCH6$#EMB90*l z&hc8|+-J>?&MP!C`tB`D&%=~G7mbJhWL%CZW~tA)zI`hyc7mZ&XsC1>3JJK?3`h;Z zOa7$j4}sCAd32>BUAs>Nbi$B7N!nFSpUj+!Pa?t5eH>F78zHqIX$bc0Z>f4?o&w)=wmQ*eZ?Eh~!DH;X3tY3NLE5YwZ$bvAik z%i8T}rp;Lnqh4>)Fhn@o4uRR582MB?n}icgM=ZAm#3uhWxZtO+W<OYCiV7&OT3mLI^5mlH8eQk?^J%WI|KF&#&m8M*1tQgMz{WXTG2D-uxCfz z+z0YD&f(TcU7kw>nFhJP3UannX`b{qe|$(taKZAy%E3MmOjBy9DX(5PZsrsuq8qhz zFKq)%XBKqsfg??b%ft=t-zG9|IQ{EE>OH-5c8f9kOKnds^ z97n?vgC7K2>6Yh32MpJHQqmART)lIE|mc!#iO zimtrn$xp^VFQzA(3{mFoI~%mg zgY~x3CDbKfa^fQp5CuF8W-k3r6u2Y)58_0}nxxn}Xkj3t+aiR+3sFUA%v8diZJHv* zK?zO8VCZ;F41wwYUhC_-`K>oMDU{(@!>v)t;96?0HF?|XqF@0D2Im8R@j4y2e+ zRvz}Gz?J+1dLk^G*F_ts8hb!oWP$U+Mdk3-hgd0~{D-zdn{=T0=vGq5?cWOH9OoYx z1jW+e6mfNfW=_(1x~CS?MqG#50{IlP?}de;+^wpmv8OmFCXS*J+Nb9pgAjGk{c?ioz7}T@VcwenmSd+2GyA+r?ExQzbYUS+u@M_a%WQfL&7i+%>xnwF1I(Z;=hJ#{oFDI)rI;T zxb&rDVA`YjSQjOIQ3~gda1w+6r3oVgZ;jkl&1x3)Ba4#6wn*MzL$_+h4K3WioSrKI zGbqbgp4h{lvaS>C7o@=3Q$hVFaq!=tygf#0EnjksJt9zx-5SC1vwGP265<@$WNZ+g znOyyv+UcV-XI03ohtH}=J>avA38>1?Qs`m+JDJQs4iilBsMRBeC47}9z5C86M{@>gX_XxjU ziwMs(K;Ys`+!kT)KIw`O!UQaFo@YZamV%9dz}P){EcWm{Y4GQMMY+kw{vIX^2ZQrd z;nm)0F@Gn@Idc+$MeKT_mgQY#=mTYixikdyh6aVOv(GL8VpSa!c9Zy@T6&2Ee;CbZ zJIhkYQ%*zhx|H$``FHz)qw~a3wdRteH@^PkUnmh@d2|NXiM|Y-a?wZAM*jBa?oh(d zX)CCp{^7~G@@SUvLjV{HZ}D9GW0=RkWsSeAQqeV7ANwPk1%IGsIhv#4OiB`KDHO1- zmY=Ol{{~CLR2c>MxgVXH#{YP(;Y<}LE`h9=d-~S5?YNE>JpbFxov==Z`X6ez|Ct>&6k9fYn!xI2Jd>2OK>-kp!z>}9QG}+A` z0d_{kwA&rXw#~1Hy(_MLMb{Zt=1^)*_Kx8IJ0#42$C%HF{j58(05s@jW&-&Lv+Yiq z_-v;awu_&A0TY*fYIY%UQ4wtRGNHbWp5!>IuNPt|2>k|Mku2qR<&BlQneXaYokYQd z%f#IDYFf^UispiyaD`!>?q3uDByB#_C74EvO3ByXx8hW;6V*bHRSQzF?G?KsXjK zh$ozhrK5<(HY$=Ek^4Q@mJC(Q85^l1Re4K=fXmH3eSH78Ls{{=Cf*^tdSeg=m}u!z z6O#jSbGXG*y&4PwW-tI_0}1cX(^R_IBLPgn)FA1z<;;K7k~j98{h0Na^Cu@w24OeJ zS?iBOd4(1JdU#*Px&$W~3e2odM#cIW0ae<|MXz}fxmrk;n6lQ(Edc~B*RE!J8VBr~ zz^6_4OXQHxvU7i4g#|sJn+XSI%yVXUJuvKT<;Sf5{jN+?G#G@iN3FH#!WPRMd zG@*q1DmtNqOlZt&HBhuymlOIBI5`Q|3ZWslUCd+c0@nBE?(XK|BCeGbT?(%22dII! zk9x~0fYrki6c&6R&c1LqH8cpunbz<)?BD<_Qi4@UXKQN`Q!NeH*BiL|t5|*8Y`XXH z4;VXk2z|t`(d$Y-CUm?~mqAUp6#_g#14`r1F0p2ENG7zqtVW9efvnx(VPH?I@X>%Z zw85(YZC&p<+u?igWUIyGKr`w9+>XR^zF?R)3Y#4$w2HHg(ZjyI_6!S8+9WELDuAX6 zUYeIIZ8oVoNM!FsI1Pt2Rru)!Pp<%>Qn3UEuq}^31N&mFIDnJ|llO+G2o@c8(9rgW z=|RqLgu-%b{pi?agp@#LX)$I0FoFID#{EeL?@l%}7ns3CfJ!O4nbq%M5b!QRaFL)I zY~Be2jl%UhZ&seX+>sDh2n=XwYvA&N7up`TUTEH|9QT;=U`F|D46gj#` zPeKgB-c^^TgZ>;^LX$L@_CBl`e(lhIN?@|jf|ZX@Wf6@fT+cs7g%n~2ORb+Lw)q8T z=xuxon)PH0f9absgA(ZQ_zbkcJgz#H5#jl(RCfQD`oX@+_x|b1Ym#F|J1!8+$)I?1 z+}a>xe>WQ5G78W9lV7`V@eppGgORM12Vu@Jckc-y49$Ug^Hw%l@c+4DH4Z(GG{XIDW4`ZW)efg0+d z19*CVdUPL4F6cf~#xM_~9ge0Hy>S|o0Sh)3Q0w9t*@6l=e^F#;ZdU%_Yps?VX;u^g zKAWh3wpKMpV)e%9I@>D(NuCGmVA=NWkhG;m_nOHQ*rD1jII+Uk5{>!oCOkSa z5#+7V+z|qN++J6%4zpZaB}^-++j3Q2W7zc0QE%*oVWO>z>T7-1NLZ#eztfWxvlzaD zn$RDUV#qXXL9ch8ruG4$4cZZc^7af=(G~Zc=KsU<&xOdw2vg~$(W|CP`@WA0}t%n25q<*Xe>N-B5t%P z%6b>VrpyLU=)8dypxd`leJHD+Pfp>EbIP^AYrtfFl(o_DSyFeLP|ob931Z>eOq z-saZDYnd_?p8q6_&|X*PC2vS{o(KWA`1$`eeTUfi;ZhC zG%d?4)nG0t^aznvO z0h@fao(L_dSt{x5R&q>7IkbvfxD>13sJ&)TMe@d#-u=QwNluVz&7h@lwTLr9f+O>L z{YMzFNw0cu)T4}$2Z+`U}C1bgM zfHyh3)PKG-Ru~iQk1}_EjrxFMvVYl!QSnqAh8aMrZ5spUAM3k>+PD7*U>3A8W>osc zWo$V=&fMntiDv{kh8!VgcQ8>spG;GC%E0wK7hB^vN7g?&xY|R8Y;4J?g>9^e>&ut=YA{j0Lp%qg*{$qK)qwO3@c|wG0uL$BNVBnvRAY zUp?D8XzTx=S8OK^Ux(cKL*8!7p~_JX!=HaVBgt>c)##pd?3Bb?p7Oyb=|WATgVE0}tl!JVoTyS+ z7i0-UPb{vC>yhPpJ`AN$zi;y{K}d6aN6bN^R_v>V@%yAEWbD@L^%44w@0$tzi+#ST zp^ID|FnP)Gc#hP9pHQ9tQUHD@! zWMSEj@#|lzo{(=iDtuAg^MC$n;Ogbu>Agt@=D$T=bmN1bq1YJCuAcS${pHk{gL#gL z!!O}^Tws>;aBgc_o@tVo52lDjq~OY`yIQD?!G9QYD&8Ea9YB5kXC?ghG*q9263*Q) zOiMd7vT2Id|0KtWSLLWsEG>#!Lo?0pjBINHpW2H3>iMOLiwU=Tn!;3{YO=oINwgEi zu#7pd+6&umvSPzafdQY%yF0eBiU_J}5t5&T$z^Bk z3zm`uC^9Us3UXm>1kIHn+8f=Sri81@gH-aRq3>3jMT5hE{(9$k&}R$1UiK+@y8q10 zj4OW=se>~+*%4NvQ+)?%Yobw;|JB}ig*CNyTW=L9q5@JkCj~IJxD?mq)QDXgkJs?_qWgS`_A#dI+vS=3&^vw)_U7~=bU5A@giy{ z_!oFb%Ml<-Qc9&Kv59@rN>op`VXsHiA2OF-*4+FRw)|m&k6Pxh56}J7rAom^Ed#>E z;mK3vQjBp~G$sRvxq``Fh2aduHzDOETqBoN-#K|G$3KdSbfA2YNz+$Dcn++oA&5ZbkC3`0@Hq4Y~3szS&N z&u^VRJB?r|Fm#1}YAh5@>_JRUd~YPH$AsB%;UTS4Z_T82k=b!p2!n}|HSy=g&OR>c zmQm2OETDGjF9OKZe(=nH;|*aRl4AE14K)D`zxYxsyEv7xEZxy25&qbC|3R*)f330*?2@RP-n9HE`&?OlZ-B zluI;98vr0gS2Dojxr`qElnW~GJ&*5qb)l!7q7OYY=Hm^%&;<&9hK47xcCRO)pFd!< z_|x`|uQTA3lBfVaL@*jt+5Wr2TBpAK=cuD>hpxZs*E`Z4w7R#Bi0J(39xv% zn=D<%0@AYn3m45HH2_@i;E?#cRj`qHL6&@_M7JLA- z0GX$=$gku%br->yvSKXOCF8B-i;e*?7G?zs5n^dAWp4!z>-A)L?EHl64^UgbsyJ#L z074f}6iDvw)UR1a!*z%FL=&G)b)``IUnngfN3TO`($UiS(QoCX#o&$e?qsUbK6q+X z|1w(jNuL*p+N^@i=Xei?@$SyjVCi^zA!s;xU?~kn7|g@{P*? z*J8-jX2)zPJ4V)H&Ki2dnBDW@{0L1X&xcg98QrBV>cvQKXsG|lWP1~U4SN}19SWmU z3`Ke;4UbpvHuB*I&af93C~?&gyc&2PwnuARQb5WZ$6NOmMt%ITtYpMPF31TVk4%rv z8%Tjk?@luR5-{>4kOO0;ji9i8`!kBB1Z-*jb6Ho%$K7)W{KovqaiGwbv0s5}e9>tf zfTcO?Y4eQZ-nJq&Nge%rUIyE7HSz#O31-jP_=EAVqgz60J#wRRL6wds_>LfRhnaZ)ORu5AAR z4m2)v{a9hDT{~d3?iUPDK8f@yxu-H_%V1dnO3Flf=j=&uzV|Mt(e#sc)!^A^Msbyg zCar8Pjgfa-THwc^Xeb#?|1nhNY;QMENrDLb;A-E)23sE0T$yinn`N#gss6!AxMF@@ zCLlItM7IOe#R!|WMORLn4x1QYZN`J(JSH8X?+^PnpOCE>Wts*;7B3%{zkI|nOBUt7bBkA!`*rm4S({I? zmy%HBaBM3)y8R+4=l4^oroFO|T#|ZU!7VQ~@&pl=5nsxg08uK6jm#^v4l`D*Qc9kO z09CAL@pMEAcU0T%5zFK?>3kLsDDH6G6nvNfehk}7gs>7)PQ#nz&&DU;w-W3au{f5+ z6tyy9h^x&_db4OI1;?fDwt_GLkmLCN?6iNgKf}}n^Qlzl=)d)VK5+q)d6Ff^h4@KC zOo)-W@t1&L15zNxkyETCx$3mPfj|4ua|Quzdnk(;g|GE_KCtZ zRm|-9nLCd|(KfIK%(L*qJd}8mWTW+yoC#^KAW+bSrvU&_ix3MZRqY77mjtH3J*C{2*k$?%WLs@~dAI7*; z$H*QRiK09PBaa3fJCK@JmnT+F;eZB&m*I|q^0kp~$j+-kL6w8VxYR~x|0v%skiG4* zp1Wk=A~7upxSExxhu1n7^%@3COMyLmD2{`|BrAq?{qYdoXuYFO+k#Q^VQS}-84Bgv zEuM~VFWai_ouQ+lL`kAXFC2h)u^&Wx+Oy2Nw*Ti-`N3EsmUMvnT4g0ORXvj|p2Gv+ z2SnHiY}H;_sYcI23vnC%WIFNPmUil$*2_)Dp_s!ET3t%aUvr=;(v5lhGm8sADkwNJ ziC7R8h=%j5q&m~!R13%=42k%gC8 zj0tbXR196M6ztmUi!1QRRQr`=b`8e{Vxnj4a&2-ITQ6p`!^2#_z{ve6fE!kzu8UJ8_c_&6H*B7E(Si<*K#K}vdF{?+;z6c)#`Q115qYpqHK*D^t^A8 zi*#&92RLd}_+u5L^O^3W`C}jzFgvf!3x(}_h9K{DaIVC@9eJWUx~~MVexZB=?Zj&Q zr`KU2K3ti63Vg#sYKF89ozz;p*iVr`cFIuLaPp?mxqJzelp%r*j88iwIlJ!cO;C1D zG@U#vIAh7-U8*H4;`J))==+%HHDPiEszVcb<5$e8KIx5Aj;kZlk8S+o9UcIZ=|9YG zA!P>aLisTLQ?0>dUPsM$yY`47%dIP}kNtHEbm+P+&a-f17cEsI_+aF>C~SbT(x|Lm z;1ZgW@;4r`-)Iv0*K3B8WpI`0x4O+l`H`Qc$ECk2MB+|*+KWC6Q(@auKs1HCTMyJZ z7CT{w+9Fqs!;i=nG1JzoMD$~f%-nl;icLzI;q)gTV&N(jNO|7g&q1Ll-0sm(N|w`( zG*pVw=IzLN)qOvM&U!qFUqu{X=*~a%E5+%DKd@>cTaI502!Je8_bc?&D3mXWyh^ug zVw(i2oaYa#0q>6HDR~J6fB&V*mZQ+zh~vOuqh0TU9`HMI%3!#vCeYt(($+hI2Lcvsx4}rNB8G~ns23Eo2N$O$_J4%ke;ANJP-CoV`EVC zlDWSEYA&n*X644}bL(fkRnG(@wZD}P)n|2HV_uw~7k;R`P1Ov&|1}^x_}#7xK@?mT zY{SAtErY)Ql$#Wd%(8dW{!yRxTt0{iBq^94yu_i?CAAF^wo~Eqtee}>2$vbE{?VFn z<-*sH#LJr%Wv|W#JN#zIS+nFclv_Hmv7R2xGcrY*J(a`R<@uFi(?IW4042z<-AhYZ zmTxWOU~gq9S1fEy;yLMMctN+-Y}S)m#BnUj%h&verHDL*18irqdP0msUMJT4*wiUB zq#z@IF60KBfP&M0rb2_ zssmEv%WG`7=>3 zXPqZs%|(iSLwbZO?YegnOxmx9RFjw2?pceT?;hP>F>Y$R!f{XzaQtVojuogdg`3=r z!=bYw06C=o2WWw4%ak~D^)zDVG9jFkz@Fazm-sx zSRzy$} zWKYiq159LeOgBzK4-fja0U(0GVx47qIBAf<-Np|V1~w=v@cR!7GJcQmNBRKBUNEp1 z3Mt%4vk`;=bOwvn+^6$DMtX6kHlb#yTe$CORM9E4Og5rgdbCBafgHTWY+V&JvO+1e zpZ!4<%~~{15-$_l8?7(_h#D>)+KSe*IA~(h0nvbmyUevq!A_~lNX}f`3?x`f-rJyH|OS3YIG-n$-w-5WDuXZ34S0b>jlW?TnUBdtVqM0 z9E`=erK*>qKV#>kxEK8+bbVQG`W>-zVnC#z3p?!u4#fbn*=a6{^G#IJ=}zKG1IQGd zQ}tWx3#AClYQlnOg|EGRNcG6)XM?97LYSk3*YYoBD*k)OHavmHizYf;K|k@L89ZKu zK3gwFMz{OhKC&(5rR)oUSV6pO=S#=jCI~FGtqXDkl%xo;f+TD_kTU?=6iA|#C?f&t zUlt-B6Jm>Z>_|;%kU(A{%M%c{q}YOEm1ixOSEWk9jcq_#PvTN}@4C?(WuOud(c0E% zo=RD7rx7Ktwat=n(u(;(J6=5V5lsT;c#fyr!{uIb+j(SPl2-7_W0e4-bVsll%?aR& zwE;ELEU0@7b)N%N@07#Yt8kR;5AO+Gy~zHv=+8Mnlw^nn;qorrioAu@d(Ks3+20q9j2 zozw%F%5f}vw`G)h<&s@6pQeKNi^O+OV*pvW#;R`|DlE z;w${+=?0_T?ms9g?LiIi_|v0(+xz?OCk@}Sy;g0+l80VU;aZVruq5r``%*+P0D2c7 zdi#5xIb&rjLDu^`60)O9*<=H1HLt{uJ|4%sdy;N1WlYCo^u`eDUBgpy+=Fxit2v1XBDlO@ysXzi>hexe^Nwf~%^K`Gppy7&&;-J@NZd1;ov{cQ;c@b z+m(g#ue1D&p@;1NV$F|g1608^rlMeYwOkS^juf&e{h|SY^#X#g_C3AI_AU`%`x&X6 z5MlYZpRaY&=y7VxRb~}pvl!yuRVV1HG7?k+dh&OZ64iK0`BkLf7_%Tj1{EZGRpfRi z0aKv7M{^-APnp;hJ`&e#TUk-e@{`@mkc*_}h+A>I*etVrLxdbn&f6FN82#30MmiG! zCjH-?BO|ad*Ea8Yp=zQM015KuK|iEr7{vhUNcrI;ui4NaU1VQK&_z&U#`f@W%w(HT zik1bB0}ZVN%Oy5kT{XFQ*09`}C6J=bx$}YL(KG3d;DX>qQX1u9_d6_(KDlpeO){kn zyiL!68mtY!K4b}2_wh!sBP9^M0uh|jlkT#P>l!t&c$1= zyhH37MI0WP+?q{s;s=yP4=!CM7F-@vf0z_j_Nw=^0^iGHCB=}2@9v6@Y14Rv#lYgg6IRJOLJ#yRrw(L*% zbQaE>v-6sGU&Iz)n}e@)QLVNf5^waaMJI*W*8(ED701N;OykTf-N$wyBPz_~sNu5R z=1TYhjS!FTBz>gVj8pWN6o_^Gi~b}t@I?AFP!i-IB6T~<6Wn8ZRf=cH_$?5OmClD4 zVY-ok$owyajQsdFY_Dus^PGNq#i@Aqj777S7t%B>Mr)H~5vw;|(qBY?uCIn4nLyZR zD8$%j8{7d!Rg=yf8nnlw+a^30QZlw}V-NjA$n(fi+d(ZwpP+8cQQr+Z7p->-A85H- zY}C9qd$`OiG$`~`C4ErPt%+KTy2jLXbp@C@bzY6xy|yI8Q`? zB5Qk9U{lm6D;Jyzu*-*v7LwbGlGpig{nqQ+)8RXa`KGX8^|iTBCs!9XudUTorv2+3 z`PH>f;Lr~qm;r#LWj0iVz^Vkp*$M8r%^h~_dkhM#aEd&1b@6iqhr%E#9Zshmr%^;E z-@6V*6L_PX&&PZUxL(TtH#g)0MLrSW$sz*_*M3Sgbm9FS$K&cdL&bAJ@Tj2d`oAI) z;dF`n#`z1;S@dcb0sj1xMwlH=%f{s|QX76Ce*Sg>d!$~!bjDxb(Z$O)xQfZ2;x)JG z{}tK@$G-{6RPNj)ii<)0&A#~8==v|?c~Q$7?rAW5X!=zi0Op{J$wjF_@Z~j!H0aNG zRS39O+U(^U7~zb%DE43SbHl()V4g}nEvfto>*)gaCjYM!eNg(R&`La|$N%9yfZu~{ z0P0m%9-}CJl>keQ1142U<*X|b-;x&`fIkm=Z&Ts~a3G7CU8{diwIu@lxd{N{0Lp`> zD*^JTt`M)}%nQIC?&<+-f+y<6*N^3uPBre|U;1|docsxMQPE$zo}cJ}fV(hr3=RRx z9vx=a`+t3lU!y4ymep$lQW;;&Ax;s2M=A~l@f1SoT#>$D*7nYw197V zFcBZPFZl5E_8Er;z?a_Slm5KlFP8&Ph(C{EmHhdrdKMmlgjb@!HM#%s&26*~|J+-J zzu)1%zB7FDf+W~kA1D9j5B~~MV$2t@_Q*(;7*tY51IaJ%{9o7q>yIZg5-}($&9oc; z-K_t-gxpdQxaObx{EsgGyQBW;pMU1h?+Ao<{_HcJGJ3kWd2TI1#u`uzGmq^W zJL$*Jw|ov(Pe(_mFQe7D5A(QCa@ircV&Br!?XOtpQS{l-v*bsn#>oPHj=DBR(N(Sa zGplM$29KdgpZLkXqFJy1Nzn&+x{b{|X1+)5rj?@{qqLv@Eag_b6SAiWkZzPsyLauKRkY7{ z-U&gs&>M+ECkt<$f1HV`aBNCrVn`jpUe_bDbZt6V`WAss=-=6t z;fgYq-Ir&Y$~hIfKePV$f9zHN+TMos{i(%z8kQ4FmoE}|+y8Kix2&2aE%D(cQ+z;1 zJ=YJNofL%LxUWNGH@|qe`pE$_wiY&eoiQ!g=~VAlIJcU`L&X(?Qz=iEHa)w+PX8t! zO{I{Hdbi0{%1i6%vnwC&VYW-RdRd7Yphx~y3;dT5wBjO9u9c8vT6d4DiaT!E97nJ5 zJPET2^VvF42}JN7M}@Y#PAyuIB7E~ibwu4!tSNajqKSlwB1~BAurlt+VA;|{F}UjN z8f=GIs*=M#BXz4!s*1RH4OS-NnL%MS#){pmB>J4!XP>T*Mwl@+Y;;^hd|t;DfNQol z-4UGvjj6u4V;!p<5j&>aQoU(nVm!B+R<^0IFQ&pYVJl&?;uVgwF2gG!xsEzKe{PEV z3Ox2Xu1~}a?d*GO>MS5anW61XC$IJ}vwpsJ(tMY4PS%MkWNX(J%XQ?Y9qIJyP$K`o zz83BYFWa0H(F=f|y3;2oXeHHvj-a#^-s?TCBE>x61@M#V-qq^k&o#+oo{}>i#gphD z1-8#9^BQ$c4g1g*i;;O`<}f(p&-d2>`0)$t*!~~^BtAS_3LTOcIl$5QpnEN{+^J;d zIE-phCe-a~@X+V{GLCBIOP)Cs4qWnBQtDWWKhj-)Bz?_86$y;aeGF84d_tC%c|UCMx7!db+8*=i-ta-Y(S6MB~_Q>#jw zes{XGSZ%dJBc|$9^HRlHv+;e+kT80X_4q$>la58c7&fP&wO4nZp#pBO%0yzkgg7?G=F58aqos9N~W$CHr^$Z z@x8+T45h29!{*B=`GS`z)W)d2Q-2ENO(3iD5j27YbLfv(el8_bePfaQEtZ|KJf^ zt$25snJ))l?k{%Y*2AdX`P*OlPghU#s(<{09@2_uZE#QK8L!Psx{9YSC=~?Uh6Gqs@MoIh;bGG5i=Y@G_e9c#FEbws;s?J-gTB zo;&m`0SVej8B#CWiJG?YnB0sDoB|=FCz=BD^u_LrGhUcZqqRfYlm9v9N*0}wW_X;- zIXrC<0-M~`5+bLzHyreBu1K};+H*hOg1NN1axd4&$QTe+a%J9uxmle~!me%C=1tE$EGcBei(D2RUKpU)9#@=>F zTwqjjc%@-8GO34gX>D^y%uI*{#J6m|UqGp3zuhYqNi89O%r>|>)t{avEU5|T=D9O= zAUB_gczoAU$z4fQ2P5BpO@j%~pK!+LRRpbQ?a`(0jhC=ic`p_St=5L8_xTA+EvF2A z6TH@6xR-T$SRgQ3PAi(?2j(KR_j@1QqB@dC@t;%X?=XlXs^!Y9#ml1@;_nHvOfWUg zt2#~fs8FgoCC)S3#RN|iC1$RAk6m66*+G(TkSN7*00xZ0Wb^s+M`!LaD3B@dWZl`;uF=&VTEgk_w1de(0wB&_w}p6!0wzhA6B zrE{*$_&A|ALX&}z(Qn6jRYhg6cW5*H)aPc?B6?vQEv9$v{r;B(9H`1%XXu2G``^O7 zUzShHtcFoA8Ad8E_I63Emq_wQU_uxV_=#ekh2_#`8~Ak9J~7`!#v6!x-6plQwD$G-?B}MepsMyU?HMZ1%MHO*e&bb!U?0u8>c^5Z zLB$yEy#;*dz@F11nsXl&33kPE^)#$xqA^y9+~J2SeE-4sjqd4MbQ7wM35LCsyyjG=KTUV)@lbZO)e zNA}VFdV;-(UtHDN11=jOWp4;r_{T;I&%5JLy0L|xrT5H>(`r8Hy&Tzl@c0aFLP)3RXjkNFTFHUu)af23B>_I2!Zie$ zi8kzMmZ5V$onX*Vp$&wuwF)zdl`qk+=3IJaHf9gys0%Z_S!=%kq#`h^fpVWIK5NoW z2Rgn|QFuc0&{IH$Sl0^8+7s@z0*48H!%cl$oe8mkt#J=_HbH=G=h5kUH&GYLP0s@G zc`O0Nsh1xq(qX92L!Kg=5X8aS>Qv<)g2ma?&_dd;hva>=z6*WpIk7e{LyE(+47mEY zxN=0Ns#mmJgRMwDX6(2AWmy18v5O@5z?JH2=b;VSn{Nid1OYGe`oZ+DW;gxe%d0wj z-yESXuQew$^x1gZ%89jTEHqv;WC)7MT-H$6{g$*TOEagcZYP7&tO5*oitWdaO5bg7 zU(m%gVmD!Hf%BCrm$4z;2M7Fm%_{Y~LAli4owpq##~L&^N@oVrt#oN_%Y<9hoiQ+s z8PHS;c6lm@xb5n1rmf3o#N~0x9jEQRf)(x4FKUSNh$((49N!wc*k~vw_;|p#$jq>g zM{q7Y8xb{il^r~}<_vVz~A?6{5)R^dk@VWsxbMU1?_fp)V!>*XOa^i*lY7T zsjFN4fL7`sx|MYK-Ff=uun`l(D&n&&>OgpRx(|mBZi*7KK(R6Hbx1SDDvS-}<4*kc zNJxg-u*yyx4!FlLyugvK_X#0+epZS{i?Yb#bdj|-i^{MT0W7-QLLVe;#}QS|DG4@< za2U`_9Xy*O9c!G=D%@Oq+@+yh0|9K+~dCEN+@Xm$YaQvql*E_`C zi=^0eVu0AR!NSk^7A)>ziuNssjd=GSd+5I7#t-ny z-@E;PZioKOjV+*)*Y@Xg6OyrI*yPAEg4z<0Tm^Z6P zfqcsDuBI7Do2gL6Z60sVZ9#1@5+(`WFsxp8jMmy06A{qMa$aXua9O5B!Kp5_PUhYa zxq`92)Xq`K3KF>atw96b$y=f*IU^lCs7xjejX_KwxGc^O&lL5a|cMjESd3>alJvjn4ns^uPDUmRT*M-0PTtl5Cz* z9cJ)}J?|@E>?q1M+}aMRjOIiR)6LY(Is1z$#VS;1cQ><9;d)pkIEMCKb1WD1P@=lS zyefY=tgys$6YxczcN?DQ`#uQB*J}oL{Hj-eFu`%v7}}L^YF(zmbaY!Aw)1eFPV4h+ z?u3Dpt!rGyYmF0(CkTDhH%;zVv9T82HSxBO|p-Jg~waPyS{uF=_f;A4c0 z06cY{{P_Az0$228gT<5&W+t^ka5MJuJ1lJ#r#oa#GJ7;n9NB!-*N5SLSTtFxQe!)T zjC`2^hVBa$yCwLLFH-Y)v;AhFa5S~ZGc8_rY~(5Z2=s&b&wg;+O3`Xg^2LRq=1pPL zMO*VmvVdEYxSZQwA-Zeg*JxSpVxg`tqpf8{d2eVapmqhZ%j)&bu6gXuF7A@7hVlx! z?+S(uF33GqgkH6dp_Viyh>F*|x$uQ*Q7NsCXI8JY0&eCtX6=6JW?sL2l+fY;V=T&P zcsNn~%}lZ3#^OLqz`9DihY>RcaXYV zrw6-^(1dRN1izrp<5bKmPdm3h-;ln-3d%)|vx-K_|J?!hkC*-gctS5|+u`OR9y?y0 zF}eIb#|kuqH?e6J3l(tOz{S+5%2SM|i=l4(7%-Fqr`s!Ph->aDcB{9&z8SqLty-#W z0#Z5ltE;rj%U)>%X|peTslAE}oR@3)`=lOn^n?!%)vg`xe!s&t=Eo4R7GEL^vE}38 zjB@;D`|iuZX_PMKg`-R}eYFi|ow@IewyPtj6|8a`n! z2$S>LaoG~Yd|mZKmLdIgv#jUiDJJd&P$Q6K8Y|m16bCXM=bf?feN%9EV9!cnViXrC z^(a<9zP36m>f;jUdw*{}ypaOf^=FX?V^ZwH#=6`i?zWzMv75Ee z_W6gHhzKCAIydkBZoU*tSOvhzqjB0RX|fm9nSgsYJaCb*wi_Fio0J3Xi)CH^Ixq! zT_buQ0O#+x{u_wJpIY~my5!94%le;<+qsE4@BX^GV5r#3w0fKup`^O0iErKmlo;k~ zs>^!gDE3OJysK7fcwy0y#RcP_Ak`hMQdhQ~@Z&HA|LY82?-=;)Jv+P+2Q+ZKd2c+7 zX0a2iMyJZ4-}S|2o1mpcjZniS@~flBtgbOkVdbPWF-A~ z_L#4W!Fs{ER&(5@yCB!p&^An9W>9*(b>d}85$7Gy#!+c);D6lwzt$al+?(5_ij`KJ zzq7=^7jr4d7vs$zTnqZ$O>RqkQoI-de>Cv9{5J!JZ{vq&5;0oI1y9I-=d}Y6+anSH zVtbO(CH1?+Kl$w!^Z+7f^%)`YyX_Nr)|b1tdpJgI>3`R^K=UQ=-JAPS{?cz<6k_s! wI_ICx`JZ$CpPBQ|vie`F<^P$@X*s`8G#hB~qQsZw58&syjPlbGDWib@2V_^j6#xJL literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-parallelism.png b/website/www/site/static/images/blog/scaling-streaming-workload/3-autoshard-parallelism.png new file mode 100644 index 0000000000000000000000000000000000000000..c58afd18591c172ecbc3788b6eda4cd64dc170e8 GIT binary patch literal 88744 zcmeFZcRZWl`!J3cMU7TbwL9$5(o$QirMB8DREeEfi7n{RVO6b~F=}sW1fi;>HZ_CT z1VO9_K|JZ_JD$()`M#h3e}6oAz3%%y_c`Y}*SW9jI_F%A4~F`hjPy6?DJUoywVpjO zqM)EwprE*T^%4!aCeG&RO$v(3Dz574hFa?C{DwZBPOct63W{eRlFVr>jJw&htUrHJ zbIH-6L(<)R5T3|lMX~<0>dI$|oS)Y(iTXJ{Z?DUKNy*hDnmb8Zee2`fr_}`ND{Mf3 z_+*ltNH(^057{jz6GoEXhs$6FBrb$ciNrojd|h(klUN@|SLDT^t~-T8S_YI%ce5{t z-DZ+A_!%7=OHp+hA0~wZQ$3lo^Zn#@=6l-4&d948$3StF(ffu}Tsp(90L6}I#j^}% ziswD)2E80w_sZ0CE?;;P5P!}2w|COD!tPy_RJ{;!hDCOYM3aZy-ZT^?=KKay^G<4v zW3cy`Z|*MY(*f916T?ldt3lLVbQ^Jx~i@@p6bbfED}1puXpa%{r+f_&2Ny{ zk-8gJuoRXu!`=OfS+)fdfLt84FAap>Z_%5oLqPV;LyIr0oH_F&f2i9%-Y+mcDrDG0?XqAWwY-OOTq~?X ziDz9c-FZ7+7-gwT9hP+~`8bIfB;2k4LiE7Sc@IS&s#|wgMuqMr{ZDr4S3jSy2eFpNtmib9~B#dVi<-&yfUw>2lv}J$wtod5XW&?#V2v zuN=2qHF8^?dPz*OwSG#y`?X_`i{sumc3#!H&szO9Iqkl%6b>Ep_b|Ut8vG@ao8j_S z8QeKY7r*Dk(!9#{Q)G4V!?Blo4}KT&90qAo(r>vN#mX)bM{CD)lY-$QmBOd*X*31l z;`}Z+_qEsO%eUIYFZ|@cuoLn66=z!fZLTXnWSOqLQWaQy-Leo!S@T>)IQ(?N-fUJO z9D;I*@X(MApz@ZyL@U43_C!|a;ikkDrO4=czFGMyANv;hD($XwW!0ZB=P#c=7~cIX z5C^Ff(0R#n@6&bd%YsvjG>uZ*F`D1U`OKU%B7&L9mztFq=yG1~y!U!Nq}3>6X?U9? z(o>zyx zGD*F8eVk%#QmgEWdq=F!?#_0Q_%uL$_9bxFQJtq<^^_4{=_+aZq@-25@Q_CTweS4ZMk;W4 zuovMVoFkjB;79=Uxw`+a5kz7OTP+BQGnnbh8m<2Nsh=u6(-$T3?JPK=Nq_PWoT`Q9npc=++R z>wu3HPfLfzk{OsqGaP?kI+G@Q0ZiTmaFlHFU-o6h@!x76;n>erKc-2Cy+ zh_{13GmCuuDX@~D{S5o8SxZRETub>Yvv!ZRp!R@vyS9k-yn!@LlIaYzyNM;oU^>o{%0ZZJrhJ6_u5knwpx_nnIuG z8nwWzJ*$<_9!yemqH2jt$z*YriD1z)ljM?3L-~9k%@xKZ?Ydl|v2oGyX#VJp4bBbq z4O-K*k~~rCq(TMrI-@#Ue94B+gPOGb8^)l|RykI0rykYyx`l2yLdBrT3Ywv@LHWch zcwYbfeq zhi|%Bx>wiLeCzpk=u+p7cXxMdb`x?ln5Y<4sX!OMUbk4Ug8D#`*SUuf#rs+l&r*te zCje#Eg~6r-BF+_D05oHLjco zKY}m%KUsf&;nwzz+(^I3>PQ*Jh8y*qOxHQN-d_I@JuOeLlb|j*Ukze3vvpsjz|=26~Oz;`mgqV>^n?TzFYW3rf;O* zJXJ28VEx4+y6y|`xx4qoXOHfxzHc@TD7WL;o+;68w;a>gV>0f_?)A_dXgNNVt0V^A zZ<6koj;()U9SN%6qL~)+E6Pj<6aezkz~IqYwC#(w_Ev}6_Fpm27JhHUH*lM9e_^6# zs$&AgD#TuieHzOcE2U`}t<72Y(ga)NsSnnd*PqVVjv43Xk!^B;jI2!z_zl=#v@uvo zlav=J2+0`9N}DafaRnE^vw3)?vq=Kt?su}Wy&&mV>5Icx?L5O5?KS=ydig{q`R+jQ zRiz6`m>}N$)3ufDBpmcRR?3qW=Ya#v4t83^EyST@3#PmJ-YwPsDjZ4KPTy!oHH-|7 z#x0p|5q{6jz%gy;bCHqc)L%3GA)NEq_?aGZ?_{m~O6yl(druvCF`1^G)|KYY#d7AL z>mKYGS8TXu_!ilvWtL5Z1@0?Bo~}vMUrFz+uqV6=eACb5PG?RR$6&`t68<6in0xp> zI}ewv-^%1kd~*Dc-;u&BzhbR8@%&Q?L2d`b*I(*Fbb);dlf%`uFRhncyc3rEej%fn z-`lZ=Nwo_;`aG0Tl>AVjT5vbi=!H?B5vK9^l2crBqQ@ukXJB1{{M!W{2AtRV?!)j$ zS#)4lZNZrg7iDkaPlklcf-AZTDwZH==e>55$8JU&6{Aua!uk)K`C~H8lEw#t-Vc6 zimnu_@?LSHuYD_F99^gj|07Vtk@Q}nFh~!eTxZ8EIh*xvS+CE%Z{ii-9_T7px76nf zs<~G^ub4-!7V>0N+ygvBf+Eq;zhtu5X*fu-tnltklu|Eb{L~Uqn_An@_h|KNziXDe zN<#p6u;o*rBF1>tdGH+hySS)jZR^!nYQ0|?zxF>ihtVD9*FULyV&mhUL-?T=we$4V z)2%1~d!_Yhd7^nxbzgB=?q=`R?axtdu_WkL1!o4?_#g z3?pRxoONO}_4*|{0KIO*R$3t3HeGY&Cf&@41;(%eSUE6T6Z!(~IJbXReB#M_Np&dX zlXunjrde8jFc`C2@x7v6(>rb(>f3Cm zG!T56UwA}^*ZO6gQ21al1I~kG+(zz&obG3FXTL zqp^AH9nXWg%>XR1K9E0XexZ5BA3m1?8#=GrzqRCmE&OU=plWdPm{@aIx;0VYxiGiQ zyvoG%@!`wZO=F7JYQV5g(W|uxsR3*vu|t-6eie!AHc4tZD`a2zukyveZ4{4; z)wQ(9Ph$rkAQ0^9;t44&nN}xP(0M(x^rfI+zw`H^)H1rgLmq$9)x-i~p{J|h;0cnn zcl3M-lnelQ{T+uwDL{c-1OXxT`~e^ju&+XZ^38wsP#~B8R!iOF|5q1?yYfv7JwtwV zPahz^tmIwEyEj$n`T6;kd>ow=jGk!xCph^{`KAj5;-w%ZEkSQPhMVL z>aMhuw6p}dhlFn+7-AnF0rtK1AA|hYI8T7S4nD445LZty|KD-#UwT3z$~SNRMfBgl z|ClE*!1aG)0{i~wXOTZp>TiqGJ;}RL|2;MtROxT6f}v{w(8KbHD~L=pG7gn{ckfFp z{R`lKH2rUs{|0LDZ=fyQvc006?)#~7xiRDa=Sh;Fd;w5V)pk(MMu5~|Ht!hxjbG>Ekl5U zLXAS}$s>~h%1tzF#>~&-^W83!=`T*-X^eHHa^57XJ$tP6EPUuOE|VpwDB||P)gYnK zAzkJ-?3Y$F-?3c1G}d{4`ug3@$w?Z*D?A)`q+0WJ8a<61J2nf32O>Ae-13Lll&6r> z7~X!!2b+8JRG0X}DgNip@I{0qo<^ZiH}xf}pl(%ng3$EA`bbQpYeXeYTI@dngsTlu z0h%Dyh}E>-CciDW{eC%kzap+hp3mM13Z+)ijN=6@b;h`-L_Qb!pXmNh;XeHP>HeUn za`0i_+0QBNwbPB#N{>HZwOSuBh*yQ<*M>V*vx2v0xSqZJzf9R9!gZtob+o&*HSO7k z7y46fEr%T|HLnmFD^PR(y=>C6{xV9!X}?d#pduz$`XY-O1?B&^xu`@G8y8Ofj=y!r z@+HvOZ=8E=^h<^v>^R@t_T`5=mwaUB@SU6j|3tDOFcrNY6Q9b=MBC9~6h_v`xj$my zQ-AUF7>8}Zf}f(jsK!4aRHKMky>?y3DY`92d9Z9C?Lwwq$nM+qETT#%@vfk=udQr&Nt=wyK=o`iSGcA%^kmgLWaFPcj*LrR_%3A45c?-TN}{$; z8j%s&AEEz1?X8-#B+2B}KeDy(+Hb{yXW6yJwi+4t|8XWfVrtp74rlTT|JZtInO`ki zhW1Ix|7Aw8Z2Uv0Y7#fk|76Br2p<;yq0|dEP5)6!!_~UzS=Kf!!pMG_e?%^SiDj*i zGG6o_d99Z1Kz#|~SEj0|`Ol2uzIX|;8vcy!pXF_+n(Cs0;y66-);~!@xEd3=*;nl= z{r`}$e-UEMf|4qe2Rk@?p{&{MjUXz=-S&FIt znWhS6;cCuZL|@21Pwn?3ax>F=lZbz&>HlN*Khfp?vHPE7E#m)=t2<~H6>&l`WSTYt?u5juTG=I{MR@(QBC@u&TL`S-m?&6bKDT%vTmoM38N z{Jrp1mH>s-n7+pwayQ5)G7QrzOx6DffZWihPf0Cb%IM`Zo&N%pYzqYZj4BO@8tHt&QSWp;y%Sohca=cT?I<^Bv1m z0DhL;`2Lp9YR)nQNlYXK)!S-j00i&0Yb@AzPipSB8zFF`aSFz~S11Ke8<+nvHn zV3M93dQ=r$(=7;-S0acVR~pmnrpk6i#$~uUv}S^lK?5KiafP`4dzvZwuJK)3VSB$E z73cd1&q=uR)20hd55E%+x6*@Z^L5r(rpmrObuw{#MCDZ5xU=J@2h$s>ZdROzrhb+2MyJ8|SIg@4J4M zKpy<@8+t@*isa)O7t9V{O?A#%f1NS5T?~KgoZ+oemtG5qr6TSX{kdYt-$rsuLz5;i z1@Daw%LevDAC;}9l}3T;o*O70P3TNg?f0`)I_D@vXwXAM-}~>#^G(|L`c=2tHAeQU z&Si{cK1l$XN)821LDe_&<=U#zFjOxSy4WBg6Y2ohN9Yc?QSsA5l6$U|I?Z*O6}Lex z&8fpzu2OZ@@d{Ylxkt2FVYiF6_UR2*O${{VzY)B}s1=}_PKc}$=BQr!aAVHk0!Q^B@{%FCHYF^0B z1}rW2LM6@#ZRd52vXkE@a-?z!GySXp_-fxJy0xIDrC7zy+Ci1t*UT!DL4<8|KulTA zQ>oUYTu|%rsw`;gK72Uvbhm4w%D>CAMAc=+ZyHoTtihE!OGqSAALw*@cky7vf{5#d zTNu^zz_BFBmYpB6ew!92!*MXj%9cItv6E_hC>4m#OVC|#GIjJzXzxOl5z9ke@d zpRl!DN`v}+Z^UC3fvHM5@vCjsfz%C49vzRrWA#{E#(A{!`l9N>EKtq?<5pq8o~p1# zf4`nO^;7!+h~_1fPfjhLcdYxO4Y!JdI~p>jx`8o+(-@vaVW%7OMfRrKNSKjTGFgWH>*G zk6D7eyT?)D5yd;c4T;7+0C!BfxpKFI?1;}Dczw%U6j(jmSC%l(cYdTvR-X5*lsCA2 z`|RB-5CxgR+lYGnxOGEQ=q*pO!WAD}I0-VA-H{4A)(N{x4~e`ydoycnXQAUPn1rot zt5-c+H}D&f$6VXK4?A7rI|)y#X~1_pO?{I{B&#*!mN zqP!Y4&Cm~{taEDn1xdYNKDGlgm0h?&eESB4h9V-v5Gv-&xtHSYdX{Cl=Lx zw?iz+P7yaB7*~XrXL{P=6BS{@6Xs2lbdd46X24Bzw_-uuGk1xu?f0OB2z_c9HkN1X zz#<@B!<3s?@O%WVE-LG(_(g@NN%vnawUH`E-3G8luHb`V&1p7so9EGyDK0B2C!0&P z#^#nlHmAl-+=cKF7FAk&ZPJbjqi$M1wPbzJZ+Te*b zu_S2NUboPR(>9VYTfaY`=x>og2l2Zein@T|T)s>zFv)p)3U}{>`h_1%9!FS9@r&xE zFzH+cU_^T>ffSo!DeDMa@ddp{!~1bov$7Tr5r-Wv>iriLoO9~f*D~sW7(mIjwape$ z9^cjc8hPyTX)i2!qipKN{jKZq^?Y@s2ySRono~&9bA+XN>Ears; zh^vO(zHMNSml0Fi)v^?QQ6Zt}PI5xTKp1HsUcm2E6GA}y!@gsj;i2PmZsqrgCb0?| z!p(aek(#7*j%*2s`TFo4Vm%_S`4^W|=O{P?Blo;8v_CS!>pAJ@u{)%hm-K ze~A2HF>1whC%UWH-lT`^RpyH0o+bQrCCtxRc5{j93q0&W{@66spylN!Bx$yB^yn>f zvlf6vAU&LhAm5e^zTBS*HHiDh9I5{eAZyoGKj%1n9WzZgvV|LrtvNqDA8+MUMbELf zt$y4`RiqR4P;Yl5KfDb?p2eJXfzI29v}A{Y0+J|q+pr7?*r-gF@M_qFE>wq9bWQQd z&9-vCIj??d4{%C#WBxQa4=44dXm}F}!c8gt)MNQNJ#B7LQVMF=&8^?4sXGfXdx4MV z&_vO~bq}Bt_s?xAD!}yFRA``mim4pM$3g-Ib z^_zaAayZeRS${Sx)yy-28FSdIoLrOBAvDce8C}n;n(2sM&FnC|e`nXPza@L#>GyaajY0>;&L`bUqNW|89noTi1^?WFwyp>Qje(&?By%bG`9*5!%p*z6UjrdT)ATSJzm7Marx>}$Ga=VRt}DDtrt zdO|+f&)ldXj}NVUnlqco3|}Z&AFS0p^>a( zQ7VFDA6QcM`n31Bb+BSK2Wes3bz)u_SQC^xPS@mhyg3mq2Hr*dr9H)`82_Px|Kx-vnnBy@>JF_ z?n~m+OSypaF4Gy^MRb{sbrxyxx*}mUM`gbB`)aRe=+Zl(FZUeS%Vv`L)5YD+MJAa1 zD97nw_zYi2HD`s)C?7qnloyYF%~84VI<7mkws&C$Mtc2r3pL@K^$uJO-zI839k>yO z4+bB-aU8DJhc4BM-mS=O&6I{ls%~>vD4&2vrehiq0ixL!Wc?_TD|g|oT;~*}q#6Pz zyUw344IK12t5{l3dp`Z+D}a8=ZkedoLAjPOmK8WtGoY9*P(F`KsHlWw6*f{poy8XR zB`-d#X?P+wRD8^T5bQizV3y|vS=`-DCt8tF6pG0yo`8|!^OEXg9BYFvj z-|C{T-}}LC?QxI5Fv$%pEn0aPNUJc``j*}5w^e#sv9jPTqs~mKA-ryoaSZMa9jos7 z1_EzhD-p*j)0u4Xl8(^p+dqY>-w4qEY2PKu-t&D$b zWT&8>inhC%1n}#-!*|l53R|%n9|I1K89mN6J}F!4B%F@Lad=xspvy3?d{6o`s9EFM zHXV$;ys&WbMXHaXAx?AVd4*H*a*TLv7{cD*q=T~e=R4&7?v=xdcabOFSI-0HrBz&t zqWjHl__0)SmeEvpLYuYUDy23@r^>OFE&Jt}KK(E3FmR?#clXLa9n@tvIlOMj$QW*j zHuX~nrTB0LC4HpIQ_On!{r*5cRh~evVQQS5s?rREJH|6;FtD^4&MD(A%3pwOgTGBF zPR|5bx0E=zO~TDFLDQ+Fzv`zB_nK|EY|JW&+oP|EP2phwbcN7JX5z*Jc`We{lwG?# zLLb=68;0T!>*sF9{5jm`lXRqSYdz?cgH6)Lc$W6fI}ogT?BI%7lXH%RY6VnjIl^8o z9ds9TEn6^6)n0N|rPYz)m$p1AyF3i_;r=MyC)Cpii0j(cDy+5e(h@P8IC7{pHl&Yc z?K8-=Ut*f0JG57#!OR(Pb|W-&{2tCdABpvZ*iD&R@#EAAVQOP0DrBSi*|~zwaR(nM z8$L~fH$#zQBfQd%&sxry6gLfhg5?BszLE9Pr(ln^-_-P{hucv;%dp)z+pshI=M$F| z1#B5>dCOkUaLzm48+sWJ@A)Jko`ez)c27KO`(-5#-Lc*?TgR>XbrRNOlkL^f=}Z|G zm9m8KY}wWLhOKDUkla~m@OPll*dCw!4Fo=s+_~nNVskVYMq-`VlJ@#6Ro8SnJx3a! zJ|$a{hto~dA8w_9SP|Lq$`R_%x8&&bmpgBXTbq}+!eK;L$PYT9Cz5wL8>XqDA*y7% zDH9l%5Yn#a!ixCdk(-}vC0e2Gq8RAbKu_EYP7O_LrNjFoHxX~f|{qYQCZ}oIkR1PN{;?v6&w;hlMjtia7 zqGzOB?+^k{wi-fJ*A-fc8>Pn62e>&^SbvPjdK^xWDr)BPXgtH`uCqq6C8hx$KOAjE z3C+PyK#-$_cbcW;^=YbdawL08SAFgV}ddZ2p9eci*!-P|8TDSlE$AuH{B`V$soi-GIfFAmWpxMhgFL?KE6Z z$d3Ft)r#1*X>jvX$xFO|p!IxjYthizpD|kEzfCYX?yV+)_%ZXdy6pXN=O@L4M@Wp> zM&nZe2ie+)lN{oNCo`CJhO*a>H5qCndFw|8IkoXrL~i-jFAi2ABq8^qK^hP%>5g4G zPwya7R&eUzlUyDsHn33dU31Y~A?zGrz{?VI>}M6FKwj4=Bs@TW0|y&G*t9+I=FBG5 z^}|k_4p8sAcf|^A6(3IF4kCtGD?@lH85n}}wxFp{`=qeZ>N?x=&RCrWhpBIz!)BVu zExVJhs63;S^=>V5roGCui;K3pd`U-sAyLe!tmmF9&a+a8{l7oB%48K0~_ zR6K8O3p?8u()d}4cme%}n6_P&OqaIhV<(^QeG2%XV)t7f9K0O}^wA3<25xJ>t25yt zyv^<%c527F>v!L+Z?sAQ7P7hWoP^J`IT6y!!Uxa39*jA_#vzyU0<0x6U}%;$QhnHE zgL2iiGQt_L?QG7x7$3BEdhQg|t3Q#}bH~3g#IA9^DU6pCjD4NDg&s%F{it84a|%0M z%ZpnaMC=5bZ{;19*EwP-luHq@ShYDNYMsB7Aa(c4XGsY2LhLI(?@Q}ga z+T9_)zh-hMzrbdmCbf`yG2tOO=mike`1(S(CbS#qfAVx0OknJf;lt8Fz|C9rqyYNs zqF<7fY<(;#&8cIxliWhIvrhu*kA02e=xZv-3m!-Scp~w+%7hsi7;;9wKvYlZIg9p|g=>D& zRd*e!@exhQ*JHHK9r7tb5X+?a^h+WtN0hYOEO7JUt(9c)q*?6310Hqw>xDRJR1iMU z4~HaZlJJG(Ulrj!F7=xz-0LO2)KgW0aLvC756sfJN!SLM)L z{^RXtob6#FeU0GYtwFUys$ay{F#|ebp8|G&aX3U(84e=eDx|Gc8+jQ;cIRe0?^%u1 z92z8VV!JUYP{K)d2KzG@LInR*`Qqaazq*h|dVIxOgJfm0@5N}W(1EA%9f6_al(tY? zpL3QcEnCBod0~%Da5{nVHuY4Ah=3P#4L%YUPZ&zZjHBwH+|)9(Oem|Z@b z8(u*bT-mB9wqq{YILV&Qa`|+^+M!prE=vvLIUsqaxy0a^&e_?^(k3)8~Fxidf`IEfAm1 zq4ik+!CQ5^DHcF*68*(UHJhTa2x|8CiQR!cB85-nX1vrxih|-;sklMiYb`6u){DM2 zPN~3~{$gQY%j6J^mhnpzMYg^TI$fag$NBsgZXH(*9%su;3V+GpH1^kV|GKF!r|vf+ z2{IO?cwSyf6m*;r`R%*-K}Z8EEgF{9vpnO~!MNn`)}zrQA?ZUt$=G`Nw0Z7qpuu%# zzCB+09eT3Do2SRDKbbHaEVVIJS30Nhh&dCR?%B#IA|@RuQ5c?>hrc>&Oo`cFwP{N>+wx+=~O7Tyn#5#Ab05b6-KvZ?g!^e`@v_8djPb;!e=i`X;5}n z>Eqrx79ZF$SH4~HXxZ&lNqC-2yko!#7dR^ZUf*2T!)E~3YRZBruLT)71sAD4R37DU|W?fXL7fZCtNsPg;*gGJD77-&wySB>Ap_j^J(8wW{AqpX5S_<)PQSp z@yN~il8T7p^GBs_J>S-4a-9KA3wC_On+mw(oOo%%XsbkQMyfm!=WoRfssB}wN0r(B zjB?QMT8r7-dW%j{%Z1FIg3(=A*sts(uO5%FCZp6Q`#e=!W#9CZtgNIMOPy?)ixO;! zmGx^1MidsH;3;5z%w+CmS!nbsG+<7O6l@`9G*9>RtWB3vKvjRD;6*JcwQwKV&Ag(QOronoc>k5 ztoZJ2&NY)x_u}bzHVdXMNoceEgI4fNyGwc9_#R&PC|6CtUYH``a*^`R_X{Y!o9LH( zQ$7;_|GEG`!>-^30R6$++=<6#_K568s#^qn~vqMn*Z~cZ!cP_w%o>3ik zG*iU{Ic8z~#}dQ3VNM}uDf(C3i_h@0z3g&KZXwpS1Aw!5@t&KEUE@%j4F$vR${u-L zR#cfctZr_WT2J;S1)cnkVc_D^jA%1@8ZkbUOxEP8ehD`9+f?=z+x$zIi%qX%+=K9hIwMFKZ}kn(2@dK@oTig7iX6gL<$7MY?%w#UTb!}Tko+Q6 z#2a1G_NUFUARA1lx)do855|wH^n9Ei18Q%FZUo0m8`N!oxETJ>kQ!+mV# zWw}ScmsrD8Bf1)=L9bR4n$9P?Zw8jja?!OA&8sb!jk&gFvSS4f5;xsrf<#&`Fh865 z%tsVReUuK;aVex)%^4-{7A&Tiz=NlJk?7K(mTv0yI=Hm5>5m~A#mvdJjqo5-TG4q8 z)L`3={Fjh2q2I~ns>dj6wTc?OimoD?{2I5opkAIhe5Pjh3{E3x&cE~VUsrxm9oZ1; z7J10oYADe#<5OHM{J=K1wG3;w$6#^@d7KRIROw4*dGE{G`2;NeX|tCU$lFCn8A}%8 zzwX(keJR3)M|Qu}S@blzjYKXb&s2>1NDob{-{uyWUIl}sLQ;GfmHOkjge@1AchuOeE-^ei`@+;w2O0JSUo%! zP>=j7OP3na}{*_pg`e*y-{ zp);qDS)#fT!U%^*DsA|U?Z6vh9b|u}o^}}KH+}Rl^_))Nq$B1lTi4jl;%&~q+0`dn z!_gH;_&aZD&wkzSpOq`pmC>|daKL=jQNDJBKIp55CUr1PBfskdV9zLSvFx|h`FX&* z?z5?@YnK};-~qq7x=W_Man~Fq-mxk%;aVGQt($UzlLNh^{gu>57{~eNpv!k~Y8WVi z+5L)z>1t&~<+RS_P?o7HuX55Z2`7^GbOF{{i4H{?krTb6suX@7 zfZY)39&{Ni49iyu!^+c~+zeOSqDrF541)XBfA-I=jiNwH;Y?Fz3PUT2zjVg z_I<}_{aK5?oIG~hK3x5H(kMk*t&!@F8FP_z@0tPSz)srE&v2`+Zmibu^iy{kK^r{` zd>wXE86aut?{DI*5JDjN*k<6=46~^(C6n)?Dw8pAh=#A_8(1b6?VycDuH86r8Ju*8 zSpb7%$*QDV+cK-1N&$ibZg=N`TtTj#U+U}CS9f5j{uvMOV!<+)Ed_m%|1d7>w8L|W zDsvUQD~>=iTux)qJ()0Ri)Ga&pRk*{zDwLhB3fc&_0@BG0(+d~Ls!YX;Y$DZr_<{Hvz~lBeGvH#=u4L#agLPhwv9J?1)q; zx=95sB~L;y1WnrpZvnw(LB0-PBCHZ}+D5`xu2QwaSW|vElOwOS>rE3RuynF^vsJ!| zpMydgm-!!Dr5_wRZg~gWv{}>Jr}uEh&X?0ykeLmj({yMuBeB04DZ0~}Px7t(lg@DQ zB*i9s#O4rG+;3H8oNhDUQR%<*`EM|6_$60)QA3=`J1+Ba8=7MF{(i_d9DBnD<*?dG zhsTPYVO}8%2x%41<@%m>>xm&J#E{8lx*+ceRo}2n(i{Q$t=FWQ`RfrLZs$rTtxGW_{A@EpVh~%c|%2)NI1%X0}<+rOxwje?!i{4(*>JG3*@I3GEAW4A`7V}@^X5fjt2 z^HVHu>(#VM-P=`o@3xZj1MDnxfL)Jg7jH6+`4%Bd4lbqeHst=z8)@tR)ezA$>JzmUdoG?9Qg*o3-wos7EW@eov?*^DP{A2WlGeCme@9MM>9Nu~3)YQI%R zxW(L^rm1r0vb2!yp1PIw^XO_{PhK~2EUh2>5V(pf?&hgPcz!_>4xXKmz2McX1i)0= z{eVeF-$zg8%>6TvX?RQVE#j_o!loQ1WWJ&QP^)IUihE$|+nl(2dU@O5Q@SvWBbDLz zG&%X<6k34Q`_Dj72>5S44|;p*E{1!3@JEEY;M4*pYz^kW`7#dBS^&-TU&Zw@E35WY zG@kf|@nD0P?i}DKrzOEB;MzvJt(p)}7c1@L8?8CiTg}QBsurQ7Dn!^RG2OM*u0Stp zb9TSYIZFH?HVyj7uX$tEzJAIFtp|#apLIyIr0O0VRZ@{jfVV4XAIb!c$6G$8h(L*a z3iT9>Lvzj*BDy%)*2?mid0fj^CV8qsLQ812X5n^B$s zsF2I6&06|)C(pRS&3P~<4b!dLp^bv&izi@9`qxG!8gl?-r>I@c$D*`oeSFj*Zlut_ zZO+Kx{-fs)lcg!upKDlIRMMHdNz9$h5*E+YmCKbk6!h6+nv2W2HeouFh_8IF9?2+} z_Z<9n11I;qG*j-`99h=6kzPMDM4v=#oQfWB?6lb%e-RUReL#~{&{Pye#!J?GucNfG zK6mxz#U-X@@LVwN+8)k(2S1)Zdwx6VN~Th4GGM2-p>Fn`f*268@z7p;(np=sC4G6a zKys+qv^durlvG_r2Ycm-=&E;ujE@NQR55OK#|EY=Tp`>J3{6fu5N{6M3%tH$vAPft zWaFF(wbGqUo~g|8)fper?QP0ba-E#2k(Nt%I%qvD)KcL99XAp^5l)Rr3{_tL8hsO4 z5lwS8QbT(1c=)%-NwYU|63DMAo!ez}vbfkuq;{`r-^nR2)9bja*t*-gckOg%(|)r`qRpDHPNzsom-gs z;tz(SI<-tPd#X;{AjDHhFR?Nfrf5{}nIbK{&|gw56OgvuB*?Y>f(4Pfn-%1r_Aj+qg9*|Kd9ZyuE1Ow|#8czAr}04IJs~ zn-G>ij@5cDtpMc_&3o9w0o4=Jv4{+++)(gu@J>JJc%!Nb=q1}kIvXLS-D_oq29OZI zh-fel*?6^J3*2OrGi;Ji+z*aCYYF%cS;O z>IfEY7$w(nnKzi!>9t9w#$<0^I19Oa=S0a-(3J}qF4(|$vUblS#b)A4FT7d2eFRXy zNX|<KWn>eB=cA>|xjA>tK7#%CS~Rc7=gdr0INV{$L+%4Ovt;#A@H8hdKc zp~!z_Y525FzOYwxJqIp&w6v+*pV0&GIa>VEr>&{q4F<4bR6h$7ng ziV8W%DT`B*kvaIbd&fV-DX#UP4!@>SI5>LJJ;g93CAYbbJ?_cuy2lab*rBP1N9FZA z15|a{$bJU~Gx4aPf*Pa$?8jQon%TAP@i6Cg>es#=Bz-5RwbU|}O31+xFLFhiy|c)v zZiv;*#c4zrJ96_A&&iQPN^kCHRNd5@vMz1T%o1}{13d1_;)yP*G8i;AZ3aQ5txWwP zF9aUViQaoDFpr-1Ntg|>K1_1q$>uKvfTp|#YD%_&GA88Hboy*Y#_Ae-N4e)K5(Fei z8ltQT&FGYxEjhsG1SiRNsc}kV;_`0lx1Mo;Z~n1IOK(Xi_Vq3d-HDZSbmp!pn^WC1 z*3l?Dd@i`}y{!c8AHY>q)}irv?zp6eHvCz+ZvQ?SUkkxO%t{ZHk4xGQ;iNgYVdra7 z(}nnElhe*q4D|u3cyL1dPAXg4Oa-d;=V#S~&lu(AWV|9jb%fQ$9zZP-*;Ypgl;{%q zQ6K0G`5Ok9*r$iIl3kIj((-7j&XIn-`^%n;<}W0yZJc5mxgS>bH&kG}GwLrYK${!E z=VKCOU=45IS~X{b@-a-(s~6>4HlXYgAUdr#c5e)UYB27&YQjoBWv1u@{ITQxGnePL zQ=Sm4oledO^nl`IfEc(Gt8b^m7DC$T zU_PsFN}2&Kna&tUZ5(=1vyt=4U|~_!6~2so**VuC%5RK{;P>xLhxmeQ z4t=8tNxdxfv&i6+eGs-B+n!gB1Tm;42#nV(jq8C)|?h412_ z!PWDzVDN{MG^gtE7x`348`eoy1U(w9Q0mO(woMLKzzOk|aWRF{UN013w^8d6v!g@0 z<{!0<4?F)sB71@l^P5eTuTgXOPkP*&#&blaGCzLHG2`%nY&DTqPnNda{7-6og%2nV z&J=R2OXUEb0@ME3U&fDw(E~6-g5B{8=Szv}<9Q-(pK8nO8o1hQNAj9naxSx| z$o1mfAm&N%>Q+}!MH2)z*ecse`~92#BQa_*W;7yB!W>AU-Hz-fr$*smN{>vB>Pur; z36@jaUX)YLB*jkHCRj|h3vF9rB-yxT2RR>mX$;CXj_8e=siJDyvRe7$fcCmjdMG@d zTu-B5+VE-_Gkbqj<+{Q!p&ZO=H^GCQx0AAeM+#iUnKRopMT)opir@Guu4K0!j=x*l zDmL%>xn(!XXz%p}n+HF8vGCcnaZ|7PU|=xR%Xp$%Th2e7XvQV+vJLG@$M;5m+rpik z^U(D5C6=&uDR}wU^8$N%6No-G%oC;S5u0HSOp13^c5)eUwShL z2rbx%dn_t&J$KOdjGV)6*gd#lXqtC=Pvo1>V3%81>vCY9BY>RbcSGF#RyM)oI!Uv) z|CHp>|60x2{jq#omU*`EQt8`!r87S{5107P8y$7X!nXP!3qkq@;rSA~XtS>8sFe;m zLf{THC7Z8}(eysWx4X(m_sMoPk@7|-46;5Z@k*ye=vs_1c>^UFuw5Lz9w|ys5^>O5 zAqVGcwi@PUrOZ#@>3!W5O&g+ZWLJxgTXB8~x~^pF+BY+=yRQ*u@CX|bCf z=E!c?;L=%pg* zXvuoCoNWuOsh}a|*~Rd#nb?Vh7q*TZ{dDuJU%s+d)?jeIv#9e1 zfE*Axy`GDbjl=|ARA8$zKdk6n7;`Rpl?kqq7|Emvf(yTvvD{E@a0K+%lnu`Ka83lm z9i*S7`J-d3?2c-9rczFSwU|_q-PW}yZ>6vc{*gPqts8nYnjPK>d(Sz8w%GyjE7v;* z`|_0OA2a%;}dptNYZVb||i)OA^tNK9g} z%xOU8*eKVbjtb4Ky7J%&d2&XAD}p`UjVb$k#j8&Y&DG`)z>d~~wP_Bn=^qlDzcgpg zB||J@=--DOtl#!!-3dt7l(H)wTY5+Sdp9%F@4=J5i^=IV`jYitz3`;`sr*nC-?K^> z4*6S&q##@eKw|Gp7ItphH2F`M4PvtjTlXmas9{N#|BJo1jBEO91I7hGNtGDu7vJ!+h#GwR+J?e)tkXfKvEpC}*1%tuvg=|wxH^Tj zSg0UI#E>gQEAdQS@zp^d_kGqah_fGP5Q7mk>JY?D3HDm(UC(t1Gc{1XEc`61dI-W` z>@srILVTgE7-hi>lEucYMJ2(?W2(k)8M;XzbEV^KXwz_+tsU+-$4e|t%e+T-!C8+E zGWoj9#TZAPg7eW{C7{dlntASG?s&#k1KP4ogYeI?)F+=dHwW-2_Y0>d>nNr1-t0>< z9?dKsgiP0edY-5GvNG*0+Wc~aA-#*0B?$xOqJaRjMr3LQuHU9=yG%yzr_wc&`=7hj zkQu0cPMj}*OXceo1)jp23hzR!FalX`Ia2N;-rtb%eYbIJ%MXA2;#C2$!F#UW+X}hN z*`4f$a2Hal$oJ>K>Eo5<_;aE}dJ^oU=(@t5o)Dz><{8P+E6ztV+8gFQaVS_LeQNiH z_EKC`BuS73T&sIG&o9nSte$l3j zYw?t#0D1RYw#;iPx=9q~dG+R`8xZ{iyv`r1K@Gi>il?FXf$e3dkDr^&TQ_<|j)cTs zm#ui=?P^-708x#C$LP*AoLt$oz8#CNV;fSbiX3n*F*Goi$OzNtkTRUJ7Y^&?W*qo} z^2_7G&RICARvpur8u8}l{0bki9dOzl@GLpYeaG3TX2h;bfpOwyVeVJ@75@?y{hIQb z_^OLF*j>Gf%vxBbn*YjaxB4kg2Z9pFUa^X^S^gCPN7 zQF|rlyo}?_QQE`y>8qP%P*|{*+u)n(`~)Y^iII4nHFx`c&&vC$iN|gNofpNZ+<&k| zMjq+Dctp*)I!vN5Ov1u_e(N^9o~6MFZMlLAr3tTE7`C@4HZ}XR26i$O6FEsmnmfnr zzb5%tNbra*bhu=PJ}Ht|w3EBuMzzbVu0daCc4gPw##RF@zM&99KDAgbyL5YcX?7kvk!|SYL@~kC5$@i2x9lzeCfFCJx+g7;uYoJk*dh8oG z=SB71oyRzuUtmed≶N3LDTy$bOsTN?VW@5I?Uft+dqUhs!^=PFi-Gj?X7A^(Qilw&Bv5jzO#M? zMAxIdqGVps3I!f`mK6jp6kVYFc#ia(L6I@biISYs#*{I|ZkWy^_8xC|)A zAFIy7E*^HBf^@S-)Gp*Fn5@LbX7TzWCzfV1@{u>9Cut8-*6@@_c`r|_iEek)ncq~j z$*y!C%}W_s2`&0|vDY-;SZnG6YS#MTJH&;i=)=ZS!B4Zwov&t}HB$AHSIughO^^=6 zz2UfZ1`g_4S4@h0S_P_hPaaIN3-@g#TNPAFGwOFY+J_FwYsG z*j5+*szqHJXb4SswsE+7+;q`JqXpH5ITLNza_F`0CX4E9oAdaQCPQwo1TUZ71P`B7 z2OPLh94CWZP+O#QO}pfKo@a%WH=~nrT(n10_;S5gPPkXAHfZ}4-2w)pA8jm>OY(+o2)7fcm`tD0cH3|!qzBvsxL4s zGcrZnqrZ^1TAbx>(`W%l|1SD6yi+NXLo0eK!>4Xv)d@Sb7L~!d-uURo+{%sYv*Cb-2z2Pn=UT}HKTl+(b>WlYP3k}T05}?V%&=PZ=}B0P14#ASGWx*@OK~L zk4$jup6&cFJ`j(+ZNy!j*yWrtTD9@kFzZZpi1=cmM%H$QqVckL=i69zHa3JKsa^hy2%Pn3}xGcFjT#<+Guy5*+*K+pPxocR=tFVj&aMYluO}ITZ zdB}Ga^Ncw`c<25H!Ka^N)aR9pv$=&&7uw}LtkHcRg3R%i`Kb^m!?MLQh>K52yji*v z7!&I>NOtd~R}??c36x!a8@RM2em+z#P`{4RNv7<=0Z#EA1NYuFFaE0j>(Op`&Za)e zVM*WKXT_S~)bB-nsm(E|J1xEt-u9Z(PNV(SM_nE6gEY|w+v=pZF%q!U#Po~D8H9SV zZIm}&@qYH~cBDY@iHN_l9JQbiAMkwf!+oVUElWq7NaGzYu!syM9+oo1r6>Ph775P5 zoANx+5NL+x+`skt(4Dava5R^$Sd!VXtw?fHf1ByP&ky@Zm05LF_SDQND)T|E(83um zk)`P1W>fPh%tJjV3(LOh4q}9L{suJq;i*W80&oI+b-K4L;nhXTx{ZkKIBBEGs63*z z+i(@Ex@y9N%Ld4aRkxpAnmg$`Sn6cODJ}pTK!PT!wU4vsq3CnW!aDuV_MaiWj;A+f z+A)Wk-^#^JZY{g;PWsg|Xsxw*n2YfqEPbXV3Rg1svzFRU=c2lUb}M@RrJyV7g>LnU zAfwSjhxZ38oXd5?J5U=Bm#!wBBaTe*V^ogWGEj>_cVTb(HEkw=^2Y& zr_Y5$q#~`D8+I{A^dg^>)WAxWr*#wJLYhdOzc=gEu4jWYwLR%t*=1pB_OQxFhiKVs zO%Ay-#+{dRaV_1h^R6tu6I9M15^u&;(bJ#Q|Lvmq_7zQmwFz!bYrBNeX-HOt|&Usa!3NxlvBR5KF-X%P|p0Az$u=zRu+JJ+mjn4r87k7|E zt-EwzQGYXdW2FnNIE(X(eX*fHGcuQ@`Xj(O&(|Hl`sH$Fnp8uUtAs zlc#z08p2qoVfT7PU+~#CQTnq6KIwU$E!yuKpPy6Cky2`GnS}UmYo+>{??cy>OPQh^ zQqc;jkW1TdvTmav_Ih5yPh+r*h2rwXx^6GG4a}SoAFi9Yeu(lSza?0{xW=6o!d^5F@C2Z!vYv63@D?KmEubC1VLLu8E0f zCE>53F3e09irkbpX|0;g+x&t-+RXM$>AJ)8Na!Xw;KE3Wx`Dorc963Bu>y64orfgR z4_~3qZz@&#J=5yQsIj-j9Iu`)#+~I+mr*4Qa7a65*M-|SLXUBBaBImH#4dzn_IYe4(eSi>n%jKz$_S`nD2f|;9XxF9Mn zU>O^v-p)VFY?}iw&Ar(gYGc`}@1i`Zkb9hngy}Z4IDdMGO0eP^I>nrUR7_)lL~rkw zxgaCl*(gg%LnSBw1?4mt$N|4c^4XV-aUs9>_P2BFnV$;_z~#jXkC6)9pYjdmjC4@? zT)FCc44ewlbD8$u4(w__0H$g%C(<*$qX4%S6IdNBl+Rbwo8%`sTrj_7zlY zcGX5&+vs6=73HFLPacXO4Z(G1mt4pEUNjEv;9xK2&Mazur@2|*S(5H0evAO(T7WT( zd&*XYfrtd6&vA0*OcQDtBYzO`Bp?usq5Hw3JNso$h8TeueLF5E%%>;R_<^E6M}FTUHUulLqJrh8pP$!-^Bj;DG>rDtaMA}{(ZU* zxkwJ6?9ePr3>d2LH}twG04I@8GXDhm8;?XdD36Faj<<{aeZUa2 zjcjs`e?|VU$p3Rr$-h?qua*C&CxZW9bnG0*AHQuXbDcaY=P+IS?NmY^xEW*nfPJS1 zln~T3L`@sQ3dILKJ9~^A4D=j7lBV~k2-{vs%okq1IR5xA-Zmn*hF(?He02-!Ahz7u zW;^ILRtQF@^?xLrb&%8cue`IZo_sSmCBdu5@ZP$SnBzC7o>e`Afw8j;+O8L27bZFO zxVUnq$j^N#vlyi;KKQLj=aiw5)iAPr&S`TwUy*&I9Ne4D{F;oI5(V3OTDhdofAWgp zqv1uA4--e<1`Alo_0I>1ZI{o@l#Uw}oR8#2t&byYiBxeeJ(=~Dttn;r2cJ79OZQU zq*k3jY!sGLy^Eb)D_U|E-Cfi|_di$Svo9WSH4RpFN4v0tkpjcGwWM%u5CLZ1NnCpA zUu~Td=U&ENwf0@WXZfz4ZEpQ{?=YF2E3p?wq{_eNDQ!bSd%d=)#rxx2TFqMo@{&^eLt|(@)}yaEOkzt2?bXO?yZrh>(Fn4< zpOQDj*x%{oTZ3|U(bhx$OZ>(CcEv*`ummp!9%hlO57+USTQxfC{tfeW@|&QoQ#9bM zrw2|g8;FfXi6Kc@dUtkK?FQB;aG*oDq4!BpdRJO*X@l9J%Dj=qqr$ElW{I>h^KV*yKt`r;nagF`f^)oodeePh*1T*JF3158;|#>!uZe0 z^rZLRJswsFW(~dP;)($ioiN`m%#ASJ)y=B@&(o5k6F}rMHuz)UZ~8iQd=1=8#3o+$ zo1R{;AB!yy`Fxq5T;d92C!}y|^tR=8#kurhQeuUOc$RT{1Ah#r%x@f9yl!l9 zB1t`aW#U40C_6B2=ls;&;Z<&H4gF*=4OGveItg3cW|*94d2ZKdM7ew;#>TEaQQogx z-#$Y{+5KHj3fGybukV%m3MSdXarKUR63BFosn$PnHWs7oqQ@VY7Od?0DggDe3Dz`n zZGMjnS7gs$+P#;x;uJ}T^{J76Rf&+Eacw^p)DLF04cxu>b}qZ{6}$S zk+ZenvhK@@23U%@%sJctO-D<~Z7VS}VkWhoKHsW*GGT4;sSsMZMayY9;{c5Jl5)|P zuSzJK)a+dsq-2oHR?T*5$Y+PD?H<3%PJyDYPkB+)y@uFn@{PEw1cBVyR2fiHYD1;MyLf2l-)P-7<|Nt z8VQ#(DWA>DXPvarPLTC2vn#Gc;-;}hxtIN19QB7Pvt<08u#lzYPTPEC@h81Dn;2Yw zV{#c9v%AoKaACv32)Nc6nJ?>qbFE$c^R41TeQZ8+u*j5~MZT+pE#VI&y_YS^u{gBv zp&vT1IQv-pyj(eCj$l;|r~a$)hu0$k;_f-3J^FQn-b;4n;|~5yn+ER9SCex^G;V+y zXBm~Nn4z{=Kdh)$ISSK=rm(kdo6fFm(A0Bds7~1FJUs6+=O4jeQnMw?Q3i$_p_-m0 z%_In1+xQfz+D}uxQ^>kmp7b3X>kEg%@>hk%|9b!+K8656!*~L1zHpmluk6~l2PIXr zODj2%suD;sWwSd|_3{tM&~vIgNmSBd%V_J{iQSrdtw^a=TRqP=kG^j!kwRA9-hVk> z{ss9XO0P%L$k)whMQ*onbFNa#jkltxlW#6c-@P=HRZ3HJWN|5BcR)pUVXccFyAZ5g z2J*t|Qm;Dymhv_kaX3eAoikmER1MUfGpw{lnQ!(F*o)zu%_mY4P++{1g-wa^k4sei z+V=TnJvMpo!pW$Mj@VZq*T-VpR5c6C5@g-Fbv_{|T#x7WS_=O)?InK)QubsC&%e%! zJwSXFUpM$k5(8tt3QhdZGRP28YRJ{Rye)@p>@1Sp80ZH%G z8PWcbzXW-pZg3v_e4Zx$&ouh?gU}-&UH_Np|17WkE9u_~^7GTb>itj4{nw`d!IOU- z**|?W^k47&-+Rx$Th@PbRTPK07f8>)7iZDsFP1?gFC zn=fTX^#7`%4TFeSOh2lfCEcWZn&SlatwGs1ODqvbQ1~b12Rk(98ptbaN zA8&FmZ|ejDwz5ME>K*jboCx&|X<0$YXnD%ME~AkHaHic$)IilBFZP;|FW*O#sV$g8J&OtriXo>uM$V}<^B zdWG`LLB!SSA5teL_`KVT2K~(a_qY?I0dk3a9N+k#Ohph67rAM49@dEf7xOJ76CmYL zn4hTNdxgrq@8jtH0KQKCu*SHfar`ioncE=>eD@ge#8L&1G%EW)UfSVCA<}6#ef=?A z#Sh%dJIa5$#=L)w4u7-pIMPAVLS=3!C*t{_&O! z_(fuqdq|5?JDE6(i|Q{kkcqWFpKd(`WGK1uEdLK7a%jOFgFW&|A&;(T!m^?$U+zhy zT?pS}P3E;E|AVs7QxRXh^*0-Ql|Ldac$%TyI*8V;{5UH7Bj5}SNa#`72axQ)yNmD; zyv);Mt?nU5tpY8lX@2VhG^IMNmEec(*G@U{7$L@nU6 z%W7K5gMUIjLSCGio`-yaIgtk`^_7Wgd3<9}lP83L3b87*!T*HH2RJsvSDZ< zQO*Z0jsHa1;t1dvpG>pYAC`U#TmY!BlW{!ThWJA*>hXQw3fZLkKdg0BAmT7C6L^E< z3?%Z-C9!m5H2Bc96vs;twlL^&+a?c=>i=|h1Qkz$k}taz0qJQi;@t8e4=BU;03HL? zepv2CaL?)A4!d1|*yK_?l4;_YwGhBlAnL@+yqbdQD?jCTH6_a9(|^nf=;9=D+Rqd);2p?nCe zr^-%dnMMa8e_s(V2Dmk8vTA}osN1EuO@4jfZ*Ut;&9?Z2`sBq=Vr#>+8Q^_IslA_N zP71qcR4z2c7VUUX_ur=y0HqU^;&>=5W!hQg#2*Gwa6l&{U-~&*rC}=WN9|Dp^Q_?I zv!%URv7!;U180LxA zbC`yS4|&I8Tt}$wV;R1$ygWWzvB_4fD(AEOrdqGq9>ZhTpK8e2j!G?yjl|o@4!e&B zv4D0a|0`y!j!7IY~6LLCh-4ZPqxj(IXTyFj%hi}#vd?VK(r#D^5& z@BcY<7z|Cn?YYq@PT^mK5gQI_@jo_Ou@pA{RI5U|V&wH%#F4~05(E{kjI=b?N|l-$ z;kd=lGpa`C3ZVc%p#g*yMYK7T^}?mW ztpV>G4YPfLNuPUcW?jqfhIwRilLzI8g@f|2P${j>-+a2sZQQ@n1pypVC{w|XvI1d= zsC|2y?JL9Lv9(#}Gqf~2A3Lk@Nu=IKYUyJZLKKELz` z6g1j-G(XBMC%H@s*zNTF;bM0e&awCp z#NchK4exusACQ9zhG(Gx&`1*edrTbHjwn9z z0mGaecO+rOil#V9NOt*7`EXMbV~~vdkPM&fRA%}S*43)jAl|B-mHCeFdlJ$$b(yLNN87@xdfb-RlTr(IezL7=>+&tDx zCzI`_0fvbEs9w7&7IqZAMy&Xe6JbbUIW%V|)SvtA#s zCLCtpX`jp#74xtZ1|sk5!u8Hp`%AcX)OMGAFIX*6_GtI|MwCs(=)dv2r(n6U_9|l> z`_4NCclV*emD%^F!cG|A7KG6bnYwQNedUE?=1YC@h)S}(9(G3zfpTUWKfdr&D!~N-Y3w_BSx<~`Izq9T;=kD;B#j7 zwThgD`{NAWe2?hCf1nG*%=RV2x5!EtlII?A4tkq;ZWEC`6lYy-P&rw7A8C_aF3VEr zdA-lKSnJVQMfPr#n2bc3sM~f-PuGg7;o1}T*U>!_A#>T~v);oW$=J!WEe({n5e`yZ z*soRoU{G$|(9FWrd@>-Q9CB_6we{xb<8|FgUQtNH@Ju6IRiuc35_V^huAlX=s5J zc8(Nr**!PW<(fqKkzfAm#ieiDO0w*JZgp+_Zm8k(pqFn|%rib=81-3atLCj$%x75K zfSakIBd$Ay04~JD2h&p6Ysj6k@Q~Xz3_l3ne*{I zq))*ll8X5+F#CAJ*@4}OS9*2+25uT98&fv5;Eh1X*laSp*JHMER6(rX*I=<_a&)de zl|hB!d(Ixuaay8j7aVkQ;WqB`vcQe5gH1|!?Ri|IF|?z?du0n}omsh^A_rPtK6ai+ zq|~D)PhgK2E9V7kky6+bIM04t>@wnYAc(sU!nK zrVcM83ic?osQ==C<4EZ6W6p3IRR;s7?b;&u={`ey6=|)gL}eNI}HyQ1G%%8-Z`!_?v&>HV%UND57zZ7?rk7I*AkUDmzx_$tE6`okSb>gu4$ylw59vD{XT( z%>Ji~bfqANltV2%W2hmaQ*lHsXgvF0*W8gGZ<{UQtXQAttj+e@j?Ebw^m)Xp%P6=p zyCVKj)qhMFyCSSAw=&vuXSJ|j7ud$;T(w+<-$&-!gTNEYx{sY4hLf=$R?@ZciC@VT zu5(A*cd8kdI-7fLGt*E`?=E;_p{_vlDOU$v9M*30p4pDdzIS$}a6EqaLvhvnzq7ff zU_~H8Ha?lRiwd$?lSctU2BtGY8to@3?6()f>(GM>S(@{ns(NP(kAZey5fyj5<2}!* zI7!y`hVrm`!htWj3l-dVCiA&!!AV;7@5}2-g6m!HQs`mXf3bhfk`$#5_tmJ(wq?ONkv zcQ_o;SUGAryUZV@mw7Ydgkjrq70*VJg4n#z4LDDS*BNB4KoS;Tq!b8h7EBJ5O0vCS zx=H~_?@+bAzOPdpj6|J1L9m-dxoU_y=Ke^6#T+}&F~N6R3-#N>-QbEy&TO#6 z1Ym^!J|{Oz$)uVJoPHtVD>G|%XWTImw->uy1?m#~Tmz)hKjFfkQ`p_PU_I_=Z=!UvzpH3QQ)K*I%RbyMJ@L?Ri^+|mo%dP!1P=MF=W~$ ztVa^W8^j|Wd{|`yV-}CmxsIi#^8H$zi1;jnkey6*ShFrucXUGfbAsou`2ZGIBgLxj zGc%ExSmD$d`uZ!Pre1X`!a5wZMDj4Ur=Ry&fdA^2GyK?>=%RR4Me_%yNsy%r8k7X= zPiR2fCDRhOL5%h@e7gg!*@#>BxP6hqb<83C3IQ9Tkbp#&6QY~@57B*yfVwL;Bck}p zo$jQd!Eh5bguo4vvXrC$3&~MptA6i{WT-$j^7El41-G&Bv@?6KGXQLD$<5#D_u(;= z5|Y^#?H?3f6}J4xroQ$IKs&i5RRJl$<#}WO8YAOZnwB0!IQF=v--dVRFB@ z$UsNudK`VdXOgc+YjRJ3bRc5o&@BQn$4_5<>jDucO)yqS-Z{7G1~=jB2SCNhI`a5` z3;8L7fZ*-~|Lw0&i{5T^7pcJ3H?L9SiA_8S9=2G{*F5@5C?dBwND;}IjOcY# zV&_Y2aG@qBuz=NHs{rvK!d3}u5NNq`2l$Z%H~pS|2x7#=V~@w}{2UWA#~0#$QVVt_ zYLdL+Shc`s%V_XTJBLB6_A)8^|beo z74ZPAv@nK)~0(~V_|#ZDkCL9U{1#q zteQa4hZ1Ov0&A>^>+l*xIP|TX{M&_SsiTCSnkdO3`AlqoJPQYkqO8a60CQ%~E>0q7 z5ZoR}qxkVS0a=20c?68jhG$$8-5{_=j2c0iphxLkxhPhDKV!{doY{n${$$%;x|BzH zD(r(B=bkhhxH%?R5(+tgYv(zA4&{U}vOHPX@IWL@1a)*~r{t3c`aNAf4j8EYn)Ts6 zqiZ}#z81IvYi}&yJlAa0j^ZDYBoK4X5DIDwxtzaE>O?Tq{mPrvPSaCHSPrWznTcMa zOoD`;;kAHEkLacLp&o!QkZW@~OCvj3&!Wj*(Y#jq>_Z@i!yiz;lYHaMzAg&!5-ZRn zDL3CeVA7C!Quh;2O+22ZtRedlp-#kBzSlaFgN0{PM!RTxCTA2|#P&eS0g!&LZv0N~ zglvcvjv|@1t;xjl+(nQ6#4#7|l51x;elztc!?j^u>8FLKf(zqQU8h}$@Z-&tGGF7|+BQ_$%VI8*Tw~DHb)SyXSiG zn(hRjjSyA|4@)9bjucRA)V-0&vYmJwPfa@?;6VO{p3T0Ku$Wqt&Q92$?!XYZ1MiiTo_$^Ey2cSksqfVD^3-5TY=H3f7rTQH#E&^n zQM}y$0+*gBNg{pd8A`T51-7!OPK9S|;ue6qxcZD~pIai=IGDrUTQJv0pGk8|z$#6H z=|Xc|ukd80;mNY7P8I!=tdH8ZgK=K;*g}bsO1%}5JjOFeHMO#LyIvFi0SqU@Fd_BA z{y$nt$_^s?q(5rQKv1MlraHoPtN?4-e|4;4xbmOue9#ewctob6SA*6AlvlCg#Iw|>7pVaqsJOb!Ahj#rwcj+dM=NOt6 zUlWd>Uj60UYi0mduxi?_Um5^SJVi{L9q8(+C7i!)k3b5rsO~weFd!_J3k+BHA2fy# zslQI?Or7(2#rMHs$oeFn*cVE`sbuZkV&3Bg%yz-FLSSfrKoiN%3MF#=1Pi1=z}dRT}~dAMxoD&bc)#sSXc!{(ay&6=A2ntTdxmUza9UTbSs zr{wO&`__Z)cL^&B9RhykL9|3U9jp_uHb8~NbkI-jN?8Baoo-A-rjnjwdux z8z|0xaDnG$`*Au)$s=zAo>d|b61aCO4lE|2clP%NXGvw2!lxITi{ifOjzF&W3fhSU z_UuK#alk1`KHv%aWicBIlJhvK~?wPc;sJ02541Ig3BySvW{^5vsVhY-Y>hL>9J@%tE_m_&bZffB_2&kec#h>&ykoM1arBq@+0fG2EmlO!4S5(t(2As2g*^4-FCyqp&SEn^O8doek|`>+z3Xa?0j z5FkqO{r`mZb$JwT-TJ{Dp-?=kJ(D+iW(oFJGYW!H>rl{vEi4dK6FeFV#yNCPs0zJgy8(IIJ#WL?=rc614}K4uC$@K68nn zWds@Th03ji4NO_Bv;LU`CA|kUdEJ{Q^cT1QD6m4*n|WhO_zVQGlx+m5*EHnXbG92t z=v;A0X8Zn)i1;!&d2S%orca=DKG3aX4DW8CZ@{{jyckq|t=n!!)cHJiU~{_|13_%- zq4u_#n_!2EQ)%1_oCuF9fUPJjpf}L~!A(77t4tskgc^uPi#+?Gb7C{G)jRyQcgoNW z6ABe1Qes)Yy}(2e+zEZ)K1!lx{?LoF5%-Z#F1u6H2al~5hKI&*gU!N}gYAKn_@1>H zI|7gS&EjXWink%5q=4WrU#^)w*get|&wI%!g>(-#h!-U*5HkBtllfQS`yi-KT}1hO>g>0GlEzCW@e(T}iXVsY4ulVt&6zL!wHL%xKXuhgn=CzN;Ix?P|6?@zaEDNu44#BkF3uDSaETavVWcM{c$kLb{A8 zA65_XC5QuR3Sfp}y{Y$;n{zk7X8FRlvZXa%oIYrViR*Z4w#S3QJ&!ud2VkGP4j-DIPTl|Y-I=|(_iw;a9iG7Mbw}U4Ffb*WuIMRo5hb_> zN}$Qd71E9Oxx+>Ij;i@n5fvoX^U!GLm@8Yo)wI(_8S@^ZLiu3tvEbC(*GRevJt(wW z{3UhD5uuBo`=)RnI5hPd&wYm$P$LBBGwGV?Mc^( zh>`Ev&Kd4yNu3}UO8K~O{1?9tz=5gMG6Pc)rZ~YW9T~CuKh)jcN0RrkYil}c>`vO< z7(QbJl54f92*3T-Bmx5E>urCRBY?LpNWObduU(S#c@){vroi!^Ho_;pN{xMIY4`6 zG*lu{4G}tF5lYBbSOD`SIe7jE8IN7#oV$6!U3Qc|;?lBqmKwrqng?H0ucO!kFPWva z$;fSRl!DYz2x9#>W)s{_X#d^V}f~vI(dGtLnD<*>iZPegm+)fv>*bJ*Y ztPY~@vhSOo5`tJWM%X~t^Ux_<>oyy4B4TrT&xtodwUXoU?DiWwt9AUvgG@V~l`FOR z@hcfQ^UD5=MQC^Q^=@B$ucO*vq0sS1@eb$Ks{gJBD9Q8deH;}q+1T54Ns~Lo!LUUqaU&h?5dT3DU!9kh6{`H zVWzKK6&}!lYP|IF{Cq)*jA)!|?E+%Ni|&46r_L|2feiq(cwsb;pT1V(2xTggt_`^( zh8OTGF{9t&_naB_HVU8FTtx}cLGJ(2I5{0vT>B+bRfgZTV4-1Hn9=B1kG@N*V|Q^i z{U9#PBM`i!ejO`xG4$KL)vu;?FNd})BrFy>Op3L(mCHlqA0R=GLwqKG$fg#pXEMpF z_a+t14s^sn2b>Q1rP?*^#HbGUo?9$29bv-cW^KYIR)}bH%b~WhHkCvz)|FTi_;mOPcBAD5- z&Dor}oos3;*-g}N2(ABz?M2WopcWtOW0KkU-FD_VeGli7K+C?Lsar*l=kSF++Av|Ynv+<%&<1dm?&-YQd#?Uot<3u-gkPb&7N&|V*ad5hVTAH zVtVA7{5U@I7lkxB-y*Ym7Xa*e#yyk^n)rzrv^-P^ZIE#3Mf7}q+pde}yjHb!`21GO z`chr)oNt@8sL{QfyvHiXVjLH)GCph-!-%+&A%-@>cMmIt%Lv#)JnSVZ8Jz7)J|03` z^johvXcAtDqj8X(X~!QOuns@%>&6>^0WY+uRn+6W(yIeQ(4}-H_zbQO(=!?#3zM7e zyPhO3McTa#@J?n;*V)H=1nfg$z2%dNErAFHH?eKoc0XWK>faRQ%Fb5IZF4_F+gHo5 z_|>`A{H8>VvG5d`%HWT3W~mxy%Xy4BNTM6&%%S){UpH2iHmH4x0bbSLXo}f zj!=n-xxx0T=Nk-0o8A($YpX%~ooMS5L&0dkUoy7&GRK7!K`Uh?e?Bnce0^9|*95kT zHJCGG`d!J{JF;hCjmb*Ul}F4^T>4MU;?9<;e5q%o)7UOiUHq`-NUm~+4fO$6}%xjA4$l& zZW`B7i?fto9oj*5XQpQ)u+3I&X60au(1qDGY=S$H_PvH4z7p08&vQ2IvniFrWO*tF zO6C5m2c4YNNZn)LQ91wZ0qsMrf~@2nWHDQ_&_PSg7K zNz&UjJ1-Do)NK&+@Qa#i2(U5V;(@C@K)N7h1oa&|xY1ft{w=YzRQ@Z^tM&P>>F3_P zsy*NE;L@t=B8gLb?bTuYg>mfND^l1JQy=A4&M;}7j1p{-WB+;XAbg7&d%S9))#1@< zfxs=-;q~!HFjSedh&U^;XB0Jt&@0I!0eRF6T4(j$?VpXy@Wg>1vfpI)5=G%zq{Xw_|l> zToRVR04~wDIM#)V2Le(c(v@!YD^zV#+!>X)ek?X@x8{k-I<14fvW;A}%K&)etV~A5 zbXLxiij;d%utP)dkmr2u{HR;}F@fD$Igg@$_HHQ|n*x;u4y#KPA9SU7w_-BVf~?-F zhD9XRycMrLJFu0+{%GkH0^@SU9CwR5!CSXvW>GM(PA^Gz6%Ct1>MShnz8M&rv7x00*B*mwd3r0I>n7X)?URtTbp6a+MT=Ih*0ox)}@-WO} z;K<;;wIjn?)jP_44s(^YG;M>Y?yYVx*QUrVMdLQ2Qxd$LAKO?p=w&A^f!8?EW*f~1 zf;Z^VJ`6{ADR_d=pze83e>}{0=1UHy2<=K!?fm2O{DwHov7X(Lm45g{kDdd=Z0Y#m z{Cmx78-hmbH~MVaWa`u$e3f9E|2Vyn=m^RH4d`Y>8mgm8+w zW_F@*)ycZgk-D5xyI}V<0Rw?)3CGR8D#?mk@!T7jW!CZ&j`Q9Ks!h{K|K5qjo)z@E z?%8-Dm|NLuu0p31G^%6VGr;-HUkRov$cbfb6k~Kblug9bT&y|g@Zqu;iGBHY$}oQF zDenP{-)A=&#mrtXpzD}@=}O~pR*yGA)$6l*zxuQ|0$mZzNeNMuiJaMwgxfudh}1_^ z;kOT%DY{WgDRrFXA=If&kAuzQ+tuJiM`;hRV7IpOLi6M4o>ClF-*M z2kFwQOvSFB#}7e`?KZJOn?_nWle&%x%(P7RGCm;Y-~_3zvFE-6fg0b!OTJKYgb86! z^7(Xxz)iC|FdgqJgt|teB?tvOYCrF1cOMD?sy!IJ$%d2NI+L^NUijqf#3djZk{E-1 zW|4yge(d?f&5M-pmV|p+nexnMPT`O89DqyX6I^Sa;9s#_q7(7?YZqlooD7Vf9AfU) zd18P2Iv!-xe;3_ixCvphia~gBi7ZS%r85+YBFQsJQA3zdv#y+i8gs!NhPI$37VF!Y zyKJ@&5QkVd^s2+4U4FFtir+!7EcJ}5^3TE&fDfup(cwwApE7{@m;xK4O{OcTxY0i* z8gHzg&vzVY4JV7koSiFKDKqJqjSZ9=@~)gMmUhSi)PF=|?=_b3dXYq7cJK`64~6^! zk6dDui${_cK|!)-m! zfFlFi(O#7AR?J3@QaPoP;;Th3)HuRMrDtCJIH-aBau$68B^jaZD}LOyr`VxzzlxJ+ z3aAwR`BCm^#g&5`K%NgE zdg}YR(ISYF#->vXzLSD{&ht!CMDVIR2{Fkb`V(fKy*-*0+}wjjmcS(D=xxJtO`}ex z{#gakKp`E|uzfb~v|WB%G|DN?yd%EFZ1QcJ;$UPYnb)gJNn9Zld>hg%pD;yrR0CVX z4i8IroeatYB+`}!s|U>2YVq|)cpS-?eUAI<@{ja&=j2lgAudxPPZ>EUGCaxB{KywB zubw)XHS{qQybkF|pLuL!_G1Yk(K?snoe}r5pXqWw`736XuSa1w^z*kqI${?WwinLC z@D146k36W%7u?lp>KE@m>m$%Wf6?ZvtUT40k&eGCm94YD)s)2ir=0=nj?p{YN*oIn z!NpEHdauKI7*=mPM3T^rkmInuhD zda7>VILUR9O+#>de<%CuG%2ERLLt<$q^5-UmF;5@19whSjjhVGG!2C0WVKJAWw40aDCwWSWsUBJEsauTjwR)d+ zwM$7Pd?*g92)grZ!eJ7j#UfJ$A7mvq^Q1JHFTC90=dHWed%9o7pd@<6XGKfnVB3t> z2^pV4JD%~;XM=)^_gGfjVAks0b3qi4k+vEgqmukCgK`HWStiV!c~o1 zjCLw~a~NO-sH(&sSH-Z8dy~0q* zUsPQR$KEV9X{#^UwT%x)Y!o9~JX=P_gw`_tY3{XR>IJc;YhyT?9nfHcTai>@HV!-YMJRRZB)Ks49jZJ#VS5YK zo13~2?5LX{OzoC!DpmLd98guDfTlPc zJnzyM&1Plp?#0|bYz?cQy#?HRr8e*NCA8VJ#Ygcci-m83v!n=4r-#lnyKn99x4bor z)yVpmud|(5Nb>oN&=&}zXPi%+7HMw)8gJ8c6P%RzqeZXF{GKp9Ynmmo=4@B3c~1R2 z?ZoMNg-^RQ2wA4_X01XTrmZhTzFbWg6tbPy*?wh11m7VEv7c>AO~?#9 z+3jU9eOJMMuEc_$m5peJ_YwaLR)ntIUr&^i+V=^vbjr|7oak9u!5~ zrkzk=KT_DjZq#@|@Bc9N)p1ciQNN0$2#5+8lp;uqN=YvwA>Bx;h;(-gh@hlM2)J~& zq=0~+fYdI%bVw|{)Uv>`aA#3}@4fF|_a8qiyU)x#bIzRn&N*|pT6$~_wzjm`JV~D5 z-Xwdz{W`f`^kc_nMz=kY?&SD0rmCL9?|Fo-5xiOmZ8C*4<3xxDl#`K4bPcntXcB>? zjB;J+X*3Dmn>T8ISiUo7vak1bYmvm8mLa#0cg$ClMc8|VA0s1W?u>2R=Am=ZX(Sf_ z9CVg2=RD>;eb1*yJx_a<@0RdII#kdNX|ZJzDz@|(_y1^$#~Fz$G+i(WdVx`rGXKWj z)nr#al(j(fY89C=Ldyk(oDgvAulZ9MT*|A4h#+;1yuEX8E?2?G&q;h*e&l|RR(G^# zZVBcQmMRB~hz3dfWHixdTZ)raV3xRrc^F>{s8bCoRFcbMKTBXr^NDlf2J7aUP;a+j zrP56m-j7mZZ<0q-dQP_Pl9yW2UB_mrbansaEp=S+`UXA9g;U{$8rJd1gh6^zWlrqW zt_UEiI|twO$x-7!zG?fS?tbx_2&ncHjB?$afUeEjOyXWr45cvH{jO4N+;%&o5efA`)5R<=} z!-m$@nwB|Um1;g~Z9f4hR4^Y*LZfgSq~E&6g{%Z4AW7AXLiqzr(UwVTJm8l~T=%qR zgeO%vCc?(G1O`8s_%;y0*Y@&p1O-J3_oD{5!wX2Dqs}YAHB%F0A`#Ubi{I2aft{8F zPywR26wB?zwDlk`ItfjMF9)GU@3p3^VatY6jhO<%vjlOpOEexi5MIHe$bdx`hT-)O z+;5Tx@)u5w1(}O}GqG2oRYmOUW(&$D?NQ{*?^l;y|3H?QsRHmIM{~xEPK)T*rq6{*y5D9{<=}wOw`}w()zM$4Q+KR1{HPi-Y#L60tL(XA4 zId!YRAP_Q+vchcmCEwgc!97iK(}9L(7h}Fx41%@#jP~*B%viH0y?Sko8h7w^7;{-2 zWGJW9J>>IbX23v#suXg?cbx|R%`ds{c`zUPxc)zQnR?n{0ow-J$vkTOQ@!^BY#g30 zF0YPC=H=W~$b2}ktO+(FD$et#col4k32Vx5Bc%&X_l#16YO3JxAE<= zBrrRGRmEzjO{>$I6*qbPb$D*ZdLDWej4S!8MF8Qg4lVh*5so)s102crS;oI+G32fQ zW1s-@ENF!nufDADj7U1D!z@}*p)`OXIrw~Ysc3yU|2cSD_9_sw%ids&<9Y`e=r%+8 zbw2W}<(GUB<)3PA|4W!uu;LdC%o?!ZyL?1t%V9ax&DhYEsS}fW5p`X2+u8$w7YmnDPKeiy<9R__=kqyhp;Z%O2jg~90Nnpsju)j zz-DAQ&;v$BPzcgrx4I9-JT}^stJ-nmqtKeZlpj=wB8wrIXieGv`|BKE7X-YPm`+-z&T~ z@bfI5AZa0h;^J!G9;X3VgUj_;V`g8n6%4uj@{O#(KMRK#08Vih%qcYwf>p%ZTwb(o zm1U0(M0^cF}?9G$tJ2jAqvrdd4v{w^lQoQG=zmCoj{;+jl1^bdEnun_Okxg ztY-;&XhK;o=>WBfy~V5EJXDnId>1(IkvmaI%Y3r28}9u1;EFFYJ03XPX`FZpNl0pE zBL*wv!E1b|>tuZL?9&HY7KUCJeyV#q0UHi{-VWBvW&$g?hxj*oxOg)L$oEYPt)w@eYo+Q?cX9*&%e7ZXkPUrrv<|n2t`ZMRYikyrA_ne zE%a5ZRu$cl=wB51FQaE89A>q;j|1>8-COgWeHH)~B=XD!l6D>jN6DD@`o_Yq2ErCU zN9lvyU_b%cOG(K@8qKOH&FR11J(p0x^W4}4owG!EMunIVcm^kr2AT2h{|@rf$(f28 zKpkh8J7vv1?vcvvxM6FEa#mZ#Q`|LR*PFBYM;?%BIfqnS;K1p?{DRMC@!pj2ALXc| zJ`chW5M`qGq(b%i{l9AW>!?;HMbJJe&_)T1Gs2^DMg4ds%;zMa(kq@$qq?D5IFMyX2DQnH9Gann$)b<*lGhx#tiVUkTiZ@@X z4yY`>=*WTO7#xV%5ugo)XC(TzWA4UIZVoxy}~NV5avN z?0BL?UUK6UkLEN|2EHV}AlED<7ja%>)qBbkkKJ3jJrmXHx}t1*O%{Ys!jpW@)5x1V zB8F<{SY%&0MY7x^in>1^&NBAC(M%GIWL)Vc+{x>7=(R_c|HG%vK_Y&RUY#7T$v){Z z`@Y&~|D9NDqmBpf;tngfc@zeQ-yU4jIB2uR3r_{kXbR6Z0b3!_Ko|)+CgrNIF4qm& zMR8wogFQXPYOr zm78sa-!?04 zx{iFvz;jyADW{2$zLbDWQq6vJ zsLQ$L!0Gy|My|1prUpBfY8oIUhVSAz;c65_aP4!kq<&OxcI2ZON8rHmrSlV5CMBVT z6$e;3AH4>Mz|MO~e6CV^(sgMZ`GZ&q-@8`@mXsF4w-0W4A0Na#MeV4Z9z~&mHB{N? z@eg!fHG&ab2BfYx+^@-`H9PqWG1a!xcbji3%j78-G zf3@aZqObWnhraSlh3keZPe}^N<}LhF7Xb3Xt@A;zo`M}-6O6*uw2y)u1?QEN>D&PFe-cpIjv8(*!|Y3a%}jr^NNYk0 z@VJ_Ez=O!e@NVssXyYH6*_&+HmPl~Z(DFoG(u7om9A!mvDnJ8?3PcJjbblr0y@5Vy z@znu=!=X(N9m4G-9YL-*cQvrSK{e)JkuR2J*Lo5^4TQL`tj5SUpeiy&NT6FEkjOBX zDQ(9qN)7Of#rV7d2;mm&lguqYf(_a|H;-{h1@6z27F-e8{DI8N;p?;C0V(r7K+d~% zUS)eFnU%ovrnrwoEhz`UoTLElM{j+(AM`d$@AVp~Inq$4%@j}iyd=nBxNflI1p2ZQ zn?HV3CJ^z21?kbrn>?0<>IkkP4JH7Gfs$6Phh+85sm#xastT&2d-M z@V^1ES7xkmY4>v^^G)*%YWxNUsvy4?k*xtPYxvb>L zlZvTBv8K>&FH^146G4}!koFPSp<>t5f9FbD7Nk;$U;&wtHXqgw*q-wCSf<|s12f;t zg6v_&qGjgm#(!qeg}jqCp1A8A8_lP!Fp_vX$s1V2cGC$N3(H~hMM_PX{C~9}famwi zgm#K0@qICuV*%*iRAV<2%>aIEW@;KAR&G!|w^$Cg>rBl=E|M8|n)1{#V)l*XlorKD z?1R0Aq-iixWa|Ama z3tQ4vqWNdeLdDf2i2edL4#-m216 zHc?IG`Axn3shY9ti4+o8?Xin1m0yhi++0rEAi%wWK7!Sy5(U+$o}wk4F3Gd-6~PlP z=e>BP&3xl)x0MX1)8@u^XV~538tcZf)g3m@{)`nJKCtZfn$DV}tV=~3!Xx%Q6;ep8 z0fc66BEmdu4IYwKF`ZJ*@P}VjpVgIV{z^=BhALcg8O|5t8jQb{6R(|jP@vCu_NMt3 zL!JJaF_x?AQ#t$kLvy5-xaN^0PGaHF)TV+W9{ngjb2t4D$MR!bOWeOJg43Mc4oCl9 z9MVbC9pRYoKJWP%s2e<{Eg5p%k(%n^&~C*S?Y02YaOP+Vy%+Z#wGzMETJ`c*`43gq zsE)BWNG!MBOu3DH_=sOZ1s=j)aKAo_`a>TlF;u(WEQlRg^~2sl2qEC8VRxjdQ+diy z+Q|D!P`uuM{E=_?yhb!1?~qqK2#1?D#W!j%cgY)Tb}39hqbbUU?2$XzJfo5O5m#(_ z8nYRER>|OXth#?~ocoOLH)I9UAr3yhmZ$YxYt*k`r&tv@(Z)f15qsS`6~WG2)5n^B z@vAb-Z=qY&UBUjM3itPlYavro$Pe6p8AXE%xze)Oz7CLhzpdO z)7{O-2cPhkM4}Z^635}#iCTA{Gi7N67>`LA)aekQ3apY-Oeo|LG-#Ga(i6*k1FhBq+ zj*4Gt8oD)XAEvV5THI*xtr8x$krcBFmnSRY{;y0*)}ypB$2K=Y5lu(qLH^5C1>u9c zX`>GbZU#801$Ww$<}@4EeMj0it1{>dpH*Wz8uYTHc-<*2MW4-2xZ>Lj`VVsTYn=!k zmQbx{h_cm-HQw{$!PHD|Ztb<~m+s_fp9AG|1+I(||0^@3CJBnVpetUfHj(Xj{z@_u z9a(BNa+G3LCur<13}OSyD8rd#H3)i_RR6^)ypkApRQox8=i$-0$CxkeY^eG9P(@l& z5)s_Tn~ibiRZRvLA-uIupI-b|lEng8OHc~+3!@^T`r5MP+46SdkG_+f^+U_h7nUD`dTXQO zhoP4C=wF#_w%gQ}AEAfs2pr7UAA0O-ee(7k($eZ+LZ5Ez@PBie^G zb<#NCDqjN&i!RM8c?KkWYFP68Pzjbe(-3J?JlG%KV)od9(>6OfmFZ+IKWF$i$zbIP zdvs}s=)YP>%%D7p$bfp}yI({Bu*X*}lUyTxbB>kZ=C2M*YFc@xRhYvJTXEFyw~|dh zZa>WJ4vj;YOb}_0an(y&LrRBmMScG(d7QL2{_8Mw{V`Zx(3Quuls{^|<;fI=aqoX< zKO}tNQ3shHXK`#7DGaIS=LapGFh404_3f4QRKV6U9CPQ2m-0X} zEmKRG)Z=8SHuTX&hp&Zsf(li-i2ox=5-6!ubdBc*CHdxv*V7b;Hk1UlgmMZdX@`IJBrZ-j*>z{4b8<n6*ZzhWyL)WjM6;|x02M4e@02!Kta?Jz{Ef-1ry{Bk41^lbF1r&XSn84Et%&Fwl~4)iGPgD9(~df;m+mseXPftNg<68-2fd#3)QgcY9i;%|>zUJh`OX;h|s z4Q10{wxrp3e9O64hNz{vCWE~CUWMKn{wr6ZKn*PhIab+pjI`Vo{JG}P%jpFnyqkHe z$~!e{_`|I8s}A3K1Z6M}%YUW5w6wvw(-t0mDo&Ikr9xDi5qIpxNQG5+dj={0pp%1K zZaFBrv(SG%$WG$Kug>43WBN8x0%wN_$Ep@rS&{F7b*R1VDh8E{#T-F7E>hNY<@ccw zvpcMXzT+@%apOPb>cDIr_!{Cx+rKTO2^y<{+4>&w68=ymyeJQa@XstyCtSM9s>V0x zLORXYawn8ZjrP3Z;lZq{hf(8e)J}itXdY(WAzHJ&6WN|{^?$_VOAs$tk`;E0r4Dci z&R6GB{2;x^1K31O=qwZI5GC1Q&od00$KX1@O?9TlMRl`u$P1RNr7z2$NSy>9bvRu3 zM?wfdi->gXyjUi_$oy;cOz`)c%%}UK7Vw$?7O}_f&Ht-|Dv#z0cj2GEgD)xYeE_A- z2mWAJe$8xdslxrIA|Zr@{0#~z0;@iMSFqEO-$~EW=D$f|z=>kZuYBNFG)qfot9`T6 zgF4#x&lxFTyVzK2or(5|ZQ#Ke2|xtj{L@;@y-I*Se3#hxGaz3dX~5sBvFJM}{TVY$ zCXI*DjLJVZnnf}tXTjzc)4#d?@Z?)MQeAoQsy>d6um5}#%cB-0_lrhr$gkxcvpJAr z{~myBgIafNO&2tw=K??H-!*>lr+hv{h1my=_-S?g_1dK1Ss1-2?|)pW1-Md4+cAfW zuTS9#(Cs-~LG#3a57{=M)fT(=E|yRVCgwEt-=5{;S)kyzlIp9BxR{~4Q$1B{@~V^_ z|H{f=xs=zIxz>leG)6al=lnaq|7s@6<0~$@#F`d3QLjsfDLg57pV0l6Krs-0H9Z)3 z%GI(}?~*Y2f9&=Slsin^zon#0OzMpnr*9|3VCmMc?`Zg->K?n#`kj5)XnPX>18QQ zjTZF?{49Ll`0uMinANzWuR+@}q$=0`dmtG+P@%U$k+ABZNWpoBLRXCbZ&^x;n-?+c z^*O z$Gv}htw6mQRct+T^RyBko(I*a0+{@d`i^&Ir1wTHtB!%I40cBGM#IDU&dRHk*51ix z@p>zcq-&X>V8B}sy9=N|Gj!zTRXM-WLRS?;}&=3Ru7y7KyaUetb6YGwY)-SO4ROzo+lo6^nTYQUXg^xi!~59)U2iCg=J3 zti(kk@j8xs7@a=hFbBx>q{5QQRuNY# z!BjFdC^@OegzQGK(l6cqm3T04_RK%-{5S9mqU!>^2fs%WYW3XOuy1I6_PFQeiYRc7d>ck@5Rk?)&-lyd z?Iu)Ta%;ZjdL>P4HPk4?NIk_BQZjd!9WL?$@mx37oFRJr8C|UPjWmQqLaaH3O%9k`?c+}6DeTqS6=Yg%BlMwA%Y=?*j=c2-Z>k*t>6(B1HD_N-n0 zme&Lh+hKAAsEBDH^_uqr&ly%5!G**!0_#}^4mj=WOM=PUN>2l-9KObylS9a$%C&5j zGw10A7xd1srGMmrk6keYq=*~mdz)(JKIK}o-ZTwat-^)Ntqb4^tV^o;=uBQ<_fY1d z>mvpqhU(6jq36S{ynq$4N%-K z&TevTZPxrLc5mJK+z%Z^peXb%T?Wz|B2Kj5qNch&h6>v*YH$?CKs<*v>-Vr3V%GMC zaB;O1^kB(!BRk?y4`HerYc3wUT0_-YAvhdCzoZp8?T2I#-Q8rdT^Y=$ghZ8_?matS8mm3S4^akvoSSN4BPb$!rbJ ztTCP?Odr9F1_m7uhU7J*v$~zhb@^u>INc!&|^8 zwhH$-0$^kFDaF+~K(1I5>4B=f1$V#Gy)F1yy-dsj7hEVkgmJ+C_x}Wc5_2HlvcD~0 zAY9NB=zKeJaX~Nng@sk?EAz;psrM0ORvCCbU$_4d6olab!@0z9Ej;neQDio%o!vS8 z0Ii*MPAYNA7UO`skVW4hysiosFc(KA_Xe^^v`_GGmbjif#!RZIs#HO&a zZmmX2ta%;bFA9PPe0kli&NnQNn0E+#&MF}|Af6g2%a`PJ?e=|6KbYI?f6gg1^mfwGk9uBY` zrF*phdBXps=c%Cj|D2qh_qZdLV{fIfbhL$>*M_V3>mv`H4OdOgkK%yC1XqhH^fBn+ zD*GnfaEj+%?Zi>MF?ao1`LO4^)*V_bxFfP2cY=enDS32B9K3wbQ@3<$xYpTxJg=Qt zu$|s{o~P>up*~C6P7P?uRy_JxdDwRY$nMqBM!V{Dy;q5?Kuby)c;#o?z=Xj~#}j-v zs-N#rwU&i02BXLO?UIWRdXKnf&<7I<>IVg{5^*y0|MUd9E4q z4#Zslq9lo3bM{_dxMp_Q71RxJCc;#*+VsiLb|f#L;*uVMyH?|=CzWnfN1MsZ1G(d- zQIfN4xLGRyvLRgq0Ws0OQUimOVxy*u#;k}5q=`pgj&ky%VC_`hiTV0?4R58V_u;7d zdcA!UpV5z#KHrAGE>>GvfzHh9k(65y4sW{YWyX~ zE!(GR^@JuoDva)VRpPdH;C4xzL}MZSptA7G5PGp+$JZ1sMGG-!7CyE)CNqsk^5BsO!<>fTZ04ePbIUZB&mJ8xxSW) z+`}Em)i9A^cg&Os@^qVt|O5DNU&iImv59n#`wrpm)<}Woxqc(`N%AaYk@$*Jma&b5L zJkjGs^BFITSH@W1#q@Tf(l1YY!%ONG^}XlNTRF0qfJvEF%y1N1w*&LF2{=H4+e@6a zAv$=z*FO-~O5IYt+c4qVHcjCNNi8*%Gw`duTitb6R1dKb-^B%2*hm*4@v?cagSDO_ z_V>1nj ztH{eZ`MWx)_~)CS=kr?Inu9_{DoY#7ON>@3pPW5syEhYZhG;!=d5ups z%9dxHYm(&H+24S9@PYGKf=N>2Yj)Bsg`G`4756wYTOq?InwxuVt0j@P#fQg0X?NNA zI)}u}LvsCQC9Qwfx?$JdKj}BRl6TI{sY-kSc7Od*)s@Yjn@4+(BnQ`qN}G7T?EiLa zT}Y5MugBy0GhwHRaUE%xzAx6#(UDQl{lR0me&%OwqmCc~C}GLzpn{U40Dg-q?~IY6}( zywY|8PQ)yJs0a`FzS=$a7^^ov?w@*6DmtzE^tnv^%s1*Uso7=k{EIusrf?)9_@1lo zQ*L2BqV|4;dOYq+vvQ-(aB=Gj_4-ECw?<$nOoA7|)X8vc+z@7x_?hWFvegQ(_`+LfSXYd81ZzDFCbq_@QaI-Y*ho;OlrZ-2(Sb53e5@*tT=q&7sh$|DNYrq>ef&2Ja!RJeRc z#iqaMcFqiSK+ef(qu;=jJE^mVg4 z&$;%c%gCdz)h_Y)93~3NlybF_!1lb|`!6R`epqM@T4=0txw`^RR@$<9ft23oSGw{l z&NL_|O~&|cjpWg$^gx-WsxxwU>(1v-$&8ps$f4J`9?-SJWwuj9bpiqLl)4(R@c z#b6Udzt%Xr;}q;JZ=InBRMxjxY~^|tyb&uvH2G?tlF!0ya8R5*6Fod#wmeB&(v<3Z z>^UHgnL_X%tx+o{+tyZ%Puclmx^20RelMG+xPp27UK$HCxpq6+d0p%!Uq_1@cLo2g0} zxb1f5`VMcS#h5R=!BE1!X%3bV!X&mgndOaOO)wrF^v*S!6cLeYJlgIWLA#BaVREFS z9&)-Fr@vDb-w7Yz*t$a*W;74V>B85&CsyshnzcMlJe&v*kZyr$hDpo<}h%a_Q!VP9WB^-_rB?8OZkrhr#R>SutvpNS6ju{kjmL zIE?;udsVjp!Wfm~OjTSo6%%s(Tc3NLYmirTBa0mFFUy*eYl*_-)A%%|hR(Zis-@r1 z?{(rU4@tBSGP#!QHsWZmOMO?%5?bqW#5KvT9iY|vxb(>=buWcsoN}B$KgWLYj(u2h zH7q&%%n?hJ_|$GI9be#WDQmNu365N7O3mjfLY|n(Z)wxj6JktSgS7@KR{8znpl?DA zFYeyV$I>jKsM2~*WTe4_z!COtUX(I3`XAxs&2kFiOpc zGuTu&8T0i5n$^gYvutVLM-whBa~pGVo=7BUIKVD3Z79!V5;x>sTt6*I)t$UCU8%J^ z>?FaCtXb!>Y&Kn1Hn455)tqkF+e+@jdc#6L*iAQHI+gPmqvt%4CeaCU4YkXQ1`}f1 z1-=^5`l;~ED9Xd4L#GQT5%uOv$JxBtzVpy&zw{;^?YgR|{M^*#$RE=KKCH?Z#)X?p z+pL27hrcqTYSBKPY{I>I8v?prg<@!Uu9L*vf^jNNVz(y7x`fK*(AusyM(EoUj6O2Y z&!La1K@>G6v=yL&V^33X+7Dqh(>mLoGH!!}P1a0%>V7nQKPBCT3)l6cDxOd35+6($ z3vbGvlGffhyvdp5bMEb22<+%=?LYQ7VAWU=*ILMi?>q7HrHl%{F&OFKs@H7VsL20_Bq5+FIvDYj&Cx5jXjG|pP4 zBz|5hwHQ{V27b4V-Kcf9o7^O?8h%fH#v)OJbJBpgSt%}`ig{lliD1}DJlhl1aN_5Y zP~zilSKnaWcZec6o{J|#3fa#Reyc@SiyC^qH88~f@IzNQ-u0jTZgOuEnp}NyHt6UD z?6KMM2?vC`{wo9-!sI)@IO&O)w8JeSS!?=!g@LKs86^j8u^qW7Zq`_Zq{wy>#GHPv zm3Y-v1X`h;0cJmc@A|QAEyB>3HqIQkW2O%5%86Cyt7WG=q2hz`MP%@7ynAh6S3kKR z$1{4k_RZk;IcQz!(($DPN=B!=F6;f2OAR6M*lVS`jQSk{3Emcd`SDIv6m<_Ardb0w zT()&7r^K-n;klP~Yj^*X7kD~rpwbfzz4SgdvdDf+(dwmg@yck;RY0X@6N=!f22L(^ zhz}OR_Y1vWR!&`Vr$SpVzdPsaKP1@@N$%oAh1NfkVZH|3Z3A@`aFy%OWFM_)&!gEJ zl~EX;$9jV-pbT;^FS;;|$hRc;PQBfYL4rX^zm=e%Sz3S|I|*GNy8~UjQ`FbpT$WbUiw4ba4C*XvyY4d04G);Z zY9JK43b^W+W8LBl-)8+&{ck2fpFO*d$k2^CZfA{qXE;Xy)yz?G-H8-Ee)Cn^2%@m6 zayS$8Y9~y)ZfFdl%kJ7OaiEZ-Z&N^DwZ6&nS|3*PNxe{r$wJjBZ2&30f%>&0CmaM^ zByJ(mtCmx7x%#TK3XMeShG~4}J{$6M+&XmJR%qYvX+cRYP3*Dn!KmfRQar9v8SR_s zUuiL=S@|+moSsQ^zJzMX)JUn~MIWx&+ax072bNwF0=lwwQ!A-dF0^B^6Q>0C7(e!+ zMi|Mb<3&Ni!pBYxvx0JpTz^K7Xe_m^rzW`%(U*ZwyRpr(&2!k3r7w<}`((tfzE>JU zXuG^8%njRst@6j`)e~_{6-7Z1u`(R)^Wpzi}jqwj5 zOk*}5nD3cNanTysPr9!EOm7!RHYp~%4q43XU>dK|HG1y<$-dFbHh%rKAtcD}v*WPv zp7cHDKF?B%pB7*uY^4hSa%fdJMxQRFA#t#8O5S*;&er;S z`u&7RxA8$j$>8^;_EX~BwrWM(O ztj7Hnue(9bVZ2SB7q@dF*95lF*K$+nQ;c?9Hy6eQ$6TBZIIlAEh+I{_sO56*?(T#Y z*A1WEfSzi;-8aw9B!)0i%34bb2`S{6{Gtj^cT%rC*j2u_y|CK+{YeucixXTBOYa8T z#fqtDcC?F{Z?6bK_z-iF=s8$a(X2T?CP zz>J46J6yf+*ge|_v`G;(xyRfb*ZiF@Se7fQq!1%@ri(&_M(Ao^aWk`yImugvF5ad` z_mXVru=5`|>>O@QE>VfUEVFGK&9U>_mz2EsR%9Cm`n=l>@22{dJ2szrIZcANl#L^% zESqYg(8=<#7AjyGvKIgBrzS;Z+B~Wjw6u6rT(`nmEsc_BerDRiFSiPc+E0g0y|MXp00aO zZ8XPiB)WOnI@f4zAYb#c_*M0h?xvJ`vpk}+S_P#)|YfI*FyEP-`-~Q=4 zzW0T_$imnbvF#<(_59qFa+9Ef7~)0I_1KT=y#B`@4XOnGIuYivPrtx*70tYG5kjNE zbP;YrXRSt!+*V9+d!wQ?`q>yY$*bzv(N`5!fv9eY{P)EHM}bTU6IrgqdhIFi%tcDqrNzV5Yd(Mk&(&X~~hU$K`LZ zve)`_OJC%1FNy+gx8Pc6UZ^u!>r0dfC0eQhr+IW zp3&X>I&VO?{8ex=!+ayP&b@bT#T@Evr+Kf%a;l(FK;a^{<3P~&##O~9O>JBU$3B&x zHLe@D6l4dW?x+<1ZahA;PC#g!a1Cc@B5336zgh=wy7{&rlpdjmHeX`KYVP)%%Fb>N zt{SRr}woYYLkD5K9RsML3nN408V*jz&FUWnj~LFWE&OH!#% zFgAL~Rf^~Lpsy5IB?x>({HDW(Cph~?+5sENu0+Os)9CTE-0oBJeeZD8Wz|%(?8>So ztwL^b)-?Eg;sZ_V)p-5ivXf!HZ3e*n#oJAl`S~MAE`2vg(1ZqcoOAgtbnWmg0x(4j>e`GX7T+880rLyZBQi9py31IAa#)F!4Z3)M zDjmCt>AhB*(N$ct^p4OqBCABE1%doZ3!RX5&K2anDE|ghH5ZbmhbX&dQZLe^S)f1b#Ppz6{&FSVfS_U6-GiiK!xbg#joH})=U|v&Ur7b znD;~?VYYBgSixE9j1Jv8#e*WOYi1lj`a?F;gQGJ#UK6^W6#9>AwfnCu+4Kk?uyNbh z?0e9)fQAoy^XIf=G|ZNmCWp6d`ka#|*ykc<2ZuN2eAU5muS(sD z|LfLQA2VLRdz4`8*GklkCM}O?GS_zcOJ5O;nn}*#RH5~!M&y!QVZ6o{VDh@QFYSTI znU1|WMyQTTW|hY=KxHGnuObpd%8zens1qC2_;uy&VpG;n+fLrgLccRi8nYp}@dLbG zR^kI1N=}cn@tv}Z^o)oqJ6r|0uiywXMJn8&#~pT&RNg3#SuAnJMs}< zy>_yg+%r7zYg$Cew|%*XHdIMXG2{`hN?T(&Qm5bN&K5p8w62FUTSn-o_Fr1kvZkhR zNE>!dFn05-kl6O|Q#>%>?p^kOD!le9(^M#%$?D3d0auPB&U_a+F3YhTeeweOxIskO z9+HdRk_>okYjSCb?_w+1Qi1$_NUnVZ9Rg{|8nb8W>rznlU?mI6SZem3e;g9`FNX~70$`QP*vh7QB&>i-5EkHxhl zAr{}O@>Bh;8?3El>VV9#(d4O>0bJ5mp*U|UD6JSr);2f)vqBVdl4^#|V5v!&+l{JN zHchjBmQlZm@KyH?k0V-K_(&DufE(Z!XUO^~?(+892}|^qQ&)pSdjx$I+L-l|Vm?iI z+7UTv=jU-Ie(A0vnafI_^7@@4`NuAzUbnTn@@S-lK_CR?xO&iM-=&veHo!U~I;OL! z8W7_2kukeZIFHw%Z(h_30lBSk-pA_a+E?E2G7&nbWN^7K2BS5wUDqF9`btB( z%Rv#KqWx*;fv#liH{s{9irT473dgkME-OBx+@vanPE5rk*7q(E=?|52+{M$eFzaC5 z!hsw2h>6iw3_M#kHW>9vL>Gkc8XtC+=hB>aDG(cC_&$U%rmI(;Zg?XrAC;rPISClA z2@2v09aZxeYKWCi2{M8A3=LB=&44*CbbV=%s_JlS1$0WX)KIjBw)1}5e_^LX6)Z@;JPUhdijsa)3`R2`O4HwFBpkY;HYNd(T#qn;Bhh@xx26ZF<@jR~4>Z+sPOc7FUnfqEN!g7jod2Y+ossuzt_A_rN~4q) zQnyrK-)oU1=2>nXk>h(n%iCvm9$YNhvDOHa!a0 z*=rQDZ$wv5IA-1KU9MLYzHhPoR6)b+$wEHdGF^+kH-7;w9V7v_!qKaUsPbFY>5Key zb^pz!yP#DE9E_wK9n13@D{?YbIj3_wmDV0crf&~Qrgk3S_0rxr#vf;{5HO^n2~HMz zP^hf49F^jYM1Lsq1Qgt3`r6c$T1U;vDuYV@L%=^uY2i}Dlr-8UY~!6*#?7nTZ1(5Q z+pUDi<3?6isQ`ga4gDUj$PuAF@4QYm%x#>*4c|e%Y1<*)+)ivUFh9l%fiqZ)_u7wM zMAU~Y^4{NpuC(>pA-{&ec4uh_o?OgxORIoBFoLlwX){see}?KlA5CMrEjsf(XdU+i zHJ)T$^c?i#ig*1~AJicHJW%}j=jFWvWa@ZigV*TyvpDjN;%s#U?!A=aS#m~c2m6yl zMAO*$F88W~pgxPB&Ia)>HV;c7}9kf2%5<< za{ob5tx#h(>|Dq*nHP*O_HR)3anFD{b>72T-|Mf@*pz`CQjNePVj4l#&ZYt6T7?5Y zRHgcE2-k1rBp1(p+Zh8Wk6S`kbq32dCr6~;lbZHlgaw<^_%JZuul6qE&F0X|8KV^4 zQ=e|Beu;#6vbg0wA4I1~ray~a6|!k)I%!DjBEC^%e}cg@)gI7AH2y4U+|ysJAGfPU zGS&+j$eHN?)0l^z@TW}^NDbLHoiuyd)AU};4jBP@4;R z{;N?l8wAwdsUk4#9fk0kgMn+Dw#|HK#T+fK$cooJhqq9tvE`h;0rzQFT?4Cp5RD2i zENN_uU^cQK^p*@Rr+Vy$_r&;S=wzx=V$8leqN`z% zn8tG4VeVSygydnb|3qHTf?u0rgM_#L3EtxJ7({ypt=4L6N*6&NuD!%~{d1fUhrG*~ zB_xvjqihB?3mZm-#F(Z>mZ*lk%2wLd`B!rP54-a>J~pTeVe}mQviDO{m3aDtPK~nA z=!oA5c6!`Y3ODoaPD(k*Y>q`rY?FdK_*=WVe`lb0jnNB4&c{77Wk*}WxL=5(H*CYW z%gY)bx)q5Z{W5LFIhF%|<`v$FC=ugo2wBnZH~rW%(SR9FwhNWqx}{kgQf+Aeu0BNe zsbho&8aG&|^ZL5`RLA%r(_%85m|;^Dcr^{*mU$Y;7)#4$=w6PS ziCrGudk%w)f2Y&WFD><>LN2^g6jj$1NPTm% zd(<3?W3ixxV85{0WuI3i%{kvc2WuwgGo=kajVJigkQ>6AmAH*oWLd<*L4^>aR@=iV;h`PI9!u(AYEWh#L9xF)qgr5n4 zYfen7BO@x~boKjs{@+k2e+hpN2zLAJ^aue**?ApQAnqQ2y%7R%O31*}*=zqIB@~5l zpUHl4Eky4OwQIdiH_e{n5;m@p2{mF(GA_Q)xQx;qvtq^cS`mX(4y>r=F)IDgPoEuV zawjUD+ifm?053FMk7=raHNDQc-{q<|Ol#~h%V{U>uW9v8aH|2cmSE!l#VlTb>^#w% zZNzT|0`z54MrTTb&{Dk+O6T!yqx!mGx!^qs&+T@>EgZ%M#{9Z`J_bro{PTP)OLi0o^$URynQruQ`nheEg3O=m4BSCTQ3qthKtE$IO6x;3@&I z9B#~o3cr6rFd-eb%-<%4+t0;j5TrsyT+3mK+;)IryqkmO81@a#1r7oa$jYyy>@^5+g+W->|@NzY+AQJQ0h3?5i1YQG-i90 zzB}xAXtweK1KXBxTCl{E>kwi*Dec7bioyv^@&`uILQo(rw@HA%U0qu!XMFK=qawF))hhO z!wB7Wvi#|Rb>x$p{iy?p_bUm?4gYdRe|yC~5BT;Ay+=L2NUy(-e1qG!?YoKJ7CASp zcQPQ20pXP|o;kc_)i(b6>`h%5R-fX=KRC)iAAhM3tm~e@gTlWJ=HD>@S#C3m&vn~| z-%jGM-vvSWo1p(?K$Roq&p+wku^etYHC18XUVWqiJA-3>p33~LI%VZ*v>`QH|=-jBbx zn7wVo3K0G@{lT2XW~&UPo%ur`Zpw5jx{}eCU%5b;-Bz0n-NGpM%T?zfUqLzOW^|5Z zp4~{W;wghGXf&EpHtqPI-`9w!cPqEa3eR{4QkAi%LLIBMmpI&*8tDR5LB_otd4Ft} zQk-Ot+hdJ5gd|arA8&tCPC$2g1xq zOUv{?P1{5O6H#4r%Yt1&ZZfHk+Ph=!$bCNB6l)8pT~Z)o_bKyt1kK;WX^ZTR(|qNVNNEe43Yg zulVPIey)t#Yl7BYpYliM97>j@H{mo@-5G7i+QsZLh^e$hZ$te-+{y(R`^bZ z^L!tN$X{}w5F3)SFGTPuJTzIe#~FL}^BXJap8TqBTMz8F>b?-G!u<1BaoV<1SN6CO zoCPRw^Q?7_Y9nVkAZ68mN6VM3${dw(?08MJ6Z7|Orn2J$5&#}*c`_7lM` zHiS%_KV2=GNghn-A4#n29^< zrO88RezsiBfm;#VUCs`aa%_=Tlc}|yBAq>_?NOU+JrFcquFF55ZdpK5r3kyazy05& zK16fS9F7ndR!Nz^V}i&@?=U&mHeAZ8*?Jt8fdoH`id zCr*y-P)?gT%@micY03VszP!Kk%;bGZBk66ZFHMQ65ooI1vya?yRh`!xpAz-zQMK;z^3Q${}y%>JZyZ&~g;k75q_3KkDCVSO z&7t>(IW1PIr#Zt1+CNGCss#YoNH1mRqhS@kO3qzu!{(f)8^Lu6zPzw2Xoi`Fl^&4h zzSkiNb4TBh>i8uV@4Cedl>XQvV#+C@Sr|&k;-dJWoVeVb+3(Wb$4j$D07fX4t+r=9 z67Z!*K)E%) zEA+{4=bp^ry5Lt{L2Y^3VQ90<3w}x#@w_G}r8LfLx1R3}{L#ea3>OR8A*!M`0wVO^ z8z$x~WEvWyAHrCqL~hagcUNpBBJm4ct$P(vCqm6CxaOJ3|ygsb>K zApg=7C%HVNLl?7BMgOWwX4ie@dz1_tiZo`0oaM*@3LJx+7hMl}i9cd%%bZN^KDfs9 z;19bO#vTe-!{p9y5{1|ni>#c91B>5`+pcm+sLak>cEQFIi0EwHIFrI;EiZbUkGfit z?e_0BS!3l#U`B0kUPv0L5<*dZHCs@!ZHADfVnLoBu$&-LX4YEF^)6C274ggnF&R&P z5O3Qo?Y;k*R^2RmPeI(L5m~>@L~DvA(q*BS;|3!SDR9^oYyq*GlBt9{Ux$l8w^u@{at+T4#czrCO1s_3r&%!# z=K@lpQ~y4%9_1&ur)5SQ*@F-$n~&+n?XzZ81-40-{<^fS>O4mG(<-uiq7k zPE-RFG$IOXILRhxAuHth`n*SSnBjZhIM5<)g?%C!{G<=TxUr ztjI*}?OrD5T0*`&YDPLSA%OXDB5Bgv@OhsJR0z<{?MX_efY=!7DbZkKmHQOK3egT@ zP{p5=qHM~Bt18?6=0kn<^_%IGi4rkHj76S$b@%)a>_soGa?v$gl}Y12$N+qag-oyiR%5#Yb?9~X62G0}JSOT#7T|P#ZNVv` zsVYC@uW5RbvAw=RV#OTNK{8~ya}JB<^fctY^I4=myI>_B!PopDtphJ{bVbdNXLsi`)0(5tq{XhO=*^l$6{}MI<+f)g1<6*U zNREGw*w3&XBb~+QqFtyYmQ6bp-%EAKtRSB!FmdrnrPlO|G*UpE2nU(BmRgQLNkH?{I%vIT2R5aLnQKBMAowti$>ytWR&+32Lg2jfx z4;z-XWBdrGUfP|TA_8IR-u!z_ZPjVnm&*fz>`Q}DJv{HXQP9ic!yMz6#*25U4oTvD z4?1p~E$ii3uDTbkC*$g0WIQQ!kLZZa)zck%q?H{oMRa^!AN$qo+I04t1-tDNY->!h z;;lQF_&>w3t{Tq%=wtbjFVa@C?nB+XYotEhY-wNGSN5*q*=*J=ye3sP-e*4f~`A^uy@7f&FD91<8>eF_Al;k(-*oSvBc$QlJu+ki#dci@7GBl3J>lGiWR- zj#MLl*jYRsm-%aXBc)AT z6(jgYj)dM{JOYP@!t~vrzH`#<<3hVNhI+N0`CmxB62bSZBp}KCXyvP<$)j0o&I$El zmB0Nl<+3Y&A(BSXHi>G`C=O>Ec}r&PX8|eDM^gMUiSO=olBhAN2DPN&m(;>*j4s6a z`X{rGj&|Hl!Pb27yHLl4K7wAsrQk;+_~RA z|F@$nX@F$mCr)-6>u*-ob0NLq5{^uegIw#{T(rrLhYZ}!&-rpk-*|Cf@gf}GHukGD z%fAL2QW=yeCwb8N@t;4A+*eB40MA&n^{qE%SU4zehB+GX^z4Pziv8E`ALoWV!t7ml z{(LU-33e#?h1#oh>A)IS?$?b;2MCQut4GfIBSUJQ;o00VHcYHb{M$Ox3trlP z+r9px{y*LBS6eY;Z~Ff0|AV~Lo{PBCPsnIAw+s;np6*h**|uwzLr3ysXExF`M|UW` zo|pR{Qf-^JDxch-L8NwBOwn90X_PcNxFKe>)B5Xl{0Dcg-rD~V?bZ75e;d(0D!7FW zJNH@)`ko2i__#Sl+{ICT3URbF0=ab(W&Bv{{y2r);154WzEVi2YdGpr)a7|nv!l?ZiP-%j=ppne;~#_6 z&ZX)6**YwGY)NHdu8DQ+cP`V-foW(yO{^NN+$B0L36S9Y^B@{u;e*IscumPfal)LP3oJ@q|2&ZI%tDT&3t)!LBlOV5`VIyz&}pftl+Co9ya z&AN;QN$6`~w#vW3g2LRkB_w4we^u2m^rsh!v<-Kjyo?s_du7C`1;A*M{aAY@ppQaY zvqg}DwltvVYEz@4sB`O;VK5h(@PL*>y0Jme)nZ%lVD@F|2XLQ061X-^emJ4mZ&~BhNslTq$=)ul=L&6Bc!p84(|N%k8ivNDa4H{*|WaUYnk5wGv27G7Z0}8CaN88 zU4aNO`Z47ppt)jBJL#(&hOCLM*Fh-S!Md$FfH(vL#2e=&OWZJR{ajw z6_oB+@5mMD6m8zZgZ~ax(GleVp^;7l#(J0iT(a+*nO_?(EL{cAscv8kvCJ z1QiZ^+Mw7$44%$)zL!Cy_a%B5Ji&^V(8+fnsF)i;*vw&z$J!QG7K&EVz{ze+ngyRL zyEi4Lk4a64bP_PxxJQzvP{+H;sGo^)ercQmD%KL^zBokChZb7}-DGo_G0?ru7b?a! zXa3-xQS&C;rY$?up?ZtNH@%r5AHn9&n8?b`%Qbk#EI@~sf-`?SV$Nl*hP^Og(7`<0 z{yd-nt-4yDL80V(ntF)HV{&fFDDzuvd%AetSlShMC=J#*WAz?{nsl*k@!qN8u!mCZ1PE?r<7T_27cy7MY-cg2@u#{z9i~TX`;2R;!IH5~7fGoPIq= zyVhci$e;kaB=1+4*5GoSKp&h`9J9=8m|toctJ$%zyKlHM8}&!8j@0k$N&9j%B%8fg z*J;EsP}}Bm+Hl;8BIREHFqoNkN3f#=hNvX6JlCz$sj}oK{X)|*8I+0TyUgxQ`H^6LKH8{e@NfjF;ZuL{G418MqALn>yJ6u5^+Y&n!~ zqrb1PIT8;xpwLU%bTCi>JgXwq$e3^I-m-k_+TC5>q*j~9u^#tWXMJZrXqZM!LCQt? zkwEQgK*8u+?!SB9tya9Bv@aJTN;zd6ku6Gk5vJecy6Yon;D(U5e70%l+4mX8RhH1U zb;i|q_c3LiyX;q*AR-VADLUIC;(z#(H>-H56wzea5GB~6R1&JP-CXc;uew-s)L1l@ zo)SdH^6R5z8IYPD7LYJvuDbKJ{M4C?h|<*!GX3r_;vP=ttX-fqGriv_oQB>sOIW_<>=O3@qf_Xyuv>xpR-B;?eJHSM=IZw|j-=y*!kEs-dBnJ0hzzS)MTsz!f) zJymfMv8q^1HD%(|r2#00`f7YKDN4ZTY&nm&?#abWOMp2`(Lji_7-itg_?cR_H8Q}f?ox-#EY9h1i^II$1Z-?OD1-hEm<*@#dYJZLp~c+J#9p zq5+O)#U-1XfnTnNGuee1O|MZ%cJDve8yE5%<+}5jm~VtkeVs|#{Q!~Sx9%Z2rz+#5 zJo`fLD4>hDnO^JoZ8>f-Eo!ToG=F9AFd2@ZP0`n!t&4w^`TNJ#1VA^M3MKGney^hF z57v!p8?Ks!IrLdeiFDXl4@3uMt}OOTa965X-m1NFZBwV|)9|bgYzxiPsOzuqhay~2 z{Y|mkG))UNotBrzT!mrv0C z_^(OchDOr-V%Q}7MhFTFNwYU}cfmC7G%;Q^*iXq-e0gIrM6;VmI7G!3$xu4Oj!UAr zW?Iw7s??XI#k|cpVhXsoLIOLlxwF*i!A-PCjG1gtkf@{)Y7cqxfzEvv6_8Z@#2Qh>b+CSHL-AyCKe6?2PJ4c)MoZJkPS znwE3Yvzjt*#_q7q%4T0=*j#GC3glaW&f}(P+SU)5KOf7b75I8}*4WWo$5x5N7GG9B zRxqn8wBaem^9CJz+(SPBDCA|Pn9-wLe1KA!x(k}5a#o*eKU~p|0dddDo zL>&IuzTNGn2}!L(#CQ3MNO`mqf!1Px>i3wY6Q^6^&PZJjk#+JST*LZLDh=vIYh_uE zUfsC#JleJx0-)%~PNYe&r8OllgQoH8@%YgUEEjsa;t|phM$3(ZM<>@}!fl(uE;(`c z?8@R)p24-ww_rVf_f`B*DX~1^7gNL=BrvUCT! z4cq$|)yAWBqUNJET9Y2U&b@)gq1)m)l@jQ7VwOkki}>o?{6B6Q%-zbn!fEBjof|v! z$YQeRi(rHHosv_AGayH~0TB1D5qOvHtG1GBH^P+1y0-?3Zt?vvY;JxZKQ;vA|2CtH zJa;jk`9u(%l54chz;-)AM>=(R78dq{k19k)uE zmJJ!(H67+Onu^%yO=slRnkjygoRAm9yN|9`Pb?t0tfu=pQ%0<{*I*+aXQ$bE$WR{L z5pF6Kj}=l;WD&i4+j2>uLAxg76j$wAnIX(*Yg3Yl<1|s5E%XOaOJk1huVaXPuK75g zWcV8Mfs!G+RMM0(-hy7%^oZli(rn(nM5=4m+55yzB#+hM8ExCVqGoyA2oC~BF4|*n zHi*w%Zt_b>a3zQ{pSrn}rJXJh?hE011JA8(3WLzt@;S;|wc`|%MVmz5GTd2x|7?#FUEPwRg_4!Bj{lvDfgm1Rd2iqzrzXk`xE zI2P_~15;G2Ar2rX!>ldN`tVEs%A`U6_&n3YQ{pb5rZp&^Eqg(m=xut~IkYh<@ksP!Y}gJPBrq^h|0Z^Fbym!@ml4w-7i=i-^FjW5`d zLKp)VFHQVV4Tn7?Q`YVEMBS=0QE8}KW(7m>RuY$gC9~-D=e&-2`coi@orS6KSw_@Gf|# zFj_I<{>y9rg%#u6a=$R?brLAk7Xs`N2K%n-*fXryEP6X{>dCOF-H;6hP>RYw0=r3P zC9v>eBaZc$(y^+sqLo5s2ZgCR9z3hVxX@{Njx{0wT0KO#edRWur{v$4UnPk$ zOH6%a_W4a&M%|ukNj!DoF>HBKL&l%%D5(VW{L%nak$!Q<1WeqwA(pvc3L4&@cMR3x zI=fF)#q1lxX{S`FJ9j3yHY!i~po!R2x3i^22~%@ye0Pfg_sBT2~^_T2S!6z{}iF*4cgAF{U|yc;zmmB*;XLg@ullQC11H2xTsH(%ogi zK4{yuKQwIfHI2F}R=NT%cuxTOD{4Ef(dX1uC1UCvzi)9w9Di zGgMk^rS|y5`%k#wuPLTqla!LC5lpCmGFbRcLQvN@%@o&k&GqGA+BLryC)E*)T7Y?X zY-OfETS|gU?1~blNW!z}vxS=RrV&R9e0|SXRg3fAem?jK#cE-coYQA&vab(+34$P= z+xRYP9ISj5Ro$h&HXsq}caOEC;qG3?Z_fQV`{W9RCRtNNOUE34;ZF+;*%Lgz^nZeI z#u!j$KDy^P#l%6WajC!it4#Q+TPGdaal!i)WJ^EPU~KC>VNzz0MP`$E<6G!l!nLp- z!8-Oee1)m&-M~i*gHms6+$ILyH+FaUDYO9&G|SqQ>AmU!e=VPqJq*S6n32bxIlx0K zYGbe3TGhqXha%X80uFU`_bU1M`v*9YlHG4U%4w?74Lf(bZ7~Njr@*eeC*C)K|2b3@ zxkZW)g*$#=9W+ttJ`8cs(|-%^ueKzZkZv3XfxBeZcp$$*R!2Zpp4#~2;wpXq&-zBB z0s<#!H-8(m-k>4S_us?+S*um3#ed8Gza(&fB}|lZ@ATcd-p;sTwt;n=6T|pl!o+VU z{=bBY)k^cfgo#z^{{M|2aZuv{=eZvqGY|o2e6V$|IS@MqzL%M&eZQrXi%A6w2MP=0 zOmlez|E%Q?*NiJ%sU#sIKev1LLnEbq;vMesc_!WKfA6+wtDW?XyMBLO!-o+H%E#@e z<2)p8ecR70WT_o;U>RH^V=%ofUbv*8{|Sn?tyqmIypM?=l(+|#Y2@WE?%gX?O?^0! zKd|WN6Hm=s+=|6Z{|I_1K zzrIrmbde|-h&Qy`%HI3+v0pDLH$(Hvqr^`KLT(SImSx7y0Djcag;nc_vv<(ZaUJ{F zV)f?_|MQKUdl6}O*edK^NEW!4Ir<9$oopIRyxJ(l&Wt&D{t?D;6rk-FNEl=qDh=az zVaMG~!d+%w-NB05*J~Jk&(SMqbEl%YqGgex!i7h9tRB=&j&TSd#Q?A;A|Ea;hOppF zI^VmQlyhatwOHm^2wEsrJ%kp7-1)EV)uHA)rJrHlpo;E;XQ^J!TxB;rSOU0ndY&sN zvIv+eqb5Idc-;YjIg1|>HHG`j6iBkCQa^<*WGq@kR!vMjeMUY(c-8zQA?r=wA=k>e zn|T1dFAH?Zk75rzgg)813g@~k!loprpqnb+SdG`{Qr)?X>vMu3r)q~7rc08FGjF}h z)uqDmYTf}DW$>jDB)KB^q?1QK;%&#dt}G7#f^(8!!h$J%OrSp3Zz~Ee>%eeY1-HV& zcPlJ9zdv8M8@kva8~JO!pSnx#APPg2#4DZ^D4GK=!WVCP76`{c5;%7*@y(${?~NBa z$n{w^#=a=-DCqWnaHA8smlo4bo z|4!exL&K-G%)j(s2CoO_3s!G>t4(<{fbSJ1kf?k-+@>W8r0biAYNdwOmU^?zCTf(p z?ouCFy}6GJ)gT$+WYSQeW&T7dmXd1!|xc=ic&HT8FtDe2tQMVK5 z>he%T%!rrEr9M%^1oA*EuEsTv>1Wib%0fbmO)g->HEM;E{`3`jHO|Q67nG65_t9aH z(DlXac>s}OOlo4HA$q~DbLYFjb~;R{L{BQ~7Y~WWA9OYK3>r6|HNOvC^Jpe*_WEAY zsc(E<^7dn}sZcTSRfY3W9SE7l1-P7PcG3p>^kvd1IyYz375ko@zAG{F|fEjZfKrMt<#zU>80ctBYS zOKPC**)?86vHE?lPTx4!3;`~$eg@2i{-Yu0&sni^KQIskr)S=iar_&!pIasDIq$*E ze1-+TI)-4UBGDgur+VwL(5Il!(ckWV%l{HD0e!^#Mep^^nn)R0=##dwmN(ZXQhtYvOB<7(h=%T?WzNSA<$`7l-2M4->uHnhgR;(E*zA`@c6iO7#g_ zjqY0W__+6w5EaIHx6@7Eh_sz2|81KpHLAtXq2E2O8;g9H_ zt&0CW;>Lo|S38YF{Fo(EnKS-~t6i$H>3(Ff2r{{CFu60tVRM(K?-Y7Rik3+!{MX;# zn+8!m=))?OtF9I0%x&2(MD!jQ)tP*LdT<~> z%hF_w&Toyd4O8l0&PI*}1W?(1*dz+uW?pvW7|{0u1EgwHnAMyW)}zmblbD#aiCKW? zJl`8wv(P?UI6c#1)lmO*C2XJB?a{ZzJ1umlYSD_(g)R&9V+~cp7P>vZ=_y34p1yux zCBvq(gW9vV1@vXL<@TMTpy0>idKh4a|&MBo!_tfAR{i zKtWn=neueeVlPjeT8geo+IPW|3M|4lH_4x6#H;%Zd9_zSi6}5zq+fLe%15*4*qR4d zo}vJXz*X{~T2Zhn+;WJnOmEcp$}njzWwr{9Q4A0Bg;*mU^IqX#Eo8MS< z%;H><%rJ%HAGez%Wz=6mB6T&hg|h|x7-FSnd;6NJie1)?Is>$2fxc=t@Mg5k$;>pK z-?0m`V|wW$uRT~#%%pvO_xglR8eG}8;!8!Zg1RVDK1f8aqZ&?p=(XV))|E0{{U=9E zi>>KbPvWz3gf<@3V}o2xjQq)fJ6QmBpu)_vyvcN`gjp0%rnWx>2A^53o z3S*Y)|9GG{tLf$q-NN@NNowQjs&93BxjrWk8M27vCK+BZ8g6Bc6h^}Fm{LZgD1g!8 zBD1E>u70pFlN&TE5FLTYC?`cSFeviX`z#=Asjy3Y?z-pL*M!7(SAzLXu6e$#sagcp z)Hl}s$oxjZ{jzrxq!TIl`vhKt(kLE)<~WymT>&Jk1~QAbU>nz(rPFKTr!Odz`Fr{*2a9TK!%5EylZ-(S&~OYo&Y4&%|fy!?0CS*tV_ux?#8w?m5R zG_ZGKAZ*BIk^SS*h^2yZc<}LPJQ!nQNAxCahFC=F^YT6IQ+_rP|L0U9F9x~UiHRaP z6z3z_KNu90*OagMefSIs`UBTDXql|+i?ttDADhJRubvE83z9C-k()$PZFZ$Za+hvZ zgtc+7jyY5~RRb{W6lMMp7{047Q)-MKh=r4O!I0=?prQrdf;?ZpwA}sy>=HQ|5{c|8 zLY&lA>qqznH&)o~8S1=qMugB;rQVF_*{e9-;5&eF#XzB#xI=f{f$2pwkLH#X5{w^^4A{5 zN=a5~Y}{hC+n-M?m~5jX8pI3l7{yQ?_T?~v8=a15yF&U-qa+~oSZ63 zK4J9Ce#PaRqEB8f1&hB#_A#d|`9$iF?Dbj|`U^RC@W%E1PKVm94(<|2i6< z_27PSWi#FQr%=0ng;R9Wb?b)TcJkL*d0KQYf^S8k(eO`9MdMAAm#BP4}HT^om9Geev zJ{h=S!#Wd~hj~wpo#$F--bxNIfnFYHsrTnlCSl%v9Uhv$4gA-jA|_!1U-c+9{P`{F z9+>yHqFov5&AS&S;9sAKbU#!4t7J(`XO~Fqi}Z!N*(KKd;lIp=EV$%)#lUB=8+W&%7+HYP7=uxmcix65#7=gO!1`T1IU=&{aJ-Ckf} z3l@M?DbQpATuQ_^$AtlqG!_L;&gp_>^3Z@g+v4?Rqi`3L=n5E5vP=?KXL8#ZHZTmV zwRHFZ9`edWM<5j$y4x^}jN68-Iz4Wsd$K3Lo~+@k5JlgKMoiLl;1Ab1V?*blss|qN zxAR7GPbR(ldSW1wZOORVwn4|MNW>{psQoSn;x7zT$?RL6C2rWjGIIHXlrkHaAcJq` z8Nn5V=2`>g@GASF5pMcK8-Li{$uGyywYE2Wfj|%ql(ZukjdP(z2+uby2aCPis8Jzr z(UF&FZ3w}0N>#Z6Gwu8WnvEJMovFj^j{XccZVn&PE4PNGG0~8b8&!kuW*Y|<-@CI$ zKpm2Oaa7)PcJXewvB9E22l=ri2^_5(d2DBsDs017$&#b)^`WFJ1&M{G3e)6EJKp3# zR~%E+3IIm(NdKc?aC<|hB*xu#MKN1uAG83B!y?RU%c1=~WAA9yGsX2^WzQ`;A_&w) zdX1Ze8XZQ=G7#?+5Vspx<(DS1+$GIG6PZ0bnyvRJ)T)3iT@_pssCDMoR0xj#YE(E{ zZiGK*I5E;*YFrM_E@WaPpeGm+- zX23_UB<8;;Rs)`=GCVa7 z0e>vPF_=~xae_+bc`p_gEFDG44@|qR%(|8c1Hza!FUeoK?SLO)(}j)`jLR(vUx))x@wzoyu%XWyLFu>O7xdr?0@2bZZeO#h;2;`A``}#{yV*)+KijxR0w+eorEBj>6LD^r>)d%q$kOQO%Wx&vgkK96 zckwLiU&f+RBKTrDIzYdaP**~w-t5N5S>Hoe7Tlg$C@$dkE)7(^)-wP~_oyNve`+ll#iFzj3Jc|c^ZC)R2-aaX#;Y3!q9w-I#vT&ho zjDCgPo(?0upzre%(s3Eky#`;Mhq^h~<}V=-f}D;OO}F^(NAJwOvyhckGdIy)1CUMA zG^po@KGjc>ulApw0^W{9iBr1$%MRWOU`}B^ZatQ6eyjH$hE}?IC5givU6AQLA%&K7lU@>=&P|JZ}QPRCt<{mFNlpEP;x6 zxIh^;4gkU--p}jGj|W3=3XAJ_nr3dP=z&B0;F$)LJR#Loz4J06 zQCVt3@`6o=#TLhl2jD246g*@)778G11~p~B{d^h^cEW4GsbfJmb<{{Mj>pgUKA};G zXr{NIY28f8pnbZ3Y|{2lN0@af;{dqh3cMt2&*x`%Bvj5OLhs)JgsFNzAiRryBAew8j_Kj{|fi#p-5(9)s6 z2BpH#=9l;-WXzSAJ6I$@B=LX+Cx4gifE)?^`s>4m;u4V;4NRSe+gaW8hiPeD17)0~ zjK1lL_B=iBi3UL6>Vp@(^GR0SVjPVw_7<45Konl5Cvo2l+lrL z{3bW`qr=I{xsWH*Cn^A^OTpV#8L7vmf1y8=VR_ciSCG@xC2V1p$pk(pl0sFZX_n-# z3wD5hO>^UbGuAhVmDNnObb2xzATSe9n^qH(jdVhSWj@_NJ9Z@Z2^DE9wl%pTw>-1$`l%>wake594HN(I zg4u#G`Bag%V6$Wmae<1sAtVUlN#lW}taNSSiD6949gOFs$NRQ52Wpzebur}4v0JY) z_azG{cr8R?u3NqRK*cGoLl~nH;pI48pWyxebfIgh<%pJLj-lu2N$6S2E&Uh?`COzH zHm!~2cb*$pNCm~6p>Of+u8ViPcU35aG%#Kb-ypoy9KrjXPujQCzE<4`qigZ_Su_MU zcGdW+>An)5&8f(^JC?ndL)>7AK+SJ3bJKgb&&CTGmX)=AG-6LPz;W`*OA&D=#G0!4 z4i3jvg;x23HovNR0PwXPD6@H$K?wob0lgKn8q?|PlAgGsuE4~4=54luZ}_kqy>`7& zH!uVPv*f6BId`7=(;n zmob;7;}AA5n#}Q|K47plQ=M!MSW`*uOi)*Q742&i@8dqut>dl2Fk@;dlUcg3%P3s- zm2+4o8MzOBQj$29)CH$}qhS$R<^xFW>X9&3k&3cQgdBWKS!q5jnc74V^&^%#(A~G% zE4Jq<7%4sqmD2s&M$BTD>gO#x@PYtTembvXW&BffB=b=YdaXhLx4i@uzV33HlR;r! zLI8xEkX707!B2?X+s$qId{JglDx*psKzjMpN>~StdvU!=2|jb8q}x9 zDs#a40&RS=Z2_H0yOxyE-szFWM>sJS8!86`I&~USX^Fm^pHIUP&_23O{khD^B6)(()nX=@a8SgcI zs_9R&KZ0KGAM(w#zc9|^W>%Y!c1E*Eu+X54x+Z9pm6`3K-mn+#jnv2-#5Sxh%6NqH z*}Zx)2$r8pWu#FtW7*BNwEs;}VQ&!V1+C{WM@hH32I3Y%6OC`JVN5pWoGBBp#hR+wN3jg2x3Y zh|l-fb)2am6-A=XxfhHX&9HWMa2k9?gPc*jyc%+z64&#SrOMuNTjqMIB01qqcyB6;D$E^dQcLdh zR!5$Cj*O@Jxg_p?JqN*8EFqbw({w*5p7qDA9WhU;LLqG!qj}@uzGgaWu-Kwt%TpR9 zj|b?sfRor9T0?fr0X-YBcav>bHe5!YZM6o|hNlMXpzCtRU6TiKwDG0atRPneV)-IDSsvni=xn6LAjT%GVGW@# zh%NFS!bJKAo)CYazu(oF2tZlCgwOJ<>oK@v^*wUI_s6Rb@k!rDt-d|k-1d)Tx7oGq z+lNJxa>wpX#V>t&veye&y9@em+Q|r)4jG^$AmMVhgi4(E>``g?(in+U?mjTrYGWRQ zySgu=vk=OjO=U1%laVr_QyM+QcJSto9*C0ac-lmItJ`r&o0d9;@fN2>a!}Mz3wJ-Z z`vm_zBJR6in5pB5!8~+7TF7clj6M6oHCO>4V2n<);IMi6Q@+ng1hGvkWpa=O?7`8Z zm1V@?pc5HK>PwbXE~|&4#NU1$7zQ8O)-wa8rAT?)rF9j0FKMvt(@`nJzu+bF)@Ot^ zOn|XQ|KYm#%VFcqM1*e9l@pY#%bH3>1p85>^`0?=$>kw4$(c9ubm1Ms&|_NxgZdjr zQbsR}AaaR^lKN41D-g3ZWZy+4=E#e)Kyuc<^P2G0bM~>xOmA*wp@n^wJad{!@KojG4ErT)nn%rWEDkUx??jxA1+e^Ao$!OtiLRnSXn97Zv zKj8;vqFBkHs#IB>#WVe@ZG7j=`b%GeN*HeFKEU-pE+c5}W*#$t_vNU`Ppv<;zpjbA zA4oPpF^H_bw{er_N{Qnu-bo~x&C+EsbRo6e$(tgH^!S4i1lC9RiQcn_Z|zA!A=Vg} zHzE3Dhf`H{;IYR*M&rJ`4-MCDJ&5G}E>=I|?IzhvdyALL6$$dO*|dBDxaa67ct_pF zuvdf1DV@AuJtNzDm+-aZ6(MY@*9y7_aFh}rqm0F`swA+TvoW4*pcopEC_Fk1l5rM!}1A$+t z?1s6pj@>s^{VTGRa4W@B;CCMbc}he-5~4EF8+b1BD{MrS%31M$6ExEI$WlL$ea!do zc+t#-s;{fm=FdkFkHScWJ~*}zC~vJ7F~X@;rmdRpPsOVN8X;{kxi8-Q0S=Nk<-ujW zkhLX@_BPav6{2AkRsYHk2&zCvGyNx m;9uMC??VK}i4Gtm64Kp-i{9m>nH%8W<%EX4cG}nKi5YX6;XJHI(UT*=Q*!DCpn3exXf4 zK?R_oxb*AFWpa+`Cv8UximM8aN=k3vC@FEj^>njybhf3Sc>O8XfZ9-}mnjwa^ZQeW z3{@I9&0WcNzpfioY`iSH@slE>gXM~VuZ>n~b-L9B_IiO#)?*w5S@acQXgLGI8S%?sBarC$xX ze@#lW<7-qDMcGw+h$z7{;{svUKfN_k7k)=hpny zp12p1vlJ3PeW&+3gP@_FcxOiWh-7=?%wXzGkxqf@M>L~%v$!Wd9EZK*t&C*Jwo!S0 zYxyvbM=F}i>#{eu@1n=Wb{@coCx!{PSTp8FN2pqzKQ3H*mP@yf+PjW<*7z}v`F3s@ zN+|Va;coau?iV99s*u!saVN2aK)zlL3jv(D{XU8otXBO2a3ia%-|Yx?X0a+JPQ^#Bn|!xe%@eNY4xMoSVfYw}=;Y5#b_kbu z?LyG}*tfghu*%rMzqtZ8e6RH#Wd}Fq?)$J%)}$Zz*>AJ~uH6n*pe^%z5 zH_G9?^DBV=MGuiH)Y7}nF951fwuEoUMSPv-nw2i|v}}|vQ|UgJSL}e;Cw%|(_R&wC zXzywsRjcb_-&s_yKAc*-Tr0ZsO}S-)OW!{EebBYyr3U#0nvAgBkM3bZZ)zos-rhgs z|5KiRYm-{g^T(mJh`V6kMg;Y|!dni3C7Id_?k%PnTn~4sfHH=2)J@-tJDI5ZY8}*2;FzMv0?U@s!@w$noJpvQtW6N>fkw{>f-I z!l5PUZW0fZjm_$4;_kPW+{$p1?jPRvy!Ol#zp>0GOBnU9Tk!YmRp2$I(+lqqy{)aS zS#xz|F%(-z?_lRYu*vk4O)|eOJPG+Ac!Rk|+GOhbyv}p_Sw}((O9CZ(EvNnbwZ`{~ zS1wv|({_Z}JfeA(MsrlhqDEQS68!e!pBB$Dwjye1i++qZz$*I5y{jF}G|%sTH@wgK=B4;`p1aTNc<8>1 z|Gr}s?flH?JE`Z!@?*~n8&7w*O<#-dvz>gf%A+j^XUouE=lk_ueAxXlXUa#rbe-Yn zRV=2TOI{WZ&p5~jvcW&hK9wD2UZ=SH$X1P7qVvw9%Ux}mWw(VmZdrSkX%}5A?Ak1Q z=$}$*nPSLwc!PKGadMPw3&)+$yz?f(0t(dTp{3n^i+F!J2tBrqxaW3Bp!K_j=I-4x znx!kO9~?j2d`@L4yYOg1s4+h93ijiR%xl9*>ld({Hdag?4?So+C_O~?ZkLB4)$efs zculw;^pk=Aa|h2#jLK`w>xMVHZw%hZ|7KA6qw-K?P^DFcU*&oB-E7)yQx)D{B?9t# z1T}qL;7@pumJMgXCH-n|fX+Y|}@e26QT5Q8F#R7){WPX|M!@SqJaRpm%rL#SiSLkC^sxt{XI(a8!*<);* zteZ-k)Otw;SpvY=Tp5FE?P@c8!KSKYMN&4Kj>}Ku4CC;rXVra9;7uDJL7zAoWpGqr zHsJ;yxJk+O_}eg>7F!*g>SJF=lOg|{HzQReS4Wl|nLw`Na#<+#c8Ar+Oy5+2PtmVr zRWZYNos2-`)fH8Lstz5hL3j|zslkcYNprGfOrc~ZKWxKrqs+(CJ8t97P+k7Po5|Pl z`F)e7MZnx3y(9SS75NbPCbRrmGdO!-8g7H|jDR8t;@ELqEc2IrFN<9sy$WDUmRx7g zkZcROopcu_HdR@Unf{o#5N@+#15qf#m#Jk+iE92 z*e!zYJ~FcP99fn&g%18+8=4#eu9I>iax!vmjtb{&=cEl34czSi+<%xP|0p*>qJMP2 zAW;f>1WYjeTAg641@f5u>D*h^Uu9yAayps)GbPaLl%W?kE&-AU)%axi6yw3{1>e#J zbfHdAOw9{mgiFo#WwfAgUJBGS$25D!HfU^i#>}F*waNOv9DsTU@6q+iLEk}>HI+4t zh;F<^e4WTQky4Xw(~}Z*Q@4iU>8^TVZ;&r>b7w)sx6})ZFWY^M&)cu<9I|>L5%*{? z=%yT{+*%;#!Rh+SPAt}^1taQ4jdjMF&LX-DqZgu4fE;vpf8!;5ob!*yC3a2^1hdZH=Dzmi&Ti^TXVQQS<43B9OL3QLs2wjqxKzv# z$l}bDe8Yr&nro3sTw+<5kLR(R_sezRnj6r*63e5=fDZ%LKr{w4(RAiqXMAlU&+iOB zX5wH6_^u#{KjMD0^+fPp?~F2L#dA-|1Ulh{S*+B&)olA?ki+GbR=_0(kC-LjPWTsw zkLFAvqOA{~{Txcpi+jRT{%{YhZJ`~Yy;iHWWEb7=%lW&|YgaX%?E5*+nyf5bk0JQu zR2tV?Di5cV9ppU-9dt2QAFim$C>Xhj+wZsPK6ldIEEy9`=F^b0=l+(WA3K4t^_X-w z-#rZ|^oIv?L8Q+e>oo%d2$fgUzQ(e`6CsI=iQtQI7VvpJeTRvhq9c?RO16U9>!;`8 zs=!LlD(4L++RAWYov*oSX(K!p%&{M3as$;(<*Uu_h|H!&F01#0`X@uV_FZnW_lo{3 zxi}a4CzN6IW-do^iJ0jVxJ$&$*G`F4rpwG{fLm$3DJZ!79lO2qRc;qX1UOTiWz?!MNO!-u9b~`r6dT>TejK31RaiTktA@J@!IuP~p`= zCIk3f3)%6O8{T1{N06Zmx3=UH5i6b}_ZhkKAD5e6YIjc@j5ZISsN#w24R}(VB zGd)}<;cKt@O<8?Fq}{a7Y1sIU3wDR5p>&I8dem_3EzGubaJB+$k!CY@K*}e&ab8gz z3jXdlHslr^^j$}2S|E3wm$k4uO*369}cj|g- zk?y@DZ7>vwUYMJKC{FLpnuVa2JeL>|g6bUBG8!iMhCqVxe6`mkYEpFOW8<)g#KP(} z@RqC3?CK1}mv?Qjuz|`?aSrO2=H85ZPYC-Q8NpC^UPiKZ%MDuFD`_dIQTB-5@$qUf zmm3T^&CWfh!N2L$iOH2jB&Ts;=y%}z!KVkQ%$dw-M3u9#lk@Qj#ggW= z*yHhe%mX*v+?GFvP!qr%IKR*^?Uyze4;eZyJGi%GjmiD3si~+*d`_r1EZm;Vaa)+% zVOYI(?ei0>s4X3eFh+Gv>UsP%io9Nmq4|@q;%1Pu7{(V6yI7vxw6fE$e1ba^W3AYu z1chTCFS>VN^!y&(W}!L-cM|Q>M&75UCJ~q+BX!@BUrcx9sGXxulj5M4&(wdUNr!wS zu5rHBGi^nXnklj?f`!29{0Oo`@!8h!jh(tWg#bByh2qi$HVP_o>H_&vxWN9O=~ox{ zC@BAtf02SB(vjlQzv^g^pMU+nldr#M{`sW*89_l!{&$mn`KMp}kJ?m#bjtroU;0HZ zqj;vH^yUrusblSFYwPOe;O5OB=y*oXpmBd~@*nNIu=TR`baeN2baUnYt6fVgHy>~L zyLbQU=s$n|d{0|{$N%cd)$3owA`ejXuM$x)kw>Ec**2L}?yp>#w~qd{&PFdBUC47r z?n6OLOiEnt9|ZrS=)b!BH&VlYBbAVN{O_dyR`lOV^}K97mE2s&oq8+$*Mj|v_}>fv zMJOlw*VzAUF8(>u|HviJv;wW1=zp%70`2TYI0Jbh?>N5D)FnU3Z1&ghq6+!O_s=Ie zeStD5=tas11;tZ}H!q&)`d`?Zp-yJ+Jn7n-)P>HvIG1gFV7uuga6iNE%JnN5@e~>H zZ=Yj7%-&RS$cVX^aiqaVlfkTGQr3DqtQ9$bnb3?lR>X|Ys_@QiXpC$~1%LfrTsddD zp@A3}#0mI|(O$g5{f^>)-iBUWgkJ$b*PXTWE?s|0ap8a5u3WqXHM#P?5POpg)&2hD zJ^G&w|JB)bXHWXS?e*Q$QJU+{fEoU*w!d%rX<4u2-@Pt5Po4K^8Ol5NEA73%uh9B% z@^8lZSKY@c6s>lnDdGZO|Golwpp<|AIz!$U8L6SScr~i_mb%d&Xl^{tbY(~qa|D{u} zOLl2sU#NX*K>gOcDSCk7*|9Z|fGJg?T1~LN26d@HId`$zZ&lEiTmFw_`^Qo`XuR)@ z;YHVaEIDr>%NHmJxD9JMp|Yp(chpc85;bGGw?@jmt5 zBgc{Z#N`;x=kal-2>p=bT^$2lQEy~J^MT2^ggQFu2F1~0$aJ} z%@9ef7nzn72--Y6s9Q^I=wtZB@xQPK;wFoPom#ChT+rAh?kYg>c%E`pevd>v%GDf5 z^^x~@{I^r){pH>M%ayNemCDlA!m7O%2k@iW%I6stVVCI(R?d)2f6t{=-&=MU+KUZJ z%>eH6ElHd{y%svMx;)6Dd?|LNza3TfrFsGT`$*^fzK>ylJ89`bR965ElTbL%-}2~F z`b$>;pm#kF|8~bc{Xgpeo3($Nf&b%{ztcl+z?^hY`?O~jo-hg-BVjGkRBm6@@I zSIFi>UJP03rp%*+|GV%t)UYssY4ce)z8x7ij5jv0> z>?;kp^yi9N!lTw}Pv$Am{!8CnV$=crmG>*jqWAs&Mc8a0$bsH-Hek&NI}7gG6UzGIK`dXrPD8T<`aTlm}hjH zo;^IigVJkAfx;py2F25|0R`fgdXP8mBh4oZAFey;2VSo~j3%2fD$MqEY_#ir=_>ai zJg2xXy}sL~TqrOi*zVRt2=osAssyfr9`Yo5yXhnp&|M^Pe>6Zl@##G(DDAedVnErz zBd{p3n=2lMvQKHM2wS&Qjim)d!)@~9_u7~b3w@Z&;1E=EB!`bRmqvbEQtTy8TEKyE zs9uGub)k2YuaN;&ljo-k9@Rmw-AduD7ex+pkE=yo%-BTp^AA&&;ZC9M7Gf z*$b97dL}^n#k!H7cvF#fiTWwK15$Iy9H^_4-yGQQn5PI+MYe_1+r{kqSsG6F`&>%u zRx48cW>J5cVLACh_YAzr(^yq*-8~f@a=KKY(BLgR8<)YrtCnkD+`hp|9|l`{4FbYQCq3EE z=?0Cl&6Q~Oqj4!;Tv}wrKyn2l697umKMF#H`d-&oxxFQa`k2NU2I4DCN5BthG zx;vByjOO=E3BlRB^Yb!IPcwDPPP&_$?2qK|iQDz4(3JVvjXihRSbutM1AR-e+NT0F@ zqnD}1+oypi^notbW%Qu~n|w3?)gTU^F|8b?7P7$z3pppwWiYyLWV5>sa-Ho`%i_m$ zlsZcqc4W{H;x>EfPj$n4p#wYej&l&Ai$C_;9cl5Z&q#8U^{|VkWw7BaWpqWK2q-Oh z{}V^R@{e9(6`il%fx5t0kzU`KGH1a3=}fZ&m5Nrt#czb{x3E-fV4DSTKIT)+avOMS zc~01lu3vr|u!3k#w(2&O={JJ}B@4rlMZT?;s*k6OHwu=RBAeLy-DHDKuD8`lU+_s9 znrU6lLWH>TngIHlHs;VJGCQ&^QGutEa|y-^0xhoN+5*U5-S*bJFq{hkU;A+X;+q-q znIHP$sNLW9E||$*b*e*bA(FJwC94e10c&|=E+2!iy96F4@#{4oM#1#a{MF}nX=Xe1 z_mM?2sBOL7pRl?T743sVJ%l4$fNRz_z@Oq!2VA!POJQS`nfep51V9%QmaxU$Lteeg zcwWj&1$ApW$>H$mxUql?Ie^X~$&;^FuH^g71=PuCP}LhlI;;hiO1hN(_(R?jemaKv znZT^(gOnO`ZVl;`+p>i9takD9(=|nxFTyof5YWAldLMizvpG)mg+xLWJT==WOdZaZ zqn2nTSX`y8Gedjl{5boJZ`>!snEU91=fQv!x(4kEk{EoF1t;&II_$zL*+Juoq&?&n zsjL^1@^&E^Zofs*A@BpGhCjgL{Fkfrxp|D{=oXB%TW{Txm|Md0q0u8?zrHyns-%6h zC5{G=n^Dc|tO2a>NsFZe01dJglv5{-D<#sRQ3zuWJ!Pd4!bk7m(tvKW-ibc}@o2lC zXXX!xt{-eG-rE#M7Z$?iu0+I6@PKYif?|&xYPHFzz*I{fZ(!w}y4<1}_8aN&7M!~gaoQR@$24F=rIHP1v}^! z?p&J11@fnx1tjK^sIlwv@+6i>C(IF9{2q>ZD7Tmzi-Hu@6AvEi;) zqmTg>4KR4OwX8)$xbtxyjW4!Yu@W{*oFeqgr3hE93)K`K1=Y34b=H)N#N^)If?(MX z7tVuSvtDUFPKkcNoZ2~IYV8Vi-7{LU>b1_d(COZr`o4E~pto96t?x*rmcwfik{j0R zN4eGLj#5yNeYiB7VJaX7q>VmHYvyHQAL{m<{p~XOMzn5Jk@Ku08Wb3#D(H(Zex#EY zCO`%VAPPD0Z!}E8H7hJl>gG$@)B!jvnOW^HzWl|#W7~o*wZQ(k_0-fKWNqyES>J7;;P)O?mn1qi_^$&3KxprT}kRa7DhGHKUK; z4Mjh;Jr>5dakRXG8xob%1lpI3uG&tb-$|ax*TT1M51tYjN+ZLX2*=nDE#<%7T}*?? z*=%9%Wp}si7J-zJrkW78i&at)1)_ube#fU>MBeEEBERm5=c!Z;g4HdN;HUdBD=vff zvR>Ri_qHzfmX{d=!YXmO__=7~3Ao|llVCOotw6LNt-Z0k&ZT za(k8E;@irE@+&lzlsY~Bwv)isQ2^t_b|szyy=XrgimcU`T@u88h20Y2vP**~NcZc1 zWQtaPznMra*2wdA6Ldp z2$($c9>{a`_%0ldvnNOoZIfi|#_b=5lq?O_ywgVprKJgStPOL%h8cfKixeB`nO!V; z=d#zy%y7ZPB*!?bHoDf#%z=4Ll4yNsINE(nxH4?qc3?@y*lcxUmw)g}G>1H|2kBKX z8HDMm967+eC^L%mGU>C>fmMUjGpU#i)0KoLn+ThQ?!XFTP^`dIcO)0Y;b2hK&6L_n ztJ#IjfYIuTU?N9lJRN}XVer|>T5HjmSUglDnrx#Y35VOM8bhFQD=&on(bcdWho=HY z`ip$j^jES#YLr)fY#v=Vm1W}zm@xVh;xQ<(^T$Fb?Am~MpAdlD@Y=RaVh&+Y@%)sL zcwdlki+(qIBFtl`6xZ0e#z7BgJ|ketMB!Lj?fqF*(_t$K2q2cMH+~RuCpR3*P^?OD zUhkA1b7E;ta>&_<@BDk}sk>iD^@6%M%; zJ>_F@g=YNj!5pVa%0gHJ-@1%!n?4(pHhm;pG0@VPD(2S^rf8r?r zw2-!We9!G$eNJSglVsjRQ6)kge>C6;%Zx8jy$Yx{rTR#8mEgYAV@^m{^HapN+(K%&l|cUGJszGsp4$qx5ZN+R3%JNVrplj(a|Z1QS&! z@D3p9z*mT9bdE02=LqrG>Js06ZwoX?q zr{>NMVWF@Vet(MFHO2TzviBMat5unJqxi$}@nnq(BdU8v1dbaiLsD}Ee^&$gFNDWf zCz45jZcLhZspqTLB<@`BZEcXaf=0PEY)|zgpzfGr7!w4S#r0tFBX!IIn=GcNl2)e1 zbA{|p&`!ore44C}yi0{0t(VxMyf!WE(pT-ne7t z-b!UTQ_E$_K)6#&mVaK8Z#09+!w7vDEE+HuMa7jYl;BwB4?fOY-x!|)H}yV*-tSI7 zN<6nR*cG|>6Y&uID&QTkmh5g$Ib)K&2HyxZA|?R`OCJo(0v;H-&Vi4F{pq@weHxo= zSn0vsskKhS*7jFP40prnL>tS>`o;V1V`s@ z9~g|t3Ew)8H7V1O;569Ay+t0k#LMc2Nt_eR3`3yz%mlx(%GM14N*ajmk3ul* z=6J;W#k9`ZHi>>B%FUeb>#@W0iXE{KPHp5ytrYFi8FAFcI5;z)pcIz;C8jOwuy})T zbQ0f*Pmnu7w_PT}Z?Dem9j|BNaf`cjv-AfVa8G||0E*^3XsPcljC6La->Nj2iQ;s9 z=;M!VckGtHk@81}i7Wk+)^BS7fk>5p>_`CM5p#cjVIPhh&Hoc^no zWivj90p+`!vP+AL3tB|f=22Fw`E}`w6r0B`CwwIBeOuPsowBB`)EOhv&ZkWK353sd*r&IrY)@+E8(H@XZ`9 z??ZHNMWE;O1EqExXop4bF&K7h?&_#p!2D9?M&RE1fezd@7o@(jg~@3Wl~ZFVT{<_r zBUjx7Jthi^u&?29%A_cEMe0KESp9h1mO_DPGiw}zGs|p^bY{G1xST{{jfD+(DlVzG zG$iKON7qgGU^llX>zkZQgZ_EWBII=@bIV!a*Pa3sI=J_cod$4wR5*>s-iQLB5U%WUv^)0-Eg zw=Um4>jb+r?scJ1y%^q9|GP#S3x%K0J9!eBU_ZdPxaMBKRL~-+`9AvK?DUUF359(( zho4#;V#lDAaXqkbe0rO&u^-5*F{;pVaB#RUNy2gn~e1b5o zBU$!h<4rV)(#zkKPU8uSqG=wo4Dz_Q&z=R>K=!KJc33KWA0l-vO#UeGSWL7L1JpNv zzpK&JqF})kfb54~GOL?)Z)|iQcYnkk%2=-#Gtv<#)dIOtYqXts<*dU$A!i3^WgOBQ z;8X)#9qakKpEH`l#cuyigxu5NKn7S}DB@L2d3V^$$GSmHTE?NffMYYHSYH!XpJ8iVllfC7<<1qrQLz9*yb;9bsO4AOz z@#{4+TBR!yMYGHfWqAOkkdHHMYPE29MENPcB)|2PS}Gc>fXRC~c;Iy%&RTs@=V2+R7gd2yj^v!F#19yQCGRC1&)7`J6_b?Pj1gd9c; zYypBY{SH}Vo%D7Zy}h9W%PBrZJ~Iu0#Oe}JG=}p68H4f=nNqb69XPgqBMiBH_t@pn zg6JFChl_I6-1`^wKOZJu15_u6eqV_LmZT49KC`c9EH)PFa&c4)Az{qvoTT%wzj>zN zypr!?6ZT-cim5dxw5yhD+?IS$%zy0Au;71ARw5DlO65_3Ld?!xE4Y^?!X>wb^6=Rc zpAK$!*^(`!CGo%D%PWI6C3wc7u7JS)n&~aIvJF`3W#GwXX`qK<&{ny!IC#0|R8$_d zG=#aODJO=CcpmNfQrL0nY|P4_Sd`?8au0SIXIa}ww)rxso?IGd>hvRb!gp5kse^Q( zerQr@oV{3)F2suP{q+qdbnlKni_|1I;X81001kZ z$%r*TZZfOHi%P}meE`xj^1)yE`z!jKs^;df(f%0%Ko^Lsj0N@bT~G?xv;2%0*|C zv>7zdC|+_K7IXz-*PaNYwHqrV*j;NawL`!PW?=kyhGf|p-;c8`y)Rnhr6wp)DZH%) z7x2qKfrJ}39IN9zeAtudzHE-v?$3k&d41VuH@ zV3z}7TNk%ZtGwh%ICZT0$*kW+(BDsR9nQBFglYJqUR65$A}oV%^iK$rY;&}l?^GB#Vf%Co>{1gxP0}GFYugsgf z64TwM^xEm15_1W~Fzi@dhGz3LqzrWf8b}2letp4Sh_=vY$9p;;d!jYRSk)<{S@C?v z#?Ps)z%fABW9hlF>0_V7X89-E4AbM6pklvuub4Xy`}}S(@Ch0MmUP|B&0ldVjB7*; z;_Ph+0m_O}c%n<7tbBE+rCe8(Om=B&;{>!C=^Dr-r~LHdp2SnEDyfgy?k@QHhV#UN zkRJeP;x!;i=cE@e?38`G*<^zbQcspc)wf5O#~L$RL|m0%F;N_{H3L?&-G(Fen4meL z85DRIbi+A^vZPTooc7H7i+K?!h`gcVoQJo6kWpm@^xNZQ@I5BspVyqnsGgc(?pHtk z;?pzhiA=PRF#SP>D#THdN6!*d_4d4~S@h%?-D4ZRnQ(pG^B;4P!un2<@qTHctf6}m zG39sb)eSJ139MWxnQwSdO&4RFwRTyK(&&#DD>in;gHm$WHYSc5cMT>X(~+EvNgIKG zm}We=;{U?2zt~~Ibh=AVXhRN+RW=WIYtPPzSo11KaEAV8<5!dB`JNp&l<%X5i`P+i z*VaogMy>kwe8HdbJ)DtoqvaSlXZ(g*S^h!l zH&v!_v+moxZJ(|oB8~cH?$HY~E##f(XQqO`1Y``#AFO_pnoVd(ID2;smhG+xtfHTZ zDzWvZha94+ME#9xB)SXmql(Hn$N&=<@;l6ol)+cJ&cL_#Z1nM6o zxQ3(5UMG=ILmb_cABlWNUwjW$y-qz7&-U+xctjtN(H^~AKjmP?HfE|v8$chyVz)Yr zgDg$nhvh9H9SEzAGKRY>3G;+_^<${2g~qGo3$FkWwSi8m&7Q>Q z1A#hQ<6^O;VpPFW4mDi+?R*}g7=APctb{8LeRx#bX{YSM#*frB6qb@-jWwo*?oAw; zYQvD-2j5+Jpv9_1xfk~h3upk0Nt;iO6f3v6zAYW@bC&K70WtXxa#gkp=Hnyb#vbod zVKb?Zgo!6CioB!I#yMFJ8b~;MFJ&0&BgUsIZ)~rpK(Ger6r$$F@m{-Gb^|>3IozW1 znmt^C%w%+Uc$~pV#v6K zCHY_re*gX;9q;%b8BU;?1JbMvWu%lcS9{*5Sh4=pEN~;+T^&;&MLK^-oe-D_c~LEl zZ%ty(0S$Y<*590}U;Hfilax(N+}KU^MJi$BT#Z8pRXl4^07WZ~$;@2Q4K_~AiB2ST)oJ|)VCOFVfhhyYm!zUUyt7f9z*;l!gQ|DcN!$1; z+sd?<4aD#u#LiAJOSS=?YKPe4e#`KFi&zndJqf&{X@B&FJeRH4iss0I>Kf-S7#dd` zQm-y=I?|fsdSgJ)X@&`y!C%B@FeV@fxu75Nbz=l&p#uhWiGJL!eyjU5-L{Vk$|jem z`v$hFqdMmOh8hqc%)C?~R`gc>_LGj0*R6Km0)8aEN6|dPc2E~4hI>a^l$dpYA39&M z48S*BbB`uGAm*#j}5LG@VgKS*J zVqrAe;}r&zlhx9utQ~Fd9op*Ta5`h(-wl`XRE326>ip0^d4sglU-=hN2pv+< zMAhIgoW1{5jeUhQ*-D0qmlW?esmQMtSXyIDj&Eyc#THoTkXX7z_1UteOa*8F<1h=8 z=CODD8!lbL^^A7^kT2E#WPJh3h@CaKx+TNrT)@y%E6Nd4ugm6~Gvl*W{#d0fKd|Gv zjBH?VQ6cAO7)!pWdNn~9FM3FiDm8D$J#4#z9@c@u|2mKo;p65X_p=)QbC%f9Y-lzVqQS}RO@a)4c!Djo>$mbo0?uu~5jmf3rm zY%YM@D#|#F;k*ta$9m8U_u*b_H%u(5Z*C= zI~G))2v0G|D45XVO4=BnToPON*s68(d98_1h%P?gcWs`4FhU&7x2ut3zJKut*+^nD zQUH!LZ3KAe;&eZZh&K!h}eQa$5b?j@*C ztuebK`TS;OPt{Dugzx3=GUYyd;F320c}l{Dq>bGJLv79E*?5p@u(;6{*?;g*`u)_& z;+iP|{nCM~%fCML=@Ignno#=QQyN=$xm}f8BaCNF$?BsH{HW{CV_;c6BjF9!ivc`POc>0tmH--C_N?xKKGo;vw>)#fO3D8VRzN;nmm>xL{U@r zGx_CUa?{#*zjra_(TohWfL6+TGn`G^KS~*2K^{x8=zRSB?zPXbaD!OUNS15Zr0-)* zjE(A+o0yvg6W%KiKNh;Pg!tUf8?MK`bFTfYSPU&FW;oO0rmP9$BlW;oftQ^W{{dd< zWVdxVxm2~XoW-(9oAmKD>T%qD0eXS;=GTjlx&)nay@LXuOU#J`OGAvG6ip@wGpJ#rYd? zEnTN{LCLYiy!zu7z$(l!iNWrd(2_aD0CDeXDID?UU;^CRh`Z%hY*+AQZU@8}^`J6J zW|QyW!pk5FaL1;N+2JQxsyM@iexP#_gVm4zLnf_W2uUCJTcP*j`FV$uo$T>^ zSi2yDhMLdkPU5A<&0s(y&Z50>+&Wngw+?fM&SkRY^s4y^1k!c*1n+*e0b1H^hKT>l z{v3q{s%Nhzop8&n!l)x1A)uz-+^M_PDfbw%2Hb`RO=py;q!{el<+%L2<*Is3MP!GS zPXXANN{5OgO~=_u>nC<@C$_WacH(nDvCH?G2LOpoY3s*w6tto~)3Z{x$+2>T9?Yun zSsRGsOB8>c=fR?Q;8+v30o(K_ze8CsA$Mv22Fh!OjFf91Oe9)Lb6xL}>kw;xImcvZ zap7Ph#x5TRguv>{Cmquo7e5rd0;srke*^iub&~tsjqe0?i|j3>zmNmsWPy zlcW9;V5#&Cd-@ZAx!{9rKO_8sKNwR#F75Ptapq!zpQ5%$EY547!JnoNGYqshXe^67ntcKcsp zek8Brc;WK5T_qnyt#NY_{c$F>(ZcW^A7#LrkWmcoRoNIYBACx&f84NojM8!9db!QqVwY zOonTo7o??z4rK>o44DX0EFwq1MNGw;gRSXKBpcd(R!%($Ia{3fu$LYA?SH!_g$XVjHcQb@(N7k!CBiOSeTxkns&>azPz;VO@{2hXQ@}-;h5)M^>nDgvx z=zIgw)DX?Z2M34ZeskjGND_sd)H&Gf+m{M%@c9gEf~RJQjY*`(?G>i65;vXWq>10i zfkG@MF4g1D2Q(E)xchPfK754C9m@;*u5&Z#NOl`_)2%dziw%DqM*_DP9NI^nBmi zy52<&z8dAMAFM(+eU8pdO;%2!;Czd_bCfXfN84h92aowT{vrJP`tx*z_gsBSYUJRo-2Aq z;iJ*fi%g2pxf&yT2wjGbcz4lH3!N{`yQ@zcdfLU@#8@tBgzJVe zZMlGo<>I$%wVE@BdV%XGGB|gqZGhz~|JSjz>QjwwIIEL;9kn|x#%d3krOxxSO_|Zf zFJ3)2TxTjS3gPB+?R|Wl*6VC{&(dN7I+O9T2JOM%!P~7^zb??NjGd`2OIac3$pCKR zkeBtAGN(-H)v8SBQ`!7pgtN z7jsR-rpeJ>0?FtNx%9IpEa+%}xeBh4LXe$q9yFT0tsO78hm4wa`3^sxc+zm5DE5Z5 z1_ixBqY=XhgA`f6TeC8wgLM#_D-u2&g}B538T5SQ!IWD_oyC=uQ9*rnXD3s)$Ij%) zy?YI$z2U=?cpY-+pS7_q3gvR{LjA!)3LbqdHzfi{hnae8`7PRT3SR%1)<@_c=lY~; zb)3bGw`<;;Z zE;op(kCy3Jfq`&192B_=(K@bMh+vPf{R57wVs`eOD9ii#!0b1WR`8wPb*M=B%0?*! zFW9h=10AU>fRrF^k)AzW(|_5zKr2{uJ(cVr;~}^mfwr2lO{vQRmd~lrau4z;4Byu% zgs|oFts4{$?C@>8t&%;Qyh8kFo@#A6((JGnPDXM2OPHI3G4VdJIK;3w3)|6XPqfqf zHQ00nQwS|1Nbj}`94kySe$y>(b_G_*{YF?wo>H>nDrPnE7N=-KwYnu`M^ zxlGBoc){$($9ferTZBN4Z@Ms}z#zePjW{P~0aR?LNqCf35FI*TExG(|vkDnjOC-lH zgCIdo^t<|BXj|qWSkS=*uh4;%wPNJXnROYf5|C;d8zSBt7}VEuvX)vt=K3WElyU|wSj|RK_*f&=N?za7;yzFLfPW)x7I+mpcT>BGc|;H zrgSbjaP9xG_ttMwZc+cRG=h`?0uqXXfPjjCbSWq;-60JFNO!9!NGKxRIOG75(jW>V zDK)^*J#-EY-#s32&htE8??3RKAI{}j zKf~gxZ{EJZ;0Avo$hplh-B=y|_1R*cZEia|l51ly;+nT?u`Nfa!TVywx_FRWIjgzd zLyZvTQa9z@(uJH{fL6rn@l+?>w|{DJHiOb%&zrq+L3x4KaKy|9lrp>!KvIsnAwDzD zql`uZ@(`-U<@ql!9s2G+l!R5NRxYS-UCy&$AmWuBF{|y8Abz2v4>I^GU2J)AI#1MG zN0MM^Iu5d>oT|BorKJ63#jwwDHwIKgo|fKH{~0u2?~l{Cxph+A`E5<(nO-NI%_rp1 z(ZSa;)PiJ46_CY_ai^$;8m)T1-(OW@8s|24^%F~)Dhusg>{4?I!()7yo z9q8H=$9y4q(X=1vC&7bf^KohN2LN^wc~pg~eY?AA>Or>Yof8+uPJ#5Q=skBuJNlTJBv>?{Zk;IK$$4AhAi>A$?DQ4s7{|P zIHNLqG-KvUr=OeTakB6P)~hJI-OJmW{jHLWky50&nAwIhzvsD?o%E^&Pph-*&!0BD z`yNOxxztaA_WZ(T(Xah2JgAeHy4tXb6rI^mT&q3(Ax~Vf7t{@7 zADW-lyKFLNZeSkRzI15j1zYpVoEp1Vdb4KYy(ug?2=*R@SCm|}TyV$c>=6a)vAyMx z>f&3D#%|H7?&cxAPHVa|E*dq(2Q32q`=2X44hE+toE-_B)C#kY@f>9}Le<|_72_w) z+u5#OYWX5c687ZrdM)zH2EFZjT_IrsS81t-@Rv1F3d($TrPTQC-&eW#Uu{PH>-% zZQ;Eq;G8{KeukwLIqaSiUzHWKiAQ#m*h92A?Puqd^RhQ#Q_I8f(N6OQ$)v)Bp#F!O zTVHglZXmu3u4>P|;qG|!VO@lmhn00^_6G6Zg_AD{5GG;agX-R_c>%?B&7=PDB`Z8K zBH~ejJ6cURQ@pJf5+2W@lmkoM7^+)bHc%gzEqoC+<$B_Q8m0u6j|)~uuq@Wo-g$Gm zpV7WoS3@;=?Oyg68nj*;uWn}>C9`h8`Cgc?BN->S7OLJozwwht#42OO;Tz6iVOTlQ ztqRSd6m7{>*Spy&Ny#n?z9o-7@GliM1}~)U1c!ZVTpjN0Nqceh@Ct<XC-ZIrBB}KC#8*csJ(6I7`GIf2&rg{H3 zy?qQ_H{u^mzy7uzj@bElJFCOEvz%^5Xf=k7`ZB9sjZ`AkKfa^N=)vv#Q*jfa+U+l4 z(IT!r$;s>7pL>n>H+hcs67)rCho?+)q+_sDbIJU!LAtJahy*hqm5U?^TyA55Q^CW zRSOt{#y?9<&2{A!K)i78t3qPCtwkq4-(a63^$o^xr){r?1^aj zYZa449t%4`1}AasuWtfxYVUU3r?CwNIdAf}0Z>cCx&ioD%FUOVM0hoQ8uzJ2&COg~ zZy2zjd2_J#;qHxSmG2*)lui{9bzkCTy`>^tVh8dMu?3E_A-dJhMt2plLdtd70meM) zvJ1ggj zMqv`-@Odrs#7h!m7Fp5^<2Py!ruR~V=;N*wlnW7o{6x%0(JQ+)lPwNJ<(D#8oz!pV zKbiMF^YOO9>W##7r^j{C_XKLlmeiCq>BnS8LnVrA`bt3}r+-I8npi{E{$YcT9pxd{ zA)ehgomW_&W&Msb%ghnS{p1{puUO z4p)2-t}QbQ!i_pj%-e6^PezR^lcMm`V3)0+II`aBH+Md2Y<6$<7<}&O=|OoUjRy6n zGImAcj$Rh1C6`Wz&yNW*oJbi(2I`X3 z)ZYOO1zf@%f9lf+gF==Mwl+{McOtnW1Jn<1OO=!;J&~iK1BY8#FVu!zD7sGS*2MW8 zc4*6|a8TD9o&q?~O7HRxZQWKg+~XGti6D|UdOr3pF)v=9MMNXwV)7hb{N#813hmK@ zJMG4;iBJzWUsoehqJ|%rUUU)NGfv#f5|caW?z#BcBm~=_?oyove;w@pS&7MgpG9w5 zYo)r<(v<#C`-|ERfM7KlCG)o9Bq|E+s{?TWYL+ zNgYlzO7fuyM}&vFb07qbvp$4?BKey`ilgRFQNj~?oX?$|!o1hAT*7Z~KI>D{DBW9X zT~RD3zG|gg2ePp?yJ8A>-cr(v!qAgf>F|BIA7!Lr2k>;)pA`M@xiH|;m8#MxLt*TC znNPwi*cYRY-{o=9_r2e$+@I9T$*9w{rX#`@mLD(XVy4PJglBz8ovT$Dn68f|M$m0* zPrM=?<2alv{iH^>fMs5zH-P#MCpvMjsoc&Pddzn#&v4Vc&O5bmbXA}2itk`oz!w7< zZ`|(WI^?s*J*P!yuig5WZ;mNi-mtQiGHt4rmEjNv#z_N8fHJEr$EC6s%pE>UjRg8&;?uB56I)1 z|M+7nT$DbbVL(8kHFrBsq>EWDMro~#*IO^*FS$ecCj`|=}>ws$pO;tas}Br z?o|JE&b~OYB|q10zp`snI~o)$!xj;0h}PrLC>LaoH}8f^6-eZK3`B7`*~mmYFASO~ z%4J3Q;m@sqFg0Q%B$=j=?#OpK@Ub?z{Hp0-=3HR*F4d-thWv*fKR%lJ#zwJC5mxW7 zU-gaZNmr*-M(AU9xGeaFpGg*ZaU?-1kx&;P<=U*I8zSynS9@t*C$nG7N zaX;8eVZU$jz&-Q>49g>jRY1YF@ue@Qaeelzi;+Ewl7#5vkq4s@eSW;}u3AY>M;*S% zh9A=#+gk@RAp~D0hJJ3)Jj(8Fcy&*rY$5mUkgzUEV1a7!n9QO^?#v`l>So%`qtb^> zqZ&q^^ZOoG5rmoYO~6D06-!vFo|&0>ab@Qp2L-ha57k@|_9)r288%vDs@Y-Z*ctQn zosZCO%`NtzGrsYVnLhaCPE`NTGZr&4pa>Ub*CQt3LAL1S`3i2PL8B?19^hGTF?y2c zeieRbnltzHC%-k4zv_+MhTR;R)Z6_inrSZ@sBz(#u7JVitDI0!?J5Ee?e7c zES-lgc!#KOw*?%s@p!&~?qo#8HKx&fevdy=E_ZDyH<$(S_A{~re18`9pj`((7{XPn z>5#F5pUv54A?|N*w4h;h&z2$KNA)GE9wIm5XFXgdqcnVPqzanVZ|5V-YR~B45!&Qs_7ItQ2opt!V~OA#%nTa8qj{H`O0khcTFDH5bez6 z91^~>_pDuKi(tG6gadooBa6=GduUF$p4}>WR@}`9%k}b;b_H5S4U3Ti*%h_>h z-J2)-B68cNCU*dKz%sC;ZtJ3+UH?Ry&NMU8MO)Jg#G5<}75ry8y)z&^hXZl4F!Gv{ zS{XjQbq8T^OsnCb{yMYDNE279jNLZuau^;(mV}tzt$B4H2FqcuwZtwx$vU5-wu2-R z7+vnb#gKk2vGT-&ag$Wyv!Gh2GuDi=QdsYV9+Vb7yX=vl#;Eep*Q;l9S3qvsmxto! zwkf~e)bn_EHaT=JmP+ZA-Z^J*WG0$8xKgqh9Bma`Pzv!@T^JZ$@3#^xy$_Wdmdf5q zT97H!s^V^f&6gHKS~#*FLcho|WL)F-+5Qx>T&ZS|=8bExiS^>$jMa8VCVu?JI+n3S zi2OEu=j#QI%VnI07-)g0we*NHV7Pf~#H2No{=)qXEJXUs%%d+VriWnR=- z!LrAqwX?gP7DsisT%cbBtZkTPl6*7<)Dmk6r3oh*B7Nh(JYtPZlpYM z@8iqZf0gH=I)ce3z8*y)ph_ZJciAtHyA&A*;2r1;-WCC~#|n1LExGZD{T zI`BXL=LLVtON;}MpBFlf{@Qm6zy(0ny--<#f2R2FDR8iXrd4SyNsUscOZxW+DY2lh zM*QB=6W6~CjD@SM3ReG6Q79<%-=93CnHV=QXr;9|n7(rQNxP%RsF~4|HF{~82x|5>_3hEza;rjqyN1npiSlfr|A}xB$kW4`|Wsh zY;Jkkb@WrJLmzZ!o1vENF> z{!F+zs&m$E-H3Sk+$%k|G zYdgvd?q@L#J%6{DSNE1=eylUUb|biOe#W=rOMcDq%g@^J&D3Tw4X80!1B6$x{7_rZ zbNHBBBWlRpKyWUP+a@ZiF1bpMTLB!dZYq{a{M*i8#diCvj><~eoSn`17h(jDhCIo| zVnI#wz|iG*)<~O|)QGaMVbf+hNR>_bV|Dj~Wmib@5n0LW3$>>jN6dn4bN^s_|;aJkX7=r84tqT`t}{zhb%EbA=xI&Cn5HQ}#(s zU5{VOw$0pFLbjvsXj47@o_#=~L-%V`B*V$gScYU>2jq-Pvz{wrx%_wpn(VWGzXHhs z+8`ZYbU5?3y@ZZp7t(gw6E!%u6v{_{kR%g*3bMWZlzJv_hYdG-6h(_U%7%tegPvRn zmv~s~ibA>1iESOuavxm3RFYJnix*L1Bm3iVB=ewn&=s?S>iwKV=UHk{$(H+m*z6o{ zxX|HFnSW5mpu-6c*-%SnlG!4xD(q^k4jJZ(Ke`&2-+X2@qG0#UB9d?DHw>k2BKtY?xi!ks2TsT!!Sj z-S-F!%0S+Q4BST8M~?(+JNj&`H@g7t4FQkN+}kYn8>OBT1;VY71cDkgS1cG>>paU# zdwlVU`2R69puJW;xl(S6{Z!Knd0%0p0Gsk7(i=LEQSZ(gm&&ngK8?=4r7yhd=}Uje zjfo`XkDX;nwgLy%A`)GGw4l-%jv0k>)8;$vak?TLxpWE*LeWjD6|ziF(aG}s&U=n_b>F<)X>X_0g}WSosM(v1=XG2a zacTKRwL2~r@?uKUE?wSc5^gXUxHD!=^GWE#j?|mNR<2HlnE5OwvT_mc{pSzCBJ2eR zrAx1j_||#bw6^oHTRq&LkY*Uh@-7555U34i0a&8T09S$`^^+tiFu5 zEz>_X@HmsP>+m|mhi}v? z=deUJQ{PKqtkKjQ(Ma8R#5!Xu5)}$cCh&| zdXTRCxDiyRmlY;yKdS}BgGTpUNUTaWevFk@)XZ3NQwFrNbqvk_OwZMfd-yl=S1Dr% zj+o~np$KB$)@4X@qHC#9V-P~rwxVS=+1Dpz!steM+G;go1=O{41qZY%dqWgZK6iJ7 z#|=HFLgCAAdK5|-@;-TF<>L#+{S55e@{^#sKOb(MXw!MJ<9^V8a8^z0>FXO17?1Bl zd72$PJ_pf-_;Fv{c(sJbBQcVQ^j|TzZgnZ+V~u+hG@_o(0eZ1GzQg~Yv|d#cWYQ^X8WIj zn75uKDCciI6rIAwDhyRpXJ&P5O^LZm3JH9p2SM&KMs-2UZrN>MLV5t4y|;-r*vP!C z$8l$8)YaGXXd7-54=XS7M(x|gt4CBiANk>sk@{z;FUS?{b+;CPA00O^d($>v7&8Jsb|tvJ5tci&**AED-6KynPrv z%df61pICN-5P{!L?5#PF9N8+<+Q?-#}4GADS z<8aW)ESJKFwi(zOdqTwx{Mn0qQ^vh>T3^PB0B2!7uXSQn;bUbdafKaGrBNY z*AZ_r+V$g`2;*`+@M#Sa94g_RAA)xHFuZWiLT0SzDiT#?obi^K(#LrE`ribxJW-2Yfi7z z7ubPwRJKl~@6i&r-+1Rz>AoSUX>O-jlwOq#8hp%>{B)I+TKS${D)Nr+{cDf{3n`pBAl2iMl6`-ZY8u&bfIO}# zJe4P8qX9oe*U5EaJah;6?@{X%ULshzE+{GsMU2_u5{SzAkh`lOOW2i z&S0x&@bvXJgm74XzD6m6Wgt)e&CK7d4QRZ%0zzqLQ9RXCZTeqx$pP5Jf4}};2Jt_t z{U8RK*&gL=$F#NsKwC=_e9)kPa!#d>M?1DoO(C*{PCgNgZE| z(e)AqwcfdQ8VK`T04^K#4cb3BF5u;P_!GE}_V4FcR)U_HcFi=uC(tznyg!)tF~7$j z+`?Y~Bz^k#a0@+uKeEsUnHs3Dv+9*Ud`e*<)U9i~*6+H_16j_sTR0okc;^rHIl>I) zYSX)fZqfeRZ}Kn0CgN2q^EK9VoAy0j`x)#ga8rx=>{n&X6T)6yF{U)p&pl;ERn`WM z7nsuFx-lfbS0<(iGD=G==F!HOyE7~-{GAFbCTj_9`zc8#bFc;dh6O^=ldv#Ht?M<8J&6#>Ej*OaaB?p@oL?i9J8ZhR)yYFm^3 z_usVjz`K0~XTli%yqsc-jWkq=GCY%7nMmYvdIgO9O)!P$ETL=|aAoV9|NUzoLNo#G zHyeJHG5i9FYe5?AK`-ORTq?`Ns0;6Qa<5zGo>}h z-Sty?KaS1`|01`A0eI1y>BGN#)|6SmW*!#Pz*DwAMd#04zVPc}Sky%BHZ$qCJQLDE z7WMPhJPlpq+n=;k!%D=bpCtuiFneFI_x$NVAn$uNL;3-nWB$&x$(dAomo)w>zh9oe z3tr|-H*NeEYu{6XS|n{tl%BJ5Kzp}p+n;M)96&zKQ7h)vzt#^Y$_Q+~gyQd_AGSmk zN&-q~yOy11UWKK>0_PHB!*DS(d5xQIVD?rCasRvr%J~8vk=mcjQx>j)L@aD@>`jBvCcZ2{>KPy#(;##>De%&rhE)cJ<&rU6iAnL)o1p z4w5aTzP!tDnsrn91m6Cpdh6fhu)%#=I0%&O)~HL8tEYzzs!9EiY9+A(G2`xJmp-)> zV1f16?hVkK)-w_$Ufo$VzrXzVX~^Kke1z^ek%f_+E_?njbRZ>qUo}&$D%(N268tAx z^{3|mL#Hcp5u9or7}^3G^ts0;w<-bW!cX16x9Gba_Lh|RFw#cPqQqH!i?5(r{F&IP zjEJ#-_XN*=UpRdL$fe3eZpQ}~K-AsIv*>ex@rO`O!KEBdHZXCkWt47$TuSWnI8r%Bi!JkQhnhz-;N!Gf9Y`@95n;1k<5U&1wEp&#Si4*U z&-G$S@DMhqv$PH#)|0|>IN#Z@)BIA3SZjBN*L4-|K4xwizV5pBk>Rm}W3*!}OL~uJ zCqKAmg8X}`nA)XSHN0egh(Pq8T)xl^u|j0_M9LiuO~ff5nUTobGG3gvJL7$73Hbo& zMtY}pF|QF`Cx659~Fd;QpbbyUHaOPQ^*A zeh0f{^ztjio*hYfg5M&+NDRKGB{qBh^ml;OH~mcl+f8;P|0F30(Ipo#(#f@$X_%-n z4$8EdWagQduIrvV9bzr!OartT|M@IN7I0PSt67!QTm|_8+jCGPP|BJ=yI~I4Me>GT z37={Z_)1op7+V9hg3DE^fLEC1bkp(!H68d)hDZ#ASo0#7cU^FmN!V0KB#L{8 z^H0r2Hj&rCa`|1mV{EP3r!VrfD`et$teJ!8wXX+holMWnd=napx9j*I;JhefnLLV` zV<^yN&DAK#F0l(e+KE<&X+N%VvH1x8jS_*=)}9=Dx;F}#H;2(>cvrc26t5&XC1d@kP@`ek^?UHnBycg^pZDpit1|+7U%a1n=SlQVlI}!yR zKkiI&81*|Ys=IVoN>pUeq-Oa3iWw^xSNQRFpOa1Fu(z~LZ*nfmBt6^6?#?lFbe~Qo zSUy_u9T#_pER@8|DbbRFt5c{&ATbH@@%U)l|L!n+(gfsA+hwR(; zT5$g%mES6^vL1I&-hiB$yuz4&m*GF!EW}*coddmc-|bb2O^+14_vdR$Z)_(GSVq~v zeIo(Wb9~WhU$C{bYjt@cyQ<)#026AS4Q=$9&IHL1Tn1O8Nf@qHxh}ttR1|rvAiAx} z|1~E@w`%2ryYGaA`%&m%5k#xPX|-Q|rTVDsm2vZlO~5pd)h$VUvZBt+z;*-D%ZxW_ zJOtnam8Q9REqhzVoh?ZtRnxQa$G1cU7cx8M_P=>Yu0+SGSD6d(=~?w=rScsd$D$SD z(WycJK2&o?EQAKC`nDN==B|WG$Dn@sdb?_dt^KgOt}C9(WhJ`$o<}5upNy-VEXG94 z3i+FdR(H~+WGyYThaEQM6<*Ae`0hqlZ=(=*H{x<R?wYPV*wczqqhND%cirE|r z2dTOrQSUubOK)@YsaZor_a8AU?B;H9vc7K33T*!gKhjtDB+)(J+*(l0Hg$YgUqN6U zx58&DYd(A?z~Drkl!NWfp4P`i?-GB*9_he`Dj0)s$ZGiVeYUCrRC@-gNT>ZzL6I2f zd0*0>co@iYH0<@2n3s#%BUbq1A<|dq>R!J02heZ_OrWZF4B!)~m0qnWnpK0|v^4*OzXE=%+A;bl)9wT9Wa(#2_bUCkqz zyk}dy?ALi1fC_vVJG3=?z>{gU`G&gKWO=Amd8Z!dqht`uVTeoNEj|fNOpwPhPXlx= z=oLO@``*=K95LkXbLKx=6Z^73-%B@ZNK@`W0{#B-Z#$!{^Po)a58Tk-a4g{9HDjMp>74w=V0K}zOE5|<%Z|ZQWTjXK>Yr{r=lH;ZjjR`vpaQ z9<#!Vz0J;<1jo+EDW7A({xOLOk}>KbnXJes`p!(5$N$NMrH#CN^&lN#l|nfeCFj}JFz@2+(E z78RW6P^!+*!M}K`!eO^C0};DOcb3;t7+V+PKr^mk zBh~fKL-3Dt#R-1VUg&zn%>ggp34*(m;G%}oy9ZjC*hwr?b9lYy@oEN8$4~)lREZXK5Gi*mHQTgiu^4>}F=FUB76B_3t zi#vswCYc}5^4Yw@s;jQo*HMAh?TxsOJs!KD@UF?-{qFK7>BPnCO?l4^hN9%{x=mkx z#{I1%Z^c67U}~xnhVJ4NkN5`UT!}oDhUW?Xu~35-AAv4tw8SHBJ@@%fsyC;*Qv=y~ zWWAxNCk?YkL6lli$(b$m);T-uTs4fPR(%dAE}cr%{7>M>Rd}NMnH=n9E$(Ah5?{@! zyZVLvDZai)t?ovCubmFPq9!6)^SbS_Jk3uM9@}hhU;Ea4JIj|Gcl>5p(6Xq_YlS-7 zpsvJ^7rX@JCnS>P?9%B!hkJRW13pg)a7Zshgux8tjMahROg^KJ3 z8gjRYUmK7+A3TEDv)RthKP%cvjLcZ45X}!!vuwUhyY&hAJnfD`GxI-t0``pvn4!v; z)**}N-fOgdj5?RWQ?QpcUSx|Zq=}ff zZ?9KAfZl7~i2B~z)4lcr&i6Km+V`MNKw3)I{Uds}+JDzktj)J2dg3ijdCSdUR@2pd ztk^U9A|+)@jqBJ|bR)as*Y}mH+%B5ClV3x?b>UXzY7N46JA~^cYOHRVX*6JR1Tv?ffkcS*k|oBTZ9>dN}}Or?+53gdL}%aXE-$cbp~rV4@k zbGl)lHIIE3pT81JDcIdLu2>4DC~B?bSAyri=&s7hbeiwd2e&`4KjEX;-)Sv)gy6Nw zBmb0GZKP()d;yZPSprE~eU%kY>Rsehlc!yyjVgwQ8}LJ7gqIKF?P}llbO`BiR_so` zs!CWzlq+)5JH5`+sT1d<-W5cSrR~hN7dH-u(FuH)B$ErvdgRPG+dzdIPk-fN z943Awzv923aHGKXoB6Zr$g}@QSL~%MwgFX*6Sop`xqWHajGpTllU+v=-Ot_XY9XX4ULsST68|gBSqEowK@1gs|Hs>MxjY+76PdKC#b!) zWqlv{zwx0k`wWz_temcrq)IuMtH6sz{Xp{g)YZ;Ao2ON!&=1pL38M4Jx@YC26a_y5 z_t8GY3$_Yv4d_VBaNnF&qmOK%Q=OT0STvKS6&YbvcL%6cjY_7Zmr8cF4IytSZYPHI zgeGEQLGSjwWoCAzljIhO_OIZ^Tdv!a|JFoYfTnP#;K-I^EK2p&ajBa~slH6Udlr9U z%yHn@Lp?tvoc)*8B*ZH3Z=S}NpvbHKgJD3h0_1D|eV4HbfRwl;3lk_$e^>|tqdgi- zgu)nc6|0<$V}$4tcdxuc$SE{1{Q&%qWl0xZ)cy~37|&tb=7%Div=fbPXNLb?AvwCM z^V%C8;XgPB`S&6agM4VHxbl06&t<@uR2${ZF+N%mYfA2|&4c$p*29IgV6s)z@6>az zUO5G<2;IQgJxn;_+!&$+u%raG+cxIEpKLA2Q70JDE-2Ox9slihlZ(|$f}nw(=3Lxq z)I!Keeg&)CZr2z;zFBlXV|s^^gp^YF51G0FY$AsfDv23@JWHm2*|0|_nf6kOT4{3U zpUD;g7}CT0xy2t~#LtfluZ^gcxK+KXn}9E~tg4$)AwY~2Pjx_X#(|W;PZk{}(t`oE z>KVImPpNmtfBdY;q?ssge8WLfSNaz2mcfBWv^s)qSaxAZEK zPZ8>Ugo>mobKsN5V{;k4jLRv`|*+~>}+j8#m;XK-`#uX6RyW1rP$4!%z zcYeypo~CDgljT>u`#lcGM1NMrsn~<@HF4X{bhjI-?3t#Wzjtd&LiC8;P@O$j$v~KL z#5?Gyt>_#tjxSnFQ!g?OoSP}^vBZ#4jwt5p8$@bIzIM7YMRQR0E@VAu!foH8EjVK`L!o>^Dc#!FNJKA1!uw>&R|nh-o}x0X5AE2+;(`m z&MypBHjV?-h~eCM`3HaE(zR$mQeJ20*mxMrfRjq3qq`QPK<^vMZBH~dw=c`4@oIG^6A(6qX384 zLhW`5{23N2`D~1lEHpq~uf1s~K;*!oYnTnMO^G{pmG7!=RK^k-;K$VlTHVzsJBLxB zA`U}*?pW;|E?W5aGFrG~{h>a~1uM2Y5p__|YhQymg9T1GP29qASVjIS;9P z(~86f{1?boVAffNyiASVcPG9jK!>I&HltV}wV!wfF=ln66kQEwMVHKxCBx_Z-;wxr zSCBR{Mv!1_9^~am|8N~T$E(86NqaQ;PkJGiigBQ=`i=%hRS4hu>2GpHHb5tP;S*5S ztNRXkCAs@tII26{dm@{~Fi$dH^FOLg$XplRyh|T%!1fUXp9wC7BF9P z>aiV$Fb+3?fnHF6$(Ip7=Wmhul&pcf<*HuV3RMdF7_Uy;wo;tp zOL;Ly3Q+jg*5I=)U9z(=V?A;&*9MP6@W?27E}|8z1SnW4orl0F^bbq~6l}=xaS7}< zu?LBGqNp0olxlgl&_T^3eLx&{b`-{Th%x5-QIDJCdfcTlveG>xBE~3Fe;BL8U|RWZ zuv)*&2S2^CO$V)g{p`Lo-}VvmU)w+# zUK?xSpr&lsQZhBO>A{ehpc|bn)~09Hbq5NooW;0h#(9lv=mq*40@~nvdLilla8Ha9 z*MS-m2k%ged5`y$bHWIVTTZpTZDywnLM&bvRCn(l3E;=G$H1Go@kjwvsSAK-Zn>6+ zQJVSCh9K9k$O_8abq7{DoRab*%s%ubzuY&mHDIj*MVq$B(OWAJJ0Jy(*dBVK&eQ?z z=ZqaaItktG^s3&+jZfMO?jlG3c?t*#HOuUu>7BtArvi4$Cs~>Q_$_GpJ)&zMx=1%0 zz)!Vim6dJSRyH%qMdKiH{~r45gm1wP`3gTiAAV?2~q7{j*jMPm7xwQ=I4d~z)t|uC~F82S9v)v>8K3| z+e&xQfcF{;R^}k-i`AJ`?fjFXtM?2P1kmy}tqH`T8K?eV#eIhyhA)FFh=$IFdis;y+%9#*w_>lsNAKof?9if2wwQ**lk?ZZl`!pDGPh*>bM9KCcp#!)c z>J8}ywbEH=#U+()XS-sG+u$p-@ZKZYok55rr!QxZjcD^PrMb=mv>@SvCq{!9$yMsv zv4S;*C@#DX1r9)pHeD8W`xJwFw@46CH`SP<=Y#s#79?_ZM~0`KEEQ^%Ik=*$uAE!8 z_KWS`(OI)u2tUT&h!NsCoiTl>STSElJ*|}bA%x=DT$~wRp@4BU;~iS8%)LaJUP@Pw zm18Z zUY^2~@NR%%Phg-#B>ET8Kp6G#XPhvGa^n1&jN4Q7{k=QSiQwf>2fJKa@P&f>2-|vU zZ_YZOcDox<%XoO0Xpnm7yWge`}p!_U=#drOZ&$EujMz`#!p!38#Wn51Z1!GZ@=)KdOhG+xeD`RfC#ZyxYd*iF%1_lS&bfFiyNcs z>y~@|>JxT{TTf5>sseNy08)M(wghu#;ct$yjiKOxn4 znIAkrzE2c7S}FWvC9(6|V6T?rB@yXz$(`F4snCV!LAb8FwL#Lav4??Y;xzW28>92j zAJm6jO`}3wd{-d(#dd*QcX|4S<t*0<3?T zJ4>)LL;We!Bnf$HxCdr6$x)u>x)x|yVEiT6Eop-jFrXi!6Sh8yY&Fg2mq_BLK76Wm zHL5>pq4r(gGv*TiY%#csV1aJNGT|7NDPkO-LBjG_V_tBHCCJXNzSxg(R+wU(G2|A# zz%MrU4hSaL69{>W{n~C|eg$GjoLgKkHJ;xeJgJRU^IurV%Zxp`^T_Gp5%<@emFGCXcOFXHK%V1Ygu9+3msmGg8d9Brk4nQC$m zEbi;;PYaRTC_E(b^HN_a6%DNmAVh2X6{0Ccfgh+zcwzh&Ke6|I4A9c^lMhI=Uyd$V z$vb=W+qRJx$!7K$KQx8OrX-+~iEDmf}2X+(VSvL=TYrdkrz&h=O@;L z+a}i9>YqBO5Wu%nM_Q?^k~r8phO^_jvdSIjFB;W$OfRdAruI!Vu0nt@e1sW{hWIb_ z$x5;mWQ`0}hSd-bJt-F_9IC<9zF`D@t!NB@6F4lD=YAkDf)T(mo@w|X$W9efd62lJ z@_E^|^T?OiacQ?=?ZmAc2apjEWAzVn(ZnE}XZ`Mz)w`*pqjEVQVKqljO){nw7lcogEur-=D5V^gg z!fy;XHNP!JFSG&#ylvZr&%7}zJ4P7Okdhq8k#oQ2;0N;dn|S<>b8{cl12&46Zm0~Q zL8WO4Fy8N7=pRL|kH)Ti%5}4W63cwUK(bC_k$J4=+0x@e>M(m~G;qzl(5_kd9bwE2 zpFL)@Xn>ks9q&EAp1eJfD}1O6qpb^?*#CMiM=SWMuCbyWg&3nIpiBdeq6fxbjS%-w zQGCce{^n@rj$~nq>SEK1<>#`6WN^95vrxeraOZ^~+Of74LXu#}iI4Z?;`104XL)6A z&|X2EbA*VvEgWZWsO0|dQVhle>M{i+o9>r|!cd_j@DWDzM=b9li7_7m`jJy}`-RTy zocj!jY8`s-O$#&v{rr$8~2b$nRG*D7hW(+3(X;_q~Wj;}I-fjFt%(Q&a> zB5HA+#AC+Z`OSSaXu1aAOlY)pP#MOfbdwSzh&LknteVAdxe~3Al8tW$pDoCc+?#S( zIH7lKVts@~(bJV^M6yK8_6bL@#0gZ0NPzM$>UaLoc#I9{yU-P-XLOD?T~hv%#mcAS zC*d9hQ7tE>C%R|^4Cl(l^unOQ4nO=|kTmlnh|UMVJ%v(HmhU1;IXjxi#NKYSYR^5QgX0PXLs0o)J@mwJxE7$`Rd##;@| zUz|Nnf}I0X_=)QZo5c#!vk4x&hznG-Hmyq(vaXpoEH+EzGU-TUL5?h$tIaIHvyn!y zuF%W%byY)6El!~8UbZJ`l3krz&(E=% zirl@oo7zENJOUY@(0Ci>`EDkvVRP;QlR^LSFm7Q61A!&!gNi(1GYx>PrYM2Wl(kS`51 zfI@>lO7N}Jzudw(a#Sd!?&dl(0sx~iG=p8{E$(9&48;qeakxQrk8FtBfb)K?!55{? zQR>KtN9puDJ9V8t3is$nMkj(q+lVwvTdc+3*&2`sLL9HS(Xpm9>)Dv@`9Go^MJt9r z8bsWf@8Uh3^cnvIBHTMmAX8witCpHgpZzj!;WGC?wYh~ff0{f;DgQ)cu1ZfkBf{F> zT0%)Augq^R@%lg>FAEEL(M6zaHkQYV6BDkCvtV&g6G?0*5#3|(ag}bfoMY=!4OeC3 z+1BhISnbWoOkXtaE#BB3(gPojkN+N%e(4ZC+bL+waJUW)`jrj<=J?GGOb&qj*%QW_ z;%G^lKAwZO4)4^x>Ux6ax++5kjTOq;CE&`*Q@x6uX+h91swe&RFn5XMSLYlL1T`zvAipvUkFJB+KMLx^Z2S`JDL8zhLAsmK%Mn*E zv@$h4dsIiMYzN1VMGc=b_hd^oE65j~8Bk07xGgv~&^*>VVKeBm=62v*v=AyV?KMBYRp9IXuWBtDz?O|g56?h5j%d6YK}ek3i@1epTW|x@IeE`g4xOHW z?R(R@sv!%tn+ZBt+yh&So!+*16#_s4BfP;5xfJjC@Q=)3g%i&|YcIH};VjwiKC?Mf z8@nSI6&H=7y_M-TbR$i<_{n3>L0;cd;ddGg$CXPOj%h~%Bgpw#jW6e7dQo!WlH_bo z@}l~gTq!s!5WEFs`Fo~c(b4-e4a{R`_X^{o*9GY=tjtB4K?*(JD}00!@en9nUU>B& zD2}ZQ((981N!`ubiT>ceHv%_fy$;re4H(B=;3;mm5eG6c1)fU&ys84jhFIb(#%iC_tN_yG0c5#;Veu zmIzR-IY#Iu`Zz+ou#Q&GfU8`kywLuU8`yYz*qOuyF+_IY9EUH|%nPO{R+fp$RNjkt z0)`(T;_G9IQYtnmzL0S;E0L*Od=!!X@k#UDDrFTi#a&W`;y zoFUki9-WF;HL8^y7q+PHE>}>O>9eKI8iZ9UEQ~#w&5}m5J0WW9L9PCJ{*|M#mlTT- z-b34nr97>t42!c)PMLT5iMgGW5xEy^_j$ha3cD@W`t8+Zb?mereXy58kK=CvgxG>& z6jK}eT)F-n_u)lIQzcF%KVHq}LNVJ4Ba^l}p-DdU?7ewRDkkvYJ=8i|?_sB+Vx&3c ziOPm65qO2KX zUy?0*b|$-|k}d1ZSi@j2cCu@=jwQyvRQ5fLZEWvn=y`g7&-=dTyyrNlf1Jkmp3mpL zulu^M>;B^JWVb{Ak`Q$tHlGp?>aMtgRz)NKs- z@NQF{JG@Q3KGSzf$Sp0N^Nr=UmA@@Hd@u#j$~*}D^KWv)9z!m1ZB4n!V)Vu{kjjL& zO0_pWUKgSC`T3PY*m0Kpg&%`Wc)nj{Hr+6ir1lr+ct_;w3&KI1wv+PLZ zCl#9ZI8!_a(n9y#y?w4u%i-kG(Q=~f?+We+|v!9|F!+IFe3 zLZ)S&#v1l`dCU7Ss9-73`8<>=LEJ{kMJIZW_IH;X#2Gfr5O@cPGCMRWBP?pv#e~jeT#4 zLpG;{Tx>S&mGScOilr!D?&nyW!o`c*>nq2%CoOi!7*}d)7Z^jW$C!7P1@raG-n%aj zJ_7&p`CuV-Bju{2etu~&gawWhtgexxmocEX&^QB0`Zv~peLXcRkxMe{PVb%KqMM_v zo%k|(RXLrssfLsCNn8&?kYSGomD4xY6w2G(Hy+0#SMI6Sc5iq4a#k8!K`ZXdpiE;1 z(cept5{cIGRm(`#>&ag}hh8H0-S&`whb{rqhW8h?fXdOl?Tp{+>dn{uAgW$b0?w$b(d6IWsy(Q4fxFu3$j z|KsLcg{IC^Oj$Za75fC@*XY$waQ%8^QlhHd=D!bf&#;^n50d^&_^;3hp( zrDVgEFykg}yWoL@mTfomJ2)$#w(wj;LkHV)iYkhOt{owYg$yS{vi!X{zOGc8D6VdAL!4@1C?QFkXFLp;^ z)a0Xvv7cG_4Fk~CRl7w?f?eU@Wic{4{pt|A$P-t*mEfCm*SS+%Kh&z8K-FIsl6?eU zcf}UD0mZ`mwOOl_XsK$CW*lTs&)2<2J*i_o^iw=-2}Yi!^(JI20}{Lh-ka$vuNHDx zP7YJ43R%6@^0(^aRddQUyfd>ekt`j|93FXI-5hTjZ7{5^g)%Xw_txG-d%(mQ+$Vczc!BcGw%6teQnw#fM+-V z%-I&SrsU7H?r@LrQyNt%_+4Jv2lBP}|9`UK+acQzPN32Yc4`x!24{3d#G81*zVBJo zB)a}S>AW}d_#B(#{;tcc%+Gk|ktW+o#;nB4D|!i|bat<-OK182;o<9Ktooi_%y6)N zq3eR{+5@>gWD@%xgi#_ExfpW!W|&0WeJvaGc^kea-FdrFfYWlI6-jn^I=9wq)g!o^ zJ347&hfB&nF%ucR|4>ivZpGqLRqs%zrd-oX`xx=Zu!jreWW_~G7j0v`3oFyt~ z<*FGNTSq)buT=25jG?$W3s|lL9a*f*s78@^nf_UqjfINLblAv1{+7^AKPC_-X4}_X z;P)abM)%i@y|$8#bn@PG{X~29QkxYI>$I3Zqfw%kbshlM@@dxqNGlV&e5%zfV23_1d?cQn4N?4;yyd z8RNtV;8}Y+ZqHw?+$~nlJQ!d&*xOrNrfMOcVVUd-S&G-z-ocO8Zm&%aM)lnvF62R! z6Rp_-Rb|31+W$-L&s7`pdI?Vp^xW9|d^!R?-sHSnqw}6_wL=ov2^B=N|GJjY)i=<{O(V`pD>Rzj z()Ayi^oO^EF~Rz)5QA8B_rXersBN!S-OiHa?2FG2f7}*SM@wyFcN@E5tkw7BH%j(B z_61K|vZ8HQ{@;dANlsSB79(GyT^m0`JAA+s0{X%=-h4*G-qJu ztMVlN;`X^%B&lnyY0fK|jhzL*f0s^4v0*oq4kO-uZsHO}4R5LgL1d!X^hd@+dZrh9 z&+cTKe#0&GY@qV2r%T3s4cd$Nt`?Mjz7O7G{1vJeE!C^FtYZw9v%zB~T`_jEnw8&? zR1VVa`efNOm#uJHS)!%AA5|{c(awySz~czdX~$|^;`53={}^zry~(eg7inDk$o#j7 zqtj>HEy%814=!k_e)-CDLO@W6J}kL)@wxU$PqtOZl^^{ywn6(~#O3@_rlL;hNF8vP z#{)TqmRmTAC-*4d_oZhMs?6S7Nt8dE6kytJc#tJUPbC)b!0MMsl_h=2Z_`|_$~&{R zBb#3!>a$u%e}T+azrM-*{@74Kg#LgYmEFx!6UZHSifHzB-k$#6R+9)XPore;TrHO^;mQa4ab=01ZWDg|e#pW_9_~V@aXdSmcP0qn^8@yLiO>FkhM22o zLYo#Ll#2F!EWgfm2Jo(7GnaxgwW&h>g1F-1>#Kb$eo>stJ540JP>)D^+}J+x-u8Mh z<82=u2rY5wtAWL*UCQDP8dH2aG)rw2V2X0|hZNrRIlRlpnIB*i(KW^BCn67d-2aQU zz1a)hevx}QZqZd|)-ESr3yautGRifCm;@vFhP5J0*6pv=m+#+V>!|(nc6V*8U;x#A zH>V(RJ(NB%0+yZgS=EIGzW8cB<9;&wpIiE@p8%)aIla1J*UEtwz4fN#vSCH4(WW4J~;^cij-ndqM{5{5z z{Ood=Ec;h-1dTYuto2@dao?EtUaa5HXUM3i8p*0tY1;Pe&dSU2aZK)9dmVSu)iM1~ z=E4jpU`TKaP-J^y_9w;dyl$w4^0D06X%E&ySe%`qO}!xK2EJc`d>?>}Fs}3tv<(dziubbEq?G?H ztFy*|<8*&s9)p}){VLQ+O<_pCwG7_5139hFlePc*!fgMo)%wSnK~0!ZxnrLi`tE9^ z5024U4s8#oQceQLpOHxN}AZYY3FSqXUkPj>H@Nhnqq06T0 zB%5gFPPt}=&DCwWKcs@(^;hxveeyrvjYS;LuWQZ=%qhp+h$5Tbh;MyGo1;Zp3CA&4 zI)BA~Ko{pK0ypzn#+z~T4@?BB^WCJ_{0OGaj=d_xB)myGfTXAE%p-k#*BcKd?7hq= ztHdsKER$1aiBauVcNhe01FYkPUh&l!tZ-yo(W4`kOw$Xq$NwRtALLW2vNg`#>>@o2 zdY!UdCQywWndV{25ksBs#6T<{Cr>Q6w$EB-=cvQ?OUIhnjCrF~*%BQbHM_g|O0&65 zdik7s2WkN6>-^aITNo3;=`JUU)$5<&y64M$XYj489T}|a_Qa1?3;Y}&th?lHy44Gh zDYM1LRW8?ZtE>AxU`ofVaegwOhH#@FF%fe-q>{L|)@vAeG zT$MnWkL1)A&PUYqFZ{ph;=_R~KDv2RhMiT?H8SK%!osk3Q>Df+V-T$;bhH1)n6j(n z#;8}xDu?CNbKWKnpUVNIdCQur@)FQvHWKjKS;%!mD-q-CnC{!gB*Cemw)c*I7}wRG zdI4c5@#cKLdikmLJtrv}eE*kfkBYt@cD;GlndP}|H*db|ThIJuFO$Uo5<1UE z=NV_!*y`G&p#T7pz@A;sig1pKmdUmFc z^&T^rMUkBYTFxffA7}m(h+_d5N`k1Xpq4GBcdk0M(Lkp}0@`^HJEk#Jiut%>jh0uv z*&%__51dXtl>kEl(=9FBW0oWw|9_d#K8H5g;*l6fsS_O))t3bOUCn6VkJqrm8y{~u zXnvQ{$#WBL)nAN0z9|+Nw*G3cv8dhBP+Iy=6fj6@md&OIr00OH={6vr8qV;t{*!-! zHa%AiZ{?ftDj5K|Pod#4UpBOddt}w+Abt2Ya5)XvB~Z0i3@uBG_B~uoBYRTKLqo$S z#r3SZEq_HEqZVS7Iu@kF2&QtIU@BL~c25EjE`L*8RkrgkL5ysfd56Jup|sj2Xn@6yzs+6{Sw<`Pv_A#YxRAyxw* z>W)h{xc=+9Wxs}Ychct<=;8lC@K@zNV4g8=n3p>%`a6UXfUss)!ZWea9hg5xwLbjv z;)7!WsetSJA7F1t28O=we;!%DYSzV|NrB0JTPit2<`ompG*u^4z#;X4i$=Y8g;myB7Z>i^~-Fp$-OdHFEIg`u7Q!90?4!ER#d-_rlfSG5>h zbti_`J#qe4N|+hBI~t{dI^yDc<5L8{F3dREa`Z`zL>!w66J5J z-&11Zms0hl^G13s8&yF{RM)&M(SoK5 zj2M?c@y5GHK?N4`#s)h;m++2OoHTS?9!;#3ELRkbgiwFvd9wfF6-JfO^bTnz7Wop0 zQO+wZ{EP5Ubuy9Xf9KKnDNO}l(AzS@!IdiUDM{5$Lpv`RzSpsqfP*23v`{(fau-@# zASB3^^XFTC>}ehDbQ>U5QzQLv*GQx13-przMzmXxWkb#wV|o+8G*AUu^l3^9()rMm zhM07UcxRU=Ff*X9Bch#}gAv8c=NIK%>FVx^HnIX&U!2X4Lbf3>Tn%q|j4DO|(8`V= zSw2DxsGzaW3 zEA1>Xwj?r~Sy=qb9%ZXHgF?a}1f)>uJMNdP!Ps(31&kT2jz4huM^0fb{+SbXb@9Q` z57wP4(Fe2*K?R#@{VGQP_T%iFjaw0KMj>4iD8@v*y2FdgOdqD+fu6y;C*&GJ{nE}; z=nzTy?JH0OzT^J#pA{_Nu8z@=5Ul8Rt%3rt29K1MPt}(Fb(Uq8dH*diR}uO;-KV+N z{S`xE;4ku~6}r1XTBs?&(F}B@#({E_k6xb~K~;a35F4WVljN^gHk)}}$os^@An>;b zTqDWGUOxS|iiGTd+Oy2&Q9*$<6{t19yWCIV zgeR%G9ZxIP-dS)H$FWe~Vja0KSvE%5hEWxSz1tay-5`w#iojJd%PAy@u8*=ED5!Ae zs(1(H454*V*Y&FHzReu&E<6%WL;QJPvBmTSYA|J%cgCS0*P1G7?p~K^hR@1{gVzvN zsR1|f9m2D#o2M5ef5Qe?$sB1CNJP@;WlnvtU*;33GPmpgp#RSYu@lHG%g{POs#0f+ z@48@wfhiKU0v{L{qw9fS-Wb>Ok? zt{|be1Tr#;UEnpD4SD<_>|f{}o=gA|1c%{swRLD9JArm)$*h*XGXn1Ph{YYX_+q%2 zAtJC2q;Bm$A64Qy#(pP`<;y}-I)SdEBOjg41|RYqAbgtLShA5IA}_8hjELrp|6;uT zX^Mf-BP#R}%N%s|oqqu-lHicTd93e(S(+g@xv zz7wOAI{1V5-CKnOoutdpMCyi_pSOO0v8j0jg4Zj70uIcx3`=tj@owU|0Uc?{^g;*k zgJ)-wS*|dHnK~I5PoD04@`K^O@mnB=6SH)((sEbsq%E>Ix1Ql#Ow)vR+SV5uky`z+ z>Pk`)L*&SIJA7OZ(JQl7I}E00>n{(#G1HU}+TFy}C7jXvEI?74A z;4J(B?`J_Po_J;XFFj{)J<(S3vS+TUbBfeAS=+Zgm4OmFV*s-aXO*rNuI)LBdLCb0J5E)&*B)FIp)$^|z-4hjpOVNi#?SY4%`s#*w#WD@RU$pds-gL0HuUn16=&!xbvbo~{4h{{x zTwYbm;SD++1xnA@v!ol=+di33Kz-1EKb=ZF!{;1`OMS<)2Tpkb5KY1D6`q~_4@e>j zJ0)cQ42lv$l2P0To4STN`x~(Jr@XJBan~ICF~{6!Yr}KsV9g4%pKMw zH*wYL#vu<6lP)~rL2}(yf7xfLcV;m}mc87e;ar ziroG4gNP()t0YY>NcR723)zbtE)jawD_Rq^Sr?^~q26eMptPi^pPA1y;0T}=b3^@B zRaAFR!kzf;9?dz}^CN4ViEQkYXBl#Mv+!^e%>Q~+w~S>A3TDf!XR?fME0I`%9n3ms z#7vDGsm6H-%vd>gLryb~j6dJ;`3s?WBNPd&o#~FF(W@tOod9f2{{jlv*acR@0a8<#Q!2wzYMM}xg;h>&&T)Q=H;Ij&cX>=PO!wzE5gFRj zVo}Oq=WCpmQc^eN)A#xvvy}dKEDD@x$(=A_o zIKs1D=$|40$OSi*H=G0wO!KG$4lw^QGZhtLDIL>mXcvr<=+~OJ_1)aFkHv^_$4(7FPCxQ6QY{EZ z4#LzGAwjC^dNpq1jRp)T7isMBuq^1s4sRH!o_TYPuJS4j;+XnWtq99nVV9D^Z=1tc z9y2iK?#VH66M+VPA5?9^SVGs+qcmoC4j=`$dj%Btu&&|JFCWROuMk>@$SQr8zWtj? z)-#H(Uh>+tYsJufFO6z2#N45())yJ!otcVRo;h}_&*WH%J-~rynRfhv^5bc(G$5hY zm7MnMXc7*vnx^J_-A(Do+w256&8SLR-p%3cd|f_*l}y(_E>+3+Vs-SjPyi(D3?0cJ z=WGaK{fR}8)#@z0!sd$Mo~%_bb2p`)MKyz}_qNWob1`XX5RaTyk%I(%nL!aeO+Vr` z5NWm`9D=uVQ?gm8Rg=p`a|R8CD!-mGN;^aSk&BL%_PeALR=&-1QS+cK6E2cWI|LdR z9gy5A-%nTlJr}yNI^hX8_F+*m&&3mDBB`7A@I|0*xr0IsAF=wDFZ{$d4bHnA(h)N^ zEYx)ROzkC`!KBjt*J6itL(qMcmq>NbvhY6ugXnsfla88ft-jXyj8JZ1IPJ5X(Y#0R zfIpf&8hkLKXY%jPEl(}>suo;c1N$&xCbcUM2T^vGEnya$fi61-- zeP^iW)c1RCRH&)uRz%hJKPQwkp#@xXnbTJ+kJ+jzMpU#47)rPGkI>z3hEfvs2Ro`T++?yR)Y;r24>0@*x1$l5`o zXD0Dx4W~j!^f};+vDUU7-D<3FjeIkCzCd3DTs^OK+L<@s`FdSl!SD=lKo3WltOi2< zmYCYYcFU^|rR5u{$+aA-M8xMdh1PlMcKJNin)K+X-+=B(i3$mHSR_#Qgl1;0ZRS|_ zX$a!I!`qd?Q1T*jqG4HgUauC}$buMUajJKS+m}?fT(P0eB!O8F?1(fp^QjUW=zqbt zlp$$&1-Mpo2{^W{%aAdkhq7&qqo+&Dk<0Pf;L489s>8w?e{jAhUK{~Q*(M4M#(6w` zKGKK7dw9f?%txTg9JE4ZS5J)T3%Hr+vRm2fnL?&h-CfMhHqQuWrs`_u_lkazb9v+; zKt#-mjh9KmlW;6&cTz5YEC>LPi}}0|0k)ZCkE;0GZ5ylZ=>n>77H5V3!z#@FlDw)# z)lF*DCk{UGVNV$~NaVv5X0qaO2D3}7(&E<+vD zS$8Q%96e;!2@JwPxL;@E9)XVG8i;ikb9^&^orzDNN$O)Hm_a|g%0xcODCC!-k)H{N zUK`B7j5XKDw3N~X3E7eJ`{b12Gm2m*OdTf<{P{`MXY!y zn&Fj;4j_4g`@Vk0e)RcE zCuhJ`uGwnhQa9#$vlhIn?`iu>S&IPI`o`D}t=gLanKVfUV0#zDvlXZWB{XS=hI&Tp z))OG+?xs?p&J3_nA;?n4mi&{I0AJ)lh1wbOh@Yr+$@9qu`k3IaBY8Bs|A_|0BX;d| zG_h7e>a8Hl<4L>o^-t++h;9?lD*8woH|cw&f2p&03Ji48@=7i;+O8~1S#?^JT7h~d zRSNGf{rei;o8Bnjt8uO3;UZ&stABp*20};d;88i-X1LFnRP(G<)Gw3Bd#b9eQmj%S z$>K=;hwa1UbJVY7T|2Ed_@2pGxtiqkh$^D`sI`CHC$MSwy(`s}Pql56U6zW3pMTb<1cRO(f<@Wt#5$)ta>5o#5VDoE^ zpWg{aYw<3=w;}@C(C<#oEEiR2VEIDKV76CQv~F6_5MPF8{;2AcF@G^s`*1{v=QK!^ zf+Zlzu0uBpY{FDS_gH0dIik8(Jh3()1TT{B`fuAK1!x74526ID4f%7j;SO*Ep42A|OI zY{|5WdG|;azX|Is{k6&(wbc~)Dy5MJwiW@6Tv7ko1+Xf1Uo&P5|IBAHUas~yd@_0e zuI^D77EudXA=b(fu;?fYB>GTZ@M;zjC}q$#gEhr-U~vZ*Ncod>H)W$y-H0^99| z{hM}2l**#VB7TQm96I+LT$yeu<@O^5yT27Bp~nf}UA*2Jqo=Re=E9#7A55o233h&c zJ-EP&f;Te5oD@cG9wfz^6=axexolLx=1jMP_Uk-SwS#?!`yF~* zuSbSOMr%YbGm}@6HVh<5=^W9R?EK`Ge-hrfoy0XfbmO7#YofWpxKb*Mu5ZMwy#asE zHl@{i^r@ZAXi!@ehJ|3+Pxl|3o45?7QqOpyrh7%Z zG5bV!7ysog;2rtF*rd?(FdMP&`TQEW18Cx^(wgoZrG!6){&7DHL>MeFAvB`+ao~mEbhwm(5rbSQy7%1&>I6-D*mqIsVLa7F zU6)@%8k&~yVe@3`j_x7(ciHuOaCo&dqTVj#G(y)&5spp|nMB=_IoFiv&%%+ly%S-> zE21L|F-~L(AUU#E%=?c#PqN1Vz22g7P;9^NoHA=m`Q+KmJsPm^qd$N0!iPSbD2krd z5}D%0qV;=NNXg!D%%f^IZ3?kkFUS|zK^d*wHj>n58T<57AKsPS*V7c9Jv(EQgp(Ra zE4NxOc)aiH?n+4^%6?7zE7Hg0y$iej`fw);1NrRB#HnSeIl}G|lv0?D6L|l@euGcW z`bRxg4SVDpS)RxBZ6>SRlrm8lQZlcm=7!^M;q~U&yR=-~2g*)_^xTu5S) zv{+fB+-pep^*w_U*F@VgUXf&ha(U!C?Zo35@IMV_Q?kE7fCk^qUg#_QEeBX%rnAq+ znPmU{D_JFNavy}C4`Nuake^?c`#E~M<1 zylNR#w-5hrlI5rf;=Kr}FB8JAD93`O7(danFpjLMi)m=o@5EC88&iuzA>-6Xaq;HV zNRs6lQA5H$%w2wontkM)cwVSpJ;ys~7)f#FUjAkFu z>s37WC+a?h(G_FSs`P7T5AispZy-^lRNV4UURHW;^mi1IZ7tA_WE;eA(D6@P>&c_y z3Ay_7Jz&w4BMX@CobJUf2bsYX_$s|un2C?tWXXRl+5 zhM(@&e^$3Z`|MP;wyTf44Ris*qnOos%Mm)BQF%&)%}5)5Ij=NGr7T4+mdPiv>Tx8z z$GS#OyJD{w3({R6uUdm8wt%S?i$+@FILrO-z)bGsxQVZyd8yqud{f4}@WjT`=Su2da!* zVSEw}&-pkq2aW2pt{J5jo);fr4S)l(PI#;B%_=TF-@NIo^Nxs%kE&5{w)9U+xpS_u zqY1J3Ie8^u9-yLF@U?~N=;Yl4mR%-f8an;3-#0z4#6{PF4x!b=i=|YNb z@zd$)l$&MEC;^qW?8IPy1&do?3Rc>4(aPLNX4d78LXLd_o&-^XIXbzZZdrAE2pWA} z+hUF#swxv6 z(&er1t1U5d?XHA@GZoD*@+%raF(3g~?rT%>N8up{+tr!+(=yGD!gH}R)}bpvLqN5y z5LU*iO=&DJ6#vFH_wDCzAT<^-*kD4c5Mgk`=V~(d)C{ifvsqqpp!x-aSz}~f)-x)e zCg1J?ar4KvBz9xWGRHf3rMpLa>$mKtMM|C);;I}czP*SaNO3A*58=sSv}C_r+m5Cc z$G`ay&vZDn;=YelJN!GSDb&Dic`pAb_i=9*=GtCI z>pM;f4*kk2@WOYUXnoS#Frn++P#ufi2iH(wL)JeKV_A@ z-Fin#|0E*e|MV$SBV=nHK?;Nt0$>`wLwS5{TCtYjLM}FyTIJESvtb}+VIedUFtIH7 zOP|3iJ-6Y7Di-?Db?;tIb>%(Dma9szR3=tSE~GJ*HXIhqVjR z&f736@z(T!lSGMXJ_NOt&yUss$g;D8dPv5kRCTxt&ENlGXFw!;89zHJcM_7$PrZIA z+X~d=#^62vWY~Svg23}HQ2Lyj!HIL{1ZPga5X&+)agk4=T7B@CZRpXSshVOqER%`# z8J9@yU1J#9rR$QQYoXpw>xiaiIc_w_P0UR8dS!EBxHKVF{%Y@loy^3u%H74rBoPD* zWjT4HyNmUz@!#+;gL>F`T8#BjEK40qQb0)K4Jx2vp(%V2Sb{YcW$jz=;fso!;?j2O z#ip9fQwt+-w?%82vr>EKxhJ$(s3q(#PM>U6*V4_4AO2zY)fAt18v%e|6(n;wG1TjR^06rebmg@sced6qDg z(qS$w#mx`tV0{C%Rz6p;;gXjMC53XzdfswID!*~1ym5==!C?0qEOx5?iqC^^zang{ z(Lof@QtY%(VeWOG7KI{=u^ljciQe`8_4Itr!LmBigaO9-eHum$(5avoLA{t*w%`7E zzCdPJuevvHP}K42S6A1NP{yN!_mX6&zwkwc`v{%_4SKLKEzP=T!Xj*_re{jQ6m4Vc zx}G|Ck=61_S#ik)AGgBG(Nic(rLs#{{S zHj`6rv8P;-X;{|m>gw@5@NiK8zyOS3_Fn()!s3yMpZz3R$q!uayzK*zFNTgHecjWr zvWedZdh;gVE7QI5a;AzttKb{S3{It2BmdOW+;!{Eako2&4&T_HRWZ;xLA*7il{VFS zR^|pb*6`V8?QD}w%Nxc6HLchwU>b@a6ts(Yj)!*O_*7wOhoiQb^$?Iwh)6TPQ5Chy z(vtnx(>7pJ0U8-DJ(-F!-`!zr>Sf5!O?;1SR4_`$;A~ElhaAy^KSfBQfistIM;dST zC~$6FSn0TI%guw{3FRa}fkPv-+A;&+-0_sy7QwB+SAGw!JuK3+u28 zgM%&mcVin$9v8Sf4=(fdvT2z{z_a&rLQ(?NV42phS|s`nwXI~fCqC0lK{Imw;cH5gGtL`IdqkAp7oT?3vhb6<}ERMJ@PP1jG}8o4qA~>jQ_pW z{$}2dfsZ|-jl1$El57li%1-pNEe;O$(B6$cBMJLB;M>%-QRI74fg_h6r(8-PqbsYRKB>$8TN9(j>k(5r`&ajtMePPssbar`d5^sliP_EzM zq1s=696E{kc2Js?fr6YzYLxa?V(X$J^~#$2Z{Mvs8tJolg69diVk~j^Jak-StTaS> zW;o%J>P@?4yGLTfD%?#sq)Qv>hRUn5^;?~Mw|8!RQCY8KD$d!QD=Kd8>y4RleEJG8 zx28KUB`{FhLwg55-6fYra|bUEbG_)hmju-0MQfph+MRmm+%cOl%47$sq&c{Q^wWGZ z%o4or#R01`E;}@1%J=3|H7P&9xVkjDRP_FM$N1#?d;zWai;JR>&3zZX4uvh^>4X5F zURTZ%^cBxxIf)=C2=JGV(H`B221W2B(#$uIM+@_0uadD+6Q%k^`PE^5^s2cY{7qVY zZ(xX=@gxx!GNi~Ma`L*r6yscU;>l8Yl73yV)=AjDOdZOZ~T&NRo3Cf{Be zmbsV~kKQ#8ljW!>6K~UHmqG5<%;)O&*_oG_Ejueen9{#E^H~dxZIpC$tC)8?A6rl4 zhQsSoFkm-GsB%nFUd@0O1X;BISQNIM-rTT6r`fV@rU0BH#%d(h6GKpdoc{p>n9#%T z?(M(h*#EKLzrP^#r9naN$@{FbO3f;0wT&LxJ5M)qq_Fs-dqd(Q59{| zA;7Da5l{TPW9EM*?$*=^a!=?2RqsF9cnC||_i@~NDGU#5(}lL;11#^OX@%BLj^`tC zq$4UErZ~K~TknKysPpCZ^u6&_gM7^!dX|5=|5=hx)p3@Bc+Nh-c%vcx!DSg@d9@27 z+@L-(qHd9+S<7`U70Oyu^f1{aT%|7iaY5XI-%lIs!VPHUetO4MfwD<%$0mc3xn?w# z*N4yJ=F6TAub!Uceg}ubUNeg`RQWT#qiW~m^Of2Cx_xK zTh5ti4PP@ijV{;h#fFMBHB%sRC=%M=X;h)Q2SW9JVT$c^V*hwWFad|EO~LD9VxnULK}$q$xM&6`E{+A!EGz}V`p4xHy5+WlfXj-6j|P{`=Wbx_K< z(oB&=yu{!+*LW}o8>XzQdHk(pw63ywnrQKP+Dtq7+<R_`ZiR^_Yn~eCgHBe6J_w z30|J_Z^b<-lB+8)oS~ybH-=J7=3hRU3!HNpEJD9;v{8YzQncym7KP6y;~3HkB~EmA znmw$l#+}BjySlmsYvqdPvIsrD%S9Y}O}rj@d9fhMW9%8($N zdiy{%kMAp+yoAq)icA*$6Z9b3Gr)=~uwB7^CwnR)VER3g!PLqty~7M^MpPJ=5U%fb zidBo}nGp3D-Lby^M|Y>2RoKb7E`s!;n;QuZF|h~ zl2HOGV>{&LM5|cl#y7wNZX7HmolLxduiO|TGo=bJeGbhlj< zD}??8Qz)^2py082;QF*bKk=%psYn(&H0e5!KY^cs4T_#KtX8hkNgJ8PsB~?_bHRde zdzEp9p4!Xv%CMUTJg4Go1+B0Jkycmgl*30}#PnI5+*Ui)rbUkC75#Aaqj$(T7p{ z9KG8kVe*U4H*q0@v*BF5Iz3Hqf4K=yeOoUsepaeN{B>s4`HSg>ew<~k?|rp3q?R>< z@Z z7V!XD`r#he+v_DQFC#0TY+oH1@u|Md@FKzFl5!u;z(S9ZS^WL9d_$sP7v!7LZVEQv zrf@hdXaV^Cu(onnA2g_SB53$L#rxVzUbdIavzU7<-W$WA$aw@-`2(MtaV-e>^~j&H z(A`tJwQRZ|8~&&B)V$IC_Rsk@e^i&r@@3J+n5hZGpFO*9!*NN-5%&D&eWQM-nJHt3 zxcmB)jx-20VT9Dic}R3+hGf3@Y`-W zZg}n1TIRlKHT%r<4JTw>g2k%&f40;~F%6FZ>6Jn$RheFBq?6m{AVhejiY?2&hIqZ0 zIKCDCP?X}7%9-XHx60bu9fqD>ouZu&ongFns{~^imB6yQD{q#0(Vo$4@BT}ziwS!W z!Kj?^otCEYh>DghjzhQ}Po6Z$-Wf2PMIR1+cQS6_sP)!K0c{PI9W(%K! zwr)+);9JvQIiFN)jWM2$XyM$ZjmP=g?^rt&jW#1=#e-E#uDnhx^!G6vcd(viG%@SG zd-mKa#^v`#I9;yWh{}i^bzgmb%^gd!ni1GFd{qzM8m43@Pjmh9_`*3M$a&$L5Do`5 z-`$JWd90Sz83)$7wRSuXQExfbqCBL-3LYzZ5EgJo{Wq&`nvrv$s+B@ik?Wb!{_)v& z+(&n`OZs*jnLebL%r2am=L*2yBZUp|Fg)lmZ0v$36FyJ1HKx2T&RJQX3TzO{=XWyF zDEg6~KQi*F3A;3uVeskp0JQT)K8RUrsy#HBUgx52JfhlEa{h6siXaE{=G+{x(wtdt zj|Cg2EeK1o?qYP!T*f<`^B*}47v8?3$ZMo~e-eS7wBXBCW>DmCzbd;$pl=h`kcLV6 z&Ml}gm2LG%6e7+AIy`$>>T<$k!boVv*7-MKES1kqOYY%&iO4>Rxp2@`Fzq8$Jm0WRC zj31pW5W1;eG59LmtNHCWZTLi9wD|;{+bTod?j=PPIT;Pg$n@eJs-qNCb3N5z<`|u; zkE)A(Ryg^~ou1u({e;5L3&zjOuRmRq%HE?Anml7*uF`GERA2j;Z}Y1M!rc~{FWeVl zKNLWG9V8tS>!UpRlBp!)!%)Z%tF|}4T{%J;LN$EGSbMqT+p8q`oROc+gu5Q+3**F& zCHee$@lO1lgzd;1iQ=1{AI`R?UY&Vaywb~C@S>mh*@r^w{E%_2r8yomDuyq-=147P zPW#cbbQ^wuo*AB2ntESXHf=tDH!BuvFdY8#@V?aP2E!le{02Pmuz?!o3PL~6P0Ark z?E~uCdY*VPFuH)KDyb-jA6r^9`8wPoX2-I6eO8)q;TEN;lt&MnK=k`&&h^M!CnIHP z$S;UPkNvhj3<}x#*TQ5B!8L}SBgCcBOV8s1baTk6-`o0ATH;qX7t4egZ`!^mXL8hg zdQr*XPK*u*{j9k*V(-CMN+7&PW$0Lk-dG$lirbtGYt7h9nAJSLgA#fcm`!b&q_pyb zDGC~_A{y1k2MuofIB9O#`?;si?Q0eNco**N@7Xp2lYJ(QANH~}L)VK-n_d0}Ohn^4`F2XBc5p?mi4tA3gEU!BVOc3pGK!+%{UqS#cbbLZTh zP_*IIvRjOQHW|Np?Df_mi)M+e*ZGyQ+_;H}ZN)uzxGq~k|G18RSr+88=jK6?#|<2q zTQl5AimPqL{;C{i|HnPw}D8ygcnb#chn9-~a0`ru*~=d-e|lV1KNl8uADrZ%~y}NbLBAvtZ*#y5X;F^x9!%?TGnX2bl+&2oW2~Z>piEO+Gwm*dES|Lz?() zSRhLCRkQnWgw6w0y|-B^n`^qbXeZ45^1OA z+P8k2skff1J{xrhf68L?$Nj*OCb7$1b2m4v5n8uy(AmJ_A6ZYH_&|;}4idX~E>PcM zZawE|nX^JAk1wT&Xs@*605w(!i z+>4Fbe!dYe&JCn-2BLj#(HkNChb-?ou$+_|_BPWBUJOsR&sAk02;D*3Rt}lp`KA(c z>*j^CgF}i`%WoXB(aVz#>k8L9B4AwK1OBwPR7{RAI2u@w7}BOFH%mwGIOu=;u@U$J z(nmok{Y0%%1nsfAvT2{Q`z*9JjtleoY~87LW4`4xtoN01$L#cCC{)0`QDs(%K6Ww^ z{K(Rt(|D|8$H=M^V=Gl{I{TO(tnQUnza$q4uO6?IE_vs3 z7an(?RB8<)AV-V*w2>c^`!S+P7JBy_+b0_R&>K+ekN&NaYwL2N; ze`T~u(j>asL1*k-vEQ!8V1Cw{NzYM4m&)UkwU2u8$@Hm(h-t{N5cMO-Q%#X*OE01L z+amLhnTwOA9DQ|^ZT2j^`XwclQPw zYnYS!xu03z%saoowa!}5cBFRgs;jQ5y$hW}mk`t3P1wFXEL$Hn)5cS_bBfmACPH(( zynWaGu(}ReFZM@)j($_`Zf;=d;(SVyE<(Nx-wC@F)mB|>jCoy(jf+zV0csHgo7|z8 zmH)3HuN&+u#U}|tt^vnxi#M}LV%{n})XLK$9UOa&RSyI3F<~QP^;OozusF1o8|>G@ znd=hlQ1-&HVq%AVcJf;HoJR9sdnz9x54=g``zoJxE)C*Hw8zk4QAv8E-jw>aSezun zHEHotg1RnSM6s5I~ytxL5Z+ z?V|}Dx@_Bvsw7|Y#B=GA{!nO^dYdI~NkytHenl1uzQ}QeJcWY@l4>VT_jDbTyiX!0r0di zWpld;7QI|Zs-w92{AW5M-Z|+kp0bfP0jsJ>n_)?#&wn^lPZcNMeo_*t|Du@jgz=gx zAYAtO+nHNz5$f840S$vkJWOMlfZ^5@+4I86JakpY<`E}?sN?P-eY}LZN_wc_m$=8{ZV?}|BuprMi zqpbW~ecbLuO1g6`s-H75y#CgQL@2R**RWVB-F~t6U293DvjK*`&6Zs!ki%@v$d-rH zx`Vl^pL08FSTRsynI!3KRVEi8yQ-)5i}&mT>eIGtBBsQSLKUHN#0H23lv1?y=`YOo zMY;&}EtfMZuh@Su$W=>)?@)fzPW7j1-y%^~Vn|sKp!tkdsk+|EwvUyMvM`+SD; zQw^hQ^GIx1Hk6rvmd|>n+h#jO0-*?1TX>(crT;2gHm>)J?a&t3Hnwj;|C(uk=U!K0 zc>*4iVv!Ppp>J59UpzCLZ?`}j9+R+r}FC)W{rT&qQBq<+&LOD_3M?Q6n=bi@(2De`wr9^{An6G)T z6Su#LlA~Z;DE)ET#NuEk`e^nRSs?;CGQ?bcmW&4$ajOzQ^L@5ro1TWo+2-A`Hs&Pt zR6uL0`NzK<))eX@t;@)+KNDmubjDrap7Sbgu0AC-k#o~%Hk76ympJjtj%ok7i$~-4 zBl^_K2@+QG}og&{9RlzS@!qIxl$mIjuFmivxMQNVZP zp*Cuu!>5*!H{5dJY6bHpUQQ*P zQB~dB0h<+exSVoBj%TxZoU+${J(w$Cp7%$BO#h=UlB7S8WR=Xi)tP@n2)P#FPBoJ~ zCZFb!B$r2jco)qLUdjA!CLr%%KV(d`)ZaPvt;M`PEqRkES*D%Q76Eb%d)7v=;~H+gQ3(z=9cRHtC2CfxxVqB$x=|6R*sW7}T=l;oaI!ap<8&%5B)&-XgD z$F5V^e7j{;_o_1*aUo-Zd}v0SOcr*uRu4(@VK)D%KF3_OORAS5I{olv82;gT?oJ@o zW#G59wJw{6g5zc^;Z~?U&h`%T>imhsOQ@HOek!bHhn4F28_B-`tAmh$%zrI35Y&q* z5VKF6=+V*3RWL*Iohj)KBQ(ri`Xm+qaC|qSDatj~@Hd@bo+aAb4Q%pQ%y41xN>%R;$4?vv*;Cg=m_?57m9!K z)Tz8Pq&F0%qcb)FqdO*9slnO{Di?bf=j#K&B6y8~)PKVXr02L?+;6WEAUIJtyNH~8 zMTt_9B>d$Svcz)|<$NAA7{_bZ->tJQQ~o4Ly*A8$HT}Deofl)6dF@(?s#-jo-QSdt zPmcT2Npa+q@3ehG#{)`-iaaIJL7TM#6K$#Ex(X&%9(D>#)^j5H2t;cPWH>4v?Rm2} zObj)dge(g)Y}oik*}LG8Z@q1DtgN;*ZyRb9d+wR~Oxin&^EQl!it`@2JFi_Q>YEKu z4+Z}$a1tR*1K{n@(z++xAlZQhnKfeBdWlc|QmJ=B?%u#K3M~ayhLVX!3ndC4*YE6e z`u-PRPPNWnnWcd7`_40Kx4Im~aozx$({y2-#Ti<>xhu2SR7aU3XYL>9L~RO*nae?g{j;Ji%K6!$vhWKV{1D2oXsK`M<-Xm9Y%;RLDk8Hl>V*;hR#0c-;v>N}O~4yu?AsCy>`5z$XIY{K|g1gARG!GMe}wm=Fp8 z;~J~2%k*;u3Tn6J7EjbCD^MMnf0kzzTb>HXZ(}I%nw%VU$3fW5qV_znG$iBJM-|T1 z@jh=VKQt8`b~6>P84&@>e%O?`-+XJ-TVr@0A-mEQm*#O zRq!GS?7m;J&_8<1;S6+GyEv*T4wm(?%x*&z)Sfzp@aS>9ZD7R2>Z?XEV>h!SbR6e< zNOZYZ()=>gEH$;gHAF01ICu<^Ih_gc{|8XX0@L-oQR{!qx-Sx&!Y7hh6rdX3yhJEw zgE;QDEo=adXoC5$|B~KpBWK|_+(5KfpkE7t&NUzX(_0s7g*gpP+a1(o1ca5*U3>Tr0%wzR`RfqmBCHo63Hw#Onob zMiRj{#`fmkK!nUT6d1SdYZEF7TW(NpR@fk5j3x`j2RitY<{RWsjc9^12HUj3N8q0$ z>5iALp;3qU_-)qKwMeWdlcwTp@wm;^+f+MPo!VcYJBEy7i%d12{Rt1-1w1Ksf#uy| zuUz$ajRzmZ-V9ixroy~=!#&=yf`DiGX@321Ma`g1*|!WkxlKPqsKizPLKPw|M{LLW zq3}<0_VUVbo>RGWyT`GGz!X5Q_OJk6gs7`8{J++p9qFvn#^$^IN;UIu1KV6!dD%(j>NqCx&Qid3dq z0CY#mA`5W@pT)Q>ogo40FQG{;6p@O1o--mTpIq$>W{QO!mr4 z85^a%K0mu?*2|_P_r7XSmdm6zjjqvNBtydzx2503*&auiXs0=PZJScMr77)K>nPcz z$uq6ChM0=kG@J5$h3K*dX1RvOeM+-j#3WtU^15sKN9GY<5{Tnps@mlFeciy z&MwRfdq#f{9arpWcf5OUJ+@~%mQ6y_@(G>Gzu8cE=>ADw&LoH-5gSL6m=c|*9V~Kv zgtR80mG5h~ILd3r9j9hS$ptQzeq|LxEn?>amnml?cJZvn`g)K&mr+RW^vt5+J$~Ow z;tzfNoEb-S@eUaNE^@%&2qcz<{zEkSB6(s|%8UbCabrazWdJV;L_5Wa132tj&xXmU7>Q-cTQi*r+MRnCuy^%+ZL@E1z3tQ~neC?&PXoi!R9C$0t zNMus`{a*|y=0WhHS4yV0N!;jd&q!q! z{(}`N5>-);>{3hqHdQ(6DgB3%E0e=4uKpsD5B?1FpE$U6rub`$6xl?SPrq2>pZ9B0 zqj7Qh8yh!^PVCWUKKx-^M(}R#>vfkRodf8u;W`3CpZ{Rtk5Z&Z%oL*{ss1}vn|Fg- z_Fh}XHNoF{m)eIf{1&8Q!2ha$^oQ9e^4nlqpL)f=gkSzxqx8xLZ4-15kA8C!3NA~6 zJyn3L^-#p}(v3~?40Km9}YE{S@8^A>AftIch7KP2;a_wp*Gfb zYCG@Z8}1R#k7PCn&A&vrGez1&5N5l??3nmBOFy@9#hhOKD1fGt`cutXIHS8ir8@3` zYyAi8BB!DI#HmS z$8V6PbzIxrV6daslD6}{Fg5fEvEV8%(-DXLDxDzhNhY-dh8s2G?zm^IWvueqmht0` zd%1Vdh%9<)qXgElT$9}^UDQ!e`YBgdf_D(vpU=>Ye1Wvnt4}6>lT%G{+1?_ak@?j{ zQw|=aM_!}|5G)o;8EIp~qI>*}DHLzIU~s7w*0+M?kEhHQj#r|@B!wb=et!kmik9pj z5f+PnqDAo<*{hA2ZFG;bKP=4G+-Zz+pK34X^k(=E)0|wzsx5ropcJ=ow!|XicF>Dn z;EV3I{q@XP4Yz>LPLUIJ0Ylw%}1UHQBO z$rAbFr*Ti0PW);Cle`_O8@eRzH|!ZV@UXPeLpHZ`*`9Po;-?1*pURV>7mjfyMg~-a z!B6U=Sw2#C4|BWqSNTnr%}qsENOA$xS>a$w7qRi4`L@!pN~Yt!9>n^? z@qTDYeNXrf_g-ZBp9Dz#K2QOMJ(xOLhZ+cKn4c?4vT3E>-YAP6%3-7WH@yuyrHSt) zVT_Ay+4SEEo*6O(b0*dy^-W;sOjN*cv%H!Lszts<*-u?zFe7^shd(85Wlkg<IJniuW6yxqE5|^qmz;F*ud7}rOT5k`wUjuCFZ-o+WQtH5xQjq9 zRB7e!ymp>DhHjIX(uibWB0y}}tKZJ^RIzbfYn=&1Lpvm64ZU*dF3tJ;$Z`eFRONnw z`HOccMFHB{CC_2({*S8-V*sZp`!4QNITk=i(zEw^|D6{RzG>N-pe2r?ED{$f2?;xa z^97Ragg@rrxKc5*2-4mq1ud1*&E&r6?S4yNciXkWsa9S1#5clc_hHp}O1t0{btP}_ zg;3k})W#cHB9GRq!0pXqIR~-f_s})YAIE&toG%IF8)&zP=G=)s9-|D;Iwt0=Kztvw5r!#{Hh zOArZ^7XIvNIA*usW|G~i!KD@^M4$9)h`KQ>!TUfoWuuMrD?k>O@Gu6!zZ&xXCqgko zB2DdVR`^omiOrrND{J`~mpjAQuGQ9gY0_3yGgmQ`gbq83fqy9s-cI8ZkNHt8zgTz? zy4PAUu6KzhO71P244+Hdo@!aGF9Y(Y#6 zXKnX|r!pG8VbW=#eC#!`-o!?`^~O9Kujq7&7WH0kn}fi1@?;Kc-x>ZJzwU87M^(DD zYAb&BH~5s# zh`Vs_=So0-h_+C7f`1XYFOolwonsz`uFM&Im%A?Nio;!no#9K?HJNNxq!a1G_Iw|zc?2{k9l-LnoGdvQ0hbj*<@qv_otV@_VOJc(>*SHs3J1?w z@rf|7I4h@<_!x#bw`E=B90egzO?RG+f;>@g&c3CGB{1=em$IKCdpT& z9Ie|9Zb`&Bsp4kt=(iBsgBN+3$o@-^P^?8iV|dMu{qWeB659T|CDLd%4H3OQR7ime zhkZfPvWPJF%$Cj{fc&WZ(NtVQxX0k1BEzX}~YlxYzo2{bd4LpDcy7g6Kkm=D+a~{&D}ni!=M<=YUk*-s%OO)-!^Ha*a-yl{3+gZmeFwv z&oNK{l`NaLtSffdiRMv4LN_VX0#X7>>s0=o1^pOW_h~=amPNO}La3t5IPI z$w}I;+?nmYMzqrfuT%a7GLnD7eVZPsGOx}ez|QP#W~^BNLJbF|F*DmRVDI<7BBlFN zy-2c=8FWHTr!&fZ8YpGUBE(v)(qd#$T6y(DXwim9o@h?}g^f&YKgSU7)ZKW|ZT?!J zn|`l11HEeLL+gP~3*@f&)_VW8iPmHRk>$;PyIy2|G9baf@aS&@5P{`;m7itW9^n)e z2?&0R0(e+nUh5+cK>M)}Wv#We9AuBwFRPSa+%_|nir$i{z0m)n&yLEN_;2;le=pmQ0w_3CKap4b zO+V10`!3*5w*N;Eszl)<53$yMsxh0BWzMTrO^R^gVyR|Gl=2z_FQ`#fujOHYyzGH= zHq9a-1&F}Q*mfhzD;LI-9@CQgb0TF*yZo zFryc+Ss341hA)<0FW|l+`|L80kweGYJ1BKUZ#*FAvM^&qb3k81T^p+XBMp6fe4eB$ zM5oQj$Wp*s56eLmd2Fm?N0YPRNXxdj1!nUWRjoUkf0YkoLpM_a!Gx~rXRcm9H$w^e ze?|1q;0Tb7FD%fh>pIs>m+3MZ_C>^YZf{?#2a`xc>#Z+p2p=^^`JfR3WU*=8zbjMH zpnbY=pS~l|saZt4{El7NIXox8uDiglv{t%5waTZX#Wv1>x2WXtsjF)BhoPaRphdDk zd2~Wg)YAJzPi^u*4{nX%8P*}`*E1*#%vbO1w-2@*oIWqkzxRkt2x`YE`zRtd40zG6 z(>@u_I!aG+YOO8jF^K1i^s`BcJD)iTzF9RVH5}>fng=$?iO*ZxFc!vCDWiO|+0-(2 z>RSdMM+L-WYY-rasr`et0Q>Rp5flFh5%WA*vMkej(_p033ZjBz^(Ga1-r zJsbfOdufPAnca^j)Uss`^8f=vATjzvic|oBPTV3Q^+!m;`qghQIq$&nYxVq5+9iB^ zLZyo}#Qj_^!rxYwKV&Tz%vBZVtwSyu-u-N@)<7%pg_P9x&N^XldD>}%YuG4&yhB~g~ZclJmrFg(U_5k=okkRyALGqJI_OwQ9 zp~n939F!4`{K46ID$$x%*AcfcFg6+BfdWE-F+}JQD9L3P z&BLd|WkFHMA|odh{;AZGCt_0_ zf4N=lXp+lUu)KdPebo-?e$$!Ul|LqRdu83=k;nk49fN*QhLr+$C3cebNK)a8Y^)#wt3t){rCuXpG8cCd@ll3tH+M(X3s-i&->G0`zxWw;> zzlH7&A;Ol)ap>r$SVYwbUH~>oU(tvNFkybEMhA#=iUk@xEYC*`zZ=E2`a<6gn4d4x zj%L}a#4`1}MOztK%BhdPa(xDcY`& zbH;XVO|2IPmTaac+D(@m;l{J7>f))&?JiMWChe=wKM124`F+Y4%C2UqZMFCbo#)I? zgC;LR`)`MiZ>Lrqi4(+Z!0F(nOyOg}T_ve4(6HO^-w#Y3i1mO??Qf&`Pkt`9fWD2u z_B3eP4HBJC-Ty-X|F>jVp7jXt|S3_74Jm)+*?4+))zG~~@Jfm!@DyMedN!bM*6V9?G5_ESmsNf3qMSX?f z*B!+t)H3|zdest(slm2F33P-xH5k=qEdi-$I5urcB8n8V_4)gT@KD>48HM}3GU&+q zUh~9Qf;?x;9f>cGU7kw6h8T2=KeEkrvP7FMWRl=qtL;q?xUbi4f9Lr( z3GkrT&Jqd~{Ghj|m6S~U7=#B=nUbI*niYK+ZO{j)38z4!Kv`m)~BT_qu>MSR*RJ(McCRE6%O zq2Y1V>tU0`MTV)W(F3;Zvb+piN{dKa;Mq-gY@VnY{aSOFm-M*v_>k1mNS)<5*mf~k zV$(;^u_qo)e4MCfL%_WquRPl`gNaQe>I-zH9H62HIw5R6p87mFHawoxZ@J|s8@%L( z%Cg_y`^+8@)JR@0oqHS<6@bD$p@j98{kuJ(ILoI`b{_R`_N?0oE!P~@&l{JMK{mBy z54t|^*RZwEn3PK12TxvuNscwm1RXcMsc_rY^B|A4PATv(+tBdIkmHDIjqt-c&?T&? zdH}!su?lz(p_B3&g$bG&OtV zbC+Ve4onBa-t>En>S9z)&MoAvU{hn4c(^NuIDZ66X- zSrD5w#waa+{i8rsEm0H6gca(KSuknQpU@%MK+*OVfR)wk?s1cJQm>mhu~j>WPGD*0Io8AdjDDx+7Nm zSs=kuX3Ocpqiw9MNyj0)Wj%?zInBperi}*DE(lt+4yjx?BmJ^Ri{P^2+_35GdRYZt z*1}nFs2LTAyT8s2Z>~zSKET2t419Xn6+W5kxGlSfAG(&_#9i>v^md> zo{XK{(3iIkE}v}XbqP3*9oMhgERwZI`ZS%Ea~OFvK3B3jZM$|t9~RcDPf6OG94d?P zdTOko(1k(impP_nafrXdA5pTh5WVdbukQ)yKgO zaIs(sEQ^8PaYt1jC8+D#8{%pFVXT|}871zZdo!VF+9Jc;B4Fz>t5iGt-m>wyy+f`p z2fpbzdgXR!4jO)OSeA4=B=a3wR&TFxZ%LhQe`l__0^I-|Ua!|~@}IVDUJKhzdeHP5 z&gp=zH)gFqg9f5L$Gn0lZfU-EI146ZYI}I;673@WsWQuAP@z%NB&D|u-d4A?v8rh$ zX=<0UYoB%uvWphH)>zs1f^SdRv{*NQ;)n2F9J_sx=by}Re?-QAe~_y1K~-Mp{3g}< z`d0)|!>YY0O#P{x^n=Hx(uvknLkOihH1wva4>oI0;&~>bM)h=^m}Yqm8;n+4{BZ8v zWdqJ|xjJplTk1r?C^Hl~--G1!#Uk*~|Lf$yvfLj1uCL#sr2Ngv1@dXSO+Cb9 zsGi9A$o?ch%?323z@4#ZAAG!=jEsLtjFaW~-oxW$&xSq69@;&~3u+{NC>?!VKJL%~}P5ZpXMvs{*h&bEh z)Wf$0w&JlwoquPP9&vxyh2HWjO2DcgRAMA6c*S9p)dnj)sc;n`so1|KaUEZFp3_N@ z7!TadIy`HB6)1->?{eCUg{=$Cna59u?eW;O@q6k0A;I->NNot_s2)pi$VwdY9yq#21+pPoDSI4bjX# z&_O|*otzxYbVzsd7)F?0D;z_!A8%nUCV9^yu)u zuF1OMx(V~Rs`GKhzL;RR11;KUDa$8%5zApM8W#{P?;D}$D9-LOZIyBGvwkHu5#fGxOGpOwX zXI?_1&PdNduNeJ1024cfUDtZ=kedJ@++i#vEwc?Vu#3$b1(0W0kvyCtHe^>2`V5Cp zV#05kZ3Y!QyD}UuLI=6xWi<=UFYSDf@?FUHmgZQC^=R^f~_9{UT2 z~9NAHunB zsg$M#_)jmF#1pb$67w_*p(psMkNOxKkcIL6Nz-ZjQ8Hr#gz3I@Z$ri+s)__t=yKPp z1H#Ij*?ZbI2AV&=;$A6yxadoxqt5h{HqKL>>rb}g1BGvGyDN-Zb-=`=PsSE$_;+Gc zYBM~dO;HFn*1Tji`>*R}b%Z@m7uu((x5wYeFe;yCp3N-Bt4a_%S|z^N&IlyQa0Atk zczj*R55wQ4o5np5yf;mEQv+#hi)(%K?su!tx`=(i8GStGg>ZQgH<>E0KcF)YZa|>P z_{`2$4OWCU{C6*W`i`o=B+LPyK&8y}>P zu4Lg23Po%A_}nuHuF`&&n(@?iJa&dWwp~Lyn!RE&!Rz&#_SESbn%;S;C8cTbuYFOA z$Cb(^T7xsV54@{BPw=gM5X3rq@FjhPnx@N-NP3yuW}S*aP%ZC<<&I82dw&aOxz@#s zKV#_whvdv>FKS?e7nh*9S}g`qKx@H+Q6mjRk#%0PHRSfwAdath4AR-ke7^!l@B-z; z&|lh|rleRZpPH*`n!+R5s^};sw}{^zW&4X82H;z1XrkxtS!Q8nKkpTEzQgq4*a|Ce zJ=le`BVzLXZpNJ0a_M;bEQj3bacz9w8vLD;Snoz|f&Zvh#+kp1T67U}jHjl#EQNVQ zh3%pCCFHhM-7%2@#&W#H9f)6mA+h4@^z${KnbE(?&dK#(w0g$ei{urwGA@C2z|2kJ@ghqosK#yBXJU&K5^3I~=kM z9Kh@g(i4eHL0dI*qDuTi_eV7lXvj0;^;t~*MNz^Xv^15=P#3_hLe}NH2BWT6C+uq) z@hQ9IUqodqHQc4H4keQ(YJ?n18b*m;@7ZaGyjkEk=^nwM~Ta;V}ehj^(6_rRk5Vk0IE0H&F${$$`*+JaxCe2 zgu750LbM(pA_=cDqcN1g=_%E|a%{)sWxV3zE4K56n8j5I(puqJJN`a;=bwX<1RL5` z*G}=en^0dr2_qp&u@=7T28lGr)gyrK+*rFuATR)7W5g5DCzvcokzI;|R2Er(=be5* zlbmPxWYs9ZRm7n5tl~(}j%JUa0GI+zkfFF2@uEIBjd!VsY&oo#FRSSv54)p{VA?d7 zw_hr%tzDTk{-l{6;b}p}QWf4W>LWhc`aF4n9)AFoyi_p?g z^{K9Zw<=)rZH<@u-<0Bqy7qM)mNg|aJ=*u@1e=0-5ScO5lrRF@m1zjf|#~G}c zf$j5A@n4*t<%!FFCl?sM-Q>@t7QiV;cfX67x(!6G@p2OTEz0gcU);{_cPaOqlPoZZ zd#=B(5B7TH>BEoR1y-=aK_O}dcS4fQlG=3CqE&+-XxXRv$?wP^Y(5qd14iyMtL@Q7 z7nW@4EOGe-6yb{`9KPWD*^qzIe9#NXtjBl;PAIwdrQv*{A2TEu8O|(Gv?7e(ttupP z!~FW<+;mI{o8Q1t;1H94qp-M%G2Nw%FOh1FM0-ZxL1RQRnlU(k#qzq zH{A2CQ%Bk%Oh`Cf=N~p)gO;WcNnQjc#G(i|4tLiAl+amf*XgChH_H2V;m4w8`g%iW ziL>5THbecI!jT^ohwfGzBjSM>v@DglQ)F1hun1T`2j*viw65<{PHX382{V9Q>9%~? zu?v}CK8qKOcFp~b0OKCh%dC|8l`KBZyPKI5s^ZE?zC&7>8mvpV?B11sl>Cv)oRC=k zm9^2l_FlqVf0TZh_xX?Ed~c=-s~WUhj(5s_WUe_Hczs3GA+f@TZ*yK%eHVS1d9tJb z4&$7B#cfwFW;@nB+^zQ#SWG(i!Dj~&OB;GtU1!d*vDvj+MFf%fl}!8nd;PULp$XA& z>l{`{H`%y8pJx)0;l~lUJj9ngt!i^<6Rds;t!!T!gqocmOe;F^eIRfXb&G0a+;b~c z`3m~|2(IbZ6BJ<^i$b+<8f4!vj&6gE2yD?dz_lY2^p?(sn#~idexVuI$|c9lC>J3r z*}g_x>Evm!vvxhc-cY(aUwUI_hCk$zyOo4`hzuM9o&IdDX4D)#j)3I}B?6Z$9W-NQ4R^RH z3L|Rt9Sh7hj@4NKSuA4$(B`HNfh4_7*o+k`*~{QI*iv^{>tq<@@hq#t$fP7!;Y8ic zw9Xm*EiD7rCOsUI{{=360{-CUdYy6jUG)+I@6uvOci6+0Xwvez*8<}2ihjI#vY&z- zC?_3RgzP3F4WsUaH&VAAE~=MDUBVBrSuLEU2{2I>V{wo4CYZ!grLkq*MUQt3k0cBq zlg62Xa(GudZE{r)Hm$sIT>a1oIxO6^4&`fwZiYhbB&14vn*Y*-)$Hw4{DXApa z{b1O>03ySO`ZWY-ONvSA#NtD!mGKHKd_&jt@l1sqoYel7WXA|hCi&jR^hO0X?0_XOah&=_4->D_I&7UiCa_g)%$gYdgjAZ)KA-=E3Rq z>@^I_>Xwl@LdMHNXB@33QkcEWudRBW`LBsco|HPy=26;TWo+OmH`{w(ts?JaF)vz& zCwu_)cSdb$keWr#sJpbBHrEiHa1j9I!aV1QE)Yt5W8?8i?v?4df1iw&-yv*f_8}8W zBxS?b!9#)mS^jv4_Y|JO7{=;;S$c9rlCdsyJCGB8G7#nGm->+Eo|wvec}|^ar`y_( zM@=CFGH|xSb0Dj|5Ln1%M`z~fQ+oX7YW(23%Wqqg#W3&O*<@-8dqMaD?sgPUq!}{e zf@(5snrUvr{e%GJJ)i|}v~9hwRvWVS;QRz$Mo#;7+j<5})G3(AGe*l{fU;Vk_3@7I z#N)8^5PwU1f*G|TSj(k5a#>i{ql_9KbC8MVM*a7Qe(oYJ#W3Dr#{G~UXyw#$=4a7JRghJ%%`^8cB)ipXY9HIk-(g)X?X92mRh5-}9@G#jF^6c8XTWh{B}>4sJB#Yzr%%5IhB^v*ik$W*j`eHGEUVi!9C ze_*G(g~EFmi~5u%uBws|rI3*nkGM<+QDHZ$dJC{Paau5uo663_MGuaG$(wp~ z23#ZX+k{K5K_-EBw5w-Py@>6ZC`xp=t-sH6x7$tiL+w_0g4U}tntHza(Y|+G`PwM} ze>Y1k98GA;Rqs#j{eZ!O$;upJDz(umrfFzgG4fnmK(xcv3bYXA8o~-kF(%d3!kf2& zQm{{w{pV8k?qgr?LKezg580@BoiW7m&yK1TbKSl-TnMBVkc_$B2yEnm&m33${t|zd zY#fDSh6Trx841ng%ANt+oKtX?S07q}x9!a0*+8Z4ceh-G$6VqH0mp)A4Ka6EIeG$I z;q|5dWciAT zp9-A%+O-AtGCiov=4F17Giy8_!0Konmz+16G<&%=ZGd2(+PDP0Fr?Y7AsHp?uewg6 z>yi2q_MTaJG>=6C3G3R-_p4@htOvkKshKHqzLaeq@%mycgbTX2qj9$|Q~j{IqhLh* zn}-AXsu$eTMO7j@`Mt6?!YstD44_pML1x0JX~U{D%NYEoT>3ZWy42#sbXWWe^H#k( zR3W|eK%U-q(LjG{=ad(WnMI}HUEX+6eB=WB=(yUDT%R2=5gL7q<5q9p4;BGMg#PxK z(!6PN=I-ZVd!mP);-YaBrs(?VhtB)oc5k>jxnayF!Ai*scV+}-v4*s81rz2pB&*gB z>Yx4C-W>qQ7K~FAeMg?7WCQEP!XbS_JO^oH22FWm9MVsgq3w_HU5xM(2N5Q$E2y6UDH2_5C zoYEq^;Zjy{sE3HH5C4eAXOPc?p@TW!aD{%xWFoeXY^^v8`BjMhcC zXY`}+<@FTSI&akrsS5P8VfNl!Ay?B3kCofcJEm-jBi=xiQ#Rj9GT@d;2&_OX zaiegXS5?=GOE65*IjGGadD5|(%LfdsXHrB66-~#`^g!6h7uug{EmBS~vc65RrF&2jxuTW4FzC}zb zkm>J~c&~VaGa#(>Yi7>DPhbCIHYHh~Je2r%3N9z;jx6|Re{sy;Q&{0u3cp5qsqu2! zHTJd4PV=E{|Fu>xxIQ>*1U2my`lf*E)HZUE0u~B$?8o(hxISYrM7rlolPxY^5>wN{SPXRA%!u0M67s~4Qp8J!H zY3r^L{E?5(vakHYso_b5d&FzeYZ&L$G^(nm^{ZKwpNJmMU32~~Bi+B_CdLu^1C7>I zslfjzIrn0s-Xtq8I;s#J6#UMa;DF3NyW4L!+FTH|Od#4AsR~dvO!%DuMgdU~`Gr8f zl}*N(YgnEr0e_$WI}8RVqoal=Ou7j1b>jOeI80q(uB6MP#2N$dqp@_*O|<#a_kcL; zdn;DK{6YhMuJ9j>4&Av4_B2yFq^}{YR4I=-Jy90?`2-f0P$UW~t}(js(1!QO)NSTf zjZj!eN4gt+)l>cI6B+lmNr0U8K6dP0ih<1-PwT_QP)Y&PE*o<#kNk@5PK3yoeq(vp z8dhb|TMSrc#0Zk^_0rV3o#ctfNz;%I$XM4YE=Zy$6nM0bKB*)V%bCLo_YDORJO%UC0)6i)7`@jGq?x8n;vMhtp3`*;V1x;V0 zwi+>9R#$3HM+tBXM7!xeTm{GlHEbaJyXa`2EP4(tAr_JH=NEd`S_7os@L`YrDo2hzN=-zltDzm|~1&K9iD0qxqclqyVF<3~f zgbPHQ)O-Zhu9pF1GTf6DjurVXQn$qjruRSk7c2*KSlqaVu)G@rsEGX%L z)nmWf#`e}MxPXA!+mkqaA@(BLG*%e>%aK{P7QoP&@C)2i=nNvC1h=|H0xT(B4-WzK zm$Cv;ORpLy^j%Gbdf-T|byPf2(<9H?Oqg79Ipr&>f|07(=EJ#&51Sb>!N}GJjgQ^u z#tR=lQ1grGo0$P~PRR|R5bSA`t<5t(6$Fe7^ov?>{)#G7i|5lsslR=ZyXcm*yo}D= zzSe#{Q_*bexp`=lT2x@KAaMKpzWcbLg?q)dyR74++2;PZenimYm+Hjvr&E!&2fKx% z-G=rnM($Gq$Ck0Xgc7eGeaqhWL??0`xjV%rrAOBbhA#ZS*LPhzHa40Gt+q5m&*S41 zEZ*s1-v$2xBD+(j<{JZ;bH|&T0{F|)A{KDEW1}3`0?XnyJ*7>B*Y$`Rg5Ys{T#a?9 zcw=`Ie~Zf#JW~tiVR<)0zUzW1bQaTc6h{9bJ%m18`tE=}^kmO4+OagO4MLR$td+<<%lc|Hd zJVw5yj_WhbST%0|O8G35bYAks+qBBobWs@;fq9ky-74PuN9A<1e6zhz|N}nu`_qX3{PaEl!N7?}JtynYqCq92+NUhJVU#YG8s@4c^m<>$JtJ=8^SD*9Z{s{tlB>JIg%m>jn( zSZxrph{?>azYG_-SX-w$c{K`}$f0@Ws&CS#`{gh-pnCQ}ZKaZW?uY!9y=PQh5jDwH z2WHOSZ!N!yetcFnugm#4SPuZ9){W7w2j68TtD+Sg-{`^_G+M(Uc@x%^k`-r%*j=2< zaXYY_gNFk%4%_uxrW$&&Q;P8y3(ad5qpzB-LN>?;K3?hpNil-StnF#r)TW;MR(3E` z*{n1`MsyNWFYYRCasQf5b*~tk6gEv;1CYljQlZH4FKE9-Jvj3HO;-UVF21JO*y&~H+0CoR{N<(R-o@UKQ z$d6}-(zN_2%Q@V}_jXpY17=n2$o;?KA%~+>&-xlRm+t~Ho#C%Azr>GR?GiLvT)0T8 zOS1%T6Mm|YQGa9gYPn6td37Difj%3uB>iNMqD7Wn9_OichtOE+S&sOKt*PdQqDAkpZy($s7<@spoBC&`+s zQdM}XD83&`gLL-*A^Fs2k?n;z9s8i9%`lIP<`BsA%`U6D+1u911%r6$&&zS=4u(d!y`e@A$`Zoeio0y4Ra>pQzG9RR!VL3A^)c5jL7!rR+bWSYWSjAv$x#Rz2)`URigt=5#BJ`#*n1jedZRd4ETgPCh1ki)8{{|!uQ2I6%X@g%UNAD;W^`8^30vzVCRFb zH~9<3&hs|%x+bw#OlX0K~-H84Xtoe`WkVHERe9hW8H0ZBA; zSCKTTYRpy-EpCPfZ+;aJmyZ_NIw9s+zTu&*X|UTir62+mZxt{`cJO#>z8t-0b9n?I zEjL8ctG9*|Ku#h>5RdyIF7dx^PBH8HN+aK1{bsek( zcQ1blQDe2USLq+B)w7`>;)Std?70?Bgg1HZs;$Ur%;4l<N7-NkixCHd+=G0okoI5>QYbh=WU-#{aww09sN23e z8j!W7F2nf z#=j2{MU@KlB`Hvl-5?^o_T%;CK0VaI)~fK~zYjrzA*H7BIiVr{LY2U#pLZgQgsUn> zV*ho>lRhwHtTbD-UHV_j&7|~gL9T*YXJWncUx%cdf*~7PgRJxaMioY`D&TnHf8Kn% z_^(6yOTmz%{M8t{f1%0^B61xN@?ne|)!6^*kQFU3;(0(eEomnyD|m+ zgO%?EECXJ5<$hT@Uoq#MM0VQbD3GUh^%C2R*!=4{2)&lD%Cv=GSKWt(NmnCV7hzRJ zTE9xlHLl~|bXu>RG>`Ef?hfk1_hxA*DF}T47@jHk4a+qgs#bbkCqPYvXo53t4$LqG-X1>5BXb|rAjg~F`BVD07N1@-p_ zdUI!2bWO;NWzeO5GJ$OoH^{KLcKhk>JQ;1(R&DwIRu=W>hp0zeP%O$`^XO-he>5}S zfP%3y{e+LpF$15VgumLXO!At+K%}vfLFb2E=Cd$L9YPe_czs66jir2M=wGY;`+A$c z62Vt80#HR(slGi?rCz=4XpBl}K>6!Vmb9F7vjD^kKgu3=&FmteU{%;St`jQzT(TjE7ysID2ef(;&0+l<@k z3-*x)T(38Me{{__ay8V;e(cp)6Rn_zxpt^`=Kg)_{#q0+72t&y|JN*cZV)|q0krH3 z+sAU(@Wojo*ej2U&!6Sd&wRk;RZ-?=iEGAzTVFN!@+BZ}V128!*FKVi|IyuDGY?~fE*fz5KEJ$#ZKHwJN z(W8+c*XZykDDJyQY6UoHg%L%`ujHM<*O0{#2PIGjSc^bL=p(>A5T5brX!N+(7P$kb zpFyRrbK|-0>ipaE-|syj1~*w<)_Ke$L3l)_d69a7qVdROar`klps8F`fFf0ji*L)A846&ug!e7i-a zqqOoG>9+U6c_B*1qyrGnS0UI*NetSrWdq41+I(}q+PQg7 z>XH-4OIqkOq}g|}{+KyqL}LQXxb%Erq&ga1ZWN~WvjNoDSzny&KDv7^nMgo>s4Th< z)J_0D6Pfp-<7id&cN*PB(P*m}vS~48k>OO&tJ7)( zw@nuW>hSG;yyM;J09qF_{xHp~#Ck!ciKARyK8wX2 zC=J~!fU8T4FIG-DX&>^1<{Ks7+Iyn+Rh2qWX1dEI zseN_T*%Rfqr8#Y`=2^rSVWDG7Q2p7v4N;DEzax;5{twSI8uR%m%#B5pikEbQlIEKo zrU%ru{5vC$+`0ByU?R6 zCfYB@kMYLH+VKW+nLl#=Zcl(<)$OQNuVyE+aW-D;y}R?#bIguv5C{Cm&55lIJf&PC z`EkF@o@Bk{csb|}uXVqXVl{~ z5F(i}6B$KkEw*qXu*~ps^c`%;*+*|3A6N`LT6B*3sUcWMbhDntuxA!C7V0|Us;7Qj*l@PS zf9^D99KR+ezEEqDc08WHweDL)@djh=(_(C&Cd{g}JW0!)LI^E&fTEnOhV#YUPNJfW zm4}6OCEDOV2jVa8v?GV0s&%~n@@PV<4(<^#=|e2i?0BcP92oe&JLb28-}%266c7xV z;!1gptLN)5-NiK-<6a3}uV@KL3b&F*YpDRf;8~I+Oy~@x!smx|9D3HoIPo8Io~e0m z*Q|fmq8bFlUd76lFM^+0JbM{J*R)uqEo>vh!n@l=I+|onW+B5s&n*7*T`xa3k z5V6ppmE%BSP4f@2lOSL`Y4_c*>f-!|QyZ!TVya%<&EW$a-6{13Cm1eAV{b~&tGGKWtrs%~q znI3$0M-8u0b`%z9rq2W-D$T+5=HJa3*lK!|x9KK;a8;ShZnKL2_`+qk$f0pmF^nu5 zPqETrKj^E=gjMA$LGC9&AQJ;4x4Tf8QozdBRD& zZ_Y;*7!o4NfGn%lqeK;iPJwQe&0gW<6qoO1ivQp)lt*E2q8OCCd6)m&t2Jt<^$akZ zFf*4bj-x%k*@3xMgUroB!48uY6Uc=qsj3z91ALc$!WvPm=-(^b&qsMC2VB#{_wTKk zUZ$_iJvAD}o~)v`c4W}DuKwWL^DMIL&EMatPml76Y>t%Ruu8xv%^}~VXeQcCrzJ$} zT!rs45D~UyY-Aa3Zn!j?ueI0SSl1dmc;AW7IKpO( zWYq6#TKS%-S}SWa_-WhsD(s`FC`dGf-2qR zd$4w+1MR=h>IphC%EKN{)wddcwmDd03cY^-a5Uquvw`o*a^?v`BFjGPrbtQQYRFX1 z%lU!?l08x>eL>T*Bj=gH>!g%u_^g2pwJa@G5su`81pkRcloz~4yYd(o~?PgIdi@f zpBv6BD%(Lb1|SU^0hDt0xTs(n=26A$t3fO`&v)or1Sm@lJ9l3EK@b2=Yq_f`v@{x1 zG!|R=j_3a0<+cA7%0PM%Q5rEI^XyA|1iJifmT3g6C#zsz(nnSA>=MorCToux$Lmt> zu7f;HeG`FUhbx~#w0nLpZ8imV@`g-N>pE{}ixW=1zrxua{;)tXLok0s#xv15@aXU= zteAhexnEnecjr4y=Y~EiXG&$^U~vV5?~aQ^%FTM zN4K9|7?U<0`&1Jf4QC*?@gZKX<}8^rmybeSAqjf*ob$fQodEHHS@JH`Ov~JFz8<(J z-|wdR(N=>QaDT1n78U-!$@*NFUkx3&HuCfCdg-A8rTBu897KH4SdRY>2wdgvTmXQ3fw?uwna<8W-EJ=x@aGlg^8WmGI!}z&``Jts> zZ)vc_vey7jN&665s-+VA`#Q`}>qu&QThG|l8!L}L05UKD z-ocqwG5Whr{)>aialv;OExk1_iB^okrR$hWXYKC>00klh0!@bR^{NC=`+rdQAF=NN z0od3}Ls{J*%mWneaA~2X8-LC7%ANp&hO^`N5i_u<2o*}RC%nCL&HPq4P?PG?>ny;; zWdzUjF?*|9dd>6rP$Aw8`b|=RYT1EXWG2x&|1mk#I_SVdrc8+!pMy{z1wX!~J^M|1 z|KsodMERsv;JSYkY?f24UyS%Q(<8rudeM3`zaie6VX1C^wVp<^6kqmxsT`T>=2aijP(Ypq`^22lJRW(+FKhKSJo!~X&cbC+%M?Ew(qkUW6x++9;^xIkP!jar+tRZhxv zZ|T6KhcTKl!q`wteIkg8F~Zc9k=IG9FBUxvaTqC~uFn_BC#e zemc0pg8b~s3|bAGXvGCkv36RjGyNLLz!mob4TGvJ{Q-2fy&xD@LeUbf=&=ns8m~IZ{!2_=b5`oQTm!U_U71ul(wJxx8x4b}H>?|!dCzpUiRzY7m zgBV4l$lK-<9VaBb_MaN;|2&FMF@35Tx^McKZt$SdY1yXLujeejp#H!4=zm@uVxr)M zt-rb{|ARq(B71fIm$H*WDLbN#$Jc>aj)Ex0`j5FvcVKE+EY7XH4heNh0Cb@hK$bj0 zyD9O6HuS3W1*zqyQcR#e$iC`A`nmYd<-B`E2E5y zl);~q|6jj%w_R!Sf1d^L7qR~{jsGum=vQ)_y7@op`QO))D95H#N*j=}CgdGNptg&) zGwhm`JKh46)P>$Q2H0LYKDd7S)9gCkj>`tV^RYnaR$?BYhSeKBT)eKk-aiab94c8r z3^YT`JwVlaLNv_RD4k0=KuK|JjS*(F%ctPdb5vP_;hJ$)C}AuRicJ90h5L_YG(?yl}^dLu22I1%PRljb?9+3P?)68vTp*q_@k6I^!De$Yn0d%PLz_xQCFdr z*8eXbaeeI?nWaY|sCPmHfDUscLQ&EP*I4=*A*}HO@3JoT-2-ckMCnmlir>9il;ZQc z0n~%TyRbr8GA~*bPs-qafL#MdH`xNkBb-w979hc1JYY$28QF%{=%DmIzy=!*H`n*1 z(g{%1yUDZ$j+Xex1kzx?|Pg30T3gM&dC~0qLmlm(sckt>GF zKesgr<#=E{4+hvg*Ly&j@CR2%$crRnGKcM}%lnm0N3~9=Rn-8W9MRdZ>!8utubmS2 zKX=>gZ5*)V^qR0~oVq+4X>=Nt2c64rN4AKsCS1!~;>S_FyC{RO;cM%+ZR@L7z6`2% zHm|OZ8*5QEhyx%zG%|;v*0?I|>L6zfaWv!Uc^stZ_I`eO2n+0*1s1hkE72A>n4nPG zFpPgv0*6DbR<-dZan;FO$dsz0Xe_tirmde|+4A!`9RG87L7gMIp?#nS)VkJdq|=>I z3_zp(qh~cuk!~4Tz4Q_*nNDR*0J@(QtziEP%y`ndVV|b5^^WXNxv}8J#UiKYDtOx! zgVvYxw@ZnpPNlt%^Ul{a8rQC7ew}+t&N`P0lbiYml zME1LROE0Oc%A%4%bQWz_Oy*=*oZllj1EI)K$*sza5E}=f_OC9|;nzk{(k1YbH=gyi zpqdU8^ltoYI4rQaA0cm5cfM7dx0~jEq$M~3EB)}Ef6}Q1Pp}Haq=MR-p%9P#c)|@Do2TW4?i{Y4k0u=yoiBDf2&z9j*))C5E2wxKw}G;o?s}(U zsKD*~xZzk-5TIg9$g@v*pY1^@i&=#yc!A@avw?W~dHj)Z3|e@TU7xLIe%)T@w?Upm ztm7d`0?`Ge$Jy#eb%H~$aOnwHr{Z3M$K(m>cmP16hqYw@im5u_eVhXGILM@{1}k&_ z$|`=Ct7yDU+etAkjyj^kzjd`8F9!PSw9j+$bUAItVQ}tZRngNPlptg#!%DtQHk@y* zi(TzTun@g*Km4%;4rS|oY!#>HvTBq9kdiODj^Cxn8VV}LJHsgXJHe?IUufVzy+B)d z&f5SQCryTAkK@{zx2S{yAY2diAWbY5fDlV+=Yk*dc*R^4uiV>~o)$0d=8@(3rdMV< zT}zMfvKmh~ZkQLF^Za1u7?F5J$ZPSAFx$St(Pn9;I;RsDv4rRJ>A7mE2kl6>Zah!Z?d+ncR zJ>EEvRz6x*4iLG4x}<7=ly`|9jT(-L?g5??&I^fhifVF~(lr4vj@4RG$+2F&eU<9= z{Ifn~RkHl&jD!F-FIsh*=cB0ms5GmXQIGxz>Fjv_!qR4~Ag5wP&-GPv+FpS!rR*pY3Dj^IC8O0 zX%meJrIvehvJ;=bg&@Y>!a?w9j;|rn@Kon?fkixV@z+AMVz1~XsWvRCjR7T)+eVO? z^pE5F8jWYK_`?gkjVClTp?+%u=W`DsiK5f4+oBU{mj@)37A|_D`PMc^A!JClnUi+G zPv))1Nj4|yZY?g)?dvTRIopj!*8F-HMuvj$8tiqAsl*gLxGUG3WpK0QY{CFUqIvH$eh?F#nCw()d!c|@7$htAF>1VE3kDq6xn{bg~16^ zujD`Xa|t;%D1<8)xOmi7Za4r?Uv}8WE#q^lnIQHUjrYnt!J#-09lE(?_-Eog&$D3rj1GIF)-zxCJ%Ltz zCS4CZXBFxqs6=I|$#Q8J``>}mv}ah!cukcK$C5dzI@K9HZFiOZKJ>n-}~{PlX;%6Db- z3@a$0gNjp1yRjD%U<-=`e*!v)y zdHE$ThfnQ$_(#<0MxM607OEfHHI~<^GIz^Xx@`j(EXaO5K})jJ)$P@cXq}G}kjCI+ z_@|R7v_zWv4wJ<=ZR_He*H;@uBIRU$Ts|)X;*OQN;Q&})t*Bnw-|e6)6w`>In614Z zx>BEB0-vTTsY%2yF7&)yC)r-qA`!7#1_mR!EvNJm$!j8k%OTbukaJ~15-GmNh>IaM zlE+vG85RHhnzc`odp|;sx?Rb#swynCr_VoAK=*fA2nxrSKW2toXvDo%=x{YRIeqz- zN7b{$nfa~JTuD!ae=fD#&X-tDw^^LA;z{jBaFURAZyO9O$b1GQT)2iyTDFW7fF;$2LH~@3a)MZqeecT~q}fuAR&KDGN0p|ribpip}X4G5z{5AK_)*`reZc0{SKk)@dp|qFirqHvTpg7c%#{^ z*%Zr`Nr+YqMC{am)~JDSy{4nab78Us~HBOD3guB-X zp%6BqT|f*Tot#4QDk5aYW`Bgx_kO*T%-qo@nbKwh0JO7d%X=`iqOa#w(+)9w{sO^I zP|z7p6J^lH`R(ewW;(_Xms3Sc!7sQ6rFwRvt14kNdQSNt`0tExPf8f(EgK5Gdt;k% znj+NR4`Xx!@j)Wg}=9|cD<|(emFo-}< z$VhgNp>sQhsc(QrZbIO~bB7&- zFiKjkC;iX)?F|`1r0e z>%e{I7ir}rRW`O+Ww1{ag<;xiJ~euP<0O%=k=|*spnvKJlDCoG-|EvTNKK36 z;Wpdf;#3{o#kJM*rSRWsPI2xx@d$sqhr|X+e-K5m$i|!Y540&vjzAnhp@g2_^0>rz z`=oht`)}v|r(vu)h@9C|BKyjN&C*2wyOR9{xG!{FqfF~oL`cr}8kSENMB7Rwg#C0? z2dOkEI;oT4$bICJCj6xB9-He%v5aovib)#pylEQbR~Kia+pVX6?iqC_aL0l$+Y!L=U~DUP*Sgv(3kkm^udJOgGKTXVi$wt z$fir1W6nP5=}Sw5_YK51Vhax17F7iP362Q-b-8KjA!Ap7=&zx!k2p%nDAXDysFDPHO_(pJA zcFuvM45+DhmQ1hcyKQ?X-|hfRW)0~D}MKx^7& z&9Ta2k*=i3bVnR%8lo?%ZIxfnrZFV~HPmP(^l+)u5V3;o0JJqx2R-flJ#&FxmX za2Ca4xD1oGJM%zd4(^1CXGNqE{1>guN=Ef@;RmWE4DP@(A{CX`8ZPULdJ@+;VIv8y z<&L-vD{}7`%4bH$g>NY8I+N!|v`kYijtDhjL}3Y3$3ck3YzQ_g4mUwdq9L0$jG+t6 zwM+lALy1*{`C=g{AMM+btq`KqF#1wJ{C-$j6)%IBU?VHeW=cvQ+a}CVY_y17ir&Z- zdHl4h==pS=y85GCSlgFut-!7(A-(#U*-MDi#^_pkl+7NQe-$%w?8dR1nd@W1`Mn>V_#4*7 zPvH0sOplj>9!Vp4ea!T}IU%_shIr?kKheaWWx=>6zO$0w-V*Nhvc;n!hD$t6ISH(6 z^-(03@EV6q?UCC-wmop1e(_*%Ee7jao={9TI;CJ0wVneU!6rHR3f6{{UXzYvaJ^bB z!`>F%>+^o+Te6KNB9r)7VeDA<<)|Cz+u_ZKvy>?7amCQ|2&v<=0YT7OeN@uKpJxF- zWai8?!CEizFx=|dC~TxYo-KgG zrl28$gGOv!RAwV*S}GCG^{V-pw9$Nqp0JXsRj+jOxvF_61V{~&jgP;uSxyy2a5~%; zU*o4V#ns!hy4mWvct_sl>_5UHN;}!lKR#F0dVj;`k9SpHoUSx`9^nH~pbn9tN4qf@ZHCOZ#dq4B%SKpPR#}6}R;|HMH)AAbd-N6jw&XtL6L79q#YXp5JV?{2@wun|GQ8ZmcC4JU{mmXzS^uo}iY~l^+3g zr%(|jbykpu9633&WvqOi;!i*FKs>@m4mw2rI`s7sjg3TjNI)Vl-X9Y^>bPS|&JemJ zJdKPSGy$Lmw6J}eH2CpKy3ePHjT0rzxs;yCMyCxT8S1FT=y`QL#9}w`_!0jZV20s8 z+8-v->La18?vF_c+rBAU7|-mScDRzb{-iN{Xu7AP$%00E7qbg9FmM{v4t^OW;5q3D zZ6Xdgjd35vV+cB+1x8_G+CXGlIB*ntGS=szE+6e@_wl2h=%j`^ccp{Bo^Nr{3uz(( zwK@Blc3Re1;mz6}GCx9Gj84)$nU<-g!Z}3oA8!V7?7kkhdzl7aPU7S~Ia7nLvSX!L zg;*NvKH!^ZFcF_WPh9r4Y$Cp?=#F#5&OoS4tE@L46_Uexd>hn~&b7;e)}vKM>RGGq z^LHf#g~WB9qaCIjVTNI>M;*wbVg2$89#7VhZz=UUnN$0ip1zEWd@W%GxJDU&NGW%! z4=i1N6F!|!B@A%EV}+oz?&t6N2r5$ti_P~ms3MqGN9kuU88E+3?k@zHHnkEV>$%8f z!sqK>jJ?~0D{zP&wIO@6ln;dG$vdeA<|FLmT>Z|yUH$gbHiuqFa7u;ek>w9E(UKU1 zJQ=;PWUr7sD;|meVH`;3ujZ4q?0+25Gxg2EWII!7htuWPIHKD6nMI^xoyaUn<1vTz zT4}y0(GMXO&z_->jmg>b@V#gSrcTwmhJ1GR^G83ajn{+<>b4D_(Z@v1LOSGLKANhW zcirC@XqO7=h1;b+GIRR2-k+(bm@1r3^2T4o^ae_bA%9&eR4 zs7_4FF5I^cD3ct`vf{yPX5M~;_{VYahZfWN zGwoo=4OLR4`9c~+5*~zNwrSF$#6@2 zXR}vk>`61~eKNC&EU*DXUo-OCY>0|}EIfZDDSAELX69xIo$BmArS(p50`#LyRz#!M z5bwWq*kQ8qy`OtWZ^#)850@hr$UWe_JA9={G{yERE9|)`$vT(2zke}Z#zTC*qV8zk zA5<38ygPw6Z^a${EN*tDHoMTA=Q&uzei)iDYS2OFd9nMdH{A0% za}>AF^1Th{feo2%TrXTV>&#GKcZDr;vaDI7A8 z5wIZ~`Ew6#?usj7LvyqYo0E3P?uk*mYJlTza5

      J8QIrHgUjtx76)$ysj*zQ6uY| zei3eCQ;{5Zl|+9gPd;p5mIz|#oC*gR^pS15Yn`pd?g%|M$97wqn5Vh=&9=x6`LzVN z_5&DWQtKl3ZjWSGT^71NBrHEPR60f7PoBJO5FW|0WxS=ecQ@V!>}-~0s$7=oji^Mv z>g9Q*!j$&mrE2gK)VDt~XU*TgJ5xIx`M$J$TGVZ?q1drnuWowYP#e)sRj*ntIQukF zF-MXbzBShOrfQ}Wf9reIMmo3YcEZW@qy3a97XQgVF*+}Vb~YyJy~!tFKXv7k_=H-T ziok&mb&11XM8UN&(2Cc5k^?PDg-@-RFg|i-d1ECTZ@uX>%!`lmd|XawIz~IAx?9~f zUJ9#!;5Ngc){znG{B-QG!o%%aK%BCOw|^Q2sObtcO!I8ZPUB<&f9=so!C-RJpT}1SrC;fj4y-ME1Al%u~W&%`fIE&(j(isxE~+Lk*+v)vZIpn zuu{h|jZZ4Yji&)7WA98E&F5$o#>~BG-ty!T^S`X6fJTQeIQ<2bsZV#z@(Z|xOPJC; z)}GT>4gC3}KU}|8V)MDIxg)ABgk${ulz~Z#UTs@X6e0i9N#M~9$)P+tKP+h{>%4B{ zc0Gs64Ng(lFRc<8uhQpVeQ)u1{M6iv=r(+`#cDl-S^9%)wXFwdN%SOj+bxGV7%SOH zG*_{%q+7dSN&lA*nIN89U4qJ&x^F5H3iY>n_xGd760+Fdri8@P;VHIK;;2Ixs?;!9 z90~kxx{eL*bW>fuGi60!Q|-XLO2RsgJ4M21pD&3&j@R3^~jzzRkW?yxTYT ztQ|CF$A|))Fq5UX-*nxSq~d6FN`s2~f1IC9!ZzOgCfeejDA>($*UMmHbf3|9pXthnTI z&jaav|2R3QD&r(TgSCS}Ki~_}a_hU&JG}X(SVL73y}87AGmBshNSS`}84eW%6p^B_ zTFy`&7=wULR@rIk5$O=VWiMlc&HacW+nMuL+=!x?9^7>|XeMllQ=Q!??du)L${^Fj zuv?EE6xh5a@Xd9Y(C{0lr(HungYmg)8vJtro^zMy!#;SXGu!@Y6xm~=J+u}x*2ZQR z?^v4LUZDZ*!k!Z(+j+|OjOSvkHfD0|Ue46UlA}Rn9YLPjTFvESycTtYw41sSMcvh_ zL#56pRoIA)`xe`p;TeWJxILiVb?Z&9&)3iZ$VltyW<+%`ly13WPkk6^ye@sz(2~qe z@)|)#{_FkQl#*?cJv!{<-E$VC1xv$p$+qm8j>!yMsD|Sc`AFa9WQ-zp^4bIX@V7?9 zEF`o5!ds|iK~hykr4nTn;3sGZ&P`d-Sk4{-Rqm?|n0Z*en!eGldZhDw^s`?d$RB93JP+!G+waqsD3oz+^2|`Z)sKCl+VPji z#W7A#$|$m1OEbs3Z&?!9|J+aAul@b0^`YC{EondbJu?z)-vTZ^CKyl5J(4{s7E86r zk*fIQPP#CQqYscz8<1h!rdm#101l(8VB!5TE8|V0Sylb!t&05mJV~vL5Fe2{%ivvP zVnCl}_i;pDK-sJ9K`?qA*AmuEH39v$HxDkxY|Z_v4bq0vdN#@My%t>xilqod^w`v1 zYU2eeEPVGj_{rV5U8QWGm$EeFm2Nmf=87AYQ|B7$-s1uT?a6B&tafwtxjcz01id=< zLY~mLYQ~i3r1ptHZ&0~x6xSjs?X5|%uzmPVB)>3D+Jt4oF*V6q?!%@%&NmaS2oK@e zs>cb7AIl|stnUo#Q{Cfe-%p;N5PU0QNDClGn%;b>} zPU>BOP1>hQ*pk2+B8_sD3X4?>Qv{wqLcV7b3y9??#wg^^s_JWYCXmeC{lxt=&OW00 z#=0M4AKa|+B(+mG|3)|ADr1K77_@F@fk1L>xqEtjayc*?&M@pM6R_NU-n(OqZat8d z&pcXJWrq=fuv4iZf=QtpOHi*(89LDbG^lH`5h_+ClhEAezp8!uq@Dv=m^|M(&I>V( za#7Aa)_u6`UGf*VlKVC@=g*q=@imU6k##n3zUX__>D%%>abj?T1}7{IBe^iHZ4R9E z6vZAEV{4ol$R6%q@=lYSa%>Dq0$;4hc+uBs^0=1$`6%BVNpY=%Fd}QrAc1&Mn$j37 z#M``#+#&1P21a>^pWTAK1?5I>4E2y{(to)k8`F#t2!Jo6GG9c;Wx~S!N3G}LKM-TM zB5f3?K2Pm3cNX7nwWNZ;WF`&^@2XcB#UB%2i_-bIf^>jhaQ+IZ|17Bhk#YW>+IAaZ z@9EuM4AbW9o(Xcj5ciJE9!L+2-)f=*$d?+u)oAr1%Y|Iews~v`sAo?W348T4#P=CV zJ2;~f(%lSMc8_?Nt9Fkf*m4)z770ZU9JTx~W;t8)A|gW*eZ3$|G;qa=8Hs2g(=q0I zrt+mVX^2srSuD%{0U43Eih0hS_k%tz!Q{cP^>lCA;AEg&Pd`HZ9C{Mn=lbA6i28PR z^YJTqXW$N{YO{$Zci4X9nHAaxJYiWp;tB_Whp?ndyQ(76P!%srX^8A1M>z+(jCuD& zF|!GGK|s)-xq2>p`VIlJ)k$QqhKZQ3UvwP#9&B(P++kycG2&|>;slM57s^P@j%QV8 z<}7~l5c5#a6~D)KnzP^PeCNs|+v{{77ObWGwnph15-5^jdZ3=hZZ3O6XO$~ z$lOCT&GX|QQVzA;hxsFTo;E)d;)=J7a=u~yVZ(g+fhwoPL4)O@u2OdRVCOkLen!!Y z`oM5$Bxf>HYOnl8B)0KPe3mga5v6?5`%GWgXoYz-{VZH#u|7qiMoaQ|=MS;uESI12e?5emjgU6w#cGSmF9bYVf>X zT(i}y6DN$<-)fB7a3O8_XWjHKp|G7wL>)Lb2O$uF6yks>Fr|n$Oz6ZRknpK{F#nM* zCAL~|NrKLjqkC4#uB5YA5j>Ov-Kwqb6fQ?3=G}}nNs(LWHTBLyoT;kn6* z!=wajX1>Xc+TztiWh6$+>}$TezWc3*n_N~6yfiqw z4`euew*w%`Va`GwZB}&q*_$I7EA0<1%A~Cc%ZTAgfMNZZePxH%qzC&;gZf8))0ldTutv^_5sao-kB{oYUSqM7D*TyR|Oyr%%83 zy`qo!9lIS$FScHwr^R4=47!~X%}_@jz3eAA34XVLo03m0d+iom46~lEWL?+z;F3$G zU5kO7-4h|+hK$IUbklmoN~gNbc1vB4 zAt@=qjJ!iVqT{^Etf==cRd~3DV0Y!HL5Pt`g_*hq_@VDVv?qN_9>gwlS!+R~`3SYX zankT3y!>u%h+!|UZDbV%3DPewh~ZHgFfqW!Q=)Qvk&%h9vb*j>a+1N0`I82p&2S1J zC@nT%x+P7{du$ff|8b+}*bH*FK6aVCm(DTr(SajJ4qVI6Wbkyku&4lr`DR>iB`Bf zy((AOc8>14Q%}S2!)_+rF}D3HsxYqPLN+zEnr9thntjCJA8zzlKz@u0$X`%G&D0W& z_3rEs1l@W_jNSaL?nP~EM%^JL6!WbB4Bw`#PM?+>+7ej>d|H7xBJXn? zi9n+Eor(U^6f#2lT^Q~%snuUBWc8dN?28inSCLAq<;l8GRnhsBF3MZ8oyAVDOvU60 z>_Se>1PwX26ue@ClbTnUdu-dxgjxO}&IS)^UXAVqn%{-#eJ_eq9Y(g_t9#S3U&#ox zlYK%TkS=g^mF?ree#^&V0#`Jjn^FX`vO|l$A$APP+Knxs_*4=)eIfUI3on<1OAs-H z7(EZ!6c;L>`Ne-&u8x^jy*EJ~hqYNrkCu@j*2l&mdAkCbU3kO-d+3eo5P`%i-@SJw zY`7UkQidG3OF_|;4T^OxVW}=OO1Q6>fzJk$6l>YvOO^hy0gmDWM5|zsK1*RacxSap zQFyFy-!p`xLer^X%4ve5+q%aYX}Ur8D0}$hRwN574$+(7|IV4fv(9qvUBC6V&(K{S zyNew`3BtNC;WT(hHX04lMgU3LC>sgz#Zn0ch*-Z8cp^fCwZsxBTjuNQ#& z&dW;(u$^p>`(Smr@GMhl+;02F!vJjror7+Vm`vGJm10YYz(UDxJC*4%m!-tQkB7}C zVGYY1u+bHv&24MojMYwveVZ&ULA)CG%3on$e--Zi2H62Pu|+F<(ok{xl;=cdB6stEL`|$a@I8r?t49wmw1HXRLiI zw?CR&4yX7Jbv3k;EXH`UHLPTlyrBz!V?TttgrG=Ck&0eZW;=wT`Bru3_+Z`Hj=dbm zl2M3GWF@2I_iy5-^Fp0iTf0K^Y{@-e18(g>1v>X%OHpS=(F7a8r4uokEu-7N~nm0s4=jC9_UzQslY6B2KRlEaia(ffuDd9ubg?%=nSozvJ1 z!3{hI2m-3w_Enr-j^=A9lIC-3x1s>QMB zku1LxqWyt()o1a`RYe^W()_J=8dhuz0A*-k?4w9EZ>QZ}QRBM7#2%ost1{6opdQz@ zm__6B5e?(vL}%c#fG73#J2jf@d@dWDWbV5{2d!mw6FEI3l+H>q#ikDr8C|dTr)LZ5 za5{(WzyW6Z6VFGp#7vM@audW(ZosN0$rD|@h;|*E78jT)Z7WtR_2g~wr|>g4)C0gu z?f5{c;`&{-$9GeOKNYA#3RSsN{Mo22WM%G*bkps}v@=lW5>>iy2xoS02E6*7__9?K z0vdU_gEuOFD1ZDVK($dH;jZdcGDh0bO(jY~JCa{C7VcX33d+0f-hIe-POSx5v&+?| zX@r=@cT(=4ZhS;L6-A?po1hBoG=b|zj@fo|7O;kdbHNkOK$NQ>6nQl#jr zLUvQLosA1!N8cazM(R=CS5rT=eyD2C9+oA<_2%Ar!UXf-o&f{Jk;5jmkJsQ&{LM{p zs%Halp4*NZX}S7)vxN~*-45tCCElkh)Ql`%7FSHcq>6_p@=%YVMI-mwwZ<{Sp-p*7 z?vpZ@!IKfBot^ijxP9G4=+b6+r-4l`kbkg10SaZ$Ib_E! zVQ3h%-lLsph$;ZGmR3g*B2=1L7Nrvem*ijG(xzuttO|?B$eE$U?>6@9fS+s-r9FEW z`xKLTR>!Ajvd^|yTJ6Ddvp>;XUMyK4!>$s`KIz=}E*pPxzouz%eB=vFy@|_Sk4~#j zeH^|aE0>U%(Qy!!C<|L<-heyJ3$yMHNxRK^M5OV4?%FEQ%=$2yJ`kM4z{m;N(I=4p zsgAVk)!k0*V=;k%wpR%VuE<>=kOLkJl(tsmnk3Sbb*h?962w5I>Kc3RyAggj- zea@peW-E2;_z{hWTqg0fs}9JX_WQddpD4OPBwNlYw!73mj-+Sf8=7ZHy(ZQP4$ck~ zHz8ii#qZ+W3ruBotr8dZg#7s>yvLPnVGJOgUy;8c&}c}!JK6tY@2$hKY`%73K_sL> zq+38jknT{rK?H8Pq@<*z1Vma=T1k;o5N<-cBn2d-yFOvE`?}_uS+i!%nsd!M*NcL99qGR?elm>Y#Zb36<<>@1K@-_$V+e*-f&!dSJLce~ zJ(nlp3d8SYg2~LEKW}Ko6lX5l@G;MN1Vf{t8fmPVNoyE&ZXgw&u)49;?LI1eD6=jX zvU0L=uJuw~yx5a|+oqGfhQGsi@~C)a<8-}f8xz2wh4K3W!~w2~0<;-p+-U^gthav+ zhlpa@t=%Tsi^LQ*!{hBPjX4z$TjjY8{D7>1wL0x0igGu#_Xy7Ic#CxUN623zI31>D-w*h`B0ijQ!gSH8$Nn;k$ghdlVgwiZtizE{09cyLDD_ zb(5RYmPBYR3S|7tzuGu!Z#=1!0QY&CHN&lsTUbLi{CJ9#M9fI~B{(?adH1@yM0tDT zKtkz9WTMOB?n^oDrH&A|Nd)rz{5$hBwx{4zN>=i4cJ-r+E5|Lq`lziz61)xkNwTJwl;}eRFsl=L8T3v? zaznn9v;+L!oh*|fHS#Y%U)oZ54!xTxIm$9|Xcm@FKI>(vJm*uTs&7vlJ33$Cwdf!w zGF_jYy<8Fd)`o(3Jtlsauh_(jwG(rTc+0M)==0BCqnI~M;i0;Vq{YE2t+o1gU>Ju?&^p*YRiQt9-zYO(>1#z3och2+ z12rJL$xZcGg;u#U6c+ba3pTw>E3S-u@;AytQzfel_lFXfT_0M@?Lm+ zM990lz;W6<-Z5NBjlUhDJxUEW4GGbHAs2xJ@lN3_)>_Hty|A8Yc+U^P55?TOKT9JH z@O`ULrw6i?q8i4fN`}u)IYd&6^t$VnczVMu zGt;O?its1$wi_R6ZUtltE@4_f43vlQXE=X*uO>BWdk6sz!P_yJ+Y5BP_yu;=oX1V? zkNjPwO0o5pB8F{(tVx31@*iJ8ndN4D*n!=!zHknPy8;Ur%_lVg-Lf*&6Q#aob?#Kr z$(DxZ&Ie;?yshWC@Wh!V#x0Cot8;8xNm59aWZqrmds8A#U7@3~-2HfGLj8@v;vS*( zL3i@N_P{2_-DYvJ8;hZxn$LfH&2H|@Ghy%}ri-{Eg~hm4!DHs28kzgiGM;CCE|3tl z_7+~N(lbUJA~Z6gIXHuGj(T7*DAhr>Vfmv1+hG1xtqVRej@a?+2h_~e*lFD(+60}Y zl!eNuo!YQm)>(nV&~=>PBR$W#+)x(x-a*Xr(VHvS#GIi~pWUM7u+7utvYOLF=e~l_ zn!m_A2+FFb_$x4(8vVw%Jh5+0JBVu@t7Q}*M&Ka0BP44jNAL1w2^5K59gJ{Kvi%9yJUS?(zyMP|@L!^1`;>`^)aMOcq8!lhWbiwa8sx@)Xn~YWIreEWC zhf8oVFvU-yQgd-Z(1dwA32|S9-xPP=@%(3y$8rt!ne4gTErA#ZkUNjt4MWV`>nFv(R@ z4R*8sZOIx7U0xnZUn{10H&Qd1+Dd+nTl&e`!D?E{O5wVYrA2i<>!hZ^+=!|sZh!8R zAdyOzKqnNeHA#)SwUL(x1bp%X_YtH`T&tl&NZ;RiAwPOAR^~Fu+QvNDRmY~&)9Fwc z2|s<18%_*zmCCCO37;22ceHCVZ|rV?%$KQ*d9mjNBs+r~wkVE|;xb7o{( zr0*Bi>`V+3AF%YQw@F`|*2vA=_kN@=+({B_BfGkdO2mgDUhbZ$4~ti;RzFoN4Av-{ z2=P{CZxcBTUR3s3ZwJI--uB*dHeWV3I1iBeI$ z2>#}k73Ro>i?U$y%6p3tG4`d7h>{*jtGVR@3-^*FZ0~VR|2&s~Q@Exs`uB|?9k&o? z60er9T{3>3wkNZQFQ14c8gkEeTxg-bdh@-d`wl>bWhu#&qcNq|mxh z;4>WC(g!7n!PZ_f!AW{@(Mj?NW!KFE^hPFxhg@~jgO`f@KJyx9UOi$yr`a4i zMbww$2Ll@QNk?J}Xg3YGu~j{|cd)-No4b+wsGhd-%y&iDcB*b^Q2oqd<9RCIGyk+R zft6WYU7>2>g9{((Z4S3GW>tkRW#;OPOC4Sdt{$(`_Oc#0+E{RY95PyHHKX`R2kX=- zVAiwF5wRu{$P(dH^uUWQiT<#+$hL%}zwmABlx->0sMGXGC*h2Yc&tbgQz5hJ)|0g3 zEZYaXYJECcayrixJmh6MwPcvXo_`%NM`AJm`1SC-R%Nl{Rp%S^PXGNl{vYWITzRZ5 zH$U0dCG52rv+UyCM>v(!ed&BtJ*L$lqbzY(`-$O~xDqwN7owq+CBN0C@^$dYl3c0O zv|Yo6SatjgEDcQKKXE9ZF2V{zC8Cw_wDoYKNAtR%{xbs3B06G)r34AKYKQ2Qos;eRg6hjY|8H_9QPq zhgWrY@fqf|4G&H{+QL4SKE>}0-<9`~D5bqRq}EAM7#kNbH{mu{2mcm5mtDI|IOIoQ zisT{67IY4_lH(t>t%e6DHLEpPRd`06oyiDf1?vpf`N)$Y#7uh>rh>>3W(}U>9HMDP zxX1(jq(pqg1(!50<_q1srj}z$tkEOtxVR#Y!KG+J>zx#odo)NGwv3G^bb3fn3G;2$ zlS&v6jH({{mvM;EuTs6XCMB!qLX=hnd?UfcnG`IccwCGp-_sWFwRJC>vhY7U(d{K;svRZ|tfuT~n=F>_=p3$R$<` z*XE@O?ACmsv(Ml+M4th#@-(lG7W@#@H_G~HAVd`q;5qinOCS?iF-0Sylsp{1#UVBd z5{yeJ__o*M_IyG&i9K@YGT_$I(XUbNJrwdwtX*VaZg|LIBHh-%Q>*GIw3-zwgZm<) z@ad@jrzXC90+?PNv6*IhB^weLMmaX)!f)^<1xfgJ7jNL>?W+W=G`zyfB!5m3aF zxgq0or>Ga>Qewlsqp`t5rVPNh^-c^DaT)#(jysXwv3Drh7dgZ8r zL{0J3M)kv;U?)%E8_~+ISLR4sHqLxG&PbVj3c^O8eMlUH?^db+*}q0XrI$EMpLwZc zm(|ct<@Jn&l7?{PN((=&!Y+LSShis<3s z+LC)nb1j?wkRcTh$t`ans4fW!6K7Hd>s$RlUeAZnMEo&^9-wPT2?A1e-FNdf&_B2j8|PnkVBGTqG=#uK!aESpT}c;E8&6O z9o}|UzM_QxejjRTIPP`R@`t;%V+oA}_+Z$XO`|=-x7?3cO{K2<*^2l=j zGob%|%(w&NzAd{4(uY?V1xTP(-uur=0T=4OhfhF~0N#hU9B?3E{QLm^EC?k}|FbC! zbkNiz+V@fz1%gmhMK>&r{wc-3KH4WGVNx3UwjHbhX`qi)a@i)jfA$^oF_@i4x0{}U zj(i1+Nbi^X<3E~$*Mnb1DmJ>SP!a+`uS(+LR^$E<*3W406P5{iJeV{B(5-iHk9U9e z{TmQN;meZ`!Wad};33WX())i;{eMdeC`1w@cZ{370vK|}^WfV|k%KoF{&=iQ74-e2 ztB*p?cnE6M6C+vtxV>z~uf3W;PX zY!pkucojUv?v4N9k4{B5E`py_(mvBFNFc{Ty04QK#Q4W!FbvRlR`qa;$H=V8pw;ZK zk)MAGO9og-K{EcLnAa#4vf!zg1rZ?i{NDn-2j_u?m6NBGhcXN)`1svA(!@XXrwj<& z)^dRq%sD4|K>?hUR1fQC&Y#-&w;puK8-wI9{T~6rm;9>-b^{%S%U&Lsk-}FX9O@ z0m)nNIkifN8vMFC;eshySMOPVg50G9o`0{{Tc-Fwo%ye)bHKuPRxrla0_zCsUg$?2 zKxTIBvBouMM@TK!f&OO_S?n7D_i;C;?nkI!I96N0C>zuVrsKDji0OVGge zm&~6tz$3v#j}#Af1J3Mg%QXWRG$r<0g3sbGMUVT~x)I|W1?1%8^34`l}W#0SJS z`J>tBj}IbYJOh2vx(O8J7o_B#81KdY93TZSl08V(>OjdQfvu;wod~GOuZwj35s;N= z=a{~N@q4H#=hgQw|7q@d|rsD9c zKj*oP2E30d!VwSF3uJm3lOlj^xTaWeeQ3Sp*D8HQ8GZ)@B9u_=@$WqjlI{W6k;c%X zbAc061(r^V7k<)>>z#^joQJl+S}sm$VE<%556L}_|D0zUKcG$n&q1SXLSUMtIcNt! z63MjzCfx)-2}ElJMx|2fUZqCi%KvVzHq7;N0&d2@Er0lBsSeb(QUtd=k3FLf9ARyPy{uDW5O)xoy&pSSFTyH75kp#To{(2=FvhUE8qm;h8 z_B9~ggfJjRr})Ir@r^=2t@PWUq-&)00$zwR*Y#l+5I@NQsZb7IK3C4(;z$C?kGW;$ zT?u7?_EMFX{pK}O1%C^^s9pTDI~GL?YW8{jO4Bl``aa~_RA=a*y>6@mur#av@b_cA ze^0FhH4vNk4G&$gGKGNzs))aP-kbgXDN4mM#j#DFs|Iu;7hu3{?2hDM$^BDX{mg-( zt}0-#tY(0Vf$xvx7OZ?2xQQcRygdBjok;;7E&xQ1S9AaU77_*)WLIv;c>*_Q8Pdwa zVWHj0H4iA$KhRN|MwB2%F|<6reEa;Ds7 zyz^{S#7-7ny7Z^xQPh z=bk(M)G76V48cKwN*q$$gAXIMjGG|I+cEF#B+{*xUUVKmD$mDv4%0{uV){i)x)wU8 zDm>eyvO0G1C!eIL+c4))L;yYo@*aCJm($hU+~mnc#`xJzw^S5HX`nJs?pfYZF@T3# zDLOjii91deh*>`1=@F{eapa-8wj4|8;0hqs8>GbsBn4-jbUXMs-)vG3P;Q9>oFx{L z0?Oie*fUp`h}?YrB^C_UUQC1EkK-#Ge9tCUE%NE|Ce_aCvh~86989C*rlBye;_7pdGm^mm z!Vg^8el<_H3Z{EM=%Eh~Qx=-oj^8RlVx^SO$ z^7Ob}+)(GO>5@ki22)ne)TX_GxSbK6bw`&;Fr^kox!t7*yPpYI2snzik(U!FU2+8T zj5xk_g?A(-&XpF93j*78+WE09f|&np^o-L`wB$!$0fvS2>4O}8O#u$Cu`aEINb2)h zG{VR8_nVe8mYmv!r#Ua$ea~e*Pr8iK(h#*Pt{gR+b`Hsx`{Gxn1oRNGg!Y?fDH|?A ztJ8>jx6>c8VlnJ5i^CWaBGSRCiW*quZ#Yq|ZsRRf_ntl-u22CdINR!ZqM}jJ-b6_7 zt152Q8ckpJK}z(iQ^2pD%2o}nTzfH3+#r{g{;5nL)&(e?AfH$>xzst}lh_D@>~0f(P1srJhvuwjRP)P!j~&YwFzNkk)x%Jc z^Y7V~6wwa{u{G3>)NN_%sFXZSHlH0J;M4K!A#f<5^*wH;&gs%$=F%*te;V}T_*axq znWJxSulJVo>QP05_YyVcK&snxm7aMT7-z0_?L5(HCUKord@kR|9gqX36cQ(JbkqB{ zB=tD$I@PCSy1hnSXqSsI9WiyTeDfAn zs8HyjPjVqY{h!7daylN~@Y4jm>?Oj=Cb8z@v`5A1jE>j~Fvcj7jxc{8psj9+9}?d% zc^M>l8RHQ8Fe_aG+dTspd4*@PuUX`&$q;UM_(l6`fy{*Ai-2qv2X)z6>DMfYgRzDQ zo%Ho3N|fy$(;HMcyLTSN)jf74umYaqn3VaWDCNhKDeYV?FM4#XQe0nQ>Y6Jh{Cnr} zdk5B7WO$+H#EH5rW&9T4iOIqRS|W+o!sU?QMXyh19MtR?TgUBq$gno_9qB59#dfcR zzP~xfr{%|r$co@0tWOJ{q9MLG=YJ7)JSZk9`3_P&Tmh43sW?k3vP| z8}?JTvp{zF)GELpTumE(K|96}`6$2s1}?@qHQ1;*^P5DePm?$$NjCOiLSm7Xf*gRr z`R~sA$7k06YpVuM6y)pv>U>K~V3Hkz{$uZXOSK7w;*f(|&4gwq6$jP4SEZ9f7qnkl z1Ct!4A?b}n2iAurDZgS{h5gjHH@(oCCCH#DE1uKShQ!qBhw#JAzI#HOn~G*S%x3z0 znZKwWnA!BL+vRi2hQ}_=1-Mu0IP_%D&TnXbVWxhtw$`z7rJBzv|3F~&EKt%gBYi(gVFH$wqI7%WXp=}y_>Re zcf^0gIvmfG^5XF!;{mkrR_Abi-A7bUmP~NHV26B16?L0-L;Z()6f^i1#kSIoQZ&+d z7OOrS#bs1M4bXAgf-)%{E*SaE;+bQbR`OqRH+p!a@h_Vg-S`C@z$ohPNxCm)3)Rk@ zy#Y!m;Rl~ttlRe9etxA;C<-#G2Q&+v;+03}5`v=LhjlD9n~Ybj0UwT114J|&T#u;R zW-!Fzc?;L^P4?pNJ7-{%^{~b}Tx{d7ck~dt00cyz`&BPJPJWdGGU~Zk=QOym=Lw44 z4xR(aS{y3#OY*=LyEX%>U5ze<{=Kb!uMQ_2{J97x&CM0A-4Vh=xYZ&rxwts7=|V%Ru5*Q-kz=M^|ogS zFx)UDR!lz~%;U8G<};9ASSXEpQRO zowY)B@Fg^dC!t)Gr;p07`!BO6TC&Cbubo8#Sy3K1T$QF~f_#c!in*R`cRZT>%MBgO z72^fI60yS7<@Qxtx$Rg#f-9k50Ripfrp)|rtdf&+;6OfxrEjkQd!4B#um^q0O@Oo` z&E6?C`KeYe*kSl<+SiRBfBN3^hR+hKk)Xi>*+^b`VZtQ+U860h#QY;{(V3H;ijF8W zpbtR85@FqQw-rE*Wy8~J$4@l?JnJy^U!E;cU%8K)MTrv3Pe2y*xx8o1_Lb;9P)&}D zX7O`L)AUZ@#)z&y!u3>ZQ2esbCfm?uu=S+w)k=q|j}v6k^a6jm?9x;1h8zHI)ok9& z)_$s*cN1`-X~qFo?c@P7;O^NUpnpRzzdBI8ic8xKw{(dbX$WW#o?2BI!VFzo(MjmJ zOx_VhOgaw$S6&-2?|^YHu$$ov%0-KELej#{$ntVAR%+SLzl3gAD|+obYODjcnoEa{ zI^abYO+UQ^9$@{i5^0uREr{GV&8#~bXZ4=FN&P#GdhzS;!$h=Vwv-D=dX?MuG&G1@1*S)%c=NZtI~Y7UD;TtB*%fg$FsVK0O@ zZ(RH+*#bm>o`gq$Ii6u^FQEeTcb@tvZDR!{a~WoIo>{U}3(f=+y+M}QTX)FXDbv8KFO8;2gO$uu0hL_Q8oXw_Ytj(+JRiV5H+_XFCpxfZ>j z&9um79zr3FiZJvnG)fmW*Au|JcUcCEk>(p~Hj{>(Ko0#)Vb6()X}-)I4POEov^{ zolup5``umBeop5H!6-uS7)~gg%wzLNbx2xNk5i9U{&p$te1ZVJ|1Xzv7)8rqlp<(l z8!!k4PA}lUyW_V4+#N|%4*>^@^lxVGfx-7VbZKQZEY>ZF#1jf{zDfF5Yj-bHo>zj}l!Jk;Rw_E`-LfrpPK@Xt3)#E7c zRH38b^#Ncggro8zuJ5k>kpDZL$vsB^BL@Su(HPwC{tfV+SZKw**RwPI3V}4a5RGR2 zjc|i&dvnZRD+`&siU1}Fy$$=l-^n$ANg zos>aHg4M@LwJ5`H079gDr}X_b0Rn6X3&P9)Hmw#AY=87KLjB+U0Im;w2Q)=oAEK0j zGMomQ8rMcbx;B%*kLLsDn8}|l^iZS^>UzQ6L*DDIK8;@>z#gmcT8-&I2!RST>(vP2 zrvG%VAi&Ak0GNtGn~r3Mzunj+W>yWrLwxOPpz|@AwgoWHnNZ*3Vjfcd+4oyeFhL^F3t9xHbtCOzrzo`)^|mA{~IwuHxrC+nK}V6 zDT5L4)9(H;cc5vItPQQ{{}(3te_;}!B>xv-5`IvL_nKn)LGg#P8+&B{K5_#Z=FEXi z2<;sIStEb{2@?T7g=sV9$RQJq%7KMQkL!C4dVm^^23AiO8z`|fCcy@YyLy|yt zhGJ%DzhwZViUk4E_w_-+ma9}O|JAqsK3i+MFSR5K;oTh zJ8MHR4jqyX5-Mu5>27Er_cT4Q@t|@YR9|hlI%^oPvv*D|1I*>Blta)#^ZKrPq`r}Lrd!KD=XKj{Gu^VhZACW&Zcig`_ zGWBhRw^ZbBulvyK1PaDn^j@8-Rs$~F^ey*x;iG*q#6I=KZ`lD*kh7eb64n4AsLH|{ zaLI8%*4^I<@O)*TySt5`{AUal0?4Lxo8-KhMFZ*C_sVlti5rfk0e&<6G<60*UIV;^ zjuA2qal}h)T_sT9jPG<61P6$U+@(*?X08nDVCz~H)tXD&#|u%aAd+;}Pnfn7;Igha zQ@uN$<8K54a5a}u?9Jsc->9xbZ^n!oG}50~e~`V_YMQkRgl#@upuN%v&_J=BC9|4?v`QYgmU1Al{~JR%Yw{IQ4XVq&@+PYH1@{ z0k8aK=hYzy3$@v3?ao4^$}XTJftHTfXfqYxo~V!_AA}$caQL2dDXZH9Kvi=_QdsiJ zxqd&@AvRrFM1uTiMf^&z4#Wcts_h)1TWSZzQQmBJprF_)1O^EKjAtg~7yvxV5K59C z%;#3sWk7ZTRXF<#(Q@75jPg*b?vF2w>k&mv9ssP4#$YcdQ3){B&#IYwMtM!rr@m;y zum+_0^0sa~Z_hwDDdlo==JG3noIEv0YL-#fRn_=V;V{V}SAwOUQD4$`3`*Pe?L4MI z;z@9XJZSNrS|rJ>ox^sJ1IoMoCD;iHc9FN=R31^;AB7+w7G+hBPK|yD;E>iV*YN_8 zO*Pzy(`VBNt;49-B)(HXLa7#eXF&!Ds;ehBZk3X!0Gmnr3;jXZd)O1z56Do1Xv8W2 zul_=8WP17`uGrycsR^&tb^Tm2T#ei$-$Q38K+_LV z2tS`}9u3|hK}@M$!QxZT z&?$E3=!N+`4+MT7TyEnR~U;xm|yWIgEp_OXKeTmsihBoJ~!7STaOL z*Yp9r^0nW0)Fk+u%vqkS=$AeJ8Ifb#gG97~slD?SCb3_Kv8`X$jsc>F*eX7bK@-vR zfNB7|m7MB@c&n#&O5kInV!spFL|pA1!}MI5^G3dj>~1c8pURm{_$E?HqOrt_nVh`G z6LHfcdfHrXTJ^--ermzv$qbn*RfbzhhWZHvFHEgD3^Q#34pVLG&dl|kyeojzxy+KU z7LI&o`}uTp0-Waal}8Ha{Xm_dI=x|!5}sl^k zewpQR;2rJ=)oX|g26;6_dAK^G8&u>6m&c^s;Y(9Si(T66wX|1~cEI(GokXbF>tbBCC767d5 zc1bff~;YEKH zsWG)oIZyOGN$eEIUH>pC&}1dppxVq)jrd_M1X*aKOR4GFqS8}X%6L~V>6x(c_?EK4U!Ev@VEZ1PMmw3dc;v>5iRvC$^RhuK^;p$&k)3SZ%@^Gw zQ-6VZBT;tf0p^#ve;T7t8*Wl=&5I<_hVRYR&U~ncFJ>M0{ z`<5$PoM9xxES4Bv5_z#bp9pv>pX(OAbGe#+`B?#Q%um%VGH`tE;cS%6uyXR~{nd$k zc>9s>&=ePDGVB+RAe;95ceJ}Ku}%YP8j0%YtnW=9v|Q}rUhUv6&21rSv*2E;bK5gl z7c(_0oq%ONWd&)K2J~qeY@8?do)c1~wcQHZ_kVO{^WSs(Yf49>i!rXO*0)bFlFOzB z2`|UTUt=)2X3XNnvqKjf4m0gViRV?bgdOr9swH{oxbYY1*H|@KFpF@r!-&GV!Og0o zu2ine_BuSM0WX9^0qbi6)_>T6G?J@3B{o{WQI% zXX>k|IKNqoq*gfprcsJvr@HOjox#5La-vSa)^~DUjEqP5G**HfxZ`|Wn4YijSi7H$ zvNiM$csdV1UMN^t*EY~n9=+Fab!c>@=^En_=h2YmgN`az1g|0`8g_oMjJxI9o^T_j z%ZouP{m}f$oOnVN7Pos^5Ioe1^d@ir^fK&&{|l9A_%c>5ob;Y2G%a2j$%7nhsK#js zi?ukyWj0(rxd|wAh>g!xiNjg0JQ{`BYU0Zl=O(j>IP&S8pdo}wo5dlI(wbZHPOSo1ZG6LVww451(cy0QqZ~oGU0dFt zG2STh^Y-Mp;=7@$1wH#O@@&KI@1I}3UZY6Zbp2jeKzMS?G+uoEV>p_tdc{P5auaEe z_?9a*_6Nob`HYCyO%~+uZ;6OS=8+<=e8Tab)OUhzhD{#tEl;JM#&V)=Q{*~l_***t zq(S|)XYW>{$J9V$r4dXBp9ttae-p-{)MSq5(#f3r^h|mwQSa$l9sF4sL!7y+OndMZ ztXzUAP?M`cdY9xcCnw0#TQ&719G zY|r|T%~k;}^?a#ZjngqTG-64exihtdmw8)Ny@SyA1v90{Z;IkmvoSP6ToM{99%ZNX z$FS*wIKpl)te^-KB*h&8Q9| zy=b^XTnS-W1v_rYw(aS4o>FcD3Jyhi%oP|FQEcJqllxMS&v&$EEQ23Wr9ijAv`tnL zrCr8k-LXzHMNBbXcryzv1w2&t87s>Bd`W<$8b|*CE5uW9FGzD3gz5VZ`RqORo~D|| z<9hb_3vFPx57?(!n8^=5^$ii&@;yrN$309ESte$89cU#MXpY@MPefw9*?YbRSDBFF zK>p!yym`JLerd*VpRQj$7m8TA?kPTKQe0SpE=YZ`ciL{bQ1+(M8f8nbEp-V!IB@YYrHD`zIei$AtvOI{f)CAUh)Jg*rPrwllUlCso1bHwg$XUt1iIM?2Ff6 zb-ujCsxkTk7muR}OVQ0lg}O|2+AO+^zsj(Qv1<-J(d%JTc1qsJq%(Aw5N1=y> zc4hPW)L`OkHf}wR!YIdpKo=K7doNdvB03(OQ;X{yg*%b^r2YUgnSP)#$y%ln$GSUg zQ1;QG+soF}Lq)v%+fF5jiB5X7ai{$nVDaobE1=>)T!C=*^rL4VMZXJSDQ5d1WJM`X z*NqJt5TrI;){2GwD=vX3)i#fXa@=2Lw!IrFN3VKGhFZw0$=# z(}4nPZsh(x$UX%bw1#bjlhk%i7C!HF87k+Zj)ELnFW3mV9W%0c%7|F_+ z`}YEr+ZyBO7ou|n)|Ofv+6IcJWG5zHzl?ct@`)NpmZF#!b5mDbR=l;6siD@+eB$;scSE z4*6F-LdgraBbsyECXYX$l5i$1F_8Vjkd`G(7eNYF-tFS4z)0p)U-4}DUjD*EmzbT6 zjJ{{Dbh765l@iUo{KLz6d3Sw4Ma;e%A%Kd{r(=OXmjW_@9IX+t9ZpK_JH+SnADnJz z>vp!B>=Dvtx)j^epSSqH&Mo@1!EM#YB$@YOJ1Yy%rLv^i?3_aQvuC)(3h38kF`Fa= zaBL(Czm{5#Z9I%DVMsZhy@kXmr79HyoA16??Go}{U-iS#(6Cx5T0t-AEX zePw(vpCECbfy~u@Fc$7(S91RxT+{e9rO4b*WYXtDlQT@|FLKmy+joh{<1Ny?_O7MH zY|Tf59zE@saaeKdxP)tt`^n`G=kL|K_kHI)@_#VV1eg2apk;Ua3!ztpdgA*Q-YToa z%8E61#?V7P_*T)#c8S;uf5OXs86P>L)wRtRI>AZk@K*&}^CnNn(h_~d&gXF57TCt5 zy>lPP%3&`rG12L0P_8oA%3K(88BG(KSE8GmFo@X-8?ZAo{4ANeHwih0=0`O%U~xe= zg^7aCUydoqrkZT;dg>&c6pb(ycy)4R^dy@1-fZzaBeO$WqOy}9MVrB=oRCU1*(!Za zqSb+~`d+4u^iYhq(B?GDE~I4Pqw#8j`|Hcs_ zAVa#aBViJEStu`XHe6A!iSca_<5&4t+>isD0izOalflrOi(90=E$&(Mn-VSU&C+cu z;k3zH16HNKnM^S$k6i4X!@DJ0ysyq?j4=4KiYbWOzioVsLZ|i!IVEmb)t)zT)+cne zh!gwt`4zfbd_ec?Qy-Pbqv2vHM-M5?2Na{t4&~#_CnRmx6p!bStSZk@e{b!Yj2~2r zi6xKE2@t#1fO6jX;*l?oH{7T>?K8V0YZET+wkOT-V4Rp!_!vRq4aFyyv3T5F4=w6e zmMb4Gz*~9jU-4udBFP9Q8PjwdhhktM-onty0VvT|0DVv#@3I| zE2@XSt|e+~3BBVxYFex2a*rgW{YhmY2C0Rhl+RfhS6&tm6txb4M@G)e`g z3hfdMjBjA+jhLL#ToUJzE)SDP22GTIxE`7>+um&4z0aCpyI)gJt;?ouf^6XCY7$$9 z>V>)&(7u-#AUUga`cf zeHG6S(;IrzS6Hu^aSO~wi?G|h@p19p=kS}jkm)k{AM4xRP(2=O=-)*VV`Psbe@rmv zeKs*uxOR8MIMqMIK!{N)Wg$xb4k68+WL*=NK+moY2=fFgqrNrad|YQ&BFDV2B&;`zLj@GbtkBq
      =XZEHa)q>~w4oz`~baLx4>l)9((Wy_U;{!W^x#G?@yUz}=4?WN}6-C%G99ld- z6pKE7XjUS}kDH*VJ^bF}EZp-%s`7U7qs$9#st?BBR7h!BI4$PB)88#uW-TNsZIzs> zR+k~wAbA^?Z1N;bD-!V@Dk}KR#Q`ewQNq`-?Cm?AIfHox;FFERnTH zRMxcF-JtF2Pm+-x=GH@+IC~1k423J3Az6tH{iiWyK~{Zb2<)4oa;k!lqKH!((;g|V z-d}JhG)rF6S{1L~e~b!rJLj3o*b9+0R;@ed7KD5K1YB&QYlIq2MVQI>OzE_zM_#O_ z@WaHizz}86&td2UN~0eNLb`Y&bkop-)K^HeMhsx$O_l6)Z5O?b`PR-yqx>PYJUU3~ z;!Sil?6z%s+r=4$!crD&uILphwQ+=s<5?vI82+Aj>28iB*gavk-5dVdFI48J zA-3$L!n)rMF)p>0cxK_A1Pxo*x`TF`YKl$i{DKMANe1?FCt9ouok<=84RNj8$svq~ zMKTl0$D8=vmDSvSu|)UnBP7!+Z+7El=1_ANdhrO3YQr)sYZf3>R?7AoAn z?)rrak+&qcR(XHRw?9=d`N6JI^~=L?izU0e7Ygd91spM9F^z-(cg(bc0&564y&c2_<$!fq}A-myaC@W=s^DUz_Q;4AMV!NNWPV@H>|2y$wm_pkB&bx_Ha3-wjh zh;N(SFz{vd6ntk-{xGmrAL}*UmQO64lULjpn=KUH=7TI5wfFt=#MORDt}X)qY?6AOtb5v zhO;p!FQHJesznwL=5+`~Fhy_p4wt8^XprH{nLXBJ7sP&)dxvoz2BVHXavIYRJ(ovM zZXTYIyQxS{U#H#+PCj{NbK{FL4^%;jqr~pL9y9V#7vt7dHN%rEj)J~5#FLbjQtK$? zt#@$SusxQdxw9_AqmnPcSSu#zJu2Ij7yLn$jIQ`SRrQE>p7|9=d%eN2U1?!fePD>* zTVGhb6D9J?)%p6I$OL5DRkT9=37*+jBUmV(-R|qaySB-fkKGxjxtz92)ma~(0cd34q2`2pN_laK!uoC`2 z)rPMEvm0I?%tMgk-jQe8RQ5(zes$njhP@;y5{Yp7l@MJ~DeF(%WOi_f-qq4AS@)Z} zt0H>jtJJ5cr4QBFeHl&5_YV_m>A1{jRntPmFX5T~-lC%4EeH1~T=nlh{k{M@%AiM6w$o~o3m1r<1V~LYk%S8#m?jg zvv+b0s^m@Oq~Y3lZ%Dh^j&TzT?y^wgeiP`CY&liH9iM<<2T+GoGzH?{YEPa76={ds z996xI)~wLbbEmhYrCYKr&Vcm`zT^yInO?nzVe5zrO>bvVw z)RDd&ieB?2*4GxkV}bft>y^bU2S?! zuOH$Ul6e@aag4$F=DcjPm^4LkU1jUa5a-wXb&^W{>_oHO`6wrcP8hpP)P7GPo zLTN{2HV1w#iCQSm;sY9+UGU2MYwNfdNSlnKnkZqXx{8a{0< ziA+=lmj+S{w{KYSki$v$d=8c`EPG7wFOMR35G8H-iHO@JP}&obCCNf>BX_I#;az9nbg+cV}SJpkU8wW zA)fxNRjq@W9K!X>?{JNzQ#8FZH-Of*tv*`g|y!Ga9=3+l$G2`lAd|Y7d*`XSOtE?CojQ{3Clf43FrbySMsVe`mNrcwh)z+HUxZKe%7Mc$_s)iIhkQNJ(xG5b2Tzm6jBca8pY6rb9wt1Jd0kEe#vk)CS(!p7USp zoX4})`|17ketMp@Hr_G!+%t2}%=NptMUf|AO+(LKlR=0bS*FUX7!NX|kPm5W2;9(> zS6SHK&ni$9AHq*#FgjmPLA6+3BoP(*VmQ^v(YC@z`J^k9W(`n8=(*`(cU|3htIV8n z%Zj_UX2DE=frK{@txRo}*Y`ig;daKt@he`Uky(DY4KbW#?^RhdJ+`ULKCX@N`G{fCaK3i( zoF;!6)?{9;#b)|rovlxOttwq#J0qryiL@PLw?c#8i?e*SLk!e(mTCkErk&@^&}8li zO%*y7q!BooP(@aWY>-?y*N|C`?wa+L!xQ=NT0^tnzVm8u`7~}s0=4y#*OOcUO9-NY zy)lTrc`LsR)P_D6Z)#*CJ!(&p5?;d>`FUEOS2b+OQb0U><*R16*U9pN@a5*g=Rd1F z>gQ%GcRgtbB~35LJr-lVh~HPw%$d6*j7quC8(nfUnMTS<$& z@gTC;k_g)ikRvCiM6!M}ijVhcRkk2Bl=*4e-z7K>nEv~Gu_;DJ?0sz*SRAamN-EmV z39sl{EcBQY4T92@sja8NQ_a=k-L$5rnFZUiWo(A^3#XA+)mO&K#jwT#Znss@iL#rq zp_JNpZ6o)dYKENJUJCd95<&0)hc%aK{B!J(r$MF&%~{--!l0D08LKG12EA>Tz?OJG z@9UiNZ8r*TZ?;W34lr-@KpFn9h^9aH*qs|v&T;PK73euBbg?7!_BByUJD{>Kr~L;S z)oHTY@bP6}w<*T<9X^W6fA4IdI8SMJn!@5nevKf-Y1d7!CvCAW!9D^f0JCzA;f6o$ zeat&qRupS8BE!y({tSr!uvx#J-?3c5pg)1PniR4_1K{fCN{d;_OOhs`JY3;~Uub5W zB$44o)#}OYhMLeevJYpi6Le=>?afp&KLgB!&x9k#*QgIWOYh=*;8uz!bR(=FD<4R3 zm}P0P&`(p3k7}`q8+avY#m>ro+48LcH_3x+NnQKhgJ({=<>pVxzCOo%rXUfyx1rAw zceyBb?zQMQ)M8O`nOK9_5VfZ?-_nXlqoLqM;IZ6%-I+G!<`mdnRP{-3V5xya3{xZr zCZgLs6f%ZMc5%Brpl4XU*c@YjsQ2YWcosA>ut92?s{3TGHTKsKZ^)86cbE#QBLG+Z zn~N@!lZ2^Etn1Sd$|GzH$YI%p_fHubjli&#PM+dnoo2fFMO%?qKt$A*3rr zyXxKiNKZ2-O=B8)(M0c?^BPZA44TfRjP_b%+R>G+{auMKepMi0$CA0Ic(1)Pn7dGp zeAbapH^nYH3?tyYfKM-#O45wAwcgv{yg{Ws%i}kaK{HDBqW)2VmDC*KgL^Hut-9tH zrKXHK&;UZ>=S;b_J0R`S&RB+1Ngrqf?Jt@Vl%K2B+PCz|OoU4?KxvS&btmC(^Ngat z$bLZ7ozQHPc*zA*uF>o=+7S72+<&z@2!=dW3}=I98i$TM<(_9Xyy-88bIKb?F0f}#ivMF7 zGff%=fr6JgJLv3zw^uC48aoByVWK$?HtMv|hN}b95C(A>Cz_T1!|0m48=g=u{qh5Le&`Y~nAqJ_N=mQ5#hcQ~i)qsN(S z%HkqV5^>{ArS)eVlWW$)lVnG!9osU~y`61lji+}frrA_<@Pf2nKz~wq>u%n# zk=Q~+d{J?&Jd}!rk=rkCYxP0&A}XaP?E+wfOeaTP?4tO6Q&iJwvie$Qtbti)vTVpo z>z9pcWDT-8M)>K86|p3952=(SyC3bRB;?Eab>iSX8)qVl^F2{Lfos0}ft~@_&wv zZ&T{&Wp{#R58Zu5J%L&yR~rfvs7QeaswQrG*^4q{Z zXG*^1roDXg*pnP|vGNn8nWXHXLN}k}be6W=TK|3gF3oy+s?fa_e<^o@{^Q${xpA=4 z;z;}1)*xvNtnyY#W)yh?I}ZVED)+fjpgX}`XsFQoKP*lKaF*rklJsK`UYG*Z( zirlXCJ7mA*lRk6f(^IR<+iHh}m?@VjeZnosh0$40I}65de=2tR(VfY5QjHx{TEcjV z<{oT+s&1{z8!27SIfRFiO(7MRvSb1%V?_|^T-{*~WCI89g5Cx(dR!RjF8wIyv}<9x zRS1dy=FG|QRBYY!+UT;W2+;Z07qg78QdLgP6*h)gn!X?E_=9 zA#yyz@`Q|3!Mk|Hi*Pi~8M{Y3amViu!Cri7$f$-o)LNA)h}WJmRxD_frvVV~w)vt;#iv56{d%jT4Q zU5+M;=~P5>NA87rK*XL};z;jrKhvhQJ>oR=%dNHP{x%EdpR6(S(PeJ`2?AyO)qo%qPAmtD1PsMES)#vHknw9Y|H|WG z3bA7MRbtIr|Fxx*mH^uv10+fi=+pfrjoAIV0H5 zLL5qnL_3Yrk>bk`OBJ^p`>{Je<2WB#p%^l3 znF?Ado-sNS2W?92mMQjH1FUjB3|=Z1h@SeH4r zkKsEqEsz)irXtN_O4Ji({;Y%UOEamji7{6zMWwis^ylj)919A+Woqtbdym~ps=3l@ z99{0}WYo^)P{8ZRUMqbV0VAYR3f2)@g-NUh4w(x9#HzVS`6tU(M3PqAT@|0X4MH4F z!b#y3Dn!qkyw?<^j;SCFhY(()Ij<845rHki2gTa)Uvx+0T>C`#r$oS-Xz&Te_ATOs zGny<2r|cPwErJhH*4J(HjLF=K`Y7d~V!EPvAu+cDNupx93AVraZo4S|s&U)&(lE^n zJCXM(nYUbK-_SOEm_BPV@bUf&Wx7%mZn6g&)F0a+>L=W-ORmQyqv%=_ayeDF$JA8CD$ zNKvkj=S{Y#Ctnn{acTso3L)*nONVZ@E#y|%3bgn2dFim?@#z(|xf!;|WHhQOWX!!N zd~0ak)8(O4E|RD=%bA=bL9i$X3-|0{K7TfD2!+*x&;i$ytlmJeNMw)?S+@g(glhwL zEqI$B{mvZtt@F*B6%W4BBh!Y3Os?ef7QgDo+8{P>xtLo0ty2>($P%sqYtQ7gcOLnk zi-WbcauRD@<45KuKW%I*FjPQoS1|2=*wH$pP^vcdth+x{wU{3f(J_wZ zu!!Q1S-J<;YH_LIE;m-@D@qd__kDB~TBn$7f^OX~WsHTwUN2FIzRO;@1x7F3)nxqp zlOTXx@YUz*&%`;~`GRs9L*xc)`9iIyEZG875b$4(s3lz#p+zhsLyNrfoxLzQZQU}S z=!ZpAYhu*K>#(OEvz#(hq{)@uwDL}3e+p5h`1C&K-0^Np`+f&4?OZ`W*>a@{E|^aU zF@6C1WD!5e_nxuv*B;nWU(-q%+sYdfl2dr{o_JA)mcUF|?%O4coy?LMv&#oP>;(q|D> z@&s+yOt%rlDL4=>66<{tuev7uH9R{v3f&o#4GMUq%U$(g9x5bhakWjN*qK_ zyMlCtbmQ%a^=;=Bc;E?vzsv}IZ-~SUH)Mrkn#6%i_E9mI|AW)wAt%7 zk65Dko0mLWzWNeL#xD-GtLuE>n9M!kRs8ib;AV{+d54m=u+B#%^?8ryauO|4i>K5F zt=D~*(xjL$T!lCx<;G&Y%N#cahvYA$L)d7mDs0K0^Rb<$mZ?2Zo(ipF5$Kb^`wpgr ziGSoSjW8osB76>>5?igk*I7TSq_%xR1_^gcgA^I?ElXylq&I$&zl0r{no@8TLWIYe zCKiNl+M-I?G85RR$Fec045qE0^q!hfx;AYdEkYs*>t|gphU)^tHDVf?FEx zV9AJI%U$|bb7kc`HIwUeeIbIc9W3IVe-~T`=TYi-miU^Ikf9ZfjoOdQ4RiY6IECoM z`Ng`L$6nRU1EJXIly|8yn^zwSWx2_? zg^KnS{VWr%qeQ%kd5*u_*pKt*z)k8hpi6z}wSf@|a>)Z1z#3+V=fo#D)e8^}?O7`95z}s2xWIZR% zYHUxL7vg8-3$-*qge&Q#07^>EO-v!YC&hp%#!Z#+GyH1SX+ zW5yDzJCj)(^QOWM_RSR-ir$;*=%eGEV<)RNRFmb^)s9>pN6Pwer@d61!fR|Da(iJ_)p;tR4zIh( za5S$btdWUkk)fIxKBhO{>C!H#Ev|@NFgd?K`q_aPfUewWbHiWfbQk-uV?rw#T=Qh@ zKE2Q=$?AmBH=2#&RsW*$F}e&0g=%eVU;LZOht=J5!f1Hl>@hhTG#`7(8*$>q07b-} z$6=PU;@s7WIY_5B@BSQ@#*$^jD6npWd(S9UJX_o$Zzx_aDUNwJVtl3V(aLp=#jzgK zCk0LIVI9JWa#Oa{V z#BX*rBo~F`m!;cPjh3hrN4I^BOJK_q->0Zal{L_F$=w;{z`ENbDz#m_LFO96_oXt! zzCXT;RbL)iIRB}*{uTR5=$kI3UClWq9y~XxgLoQNee#@Ed@1pjPSfKP9^==W-!3|1 zKf-IwC|ud_GxG&Q_AncSYWfdagSz~h_MfNX>Ll+e7}VI1YkU?uR!qI8)3Lg5Kk)X8 zpr}2wdX|9N43lq@z>3evzc!mh3I_NFH@cD7O`L~6GT+b57ph=B%&|RwOj1qc(bY1` zQO4&$HApj5#rrm?T{o*&>7bfU%CcjU%Jp^0&CcQ{e&Hb-^Awo}6Jx@c= zcwd9{y%fQa^LSnrUs%Nh_x6IruN6h!X>B!5GB;lOE*B-`BZIsW509OOfAg0OK5k>j z@zN3-V!zaq68bHia*FMDdfl}vh|H&y$HK}w?cB(~{paGFZzVs2uvsY+du$qY9)7KJ zYp;z;EVpP)ej~0Zd@5~9UO&$`@YLj`Xhw?#!_JX}@ZGPaq`g*fFw=#6=D+qQV$y{F zI#b`|9Xej}K9WFIeJ6WFoH0*a;?D=3tBgo!U&-+MY{d=HmHB#fsCweP_w_MM=1K!` zI&_&fN42aI^&K8vUfDVIF^nst{Lnt`*7sJ2ruI)#QniE@cGawHA-OVZN3xh!rqSa8!4li)Pguo1V+1=3yzD z#jw3ZN4{=8`NhcXq1eT}uc-T}R9t9RGg%cKzR`R6=nOZo_j3`Kh4Kw{H0o;p(W#U_ zMo3d_3-Sm*LC8ZD?Xyj&v)e>Q3|s@jzygeWXX8zhE2NLKYAvHCQHTV%B((1GW;U=Y8RCm+L;MF(FcFtVcgO-4iS=}N8aqp$*cuj4OLYp_b zZ+|lVIB>u7Dv+)i#sgaz%=62pIfVgt18$h^#Gk+5aMfU5IA#KeNd5Klm$9IgtbW)QNcX2;3q9 zTpM%$2*KZx(Mo87WIb#E_o>MW`l#N7Q0f0V89KcC2U;Tuwp_gr{?`T_)f-=3`gc%w zJlZ_bxEC85_v0nn>eyTqI`~H@GkC@pVAZ?VKpfa29C+XY+gNcl{|BJ{Z>5F_4XC5N z;~VX*e;>*f9q`?kWk1IVqe3s@WF#wR{Z|n4AJ`-t{ABrY`X@LyAt>p{YrRVU8R-14 z5cUEz(LJg(fl@R%1vkj$NB#vk)98EukFe+;aPX5VRo->*EQ!InMIh<7|Mrq~Z-F~3 zc>Zx!`bGg-;i&8$pZUA69|Vhd_TOg%Cx&sLZ!0TPIp^kB0w~|E{>`0*qf$)Nd%E3%3(zVEG^6>EKY= zXo#EQ#a-a|Bt8UJr8xA6YW}bz$ah_ ze^pLzEB;@P5B((HGoqal*+6p*Mi{zcj92{Xt@^9Zq?3X}8M%mqhs%M^Rv^c6YxsXq z2Rz^`INEef3LwH9jjt0E;s2u$(6tBT1fd51CHc29BFBC0e<0iFKLi*86AbMmzd}5@ z0uXc6`m6D{B`Z1IDE7rX+T-fKF7w~j6C~wO#0mEAn3=Zx?H8*$KNHwHH;F9#AAjC} zRdcU_lp3AKBnAW+niR@mE1!2d~0QVBY`#2XI)` z`>VlguNr`BNiUC|MXxId z&RC$Ezp8kn0DaiXE@z0df363dP(j&npb+EnuNl9qi~B;GCa8Z{M}H@HFp(f~I2=^S z;qMbR<4T(SKdmjHCIXl#e;T@fKh7Uz=n0Rp82(uzT{MxusX6A{yADF#c$}?!fAVNU z!SK)lb@&hC4Wudp6-*LpBHR{H9)gJ{vK)N~I8N^Enx)8fOsSJ*dZtnV zl_ub$4ne^})dp(LYc~auBG!-2vg%ocjN*>~Fz>@gNGE=Ub#g_F#tZ$@>?T-xKh@dkVV`yf^~yY!DCZM> zfx~Y1(Fxf9MQ1cqiuQ)0I2Nx@!O%?YdXc@)=8$ce^G04yc9hSSnHAKwN)fG= z52K|^c{ti%KLZZ0oklvJFZxG|-&VHMC6tO!pM7Y@RsC(E@24Q~ zweeRfzyp|fHkmqvSMJ)o%bWr3LPsLb3q;Y+2-Cc-d;6sWkRsz+R-Rp&sWE6Bm@U7o z7fo9Rd%oKT3Ql#t-%a?*CfeeE6gw}pRp$My->s&VdWiPwq{Af^I+I!F*6m!dmm+uu zYLXH_Y1)Qqyta6q;(=js{K72K=sHi-N6sM-Fx&qSBrV-QgZwXS9b>Nmwt&A5-ccKY z9mv)J*rmAxPIP)y%^oQBvBDOAiFMjXkP^SIR zZ*s6(iwwSi?e7{XBx_P%b>Uw<@|7jVTF~qF)XqUcA4g00vn(y1*Uy@BIVY($N{gXs z59au|8HlSdz-V;*CMjN-E)Wf%&0)5N%-Zc;r-bhY%s9#^iXACpt88j|-buQtWr;c~ z3^-6?9B1w8E6v(#f57?6Ry?snfhh%v4fW5osMTi6j3#lD?f5lj$^FpcG!AX84z%Iv z7S)y@{wuF*NzHF+CLWyJ_;*l3U~kQBzHxoA~! zdk%)RceR3y69;w;oTs1X%iqmL_`l5+AKw)-Lopq%aQoFd7g-w?2ykq$m`^NwR<@eY z+9^%LE*z(QM&bmc6ivwfmCG=gHINX6Bof#(W%ga$w)m6k!q+?JfsF-wrXA*6L?`pH zQp@u-nF-nYOaRQNHl`{7{#DJYE04i-Q_i)jX{4~$TguzJKx&qv`c49&9$9S z5Cpmz^Qg~&{(22<(_@x?tO40OHec+o=B%aIgw{RSG3V^B3Y`n;QQ1?Ne)X0B!f!u2 z(5Wf)JMMFIoF`Sn9!Q8ZG z1Hqz`8|kq{I^WlF?PGmeG_%ds8`YF1lI6vydrav+b2irb;QI`Ktt}O*Rif+5pKoZ1 zScLWV0Tp$rtW+vzG>0pLfP}{*{WIr1_BnuwTu}uJdHb%3&9BetRFIp`Yk2msGLQEm84@nNgW{Yr*<| z1FIqEK*72xEVbQI-%sa`rESY9I#$B&v>x15nmj`O4!^#Y_Ike5t4P5xQJLAVQPl9j ziGEiU=NB+NPqcIut14<`9?**(1f$0d@2$U z^UyM$Z>mZAvU)vy)~voA74F$pT2EB)XREX`+G*}$c0GU-t8KMY8-u!}Ud=p-c4>sC4%)L#lwE) zXJLL}q4?T}&yBvKy3a|3pVYjs79&Cms%8xIUmJ+k8%{pB(HBGiEAKX>UMZk=LH2Z~ z-~o#`NP;sR6w|yCqdk#B#xwuo0mt5Y;=p>{TB;Xfm@_TuLb=1Hhu%T=TjLStzNN7n z8-y4Vda@{$(lhoT^MR=YK8E$pq8jLz)b+&3EVViNy#CVWEiMRgcOfKnHeQ0H((Fl_ z#=bZ_4c#A3yFYNM+*t^woiobtk9s`l+?tv(?e$X!>uhbut_&7>U#0(U_a-;9B<#6f zS4zH0L+BG-b5WbLa!!8Gv+8extR=yWfeB!~@F)T(Ux=3%AKv|RWqNw-m_sDk1Zst0 zyPNVO$|}9udv}6XhN@+cnX=b(^&PsyI8kY(NV^1>;;f3~f_)*Lu5iAWNARtEutwC- zOFAMwDu+_654OF(u`;;ZTDH^ewlcZNWTU;1o#}Li25h#EWJLRO2m}c^%TXU|qG}<2gE#@s zb9b%#gadn{6ECSKdsMNZ%iLAh-krjPhUKUyl!L4Kw3zfmrqO9M^lxN3r$kEX(T3iF2%p1m($}Qtu9v9Y?st%ohb!oUhTC3qqF$Ew)I1W_sWFKq zV(EL^-Hj`=mID!N zXq;^!>tx@}uXh#vf)9n|CklNE7!$T~qKu1XW9M2=uTQ(1P0rofsSWO5xK_IDQ_o;k zU!pWeLoB$z7X3d;-*7C+Y`g^Vd5nZ2>aP7n{kf^I!ve`Gm$}oF_V>3_n|V{nWV_KL z>}m6xl?VE?;6T9`Tvfh{+$rhk!Q6M_5r>+#w@suJyz$Sq{j7e~_zCNsmVIi;|2Xf7 zaR(XG%4a3~ddYMm{mN9>ec=P%k^@~Y=jGVRPT9n=QK4XY)GQciYS1>J`qBYLaty}X zf;Jq2x~6knpJ`M-HZBvW2kO0;3ze#Qfr(F{P{#v!t zpdxdC)m*~GN6*uQYl-Tr2EPeZMCR@m6ykgT!bqs5hNkx9mlqn0(@%-=Z>TIkl{I8UWl<9p!} z9j8J;!Xmujrnu%4CDBhMtj_ix-TGX*-$>kEOY}Meg`@!}BpX&6H*Y`v?1k!Ilf+<+ zPtUtqRD5!ATOKauaUY`(}IE>4UoBgHhwf0Rc`%h1E zjOC5rdEOLUZm6Q7o+5dOE4g;)xLV<6)kwEK?&l>9y*c$sYQg7a)|3GHMw zew}~zT`7rSx0=htLKVszFS!Pm&ScjZ160m_WHMYt8o0_&g`zx{Pd*wZGt51)zqGVHDCCbI=ImLEe4yi9{Nl1 zeK@hlAtJWf>9+U`sjo-&+h6%v%EjYq0U;Yf!+PAUtBqU1ZEm7I+eFb=kdPQ$wu1T7 z$fwh9DTtUBRd$xjgMwXOiPiiP30TslL@0NsnwJL)50Usy_2Jx5Mgt@q`xXR-)TT`R z#5vmaW(3WZAugqH3)OSRctrW9$NnI3*z3!PCILx3q<^&2Ys|gG!&f9A+I?64n4ffk*fm25+US3u5Am zt$##qj+OeUZ%KD&ftEBve7szWk$UAV-`Bg)D@{@+<(6()%%)QO7ezRPSf@V1dX;OV za!3jWZvK$X+7YF=qA~O6iLv+FKAKl0ryjzkhltq_q5Xp5qF`$pQ@S7Wmqp#LUKI6j zN}y>_uGYe>nEAYJLXFmXU;3CM#LqZb5~|sxF+OyI`-5&F9KR@MGe!ed=t1Gv1f~u~ z$D2@%{f{>?fSnrO>QJ|3>z^Xi?GBAUx|l`jOAiPz)m}@_2@3QdN#;?;yj8xVt&qxq8(=;0D%$XtNi5OI%ZRnwFxVW|Mmnk)5Cl`U*{s&^RCVGAyEJ9Omu_)q8D|uIV8M6av;n{lExWYWo=z^gtQWlPs8# zAbH}Y+F~I*+Rqr!Td^<_?3mfh`@vvJy~_`J97LSXebDy6)pIev-$)|ax^r5yW%Y#@ zmEXyXR^)FXbxUUZ@wZ{~Wb=msD2n#R`J}kP6m*6w%J}S%SmywhRP~7&snWd1(|GdR z$^)OW%i-5KRSRjOgUB@qQ=uy%fhS-_9{*uRPS6ucQytaXqM?*t3%rz1{ln$1 zX*(I1JtkvU6pe?cH5pivh&7F>QpQ(^M_v0citDe7lJDuU3eXMGCRtx*G`Ov&s{rx& zprw6VLwx)#6*|$I?ePnGaf_n`1@9ATB|H?IEn76c(}FJK2)mvDLlFbM3-|AXH{w}8 zNwrQ$D!+TJiMdF(toLgAVxY_*@Vd+9OGs=@kOZ{{%FIL6xBEZ~YcbUaVV~Mo_TKWu z3(eP72J}@Lrgdam-u|I0$B&^*Ekd`Zj0XH9AwBj(04+d?BXS;OEgLFx)-? z$%6Y3cAFvf@HVW|D$7qAt!QB61PzRAkSuomNI71kSYgC#M795lKDMPc57}G}Ol|cO ztyhj8DNFIv9M}3PGLF2kIpAOU;Pz|p0mspjMm#}CZd7hb>Cd*INB#^$1rijc03U%xn;iW_H$gnPZ7T$9DHCgoy zZB+KfL*>iY#4m*eqFbIsi*oy?d1A?+L@L5}r?WMfk|yZpupZ!=b{@b(`3M{X%5)5c z5mehut{7GdA`yND)|@H(O9M{r{3Ajq9kfRr9)aG{MxCj?hl=_4>HAe`l|3s8o;YF2 z-fp0(lBwDLm~d!b(^NP2?u(O*er$sntEH7!{x4cc>>jeS2|M99^47hvl)X387c#UM49L7mg`n1PGsTD2tmyru7{KcBY%o zJ|Ai+TDFT7nc;pGk5-ICrutl_iH8}FRE{#*=R}r_Bdww~FEm#|d|8M;!{3jp9LnVF z()ywHx8Kw&CzlI!5k+36?x4Uz2TfA8+1RI?la$;(5n&d`+AfTM*M*z&pC;TSjauJ` z-4Uauu}e3#O;fUr5BeA>2by#~rS5lMKVr>PN5Mj2182|_se!*Kr{zS~02=%8EgiaB6sI@FcJbP7Sr0k+5=5LW-H<3k0 zcvi`=WE~vUQw}Fmn0y3Mw*^0oShenUmguVQ`KeZv#7CDNzQ)WxnY{x=k&q6$jpzLi zUNv4MA%vd9kP<#5gB=xy5YtMoWix+(AjDG+$J)*a+lxXH5o#R;)gof?QT3R966|yc z1&%U(2!GC;ne?ovtcAtKel?!a#*Y}$ylesA=ze;>J)5G~d{}2*(x_-@LZ5B&Z{^lc zO>Y^zc6ygFjua4V#5YPg?9_BtQw=D~7tQ#Bj??irYOz|)NQ4?Jr{?nk0y#9+F++dBA7|WMOwce}-%_XY$A@OA!Z*;pF z!h17&!)i#>e}7Jz*C7H8i9p@>qfN`0Z1K}7g!U7T+cp|}kcoVa1uBg8E&ifuios** zJGFhr4XJhh;!|VPxT{0$6&_xyA`Gg45y2ZdEy~zpE}?7R*?e8FAA8->75CSvM7JIP zd-XQM4OsF-cg*2N_U#cpk;i^t*XV8fI-5p1))NT*m@3I?4!oc_Q?Rw{wHaUaE~$Ol zT)SCb+vXX`KHhD^`+d6Gv5HP}CF2RU^KPnMir?Q|9pX=WO2oF0h)G}?Gc{Q4dKSdG zGr5uw6V9Yz7f)H}G#&I2K68dCojKMXAGAKi+PAn-s=Bx4IdzZes6Z(-Y1m@q@M{qO zL`DE0f@~)i*n*x(SiS9`f!fH)c&c2KKKUSC7G6*+wrcSj!1sHsB%>Wmwt%JV_1OZgu#c!qXQYp!OusbuWQ;Z13GIV)Tj9h5~^l~ zSCI&ts9>Ipxp=X8Qa_DyO*G|AzKP&k|L*WPoQ-|#O|`f<3iIoS>yp~a5~S8; zsnPrbCKi5@i#KA8n4@5?QDpA)WALol4rd7+q0!TY2WN0GP+?OBE{Jsl*eev`@Yq>K!eW zYpdDJ9zotBRXs%?AKOl4qN?Mch8h_i!`Cdw6 zMRe`R%o)olm4GaVwOF5s#K1JCt?9M~v$4<1)gd)nsMh?rH3C?HQY}-I_Vubc44u?% zSNB?O?Qad7G_ZjWw!7Rd?7kys!XyZkks-iC(y5%$^Z(V3=FdZOi4&xM8$gH)vU!8= zlX|}l+xn6-?M(y;G4GZ*2C)CzO=gUd+gxGLE5;rW?XPOj$Fu7qRDEh_p9Q$8MUR~H&l ztS>$8+|5&RuZ|nsd*ZIF9|cpMP~OPuEzF6o*8XN?=8;`_Z%St(Io8jE8*oC)9?UKM zwAwB(`z2x$=V52NCYEg=he)_7>l0iCBhkY(coUdMJRi2J9B$m5LB>ex3{@LOrpnx( zzVv9x_FbHula=04Ot~+`r>8ofeZOnCZy?Q#AhP?Fo7j%YhcMB5*X!0MNKHA>iRw7aVPI_N*WO%n~U<^NqmcXK#FQ}g3`^6oo zLiVu1C&xZ#03mAl{yw~4}ks6=_8bieKo3`E_H>iWKIlF6@+$zfY>n&%JGnw4x> z?GL^n^s-ZS=}P}lQ&m3Wa{1bzcC0-2nJ~Gq$*hiHfJ(q)QIVr4X|eOk1u1HN;fT=i zlhCIh9G+$b1!5^>)89r+*5zGymc50SAq?N%SmwQ^wyA^n**VXi(-SnYb!9s(O@!h}_!u45<~Ch{=1&TvJ_KVwe&-aa$+^qV{s zSkQ8V3Wj*btsBuO)x)Mj!pK5crtp%T)ZkcimNrqn4#i_BaaXLtzk8frX1w-Ss;N-w z$JsqjiW?Dcj+!3`oOD?Bn=ih}<@aiCGmmz<1L60$DV+GuHB}_eF>Bh)?wJ%St4}WXwewnT1S<_IIFm;y`@N1X3QDf z7EwE*M=nH%$Cw72%C0*kCk$RP)QD#V3;PAF~+4anw@W-^JZfYtD8k*^;rVU$3 z^>%VMbk#g^Kp;!E_luom=egwZ!X-*CcU6qW+-VO|IEP%;z>XNud`rAA5nsJTe}D9b zd{b!L+3K(Y0~?ztq&F-`U{sy#AhN5;hbtwH6~d2M@9<6E)f!ELSpCSR2&!AtoWjOtmW+pBpg~vO|L9#uFy(3H_Y;VYcx}vD5M@1 z#A~=xKw}`D_DK#cGddIDm5vg+);8QMbNBZPNvr4VD{0Jm(kvM)1}CFfCZSnthGjX7 z@dajF=d(yzJmfdCs3w#a`&Su|NB!%FrJ2%dDD9CVlOUGZPFxEUM&DkEcCHRso?6V8 z@DQLvh-jx-(v40Jv-fB3$k>?8_ie1k@3TT;y}xv}e90D=Dc#MZY(K-3fE*+`$%_m3 zJRrGT#_1xXXLgKoUD5UK@a;!Ta+LJOxZO@4HJ|V0Sbs$yBW$ui1z!H6mn{|%#7{`4 zPhJILc-7Jjk^Iu(mLP6V_8lxn?j%;3sM4+_BJuFFgfoocaQYQ?uT1`0MbX2V86;6t z3t=g9?0e<7XrZJ>J0jX$&(}14H;{DO4SD4o+ERqT{!o}ru9rKfO7PXT<#vu>Y|nJW zW!C5)SL^b0^%!Rj>n#<`$X44fFjdci>9NvYrkn|_S6O#w#r2kA8$ZcOBf0XOOA0q1 zV!fb|Orl!5d|S__h0ko?ZyOc5*0>UKM!4BuPn;H~!ep|6q-uR9wib?!B1i&9MV+~YfFf4mk8zVzOn-uia`1JeQxo6Qi}fk8kb z#p=lMzpJ3=R#PAu-w|fmSm`*GKflFlc?a3P<>bJ~mfxn@C2g1xGKrzD^taUoeJ^>y zLC5iyxrjxmTS`E0J$ssK)3-O$B0>`T{9gU^;t=)rA3;7Je7!Sl}dh_*h*%_LyrdurX9q~D$s(EA1H>J?bu zJ+fBZD5HiYR+UUhTZQ@6mqEhRR*?q67jV&o=c=r_FMh;iE^wYwH6}Y5EQpDd&HcRl zNlPllbK^i*k~@T4VLtHKRwuqkQ4N(c%{?_bM?Fl^v%W+%JlFj7j-~4dEXgF?_PR3W zhkdr(*a?qRM9|WTxZRwplIw4;VEHX`wr@g z&dH!#%?Up4Xz$~9`9S>M3>Cz=5cKUCu%vr0S7Ym94i~vewk6U^X;h82i)eoGFE5>p z+m%LpZ7`pEIb98*XIzv|&f^(&vTPw`{)!tTm>vpDcx2}s3!WF#jHuD50>Dz=s-WoC-sHz&f#rC|~f zR5Z+)8^a79t;^qRIq&m!5tF}Oyn{A{8u5^NYU;G7f$%O?EMvVmzeQkm6-?1G{Q6!0 zDOP}ez{Bd`UUr@qB|@U%+X$Y;=3fln1<_rVpg+b$!7?3tjskjJPNw*TR~3BwA)>nX zB5FbZcw1p~koBog;A;w;K~FW+sE)If=Lwzn5dW4wz?$&n5nHgu5pt;rf7yX}bQTurN_2SN3Jh`-}WX8qg?D zdTAf7c}@FL9hVtibMI#pa6hC9LrK{&NC2xd`{`|k;YVL(5$h!dz1A<{-SE;UOFsCH z%2RunBlwftMQ+g;wvv~c?berJHJYsk`V+6pK2-%b%~f#-*ooABam$P4`ZH#Ht>Xm- zj6Bs*{M8NqyNM3%0#k)pq9Hs*pnq*oI|Rr_hB5((Ur4G1Df`gd{%z!3DX3bH%$dBT zIw&}``}di*pMlXLNAtlUzUPVniC0L@=yn(XjM$h(cvch%^z&Xln)vWUC{T;8^2vXT zfI2C=biAh5S5ukwV}r!JZb(frd9s(V2 zen|{RS0bk2(gk&oD(Y~<6g`SU-SvtOS^*zyI8u*AELe9(=y1Psb+5wYDVZ%E(8}L+B2ngNTUqH2XI(+8&>!7nj|^A4)P!RT zr8@Db9rxQy1zqxcBe7=F=k1qi4MlvKv;~RCOO8_0Q%}*GW+}%mPd&Ta%4PE{2YH09 z%4F{s>%zC3CUq+CoK~|f6Gu=lTczYD!tXmp{6FoTXH=6**Y~l3XaoeQ(oqmWib#M^ z1q2ZYO(`N>KmXDI!HM6s1Zh28c)xO}ZjI^xk_w8dX}uk zTC>Q-%#}TJ&CH%_|9{(Z3$Y?;f_g%tk(YtB#2z*>oD4thC@G={1h2VtLZ23yS|cyl zmF4QX+NS{B>*cx{-CJS7gL(}EXA1L1w&dFk4fA$bIBcfnhqIoCn+x|47Y_#&3Ucl) z!Kf31QG$)zTv<8ulJ|5UlW6w$=A+$Nq%-6*vq-NHFV*=+`JpCJ4Z$mn*|FQNPmpC_ zq#~V{bZ_*+WYH7?2PF#{v$_m2(3fP!?z2!h9#u47nkDh#%uCghX*MZ*7}IpL^R&F_ za-QR2!lp!qQBh8Xc*eCjY-2ojxaNh5=vhGY2u=SsxP9G|V|hYEyH4-DASW~6dF(8| z9c^4e!gvC4Un81nG`ieu^`$v&G8nsId@Gn4LCmx+;mr!zqT!cD`lY<@@h-lB_4GEd zvbmIreDf_R3Omo!uk84yC6<-$*y7tR$@Id0>xf_+SRRoVm$dV&7s|}?AmZ(QbPGCN zI46%0VLjx6OM#IZ2+HggGT09329AVdVb7#_I5or0am^=$39AF_s7TP!qa{YjD-&W2 zX)lagv`!(J9i=U+Cd|8RqF@ zCzqZf8@y+9`*t-><;umTZ1D01iLa%c=yuR+k9ewUK(-~jx>&)loE^pINN=$YU>;rA zywvu#;q)abT@c-nId4U#(%b`qymkxUj&U-l%rzI}q`d1jf4%vtI+Ip|Q*l<)PMQ&% zxWGOoR0F+gxMl1kCQEm~SuWR1A7y0{;rP5mQ1CXRwPaUW!H#|>r=y;}+B7{G{iV<@ zTZg)hAZXFC3h%`SCQ>8_BqQp+Dd+NDtKs{lll4ExWc0dpPXG&8RhK4OZC8xNITQTF ze;|J{JYZ{r=!(#h(}%lC!z#F_A4v{sQc(cmQPz_FplFNV?QBMJ;t3zuKqF4RrB58O zARbG5bTdGt+yuv`qJ8(LpG>FB;@@^>cYCEvW0lh`Y^%vsrL%{-8*P)02?;Q)&{>%h z=XrO1ntgCU&1R*|Z?aSQTjPKkW(3VRCADua`6k$CIA(uSw{iEDlP8FawsU?P9*ijP znmjaoKx?f^!NPNc`K?N0K<~n4rjT-abFQ7O1O-W_-h6zMB}HLcJ*UcbxgM&Y9FO`k z_n5$P{p(X8c)XodCt@x{kDWTf0la_^1v6a#9!Rr9;(chU(rc*dk2%s0=h z>UKU-&#ho>oRvO*V;M3qK@oLb*FqsdujAA6^#)>MGYn^kWkA&K)BQ(|ynBjum(;qf z(@5+es$J6UuA#CA5RZH`YBiy{nb)-p@;M;BhPh)yhve&)RAq15^xW6bQsC)mm+JPG z9Ity|5{0jZo-(Opjq^3nPNhS0X-4Z#`w<)3Axq7h%3CqpiOLX^pciS`6EK4JHyXX6Z(xjTpz6@kRcM|=s1QZ&L zWX7(6MY|Iw3*J)$Xo6aF$)4B=!0%Xi%{;z3n;P+C2k<-QNRzv0g`aFy`?tsh@Qblp z*t}8){Ur1wS`4o+TWeY#E4ge!@d|djCT&^xlbUCMGMUEvBpe~R8d^Va_wBxqw{lsa zR<3^#i*~va74ZV&wq|_SK5Q>>p$oh9^%(Jd|8NRdNz$FZL2zb7q}Xq}lUAomyB3BY zjun#Ov@2#mwndz%o%`&16P{ zm>en;O-o20G>tz1az2Bevy%p#uT8o2b?&}$itjU-vu<-M*LOT$)bw(4twi-pg>FC( zK6J~xLsieFGccG%GV(=!Vbeo}PCt)q>mRoml`Q9(*SW^fJoM{}y7~gKNe;MM+=pgA zXvZbc*GBuh^a3~S!>v{t{Dj6S6)(o*(NUaSke?(>;hE%01DfRB`!{yfnd8(`;Y$V` z9(yWcwhO%ZLMHL<1ab%ZiAtaQY7Gg0a(X);ckE9e;8v>21F8gGDkqKY$YDc~n;Lz1 zT7w3+P+`|tf27Z?*8H3X=hxRbzNn>-G^c}NAWuxcH3;1xs+c+j=B5Si28cVD#w+#Jzuj!I5sVdV z-Djssw>PKg#adp)aIVPDR35D9AFQG*n}P}>Zywt*jigqlfxWJa8QFu^>M+qL>0MPE zoQp=LLBng{^6U&&^28Q98xTwbA#KuN_^gU!v z*pc~Q5%HnUjfL$|tDUpCnB)ORl8I7nQq?q^aW+S#!}qA0&aqD8B76L@#D@`= z$w%~C7NZZ5HN0!43Sv}@lV0mqeiZ96omJFOrP{aJU9^+5>o3-Y6H8%F*(B;lgYMVS zBLmkdd3tU1tBZpkNf6dL4;7latI@6i*AZUSAuP&1tP)rCCr!wpYr&JAo0{A$s_CoD^ z2zO=|)uhPy7nWEk|m}rWy zD>Ss(@^84eEfncV27>&hJGne=JDKIczxa%g^E4{xfO6V&YFj4TkIQ)^yX|=`p}Jz^ znuHt%Lf=XiAEX<3WG07%_bGUhY!iZ8?OM*LDAYI8yeRsfpkmk<9I7)v>9}<%Ct$Xv zTS~tobBF2hdi?Fd-PF8vI7d}iNmTGS%&?{^%T?6nqO`T=2hV4Dz053{;a6z;pFj>r zTX)&0zTp*y#7piMG@8$&ALFk(Cigy}0k6&xF@{5Z+quxaC0HpT#AF17n0%0F1c!v= zkKEa{Gd==N$8b_eo#ajrT&!wE=oK?nfHSjl;lQl%`LD28zfQ4;(bvSj{lkxK?hFcJGr}f=JSrqy zRa`&bfV9V#a`LW|?Zwxa#J=qIzOXhZ95=UML#mXV7*!^&i zh{t_Fl0l8O@%07so!*N0Sus&cJKkn;$JXWVH^b}I=vZ|NRT63}i;05{ag#HH%2Jp! zFm@e`OkBZ<*1=>oUT*d;lk_LK26$nMdwE>Pv|16Z#fZUdUjRl^CMy4#3Zn_L8 ztq)h=V_`X5AT11)OGOh`ex{w0_*d9`+T>QW+5AOqqAp1@>%{vTt?z9WRFnN0GcnWk zB?TQ$ho$vJcP~BS`8$P-t=|bdq=B{yQn}Ev^D2N$v0vpLZABa z(4j-MJEjRM%)jc9yuJJNnfxGy++1e2AkWyh>Ff4+QPmf`wMY?a>|fb9hNp?GQ}^Pj*F@D?jG}j8yk%umhsuXD7sk%TELuEr zhRRvK5ITJhdxbnPq|LbWBfUj5)HkN9vjF+Vt9I00==p;Z`-}4NZ350KQX)4p5bu49 z-ajHU#mssP1xD=JQ`A6xsTV8?I|m`>ehf>UJArCreQZn2<|qVdhx+z?_jo~J1+A^N z6ADs7ynnwmjGBz7tp*=1gFjjdwI;1hZLwCQUsSbRV7s zr0*b1_-pX)dtx?oNc!AR5rVPiZ z@~Vi6x*f4xXm=rGd_JJWR6wNjerczuoaPf6la@ngj8zSPwL zHLJ!NpXhl@?aHh~jUUy@kbQH|jdjFsiO&TNvu?p%y+-Z<5o5LH$$*#}h#Jc0Lb)=1 z5bb2Ni}<2_CH=O_2!uxuT00+)%^&e&0bk~d&9Gl|m6W|i;Iro*j~JpnK#bzD@;vb% z?aMYNJX+t9r9iGrjqL`!qL&+^)tA7&EKJb9$DdpmTT8ZgTN3n!X@A z(S=Q7m+)`n9ycUNUC?1^$5ZmYl+*>d?a)M+5s)wHKi222u=wsqoR9pw*uO_L>5h+ zQ$OjOblHfEaG#ugM-7co_&@?I#&lWPc+)rh#vDfkBy;jSTz)u3qKhxUPjE*G?aSs51i9@?!CE{eeFOc`T zq)iliorU|m5lA5DfwT=w@aHp;y;3jfeuZX?z6CEA$}LDup96(F76kDO2h_2^ZxPv3 zCpy&;wWEJnmv1abP-<)1my8!b_tZZ&A9lZ(#`n?@9EeGx@1T?7QXRCg+Jp}mf8yRWQ zYthw-%AnY;>NfJ=NLLHo0T#LtMhm*fn(z;AFZq4@yOR1(^FH7r*pyp1^ySG*UpW9Z zC3W2HXp7QK5w%{XcDiH}Vgw#PXkaENGsNtCih9JBB z&`9IB2%*|@YBH((zl!yN5?_=shPBkY)qrSP3Dl+B+4cVeApEqTF*Ud##~4a+q_Mw- z(C-@l?5p2)_-zNU82-QK`}cVGJsy6Kha>xp-z(1Vb@P|4#P50X_dNN3a-PgNiy#1f zptQ8K2-=F-TL>=6IVI)Fm!3!AgMaxdPDIBxZ#`E4zYUj%qKG^AU6~K&b*cvN6D)^) zgMbQy$dua3s?f(Dh3);vxvIY64V0@Z2teiOMPy1$aU*CeSN&6W+DZkk+8iA0RE^*X z1PTJ=2Lae0{SNqT`d=Q$zwKZ~$v9U7rSU+CB7I=Lk!vYwy^$~@VW~lxikTic*hUki z#t2*E`?h?|4?2#zTR(R~Pq$B+<~6;n96iO~C$VIuroWk}=&vdp+&xVQ%xiN<#yxnG z#vfw%um7KPl0w)E;GAp^0Ldq*?mpGyC=90v9^@4r0<;OQv4t0U`$FKPVCk(h`Go=Y zHI{$-HpZ;g8d3@U8-N<+Qif-|(&5}71nqDe-hg8)GYgeIAO|IsAe;XnV*Pjd5*0t5 z*{6e&9Z>F@JhnY3Zwu6Dho6Pnn8Z{n_LrRgt<_O7w(2u>B}MN0-;*m%x_3I=9+%MV z&_J;yKdSzG?*C@+{}jD*2!}=YCeP{ocRTr?&;E4g;t0e?D2yfgzc=`+`SNGK(Ubhm)?07DEN(%mq0NryBd z{onJJzVGjx--q+zd^v~fnrj|rp1q$PYp-?R_gWJmFDr5TCh1KyG_>22FGUs6(9mIM zXjd^XuY*s>(bMhF&~6EsiipTdiil9k+gcl$S{S0Cy$py_xdDCML6oc>9x7y#Dvec( z^^oITB%T`Dniw2E94)o^9wxn`fkFe~v)+~aRrG0NSBf5e^c5>Qx`uz(P?K>iiu1|m zjk>+s4o)`D3-0|Cwv}Ec^mh|af?q~@=c0!)bQ8DvVf43AXZ1_UUBO}fe9QAO4yRo6 zr{G{T_^m@vmTjx6q7&M7p=RfHXZ1w49|(qEqY>P;A!P|kz}}@o+o3ObnTU&~(3v3D zMJ)LgCM10e-N!lXj&Zw9)Sax3U4b|mH%9DlL}-ypTx2%a(Q;KN;`_$T@B zy~ji2;AyU_tbFZ^mnCe0ZYhcYFXH2z9Y~1dSj1mL1gK;L9K#J z_3RDP*xkC&IMy#sefNoWPMrZxJjZNi?UG-x*~2(CG(es;oGQc~K_#t+ z_cZjL)GeBcZ`UhWwt^(SA}JuoiSOUy%n8Vo(=YH;TzT_VGnIm7>jr=Ii|7o0_4vCdauc5F4Y(>Ix)P2L z(+xC>1;>|H;&L5xt(xcuH7;W*Twtb6$40k9&NfpHMSi%zaWBUS)P2aC)ryA3u5I2! zqOFce!tl+Tf~9wNwpBKLHu3SohRmnU+V`ua?khxRSKjq?G&D4%&sM@C z&^C|Wm0yN!C*DS{|wKPu$L}F@ZPtTKM*i>B-l7vFP_J9vIKz)VvqOysAri zv)S8#6-zt?>!|YHEA*1DZt_<6 zcf_A5YS2^gF{r4NeEe1)(fCvAg>XH()l7^f{4f;ym_$MXC9cWsmgOM=$$kt}kgSWR))R{j-5L><^OsjXu8~5H7u^`LRGO zZ(!Pl-<7o1XGVy3fOr+{I;-KU8*D9Ptk+u`)8Ka)$qDuC;EJ%Td9CYk8t0@!-6SZ* z4}98h?1{m=U&+Zn(#~nP(F@$r_9|?1`gZ7y?Qwg%@pRW3gT5hDM{egK9BToS#K+W! zK=_(2?>y@~V@-@J=C;3R8qPrc>Xq#l13e-e8XGJdbQ_l4J4N1OGGvrtFHau74aa@* zv6*TqLh9wl%W6qlNfk-{FSt^jQZ!P%QVmj1q=YjbX57rsl%kC+pyyXUc?F?W53g-k zFnHi{-G#*Eer14L+?2@0tAWqZ*KTyXk&K;gooZ6#ON>hbOF_lO#c{=1wx5cHT$1fp z4;mmWNtOWmp(4jWU|ao6eO)KG%3z2y)p36Ox3&y zQC!+t`ojcaerRrPR&7RWCO29zEKo3=4EhSbbsvr5zo(>70cE=)T0u|nQ){Y z3;l$MHB>OS84b7SfOnT_=#QHn&vZ`EcbKIrdylZ0^P87Bq&nmuy5G+YO6gHbFiY4d z6IJ)KEcTE%2>wi4E>$IFNMhC>!q zz8|&EPNl5WPhUP6ijHfU>Tx5PyF-b?MYfZ?)DqvrbJzcxA4c@`vKyw?sWI|#UGG^C zCE{z`pQ89i^o(s$iH?e$&t7bmsSH1%t3dbYgNsiOjyaYJRtUB>#RXj>voP5JI}!Q) z=Z;Hbr(w}yjqQGPcrC$dB!`p}Jg#O31NZb^*}pREju;y#D$!G4FtLePaBQg!#P!!E z@?>eCc@f^9m>tbURYbGvuBfBvqPS9_uwWEY9cdBD_|obXRmS5?3ptW|6zm>{Ka;Vn z2&HJI5>5DRPMWbJZqY2g;t^1_d}h4Ypd@UjxLz>Kl1L}ZVN4m61c^fS8QP4#(cU?8 z$#brCqwwIqG_8_zbvY@ymGUWyq&ChY?rxm>)o7jCIc10mG81+*t}xD9e8Vo>_Qx{) zQs(jle6yP+zD%z_WxYxnq$(zk^5@BNmC@u!Xp=F|B!5_x={D~k^`h9bB)H$f5?*k1 z)~nMCcZeX1Jh9-ZCReSc-}I*zwq&B~#23#AQ#z8y`MT_pXR4YdaV6#5FP6Xbm?oPG zlsjAX)r7k6uDo70?z^mQ&&jS?{qFVUnoNs$%l^k|PpspNGEs!6hOK$(QKL-Yj+mF& z_drdeLiMx!NEO$j?i^U!W*5O$cwl|-g~RtkZ|R|!G8Q}SG8=B2$~VV7cRd&!qs=3U zqn!%Ls*64Dx+|~cmgmtH*2&MS6`9^P3^I^eneayhZ7?F4w2utkmrw3TU5R>|cPo!b zrFQn~tbv0BVCF3gqj2#3J1Qbh1j*C&V`A#n555% zbOKMEi+2{kJ{icV8zRZ7PN3zSxtSN!fteYlx5n#X26DR ze)+pPp_RkT^0bE|?MiQ6^))BK*#xJQH}wbaPrN^V@Wai!grDnMXT4q7E%;hcCSen@ z&DZ<(EFxe9lzQh#rjoVwbdv0g@$;4^IDW_5w!#_*bGkUs2JdZC& zkb(vED^Wj3<~FFU4`w%=H%`i2C|&2~tEZe&W@9}1FX8)-7W6l=zR1Z5%AE?I6#vNk zKALGgKf8syjDz!$OD}lyHJbNb8MzyChd5~29ccY?$Df{Qd0a%?74)L{-#h3t&paM-$uWxjAKD0(L=r)7Qtun(FxZt zWWrLoJ^fEt9=uf6Z0I|$N6RR7V|15!(gz5|M?SH6(a=7aqG9}Xk1Tjc{e^-Ts?DG8=;3~7H^6@c;N|@J>fd*x!#<<` zeSHtWnC8#}A z<`s{=wI#Ezfwi6?v$N$JR6l5Z&OG4K($HR)(%I6&%8tjG|KT5Z@PKR7$1D#i|G358 zoc|$IMxIi{+SZWrIWsFW>qCK?l$4ZwwgyH#ilX9wH3$FkKQytof5XGV;^gGS?8L@w zZEMW(l$)EIh4mTBvu8};4kkMnD|=mMCM&x~e|GYBKca?q`nINT>`kq$C{g|D>RCJ3 z^FMrudeA?Ae~#18+4R?wtnB`p7MLIl>Km4)%&aW`^bMNwp+4o2H+43&P!%<`1ZxJK zA;9tUIo}`m|Br8eJ@L1e&|fXN*;#*Y`P(=DZK-T$Xe(lE2_9)L@N2vNYW(|`e>LP| zLCyTPmH2a+|M(QFv%pP0mVfq4;HH#C+IO&xWTv8WO5hzB8S3u}>YeV-cht2J0eobe z77a}ZO;YrQlJk|#q#KEM#3nvq%il5vYRXYGOt*%zv|N)qNjPDo+@@6i0P6NUj7n(Ub$oS>b&Zzs`Z@Y%!~6nvygsbm#dPtufzxdb3c<+RWFz{f2 zy5Rn&0bY9|niq%N7X!@y(G{K$+~`FZ-(Nld^AJkAtC+~{uyCrspZWJU1~1SW;;)JA z{Tj`m-@Na}qJ*+ZNcsJJ z>S2neI1j_~BvXurlRnRjAy`o>saVi{6Ru!voN;pHYHZ*pVpL1q3qrvJ(R5JW2UAWU zF^X+SGQ$4lfBJR)81x)!(3r>{9BYcw*Yc7-^$DDIyY2UI8V!gM%XrFjq5swb?~fMU zf3$#Ln4b(ako>a-Xz#@zp~gwXA^|ren=LgRx|y*tR+PRrob#HhKvikhb%SZY;RZ!> zbao4oI_%4af!(qD$y(Mj7-^s;hYD2p8e11*$(^mJvW4m`dYPPYjiqmEa+&E>*|Il1 z1Z?jfZn$sP9>oVepP-W^e&&x`_RUi-bvN9){=6*CvTVGYK$=mv%3%ZfG4T^V=Te4b z>2iX#K*_&V$rZBNZNSjcDm7q^x!NB?D40NYy2Ni`AFn=0Zr{kp<35dzvq>@I~;%_un3&Feu`-2Tmv+m`q zq)`Kot*?hB4c~qo#3f~?lRcgPKtV))ROxd%Xkt^8d3Z?YRPEcNc3GmR38Jm*o|mVyfd%Co@T%9o-$iiw zl_JHzPifqHswX;JZatf}p~QR|M!7y+?Zj@s+P_>(muP*v?dNM{H zVobb^!qbg$j4Z3vvQVq`VZi57oqUUdukX>NZS7oG>VAG3kY=fkj8=|B=qU3afdwt@ zEE|ESi#amQ4Hqjn!XRFUu#(=72R1FcvXfHx*3sqFFt2Ucw2>2kFX$sOl+Yt$7!9|Qjm?_zC;Z3Leoi73f|tEOFBgZaIZYPx)UTsy+YgAznBBdJ zEbVn4+&o)c?WMS5H(5~}*Nyhr!*84Jh!*&nND?P2t3yr&~ z3^}<+5$2=e4{pj>HU8!pDF1K_{rUK?f~M+D`aDr!J+hBSbyN)vxNqebIDXQIP|LuA zs9K>Gt37xia0^UXv`aCDS@R*UIGnBzhLOi}tw)Q5jb9qNP8AZ z=yJB(7P$91**qTsu|Jx2k|3m&{iNl2ez-cO9}*-S_K?Bs-s7qBVpGrWh|yrptfjJo zAEAN9u`yG5p5RQdR=>KIH>J{4e7HR;==v>y_93qyN3p&n;Ut8tQT_>AWkuEV+NEgq z_{gr{TIR&^k}Jek+6B@(U~49ui_O}6B~*YD-OROWqmi(67~TGA{4h1!)|cuIY>U(# zO+J}^TR()KsQN**omKVsvPr*ju~ePW{WlHQh(#CLLPCZ*m7zql@)heI7{577JrGv( z9&iL8AaE%%>Y$=Xu@Q&Q(@5FtN=h-08|cgBD?R*PB-zJ=O!^^nU>N4k+wz;^4$!8@ zhHc86yvMQ%n(WcfI}FwI6*V8u%1qH z@I&U84X$5^nKS?uTh*#Lyc~Xv8;7`=M`~N}ZkQ>wsU#U2QudETT_Fm-MUVB-C9o7KsTQiwUS|TM=bUvb$==MB$O$2x zw<|ojJnr+nc{n0V2OU*4PDxq^N)jxPBShSYmIe`1qqgA zsskw=UBB<(CB6{+B1Ej`huR}3M}&0_<%^BL&;;wcN;;P@o6%R58D1J<=Vyz#8&2ID zFW_d159)CwiC(3=_wl9awDBC(u#=g$r!zX5vg>QC{6b!N|3gXF1hxOia3e$A+ie9~ z0D_1eVLeK)sxrlonQ!_)9B$C@D$a-rx>WbQJV&$l$w2rf{THkzVTO&vQSEKnwn=$- zbbOE+^Ww!hf0w_kS@+4 znch*(o(bJLP4z98$8-D*hQ$K_{``gFIrF#)Wk=ehcT``O#*5|*%aVMw=zhblIt$6|9-@>T-2 z!~++D0gl{siee&Ir#kYO#5l2Nwa%lG7^?H$qjFxh39q@He5Oy>+6hvsnn3YYzs6kJXFQv0!Izr7WZU%Z=jqEtTvwO~$)~;WPnj za9B?ZNARF8EVl9tJTrc=y)cA^*^Jr8s#h&<40i+%)_FOiq8;S=V^dX0tA5i;duw_A zv8l)L*oS`@X3#6NhKcL>+PGdEl9;C4_{8hLggPG3-Yk_-lS7VV4MP?vjlsavYLead zIuv7?SEALeD$~qmfzM1cFUEt>C$v5~J&ZgoaN3?Ry!;N_t*@pH!%MDK9CW`Zwr)AcrW{FxTuf?uPUAjQH;Viai6El+ek~Enq={%h_U2 z9=<%i`0@hmQ`3@|n=W$DC`P$=CDlPL0eimUn{JceUCnS`9c4d;{8B}Q@zMe-bCKw7 zB|q4;JlL=YZ+m`ODo5 zDS0q$e7l%=yItE@2>?A5bg(zjOq9lu$(QD!{w1 zHyCYJoc1^z`*Bf>=1Ydv8G8u-kJywiCW0EH2#p5U%kv}RrJ-h~ zaU!dIz>37@pK;hSMXh3bGv>@!3^v;xg)BAbYW9e_Az7L$1>8!{0u zlv7X;Q_Qs3AoCM-q6mKcbG8Clpzg%uE9D(D33oCgs(;X?xG5^(iqXg=lVOVs_T7a* zQzfey0hJ-S^q9CDt-lxnoqhg+HM)L#)Ly10)ccoJzyHIki}oKvN8)Dehh)<4TUAh% zm35Fhc0X+sEfzgVvI=NV@i@uB$^m1qqf06n;8MVP?l8y!u?Bq7DqMVR5W~E}qC*&) z>5l2EWkC>BxMpd5O_sk7$q7y`UYHlwI)33$pnVmKS9^$59{}u(q=#n%Cbsr`#(9B; zTl?*IgqW#PhyzlUc5t)9HVi_7un*SYb@wPPkW%uaV!+PZLp2QC$~iT^l|P2UMFS{? zj*B0EW(+_u-IpW)l@p+9B?g%*G#1OBwU zf-3snZLo_0>{?uBNo$pm`@U4t4Y@U=YUcTA#cNGpdpKt`4s1jWZ^hsZero*P$p6MG zzT~C>{;WRVUZ!O!VbcDr1Q0PW{DZ!t`-_pOIv-=j&^MUVUI|@YNpgfRz4<{Ob*%SD z4At1(RxPD+Fr}s>@#%K`Q699Eg?J7K3{?SlC$LyOc|pt;ljA>B8aVWLY~GOT+Nnks z9C*;R_+z9jM$<7)J2CB?G0xPYq}sZ|GG9ko*AVgQE#&MpyC4H;-lGQ3wJ2WntGBCv zbx{Xv0hBI-`O$B3$=@-jDcC|C3gu{3d|?$?Jaa;Bf>pf~qb)Fq!MB8w5h0zK^5Msl zRc^!SvI*QGz9r@4|a!6;-#>qiqU)u+Gdb9hQ*p#pmR%SF-so z?kz<_p_-{n>oA)RwYNw_#-vn*`FH~0$J5kuQxFmOy7h5G9bCpR%an2{T$)k7Z zIQ}#;@sUtwoZo~+QH2|o+!MVKMGM`f9hz|R!HQRa74~V4ujdFkc4rnzcis7DaV@W? z1VQx8@#;N&-?XA6H+@aLpd_)1Hw9DV=HwiXuhNRHjxd7g+j?`}mc=r>_kt@(^JrvD z^6e+`w~z5x+Lto%RraP4jP8|`+P?lmYY-d2rl|^Vq4QRoHu4cNspCL&@V6X`e@s`M}E_qHf-?UX@5I+me+ zneqQPL-0fs34nXPXFruzURnqde9lsTW=jTDILdzF2f$$lv8TVrI1ZS#^LYP`3zF`oZ7~JJMySa%_e#;A6Zo~xn6hSf#{rH3Dh|>RdSid zQt;f^-uS_+wUidh|L?KCr@=_1JS@McV}7$sL##3nVBbyoh+=zTCG|@=Q(ni-dz@8e zzXhg05Td2z+|jIO2uEPqO@i!;6kehf*xn@`O!+!&GopDND1=6t306EqIj^%wBj2h2 z$J$J|udGFz4rR7fWiW{{cp;W7k!cLVMY)A{PDEg3n2!Sfwd(JmKb#ZI;}cGD%C^IH zoHV<8yFrbq__Alz>2jowR6U6A{9s7sxYq#pAL#nW9#Y=Fib8@G&JiE7txd(?vWfQy zw}5MU{>3~t&{{J}Sy@kcGR`PI$1h_J@*e}Ir`-W)qNSBH0Ag&aTrhGc9|~o#E*~ay z+sO6Fb*mqNvvsqItaBv((!2iXAyndO{n?(hu5&setam~!fA1kQPYk)dfst2IAe+R) zQC47fMt|ksCrpSRKlF+$1lgO!+ZMrkoQ6<46oNwnV0Jg6p(AzXQ$Q9hk#c(cUxOH- zW;N`L^um`#DhEJ(%_^@wg60h;_g3ZRi{@Q_a`|&1#AvH>rC^Y}gXllcbHH-jZ6P*V zjx!FX4Lf9lVIp}F?4L-&Jpe*B0t6>#i%@jyKMRs8ItR2FHXu`AdyVWSL9% z(7PJx8zvCuDk>ah|7o%Q@oQF!hZ-Jio`7>C81+3G$P- zlQF;qzBHsO+=+YgIx|)>f~G~i#W=|U!UoV-+yj2sSR8VmJNTr$jM8?u{^*9Vyqz`O$0wv9{X)-^prv5!y(|5^y7&npO5s z_j`E@fh^?E4XC{V5l{-QW`sSGh_Amq-(K#ZO<~_Ctgj>CvCm!`FEO5X%XdIdV0ByOp zsv3&FmaJta#7D^TbTC%Tc|Fak!9wmXv?WYhRri=oRy$dDN_&WV;FF_;i2FdU^sd6FZjS1SAr+yuvzHKIX#+If185A8VFja#vv5zn5l3H_miv^n=9pb}G8D1{P#+m*U3c6KxD!c8y|N2CjtU#G`IO~so%_Nm z4#~4-zz4}Z9Dmn{geaX*OArB05#?>7DBhiM;}lE4$YPzJ&w1nUv?n@$3&_-O3zm6V z14!F4NEZdRAquM2bBnNxSfDX#W=~M*uGpiQw|!?I2`cU)$2o0ZBtHdMPIpdLGO2dn zC+c?`$O9ijS%2fl$XL~k7%>%nV_>Z|c0f0YyMliGPUk?q=cThPEfx~cQ-KF|dAkUP z!kXf7kXh&f3X|mwNaS%*EESRjO%w2v4$1tLx*YXO&gfy)xmp(ObJPP5{TNH-DyPtK%g6tWq<0FU$L^)t#AYCFs@N6Ep zh;@Vms1QyTyHRsfI^MVS0tnN`jLEdeHuNI)hh7ItqDEA8digMhDUDbOh12gKB5kXl&zM}dIZ7@nTsK1?aUnn zs$q)OBPRQMYMgT>5i-vMHbXdyy9Pwdr);GII}d=KAihyhZRJ**6vSBtlCWuj#y{y5 zahWVC+N}gigE(@eBD6fol&je5Sv;_-N7Z*^=PT}3ggI;n6&7vOiI>l~n$K$h9r&U7 zaZn!<5XA<(aRu&;@0RbfeKvm4P=zjm{6?Bp6u~S8j=saZ>+ez~WvJ8DLH5DtgIt3= z>nl>HN(}b$tu9Ji1+Sn`@_^PUv#DG=geJ(|^Z+NLyy!Pcc6YWh*FlY=U^vJVA;_$M zEo&U8ae6BA;F)8IK;pBBcHM%-emZ)4xPG^KyORlO-k=3?e?@`CQ-OVa`>#r7Jx*KJ z+^{$N@gM00nnMMQuy6wKehG?r*|b*?WpEJ14(wNX-(9dLD|pIoLZ!8$w#EcXM()X1 z_H??d2%}QPoai@j7^{oxbTmPVB?uAST7t=djtDe*P)0Q6%uUuK0P6_R=$|MgC`cC% zdWK5h9fK37*N__!rSu>_#kG~hxVdY1l~fzP9uHEz-G<6z524s}W53iTylfE=NInn1 zf$6QYpR}rP#~ytr%>`ZM9RH}^?zWM50D;J``zaGXoOWIaG6L~}6U2*}aiFIt79Nsc zXU|cv^HPqMSAB!<$YX&bB71>yVFA*B*Y_vP3tI@;Oo_CE-k$^6>D$w61QCojWmoWN z(#?0jvT2mQ?@q)^dU*1Zh`;2icDsh2yTo0le0W67{edbJh+m{SsB^K-NS}JrZO0O8 z4MPm#woy1#P{>Tj7^;;VbtZIAy{YQ5c;V)>6U&MyFDV zvyog3Gg~8B%20LVkcLM$R|472+)7sr?#Coc_7@U*FNN7HFlmw57jI6f!SH_U-RMK8 zw>T>NuG5|Dq=zJ^h}S)caejZi!q&xH8@V>%{(>zFp+v(Z9ihY#(AM_gEu-UP)S*2P zq@DwR>%pi0CRVEvQi5r2v3BJK8D2Fm$QXKn5{V*Z;B2-z?E3i2UFP23_?BU6FIY)8 z3v{pDV>NYX(+R;Zb77+T>@jkOSE9`)Afh|Lx?W!(@$52>8dln9Yn1M>QT~`=@Hs2+ z+5JBOAb6Q1>RvSA+2mp5yA*Bd&_Rom;*^6fHwV%`aIa^le+vjNo%)0gd~l%VTSRNF z#)t0~UyX)jI4z@mm8HjTfqb-<2MX*3XnI9uc{OjEW+f379?>`CK1HZqh^Hn#g{d+8 z_==9pp4%nGkZ@m5)ckiHH>oJ%+dVO4stU5 z%OH=;3UGJrIEdH!46xS3vew4j-8@pxw*Z>9Me77vlC$c(8~D_sX6*2DXMXo2-xfD| zatB|648Qtj49E$QL?FZ5*K@M;QgbDf7wcdMT|*L@f5^vsHG+G*RyCDtNZxPiR!GlB zQ{LuS^|IR_6-$Z(0%4B#s&CA#pf(u9O!YAD1ZY6i0#A_-WOyxmR(|T1+0&O~2ON znQK@EafQ{f7Fo;)Bbc&PiB>OaUFqfBt`U#c3(&_>x%!VU&gQscY&sH)Y&iX{xiM*1 z&w~OQq$r@#;WDhM!eQial;qS7DH*7V3?N+=muJt^WQlPfl+Ow= z0ui^Wu~N)LVnPh*bzMLaP3u;@{^n*YK~cOUG#Vn)_&9GJ*zW_DqDSRAOqH*LeU)7R zf9cvi>=V4KiF`xq9G)+={^%x8RR$DMMK06?Yw|#P&Wn{Kf+k8Z>geg**6ZoOHN zK4P#lO7*b6LPJ%pE6KkzDun6p!Qr)kd{MH!$`hz_dxG!}%ZVh`?7i>*?brFebM;^$ zyul=(Nt(Ix@$Mq1`$}=H;inLVU(W}v-YxQV1~RYQ7Tx_vwGp}r2Hm0a7(%TU;RS;M zipf!(^mwi}bHo(BsPm~Kha0HL$r`+aC#iCAQP1wVuW<%f5W0p)B*mf**3yBIoLyE`Q@ac zAHV}(Pf+%~f0p(ZXad;!xQ^q7#3eh8*ei)flf>H~nm-4K<0E#Or)VC>sruw-sRMPz z#mPnws3x%jf`QY13$b=Avu0H(^P>VYVhk=9mCtg-o(Yz}t@;%Ql=yd`TQ#nOe{Ueg^Xu(Z z+piRVm@oSV@iZv6F{}gr2{P&mDuy`BVMXF@=fJNSjj-%PviXKeM1>;uZcfnt)!AcL=uo%gmZzM7e3WhB-0U_b&@2&jT8Ut7t5 zI%wiBk$JXLeQ0)Y2DLCft{eF(5O=M&-#JPeZ!$EShVX82+3j`E#yF4%d^)~3-NvTC zIyu}h2Ks(kRjci&rc<0{`Sf#-QGp+0`bPN(WbVcZyWaevRnK~Q#EB5Xk+8RlVss|? zN_kCC!d)mKOXc|w$f)8WIOX<#x{eZ$fqcMSmQjPsGp`*GzaRJEfh%3zgN6y)Pnbvp zqhf!_#Wnp-K$G-k`45K|eYp8gz*OS_)Q(R9zbU@>;0OxixS34d?_xKoo&Ko$_K~3* zm6$xRu^s#(nF$g!2o}*+r{Npes$zMPzQ|kSJw|*_2MwG`NOUUVA}S@!I_}a=uG3~#+R3H9&9SFkpX(oNl?et zict%)RYs^fQ>Q5#iDNth#jVmI+2z*z*6g6B?fK=!QN7)$!%0-74ylA2%Hykcb4fSi ziOim$rkOF4GZQQglPab071x-_M0zRfQ`?f$fK=jI;fxpg5RvMOyW^hDX+Tk-F1C^$ zK;=OJBeHRk<2m1a>50Gu8rfR|ZL%am7Kp za~5;~)UU0;q2)a9+wLg!t;VYU{Kv<=Tx1I*vTbPtq|?`#W;`#?O#s9QLl9+U=6oOo zG1+aNg#UZ#&_rqbk924%Ba*sev-lGkck!Df2Q22(Dj0zp?)%2`L$^_BXAtdgE|uwj z-^dsOo~oiwmd}CJEqrr8|7NDf5?W8g{9}hmg7FPA*0(=Hnqs|;wABEbR5MC{=3#JJY2Zj%!C1gOe_|P8Q7PO(V z-t@Zn<21bG3H5W4BD4IAu#3h8uPpt#{=*{&875$5akfv2=$}rSTxlRJON zgEF@H`P`NgIDF!%twLKc`2vnF;}0gR*O{*Z>VQv+)=+$DuIUW2k0;rig)t)pFk}yx zVl^l6a6x{4p1_Zsf)hP;*=$kxy3^!@F?7_oKEn(OJ)G&c&WKMXXi&ovIj7m&oXF3e z0QCszf)~3XyVkpa6*Dv7x)Aysn~P98jM(LGsPFZjPR?Esjmm8&;tKryjA&V|BE{SB zHLTdiejU)eQ^O(}(OZ3hC@w}8Q>b`nc5D&>y!4ghf{KpC2_{XqIX_u}LrSwYeZ|4bp5lBQOKfVMW)f929oowrIIviQ zLsv*8;wK8U4%^{MyOjzukV_2Nd=+U!+b+AV(B5?U;q|DWmol?6C~;IS{dyI<`FO0T zX&2#$b)DLpAaZ6zggM-C+%Gnc{FDS+(4qO_AWN|264T#Uu>wRJY!|xvQk@w3k3&qL z2F~}fBY#sP0)_NaZ)?P%>QjU#Y#q;n-MN|l3AD^EU6$jr-fROUvia8cv99oPC5ekU zZ`@!ajW-RA``kdP6YFBGw)leM`tyBr?@RUUyp6f+-hzVDt!nFm^Ubl5pNZrl>h>D= zPO@hQXD|K7i@d`0v!AUUyMSWUo1?)Lf&!#`&TkCbX2e{Xm9Fz|fEq*f1Avc+2dTej z6MAkIwTU|#^|YTCP**K&c>>hUdCdPYFE6I^+qyYz&J(hFP`MXwb$6V^CH+164ROz( zO%&fY>?!!!fDEavN|Aq%su!gUd2$v_76`VMb$PKq|HqN?6%>TGF#Yk ztbTBN@5F;~EOUbG82cT-Fg_KgfNF;X*4-h&pL5=*;#4gY>*?3q`n=ECQ7Uc7mZ{z!)W0N zR+fM$)}Q+OID9)_+Y8(iTx(|TT`?o#S(u60jo#BS89 zLUU#yHN(}yMB@pE3M?)h!^DUJ4!UWsxi?RMidMyKhAT$2PZ$!d^L0k}AIqNG9-lfs z%%?$?0nMc*RI5)@vHu3uQNxSALX6rJ>%qDSj$2Wz!x9L^GiAS>CMjo6;_cabPmK%y zA7f4AX2m0?964HBu>x}$=hvfDsU4y|dm_9x|@U(X8;!yXq>N&0ZN zGEGll`R}Rq4BINJd1`KdQ0#6jy>R%o%^^NHupBGCJXk>xDUf5kP+1V}5agcN3eN#3 zM?SgnB8jv5x*@yU&z;qb7_E;$rY>%0p3qv=F%gbun(5pM@*SMk}idYdj!O(`}c6^Ta%xmWI>;5>6Z5K4S2qn3-4iTc@HF zoIZ$=uy~Jeul-RGhv;Fn&@i`Zu5ex|3p7gH^a$X*)v`&ez~Q2@l3XGff+vNYD4(E# zeg=iN&uCw4cCG~@47^AxDR~$I7?m>`JumiCgw>u3Y`{v=K(PgXxG%JC98iB%=jXzC zMLuyQIvZ*Za{zG9jFH-dgC~Znw3{Cv*x&F0TNt|C)z?rFfVg_xw93uP?~<&7WvfsMyw!5hnDetto^8RlrJ>#Hiu8Jo57>d$uk~w~X?OJ#&W) zw4T_B;vC1o!Lgp2e@OOL75j;fvK6K$t*deSO_)iQnC?WW-B(Pl5TJC~vw zKQD6-jiaJ*Q_u#pMf87o|q|dJg_a^xm)vS113J84^%4W7zOoY-OlMOzFCB zM+7bL1jBbuHxOBg*YjpSO?35E-B%w|_B>KYZc|Feswk>NDa_ypmop+ygg%F`x#G5) zf8}R6d6R+FKjK0s*+(jEcac!EA7q7UxD!6JZJy`GFFcsv+ z)cFFDhhd*Eyi;r_z^F1c_&z5yqxj?Z>pWrf(oY)bk7QNL8Y!it^FAa+*RPh&;*Yv- z)tzpo?SIVP2l+7J@~=`Dd7dRwZScM`@($3G)u823$IYz<4L&*Mrq-9AVijH11B9EDV_cX79uzc{G~J*WvNIfXwqw^8eF zd#ci__<+WAbacZT#CPRN`|Lm4N{p6R0C& zo8n-?k*Rnol$}m^ofbNRC70kPPA++b`^owW_Ws~ufj1J+z_eaBA^@i6G`I~KUSb#9 zSJ-D@Y=EEY#K!uz(pGJ71P|lZ>D=b^_28KvUBa&tS`mYHI`l_9wRHQk{F>e_a3sG;a1L;(sdxn8zrwDQ z-`&VL00}JxzSbj<43n+q+lxt;B0wx*LTKM8&9DvelHb+nPMn!0ik!5wIUv&OVz4Js=7^vgEGGcZ$}9Y6IYC(3!M*z z-4sVwqOBBFq2g8-v%9+W`B991|5RCWORh7gN`2{mDwHWk6ERs4eO_TU>U)-lN}|HW z{t$ukmjZiL@2D*hXqb^hR=hG;7IkSGu^>UZGT0Xt8IL#OJ(Fhfe=vPk#e}Rl!k%Tx zJj^J5$zI_pFO7u^_0`5*nIZBxKji%;8O;;(VSR%q)t2}`L<=LYFK2i^U6O=MuJSVD zn(BxqPbIXPRj=8<(MF~>isqi?$rnqA7e`0e*J-)vH!2s3jT#gmG^+YtxAy1}sGoD( z9PPSE>Sv3QCmPU6sj{iqS9I!6*@d`F2#b0(>a;dUoviba(K}E@KUycIe3EC>vGV6> zgEcO{x5rLf6|Jgu7d|ZNS>z<*EzV<1_4|tjFt>wGSwVr2nXU7zQYOu|(!naTJ%Q=+ z%AzgTkan-!`cmA_bxYdYpItGRZn8Yhbt=mJA!s!-2El*m^0 zr-}r~gR(UIgfPoT+C1_RbQG!|^pk=GYeo@rl_P`EaEpQm>n<@U=C++zo{=QF>qKGc zPK8u!pvQi{l~Hw?s?m^ZPUfU4?zZKE53_Z0w9F`aW7;5d@dXDlp`t$@e&N>D0d!-n zc3ELba!n_eN`CtM*6e5oEpt%%Z{uWmi&xvXbBsJF!EsGWHNhZf^e4137WJdaU5mkD zH`5Pl?j9}Rn}oH1<%3fh`d&ca(l-fP=3l?5I94e;Z-9=Mq6oT zPFB{c%*mN~ZAgg2P(p^e^ zzmxukgR~B1jgj>eE>%mGQ;X6}?jMw^-4LBXq6roM+ccRAy_6Tk4nqQ?t4*1+(YG(C z_Ue=Kh8BZAFc*6y*DW@vo_h3GC{^zJjds{RVn9`)s4LbWBXmX85U)^X5)zU+geW~T zG($=VlF~6CEeJ>p2uOeH#=ZBmpLajc`~CZVe1AOVkePL_b=7^Xb6wZD7(G(Q5c#$X zT}~j^tZd5DkHokyGSK-_3@EVsvMH4yU~rvnz42O9atc3gdXnU}%iht^y8mwX5C4jI z3Z$T^cD{PpGydi_Tur@PQTc?6pb4rRt1^~=UR`_r?9#~JM2ST=BM^zf)7uGH%WLmO zzmbR((Bjb*$%{+AhXCM_(>D%#P#t}AyTY$kL@;n0z$Lt8+T6+?^g|xN81$6B?rBb% z^a`ar#n98O7+jnv{g|J~OKRoh<~bPgY{*>rI98OtCN~OeIi85q{`f`jg7yNy|NTjB z`2OI@k&18OSKb297u~3#*=6NmI5AA`{e(-pNc>TH`)g2YM7zQmC&$Ds%&E(3@pDR( zalo8=I99J)s43wJxrgbQGwl5k7l57-ch-3DwgFfECj}qDi!vvU#;H72_mStFt!zAY z2Ob%Ge><>>l({fnolr!~K^t0%6g+Kt!ZcI;0sn!_;U59y@`S%KPn z){B>p%<`@wAv1ziBxVMcuP?o&`|P!tl|(D?(V71VnZ4*tV!;L~@x5fhJ44*Xn8Y^6 zRHjI)C#2gfx~bk6tP$u?a?#Lc@yGwbR}v!e_F}9qm9%yB1Oi};$NikI8~T*$^Q*!hH?*9949 znO}Eud@vg=yC%l+W~Il-2pov4xla=0`%WHK@0m>uNZ+OJ7*%YmOOlq5+vIHc)`dW^P_(WT4Dm zbc;nc(G<=T;DI*na9;xkkf3T*T@3l`*20_0UOAYltoPc)O=uWx*MBPVYAa>s(K3$8 zuqa{qGOgU;XKryMiqq^w^%wVRD@ihxY88NQRjXxPjO$opjC+gkT=814>2$m+o;dz% z!h0L^mv0N&4b|}_t$EB8ZeTij@|QJ~`Dv20GvL-@e0H5aKRXA z603=7MC}&w6NRWT_|7a^FB-e0T3WF5yw>jtI*2k*B>mOti30p9ouLsCMuKLIk-17J zI~bG*Gq1hn&X-uDV)1h<$bwsP^0!@Tp7_exg#LZ(%<|n_$oQD0I;x^wCbd0#i6&eS~OVCw!#=(K)ao$JY3L`5VA!Z%UUC~)Kt+u0q zu>6s^Y@r9sG4yvPQY8a$&a(i;ewp|J)-_;oTAoecE!of%I&K5lCa1O}m8wv!@|8WcLa#mMP~fZ*RKT^@e$ zs^T@_qVw7{Fa`*{O>^iW`Sb60o%9Tw%f+K9Ed(k|J!DjN;%Q-Uo202*fV#(V5#BWLK0Xr_sQuNEZRnwWfIT-efQ7ul%il4N z_3f2z&{+7kqF{KWFO!Kke3>{`bE-ex-L3Rd65Lp)G^iss$6N7U4!>PSv3<$74a`8s z`Y^L#=rww_wms)ic)zu8NNz^tBHT}IK{+zR-9ADQHmI>MUdw|6>2sv_&Tl7qM#skV zu;>)Y!X+Wwk)m6Q_ufRC6$4;dOuX>i{>XHTAR>DjlwsYuotpTBvefZ)Nt*uAWj98> zyK`{*^IbD5OBt?NL3Ypc1$Et4K@er+2$Qk4xmS%@X7CvAV~dnzLZX#(R$mKsU$trc zST^mgm#MeGW5T6&u2rq)5DXlv8xJa@Bm^q@_t$y5%GatD1CDq zHY8)<2}{oGu9mPY7+|Lp>bi(gT>vF(^hZ!%SjyNtWolAbLQ};RYtQRp*Si9_DcLJv zVUIm(%ZILDJB(+SLdY3!s-)c$PZO47aO3QBlWq@IO&v><@@|>aQh?c8psvaZDo0Ai zkrk9ma7PP&UQ(OPXjhL{KrvmZ$z*cM@ouxL(pu=N@KNToUdda_VcN)OH#4x(rsh%n zR6Si-lk4@BpukE#)d*M15!3gIBH>}97ByY)eMT-f?`h7g;aNl%_ZW~_0b1`91+jDU?%na?lSb_0)$MI$0;69_O ztI;II_X|`yv{EktM5dEcP_;3zMqh5HkZ!P%=PD5I9jR;_VaI>qgv(V++>N<4biwRM zF?XqAJood_^$)wKV}_vv^kBVmf;0M<^b1WQ#b-Ek1HG{k2$pF?n4)Fg`qH+lsEB2X zno6ta+pZB<-4R=GN$`wpm_b!schwV+g71gU;>q0Sepi%) z+k>W2qIrPd7S{Oo;q~SZkq zTeW4R==^>H^voyqkweve+Rk4SIlG2`F2{H%#1fUoR-<22wbscQY-<{(&S^UxUEcv4 z%WRk;%dk^l&8rxHf;nP!UmtaeqXe@A?P_=kcxNH>1lJNyPcr`=#Fr%sR(~Rci448H zZD#7BBi1tDCe2o4e*evOrj~k+73f;zTKd67t~X~c9yj9T$w2~}%P%x8fr*Pf%}UDB zAkg^O#K?NpkH6FTB8D)CFPI*=RGJ~pWuMyqf2S&yf$Zhe92>dEk9a#zb$qrpnF1pi|QwR ztqba)qS3Iae3j(!v+D8!SAAqLj~8yG73Yh58i=RL#XVb14;*FGvVVE>N>U(sHFv)J zwIUHU>_~6x`Jt8M{TBC6ua92waC_E>yKi25I8%MYfAITiwyK`iqn^e}r`gwT0Yc@P z*1zwK6|P!X*-09;R4dUhU$#mFj$md*%zprv`LJ2Uz-VUmS-8CE>iZG~2{S|fSzq3w z*KS!`f_c^_00a_p?}VqkqM>L}Ria+kuefW*856dvO*{6E??8Ho{aR_SL!M5dQH#6f z*E$%FXm9$75-P1K7sA~H>GqR=2aDiUxceoXmOp~YW--?UnJMUs)9mYjH~iajH?^D@4e z@!V?$>g8}lXI>6;(j=vq5jMi@P*Slbmnvkx;^|@joP<5E)EI0~q$+{Nu7&j5pW4h4 zoM>)pzJ)~Hc=c#&{Z!%)ZAC*jMyX+U>6d>hbh{;CyQL4r{2WGZZ*55Cu+OpEjZqgh z>AZFmbI`tzof{eW&RzP+5^HREBw4%u{&2#O&Iv;Ff#fyGP5%ppuj{Rd2Z( zai!-@nZjpK$C5fW@-EB_S@k8L?hX}G3>^VpYI21Ag(+Fy@g@H4&pd(_**8vDnU>Mm zqRp#CaWdER-jQspG^nOn{0ZOwz{oEXqr5CaoKKr`0r#jZgCqSZ7 zz?u(}`nqkUayDvB-EwS)N0;caS?ub?uNznIMRpBcH6$P$4SbQh(ms-<8SXdiC2G-C zOs#~IdsAt}OLD`d^LC4By4A`Tqm2upSQ(b`5?uqlET~EnQ8&|3*9H{~wB=nucbSqP0f$nRXir8vFDUTttiD+DPku<(T@Dh+l0h}C{VEl zr!MR{VAg%<{_vlMuztS((S3Rs@%2wz_)H-T$$Me4+(5pnRVQqYp?i}7C|sTggf<-> zvzpcJ%F}>cK*)VB;`w=772<5YXh51d9m*PY>%t93*?ZD;(n#67g@c^80 z{0q+fFU4aw1OTYZ-)Qo8!~H)Nab>;u?s&>#aTzoXuo# z(oeVff+D%K<_^)o*!~&%$rT zAJLtbM`CyUjV!tUeFcN}2djw8Eb?Erd<^}kyoK+bwK8mGsQk_tTyh3T{0S6U7#^i2 zXv-N^^q}SiF@^98S=s;kzHIsi@oNUL$#K_Z2B+rY0#T!bitH9*{0n+uJ{p{TuNZyD zKi%2ft}N4qFK6~~as1OjF3C0M!L%0>V7lG5gAPX_Xqf~%2B=8v5^&?Zj9fvuNVeOz ztfV_q)q>XZDQ1@wKjme(XXgHMqGjX+@M_DBVdG>HYR-2+3!FDg(I{rvn8E=bO$#O< z3{Al5Y1IXpt#Psq4KX*EYxsXPs*Q@AjnrE$ou_|NGFYH5e$W9bUzrXXCEU+v#s--}0e|v3bgaDD45VH3F z@0!B?|Iz0r`{dbZnazBHjsmLBtmW;-Y)djI3~4=FR8vM}tr!EMD=KcOqu)Z7E$=Mv zUxd1sRzHB+UGMh)r7z{ibrCN1-ZLu7A3Xc z#$j5z9&*%mxCKx&BVX;5-yJ48@2=a^uL&SaQbEy8XTZU_L-~l$0LOysw9I-P^89K| zz4&?xvt=_)qS}*zw9BA30S}6D|9fj-Is?_unm&&C3Y&V=VzvXKYus)jwbQubx^C|F z&pxxxx(heW{r2`aOTl*at;)f!ScERw!i6|#;lkSEYR}C)tImV<=sRBpN?%v~4BUG1 z>M{fAaGKK&#ehYWc-N%&5x?|2;j;U=3n&^L_=gz#4joO?ugj|*6L#6ho2BBb@o`-! zf4vOH$yEdC-%2hgFUj8>tkpHSE)BN-?}8H|0EKf2*NV19EvJ0Xzgsm{CQ&i z#mi2~6Tj9QUcc^JZgrvH(x9ggaasDeRtzLtwD;qC*y=7Tv z7Z{hA+P#e+@!2xXE_zJi{CggxjNL^fDtP<&!p%DmyQPM2`2G9I0U9$lkQp{y7MrgH|D59XSl^7Iu zih#7}zjwTjUOTHN?m+KX`pu@5p;M?w+KR*X&xdzrQR1ENSFaA#mXO|S7o9^5r4oj&e6Q@dS&lE}*PtrLgK9y{GP#RRVw z++_NR<$Th6KjCN@iAL-5#&pteh1(50=T3SKm!qmy`xF>VEK8KXab=PK3Y77Ve(&kC z2g&>uBcIxq4%z_|WWbmmW3=JWA3Uh@=)w+roD<41RJHnZX_#g$=P+c-8MFB8WQ#M3 zbX`*rRLb^h{7A`$)f2Jw@THc$gh}i%BiybNi%<3vn~u;xs`qN^$ZlEZ!!P^!uE7n(bw@# z8~)~1Zm2~Oa-317eNf$9_SvwC{a^2~wvZRgYB&8XHey*TQmbV}#hB=X0HK;rWMg;D znmxyp6zjE5tF&9KGRJ_oM>k6OSAq9o*X!$Ib$p*uq9MTKF1ww4e)ZO4>L^-#`Sm~- z0#iUWtR44AWeRY!K6-+3TwHPK7GX7Ju<@&yb)@myUb4Wil_t88OtPe{a7*j9o`8%w za>3{ps6Qm{{vnADnAtx73D>NQ)XypZlUfGH36UuxH7?KuewOsrKU)`auM-r=ggwhQ ziqqCCy6JHkHveuRO(@Ceki2y8eWEaDsx#ViracFwbc`1J+lDeYjirdfiqGouD+?wa)7LVFM3zb<^g_#)*D2Z^^^Xn{VmH5+e&Qy8vCAriMeU0_%BT>-GZ8+oY zBVwU+Pf@ccjVxKlixzYO9e-O9w`LS_nz;Z<3=8;PH%AJhn@+$!MIPFx{CjVD8R(kF z$yFlKbWL4>6hz4ciqdIZb>0Gw)H?O^6^V%T$=a@jBx8EUq&e63N94i~`Cn&)UAgGN z<pLW~MoF|BnznXLKA#ZTSgH4~*Z?dNfSqwW`hnh5 zg_+KYWowar-Ja!q@5}RaQOJcaajLh~q>sv~j!~E=fNLqddq|%U92-K-693-*pAPlU z@HY-|gsD}~4p8EqtAWbzoGTyc1c8jPD|c5R8B(fWO(G&iF4oso6omL+u2WfaX4(HEx+~3l#S&&@ll8nY zC&}2e*}!S;)_}|R;I5&j0gfO5Hu6XF?u%lElqo8Un&i3vNPnx|X; zu~0d%P##+Y!T=U92bLQx|M8Un;|pM-7yjn>)@K81yBREYMqQ;#XG}}Wr+H$*j&l`k z5^=TVeA{Stu%nT^PWG?2aJ<}f_yvyS54*m$JCn9XrNbP}WHm_%XTzAz1h_wZp_nwP zqN(xb2Gn;J3GGw=5?KUTl6=}f_OM)ck#An4z;?~BoFyEU0z=n0nDXdC?Z9V5|4VQv z_Fomu>+gmY{26of?fdjPi<;;hcIc@<_vC64nAlM{zX%5Ub;FJSdhN24+|_xOL=1YN zih^Zl!5&rAeoS*~1vI(;-4;nkMO_o?zPCAI7EI4m(2KF^d0rDbu>&`M^WdIz+X)n& zpA-7%JX1VFsJY4giW{xAm7<->5Jf@$ zhZYeE>-U6ZL0xT6r`TckKTT@&MmgSGyujRh6-0^1zV5*iQSjlc6$sFosgL5Y=<`iK)pq`Z0N<_nHYuQVJ1IVhzF$t9LLerIlL>|h!6H_y5G-8 z0!ciC)~@BE`Hcr6zIXwxT_P%x_twM+Ok4Ya>LaOQ9AB`f(qSSpFH&gEAN<0Qn%FSl zh~3tQd6}_rnN;&RkiIrpeSTw$9K~bEbBZ9FcDo_mORY4-=#Hy3p9UF8ZK`uY z-WSv5iEu;ko6E$RRRo2VBUqX4n5voHNjVxaB3|XOE8m$30z$wl?bG1yF&J5I$f=iAsyOUXw54X!t=vNdFr6#b^4IC0H!@*BLi3^N&R6fa&VG|;8n6f_t)m$rUee<#-B!^k+A;V)LYK2P>1=No7_ zrw;-%zfX@s;K2ux22h*?W?m-#)!IdSvl-o=NL8CKiK!}aya^b}Bndu9I&$kg(@~mY z(T{3y8!iSjABC9G4iIXHfpW-Q{2_O_oRJXvcuE3;PNKp{7`NBDN&%*$Ho>#TKIr&4 zK22)XUafACJ$Bi4pkq9U73GOHhVQ3yLrVeg?bD@5BvYsOZNp9VhTJL%4}JrWK477M z>%=7%E23`)iiDB2GMy6tM-I;qR(e5C277eq$@w(cI$YU}56ff*mLvsD7lK(HrV|DL zdfwT~*p&yL;cIT};f>sotpY1tF6!fQwg(?xW}h+++*%^TI|pY9N)IG9k~}S&N8KWy z0q!A9@@K7q|5^k+=|rtq^6(0-{csP4s%`~{f8{S2tC%k8RNB&ML? znvlD#!I)75pDCMbG8sRvO*Dydf%!~;4+XKDRn=NU!SCbFMq-#kFh3!E1kge8X!0lw zgjbPisFifkrfk%-zoLM&4|s=mP9}sOQBaL6k3Y5ik-JX%N`DEI-14WJT_Cswn2tt^ zRMeU=vH;kVh0ghxgqO%VoW}!?e$N5f1y-i~b_;)F`Y@fDqzun6@)`ZxD~TMCNTvHh z5~x_^@;oL6gdfL@;L?|qlc<+A9sj)P1>Ey8&LB6`!WkKJlbD@?DL$r`gsq>CNh8an zm?=Dnn1%ht3)e*z*#`9o)Q>1*c(`hPNjZdV?t3X9couh)Pd@G1&Ue>%FHib)3p%*3 zuMJvtuXPJ8bUsOXO*7E2321m~`Y$m^lm#e%tFmtBmox2?$ZT=sy%A zprngAi*Yj7CP506$7nysJeFpG^OI}{VqvSU;V zZoqxyn9c`0HRF)i8P>-YR&BQR=VvJCvJ!7UqO|rV6lFZHy?h>F(iQN~pNl|_IXy_^ zg(3;^%ww{{i;BG3Ji)f6c}F#F*JyWVG-fN#1i^{q()lSKH!)mI?^nvfY! z52tkidm|MJ^+x&zY>>#AOgX=nH!$>hC%KUPl5mqWUY0LcVwX^r7w|ef)+HZtx+Y|K!2Gn@>mv~1DzvN}8)UxCHZ<}x3^a1EAIBTK zDTesN#Hc_E+n4f@=k%&T52$ki!s7v1*?w>(T_PGF!qr#MkC86ifG-&uu5}%^4Op#ZtBEt`F7st=4)A(x za40HAhreLy^&#dmv8xjupb`@x;D4wKDjM3u)o+R*;v))a@!`(~0o4FYIu$FmZW0qsWmb9F`riM5&-bkXP_?U>#SmR>(dTJ@sC^g86=L3*|cMKHB?VGCd_ z>V))^0f?H_HIE(M`ESG7HRxr$Xk_kLIENf4-sUpNzgdoQK7b5(Y+gxsOl=R>0*yaO ztTn)J)f{6}Yqy>_Fv?3184Xt4;=sypvc&{_83rVzV1vs%l~88Di}e!yOJvrS8ox5X zD&#!1Re1>6))_DvmEOnjFWL9OTFUc)P9nPz_i4_8iRE4e(>kXq%Z{ZHi{k*QL3d@v|e>~LLU#qlH8E(QrgT)rC-)1;6G!i2$G|HMLDxBv_^OC z<;y^-y#=Y`Dz^zgmXH1cn8K9+*!}d;wT9^{RQPl51}(3@`8~$S=59lOVFAtoCwxr* zh?46CSR9|Mk}jEb_$2JqM&u}n!^~0Mg&`DsUvx>3p;5;$LO3H>PySR3ylDy2Vni_& zeu+1EKK)dStW4^wLjgw;JpXb7U~_(U1tO}@uk@ZI6lE70pgsbl( za4WDd@~iVI0ROoyCrf9D7Cc6IBTv=!d!IDLTF7>6xw|nYD zCyvG)YCg|5q$bAcduah_{a+_WP58iL8sT6ytAjRr^iA+(8t^+=J4#?N`kzE9@bZv7 z`z92Xmw!aALWWmMNlb#|*IB-#Lt3!Y@nvUpR72l@J&5^tn8ng#FeI6&A zM=qV7*j9SyTZ%d34VgO&6~~YxG(vhY%?!d7l~@y>+6sy?z{}2R)@)Ayc|y^C+s`VN z-cj%+yVaec(V_E@J*OlT;pM*z)17`D5eeBdeyu@6haDRjbdx4v=K+|DH?9I-E&&Zb zgAT$;kY?xoRk|9G?Ks@tYIeAW@^Pv0O(`(hfKd=uaH4mS0{M}^8Ep@EOziCBhDRZj z^JW2->&5@vb#Zn59f^nk4|mcGn@CZC1G`Y86a*w8j zp_lz>takXuHVZvZN7#V7(rE72*USaT_@sbNVC4ELRY#}Z;=+zy%{hddfnQns5*}7% zY@_N)JAeTi50{06zW@Oc#&`+LmFM)eCIjTm9ozCZTJ%v^U-5Y6Xp>nY441U=1X1S^ zSU+z$8lPtbt)HbZ2xr8TyZ3kf421M1lGpR}9_zXxuiuxC4wpqjjwmvwjnrm@?8FH; zA|uhD!CyzDB{)7D@~S?keB`rh4&)mS;#b*u0_LX)aFIsQJ0{=gKRgMHz6mDEWC&)S9g&lb zpM)M!UJtSeKZ>#)An4vt#Il5MzOMbbRZbOo%ReEI+a%KNW^FEU85(ud-?COpIQ>H7n9CEYCCZ}8Nc1d2i1D@{5! zp=ZnR4-Jrm9j7xcpu)poF8RsO5=jY;8|6$=ney7+uD%MYpr5ZOxH5iAJaM}Bi+L$#O;p#fVP-4PG;3mi9nwQWh{82=^gZ5@ZQAp^| zbp)#;Fk!vIpvCzPRvnBx4CVLY%sN(PVq39Wo0zx%$$EYY=}&Swu(1as3%-HT}k(=4ohSnSZ;oR ztbsyO_uzTVE@oM&#;iNpmO~|tHsBuOk&Me;C1u~br3A@sd)UmxleZO`de>}Gj=Kt; z7;L79;60j&-|yGIlK-6Pm)H!q7q{_qIAAxUiyZLG7r(VgF2{5M^69(rMe{#{Pve#7 zgNAkUIgGm$-&vDy@XwL>_pQ}a?k~FnB>9ub-D`7*uRLP%Q9UHS7=CZBzpRZW)f=oM zs9&6Xj47;0`mSGQSn`Z4x9b_+)bd^}IFvDc6V+}U_3FqM(Krot?(w}GJvL9UYzTql+BTz(CdrhRcMA|#S{vP{0dz9Z?oG-s14p+yyCCX!!peDBsK zA|c8k)E6@tA*zwbK%n?iWe>Zf1Rt88e0Sb-zEERwLvY znL`R0pSDa#pMhQ!fsF@239?R$i{2kTmG|l9he5Hp2E<|&qG31`i!TtkhR%2Od=Y4i z?O~anz2M}#Ka^KYOC)2E1sjIB5$)V2mCVhbURoJe>%qK~%@vDPzWqqM(OIj1DdXw5 z_y}v##D<9}VLgKqGnVb@?xbcviWoO(Ed&|obpsM)vP~UKhtiswFjk^c!?*q=`*658 zY+z{=UC(|!ysrwa?eQlR#d53%PfhLA3B=G4 zVC%$J^rC?mVndho$t9rKgvNIK26fg z+d(3bx`qs<2}pdHZlEj+m+ftj(BHlz;LGBG`MUi2${B7vLdu$5M zQfnhiY@%V<#C;!-w||zlrDM-`qR$&1Jqy`B4AwY$d8xonix7Sxz%&$5ocIFqkp-r+ zNxXZ#Gsz(@waLduWV6;0N##an{{xRU{5I2uz)4+ZLOkZ~$z$s0VH3T)>kKX~p@99; z?&pDLx4NIEN83OuTW0BQI)o@NZWw~u1Lrj~QcmCCq|H?FDG4329*QpRi&JhzH_<}(kb|GW{ek2v~}iPOy#HsC$o52C?q z>S--bHC)z(m5Jd?=8x^@j{5~<1B5Ey`k*%K-!0%feXr|i&Jog zr0k14{kW9TH@?7ZqVlzb`W2}cY&FVG_x(}%b=%A1HdU1~KXH5n%$5cPUl?#{4IMRn zHhnmG)`IV{_eV9U`q{8$$j9)ba$4Ie*y_S~j}Axx5T_fKB9O^rP}zkbUP}|WE~R;S zhQ9-0_;@aGK}n13!C?GZpNeS&`GeG5rpOhQQTAmfqc-|6vPNx=b+E^!s(YI@+1?b@h^|yywi^!^;!8jajREpm zo$HZQ8oSKNRXVY*fIwk7!_BY!3t4^-*^KJUPyi;5p422%I*op}*|AyZs1PWXbf)AQ z>}zPm*Xq-ig4|&VuPxA~LMym(FUQ1}bl^9$BCBuz*QC~5%A?NA_JNPK106z2MeS;% z!B{1sxzch{Cw5!r&G2sK;GH!0fEEAP^7r;Bli4bGw2E$)-*h;eUJ1KFaqHLUt0wZE z7zr7hf%kVk!;g~JVy{YlY(jYVUS66v=F6rpaE6TZwpJs&>W&R5lyF`RsEndw@MO;k zE=F!s)an0fGx6;T%7F3KbTsSU<7JL<^5P5L9@t*~;(L-7RJib_yH3p|zxtQ|hs|TA3w&3w&cg%+*91GSY)tBJ{_u6i0Cky; zzQ|jb>hZ1}13z;)-+ZsEs&JaKxib8W`Smte^bh{k0ReOErLVVD#SRa$QpmmsVcXHo z4;o3cSe?j%dAI#)iII78Y@f=4Ke-50ZArNdr8egm{u*F7e;rqz__Df%n+S^0W${Ic zA4Tb*yiZB)MU<9gr#3GQ-KBbp0CfeW#mCC8TucyTJ%_F{jf$LZCa-^#W;Pog-fJ8& zN!x1-al&j!@~P#wo4%EC0?eE{<_Ahu_JKGum|dsg38%rh`riy=l8Q z75AfO%67c{k)_qJIIKprB`Y@m+FpBAAzSrXkD#B2GHai87@rhs%@9QAk6`g4c7hDOHh6+PLM^2Lj zi9%%AKzjhDbaQ|snH3ZN!9RZg=ze94<4yGY0>;Vun;tipEH%cJ+R;B4x_wnT)EaX< zFD@&Me&=a1z=X`{@cEe343!)Ys$`WDmt8kG%Imhs&kw}u*+ZZhNME~zjWHPF!7~AO zBUEt;+emf(u8$qWUnpy0R)XZXk`Dj;cq`oG+RuW~n7mbwpm@|CVKy$*=9Xvm+_-=` z+ki3<^#iR^9n3d6H=n;je<8q<4LetPj%;R$mhbV;VtOb`{s^{~^h;Sb0J)5i9ico= z!j4(RkY^3vQ}u|Q(NQX5(ou{E$WnpjJ=CpG`uaXd*}#ZOv{Q(!@4Ii9&d;lOe(q1P zLu}^oL(|DSjmCPilOJCu}fEUUR4%F>@NSLzM|Pz`A2cXy7cBm&D%9v zwS;}9Ti`-jl=(PNVyrlJiOPT)hZuVTLQAp{00^be!JXuoLG(j!dj-|T`PJ^=XySke&Ib}dv?Gmw&V4Y znZ%jFvfQa=GCc^louhld4JaQ|pM77Zt?w_BdPgkh=MSC7<4jLK$W9(T$Q&SBQf<1! z-n{D+D;(R{;VXrQJhJ(_aa4kKMH%;H{Y%=?P(D8b&_&}g;cS*(CE|Ex)rS|oc2_LD zeiz|saV0^$KiAJIoau+zm5>rT(RP%jb0^(vNw5h!%WzY7ufxFJ&t^zxfcrIwH;q>n zLpA10(7Z+I*{z6Eya#MR5$Jc>`NYB;1U9jnP4`5`&nBdcTACsA!sTBqe1v)3b4hgb z?Vi;ze)8ek4W+F;GDS>G->>R;m9?)9QlcDSddf&TzWmws=TrAimVS?fR-TQXa0>nA z_`H7P!0RcrdJtrU0T!7DLnX5GJ}2f9xBDd8N5xIR^upV&xIZJ=kh$vo%ZDG3w_ZN| zAnDAH?QNd<_??th9oabg8bjFnArwdR?rzn~%BY)t!>@1NmJLJP(DlkkK(6=zK-36{ zY!Y6AJeC&;=iW&QdR&jneA1|DsZm+EY@=^Kv)PM;@POyUYDKz@=x`?GEzQl-2Ak7@;fx z4PfN)pw0LLvG7d-QxyyEXkoc_8Ke83B#Gu zFC>?;HRAoY}N?=c#)Y$k*!^`=;*7vtgucL4qxh3Q^$W=O=R+C&{ zi0kR&LO`fHm6GmtIdnmK=48oIc zk7Z3}Ni2_VbxwGkbb-sno|U#Nv3Bytogk$Tf1N7@_xHZdh`!h8x~MV!JHu-@V`xhU zB%>xE87)|X*DF!}YG3U97EE_PRy>jZVemQgo6D!-W=tX|1I59z9$5}W|M5JgaS$Tx7t=}HX5 zEA8CA^!03*ijrCwA~X?T>;RYm;1d}b+lKj@#Nvv~j;xU9rukNq%#?Ma5h5?wl<{QU z-!i&(M|2rmTVD0&BiYAgFAW(e)hY(CnGE}2it@t-&YWgl31#Kl!`{6%>pHE3-1&zJ z>!)y6Ix#q{pu!8Q+vW?G@2_39Y`gQaYlynwr&e)o-3jX!;0{1}uT?kwUYn>H@U2zX zheqB2u!(*%FNfGpy{29$FdS!-(9zVnu~seO?Qu_r#K}x&-lzG7-ZrX0uVAMs2{b!q zc+PqG1S(ja*cth1TldX!K9f4Sr`EQ_F}{dap8I`PsiW;B z=ot3zI{tyMN950)BbPQ;J?_f0T(8}^Bf}_}ttq@lDNlmrv|e()51>1D0NwS~tG-ix zFU?vlYIQ&Qr-9mQL%45%iPLt$&d8AY6{^3}hV;ut72q-H+Zz07uh4_$8f)q^^7>K> z)d$J>j>g~=_~|tQ&Kut5dL&Q0_OH2&RI+wPN!mI@-90Q)wYgz$7|01vn!bMQn6Kwo zH3w*`$Z)6mzXL%3wLfK!`!u4i$QPcTl0h&WcGGgm1cUnO z?_X?yH)DX*OawNr3^?`Df0#QLg!@}O%%L2E4l8jsE0)aIbf<35g=ER~>aoL_^mAQl zHS4rzU5^+Ygp-V&vaWeIT-`caU!^e|j@v)oYt>TW_qjjJWB;*Y=tcFlRn@GV9DG0y z;JY5_PMu7jbLkcIv$6CM`MOdyCIGmRhwt=AZ>8*k+X>#e;f=-)cRsm10r#>Y&!x=A zo~EPEK{`4_T?+wxK#<_n5u5?^hV+?=Tsa;01{nALk8QEZTM7)mSg0}MhHT~e!n*yX z`PTz}X0+Gu7Kx_Y2oOPQ4t`96yl>cd=G3sb{_VuI2`m67s?u*8q^<6Gkm}I|TI`T` zxQmLA2(*d{=Dv7)2Rv?!!}JNi03@NaLQLC2{rh_hqo z&@nBVf1?Sw4Ma(IIjnBpke_hAv+J$rmgMogVB6J$9<36hJ@56p)o`zo<^i%Ytt2_mmIB|B;V0OlNlwo~sCTo!atZ^wIBPS118(9_= ze$Z8k8l?NDN21J-REf(iwV`xQXvLTvHs}+q*@z-f?O3-fZD$g4udwJOqC3X3<9{9$ z!HmpQJ|v#zzyQxrMC5q>Gr$Fd*_6c$x-JErNgP}>?h>m!qoWDVI;bb2%90GQuk^}W z<1CBiWP|`>9_=AH2fbQGZK(0Deb4!W%QLc(!v&Uoa&-Ass&8ccf4o=A23;Yb5u9}Z zT*_2BsGE`h-f4Apc-2>Fc<5aVN(>_zrwzxSw%(SB&Ir+)6 zzz2{pddyw7PP%2KxK(EUIgqH}Cg?{2_k?UTcL5Gv(sD(E$#|FU6mh4DkI0{TA9gr? zb-oiaOaXKj(6v@_8xc~>fLEg4VSjgh{TAyE0 zxQz>qYE1{iYT8_sJ(VNrk`lq8+TRz7d_&!QSJ}I*6@#H4w`?Y!Hlk=_g9pS-PCi(c zK5gEfAkE2UzH)YUKL+%Xiu@AUuRiQkFjC%b@tmCdH5VYYw#dTx6uGythKv|8lNYrbozmfWK3^XaOghm=Xy1zS$6fFIG z{(4(hikZ0QeI>wdE3no3ko?`=7M&kAa{}m&PQE7P<*Ir!>-epdce(NjxaV3pMun32 z0NnX%B|7D^^`-nx(eXy1{}G1LaJTSa;nfZ2(6duMn}eEa??BT`B@?)|Nd$D-7}ZUS z@9LUtpfLr`=WaN^JR7AVHZi#PB&FcK#K~q-{abJ+&AnHS1IIE)i)vfa@1ZQEYC4dN z4U@w9$UBTZ@fKj9{`3vd^++yK?;Rg5$apyQE)S?mc3qi4=F#u=7& zYNn0^IO0KF#9y<&a@Ommrqy5wRs#kO)yj2kHhS(ow$CWl=k zL_U8l^vWjS{+e=ID%BhM5ffc>d@yw)3g|Mkcv}&e{Q1Y{yww0R%D3J=W57q%1+ItE zDaUEINnFF0bnsXLo?&!Q;EMo_kIELS5bu@cq4B+q36GzS+zr~ z{0w^`kD<#v#Q!A3(GhO`KWx2aSX5p3_AQEpASK-?-69RrrC<(2Go*BbG{VqGcM1|J zHFVbi(jXw+4bmVC&AYgt{}UhHPkJ1V8)omdS6tWcJWpUR!x#Oen=kiqIm;ST0NwSb z(mK_`nWDOW=MB9#)(9q@*CGbTxSpGypkeF1qy-QjwHSMTTGL|FjB45^Nz0|W`dSkG zow<+=-KHaiFx0|m?k?^3Tbtio5IRH+5SLJ^t^|m74=&a7enX*rsc7HnZL*d#g@Q)j z#sk@7Kxm0lk84|T3lkhI+}=#Q@cGdPp0_pdf|+NhC4(bmMSz+t^+}M#)GC#C-Y&Vx zlw%+FH$ExY+Ok{ArCtSi8Vaa5^_#OU=~R#u-!TV4rov-OfyN>hUCR z9_Vy_Ssqft+qW7YxRFHtbtXHUZjj8CtDatxX2ke>E}sRzb#(;kbeCHCrb1 zVVOEe(MiG3gyhaPc4&tt?1Kn~u%ZP+42x!vhfDJ#=uhIZD#ycSV2{`jJi+ldET;iQxLastX_P38eK{vWh;*6q5ow>>ygO^KNm3daPZm_u z_p~A_rPO=ZN>q!#fNbQx4+B~scJc=1VZqCk<)UCHiJa)kpLdT*ccXs?FI_F*GA}mk zJ?c4GF~;s}N^pf4E(2waP$v;Yd5gUueHp`$zJwvPYYhb!1vlG2U}f-)-#t5A84w(j zFs`05qpAQd)TW}g7=hM;|{sHY|-jRaEvmot`W+{g&TLC-g9|(}BeduSug8R@PZMc=2 zJofERzFXp>P*b6j=Gq?F*Y?L9o$qILIxjHG+IGomw0k2HDC;_kmcPNG$*!S8HHv{1S!cH55E)#3gNB-2Y6AEN{#A}^iswMmyV>Ldqj>-WdU|MKC z@6qrvdOm9((4E7%wJt8(5h-=?apR{4&~%D!8}FmDCj(=BSr`%cQw5`Cu0c>SUgu~GpT1aD^(+OW_{@3f`1K3vBpa%~A$ zSI0wcID9=<8c4qfLug9>M2H>I7JwqkXoHnZ4cf__{;*gb{Gg;S8c<-4q=Uj0_~eJ< z7;D(%dpsaKN4)~Juiz9%^xC6oi(s+o7m8l}3RUEr)yAB%Fytw6m3Ri*m?Ci^<=TYe z_}Tqv6u;sFEECp!gQ;RGvq6r9`bS#3cE!xu3C`~XUeA`ls(IbT=-S?*JHV1JN zN-(}~eeQj{m4rj~iuz;EI)cO_ZQsY+K}jU4@xVoJYt4e^rv`N2^T^5R*Vz7m7 zLG1lfOgfLryw7dM?T;P5Jbtr$3P;U>QotV1aFVv&YW>U|S1~5u=qhW41dHa?vY>YAzIPTI=N^Jny^C$G z)K=@GUu0I%-VC2foH)E;hOCuTE3MD?i2xY>Z_ab=FR5ifW0gAoo}w{uH3}T-rZq+~?gpJ@RG)|k{0yG35ha6F!ouCm9L_B>?!f)D z(SR=L`%M8>2YqV;Fpx8=lX*VqZUE9}=G28YjfccByp3|!beFFM%?EW+H%l4gbIrHz z&BJZ4f3HlPX0&?3J`Y5ph?bU-sM&EhUNO^cFHfxbt=Iy?-)4xtTd7ehA;*l<0=hdHPB~`Yz zZr#UaB@MDrl<1-9@qAatREoWtV!mlB)S^;@LKESJTl9oBx*K+Vo3t8;~SwzzGRsKdcc87RlKNZp{5fqf94!@Q_@ zozrz`-Q8){Mw)?1mMEOzzQZeDunvk8{%m$SUUuc7v)YZ>4BpCn zwOBetNwHzWpxCwu@*%17EO|Z6M{y$}R#yJ~u9xDshlM-U{PSy@{~m0QduCR}j{Gpb z*S=3L`=|^dyXfW5>Xfs^cSICCDfmK0tM!3id!V>FqTV>c)bHNxrm{e=Xe7c2)&xR( z)A7n;E1)XLcD;URxv{`KF3yIl*n7wD-mY5*p;GNv=*Wr~V)Oa zSdUb7Ub@xnpG8#VuIC4=k!Wfaj;}jyJ96C*RnYfc;dYbQS+-dBQakfN{k8m+dGoU& z@n2SS?-O3nQHj(3_;kaTDkll8h>r_#`uzJ|&p0<|r5_POPPwj+DXdcQr8A6u>}S<@ znhC}^Z7+ZCv>Xep!w>CJyRrrb-1v2FThi-Kwi(=QJM0!UQ?5kcc0*7M7|-==qd z(Dl3aga=pt>S-3oeST>@vE{SgK1VSnaKy-=;SHn3ZIef`K#az@x;~PbnSUCH@nOQaH3@*fl=??n>U!}2A znoiU$H;(7Q?edod)glA6d(!rW7mmJQTi|NjC0kDK$kicd?&vq?S*e@Nq^4su(7yIu z_fUjgk5t^>IqXI5oJ@6Uw91WrW2^CnQ1HO6umg-p42fd+4-|kj>wE zd2D^L27B%AK3d72OFK($@)k)c6qw0O-%3BBTj+^Pc|od9fgbSUNc|NNgQu78r+kIq zGDV3I@sBAh?LGV&!x8hRf5&7@+%dh**N6*`kLQG0ZC}5_+J4bcAe1W>uXRe13|qMU zBa=^!1p^(%V5Z+H^6?u%j)%>;vnoqSi4!+Z#^@k%zFgPRPG1YX-C<|q1oAvxSLbm>gQ@oeWbf%}CmCk4`zU+*) zae|=YpV1b<`0x7q^GS2PM247?Ip6lcl-Q!8?Jt|=#Yh_4Hd$!Opv$sD26yp4_R~4L zh~2m*JD$tyJhPKA#acHU(s7bq#(X-je`Q*3yH!Tq1ztmLlcsJJSB`SCvN5Vq1*_EC z?_eMX0d9yPAD9=`fBqEu0=@G{E?7ULXqF6_`fTU}B=KY;;StO*3)5c{=CcGK0OGuy1(TRYqL4;$BAX@t2*n4mH;kEjMc;X8Hvh zsYiKja$zS47YG-ej<_TLVGuOEi$TRE%ribR#ViT=Z8~P`JI^~Y`n0H$J<%bT8kxFk z$pt5A3=FA%x#Mg2St0Yh0!yr2@M-b|oP>xY(H(&+Nn;Ypqr;aST)Sp`4Qhd3$8L}4 zTVG)9KUPlVm6b0m=V?AO8s^5LsqaByG?C_z6X`SZ+4$8#y?DC=Pz2{LyVC>O;+TpS zjsl*i)Z&XG1OLh&I5+J!(8mLGPk?`6RznAibM@1DKgHf0`37l3r?7FcUO3lC`(qyt z*yG`e1*!YZRHeR>Uyc=uSgH*6Z6hTVA-ZbvnA&7UtAR=4v$no2Q@Bn;3fm%mrb7z# z+y}^9+xK^BPr8nK+ocH*S{=^`8qjQc;Oy1-dvito8At zXpNey=3fY7Ce|TB2*xwInhez`D{?qzcoGTZek&k$N+fcVrgp!R^;fZ$W`?Qp;c&@F zc^`7;%W8X&Du3Dsde;!gZx_~>y*U_cw_@gI7uHJ=M>0K zY#vnID8w+q8Clc$yy^okpPIYt_H~rL$T@TK-)J53I{8`IOWBAi?I3j@N`Drza#i#2 zV#m8!8TB4gHKAp z=&`?pUfvZhA3T6GFLy0CZ>RR%=+wUrf1^$X^G&QQ=MfdrmM-5MM% z=+u6qOzdkoB52}zfxu-NbSD>RMp%ouo^_7DEi&m3`(;&9XC{wO!HK;FnM-x&qsxv1 zFM@6x&)xslG2XjRlj|JjGPS!kyD0Zws$yRpE_%vXi$7GsTt1CPan`Ik)-+CTe593qd?;J6!>L85-n!%-Kx28pF^o zr|wJQz7Qgaj9!!j*2^l}MyFyO`)U6fUMFZ-yg+GOU0!xK^*D}g%63uGtnY2y)Vg%( z0AKXnmmQAQrrqcKD&aJbYj9X(*cxWQi*fkt`m_qV&4CI;$X=NZ=qrn0>X6No?9M`0 zeO*1%vNFV`2D;X22q5bvK2A+&L3P!v28W66%nddJU(qD{Uyaj{zxCdun->yJuUF)N z3q^Xp&5W+RC~57$?gTZlhU> zfVqzDc^lG>FDSkjJ(I;u>jW*WUlzD)5WPcEK9_xR3rb`*jjGt3h5NF(o6ZRP=mqzu zIHbnN>pwNIVe`a{MG=+Zfsl2qd>JTadW30de`Z8geL$F09gQiYo0agOUjjPW|N+Iz=E2_y~7 z9i|>{EX4%%!vgIE@P&yPilDC?-k}2Vq{dyvej9x_*}sI7^p~|>@LWc_uLtxw_038Z z{e+g7;?RS=L5_$1qx<~@S!N3}UYcH-3tOY+&>ZohCY;|@A~J3iA;@sQ4U9Q?`kK*V zO$EAbqsEUznHf3dM^D2g0&`>i00}Msglt{(xkg9<^o=+25r3Wp0|klOBHH*PTrE>> zKfad*{(l3j6~B%=Op-k~us7#@i^WRp5~OEsQLX39t`0}2*P4H3^4Dk)7|=aRLNQr! zm>l>s1*I^koj!ALf}}XrU%Dka>dY7>=;bPDn&(Nb`~?mrRHR5T!AVQZ?cLz4f#c<&zQ4SR>w(x z->{Z+9O*j{YdA4&zDRfHGcjfneHN~w-Jf zA6`jEZUERr@$i-eZqtqc(Zv6fe;YlH zV7M=#gByQ{8!tOAhVeVOW_oxEucsusT?7T|J*Yb=Zxx!zI_5Ba0%qD9za=lVNUcru zrMZZW;8_$l=Meoor_|@CkxrP^C%JO$$F6EW1TDT-lHIo;8rPR>&<_9Dm2M5e7_7nM zWW#}(;#tL7i&=gZ1=Y-fh0INa;nTqOUe(&tQt{SwvyC>XgFj>${*Pd@~l3BoPm;j+z?N)=jzfTp}z?CBZ;FYoI&HfwfG6v*MEO`xfgl#v)wGQcE%% z5}2f4l75OcSxNd54~_HU(18AwLJAyFV)BJ_LSM2i-RtkxIzvO%O0n<)(2RyEkm77R zbkQ1LhHbOKNDOsW@Afs(0NBxInUqBEb^)n+ZGWj?&y@EBD8xW2VP~L<NU{dpqTUQPT;V{M9|MF>wf6^3Mk=e~rC(Bk^w@%r9D%0899m6!& zxSV?s?G{2yN2+x`fzPQU8aV%;R%*p^uyIu+C(`Fo0?~vYK%D(P05S5kwqW!}kjV;9 z|97s-KW}~HGli=VLEJel#|*9X*h6CbGfe~l!^KPRSl_qv! zfWj@O-wR5fwo?atrw>!pX3Z|VnJ|>|7bi5l@VB@yJ@@XM_1+atr*XBSWUAu1g}P(e zY1tU}ER;?;N4W4gY=pk3KS9zmi-elptWzIC(tLRscx*PxSoD^xFCltmp>z0%%@-SdJID#nMC6v-1j6>D=25_fwf_7*vU z^|_6$w}0k;djaT?3u`%mdF^i;zqH*HM^_CGfETn962`RJ1GAZ>vC!r3*YHh^zTHc3 zB<>@DNGRit*LX4FEa1{gOmaVhtFfPTKT2B`*omXU;)I})k|OU@t&=E08Pd2y6uR)$ zgwdl1DQ#FCL6^5KFv6>k&O;_R7C~M1{A{G%b@M{UDi6ykz=7Y?c1z#Wd&m~M;zN$U za(XV)xB_XA>hw*Z7b3M6A`}1Np9s&ZZVGuKu3>g($*g8$7jpy1Kgzj-GDdF)3Oq75 zBCGtC`l@(L-ASsXXJFNtIa5xyB!G;Vt(lNgxBnfw7YWU0G5Ak37_Q)l>cLfmFr+lr zs7?h4vEyUxehLm%_)y&+y8vr+)bnq5UUgH3PFW>(Cc*jC+)duFf&Z9Jd(061D@V zLURyNaq817b(p5XkD9tw&Gxxdf!c?`u^2GZvK*Zu{XU74k@v^_oTCFWo_G$&W?3n- zGG7Z-px#I9=wk-&RZ7lJPWLY-)byNtqWavGp<2;+s0CPIku^{A=6&JsLw{STd~7O3 zcZFHgqIUPNRH1UjX_r9PX?^)Y78Ck4CA%qq!ycliw?^qiiD%BC_#r&Y0}i>5O$ z{yYnsc`d5zuvGV}FQhK0c4XIVLnX5Oa=5&UzQhcFrmh9}D}URr<)^<_7yD3|7O`fu z|EIQb8H>|Bl=E-ccon?U2decBa+^@>MxP)34C|S{%T#mK`oisLApL1aG+e5TkHoVzHWLf%QuCprtyo4pj3pzp0U^H#rLj4?az%?a zSxvqWs;_%pFop+}X&iv&4P63$IE!s7G@-SRm&)nm6m1Zm1gWcT6rr;n5{_+a@W*R> z4lI;yFaLI3O}_c2*rQQ}P*mPj;m7J8e=cz=A$~;5_LOIz5W{A9`9QtnC;J1pPcHq* z{BvuBgV-BWYjaCOU8rNSq{Untn=6DPEfFMB2P3tEJGt1I;51bs9wA2lujM}0`}*Q9 z-nz5Tg&;J$Qbot)OdM?EXua4A%&Wyi806JKqTADM{R_8KWg?UlDwl4qZ}Nm7e~lxX zRv#ai-o6wnpskIaGRZ(IbJE7Yym1Wj*5+;A@^;z|$cmPa%$0$v98?BCo`2KaH=n=$ zl&Do?fM3xy%sajRQK=KmHmh;Js-LUCtKw~J*wokuEP8m?v4n z1?e8CqD!>~Y23{fz&U`RmGrC_EHEK!M36G#VVEBlSIgCr` z2Xmm+fvdB?914x8!w>DCjAmX9NYFs%+y=QT`&f@TiScP0Be;i%l}H)vqtzJB37Cr_ zSGuCy_8c7CKeL2qy|7QxGfQFP)!l|dL#rP+P2cB4dw^a3suu*GZs`WGQvs$xWEjEy zcrErrSl;YXOkkOEFy4Imb1#iF5cJ^GMdRi?ZmWx3CT~nJH-SgYXSj&tzM}u-HAXjG z06u{tyfd1dk6keqi*sBtg#G?-uD3k>D0G&cwWOjJb2KGC6Bie5yGNuk8`qX{UJ0!n zX5#woyjSMR$}RhW8yQw2cQwrL?;VCrZoegrg!vy73(aAagh-Rk>r<8+0nJdYdqJa{ z7m}l;ogFuqJzU~E{OP%S9(agR&YzU{v3kNfy_h;Y6|oDWYBO0G_=f5a6mYH8hGwldgx6ax}fm+ICg4Xa})`gAugbQ5n(^fIQS;_w$F$!vP%}H-O${kg6Kg?ir%Ir%j@Ns9HJ-WMoS321-{KU~Plzc2Nmw9fh zrc9GuX2OZzKE9WD!$)1I-tqOE%u=HQ3XxpzNIcUj76^-`&icD z3-Rao2-c^}y{v_^!L58^*ln#DA}aJBq6>|f(~p(Y$qc-PuYe=>Otup<;p-LdIy)GDgSg5Y&OS+)BjG;Tl9 zZT0t$1!?KV00?f_Rwje2c=_kW1%k>U?sG#16(R?R{32@&In0y0cxtXKqNqz~or7=w z4%P(%@JkfkF0)=Hp;WYSSe)p|pRs#AV>1@mY2JiJ1=8FPn-N))p}(l*2)@i!rH<^E z%-{L*7|neB^@^!qGpkA!7Uy}7pi-TCwHK%^e9PWY!E%NQwb<{|l?glzOxUJ@#JpV1 zCaAww6^WD$qL?HM=YYI|P?&v~-*cr`mmg`A?km!GgnE)=d=7O%gkl@67d-R0*o(9D1G{ywj9UVcAoFH9jp)d;KS(<@`az2tsJd z%yh-&B8|zn(w}0ov^`w4MEasXY5lF0-t_`m9j&ILG&fQ;!%otiyN2U1d!uO`t9mu& z>78UNoL41NrK^3&mpgTZGN^93me4F=O)NhMdpLK+qg|@7RrJN zytBQByxKjuGaW~;d76pFq2y^7Ibb0>PEdo6e)Qgws9@wieoCk~Uv^LCuUC$?wU2ZD z*|DD5%5p5GJi33_2`3M)G8IooG;xvjITloxc1`!2G5mypth(9zB9+$l-9^c1Upr3N zF3scvKJQ4we=#)jt=?qejD-(JmmUa9mv+&5_Q!OJ_^%YOZWt<3OyP32E=-nsecVGguVF2 z*jf=Lb==%tFeHGVAnAcUj>vdbbPCP z)@S9!^U)8*!Nb!w&+yEwyWDGWuv8a4M6Z}KOREP)1$aS;7aHJbDN6MP0r_d#Ni=SB z)JBsSn1xt|T^A<&7DZl&y74gO(HskPH-xji-;pEN{@m_JA_rx`lYbnJp=8h<9?5L` z&3eI~lYP8RR+crkd}CXlR8?;O&gf!=(a%bh=jc37EwTal3x(nwnkuMaEP1t^)I`xJ z<6=h9SM-~%(d>7e)_d|G@4L`kR$W+hy90gWKl-isDL2gZdLkgxWXC{zh6?}DIUvpb zw8^L)xb94T+9l<2e$4a!*2^TJw!f`&Oq=-?Qp{mw^gqjI$z98Kw>ej}XCbBV55H|Q zzJURz!vT8*v0M)=-y|r>t@Rt1qmLzkmia&maaTVFK4`O)H7C^k2XX7)$!K zn)XVfi01ryUY%BPbDKXd#&W(Bv64PH-@G)*6Vx56BibO+wNSUvhPI(6Y0bvXd_a^e z_fl>M%#|d6Z2FHFx%1L+BvdvIPkVa=fbmD#IPJYTYTMK9Em+JXpAqYEofjy+BpUHq zE`Eekv!RCuA=@JW{AwltRe@@ik=5vsO5BBzhgsIjeg-Pez@w4N|Dk37`y<}}p8oDT z^M1i&vGi2cP;CelP{VBy*IYq-{V8k2x~%46`IC{5Fuywn7V;Qev@kmq5T2iZ=Kl}C zdj~{YFtq;JwGgzD(*`P#G0Xo0L57pyeP;lsIREj}KUhZktw(B>Uf778xoQ&+}e)2z6Hfr~W+$>faNmuL=LrWp_Rt z#Gl0&`K+q-LV9l8)9&YLQ?eBOKu)oj**vE(`L$Oav9-YJ)-tV*cH+kS zM$Yru|Itg_{r2n(h*nB1hLR#iE^*=a3Ixmy*_l3%D1veDe# z*?u4_ERmqRGb8%n-{rFGKg7uU15#UybBdwC&;7-%y zoMXJBdY=OUE$Q&b;-LR~5w`ni;3^MZ{3~GmE&tzV5{bs2j%aiL{C_{ga%ZK2j(GN+ z5hTsrT_^;YTaCWtj{JW=#C``Lv(O7s{QtR4{LgRw`T-y{#_yR7$P#`hy8B+OK++N$ zWO3)d^nbqC9|~}%U2N|S=o5awGknrI0HpC(1xg@o{=YXJ<8Dkm5RN2$_5WkG^xxoT^0y^C!O~QKVl+=J$@+1hbU#40US>&2s(v*e%FG90OvC%R zaiD?N+n*^ZUO!iAAbn?RyXhvRVb-4vyExt5E++?;VJRXGMq}kThdoJe0HFBIaSdl; z4`nuxYI-K;f!ebn)iQ~JDu3LlbK0o)Z1%}6tqkW4=c)g`R7!phjMKxh*Z-w>#mV=! z9V4B-)yKl$pHFa(kw`bk=frP3-MS(ms;d^1UWy_PmL3Mq1$3K zKjr57l1#7o{CJ~5q*HNmu2a0tEaxc|?^gGlk#Evt1sZg9vwUEi_IQ+7n?T{K3A*JgD8ri{ zy?|qlTN)yYO>Y5u2JZHSbWx|Ui#XXSJky{Sv$)XGmjm4f)24S<7lc&gwA}G{3Z`W@ z7u!k>A-5n+jg*A<%Zo%dd3POTs$y)Y1^ny`8n|%>2OEib7&F{&xfmtRU@6p z8f-q!*q2|*^g8O<+!Jm-{IXfUfC1odlV7gW0^sM{{KES#zjXa1(^p!(`55fdW};G^ zY{l&;W`^A??qUw@QSUV!=~dU1^6DLD9lAQJYiL5cBdA3MoPJT5o9qM&R2Nxnsah|) zG>TY^v>exu0!VH@V9y;#ax$vH($Np%jc3^L$nIcl zviaD>2$sP&$;@*FI{+H|WH#qA0Zj7${z&0%K4T1v+c&wh!#nr0Vd}k$F<#QRvz!9L zF%XuQW27-cITBP=l!;&o>dalB^(ZOVB@|z&+--1k zM25oUYS<~?>r|C;7>YG`daLWgVx4lfknmTvS-qjM%Ma~=K+S@WXTkm4JkAWFlI1|z zp}z}&_KY}@-~4w&%0(|dexwbEO4cnv%{_Ahw6pnh5KEU{{~--Q3p%4EvU#AJ`cXvr zsGBLuBnX!x_MNN66H%glK(xPOy%&s6x3{cS`YjzT*vL7n61c60_5<7LDR#YEB(RxI zE~93Kym@EVe3rycu zkxm_8`2?IB*BjI3h3LGV&%8_%m9Eyl_Hc7T{n|kkiMHM(aNlv3Cy(m{YUyS;MhOjaCCSc#IJhjPYR(OEqPt*6>=WdS@OPetEB>>oI= z)O#l4l8a-(z>Y>3*m{nPB(MRqTVkZa-Z3uJ-~1G)0*tvm)uTXSmAWPp=nlH59Xbew z*lfhKBIu;&sU5o+O7z-^xnV+^FUka**84a0l883|0-kXW-n&g(L7KpQYiUY*-uedd^ghb|@){tJdRO~Cs0Op0$ucuQzzg2RUz`JQ?Jsh)2}yzi z_<2p)1IIV|iPrUQ!vKi3sQ@=neqzNrD{Zrd#clnrBQgGNAaLQ+%9nxL>xEkdhxWL; z@wX1d5N%{4N3vYc2)$G|nT0j-PmJz&8S}f{A*!wE>pIez67XLKT<96=QtJMvy zcr@iM7%Bq$X<+#yZ&sO;sTs01XCMxRx=q4@9bvYAKUF^=t`gj%Oi;~(2reMQJiqbQ z-ron)+#7e{DU&$r8lnZuAf1n*{z6@N{&f;w!jiXIZ}s>Gs|u*Su0<05hS`~lfm?Z% zJ9iE=C5dIL163@xv2|G>xB{(4y!bqpMXSw@Jt-F5J%MxrvNqfeJ3vOFE<(g zaO^N!b?bItX#c`fHu1R_%~CwM0>f_wv-oV5NZULHomJc(;_Tr{K+h3JGuazzjw$oa zv(|Lw5m_&o;VA<}&lUsPXUbY|wxsrX&V7!#YdS=q)9~~Xf5qXcrRRUyWcNL=YM%S+ zIp_T~Ai~8dtVd5;fV}CKU46E9GheXkw)BN4O7u{C3RD5#{6nh<7wLcxniPFte`Ks= z`t4W*dGQxufoC7530ESj98Cd1+YSn3#saSVp=>r-e8;XJ)i?>4Zmm}IeaDiNZ|&1S zC#XYdIi$XSbNJibY!Thq z^1Mb@xCE<;7`9`SgdCl>_)%tCll9mK&4J(NxeuIXY--kWmu~5Az!*=N0*l_9ZhqCc zHs_u0#jME9pRPY+rCD))JnYXFtrId&*Y(-lT)|40&{p?IUADC@NPydL9ASGn3T8to zq-yr2bwHN$FIo^8`@P5_nqYk{)>WkIgG=6_bKdI$n$834!BE!B#`(6uSd0x-*0vFVbpK5;X!S{e*(uj%KRUhNI-? z1mkNLu+@5=5=gRbDWWM9`88-FuPO2e_er+cYL4 zf~-#)w#7ze`*w(}`R-7VBwhAD{udaPJ!@QoxBO&0FG*hvWR3L6!Y@Q^(bagm6-&T+xCzHl*Klhar%ZmF*CoS1jDL^!fnAlCu# zf8Uhi>S+#sJQ^bdc*oQj`*q)5df))_5Z%=x<{o9DYaJ}0!arZ-bV zK2H@~OdusmR73j zQgox4?|>Cmq*m%%I4b&}HCpNB>a)6Y;xqcjX5tZVQuo;Neq7Zf+NdXGhJpvwj~1*B zpT~G~$7~DF+RKsWS+)Sp_G9f=P{QwGV9S{n0Svz@|17M1j9=&pLeSc`qV25&dyWBG zH!+TxPNd{mrrarLRyE?t;Rw4hgO1c2&QM(|A!B$al2F1CMj(IY4oRvgb;p`Se30GC zut#F=m1{ujx{RJc$u-2*{`5m;tmy(b0?uc{?M$T39o5Z#72ydN-f4-3y&;kvofaDz zZjrP1LsxYCyuq7QL>ybMTDPmqpt+Z3ThCsPVR8D9^9S5~=}9-iw?EA;abtGgZLu5< zP;0n|vuc}M$;`RloA|flZSg&W9|q4l5*EnxoZqZxxXp)-aFdK`6F|g=IFa}I8kF;x z<+~r}F4#&k^wb+SY(Mg7V|<_E`QAg_TrSq zfAFHvHFa??Ag@h$hMKXHzJ&~L0P?l>AJCxH*_mZGV)t40!7GkkoqA1bMtdf?n`R|3 z>dVYl7~p*Zh3Zc*WvOcz9o1#oufZ9NC^|KQ@iiuRSs=<#HI|6AAhJ0&;pPrbbDqM5 z9y55xuG}h<8CgF`3`wW+1Rd+vO7_WUUxDCE3I8jB?rt3?diD;$ZGP7 z#WbJMV9werczKbxtv{ea1+Pt!(MnC7q9alHdVz#)><25xgCFa#YzGCRBG22(h0sn` zjqxKpn4~g3cJPQ1kHyrzzu6SYRb3|=)gofAdh$|2mqN=yb6PUJouhEgu?6}Pw2A*} zG+V!6I$s{|6T0!T`mpK9?P6-Lu(z-atzk1a#$DuV4ypErT1=En` zVGc9Vxl4InwR1;*qQ%-C!Z&%L)Js9@P2+p9RiG#0toQeAWXWr(g%OP=uR`hosdCzT z`^3Hk-_y+#u8sy^*B28lv}0X72XY70W#PdJsFWFcVjiFM_UgSr9+XPe!U-PE$~ z(0f7BS(S2LgjxUiOGSu7(4E#c`+wKT^=_e$zBv#-j_)Jm+I@R!#DCHkDFwfOm@|K) z`$6i=Ke)w3VTAfcRD5v(t7x*V zMD2QcEDQLp;=ffnpTc}MayqBb=MGV+fFl3yH-+jSZp0Pz@$O4x$oA^?$DlCCM@|Fb zC<)jW;_QAM{tWxo7xf^BS{CBy3hr=po#>WlfH$bcrZSCv^tOmESy=*ueP!KO_8+s} zzfrPkZKFpjvvg-+XYjGoH0iHg?rEJ5Zh~h>Lr&l3 zl0XZ1%k>eN6Av>!4brkdE1rEJ#-j^+ZAf?^Namvi;U|6B?7>)bWktzOFBP z)~14cs8SYN<2d4l7Im>)&9E~ahsmh86h4^$OnK0*7(umQ8IEA>x83AAt3;F9Vr8zL zi0X%B@T$@3=cfTHdoxjqWqWZ!yC1DFf7zhDkac}Ff(XQu)tQOP6tC%+Lb$X7HL*2N zec`(e_}s&t$)?XW<}Tm!nv2;^yQI5G5=Y%$PInWo1F^|7Z|mBc`Rz#_L(pi1J$>cX z3sw6uQ{7hYa`7g@WlG338_R9MRX3vQXRXZ#F4#_Y-+0#UljRQAa&eo_M$!_$c%GRS zbv_LfFzUcEE*|~mYM5j7Cwy0nfRBZH2HQ!8-u}y(wVmH@eQ;2$D`RfAacixr*&%(H zXm6jYxkGJeTwEIDF*9hW4sDn&DJ!YiOJ!kr(=d(qc}88aI;;#XWUfOco;7MC`tDJ_ zkGDe76%s1D+-pwu*kpTP%q|y`u3#MSF}ikS5b3PHQ>1d-7ZI+yc1CYr>l#JG571eb5oqh z=z7@_iL;_jMmntGU4wHEnAH9->KTjUJBzAo{&uV0f00zDq33>M9Xt=M@lML5Srn*V z**JoE;v0NY{K4yQdm!<3!+MT!W_oh)E6j=teir|#gU6q^DQWUiXmM<-eCFs|@JgFn z5mif+(Wn`Vw>yf8OQqvf`* zwB>jp8ND&A&d=`}7ScCLH2$5|hf?KXx%|`KE_~yHsN}odkGimX@P&@)`O=kBro#5) zWuBgzv6i7a8aRK;%}ME;DPrTCvnW6DrUdtNK9sJb&g1P-ZQH=O*oD%=e=B{>>Oj3}ep5-c zMMl&>{;e$BhN2p}qN_mo(G305#4+tg!%&T%bo20yo7mXJ6z1~}ZAR^MILI^KqL00F zOfNgHRIkl<*V1Z3ce{jD11=qWZyLFE%Q_?@MFm54brlbHFx&N3q%X_DN}{Oid`x6Wa5%e$ zsjP_u-LH2w1n7LFI4hK0^hcep_W{&zd33}e_!gZt97Zc)XRE6PjjuQ^NQ<~>dHhyI)C9OIrAJIZ#^P~ zj6BwxTXxQi4jd!Og5uDT@*NS_h2|bzOoWe-AY{C7w(r!@N9{kihhQh%MVlp9<3#gT zu`RXKnmL@gUjoN{to7=dlp^DI93#1Wn9nbGP8`nEI%*$|tm;eQa6Eyp@9Q2zD}S_l69H?pfuBeA4P;!)RmzU(__ z5hlXJU8OPfjVJ!ue<{!e_0ONyD1PKPw?>k5m>$bQk~F+gp@ z!ehK@C(RyCK&7m`J{Ac*t>1V8IqGE5T3(tC2*+*FN*E){LTKYyaltxCl4}R`)+kZ((H@@uzLro3zHe*`#x^%j2R8>!9@A2TSznQ)15 zRKenXbGnr~OVB6sLtaxE9E7`rZNAED!VwpHmYZRQRBl{~njfEtU5-3|TPmW*o&=nH zPonYM(n}DnaueA^BSmH(Xf~NXZeo4O`Y0Sps*@u``H1RX0G=cBm$01bP$dg|LpMM(w z(hUi7f3>ngRzU)^Lwec;Kjy;;Mn8!l#vw_U01;JDK+{l$GZYaRP=p*Fx4L{7VUFOH z4N1nKVqF~&|Lp7@MBn*G=I^U|f@i12>S~*p3z}C(SUc!9+%eA)M;c_F1 zKJWnOqj|5a@B|>!aJJ7%tTs%2EtP$P`x+tjK&&(e>)D-YR0S#2LRtUjO6cbwPaUc( z#6`FQ!Xvv;d8%BYW8Y8?+KD{>V#lq^wK3>P8&TqPiwq}=oqD1n#k4UmiISvTJ05sN zmhbSFVq*=?DG!w{N3XJ5h>ZEOrU~T_6Th&cQrvo1BI+a_C89RB3dP2bA+$t1G3pWx z7d=}ZsxXcr*m^Ot^{t%@%LubB1G^cXmDD!d z5YKru-Kx+l3<7?wE7E^_;QKEn3 zTxkb%;+- zrr@>j-(01>29-Xcp-{xX@1avgAq)#ksEtab`Z@ZM(jC7|!zO|zs2a8#WD7)7+ff*d z>u5<8gbk#>I@hIRKw3qU5+r^}3O)Ral#tzByP$XZEQ`NJO>TsL?E}Kw5Dv0$#CgUJ zvM1XdJl4zF>f_jyUWZm()|`2o<}bMp3igh?5cc-0b%-}@Xrc1JVtH;ft>_aDaV;sxg{jJsw(VUsXVVQZN@ZJLwFlx!!nB}EP`n<5?%XMC{6R5e*=X;yLu#6 zvfa*W1(&V*=F&2Z%XhJ&|xz9Y5rkp=w(hl;b%pD)Jqv1RMjOcvNISomQ2+gApa`*Jh^GS z_(X~7AIZr0_*lOUte#p)bd%=hqxZ`M=uAm!i6p`JRye z5Ij<&uvszUW|SdOT6ASihZH$=bpTNnp~aWieu<+ZG{b3m?whw5ngd0#FUdy^J_(OD zQ4_rH{p~HTB;k*)#7_aTDt+zzr0S(!^e93&kS&(Ma+akSDzT!}g)5_f|FqbfU>k?X zf7_4XS^E>N^ajb0a1^&V@y9nMz+wr%j?PJhr1NCpjoP=Z<8T*1lJmP+#W{rK!2wAa z_?LilVVx7e;A?Q$etXG{A$D;z7JzZj!*lUIoZY5h=qk|#D+`y~_iL*~QEOO_AHsFa60AG3#_W&VgnV&@{+RBfU9ngkDmk+f5fdzD$z zF`}~d<*&^b<$R32g`ew3tTf@1e)^>o~-@<5YMD!x^<&E3belM0*z{2s_ z&Q~Y^ON&v*vsw#fs%;Pwc^c9T7w7W?U-j zZ+q~XQqxE97?mX~w6*n-%+kFSJ`&0mXG($l!w=DC004A%$fZDdS{)_yDHOFV9{Tu8 zdpPC<1G9UGze(`s%xKJ1 z=Th3_G0imI@9QUf0SC`Yq1J`9FUhG4e>}4c_r&Vm!qLqa`en0+dN$4AB`Q-`xAmQ4@EB`Du*TJA7Z zg1UWeQv3Xv04i&k%+8+snG}MYkc2Zh@jSYiWE3 zMah0)TDq6TS8J`n_@woOk3A^R;xuv;)?0E@#5w zQv2xL%in$j-`@~R-v-y2sY@)=_Kut~lcPP~4wt@-<|C>e#e*jCvjVsqahTx^#q-x& zI5&oyeInxOmh7ydJ=KLhJR_Hxy8w>!_(<^8U1rBa)(lAgRT!vmb&N|oQ$#T`_jVtEVBz-m$o%jU%`9f?jjD79s79~VeBD3FQKEWUeg42h zN`Op2SzSY&dz)t>fv#(QJsy}u?H>E%>xh`p>vJjZimAsZ8!CiCCu?V1bmU~>RE*f{ zqR?8*N5cLO6z9+SBD(8eMRiH1{`ld122ghE2p3Xe6~?0E>7tQ*>_5K0g6mSd{y9iZG^pKFc~jHkCUS*4i>PIMR|wm8ak6=O&Ks2#4^OQx=eI zW1@3PS2s$UNvA+>@W}uw_!oJ`&hy-1?VlO4`aOin457+ewXHPie8yLD-sQueC<|1a zjSXF296jE4emCS*aQ0b3i4ykkmS8trcr;y=2#jw~a?(wrL*%$Itxs6gj4 zTTh}yQae5q^;m9VEanDvb@S(8+M+a z795UpI~YCLnf=YR>S)_Nmk4AX>MTCrb#Y4LCcY?GPr6=P3Ri2En zmwoG@7vt{I4ANNWNw_+CP~yvK!$5{~^yjDdtpuL^l;Y5yLUMOyTccGnCE=b;g`KzY zkG@`=Uq}r3o7bPL$^T5jLc-biL={5oxbk?bKgEP7&=V#Kk>y&&B9PyO+$aRGm!wAf z@BRYC^^YK}g}j}vNN#m!l*Xpcd*3=CXjMwXdOZ!6yNl-qzk5~5(o~? zKltvpW5$+XyKy8+Nbv3Bex8b%a1TPew$%zt$){e;iL5Ufdpn`cG$z=Wbby`aquaOJDJC5+6pf6U{ z=!wrII3~awFZIISyB6>#N}-)m*fDKk>)B8-o!o)IrKg7&gvVp+&^;U?D0P7IyS@UO zwCnDWI?*bv!1cj34&VlE(1etwvkv(O*3t9de$nVCSYryJPrVAB1EEf+Zb-01CxE(h zHH&98m(MvS+aPveN07oNFwSR*euCcNwGrkOEYZqvOB=&}b+A9kAO=;yU^;p!axM?c zpm}1q9}vGFbCq8twgvDVfl4P5X}efbxoFu>#@=dB74CNs5RIis0jn;Unv7_PDOJ7j zwkQ%La($ePQgsIM$H11zmtYXhHPR5wCqP17fAG&^ih$f45a;tae|FFE4`BzLGuV^5 z>Rx&7<;hE=T1jRbLs-)}X6s^><0M=NS3 z`xh~tzMecW7o3hke7BDU0KA3kYw@iW2*s%_h14yUi=QwIqpm(rGoz$=zqPZBHej*P z#ZMQO(7D)`xZ2-HhUL|iXMzgbwUtO8XeTCKMCIQnv5C%J%$Z1*un;NM^j_TMzmVCw zFfH-l$5A4!b(!>h9Y;5&>3th~`}<7ZX-dg`iKRy3WFh3FdPiB*fBjSMFz?90;k+a->-YX)jhpAEPwne9 zmHA$Kos}b2h0njIMwzFC-TOb}A<(VWDuuWP7KC-gf^XLBKOnnc6KPKQ=s^?_)CMR~ zt=h(gD0on#^zAlJTN#aVRQfGevLMeCR+Lai6E-yrQ^T876n<}t36?9Rop^{ZCUnJY z#IEoTCr+}JzEBd$qU4C0*-k=?^$(=Hp5-hgO+_w$C~8VJPhE`N;l)a%7SLo5l+fYmWT~5# zvdHg}^>N{P0o~yc9%0*$*6Y&KPpe-Lq^j;(t5?JG%5>3?Fu&uZ*U4~QN-4NOyN|!9 z5^9#Kt1t9E0|%nvT-#wEJ0&B%th+JY{hjNg+^wFHxEweyu>R;VZ@w)Hq^6BmDyI`? z?Wk+}?u556ZIEPj=t@0U&`Zc*b91l>m!W{J%FB30)-Rr~7|3k-J8`NorJIDV*Xtmx(bc5(x)U*_@kG|%OxSpVzy#TmP$$zeM zry2~4?-~)h1S3=_+1-9R3a~)5#tP_JQPKwxj~yfY*UI%y5^?w*DhVyxr)Fx zMfe#ZMsmXEg;BcGMgI>Xy}nReYbwdiCe-&1yK&U5%r##L-h@=aG}in~s5DX11QqQi zdMlfNgXZgAk)&3sUy_0{kWg_rr#?>3Png+&_+%Q;>=*1I! zA!`;>UavU2F}eLxg{50sag5)s;9LGsxMR8?IKe9qpb7Y}6s|KR-{!8tx_Yj+Fn=Ra zsnV7^A1{`6ed8DZp?yYR{H@?7&M$5_(@dT5PTQzyiy~w@dax-_+^;l%upNh!OSR4~ zZUc6tUgzDp^5uvZhh?^S`%M_10TQ25PNtF(cVvy69aJ3I2Rl6jAawQDdwBW14_%~M zX|Llvb5J5BqZNc|2bC27n*<5ec&DUeTe*&3a@BY^LGy8rJ-Ret>+}Y|+{C7fG{L0w zp~R51ogv3wUQneh>15)LR@W6=UWGY6S|tqDiG7Q@XZ*yr>|KI{x`ee`UB*&4h2R6s zIyA#DoKkf7)kh`k3N;uZ^>5p<7bw}5l{0zu_Ej@mV46v^-vM%qdzHUo3mt}y7ulya zu&T!dyWi+p=MYiA`P2XjfwEenls*;4&11y36t3r}fr>rN^n$6<;Z0q9Yg$$Zd?FO6 zRbg_d;F53j=Ca;Ya~rhPcedIZ#yPsO=k(95UOk8|kRphAS2}7+)`fH2<0f>JM8|^U z@r*4*VM+m%Wn@RY7StZpv<`${ya91GimlEYZKKZD#MG`yycjw1sv4JfU%Bvov8G+H zbQN0yRf_mEtcaS$UU#^5upE5cD?zD(<%-W8d8J&cj;)<7e+bYX5w9Q;5$FDQk_ z&<*si7150tTBK=zBCo7|G5yPPI}(acnAii-EPdtKCr$N7Uv3zG!hN1)PJBLJk3MAQFSgTe^ zMW$gupvFQk{aQJIvP0jn#3XB{vr0_TUTF`f;7azRrDkRmZ=q7ogHBK=?tmT*rvsv@ z{tl)9D9>|&gk8e=s`!1#_mufIzTKry!PvdLSINIkvrrTaMYZ*Kx4i(q9>Yd;>MA|6 z;$>Xt_s@}^&(gEX1tFRu5+Sj$Kl`ky&-7FuRsL@3q?XpG6ERYevxjU=OQNH+Ee%lv zRx&=z{YC%^`(TWLHK2tu&T)H}GEiRN{5Ab~s}wUP;#LvALEv=kOIf2MUlWCz9K=B- zlv-;0X5Wj-n6Mrx#IcW_nr3!Y87B$%>=$k;7}*ifcY11~XE?TlQu?J8uO4F2=qW~y z1>X3@5<6*y>D(9Rmb|=QtR<%DSP&>=vo~5CJhE(hSy7799eKvq27aHYGnrUR&Im_FL|& zj-93+lYB}IyE~Gx5WhAhfPMQ=Am@oiK=0QW^wX6lJ2$t@Bn5u(fs6m#C3ypykDA>@ zqx4%Vc5rum*D>j56jijxfw!4M5p`4oCxb_Ct`~)CPrlN68!KKYAxq^-5W5Qr4n5st z-^~PXLOFZajgz02g@=Mhge8+ss za#B{!7i^B%%N)#_3{zD11$R=Lw(EV9hXc+|)}EM!3E$xNhj7a?NKrfvhJ?22w|p3n zcIioR-b734MIGhG?j}@=LA?0zEqV+5DZ{$5MwKfc@~&6#Xl`NTfn>v#gH4C6PHKj6 zX4Al;!hw}>;x($Bz1K;R8JE1>z5DFN%MQcrO60BgLDSuH1&c6w7g_tLV!mqM@?%{d zRNaC&3!`IS6J)*(+85_B$wKsCV8%@l2dqIO)SfPpjgm%1|7gQ zPY%NyXWMPuJ{#ki+)vLs4m03bMV{wxQA0S6mk30LZYL~XwFiHe*>nLv8KXbws@3Zv z4hB@AQ1?jh?i4pL*`05aCNWfgt-8`w^FD^L^LlXW7~40RGqVb?#Z{in@IB)!5UoI92ec1L-g0&pq`dmU4e6q5uH6RW~XkurUBbN zsnu67R!3gETP1s*t+q^;)eDlsCW|&ZAC{%oTH-;6ZjruCnZ@b4T@{DS!82-*noWVg zZd7?k@GDqA+tH@@#uGQA0l_`lnPZKb9R2!l(*WahVq>}$HikKyL8HS=si~cQR#?Ob zY2}&2H6I-MIS#V%nG%|?Y#@ifM(NYes9%rH8ll_W$K#rJ@fVL(mViXiuO&E0>vpRo z!8#G}9zDB>`sFg11(|rfF)EAogMn3Ps05SmadQS|k%1ec2O<}049eR(#^~H8uzV4v z%!9+`O9WofNCze@CO%fHbE!Ewsrt&FsT7qp^Qnx^I`>HULgVR#a2S~U2_Vh-we zimGYiYszYkys5L{N;vd{ML0-Tb1+%yH_=#ML&O3U@9Xgk+!vwWO*qOMnoCJG1|9^f z=gq=MVt?$s5m57fE7W>os(t|U<3FG{A$Auepb(5NKEz9MtyQh!a5LM(^O!le_H^l`ca?3A@ zE=T;Zbt8VcUR^|5x8Sm4$I5ux^@0DOtU8c7SIL~Pcf>2?ye!dv%zf8yf3@evs13kL zh~cQt7fTy2k_KdmO9okbeFtuoJNyt3R!7_sHQfL*%1vRZ#B1wsM8O7 zR^%X5`fohdEdZJi}_BdRB__EluM11CGla!!eEHx4_L9%$Z3|hV`lmE`QZf&eXZ-$%zhRMpwPsU30f+Z1{-Poc!IV(HqGDZ#h52A@XFI3Ulm$^&L&cNnuSck! zQ9rDg3MGBl^!~ zDoL(GlDgQgaFD4g=t;HFA&+ibHM*TEz zIPd+Hx#+Ho#^Ma}tQqfN1@aaP*(x?H!nLJYil}_Galqw(gkxR#r@L9v0QvZSqQ=n4yUlWS>+5! z6ju=)Smiv1jYu#BjW8B^Pl>*20m0t-JNFX}Fp&#hym7ig+#===SnqG`JRvMw+CXPn zYn}Ct7b$QlCEy~X9|zqQcOm!1^Nc7g1s$kw(RDs@26XSPG+fpk!0nJ6DM>e8~zkVum^8hGs zjP0}(v60QYShKE;$tPWfS~?#4v(7DDG>nuFxYv@mI9 z_JH!fUiyce2ro*H{zFu2guM9{HK${*Fq$HDzZ*Mi_w@ZlWM245`*V5oS8%sKkq1Q9 zj3%_=DklkJ8W^D5P94QVUKm9od`x4Gm@bYmngn6L*L8Uj``H|B<2B1(Vf{##4_k4M z_Hy(G2X(fswz5BM#l5w&L5zKi#fdPA^%!S<>uY80jItDb*)3Jm*Hg~$67!?N6o{lE z&8dfACFB90lmCoD7+eZJENVH5WdxMcnD+`VJWyUH}y+oqx?cr>Mx_keum1y+j#&K=*ZwX90xZ7ym60drqF=0O@CbFVP*a~ zVZjE&JU)t4s{Gede=MMh#R($A6B^_}s3M2ggr5^+a-KI$_qPn7*`H}KIYD`dmFhDn zs>r}+gmQtf$;?`A%s($OM=Xm2D%phP6)11+5AP-rxL3GKeJk~kMgCJ8P8IkAExT4$ zd2>OyU4C(c$Z9tPoBmZyI=s~?rKQH|G!RVqt5j9>2|Uqn{|tuzlw~eNY2MpTYB=qo zUjZ+$xdU}inGB(}<)308OF)}U4%zl^*lghMFH4LhR`3wz#n)R@7p1!BfvjWW7*q|+g>$l`>&6z!EHy*Gsub17y|Dj z6`(rG8SX;=ZnP;-gTa_`Z61UfS)iNH9dPEso+3L||0-se3K)7~)p;pcgaYtaMP)r_tHNu?U~S?J}zh@GgtTTl}k-Zwf&97mGu5ya*0(!%$iD!sg}n z|J|licrQU6B{Bl7{tWjAV2JrqwqE?FgWz3#0v}7)A8YJB-T{rvv$Fu^(&{%8nSbhH zZbc24pL=n+*_8Z``HNBaPJT?)hW}%Jb3|pJ%__2Y_K4|72pOin;M2*VheiXNJP+b4 z*j%k=GaT2q_n=mUU*(ZZV$%z#vS_J00xA$$8T&{i_MAs;t9`_F)k&eo9&tB>mS?wop-@HA&>0zINCOokw(7( zP_wEYYW$kYR2LP~K4I2Ml{|=+Srl_n;ye(MZ)n?Z7|n(wz6HwTI0o|uKyUp_|%|77@1hsKm@2}5T8G4za@6yu@qJ!=> zD8}J{GV2(&kG4QHb3lbI-v9LApQQob*RL=I0h-WkxCF(xN1flYb>9ntvF*>cdNc-k zsh(-ivLWh_EUHRDEZwGRR;{k9^_ZOQ`va$OpPjr68+y3JlQn?MaR6k8maBCO%#(8e zJ(K@a@%MO-TSwtpxMMm126f4v>6so#d%0Bvg2IBD1);}G|5@ZuEczg2_zgW<{cRcE zx$xyxVKANjc=Po_FKqIk3Hx_7nnDZaPmY!#p5T!R*FnM`(te&ldrDLKn(WD`ZGcW!qZo@;#&_{lT*?|S71J6*ns!Lxrh@;e$LrI_~uKlIvG@rg<|?x4580-Fy$7R>FnL|1Ank z`3QAO15rosI;P~O@GiGOpD;K~=7zfXnD_{_atCv~<$u3ZK}ktXTz8x={lka&DK{t& zX@n|4QPHEWDkN?4?=9^= z@8ZM=PR>l9JVns>)f46s@*O?9Y53=RMWNh(&Bs68J|k_elQgR@0TdW<^soKs&lU=w zb=L3=C9BL?>hJLW*ZNKgtS`jVzdHZ#LV-GrJqK1no*7(~a!L*$`6LGbgBAf)^kUgY z%RhBh%D=N${NG9;xz1o(_|`-HvklHg|y=gDavxB_zZm;HapP>dYGeNO>6Mpj@SkOW(}18`tR z;%<$AcpE*W0UScZQ5S6}5yh{1F!I z^1k&rK!Wz|1xa_o$tLINp7Du*~5jsIx9`>z^|UrPsRzXw8D|Wi>8xJ z5&`#sM3H^quTTo!?z=4xfP3Iu**5^Sl?}W(U`0P~SGm>s?>LmCJa{RhdkJ7oyZTml z%L}?bNZ|d1@Ti0&s+22CVMjD$k5hOQj^2A6q&PJ|7Q|abhJH$II#u z=DG8A$`-Ys<9l}2Mv{5z^W|-AU&D#x+y=SvL6Jv3?;PAS0kGvpQybZip^C)=)k0-Y%0 zP@S%`FCfFh0>79wln?@Urhcop+x&_t?g2BKxAM1#087AW@4%6bebBNKqz5Ta23?3Y zBfzx(*A&N#(l2~Tdzfzgj>trpaApc&fwp0_N()Z4j?LBWZy(}RK>yvA-%B9OL+C3k zvkyN%!LVPs_}Jpncn4$zE{X1gF!sb-mhaBWCB|V$of-%xr^`oydI{Q{>PCI<%C-Ph zTVO2>ES)y$y!wd!(>`=*&-C&qk+o2V?BeChWtV^*{xor1U%-yY>aZa*RGMdIbTdbk z;@4Z`O?w=WgQNcjiB}8+xXThyvUQyDr!Pvr{|%6R@B^E$?r9&z14IbGoi|V%O?8;u z!WIq(=G53;0Jnd(_I?H|qURMF4pWe>i{?6q)(jH*1OTBUY?p`U)n+;gh~!wqk41AL z2Td~oa~`Qj=j#38LN36?(UvC;eDPd0`Qi?!-~w$gY;GM#YE^@`U}*!cJ5nuGVy5Xy zy7haLLE@YF7p-z(y{+ce)0p5h zdpf)V|88!aRVYB@eT0n$q_l0g%ekU%Xg1r3`x}^1!b(rl!k?=WWAUr%`$>PKHod?x zR(K2|4%68ZUJJuCLG;gktIMk z`nHV|(Ui^zc(-Be-pg|i8@K_K8Qud(uO~N?{y7vGz#YjEHd@f`bV8u}&bC0e4?bz~ zuH&4jlQH|vm{=CpcjMGwhP)sFWV*qgohEEjRKe7ixoNE1+3!~0PO{+QOZ5_og&L3 zTvzT7y^bK_>8N4+8T`V-c06E;Cf{mp`W3h&;FrvE&&|o(?A!U4t~oZEYx>h4hi))i z`pc&$nqtXBTOpRvZAG$q@0IUJ0zXQ|BTg(8xMTM3068x+Owa7Fw6S{5x2p6r8&9o> z2(nVGF%FD&<%c&$%^q9{oDNmI*1`!qr-*0hIee#MMi%nm;+%O+GoCx?Ege>l81!}I zKpwCcd_u9IkccI>dG25(c*~M6EVR!zpJZ-`B&&*9K>R25seccOU(}x(18NK*$p&fTi6&-C^UPovmnrFMqFRp zc{sh^wT@T!V;_iEk#$WEKW|A%XLnz@+ZJfS?p2Dr1wvfaeYA`Ym*E`#g@*PuJM?34 zXYmc8-(SZ#lMZm;%a@3;)1mBHStXRboHNUIr==pR*AoVhd;r6LO`_DLsYanlq=9Z? zq>MI&T3=@QuOsP##SAV@)P7l?rZuV9qn`qcTW3rv%AO4QNg}H=OSiY0h~`H}B=^sc zk=;*A{jniY8rdi_({alBJx_|1+7o=nEM6yPiBg-Npf{MTV-&`pCtWiK+Hz6`m(s)m zK=L+{1Yfzk;esvoh$YZ7Gqly;Sv#QWQ>P=fPQL_ESW+;uy%9OKnD(LV%M)h5Zx(#6PdO`G5E{iwL%?UD&y! z*C!+N9RBWI|L;#IhBi^w!uZr|Zxk6E6lcegAB#J*C*9A~)0zY=zhBl~(S%?DS_&Zx z3KBQd)nDFMyVN*1#NlS6tV`ufT+}C6x%vWR%pOoK{5+3&1BPyv@C!{;mUG1%>@8lZ zgjt2dOpEy82b4_kDYMVco?h2UTE_~OmZX9HW%xG2@1`u3_{e#FVbwkI>mh}0uSEYS z`g3U(Wih!sKt#5V(80C`uc2R9^pDIFZq*qm$(D4O^m^y3pF)NK19q+}a$ z=L7wq6NDEMf$&sh;Yt5(o=<}Qr#Y*!)4~8x%%zC!tLVXgG@qH(&xe``zOOC-DI>1R zqkhgXykrH5*O6W)>j_G1cUQV{fGTc2J*c$WR=7f@4asozJHSsw&&(?|B>04dKJCi1 zW6$L1=SKsFf)Ehu-usPw@zRe~25E260@PE&z!uCM`+ptTF3~4lfYU4~Ztd{VQPuQ%ObGC$`oHtpU}4Our^ zoRzg2ASF_ZU(y5o3*&)g(VzE5D<|o>6nh@UZ%I*Ew!>1Z4mb{QD>vKq=#f7l(*_F{ z`FwH5=-v_C5n6U$dTBZ`E}}}Sa+y+aSE8cGNt6z_H!B$%lDd>(rF}~&yw@>DkwDf# zJ2a}%^XW}S$0pq}TvrBV9g3fhs9~8L{YZ6k@!@sQp)T#Rooh%2@o1D-rzo{ z(w@2ci`Wu$(ecd~(vzR5u}80!42W4DrABvOy;&Mz8C1>6#j`Et{<4oNaPl74R&=x-0jsK)A>sxu$AFy(oPtduud<5)~La|+!zCsl) z6qEAtDd8h}dI5bKEs96}e`AuIc0jB|y-kjmRj!w*2f6X{kEU*do(tf$( zia(r2mwd&k_Qg^Urm{{=s4z&Zepmc9BI2>cgR+zlCP=}s5WadWSC^pr9=a7Q3IXAV z${?5+04(#j5HO{!w&xE0yn3O~W{2Jr+dkBUt;Vxi?QS0-BdB^08;LTY2kA$$mh)TE zRDqUC5Eq;PC$}v&Gq;$lYECWy;ncoOM@pYB>+Q9pJ9Ucf(k_8R(3Y@qX<;<#fM$+1 zd6Nu$v^qQ?xyhk7m~Dv%E(4x^e)nL*{#7bz^}EWF-f~Hyi>_}`$Ir}*g#SGW#xU^r=$j?1n11Bqp?_S z`gdeW!ta;R3>p`Z3NusTe>c>+`I zwHtSDN>8|(9SC)7FYz{>BX~pKlezFQ?bCJ;K&`e@0`xo=lb#PD@^rLnCB_%8`$Mj_ za7x#X=%%!0!I)v0T`s0#;rBJ{YI~>zY(%&p7j#IdzDE=-fM)5PL!9Y^%-d(sVNe$$3$n+Hv8+=SMbwLrqoK@Y8p( z8BRJSny>)osrXWsP5w7L(ITHh{tGOFcV8B=na9+>y+Vc#TgxYkdXA)0YGVU>p@&|} z&AMhvBxq|*_#AH(Ph-xUFncS{Ep885%;EAjK+In^K$~@99XL!?Nd{XlKH|TSIrxfv&@p<`0?M6`WlJE~%@`;_)WhPRk3`=PoWaSx zx`k$R&GJ-wn$z32(d@pL{RJ#bE)!^OhG3tx-S=Qd|BjVy{+8k0fpge#=OIA~G8kfN zl%n?2sI)B8Idcwe0oGO#Rv|QTLc4MfggRh*y4^}xy4<85q%Ac}H9FFtTryziACGLZ z@5O>ZLIy~_^b}BIsLQ761^kV6ow0f z$LOxYp2*K5Tw>^|?d9pDhi|?CQAM5wJS}aXt3AYm3w4#V)=YCe{`xVg;1Bhhj?6 z%a=T58Ru7WO5`2!_n9}Gl2R2{7?-*J=}n0;1f0X$m(nr6x)w#MO*f&=1E=ahxg$R- z=-Y>$>^;~-FYRYvj~l!IIrYH_$k-#E6)2ayoS|i4HRru2D!B*g9?6_8{nn&!K+m3A+EP&<2J`Zev%G)ESPw$XmVGQlME_K)?@{{ zAciQ?!$^A@iXF$JA%}ph%iOsyoQc!TAMwl$x~=S*3s!FLPbg74YAcFn7Em{21*I`@ z?r=BcW1X-Uy~;*X(N+^`=5b@CxJ0=vz0mZK(eJl!a7T$~da>(TTebx77iQi^=^5RA zu!xM(tNk*Yq_}TrEIfk3MY#ejKZKw&r}v_%N_jnKK%pv0P()phi6 zcB@tQ@vdG|9w{+TGYv_|TyFg6+7-Rf1gOm6%HrOB7Z4Q>JG@PXT;wcinNfI>4l3c& z4*`Z14}F;i8#P_kfLbg#kyXhpQ=OFVWylkC9b{RA4TWCJ!hRE`pW5+V<75b8T5E0O zH#Kvu9ykU0krN1dHkB>aceA2cV*Fj^c*1~Nl~j`A3<&KhR~YKU6jFL^9KECv2uylJ}a$`=`=@1n^l{Vx!x}&5}qu6q8_|zws zrD%u?*&$PMCQY$NK$LSN4T>w8CZ_s%TXEmP2|8oxvbk%Rbj3KITE3v(h8~lBig2v| zf-gkgFKZ_q9l6V8Q>caHX&JAc?EnaL>RS>(dmS6E_UdLMYze?421W{z+mWrQX~hCX z{8=Xz9AB0TnMlG6VKkm$Y{Rsh&q0|@R1c;6NxZWY(k$PQPtFglEiz4Obv1Ie`Snmwa3Rr zWy4V@LB5X0Cr~vv?|FYrI81c^YU?wu9?6qnjJ;|;c_o2(w6sVXSMSJS&s}-4!IDtA z$GhAETt*493BtwUIsy9ln(eU5loRI%Zu+10)(SUwY4?WUIq1}wL9o543|~t0bV84-pVjnfrC%Y^de(f83dwKkkf=FRF@dqhK!* z=tmteyRI ze>j6Ha}r3KBD^>jQ}862hy*NUTND$Pc6tB6nf66a`+MgFjDW+F~uiIzyMN|T^bX^E;1^|ZJr(kAKtiimB*%9v246Y~skT*MQa zS2`&5{h_=()!r;rilyXaq<`JmL%3^kAr_@g-mfXNSbO$F2&{6RO29)FZOhWwLW@|*CQiR$Eem2@UBA(C0wQBR-wcR@ym=vZlx589 zRYYz@8oCUR+Z9e$7|#a3_&~_e@asF-JPF+}O;=Yzn5GC5n@La0T?D3{I{5i(Zu0Kv zW*(I+`y~9d_Fpfy7YplUj6e-0?Mm@}Cm|o6VGIzFn0{G((v?(dC1)TnLGs@_?@YWu zjurNO7~Q-iwe9<)sy6;!W&Qb97twd~BT4Al&sAxu`ndVzQw-fuG^veU4!}gZ(j3w( z(xiL+w4}$iv*c;VScXCxeB^GSQ<0F!vR3yb2y=m{9&`YerRi;^P=D8x?9U z8uO{`Ny^luyp|>p?u^`N1cZ$$-8)efQ5MC097{puL#1gVnP=3TPvuoce^e8?*x8gL z8H#q_5kPe6&2NOOL`RDA3^V%!D*I>z_yQnT=dWY$mwPlY)*ev-jgn0i1kliw5@#4q zAcyS%E8;#(U2wy3G-g4haq;B`j1v@RI*9OA8vu7T0G|=7-;_Bi# zDI$zGTI+)m5-r5Fnr0!F=^F_k_L_Go_IAw09fwvqph;eT8cD&P(~oT#vY~%WWHd!!{Nv<24xVF_y_u%SQL&a6s-b<$ z#ynnfK60G$O{a@~&02G}retV1wnmA*Sq=M9jx1)j>?@?oK4zOQ zn3$0~I9azrslP!9)fi2f`$UYb%uY-sLoX@<`$RY@s|i{IA=x&nVUK+jc#=8=h2w?EI#W^-?*> z{1R`V#GORUc`>j5Pkz9kIrLc>o`gX6(f5NJth!VPh~X1AX@eK0>1&?Ysf(S@V)p(& z?iF=Hf&HzrC&1cOjpQ9KxUtNbd_Jkn#ORdxa~9KkwEua>J9rMk7ykR-%=z)(hDd%& z>dp{FtHFHS!)exk{+-MJ1ik-c$|*R$*V)JUs^@?1F3l0G;du}S zVAPDh|Hs2^cuEBZy2Ss_QvdnmKQ0ps+nVeD>!B!M%$=S!O7q`On;M!MmD_Wiz3 zef_;Iu3h(>^Tfz62IS{c$reN$*ApB%3fu>?1sCK0&cjL=)6pPE>y&dQt^}wvd zA?`tb=bmViqBk?f0vMq?bey7}L1X!Ebrg{O(y8Ll;?6zk zJC&X>9vYhMPon!MRIzi3qP;+G#Y6UQg@$NQxb}2MU}#N*G4vQe#+(eHDH}RvA&*$b zkVITM*M`*c^J_jgOitmed}{5K_mcGe7nFUjMaku}uezT}bS~{#CLT7zlGxw1^b_NA zRN_4nV}~}nZV?(L-p?O6Bk#friR*7;$WAp660vRXM~m7wyHi zS=HCMURbG~{VgsYOC*XRCM5y}5~3jNOY)t(m(1kmyH;xfH%nAsU*2vdzy0%7fIne! z6g4q+GcV=?e=&-MfchUEx5`yT>0e%sKi8ZSeCa%C{>oaK*B#M;6%|F`XOk4KJpUF8 zws3IRcd|KwkB-ml1wP7syAcs<^)*X`g{rVeQ$#vfQOLiz&jBNlyGuK@aOX4IPPojM zF)~fF)K^}Z9~SFGexPLq{0y-V7?AzUrK3W9&CperiMxTqh7f9NwTQ)p_yhSjEur?uEjBV8tA;0%?$CG*5 zKyvPd$)2$!=~uDKdj>idvhGQpH;leTQB=NzkdCiGhg#D24U;uW%W(eZtA_;je@ba z`27Nzu<<(t^r!BBSChCmXL0_8An`Ww=SODWG3#H6p&~vfM{f-x<(@qGFVkFPa~5w{D?`-{!xg6yWq7=QE=1OE3GA!fN{Pf(TTK%Si-J4u$Q@{(kQ+eR z_KD5{UlgDh1`%axuF=Q7;vBT+ev}qsnyEG@QAuF%rsQ$a;H&8 z8AVYH{Y$^OEgZRH_}<%XzSy%vV*Cnwru>ugBibS=VW34Ijs)^^p&#r&nClZfQFlY6 zvM~pf*KX~$8Na}DpmjiVxb48U_n<6bLXm_#3VKfM6^+I4rj=qjMh?0Ot&^pb)shu? zhb7k~M=RGS_f?KTP9m2w7d_WNjxM%@QAFchL6c4|x}ihW_>spQ4?+*(no#eg8Odvf z!Axy6Z~DDh<}U9pT{+Ta=4H|4@bdEVr1E^nuyS#a^nKlBpRU!ox>&J7^TLVuAJu6K zpz863TPgy%jxx*l;^eBb&(+il&PH-ah&Bi}BsWkrk_&Sf_2TjcwW^*}Jv}Pikmo5+ z&LvW_iPp{14Vr>fbzAyu7`rgJ#0$#!M0n<&V;|{l+$Q1PT;Id$@d(}jr z2r&r=iGv72!^Q~HI5xP(IF9-HIG>t$H-8=vG+nTwWY;lq99j}ENa=gGIxs$@w|12m zoR^h{Gt82=otM#D+>6umrsp_Wggrlzt7o`ZD~T`VL@!Z0tSZq&)yiQ!+Pd>&Po=)m zq~+OM*A!!?WtK+3D3_Ip)hCxMmk&oi#D(D*z3M5JDVv|9^nz_ZZQq$@axF+pF~~E> zoi*_qnVWt3tm$i`5%u$TtI!`E8&TCH>LiJn$e2}_1`&c0*b$E-?nSW4=!D4;R=rT) zEU;6uRT5B|PTdI~BO&FjHFq9f8}D=N(_fWa-DFiyc$QGZ8qQj(zin_{^z!V7gAjD?0sH1!ncK2Js%xht}XAx?Yex~WV1ur zwca(D>u=YN{1G|Hn>YQf=k;PmTmEpuPRd5zWc6_WNaUi{_DRS5OvY-{>BdA?{?0Epp5E#WZWZ?FlT z4Qt~{$Gd}n&5N7SnKmih?L2m=V^EP)QQZSsdDm-^ZY5gnZrfiE^AK89TQTduY3O)g zP`|eA{|;HPO}g#iO`RXwS?(vPDk*(OtE`h|#kW5n`#;`(Yk*g(cljY!%d@QKeR1|y zH_lG<+op(Xm+jI3`Qe05Y)*!s90VL{?9X~1`ZBr3TgBqXyOomEmHR&Q(O566E}|=K zRQaJ>W^vCX+*omSDx@lWlX;BA@WjMt<(xR~R-9MS-6A}#hWRhk_AWU$?X`;1DP6oi zQ_K_IUh*CB=axkqZ$7ksK-SB!mY*3rCtj>h21lj)liw`G%wP?dtC@VbjAAcqi$S;L zpsuXV?haaA=@#0|u=c7-wMl8;T)EG)4CDEOtM?ank5I)1ykQO>cVNp|g)NmWxku2e zu=DHSp5VUrrS=G_=Q7DD^(mNSF2c2oiR)!ui&Nhc(lS%&MLFEde=dDt7<}J2OqgGn zLdQ2R#I@6N6rUJ>T=B!KKLbP5EiZ1;bHBd-%m}}pm87QQ^Mrlp)f1l-z3CtGv%X?8 zJ9AI{CKnwSANDgTk{Ssr=^xd3p6hEb-*ouuJmAgiV638>k5@)^`ChAgC0|ND$v8yrxH#1r3io+k=H{QG9m%$- z#pLt!r)H3D-rH%|_r5$x$Ir&kxRARZIjz{p?d_KIJG&Vj6Dw(2jXNFvzDZ+uIKSn- zdH%_R-1Ga7x*4~O`2^pAn~w)ni$1}D1K&qybPmf+Cdokx_go+dg|hY@zQ7d z`yR$dks<Hj_s{wG3d?(A$Y$j0X8=EmyA#cJnh#>OEaAi&1X$;Qda z0`6dO^00M&?#^QCMD?FR{x^=4iIb6|g}t+doh>3WeO96GhcdEqn5RyIYnR+bo-z)#UiCzf- zlFd)HGZu>AFO?=9eeYk7BfR3Jcu77;OPX|SN)CP3@VbEPuj?x~;xKd8-3XR@kZhy6 z^kB(<4-ch&Ngfpv^!AAMCAq3y_7ckPrNI}FS%pRpt#rMqQrO0(*3By_p#N+5y!&8y zbW?N)?AhJ_?dgBp6Ne%|DBpecXh9{1a^Gbk{A+4+SwvJEmPwlrS{6v5W1C!-cmJ9@ z3h6#Ik!(VJ1^6td=YREY$syd5M}YW0A9w&pQT(bv@t0(EgMqxdoi(hq9ze3IE+VJ? zn!SMU7E-7uE=`jZcs^A7GwNTH8-a~GTaWWq|3Jm*T~@UCYxp6c7Rqgr5!xe3n0ZXZ zJHo%NZ%6^PDUo;P@q=j=ygy_7HT*D;iL-8`o2nN!buFrJ+h1CTGJwZoT@~syFd*5p zp0US&$+jRI?9S#AYCsF_Ys zSRP|-))mt$^1*8#1&qI*SC;_N!O?B~kPPk{U4#BDd=|LXnN;VMs-ohoZ{k`a{cCpd zOeVN;Xs$3Bi6Pkmf+=}_Nsm4N3`+)-(Qi3z1)R46B37nbFP6b4N zU1uc+*O3Q=%AiQ0N?R^de;Xz{7`EB^vziPr9hX2^&HDwoua*-G=kAgUa;Ho)ss%I3=(*vWBEYFzUTUv}1;~iGu3Qw{ud_Ps@QHl0Td+*>@ zYgwD!z~oP^YH;Iw(95^7QB=yy)_9tI_B{ZrXKgU2_i$L*ZY9a=QNg)^79|j}KNCN8DoU%rxxQF= z7o*hm{+MT6--tHdeIeCqdp`>oDO9d~&1KwRwK&52Mv46^lIci+swoiu&jkKIY+@t| zESdCqw8nF=(V)pdH_`JosvOyc)eOH!PhD4&AF!2oP)a*AUY~E)U(D9JaA_I1#7%;6 z4|aGqnl19v;uoEsO`0V1?#T`mI1toovV-ENcT14k^e7eX#)Xgjir|3DqI{!KPyu`=V>(;{WhF$ZsINx zcfogJPZqkh@zRX5tCRT(Wb4X7X=&4qUer;A<7CxR%WiQPZ~(4bFw9>jDxT>Gq!`CV z_4pz1%W{%>p6j{2Or`bF;ryG$6}3m3xtI7P&I6DYe+<@+yx~x9io=~*q17oezsu*> zmk0e@RXekF#2=#A9{65tSEuT6KG#A$5LgW3>e>57mJwdv!`V%sxZ(5D;!wwg=>wJ{ zL4gS^dXhqW7y(Cf{n<*2DXwR@E9|@h6}~Jp+p})-BD+uPcdie-b{keJdL~#NI1L$i@3j%u^}aBj_c`fR z;OgZTI4O?l>*-=DO)W!w%wzd*$5tP+)b2WccW0sRD{4IhsrL5bXQ?sL^Q^w^} zwru-3QK$B1Qnb$GvbeyyvcPg|J+};rgOpM#&zLq9o09vFZnmZ#?I=br;4_!ssm*R7 z@f&^c<#fH6VwW9s;xGgY=W>lsPsli*IWbC_9Z zfyL?c$7{2ZlQ#c{sE$Ke#JtQ#$OH*vZ-@m>VykEO`HX^^Ug*?%&HLT>vT(!`;r{;6FQnQfzyLc_e>baPE; zbrm&XYEX%XLU?i9VEnYJe`hTEY$-+=eMQ3#(kvLzL#>$U<9D9ZnpL;kSof^KS#r^D zSdovVC=Exp@pR#>wQuiz>f(xFh51IG?!t^J-Ol=-%@~vCZORM&fd-AsNR0hV>-5;& ztgCO#;;xDtp*TU&`$O*NXC_=fkbWI-?nK}%18&9mz#+^jbi%Ql#fg@8jJeL;f5M;v z`5qmSX9cm}W)d%ive>T7NpUCxgbMAqdNQzq)&W7P-t~+Z+oKJ@)`dzNYM;4jcWTSL zeYi~X>RC@BN9ft88tn9>=_ajA9-Xl;#h@!rn@#6sE8Ti-qRr1A#LFCW}WVboTvFK1YOX_U*kjm;@>-tj^4y!77Ps-(tLN$)W0DIN}|)rm{| zncoDKV1i3P(+xOM%cD6X$Sfjg2eJHFG(3{8Fxi>-5_w{n+YOQWmq?#RwtJv3cl9pU`ek z$3DZ;lJo%g5Dd;SH<^`qJ=;!ifI(ObkDf;hY<+Cm-!IooZLe71`W45_=|LO|UZ8PO zMu0TOKt(NC-1rAS3Yk&MFxzX`CAS<3jw6ZPJ;I@G7G7SZWahJY9}<~C4eh{^5Y|TV zOEPesq@gp6%R=)e*r0(1dWfFPIC(60aW_qo4`09{1Mh6|bFm32KH(9-TC`l3Cg7&H zx4T(CfHbt@8;z^g+#jPXN@%*;w@CwqK116CSSQA$_;y$QJ~u_W$4ZjZ!W>w>z+Q1c zU)a#=GYeU@Cj)^ZVRR7YM~s9y%cwYGUyY@&2;%E{d8d~`>>4p6MbFKj>KXSwz;b{{ z;3=KV`}qkcA3Vmc_MQN)@?gOt6sp}M(Vf23PZd^uG^V$@+=l$yQA0ss41Xs-rvpB9 z9)!|)rjRJzJ#?*#Cf}n_C_SK%3|=yh;)N{O`CaHKT{UBQ1`xYB6Z&<}qx(zE$h1cY z?GW&Ct?I{&f5h6xBoH`RyngT|R7{Sbx)#;OFO<%4Nr%2Qh#!q}?>;{Do4tyPJIQ zSk5xh2&ZSg79Q93<1lXVew`FU6Gr;I5e2Y;-gmQKO(>ubb}t$75-H6PIi7uaxq$Y< zt%~?D|E<9@sz~8zTqYa;KoSQ$43tOi&y-RY5T&P((FF3sF4SQRep#8Jm9ytOa2!)3 zc1HiRk7W_p?r2w$Z&sb2$?Ynb*tX+G{;bGC)+2wZp^0If1PO>JPjgvennW}CE6Vze z$^#h!zq;*D-JG~WEM;ht4alLV`-PuX@Zvu>>3FxgX+*KX*jtf9U)0s4y3KhAcvmh; z3l{@;Bj7+SII(TDIC%OSx7VfupO)YTF;fRV&0XwdZIH?gF{?T=4(Y>tEIJ3lNVa?C zJP=CJtO&c!Bv*)po?iO}_0D+CjA%U8EBB{uv{zjx;E4CiFZ!UKI&M-xTd!=*%^tA$ z&>!L(dQxJ%9=KI>E?`~cO*VqF!2<1JZ@DXQRJY&7B(G^VreoGC>ojiQw*2nB6bzOT z28)dMY70OLXOHL`WfR)2RFm!>1?Zeex!LTu3TObhszvfzefWl4wZ8vmKcQ(!fqdy> zY4rV!2bPHYAi^yf4_f-Sf>RyUBDHzk5^4n({ zI#&Shcvz_CvyVu^_`w-F=n5UL8ZisyLwEEin!hFCD`JUu^niM)o@)*djV-qsov9cW zv-5c|qFeTf9Ylg;Ie_}<_|J+@St1J|RGhmFCr_0OPJbtb$+AGMNtHsMnL)OjM#fh* zhLjf#<7Nm`*49&mxgU*c9nh*9`W$KC$xk7NLIUnAkBD`;iGSjtQ8K2vuSC0Sn3Hxz6bfjB1Nqol~aRk~&H1qy2WNWw~)6Qir!wvcL>Uf&XEIWsWB(7dD@Q~T>;R5?ZRs>c@mdn0GV%3^6;tQ!>2RF^Rw zf#r^@>`Nl`{nf`P0;OTQ&s2MMrIn1(y$kZmqn7ZO0Fq@>*+1>xBUN7zeL+M6Ex1!2 zTeF((HRBk*o}VF_U)sD^1_Fy&B{J)+-+?y<7%7*9o4pQ5$AtWTe*fx$8}!cRCfYH| z!__<7Gdib16PLQA;dqLK)*(O4q8fGzwCc1`+BuJJ=p8i@L{l10$VJP>CLqs@$K8oV zmnujm>g}!+?TLzmCHzxdNdTZaH6_A0-w->U_aD@(;2KwtJl&xI#%a*G(N-5H=Tq;CmM3-At&870 zT9MyQnx?QkZAk6kt8Uq^!^Yazx^H(9DRxZ|sd;QBfOGeNX&DO=kr|2HB?dRu8zC=cC{;AcTPX*hnzZV;aZPcJm zbN&d#U3u8B9Iv<1I>^t0Cmlu*Roxw2Y}P!$454IJlgM+ON1d+gVk`<&<$wJvgSrRO z>OuR~2jsw*a=$^Mackl5%UohRnS9mx$2*BJoW=I=Zh5z6G`gi$J&WA9{e@Eu&TXuN z5f1_Xz)v5TAIPEIeJ&V^FsvkGqeNQSCvw*zKVi1cP1(A7h`tH%1)OwEj>w^8zNZw- zpRx}Cl^~km0}}h1XXn75G11x1tOhDrQxbyu-7f-UH_R})jBMm3YrQ~n!wTU=V)a4Ec>5dQC3?M z?D6cBMUX>9PG*MD$oKDMH33+auQJLh+x=$2g()$S_!p^_i3!}%20|3G94^Qrd!aGO zr$ZQL=%2UByK!(gd!1+M4RFV%ZR=P#8Wp-*&_3)d?SCxIMR4ln(Nda`@DJR;TVbGk zvseAIkrM>gb=Jny7k)yk1Sg?cp`#z4w>T~oq(7e&Y&zd4a#~mmXicgb(`90_1#uIA zlBG5;klXCRAFgMi`RSty9S%sgY}6chqT}G}+e0vMS$ae4PBxhA zzmd8vwZ_e3A^t{1a9+-IXy<$Red?~w%TuJU%AyH6rb%J6hd2)p+qGh}wkyByUKo{` z+)uXde)Hwqr~^oA`(^-Iu?-E5M_AjgED3VB^oS0Un%R4|U}+IjfvC3b0V3Zh{D+Z=;O(Gm?vXevK&)KpaYnfPMFRvZGBA}}IC={ng&81K zKkh-%q>s(GBa)Ak+3a2(0mVVtV1Pm=!uhAGZOF`pF!U91<6!^(2)VBB4aIXjjov`2!P4)?bRC zdZ5PD0QLp;V8Is$eL2cmL>8t(h~pQ%usH?iBGkjP$Dev_SA|K1NO-iD7VTo_hoah>4?(j;w=2 zt4J10 zXjkA_+#GKlgsVIsho(}2YDZgjS>Aikm!{I1qS%lEB0<=FDB7rzGnDQ;Bs=NIBof8; zHYY-mF~{ZH0fV4y+Bim$vpkQUuj7rx7>Vo{zN52eCryGeoG{Iv90mJ)zW*ch8umSw zn|6e5f!%d$BHQJF@92}^!s91GRU|Uqsd7O;PBi|3Y-&c{qZ(ShPR)gTl$#IEx=#VO z>vPy@M1_R*@ZB=ic?9hH{Rkhw5*q4_BF78LvVHx)Utx*o{#st;@i#QU4ADFrXR5pW zAaj}ARfdb3Fqe#1my4bL7Md3k343SZ_7x!4gt8PG)t#E0YcdVF^5+q}=)w5qrqT{p+ z>N;l}@fcs-C8(41m#^!fe5*dgyG$Y4)$#_n{hXWKmBr^%y_|BZjCE^=EurPeBcZRw z+$pW6V_HD~7`C(KvX80HX-73r7(B*VjIT9ECj2%lNYA4w#o3ApO|B2~kyxn|GiJXSw^PRngKZ;N|FGjQs)7|s{=@IYUzfUS4{|Ly2y z9$0vS1=_OZWwuXF(MjOg1Z4M$d5Xr&x+0o^ax z4KL8Ch=EXlWF)Kf7(ko{uZAhA?^6rD%dw#S^R6yXI#Z>k&mNTq_R8zK4I_eNlX^z+ z{^E(i3C{F|$IWlisc(RB(s53Mx`X|0(KBQSkUn2^~_} z{8=jen@oQ_BLOe)N3vlO434;83#HDVRq`cr=72(P%|WhgbfN(Tc_hWhMe@pmDH5j?Mc zE1WtG`1pcULWaMlujB&zblsVKP<0PN`8W-t@z?ctFTt|V1+#Dq0f0kmP$2$md$EAC z$oAH3Eq(>=yY~}{{>$h7Gf?I|cp2(>eqNa@Dvrh0!;QboP@C}WkzPbF5<;?bYE#<( zC^^85;6ILxl5zxIjuN4d{!RY>quE!-@Y1cQPtry_C_~kH6_ETTojiDC6Ws4(z7NUf zx=7RfYU*t%~pL39rc6YiTG|EH;d6(m0f&+EJs zswY7T#Uw-ECi|6${A0o5VPM$1sAR;F;5H8Z&WC?pE`k@9N7Jp!JgK1SbI+3$^7f^G z!2h*~0dYi7PASsc0d+D-S?Mto8odiG~N5Rv?~35h#F1h4o?F6AboyGuzi^EJXcINZ0!Uieah3S9lAiga?c>@b|0Onb zRP(w0i`}L=4|$esB)EB2z|Aviy4!(6f-&d%K9!@u-tU4SToq8dTWoS2+0%M%7be8l z+J!414MBBiVg+Z{kt4X9%emyW4%VI&u3{{4l=5F&eIob0dfwNGP1EEDsjeKbv&t61 zDqU2ZMK96H2$)7-D=H~nigo#36o0^9qD4gVy*@KLan#5Ttd3Qw>c*z|2s3|$Y5SUy z5*oJKHXtY=5jggA{cD3%1jo6@-?1{zYR=I$i#doh2hktkXJ)`>EsUQp7V zp%qc&`j@|Xr9}0!4c{;kdXSZjen=~nLw zW&;CL<)eZ+x7z1N{W;1SJvdsVqq85zwUI&#`oZ0HTThJsy{P~8J50BM!Ll(y7sA3T z=-!`$ycC}WZgqr|fp82SuB31w+37iN>i;cDZeS-+Nj8<&{$LnNZd4rP*Qk8|z6Fm4 zxVLQ5afVT8ScoTxy=v0J+kc8yd=(6`mfC3B8BYn7nhho&jBZJg`?nMQPke9rPy&1< z_R%eWsqx6imcMB&_>3b7V3j@|(RyiM*;3-z)EmMdJN_t#f&;n+uZSAVA**K+z=ezH zFIA@6Eb?!;JcP%R7o{#LY1q`+ebO-Kw@flV+tjoDZnj=!u`6TR3~$Hb92qPwKt$!{ zIF2Cxd7fC{5Ti}7c(@R5QT+Z=M$(O+BM7R}H^GE4VRsJ5MT zM1REbBCrN9OufW*_0K^K0u9t-E)qcoN<)W9_HRM%1Ivj07OHIk;t3^UB)RI%Eq>Q6 z?E}CqrF;!Q5qbqN4EDP|%TEFTx3{>W&r(y*hNohub~70?IZ)}Hf*jL2f`0+^;B+xE z1(fVs^g#nco>uA8$%|f5RUf~LZ9c7CG}^>33c`D>y(Ngtf3zToXTrU;Mh?|;F0cxz zSRh@Ysm7aYFL)C`l00ioYbbW|h}C)!?U8Og#T=+^=S`3EexlBj@E+hZKzW$#{4 zR51mpQ4EfRtCwEng!6JTm3+=Vz}P!m%}4=kWfJ~V_pb_q>+ehf^U0f^?pc_!mahm? zA%)XwT+VAM6@WKO2CUFDyl{FH@L1_ji#I0+6W(d2*_?t~&|Pe5OEP)U*QJf;U0apI zN-J5fA2}*9&?o>dg>^?C52xvJ|E&k09nxG{0V})_Z>;Y!s$tt4N&bVu@n8164bFz8 z85Hv#0HWAmAHxdz$#G@u6v=-)Z#Wv`0!z+H0>rozx{C1G8-9TL)7oD3!3!L?_|)(Y zgM+P#{$w!2R2UIYmq4?=S(-}^P;J~=|BnyVbDUVBree(}Z;>2lkcL@5>zk0y+_JcfHo1CQbWpf;Onc2&T>?ec8tJofNfpbj;FB^3q(V z*oA)ch9J(s1cBDyZ{jdM03ag7oE#dMlYQ>QGR}X{!_~|2jbzbvn13xZpeK|r-MrRq zH$7qfEnA~`pU8>+s@993hnn{C!V-`dgOBlf)RY8rLDk%?2ZY_Vcz*@{AAw4!6E@Ye zJ-`hw3^5uVWq6CgJ5+3Cu+EBA42+{F<{x6Pe$EsC#`NQlL213-}o0c#!s2 z2TI|Qp0~U=sS%M8{_PrHaRUp08kMT#GlN5ditI1t3LSB#Y{K?sXa!`L88t#+H?^F} zBXk_lB9wfv`Ta`y5^yu{$|MJH5ovl|4uH!fM#Dx2>}l2+P};1=?p(v{K03=ZOq}AD zst z{16AnnBoTuP8S2k1L`qtqkB$g%LzjA7`bt!DBCk-U+)m}zeU@gqgag=^#h0jvL{3O$d}o@!~|#RN2MDwiRpkzeuQNyTo2gFl~*W_ z?1i42%PZ+Ht5ZQc93}Kz3l(&v&8?J{a}C%lyMsU7jq^iA3U$Z&N%-I32Q!X`#SGkQdHS>Um^tVIeFFwN*?E}O_UO|Mzy9@dc$ zcxNLNBOTZJ3e;Ucn>>8Mh>`%zI(^!K>XBh{kn0ba9g0By$bN(+QYfG~+I383occvf z+X>B_F?p(7^8MpsGtR?`JH8djq3&L1ziWw%6q)lHM#YiEs$>zoEpzU}-j}kK(-Y=CAEgn8!>WH#kZv!18i}9}dpsmWZ(5o5w^CmU z@}do}SsSjKFmj4P$>{*b*hR?&06w`TO>PvSi zCKr~l@I+h){h-uOmZuN=rQW(dtJ5NbP;UHC!lSNKSe!$7rz>cat0HfXmvJc}=v#xq(k0~;+Mzx_Geu0;o za2CEVTC$I|s^Yclr90rEdov*cJ^w^CJ9C|HQ8JI#`?|5Im$n??aaCD(FA$el!Lf+QcTyh$*ni3PBy% z=kE*MY5gI(EL6?UZ$Mi}3W(&*si3f7E2PBI4f~#%O9sdOQ6c19t?<^ye}bzFW^So8 zOhpv9+YbVJt+>6QG+>=q-j@NI8lP7xt|f99g^}14Dg=QJy#aVMF|A$CW9pWW@P88@ zV?O!%(x^vpvsgO*5snzO!u?$(P3HxB2K!Sxqi6Y5(&+O z@yB=2EMVWw6+Ar)p?=aqBpHw6=*UnM!q#*-WCJNQA)bn^B*8#i76I5K;d;_6g9r2% z)v+eQ``?&r>C;O{?_U3})mwZRP2cHCu`eG<6Gu^I-VvD+TB)Dlq8S7{(e;ixWMNEb zmv^`RqVavci_|GnryL|6xm=w<#IsV5kz-unNq{K;#OVo}Zo>8iSSAgbTs|o$HKTi( z#GbQ3b8v}8&FlT2kds)=WvI8dc~;O&IXW7-Zey0{*co+Sb9pOe|@?^mw%U;_~iQ_?m-yMnk7L`Zh{BdcV@|59|~DDVt%$GF#S zsfUvR&~SX9p2`QvCMQ4Hu*)i2SO3^OGJc6nRk*mz;Y$qnZ12GTH7mD2SmX1;pkiuc z(m@L$n`acN?y2^4$l&&DC3w>-^y?z@;qQ|?2`-@LM61c$(PSMcIh(GplSAEcNYnsz zOSypu-<7&(raJns@gD%5Sry@Kf)hw6W&i`T901zJVMpPC( zdp9_*rYy3IK3>yYSX8nR0Sa&qtJf8E6OV5sVQZ8>Su~md@EZ>B*_6DFg{m6*g%836 z0J86emvG;sn)3nbnhp_kL^KL)6y*HVb2-W&Q@$>Ti1SEEaGjb?I#w?dkW*h%-4{Za zrAxhpkwR;!Y`Q^CGF7fOoe$dWRSiLeRff4=pRHC{-MCI!(1D^s|FOqUqLJtnyT-|S z&}q!-b_I<_F==*31RPESydu{NdOj^0&o`L3t`3JIn>uyAVY`DS*RUHtcyW}w!Sjgm z?N_r5A5+$*OGkVyJ2@>qo2v35?Dq|*0Oj$S#_O1JoL|`k-i0`vLc$8CKsp3fZz?Vu z5M&d$srUeQsD|7fSA%@HAxMG0T2t5$no+lh{6I(N4!oUZJl3!RmkK%}0|OhQ>jBF$ za5?D(KG`%kPSbI{A7lbcyu^D7dZSCaiaC{3IM_S4S;Z#A%=7IcxD zmajSh9U8n=#bv=~pm{DeD3DAqcPMwc(U%EldL)$UdhIT~RJUj|^A8;Bt8 zqJ)&wI7G)JPD8g=Bu(4Hr+Q3YeZ(%)lx-0d3`8w?`lF2Gn~H)-^v(|!7^XK_{iVE) z%%-dE0JD376nc(|!&Lmy{|wa0433=2;Ym3PnfE}h46J|zWVoDM4TEFUPhU)eoXSWE zbk;6~Ff^QllsDy(h-adq?^)Xw%oE^NHQnK{LOg$|gNw~KOeO4B=SN&P88VTNY-kgo z`LAeY_tD6~hKTnG%~A*N67%-JTbo^2Uu_XR&P;z1SZY-97!3(cS@fj9fCp6e7iQkx zT)f-8R>A?!bMaDNLxUu_nQJd5vG^uakcBc!?MhYajzWtZ5e81gKhn$m{j zV$uUmKegrp&-U$bcYc1<1!U2DmR?#xnh&WDh0TSTQUT*<3R=9k7y}8+7c1losiJ_ro10X!Fm5%fiz|O{} zb%dsmeW1x34R|Dc=%mKImp_YDGH-YApkF8%ox-~)* zRhj~bi>BzM_Yq9k5~9@HUo|o=k4pU%U3}iBIxN^xsAiF5+{d9s&rNtqp_)y23qqOX z$N`aH5bTvhgiv~<+VjBM&$1)_afF>zI%+)ksrUCjRP+hjUaf&ztI1P9@m!)Ze0~x^ z9_4R2mWz&xQ)OhW0g>?4qvp1fLEm;AR{8;Oo@2L)ahtk=wsk&lwWHpfP&7btcMCpn zRkR z^ie^nVf+e`8rR)95h4V%n;Vb{hF_b4R?on?-Wuv|1c6sfZhRpx{98e#dEH%+vy)mw z1EfyA_I5XUsg7NRLt4oNkMM8+kS&V}FxW110Db+&qy+J537}J8tjaCfB9DR2Q;Fov zlYq-KtiqVy6lV*U2S9L6hHWI3E{n4E#0AY^O&>SFjjxv6WhrHNXo*dIIMX#$n9Lz; zJHvk>{JQ@M6*Ki=TGk%#sJdF5gzC|iE>fs2!A8;mf70_{LdKH1c%B~6oFP3g3D8qG zz5|Ew7{6mrlUU(!z7beh`;w@;x zpn-+(s^M@*p6BZ9&6Hmt*SS^IGe2?DEvye=SA(%-V)|UiD4y-t1eR5%>iY9BNuw{Rr52HfNAkUtodA*;D8ZmF`K97}rE-e}SfPqaMt zu8_mY;&Q|`KlU2-32g`*>}`E|<#FNy@kj$LcS)dYi19=KA^@a@_{hSA3Syf89a0)9t`uLiYbeMZJnlCmv?f? zGnYxVD}9+ckxzUzJ4i9D8SZfPzPIrM2-?a7B1S$kpS{M&r2AZaex9@{FMewb>YJ!K zXBAs6612Fi2S9DI60n<)l`prkdM2uN3BxhkhPA74+rrYgO&4j>v>h0Y67Cuoo1KUG zm3$!hk-iP@cG~rLpf%4r+EzU-c&q1W69@{?Z@MN(SvY+zj(aC$-^=Gaa+_{Ve+V-` zj8CMkFW8NOcbr`?d*EpnJEnRYaL;w$8yljXz5}!@Ev!!F!*{l745E487APg{%tx~u zUQK78zkPiUz0#oH0&R^tqGOl*Ube`utj{bX%`N~5LuaaiAz9Z$Spld)lM$`-xQ9WU zuH!jjX<;<+3C?@)cE1jg2BdtCeIc))Rem5_}9z>u$~=H!O-Yq{H1jvToULAICIPJKhrCxbZPy z`LR6W>#2LX+IvZ=RVv8GM0TYqdtM<=o}v*n5}P9setk7aBI87QCCz-i8(I4Kn1@z8 z^9OKvl&zlh*wjQqGncKoL=cG>{oI3_j&G64>XSUD9`^t^V;^>@z{L6q_(rPpUJnTcEX#ztZ7r%f|fV*nLki{ZS@{e z5^G#szcfQrIOq5-$?-|9qLVh|v|xwcL2{f1u{EH{tXj1P-YtzxSm={nscV*H8@;-T zCuW&CnI^l?_PIYfQ00S%4P6QpTS3c#g88zs~-DS`3H8vPGRa|1{D!Cx53bhLA`wY*54`eR@@rILq^ljD*C(rWxd9>&af29$; z7)O85jKOBi3Vdal%TB`8Eel1H1EL?uXg2{6F?t$;we6BwUshtn7$Pg)-TfirKJapx zFuVX_cBp9(kL?+=elQ><_z1kx(D8rly=7dLTlY4q2udiiND*NHi*D(VSRgFAq(KCf z4(SF#YLScXPDv%D6)B}#KtNGKQo7;HwV&3rF`_rB-4#~gEv8Dn18 zbq9V-&TrTUdtWzMx)9gy)pD`(&aoaAMg(*8Jo_3)1;2rd1Mh@A&b6ju!GWEIOHd^g z3U3AlGYflA#*buFne2RMW0!m8#^FKrsEAAo-jeOL#&X2;;p5bp@z}jyr&mT>alU=8 z1qZr1(4$y!oukkJ2S${Pb#W<@i@hcuH%)zFGYT>906K7}i-W({U3(&Js=*g@OxMYj_R=cVh z9Z;2AF(_EkReDgaRM)CepU#2`ysuV`+(AskJOS(R8uL2ikO3JJRMKUk8dO@^`8NV(@^KpE zF2PxOB*tzQ7wm~`z+5BI3I54v;th(}d#o&bhV4G|R(WKg+Ff3)ut*@Ir+th}`8*;6 zrRMuSg4uUviEL*m3x(1WdvS$ZroIPPGe0JJ5iftoV@?#NKxy~9TT`QNwIUBU-%xir z+lLZ!B>qJ__&Rk_y&irz)3anQADbCITHSc=oIgG8+oi^h^l}(!#uH z(E&5i$e~DL9L*TIlOs5%U!N;PS*CxJn4nFyVj=RlVq?V#3rBf=QOa)oo`JFIeTD~J zuac-eVpWMH^`e4J0GNDW*~I+@jzgn!7eOCR1KuUVY&G73U4N{YxhS>3l-0bZ83Coe&PENxUlsk-z~65?dGGQ}4I}oH zi5j4-`#T6ZAF=X%s7lP0xo7OI-2X!=-ib@a%7+{;ZJ3QqZ;}L8e=ASOd3MQ4zV8D} z=IpfdQHK6(sl}7LxZPxr1itysX}r+tLuL|~->DZ_4Ix`XTk4dea*?lSTyzDiY?uQF z;NvXPB9%RWXhJG|T6H+MI%Jj&^)^DY{?{-rpwB+ZnuX?HU4If4EHk=$)7apIZOXSrXJKphp-26=1X4se7>xLfl zs&ZkBj?!PpZQ>e~|F!HVepDUczG#Q$!bB`_eJ`v*kRvcisoD2F-U*Xh`o8%>&zMsR zTuM>br)5}GqX?8&Vh_k5by#Qo@tBzYnG;FW|Hp zVIiB3&jEyQ_~gqQ16MMIEF9l<7?KQ-p!_h(PS+Mn3Ub`Ma^=i-O;B1Whcb3H++ee4 zVa9u0B)4(9j;21Yz&O=TghUj$4-Vd(&4)H*H~{4)PS({0I*#;_i-J#nfQ(#myHzVH z=nX7DGAcGWvb&buJ6!L^s;IGDEUApfuoI5Nv`(YEl+v@;MZ1fA;?{O$GvCkPNshsR z#u{{Z^b<@h832ww-4f2hC9g0`I@PhwHIy^{Hn9tvhjXG0e)Pjsrg4LEu1Xidm-fQ@ zXAE?N`r=D{1-L?NB9qG7k!}IAqY4KemIGaxpy3V;@(Bt>fP~W;3TEuPi7An*xWMs3 z02jhfFKo@QSH5^HPd@kR!($~>SPk~<_koxn4h|hi?a_3%?9iEYrn?VT@TEbvpfum^ z4y3{m8 z1|#en+dBgN!xe2l zvV`D(5LVj>_hf1%I#^nzP^m2gC+iT^=dU+Sm~~KUltV@i0@7PnM6g<_uA{`sJaXLm z#v(?gHRFm$s;77hK2N}MGrhq_G%3@a%uwUfl^7G|)P*$1hE%8QxJF*R8x-iy>UBWI z=xo@%Ww-m84Oe`L|8QUJsN2|~vhFNUHTzkrh60L^x7|hF{9NAmdlguo+@`DH{>*-I zVehs>-eih6gg|M_*oNq9`P4+0uHZ+DeFC-4s7ggkq`~Ww`9dK2!JUvDGR#tWs&y&B zdZ5w{_>WEXt7mz-D6N#0f+d07S9I-+g>F8t>ngnbd>b9*db-vORY3ex2 z8oHan!R}%yQ0|0(XXha~P8Q@20|uiW!}yFgj(A(S#-Ib9w%9vbbMbciXd$|H@t0$! z#G7tJ^y7E{ZrVk0;zt0A$3}u%3}T2d#OZpIzN^tE@E>}z;`-1qT`4$<+3d|Y1q1NZ z)lNH5yGXFj?Qgn>xuXo2R$3_3n9_EJGuT4!Pcu@F(IJAwxjK;Y`k=>3`@UKm%Iyn9 zst(Ve|56(?j^mY$x_&~7GRAkn*y@{d1iR*;=|kR)uiEI_C$6u%nr%1u)JLNEtCNNc z#@gp}(#wF4Ip7DG>MH)s(Z)_YpeSX2W@K|25sZ#j>YX1?zq~OKo`|hM_z!W-Vpsi& zdZNM1fEfSG&8}(sn%$_j@J&Pcu4|H(-rFTAO7d{F^{U};)_w`UN}2)vqP+V)ErpR> zs?jP3=?u5)>a{S^hm04j>OzJw5P8aF!38+(qDZ2(m;!-Q9+Zm@sA*Qu>00S`Ik>!s z)1|&?lZfxcHj40%fvpTb(E+8BGKtwreo!h&rbF=gSr!A|JPPy+T8*memU>Glax=lR zouCdE@>$M+xRiQkCs!8{0euUKUCpNo2pr`R;5#&Xcp4d^)Q*SMo*v35dT@K@uGDOS zHJ;QJz0M_O9J~z7n%>QYHm)f>MfCn}}!g<&jSZat)W z$j8LB>wTd3zak@k6^X#aalZl?c)E2sDLxV1@Ww8;Dit26K_L;f^1YyWt`mDF>HW9% z_G%)Z=QOh{A}| zjC<~1T4Ch+09tnixBQ@IF2lX>pAA{#(a^4-SdeTH9Q7$SRcrS`hNs{>D=zcyuNbrC zKj2|a*q_}M(Md240*i{*18_MAA1Q6C#EDPZIp$%E-}3k!U!m_l&AdEbO)U;PA0${y zhv#YolKJV|LJo;CocY)SA3hR1K+f;p|1Mbk=$$HMF)x63|r)WK!;ZexK7Cg_W z;yE9EE0~r@=*5~9lI@PgEB06WYUj)6+S^`iTk9_XL?gJtX%Nl( zAg@vnl(9VJtT}D%diauqJd9ob{{#L zH#ACKrvkVR^j;Mu9?o+rk+OY79$d1N7iMT8(q!%zw=5@-)gEhxrJ$itmCajeYOhw~ z7#ICml9He>EXM<&X3^$bv$OQBMhz^&CBeB#LMM*CK$eTFRF;v%@`s=PGk)6oLiVEgvqni{jq@=pyb*AB_$9^!UKL48m%6WIcK!-YoLY&g5@_+)AbA1TAL_&SBeh)@zdOFl0l&#w|fkJEj81! zta6lG?m`xriGx_$f@XbbmdRtV_ z$d?()Gj8`om+DO8j%ac+1<*=5ogOn98vTpna6bD3s!}RF=H7h?>RP;ThnPiDV$MY? z05!4#tIDYd36t(;nn;IrFvDh%_ZXx2gJOz*fQnCAtaLA&wlpTskooqpf+ zf82+uE6Qf6u>
      o?ikQu|vi@;9) z?)JiOy}0ZU^i`DO!BAW_xAzuU@hLyG z>fe{{c`Nf%&8perTW24xzTEyX*};8281EXIL}#h=nH4b!%hjBHhhV%_T$o?^AgE1y+`YPT^r$!s3n`pyna|<-BR8p@l?!Q>_4@N@P1*JXVXDcaS3x{;$^4pz z*Y;dTC^A4D-mpdEQqhMp+DInwfltQ18Pf{UJ>I?C#dTuVh!f zm9L`L&PubBXlLvq`>%(!3v`G;c-%(;+Fm49b;F<$wKICu5gI8Uu8@`?x212TN-d`fA73;0AJuTcO^5=TkP_i5Jt{UMgVyqY%aSiGA+>kkjM01%;*Q zGf;w#;Syh2T__Lz^`r0kvqp#t#P_R5HUg&p)dS`67SN!dtuKz9$z6wOPPYJ z+IVlHrI`u?aACQAXe2jI1r9I({?kox zREr0p<3@vcsXm`B z&mF%nyeETzM4A80s+=MR$b6@Ls!#Da2!0<-sGE+>*i>cGDLy$tFoZ zq0q<@nZg%8CDK*-4Ud)?c`=m+x2oe~XE{@PK_e>h;HZcz3CZJ`8$4RrZG(M`ml)G&eW$^H*%@+9rGJ=cq zrp@*QOM}vPCE>K8qxk!m156n7xUKTwi!Bd)A81(k-L8Y*DJ6w9y>>kO0F}(i#iaH- zHc4*hH?DY1)jT*}*dWpWbwowR`g7GZ^5EUCAzb3Eq03(W!_d075;AEYo0+rU9{1i#+LpUjgjE&dGQ^3R&Y5q zsmrz7t%CiP==F)DXn%M0A_q%6|4dqQ` zR}~)5%C~XNnB=-u+E`gNf0%cM5q0CL+i!gNofMNqrs#e!m~L7mivEr+6^Q!~luhwX_!wG<7sqPy!3n0{t;IklN-WsyGk zdK7Y?_oL03^);7cbJk0vCzcydIi;+o988g;iYma4J4=CxiI;p3w6xWfy5KhwaDm=4fhFbOjARROQ#dWq@ z-}COeMc(sSb*rw5=fQ8aA8&c^i`%}mZhsp9A?K10B53BsDO_RweuFfaRwl$_281gae)R)(MnqBJ;JBICbv-#i049@|Y%GdR#Llh5(e0^~^><^U znIg<&L%dX59V?CzJ<6Wz;yeXjAP;bOYYHS(%Rh=#H2J|l#@%6uiP&uvSG2lu)O(i| zJ*S>j&q$YMs=U{Cedh zuWJ!w4zLtQ$!l)5I#TeLKZ!fH)D|b-_X&phJR^h-lVXsLWDb0}Phkib?l$y2m>2*m4L{hKKlfEs>u$0m!h+^-K7Og7 z&IFa`g2(H&wGT==LnJw`<3P|H@58)K9N}tPcc#zNHyb`7GmxM4Dm%l_EREj0sA~b{ zXnG|&i~)m>|E$8yItuL1`2zvuh?2%j*fe426<{yavBZ-Kk0+4(V(FRI=2u}gbL2Jx zO|^md0XN3JERDaufqf2y1-tDDR^3psAGJ0q;g-r6;Fhmo+W+bz(?4y}_IjPYD@ccm z`BwvI4wuGeXm872h-%<^o~dsppwmCLn6t(HKCH^-slh z|6l+`Ks0M_e^?%;k;)U!U@25Ur<)KO7RE|L8}~2xkP}TXW%y!HsfLM6TCW}~=(TIb@drX)f!4*F@O~qFAcwNVw@`SEW)WlJqal2s<-(r*>4s#E zBG_BI;2b8g0g2wFrt%urNWg9qM8DVK;Ylso-(b1`Z8VW}?ctYcR<0cu!!Y*Wi)uXHEdw$;s&@Ea8;tjGiwctHf%*T#9E0ta*_OSn zGxH1A$p?yDjWYj>MSoD*?C;+D6m6TQ*IKfbxS=Uhe6Jone++~m=-^WTv}#k;Znfq6 zjh43u=(`!Yf2~||l!5=_k81(TMDLbQ)BAg}eXIra{f+jLyU6XHd`K8~0*=zJ>=P;L z05v0j_flx?B%qe+%DYU~sM`D?NrOP17YSUVteEy6s00O;8l>ivN9f2k%w$J+u;IiW z8)PRATvfb-KQasJ{by7GjhB|&^yjVRA_IH(V3YeJRLd1lfWR3Du=FW3%KRHZnah03 z{ScEzZtOSl;k^bOCGq*c zMG8&=SBHx;B9`u&QC<{#`V;Y-OqY?t-P0VP1D zFrU=(PC4;Dk`1$@KCNfPk;HjymJU|5<^1tYzOPg{4{R&>R zbR%)?pTzzP14|0xvV`I;4qAGPE-wl?(yuP3_g|v`UjIJ>CKRfEKuAaXCrkf&6r};8 zoMn@uloY)P2tg6QW&S`@|HFdZM6aN8UbO==z*JX|_vv?R({B6^o;(F6&P}OO9B@pY zdj*YMGf`OjXU0HNiUa%z??Mhc88Q0{Ao!A3>Eu-ZRof0I3AQe=&V99!pmM3mGy9`{uuKGm`-1=E9r+VA_H`+z(B5IX&>Mf)Jvd#}+wTE_; z%l^w#Ytr9>;%}w^THuV~`4Jw3mkQ8BJ@?=C42%tlTg>~N+3^~0&zq+fCJPuG%yqal zEJ}*`obNC#7ig;RZ?~PSG=l6bRsmcQL+&qK+wnj%Y781@|(-zaGoQV|0iCh40=d|rAIR^*;Dj?ur zeo!I)C#|kwmeO&L>T_+>TS)LuWjLp;S4IWB^pL$VB;Esxc(01&V)Xhu@Hl`{$B{2c zi|jN25ANBQIOiU#FlexiJ6fQzKNS83<=sloMgw&11PuLsd8_KwjxG0lmJ@Z1`C}w@ z5^AjoiSSbG*uacUg;lh0ssDNXv|DIqKOG)d zf(I9u6W$ZBTlt`}Ucic`6oGV?&(ErdgUopOAKrcf9n|hsC-dIh zf@m?SRNLK8%0pRuDR(9frj+<69-E9N34={QYUNsgI3QtPL_YA1e$r)IGl!4G0h`o!kmfxruK`8)wg+4_74>r;d;rjQIrnUvKYkYHQocCD9Ehg! zIG?^QPD#{0w|R1|L5_QZVtz*JxNt#`Rx>Evqi;oJUBNfm_vy0fo;{!7yh}~vRhH-g z;GCF-v|cs||NMQq++f@@y~N_2YV3bDFk#fEHYmJ)yWtrF(<`!LF#lp?j(^gU0o)R> z&5F;)n8E*d+Ea^s&!vshwV690#zJznpMJJ&Thd;3D66zq5&QnG#TNhJIifkfThV+t zYARiLqmsP&Xtw7hl)~lJg;>vq=Z>Ng7$w`frKCw*(-s0Gh>w|Govw#Ra~1&Nrm}am zH`G_-^u=zdwZFWk-&E!9i##?rCL4~OPZrtf>y?e8B7+$NM)$1O{Zi~cE~ndDKG=5zhRG?a%7N076)!%~Y zj!>7_VJEShdo4?Gq5#DdSZTFpZUY@vdhQvQFL*ZU-xYq@vrR78xNK@0xX_ALK--C(u zAzf~3-r`1+i`lfvh_~CX+GgjZ2L+PrXJ4uaZq}M4>TdyD**|+$cTpUOv0jbvziYoG zKZbA2g;4S5hO7s2gBIcms!sv4kX^#*E>8pwgsXkGbdKl;cg;x1y_Apm!a`gP(i={p z$s5(7>E#n2GMxKO8VI5oFP7M+zj!vx4IrAAECIZNmIl(UCqa02-y;kV#II5UinGlx zn>uzYqt9qO-lSSCBAuy>KIbl_x}A+CgN5we%Ke7=>RfY!UfmF@+ox{E!PKJWFK|FA zvCJJ@>nsHs*$OfenV&82hh~Rj1nc9$dXzaX#IHCUl$%EEPVPDcV`A$uKg-m=95P+} zk{|QHrlL_+%&gJhz4zS{WrR!kO&dFj@8zyzlhmM&(Io!vbUQ1?l$>TDqc>Zl zNcVUay;ZlZK1nifo8L=i<81Ba|CSp{!&6KkHr5Yt!pm!HoNQl8Ud)c@g9C!9=-lnQ zyCtp)jqNoq9^XEMW-xSSHq2R@rKF8nX?8#$2 zy$@J*SNP`t7Z&r2KUAe_m`y=oyO{ixt85rtQ9ck_&u|{_J(75OIO|R`<{mLn<2f(1 zYVO>XsFT`yx@lEE=f&o31lE!H?Q45L@4sh%&Izw(=*h)yIlip#ajx+19O?cPhjv#h z06hm&S%~m%iCMqUdc$J(dZlZ#1(k_}ze87CALpfaedV^^0#UPpO>~rC{h%Nas+xNR7Zw zrffg!!#cW;F_W>+76t_QBRK0zJU1syma`k?t^ZnF=7lYmHcEJWIZ{Ng&5VZ@7;;i*~Q@=h7a_5uIRj=bh;b)QIR znzXUc2$Hqc>yU5C(DhL?7NY(QNjBK*fscHo@nAznm?cr)to~jx!T!+*d_pGA=7gNB z6_74!>1Ct&aWfE^gjK7 zMQC%&;)Kb^{*fLkog`>%Jn{l;Gl+$l(jbu2!?_N2+x5KP$Hb~>#J4hRZ_atNrkWj2 z&+%Wi$I?VTV9ItXwTWmi%I}Q+5&dg>7q6SwCyX+0;vCK4VqPlDYucOO`Oq-q4R2dk5$b3n+f#i;9FQB9AJF z2ZzKj`gNVL8#?pdw>=F8A9V1-QNLkAMd(BMI0uknpvwA(w-$hpX2MHlDr=^r_x)BS zEG3j+J|F^oKWkY6tABQ@|EQ5ky$7d||NqziKl9sXxuM@cj)Pu0sQAo z-Ww0MTc(G$=&t@PC1HVD0m-$G&#}iP!9d>hfZ2(GgHsq!pv;NcrU@HdY(2ULJ;0lzER=BgL6Iu#8!ZAxkoPVHN@Tj=cM(UQ{El)6PdyKl z+y3u`P6SQG4s{h8u)_sEQ1rA2WsQF0Pr!4MQ)|QsJmScua^cLCqWy$?E~bjSx1_KC zPU+;Ik89bXCl0&jq!3qH(3~?!b)*p8JQ6hT`TRkSI<-}|H|_gkAD26wjDYD24V5%~ zcFw>&q2T{mEtzu%_^VcmX*6gY9J~=m$!fVMV35oWDLN|B?oN)ogZG5E1p_ZgV&cg^ zJYj|j1;J%$ztgcuRQ@Rf47Y(Z^Gt(-mEIKig;kNm1jC!a6l@5%wtmX;{5}ymJ&bif z%l!$HWal;mN8j)$J`y4h!=b*Umisfci=c(+EcSUTY&k*G<=vv%zc!BV31Qs?8BU#n zYAy(H@3ft4#t;kBkE|lH_;c8B!2r*ztUjOl@*jb170^I z?6Wz-a7qAP3|@?^BK@avE&8iWZrdg^Q|J%KGzE%0M=P7s!GobZe!b&;?a~irB6jNj z5flq>L)SvK9x9zR1p;$*)QbD7(Y`K0TbZf`zD~pPNLZU}Nmu><@klc z74;NU|12drzUc8(H6D%I2jPcMHCBtqL{FR~pw%mS5>f>m+#i~EHIdJQM&CZ8FX;Ja zSB?#d1QJu(;7u2)ac1|?9pfef9V@lQLJrU5Qqj_raDRDjCgwl*ccx`30P{4xv)lyH z6FB{^IQIeIMY|Fd_Z0TykJCD+0pf)=A= zA~kX~BEmIJ9RC#N0w#e#-pI^_+~yd% z;FNg*ueg*0f58BB$TI|=K09^={o`HF8@HCVU;ZOP=RyUBI(prS@+;aK$}g3OCeYQ? zFSLlC(Yb2e%ofwK6zv%d_j13ftDJwG`A7FIzzxmqk8V{+Z$f`5;>6jBKj_c z>ZM^!JdP`oL1F(WeOykzJMH|p!rm|v^h)IIl$CQn@FHIt_sJBRyxtKCjg^Rm8$+55 z`r~Lx?tYHaVI4ZBbo`^s7vP5O*I=vSY!kYurxRQJwHl12ObVc_hbl7jG0XD2r4~HWdE2>sdscSmR344 zTKsX`eDU``0-2KDzz>$&mxZoIXn94Ocbe9z=wVa?xX4>dam=|S5HgaE3h~z8P$KvT ze>a+qD?)lPoPXpHg22OY$_(ioaqyS5kFkc7e~EWRK-Ilol0q01iX%-e8=~u`498ag zOT^F-*y%{;6C(M4R*qu!M`_V3Jtb8aH#UX;r>5|~pORT=6!gQ<8$8}c|2A4MwGU%p zM1*~AjQd-%Z1^>LAnI1E-09H+5y$B{rG)lN7*u^vjqb*u$p@;2F6E{Dj-|jrj~wLf z3my*eXI>V_pT4jH>VM`MR`8h0rsvEY0+^5fyaqoyh6w2p<6;TD{}w>Tfd}#!ePAJy zbfAF@yP*<#d`g%=s6U{4{g3wM|2}{C2^b#y$>(br{9sls!J$FYqR?M4B*G&f`%Az8 z+;!AOBG#_|T?&va1}o$0cvkUT6gpxNO+Ss$nxxp@uWH=4n;Z?@sb1ca3MVlg{fsIV zB1;^M4r&Ryy-xc~E#x!vp!64Q8=0U3LV-mA0z$-#rm%X&TW&ULg+!S7uk>!mn19O5 z?91xwF;DJ$*s$%72$L9~!i6z9Fx)4EO1`{y9RrgQ2OGw?W2N*|O%lq}yV&&@6BCjP z!9YHCEAagLnL1o-m~};qD;EnCrK_Q{5RZik2~dObc(VQa>nE@Yq1^JIHB!|K;}r%5 zQi_bLhz^RvP9#JWRMQWaWP5`=;LWFUVF{BN{5 z0rH@=ybmQY1qmVIcC`l&>Gc4ak|0UC#QRp%|9M8T0z9J6<}jxSj7l1xIC=&ifs!DU z<`qQ1PrRSUpz%AcK3TntTHdP2Z#^4ET?;q|_MS`)#kOvLV1F43FP_&PYq9<8Si^%Oe-*p9Uv7u?cmbNJ8JXyd_!#ZyMLxE7xX6VJ|@t&+6=yldc6| zV4j-DG-Qw$yEhk8;}DhrkE@%EaCkZbhb(s9OhNn13j@j0j4%KWizKGNrJte3g#4gt zO_#I+-5zDNe#nB24bvzI6t*QLL|{?|bWU9xIs%eqUtWexeWU z)`N*N8So3&fLp&J(ukpw-*5wRxfiDw>;>IbvK;i=jt1+NBVrsJ#E*im1vG(|xK7Mk zPZ1F!%754D8oPagZY$EY% zpbjBu_R==&0yO&onq5(k#(`$ZIEG}hwVRlb@Bv zaE1wxWD%CA01q2|;_$!%9>(nr`C=>yCECCQICxqP91ygyH;NP^b?e#mw5Zj`VvrJ5 z@VN0ng@-42e79RR1quel0|SYZj?e@HBJ-cnT5i@!*O_fE8lisLxS4C|FhO6KL68mw7^ z2PaDI)W*D&q$|OJ9Zo--0m<-9IpA`Wd{Za;^59*g_xkF9;NcVG4W!Y`eww$!OG#SL z2##xYS003N9Bb&OPVnzgI#if@$Pl_?Prm*9q_?-Y;WPGPY7e?t#_DF*aJQEj12Y^r zn?zkU0-+=cjn$P>TFC=%^}vR4ZsBDUK_#D-OH#2$KE%uOe;zDpTHba<%q$IjB747r zjspaQ{sDJ70=Q5T6Ef2Mn+2q`Ebcx*k!^Z(>t^19pKS_{Q|Y`4=ohd?2G2}i4-Xqw z&1rlRPmTzp#DjfaSl33peD8#@bC=LPIwozbrzihrD0rSbFaYa7qr70AigGXjddXGi zpYUMtC@jd*wksP<^6A}x%L=khY|TK-6$HB`eJiB*5$L&MVLvYuXs0q!`l1p8spb+O z6P&&fheMdcPH23e=V0SjK@wEhMHsKKEn9V8jnGU)tR;7*OkmqIzrWC;IiFL<$_J67xO-=Ia>5n2QGdGnLEYE8q9H z{4KM5FUHK1oR8XneF0Gy2c|=q4uqIh5)79lk>Zm;NxH{6Q1xBT1_2DrWr*&sc9ebe z4k=j0Rl4c_uTv?26_`ms@4ceG*VWTt(Gb?-7~Q$iDFQ>41Gi6)rs#G9$3;fyd>qGu zm(xw|2nRT!b9Qp_Gd>d(C6v(`>kj;a2@B}N zPwz4S3zmKk4TRm*RpwU9vhG|8n6ZUgUP>lwZDI#JDJ`V~Z7#D(Mwh5u_BA>#*fw2PXYPpLN>%dU1 z?wE zt3U!_(g$=-cz20{QB=(FFZns6Bj+-DA&ddO*-a zR4o{e1UYEzz5qYCo**N%lny)m@k>Vn|1j&bab4KQIOzrWgRrGX7?|cnVKTs|263lfBXoV<$$YWcRSS+ zpsIdhI6`;!uKcWKvgMez7gFXMsj(k}*eI#}-N<46PHcds_kbJRa{DM@4#ka4#PS48zp}b zV~B!Yx)tqqOQT)hB)Le~Jd^_Rigx%y+J9Ep4tvC};9LTAA-VXWb$uZJOt68YXB|@( z&qYec=S-vY`rIlO_-CkE3EYy&pzmojgmrxcff&g%l%v^XTN z;S2cK<W3%~{og>#-Zz{z3 z<>mt4g3r-fRKUe;k5`>|nDrpQ>PL^^xM1AXumWUon@6aye=@z8;Nsil;@#|_g@@<* zZ#3*#t;Fq45K1?HJ84)M_&7Ak>GSD(61kAi1NgVn?b+N)Ok!!)vsQuqn%CZJl_P0s4SGtiwC7OM+|HuC6T3`WF9u@E&~bKIeYdL=V#z4y|OMcc8)X3kNGl zwkND7zqnE-(zPL^Vz7u|dWz5(RyZV2?h6cwswxb_-%1Z^hS zj|6>BU3g78FvkF8z?%Hgo+c(qfQaO`wNTk+4SfG^>EJ^myKxWaM9Vnl`pO@?Y0z_h zS`|3fd(?5c62({LFGR~WGhDIEf6JD~f3nr9>6$amwpW$vLz4yO=L1J`qs7=h$vSfq zO%sBhyqoShY91cX$36CDN*v8UkLxXW_q5q0zV}Nf;J#(o`f}Rx%RRlJtR`y)_klBc zyN(nQ+w~K+2TA6nvRuHZN8bQOkQaz>@MVg)KG>XXaQEMwspCCg!re?j^;^*JO=FN`?hkbt?%lKGD2lV3ihemHBydl*Eci^0nJM>hj9PYM#~yPtIqRGzW)|(qm94m-t=HY6RhE zJ{|YH>kKe+vs?@Qu$K@BM^s*r%pErsjySsEo7mW~YQ3MSyCSQ~L&7<&bOH;ol?PK( zjJ^9ca`VOy9!|uStS^^&dfB-+^jST+Esl{!Vq2DxW%Ox)Ls<0vnZmA0P zp%psD&m+rmG1R01UXLXCVQY0uX$4=wu<_MGAj+fP>(Z1XUSeN%p$Nn+mF#h>_~_1$ zaD|hr$rLsCK4OYhGVq+QO05l7qEsd&9rtj$5|7OvUAx(Bonkg&>Ky#S*b13te(+qu zdD7rEE$wm8=T6>MqYKE1!TMMpUKQ@k8v5Fw`B3Lur@}h zvIJqK;Oddkg$JcTpD?%y(#<7r*h8y)xF;{`Hk@kLa>#hIUNsoL?4cwH5Rd|X6$mim z1$_i#iD03Pt?#iEdz4d8NGO#P+Z$SotvE;Y(xxoaF~U@Yad5-J z%BU&q<|z|f=c<3!bx1bz|I*$|pJ014Q1dfa?hzCRgjO7ClotHCe9M!z*!2kWxtZJfSLMFAH(z5=j4gi zFjP5v(G`mFL7Pgk|~^eSpkGLYR; zJf?iV?l<}eA9&`cK>|al%g|1mkks2_rbOG5qY2Mjps6 z?8Ti+P5d~zCmh`D3AMckzFlKn%T1#}GA&BtQbhhqE^}?ZqH+il(^4W;#X8tx*s3c7 zks3TNZq<5*fgad}7o>nl_?pOf{ZURiHmHz(${8g#{UjH2@Jf=C0}0C&dEERi=}w2j zsDibIgi^Zk=$T*ROn3r9Gn8-XXdNQKL@;tKOk%G}8pHXu=h?5RJX6bgQ-FP z5(P1iN=z=}xtr9ouWO9md3MJ)7&V?;;3kieCYEd_=G(Y!qX_GYeMePj=xB#6p>p1m zR)FaW{rOS(vu?j&>P;qj2^V(8AvKXGuOTLK?YBkO?~B&5Da7}y7N?0OkKdrB^)04w zzT>GgAb^SUQ2Pe3pfsR*S)C8PLP-_}$&45wj2~kv$VxJ{X%-DW5EMT&D%&TdB$&}Z zdf|W-Io}J1Rx-pMaJ#8Tswqk+k0ymNEUZ-r_GEt0f({~6gc0k*6iaX9L(lLp#LV#@3hOi2@&+?$B(qHcgjf* zIn$z4zmeuP5pD*l`B@>6PD@PBnQyVE_aDh!ikr&|t?aVJ{=~a!k-0e|e!Z25r zGg$ofO^gE>M9C*vey^Zqb$rYO6!Wd{G}$gfFzi_R3^3NoF23)KvN>wsdqNeE*fe(@ zszyj6bSobl<1RfCW=o{83Pm9KFnoInZv0{h_IZZ8a5(&4J9MXs$wa1Y=bilXk+Xqd z$%jM`u4%w!xuWJSCV2jT;)ANjc}^4R0|Xw@T=g6vOH$4|Z5HNYaI z&hm_Zt{UADyhxB2g`!q;{6Q~O#UnkAX9c+Q8iTS0s=~-5iAsnt4<#I)t;Ho&&%)D` zc4Hx>CkZznMcu$SkVU91!+qq(^Ivy^k3T-8zM6_(mKb{IKxnID=;v7jxSJ^lQwGUv zHYGQ}(3HHAwO*!U1-$^O0(^TqYFDmR{JYm>5?_{F`4#f3Vd6@?bY$Mrfg{7gOiaQ9#K#z_bA5@`wjns%bCT-ReW)sk3(0%+@~#NZ zo3>gqkT&|^FX`0CvAM;S>t~N1+;myjm}$x5Wc>;YMsIG*w8y*{V3^Qa2J9-IVN!ok;U+TIqo!ZG**-^-BJT$8Y@+ zjr+P3AX7;~bSBK$Xmf*t7Jen5{txQTIxNcW>-Q4E3@8kPh#)nzA|MUY0|-bPAR!<~ zNeM_u$Ivy>T?(j_NJ}d%-QC^YarXE;zw^H5T-UkYzt3OK<)d)V?0fIESA5s{Z1GQc6dy>*j}4rc{pn5STt30_C~ZXcvK2=Y|SHc;ZTHHM?HVe zHA`16`_<sa@ zqrG}|3w^B-q$p+Tv9a3Q8T1 zVW`qin&H3?BY(laA>3_IJK?a{Ek)G%ZS-jBO22f<%!J3Zo6fG|g~tE+NlO~_6JL1p z*Qg@t$FGG-$b}>eGU!w=SFgzha1Xqa3({$g=BfSc8s#d7e7wbm4PILsBq~TkyZvJ{ znzdx0?dXzp@Z$o7Ns3XJ$oadXjzR6xCr+P2J**ewJ{43f^D9Hl zDK)$a=Tz3dQM$ujyifv%zuu1qmCasLM?14S0bd*UDcHjp{zyKZZ!75eK5)wV&v)w= zz_#R)1^%VOY5NJe?i(7XZG>H?UKh7{exY-Yx8-V%b_>TQmwR`g>i)c)>tEJw@ zR{FbB5hf%pqVy zsC^xLz%em#I6M||>0C_VzA)9E=7sZ$zWrmk=w;VpW~?r^#w(i&P2-i!UU6{J6LY^s;Foe zDQMus(kF5#o1V(Qdz6;|d=uU$3U5X4`~G7AzC1U7`?Dhi{2<@tDL4d!a7OU&wJ|@) z;jHo3iay4~OOMUu^Y9^xi78w>#--;SNzq(~9+mzd%!XYL7M3`?7&vJ4w2(^2{4qBs zDjgnPoZ0gScTIIXsFfxzS|o5&tQDg!>i)9uivI_}H*)*XzmIc$BDY+cn$GbJB@H2~ z&W2zP&>ZGUo+~yg8pQ2=4RRC4ZN-{RR<5dt;D3)G_|}M1ep{5yjEAMs*)uV=g%@TG z{|y5SpFVSpLIVrfj{p`TJ>#DvVoHVCTuG%CO$idxip3t8Zzbn{DMJHwmZJ_WB1=x= zUwgz2>A1GaaHL172xb-I9$3oI;e!>Lzm|QQ26H;3ztDoWjFj2B_je}=Vs?_C<%lAm zB&or#`S?!hPraB(_6MYEqAW7Lpid#odMGsn4v%j9Ir2;cyjd6UO~HEp`~rs~p6{ms z)BCTLDkj4&%0)ID%k$(G1gu7Z6<(co-9`I$;Q5@Xv+#6e-ut^V5??`~OpM4^&!K^^ z`_3x=*iy{lAH~MvWoS~$rJ<0d1{b`pLaJMffpLkEiw37u$JehIAFwG=`~UgFUeHM6 zgs%=Pp>Oe+7zw0YGQC762wkkW>Z*j^dJ8WdHx(>ZJY5^BjHGb?CBZX9m24~1XA&f# zhB3;wECyWU|NiD`rT_b5eG9!-X{wlHvBCF^>YKb=?oUO|zDk$Fn7HSTF)wbOY%ZBC z_7!B8Q_HwQ@Hi*_?636fb`}+c+l-~0c3tcZ#I9L74_gY#Z^dggABR5?oB9KXfy|PC z*nZv&r-nX$*fYNe5o|$X&v1h<4t9oq&ZNQd|r#L{M~Xd<8JziqTU_OCheOabGA-K zMb5`ndN%%qzv_KZ^pe=o?bnnT*Oi-#&VIZ2N?*~ZinDPe%Zagg3Lfr7M-SIN51oA> zN)nK8HkM96?UIn1&{V}~+I`@gkn_oGk`D*_W~K0LWZXaB&gno=uKfoO_|(R6hZ|#w zVbt0M{BmyDN{Nfg!j793ED#DFkU{(qbu^rgLwZ;E_YU?+MAMBmOY_0p_)Q_3pYK;@ z8+)Jqy*5^HuqYU$GH-@@HAFg&mYAD7vTw_CX1Cc39Y{Yv(A}eZIAXOSWcFldlmtyu zop;-7BO(mt8M55}HMuZ0jOq^0>&M{cKsDUrDciI5%z7+Rby@FZH%pfKxQuwp2&-t| z)Gj`?Nm*da<56oo-Gc+Ro`>Ub0uPV#GuC_$dQ^jv2d$+$RUfzb%n4PD;!A_{P2-}H zrT3^`T(Pbex^5m-E(0u#DNZXUdqZor+mRuiWjbZ4LI!2mC+vlwf!X<5_u4+8BN;f) zS$jeZqcff3_lWgGo@_2?=yJ_9m-Gw#G*h`FaC(&5{GfC>Bg=d!|B_*Ehb1)*OaUIB z(0+Zg&<^_0k`Kmf@0`@L@;L9CI+_o>z)uvbj=r(8V-{rxF2^dS=)cB9W-T7AA0h01 z?xcYGo=?=JG?xmsdfcbt5MAG?dfFkBjqwI4c$ljR43B6;6$Q5N-?c^EGoEdb=T96w z%sV$dTp!`tnor_ZPJT3S;oG)0wf1^1oRt33PE-T9r=yRr?b^0QeT}J0vhqz2S<%nT z>P&u=xH(aod0)_QaK7~!)yX4L>|!^3fhN!@IiS=(7l@vEF3hi7=CQ%9Hrg6d0+=2} zQgjBIg5_M?=PeT(-RYn18oJFo9tv-?C)ph;De5(u!B^kN&rDW3FSN+h^6_B9@JcJm zshy9~YRh)E7$p4$+&z`@?AZFNOENhOEA zuCqy!&}5bp`0IZ1$ad_#_grH%7h!*XdxY;1hQ0iCL+4APJ;9^Cc1t(jC@CrW(D=ty zRF?;je@||67IhW~*`PIDy`z(ps8{5jY_BJ;n@- zU$%Ow^>dw%aa+5R3IZ6s|qA5;5X&SQ7c zJ-?tE)khP6?7D#4S!bs;KjQVci#(+Pa7_bRG?*A&-ye+2UMmVJB>nZqi=ZpBiwG)az}O*pIJk}8cK zN^%{@UaS!KvA1Eq<}oz#CxJ!xv@zbXeXPK)6(t=IQg_Se)VEy#+4b&yV3suNdiBD! zqNPQ4v3n@A;5u47p+ZE-ztDlitlytD!T;(cK>PX6=_ z%@v$A=e_IT*_+gtYDjwTj2UGROz$wiK!S9ND!C;;k{;!iJH60bT*v!neYj8^vaAmR z7nN;%>jY1^I)xxU*6|;2!R0!4|(bFJeNB#*4W*%Qny( zZVD=JKg^Q2#t7Z*cd8;=%+~x72@ZO)Nuo>|MN|0zz#eB`Na(cqa2YoJENrFyq*eTr z`KRkF3TKOij+{;yfc?g784e531R=5++#SNy5AOYO@@;(_vG}{|HcoNXmjXDPOaFiy znMSopi6(lM_q7q_`K;aDO{@Kp2jv)2E=>r_OLCe9CGE22`1-`?HV^suZsy*U1eIKf z>af#6jj=D2O9IJ}hhbAls*)}2Xjh!w?DqQ9!1+v^V^A;prPE~o;_0&(Zd0in!sbzb zbRkeVq7>1KRnA4uA@`IQ#d&!rEw2Yda6?0-91^DXx3nK2NCXD9TYU-HcV;8uHC+M6 zF7jD-*@#z-N*-yo^P|w^2a9(*ya^wmUkGq)dQ3~&)yt44@~P|nPI5Vn->iVaaqx+_ zKRtFs%khrC>+4wjxn}(kGuQB>JK`=#v3ZxhwgL{roYGem3J(A4=>@aD7wZczG$f(R z(#Fb&O&)e6k~CxCpkRTu|Hz588x3^pefwtHXp6Z4|J!W{7bPZi6eEfAa`1UI@>>Uc zwvHGHe_#69ut2W_eneCbzVjD4+VDPO}A70qWuif8^#n>do!_-9S6j& zhLe}32A#e8sYOL2Z}NmDIBpSNEV0P4?-QGx1}>DdETEzDfLs@r^cbPNFeHi|b`fn`gd zFN3gg3%%Zx5Z8Cw78sdMnpZ)=2r<5kz>hkC1jhVSY@yfCf&|{ax;0PTduZB|8@E{8 zwmmU7RMvZJetO_rC#XQ6SoX`K0>_x*7%ee|RZDQ}0J#nc7)*sYxYaH=EkkHF<*Ur7)#dL*9`vh0$?@{apT(=78Jf3x0=wkp29<*PeWZ zFB>B0P+J>^sb^OUI7`Gn<_N}Z>cY)7xAu>NHVbA$uEv)aYo#d{r|-EPwd_J=b&p$< zJoaK22WuZZ<3^Bh52Zc8P4$#e@cs!yrM)Ac6bA>>aV&Y#qkcClmdSsY1fJ@4SutX_ z<#uxvHbhkWtC|hf5O0#S*)Gt1zj7(R)+E$xcQRji%c3ca)&$IZN_`^Na5vp6ACstS zdj5}Ldx4}EM(gX%Xa1#$MylLyZ48@()qm%N=b?2Q8p0zT8B zq>m=|60~VgCQn}<3kCf=hl{pf_vd_Vtd(8tqA43Sbrn(lj9iSi^r}O|YE6Zg{E_3^ zt#je?gVU~?S*$yBsD{tO>Yoqy_@>Tz2da)*8M8mm2v6@rp}mIYT}x9`=E~HU@e1x^ zH2`Jw@Uj^E-v0(A&NoKj*pkwgA7ca@`_X zgUHfG_a3T&B0Kr5*u^@JNsP@16L~zD#!D}YSLK&}cRLOQCg0>!xMx-@)tq$zLfkhL z%CpZ?7US@#{p;{Wqgzzu0!9l9bg6&+=y!>GTMVISOmvCT!gJFDQP=%oPg6-r#)jml zi*1^`bc|stMDMn;Q?*?Arp4)Lq<_~R3C%Qn>C`E<2GQ>s6 z196`F<@3u-9Y~y?aU0=oLnJKGgXpx7XnICX3R92Lg51t#&Mrj5zQ|@rj zGcZcy%xG2qX|xP?HyM0cQ^r#Udv;x1RPj#t?c-P`hRz?KrA4BA&~+M(g)SWPa^V7b z6}XkQAe^(PC0n3l{7kd6Yqt?e66f$vNv=0yd#*BIQrsr;rfX-wB6-i~m|vrH!;e1? zEDxs~w;mYuvoP|%^bjrDLZ>g9D14u^J4)sV1r)pf_3Eihbg6)2SW=&>lM z1vP1x=0Ty%jO!A0bfl@n&v70xzrpoD6D|bbEaXSNgEPgj&r>7LulYz=mpQF-^eSMemS>S+y9eS7xAKVQ@G6U{Jb zQNmz>UM=gE>iK6E_Q>O@h*K5<6 z{zh-j+HV?yt?(Yj%XsW}av97u0$HG4ILc|o8{&SPQ6ZUZ(=t*)szoL`@$w5w8XnI3`V<GM_pGZmp3ZMfI*2$*Bt@%XOgJl@TU)Ga4~ z@<_nYQ>+@%4nd@{RUX5y{e_lf4V%MkE;u&7rU(ME>W>Nl_AiMh`#Q%X`5RbuGp`za(Y^jIo|*qi0Llkr+vJ*%T_Xr0t@ktOHX`j`$Tg ztKW+_zrO(F?L@!;7yNmsLf3==8x;WbI$3wh@G%jwS+nW!kvyXYG?oV6nb=8A1?R0 zzW8a=!W|w2;_qmZ2<)MqPUN%$muTH?LO17z3|9N^<%__W&qoyVTwvF&qZ-?-{OU$S>-N}vuKw#O=3aJGy8vqfdO`C-+6gxFn_4Id%- z^lwtgenl6gpVl3Raw11&${jb#2A}TO!r=2HQ%1j!t|Q*y?m)y=U>=-iv7yM)3yW5! zvnfHuPD%3bt*W(62b$svp~Ei>9ab*uV3?z})z8W5FU)sGj`?PzF`nNyOC>oB-pe6; zI_4tEal&4s2#hDzAS={CHLw*_&EhH$@Pe`vOFtq&O_%xU#o3Vn2&}6TUy@15K`5k7 zgSGt}z{p{{U(4Fo)eFpKxiQJ_d*$PH2bkC}*tgC~fbn{b@6XT+AS z-*P_^jtE7je?Um^d=*_vvb$mLxBe2~qh6&v6@yO!avug6OuF!_1seGbcN#MRM5#qX z$3`aGOu~JuYA;OTEucU13TqR$nBmM10&vIlJJYEepI1B9JBk~%;nzCnu`P)`TtotY zUw}CkjnkE0+>7*#0SdJ6=%tSgvI(^p8`k;M0DGiX)Bsl~C;O$L=hiWq_!f_t#^Un0 zS^SB5&rXB_2N&3rDGDYP3o&-mLzRc8MJnR*BVfvZg8B zdL+uFV^+dNV065!O*tnDS3qFQf~TmYq^O-7Fq`*ER5x{kEVuC7C$Vd7!uuaNE>nK5^b+J zTkl{PK)++PmLl%@?ejOU7oy`o4Xu}p zI2|5vEfVBVlT3w5>2iR`C3#E8TBt0id4}Be&E)C#Mjap5HW@jzx(=zgi!UnFulnPi z3ou6_zLDIN3&Bo~uxUO zCw)yCzN81BW?wt2@D!IoNj|x%>$_#&h_X{&M_jVmLul&T>#HHj<)6H~jeTgl+w0Cf zY45f9jmU7^gibP5Lt~Ooo0_Vp&k5IYYa@lm-o{KnSKj>6sawC6>kstIOWC(7)V@2#wFC)VZ+q%FUBeD!d^_2A;UaCQrBBr?Z)f_*VH zJD5V;CW|Na7*Wb@y50Dwl0u+fPWxPElqBcDz@H?K<5{(*;1M-fm-t1$J1fh%Y)LAw zIrrThlKB8CN|HU_Sup0Puapc?{~4P8aloY={xDpB;G(SdB5&liKehLlbo#JuUrl^v zD$VNbbH(?n;d*~Ymx-z3@&>~`T)%{FH+qVDRTi_B>HnRfdBnojJA8Z zP7fT4T*Xmm&E^PG zEk<8(;XK8UNj>{e^B@W>ht%CVgV62HrPe#leZx;he{o*4k3dwKZO1ZCo6j4W9Hzhcf!J*)t%WX|d5%E}$`LZbTJsd%&p-^-^<$n_amSaKaI$fH|lk2x%){($- zr5T19;!<2TD?VP^{Ot(HIKtT;$4bWlW|%x3nAW+<<@JK*T=jU7%%JI-O`V;Ma$pyp z`mzx+wir3Tw3RC5mxdR98yjf`)Jv)A%Q`4ohvIEtjpt`-d}F{lN;d2`u0%rsX)>oz zu!ZUQ>6Y)V0rnp~I0s}}me#G9!N(Mc(f0b-52&7n0xmC?0?Y&+_5mLYKLxD_?`BTx zRQ%q9=gCSZHVWo4fwkMb3=$$Smig^>)9#XokyZ}g0>^$s=@#etj!5$d zaHk(}2h^-AHR=bHh8CzvR1iH{rqke8WSYe#T+s#C+C`76v(&w-?X#(?6m3leI|894 zVCvf)4iQ7tANi{j97QB)Bl~-35rz*h=xX#*bG=u5K}!AkEv%Xs?pJqY?^vVCmD{46 zV8ztJ#^FlxI|nuNs%#L&c>X_%Ng*pw?R`LXb6Z*J$$W1RT4%7VhF_WDM}W?%8VT!N zP=1fP+1AF`ua0Wq8`7C8rgy~;$sK)~Qhtq@1B;w_D zW(cBQDCd0mkh5YwLiq))-BNt7ZM z?FWO+e%J~>$~X;&UMoi_DrE#r%CmAZ=bxWomsI_FE9NaAjW@5_BLAs@{cxGE##Hr~ z&E2#j4dLbUSkIF@l@T(&?u`_dl;D!bCqAWgXF_7-Q54^&^Q^l$OCN}qkn&E}fVGW4 zuh05iBf&-Bnn{ez*0Vh8j*{^4d7%(MjZI0BKJes}f0TDNmOjM_oEOE;#p>vs|U^B``m%2UWCt0#7+SOb=0DNH7#JR3A}7Au(V7^fco+V7nRoRa6}y# znMqQw#}Nu?O)uc{{ZHk7J_e}#e_)h2BWbycNJ~jMy14Y@x;7#tENmY#ON)?iAwyyp z-FUv|_L`!efF7F^Tk%nM6deTNrP|i^=vN$VFAQ#&8~Gh<$bK`Nv=J|gYY9|w_X7gu zKoU-~6)*N#o(gUuQ`Y#b;|v)0Ol);MaEp-Y?vDY6obeOnQ(YS*w)m}MyJ=#3x0GOJ z3AupISnK?yeZpc6;C%7O(l|tx(EvoPie30M`RxW$6~V~Q0G*k$kac8DiNyYO zJ(RIn&ee`K6+L}c?~nTi+PQnof3drrH(iEn5`*TS#tZy{LJi$<{`QY3LzM(~y{nu; zz{anzMlI(yC^J#^FRn}7=E-3f9((@DS;$Y=i5V zrF})nk@T%ysP%Tw4|v$jk`SnaSsgAN4Ol)jmCXzdOSoW|^c{w%V04`$E+tkJ;gYw7 z?+QMGk?svL=R%OZK?iXY@sX`S5P=9I&wPMp1cZ?9ef%F?79Pz$Bxw!|e%TxWsjE>< zVk5@8Tg=j}U#E7z0s9S93o@kVQT~R#XWsosp`W>Xf zM#+#vrGGhYl?Mv#`7ZE`42zKW?Y5p|9mV~rIQ>;pIQmA0CQgivQYP?%JGXBJRgfA# zgULRTa|%t*{#R5%CRLX&8ov|yP4)>W>J(qE4H|faFp3qRNTSOt zrDKF`jzA5p*Pchba1d`%zIJ&(U!+S`4$W+KhW!*oHR|3gExF@{G~S}(2@n{qsXSdn zv=3T>6o=dU#VUK*c&{bPefwBLEhKh!YOWKDuj_|KD75&-OA3EpY!g@0cVUOf77@ff z&^ox9=*_i7j<`fs-JM2HGl(0WH@4+1kD<^faJVV${>IqQ z0UK9dYc`^=^U+$I=waLjN`CbL3Z>&82IDFouy)@|e6UBJimv;}qP_KlwB?oK#DbzX zN&=xIcdJ;v!2`<7b)2zXTc$esUuqmya%|Hlig6-+n-89S;EV+ZEN9@?N;{AQ2|0B6 zwohle_%bZNl1WpA|cy3YPWY7@vMoT7=h; zZO?bu-3t)qN7Q;q>r3?+mJF!dH9Yx>!u@p#c(O~1%G(~_LGU7713h;o(LMR{4s~ZJ zg)RjYODgG#lUZ$g=Rw-ssf!qP%ajs*djpE~^evjq31yW2bR_h8A3n+^ z0ISBUod4R%?I&^p#L!}%(kb^db(teKx$h;c-c?g3Twm+3Ohmw6L9UaaA!5ih%Ayv|{H5OGkjO$+tL^R<3@UDHSW_KDSwi>?yR3H*E{5E5NML+^h6a6bhnYrthV*?Is3NxJYdy zb#O6z(X(wrOl$gNidi8iRmo=uvN$-eqw=VCQn5y0aGDQGAf4u*oeJ9RQt-$wo%GG0 zQxPN8(aIv9JJ@Iul0N}`HTeSoT#c7z#`Q$OAM({7T|Y2>;ohAlnaN1*s-fun(7t|N zh8N3}ZRhSxZGb$s82LN*AiV2Y1g80n*LyZSRlO~O@u@5_e6ePb!tgqZKJwF>zr^Py zMZQfFWm-1U`Uw{8x84RKD@m=ne{FZvT`$d;5YIS$aw1-q zHfmtO1u~kxM>aX!?X-;1%aA=G9lAvZnsb3?b1h5VN+S|2ai?kClkBDXSCU03EPPi% z<)~=55z`VujcAYSN^G!F>>gf}hB7myE8}%RYE+vWA z+s==7NyPIaF&-CK1vd7Ns$p-5O6luP-%h0)6h?G?<8mi(MZADO`%eo`j@(!!9KLVR zd+)2sQj>k1CRdQ73rIJTbav5RN<0Qf?g+`JUF1+1waCza40mYoY&&_NX3Z{(hf@e)1k^PI-wthI3Uu}FpP z`1ysP#rLa87#_y~=2N3tD5IIl^B^>={j>XV_8k$Y?>gW7q*17pe-)X$^wKFl~x)yyp)c~c1Q6~9oB1vn29#Ldf`(vqA&>SL3^@0HSM}hibeNvapZ%kjH z55MK)ui{F|SG@!WKhI22a>buz%LMmIT+CYIovVU7Ql?RUMeGz7s+;3-$ehLj| zvc(eR*AX+;QY$ic*@9$@qEJ)$aoa!a`8AM&^<6K_yucPp2ZL?Cn?bFF4-ld+qq8?) zHL;q;6poCXBagY2gIYQXPaICtI0R{!r~@Kh=Wj0FvmpJ$A7dJkHgZkv)>Nhu$YXS# z`|j6YcO90;L3(-?$wLZx5rJrQ-V1pgdauKe*ED~1l17pC-DnqR3Zq_I4HPFBy&l0% z)8LA#2^A+)R5n%Nm8+w%jAmp>1trAgHJC7(b-zonHIov)quy`+j~LzSF)~fdZvOEv z#kCAbKRs~c`weoOjJ^%(v$5@ixGB{RZ-oru^rZRY2271wwwx5{okvLGgkiR3uc~0j zJ<@NhcSx_fCkI+NSLokMhNW>Ku+mBDZ2@jFpfvCn(wmwE_GAa&^IGz)Z3?gLB<2P; zqU}!5jBJh}$q9$zw&1LW&;}>+Z!ocbU@x!0$qTEm}as=W5 z--$>(2W9g;v#`VD`f#}M9flv=tjF7Z19HESUp|D( zqi}KYjUSovPU|78jD7rW;NrN`HZYQK@^9Bd z8V$*pY$03G*&HKv+;5o9k^$t;7S~m??|JsfS#XYQYaYC7SvezP{b4q`Ut$1(p28~? zEPk$J)w9cw6(bWW{!2&AihZ39^VzV!WvY>RWnubvD6My$DJ{&C(GoJLMByiL)j6Z& zQgYn`Hco+GCC5K!Hi_e~|2HKzE4C)2fBntf;Q!15=%W(KnRZSG_m3tppdYWxZoKO6 z(+8>QBf+8okaHcgs5w~#Lf0hbySL<$?(l=2g_y@Z(*M0H!_*LW)hukhN!9EyNJ5DL z#9m4Q;anYJn4W>nw8pn`sAqluDDwX`KKU`Kbo8`o{W1PW)wp0xYXPM1j;fgM2jFb}?*QDw4D0aee&S&8C_6&=I1b#R{SpAkgFO00RAhS3D*}+Dt7-r z4(?x3A?7u9srNl)!_LFRLwwA9_Sqx&+TgWJs6hiq%8#{3inbQT*7`nuO0b>xlLI|A z|5^pUP>G_hkLEvZir)oEV1cKrU3+OsQu5CaSA)KVk@9mvhtu`oC==>O#XN2<4vG}o zpZ<0x{B0|c$U|TjLCc(&J)B1l zKke00;qu4JZvWXXX@exDheEb_W#KU@ByyO8Zuar*C#Qcmt`m^-O+O8BK!q>sxIH?F z{jcxMif!+W`S5F>xqput*pZEb&52beX zr+-{LuxvPgcfUV>FZX?HGjR3N-+W?u2eWLrTbf126}3+j(w}9=L~qCQXqTeayVHH* zMLzsE+3!>7IGXaXj?uOf;u+Ghi;j?aF{MP$n5B|x#jKpP+}4|^K#l=xM6~f4Hr)X8 zHBjnio_Ynk)mjtbq`20#2Yw&+LWkOo6RnLF*EPgk5sbr{hV%ZZiCyAX{qNU@dmfZe z|H``Z{Gc6uZw1fzZgu5Qft`X`qC+XY+v&c|z~{VjT4FE)LjaKQ!lv>^)>K7*I&8c> z&RLrFNELP7f5^=d{Ye>=$xE^nCdJC`!P$*N1Q+8B+d;~F>#4_fJ$?t!+kB*FF^ti% zmEY@;rKM;JQ>9gkZYlVPyzr3?aqRz^f;<<_R~H^6eQeg&wsm4&P;GpiAM?@x+E--K z#aoaW#ADQU%Y1Q&s|&xo+3v1 z&9lxV)h%YGK=T(HL$lF5r>=MQPbOHySuWYhzkFh%k6) z-H+21&iuLhxOa~_Te^50c4sH`JbXek&UUMY=i;D!JRsTOW;|jPYyP5?Ohb#wuH&FA ztJ8dRU-P=*Q#Q^d0Gi3aG|wwhfBqRTC9}zxd#xP2Q#8#oqq+TQoK2e|-t`EVI7K`U z2Qi{>Q9a(>H&#_UO(Dt0Lk2x|F)INKVF9B{<#S_-GmNIFZB*%wi);tb{2t(*d$g3(FZlbw&HQ-hf1HoRE zCf*nrP?4-@)N(aGb?RSzJJ~MW1Rli1G=Jf7K1WMYsYP{x{zF{rp#$L0+8-sZfV;(` z=zU_CF*cZf8v>=}L4^R+J!de)X{}m?A$Lg2r8?K7F@i+z>LFKAU%tz{g?L-Z6xxS- z>XKK;I>mA#EOTB(JTo77;XW^j9@6jHplf~vOy8zbB zeyEgdj4*&N_!Tz-VM?qQ9lOT%bwn$}RACP!(8&#-h$`-6#-FzBx{XY4phxq-+vcF> zS#5z}e|dz~WIK)7O_W(rwUg_)s;PdZ<5GUK^7#hgE%$^9NFYn9D|cUo-BRxHWhE#e zYkj|g{wC#jL*7-T7eol}*syPBNTsCOw%t=ZtpLMky42i@K9Q3u6Nw$`$94gAO%&{t zrM18xstQZB!ErJ#;3qvm^n_)MNGeJBl9;}?FV+yhW6e>%r?7XR@~q=no_+C_jhu`9 z)^rN>_XStuFBhNI(qmrAN$^oV*qHB%=S_1r(+~p&$OUic=o(tpHrAnKv|vhC3M9!80TiIXg9FAkgX zZ_6A7iwf;DkaonIq^#)5ik^l0(^w9^G^h5te;(X*F{ZOq7Ca$DqAYr0VYi(wyV(B3 zRc$`xxeu|P$Ax_nz~kN#hVoCj?uEwNbqc+@2Qn|OFwDY=J@CiouNStGdMB$n1v)OR zcKz1!_yf~ggz3&{3Fn={TX7hvQ4^dbzhz%&_4{&0aEy*GhP6?3FsRKz{XDC_G-(EG z4j`^5Qc{oTHv*NVj-MLq*A1hX?icBnVJ8++zTTV!Oa!a-`xb+C&&R$xBGPXw@&k+d z=o~Cq6a15(O)!^N-JL6tuCCnEI%C^K2`9eLcr{QWdT|Lh zqpiG_64bDD=y~f_VKdDlxNo3MB`8$-R3pwFDq2H#BwqPpmM##_7|(?YI803dwTJOK z<5;|qe$Z<^f+AgU;0U*aJj(j!YXbv5Y8{%Gs`nR|BPkH0YSR^85ORl$-Z@pM8|a#* zcPBt;QSoSVVxs7&N71Jn>(D7Qlk$Y)i+~R?sgEX6?2u)V{oT(&vE zV_kb?rmEoW=v1cr`=cz)%EK~jD|Is04s6B9Sub}(mI*bNbTZXoga&Z#gT5?1D|q@F zg!8Z&?>7AkpOQU?743S2mQsngbVX6?lW=SGraT?CDoxIXiZ6VTp{n9#zf7N7uUfa3 zi%A(&!r1q6%T5gQ}O=tkFHL- zJiJ?VY+ISsbZwL%9+Ud5fw@OLh&9t-}LPV#Vwk zH=OPOHrj7k)jEMj)7vz`BI0wkZ6c3(&KV$Gu!|!8@Z|Wz`Tm36*Jsp@p|i<{v38$K zz-VDh<309L8xL7pmy9YJ>J0*is4Y?z5t4~m(wxBQHU+&j>h=3C7gan?pDkvmx<|0~ zaA}MgoUt8zx;a~)|4ychv*vjA*f4uh9OTC|WEIipXa#hv(-+1+jER@b2#t6qqGnZ( z&nlW5*y0Jowcv`&_~}VnLmS7~w^Y}a4=Y`@tBNb2iF?#-@|i}xsC8lKN=h{Eu~Jh$ zx^nBIgYdK`4!s8pJk~nu5H{nW+jq%^#lwL z*|KTSvA__yUn=>942-v(FACJQ(j9-^v5!&7qF3Ag;(#K&g#R%5=$L5M zODt)hx_K_kOg1O*_JY5-4^YRAmk_!E4is%Di{{bL;Ooz4DchEw6E-5M)}u6%VwaV- z!`X(8_tFX;X47p8D^Q*TX_N3)mEEnSg;>3q&*M9>dhUVkx91ulwSf^1z1YyeZhyv< zfysbg6CL!WQHk)Uv&?g2*}!vW?AK2wFSqgyt@gz2!~ z;G>mEyN$_Y-PS_LQc&7|ml3AlWTTe`(1V?@<0w{X1?+&l`xg7Z!&Fm^BWwp=Xvl1} ze3l4P3}TXC&8=8Ws4vfbr|(ap_xk&4CWlyOIgFd}6)rneF~l|RX5jCSFUTOb#OLW5 z-*|_;Vn0lC*RrkYd3^VzRR}4O;;XJ5@Zf-shUh!v_xty@o{EdR0+CH;V64bd_*y}& z`+ka^_&sr(yi@Zis<3-*skey5cAxKGQkv(Jin^H0oAg}Vjg~i8E`G-(+-5m>rMr=>FC!DbTnYvs z;MBZgw5+X_PtutxZ1`xH;NZHP?w@gfr`atP4qwJA&sTyQ z{diF**r#(E9+p7-b8$Iw{T`&Z3ackRP;I9z$DWzuvp`=m+xn;QO@V2|oDUq{nt z#!v@TuEWE^H_=GfJnJtJTZKRVh-yZAn(7-#q%f%J z0Wm3sqVV<)_MM|dQckATk5*<1^>($sSSa@|vPf>I83elP z&G?Z_a+RU-_4bWC7_@>}z=5###$n5YF&)fmiBdwaO zqYglp7u2>~MoX%NtMvUfk~*P3{q&Dr8ws%)E}vd9HCzc^?iqdjbntAh_c2-0P5_txE0q>vFQ!|r*wF*c7!@i-PyR<+ZodVB6WkGl zRWj>ma?s+&@-4%FhfDs9$c(rxHoYqeIvGOQciy;Bb)C>zL3l4g2XE7h7QDc&@=!Wf zFZVw@r;*}K)K1f0J;J#yXILEqlds#;!gkPW!ePxFLtza^Q0Q0uQJiTSdg<8@yuEx0 zhTd2S_Lz-?i}IgeAYF*aRtez(b@Idj$rduoAOz#?h4GmY?3VyF#col8}_fp=)SFx&&ziq@)>AKw3b$Q96b`o8SAs>--Jp zeDND=xn}SEJa=68b+fqN-s}ftODRS6Jj?ct2)Of~T^Dtl5ONjb9;+n@`tUyms6spZ z`Va_44~ncI%Fk?p*bEW^Fv^aZE3$SqM;{|4Fed7y>-#iK7oYO(NyGgc6bODk3P=V8 zz4B#$Aw=>p%+2p1A!G$O8B+ev6Lm@tfW4Qn13G)U_257gxU-w!7PJcB=`l1HfEUT^ zZEnbXwpy(^YVl^zvs||^5sb5|&Ha@u(PokJsZ&U`G`32OciYsxO{l3%aH^?64%tc*3ljaF=vCC&DG>R-;;xVPHR=av|fhgXmW=X+*pVr@GnG~hgm+(=aGhU!Z-sq-WJ1C*8MVUZWDK%N;a5lp%GeW}Z?oK|K$;?^Ql z@iLCbp@(%}gA*q$x6jXPi{#rwZ`6JiC|q~%Wmb=L>%Jh3gqa9rkqJhAK3W0c5zjJZ zFdN@o6Cl$hb8NAc1}KR(&Wm~Ky+U;z|{d-L5SiNmaKttk36b~F-zONid3p?OinUHnfW>G z{Q#*sPey1G5(U&$aBw~5%W4N@;IU=oVG~S#L2Rn>e}QT$Cw1+Us%Np-W>HI*yOnV? zR~R`GE~#K~g@ZnZxIRZu>pTXwApr1pK4Is85@kP#Z#8d^j0I&Qk(;>;Z1XNEi9i?c z9JAxAlPKMZ5)J*@F?F9{F=bK$!FM4zh54Xk6ImcTN32!m3vB&LJaY6I+92riWp#B2UQr$;zLYg$SB` zdVeYh@={p5fB2CmKBDN-DD(kxX}T-Bd+>eiiA4!suw1%iK%(bLfiWkXUT?N>Y)#B; z=_~@%EKn5SbPa8Srh<|t_jVFbsffu?+_&&sN|-C9lDPT}-uj(4p)rAfX~HY&O{ z5gcLJ%GDM?=O#6F80O66{a)r<(7tIW5PMOIg%L(#w_;O>T>D4kgcS@bzIe#<(Pxdt z-RhYl#%g1?90N1Wkzem2+IIHWz8dsjiC21z*H7Q}Jp_@({uM84Q7drdE;?X$n)b=8K!j2+2Yy&`R_Q>>5BvKUKJL*&q8{X$QrnoJZ6Ya}; zI0BHe7iGQ3glMJQ^BVe;?L6Rh;33sZ)Bs+G|F>~%U&RqZCx5kvJeCvIhZ08~(k^c@ zAY>?cI?bNL!-*_1XorDods>L^IH2kiT{li3aL}{Hw2XXJX3n8!-@ZGH3y97`W8`p= zD}uOj%sZnL9?68?F{w6tak{goAd&ri>)i^ez`I~xNR+F28%jXCr%N9)BPBaK{Xl^p zM)wgE*Dx-5giizkn;*rtQF5-I#Jw%<8xVolAb_rdGNVjiM|@#iTgJCB#ouXQrM~I< z?!+@piE8|(5)>XHgSxj+FF9ospkM&wG{Q7fCvJhu@nWv58MN{!c_&`x7x!J={30_2 zfW_Csn&QeZ82VJ69@ehB(4$R9^VVsofRNRhBYuVb|E*qv6avss@XF>vZ@TDiiYdL9Pj0bVXX&nRAw~ zm0!cyoMFTl$fb;d_kjr(E4)+e1;G#aDX{!G$;H%8URoZBO&L7)M)0yI_vnh8Nu>uw zl$+(!NpOw2*zOk+w}Ji7agb8`Z)%n^;7&j#%jM)I|3GYYGxCr(wLay}UudM1Pmc4= zC%$|0+S_}*PdRre%)Y;j8heROT^9REf*&Nh;Q*br z;mkDk7Z#G{;1B|FG+nwex`?b1BncujBag>X7$eeAfIkPSkUhCemEo7A!@wVuifa;> zshEvDA%ch=Gda5~t1PTaNfQDjB{FuKEg_ z&iyCI(x&`wTPt@7&kih>=68VIFC$Xj`E(Iqa;#rx!}7{Xe0fqpEI0%BtQjJZfXa@H zzGcPk;w>3i1D1x0bIIleCYUY9x;HGCZWR%J5$N*WlVwQ(&NaN05UOzq zWayEvgw_cj$bE>~+SmlB64ul_O{7`5^hbn);=?`>2DCa6-}xv?17YcB4$SU1`0JLx zpe#B6B0M06*AmJYtIZP63pHY|Ces;G^UKmhfz}%AzG20~&v#e=1cRCi1e9U@UX(n+ zg37py0e60zCBhLsOkx3;lZ(rA5y%`WOS}Pu2#K$|SpXbLOzJ-*O$kwc?Ty79)Z}|j zaEIxtMs-pLG`Xiom>vKEXR{c-f~ik3^5IsJFK`^aS!aOV;p4wR{hlfMkD<%^;Xzn| zOVq+V-bu(x$B5pg!Gh#VN$m@e?Ay=cj3wPa*ZCI6CK5p2ELPWPl|vVo`Ed~^y}q4n zbAg&nw=#b+T?X}jh>7`&{TYkH85OeVw3uf-VQIUkGwl2$qx<9Wn$T^X(1(M0H%paG0KqJjhhc4%U;% z&KiyPinj{0nXw}~iNbT~@(kA9^N4Y~-Im?l!mgO4#YzRMqukLrLg zYJheq?p1v)rRZ)v5qai=cR1s(Z-6=a*~*g~@Ayz&qrS z7ca8Iuc-Q`>kE_)_&}g8i>}Z*(UaRA4a$*{&ogyCarxuM!GR-+|hJtYxI5w^Z zCMEQfpK1p2pk-zR7JZ3hpwR>_VL7cI;fsxx6GU=VV{(FH6%gN_>=6vJ#1N)`p6rL~ zX_eFw)6ob}(L?yR40>;;5|hLxB+O-EaiH2*<}Y1T#egYC<&jo=J$T5)q;N3b~) zTO!pen`XQXd=xG|=k;&Mt*L<%Fqs^h;gEf02g=dp1 z4b<=6rjpI3)m(_n!|?CYl+=$`Lxzu&g-Rq=s8aepizO~Q*{lhNN5$*dex3+Wso*YT zTy<1p7ri(UQs$*Al?mT4sa4o|<4+hCB>F~4{3tKeuOs_S>j>_``QS(XC!mP5ERsaV z<_`Ir1)qJfws3>Zv?+N}gg4G@eNL@}U#wGD6Pg?yYrSqFv~mAIr!E%x9>1kj#~!Ey zN}e+ALCi>|BhhFB2Om5$DNoy6!{n$8@>8>M#^ucW05(LZ0wo1kN2keusp7C9vNHnhr^ms+~vhTNs%P)h%}y%%eDfIVJn7__`!(8qC)V5@I4LX zg~xYkj@J<=5nBVSNz|t{JV8KsT%*0i%53WB!Gi2s$3i9$f)UGsKrg;)=Q4{qknU&`^%UndqC>E&%>P*kEE(GT&!;{}F?okA#V zdMabN%8pHe`ph?Y8<654z&L#mvk8xS--DR${5o`FPpDB})B~@Rysl}`=|bIdrSPm? zzD(SU*j4XMd$|>XXHaOXIB4HkZt;73rAd;LKmr^baX&3?=fi>rWL}*}kB`==ArNma z|F!SfJcHTNKmfb4jh@b3bTCK~I%t2yVrUu2PLvHV=ofgz!nOT!>MC6!0w;M{Dj#@e z58@ef!4sJOJ+qg5gXV1A2*KtP9Nb@#IVEWip5{?NT?#*aDz@aW1{$|bIuJAY73^fv ztpXFUzBiHkD}OJ)D~quJpf@|;y4*VY5gCOgDeul5^FY2SDRt7w%)E#yO$2R7)m_dA z4SN0H_T|~EJ(r&6(YZ7yJlG4&K65<)!@4$rT81d>T?7LnYP+4=chmr!B`Pf@LNaQ1 zpkreYWQh?<5!XV?1d|UV72;+0wIGAy1w3_Ova_8_aWf!F^o1I&NY$I#V36x^$Mv3K zmLqeX#_gMZ;drkZFb}l;jZWpvyn3!f#n}>_V%J#GU-=tce*p8J1b{rjRJup#lthfh zuazOvpK~u5SdR-Df$96#KVpSKDu2mPf!T3F3KX)Du2xO)Tuav)E3|H-QW|b3d=%>d z>xj_GJfwz`lTeQ9_Cvy}`&v6}_d81gj;dXc3vDnlNcpYEB9P{7KRxohxM3*Cz7ol( zVvFD!`sF!OPy7v^R&O9{u!C-~5o{X0d$@E(8M)7^-sefBCZHn13gtHH`st)K{5xHz z6QFgsYiIS-BtCC3`6&DQHM`}K3uX+*5C8?15y5r9hT@aHYOi?E zL{(Pq9`gQc=9<}!xvzE_umeDL^Q`yJj84&36OvdY!tK|(-&NOsMS0+xGJ*$IOiZW> z_%jilzE5R!)w~{%cYnL*^pj<*m{!VH=K3p?#&M-WJ-%T$l?wLE=iKW4`Lpi$&jvr+ zy!Gg5qFUWzwfMv6x*I;-88@%lKcHxL^m?QPlN}0|HbSX-{ls+1=+1Wrd}h*(I8)-&GdPaA z7#?iWRw_@U`8v3Di$^|8rH}>Z`r_r*%;<<&Wrj4#dGrkFca9wV{O!0GU3V;yLXP`G zx8COrdstsCzQU>S=1IA2Wc6!iRKf#PI--_UiQBebu^{}}6m4X65he&4;4^)7|-?L*NFY82xk-A{hHn{+C^yt zbG4cG1hW;lfg58E1k>y%?s7E zzTcB9R(f(K+Rc;c)ukyHruN@9l_kqFE2~R^CRM0V4Vx*s{WHh1N2*qO1JOFmaS|2V zyFh3(oKJH6LA z^DXEC#1(xNks@ge;MvKHebQk)Df|k)z4sma5voL(QI&={PqV z1rZ;W6U%-qQY^=`FIw7h%9NtqQK|Bmf2_}|2~_su4YdF+_a2pV{ErfwwZrF#p;s5Z zr^Mn+bQT=`7d0&P#K+m)?bLtfVD&xnvF_;J;5+?lh0@ow$_1fLh7u34YGi^>+sw!%qIdzIR8xF+)LxO!aa)n zCEE8^;{(gG&j*RO}tmM^ul#r<3ve*+w3#?qFwY)%Z(l9Bjs z**9Yr55R5m(s`-98v-o+0joHQ{x6&;QN{#JqGX8sHC^TNXNq`w1OSVzaP=xw_ci6V zD*+KYyAnrQlFBEZ=V`iOPpBUUfH45%j{udM9Zf5-P-{Ou3>Zn-$iGZ{Uh9A*Tr$ka zH#WKNZVW4_w9zbLCq@u6HF<0={Aex>o>%%BSNr^*g3qK<8n11_Cq8>!T^E+NlaH@1 zhi$cHt05R(t+(-*&VPf&Q+Tnu^Xi+ffqv(o23g7rL>qeW9;RuY1W2af{%9{lQmodx0v7lZuWH2 zI|Y!tzSBk+70@7n7rok>-Z3}g@RDg~^Qd{go}|EyK^)XCDTeLea=}KLs%RM)I6&~Q zS3T!1zzBVbuRmN>7r;5C&*1)4)Qa#;WG*pkWeC0nzUd>`m&)o;s+F6LfNUI9@NP@Q zW_UX;OMt)6#jhwL*X#4$Q!1&C3QBP_VtKSy2oHbfW6Vnt#MAvL>BTz$9ik>m*5*PP z?f_-z;FBMURy~4efcT?{l}?@Kkp0clcO3R89H21m`HK>_PDXxhSIJyDz_RS)LD~2A z^AW+(wtxVE*Y&bK)2y$-oUJJQe}e*l00n!7Y{jI|5r9cM9LfMBr(H2xZ-3djS8>$R^Z-=@`e>>|=i3#Gw}Kw0 z`#g=e01TH$D^*KxtrPwR>_GL&`9*b{nlND}QBfm-tp-6Zus6t9(#DeQYx&Y6)XnAT zU@&ep&|fDDdnuTr_4cE0?l6dvzs7C|mu?+G&#GNf2IpVG^rNNnl>!RLAhx!hTG9p= z*aY4?dKtJ!zvs}5%+2qk3>c#J_kBxLzf#x&ceTk$w4%yOpwyFfMFYlSFu!|(4T1ozQ+Sj%eOgGW#2pM=SSI~M6YV6fJyv^%vKc&&0IM>t1Q4i zoIRi!Ww}9&+x4Ru-JJI!!- z@zm>m=G*3Y{%pJ6L>}p(mrpiuli%$ddw-vNxq&-G0lol#_M+Hzj5WrGV~p1TZ_f}1 zmLWpGQzBnWWA4ZXy0nmX;%#leUs;U+$}O>Rzl1 z?IR^HB532>a9WGt^I@Pwf9PAUR5uop)3x+T*Rhz=Z&7qImyMtgjCAIb!eDGjLs+1) zJkM{WFeJ4mkk%&mc1ik&-y6@Q&mIis^ziF~CA?^thncSNOpg0_dkHCKXALZo-4AJ< znJP|5$5y#$it%hVDcF+MM)>xj`t;UgG!sk;pFiFzm_2-Rsm~0i5HN2KEIV5hJA}ro z1s>0tjO98c#Y-A#I<2_SI@}14Pj79Ce!+jMHyh6rHmg(>zv+LsCe{I4kYG`wgeG|9 z@yUKr2o`l;@yQok_bDo0S;TT*H7d&LetzJnDIV)P=T9j5)+s_wsGWV{{Mr*(NvQSY zR>n&)zgqGtkV|eh_?B@M-0K^36uBSCdl5uCqpoM`bCjz#P5`(UVAjfOCIg=<{{cFi zHLlgD+AkAg+9|bu3Mi3Tu7XV2i}71d!ql_e{R5MpkD_1Z5B+70DL%Fj&@$b)wFioB zi6~e_b*s%Sgj}Z--c%Gk?zuJXzdDgR_(Je&@2DC6XZsG~#c4$;NZhWm638^|AO4+V zk=|tsPY@D~UHjlwH2E^<1h^z;r~VZeb&Ell%lI<`w_N zdz|_hP)W77Kn-FO`h=R2U4-lD z=S3iH*qx;;$DDoglaN=kJ%gVSzPpTckO% zEpwmH&LW*2F0$UwhzcF`G32+-4&SzqoCz~MAG7)mq$rkOyR`jtA~E$D{U1L-j9M4Z z)DB)O046(|C!zaEN zaVb4(P0=4r?xRrUDB z({wWWqsq@T_*vdBkz)F@kWTzyWVh&`&P};qE=bx8xZNkv|7i15ikuECgJ~E^ww{oq zte^bRL+Og{@p@hcgmKR~OvHLQ(IbRy&eA{awzyT%GW>i(4+vCJsPS-S?XsDY1J}m(ZQ`%xv^lfx!{#{iS^i<2v3vbKK-JZ>F`Gmg#4BPYmHRy zBskIr>f3%T;{*q6*zdYc*ogfV?{L3}Ec0@*EOv*%XQn(z`D1ZDJl&7RW-iG$E<1NY zATD1U`PO-qHhnQupQ2Qq5P~}yoKsHQf#!Iv-U4lr>insZ32b}v(eK-A@hwp|HK9&a zZw9!NczIi6?C@AqXVo~IK1|iMHV@e(p0TY1X|y5A+P zevi0T`7B;77`RAm<$Qc9a}0RAL=y>_&Di&R_ugkeYn@(>XSQOVOPjGWjC-shOd?Jq z6dnNx3JZNR@51^d6=>HPp|H0;8CQW960x^rSAdLT?{~{-xsyp(owC8c6`XI$Ni+#vtR;pi zWO}Vc`~=wYDc>EfY&$`wN4-=uo7h>Iz=`t{xY$IxQ-p&2!>x>Ep#Jl#G^ux+YI^Ws z@kocRj3GTCnF;2+Uh%W)Pm;?4J1MuUpWffyoR7F0K{mD!R0%AaqEpz1R3El-yIv}y z&7O2F?RmEOq>Ok!?c(ONROqUw9^fR~t-&|ChU-%K9&{LRYU}^m)>?}CoP^BbWZ2^# ziB=(WZ){^4{X2z$A#r;chi&ZVZcF#fxiG@MNC{mS!vB*SS=Qx}0(&?6g^n%jWD{(*Gsfts-3j-Ead(|(97Jr)O07} zDA8<1$J9%&^!;ga{Ri#a2&D_A+bzU1Kj~A2%M+HJ2IbF98-Cp&DzB$=xNVa(%=tSZ zO|k1Mo}UvX4w4w+)>x+E7Pe{Th*!bhJ@INWPTjepEXLk(HriB$w6}WV{lpDH0_#kb zJN7M6XhO~@F?}}n<<&IZ(yRK*0-6a#(#PPA6M4I`+Z60@btgp z2R}O`r1jDWbG=4okHPLlqx=u%fKL(KL4dXWKBcRggXl@5@S@pSO)CE4bIbQgk{?o< zN!Yf9qH%{_iq@!6HA6pYm_1_N|NiTJGR7nrX7z@i;2W?o4}WauF7PbOH-)78T9nB8 zqYRQ=zU?~fP=D3g(2LcK;O@(xzrZxeZ>P(=E;dch zoYZT@a$0{%h{qV)2m;Vu*b&A3t8(;?Sm(`)H%G@Z+OMgSBv~FYMkPuK%kxYJ&~srW z@Fie~%Yzz&*RL!KQu`7+l*}&29`e}Kfcp*^?Nc)J1~ks3cck?hZN$SYqGk3K2i`qN zm5=DqYduP~Fii~Ud?CJ8#V^;WGrb0FD!AVCj3^lT7t<>3qc|n-sP5$e5p@mydm)#i zf3%JNxJ^gTZ8sGxM)ng5pR{`Z!`&hGkI)_%P5x-W%BPEKW9 zAFmd+b*;>lV}g|BPr{J$MHXGEp%^k@1;jj3sj+-)=B6)r6qP}giE86_p=5%`d0NzyN!r&MuZ~RWrF)>5 zj#Tfe7wm1ceFxrj>+V}-vyi@-6!R|cQ8OaFwQ2ixTqC9?68iPAW7=pKiGXcy{eZ|M zj$_e=_Eb&lBr04)*rL*lf-H}?qQ&WbG}&5Rd1lgb5~GFPDfDh<7tOXwf8E{6ZQDk# z%DKZZ{7livE9rgU0f!50zW6ZCznpjQtfWfuTbYsdnvOy>kRtv#5=<>?d%7kBf%MY& zltor8al$Rbs6EJGT4$L1Xl!p~tG9->W~+XNLA0%5L*(mC5C}UD@^#Ht|G%c_8v#MH+71^lTW~0KJ zg~^cNfsFYy6>CgtM4u(RhGns{y87CF*3jXSe=2+-7A60yfg%Yf*nt^XvC~ah zn$%l-W>r^^dW-kf$3*Js8aV_-LlU&j4JFlgl84RA=(U#sFnVyOdR1Q#fp%yz$ z9i{gtxD?kl_Zz>*GY#m$V0~d(-`Dm03aR!dQoLfbz2e(u1S}UB<0!cf#&s8~IePr5 zxEVQxV)&{#t>#(f=JXL}M=x2`6qU}?Qu(hBN^9pfc&?HDLtG4NG=6LiR(*Bla z+e4QNv9y;>4az=01Cq8@8mpFySCw0{)6WLop6oGO2hY1xR0$R6FYrz#G}=&Oc)_+GL9cxR^}A0d6z z4>Bcoopz^~9h(OAw=;Kq^vQT&r9Uuom7c={jrVkTPWXD;JcnFXw{=URs^{oqKiwfc zdMbm2kdd`6`H1s*5eeut1Fh8ivZ7vxCmF{A+9WSeC~>#D640se>p@(NO0j4G^&+>z zp>nVl3`Usd%&oTF{skIp9?@@R^rr_X40eic%gW7)vXL5J#1lqlGi$$;I*h(fcF(^n zlnjc`K&)$AU@@$0VCdC-p8K=s(e);>vPJ0=JpmutsxGCY=MgO_a;JD0x5K!sbWF(V0N$G@3$_JHGEpE-b8%kfDDGnCZ0>X z#B|gY3@XV$ix2wtP|;LVCCkm0jzyzchl-nG zcwB|ex#MhYN7{m);6wlgJ8yK^HFqo$4WtOdR7!JTM-nyET(NcP_CWv{9Amx?tZpQw z*TQ~=WHj>2AD@X`F>0#a_t$^=lb}gUwb9M|%~|hPp_HZ}iuJ-|U)|pIJV?}Cx>aEo z?{8JM?@8NrM37m{py!=1P6(v4!b5G7R@+A0z@*@#+;qD(xs{GohW>-qw9?vt366Ls znFGW1ks8MYb4d9(4i(qcodj;vb=5b2SZ0Dn{Pg^R`(z1D{#5nVy1~sKL^noX>Rz_s z45ta}@w1nS#AJylz$+O}*LC~d?Ld7Y47A(7!-U}z^Ha88g^6hsPb!@B!j0aM-(B@f z-xKqRPxAIpLV2jwKb@2DP~?doOMLWk`#VREk0t9q57+~kB-qY_e+j9HolZR5w@~74 z6bFEjZ=~?@)tYJbi#ItUz+~r<0Un}IbX$9U*@uKY&3LHuozL9d>({9db^Z#XziCR& zZYOGfbdwwo^m7)a-5h(!+votyqf{1fJd`08y?A4jg&wy^@|K`vd!XD@?Et}`m`9Kq z;pEXCF$DaweoX8+#t|aO$ap4fiQ6kv1qk`@RTXuq2J-jNUE0x6Et1xX$Sqi;C_59* z!DYQv@0DpLxXTMNwZ7K|;7lJ$#3Jch-HN9PoywRLJ0z&nwK<;IA_YLVTW=rzZ4+HX zO7r-qsn%GwgOnu-7x4rpIcz%rrnDjqSp5jK6fY6iC=nlm4)EWZxzDO+!|Tj?jKFMs z?R;E&s#wK#;Fp#jxqnfUtSgRMP04z*RI$Eep1%hN$sgr*+-k2Uen3@nqT+#alF0Xt z;&~!6p8}ncG*kk?EcXb^D?x1*_g})K)U`U+`rRL3ZvZ8_s!e~N$Qk@n(Ir7MhhK5Q zS@U4F%HW@UU4k;DF(2j=s4;HNb@I7pM}%U)oS@8Hu6TiSipx&Gx|V0>wuu4LQGN|^ zz|VNI_!p5;!e6eL!%e8xHD~I1SZT(z7Io8(OgD6HY%SD240Yd*61ze;}8cBEIY7t>J?LF_GO>Rda*<3H&bLkErCXsr2jRPocj z+-XulC^h*|b-WV6XJKFA$lmT`aAyvxuZ`EA=3bbYCRbAWB3wG@m=qtcKt7&zl=S6s z{%8umbrrvW6QUa+bj7GX%VjDA)rF#*f37DH359MBl^3r5eY_!P;^3k|rQxAqrc5qA zeWl^KLoJ{h(W4fF)E_u0NJGSN2_`pu>Xfnln zB(O8=VQg$g2%hQZIf*qG`#T}E8>-X%S zJg+yL^IcTaq~-e*a_C3E4km+)P%a3*3qk%$D&a@MpZ?+|1!N(DNoO_6m$XMXZ3X>3 zLDCq>?_(HJIhFWFcnW+0E?hn^WTpMu%=hT~|C*tF{ChW=&Ge6uK07Ni z31?Y9EUZ6!ue)n4;h|v}GdP4NnWm(`ViWKYH=nPGg zN6U8>s3yGqxbBv zG`U_AdKnZ;_1c$L^6WMr91)-aaho_JRn&6NmApI<$Bv~Kj53&7XA8CxrTDY8Kx0wM zidQ{7{J*BSzV)ue4TL{_IZ*v0%vbjJTVsvxVMK zI+*LJp4;CHakKOq*0ZmMZ6`w}(*OY3wE;VBleyrR%T{uIxjByXvtTf^WuPNjxT z6z&5bIvvh4PIz$@Hn$&_A3B57H#wqJ?UoeF*Y>KG_@Cbz^$nx2iUV$MCI>ELVi?d( zSY#~WZNme4v|l7V2je>uS0HRjUL+yY)IeQ8D)%nNHzll81puxad(eattYxX;WEhV^ z=(1$WKg7<+z~j*E!c!V2gH8*}UtngFBX$GuXlXx!j69^&pRZZ=1K}i6*WKJ94ZTDV z#Y7jZw!%b+;ymzAK}$??syEx}YQTUBMIj0C8|@8y3wrVEckk)tW=Y5=iPL1{=ICh0 zCw{z$upx|pw2>&hUOkoa7MsPyog<-5Ryo$;@DqEaw?ecxVey-cFErsDM`ivF|3IJq zJ08@xWXX3xPVGw#e-B8|PdP|!Vtg)vt--xi z;-BfIo`}_HIiBk1|Dj-8`@#kdnq;%k4~(b@UYRffqACQT!HN%+L#kj&V{}%3L{z_f zN>pbxAI9_aC_lGfvddyM-9UONE2>%`>=hFGNQ)_qG*p-_QrGRp`G+tbDicxZJd(Z@ zP*0=~97Zhm#KO6cq^QL_6Z$10g4aODW;C1{NM4vtH1uO0(JiljeRSAMI;F45!l^-* z+-V4bfO)BEn3Mtkdm)GrlvLvY=|1_*(pjTPXhOo7FIiFrmIJ>Mu8qLKct|Rh$U6cE z6QIjA@9FWi_svUN;2AO&NQKCw!z?eWrX;21TFlVv1hsnVgC{kCI1lt75C=ZykOO&n z89baM%|c8O&gF#K+Rdc%w42h{SuoD*1HX`u9Hd`{wdhO_qK$+GY9iG093W-PVd74B z7M_6k6WB9|o$-4mrxq+N?BqRY6gsjMmR5~R;5-c?AnLnwjYkG{2a0M4UU$Egr$3ar z`pBy(zV*y865CytX8Rj=HlgxPenB0p<0s8VD9lQn={anIX7D9Udz$$H_o zP5IbqH);?uGsw-Ytx`S-P2K@+wopuLmPGSsnFsb-@4%(5$%+z!ThzzW+XW8a*UMv& zIs2@pD7{;!=7NTl^)a9(Dmj$xTso<8A}2baR343v@kbSs^?G&U=4-fD@}&4DRm#i1 zeY|dBWGerC1EFB#XM4@KC)f%}UFhL%RA#6z!8a~$nDuuF&d znJ$uqxSQVU=p~%|qHUW4M>NRN8JZ<658m~%RHLWoWp39~cz*E1;oT2ijfDK!%ucM< zS^>VuB<_`=d|%>|*jHIBn;ZuADLi$ZQ+wUHn1yMnTd9XJEv~j@!`VH`JP%w0pGsMg z-%L?3>yHe?lqy}GG{5w=@Vvh=Z{_Zov5Cf1!z8Ovzp7+mgEFln*1eA&W z(qE(0qmIa;3@qOTZZFwhSP2gD=C5lR4I5{8Y#I$5<0FQi(1Tcd*u5>vB7<2>c}Zht zJV>&@N8WHb6}SUr6t?N1LH)+t1?mtkTpc&cA@5!qKOuw`4lhL89)Un~0Z5U*)bH@H zjwC_C%YhnhgJaJ0eX9f->M7EeP0NF6On!vy+ZY5y7b@tSOhhlGzPV#-si*`F;01%cr6TAZVCC`x!c|EWbj!9~f~|Dw6Y85x@2 za9QDrXBJh4k~3=*sh7Gi3;8hqPEs74VL50tqut&Bwz9}YvekTqO}=SF&p;1bExA&kMO7ui!Pc4Q)Fes`ycgkp?Q6>k{6Mb)HXA9!oE<5X1znlss2;BVDII;z$B<*IW9iW-!s|yd@x{d^eo0E^ZuY7 zywq(a^@KSf@Ga~ekySf!&nwo{Y^P>>ikp(7*~i^BDg;sNa&U~YT1DU0XNwK9TpbBQ z>D(DSDXMrv$ffubt{a!4Pa8oEKPVjC>hIxx-t-K|;Y;X&<>)EoGas624Colbla$cNa# zMbTTc9L>E7?EXh*NWyT->E8d_zB{Hu{k^atiG&FUOkXhJZ$yUC`}^8tF|-ym0#s|H zCI;yh8JReRr`c0+2W5K9G3=vxx|cm3&%0n=kyd(zv~2qAQFMiq9YX)o1vz*-F&2+(pe0TwG`AhW0@XFLOQ25`=qn=}6rgu}zc;`uS8-Sml= z)zYTr$VdrEA`_63?MP?H2xE?eAP71pz0~Doz|Y84J+uaW=O`sLN>_5dJsin%NZ&2H zj=VWzJ~cK+ol&Z!BW!#g#rDg&aMQh#YdRDqBr!DBZjaE~-ij>@ULj#2!NFMFGvhxd-!Ae zO|IZyjS;sL%=KjTE-}{JB&8c-C2$4_#@-ssUg7$Z{t>EB?MwWEn*UK^bbJyl9Dki5 zon-?wMhn9H!IYJrUzsR@vtc7{MgC&-K)J^oy&Ohh#`P5X@abBZQa8zV1d^~~O~6n- z97&gPps^9I&Ks?SR>H)oP2^s>l zFovue3`Jlu;e3Y^2U9IQSNywAEeuy6gTKy1HgZ#1J4^f~cQ7Tsov;m{CPRtc2d-R2 zM_WdYdG@&_o^w_cB>Ty(JC+(*TayTokdw4~iIZ8RvOzn13(MQ|C10crR5R!F5Ns69 z&h!QYqGtLpu3*oIe+Ju+9~cO_){~@={@Na5ceZ&q-4IMWD{hqf$t>;tdJ}VXLODHD z3SFoOggA66t$lC`I?sjUtn&kA5ft8&kTBjQu_z5!6EMzoq&A!}Y7KUpBb5E^t9twT z7M?>8<73zNFm}NxCEbQT{q%1QRu=TTV-P)#0*$!mhn*MB^fsd}^L8~;eO>T<8-VpT zIm>AsqHv)JEN`lxqTC zf`jXLD$=8M$V} z5)|~uyNDYnaSQs~)?Fszv8uD4ozbp&Tf&M{T07qMlYq3Nd~sD&IXzka2m>QGKuS)w z>p2{z3Zovrpmie4nFa|Ri~ldqOb+Eam)f7-N>K$+=}w}?rH^&UtQqoS>R}7{>2imm z^S0Pu0+{$$`7&URYSoP2JiGAMDH#pWG&WT1WDjTdx=OARzMUI{hIjPvo21bv=yVig zq9`=bpZT~m^5%6`51S7iR+FwPws{SldU8mBo<5>xg zyuqEgbN5K>nrY;V=>5dk6PMaToxr!uDU$Pr9aCmj$&n2tL|jRh_o2ZLV##`MQ1(~S zXXVZj>(C2u$vGo@`{LE-t`(Z(CRAWoWT+{~Zn3C-23naowNc(bj>I=1Xj@5)+G1e;xe{wxp(* zr8ue1IQ6D zO;cQ`N>u{H*^3!jR%=c-3iKcwEQSo<&JQQvin7cPVK@nL zNUHIQM2`huEL3O9DF0AakS#Pblx{N&G|m(E$L#Tcf)=RCz>HY^>d7L&oO?z?-L)of zMTHdx=tnN7;jxzN!NZ`e(8`CPxgHh*K7u+@n`Zr42ovg~v!GANA2jO=x18JZ5gs5= z#&ZhLBHpuB+dJ&Ez&#}eo>L+HQbUuKo&ru){SeWN_9yezs#Me~o2C5t-0!oOqVME& z!W8(C%+{6M)z4j`?9}mS@tV}1pLsCc{dkqsA-a8l0mo7Qi1x+Ky(`#VkX@g7l(yxV z@Q&*Ny`;M;??6#P$t!*)e}8<+^|*TRIgWj6QCou;s(tJ?XHn)pl~-!f;40x7GZ6kL zaG@6}_Hz;ny^{Y-P+#XUz~*HvB7pMTo&yziMXE0}kt#FrJ^IRcc-3hCXdhiP+$(wZvDRsOMgtT)w2c>|+G+qPD{3 zwYb`W<|ENj8)d7))&#Ft`aO^JVhDGdU5X!1y*byvecWvGBdnU9bv@?xd2eXJOG4!Z zn@9l#$LSpve?*fn+~qc&Pi2n88|M~kujSWEj1;s}(~9QDV`x`7e<&V0Y78mU=F|dAd;EISL{{$z;6>iwvnHc^{*og`6Y12FE^^ zC>K(=ljhGVSL%UE`qUwg_2dQV5IZXB_@Hj(7x=Jj?lI)rI=JvBIVrzvg4ABpPzRHE zgkTZBd@lYu>u%*&tpTAMicpFS>utsg&`Wc(hM9TLs6$mW{nHbYIQ+;Oy3db{A63hu zX{nKre4NZ*F~?z#HK^5?qc`!9^#^)KPawAVO^;Q$*iVy(>nqWRmX69=MFkKqFC<7j zvzIVFL_zhOPef7*iezO>t+AgCKI~bdpFdoF^r2xEmxY88ti|u@od9?6DAjZl2PWfWhV{B|(Y%VAShp zP0%-igWBO3WPhn6mI=Ob&|eiN+l>g@-uv7LxY{TrxFn!=C@#B zhZyAap{KB~)_4}*Tcb&VRwfo5foY_@UOa^!m%BKFg6i*j5CT8j;4!{=-(rZa17S2J zUBO3KTh?+#UBH?E_cfh(`m6S47fl^{HbG%-Mx$c9DIH?3qMIZNPOB@lnM8wx0#a{U zB@69a`PON=f`}RxQmfh+6g#68=k>aKgIga4Uvz({ucCVtA*XlLNaQ8u|M}r&f01|T zurTUvexY0UJ*&v{=5eyqUjAW;Cjv>X3__mK=Q{wZ_{i{iEMdX+{LrVNK_1Ftg@?4K zHT4rz-hHG)jXVM-<5wyxnY=^r+tjN_)$sF=Xv?q$0#7kEb+fP7-gn|Ayqv2ejcEW| zd(kk*a71m zeyTRF1AiEY-kglx9qF+Mya@oB=2yxr+=nbBzg^iD{07kiMtyQlJI;7twGvwhlZ&m{ z+TS0+=0aXqAh>I~p*nlnX&N!{{*E(B^vS)WAPGIe`m(~7T6q6}fQ;>Bl>n0tU-r)+ z?gv)1{CFKc?g(3c#6vY5csjd`o?X2kE>K>heWdQ36J`+*?~m|9%RSr_{#(25e(inP zLrR1`Q^Fdh9VDZ=aLF4_K6n{JVfdl`aMTI<@Kuu!Q4ioflVWU=&r)zSBT>RBsbvy+ z{k!O02oCj@kadW}Sp9FbSc?JFSGwq(8We@###jWmJ^D7M3DDwo!W%R_v@nlXO?|?j zD<`@X9chmb^AS}qNKhGSGlZ$Od}*SfiA$JnF#eeAFSVd7?$u;F+eXf+EnDhUeE>B7 z_@9nuR7h7=C1_tN<*~}yhe=)fzVUgOXg3&TW|tcp6613iErFh*PW&ZN7RUy=GLX!Rm~y~BRC(2b7kHkSK=d6-VBnoMY_x%IGIiyv6VFP#!)A|>{o4mUu(}pr6 z1RcQUC$Dr-s?Yb~@No5Fv>)GGhNmH1YWbC|VmGmXM9sIAZSBKkOCSB$OJ5X%ijr5XazXZ+FH@ z%%cS(`%2i4SL0yfNscrGyN9+~ZAQzFRTXeDZjOnzwriX*EbvRf-&TF+P-aFkomIod zbu5vpC4zmmI_hs^w=;{Qx$Dk^8uCgcX;0 zC?|R{`pT~B%@2H3QIeyH0!B!icP|YA-5^Yb91Wn=`8%PYi45auTndOu+dq)-t<;oi zUZY{tTD?PzZ!z*vjF=giobb2wnCme9??ORRMRtgqtRRuOv)UxK8b=1x25l$w?(1hC z7PL|}pIKtQoz&+J7$eJ1>vTPPS<~Uv!D%4ZZY$aZlF(NuJddiJ1ywb-Ki^a$ENAs5 zw`!rHj!(2do_h!<-u@xS)q&P)%fNOx*e*(ihl3*?d8%6_W3P5X89n}1Za}7qAKyr% zBJ_UH!7`stth`P76b$KZFjHT%EJaSG^OkaFWwQr#@IG3&vUiUd1S=^ZK?a zqafjX;aj@QK42NcZYP5J*wZn#wlZtc@*_6aYjxmoH>kyKC%5AkY~fPTiS+LU)g*DP zBDp_kX{jlVZ|o4l+nmI+_BoH7is&{cP{Jh*wIz%nQ7U&teZ{4U#zg67pr$bod73)9 zHGyV6qN(=Ye85nUk1|N*{lvd~pmKE0JCAW@prjyaZVB*?{jdQe7IaatRtmU=|E0UPmXY=U%L-z06!z1Pjl%KAh>;~|Smf?Q_pLIcVqS;J3n zUxIr_y`0Y>EmHi<>H0CVTXq%fuk}6`%X&v*^Ss8v!qLl6aFd8P{Nq;%d~!03syZX z^^V=JzZ4mvD9?X}U-&?;^_r|!m_-IDk3;S(zdS2>V5%>&!03jvBAkC_{)~li!AkWj zH&JgugaS-DV$-{~qe&MFHO5NM83{?I#@IAkO-Op5(SlvXdl* zs3C%^Ue*ZVe-Aym>(esWcU@?fRE-6$S@N;=-ld_9M9!iZ80~oY6#AOqL$k=C8@czG z-$8m^# zT*D#C(1lx#-T#Z{N+jNesg5@O>oZ?MpE(+tWx@`B3{lc~VbHYh-&FhH|CN%o{C=cc zIQLig&OC4cFv&NHR$B^Kw`aYg1T-xvoLrKI!6*V=qoH9dNB^6A2XTC$R6hapd3RoT zndRHx%vq`3-yb#Piyco2>-v67!ndG9jXCQ#ppI}o zm3!fwHota2`6_XKEuZCq<|_w1_w|Hfvn0J~s2d5_x0Sa4@cI9UFV_HQX9Aihe)xkQ zQ|u|xTa&uq`~UOOzXYO>8T41PXIh>^#b8t&NH&TpTQUPn{znc&6Ecy2!)5yDmE^gt zC|g=CfrmTT!tn9Vzs7)n-~;JLf9J6E7<%>8sgZr*Ps9A1AL(mfd7d1BNUoW7JR^*@%Pa`hpj`y*43Uq8ZH2%?!S=8 zDIja>{__Kfpbuzp(vTQ~K0q4$Bb(7mxeFoBz>S{+icN z3sOV0o;~&EvXU@%{hAiN^@7}?|KGE<$xDOlDPJ~wUyT(Hw}ffjzYERkGi%^~)SACl z3`y_-7}*C=?owd*jy4C3BdTn|o%7$PDPu?gF<{?}PF4Xxc$sd)dWNdKrS zhAdQIWh{61)@UK-VBM@P2s;Gqq*~xd?L;a%YeUxgwu<+h?xx_3l|2a1@kPC!%uTh? zU40P8lS;4z>hXT1x_#Q!8wR!a6DzB?qBj$nH+OwR(HlqSU7e+-#$CPr zxrHD0ZkM+iGFNJ6P&4RzGXOE-*9KI4Py&i3kuCs zXFlz@4pTim4CYyMbr(zDcOQL|3b3&n&pQZPqWwVsrdvx;vi@m$8H##1gUX~5pJ()B z@|OTzZpOB<9cjJf+pWR%jz$`2K2v%A4a=L~gTQ-X8HvwQAN(&L)PKn%-MjUN-HSnL zj9ZVxFaz7YSG_`**9+h?ZCh@y_pxp(t{Y~D?Wdd1y4ma>9z_RD>Q@|o2SxKX_KsmX zraAsgzjgAz=@A^Sp&cbSQM;TUwX`zJp8IbMF#U`L!#-|4+E4MjU8582{{``^#I5Pq zUlQbH#qgoZLgG#&{u1gcwQN3~A_bXBva~qg$4U)(Hbi&!QKoC3#t^(~%s$hGMMK8vN<>=nkQYc^F z%|7_jD^lMGU=LEs+!m^4y>G9bTlhes@rKaSxw2nEXNA{kiFxNeK3kE)5EX9gB(x}= zp3a9qD>rsDT3;W7-D_yS5Lfm`;9mgCm|4{oDV6V|qDCXYB?p{34G@cnq^O-}r0`K6 z5_$+o5NP@K;oZM4k7gQlUDGUu%0h6U>sp-vLi#5rEN#32omisH0wDXCpcX4*>Eu11 z9Vl=#Fh$gz=0q2Meq;3H<8)m)kB4*8#|4@=QjjY9@wf(ra80|CT2AuKLF2bRG!ZG^ z1(Kz+o{bO{4`cN+h)6eCXkPbTyR}~G)$Epk+z>kSSiVbcSq}=dxAGgmR}VTZXsU|2 zR7T|OCgS7ZLSZJ_#{^R8H|N7Iye6KguQV^7md{)*Ba701Vm>1uHw7d5=6Sq83nxB6 zcBj6-ZhHUCDw%n}Wlwxy)GMli@ulc!_O;TPkCU!UAn$wQ?On`13+>>uLAdKbR$oek06Z-0di7lJgP1<1xT4G2_N)wJetGm* z<;_O}?FP3OGveA-VV~mKR`#e3hP{c;eDy& zF?B&wDi7tPKI)a!t^0MR5~OJ^x>hu-A}C6EDYw?{f{ujuJnR=CM%wxqRNDcvr^pYEx)wHdKX&Qk47OY?=inYa(kt zYc*?^LgPdK3Pt#S_v^8@($XiaYV8F?uvfKGz>$#fTKp8CD39y7otJSl(k zb+`VZEJH~oWC672NCcTU1^SK97}>}PuNuV)dqZ`|d!Sx=gTmG-M4bRUrGGx4!8_`) z!IcK_$XR*GbEabmTWGz)32IAr&of6jFOG#K{YCo>pF?x6lT@TIEhUxChpm+Tf6Guq zH3VB2jupYJ{vkwU=g&9@aJHYOxZfAG3rJ>I1+T)Ljh^Tj&mPWab1)UP$|B2CNx=N% z`LiaIOhl!t|KL1NG$%lkl9~X4+zb|jxksAE#e}q7^G<*qfacW@__N%~W#SLtjPu4F zJvV=kmmRZIF!~O&GD|kd4725DUB$hOo09@8-2RxQ{L#2_oN26oOYb&nXjy7#y*Eib z&?bsZ{R-7+&XEgstpU|vyii+{)Hwynk^YOyZ`VYgSBZ9l79w-(kZS5JP2aVv)uOD--2;KgnwZu`dmj(ELuvPS} zz8<*;`2pTD%9tZap8*zQ5LZh5pe53A*#|lDM0C%LP{NmgTmW?#iienIdj7N$0j0Pb zY}WL90KQJt8#kWKY`KBXCiF_^9m^Q?uFn-oK&uO=DC0}3r7QPH?_64=q@FWYF<)HH? z{y--$rpT1_AHwuKq_x!c2`p4Y`pA_A$uzb{ODx-qTF(`yaWxkjY7tO!&W8m!-?2L? zkn%5%L1lQ+x8`5~g;ORMB(NczFZr^%4(U8j2w-BQ6^NV`of`xPxu6v2uM1_u%qm&F z0kv)Jxyeaa_3iHv)XCtDcP77du^oeHTcz)g0Pip1lJi+%msKYssMGoYJ z^puXYDN)!nV6G;4AR!V^Jtp^bxl2&}P>bp<`9FHz-x@Ll5C0RSL=1Et&I)Y>&Vbf)5SSi_|tIbOQu4JAY!D~HE zOf(F%nFj_~-cOO3sm{8JbT8&)Y5?@jCOY%?Kghcj(yi}{2N|eCKbCgnGaGI9UViv! zMnfe1t_%@C1Z*(?HT>V=qI zrv3l{ApixwOvCcrk{pJ*N}hDi$hPIwA`rhm!1`YnB-rZDa6VS3A?C4hTt8+|hS1?} z26~z&yE;6s*4ey?^SJIE&5%IKCDs6&BJ;_kb8o$t>M@$>N52-hHV?m;MCmO*vRfk3 z1qNzSi_Y@l?TwrcP4$vgGxMuQKQwrasjWULL|(RIc{u=Os-Tnc_#>O=wt`JF$?03R z3{W2{;0LtOLq8Wy^GLt{(v}Dit`Z_JOCvs&k^-*A;qBUAd;^C}8a-;Y{G>ncY@?Hg zkgFh0TXu51N88tqI(@8-kg@)YaTwwvKgux``MGQj04L(Mb^q*>gf4u&B&uKr6F-6U zM&}C%D8jh|l@22gYKC*9j@L5ogD9J}dZ65bUw!>^p6}yc)9!;s(C>0NkcCk*vd!{? z=dek{Td#kV^S{kAG-zSy?^Y`ZG2wJ%hYS+(xp0m`->5kt6Y4s&$C^5*qlW+yF65V& zzUk4jjL25F!9m6-AZx*ADyrf>@DwS;vgl&fNNuO4wYxt0tcy-0W>snRYMBKj+xnhF z@vj75?Lh+EcXM6z=mRK1j;}a^K<-{g236CpPFVJ+q{=C~HB zK<2h{vY!jPsm*Ij4f+X5pxlaINNvEkZl{O0Ph|Y8HSz!k!IJN<3ESB-P{k>drB}da zgAscwGh!j=;&cG)3o9ZwK`i6SDRrHD<3@>oT@qNCGk~$NJCVkJ1Eh7n`~hfTpGl0s zVlFp+*Zv+fdjE0tt+105qcx+tV%q>`OS})iSj{0i)jj~{X0&g(o>#w9e)0@QUqjVmR3&=LxB z+bJs8u;U1!w&Kroil-{y6umuHq71A406Mp2HLjBFe>n)iv^&je5xL%0*&MBj3?1D6 zgHO;Sy#HDa40QZrFTM{DtH>-C__g4iCbaMk^Bq^$CL`1f)Z#W>$PqHo+hGiaAu2m(|^WJu_$!ypt$4L7?kUZH771NfC%qiYZbL`Xb>5 zs8}YJ0}QfGaUzGGNba10*?v}sao!vNQ4Djnd*-7`S?hL@*DN{!@?O8rF8#f4}M z4(pZJF|ed6<&aG=GtYzmt*a*^0V8c9YlD4|0Q?xN-zg@(1qRobydMvz+-ijVrRdAB z--qlU4YIwkbhf&bimO*;p0iM->mLN!Gm}8MW*r7CerFCy;7xFYrjWa^GS8?pl=!~7 z#4i7|`L{i&4VeBIT7*F<9`e6!M#%r{4{abglfviZI z`}TD8<&ZH^Zqv3+@;_$S-*;CXnAaUF*&1N`x1PIxto*ko1%j9ykjQUvO0VmF48-S( z$qv!KFB{}4nSi_p4<<>?XCc#WR{b59{l7mTybTSq-qA{)5L$4931wDSKm7S7BA+_Q zWm6YWJCTA8`39tU`0@UGP>;VqiFgHYW{j2WyfpfAEy*U(laA~BHepd1#wAbiX1zdy>A%|Hz(Egj)p=}bG9Jk9dfOcBL^JDEltNCZ#5F5}1lb4zt zM-m}7rox@#+&`3Y@Ps1dI%E)Yj$Vh{KI=MeBaQ#2O5i7hmcmpbqL<16UPW{_+Tg!W zeHs@W!Zhw@Wpfe8CrEy}tov`S1JQ}1Y{R`1G%(qxVtc~rKX*t#2J9WFMr2L>8F-DR zhay?-pDU3ud;lg<^h@nz)&W15=Zl?t{@glEBj{*rrtOaEMpGvuQcC?~bBJA^L7S+bzazf}Mr z)7rSfT9T1?g0$+PMb$sw4F7ZMDm$R1bTCeK6Gwmv$Ii~m{(KW~a6lr*4)()k10L+2 zv+BJO|Nk1)7Np&D?*!dvU0D_b$!`NFDsPmiQq|{9kN98>azubga>*k}$qLogt6K!N$u?M-5TkI{UgnSs9BG0=WC=y%|v0 z%l+Y%kjbC4OXTwa9(dDK(g6SBHejv=alh;}3nnz5bw$}HIGT?Q(@y|zE^E9x8mtMy zd-n>UTiTboyxKMz6Lh{)*(Lrq-XQC%u!@n4>2GoL$6gOKB>wB+i$h3H`7;ua%ldOhr0VG|sk z^#piPq^AVqCMYV?QBemJf+7j?5Hbg|<4hQQ&hc8^0}&CuY|4MQ4_9RV_hHMxKFd%Z zkfH}i@GbJQC07rV4x$jKxT|~Z)Uaf5+Udx~2cO-V`LxMXP~XwcL)esaaLhoM=kvO~ zT-tP|IUD98^KE2i>`nQTJsEaW+-ZEfe%C_E0b6ua+P12j;GA#t72O~T4o*|l-FE*j ze*Y^kRvgVee}t@oChiQxcouu-q8%s!#973K|rg(R}qb&OG)7};~azC zC-`f|g5f8LMH7tNU9P<&5nq9RC)fGFgNu!Ir_l&PcFNZmHh_?ts8@ z`HloZ9Tx^yNJ+7>f5t4b`KJ@_w~v$J3GO{mi+qIbtrAOzhZ~n=*6QrnCQn6;P%4K4 z`}XlqhHxAxCMO(!tCDH}22+~fI)m+}p{8u05x`kG|M2H%8ACk^b|zGuV_T4H^lfPn z$wvemckn%Dy6S&3H7FpG49|_M^$TKZzg(9||GagE2=<@p@v!=*Jk!BsQcgFG9NMFk#CBin1Ojx#V^uz!=*A6MNl|Zv6nRwT(!( zW)tTUov+2SpxMJT<$yxkQeNB2jmo%qD$bF#1OS86Amjf~_2$#f#V+5x z-3PrIVEXd_rZIfH&m)|V+zR~MX~3*+9464QjY&Z6V5z?ZXaqCOz%OnyB6@qZ0hC{N z$ak|Lx0&UKX6@UpVyXMEjsDYN(bH;B z^2*EC(0(s9?dx~*#+KrTn+M-WKoDUAvJ8CF?V1mJhU+Rke{3|(e6pAHHW>2DA)u~J zycO7MSp6%?4xIEg@U6-H6B;V~-~5YmJUZ|)K11B_X~0*2$(aTYVs#i6L>imWULJD= z_U5<1_AO1b*_KI0|FDdspk@K@LJ%I&^zjW^#L^MF8^jBAbbW6H@R0X zn7pUQRdwk&bOK7#-N^XWY>{|B{&A1=+^iF+==5`Mf|rLkz*$STdI8R|28t&nZ>H`7 zujnv@PP5U&*W!56MCr{|PkI8Z`)qgM718M40aD2vazFFVd|W(Dg`5`8MLk~wnRQdu z2pDEY*cEX2Qh#V%j2C^k_5#>IWPhUsGWFBo8asTP|7stNl)2+tW}|0w0Q?7a+G~(3 z@}o}1jK$UP6pg}~K;U7v(B9UHXKn$82~p2%po#XQ2h;(k2PFpA))Cw)0_*WcXAoqd zO>^dH`=I{M>h`Jfmvu%DIHKgNh=hTIi2Ulb!uuUhiW%VzI1am>wSM6BWZw@dEn30n zSp5Zp3cUOPVij3Vj6Cq&CV7Yyu~xa1JO&g-!6GP8kzx*xbM$3mU_G45xxImqA(=M; zw(^Iq!wT&>Ya;Ji+Meso;K8C}XTW4d3(`8cJ_zijeFDvG#Pc8Rs74Kx>6?t#g^ zO0I4O1w+;~**iaudhShp2eFGrAgEez-!zO%w}||#iYlYGNp}4Fb%o%A-eM-6yLLhEDr4%fQCh0~rXh|r3wh>jDj!B1IKO<2Ero|?- zY2qD^MHAtG1EXa6RYLsCM(fRP>ydMhG_#7l8wdiL4(1)X1IW#O3E!qXWfPqoj9R)3 z?+va@E6sOZ*YmB{I>*5K$#tub>0KF_wIsSFj%9uJseI>+ewEr+(PcO+%gSQiJ$tBs9L9}$1{2t%Czak zEE8i#T=Hca={duZ+Cw4ZcY!r;7kkR@+{JKQ+fZZ6m0 zm{()e6c$tezBGnSy$U3dCv84Ax_GU|JAJ@4=x6FB+=Yf7nD3-nl4qwPPzD@}v`bN< zH=wZ0Uc6Z|#)nK}PN#MGtML_A6aj8sy_6R5U7inlKPdf7eOrjIOB48GC6ad!KnRLF zWl?SBS6|o6m_q56W?4bRZLjxLui!xUoZL&hseN?)1K)5uf_S^+_lQy7krxb(Vv|bA z?x>3~j!ER$vEG6Lejc$K#Ca%k8C`!-_BK{U-+$c{R`ulwfhp*vjNvQ{DAA~Wfwi%j z$j0$8yuSdDnt607N_A^3-y0u35KjLVtiubc(LHGEMF3!?JmGQ!{_T_2?aHe3#Ye!g z86!ts68Rk~o4$-O{N$sWevTr{pE@iBQF=#0mR@WwDsxpVI>n7IH2TTnguE^*Rs;1e z{iBTYmM`#9I)UTuQ3AJCFAkOg5A(cvH^OOo+)_B2r0t^xqLppcU*T}m>f=kP3GVOa zhcatDHVa6z3k}cAT>x8BSUWQ%RFCl_Q+J#6y=y{5UF_am;Sv<+@&nqd@M?czget3+ zp}BYX>dUX#TN1L^PpkbO)bHj~4}DyJ{q>9!R%P$tRxD#^4Jl1NZS>zLQyxThQKuYZ z#TIQ)kiiRxS$>3qSZoM2Bx{QFrVEZt zB@*nb%NK(YuFzq~ZMWl2tcM!%abca6xWZ-5Q;|1w45m2Iu8h8Th}FT7i1FQ3@%_o& zqod?rVcPu{%JZUun%!?RFD45DPcK2}l(I#8)om`6TbB$ubF1xbb3*9HtI7wZ%`@tR z{9JE8W%pyaMq@?|mEe?0)H5_g0V5|5`gp6b_T|i*?M%)p%I;XFxQB02<`;aBA0}VN zBcxu%jY0_vezi(g5uS~4*tpwy=dhmisJ9BparX`lV=V%1l--)1>Ngg=lS&$rITpZ= zkEmB5DQNC@;eJhWL%{oeUMBg`)j_B8n_k9c5b5q*E>iK_$(r^G&O(Rzw1QBaexu^H z4d&Iyz+90ck~ww@)VC&_U&*O!t~_LCtd~fhnJ;WOO?V;nR{e3A_TXH_^70 zP|qs>{_knew=FhOg+}Q}zpr_LO^86U`qWNQEw!njOo;f>rA_#?P_OsYO!7cAH#&ZW3-^Ja)x zh!Is$7biA4p35{`I(c;p{oM?(;+oT?*dk~(YYxv}x)_ldtxB#s#A7<()e z&ma=&X3+=zlG{lX({p$YT);6CZ-9eQ(D1vdh`Fs1hOw^CmWs|!pu1tDk+9Ny0a70$ z_AwMm2-<&qy`|k6>emu<3U8Mol2ZT4JY_TDg@?(m0AqMmEnjJ{{LuZ1J#4Y~xRQ9E zHH%m;(oKd7JQM28iM%pOtsXHMlLazYq@mjFy5TUxa^5N4D2|69%z-uPjJ0W;-uZnfP^7-4^M?Ad&!f)@x5^cB91^l#7L*y=nJmZ?YzMGJjl(IBX{VHLx#LRe> zp^5!1*c*85;YnLl4St(51Ka4>?i9*zh)N{%%V<}u3y=<9v1uq1Ew8@8M4!0D7%#O> zRQ844<(Kfka*jgo%x9CqP^xt&T$MRjvY?DtW&0wiidIiw*adk^9V~~jCpP#bNbsJy zfGXa2!y_qD2sy*}KzNLv41aj*iAxdrrzp1-$@7g_&?)7ipBZPsCcP_&_KV)&JjspY z3^~$qPUdA&QKZb5+#Y{3XH` zqC1!7NP{Md>^uRX4KI#s;5zB!?8ykkBWd}>=riNpny5b-aeZz`f7dj8p!-qceMKvc z{HI?fzZbEZkX{8!$zA3UB9J|nRpbf;)o@R6`w6?0w`t!IQgl3TCGFA`l(ihs*{j@P zde`#=Wtx{x*9Pq+@p~Ijx0FE$0Xb}LIPduWx4M;bUi^9CL>9+~qL<%m2x~P0#y;sj zNUS0PO2ek?9ubdM3MDL1P+IeNs=YWO(J;L13bbh~MqEhg)84mTqfjzWewX~Mi7 zs@qm}&<6XuX5UA-uX)$}BTu&#o=V!#CW!y4clS;_ncI-Yx6juZ5MPcNVb)^jm0ar+ zUE>OMb%JI=N{pS@f$mSxU#@YC>|d#TogwvybE+D+-x>2*w$bTfxa({n zi=PC+r{c*>y@V;$P9Cv#QIXMrCDLI98fcjJ+M$K{Q1g5PlQ7dZ%wk-33=Um_!<` zJ3%fEE}@@1=q)26Qjaj1OWdqSJWjlIhjAm^G09hYbX3$={$^9D&w2c%X$Ma1Hel-O zJ1zLS9aHtTmid{#*cbCNRq;5O)cseD*D+~uz6CytQAGi0`rpFfC%Uc1yK!lov`|ph z!~6I`cR+esi=xOAa?Q23clYJHbwUaEHSZZqPdKMqx9)1f9c0|+L3o{EROi3VI@&k_iboLF z!@gvBsCji%=J={n=BFMdC=aO5qg7vV^)OsJEtRRuYDt?&<5h>(%Pw{0_&8`D_QAF| zSht!n7YkH0HY|2}&+?UIpBR(o=obp%OES~OlMj5^h!ogJ{*Gk>*9Z(<7SUX?sZzIt z-kpjb`W~E}s2cUY0+f3Q?e$X(`BTq*RNXCa!=;ypX9-WRn3d?6q)e=SVDCQ6VI_N( z#*&BKlq;e7^Oq_Jv6~dex2xKPa475WR+fg@&$)jCYLRPC&^b)-LLdL8N;fpJKJQ1! zq|LBV0>aO5H$B~%i#^UHrMZfDPW zhR5QdlFkP`2V9u=Lre4%;+{;GLWlbA(P~AFh=_qz{)dRT=-ru>wS$&&+0DIgW-jvW zLh_^^W_yZPONNeN@{~{Yml^j#MI2>8s07R!>0n zX|9=H9y3S8pJqVVnc+I)@afuvJgM5iUEp=JuiJG>&y)!?((hvQ)ooEayLVh0cN8Dk;c?JS1pekF8BHK##DJ>I+~TJSFm!*>>}<-q!$2Fb7#y~$d}<%j?|DMs9NlDMZmLm zW1Nud=*>{+U{luKr1!i!+Ie0ZnVTn7{>PD-HnZF5h z3xJnZ84#e>=JQ*OaI&!@smZKFDtjt3g?U7BHl6TflSZBwzN}>Ps;lm^HXxkIZwawj z(B)NVjrHZ&_Q>}W+K(y4Qs_OSGsX)L6~rg1tFjb_iPlX1;7^y8^hfX~J!dAHpNH1}vLH!sgtws(b%r_x5Z)qPz7CG=itfN8Rw#cbp|K_bHat)1btJog%GAM!BH}IiplJVrr^l=w@0AshH>HztGf}dgoOJ; zI+Z?ynUAO;<+8J&*>B}I3aZZ~)U3hME=wHMJYBrSPCBF#Bu-Y1&^Oe!gX(+C93x} zT)dcVB_oikuPQ2+V?C%h|S9nEN!Ab}U`HODg(bf%H;siv)` z(RGwN6n+Cg=xT=0vD4zCUSfwJG6X~`gnyf)S&y60tpqKl0EvHOc1bSL4A@<1n&dkS zPa8Df)|eR4C=K+Z>SF!z-UZvF_M9Jh;YeW|3ST2TLr$b+4Tm+HQ`s0u-pnZ=Ph%rD zv#}h@x9aU`s4JBkeh|>ohVqegmZcL(>sXA!P5OB2m~hV~ey;MX7@ey*t4*ptTnDVK z-d%N-|J3g^f>a7?w80u(@Ju1Icx0NC`E#GN1pKVhD!y9}M$cWl3ULnB)eQB(Z{LxR z20HYIIYy&d3H_-f8&#wX(}wOfW;YTG$gobEk<@FGhJVA)5HiVPXYD`OV!O%62arU; zvKJq4lHq+oD(@Y{4t6FKl;%`=9-d>n1 zADy?lTx>g7`G-+_%BL%ZY-T0N(+nX3DU?BMHPXbw8<`f^vmBDgvQn+#T$)H@y|Yf@ z3*Rs4^mm#!o|&tADIljvY3>M@PY#m5Z?#q^eG+3%!jZ#X_eKMeHc-0Ca1h0xbbW1n z`fmKzR@3As^^KfvZ4wyAAQ7X4gk9}Ri-oL*3I!Z4sqQ{EC63%bh4Am}Bsq$g+#X&L zF;v?=FBCwr9Qa&c@sR*%j8SQW0}XWB8kE$u?<&4H6$(nKnhcsoP%s_yMn)&++kafn zdu>Za%}%b-*k?0rm1gp++UBNdl5L=|wBK6&ZK%d8ki|RwI)0z@K*_7U%q4a3S;nh^ zbSFSfj>q_iVf^0XaT$4QM-7Vlv)73# zSKl;lR=S3MlC7+V-R0>WxVvwaW(TmVOZjW)3<&KKB&D?O`Jpge-gS(`!_;()G9&*b zgqF#8Oka%zAIv{kVG3kh&bX>Jnrek(rn}CfLUsYLyv4eT^VLC5cge(rNIQb++K{zp zCqdiHhg0g5hrOYd^)HvP6wno0U|Mq*&f%F$diX>EK9%B*iObqdHgZU=MQ`jr`qIbQ zP*Ix`OG?XD3WggNh1Jv1G*Ul{&(zvLT>0$f$Pk}SjNm0m6n$)UmRWJQT=@VOPIG?4 zpZrG>TL|}JT_eD}v1g^avARsC%!rSs7}jWBx!-wT>p(&vFx_xhMM;24k(9(*hu6Gl zo|T{B#vMkTL!-l;ja;%-IsJq@N-Cr$f9{p?qtC3a+hIIic$@xKmxm4Rp;rol!Tw!p z@4<<0FOYd!F&PW(xKUGo>B)Bt6RUO-MAKur z+M?rSM&V)B&Pgr`h}o0!(hoZ6RgOuG7CS=LPU*vEZtEJ>rVpZ|;g}X|nZ)1_3L0|w z7+^l*!2(>{#_s}VMSa;^6&Gv?SM(yH+n7O@zDm~8qydFBqD?*u%W2X}Bdz{5cFj7f zm4NTdTz;#%8i)xONzzECqi1M`a48a6r|Y?3(H+qi0}L}VvO}DKq0xm}^Yr@loC%u; z(FNexI|d)zvmr}3`YcO~ zRP%*00x=^)n*hiCsMLiZX-;{5)&YP@)SS5WGtg2$3u__qLB-LO14tV925g47MC(L_ zbrp^HvCxUXFanw0B=@fmj`dL1omTzv7+|IEm3O9TU^sLOFF*=rXKC-|n1saTlB(3a zE(t6<0mFaVv7~vj3Ns?K3)@OrQWjg;Kri@o;MlP|Y+Q*n8hef~q|!&CHxC_7%~`YX zE#f?mHL?=JkQyGY+S@*xc`udq{ARayB+8(MLCLQGYaaJi8?QD&cNtdQqxb16EZGIr zti4*YThz`MTwV%vM@98!k`npj58W(u=J+2sa-1i@OQm}}H;v!T$#UBBsqY+nVWV8~ zK0iP_scGhwPY%^zrwG6#DG;RY90p{K4|B3%6CO$`juB9}ntjbu`Ly!bU@V&~G_U=2 z?uIEbtwiod!Ov0!PdxkII%sM_bntpdX<8>b%-nKcGa=s7@gH5Fxma`r9oGJmErQSX zleIHo%jLPG+AIK2 zB>9d{f|RUfD+e-RgmbbtSJt83{uaPcx`@3KgfaE=7!NDxa?4?$Fe?gAIY&!Z*D=`L z=JFG;H`KLxI4Cp+>y)|#pQi{G+hI|7kbt8#8M=~be8q0I@J5LUU*~8+hPk`)@Q#dK zHv?*<>iM!BVYh-~WvudI<=$rj_`2(cvrRmmv$C=sMv*-QCD}?19Xqf1e3_v*FSm9X z&xDF*DFBWw#t_q?n_*s_uvLFQ0^9?;Ix8Xmx`^mtWGe}x1Mm6#DN_H!Ep`iOhVX0E zebtTA=iyFi30GT-(&Foeil?`-!i<=EHQ2b6SOUk7%Oj$b%`nKMWuwbcXy@GBpDZvR z`wmd!!k3V$|D4snPqvBU!-baFg7QTB$8u&%@Hkwfs?cZd9<7@_wIzRIzNF#t3c&Ix z94+^CYa_#pQmg}F`2qe{vMu7u-D+%2R;ENN z(TKW33gLwAnifd?YKhLe+k2wQ#cl!r$s`u^ka$aLM86{r_#z~N*TG@E?X zK2XPQYI3b9P04}plyn6(pAVGCc4E}<9X&OHk^~otiCJA6(t4GU7YS*v$YKzPX`z^7fA$n|rc+V6$KVHEm zr6=VWa7R->G;J9;05Q=Q)Xl_7>{5nUzT3AMl9^Vn@ziWG39k+?Ynos7*pU8V_`%!v zqtu=ow+3X|kde)b!7bjy>66C9udQUaY2}QUkia)AE>T=n0mPnWU(0b#Cv7pPk<0wJ zeUn_es%5w1^zawo=PtI-knxn1YplJ$dI5}3k!0u2V2{Vd2R^**52eGQdwBUFkp6d4 zu-Ap#+V)dC+RvwFIy@?^k~^Gxy7MX8r%@zOvneh-g8jL-3l5xP zkm~g?+h>_p@{;o_Iyu#R-0vhd7MqC(M8&ra`gl3k9Pj!gDnQpxFtj=HrNb1v8X?*% z%%P_y-K+Rb^a-H>LCKu&1VUZdP2+VPN<38T=zBLFc)(|jG=)|ep`%Q-F@z$kgNk3X`!FJY$yr-{n^()qcU$P_QJB#Ja4)^%|8_6o2&%50Y*9%Dm#}=o&9~Fx zQdJ<1xiLkU2TLU&qr$U&N^9&J4!0u8#h2b?%{PZ>ljwmyIqq|)m}b-^)b3VQOhFDd zq2tvzv-e-%G3Z!((E?RNwWX{UpqaIDP~Gs}9j71AYPORuNHx%AmO5qsCdqG6182;i z1acy~*?b1>(EieO;;b(floXc2YhU!%iR`!LB=J*3mIvf~wRCGS7I(>~8Zw4*q-Rn| zKeB1~ZS>335#(o0BQ1Z>4wrVP`CSR9ZB@~r5XvA-#q%-VwT>x$vRDOr1th}WrlVQE zP0Us@kfc!`K?=aH!`DXb{9&TE-z|&#o9x2P8TGIxUnu!rjtDx2e1t=cQEc++TEZdW zsi**WI&1Okptkb#0*8}RP{EkqSFA^eQ>;AG=ueQ9W*Lb3j7(dtr_TyDIU!_b6VCOz z$NZH~;$*u_;7(S0QToKXrFCBg`rMGq#RrrdN6V(_E6;E>j21gx6;3Cw9xEeyiNq$W zE@#Oe88RRXF&X9VuN zPx!MkuYd%v!o$$ZWCN+I&zz7MDJe&qu~fP}jv|P~nH()7exOG;`bPeGfzN`G4VH5E zR+nH^M>xm+KwdcH=dif-)*i@EvTTan5Lesk)U;+@ntl-2XibGov0^{SbT7+$CdaK8 zZp>DM*=Mz~7+n@;5-H++#&q^iH_Fyg>7(F?rPoT zTVHv=>=UgRo2)Ierx)t6?_aX~`&UrUu~nBXBsV>IQ;A;|+fU;f9hAOgk}M?31R|aN zLm=?h0osrusK+1{7Vz)tZMtooX`Kp2L(Bo9Qp~2P_&>rO4d&dHV;%$%y|Bts<7f7& z-k}GJKYmTpZanfQakvk{_HDI(Ds4&MavGhKCl;p^H*EQBeWnx2=4L@ z8KN zey=1*M6bX;a4W&g_?g3v50COByX^`ZybT(*J~^c|Wo&?F9r|=(jH#6ACR0j|m7B)x z?=!7oy^RU2M>MYe6_*~pnvipt?LIqxIf+!>#hNpIJRz}!!Y;u)oUNJ~D4+FkEzdQ;-PRb?38mr?r87)l z-R7WDD;Ym8S%pJCzl5+br-wHkY2~#Sy-Y?Wu=3Q0JvH}k?)#bc-hB!tzyaU&eflBm zj6+V7^6{JqGG#EcgcdO!l1ClJ3OeVS+D4iY|7`tCqf1qv90ydNZ2V26_wGW zf{H5kZ$CtvB1o2PSGGevZh?>z>Zfk|d+<-^t)N;kp;wgVN(22cYMA|t_%b?!X;PG2 zRd=U*J)WP6==!Tle~ddzKdiXoI+hB$(bjgyWzcbSx~*uiV$vv#ZY`0rg}oy#YP0_7 zz+>vKMIgvLrtEjkk|I8Z-;0PCMs4Y)%IRoQZ#T%&n47wXMUjIu3w6z{J0g2kcM+pl z9O^Gn0W*VF>kYor)&20M>`e{YuoDR!@vge{L6o18g~gKXqgmQ-LwtQQWYRX2`tPbz zXiC8ld5IqF4y!~_Z>!>F{{sE3S+fa~p1e8=mg|mIDkuc#tLkeISW2GG$ZcRtk&C?W zO6OK%=p%_@i^ zA!{*y)F#C>v11X*(wgUq35w7>LY;!h6~gO108~~Nmpcr zG$SR8$d!9qwsZ+h?m|%o6&tf8teWV$9qLnHLclhtJ|ww)wwE~JwcJ5=E81mRE1sC*eXilIK!&%SUQ`+O5ezgBpW znjGDcB$M=Z^JYr=Mh$ktmq*z_Y)!Af=Zkxg11b@3wl|pEh=8uS`B6mz7T>AweNhiP zLQRi)6qBK!vXg!#GUw zJqvmZ@Y$d2ohZh{T@KhpeSY&8y7i{7_)3cP{v6s1cM)nK&89_81*Vp}o)F zk_`%li3Ba%FA8IR$plJ6z;WH89_Mv9;5gpo{$?yAwYRu!uJ>`am~P>y!kXnVA(u4% zYVwVSI|}gYbED*^^A=&>1^g1hZvqY2;5Y+cvxOEw z9-DvIAlv^AfEk}Kbi@{7Sj=NA2IcLSdGYqnpjK;HkQT% zw!gdH+TE0s$OkKVvA?A5Uzt__v-p5j^B?Y-fRIkg*tTpePtJTScUj$Bov9-1+#G8N zRjUu#lSrIj>MuH|W+1ZWIlE*?9X+BUbU)sz($i-C*#9Fb4UkmMJg(;pNQ&GxM9Mvz zRz4uVAXYf;JLNu)dlbgOk*~D+U%Fs^;>g7LQvJ$4B(X(t`;z1VE1I{&Ap5RQ-W-=F zY7LJUc^`nfh5`B@8ZDKPASC`^^M?m)=fgdqWc|8v6DxWI-;Lbd1Co283#>}4hOifR z0$1F?^7`-P_$@4n8Rw+sOr_L5+Wz_6$u^}9t;c&NnA`=0SpXc3JeS~680VNgh=+sG#D5n zpQ(_LoVbt>k(`~ik*S3t7?@Z{f+~!f;viaz90@tQy9&Vg5vhW-$#x`9UfV5m(qm%vVyeb9d1HptHG{mglh#k~86 z7Sd;dDpD-ozZf!_dIYOK5Nf>tQ^B~nEI2YlHljBfGP`VVcw{74CE}Gg{fQNX@Pe*= zwAqvWV+SS@0e=(%7&?+IE`3xQ!Wl8x>F08>bQCc8p)}cHEOEwC0ZBy20GF8e#sjtq z?+XUc_)?@jXb?6q!Qz!TKH9zmD^?|vrC&1=K$`Xn&IsV~MFnH!>TPdoQglHr_+kNR zDU2$4g=5OU_x(|t*-|a8XAt}MdEG#$LN<|Xd{4@mcmAe#($dGlXcTHSWv0HIstMMf z*5yB$=yZLbQE%WTNq!Q|1l;=jP}Brr=Y5y>g0Xd7NX#AuZU4@J$Z5kCqKB9*fI}@A z11D!~ass-`B6uBHumItF{tWd%ur)Xf>wQ7xJWVEg$?1=of-rR{XzxtY#JhxhH_Abo zZ=Wx9jnC)dJ*DaxnEBxJ;d?Qm{d&cMm5H53h9ePF=fbH*zKbBw2Y;%XxTPhE+(rgN5Nd?GR>@XqIIR|}+Jp;BkCtiwX-V-T#c6#hav#2e!qHlQi zT?oI6{}D-B-IV+y-7Lc`qkW+N{BAOD%%n4wB$XKR`}dv6l+%dLf|?&%{V@*46vkL! zzfQ33HV>r358HhAYCU%`AHjXb2D`etey%oE#(^E)_%yu4oTMW`wsXaUb9e_(qhSqj zYAv9yDSkm(F}?4^PKLy5Brsk>ZuRAdh0r5{@Ad!A04JISchiI|1zFSSAqO$kX;+C` z3X|5UQi*X04kG}`@lM9yehs}5!pg_p=H}7|E1R$tG7Algm{=(waF3Kc_>+DV2PtAN z7Tgz-Xf-k%aSUmw+@ECkJ>p9avjSnR<62^#Rf}yBzGtN$54tjEsxK9FuHyf{mvU# zE(l&oC*2Qc)=;0jqQA+Wl2pQN!r}y&2B3d|*5h7hSf^=Ca)Uhy7S2Wfo4N;n()V2- z)0W&8&KA;^{_K5~|D5zkq8PDzGWS>%s?c8I?Klas1F;ry3UO6&-d`vZLlWc?etVWX-iPVmw)C}zL!#=(2Q*xkpE8L`py-{6|X78BV|eGS?X`LnxY5gSv<{< z$B>2u{x;1v-*!ZGb#+R0fn9jDfNSQt#jHr#a9J-i&&Zg(hW;eWx9q{q0qflh%{fy_t^xb-;5*f@AUutUOU za55P8nP(WU+5a%<7|tQKCEY$$JTI~!{;SUFOwWiW4cw=%TwdC)9J z*?*eZoY%bGF~_m&$`h|RB5PDB%`EMpURX2Gvi|7ZBDGUtMw)iMcHXj~`_#&^&bN-P zb^|iKU%O)K1N$)zAC*2PBSRzCA!|o+MWRKDL?T7fe^n2cz^T($Iw-W3v6A7GSxi5U znE8m$)@5tZ~#O?u|Qqs4iCb|f^3aumUyK+2j>z2Puznf_t%$*+gkJss( zD(p|LDo@3(3eOw+#`T4n6B+)vqw_%W?79(LJnn5DC!9ET9?)CEoLHP_uZ;axi&~GG zXUkvg9|_v5=_{B_I!@bfnQxdJn~K_0J-QiKUCP?+Sbm|JOibxp8uh?gdrySS@$ocs zyDxQ=>q9VfAXMVJdKlAppPmIya6nV7efD2Z8k97OhL>(SDN%k^%8 zTmwdt&EZtxq7ZZmpDDZPzI^=4f{BmE=CnQc5R(|wJrGEV+83#Tb49ekZtibYjlSlliEZ4bQ(s#cO0X`bH+>hmVC;B$VIl`{75SN`!Z!t1h22K^O{1um zc#CKnF)JzJJhFTXSsZLa7O$(@OgJkH3G$_M6JFcm)^|F&vRG9*vHyk?|k0Q-yuLVgRl8Ps~BD&l{G*{Noe|~TMjwIFl znwr*{hD_+l)4ZAduWD#>p+6}jJDrk`(WUxytCQ+)QTrrLK}#A1`zkl{@yJzTa^iK( zy75>R0-tk!!o1sg>)4$ERx91drh&#eo53dq&os@&_0?rB{-xs;9q;)~yUh<{)YA9{ zTryf$EpGQ3YjyUs^Rx8J!L5I7nb&uYG%>6kS9X@YoG5nxl(axQ^RK2kXW4XI`riA8 z1_h#&yi`6JSQohOo|Si&*MGH*I(D>g(dGH${+L&A3wI^nrx;hjI+mV=e}Ht{cJA?b zk%^UymGvO;Jat>MpEo)z{@)`f~Nq<>0>FmB?*vy=BQcYc^36gVyzH1G)=_~8S```@2M!70HYU!OyOfd!d@LH+lR4DcKD6%G7=`n>&y zj12^X0scYoa5|&;}-`C?qZp{8luuGc>faH?em3*?u(vJb<$i zQ?~~L!~6vL0T)*wI|as{F;!A?P?MJ8GO)I!)BA3%Z%F52X#*MujK_rw_-JY9phx6l zX<=p0<-$wy`UV&98T6Q*gy{7Z2XkH$HEB5_A!|EBA~re(ItCIxcp@Sq9=q>GTnfUX z|8)oc;w3S0aIoQ`r+0RCrgLVdv$iv)XXNDMq-S8FXJVoS-k`O2wQ|sNp|!FneH-L| zjU#MmZ(wI?<6vrSMFbjGPv6?nftQ2?G|~V5dRwQVi|PNF$;$q}Z2=pk2er^M(lOBg z@3DcdJfNpsa;7eZ7V5&LmVnHFIrumk8F*gb|Nm9cUL{s=XB>raGI(~Z@Y0S$n zj!xvOLMkA*Y{vTMJXSyxXs}E|wPH-?jwF<(g7$>X|C=1xAUL{-EwC@z6)-gq^l{*oc|>Iflkt#5I(}wZJTZ19!1g~+wKZ|FBq!z*a3;OLiFac#)?oJrXjq5xC3Gw~^E5!e& z39-`?;tDF@0ZQ%5+mVq$y+i((Os28E30#==&}2b$N&?{WZA`=23xh0*AxH@VHS`Gp zWl8-a0ChF+=V4arxRN_jKqp-=H-Duw#INUr*Pqyg3)5sF+g+5jmILT~8gb@fKL z$9F^SNbPel-~I8@Yum->VX5x-Q`3*Gq7Y=`-B7}E-mrqpGX8~ogsZzDe2laW{*@UJ zjI=0IcbAJF+|i}~oZrtKzT%pY+O55x)3y1)~9;B&Meb@1~NVQtuhbh;~s1eb9tB+$iHmILRf&9aIMwxd@5{5P(V?oI5GF8F<_)NAdzv$&b zPFpOmN%@NB!p9Clu5|Wc8|Y<0R&(>8nm=kZ#J66j~DQ zH+t)^F*A0^k{!U2F?ZozrX*d>n)hO$XMMi{z%Jnuw|Sa3m(cs<0`2$#`jhe0$fZE5iA1;Y zT*}UNs$H*EBa8k7KFj@^X8tc5=-6sW-nyw+Ow%yKv50|&6yyn)dELYs>Dqmxpx{x7 zO=c_@+!RH$VgK!pG3k_}?DCo+#4hac2={;oa;mdw-H3;OtG#{9OUx^Bd7mnfxG9PT zFC(ZOpz)1BEc-iD^&Yl}SeQ?F5EKO7hTFi}@&WJA14vK;5S$nT`bz^%V#4#WV<0%PE)RM05P zEXLIP$#us*ZqOFwpdcb5@kA(v%6R1DW^;SofrkJHI>KJxS=aSYlYBpmK2^5?ZIp@i z1`ASUvj5ut{D%{nLImw7_01z4N=%?glYghRNOEh2kV$BC_%z-MNtO!44SNtRPunh;doHZh~^Zmg{9P)+=wfrfRYT$P$9q%W{t8pQelmW#X$&HJ_APc&j8*K%cpFl^pYNvOsY|mL>ddjsxJN|Q0`ih{ zG%TwJ$@Lmsc|Spj_eaFXjx8ZVJMILhfWb9>1?@sN9Gg7ysaIHT)@_Bp@MmM}?IhLy z-ENk5IpDw55_m_hk2_wTm1q5A>KDwY9dDMM-5W5~zoT7s!4SIaj&cnM$-&r`DC;mh zAJ&YS@ox0Ct$IC3Kb`iVcO8Pbn~3LM4v z=maS#1vz8C1X1x$O9*jp(T?oM{X}pp>6JR_lp=ZBN&bo)O= zUU%DAC!FwRC?aY8Dri%L-j7upHeJxtcP-8*<-el{R|7?25k}FcVjJpabu^c~9*&A- zv-s==JrsajNjV}uVf&K-x-bq6exJ+7B@MWYw7$Rae~z$(_3Z}oehOmcXMEuKetOKB z)iwI-`T+OOoiBs$>2b?RaUZ584U4l42IuXXf5~w}bv6N`zA)aIVNaI@#ge=hiXr-x zf-we#jFQR87~ezbJs_O&eCaf!V$-W>Rn;+`i*U2m_BK_^=T)q#b>21ZJ>Yc7Zm4uw zGJlgrJ1hmKNO?u6V!vK@a_}ePGkC9!z-d{n68)Fn2;CFcMYc$7I5(o=J^AiOll3)x zLf5Sr*09FI>f!p3pU{q@9O}!DW8Tl3+nQlBrLjfT=Jt^MM-AGJ9m=wN_j{xjbeD*Z zX4tiOvBs(`9#gd~OLoJqV`fPxC-an!*Gu+e%C%{sSTChDf6g~Hi9#caVL$P^?iUq& zHZQA7T(_MNr|6?Y+7Nqz&{Y5NYg1I^PJFRkVv*4nh7K)Wa>{*uksFux%zV-CW$+M38(e%zUaHlnW#RRnlE6uW2hY~fd)SIqm zG#7cyb;p@Y@@d>=rsFN!?{-QoYX6ERct0KJUK{M(=BHVmpW8g&?KU*=+ii)RJR@@- zemh?EetFu#O%Cyys5xJ;kK(&G!Q-}dVV9I6jWf>jtXF+ktNd_7WZKq1@yiYd-ag)f)iEYB)Nn;**FALtW&s`pyz1ZY@R4Wy@sG1ef#Z(sKekI zT+I9Tz-$b%Hd?OR2`jeI+3g4AE#ZY9(w=48xW8@}WO<#(&O0qy^12HJBA%5Gce(Cn z>@Lr?uk2LQ7GBl6ewe!~@VDPO=|SOtuc(~m{jj(Su}P$}+U`jm#e1zbeIfp1+PtDo zC8h();c%`hR~w0^#pEG_+xhzZY|S6V?Zzg%0fq&FtI(pceo}^8`P8tR@cC}cTg52D zQDv;0{~1%$LjDu3Ea&$7lX=gZ6!7^=u zktAd)n1GIeUsrTz!`D=Zhhvnv3-tM)yi~Sj0gI84RePi$S0Eh^a0AdFFYx5YHLCu- zBU?tXbXPmCUSNDNPm8njoNsh@Y5q0s)$x4yU8ei-S@{99-iMLCIR!=n!?qp>_}NOX-cKb(?aRR8?=!MQ{*3 zsU1=lMV3c(k`}t))A=yHDl-40O^?>0)Be<;R?n5HuIF@3R)8}zG%J&tJ?~~Xb89QR z!bxTfI(`7$rbaH8c)Dl|phCFP4eDo9g=r$G|Dt__UR9_8afGs9{W>OLIaeu#7IkgM zm%(J5h_s$4_|l3%S-tIM#hn+(>NEB$5zPq;+(BRIKA&Tq5C|9JUn0Y=@HVRM9nFj! zQ7PstjppzF(AQyUwD_ii^-nO{`>YRZCsS9hN4f|3G^|eK>SB!lWH}Ha#Pj3_md@$1 zUi}SVoobTCI-aj8R(R9!&LNFZiv;<-yd7ZeJ#f&h+`PAkCT01LGH&+%{yg7K(seUk z2RkM_=h0$4?GuH-9^`FooO3kt2LG$s<(iH*`+V-#4BWV^h(R(xjdO*uATFvCBZPgx zDX$oCIqb<%>EIFW1KxCY-)jx;>KKiZv>%_n~wXcVKbQEdrO1cJ|@cEx7Kin@jeOS6I z{PhF=yd#YZ$4FUT_eDt~D*z=o-+P!3r5W&Y7Udm3tjs)RV_E5`vz?>uygXeKf};<2 z|A}bD@z(Ld{|Op`juG9$s+;r>gYLsK7(1e?&Pl#PS9neOLkcOK7WLLMhAhuIyK0O` zuFJc6Gz;6X=4s8jx{x5I5bY5r((hD-apzX`*}s!e)*;x*;B5Etbvd-*pG?vlkVjzqf8KEtO=$sj}PHd*+*wo z?B87||MT;GMAFXJj_5v~*t~3+p_X|+jgGTP1LGIRKHZaU_PxnZf)Ro8k_@xWSX0Ln z?K6N7oOG%yzY1nRD=(0N_k&>XyHQFCv@St?F6oZLT&177L<8k`*Bb(d`@Y}Cv<@0N zQp^F6l10Je2gM3S<&oc~Oe*HGmP%W={>(V5W$)^w2U+u{Is?Z-a7nV$vkH&A&@v`+ z{u|u&NabAN6;=C$mzACNq3GS?R-Ei-?PiCwb7Wm(n%fH)54fTPiWlBs!2krh#?G)H zoclsz;B?}hYoM?04=PToYJOC8tX%O;=O5)h-FyEI$HDmEYMP0F@#mQ>*q»PPz zZ>a~lbdVnx73XhKUrh3G6W~ZCBT@{5^_EJoPNkwLWrrzRbnZvnPq1LC>y`k6eh3Fo zqc4Qo05L)%o6OLq)dRtdBLyfh0_M^3r^-FFnvv78Kxs{>1wY)tD2@%-vE!nGOpU6i z0Ai;@R1vwwaw@e-O7JkGPcQO(>!QAEyNSYb22R+TtK!Yfqb!Od9irORJD%Op!kHoG zO&5Q-(;9cv?IW@1O%V9#!B+(nfwi7+_-QOqsdZJTe@W+>w06IoBesOIwGjI_%l9*Q z-ze1d)Y4&z(hXl;4y!FNnID~!$d`rdH-kJa9bU3$eIIM2Ev=N8WGlpm8Hc2o@9DDh zDo&aWceJ|x$o&Gh)3+N$xO(azx^j!|N(Qgy7^qM0qG`f zjT*lf^xVyGb~~%`u7h%ChdJCvjL|&!`dIU{6boy1t0I}Qg;CB@EueUA;@#A>vQT#( zrNqXh(&WdioZx2=`H@IRw6=GMJw;DNox7EgeLe0flT^uB!az zA^S#QrVKJb#AT-q?5N2pnH`;y(zg}}ijt6EH_7vG(yq5G^p>e*c^YAuQrakEpaaV)C#qrSGt#fqzv21+uDAmv;nMoPn zZ0IF2)`t=6RpM(O+jr$Hh{{x81#`Jj7?EaxkW{3M9ezJ|2Ju1!0-8M9s2`SC%Y3FO zvoUg?T;W@~W6cg0Y<+ehM|B^ZaG*CPmz10V7K*Dc+y?L6R-CeHSEA+h^MGJEvdtB$ z9%)vZb-QZ@>uuR@CLUB%G z#eN%)bvVboXme6E91(o&6nw6a#TbrbCyvs7tgm=Xv5k}yMo*xR$G84W56cnM62=3G zMan%@!c6nrt4mMD*WRk9DaJijqsPAp-%nCpg*|KKK1%1Fx+z+7LGY=~$wiD^>K!O> z4VDr>QQJ}H{fWsb(*{%Fx$ow>%hA=FKbvc!x%iO#NR{y2~!dc-VD3W`Oc+?O6Q;0(nWfDP#F1VzaaKiEwW1p3(sK&`+#gs;59< zd)w01R)tYc8R|FE`${qhfAA>Jv8 z;jXjE%d=^srcI*bQPkA=W1dLUgcY*{fk(nRZ!b_IPW^QCCKT z$+)S$uduqpLd#p0j$gqYFZoH;aS1xHYXhY1%$K;Rk?31_vk-KvGhd5@pWBOx;+_ap(5m|I{H&AR4DzT~u zFRm%BLjxCmq4?P!=Cch(z(~J^aq0Qtc--bo`beiR1|rewY$x9Jmg84j>dE2Wns=)k zXDEMwp@8{lBS3^w1+MGdA&IPGL~`?#<>H^GbulJK%G;`^csxw85=})6XB`rTBfm~j z|CFV+6oOTLD>TK`qsvU-c`5JhV1L|zwo8xU`_7f|1B7}cUeWhTK~jf16I7y~aRSun z0LMq8k#P_0rzt_4if|!=U0$rQ?raO_0mycg?=-{po zdi=H6PvLDsTE@R1xLVW4KjxO69@PC{--juOcV2c(;+fgOpkq4~Z&%U)d zZ~X#!e^=2v%fXf~SN8s)cY(}%E)jsX9zaC!!h1iZ_bb(X`O4Oxj_{s6-4~$Z2*7#2 zrNGwXec?gyXby(DBw@;f;mv!}DAAJG-@dAylu^9zqugO^iVtv#EUffu#S?Mh*RzBzx< z=Y|0Vpe-~a-x%LvR#VJmC|nAFxt-%6zM-A-(>NI{yPrxz8O`_BcDaa18|!lc5>7^i0Oo6gG*#+#(B?s; z(7_lW=L@uH7m@-=UtlMp!f$P2Wf88#zOkqvofl_iNCS;2hYL|21?WYdY3K+g-1{KV zVr<2jDL`#&@7_erzCa}@2qFCmBmt#lsc_Jk8a{8*L159xjDeaU2l3i$^b4dQf>VX! z^$?f$Y16kUe4#*V!64Fm|HgjeLTM4Q1FWeMpEba>Kw$ujL8N1?0EleB8j*c^?U_+) z$Up#B%K&1;G0QbD0WtYNF%iGDiPz-FquT&<=#;#m zeiN#w2(Nkq{l=DSSH9~%j26Iw1u^$I$jM({*?EjNi89rR?j%6Le*qmFQF00rsLdwu zb%%A#(h1PSzB{07fCYhgzEV_Y7=3#oxoqarPF0Fgs`%CMvgRn)L20q~%X6{k!?k55 z)#4Apjf(@nRWVr8xHYZNI*EgiFDQy!UL4}kp)EaS63%<_bZU7 z1NpA{IsGs;oD>e*+-87KnhqlLDg?sRY-GOMZ<`PkJAyiZ2nQ10l%#w#EF?~V^EGKw zqp~#O$SWA307CGMlHX>UM{1_5W{4z3ASU0YtjPoJYA;Bmu<>9?zl#!uW{YlM>A2+F z0}gGvdYG=He$~r8E(@THzVHI%<&FzL!YTs{4qN}VxAUrJgKdqP+rPYN3F@+~<>sAK z^Ql_X$tl1Om%3da7G-*zi~n$X0pjY^AJGW3wb5sb$5TadSTHc8qTzBCt>>Ys29cao z00mYIuz0i=7eGv&whLmyXVi`HSzjJ6RQmwZZ*>X??<~v90RB$-CT>Rry_4n{FxC0d zFpYzkM~ky_U%BWz9;@A6o^GbmC24Y0v>la0x@FPDGTub8FCF+!97s)Jlnwv=OV~!s<)-+B5 z0>5(wy*rOdf;^xzxr<*wK3%0jV5R&By=2!6cTpO$QUx?U9hCX{Y4+yPLa zLl~R#P^=ce2N5v2GM!gk%|?CjGg0Ad3Y@^xxmHkdTkmgNcjOX1<(x5(q z?9-k}T|%|m71Y%)@wh2a1WzjwAlA1s1#1dx2CGNoa5z(z){nB`JXfh_$3?maQGGN3Lj*lu6Qm+eq!o+*wUY%`kulBzR>GDPUJ~q)dApBg*PrR z8_=n;=}_wOy*z3d;J&4${Ysswa?sGD7>+*d@ld73ZfRcFJu2<}%b9h!!ZoVL-Qf!C5SWtE zT9M*Uh4PfGHrwk#glvmr_)c?rJ9(cJ)M_Us>6&`eHwbjeo5II!;3%(0q$LTg5R>*ITjU z00@rHtpSFB*@u7bn|s;->>g8=%O4Rzq?dopV?2t$&6YKC@*0RHUXh`6`Eeh?zG9+t z?{}^!=MN#2_~y%NEO=|BLD_5|UtYT)pB!of`MG-z3Cf*rllBKJpasp% zn~~hD)dp$YBU%P0W+O}bfHZY1j{~8 zi*UrbXhbw;@Yb!A+kd|jDieB~`o=L&L4b8s09K_SC8Gb3XDBMgZDelq9w6kVK*Zpw zncX||fzUTAtwI~*B;Kh9veL+^+QXPgU(op`QKtbfuM8rLhN}eT&z!{LI}{6K@*htb z`@?yAS%g_3;l7K!X?RJZPz%`_2*-#9l|+h|GeV#agHjDpwc3xBv)PMdr^!$gZ;&@b zMKT8;-Gmr47p+nPO>qxYzEnysUE8ICVmI&gT#sgSFanT{V+lj&f4WrMRP_YxJg&D{ zkZq+2C&nub_|xRe=!#BB3W60(GjJDD5MB^!7A}CdDgk)#vEXfh^4HY+OYZd4`8=f- zKHu)?Uc)>Dq?dQ2{@A07{pYNUM}Pbl%Vq`Pq>i9ANu=Y(^p0fZqAZ;*GTB~gY zYtgEWj_@8wjOhQ?640Q-6#^-(RD9k_WukhN!|9TwhBl(JXrQ+575dd!3D(L&Ait_h z<&YW~2yFe|BZXx1SCB^cW^Fg>d3VC0;3YvIt#U2pwNR6i>a8u7P0A<@>YNM&Ln$=# zr9lM@bpMrCR83b)kwr6tq!NDim*=&-;^SMlMDqz61~?WO(2%xYYG3gp*|#@o)|;pG z1Ea)%1R?6z83aI3;G5*FF>>ksEfEkrgRpeG-FY0SNsI{p%`pC}#n2%3O(X#pGpCR+ z*bTzA0vK=BfISC|5f0rZ0^~r=ulmkGA-KTGn_w=i^rW(&BwmX>0+hRH02mZ4kn`r_ zu$u1}?aAb%KvQV`lY>!<3)J zaOnNjqIE|Ropy6ABd_g-FstQ4E!(U=-3ZI1lrNiKru4%C;9&}O0In+k7Yad^_jA)d z;KZkWz#&IugOKqT-WSmvE__eNlK2n0#*fCPko$UojQ)-!d)kz|F`iXp&OHEhs70B!sH>t>j~!{y)Q;k+Wg!?xcrWl;Z8aXHI`I_H>J`_{ebbw*A5if&^RB} zjy?fsu^a^1P6Bn+T-hOzQ#pm60EriaQfQ6NMAXA{Ev0Ay$*xEkQ{d|5%>j)i746Nb z40?WU$4$FR9bQkx$Lt4NfFM_Fv9t_vjg5)eO#0o7E)9;f>L9Y9vE(MC7T z6@ad-ji><^;q15(LLpkcA|)7sa|z_LH7u+8Fn5fBx)9}(viek9Ubot20Mbosn3sK^ z8oL$()j)7vO({+Tn=1dqeZupQsMt?k(OP{EJov)-1j?SCZU!C<(zP6-7af*$)1f2s zO#xOBRBRQP28$FeaPV4eRU~?|Xw84M_cIDt{tg(Z5XeXhefls3Te$Z=V}3H#xHO=Css zhZ}Y=qh&7*)Gp|XWTv#8{l<$hADPj#wj6To`=Cl`gU4=_t^ufxbjpm5yQSj0cr*)L z_aoLC20K6z*{nVRoQ{by&bl4=aTrZ?@cppv^Tu*dQmS>wlS^KTvrdN8)AOi~cFUl~4ED-R_lc)|h4`QDS zodHitDzVPx`@^!jMzlW?e;Qzv^MUdJPI?5?U@;hdYzVX#aUy{!e6QQ}6?=JPT(l`Q zFc-k(6f3K!_caO%i=zMqaxp%rL2?fOOe!7)QwJs}hhZYeuV@V->*!-*_%- zSTXC(_yyBpN&|7VD`fO?mIq^n3vyw~8s89{@f+J&aODVsm1&LxDjeIQ1D0>kHY0|S z9~;q{9S@AOEN#N+#rr?nhlA{t?LM>zT&JIStumAc&h1%0VZ`DX5K`m=PIBUNqRS4D zZ&lW`tV_6y^|udp`W3+dbi@{nqW4DZBUyWF%)w)Z1X=%Hl2f4GN%^!#P76W8W zTVyO$1yofMCr@kg+3lb{^tr}+E8LtN0R!gj(yx8U`}ra(R0mAe6$qiJ@hmERn;HNl zqcR`ofNH!M!Ct^wjyJJDsnRM;$w}evP-kMi^-GvIl0RcBaTPa3JC$3u57bU>9Lh% zOdrpLTilW_lQhhVLZ9t0s8rCM?uUcLzqVj2Ut$4|CY1j!-YC}&M>Mr}LLewzT6inp z8>ia_HWfhIK_j9?_k(iOJI&7|-)<$zj>2@hSf_dhDYCTw)?9pO%hMNmsmsoNxz{Tv zJ-J7S6kazL1i7lUMF7XV3Jb6m*a)J6gqV7T4e}Rulwa6GvS+-#*dYd)^HDBGYS9?N zXzPQ3R1})8nn(U^@b<+OJKV?u< zNI{UcW^f^kT?G?J6YfkAGGH`MNKmIR9YDK3FMfDb2BSG8ttoRU%*>ALsh7@mq#Gz@ zG#Y!-1?$cwH*5eR+O%r_9R9Tm++^4KMbchYwL9bEPvttdz)}GiJo^K7tmWgH6L9rt4CG}u008pP}9o zz07t!H;j(S^)I)N5`#P5CzfuR2SL@#FUoQG*OTQYx)<hBO!gSSFO3OLTK8UNMsY1#74&Zc@Pp6FCMimv*3ZyNEc-oFx^9_v zsD??JxW!4}Q(~RD-#kvPWD=b+%D=u4_y>iXlT}?ktNS#z?Y!I{oZ&2F<{1GPHsNr1C_vXq7da{Eb&z`(Yqk~w1TKJE|v9y zi|ILyP>mrHon@V@={FLvyMCT?Jc;`l z%g`U@E$&VkLN{GgMf&^UAI_>;d}}u!Sxp2j52x%tg_wnnu_l2Kgf?I5VrDWf90w3= zMpl(31n~R!7;_wc?Io!vi!v zT_NfEM9%-W!tW%UptC9l}?Jr_MjB3JHu zZZ!{}qb2CS$8Vg~5J9CAvfDircLxB4&k}%zNeP@#k;({f;Sq{xVAOX)cemb@eGwN@ zH!hU7isv7S@*eJ}o%7}NXsjBB(YYpd?NV`Dy9a<2v;GyR{@wZ_h}bm$auy zhXM}+a{XSN;;wA+Z2${yq$=}ccV7e;3gWKYkb8(?QQ@`oA{?z~G0WKwKwAZf+@drkoRH7_HUP5y?~Qm4m2 z$rwRUH~*m#^S5q;Z;>B`HD0(l1nsLMFr-d|rU0c_!vEX*l#K~M!o2R?Zn??=)G-?7 zZBQPK50hlRv9Do>>lMQTs7;RJO)fvo?M6`H-oJ)TXXZYfuch^NZ;ppf)&0Abcs43% zF$mx(DL`~i1RnG@ANQ1+`g?Q}OwcZjY`=5Oqb4&MhTa1Kj#3hxcD1;V3ZPN?|AolQ zWT0&VqK{$Kl7X;pJqv6G0@JCdo7D=s#x)35m(w*n%z!3x5QPZJ%W=<{sFR@&q=N49 z%%cfEFT^V^{a^T2#CNxaF?Py2s}iw0rmLxt<&aU`vB-L5= zG2kx}(D8-)_(Z+;=>!7YqWXCwyv-HgLmGhn-uFl0lZzvlRv1N{GU$&~25@UD@fi^M zmH>dg4ooQ-8lCdL)6uWz-U)?*f6eNRpQdm|bxP8={fY$YU+aJx>g4~6wYLn*s{6i$ z0YN}SN{71}yEo$aNuszn(Ka&6G%#-KxJqC_<($Qz2 z_?sa-btFjGO*x%ILPN;%m$(3cJ%aGh5ud*jnDP0VR<%-}p1Q&R^Yd&vHbA&5J_C{Y z3Q$s1@DG9J^;XWHUwD^yd9r^q0X^wFby_#U{%SeO>jJV^DDyJn*d^$F4j|Z85nL_t zO#*Oeeym9IHlJQIx?L#cz2c82F7(QOY8J0JOxt$k-_L&7M%qj@MypFO_thN?I~i)| z*6at@n?c&aFGo~lNTWhs32m$)HLwEcO|1Mon5aw5Zc$Ri++!y|M@K3eWGx#u+Glwd z+rxPDPZ0lqZL0byK)JUv$ne-oTM1VHMcHh`cZzZnLwmr01OcATq_8$*X^6cMTDuSR zMM4%bZ`G&IDl<&HZUTVvc(t$y@&ru3#W5+rRsN}-btx!=b({3dMY=Y_q)oUUc+eke z!~gybEqVFu3d1Y}oH~R0Gg@DMv%H3KfvMJEp3c|6j+UjmorWj0?p1pm!HhU+zY!X| z1pn+clGiUgSSfH_UVfv{;N+6ikm26d_W?ksoK5IJqN|R_e)h{_u$x<{KTxs1ouzeo zMjjkFwr9yUT|@ia&s+0Qgg$91TnMpF0E>WfZW;OoXWKc-0#R zOM;!+Y!#&p;D@A%pO6z!W3L#H(EmKH4(#gJ#NSpBfFI%g*5*ouPwM*OjsPC;+_)Bj zAvmvG;W}(s&Ph%pwRa6T0O0KrLWi?eaz@e;kQ{1og6Owu5)z1gh1)>oDxNd>EQaQM zZVEqVs&YRdWVwTwis+|G@+PABZxMd$S#`)!2Cd>ejwPnByhZB<1hT8xUo?Cs!^!On zO+v#20Dq$K-Gk`1sHQ7>Gq2(AOmk3AHO0d4JzHywC>w|4-xu9oEwk+P-lAG*}zDaWqSj!aXCow3$K+YM)_o;E8 zJs=q|R}ykjl5hGZe#rnn&zM@_&$P;7<~$k50t8&4d{;6~pckiZ%#A}-+PMz-J3bt$ z-AQxK@<@W+jeriYTxv)`%1UVmRZBMiC(b%k&WlJG4mNn`~09IpP__RmG| z?tF{9acZ=>?p-aYe<+xk-rhKg|Ka#>_vv^NC(9m2CHu-YoizQV@4`IU>SW`zUjN2j zY{UNnL5yr%xbiP8fPX-q{^f;i#ym?|JdwS(KW=P5IP8=$LR@}wn-S>$A!fj6e6rQc z6EP$h{&?**?i4UxJU*|#Y|oIWd9JHt?g=d3X_EK|z+8KJa}F2RQ_x?DouAiOMWN0# z-qFY$v#C39)f0PX`H;Q8@o}^JW*E%}K)u*QhVf|aX`Qy?UYh+Xs^%xL6vw4DUFBh) z2gKtc^BenzZx1gz;(Sv0i-ZMy_riV^6qdbe?lD@mZ)GA@Y_2B%O}CRF*}x!eU0|~5 z?{tDv??mak`+MfN+DQ@Z0X^cEcWC=5KqmO@G`iMTrHbjw;mKO1Iy}Ce;C7FpJ?Dev z+ya`FYNjV8oV>s1$FFFhB!fwrY}6Xm{Nubs6rhMQoO-wM`WZ^aHm3=YMY$mgMdI2)t8{$YV^N5Uy5D4m>C8b z&K)hQ$(KA#l4!s2Z9H;5y?JL=8_n=0{US>*i-9qzc!)_Q_}@vw>80^JvyW>axmEzD zT1B_}1y`xX(FLF^aFzGibdA60Veiu(9@ilgd`WYeLO^*Nu;Gb9lds_MuR%U(E84ykeR}`R(si|GbrSWs%p=*YyxpyHK!s0ZSL?k zKbzNLKPSp_Cjo80x)T~uQ+dIUQ=2bbIE$bJk9Wscy6K4{urG25ddF|zp}NYa0X5rL zoL>9UmD`t3iJkFB$CJqf-Nl6Hb$>+6w*-=Z{e|SDv0NL_+pnmN1U(Zfc}uvgcPQH0 z;xRfdj4l|*6)C3uPk!D^T_GO*J_0_KwyL%i=1QWRbmJGVhqU+JYBg1B3gxMqXrXI{ z!=s>&jmu(L^ko5LG#+d(4>R{FB>yT2iLcS6wg__ZV~p1tYGB;}hP|phTd8ESKn=|@ zE$2*~1CCn&iQL9+y6H&9%(B2QSJ z1&3i2vOueMFU9q9pSy+Od}_pj=t*a>9qHx7i`eJ{O%iR89dBxQN3mO)K;SX5kojTc zp?p3hVqcdJi?2Hs4V*U?XXYs3&Y8<3*slN3q*%ir#VxmV3d?clI=8-`Iz@hU8&7@c zoDp~BQLUywL65q~`74*qV|uxe1jM`c>=s|<;Mef z3=Lw9#f`p0871=6Z|9wesm5(*@QynpI+b1P@i_xvq|ZO&F%#gGN8>=;Y5UsWYW!!Y zmGAjOsfu2iMvHY}%XX-3>Ysbs;B7E@#b2wv4&obys&T>Sw?HZ44YzI*;=ZHGAfgu< zrEJ7VcOyYz#0g`kHJE5>R(AP*nuz<}!D_4;e@`z$Vt@j!Wc3%2131g@^OP&KAf>{e zl}+U0tB|_Yw^``z;gkg)U|xD(e0{GYI$CS@qjz=K<^bK}WZ#6(xHsW1?x|HN*s6Cg z|I8*%L6a?g7p}cEolr6j_0U=XAf`v;%A!n&s%-Sn(wX+6g=e|EjQ_}=o``|!aCYpj4%VPQ+_Tw z@D=(%jDZov7``MKG$LK4F+d|?Qcwo}%l%z`h548w?sYD&-&FYzEmOH`+}-w<;@>^Y z&r14bY8I+>-={FY2lhRKi2IWCZWvUx#qw z0J7K*7kE)9!ZBN$s^a%fx*WMte@Q zUKv4Nf<<8IQ}5Es*zqIeJ~TFuuf^>w6h+0>4aD3VkXDZkSlcK`#+>!)nBzcgP$8=K zl$=n+3wbzyFXD~%b=f9HP%_~dW$KYYM&{P`;x=M%{J-(e?O%WXyZN z@VyQ(kKPS3t$s9Y5+$=l3&Go1x>QVcnPH9$^i{^pVwY%cINcwkF^@hp^l>3Hs+j*U2cnA41Xz`Tb6PyfJ zqG33&-~IiST-e2aG^4eI(7>C{@yN>>(%-RfU?wNbBlpi@ZYILK>gxpR>8JKYH?zFt zr1pmxC*mys)16?8x%`ondNN^r`!J9>&Y+_;6{>tk|0}L?@QUp9-@W1S(-;@=< z88>kWq>3nm{QoId7o+K>GE8h>17$mLH$Tx`u0Ga|3N`Xv!@x4zb=t zXEHvqQDPKhS3LJwa*idPsHAlv>@hB-b3`*`5L3AKxCgJzfu5K2HZCj)Q{}8PvLy?x zcUJkJqY7(|C7Dq8{W*Y;ZZ*3=K1e0gPejI+@>Rg?Bw1MH!6k`Tk!|(u0SX;iV`125 z{0xe#!_H&IwZm!rmPrJoiyaY4MKt(hY7~OtNhT}(z8X!YLG})$i|Ve>%p%H1-nWRn zY}%DH=*ZY;)M#=xy2Q}J2%9q6f{B=usDoB{FA#H2q9t=i$i;A2PQX@#2Av5eVPa*? zcM}A7mk;R~I(<+$TVzPlVwQ@QCSTx|4RL(;M6hjRq_0w_0y-wWwK1N~944Y_gF zICFYs?Ve031^iIQZ#o&pAKUA_0kx-tz1kr)?RPE!>lKcMSc=87?Hi6>mf~L(fQMvB67x3KLQ;SKKsnw}n77LHWtw6FF}k-z%!q!CYZtce2^1kQDLy zco$QK4H!(bVKw2RH`kFaknIPaRtA~(>OayoQ7QO6p7qu-mqVQpe$#>VjW)wOvV1U{ zj{!5p_bV>J*vFFt^GEle*9VE|+Ibs1Dv4);vhtzF*LdzjO6>;v+~>jG#OF5IK~}H7 zENS17QoCn(TVBgIqgVHy)+62evEqfeAfLM&+#*V62L!&Xo##)NiA)`PVTlHvCuq0q z!uku{i;qJMLqrHb<1jNVa_Yy`5vwsyc^ODe*`W0$gtepK=_r%_#0mGCp!a@^71Yls zvhgVY@xxN`(V!1aQ9R=+`LB2HTw2-uLTQJj!1dA8aZEo;#Mjtz{mLiuN@y>H&QjbK z48p5&t6ie&Fdh@DT2@gu>52bxc38d=dm@okKA@W5ECCExvBx2upYQ1!sAL|^N*HeO zEjttx4&VV|RObTD3=L{LRu+5C9h|j?zjW0%8N5g>dbOwM>L>QS`zzO&d<=dw46SO& zrcz-zyl{jR9qi#KtSr_n+Kgo!N^C)DI`39=h{M14Y2SN(+toYMkiv@gmMeRjlmFd& z8;Dxhd&LAq5+V>wx7{3Yc-+@Pbak|Dp$cfYL`{xv3sh*{!VW7V0hUaBH09%)!820V zH$jK$jVC1+CcG=0@VTVI@SB!F=}-E1#8G&@R5i7SI%=a9(-PHSHGu>UOCgSnE8H5GOP`GvwkSJugGO+w%mU48deQ%!<4;}=14CyON^Njb~%DPX7bKe$!|8yqKBdhQ> zwI0*>C6^$$B{2M^_+EMKXfn3fx(@w$iiCV0h`*5z$vHD-1L*70lD!WDG3s3WqUIgI zV%lwO=`~$QiMH)_TcAl~IkDp!j=rJ4$PKis7pqu%yd=zmRagnzr zplHW%{HEQrXvZ<3dAH%4XuhX=TkzkfolP8w)|IRF_`r90#l+D0TlqX}QdOJc-0(-~ zifv~@L9p#|w*qAJ-!j4`td`^}kAyei65^9Y6}}``9xu&IGKy30Rr|K#RCrbP|je=Y?> z<9!*svsoI;q9%}+48|$OzDF3oXYRFu7{a-iure&brGPfOp{aFZsR)}6Wd*eIIifAY zdB0H_u6gOwN=@tD8C}i~Q6ofc={fs40~%c+ZE@~{aq2*zxq(;(7c`TY2iatCd< z$(ez_2y2+k#wi4eF)m4<2>OLmi_Ql9Vo`7Y;sTy`(LM`?!vmp)^NB-9VNc3dSU?xR3j;wt2 zA6hdkxJWTH7+37K+-G8uSG`cs`TQJ;kQk?2$zg!!>IkS}!d~=3Cxj7IL7xy@wH`JWRZTPVDgn0#D>CaM*NMxBYlO;irk^?C9^;>)DT!- zk({6yW7}A3yn=L33{8T@R8&!%9sOJ~3ztxKlHhg%TP9+<046zAyKX_p=hf8*^iY`j8*OCYC*W;|z_{KRYq9fGl5^X^v{ z1Elp=ke>?rc}bmxBoB#XCnFvy$Jb3JI>UaXLp~(Qzub$1JhtMNch<20$WvC6#UW)_ zsz{3-^N9b0*#$$-I#~0F=uIS%_vA6bTZ^lj&yE5b?*@?SfKRvHap!=UouhyOEVTn~ z@@$e=&mSyzGi3rCM;0n%bT{rdQ=F|2a~Mu&Q?;?HUOvplEgE?4XPTS5XA6=c((AlC zs20*^J)l7vMWfjO7WsUqBJ;NLeeBvJ=st)%426%YP@HEGV#Cj8++MZsGGcmPBQu`2 zq}O;hH>2kb)-1{{X@Ey1B12&kM@x1t1sWuEHpG-TJk)0%vJLWDn~;XH>0lU8I9o0& z#$O1Bk)sz`OXH?2=Zc-NLFccPeBE0JtMF-I&^x^ z#9U(ok>Lws-DZ539hgiQaB5s|%+L?S09c{EaFf0-mOCc((d2BUPW#flM6 zDV(D5@W!soU_JEe4;bY~_giOohzZW^Cl0 zGxgKz^W{5ZMZ!i2LrH|})8qsxrn4*vgR<}Wq+NVGiwK%|3m^k;J2Y30tTHjQ30NJ_ zNRy|`6_Dxy1zPx&Js`LZc`~FL&p!O|I2+RN1#-R>U^^)OWTq z?cS-K=zm8~?TQu)Kw3{~9GnAF^^mT~R**3g<95vX+R@+tznO$%$a5)lzq#ws{a^jm z7o-+FJt`;TF%C@#vPwfJ{FEUhkI4>mQ7aO}6z}*GPD#&Zalv?fzLF$q(7F zEwfM#mH|9#{_V8WqYfV6A-nSJPTLi{eyXjG$T($RGx6N-(HQ|cY7tVnR0%X)*CqKE zC?be_iS7tE7UZ-cya*8%RL2k;_>5KysNwt10*j$yM06n7rZKyG*ih@0;JA94Qu3y>ShL;KQ5`4jJt+0M&Uw zKvU9t;)sx`fe!SsS)tGmpbTXo=$a#SV9E%2T#IAcgxH6S=OYeViPzIF0Q#$jQ3^iM zi@UA|P_GJes#C_wK)UZP46jfo$H$`=l^_j~he$`%&vt%-Oi%%$9PmgckNikA9avxA zaiYH&0VdAl@PlM8EUTvmS<>f?HhDx{fyA;t=+XAAaDr$0%=u}yN|Re=AsTXgU?rUX zN=Hx?1>jLz89R+>S_>m&T$kPA4?7wNE%}}3@v@Q`QVg@5L1{>vM;mKhyA;PS)fOKE z0J)gBE@0c?X22FNaxp&I+qlFtVOpa<_x=+8_dU9b?i&Ty2QBK~2tiHF3Y3~Wfy5}^ zb{$mMJznxh3U1rzBeSSwSXuJy-t~ZT(NvZjjN4M|1+Wqfk^J5alp6@~fCJRMMl)sb zc!o(C{|w7? zxw{ZV-{7)AHGm0Veo7ta{DmHN0WSOg;P0}**YCFIXnXp%EXKL~`(qUE%Dr*h>>|+5 zKE>)-1I^Gu{b&!0C6Ln2=daZeU|xq^e~Te9i@p)2c@$YMpv?E&IK#IruuU1?A^}_Y zcg^Aml);*4kw;QZOO&8o+29s*84$-*O>7EfGcEJMULBQ*z)<^!rQ5ryob2mtEg51U_`OIRG9IPC5W3$cEjldF8GdXjB4*5BV z)@=jC+A{9#g-R7Ja;Y*66T&L@?0_>e5A@RwDPo8P;L^AQdF;L*z5|(iC!VQI&!BH z`@ig&eJfa-9D*VIATEa#?K?B;r`My>oiz37JM6#AaRz&jW zLQflQAcek`;Ce5@^s#E(H>N-8zw?74n0U__DMfPqT`R*5)x+)YHK``y*bYsQ(vD;n z6l3ruK|6^Ku8Tn-2p6!yl=!enl`7NZ(*w1N50U!n&6*C;JVm6g5Ea^N8nOWJ28EEqoX{|Hi2_xdU7LbXcm-)=7B&vs42-*@^oPfqA+)tBwnkjIY-*@AOGy9h-2 zjFa!Is-b?hQWu*jH&FtyF+y6|(mA}7W`UEbuP4R}dzJy9cN2K)UXXz?S^4tIY1_a= z+TD<^8a`y@O$kOP9ik_C#MTltpL0pD?_&{-Acevk^#KAiVA(@$&`)8hi#-E=#tKmX zd6+CfCXa6{A<>+xSqIMyqxaRlt3X^?T>&6r(o)PY7Ei))buwPjHg@BpC!DNrUL!9^>-DL zem-r0PtF-VL+9%@__OIfVOgU`GlzpV&28wYPKSwY=7!o7`Wq@A^`;qVl-In}b+HuL z$SN`T8hLduQr8!(f})FKEPsa$-yI+@CN+Oj8<4YqgR{ z=6t1;L?^HE_to@oC73Eh!IQY6gU=Au$0be~OI+3DYmS3v99;UGNF2TL4)(tF=Q}@m zf3rbVy7DNIxX+{_Cfc>~&J!9Rq9uG$12;ME(_1^3@h8*NN}svSFGmG=eQ00mc|a)6 zuA{LY9}4BpL*o^vzCSw(x4`SoLnx~S+^;a}-WR4s&B zEZm4MsTM^`2nUWcy<+K43#cG_(m~}~xa*Ygw!N4FUj+;Q&CYD6wxK(X1VI|8!)DZ~ z%DZ*xi%v<8mkJlK@edP^hy&F@bbYH`LOe(uOLF5FmvE%alD3f{1H44T$3?GkqFe=$$Zlt+LmaZQKA5h}5^NC=z@7?=egP%rvd z8^r)e)AR1FsbDFqg$ONqH#wJe=K$S;Yh0Ne95?jK)qThsXUAtQSn=*$A%k%q*H0D- z`tuf_A&=T=4s}Zkb81wts_scsRnRyU!F$?j_lm_!X0AUX5M|Quh1}gdfEy=TV4s3@ zE0%t)Iqrhzb8QXRqGXP6`;Cz-U1UsaSJ~x?wJT>)aiyT87bA>!XQJ42_;nvVI-;po z@xQtk@3QFTkV#yY7@TcXM7N5ZLS2>c`t;*!W9bP^26+aF=33WmTV-o}joY3{_i@$n zkF#bAl{S6uzbUJ1AkH#PSyF=+LcJvK^(=r!-179=TtoD>z`Yw3B#?sfH zvi_^4*VSU6zK`yXLqi>mFnTla5$JEa%8OA#R$x2p$@@W29Gq#WAys zD6Z=~#Wdkus+Y37vO#&)HqL=5oGGjFZaC(gi-U!nN`70y+?9&%)ARu_p8c!;`(f3+(cuK0p(R$NS zs#i!q%>)~yNE##cP)~SA?{H+{rBS!VYL6&aZpCm-lU8jUn3<`v?OoakTjoB7jirKL8 zlp6F1%4hLwOX`8ElIXfgO?)VB)53SbHL6FE34a&=bpnP=2?ifA#e)3xRfGBbHJ1w; z5mi2pK_3k}^&ijcC@C+pM^#bR9*^O7*%7B9NlYx;){1Ws**6-}G@i;w361?^?pTw? zj-&SgmzXgV{0xu(5lk*xI*ga$qLB#R*6qLzy4m{*iL2%r+1HZ;4g-ROuU@d0p9VeM z2fHI z-p;jsV6<`e@dYUyt_A4xJP;2P>QVUcjoSIbP4tGKiw%-l3mS0DQebI}P!rqe2N0(v zxCh9#_3~)vEs%_q1anG~JhPA5rD)|g{)trLmHS>w759u-+de|oB$4s9JBEz{+CY9xRQL6ml~&Os8&!cu@1cqcE|+NX|PURe8`L>g=#$Cvd?qo ziI@Lwe~xp~VO!64Ll{Zwy?J!_Kir8>Hu0Yb_vn(Vuz@2;h@uLrHvJ*CLWQhB%jtxN zQPqcj?gayWvO|RkGQMssbFCI1zH#3_m7Z`!0d~?xwjazdTC@?&W;fb0Av71_8x!q&2dD;`g0!k#kbOW3GsJUA?jkh+w%X*?{ zHrQZTYxxQ6U^bw+gF=c~47E_8x_eEw7(TsN(*oxeT?{KM5AiT_HLjou>#VRUpG^aA z2CauksQBmob*CqLAUE$s?CiYR#oVVzz1Q5v6QZ_A-`{zlX8~-E8b6D;toUrlY zN(~MA8sDlX70&qF0?MbgU7a`+?%_)+L5KJw3;cAuF&^E8T*iLGWr@6;rmem-?Fxd| zgAW5%$K2aVP4Xa86Xhj^O|o#WmyKS;k5wBD?Ni7zB^rx0QoAYq6}*!lbIY(ky8cE~q zp8iE0ivTl7OcglSxWt`%P*S@5XrHtPV*cBMU}aR|!N%^CSC~U9hq5l$^3HYNb~@!= z6i|mam`x^MWV+yKSJSC`^JB)&{@zVYM_R;R9f5k^Flxu?7j-8>9hPI}PA>arspg_V z5w-h}{9%IVp5QjExkAQ8)YW(1GeSU)L9I7vdM$x|mH1dMq~5r}qh6YyrxpV>tSt~? zi=g@rC#J*<;3P$zWy0=u+&3742;2(EveZkZr(}Eo;U#95cx-@Yyc+5wp<8aI}1vXDhjC;O(p z*kb1Ae==ty3?;{eXeYh@92$^1CeJoul?BCkit3qS9oA<^@=pVce?O+cKlyS>R}^Rm zAnChEKDd>FVSolTd ztho(PLE{e4p>h2cLBuf#ULaNY;CnFScViJRCYz0o=S;le%K2akj$cOvBD~#@5!mWt ze{`1JBwRnw>>tW)LQxH%Dez?iW`*!UB|q5g~RC##0p6!e0B-cK%%1Dny^V<`Je*xkUED+woFl+;gs zJ}ZUl?OwIK&E_e-v-tfEi>LT>hf zvX36xFDI@Equ=kJu{yYOQSx%=+TV|_PJYZ>5ukPyo7!XY>3RTIK1Y$Y(hxcS)r;>I z3%+iV82)pej}muP=7i`*^yE_f(Iw6Z8L|E2ebMFU>q6_<+0y&DPscB7mpUh)^#)m& zQ8$SqcfdTeh8ul~`o|94N3KPx9tLe4t{pc@=80DH3#?_nl0iR&n`9(~=#D%xW$wcj zGsnhN5t{Nh@^W_#j=nmeWtca2%ZXD;2aJ)^(ct3ojEK`NuRe@qKh${Q7$yA#uI`Z% zS)Jfd8M|a_(NP-0TV6)l!Ldoe{1y3A^D325FV8Q0^mq4CT2#))l$aNOmiP?w0u5xY}1ZB7I2KBGS+YL z`P!J6$j_s9SNcu=GEc)Wb%?&GVxxbcsOO2j&DYCw|GEUK`sNQ?XIY{BLL*UZywaO# zFq`A-_YZ?wNY`!^?9e)zZ@hNh3yra^p9v0odk|ti_MO&oiKo*eWe4?Bh!qXn-de}f ze8e9mncp3y=39|_9`!H64{Z-3G}e55H1amfgU}_%{FcIe?uIHTj)4-4WXvXOZc6l{ zD?@Tm$Wp%s-tDq@_3<9w-QAZ%mG`|r4N%SbtWNAMey>dxhHIrby6MZG5bAVaw^yW+ zFge)e{y*OeMNDOMZKODKibvyegb!9wF?L;`ImlKTzNq`%ZKh}HRE1ahBRS8V=H_JX zlQQLZ)-wyR{N@QUx@AUBYaj?@zP9EesDU-t*ht9vF-}IU#?<-d* zJ#Amhuw~cqKnCw-;miva{p+Qn?_n3S;wri2#eC)%ZFvKCJ0-tfgZajDb;SNrDM2x1 zyGqwrh^#EP)~=VxpTzz$q+7jr(Y;sjbE%$L4I8~lcQl$Vwt6kJMTGxs@goOts3bCI z(u$At@3;8kIZTUkx_XvZF(nvg44?FNa}42aj$SsW`yZD_9UkN1@7ceRNpXy4j1Wk9 zklgAok2a75j$Q1OD0u-K@Qe4lJ*FvWDAA=>19i+=T zRWsjR^K8IaCBZ`O@N$o`T77np%)fieS?FyP8g&Lc1JzjiZQ)`7*tV`D`Qh@LYdmc8 zTp#YiW@B6Y1tlrGtZWVI&7`@pMb4&9dbn&;Bc^A{kFswq(7W$ift`yl`oJsraY&?P z2zOWzRz<6mFd^#MccTeJ&TemG7`O$eOND}JeA!$0-Yx2!mn{D_M$Keu4D@b6=Ww+V zAV$v<-*Fg;XS)X&i75P0Rt_Xs+?GYpn)l}`k(J)VKex&~_;WRbw^W~X``m}_xh2e| z!#0lF*hV7lEGRBuHcWX~^*tN;k;S82VEZ5_Ss4M#ud{Kf+8J6sf*;D?K7T>aUfs7n>h{mlt@$_q3L3wG8@wgq*CvpEC%7^>=liTl_lSQMltCga+B1ZTEiPf_r)en)b?BcUV$CNtsNmLMy9QH zp}>ftb2KxR1u`_-3!~kb2#X2^fHy*neo=bsMOUoGYp;s_lFh(nxKN8rpHxQ_6e}KF zLQ<;jtVZVAXU}+Dr$f=wxA@03Md#+L&5lnkdirEO?yV-)qm}AB;+VmklES@Pc@mL(v?`NM&mUn|-K$^^>Fq8MI;| zFXaSY(`eT3NK(-l6lgl7L_l3<1ZqwDwgM>~6IMVT`df`#2_H6G(y^Jd)LrxDc1K1o zC+zPuJr`gJhN`uXJ=<@K4Iuf_)Xa4-Bi!s)D%!}N*RsG+xu$e8WD6-w3?skIF<@V9 zH8UcC!~X3U`28H~wRGB-Z#;!k!^hxCUYl%3^!)1Vmmjtkc6=U|B6*etdLS}H*3c$| zRv%`W`NR*@pB}ldf5NP!!fqG?d=#lP#h5L4Nq}|$2;}XE4{&LU*`O3j5lLJ9*IGLh zKG4Xr6EPcMfy!b#=iO+E1`zI8fzlk@(8&+3FZm#`o0J9xI}g))FE7!<#YTIcJH;2& zUM)CBT)N()CLMdjE*SelM$jeWHi{lEx&-#7(DcV#DBpufRk(f>UKf*6Q9&>F$-nxz z*C7~>eZU$9aXcU%elOY z!GBoN-X&Vu0;y9&5r7q6Ex;9F z#)xKIIbc_h3#W*0JWd+Z4`DY((7n%qU6liNYIxT$vKC<0(XLg%RZ*8Oy%ysaeEEB| zE#f+N^JPc`d468dM@oHnzj}I(X288pCQu*t{@ql%xF+2QlqPcnhHKO2Tv*ypW(;7c zF;X+^?6O}dP|B32{Htzoa^R}bjFlzDl*?&W+K>QaW2T>m1e=YsN@GHnQ?!p9S39)he0Ni5|xXgB|?IPw7T@ZGK{RF2Gxrq#S-U{RHhKAlc43 zn+(>ezm$1`vLmGBH1Dr{m93U$LY;|mWEqQ3PXtg)JGkO$NT$!uORVUZw7F6d%P&Xs z#qlqP%*V0qtQ(Y&bkq0l3E6C8Xp6b3;1k8xXYaW%fNihNAGlqs^S;sO*PLQupYuDjN+d)v;t1qhiFZGhk|MDy$NztZwz8CM9_#2Z9pR<;|CFCO{?WJHv4V?XfduWi4EU-yG?F?@b(*K;vQp z@oY(pl1~@eSJ*(#i@kg3YK1BeqO_}@3K?lpj3Ywt7Cr=*hI^g(o?huW0+>+&w7}Zu!A;NXayX?rSh_^BAM zuHRzs_N-H8ygB<-m7l^9Sk_} z^eI%%|KdPD;xyFF-d*>`v9V7h;_stwMy*i$dp*~HP#f8to!|58a?SEMp~P`}?xX#+ zx?NtiKpdTqYWmRu!4^6LiWqiGc$>9pSXRyY0kq&<(oKjHw%Pn&C zr+z=7{5IsaUD|N3In+0JD5_O_1&Jz2^-P(F@GEzJzI|R)`Q=risC@;9_DUl{_J-^+ zjN9CJx_U*s+8p2I_Igu=>$4gwt)#z{k*)y)FluI;SpDPg=WvNksG`tD>rco<-^@?` zi@L>JhFncmS2|xFW=1g2n5R|tW$hkJt7>7GTB6y?R*aUCTJZcV^$>3{K1si%D>-g|U-->*(X zSm9pYV70^6djCC_sAjvBnfjMt<(YRI3a<&SeEsOTpj9XsG4_VRpaa_b>-SV16R$&> zL2uSeC??rn3Tvb}&T1fg*w@87>6;LEzp|J-y*=~AHPmpF<=ohFW!(_fFOHDe0l8ApFq2}>B{${8CDjnIIk@fXwr2#f zT6a;s*k34F+&ie5D-)Nfu`mm~6*=T|AKGjT!2O0n83F74R<);ropnunha=2O{ARL- zGlFi(07UB)hVbv02a!JRS>_4X< z3m0~KBdzGBl3`}bUL8CsVGOFofBljtF~#Cbt)JOT+*{52*yw&(_0xyQEYjv(k6i;g z!w2Qp^dpr&^)w{hq0;@(b|L##D$r^!d#3eY7`twFgkk@4AidE;tHW~Snx{43OW`Nb zvgqII?&%`yw}v%>+ncs{v+gS9A0em=*v@$5#9ZaRyod@Tnb{r`!+=BFUadYtKSrFD*Bku z`4xnR^XqyR%k(~BuwtV6`X_y(^@KLupUinfA}iYEU75nHM!HOOK+gMI+3>jQjekHa%970#{aRmwlXLJCPen#3YJ`kl#yW~eu%#0`*#NZVOS3Yy!9?b+gR1991jt&S| zG^U0PuXd+_z_3nxT9ozQQxeBV`R_4P*6D1aNKXc|IBGdN?7T5PYQfxrjJG}~(U)V8 z#^XZhC>elbmiZY@$&xWXvvu>r)F#u7YLoAYmgiVDT4-wtaAz~s42SzX15L5{JonMs zg~BNA3`Nw!`A=X^{ZZ1Um%AS+5Wnq9j=S6pvWq`^Y*9BK>_#HB;3p!Td~EwHI-o~M zZx)D-5QzOo1I%6{8dtA`VY?VHnm9?p?K#<+uko0yED zQJo@eRW^6#u^TH!uAoQd-WM6!n@F3~*XFNCu@1lULjH){x?%p$MhGH})$=wUyVE-a z+ibR(53dW;gd(`!smMM>s>Th_n7JpQ2MBvIc2N~?BvZ9Yvk zPbzR~?_C)z*8sutnfF2?T#Bu={rp-{KS|p^l8iJ)IrE{{Ce^8bP6q?t!#zS*!1C*! z##d7m6w)v`N%6;y>~_{U{IIAw^wPmEV6wSWD7x3Rb6%Z2?1YeAf-kD&!Gmb8|=}m;C;q=>D%EI~U{t1Ub%OQ_)|7Q#1t8M^n7(PZh-)YwR ztxeyq)?YUmv?1pXj%nQY+@(`mu>;m6LdEkvPO=mtcj@U(Eji*P4Ott)#Lq*V^q^+Dud;OrWlr6j`A-s@E^-s*X_RbmLo0bM|(P@{NQe*r~$yD_I( zM3{@vJoC%F8!dQAD-27O_+~Y+O%4}DmzcY*MC5DY7f7T-7B@8~Q^sncE$q^X z&3BZh_aIj^(s>0ZN;jI&ljRRR z2?8qj7zppXX)j?_3J(rMATe6{ca9$-jqACyz;jCZ%Bl6ZBtQK7L-us(SX?Xri}|x~ zEhk+~uNZ%IRy#J|OBM0cpc=x{MakR0!zGCPj3~%yYYR~OFgeA>{WMrNJfgKk22Q}f zvpA>wi1JQTaVdp{9TdhX-{Y^Av^#atR`jc*LRQYlb-l~emKf-w!CZDjVW(-)gwsM; zAx;YRijnjD_c-;|KS(zlYgG7q(DBlDMNP{`A6OfzlRcb~y%knFR&T{iC_4L8<%^{V z^9QS7#E+@A*w6_kG{a78T9Vq=k8U2l5HI^>>>>P4#4Ncf(}1*$}JazQy_{!=*lCGsD2SQ5y06y!QfP~59=*v6!_it32g&MRluy5` z4deH}aL=Lru)dTnMI|y~`iOC2Ju_RDBC;Rvo*WNnV<|rEj6ms9G0nPqc&-Z;A6u{= zCY?uXjAR}Gfqy?q@*@5u=W@hHmi3=x>-mVB^R$lJJ-YT!w|_^4h`YJ{ttCuvKV{}s zk~|DTe*QPeaM4d9H5!ZWHyB+Q7>O@~YLP7wZIRmLp)I2sPoqdF2 ztE8;deIAp~e?OuPyeD_^o*%T&v(oI2k##VSG}VnBts8vq7E)$)o2b#d!l+SH|s%E5m#)|$L82-A9r@@r8oU_Y7K z`$Vk8FH3)9?0gq^u?<=&n;nm8o%<4bn9=0EH_nV7wGD%W_Xi8fnK|smK%+$2D_C4g zCHBJ~YIOKl9DaT2`-++W1ABfSU(SCrMu%->{W*cn8}MMu?R?CW=k|!4@rI7IBW_#%EIQPsv~q4r~XOxqUP!nEkS-e zYY>PrJg=lRskN$@WT;qdJ~JLet*`%Ua1_<^HOw8Tf_NW%pxDT|yYxcQC7Xphh!j^8 z7;E!6-h2H)NvO@8>{UVv6!BqIK`3Z1F5pmwn=?6g>NzY0_v6TDEh-c5N%~?vvd}TZ zE3go1bALT=jw?^AueeAE_{JG9#GhGY(6 ztE}@s?aZFC8VhMX&f|FTDl_as*i|#!r368`&?ajt@(GQw< z!qbwlLvEb$Q(|kH*VZ^ED zkb4Z9CvzzupT=KawYK~I^wT8GV%rCvmJ=W7P3uy`UQac|j}RNB{Jrl-wyo>loPJG` zBKk$A&-^szU$Y&bJkijx+40IP$*;b}Cz#zp1>Thf%+7Nx|%T;D%j z5vyXBxeAF32PlKz0w)ICyK5~j(RUhH`MPaBaALe*LS5&$wzBa(u%bycrQT2ZqtG8v z)Dukz3}&k>l+07Pqxjc}+AWpk2Qv?0`b9Z^#y8cAV{rR2D=CqPtRLB~*`+k4l>vP?Y>$=zD{xiML_jrxtI9|`=IG#_vwHJX`uWygdjgA$Z zX732j-}UrZze+XzO8)1)F5#(MEAhU`9b*Y(URo-;V3)Mf3& zT(Z8a9RA)+YCzE8*1;-e-l&2|zS94WO3?9{%+>x>IcjV2R}{>q(r@W$knEBT_dXb|&Ualzmas z8Cj87QgevD50TwtfxZyhEonndLZwz?L({lqJ>lAg8x7CHXhj3gU%XKrsBioPGiJ+h z+;1e|ISHAF*yX(zPuS52`RQw!{mrpz7_YzDzZ)ZQWTm#7L&W1y9;!)&V5}=;Q$nm6n zjNw8>`pYvNj)tRt`*%$IU9~E<{{R$mLUmqV5BDcqKGCM96Y0;w#1cR1{)p>K^|tws zPMC9LV>cjpn}c=areRW9HpIaKRZx$lY$+SylRR?0>?+$n5dJEW_>RPXsiU z(gFnT_T&s9%;BCWk{v2tvbjw@M7I_|iBSpYN=0DTy@n-1pOgsC2l34O~ zEJS7%ereOCIq@~9@L-nsRi@fMfMR1UZ`ptxvpyHN~pG> z+6!hW{|umOsLYkonB0G1Le(BLu zME*V{z4@GXBRr+$&r_ZtRvsCe`y@~E{#@IC4zpPQ?p&N$3OT&{asEH>v~earHu5sM zc<$Ak-|8RTC|tGQ9&c0mV#-L?PC8wR}Yias^xExMQ;g!)NgAc5o&uH7+QI=Z>n z(dJrdTvC73rM`Q5ljpy1@@FQg6#rL-{G$R!Bkf!daJ4h?*-RobGr;bb4rHml2=%HB z&_~|X;dH`zD}B(o14YXxaD4|>X1c09!WmlTUm$ltZBV{y?7Npsr zyhE-MY>%XWT96{7D1`z5X}q2TbiDHb>MuWjM)F6De!nUn2wjim`RLL`ulY7p(l4nYZa888S~W`>qo>Z7@j%#g=R+@X0&uqh}1$)+4WN#wm1`E&SnP3jI5 z*xB4{+4m5#>^uP-ZyTur$8g1{$XOWwV}l@2;9q;^E#A z;L{4hLY7Ejur0eH6AlVaxE-cInr<^va@httgG8)!(k{3?$$3eV!@U4V9+l)2)Um$l zvw)^SVW6vv$K+JC43-E@;l+T9d&!14^*x-$@5xxi1z87*^dIPIK`X!9y9(X1I$5dY(6{A|l zGVr0xiWX?qv?+X??E3xPTVQUqfS2)o4$_7Xpp)zpc^_p!FKNF!tDB0xc_`vKSpLZr zsszXYWDAN>X@Et(AeE2YAQFj(&cwD&L7;+v4w?#;i}%8}A!#tvEu=3Bxz7{4Q)58Y z9<3R)9=ymBbexbo;;bN$+-`J`Yl+q*A`%%I4@Cv0dL181KpR*uv@gwl3((F2t%6lI zAb9HqhS8O8!(vpJZGd+AfJChis7)6?y~F8@qel5;A;L4yfpm6&?|=CVXygb9zCst> z0_JQ<3!sk~i4SpdLYYY3LVsazN%QVbUyP?{N+BQ@^|ZizX||P8}!mrx%Fc6Jf&jJPUC1knAmVo) zlQ0R9V?!#JN$g3k-MD3Hn-d~&mZ=3Am)zGY>Z0(to#iqpeiKzZVZ8HIfYV`Zy6rgg zLHq#8vV9loHo1Awt?ijGTBBYsxnR)d^Ec7WEr6vA#A)Ftf zr>jxu-_7=v28JXD(SN+`)+f9_B#W9MK5=lB?w25N_NIfPPJZ4+s$lQ3rY%8Q$I5%( zQqI81jp+L0&fC)wFG%< zC9Gu&WCuhho#oGrn8neGype#afaB>Wv_xmnep$Rj2xEX>Pqw?xT~bKOrCfl0g2)sp^ETVAL;8;C zjph7hwUlG&mZWAGh)g2#)m#dSH%)G;OAU!~A*uE!(1vUSJ|j-+gRg;H1}2K?xXh8g z#fq!cW`QtBRrN4vN=?p}^GBSlo?K-DOFRm;2aVJVgvWRX%%CPTbZSDEcavUH2J}s? zZomTmY9Y6mzJ&_Q@Yil1;G_SVa9s|#Ti7-rLzw3E`9T*wO;9(h1T(`Hq^>`IF4K_` z74)V@K|CmzT_pHi4x-@%&VL9sWbo14r1s&h>+eP7=$67+W6y#)eb&$^jrh=fx-*c! zJswt$(3Z8foP5WR_&e7j9+sTYgCA=%?R^O1Ub3;jySZ^sdm3NRruXFSMS{;|!8Ba9 z|7afm$WcvosB9wfb;&69K7I8BZv`-_swEq#wB#O%BN%klbDAjSwe zWH_;)2T1H&11y`Inl=r>dI5XC4w*m6xPkee#1BisOz%{fl0{$8;y#uDajVZjyUdjr zl7`j#a}%mV+($f(76kl^makPP?;stDpx$)N`CJ``A``{YQ6kK{RPi1JRDH^SLq}@s znzM*+MHVe+jbr>(URa_R(pN&DZ?Q+6H)i67p7==o~3rRsO^MuWpEQ9^ag_j88GgNy9+ z6!M=jzd*A>*cx1W--DD4n*c=N?Rp2&(Q6m^n3qKct*c-Tz4C~DrqV1ZeiXs}Ay;J- z_b5SBQFkFNF$`^;`OsQk05Ap3hXp)R71{vpO(mTl9^?C$&v^+-nXS04XIx|(I@A5D zliJ$iRjV7}*%mHcF;d`9Awg(LDHx_cdM(h)Cqomsjqsp`P@#ZjX93b@3spxK-B%|s zuYgOm2g(F3mIS+SnRZNYkw|1NkUrO*@8Oyo95$u?$eiQB{P{vN7x|^*nGXJ5q4#DZ zut-(Hv92gVqW#e_)K}=2t4&~|?|`{A1bbip$m=4yk%^>eiYszzYoHK^j2mvm_V-4= zfp}kl9jHprXR3eaxq{qthc9CQ?rDogVp}9HDz~}#PF@oZ9Gy6MP+SrdZo$B!ErQBf zj>%nIsTVKl6YSo;B|P!+wn~9nxPTNXN^s5e81Fq24i=PxKZYcrTGrqDkz>7Bcj&9_ zTl!P~?_B2<+X6U)1|Fn1n->y(MWzHS(3sy<&%Xsz8;(VaAv~HZ+d1|xyYL?Ozs!om zRCP#nQCnA)pO6^E+Bm`bk{lbkT*mxp;E&Osd0=SRnO|)Fcf9^%;F(YgOlQ$@!?cR; zCISdkdW(7H2KCFMzQt=dkZjOhO^P^u?5%LCM7zp5G7{J9azEIJ# z4AGe(uxd1aRG5;@1wvq9E0I_>ON|WA*@$IOU<>~~ZMFWA+K}_1 zbD%L)0KA?xqkhUZ#FXPnf=HNCI@f}GH5=AL*u49Ud9vte!v z!h{x(6E1{vrz}$RU=K7SZqT1Vi*RWy{e*h1vt+}FA_oA#g;3sQstJ+2j?ZKdKzv0A zZ%fb~4z=lYRjJD-p)(gyE4d;GtS^$h?Zms^qRb^8TByrE_UDkA^3Y`i@N zWIed?u2?UDKNh0WCy^#8w2>pl2&?o1m9CTg@lhj7LSckS1I~{IAK*xaPvu842ShYx z?knTMf%pbrq1C=Wz8BGeeOfyk8QLoeW{aZ~c%eJTqwiG7jRe!oZ-Cf`K;A0kk0QI* zp$s!mJEa!O0a4#5kztX|4!8syj6+c6L>(hSvJ8=s)R~5#qK))286YgJ?hYrwLI^rW z)E{^TIRnebsrQJg$|+i;WegFIrEq>p8wtOzX-_9K+E_0S+&Nxcjs7@)_0>#;z|mfg z*)-mpPgq~FuSoo6`vp&)fMg76d(^fGByCZ8FBs(u^QcpkD`)xbUVc1Bm*#)!GN|#% z4n8wxmLWI4bkOqa6aQo6m{@`jTEDWY4cAil(zcU6On~#zZ1foBg4#}}caoZ|>J2N^ z>2>*ryR;tv;hjVxz z+&vjF-vrCW`scI8Z6LW;-z5yHVF5YkD+9#zPo5&-`Q;0>HHVF-fakrv!U2S&c8}% z+NV~F!MmVQe$MkDaD7*hJcRSv=ig9K0&9*Jq7HJ1 zta{286mcX}x29Bns>}Oers0cV4R&&2kiq7)9FX|EJx9OFj3U;veLiKzv~`!11sS=) zPAG>Js=k1{Z5^@CO17DTTB;MYTV7cLjS}ISO7{ltOpm*nHAF|2@{6}rbOIQ^*iwW3 zPwm!&Kfwm?^L`o*-=GY?PPf6rK!BNff_>G&QB~8FZ~Ms=;G^$c@CMdXHIrL0eia-^ z`qb>R!*hrhCISM4yu4%7SOw=*9>*DdBZ>(Bm(EcAoOD4NR5UHA@7q1IFy#heZ}koP zk8C3cvQodYgmrY*EJJ*{uRt)zraJtHw7cB^f(9Cc{`#v-i-uLB;_#jQkeR&ncs7U66@aWq*_V?BUxVnbe6$J3#{DR z&g%j}B}0UFPky>f&QII#4UBWFHlTeI0q3_iFZ2)+nOe@b^yTc2X-B`yuAF*?0$1izM zD?Ug)3|Pb!Obp3<@@17c(r*aLM{{KPd{?-Sy@S<;jV*(9p}%aDkU5lb3v9{cwG`s` z^Son;jJLjW1WJS}%ScX30R^ET9WkiJd_?k{BWw7=mvvFaW0MkH{c1}ao z8fqs2E6X(A6fJ@A?^X5Baub&0TbehD(pImxS0))m&?1S{*3o+7ZQuuretQA44*56- zs)d~9Md^4Cbo??RI4_|B%0_kpVUt~+wLp8NX}m-1wU!+qQ1fFzJ>9m=1L3%WJ=G-V zH}t|_e)vwuu?1MNk1IormqcjwpNq&BR4kq(PHjaIKW6GTIva^)LFniARo4=C<&71C ze+38D@BQv$EV1Q?P!CSseaqv-37**stxm^3mqx4pVwlb;SY*~~(xGTF`>{TJV-LLR zyy#Bs?{#xqY?PvxJ6oB8G1j%4kwfUwMjI(O=#Y=9CO%B4-b(WkeW7f?qdyl%B!buY zxhtz$ETw0N>JoWbgr+LLKTlhWzDa?)vr{U>)V-3LKDiQxrY<27CP*!y=!JI*1CxWv z=OVc62Fz!t4X~w$pwOo-?{*0ns5qbRga7N{@@3_WFQl$Hb$K_38DzGHvNK!Jj%s8b zpN@G|vNxhdXet$*Gz|yx2D)EJI`Tz#EQO_WG5oGW#9HuvPnR`*SHtr)&{CQGuA`Y* zJ(*T-OK{r`IM;>~((dfGohjObx_5&Cwwk<;-bv@)ov|aZS2dCC^ZoINBeRoTF;RLh zGq=Yo$&kO9k4Ig4^1+?+7h}v#IL@`<_$=yBS2DUQF%IF!)ce@t(8M_Hx7zx2R^d$c zV(-Ua8*c#uo+QdRmtYqhQ-D^zg{beznCw4Vj}b7(b=*Jy?8lqa?2eyA+!&fSP7WG5 z`L7pH+m2tx_hL9a3!ItHR;$o~Q;V{pjwj7^m{@QF)liaDN$aml9Q}oPLvDA?SzEPJCRWy7o7fRM zCe4!EUR+8|yGvCln8u2zd9>go9GNl;^wPB*ent^_A<`kN#Dbe!uJo%84o!7s9~57$ z>az}Q!e|Y-&g{m>gyC}5<==iRfHqP~ zQtaOHNxZWt)IHcRr0{*sB6^#$&FFGWK1^P@nJ|k!wts=rEV+7t(Td6V;_0Qs!4pNVIN}p%0)2pS?j=bn#;YT zoqDZO;N=7x$&LAd$900{5AnT{cOLt$YkGfGBBM56@01r{;D}@xH*=zB@{@PUiq;x^ zH!KAQ3c+ZtVE+@f3K0a!m=3aew3Q=91^q~KQpJuvseRKzX@zMZdP(uxV_;q4 zag8XLe4@#Fqk00DH>uz!KpSQTQuD9QTQhMwTfQU6x4<*H+=-?hyN`YIG}Cuy!$QjP zCI(Z?Mi?S{FW11Yw6)1xt|FH=P_9BDg6~KE^>&TdLB`av)HJ|R9$J+8xWUL7~yYvB!kC> z3O;ssINokgL0(iPVEcMt2~+qT%((U`bF+5e?of$e0eXn+yY@`O<(i4G~s0V7B63DzQq}Z@30aW zIJ)TR&Tw5gMQ*L!zdnd}6rY*Yo;R(-1WjUZG)s$@LuN0blKoLYX%5xNuuJDR8%LKy z=&#~oZh@!ZS{5~lxTxe&;Pt)Su@s8tkCYIx<|HdPzggk4ew9^&6)BdvWPfqj#=!eK-^40NW!f|Ck#tFUdLi-uQA2Mpy$a&@Oh`7^>BC)k{nnAUyf11wYFg0&M&q)Bcc{AkG^vI zzZK}hdH)i72MTLUoI|v(&q+~f8mxZCjyk`3ZLi80+daW?d^(8R{b@3B@MTl6@{Q08 zTw?li=SMz`p=FZ+Y!`20f@z4A#>i<_p4IS2L*rGgHWaAtTALcy%Up|IjXAzzW7~kK03{{Y*#7LfTvWFHJNdJyR!E8 zbnwnNofO{(+x~gknW%s?-ZcJ-CCA!c&bW2OW{}-fvb&A9aoTmI>=dO7pdsn}lq{&M zq7k(zpYKNG#KMr}OM4qW#}hH$Wwg0G%=0RY+~*11RVVpY&JKPMb9B3T5cXE)8#Do_ zkxOn2i)d<7H2(_MVxMzc+;(MY=E~o{ymEv7O%+enPCetzY_sjihdk3leQCGSC&nZ5 z&x-itN!>2J`aHdCWM*)v73XT=(8RmbJ~?Fu{Wpa`r&s9Y5(vsto0;iMgS1Clr1q^8>(*y3)Ae4w9oSG6BzRBdu<95)F$b4 zSpJE0!Y#S`&L2iP*6-w2ywTt%u6}tyW-fq#p*!*6Mg-q{r?t^-Bfo3bq54!*&y2B0 z1XkV_n#W`!QUw(T3@f@h=8C}SV)x=^Nnh1>TM^Xb50|2apAO7k=5c!dbF)3@S(hPn z3#L={{K95Bdvj8B@LSWJp9Uc4sCBM-@MiMwwR?dt{eWTUoKs*zHy{d0Ue6zmil&MX z!QCF;7)O{ch1*XE4XIrz*xPC(D!c~3EAzFf$a)0QUHMPA4nuAo=)d*cz`G0e_-CKN zI8CzZ1KwZ+&D4&-+Eij^X?#PDf)Lo<^$%8h7}@|7tj-o3(a4%zv!lC48G7HiWmhEG z`CG^{ECiR8Ie{9Y705ozErJ2qTfh5%gtNbg`cDmBo)KoJk_AoKyx;h_HQ%{X51&c^ z2&yq={Voq_{Q6Q+&|jeV05(G^S+beO<+toRl>7*0{44q5QWZ(-O~UQLsNG&7#@$C` zHp+p0<-e%`awdNqgE5P^0OU9E zxJTfY*U4jEr9shJhV*7WVZq}esSDe=8X5?|`T3MmBKgnQmpu&fj= zZu<)i`si>x`Tqf4-U9T_8hFqGT>ifU_x}sZ#gLQ!0sAMIWSZWcH>0Z6OF~7Kmuf4zLNf#!0R99uHA>^k-oz} z<}hS}TM<>b)y9pB?x09o1PD1>B=W9Joz@@m`YDCZ#stSnTy zYl7s_X}P@1a{B;-7XavE+P1q79KS`UY*J;BGBJDjGF=2PIlX}7+<@x5>BVdpu!O(p zXuH|~aJGFEU$r*8hmBqq)EsGkfb1bY)mMPwCWdZ)jURV>Wzr&yoxYX18qXZF?akvRGGa_ssO|_ z-F2l>f)jQ5Xzbq9x5)Eo0h&Mcchmj?FvnU=^lN$hzoGd^5=aEqN3sm+(&~31s(|Ews%AoIi+~VR@0w2w^vN#+;;au3sZ8@2x1%f>ERSRza@`q&>Lkr( za9B$8AZ?Chlg_g30bq71<%)OVGIS?a$N^9ELsBx(E6-w{VOl6ShLP(21%+T%-Mi)2 z+BJ;RwcB+O^bRS^yX*+5(`AUQ4LnaG@Y_%uF5`{a8LA+Uvm^lq%wSo#=P%zRL$XN& z^?ywC;o%YTtn6ej6ck#If$W<-$?$yx?)eE9hyF|2A1j&v1r6-;C`_P41@V+wZ!ZA`?9B^z;WiqTe z6d@7B)AYwVSt4l1blq_y7G@0vv;SI%$1Wrh$ohhMp}h<>(8#s?#jbfGINFjReuqzSUh1g>i=(+5ve0TkU@S4F=4{rw$iV z>qMf{{zP6K-$65Up;+_$`@D8vC6WMMlI*E&ly&kuF2q^y{PkL%ylQf-#UJB?5N||K z=U8%Q3{GTfwhe&2d`#Hn#z$@Cs~thQwvi_de|tas*;bsvx>+a=+W5og^z&eD&6?jx_}QKL#PpB|Bdcw|=F@TFpzAzrJ*LywXJ#F;EP88&ccHk9beLI>%s&1* zrJ-qse0?p3WEBz=7gnj}!u@X_z0Q=l=o$(ZS=!7W|MlkxhB%qS6tOc)KW`3I8{||1 zyccuAqIqzt3+4T_egLS?tXj0I-L8mUrDXK|aqj}dt$tbS->=7Jj$OT7$u~4+NSjv! zSpl1MOTBn^UDX0dT|eM!KRIxdonJh}bRwnUD+hnL|MjES$ZQbtT!%R#borZnjuI5G zhUJrb{p9IPj3}u}l=cKp_Z!;qJ3x-Ys%xM4{uMvF6o3jg3o%fCsVE7FAnQBU@)x0v zaaMU-+){`6ijd{iFR!#!3tX<6N+G_^QnReL-Qj;dp1Fty@M~Ld2Zrw&W(i3R%1VWw z!SHY8eE_@G=bNr=`C1G#xUo&<=>2}JSFF;G@u8C!p`50oS1x87puCGSbAlchSjY)D zXC?@5litx`ks{j|x}^2MyC<_Q0av;E<-*uZdY=s1_$Z&&puf+cRtE#I>Ha?%2tcCR zSsLFd(VnV<v(c?`K+>rc)wA$${}}b}EXkuV{oVI*bXa9u27-rVKM&qst|wL z005oXUgVw_i12?K#NYH|z(&jV>!)ab%F^d=#PH|Geh*#PitNmPnV)9QEnhH`Pyu$B zw)pH;O6MDTOezmlW(j4T3*+@+JDzMC?-qXxk7?4xvFbi^D`fxAQMu7E|B>s~%A5$M z@tTqL1&8?H>3(?>$=0jRQWj@LXkd(Ho*X0UpL%5cYW^O^D&Q!A86`zjcLD<8^GAX3 ze_Gi-5_>(|EXpwJlTmorD3umJG}MekMFHhj^zpi?W4tZfL!u=^m@!*I7K%l0rT9f4&mO7g6rZ=u;{Sg5HAf ztrC0q7$Rg=N#Xat)K%^{ur_3z|1RZqZmMgUUZ2-Z6RxfwT0Ezvq>JdtsMdI9`5`EA zu%$X@A&~QxE-Q5G9wOcLU8aCR1=|7%jLX@zY z2ge-I;&;qnOYz1&gn+E~3@cDl%j^5C${lO(#ECgtK#OgOvY+5HBahC4nREZmbRAZ} zpydCAB=lp>J&pD7RGgzBg=5#BEaD&I4+Td|r~u|aQ8LkuTB|E&urvezIPdqNABSSdtq?$Hagm&o~7 zR5Y1zj?PUcOeAT)$$hDfnl+9htA>82rITKYJ4Am z2$f}xv*{_`!?VJrF6we0avA13Zji3=R^LRg;eD;J>>A17w+n@2V{F#ji}(f?2V$Ut z4&kSsvKIoES`5`rBk4Z)9HuqB zFt;425u~?=58E;k2^{&?lfGaFfPilil%Wm;%}XmF|1U%mAOUe;M*h^AAvKx3-y zL(?ewdpGFJzj&oAM{nSVl=e&xZG#}uiIUGHH^o`7j3GJjaF!an=i#Sbl3T@9$*^hP*B{L2$7^`km{BeNpl2<01aQ20+SGl*X$_Az*eNu1T@WI>YkWLfZ;iP zBc+Q(3N zU!I##W>#pU;toAFdQc!lQ;g;Ii3~eB0?GK!)CXu-YW#q=xpt~o1?|Hs)3ic6ktrW+QfHO1_&n@&&<-Go$V&qy(NF1;HI!SA)f0=I+b`*&KM{TikQ@4Qc2u9oK zdW@&TE7qKwgsX%puOi-`z15pajxKLM=3|@4tJc2dIo5aVYO5vJ8Ps+zhsS}+vkV5R z$1lPd#GBwyKmsW-vXGaNHNWH8az0qw~i{m5LN=nJGYwu1A zxA4SE)Wx5^DS_wivR9p9qSW+(p-BralO_g@*P(3?<|2j_b_QzD{pSnmh_CDi$UIVu z*d`5VWPz1EnyFfICFZw>`v5`Yz<#fO{YvgRSXHjEGJJ9|%Y!Sd3mJ;zJxaY88zo3qmu__oNfVMwIWRR&k;f6<{frU!ez0r9v0jv&V`r6sx1f zSmJ{vn5psfl)pyOT~ZgwS-FMF{T;()Fi}twHJnvCGPL_4H)K4-G-|faxK^?Cxselw ztwQ&abAr8-S-g0gY8P77eYHN8G7>6S#g8YUltDrhbLs^%;>WwKYZ%wHA^X%JI$o+ptbyprTpwudK@3=<1bVrHp zgu4Qn+AK3+0lm58v|UVqS0#O;N#%3L4-`+xXLNv)s&EmpU`Q*y?1>X0J7viUkq%rD z?sVMI4wPWLp2|s0xH{|`xae3^@9R9T1UPv#?-Z~s^;K%fpyd>@PwDclu-8$sy^tQ6 z(=vN+92=yj;Q0X`T~K2Z;Jsy?pnW~=f;t;YA^SMWF5dohV4hk$=4WcBXZEnzdI>A3tvuTNLB~gECQ3E_VlBWLyKRXf2=MkTnlt9K^e>flSv(sH5S0=s zQ_f9_3!lN7Fp;AN$}iS)bB{QX-Jq7+iPCbDh&Np*^SjK-Lc)z=8w%K*1cR*+18iKc zTJ7H~lLdSxQAt#(vu8o4kEp{h!hZ@-1)FZna5MUv<{4`{Q5TAQaNGfbf?;5O$-slk z&}_tv%-qdSAfmT8HA9bGO0`8KK+t^wCDol&+`~3@#_BuiH#gqwZ>psm@80H&26~JC zV9+o)fY9pVKJ0-v9l-9S@6xGFX3uGL*U8yp_hrgJyDXL!KC>p{7yGBw1RPIChqdkd zB(pkb_)v8=Ej3S^qO`d;JxIMPJZP0-=?X#BZAD`Ygjk2muC@^UR4ovfNzg=d+ME}C zo>WePl|v(axAAgEs(zncTQC07{HN`^-Lte&^S*6Ki7@M#!HW0ZLfIXsu+R2dk!D9w z-LF^`IWl`$mN`~Vn@<1dwbLX_bws7F8}(8R9Qez1ZI6M85WzC`zs#YLnhnZ)wEC}E zX$$;{zmDGWCQh5Z|?LAO?Z9F z`LUcbFWYQVbJHG##`M9rh~Ok@=ZE>3B|uf5R^o4j5R-?IoIJooS!S#1%`4$x1s*@v zH=K1gehXzS8@jB^D~&&b54?W?V$`X&4>j5bd?nDF5~^qENQCFEU*MSt%b}5#Y5tW+ z+h>bTpIG$LK_zw7cS1#}z4R&;q;;Spg~okpvhF$aFx$}O%@ePg4;PIODQ}@fL^9L8 zmPc{T*WqbduwddTQP8N@L?d$3GX zHyasK6T#kKQE^xVuuyT2;Ij=?k$k^(RZRKedr3<93izQBCeN{r?cKB1zI8jA{Wr~T zTyU@J2`{fi`@gv?ux&pJ9^7p6OGJSno_my7Y;lMQS|W-a<;N1Aj6>c$3?~Um7eX|4 z{1>JDmG`3ox1QzO>pI;*oyV#$dbkaIQTk6$!w`7>`Y_+|BaDxoJiwdOHrX`Mb~wuO z%b&3^%C8Kq~~c90K6V!bDz(T51W=n~e8&9gijD3&??yLk z16%uW_xt*#V9nHMT^+Tzn_Ojt(a_$wl|?>CUQVHo)>X>BqO1AE`W1bAMQz@GUs!&$ zI0x=a9jAqCa#y_&1TVhFs;nf;k`tTVy=4Vz7J@j?V|i{O@G=d=tc1+*Nw%`vCC}eL z@k4JCF^KUBKONdxEGw~>ZOWh=m`MtfEP@)=(lu*EvSsJp7-4oWH!m@4?8cS4HMNg) zxujID4yt9K`YB}5C}pgP(=T*2<>)?g+A z_v>lI8kN!DNXeGBmK>^+V5udMsK^zmR_dY}LM&vS$QBjjY`F>F@{Ge&5`(Q-863TF z8s7Ib%bFkbiReFmxAKTvqHVIo`mv2P?%@3rF4DC~EW9d1&krx^jK^XKzaehD!20nM zKSxZHQA2E1$UHN9Kx=uCfS|TGC z$(B2v9mJ?PZ3annp;hAoip`ksTc8HFjP$aG`5Ry6p^3V0xyc6c!S#x3ZWh%TPb!!^ z#78Qgb>l6f%1JJA3&>^C>1EyY5ZDubYH&|GKg24-RaD>$I=Ux9h6oJvn``-=1-g9F z;I&jsct}ro3d6O#?+kfV3VfN>>tMhs7+^$S+;$miHTI=vue>NzQbB}6G<}5 zD^I3*d~C>;soVt&32Od*td3FKudW)$!B(v%cyME`u;%$o@yY?@8I})I`B0fJ$pR=~ zq%M;7`YT#1mNMaleG#QE=oxgRi*YC29Lp(YLUEO2A3|Cm<- zZhQj=G$X;uLH}qg@e7ed6Fn97T}f<1e9N2Al98IwaQoX-E={=+gk5PceD#;n6FeM+KL45&V&GxPH8xa71aQza^>3KNrJ$~Va}eVLb3g%s{Z z(Uw>6gw&Qg*YOy8hZw1B3q{XP=_6g4hGzA*bdcWuQ|nYO_`0Ht z+AUl|;fYaV!LV&17-flPfzdl1sH!8&jAh}N_{D>1l4%G7xTH&8GperCW~KDBKVG0b z%%!5)dFXY#^d02J-v9SGe$Y~Y!MsnlB*pv$MgYy*no>LSg}2g;DjjF}TwFCCGvW6j zqpo3Y!;EZcWYk!;xF?cuP;!-@!Udyb>IXY7h6}6S9`Am_9~Ugl{lr9lPSSYpDE^FT&)t!sOw?0mPe`gv6Ck`CMe4 zc(a3Q{hW&`PjavrAKRCt=MZ6~(Lw)ukq~Dh^d-ep!whW{3$qB&`j&ChdMK)c z6mMGxE?NlU5iyV4|2;`2ecF2c4YKj`VL$!b#@C2sL0wR0Dx9L?siAvUqULb>P5Sav zZ@H2X=hszqO_UD<%z+L@CYFa~sU!OJYf;#H-veNdrFH|Vm{6CQ;;W~Q&s3|5jQ6jO z+$8y;-E$t#o$C47C()IhHX5YS0N=CmkTg1q;43hV&8%bgmso=GjQ^1w?;NgUn2mcJ(n3?hCxR ztPUULiWn>+@eqCFjcHL#6LPgW7?m2|m8a%$frHPi61DGq&9l#i6p!sqIFIiPC#)l8 z{DKk;eOE?%5otgYK#iq@wmO*GtOhs8mdEGX)gui`sI~@3@Bt=3&iu5urP8N7kbw5% zryR1I6&8kW>W<4Jk48?WE0kB}#4H-(=HF*1@Q(TJ z{)muaK8#F)$%RZBAtYIYxu_!6JDrG`htu)O$fkNG-vrZWxoM(yjjfOKf4AJHBQf)I zaFvfoyk@nD-7rVa&3D_=hp5zlS!`VjQaDcxOXLo;o z%d)QEje_0t$7}pqJaDpHpij#7exwv5SmY1E!G#heHn{-XUhF#ki{-;B(-dST6_=|l617F}B%Z?->TiZ!SL5{Y`6u{sB`(UM z?KjduaXvM;o?GJgO$JCSDQ!c#ajA?%jRWS42UgjYna9p7@jP(ga~r1TE7TDFB8=y* zUaxSvlwnn3j_VU6^ zFkskr)5)Lzcf+Q@v;dbZGzJ-)N+PBoi_<2`|9XW2lOn9nOX$ShaG5u-Fy46KR@HhV zuFPMX(}oZAjzttqJh&c+rRV}6wJp3LTymwU|Ht`F_s!oLduQ<6$HO_2IFCtIGM+5F z{lZ4@)Ss(Ku!Cf5cT1m5C>(?g5EVLEi+ip>Txc&{OJz``(NFCCTho2uDeq;zsrQ9ILyvV)L1KpX!&XIa{X0x`z2n8@=aquXG&a(!lR{#7Y&eD)`%{ zFGZ>IR>H2u?;%!`&g1J?sKmSC&Y9?MC7oTG4Q7$th1%f-g*UK&KWhH{hYuy+=I*<$F4euup4HAiLqaW& zb`r7=&i-CL#c1ux7XQt`=UlLI;P3Wt4(-T4zx-q+ilbF6=)?5wwy0eFIj}pv>s#ZO zHMjfv1Y<_#ZejiJ*dLo&Ez)pWi(6}bors4|;BMMFuR$~4b4Jzss;f>l|NG5uMaghn z?No|zWlJL_LrKfG)NcyHkiWAt$Ye`cNlMw~oavG-=2`wSuQDpc$t1ig5*{peVXE*6 zQm7x@2;$(U_3N5LdeHmra!=Lw=5$8r1ejcw@W-2}K7@%MXGmT7RqEiycejdN-ssmtOxhB1FXf>6NI1hfhd$ zmP*0n8X>9BYGb46dw9l6@@8)bGbG;YUo5Y67@ber9aGluPVevPdH!wGWWo6R{9Q>` z9lzOYKUvnx-^Eu1h@QIYD!n>B>SJf6(KY-w)6cbq;eCg^_d%Q`mnTJtcG#1e!ZOZ2 z>Yb*0C+kWVe&Tvrzfrww!hlzmeQ#Z$GUD`FH+sMSdrzd#dga@zs(Xt{&4U7?oWC`T z^`aHXe?+V=-Oo8a>NhFu>d>9|{o*8XQN6@L-j~?#F^arT?y|hyj@|eAxNT2Ubj9R1 zqg0&VZYF#D0WJgm7wRyrInSYFr{#?7=GHEw4zeGNYfdNPq>ZmWJ&4bBk+6ULwtYKw zp4GJBV2$g&D8uk4S_2R;!NQWkLP(nHjPK@C#l@+54y{UO%{{s+Q8ec;lKX3CPxr0Q zgJ5OBou_n+BCA_r{N{^MWb%U(ON(DdV#{p#WTX82*7gjeTH>`|JGpgs^>aLFWhK}1 zDfr@5pU5eGPk&`m{miQ$>2_=%`>+Y^H%*NHP7`Z(%Ss9bt#@ZG8iy-Oed_1TC_kHeRtuVgX$(Xp8J;NWYS5o9HRYUv#9!vm*Bpk8=DB5*(2vm zO#crL9cNIc+U#1*j3zmu*H0{r?zECu*FZ+MfyW z+_@z0CwjXEe$HbxB80nz%P@fvo&Yo=UXKj}HHg7a zYN7(~BnTvjokM&;2r`=;c-)AyIn1=FlKj`Vv03sG+aZA(L5`+IK+gi&cjfu$@N`$0 ziVprLCaSNx;KCWP2^RZ`_tY_^nrsJ$i42keQ`YHAPFC;N`hQewO%|4k$4Zbb$EI45 zk&ky!X!~erKS-6!e#|>cil&|St{L`lVZV0Nc0Z5CX7qkUsY#&ewAY%kJ1z+#A|J*) zn~)!#9e9|%c~PS>%aCPrpd2a>=RkSwzm#$~l+(=2GN04wm!mF9EV} zoIg|Ein=ntN0v5%emO7Uyw>`p7|&s|w0K2BB>u_0yu%t)e@67c0PS7-6jOU!Z*e|& z&^^?%ywR=3I=Wykw6~%g8R}G)wtEczf%pEcdQ!R7z4(LK*}?DMd;;1Oz3e z8xf^TO1e`TL;5xdnG z<5J~S?e+Q}Wr1a43oZM~;-99;kpt|Uu(`{gYG=4Z<(OwtxJrI)5E_i-{aU*iM;@&Pd%q)P`k?y|~TfOS=G_q`rP zCj!A3-!+e>uxK%@pHj?}FWZS)meU*6?mx1TdiI|cgvNuzmo3OcY4j8fAD zQ7r)3o|xkKIY@2QfbUlDdM$F`DcVQ zpYsTGMykMSrvPNgI`Zm)C^*n7&rk$9XA~|7r4#5_AM>oQIL&p&oe>R(@m9bCNfmm{ zi&D&@Z=s-Bs|XA}njYQ0$wR^}lcCnAc7+9-5(ZMv(Ox4zCJtzycS2%%(yK*Ecg5SPbuinmyXkhG-4+HflAqn+q9e((VM! zy?IW*%R(2;UKo~X-0Z_DR@nW6OhuGr>?5f4aVkF99?49^8Qgw~@A4C0~@DzguPmKXVDfaT_eIJtHJ4zh%SDgXV==K)2| zwvWna!QTc5Hc*40T0U!m<)h3?p) z0V*G5mNU#%{D!GROxNMy0QA<&e zx|BqGAtX^7=fHJQ+vX>@I*KY=f!$*l+~x6qG1B@RP7%AzL2e<|W4zuYw0wE`r#Y3a=V=nR2cY@Pm`3@@ z32-W(BZ_f^8{M}pv=hsnVBxR#!LVB%{DCe{9z*C0%{6NDkhKI>VbP-qGb3gbVhauX zxeHBjGiKKczEzRwACjkayuGDA1SPOOV3_I#OyBtc#Af|Iy9Biq*j0vhF9D6cMB7k6 z>Hm(+qVItJ03%kzhrZ9hGeTDF(AlM+4`jKhCX$^;8QyVDfSks~+&7iH$*k}G1O$1I znj}$JHK@NMaEMui5;%&I(!fE=JRVh|c7@*cp$#q2Ov}uHGK`;FY*;V1SV`n*BP>@< zkJixGV~uyu7~!0`XyZJ6D4Mf>C8wMj=0x#ULot!Roa1hhig%YvM7-@Q*Y^^LU3D*Q zYia|qALM2f3ZMwrhl37>mz*&Z>l?HBTCW9J|F|x!L`Ix=Ito1trkfjwpkZ`;PUAq0 z0a$;%jkD1r++PUO^@webTJ;V}e2)t&zg#O!{EgyXWA*5jOOzW|*kZmzRs&>Bd8g?- z#&UZ2vjT!!y^&Lj9<2{?xv@&)dBCrTwbK(QjI0VHWhq^uMyi%yK9HbM1anmnX9a1y z{=S7h(u1$`Wv;a@=3wMGHVVrT?lo^+J%bkYca&x+2Orkmf!!&%%y9B2{nYtRWEe{j z^`gKJuRk~yfrP?1YHC8D{e77LcJnj{NdVlHPx_97<6UXM;^jSqS%16|7T~kxu$S1{ zA}Wqqn_K7~H)JmOaU`xY8l>Iz(@(Kl5#x74P9i zEnP6)wxoOy8=w1tx;c>7?b>$GGZNy3#r*Wi1+s-i5H6^h>tk}>Cq}v73zF0$<6A7u zWJwC(JPfClI4^sFqo`7S3&R0+r+oJjNQWuv+zVmACVmYDJ}FN`rN+=$bbTbZ>==5v zVpYSjdn>%lq{fMD@b|&)%W`Im@S>XdXM2WFf=dtjKKsH+YBM6^|Ed~5Io8@LjS#D# zQCAfqP;t0SVDnz6hJmw`s&uo-KwIyG!F|e~bcuDBBL)iN_i^#>uf!&*0>NY{&LCD* z1TJIY-bN64{MKYnfyD#MwAZ^{8ID?QOQ3j#uPsifWi@~brRl}f7~iAI5iugIS=w>< z%}-Ok#5uU7>+;5vEfhwQ<11t6+=ddzq znlOvPqdWx$f7tBJLCrnVFsw>_`U(a8X-7|&8ODC$Kfi;^_%Q3DDLKV*{;K?i=EqOx zi9QhcT0q}4#VsdHHQBJsC)0{l{^8o#4w0rXGeD}MIvkptYWj5sl$wfne^XHB- z6(UdfR0pkkhq(>5Xtzl?Hb(c z-QdWs=s+@Sl~Wfd+C4K+FlkXeUik+yJDN|8@v4-X#hKCw+bGjxQ zI4L9&o^<0ljKnrLRFulFX0{WZ?Jf5!6{%t z=?r5pUq5+nNrIoHj}wq(A%JhsPAd^CLzW5YX&EFAp$qB z!sXc5HxJ#HiV|_NjOP4`Ha=Ei4C>%Mu*DjYIu{qufW7yPc}lbPqrcx+>lGB3tND3z z2-}CKnmOhq(mjrot+yG9fW98_Yl2rZPky%%Ws5pJ&^Nb#KTi=Z(D8*q=I}~q4T_Sb z)JdBx2TM8r5ww&Uwz>x7tg`wTrQ)hdub2A>>2cr$B=& zjf@nlVPG!}d=?~!>ymBOas^e&H|8^~&PHonZg*I-4&j&D*s2SL>txUO1mbxXORwpk zXOq7*^X2A3@@i6B8`W#_OmmA9Us8G(Nu*FTHdgtvIYzr&yZ-QFup*tbG+n6rc}FCP z%0VJ@XAozS%Pqvd^7!UvPSjHgyr=CCPk~&~J!rLg*+gG`duW>=t|dpl)D5lIxRb*= zPpJT52bX&~^%7(@ZM4%|Ru7c5b6_es(zkm*pT)O>{t5Qk?P}f(&eossIXv}-`(_qD zD!Gny7^x`XLLE6M3{{I)l#UYa^y(n-`4&$^B=2ykm03ol7N=@eCT8H5W9#H&0|Wlw zc~bNrRb9sgypzgp9uRVr{2dP++BrM{)5@BfKI6P+zuN<{vd-5Z^ENr$_NwyKo$7ry z*_q{|&=vbMjnBKVpFhyEe$krB1sYJls*fLBJaT<-xABYW8yN&E`m!JlHIh=kn^H`d z*Kn&yNNMHs`rQMbyZ+n{qvO48`oYP>^QwuA>ebdC zz=)3Kn5u=BO}x3(K>PfUpwZ{{942 z?0b>H&h`y)wTY8R^A?fQ3xKSn?c>5!XDbv_awW;Hv0!)Pwakwn~c90gam8Y+* z%{nFJic0GkHw`QnpPt|&&4csxINQ)KL^+SZX|r$5CpE$hHZtX;$9~c5N@>iZbFPBA zlCcJ1bxuW~ZDY)s;P@nIkCD*G0u{{UY?1#;CZToHUK9`Xu(^_r&WJ{9)CAnaSgG8S z(|0zAh`NWr%e=G<-ASAWQ3>nUK1#*V1fBYkWPHz1424}d6x=KpYv~wb{W(^{EJ%@ffm#33+kMqB&&O>ZR-vYCAumgW6jP_fO|UslT9XgvnvS2Y zL7aJe$mpmX;-0T`39vIA2tc;$k(l=PNI;Pw}`M zc|ei-jL(rLrRluD&Np@UdUDEADm5SZjJDF;w?T?@eYk?o56&~k;-D!}+!HRcAj?&W zUr6aF(F{zdif4JFP%&&1`PNW-Om}*?SV|u)sX9WgTi$npA#6cUc?;V((RCyCZf?!H z`v-yCiV4N+glur8=Jx5=z7(=u)Jh3Up6CcZ35)E*r%`I~$O752#@6Gyq?4(z@QUSk>3c~hRN;LmmUzDoG~f1VndW{@?K3_AO~n1YK2NJu zuLBu6MV=yrPIc&P2Wy%bOBMe6roP}m!yophRtI5 zlRe7P`yy2nU6WP8#k45AU%W=~upsqGVqp2Uc%NcDA$z54g60#I^%>JrvU}Sc(&pcZ z62p&hT7H4Erp7`4*iY)yMQmcS{BVnl`9(8|zMuOHmU&pO+2Y>#)uOxJ_O4Q;;r*a7 zp+?HoHPk8O!1otKR)tAF_m2I%+PLfFT*Od^f)<*$TSoI&lzlDai=f3((YMY3O`=i4 zMMJQ$N8Pfpy3dYG%_nUEE1N{l;^y^9Xj4r*xn0{TB~dE7c!O}O?Y{FwCsmJUtDM;# ziQDXroj2nt-JH7lhuDi2vh+tE6m7|*^ieFcN3)GEPs!Ln91FO9_m974+*Z|zP)Z-s zE45!sp^mvfTCN^T@#P-pCr$bqTV4+7t7m!1=OJ@07uPkhNj=m-Elbqup;u50GaOXz zZ`;yi{Mbx3lB&6egp!+Go!7(KsGGbzwB&tv1Wa8$_B=;Q3QZa#@w@d3nC>TR7Kp10 zDrIENl)9v@mNQze%xc_M()56qv20PH4fD*w95E-NSmCL=@uXAzHnYr@E0@Q$SH3%4 zF41~G=TQ1*_?N>-_#vIcdoM|G@pAMun_EuN6LL2ARVy-4UZhj?Q9SaB^h3SoWvw5y z$Lfq;{+LwySgo3UB-bw4NMXgv@&P+K$EdKVI)(352L;nv2{q@egc_MuQg-+GaWBXI zM1kl`&7{$<0LLMz(1*9EbO%{5rS-Wu`FC$gr{(r?yUy#t*{HQha_h|A`~OGL)?72!XiSYsicO#f?WIZ`!SN#|L> zko)E=ZkFDq(pem@-c{@b=wZ#mbW2fm!UAXgAaUti5x z@s-z{YNmg7k5LJfJhOK+>YVTB4u2_e(r0g#j2>Peud=_n*_J%ojyXJyp9&&U8a}L< z&o`GZVQiCHCHjAlxhryQmmyjB_X|37*N{3z)RKf%%;6~I>-DKt&*FoXf$&FGR&HA@$I-_lCIxOY zst!N=L}E@DBd^kP)6^0rlWO-wNLfnQUFL>ejQQJc)S_Aee(u?J3lz$Wc9pZ}_ktGPFF4@+JQ=C^C=mtNh;zNIk zALX&`zoLW?Q4F`^-goKL7wCj+1&pmg`@Mkvd$nKdiAyI_DEl7vm{f^@wg(M6S?&aH zqEVdP4`pXUU$ea;>mlLRWOpB!ZNaZ~mN$i&)t zIvR;p1<&iO8y3?)og=9bBE^^eLtANFquYbWTO>iwz(qBnPxdxSV3hs4rhQ}``M&@I zdjdkd5hI7v{R<%}EL>=S+QFF`m;tr(ccq?ptIqX#4;i{?-i=?N53AduR7tE(3V-df zbTzjIld~{j=Z2t2f1>PW7VCc@i@WD-F@ob&fq_e{n{kd<%joGm-$(N+fn7ovaV9QOC0?U9N}?=^6sN_ zyJM+zLeC<~qHd*c>M%nR-Pp^W6^&0|tqR9C6xF2GifhjM6I1zVsT+Q^OSO9YZ z8A1)>R%S}e__TPp`b;|GpKiACGJ=^&60-CgEGY5bBIZLV)CsuJ7zP{5Ru1W>Wt@=- z5L{{czu=7O6-;0OcwB>g6rJopOXjS=&LWk;?s1u~(eUDU!m~2b9E3u}69fs^6=hYa zfHTkpG_C_A|6LYrm&~5tkpb$95mLXumzIIB|IYN4V=?{z9VQjt_RZK3n9xSz|4!(C zz%1Oq+x7LsXY)*e1w?uMhbxE9{M|V z@P>KH-}7pZRXx026S0yh6Pi+wly#Cc_Uk|4rZkJ2MxD*#Q@OuG`bV^SwJuI|%;KRt z{iv0xf5?9}(izMW&!~E_ktAi38*fdaU5CC80TC&Y9sPeWJwI?np`yG5W&vxe_dwNC z1DHqyP{}R=!b=JC?~F4kS&)FNM9#rKr0*v&ROUU((s33P) zYwGea_YnV`jLDu1YpE+RcYe{#GXK68d~%TLcfS66R!E5}0_uG~P~aXg0M_L1z+;Az z3^0Y|LQQQ$enqlA1(0=yZU3rF)gCYfs}?^rCe3sgR&@UV0+=ZnoNTzrK}CXS23);# zn=z*ams;IE*if4Z)x}klhA31B^F0D}1w_PH14^5jNW+-$?-qe{5VP& z*JUsa+lbzm3S+&+?`=A$h~7HWj~1|bp@(>^Ya?xG>f!vjo{hUJBNZ}TPzH6KeN+bC zI0YbwaHY-{vb92_eeHt4u;ljIgud9$u-Q+kO~h~rp9u^v?PhBxK(2?KNZH8u*abOn z&g3u#axJ!UoUH_;mo`@+J3&prTzF9zTR1=g0bVIM;k{FvOX%akrj)}p`z=mS@x;7CO>P!SDXtgdec8JFn9+PTUz=yo^kP$x)yl%UtunJp3}5 zGc1P}1Uf+%)u%8oys7z6Y)y(d=75&372=o!El<&w&eTEF0g|HHdX}vGh&M^V0Srjt zkdbG1V%dO}K7ea_U(#kJm{39;(U&{1oyPN37`6eXqYk*>6wP&tAgf`|*bvJc6~oe= zjfOwp?jx>8Oy5dhWM6PsJtt5)*%hEnYL(G7MfUf&+`T2xpQt4L>Pm`BC!TFpDPI(F zO!NxE!rMlWOpxs+=2FB|_%w;a0nIyNQ%>hJfZww60*+oob@Y+1&DbYQTpMY+nq#`1 zlqNuFqTa!2X<*tV5yTQyH(A#*fjP{$0%Zd{iYFo_p=VY(394hAzsIrv?$$N4dk3)! z(yJH)n7puC`&iD-{~tIwnTWy^3LT%5m>ioF$Jb=$KVZ8 zODr(B^&W~P&yqX_bsuZOET-R<52E%N0zXCPi#%+54qb>63OAfsDpN+2>4GlEHBMepP>aq!e2^KEbOdS}&t~R7%BC4^bh3$~wg=~- zxUUBI=__5#t9IAbiEhP1UykU(e@}1knD3Qy&}@Uy^WkXh6a70R#Yd(1M}cKiM9-~a zW%rb|LtNQVrLm{bHTvk(Ze^B^@=n35s%aY0-*=ekc(KKl5HyPY7WW~PH**!vF3u(r zVrjZv(`3-)mlp#B@+~x|;hl6=r{7!}_y%%%h#k&U%Q}Hq<19KZ(oOcq|L;Y7NR=S| zFkbaglHdO;?2opbh(mOPfEBL0A8H45U_5(;RGIv55MmNU}JpoKufb zKq!V4bhjM?s%3b7WklFl6XZQF$wk*!P3IJSWeJeDh4BDel8bPb!rC4Tm>H#K$ z2wG-H?%1|zaxCsPDD&ha>PZ8+cPan$Qmu$Sz+MUUZpET}vGXLEi00DkLpY<`u-{gh zlkSdE<-QCE5?10HlcZp?pmBCmP5~L`?36t(0dc12QrI+wAU)HcHrtHIy?f0&07|W~ zhtmaS^B0dI%ML#QT=6*ij`nTo3#W(zx?$9%o9#Nk+`whPg6(2$00Esa%4-V?Br?zA z5r_kE@^^T#zJQ~f+3YN+gAr+vbrN(_-c32<&MeUQLqSCU8{+8GZcJ70Z}Q#TE0=g0 zQDfRPziD8WDxyQc}k4sT)4`TKZTN1=S05AQ@ zV@C9*zcHqp@hq?Hdlr=CpSiWzRED(foa}T`}?gcopea6!Esqp=6Z^ z{h#w9ItY1P7x6GDl(bfyZMso3GTCD10^e2vZ)0kY!#^tO?DF2V<&wR8w@ow~sy=P3BJK=WZ zBS?Jpz8)5jwn9=fWEuO*WP@k!B(abG9&=*f&BlspaI2!^LVjfyQBYX?XV(5dhW^s9 zm`rB=uQAj9$!7oeAUp&yZJ7iaga2PLjf>s^J|I7X=Zj#R8qh(%v{nn9d~YExsax|J z;H7mzsVzZ>p@kS$2>*Ox_LjB#xDkmJ!9CN!8xD3PF#cHrJ)JHC^(4;_Q6(1$bsk1c zBf$einp#!1_m%iHN4%$Qx~>eD*M5lvy(sphD&~y!>pXd>cEanQPN6$=2^1JKz+2@0 z&9B!3Q{d9qh%bZar?~(TKtX(g5qBhQ#Nf+Fn?jY&KWjN^smOl95SH1_*)mJo0iZ4Q zPt~9vM+E+KP8eCG#3_6Qs5}>TAwMxxycV?qO{%``QTAKYs zo)sV=JGTjXj2ck1Q3G%BFh)cxMi~4PQBVjlNaEy=bmaI?ykP_)^94nYY=H4KA*YJ- zG6GiU+$Oof=6g8y10?}22`@x^vVpN8Rt2<{7Y9wi0(q;FE^dT}8P^1-C<-cq5U z4QpdU=L|PhegcH{6lc1c+_MB?uB?nS4e-pd`#~G)|!Qg8c~+}SE|38^1FMj zfB`KciNK$H12XHrQD)^H0RQw2Jr;BE@H`J;l+r_$3p!#LN(3g|&9)(ByGFbK(R`cE1*sm0^l$@- z!0jc5uJ2L0IF|83)DpBp>(b9v=YD0sUxUb;r_Q&B~~9d`cPpstGx#uvlVwz7Z|LPF8cJ7yr-;h70c)oZ7JIG7- z>1dYi8kCj|a@{7Mveo4?d{I0=`!uc(jY!$V1ULBhDX_iV0mZQ1veh=3OE}xIl!h9vr#&)6{2K>K^17nj;DBVh#*}Wv z&^s-D`RMZBO#ejQLYTE@K3f6bH+w|J(}iV~bCe?o-u%Ah%)2$X@hgbcMz40*)=X#} z?>3)mI^d6h)*OpoYgDTdRUTK-6727@%&biug&-7lAo_=bWl;RPj$!Tc?@I$(cYkHw z=JW_`UW9ZoYtrY?cJ{~H@W4?J{sKE3F7g4&lBVfqf*xTIPfVS@?$ z`t8E<>7TXk z=f3W^VD>E3Y8ohL{F{E{1_ALQKuPS#844mx0w+)~uFaV@TZD0n<9v=wjQMdm4H2PU ztDZ*uo;RuPN-Saf$NPlw&g0Uv)A*)5Oin7HOipoM{bLvsXk?@FoO=y@(7kgUgx0dh zz*vXlPF2Qnppy8p-^2LUZ5-e2^U!2IXTdNRzvC(JD``~>WEd4wJ|^*v+}uSxik@5) zUD*iSlmJft69#X4>{be`2aTP=)$bH-R*$yR+R&p87G+z^M^v>xnu%+KK!%DM?`LVu zVUXmo?=WHjA-*d_4<8tP@%p%J@zKmT?x~6%)QMD#WP%k!I zVi7y7ZC+MlPWut=PTjx7RGCYA&ifW>D$_}JVI?k+0#e(`^l~ppe3Ex!g?^8$9o?k3YS2Jf6Igvc#dFk(bU#%ZSytvVQW=QN zAbaGb>-Ue=_|6yE*bpbJ_t^2QQeG-jZ>t8zV{sa7^>9DZD>AHIPx2{_7hEK%m7&17 z!!t^hr%bbzITp0v|WfIYI=6F)~`j7R;V3Iv?oKf@p=j73?c7%^m>HHSXtF@g^BW| zbO957juOuxh|MjRI<#=a=X_49RgBDEWCoNlD5jFGyH z?4!QR2lI#p(YCa)Ml`vtdCX7Z11-*!L2GfHOFe(~*YB{rgf!Tkd2XuLrxV z@$htbfm5)3?NuV?6X7RP? z__sPuWZv*k5&v=Tl>+uEMQhrr5NXU2Kc~PFm#Q9uk^Xx97rZ8hg_S1KFYPOP<$g9P z6=Y6rb38S>6--)x&-@}?Reu3iMR9ImpDw80VZ5$&!Im7MEbw@@jS3fUe(H#5s z&4LL9W_&}^Tiqs{fTTO@p+=>e`vh&JJbzCyO2I{5qzSx$@Zx%W^?AMDG&pYuEiX&5gmq)OkBwigU{0$sK9elvW)am)x;u zC_{Q;8*OwZR4MNejmgMI8cZEeskBc~=YKJNZqYoPeJ&=6cAr=R-XnPLGW{%=DditO zv(}Qwf$--N@srK~7jfj>nutl-gYhwzJ*1+#xDl&ic}w1CHQjD4`v+^MKFew(kD9qW zux#vS=(oB^&pWB{HIm@0Jd)=&`5t=#x;RIs3|^>5?6l!TmKt58yn4IQq-YE7(FvPi zZ2aj6j$(_o#TZ$Q)i$}$^cO32SZ$^4DYYV1+u}nFiKf1wu@V#>;&Hq)S&HIXUSXsV z7+aN@*?J%=Vu`Dd_dyun3GIO^T#d=L3R=%=zu}`Oou=_Jn^Rt}yJb_!xN}K!*`rDN zuz>QZtlanpP`J|_i6jrFo#OdA$1c{Q4d7M%YuEAwGJ1hyv*u|a`*fIEC(jFaU|@|~ zONR9B&frJyXxqu+6!NdN2}gyJ>J+LOzq`;v6-eZqwk=d_Q~PKEt1<7`L&{cx$A@!K zHt<`h_{OUcd+|>9`SjX5nS`4rRiMsGwZG#YS$J1%CL&KzdXaMGwXI4V($}Oxy`%rh z*djBG?Ba8-Sc|7(1X&o&QNiXjZUhXUtuochA2%C?o?00=kXY>*;^|7gGW8Cr@`TcO z3X6q&(RpXd}nEjb-^8yX)0SdCrE^S}c=P&>m%5 zmUax0Yc^H3;!|c+0nxk37IoiFtmN>31_eJ3C%h3Cd{8BIXyjz4mq*Voq#xg4vN$4JN>MM+L^H+b(dPVsc zzx?C>AN&$Po;RIj7w1R^WjOH?K}In#r^|zLk($-h-Gx)JR$VP}MMyd-^xTLVnwHBR z+L%2K$5&eFQp z)&7yY5%R8q#7Y+j*LXPxje-7y4w_F=(@qj^#GQg(xu;ZZLW$=%KW5o{H+&zHU0Z8H zFhDl-!GA(v1=}vs{II!ea@swRF{sNW=#VgAWgNqJ$rb0{;t~B9S_A=92?hi(&ow1O zHG=WF&I=t5KAD}5dRR_0lruP0Iw{kKH1fp;LDZYm+EK34145j`Ld`4XT{98UQGwL%*2ad`L7X5R1lX5{Zwf{Jyv6 zIryNB6^HGUnuJ!;exu2uxlcN(^7q|OVznNAv2Cxioqh#r8+#9|D5C8~gHf;s1&M9Z z`7XnFztzLvPu&vnn+N5jZ&E#0D3Ho3^A#FJY#E&`-kyXhR|;KhY9cLeI9_*@R z)ED}2TCIhXq-vJ#Fm>MdxR&uysg0TwZPw7>=1Fs7eRN!o`>Bx8uwy9Q__S*%wp#$1 z!HFb#5CbVoEq2Y6RI^a4lRrpHSvV-%aTt>PJ+Pa_SSNRN?zPZ~-qTgspxvTT#9GZ*lEpq)R54Uw%@h;imV2op3a- z|28Y0y7aksB)EF|di>S)d*NbT?3f-4*F{-wz46QIe;?u^`^@u_>atfG59ZQrF|Of< zEHyQhto)isnDRGMwd)jh{R?{f-TZ0Bdzu;i%`&4qLpm5%$1_YcN=^?dF6_ML+zuu@ zJ|1gM48|7NzHWYq5jj1re#{$hPO$MCOM6zkCi-V)0&aF#&> z7qr^H+b;u-yo)|jc~7b-8lJ11dRyP5Jg_cj>_JM;y4KbG7w3iZurTdFsCWCm76Pgs z_f;1ULvHkHJ9(!+Yb-FVh+*5)zU}xHe$!#mPy}=P&i&?!lbUbzyei`r6x#%`Ur)W&YZm&mnDFac=QJ-3=~Fx0-on*XD6&f9hKhlEWz)8l5meY0rV zTYb8eG!m?}dFO{vE5OFDsvwa`^PaZ3Qo}f^f32B+zE&)MN`w2|_UK5-%@cza$}sVL z+h!x7L#KI-CeEpyr{=Ad%|umZ$3M)BqfMs3rq zLw#vgey2y}8#Rq)hl+0g6N$07FfB49wn4P)hY!DR`Yi{|f$zT!t z@Me$BxUe$GtWgc@l_Ux3F43<6ml_5(-@H6Nug8}n|AJmj$1*4!)P}Y!tLWJq2Rq;^ zp9~FuU(%=5a|rZ5DLVO5RCF*Z&RG|e^gZ)X{!ad(y?XXmkav(ae%diRL(%0^3*Vjj z2Q4>>2A19(5l(nlHcl4u*Ed#)(aLlEyk>0bu9##l#%NG8y|tun|53|+=#yAt^7M%! zz6+nRcSuM0;5JPOxkmo;nzxj%LXX)Nptd&uDBXMKUj9wTncL}dTu*@YbvKVsK+*sH z-FMq$ZdT;4P+1g?H0eW8R5w0!1N_od^;F$*@5mT!M3=$kN1d?G(cV|rL6=sH$O0D_S#n;QO9?icwRS1(y%r#<36CFHNdyNFH+TFodeswjIy$n(4)NT}St5A7EJKljs^`RA^j5ljuw0qB| ziJ{|Yuu3+nMTxF8y0B%e6}kPf5WsllYu`U3F!bqn2s*V4w~M&^vhk9B?sm&KREpN# zsW_#F<_PCM;6le?RK1tMy)9?6?P@q4IpN)SS4nb0WLaE3V%2imVe?`SBgv0KIp-dB zGJ&{f4DVa>nxog5h0HLC!<7I2nc4ek+yag-Z-((6nPmB$58pEym}qN{b%lyp^!nT_ zlkEMe0M;B4W2nx_V4FajGMKY(x5LzI5AZRTzbKtAr+vFNi2EJpA+5Xjo^dAn&p|- z^Yk=?W5lZ3iTIG9zm_P^dmUw&>DQj9pT>a?dy_G1wY^d%c*o9*;-Sc&>* zXz*Dk>;P)W^j@>qAMdL8TF!>P#Hnc0e=);P;HrDz+2epv=w%FbN`cl!{DQ!hcs~zf zTJa?wNwih_57cK5dJh=(^GEfE@~ z(|<=XTE94+1rwE2Dnf}|q%e=I)+LzYCwpdYGvRD?&~o{apE~NNDHJGu+`iuEs0E0= z{(ZC{4&(JB%zrnT_ufSBtM6fcbrpAqhT35UaF7`o#rKD*(-3^^qcspgbb*d#H-rsJ z1HMh3(C|C~Ww|w#X~1Y5kq$C#99bgI0Kwl{zgQD;Rw3$f=g2wQE+fqe#wD6Y+sBPJSQo0j> zx6jU+f+cbhn|a{%_}FCzQ#gV9i0|Uz<4~L&#{cn6B>(SkYJhvx4l2+t0BJcOJbfB5 z59f3lq;*G`X!|WqSb!l`qd5;I$~>~X`6AZ8gCq8w7g?Fg!Rawjz4{&Tjq!)+)r|e| zJN(YjHR|3%2Mu`Qs!d?)F0y6<&D=0%yRYgY7o33B?NP_nog}cG2uZze8sG_hg$Yni zUHA#*qndVV>smUu@!lx7Pwai5U%M7DH};0+$);d0pgp;OoqOHsJ+4_rvgg29>Y9KO z$r`lQ?oOzpBE+Gg^8G8f$q@u4HLVaYQOtg#noOGX2ADR(yI4Pf{y$+FO81kYK_l`A z26c$-Rq-}Fj}w9dB2e!H)F&5W$^a7(&|lXl1MGoeQttS>#KFR4L2%&O;~Vt>r)Z0b z#9bU`rN5i7-qq}RgBki#ywq}&2%g2n6QS43hfwkNjuiob<2|r##=9ZrNf6zKwK0h} z3^ZvXOJx_W^5C(xTC5l3g>?|@)&f1J^J&@QoLI7o)|h20hT9<`P|p7-VeS1 zE;Rd0ykPJ=UNm_|O|(}Q4?gfms!nGzY%iuE{ox+V&Rj?j@)#6-kDK@L{q5lNb-wR5 zp;-QJvEGrtH|1)L0=H?V{x>@FeWetJ>Fx9nZFKPM z+w>f020g}fpqKL41K;OxWf^)tca{Z*O#;Q2+a%s;j``Y2Us`|P`eU_C>?KWXh=-EI$f+T8fIfW0qF{lRI>LA#ZsT*7O8!YZWSL{Y zN#ulEgV&;cV)`}7+njFHwGRQ3kI?We9$gby$?|hCwqR$}RJAB-dl`+;m-Zx`imC>| zWW^z!Z~CV7;`i(E|rT8|SzX^g-a_ajsby*LqbO3Wa#127xXMUSqTG(tr?IL|aA+~AuS>F{ z781JyuXYLIg%IFL3Tm)5s*JPQYW$>tt!PDPYHZbw$E7E8M{7UtV$YWf#(lCqu&AM? zv2Q3N!U5yh#IN)MsMjiHL_9mVqE#4J!=oi9mE3_?sG>0!5BmwZqYY)?Q2y)y+IFh( z(A9z=Ib-kbq*H^|V!fg^Xuqr9+eTWtbq0R%V|K!s;`yXeue*w2Ibp(paZnK3_ay z)w*CKFMJc*HK+kNT*R7g&F#+e&uJ|njWrt`-cPsTnx5{018@?F=zR>AjR!|pV;)b4 zd!^fXy8^M|AXM080`cFfc5Gx8w6%i{~&=tWtz7f@S1i z9lZZ@d%&u1jq%IG#^%2kvVChOTZ>>`^qu)&%wt=wPbbq-xvv05c!j`Po+Us4!xCew) z7jPoce9{c-V&{5h`lnuc;jI$t8I@Km9lj5t+udvZ(uf_&t>xfs0Yh>2IIQfkR7 zbgK(7kB`fa1t<-j<7f)_4Qb#OrKRhbJptf)0*FWp`aAq{lfaE!1cI7n{xll@4MKeM ziHm=)cDu>c+_n05O_x_5TbyZS$oM|cUq~R=j~JOuP_1ynJbqrHL+;9;4y2z@?yZ$>2 zz>=)}@dIEsGOa%ml0$!Q_C{;LN1sK0W`2y9s#xQOYqvP^EVxK&F?5?4cmY~{#|89* zJhygDiy~#(4%pmiN2Igu*?R^Uq8Le^x1s~>yJ^(EdD3G!xI49kT0zFVK`g@u2m!~! zQJtnm^4+U#G=|1PF@vF`ZljiL%}xCnA&P~wBengefOtA;o=*lq;1v%u0ErW7W3LU_bZ=d4($klEcI0UxvTZ$ATL$)e1kmp3tQs?TuG~0% zDPDtpE;Ze9>zNGUt(<*bPhwiR71jB7nY$md1{bgEh3w4Kg2HHvn|^IMCapr=-f)DgO!To`2vrcJ(|71a%g#}lya(_(M_W+ku#=ISCspj-c>O_wTA5naJsf`%{ zob-#mUFj6NN3KuRNmYNT-4(ik#W($ReIwTz&mJpPmom!YMi*xVLEU@w3E-HRnz0;oG zPl>TCJ`;LQ6xbU_i8DnVy~$W&7W=@#781MtMwZC<+TP&Sl1Eq^mcJc7-1Pdje@-0N zSNe*oFi{inMT?oxlQ-^B<%w_wwwzcsSu|iMu3!BM0;-sw}9{W zy~c0Z1I&`%qSee}V1A*u7Kvr8bDT# zh(+J}u>xqB0`-P?w>gOSuvgSh!Cn_~^ z1#qD6WAbgk4_E^3!njJR-DS5uUGi;{tmiu;gn#IQHE;+R{OeFOWdma_jtt^%n^TXa zCBLNa+qAVqqQJmy)RgXDC6k`#X* z<-1eNg-}}@$WVhp3gsKa$Rne-3j~{V&(Tx9M`<9x`}Xgb(%*g0lo8FEAM*LBx@4$= zoC7uTa~8<|1|DAgH^1JJ&-Rl`4>mwjzn=R*X?yJ?NEG6kki}+?Q{ef*AU#l3aycQ{ z02lc=;{9sGzAJHl?O3VqeTZLOYF*%p$RLf_;2(Mxjy)VC!OVy}dw}}r;cqpOH6|*q zkb{%`Ji!h}LI|u(zYyO)XK63;R*tjn<|8)4c4YdEO5PJ=`r$}@pga4M*7dgV`;vi} zCGf!K&}B#%Gw36|^-^&7?GTG#A#D zZ)(>$0)CaA{RC$YF`Mvzu1X5I5^rjNE{SO9*}zLOhMwDYC9DjJN3!V=(ANo)*aSB@ zwcbk(I!L06OwKhh2hqINy^2%N|JALS8hJtF|4U}Dph1SfIC9}XUylrI3OXjr@~4P) z6f%_XNRW)jL-ki7R%W6bvYwv_$ek*q^kwItpZ1u65-WY6ysHrMV9)Y{|BoN};@60& zTb%#-wp)niX$EUzkY8Qa*E|y~%9AQE0+^qm8OSxwf>seeX}F;)|#T?;i*#z<82U#=VQ|NCNok_zy$tWxDP z|BlBwe?F}LWsE}WQh_`-$OgV=UsV&zp~Q1GI$Jtm1k^DVM;8rGOfYLX{eSOWuD&2| zee%x~I>EyDC3%+}sXV+?YJG?Ex{dj4a$*q^A#6Qq1!Oa=AeLdQ3=)q-=9~7a%al}`478-pI;qIXXe_^{B)m9g*?b?GOw6> z{xf0=Ww7{SS}%*x!h>CE@Bi?B`p7nTk7R$-Qk*@3KP@8I+e4}-4N3f?`~T;QALudl z5}V-VbWIrJpXG55T^FmMrb6-m+0vb|jHR=Fo(ok52OEf(qT=8XE2Od;G`s`J4<5k{ z4OVAsp09$WIZz()5>i;8$62&u_fKGxuV#{n=Y5R(=LUS$9|Ein<0n=Gd z=0S@BpkFVEC@k)+jdao5e@{1_{{MBS{vrE&3hU^@9~-SFBSo^&)oiK(Oi2qK_Xatj13uVXLxix6rBh9>vNi4w5% z=AcjtH2TVWhIR6Ap#>9wrLCf!5F(u}(Q|N^`%Ct~SA>uVC{-SDPeMf06=1m)yq|eT z9YV+n=#$nT9zF1ae8MiE(XYxGuJ^-*0z81ZTR^@O5h6jOfixOO^h`zopr0B1zs)&Kwi literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/4-format-transformation.png b/website/www/site/static/images/blog/scaling-streaming-workload/4-format-transformation.png new file mode 100644 index 0000000000000000000000000000000000000000..751a105ef10df10df3f72c5a2fe05536617ca94c GIT binary patch literal 27127 zcmbrmbyOWq*FA_saDuxAce%J*NYLQ!a&dQehX4rF<$tQpqobz7>ctEx-(*=HZ3loe&skcp6?prFv?-b#Idf`TRm4qGGy;LaZ$J96Mc z11u@2EGH>Rq3rBn0k$=Vf};QEme}| zvrbEB(|t}w&6kK5k(WkFMKI?FwZa?1KtPc0XdUGeA z2P1sZa+anQCfC+oi-gO;ltO|%HJvoW&EbxHQ@h^ck$Xq#Tehb7wK4YBK|9yZpY(_| zBHoGpUxop5dt?qAv9!>^&T7l+7zq8U4l?D=of8l*%MZz|{SuP~d%Y)1PVO)!0>o7{oH~xK0&J z!$1PB)r&$OWK1OBgvzDnRKK~_Vq_LBw;;6k@%pL&R zFSq$L;yJuYF>*uvp*hSU}N*}@L=`eVs&t~WaHrH=VxQ*WaH#y0aCEIc-gxed$QQOQ2$fNf69?E zcQJJaJGz1$>?vN#H8yc@a}}nddU?_R{QXl;b5HPp-(>IdZ?k{~vb{WE<6vcH`=7Fb ztU@n$1(d;_=C;~WU^}3DfOm*+fw+bKrvHCD`R^P5S5B?}=H%q#`QJJJ*OUKyPIVV^ zXGsS;;FYc-|NS!mX8zv~|IH}G_R{kI)f4}8^WVEbKZ_s>vHj0C6G6U?rvC~BB?cuY zC9dWPeUy!uh9iv~zEqJ08=Y@5N{E^UC5{CfO@P!<3g;rF7E3~Jfk|zF8fnrQN%l7{rDUs)&HOFud}#n)lmja zo^8aikrh5ZUazrw?G>e>Px{{<@uoI(BQwE)MW1d8-j^G8_$PvVu543hzZv&X9L2D; zJAw}?>J$?}HnYsB4Zp&P5x`s)!;augy9VvVt5Ly|CP-!Ds=!;H$ICf=on%^tV8~b1 zl4eM;LM8$@nNGdPLb-qoo=FV2+A&}Bnx| z-SzGv<~jY%LQkRBs_vAWGrLYIFaT1>VOAw%1GWZF-b`Sn^1{fSw>FO7ZYGEh-oM)2 z#SP+=qyXn=W-6w)iSZCtExK9<>#Q&ofU=v??XZ0U(wRP|ExopdGnGfPLUKKcP_F06Wf!?Bt%#M9yHBVLm4J;z~HD9m# ztUQL{8_a)OH6LPW=Cq8JEqJW?&AqBh5V`w#*sx@8KXa(-(6*(qkgw0&xa`d8 zaMJF$BZ{o&>v9-ct1-&MLzz)}62 zd$-lsqR*UYMBI+uDWCeY1>Bs~YdLQVlNC9E8N)fPkgKT!elzF5+4`rC*rKw2*7-69 z;V1HD9PjFU#rN%uD)hTyt#R}{LBnVQayazqn$38@=7WrfxqNAqsS0f_RTUML#2tR9BY2p**zP;%;nZJ^sB!G zI&+90fy9lu` z$F(NlV`ht)F(c7Og6=L4cJ3Yysyi~EVG#%>mW`2vpjk=5pTi#tA+3>O9 zoz1%TMigive0ij2g*%P#-pz{`{#uf=&mAXxy*#xp3ppPZB!=K2jldfRJDM=|T@T&v)rZJQrk#V2I4W;RwrD zD3(aP`INWyCEds&`n3*DDAule6s67pD2`?@Jt^BFm-#*SPdQbVl3A25wWE9vnk z0wDSC+1jQrpZ7{~K~u#FDADYGSP~S}y-yG4N;^12bXsp58VZoIwns7rDXDK_)(13L zb-f9CY`uIZ+jz38+wTTP9Q=qTTDO9rrh=oe=@8tLBr$}i7kJYoq-O4CG!0kSvoR&s zzsPmYt5Bh#eX(0==&JlyN&B{98?J8$!v~oN3kD%mG3{IVTeVO}fg=@kG?e>z%()A+ z{NtuQlE)H*KRKzbRm#6b50x8Nz019xZg;XzvGec;K3~3GLhCNN)eE8~+)G8GHjJYK z=j1-Wde2rz;-#H}6T}B+gjuZ7deQp(`SCjU_II)pj6xv8>yGvD;!OK}2xg64r+G6;<1{X_P7-`-=ak<2k3^YqoXyHSpW5=y6pIsR0>X$Z)`6g9Aa7wPx_ z%;zk;uIZ?+M7@(xha#Bn)DVWq8k1!i)6V$bX{^i-Bzd=X9gkPX*d5cvscf1$YU^(e zxN}7O?)Z9Y3%6Y4Gj@R9^A-5&HHZz}4)x~)Ht|mT{jrOo|Eytv-&BDt1|kv&rNLkWx2M98g?zpa23sxMZL#qal)*>0c9bN5jFqUuslrK9)I zt|*GNk)XK=kXo?A9c?A8zz?LZ%`Jo@kDJ*wCf+IXW~L$>Box%zB+vk-pRQcRbw< zJ?5X6lhJa+6@U1(hb0j?HaUZ}Rmz*&Iy3X>NQ4O8uQ>PO576l#IGPx1BP&5GCnUny zQG(&Hn?<}QIJP|!EVUVhbao4D)Y!2jkOTSV`t^Yf$hw`FP`)OF1un&E`QaetVehE%Qnb*x-_lti=P}}kn?K+ zvD+Gb-5vqb;&5=YW3O-nu-vTK0{cn*s=b~e4GS>W@B~@|Z%gMUjfFppz--aFzI%>( zBNz6R&{c0^xbbpw%>-BfNl|mGc_?_UxOqyz=4fn&@QO?wj zw%3wRP8axqdPE0QG7H#FQ)VtfnArN@E0^1mWB9P%_5$OczU6}&rv;V zrO)-TI8n}wa(_H&2)D+m1R{x}SfWfRTx6K#2lCNi%y!8nT7yxk_GE6S$k-)#9(zP} zjlFl;P8>+peQt51j|sjTrE&%gbn!9Fx2fEV!5i+?g$YSY`Bxlxa;CAErk}aBLlHqq z2o{(G?}+9lj99=zu#uKjU?HYROFFQSYNRDISjZ>RLwwRxmSUAqZc{(lvympw1AT)u zg?m*Y(nEdHGlFsz3_OVNynch06S$5gmz+p+npD_jiYY_~-pHV2Dv-lByZ|Q@%1Na27i&Qp& z9m7-MX)>B9u|=UHQvO6w>R*1I7x40DPIIlSCYNW3ShLBMDmT-F_%2P`m1>_zM0}f! z6BxT4HSkkfi|w#DQz#Bz2k>bMhYFYzXVTZ;Nza~2onQsV99f&`E8K{v+|S=h&zgjt z4Dg@$dCn=jXHisGXF&HA9zhaRa^lZ?^NxT@xi5lCB$lSn7$`dZm6F$_ul$GNZ;a6eCTVeQ$N*(rki;nXk@kD?U#pB zY*=Ap%J(oDU*%BgBXvoyyIko}B{o(le!eGprL4#;+>l|+B>?kHG8DUrRAXZBb6aS` zRx*>_X$(UzI_$I(#rZDV!4O-=;RbfyQc*?OC+EVvD2ktj`nKq?nWEf+?!SKP!DQ}O zA7nbTI}t>FrauZzKoW0=6)#9~8t_}RTBt3GfmO(?HOaaU<$iNPA5hCr;ThP1xk*9Z zJKx!cX;Km)u}M(^3_xPD$pMhx=mRE^K@4k*NqU zNe<=UO*as_X1gEuhvIXr^f=+Ep4e20@4^J3T71J_8A}DaUIpf3NU+mEFDwM7?kQEV z=YOEp)HUk4Kn{9KPUd%^!hlC|zM`K3gCvASfrQWQ32Z9M1W3pU>&B@_P9frNH{Vf; z!a20xT6`M($I z0#3rx5{?_`>OuxHZ{?ftAA#9~$ZUK8kK@BbwHuPW5hpa^htnQ${#49!!-j^R(MK^~ z4s7E>a`9WVr0m3yI-6Fc&F_gIyrF`@iBC~%_^Oib35nT)Sk}uWq!6Z1j+TVlH)wk| zMwGkeb%w1>khyxx!v|De#l%U@2QU5ZvB6tyk-}xQ|lAg@=w~Xe z%E6&~euKxP0rOx9{he{`kAt7bba&amS(VhPSMAluKpFJ^(x<7bALXW4Bc&^lC#b@X zgOdrX!6G{3@b4qEGG` zSYhkpuCAx*yQeR;^XUcL2LZsaK!A>hqWXm^rGJJqz+KRk$;@Sz||K_BP1E2v~D#~U}< zseIIq2X>3+r^gV#p&BK2^+I8h87_De#(r)jXi5;yeq4dA-N#f0bV0jrWrB_g^a+#h zl#}CpB;tK>C^3S%bOX)?@_>g4Fn{gOc!ufFSJqzd}DU9o`xOz{x38 zS;756KC=XXOsCWD5K8(dC7_B&>Njb8`2R#@vI)HIp%;>{WuK=?Gx_a>W zj-6W$Sc9sX^2OTnr1M$v4Dj8i)1@l*b|@JmbA|y&?PpRRD^l!Tg-GylBn?>HG>3BR zOzwHW=%@s24#BwSdeD3mrnmf3{!5kF8999Izi;7LHX4&4*@_gHDkWFzO|#|R!0Uc9 z72>Jc`Ca>cLc{HPK$Y~($ayo>cY)r6o5$VwWM73}bEMSt9g|iJ6Nc@2WuU8cUysJ& zPrUetRtgZxsZV-dTLx7PX|ug?s)NoId&l_~xWu}yB~y+oKio&pM=xVhj;gOOtcTM$ z2(5_oact1w#_>?6+BG;`v*^nY5k?dKaJ<84vkz$#7z0iDALO+H#xnu{l z0BPXcB|*1`J~Psb~4LF=2fmpGB;7yH! z4zkSO9Wc;Z0Og5r+5%&88|FUq+|7W+=4${$Kw|2k<5LxlRAk3;iS);A{ zvV_H{$8PUCm9jOGlpUMgrKmzqal&f5W+*k^#kge1pCTSd@Os`@<0sV?waL3ARK13M zB2iTzH1#Y4T(1%RhzBJmgyE6*#oup3S_@VD?fo@RmhYNAF?E#lPPhNA<{P=9o^GEe zPhGnNe#?6d`r{IOoNhP!@PBX7dW_-Qp6Qijz1qiVfuRa? zu%{U0a@-OroALM+0cXnBd^#A}CY#k7ll2ED4My)Lv-4<3nn!GTD(%Ku|BUV@&3kBc zi90>Gs4~+@PYT!kZpykJzpMVilmaP*f;g2O?_9KgUc_1RX;`QUzkNp`_NC192NaXz z_5coDa+&`G*ZD_aVZsuyH4FZtr}xz44W`TWfBWckIcc?glUM{){7(GQFAFLsLUP8) zmJ|;IZ0HzO&Kn!?JK0q4$x*cXawkN#%6s#U^3w7kVmZ~DpLQX^>T%%?%iho*<$Nz< znq1T2@V?7bG3w>p=3(dP6mf44TN?{u@hyw9YdNc?e@jWbhrlYB*^BfyS@C_kVo7N< zI_A);E{~1|v(X0VwV91IlD`y#5Hezy8{!~($6h}aBwIi>(}Z>ojW#Q^T;uf?oTgPD?E1SQ1VClv`cWI zgGJ8+w_gqMw4N&G_9Dt^6HQ-po6{#8D{0vNL>RXp22f;$;bJKz1ww{aJ*ydtiyzp9 zC}v!{t#-dtv10^r^JIHfs$~OH7c2KUd*>H-k)fdVC$|gqUoQEqlvP+uO~%1Y#!{2=T_(hju3u1 zZ~dA}uh~%;w!#~ch2+K(d*=aFs`&@D)`SVY{%5Q;oy>p*d<-BlL-DlOIWlqryam_Y zmj}SiC_^gzx+ldU2iTl;0$>k%oqz@#2nl_v@~bs!(M4s25^c5)Ziaaw*2NMu-sSm< z4*@e)&Wv+YC2iLBZI=k=i_@CJrb~nv=X?c)&iDDzW4p~`&0UUt2!YsE&1Q^t zIpKqdlFm=58o?=`kWol#?ps%_G8XPmZ8mE)GiTfNiCZwHPik(f z$c1`IytQ`bedOqJ-1lp?JzC(8RrC|m9p0zXEP3yC`6zTmyO@;QLPnb{2Fp)=td@iM z8GW24Q7qanXA>$t`VdbtQ)!i=+wc`>lv1DTF_MfZ;C?&NWm%)!db~}nqPwMjLG(+m z_RV|64?dYDiZxTcU{>QGN&Jno8~39X3=v*(zq4$wKU=)|6FF_8PIHB%(@*Sc`_?JX zQ%a&74y__1uA+BvC1}hwy-7S}bh#4!BtFjU?dYT!9zMJr8c%&nrFRkpno0^oKW5b5 zF4{>RyZMpwShU{W=XQ*|>i6JT3*c=@WE^U+1r}Bo*LggUyh?%U&1$d!#*#<3AEr32 z;?T7(z{_(CG1pbhuzq%u{`_sJ(|Lq^-*?lFmvbk3X~?#HU~&8D*L51e6^&#q9MIHV z;vHm=Hm(hW(GEFZWjijbY)AHBfJ(AlAq$=%Gb6?khYkk0PZ=%!0dt1D;}F}Lg7WjM zI-fQjpLIldn-c+K~g zYqG)3J$L{W>7~+fzI-Wjz{_rVEixhm&~O%w*^L}aw%>DF`nk+AbqUu1%4K^o>A?H) zGiYJ4vTB*GP6U)KKw@R(lRMjDd66n3QnSf_`P-C))ri}2^t$S3HvpSX$YGK6TdH}J;$H}AU^W9XZ zJcfv4^^`Yg(F)sTB5Wtq_<9+9XUI98S#VAxrKxuKswbX_!7yK*o!vVc4T~ z(Kaog37Rpg3#3&babRTKwTlmG1wnd>!3aVS9UpOQW|X1txY zsR(L0Y;O5}xqYB8rW#*=dCAG5dt_Hd=K9_1az<0{*7jL_&c4C9(4!{!cd|BldZq)h zb;F{!u1nt@-C|pB`IXR)h-$?Vk%L40x6OdNJ&x_|%qnXBkkx=^{qqh*#B_kNp4H=6 z7dRhc6U}s~bc`>^4R{(5x}3HO(P;XCr@Pf_dtld4=U$O8%*57V^>lwCdMzZI;641R zJ)KLG-JzXh#n%3Sp7-6~1xTfH3M3%Clx$P1Jaytf!-7;Xsz^;Vsu9=lapNn={Z-2_ z$?UIys^j`uouw@IMQw{c^UoYymcB-_S_OGt4>b|PT@kLX*St%CEZbSgpEhrOCMvt>r|eJc!x!>FI#&{6zFQZONY0Rwk}uANCr9ACkQ1 zEajoNqISj~Go;bey?V^T{#At0SqwJm>ij4Xx7JE*U3bD$Ef2?7VU;RCFtE5PiroIr z?<5gK%Rqq%V^duyR$DeL;!I&t`T$#|S#6*^6G1-8?pH~9^?1F?0eC{%v#ItuEf_bz=e}hu`N`x4WOzbrOH4ZtA-5WE^cHBkeC+WcFzU z3ct}F{_uIC+YX-haNxsx^-GVMA9x-E&X4!Y?V;!4W9MMrEKy&P<2||&77d@<5Km=;|(KZo}I9E-ExQKKO(KgcTOdwAwRra2^g#WPDMqBjeUsf%%_x-)r{7<;Jw)dinRSMkBtgistfP*f_ zwb?6RJ9WEQ_qf5()_}X8+{esz0pNsJiaMU31iKy^*SuH!X;=B06gLZtkDhKeF^Vp+kt5y!x<5QA zPHY>ia%`xGAZs04DX%(Vu_d(%r6v!UKWw{xvdHyY%&_beHm6k}a?pWT;Z7SW()}ex zzdxJ}ah=8!WpmmW`vx^ocf)n{?3>{soZ;(#&1(3JO}dZFRIwX7Fb{o9DA4O`7MCXf zs;nT>zP0bLbxI@rl5bu}!)Cm*qSg69C+oAj0?Lt|(}MIUo~zTd$TmQ(u-_kf7`ct$ zxONmDu{Q7Kk2c){%aDLL`)*m8$1Cb;fNmAG%?a?&BJ^KVK5xz5KALcTva47kbuGO; z9}}%`da^zLHB6{T?|QPhU@djdP5ZU_v5PdT&t>3;UzWR?{#qu_QjbCr;69VA(>(CY z3e+?EC_OAoKFV#JM|(NcJ9Zbz~8%F9Yp}4WwGRzdKwv%3FD5d(=LBXx8Cfb90h)&Kk?o*Cu%4 z-`UxVc(^vW;QTGX_0U(!_Q^JCB9&QJM6Md#B2Rs9;F+F&wWv1b-tigKWKdLBXGDP4 zo#t~mA)AJMM0Z?Y>HhO9kp8Gy*M-Y*+=wnk=cp!`|2#IiQp}1_9>3={z3Nh=q?+qu zjsLergeB{+hK?65-C?F}NAfR{pegSYKYf0GxWSJ}+qc0h6qDQnDjr{D(SJ{UpMU3T zpk<5qtOM)ZZXR%B9Ix$vi(9+s;JbJ=pquQDkaU7Mb<6di^QRl3Z=Au z)pCO16uBuhs==v=1Bw1kySF0e!(1qKbrpj{bN!fdrp2}*AsP1Z7#+pOUPk@k)u@3%uxK`_u0GE zPT=wGGP(A6*9PM!eFc7WAeFM~zeK6~JV2DH)1&_tk~~ zj>4493mdKvhsg;}Y8?N*9*i=1m@xYSq@}}Oj78zL=#7t%ecUR;5=~?73q7o~2(gS8 zyq*Qk`$vTkfmdm zZpl}pbXshubUB{W%wS0t5{nR1I}_o)9w{qC-+-%mQ?|N23thOzv#U%K10KPuC@2k?jh@_VYQRQ z)yU?cke>q&#$*ZN(Aao;Q-F~jFk!&r;okM_=R*S8d7vrXF*FkNpNi2&dy`8{-JSEX z3hJ#kk95ClFqu^zTs~DhhMR%6Tp0iFFMh#K`p!;aC@5WndrJ;21CV;?8cfF`DU4;& zVcV9OJ8e@f+5T>fp2&z}ChZ_>Ql-f`q;&29-QA0mHPMl$6tflfJwQ-l2Mg9GvchzN zC3L=5U_~Eo>mlbc?Qh$yMJ0AG5!}EL*vR{}Jy3%^d=pj8|4eTGTlZRl*TCkqg3Nj6 zaeJV|z%f2C6!fNPURC4%`{7ynNMN}=dq`FLK5y4sb1sJA{TQM$+))UBkm>^2eqLDGt z=V5rUr z9*21gT5)wbXcOwu#~wzm-u7Ub!hW5MS(L<<+w63w=g@|xpRTTHXEdwt;Kz0KRRv@* zk^z$M(`2>6=Y92KfNcSwDz_Kcb?zYF(U|*$g*FPLLPa6$-Fe3gwO_psCnEB-E7tw` z!!1^pQJeBRC?Wl;K1bUS7L)ml-Pr~174ZA1orgO4>#FRmK9ACwVS9)QR-;2}6k5t< zVL3)n{S1ma3@MXD{o{?Nh(FfZFb#h%y2=e<$#&8%L%Y*uD#S z7C)O5Absc`6|@J-B=`?Q>N4nlDTN_biB5Xgk~_bpOHE;_PGPt<{cBz*0wh_iWhdL0 z5B^A<>#t4IU(c)iKiyjPr#&=_J}+}w3^K717NacLwdJ*4tNPZO<1TW`{O;!lJ z!Ez{Lwp~4Xx7>PvXd};`a5laZRD5K&E%p7(+<>yO0KiIdKU$}gIn1zXnfJkjxfW%g z1Y7C1kncz+NeOziqY26IjmfdK8Ph6eSVTr($3M#7%4q!5rR6ks!a`&eBGIAp__uqqnr=wY zRNtvU$N4eh$ciMnD~K_@N`&9Lu&#hIg^~&Y)(4y^zXJa8l_YwiWCy__;w@S%d^YSw zhBBrQGunsDo5VCTKzDSujqRCMHQ&jc-;`RexbvFk5@qvhAO)*y+N>YGd%VlpbQhyA zE5p1bx95q#wd0FLxi|ckw1%=5Ujk*+3LyO-8oLsa2P>z)Tk^81w^P=R%!mUXj*()OsK0w zD-RJ1TRRlqy#48p>8rrz&nux`P1;%t0;&`mOoFaCj0Q@TFz2mLLRI0D}TN zMKp>)X_~=*?k8BZI0qID-Ol2&jBg3EkdXfkc=9D`HGIr^MB6EhRlPPx z_O^{$l;*BLw5XxVd+us3!*7|!dwVh&@D^d+vF?H%5#x+_i1GR4fB)hKD9x5- z=OOsUw;0%Sy0+izmy+L4&guA&p5N}|4w^g0K4myd?k}~BJX!RUW+Hekusg(hQ8iT^ zhdx$jEYSy8@iZ>Py*g>Q%G+}o=I+p{OsbBUTXED?mTqVbR!OXhRh-=38*Ze&)B8P-X&xYk1*E=bLGMJnM#Hh(;xl}0Hw z&mzH%TWm#8yl`isB)CFV%r@+|KOoz(!ck4)*;eoMu+g65$9u2aI3hol?NN8MdV>11 z7u&!$>dnjbh*e{{xlu9L_mUEa5kyq%SQrK#Yg#8Q`diIP?8k-X#Co~K`QnJ@j_ zfTi=dfzP~}K3ncDycV8-u+J4U?=T7xn{FB^k!O1O=KQ7-hg9Ka9e-}W%F#j)w8>Dq zj7_zTPUGT_i|^~qF5xTo;iNRqScU=fOZy!G&u$Cz)F_fC{`d7<9`mGB6vZqAS~YYv zN;opQPmFgL?$>M`XFBIgfUIW4V_|mLpv>(gBN-KD0>_3w_j3ihVs@THMWkUjgo z#R~V0PF>JUMZW9A57q$3sVvVg6rYbq(`+ur-e#*I>g7Tdz9z zT2zM@u~`i4i{`-v&EZk2ctzS|kQZxr-v|9urbF3j)$DISXV8NqM$UZu zdI-!WsNGC4E$NUV!8?EY-FQ3}POGe3RFHS#R+;(^7WGJ8JijQU2E|RcoM%&t#p3|GhJq&`=RyC_`d)uGv_F3*S}X~v zkiix3Mz^uLlB|NV7c+El1=v|$q?rSM-NrTo6UH{EW4J+HKuN`1)iY+J z06}lR(rgaV)4j@$mARC_Qil5=NijI3lEVfu22hZ~2(19YjB9@6ZatN9m{>-JBF<)^ zmxzm-CZt}dD*0HMOaadh53jyW;PTV$oyV?m_Kxlr@u(Di%-)nC4}OG5dBh2wQ@kLQ zJ%R*ZaP1Z=_f*we|3QI>D#*s5yNYpU@KsWNRjNUTnSh(f>0z*Jkt^0)z&lf-*|-97c;(eB$=0^0*&wK3|LvDdmtn}voTD+$ z=Q2%)AdGOw*K1?9qm{bGNAKFc`*jyQG?~UjP_X7){S|L4AFmB(OnNe0GcIL$R*W1s zdjD+KzQ12oA5m}D2<_(E3$?e0zg)Bd7_P-12{#@LGpv8*jm1;-Hgk)lPOs`4qR`a4 zaxr7Buj-G{n44o>ldUyLu9xE!DT&<9w_fez)QvVDq*pd9&ayx<0dn$y>YM$ycV2f8 z&C2qwGm90sxkkmLjs*g~u_b9Art~8s-WrsByeJKD`|Y-k%fqV@i;D#LP_KE*HP~Tj zM6$!d7gB3h-L!VWsYIW1-$?YT^2lH{*)EV14&@`3_JELA?NRDwb+-rVi==Wy$A-sw zClsz?)YTnN89!`&O7!7*FKA$c{;ELtmc0y4slD-d1k5iQ zV%biiO%|uWo@1ToL4~*;dw$6T37Kv7qPFbBQ(&wv(UGl<_?nksH!i#SVJaa&yX_=q zhWG+p-OPg#sg!nDj1TOSGRFM6nyfoN^G|FM%EVbgcW>$ODy0h3lJSh$&3l%i<$$N# zh}+paXTLu`c32S-mQzjFqU%NoXG>6a#s$Nd&F<%Cq)8suAFqeQ9!GiR^i{W#ultCn zhFNAKP`sTsP{R3ydWVDdFmi}_RQ6HHUuff)+YqM6ao*V&UqG&u^nR*oAl{-DZHK5x zk6=Am!s-J|Sl+}hE(EO8a@p%q)_Ls{E=7*URXxG-!va6HzJq@}Fd~3%qbD4V3y}3H zU4DU=igGNDBdWASSaXeN3G3O2B_wZh(W7SV*dWrp@;E4+w@%2`Bgq13 z5O+D9SJbgNqQ-(c*?7Akv)Y#WE`wjyeTFxgqw(G-CaMh%?&2O9?C|Ao(2iHy$hve# z30F$)Q64*Fp~2SdkSSEI-%e%u4iW|x12)lJ0v~VYLemJKyZOdns&50(;z-1iwkVc1 zU51Cf$y}x^DmZx@fRU*=hW0B~AvM0%we~CDzFdKgc$5tVLys25q<_5RA#HM845NMygmX%jMy(6=& zhkXcoY7n2{e?8;y-8{8X<;=k89vN%Tl3HKjHyK$ z&K@-^iGa=-ndUqi;+rzBQQLZcv{8=ZHGJ7_;eAzmROg_JkApJu})pck^ zk(j`a8qxiDdURRvfupXO#{}ik?1*aKmEBpAM6{u+fda;YboQ6yb&Lau2ca^ELf=WE zANTEzDBLzP6WnMz@n;+b1mnAv>pF$7IQ}>~5!LB{=f|-yCuy6Q4-#4EZ06Qb zN_RMeYQP$iNLUwH->{HKOrD?Yk3?Uym5hkk>A(GOC`~-J(i}dHhJRUS%zk*WSz0CA z5@;l6I;?- z>|b0hIW)0(gc1;%!?Q|}kQLXgmvM{!qe}+^LVlU*NwpqMBF2v>?cu(Eeps?bYBaE# z;x7y;7O+DV+_AN?=iIA*Z*do#CH#V6+fqU(cqNB#a51H1`8^W}5AMUkcMi|DG^^}g zok%BMKgI7eK3D3DU57W4-u9wP$9DH1n4(63t-gb3EoOb#pcFGkQ%u!dTG`j zxt~LrQW<_ckISU(!v$OoEm2)5wRcit3Xm<%S{rQ+Uxq~VF8-`FhBCl?WaVm86zwwh zd#I6xw@Hj|m;zBD2kVYa#q;7{c5otrGd3LG=kNcs`iSpF|CYdF{vrelx7`=EQOkES zOw%E8|LJeh*-5My+U7VlUEljU2633E9bZH?Gb+YK(kEt-LPz9}P4^{Uj~6Ry?T?{> z&jzTQ12c&FADKm`zAI1qoKq$$tS53XI<7Vx=pEN|3?F8Ry&a?9Dr3sxRa|!zT{LKt z|C+VaEf(x0-JirFoKc-sHTA^z+wQGDAcr1&^_)+5Q0@M41+)a>xL;J-0ffeORR8MN zLKIXPKX`nNV2ax)nu`{Dja10r_Aa@Eu%(gGtK+c@CJMz-r69TRKm|Y|@34dZX{L&o zjR5$4*J+^I)T@HLo#;*>xXEeP?fMUHp8@z}Bj7|DUA$NS5|zLT^A8abOTy1N@?^Eb zKp7~@;{;B@fj0ctRSM~mD>wLc=jM$oQ(&=EVcmYU&on}pJIon6*z_2G-^!SAHu!!N z#UKmRhHSGv{U5(+WMlDEnR+=YMS)C7Y^HZjCzjUFsvF^gYxwPlL|yR)#gUtJzY?@4 zRa#1R14tf?#ppw4tdDn2551E!5}G)q=k&ZkZhfz&AX{|Kqs}sOCzlwXJIs_$yjhQ( zTkj`aLSqH+U<d4#JRvqIYhKQVoxev)%dz6vckcxmB;2J_6np-pz~-Gp!@v ztQDDb;ogPRtQ!FgA@0FxU2m|a;xm)|Up;9ab{L(VI`d+VK}kp_c7mwMo%Wb}7ERRZ z^%lNi6ZO(PC(i*uRn0jzc#1YDwLYB108%eQ=km-~pWVF2$~Y6WDkCwAHBp;JXPj2Q zx9P!k3+o(z8a0B<24lX!wy;saZK*#V2yt|xa=fUaTR44PLI|UG^nFf*Rm?|EsqV+U z*L8NIejuAui54J%(Thah_dJi=+g25PBiMZZtoJE&!KlL>SHKAg-%#f-r=ZhdtLu=c1P;IxIp4$mau;H z;r%Si=w)FO)?2`OMH^^2Zni;?0KA`l3T^bS;3A9XYJbF!BJ@lnLrYbdMP@VmKR-4a zkpSh{exI4O&m@Q)8ZQUQBq+bwaNV}~+QSI0mOg~V_H~v4)Qy)~X8++Z=X9si`J7Dh z?5uC8Bm*74_R1W)tI6W)e$T1zFnSx_uv}}M8qsiH&z%rQVi|cYInEwDa5C*WN3n|W z5kU#!l$s!ThgA23K)ld_7P6;)(01+Ol)Xeg-4{-5XY{#o0{zF$A<@KhG|>w#^T}46 z{)yA3_*!^hBVvpOtFoQ7;ZxLo8BUxaHvUP3(7d&$eJJG{+k-7l17hyGlaA*M(dQfL zh9`>ToZn$$c2!ey8U28l{}ndvZ}?~nMg_PHICSc$zHF?Jn%E??JH=Yv9oza5chKAU z*A^G0{Uf*ddNhqIA9n*6_Q!sf;!B79E76&FtAiYVps7F?EAuC}E*D>y$rRQEWxezZ z;}cejB8H9XlpGPRL*7)O9CpjmK}z90mToR`ww4yR3KlyUIh(%quRr21`KuA8h=T

      DlSBV<8FBC@ST}e^op5zJF)=S6$Kc z2&@Sg=`YAZ+$EaeGwwDZ^q~QuTw-^~^xM*$RM}qC*S|3%&sP)GstI=KM{72_>W zHan4b71FRIg@S^m0h7Arf22xbi1I7%0Gb*zK}?N0YHTZ)Xlrk`J|HgrO>2qfi<;5` zjllrgn*UMw;hQ1qW<06jthO&fCPt7xe;X6{RTkgyi^4Lm?shw~|MLCRYcZpOIM7*{ z`IG|h<)Goabie-X7<~rgnlJYmFP3388fcyZHe@lSi4bkBD`o_=G|X+c$bIq=Fhy8T zT8lnUSlSvPxW5~q_=mb)jBXrH=XbGq8E_;>7YB1}{VQILqkNmF$rqYoUiEA@Tz!26 z=Xt&yLnW*uT%s^h=dGWt4a{0nK8kv01KnXA+NtlcqW-FC;VEADYMy9DH#&^K1_VTe zF%_9%LCvvCSNB|7JLhQoknsRgF=}efNz%d6)>yVx?}N(0Yhq1B zafHIiv&{9I<#t%rW4%nHfB~OF8Wk)7oIXro@SRf}0f|AsmqR`6&j;%U`f|J33T>GV zAV4S6sS^J`B>p1c5&(jU$^uuyKowgy;bPyBvrC~^HA$?mW^pta|3cG=o{$!($dnj6S7#1cvSps z6KMJASuS#|3}yCzVaEJ7S7{0QjKoo-BS37*e+#}>KvlLMM z13r(D#x?(gH5V2YOc*O(`ATO6I-|Gg@jo~$#YyS&yT8L##u|nxJ7X~at!L?_%7MFc z&s0X4B$UPA&QKf@X`1qAd3j69@lHqx@)xrlt4SflZ{u>!k^`M;jWz?a2T@dk>s_?n zA3K`GZjs0cOOPqV^Y309U~@BbiGmTpC=vE)&^G_19;^dc@yazhYB7QG^<|mvKh#6l zzcT40$`_qYKy^vLD=!msu(7-_0=cPm;g;@YYM2WaD2&x%$>NPyt3(WBA3&t|< z8Rf1?lln+8g<|8+;5b~Z`uU-Y43mckb>2sl)>vHzY@o~9jq;LRBRC_osNWp`V zmvmO>+CPI(&t0JC4SxDZyYIxHWg;c0WfcK=2$rNE#-26rf?agll)+48DtZy{mjkMU z;Rhffw88M58y0jiS;b9Ul6j}qhwSEf-E-CBDQ2(Ef0im=e>&7#a_foP#nuH&^=%9`bf|n<4q(upnr90o#D7;r=S&cPd7p%A ziH4U@y+C4^;VnDI5hE{SKS#gtX0O-koR)=PqDrqjrxx^JoMFHtPg9jrf9NrNE(c`#R0Z?^<`>jQ-P=~LEuT6)CM3mA{dAPa{;@L)rtN|i~pDCqtU!PU^Be47dI z#qX0XDPQ7$n0}h>+_GYLZHc&vO*4QY@DmM%6LgUx?}+>&2x1C3WFk4<6KRPVQG<;T zg7ON3Sm0)X@Yq04S|B_&*wZ};&*S%kWGjkut_K|QiB?w&+#h>6kyZXk59LWuUdmN< zxlPh=&yqKB9;6#9sob70!Ll@Ht3m5;JQXL`rIA)sxIOWbY~qch&wOtT6h>!? zO|dkhWhu%mmhhQHOiACd^mAxl-*nOpSOPxBvZ@eoLjb+v1nD6GgugVtKDVZyw-Rt? zpWtPkxV|Wh|Mj(JAV`c~k!R3pnBO2s%^m=_9S>*k8A&JJy@*fw{^Amz?83NrKVH>c zA;S&+(d;1U*IQZ2`3rbw+>}p%D3L&_L=L5 zlfM6}41YX-KU3l$i-ST0#mLLnr7ZGylgv<@lqov%axLPWbbgp&duf6N?O#2j8|m~t zknRi8Z;4Lb-O%gSLG_wdRiHu5LI1c_%ZSCeQ!kdzQ@@ooFAhS8h@4poq?c+!!L z{>vMF|Mv}4{Vf$On(-ZXK+1iCv%mDw*Z;GN!qJGg|KHT#a(g)atE&`~I#87jsh5V% zx+Z}bap{H^nSiDmK01@QxH_g~tS2!a4tUHU26w%`2~nvOCwj+n}LRHFjo*X!*}2fXQ0JXSQg!I}0T$5$ws zr&Ie=B?&FebkHe(clcA-@$=?5V0$9Xv1aYU_TQK9ACKodD^U{$BCIXnc^?GB z=(;Uz7&->1CvuHX(mi-P>Pp%3)fkWp3%v}&6#Y-$Yj1zZ!+xH}5>p1XB(&_^8fsCf zq}V(~2jMl1gC`54(mBoFaTw+Ky(NUk6_|k?PGx;Uo(XoYr|G5>GNS*QA`aFvOCbA7O#W*%!Hp-AGB)Fbuv& ztk!#?SRyK2(*RzO5+(aO{Df>A@m~&exdSf?%G#$$3wyc#2y!T0syuz zPiHrl(9_S*=*LvfQzgQ{38ImRBEt_c$|t7^%+E+}9M9t{Iuiqew^qDHf6$}>1E$Go zQ-0_(%faZ>_(!w?GL)Qfp}PHOYkIzj?L)+)t+XXR zm7)Qyf>aH1x}VKJpV^zlKVH@DAh^5G{IUonKjW@E32XoDUrH7mEB3K z%m^5W=Q8?u3y!<3(C%Re%>B~hA;BAB9%#8*1#tElW}@-WkcUWmyE$OczdXY#J@~zr zw8dC=^!+8p`q|ac_a^6pb_|}MciVOlZbUrDQ!4QL4F@ng9p!p|M)mUCuZc34cm=sx z!FAHFlRhLG{*#_OAS4nAjt{sGy`x_JwVqdWDm$NHh2+YY{azeS&hR4H?uc&|JrOgb zNy+$nSNg`*0{Br5QD)qYFW;Z{JUm~Ky&N=6|H7mIp!`=HfLNc$8-M#xx6x}Aj+IsE z+OL0Zr}NZ%_sw}Sp6okQ#Yz1#JzZRw+gIYBBwE{_KsIWB$q#=TxNh+=X`OQ6JsFr$ z;hV?Ptm7wf?kl>NFnyG{pz1^gsGvdthA++aRf5!o%_WfOf9^=WofdRva8f**^$_*k zd8E_cn1dWwG=Ia`_kfI1h6?Vz1)oj-gIp|)wiW?6A+Xo>61fWy+%CPuqM$c}jTKS) zDd(vVj^wDL?=Rf_gMOZOq&J}2B2PnhTW|3c9l|L(G=vHYNqAOf@mf7quojZgWppVG zxTp}ZsogIf`W4T&{{xB3vp+@$BfrL^jT1q&Fgx+YXVE2xP*_dS&heQ@K%KW!=Lp%g z45?by(Nl$NB(4+mx42j84xAk>&qY@W3O`dc=;OIN{-COF+H55;k|qRODGl-M1x&pM zRw0&RdPdQfwL^@Ly3Ys49#u7O0V<6jbifLM7HXsMZvYE%TJk&jyprwv4dCZoA&rMa zsTz{@rK(v3>cc;f-Pm3~1At)Yy>UKtx>H%(+PZz-Xg~D{)L1jLu7K|dFP_wui-Msi z3DRnT40wouG|#zWq~e@}2AOGn6&{P-MSM8^!r%@nk`2T^0($Dx7fu&Dj?>9>zK_WM zT48p^og9*dSwD^Kk0V>3Rg?l;5j+)erApQyxC?P0-M?O9Fl|o^9cs^Fc~g zU4=N{v)F7uXA_MQ_)2;oKhdwfwvPEgt=C%Dp*7z|J&78OO zOZdqkRR0ES-a>q~qmBD+YjNox=IIl~CjkNE(0@-{RtiD@TnAX(K?kZeSMl>59I$6@ zUaTi_&3D;097N?;#3yh8n+zMIKS*u+ag0L(C9aZ@Q`1b`uGioP5I!%mgShK1{nsVm z1E;g)z(6YjI{kk0XzARq3js_VXI11?QS)oJaGN(N{D zQ)q-no*p)j(|jY3YHH>Vt#m%PzjSu%z!id*!?4liTIPj%jxbNrygac%FFSDku!$F# zA{m~5#P*xo>xDwlJp03)(JS&erO(90NwWuAe$yDjsF&S*oFBy7?Wyi-VoHR zzg_~i*k-@|L|w?q_i`h{Bp{5S_cZqSt!eL5Ib71cS#bjAAXV;P$MkBNr*4Z9JzVo43>|v#ON2cV zORo+%X#G(WJnnrqXG_UP#~`%>QS4y$&xL{g6+ThSc@6d4~N=g)R+r z_n#5X0kMO2tNCW9ALw%+o4D^qf&Pf1z*^$hR#CzcK4;^|g!<1&9Vn3Qu> zAT=1(fY64%v@O$B-!kxDmU|xWI9ruPJ_3@E%|`0uSd+L8mut|&-}=iU1i1bz zeScGw;1(wa5|QVLlKK97;K8bAhMr(^wVT&f!tIhEYGE$(y@IU}xiFP9#nf|JpISr> z;imU8)@y$Q|JWCsV*<`U>!-|@qvyTb#dJbZ&-%R1?v1ddpXoOpXFJrCw}bSCZLij? zL?Ex#S2hyYQ^07iX!0h;79qplAWDu&wmaRk@lJK)I5d6+msRRwB~o_sGglJXo;Dvq4;xs5csFE%IsrDTQh3!jPFq&Nt;7uyzoK{5fxBwQ~WR4 zsyki2AUs|gyA3H<7U)_Qb>oJ9wI9t9+3z6A93AcH1Kt>W!-n_!zNQ9fWe`F3mq7dD zqQs>JH@TRXV~Mq11^^SNpTDc_`f7qlAzXJ1__iuz%VfAw}F3d()DQG`>+#YoKH#C&%bscWZ%O}`kl(3(2*7^4$iV(Q1>sWFyo(>u%r zkosu+>l;fx*`j|6mt`tzJ6fmssl318pO?fvpwpR|r~73c9_xU+r@&7!+3jCbeKTF0 zd%Go07f&f=Sd{RDX+{SS*eZt`g+VCdMw-+RWV6QZYJ9-=X{ydbwqd*Ul_*m$CmgmT zp=v;A2uz0JXM}SsvbQ6ykTxFz<(w>N?sWfTp~4|5r9x*ahDpqRAtp~i@-8@4rh&)Y z25Q>NOJuJC!GyouH;5m4{MoUi9OLy`y%x{UhU_@DNSAnm^+;~f5roKhOY`<(d^fP* z#lD(VqMT@*YQ@E-nrC4QPQA;tBvWQPmS=z3Q>oraiO&Qf(PNH1(Df*Bb8-;jh4Pa* z@>wik2$D^98zy`k^OCcJdx@^$R3J1j*4%+8O(xdo)<4&SEB47=YHlY=Wq>9wnEXDP zFIHT1e;CKz--@-SA97rJF4)oCt#JC)!;*=`a3gkE_8P_{YdAYxEOK!FXwz~3KHaw3 zO+l4)B;BQoyLD?q78ELk@b`h~%HN`q9wcM)ZRzcq{yuDia* z_F43H^DqmO>=~~Ren(M+Quja2+jaYWFar~jku=T)umKp`H{oR1l#!(KZW1pyk&5cZCs=hVP#3P41?ZOa^GV|D3m1FCpTDFCB_fIrKmAF$cvCO)9*Piy!WC+e+ zn5XtbdTOs@Qnji{{rLzaxOQ8~ZFABsAtLjvX;pW%hfw9hQm!q(hehPBebVRXx2eQ< zL}$D^U2zN+K=JDZi1uz`X|svSKJt`Db^>0ZkGug7t$EJ>vFA_NK!Kgf*kwKAMDLaQNab<_5L zkKi;_PpPSm?nW1sOks4t-(yz+5%hR#fOXd~%vEChI^8a|FY3?LN#c8unk>%eI_^36 z?9s05SqF@YY)+dVG?t{u2XQ^V;;>5@o5pUXE?Eqi6n}kP#*D4(6_Bx!z=U+iF!?tG z^*hti1`+A;ZO%e0T2&4H@&A6^Kx;~P1nMr6(x8li0M>7(vOgYyzthPhQygN^rVl0d z$&!(A&+uDQHM$A@Fs^@J#D3YeZYdoV({dqnL1p5-k^UW+UJbJ3X?z8yZ3OiO-33xd zCU2MaM%98ep#;B~7E%1mhism<5UC(d89jh zTV^)ef{N;t-@DTY4UrATqMDqGGB<+2$jL<@`-~Qn7B5JI8;*KV0c6crz1o}A- zguDc5X^YX-^VfQn#@%J47d@ub)YoY(3+%MzpyF#$^c6FDP;+4`!$2;5=Vl&)=tLBQ zuzje8Cu*{*8q2*fj;?lSP*|b)qOd~*)|eMRUsxklJzF?FC4)E2IUI5p4uvD=UX-X& zJdLhXe3QUJYv-g|KqI2R1mveD!i!9(k^bfFgMvHhADOa9dbJO-Px*}RVt#VGZl0Hm zAS@)39+e0>7Hlj(!J-=^k>o@#iP5fU1@2ZRo6rZ=5SYzJ!4{g$3!dnY#CIAY?To=q z3{t@@yQre?a}E?cJt@Rr@!PNEuyKU8i3@8gi-^g)-)V=G$|rI6KMA)fAtD~#jFBWz z#nDG!q=nc-q_SPt*C(6!d&O+-PbHvluK_2894&i&0WnN{@|fwFhvvSSg8_H-_1)V~#TSC{-G2k!9ji_c`kfM!MbMasn9{ zsI&^CItpd;Oy1SiVH|bbBa@R`cipt35lo9#C?5JVW*eMU4J;8jVj^A7aOgc1)F$na zAdE2{yDGh!gP*yPq1zyz(cr*|M4K*~+Bb!Wm9vgSkPg+UKnV-~Ce7sy`jdwag`8{r zgkzb~=BhlvK+ewAd6g=g+KXwDc9FR=vNc*wPMXu031WYzV8S3$t1NyGq<%H(#|TQc zkZo2AZiuP^1r6jghmmOp{0}x1DV2Vu;SOWHqzyuW-YsO%VMaCJVXQN1X?fAt%ih(g zgz~)9LXbkvG@b@xt`J7#YLnRkg{VaR$_lU;s4Jjl5UHg{MeV-0NBb#rR4&vZIg#cB zT|bhPt4$2LNSR_6bc(&o25*FTMbycm1Y@AuEw;GU1dxYxtC2iN{PPlmJq-OVl7O@# z60$CqKz&P})=3tlUhTFKU)h#3b(<~ALS1$t%9r#(eDs14+Fb&qPl=VZm$fuu@VKh^ zlH;pGrOU(Q!`FTvx=TX@yK3J#I)&yp=_zH=?-y0>Wjx@I+9hrDqXKg_(qe#%6R~tz>AO3dZXP@Fu z<;Vb-_`W{Yp*l0xeuU)YcQvO2AaH4ZNmD_miWu{FAdZ`hJpc`Jb*0-T|M|>qW7w^a`sQ#vMieHTp%7Kcq&aNXo*)8`d(V@Vkx>^rR*8roaFRPqMHV@DqbJcT-q!X z7&opCrWVRN&JgLxJyVmei>m%LuQI6Y+nHnIS8<>3#pL@JGEF{@Y7rvMw~yS4kQ3-7 zji)8+X@KIRRlnTg#1ia(_9m@9q-^r3U%Fn8+qHS*eTPK1%c^(2b(P`MsT{S7T(ee- zzmvVpx`dRDYvT9hCrdHzI8Iv8$!#hz=cZ|KXze3p7!#saYy+#;pMB1_%h#1Ja~!T* zGbQ|;P%Vi|?(Jt6wcnk3q8;ne5j66H*tFC5&**+fMT~!jpXC2B(5jN(RM>NsOcz?$`Y!ydi~7ll-=7 zE))6l{(s8!GG%`Gv&pPr(}Mvyfl}`c>;L?quqj_wvivT=nYx&fih??m|FYZr_oas|i`JHVU#m8V)>rzJ{;nRq;eRMF?brjKiO{aR1 z_UW}r-PUokx*gV{;A06B7OH7B&jBw6rlaEu=-H>J`mXYTP_w-Qyor86?imIS+;+ z=^yZe;@B4rV;E$``dsb#_EqHX;IHK3hfBE65QK|DQRHE>R{HpDHwzMdxwG`ZERUak zCr8+|uxm4}nXwHP+qF=35gsGRayS49JE6?l`f6OSi+?! zf0PA(J<^L*tcFrXoXuFl1!^r4A{=w9dNGkD;PBD{uX-z+=0WYi75?F@$FTL&aOMVX z$d6}^IV1y$T6Rar&Ay&(_6V?alP^Zlx|)9rx6Y18>Xd>oD|V|w3gA&fLf!EXaI(>5 zSUIbh{MR~61b$rQXa4HLqS$HQsi@2u?>dOW2#2aCA$>p%3F3&E!l+g8dX_-RPho$` zIg>~s+zyW^QPg9NCm~-!mSKw+rj$q}u_&NkBgf0Y&er=#k$BG_15_S-pGOkubaP8` zo-F=>3cAL!Y-6G-MU|7>IBCfsG%?SirWvL_T7c*SM6flLC(yP8V8pjI@@6{ioM?dZ&RNWa+^gY6PKIT9$ zkKTOO&(wrMn`vZ1yB8sNq-3-heBF0&_ydDbuLy3<)U9C}Op`~;TM~>eN6YO|bH47m3enr0%?QPWTkI_te5S6y6NTy`# z)Aw4nD02;8SsC7 z6{*a^=)-lBmD<~OTvZ3Jl@V+PFwRB|{E@3kl zjb^5y`G~Z2Bvg;Y#3T7Pqx?uW>2U4SOmp@yl#sh>rJ%-`FlV040O`jjw`fF{Bf*G3Lip7k#TaDt}F zaV8jA9F$S=;5eOem;yon*qOlOkc1=gb5X1K1n;Hk6ed5c{&U!fMLX_7P4_wRsmMGQ zLYFy=M|@x|c07gD-D9y1V!hUpGjo=y5-+rNc$8;)gq@!TeCOjM@l|z>Z zdtB2lJU>6o{FcgW_`KuGDedqJ?U!7SAqY!`Jh&%nWWWFx#pDBWp=bR@IH-}5oDie| zL^y7QZb>$MkX~4)`JU_}nwfSzj&ewGBvJx%no@GM+tA``rZgDQp`xGFVnxkDT0$d3 zWSjdwB}kF@7N|4Y8f;Awjem=)4zftpzm#Op=A-|ZaK2K1k>z*UX(~$J_b^U(q(u*^ z&$L5iCQ3h%M~rD{fDQ`El+$+-W01$1S3s9YWK#L&MyTPL7N$QLdq(1;*H2q)cJg^$HN+K-H`BY%;< zE~TJ|eT=n=7`D~hv4{?W#^T~T7B@gO%K2}D^>AF)Hf>kBpod~LI8aGh^oAIpe)q(_ z=&B2aq`C~zeVDLdA?4~CP5pDoe7L??hgFL-4y90>A3_ zK~|?p+c-KLG7e!_7QtuqI;`k;79zA1R7a+f+$e--igfgr=lDh3v|I?(Na4vB=7bFf zEJlvm;EI(feb_nwWsyZ99N260X?&dzw|AC5evAzc9T0#|6zQ(#novho6Eb!e z6IA8?l=ib3c*fwXu4Fpl2ezEce(aPhALGN@XjU*6126c@98TZKne8{T<}R+ozAF%R z{74)97~W|#MZ1=pLf1*cYbJy(U(Nq7^>X+tZZb^M7um@F7gdpo%)O5k74!Z69oN3@ z{|S4sbB}`!A69^G+tYwI3#80<3&wZZkuF_xxA#RL*O79{ z19&26-rn4;x~}a1T=r?je-a4?QKSe6yB-%s;zo-9!;R!)8oX4%0lwb?`wpVN7{E0F zmj!eH695e^M`nKNE%GERZ%<;XlAuSWtrDDpd_>T@(uQem6{0z4qZsnuBklb_%_Fr@ z>w#Kb<1^$zDE?}(5Bwm8p~&aKhg9p%mjk(M+dXz;q~~b=HVF*lx>6!vGQki7j@fgi zt_03!81t_XorX6pstt~%h!zaPqGT|8{EMUGVr@UW5eP^NKi&WTzePf>IKvAFrW=!j zz{H%!3COLSPCy{Vo?Pt7Y?JO_r+X?o4J2I>Cz7^2Fy#kSra1 z9%x1@lK?>{E0zH2E2k3>t#Buivh5@g693RReH%>XtOh_3d>-Hx#((*^)t<<zW!M3GuNb?1`vUZ!wYUonYPy`#+z)eT;eYXPa^(SpDwY2QUI+=} zeX8&k1EtW6a0a!SRM84G<_8Q-RNzSsVh6+QMZIULvTjFG6db@bI4s#nfO{LfQG{8- mvB?H}?f-rLe|sNR{_K)2umnFiIRUTwK~a)dm#dL65BWdkO@G?} literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/4-ingestion-scale.png b/website/www/site/static/images/blog/scaling-streaming-workload/4-ingestion-scale.png new file mode 100644 index 0000000000000000000000000000000000000000..dcf2bdedf8af914119b3fd7b998323ff41034acc GIT binary patch literal 43609 zcmeFZXIN8B*ES3Yf}o-((u7b%dJzPq2Bj%Tm)=AKr1uU9MMRpQQbbCSB1n-==t%EG zkebkY?+{2x_%_#Fp6Al*`~CiSUykHpXYW0GX4aZnbIw}p+~FE(N>|9~$q5Jut~_}x zuSGy`0Y^YUC`5JhT2`SYYs^dK+k@D!H4Yp{ALlhLM6{A>2aA` zo-=8xJi+k(&JynUc&fMKg75v0`@t8}^<5S2N`nlHrr#2$U6Uufuwya!k~-~v6np7R ztrrZ#=$|*O_6;62yWSwYyqgp@O)oaEHfshQT}2pCE8;@9yQ`cDr-@5S=3U!rTg@MS zc}sZ{#RTK*_wmOs8gym7EqY6uWU&;!9e!vW=IG^IqG4HhnLFSoccIm{uw2F1M?__v z%&KDS#~sP9eBezbq(OXR&NHmRcerko=(kdE2AR`KxLg)~=29^~U2A8D?fEfrJ)mN2 zq|`!Il|atdh{20X5zR$aNp1JRsL95{%Q4mEsSC&6Gq11=`K<20Y z$&Dk_i#z%scxC}X>ssnRu~Jnf;0BJ#2nfS$2#A3rLf}ged;y(&ctb!6{HFrG^65nX zxq1PYe&IjIghJ;xKGc?b@&x#=ZQ*KZ>FD;#$$k7+iz4u-QJZJ_?)s`vB`lmA_|0EB zy|Co>c5pr~LLlWW0USD5x|=h5JJ>tANq9@M{&j~0aD0ARfR*{LTiorWS@l&lnB|;Y zEty661^ES8WyqPCnWbD`T1jZhEB?DU@JpKYmAkvMgn)pTmlwa6Fu#+lwZJ`bad81b zAps#FKHv^MHy=lLb8kLJH@1I01{`>c@ zc3OJd{7+4eZvXBU&_RLoD+2fU1qJ^5Y@n#r`B@1K8*fW{19=+$?oE(5U-DUnqHUBRBzc2o~pp?LQ*Z-F& z{w3$X&H{p#A(s;PuhL}5r7hvWz>y_*BLDE2H{nJqNy@Wfnhwgz)ZoY8-d^IiG52Ee zWahp*GQH)f(bU??e^XQAX|cxj8zjUEao4V8tE@pAz3N*~mR2E?b-DRx!D-VTMjl^Rzq?c z!t2+5`ve^m2Y*0_)_}mC18)*sB2InTaKwcMW$n=J1TMS9?nEA>R(vwq+rZyHaY97- zdWDrDcBA^^rIt7jT7nPZU+nvPLz)|Hhq-yo8tQK1Qv%(2iCCF`|1ih~5_%liA2?h@ z&PO#l$$qcq==yq~%?#q6NxAyMk+jH`MR`~}-M1;t?tW2+S;o1fS<5_+e z!+95D{fV?N9})WOe+u}mZ1}aWp9NF!7XwAXma^29tQ{a|-4*)#o!0vZ@Yxt4>FwuW{IP|38J{~QeIs+B4D=eu z;8@kfK#+Ax%8mSe=*aqB=Y&NJ<5qKE1<*6RZjq_<`eZF!C-2!e*v^kS2rMeOVk*M?mXp+D;SiwrBc%rywS~6ys~Wyc0iWvGeyz}RI2b-w?0TmFy3#1I?{a6&-J-zyIoa4h&K=`~KQO?RG=icavJf3fp0SCK@yU5;pR81`DId?hE)JtY!wP?2cq#f||#lwuAGlb*e{#UTh;CV}}A%9X+rsM=3pY_vxqS>e7dCU($s?zYA6| zT!`TLUQA=^-4ye~LjRMb*Z0lFGyE;ZtuD9YomtNwKKK|1%zv&2<+ok$Fp#0p(BOU7 zzN}AF)MxLd-@<#P(^K3s3_7)(x*u8d>rFh59!8sjhW}ulrrhRNd;2@a^(*9hraswX z#@A-UK#*dK80nE3_th6>kdthecZXM(`jQic*Zo0IKy%}BrGX$R*BjAC%Zf_NzYjO& zL$a)tK}4qvxkTnJXEh+my$D~(fQa2lukGk}hu&nTsk&;NLc?L6Rn9$1@M$rl_e{`K zA%4nl`%s%eYn7&HQ&m#fZ}dNzI=z>fv4H+fSH{v4DpBoL*L)?)7W} z)2vju|3M1_$a&w|9a2<(MBY^h+;%jqXXc~l7j>DI$hj$}L2{!wq#hV@?PL%L37b-E zsb!B!RU2-{UipA_YNTxHbgx3d1iB0_@>VU^7_O+-DXDsY{0MXvhUHxDk7ANAbg7T z$8DVFCelX%BtU|6`T3Caq+DTFfiY7tD5LY^`%~}WQp8(`R47>Uv-b(-(?3@B^o2`V zaf67rGLhr;=qbOWe#z})1TdyvF-m&gpICoi{M^i)rLj&I)nNkjAZ*T+IeD5S)CR$B ziowK8iP4i@FtF8@R1wy1F#ey)L;yS48>;2 zBS?-~YH5I)_~K0lDeq#l`$Z0@$|(|9=!VAW6EDD|SZ79}WHPr4`VpMC>q@t0O-{O! zLwEEEw%78Dj@+;YE0hX};wN?Yo}V7?HrzHw8_*CqUC`aICT=uqgowUqCELaz+zFO; zEM7k!hFqqk<+JMjGz%}dRQ$Xt*!~dtGi0jX4;xB?kiptFR)HdSO71>1f31o2W<%Fi z#;8j*QOL6XfmC8;h_w_~t?xjAd0)N>pN2^LDlvM*>E#)8UTs>00*f2O6xT!wvaj8$ zv8%j~-GswHsY(XcBsjq4_!bc=cDSIXUu8zyj}#b~1}#(jkWpZUziD^Wu9m%J;@^6` z8v$O}?Es+_iMlOhz!VpR76F+X5X9OL24Ig4ovzRdJ`YTFAAAC-kO45_%N2n%eyfQE1C6?k*$qxM`|ADvRipJwu#LVA=Nof$)3D}Ti9 zV73YlJ16Iz=eXRPbVIv)e)zh2;KeKSv=s9sv($dK_40N3h%Hfbbb%R6QH0}aH5dAu zavUdX#MMvQF1qY@ywVtIc50y~u2a6YO=N2Gz-{U2_NvCG(mlgsN+*JwkV>jQL4F2H zp!I{0PVY})6-yC!mp7^le!VFR%W2c34OM?;5I3LOtS)t{RV8c3Ev*eh#dVre1WZHY zP?+1PMwiC1fnJ~(rJzU*rIQhrbVVq0h1|T)0Ky&o3^D@=qBO(TzP7dGh;B^66w`S%)cj)>nSrWke+4ZEkxIf1%oeI#Dc6_<*r$3uw1MDFh%h3cC|6q44 zfTkeEV7gcS+=_8-p6^SJn9`3T|B%>AUZ5#-*k5_(e>NpMRrX}?>OXbvu>+AcS{zY1 z?rUe(6)omyOA2yNkM>?RVG!SAL%sZHrCVqyvhLD4Tj{xJSX;I|sax+Wl5ddtrow5o zU&3oE4jHZDx|wyrL04S*r|8C=nw0Mf4H%+Txi$~@sqhC8R^mAsTl+@VT*JbJ7NT@F^laYs2|= z!>MWpa7V_L!$yNLtE1JN%*}-~4rJ+KkHfK%lM%XwvH}D>?OGf9{;^0%NL&l3DCx4c z!#|8dLn2Z}_rODOv_AvbJY%2U?1lA#OdpN2ZIP*G(;0tdw+=Td!2yNzL^;^y>lT*M znEH41Kgf@P&4XvAAPPoF_j?PY{IZ*n8cv=;tx{wY?0+ER*vmxcfO{Ok7$S7zJc!G8 zb`W(wx^T6l_kOx7))?=4-^0=h@BS2VnJ6FpH*Shsyv2Ds`4|z>3@3KZHH>XmV{SM% z_-{HmH(f!j9I@K7IRv^Z%OADrMeRQc=MFKctO`_s>*@LbW)|3$WZ6#HJ51RV`H6fj z%~mn>!-ju8PD>t`V$J9iV@ z4U+prds?UE9H{>Ew*;}W#qKGeCAUnMicS0S_=TKFN*xDY*qFIM*$Gio`hB$i1P#hH9!yg`C=@JF16w57NvIG%JJruDXZxFA{$2m`gb&K#2x+#{s%_!mnC9v2DSXK^ zg#yTepc}l86;J#{do8kn@ehd+Ugyjv?k@14LiTR~3bS7w%+>|Q$Tj0ERFD8Z%^G3` zbxjl*&&qcg^5+0~4OY=O44m3jj(aL$=M5TTj!=DY@w=!KM$hV2(E494BcXWN>e9th z5r;4bTO(}ca9QZ;|90pxA70|sFaD9ITWGu3lL!~G?7maj+$J-4c3Rx%>hLsvugu}2 zt=0TpzOwi8rE?=i&!!U!rjQVPRfKx&lZtrV>{DI}A04j73KikLH&Mp}35B+Cc6bS{ zbD1Q`6J(qHeI|yJ-K-siZtED`$%~`}VHfXX*LceE1^VfS5fOEOT}aZlU$H9#c8b!Q z1!@Mf!q&R(=2pvTW4)j6+rS0ECVOtZ%@aRxGrSr93DD=l({G!9rY1>HM!vvnQQHMO`RYV*}Z}Wdv`=C2i zoXM?qky7hV99_T*8Mk>3!}N|za$)j+panqs%#r7t`GS#{o$iK|U@b%*F8ofGs=Mjz zKMY=t&*z&#N`KSg4j!!Yat!&Kz?60Z#S@=Ukj)=<;jh8IPL}m`5IRE$nSwO>VU3(8 zbGnnAYLXalYHr;lPv3#NFcFZ}-v1NDozH&{;)np2Mh}JMhQ#a%>*%?#Pn<`b7=PN` z4hDiyuq_#xO9l1cr{rT3BEt%>2b5u$x9ohTAWlz39h(|v8Yn##4>N;R8WoQ-<;(ws zH_SvpJAsLn_f`v}cx+G)F8EobfF)x2yYGV;M;65-jnp$_!>^)I-`ZCdm>1*mfxlu- z&)}s{vR}IYDBgQPamJkjE*>6=|9H6(fC0o2sBm_P9+%#=@WC5sa%`Jhw$x8}Z;{l@;Y!fsRx`2Q*m`$1h#xV_E?0jCyGj~+kzq|mSo>D1wW z2H$=ER`%-+PQ6Owf^bQDZ4?q-$Rs)P?mr_=_64x_xKTDhcV76hI8oU$;>g3W)IWS$ z2E0)em!HXfhp6jr;Gn?u|JbZf5HO%m=CkzAEv-fdF}Gf>eILX9nBm!sk9|igdx#+AH2SOf? zx^CzuCj?Gl@*@9&R_r^AF_AA$JdibSfDU$dC&quK578o)%rGFn z7XJ^fay(gqas41ufXBP~gLc?B)$M>zMNi)t#qnl$4R4@+3WSFEe_dC1k_aO8Z3%ns z$mt%j#s;m3bKuxKj{g|N;&*|!ij6xS`JIQahE1*?+E4RaP2V*?@EETH z`QO;8w~bs7Q((`!9Mtz&?@3ZkTtWn}J>_dGNB#vn{=0}WJZ7f52p`BS`Csv%K ztoxL>(#~k|u9_*_7Fs^0{q7Z&^bq9wc^K^fPXz}u8DC;(Z*5V0)31ky@r|O1^~+zq zpBzU!=9|`k9xl*qLBj2?- zD5TqsepiT(-|KQFQc2{mAgAVTH8iMK^x!cw&9lG!m`SotZLKulsWVeqg6}`d`EPg* z045VI`yQ8)QON2eMB2XIhJUrIC2j(*AL!}IQ7p=82YAMSAkW$NR4^2B=UHu%;>=+F zcSQMJ_Co0wI)deM@lewj?7Al$8pWU1j82w;Lu-YnyWL=cl5TK*#+K2F@BjkXnT>uo ztmuZ4DK#g!&3<}7Ws@c%uFF3iO%%QT^$!Vm(sA+&h!c}3% zi5NS?mYZpfSl1}%JT`m!wafNdNzB2rWAIOdzppri*8#JT!+X*;q5qSwW;-rtjMWM7 zl~*`Fy?y6)nZnx}mEn7rEMJ!co7#QLH|61j)Rrf4jNM&MC*?dtN#$P}8k%hOD(0`E zP!s|UFRfFCJMf=G`z}tP1XOL)=!fG5Vf2k5w!g(^W7UZshARXqy!T*RlVOh##HLuo z_&~3{H-u@VHb~h&`A@tJ=na(hfA>E9&JD3;+g(jL0+){_GM4h|vD15Q?QAf2Do7a~ zeNZOuwxkd2_oSBH^Po>gb(En`&WN~w^PDhG6K6z7VL8hmj^F*5K9g~}|IlVx&bX3C z@B4wvBZ)YloBsWzmsTGc*k6)6xon;#*BRz74%#Ezm6K7>kvY+F9J|X6bDj&`n++*u zPm$PvB4#sIY+j=y&c4Qz4Q7X2)OqxGOu{Tc#97Ok&-wk~Ni=t{BcxpK$0Z7K0W2qm zjOWM4tL#z|rRoiwtt|ZA(yQWkWvFgquPM^Mo=+bN+XI@#qCR)4(!YD*1~M)8^f~b* zhX`N3SeYS#p;Z4nNh;}4*a3^Xe4eY<>ZrZ|K7k~ZZN+cCHD(90^qifCPp&k})Tn3# zwp2UczUeX-R{rokWjFuqW5H{&w-m(VIz)Y+)~tlNvru-+JwnFvzkdJyK{w?hNfB|k zLM~`?{u55a460?u{s{0hljv61>TExxg5CeMcJO3#z}4HQMZmVNe3v5T^UWj|La zS}Xb*2QBdf#U^$pP_t$yUHtL`&PEoey&3a`w1((D3WO;xoJ- zle+v>Xc7KxB-i``V8ncR>&iEZV>>h2RQ%m|bUB^+K8x&QdlCiUbi!xls-GP;Osa@t zR)wwf0lQLY9&(W-cgUWG&+w%i{_{F&9q)BJ*=j1|$&aTU5#S8P?~No)s1<}kS)szd z>I8z*T8+RCE`=~)@}-qYCz=C5_|O_-{kygPLAIx=4l4YE=C)e zm-R0&qZ8GYovY+hF&Qt_QzeFFup7b*U`U-_rKV|hd*-Omm~|v^wiIXMZPM(-2T~ue z-MW({=b{+=NHt}pI_2cGl-rUV{T^rc9#@Ee*T=dR$k>~jatG81;KkdR-+4-pr0Wk8 zeIsF)>`!#6xc@WsY~4)3;KlyZ`avb2ryNz7VX}XC4+d zheuC34d?5if89h@O!ps6xgqRb1~&1$C7U(;U zn)pT6?{z=-!;CL)R_K{Va9K|WoYhA{gG~H}sVk=;Sj>HPTwACd$alYQ_B?bfx!(IL zAl?*&`5LgdnqxXH|3#{LIlOW}k)bUwQgi;DB*yTNez>qQ8L3rdY9hHcW?|ndxsB|6 z4-zN&k|Y**@L+D|Q_wt~lxc6(sKR!Ux0+`&*{Km;*8iX{p-A6++05HTl>hbNhR5g4 zDF?AyOm~mPV#0x1oy>=~3gzF!F{T*}ZHK2z#pabg{gSodPs3YcpY0hvC#K$QxV$!j z&Z~+}SBuu4HoKLudb9V;tIJpxHDXG$yMy^Aj39WQ57NgwY$H8~u>MrZ6U@>uT!uj! z`R1}u3;GKQ1r4SF?(*yFU!K=#D8ggYZb9bC#@ns14kXkQw!ax0Lg_y0YiU$z|1AKG zqZ~o^qXMSY5*tMN>eZ}!Kb0J9VL@X*!nV*3*xe4;MhQn+`B&A?$MYthQ(W!u378HX z>kmbF)|sl-h6~qG&^0a&)PfnLBep3qx8jAHef*d)Mc%@W*8~O@wI< zvJgJ;(x}|VqUZ|U+V}i$-5gIw&|^U4PM zs~4Wa5Pk58gd^p%I4*5_z`?e#St?B#Ft2z2PLs~Q=d;nCxK zV$-j72hCX)@A@p!)z6#Gc&rWWDL;AcJJS?aeI40&YL4|pjr(mkI&s{Frhp;ndbC5> zCMqnKtE6oW8@hgBs9JfLyF{XXXHey6&JA%a3RJ;^f4H4ehA8JCm0pZim7;a1KTPnP zyVQXN$KtL^wO$2pXw+`wNgY;Vn;_3Vldg(|U4dJ~h)X1FZ`E!u-8^x`(=KjDn>&rm z_-)&4R-#6x&Yr*QrQQ2h_Q+}6Gg80i*?9P<5)c7(H@-FPzh9``0I8@yLb5qdquk3> z-8PLJ;A5;KaBtbF87PfkHt3c&_^I<7zBy*IXdKe8!oJC|n{2dSY#L$*c<4)q`<<`f z(VVV1&9mG1aAxw{q5*^aUUf=!=9^><^P8H2U>Sk@5)=Eug|02eGB2#XZ>n#N1CT6H zS#3ePKs$=Ctp-~W%dJ{xg3;}{?yE+$+<0fo7iSVJ=COKVmrJH{j$G3Wjm8BU@wV^@ z-!MJ;6}N*z6VLJSVrAIG{v9WR2gg6RJO{%&YW>EYwj@x)SDi1;cvIfp#oST`+x_}E zDtPyqGlfuJDi-ed3cJn4_l%pQau=fu+1h;;Fi{0^XfFU;7_aD69~M5X@8DQ7nq``< zu%m%Gfewa*M~3JoAAujuZZBet=G&t!%m?_dZ!@W%sU|k)#QL3Pv_)DVD0i!*&YC0t z1I;G_e~?SA#I|fsUT?9Oa9uZAb&e5O*_W&8;P`8XY#OaXVqz-Ur*LV@#XC~;+Xmzc zIq5i?%KhR6%rbj3YM`A{sn1a4#nwuyglBmS_+&_Fxn0m_PoSOCi~ABq!gF4vPflv7&U*GT z>=1gh*CYc7U0+l5rFR+{^OH>9VKb9ltPJ-fFmCV<9sX{@v-GAvWDcYj^p?Dx);pSZ z_g%fzG4(o<-R=Ocwuy9Xg;t|mnel$x&=tk%vQe(j>Q6R1*4exv3okX>4)fdlgStIcT%Vih3Erq&=RcbA10A=RK~v{)lwXJKUCYUWKYfzn zRM9`MeB-dNf6MpVMwN*-zIy8OyVBkE(5ve)Tsa~In(|emS_Nu408~TWK z%VkSpq|5x&n=F;-4A9{UM88th#H(-Ebe3aBZW(ty>dhCY3di9=p3`#N8wN+$Q5QY1 z!kdP}p)!OI`+Uv1lVIFJ?wvIw@A!>k+|g0CFfl6#@<52asKq;gXTalDeMi|bLSr=N z>RWHa#bLP6ArmauDb!D@5v(Y-B@)kVb%p5~kU{rrW02>2c}op?XG(1LOQPH33_t&x zgTmm6`OE+wro@7Vc#-HUF>=Ns%Y85yNq4~V2z+ps$?WHHm~)J}J2aw0@M6zq7FJpY znc$7a{)7REMj*679y87+(9dedC+?f0ab~KdM}&0DcjIayJ5vbln;6}v*!AmyW*Tcg zci(`52Z%4o=1Kb%juMdA3!cNy9*$zBc%HMtSV zYG`>oon(zJQ5BmU@{7|jkx~I{)v8_p6-xM?qzSZUZcoMGn_Iq>yWrz1%NUSj9Loj( zCid$ZPPQa#7cwGWWa7PFrY~fU&BO~$<5oo+hdA%3oH(f6dPW<{TjlDSQnn4K=Bq?o zbc4uRW=y?UD~TH2)lwiw<7jx)0tdUwbfGV{4IGMV^(X@GUfLUa^w+OB&+Hbld|0{loQ*q>l(^QRL1$bo4o4$IpxZlSBBiCT_-aa zS|}F5z4Mgc9>9coQ+|o!E)~=o(P$feLfl7hLrde^o0U@~t)tbfC3mL@DNa;B9hX5h zu4Ya~rZOOXXigDKzB4j*<8`4iU00Ua(SqKsvtd4Pb7Kig@eMLC0!AV$r0QIeQ}2bY zXPb`HnoZ~=oi$%cKy3*!#Ra<#=4xK3oyDwEWguIs%No}eUs~6koJ~?nw)TzU1kER& zN$e&IUo`PK6!DxAUv35R#Sd%CoWR?q$E_AI-WSNlz!Quiy7=3pZJJ%1p{w)o9|; z=h1$U!VIi(v?Xl;2k_|}dM1k~wGnr_fUL;Y+WtqA&|~wI0I*}qy-!`TsnVW(9QE?ttgH=~ zzc_Y_SASVmd(8J2fXX}I}g>|?M+{)uk_-6v2Q5E&IGX7ZTWJ($RaQ?{y z7L%{`E>4Wco%2UCxC^CRrxUl&+{t}deefi3+0RCgDa=&~ZWU0FLo@CVID>tqU9*In zpBD4MCxm>tv(ZJ>N4~X-Um41zdo|6M_I{CqogEvaw2b`@bxJ}pRpJ16@2H02x13^O z5DBEurSg6+NrS+5J~0A~dwIf_Hxjnp*%|LKQLGVjc(bqhg>D-8Jy1l0chZeY8hu15 zBR?#?*EdLAcpoG`kcI{Y3O@MnHR?2a)XYvyB1E*Vax5cy z-+O&E=R2ccb9g&ZGFcis3htyA``vPN3{EvUFE>30 zpPz(L-Rl*|cdzeILeXHfGpxUz9_6IJeRFpCrq)qyRhzrgJNk;Pcz>UjJ>JjQw?@pz z8*Ye5UEx`X-7<9zfQ9c!!J=-|BP39tB?EGU{2)?Wtsm`^S&wqtrkb{ch%b=yUW#NQ z9qD?NxyLI`1O%q_#QSBk(|PT-9N0uzu;KBwW_) z;cK5Lr)b*Q<=nHSwb-k{SRbid1^Yi381#d{rqew3(`!G?m1+m0XVRQcOI8EBM(@EJ zJTOVtsLTtX^7%<4X8zheTQ5Z3a+GY(rmL5LMXS#l?x;*#ZGAI3u3lZ5JvrGPLXc%S z?TWS>8KONH(ryU%hFtMqA@rLr+)kRUIMXPJx_b^xg>~tvP!W8NF&(fT9{{WkH-imN+VuM;}`{ zdK~PWUW=iKAkdyUPmM8~w)#O=4<3v+u=ou7ezJ zgcdEy9t!3?)%uo+m?U28{&11mG#$Yne9xZ-y%Ah)*H|w`|9W66*1F$;XL-D|!_*HY z>C_m%`Aj3WjHD(+ezsM`onP~TQ%lv{he-Q0hF9Ur8OPzxnKt7HEp(Zw0d0R1bqhIY z?-CO={VL^aA#0;{lLz-uw6(^sd&|r%kuNFO291jc#9#QthZ~*QF3;W2tx&rO)A_tU z^o(3|c)E9#U^BQ?3j8(+W{`?lvcJXiuP#6_~M&t3y& ze^JG6xjL?Lz#Fdz4dx4e6cOBA(Ou_$baJ{hbu7|aPiHk{R_5?EdFiJ5W4~Iz$%Sog zXxWkE#w+xs8yvTT^BU>*iDn#`td|(`@nIzec~<#RXtH5#dlUPsJl^Rz+iml@o=J*L zJEuaIr@C^EJg2OaB@?kENBG+9V|8}4+3+vAynA=kJYK=suUewQeJbQvj=Eb2DfPxjATs8+9ksK# ze2VO>=Z8b)?~dfCgTOxnSz^u%zFW!^Tjj|M5-&V0nqRmUC>crE zWZAORIv!nalbTIAyCN9Z+$fVT&$4C|-dOrzl0#Fbr&mblr|#AY0gv>zB*laL!uE^2ga#0!nr;KkO=s%^eb%Gubd1()cZ?GkKeSJ)Hn~TrL}+X z3cfa|dMD?7@W3e4J@z`Wa%(-9t@GRU|3=~db?}5Ourx-iJNG{5ou=A&M%5mRl$d!r z<2Tc?H+;^KLfb#TWJ2Q|Gw$;w2zW){rz`iS<3%UyPr|RC=@yE4uqqt&!Qej2XET@Zkbh-(@>bIvy;I5uWFUViBkP- zvPN?to z;DF{Q(zwS`q*48B)DsK|ozvTr*-~lF36z!>4eG5b#6)KzpzG12weaB6jkvAvp13}^ zzBXJ?z~{S*2Su}lMdaj=i1Ds~m(#}=O$|Q#rAr(8dAk!Yqj|*YYEt}Tb@==8ba)Ei z?9<8%K7IC0Yj}J4h#E;Qw)ZmfdZ@#5y19jfnEq@~&~d#a4*7-tw~W}A*MbD|KsN`{ zy@iz-CzZ^jIV7t{%u_|}607wtmQ`LITt20+&q?(v0m5l*)j#!0O(v2}zT$Sn9F~_B z5h?d3I}A$Z%)r_9H_(PVUdb(S?3H|x-2tDmqsL-Q3lC|1?wALelQmfr~5x2V+ zDd0^RcxK|7eBZit5@&z9Ob3HoeB0?$(SxB@Km@nN`8BmpovKT$}w41stxX zvz9FTGa`ch^_o_*EMa3hTMibZ#x6{Q^Usc?OJ;aXTA@w2~s*SYFMMM*R)3}YV zsI48Fjv565j;hg}(%k7{u{X#sVIar&NiwPLcmOplDWaFhog8;%qAt`$+ngn=&+v-q%+n~P(nq^DL(%&k^2k357@*JOG){Bk+*3Vd-xZ@!{X;*I> zk#EZTDGc^Qd3lS8I*PoGDpvE5xmDo{WqZ)82Zvdvnh$+|IRmGW4<$L)Eu6bTyuSA? z4S|qhqVVRv4Bfwi$~w90IaJbt%A^FpvS>JFb=#GS?M1PZX zkb9LJhp!_0NJZv{G1OG)AI6Fdnhm{D$B0O1s)_tQ*_2xUGR3Ifa#oJuG2eIzmYSglMN6cXcnrOOIPbq`hxq!9W7&HE8 zUQFh8{EfkQoa&-k)m=Hlz6vb~170BQiH5?8`tLc2tQyyd9uitID8lGe5n7#kekAd3 zET#|l)X-IpPt)v2O?`Zxnj`Y1``^qe9nulvy9$q{t6VJBe|`zsl{f$>iNMfXiXtAq3Smy288UsM0>o6 zXp5vlz1woQs5)Us>Za_t_}uQ7xSTkv=@bT4i< zF?njYivy-V`rSl2@>R94Rd2CCt-RMA2a#EWU~Z65LPQY0^)f_07(T1+ccB_lsO?^iAwM}gG^9% z^(shBYYOO$ZOk$oN;aRNKDEbqbr&T+Pf!LXC{V zR#9S{_wC1iGAO0_u5mFua83uK&zY5y+jSP(^=Q+|`?sTL?={1W0LG8+@}LNFrXt2H z!1`UleXF#-)I<^3S4b*(rCo=_h{g1qRSY(FidDm&_-fr^v2hPQWO%(gZS=|i*lkW_ zqKDro`B$U%GC$3IeOC(8e!|yXD$E6tHtqaOBX9AS0#Q)|RHXy%txPutVE5I{F?lP- zLz~};FVQv<%mPeE2D#u2{G`|Pc(mBmtKmGIGGL=P$X~rcWpS^ydrKrd3hxA;^pufl zCs7;&7+l->zMkzdASaPY9#YB&GXx-fj?u;rK%9iPT$Svlwdu%hIRT`z&8?F+_Fjo; z^_MpPy5vSl2NV8*e+9rUl<-(vprosA+$imd(u8C|aG0s*51F!*5~`?w(uDu#T>xU- z8fQ=GX3^2EW^|bcl%C{gJWhy81GiDW=6e?XHd&zt@GosQrSv zcP7{xFD&f1{&~n`dXn2wp5`s}xohy9pYItC4R@#5n89Akj}G z^jU6 z@tVyp;~*LTDu_B~CqP-!g#ynJ=A_RzF`IId?f$xbjAKiVTScRT% zKPzFFjp06>Rbq`AI18@9Kmj)HIWrfPBJLJ-^`P+#2Y9$vzQj6|IIj8wL|WgH1Ou&5 zmzL^-L6uE&v~655(OR*2%f>ke7}$gNB2Zj>yvl20oc7fP_=sd`&=NFYR`>mtj$Q_p zuPcTF6(4%>*>(TQJBWZ0M7TSfQ?lVX3pt%{pU3lb8(nTR3I%Lpt`sV!`s(IGF68Ux zztbtyLD>#tof@qi^0hJ(biSMZlEKZX@R(KS=@glacH|ppeht76kLrGXq>wBL{Spm3 zh*a-6G5d-)r%2?7=kZIs=ziy$NAu|9JAzd|Z|u*f2y8bB<&A!`i%cji2HkyH{@G)_ zqrXtUexw2GQkSo!ntTr{aK>U1|GC1yKEH;~=OiXya#4F(dDwPL9jspI6Pa*qvB$LC z{aqK8jVn@x`Qx)=l#rhf*RbVh5}7SAH#q2wY2C*y6HQKME^lF?24UWF;gI{OW+GF@ z_|sFQR4sVFSJ)F^cl%j90{F2T-8@B6ecRS);LPJr3W2)k>QcL}rB6aivBAbkf zjT76uN!cvJMEH>M((G!3YV~{kq*`agPdNwy*wYR5Nd;PWT!j@ddT&`D;U!}_MnYr! z+-t%m?4D3~@z1tP94XIpQd}dv#G8a}P3Z2B%`Al|Q1mb5@vn_B?e&Wf;yW=RQM#g+fkGb0;e#j2sf}&X#9?}_e0x4=o=He(7 z?5CKJf!8D=#my}GH|UzI-@lu9eSPkEHpPg(tNFPDXp0PE!|n<%1B{yqYPQhXyV;k1=|M98qajm zy}Q600oXh<`^Q0UwYtZ`%x5!u3JnJWy#0SMQHxwAI#-Mi%w9KuE>m|^=~k&JRi9WN zEWTruY`-ED1De3F9xShVv^d6~y! zls-FW-~fz!(axUnL;tiU&8vt9^OwP_eft8t?}7XvA4>gO{uVFk)rV$We)t^SQ@#ds z@NTy%kxDBXV0>f(@}40C*#D^0W|v+LC1*blhfWTAxqv5;C<(UtsxNL)RFO0rfUQ{i zwJS4)mu5GK z2>0VFu1jtJCB7rrbIQlMcImU-DT<=`k?(Sf+dJq8E9hcG@El0xU|KE1^_8S?(p_1U zEBQLr4O%wz^6NMP6Oo2rcj&*gM@o7*dJI;Hmc^|?UPG%%k^9A$sGKY!zstR zn>zB7(eQ8g)h(y_oko0~An-gQm8Jt}3Bf{Cj9qqs0HKIl1d z6ZxlP=d!t_*B*l0`lJTV#IuTTX9JI(48{;Z>pGbH3SbQnq3xUsjqo0Uh zf%ig89RK>EAK1DcQ*FX?I&l7KfvLxez8Ln%QD)5O82IuR4wY@Ek_}o6DeVcf zZc4usHUPQ!bF5=CKhkfsLkGW*WjqIY3~5fG5|#;< zytOF@mw7pE&wo%excw3AqA7Z5{w%F27I-tlz!^dqmS)YUn9KFCZhVjBiiai4+BAyl za(XO>&zG!^G}zWzgE#lT-~kM^kC-eE3O#zPgx5~JccCqzGTnF{*)I$(Wl`qm$8v5< z`yEB0&W`J!GY|_9tn~g6(7uxokSFh#e*F9v>-~1#T zqSf9OJP5qItBDmUQvj=bbAde~ay?$pbl1||jjhJt^+xy0>(>fGHJcb$`^2UXx@8J> zUXvY0wEu$K3Ub0mk2Bx&y3AEGfS0G9iRYcH^+GcNYw!&o;a_60ugrioKKT^H7jWqd znB>V2=ms!sPJ}-6!!F@qJ{G)>TEcvcy}$2tgnLnrT312t-E>wp_m_>>Gn_>}KOnFm z3XcDbj~2Un3pXJ@s_f|QuJ%__Bl9n^9jVBSnDi9aD6QTydG9pl%jOKt50=Igt6pcm zcI&;R@QD&p?@gUCmKj^h$r$lqs)2KWvyC$`igmuJo7G85Yo%?gP1t94R~mIlKn5C8_Av~jMJswc9BL2}5<-e+RW=J^CU2FNBb5(6QFB&T3zo(> zn_twM0~qG5AKu@cvg3Jiec~#O=oA^J(8T}8)?3C^^{wmQAPPvA(k-%(4gu*-X(gmV zO1itdQ;7wOScE7gAfR+BDIwjx=FQghTpEs^Q0S3biD@3EilA>QI*d=G+42*v`SD|v0tWzWh)-h{ha_f z*c&v{rh+YS41}*2xrYZwBO1V?HwbYt`1hkvM&F;0&l?E$eHN0-b0kC&H+fjn%Pq7X z6r@Db^22|^ycGD9n*p_ltJy`DXf z%^sbOM)QymOdMW|-*(WmF1FGH*tL*pVhK3F&ik0wUj<&b=XvH^kdG6-Z)Tgy1m@;y`VVqk@j71ocBGu*#8qIrK`O}5dT2M zK%686?#DOd5PY&TJx>0DWZU!Puhva?r3X1u(6>O`@>$7!0rWxLYN&`^q=&q*XO=u$fO2JIj0y-tZyy}U5i&Z54e!n~9=oDAp8$zeVli6n6}W#h`i zZzT(3i1y^y^OcPO2OX88%y$57Q6>41K;F3QN!iss&4nEK+o@^6+F-h; z`;W-7k24z^ng3oKo)auIr>#rET0P681I-n+`H5l9uZZ8aMYeMuL`LWNf_=`F?~>DN z3K3)K^0!#XLeM-_`porL$mOE2D$nOK7Ys_j7{Ac(f1(!2Wg9w-jNkH=>xKvp?m)6q z2&W?H2K*q|H-8L-NJ#mO9dNd~w0Aw}B2nW5G(KZq1^0H8K?in%SGTi& zt5>&D3}za>e~EX;dRX`{%sHRuO%VP#4a93-iWa_uj&Qs({`rt^v4IGlkXm!W$Z`E; z{D|5U#Wc*7$zgu!a?u^P*>QjcQA<+c@MCEZY5e63;(Ukjqk_%xDA!+*E}d(k06 z8GamzaUiYlq@Wxsn#pK)ev@ZxY&e}qo8W=|)$)-2R8RGg&syy-uX{GO+PP$?C!R$$ zH3LckMJRV)W9M+^m-)I1j^XZ50y`F3-#pfG_%fYjqHzjUZOX=mXM;;?Wf(zfKdq=z%WM1~t zIoVOa>sJ6XzK=`{$zl=pvlq!wv-1#ghdQgIFwa>QnqM0n&9TpfAO8_WNFMta+Hg$u z6Qy_r4Mj?`{20(CO7c_6UQYfjy5?4@iaP$t&f@sxC23c|hHI#Jr0DCO4y1s?Z0N8G zjNGd$<(3pHFlnZW#_q5yx zCY|;ml1Tbug|H`TEVcoElc>#e*O<*8laG%5x84J@nfUCJt$N{>=5o%5C*tE@7c?Tro7H_IJBq)Cclgrx?%)s&-gHAZf z3U7wxX++1B(=vjJXhU0m)pfS7*V96V!MZ-dC=_D}Um+Oa&BTo2ysM{GX*&(5kt7OHmsyQ%_1)N`d4=b(W&NIKEySpihcuPjT=&K5nzN2c{4qIFjI%f2Lsn*VDe znBgY5lkZxeV6$t4)~B+MnLMulFIBXK{*XZkvXx280y(xJ)stLY)qy!1zhhfht(KU{ zK}tr$5SGvzc4q5fj5kuHVR^@E)pn6knBQAeR-Nso%6AHvNXtLQ`y=O%xG4RrjxQ(p zfTfbpZCl$kP_7X5e*oEN@Qir4OC}{rdiaCXsfoO)_YJjG{U*ay_~X?nQZ904!hQ0( zUyX$FuO#EFhM#_bl0DoIJuc3B3WB#U)mA~qEP>}i$>M(yA`Z}3-n6y<5?&k{I?qXf zw}kUUE<2X>RJSDs+Ejl@@S71nmFbiGVXO#~`z5#{NhK%$k-DcqBxzVtZ1<+%tX8Y{ zKawCcaBweDWf;_5Bq@k4KRT#*Wpq?|6%sk*KKI6qz> zHdxQpeY*kD(wwOgU<&d&t;k&gJ=Zqd_w*akgeRJfk)$g@ye=%f70^j`z-Ore(Y#6P z@Q$WkM1rk76*V&Cg6XRxwwo#iHJv{%FwNJf03cY5IZw1;8)PDJ3-+W>L? zNZ~`Z-?*N$Qp@{z;nTD%$1c)S@I0?LX2t*-ZKdcHM`bMW!x1W{8qxw1KKH4}z=!t| zcLQx3bv}aFc8xE{G~tKy`qVw91Bkar;tnuAd)S@u1bYlfqyF!gLXP}Feb&Bf;MV0t zzpd?V!^g)Xj&Yv12k!y&OoSx$k_NH_*}lM+sU^L!8!yYECN-#cD;2)GJ{oKEcE8uB zKz2di&xMj5X(^`;u1XKsZ5061*IgeEy9 zKz7SS*S_()@X4sg9HuG2D&CG7`rkasv)QlNMlE_H@&7kr{>{$#)kmPS>#ihK~0Gq!S|EI@bkke|m%yOu|c#tb|OD4>_+diQ}LC+1kn?} z^(^-W<6vo=L`x{?O}X~BJlpCanYoR?1G=myelol1wehWCZlwl|Q~qg%feVP%lo){3 z>};UR;IrQye6prUe-oNpRQ-=R*-CN&ZZRHPw_ZD3GGW-P{Kyo?Uucnn`J$U}t(zXq z4Cw&L>E@TMX0T?%ofoYhw^U0j@ouf3n@(QW?S|ikdYU(Qj1{TJKs~Y zVX7ckyT^&w(~=Ny7^Njf_v-e1`5r&&XDsj(FMkejqYgAs_Mf6lcqkjcYXd14tnxrG zX%CY>=8$+;_G-E4lJn@8bD*LNC7Wj7?B^qEcHm(XpDU#<$G}1YU`OeQn%3!)-*yg~ z3Xp%seAA@^eq7eOmYBwFY3Y?4hwN+)FnGCA!>b!=%IxQx{00Q!GpYJ+=~_Nub#WFg zqSErzn@eAgOG-6-wGe=EE#y92Yd1SJ=dp}k&a(Cqc5&AiZ+PFuQ*&Ij5I~OJ$z9k3 z5`0&tZyP?5`sOc+YcSBJ?9Tv!RF62r^vNYM3hH?=>I!!J;cD+R=oempy#`!Lw7p$r zZRJ;hXxb1o_#mjnB|h*yKz6=@7xZ&(y{d39J$G@Qd$QjSAqZ)50Pw~MFgfmFGrUG) zDgiGii+I65NEp8?ACwIEuiVDk)bsu=wQ5(mZNF7xE0GQQc!h$eIo$n{>g*Imj>hki zStY;X9QwlFA`HC{49%B*a;f8|qZ%qFWk+?b1W%vxE$gC8&bj5}-jTHds&b!%}T zx=F3FFCd0sz{cR(dXn?3=<5PLa@GD~EINr62t^ZH6GRIf7NXSYfxX_h{kH8C$7t4$ z!s3eFs6duEJlF4#*T@S0i(w+I6!38{+N8-@e6v#l$d9GGM4w-L?CsS5CaUJwAb{(O z>3*KsmT8yLjEyhT3~*gb#0|anUP~JBARE5ICtu@H9s24#lrq#j&U--X%8YrV8D&$q zK;g+AE&iU*&#)otv`xH^08~o!GwW4(?1?WVkKw~VqR4=M(?Te>(v?ju^n+=F{L8)u z8|LN{7dC(6kFHZmy55KF!Tc9$=aE_ymr1feG2h75}e)moWBF{XnLYpJpkmTtQ%lur~!Pb!~9p z*D&9~dwow%2J;V-3uVDEFv6EGc&vNgzn@?lmR`UI!<*gFwE|EvWy# zmp5JZfYjE8%3wNoH**G{E^`jtfTE1otcS?@@0X8vc##6^I+Y#3vA#Ak`sHn}eLL}m z)@Rk$_&42O$qyh^0NBTCH^V=e59-HL&=Wa7wFBjb=6w;U-({z^4`7$p_fnP*`F&Oj zQV%HHH})gI>E^wW8y+Qv7|*M@5!8vdU?En_@_xmaaOgkSym*nj2>N&70 zd`F}5Xz{zdai{J&0PWO;C(FQz!M&yskH1g|wX3`@TFG(C>XtCdW9}~2v**MDBz{;9 z<=LRCR?7mNpj~xSjf<0wCgkpTM8)r}xCnOWn(@lH5IjBGZ#;GRAXU6`3R<=v<1W+} z&E-RI*(vdDsBv&|J_Y(f3~RTQwvFVp5f-)m6U&r_WwO7XiCwmn<3Ux^)w%M{XaAZNe#Ns z+^;2}#0HkEx;4ib3!ti_()a_KG1j1WReE19J!Lo8mhDKVYNxv7B0IE?>MWx1yp6Fg5d&7m}RYvH8wlI9IiQ$Y5#R z{N0~02)b9s%$g+=3waLMTSM?})FO6EiIQ3tiMH$fh7eK0+mc~%g)e2=KHrv>r@HOa zZ5CBh=h?Nhll2YMRtGm8j4KX!E#IDl`J;@F3!nAud^x^WMf8S;>`lqleU)v9$D|V~ zv#0<2h2%w<%0dXCa)pY>#rqpVX}n5@MG0@PLAYL0iJ^GZD$RfGUXdl_>BPTUyC|Ro zN}w7CDxT;iQPR*rt^}~vnh(RF+&oibGarQQoJDUklX^aVJDYcrGm1sd%R)xveg~}g zRp9Zk?*7QbV8Jj4gRb-*Y2Bo7%=eUrS%GQuigS0f_!(pxG7I?~bAII0jV=s7g4{r^ ze}MtMpugHKfCLFMcH}!|8qP7@Urf(PJf8`1A{AZ}C6J~xMHwW5z+dAV-H%unbTfh) zK*dgz6;ozNxx%AD0q;vyHeC5)Mm7xt3~Xa5Xq^yr2_Z%$L!lGxH83*Q% zlc{cDlRr7)b0j<-`$nRjLB%AYAHn16ez^+vyQgEBK|CR%Y-ENL(o9h=vaiP|SJAd~ zwEKVnr64JE%aL}afu}`9)#HDAVbQLjO%NnVge&u_m%Vv=UHyO~3u`g2o9vemE7d?< z>oN|HQ43Q0@H@Q4_!V#m+2xWXIKR6qr?}ff9gcS|a9UeJ1($)-N&WMZ+@K_?U6>I6AKZ9>O^B$L9oP+=jPKrC70z-W zUB(qKOYMC|*%|+&xck(^q zT}!`-Y9B3|R#~Q2^+mZb@ORuqkU6`*if{V^gaU0=hbL>s4LLFs;B0|8aI06>5qS;! zcQKqM0N~7%BDsc6zvBN3ST>o8G6|TgasIN234EyaNe(~9M_42bD?#qb&qf% zr@1SRsAtm^xF#R#jSnJd>>!R-T6F3T>{m&9 z4jEGQmev=kfxn|`!ykBr8iuJiBj$qD7+?;ij<(WhF2KG;#QuQ=D;W6(ubl$*;VyB- z;sxFDAXp{6=*8qh+Ts(DOr?Zqg~x^LE-3|EDnNsChWvS2)xZhWGqu^ePdfETDBVCJ zX-KTJ7=+R4blGqQ#W)?E>Xa{n4f+Wc)pZTAzDwfSTmbQz3l>L(QAPL(4hHjq|M#}F zjFsaVBvmgAX6cVA##?GrBc_WhD#4-+4J^rOn)%)boy2f)hMK|943w+S))pU=7h;N0<&a*qAiW{-e^5MTcli=^u?-!5p%-uL64c-kV3hL2 zjLbeA%wx4J7)r*Q`PFoQMG6Xo5<_f?P%_-7Pg^%fi<4);H%vyEj;7t=#(d0#-#Jqf zeNS-T-&|^i-Ms)gyhE_`5GyP)imr*8j$*14d4@G8nCKFnH3f~;e@-_xVpRxj&e{2` z;uB*4(JiLaOK$iS8+yi(k<)TCFD$phtRaK@Iqm&Z_1eUw(xsaE`(A#g&z(EO9d;D;Bs&*<+eo#vmrbf^6I+2>Q*a5 z8t=D;u0_|vrxWyA@eP(!F4NsDm)A#w-ZC@XoGC^wt-(C#C_Y$3N89Q|%xB5=04G{k z$PpikgY=LM;~_&hnt&d>&4f5+VZYmb;MMxMDA!v2!`7(&&-l(7+5thB7?G(ea>4Z) zMAvuIg18RyQJePTmB?=Lm@KbYY3MK_0-_?NAE^bIJUph4h%yn2p^3^d5yOA-Nc{F; zX9)>MPEcN6n%F}W8skJPMdjYE%K`6C+`6^3MHXwQyECiqu=P0vqO7dUj~S*>^kL#p zP!^%&XS}{`Iw5jvZc2y&=6kv)XvS68V{;2p(O(tbrZs|3YN}giue~Nu%*NZd7FthP z^|pcfpu3R#Z(Y}+JMT_Q<&XP%50z@Qz5=J%J%LX8PxaC!sszc-9Lcr1@L{DbY=1e`NGwcN%UiGU9M-`rX z-t${o6x^BGA0;OO7-kOnyRm3|Of*wTWye5QX3=tAlEh{R!cE?BS*AVM={oaCJG<+u zic~SHx>pia1gJsb-bbp>Njk!+y}RoMEC}xk&pJ0c>Z^*eV)wkZO4p$qs`E_bHib^- zi{bUqw?ep&2vE1*DT<_4RqdSrp;GO>FFmJpL=U46ABzfT!bokAxt*{^nczaTM5xyV z3|ZL$_xs&<;+C?O8i|D#E(+9&??AazJH}Pu=J^1(hjK4K7GwAg+7D&D72(zRh6rR~ zD%ui;{F0&Ps^L?QZMi5Rw*0|`crgw2+m+FSmJV^1p`k!8YTHIbp)nUoi^WoTB1D~* zXgo{L&AcDQUl)d)fJ{+EF(ixcKPuVt)KhmLhf53&Q;P%87Vo`{t>KOzZY-8xjk>mx z2D{}*wq8zmK3SjcjC5B+I0k+z-o|vFb=A6il^ek*Vv5bQ!(ql{j?9ExS046KA;};cEB_>2TR#admw;UTY0zoR=|{iMnUQDEZR#+4zqsgZZ;7-su%cJ>$a1> zOkXEx7Sx@4%gC3i`7z0RJBNkE@e56EL9~ga{MsX39eZW&QSs@%Q&$^Dj#*_2^)>M| z+20=eI=Hfk8~!E<@(MfDjN_*B>!DpH+oUX;JhqO}l29;DdE2FZ%sooIDL;8v!wknK z&VH@%=ag0rZR)$clIt8j#G2;JL5YO>HzeT`)6fVlTZ1az-}S+s{)(a?zOBy}&- zqL32S_nE*#E@V}ItzUO#r;F{z)qe)05%uEhDGU!wK4e1bMsbJbZiIpV%SW$8#>YdD zt7dufz^|i!Tj1v5&S7pil|W8*O9d)*7tVEVbW4Q0Nf1(2kU-~EBJXg=%S$>Ketow_ z+ycHqNHycJ|Axwj>1zRw=dBTVCGxs&IfT@4Hm(qiR+5f7jC+5w_XX(YCECOS*;>xS zD)QZ*3Bi}r;WzSrpZqJ>gds-KKLW&m49zDD+dj!-&_t>-^W!})b*A?;iZ!oMwuqEX zKy}9L9YdHiYfp{^tlry|+Ye?@S0_>TV4BJqQ0p3G+nNWNwzcxj;}|vJwc@FUwHTk2 zjDPb^oL+4%9guHlzx*|cVNmk`QB)GN3U)s1y2}gmr7afL0oM~`n>Gqcyk<w9y>02_|SZr}V4|W1oI4R}-i{2ljkb zOt1a=Q$lQ>Dy}PEd`tgp6I*S_o!Vm)3}uK@TI|>KvTbA0}Ypx-OcCoHC0}Q(1>jygFTdqlmb|{lJU66A3QLQaJ zA*!AqI>lMEd-e)>-SU%c1#GyB5+doyeIp9D8TFs%$ByI78kDY~bEysMvz}BZA1i!{ zuqJa}YRKBsq$qhFcPDB>S%2~3IoXXlTIg17jc1%YWHXh+&Qb!uXj`m%D*9H4>(vYl zPm{E%8A8y^e;qF$8cd05I90h$_)Q7S+Gw6>3Yv#K_kT_dg{dh~pFF}@`P$!N!R_~+ zd;gp}J0-B_Rnxs-S2+SvN|=Xl51tG*E?sIdU_Fu5PHINsx* z((cCci|{bgOODzEh1tH`WAICgt(Hb+bun+k@icMl>IAX9Db*R(nWK1!%jm&v)q z(}2f4W-$@=XFWY>(GYt6rdM5~{2>>-yLMr|>^< z8EuGeC12&0@E?Vj{NX+vtIW`$zgsWO(nQd#6tt=>%(u%}znW{?em|koS*7-6UZuN~ zXVb8=bx7Yi%Whq2c+#hB<;4AgSWdo2?< zzUZ|52RI!n6hhf9M>v_wH~fR{14pm<@rWfnh6x7CoJ?Yff_4e^@Yc^`Tpar9@kIAJ zkLqO7BdqgL7CrYdc73v~_T;^&*$Mi>uY9ex6y;ZmsT2z1Ylc+>^6=CriZ>v=xisAS z`O8x1(I!d-viM7be#32olx5R>pP1jMS@lF|cS%mDt5&LQlTPGlQ-aOHr1^x9AL=QQ zIn0*ZiWLRvj9W}fUTQ;ifFqPEM4{VzA|{iZx<~obdB4e=7aA&=Iqi|I+GYDMJV#d6 zY)Bzzz*3f~Lz`vWnVW*>ILXXj^@uXXPY$FA=_fpfMm^84ADc0u z&eSBzbX5?VkRN_OjP-dgw2Dy>Do!57rjC=@Ybx-;*&=GgrzMy`GQCjTtl~*5d7vtr z1j2sy%_}j41&0~8s%IDz8&;F&d;}T_S{PjqvwupKDjA2%(2uaR6CQX2y^;b)4?&M0 zr93)Hbov;HMmGsd$I)kh{34w+-C&>H<;LG=G5Bwb*Jy)rdc4t5h^ibe#|zzK_I-E{ z?7ikyFX5#*esCjB53E)r6Kk3qW-HcS@&+5;uIbR{B7fau>Mrt25(i0wB~*DyS3+|7 z!XJ=bf2e8E9B@KlM2aaX+fn+Rfnm+ zt#YE-DGZy6=!2}&=mR5cs!zWLCD~Qw+NmzTD?Wy+a$By8pvJASt*=&6{F3s?Z|p;X zGFX$ILk6)vk#rb6tD4-?gsKPzIX+sZlOUI@r#5@cz`ThSWr=mo`GO+=GnWnS^0pvS z8mq$703F5gGXG1;&RdPlm`8om)6VAyEPy-7Doi4YN3`r@ZPFPZ+H zMhRxTFPX6fzv-xxYsAf+o7i^uvGvuQ22+96ppkSVo!zEz*UcA4;lo-V5#JZf8u{-v zcSbA)Kf~oTNLioU9P-2T$Dg}?IlBELd-rxrG?{gjKIGJ+FjgLKig=FOg`~?imyaVZ z(sY;Uqu*39-crt;6yPx)kp|_xi;G+m!r&OMqh6$%!?#ku{4u}{F+{f|UzAiR$Uf(c zN_YRZg-N0$$!Ekuf(YV6EM8nYiW~;idCH)#lM>V99(R9FB0LDWCF(vX!-oxWQSQeb z5RH4t&V9U!2sRaw$Gwn&wZwWoMMd8A7DA_xk@%u;x70~D#&s6GuoN)OU{_KXpv|}> z&ka$0HiYr9{>#2@d_jkB-O1vb38gC+W???Qp#LDtbP$5{C^%5=NxzT%JzhD@~3;G|Be=uJ5w z^A(KG4+)UCA9R|PY8jth-$Y}JmcSiS-N;87XH=D1T~Z;|uV)zIux;=()_gr>gmC#7 zZxb+WeurX539HzKjf@lPz=_1C{XShN>wN1PxPy$;aV+gler@TBQ6t%$nZ-!WjtFnW zbJ!7tPsri%@jo2bCto?cmXe8`e3?g7JAoe7;#5xR_b>N{F0((bXIrP7+z z316t?Z8Uwk`+=ghnXC*U=vZM6`$^X3zD2UU6`(S+WJx$tToZ}Q_YuKNpIh?IVux+p zaSk07wW-yL3{yn3g67Eq`;~? zX*}u1)L3go`EIgTtfD_tT}%6y7c5XCR@$2|Y)C_y>bzC5Qsj{tV_i+<>uw@#3&N9p zGOdfO+W<3@2<*y6(bkQ+OAVcJ3o3k=`PR6Np7&d9uyQTBLMp*S$78`Jhpo3(D{%-MLUDZ-XJhXuHBY`iE zqD4ZXNVsc>T)JJ;vUz_c;bmh&p$mQ?ne0qAQfDkeSQ z?5^S<%II{m6ENAzMiWiUfk(K zinqP2)fr;TG?~AL-&WN0Ic&1{?pc!u&e@-FLJNeGQksDAZ-yXc36CzNS)p>aGPiY0 z$^Hn0Dfc0)(S8#quCw}+)KFLLfXcP)>aZ*;#3996<5wdtjmXr|b6WPE1JjC_rhuO^ z0>8=I4zQY%{a_5Feh>Oz@m>#Zy^yKGxfFOgW;Y_PMl*q3qx{*Dh$!V!{o=U|$zyGD ziADH zQ2G6H9dB6*EI)10w`JvJGc}{WW;x+`JuPX-&w}#os=m!7RGr4iM%Fc=Tk~l-<#J`i6YR$bC#}HF)wSO+?4zjO6e#K9PY-fx2honpAIEV z^?1W(J4nq!DoO?m4=h;e4>-^R>DRxs<>csH85-CxVG^VM;c3BA$4~J}5rK8nB=zTh zX**gFm>E8fn-%$r@U4%9>+|>E**>t9Cd_s_bB3M9od(pY3gN{tLkeB9>d_###xrgE zW(I6cyEu{$w+7{j#}isbP$XHxxR+!R^jVE7@*0#Z2F8(+n=iX6)s2Eo^f7GNZ87|} zDPsP(s}AZY4!nEI+a8JS2!WR*Z}B+LQW2rYprYIIT3f0G;g6|I*~C!8UDWw>j{0ti zM^!E%F=_iI)TsiiSj)(+)f&yCte7gVQ||;i<$s7D+>|oalUP|{+I;#m&Do?CTB%4K z&iRp-2!(K!J)|b35t(G>AhM+@Qe}obduPh*(g5~Yag1H7VA?CP(x|Lo#DGR5M*p~< z_INAe?i zQRwluz6i>bd{_MxrjeN2r>e{;b@HLb*q(VT(Vd`8t;jMvqX%NO@z$&KXlJQN7_>X# z!oDH7jgJ{VIVpJ-P#aN<@UWEwMM6zAGXAt!{tut!)`e-Gf?>;%cR#eyB8TUwdjgYI zKR@2NAY>>vvjS3%dz$8MOqb%mv(_kw_EQV}?10#p?Ih3-$-tNh)AXHBy!Xi~p@*MH zVU}ThPhUI_O$YmK<96~8g=p2{iaFX4Gmb8#m_XvQZx<-sj1(g@Ioe#Rt%u6A(HQDo z%ZX3!(qdOhv1lg$1Pvsm8=#5+Pnr#xi-7ivpda5o)}u|vpRHwVl9#4WYi-rOn>-Gh z#4O21r(~nN7-q4UvLO${;SFnRX`>F7M_tmBbgK>|_OY5E8BIl@K~>a#D@h_&S)A~L z`{yb#3NH7l_9STuC&*hTWv4tMa|sE=hSG9MDlWH&MPkiZWs+nXOBb>v;+UU6QiGph zk>+=Qa`q6?9#n%!@UzV`4QPpy@{f?hLuhRdJCHKnWg&_-uwV}Ia003HJe2(@8jdhS^ajg`-^81=j_q+yaZ6JAVi&>KtDnT5cS9aStVVU4Oj)cM{|ea<#~mr{ zO}H3(mLhz69cHPHZbwZwMY&rcc`lYpJ{e1eS)8e!^I{2;UQ^VE-|Y7FiE(%ESHqp8 zZ*y&O&oy{WG~Fr&TPoy*CDP&7bpL*H=0mx}b+Du}(v5ovp|wu1SPkRgbulLLbiSuM zeiz;K>T|$Q{n^x`X+nh4U~t*zB$+p&m2T(3jwFfF2A?I$XT?6$kuxc1KoPPG>hNz= zR|u?{o5W@!CYJYcFx^SpC(>#^f#e3`cKVnU64|`LuZ6|BPyb$JG`UA5pYigUYPne9 zF2oH>35rdA!RYf8#e!BB0hjG?@E=hw5~)axf@VVM=7S;stSK}8c3y20BV0s*!Qoto zq^}f6RW%mGb(7>34@O#OiWbXy(6lWgOV38n0nr0KYAr+Kg+m7;9s3SiZ*3_IU`V9(0 z77ID%Mv!SYggw+P>NZnImIUpif(_{>E@bWyUePJ^COEUyl`I((S}^`xpwDxhko8cu zH(j!VN>}7otDdVR{-iZRYrcs|hMvMM+)JA>2=)>2%xcnoEZz=Ty?r|SbLD0Q;poW1 z!wUf_fk5e3LdPDclqkDQSOQD&Iv#pd2urnww8r-i;<-i@Ax79TiShcO->@|59O3U} z9mw9MFk?LZq;0On?_aHhVpUh6J=VF*NPQ!V{9Y_roc6_o-Bj)DG-Pb!wKR*7NbZPd zyThGAZEr=bBIs;g+^{Y=o^xpQUU4jJujIXrb!PD`LSCS>^$%f4itWs{4heiz<<%!s z0e3<~^(xQstm(nkWOTymQHfYaQc#Fo50hrM7kg&y(m_0^8_EaqTy%6VP z!69Lxgw`35MX9d#B8(2Jj0D{jVQwUo9?2!+X%a9ikFH`d7(<8X90?VFVV2^aBQHOf zkA(ExSYAr}jZmHo%{HuGEV;68TXvGkBp`2Y0d*h%P7!0Y--10lcIcV)z9OC&6_wNc zC&wSvvGiA6hggQ%N0eF)n;a*P>;pDfzU;m2Z{IvQ^d1rXw3rbGovTGOxs9ipwaZEW z$XALoBU1920+RBg0%JcG{cBF%VKz7VeObgJyMX4DP><9k4C|DX1Yb38vXxr8Ayvrg zOYmyFnmbdk&yqe8irZpgiJFXK{2K8S61lS%kd2gAdPZufr2Msnn~YX1e>%OxgA@eg zX#*@_*xf9wnz7^CAF%QJI`vub(=%yLcAZ_LpZ|@tFdIpxfF?r;P>8b zHkI3!q)U0hr#sf{CrMntkWywV@uxJgN;7e-Hl68S*Q;PTraZaKFc^Lti#sri+n}(4 zd7ur~8%}j$hWy}YqfzqjNRsno`aDn-du}C*kYy_R+xlwo`wdbTXX4}U z*Wa$oG<4c@gT4jr^j3AiQK5V(lwA+sF6xXvnoIww^iD@2glHMu;M3Q|Vl9<-<*ZWyoQK;n`=x6w4U32Kp}KPb{>OOmNr1 zMZH?rrrDd!B4L@wGNLU;lAc&*sar9^cwIBW`25^W&-{q4EET^v4$1y8z{7#_^w>hW zgapeHyEdR-9bGO5`h<7bO+{dYsDCznN2#$Ml8DG-b|LR>J5m>C(!=@e|OgSvk% z48D!$eenznDgR~2dg}aggGhLI3%|9;yXaT^`ISbVxk&Lfs)_O{Z>G3}jj>;ptI3gu zj6Sh@!TPA;h3mnac9)|p3ysH+4oUpE^g7gOG3?dJ8z1&Eb*#EC66ve&CKD2hAT=%q zZx_k1RrGZhg#2CE)Oa%5-!fK=4u4LxJ0xWweH5jfcD+}~S=yxKQ`Dq7wly3T_>62D zojN-|^5;m|H?z}k?Wv; z6)WSsI+}SPIo_qrbw?+JbaUN782;~3d!<_T`6ySrElFHUUr=?9FSC18)?9s44;wo5 zkO+BGUEd$c88wcRhGj(8cm434rw{>4n4t1udb7b*<#Q1Rl(?2oNjzfD0 zW{QhkeFYp(GWcHeF8)^cib%1_v(squ+9~obHdJ$i?=WW@y7y@Zi8QmZYp+ElhH6zpyrLq zO@-F3i_igAa;@6M9*dW=lH~1a`TC?UmYL$AODyfe2Jx~{xURvG@PsDKOZvo1qQHVk zB<}WWPIz|=hUQzpY0ZDrb~tz*c#!JZ+ikK48&gx8x9Wt85}A2eG7Zm~Mj}z6rewKs zFPb9s<*bX3-M^p|42yo6tNJp5WiiFM0bel*u#iZZJh6*0X-^OHi@Qd@kovd z2`r$VbFLoV*Jiqb@6L(73pmlkCt-bXs$nF8`Rnxqw&2q$e)vZsVXgZ}z^4}vN-Qk) zaCqNA?bv$05t$Zy$v5^z;2TAb4d?BnM3U-!Qgd7848s&vq-2PG98yb)!OpTJbon&g zz$<<~tyS@v2TP;%Y_gVfbrNmp6K5aha-M$002<^DI;cfw ztOB_E&GnDJaQI3gb#8;R<*CMq`m+T2Zg$<96(K`!(_cB8&wok|#U%Fn37ir7N^X~1 zhIdo0`r1otR4O1M!PvNnZ7+9k=Z|+kRoRdC-8My7h+d^LZs7W#OA72%kGzYl2PyQQ zUbWp(?I84#bsoEz+XKujNh0U1xHvD#X8-3Ig%v~u8@dJA;#d0avwxo)Heau@Bf#a_ zYZ*TQPdG<75THqT_5E*75WiLYoUX)t{Z?f~FVzJ0TTp~Q`d>}Y57#!|?7%dxwH?oA z)zqK^+)DwpKn44dOi(VVQvo8HUOx%3ldE7?a^oNwA{ie*lk(pEQ zYTH6okkW0!81ZV*GtLlr+qSN!vTu(WMP=*mzx>#(?uZZG@}PoC3JY`Non= zhqlL7pgDeGR2R{VP_kDLT9;<`QX}KD%S`<--ni?ElGj=lKsvVPyXXB_`rt{qhxFxn zw;lHCN7Xp~f1&H)PA9Dc^RTu$wl)c*8`qMyMADV+c|LbMy z+ipP(S>6dFlyd)$tFMMPJzHoqo^yc61bwcQvC@Ytut5{=4-U90(_6Nx0y!yfkv=FN zd+Rr%iT&FZP6=Pec+Szqeo0E7n=Qj^WuXJ9^=sH!LwB}D7icEBdyp;&Yw0;kmp4OB zeePjQMQ8VH4wX$2*DMD76~Q?|a{C6yd+VdH0@3@aC+7*Z4diOqcy zbxbHMwAyw*RtfehyU46PCK>#uVAR(OXia?~JR$(ZfB<*N)eHD0pmX3YN|oPti)gqQ zaBoVUJ8L>C7;jj6iP*9C_YUH-qNm2;UkGpkQ4jA|ZS|pN*O%kXyCqYrcLCuW|L-u$ z7{>?gi=2F2_>c?JG_iq6jrcOoJQLgxK=7+#;=BN!QQIu?ViXOW!|A!x8XE;&-bc@% zn0CBoqRFTl538`vk3{b-25nGuRFat`(J@}QI zPD<_j3$dz`ZbR99Xs3;)o2La1ZK4dC_L{CLnW)n}E~M(fLgQEBPp{IFGH27{So8gK zawfToiPy97Wb6nHn)gY|z;o3`p(GVoqpyW%OB6)vyrc`=El2wN}YUoX}uGS_&2 zBQVHiD#a_Q&TIA(`XE(B0P(C@^Y&_mQ51n#`oSv+QzAV&lqZ=ReBmH+u>!8DbdspW zrYCn^`SNw-fceV4wRz^g3p5ZOGZ)V{>oQ^!Wm02?36tsJkoci*`~7bv5FAsK*n7=C zZ+OI>aY=@xo&hW0%*cBgv}SwYH-_PG|9ZuzO~81vR6l=X2NUwFTt^G+|C7`YSvb&h zn>3pw8nB5f?$)@4+leld4q9e$p=5~$*Mv1JlJ18mGTnMEbwrj5O#b|h13OG9de2R8 z$+UTQez1IWZ>N}7G9EDBKQiT}^AM`WP9fLmIg-Q`UK#f|176dYh7PTdKribigVf~u z-9+k#3Vol7=d<`j6;Y{KM4*Qm{q&8waBtqMMB9=Zoezl}IUsQ7#F zGKfMQpY2ulvQ3`0`~t6Nb2dG@k&NyK3_srG@NW4jXphXvtN&OjI0PR4sV7C_;-!!$ zeR<|*Uhw+hRFu4dk(}a+S`)|oH9q+@V;N_%Meyoz5OG6`FcR!IpOHbMB8f>^bGf1vQs)2X&)IW8^)CdcUFx1 z&sXPAJlZ%g@8aAW;NL$rBFOlfx6RlUQ{0EcEwH$+c0$q}8aUDty9C$36kdgP17`AF z?$M6mq5(ke8)1uzTTjOi4<_9nu7aJE9D`NHDbm|*)^+^7LR0qo4bH0-ytBfE5T=93 zt%8wd4x z5mF)VOtCd^L9Wba4%~%7&}An2;UGiL%L|z+ISO>gVJf$tJ1$m%r~mIbPEd|$|9l*i z;Vl~chUlf9zAfs|$1ree&DT%7DDP2gA>@!VUct4+K00;sCd9L{S`vzT1fP}>$Fm8n zcO$GnPf75_JSR<}S!Z@KUas<1?b;n>;+M{F_*(FeuNNJHeP~RdqlC0|`EGv_~C-<+|2Yx41)Njmi5u_j-EU zomsC41&a<6MnWvX@vwpidHhnHBjTk#bGwsG)o;6vmIRZk(Z9VYSojSnyN4te+HOO= zf-LxsP{Mq)ahDi|0;8Fh#;o-uoH{t3|1q5UWL5N=OdkznJ5D!+|2|(6{Dw;Op^Os3qGndF(pR|p$Ja|TGopi^ z*&|%UMuzOYeXI(&!7gJ+wS*~BL|+9sQ{Neq_il4wWc;|Tf6&^O{odtqh{WA!k;qH1 zD&0B3@v3gu)-oaVYrA`oJD8C=<4=}3C8P3mU!0RfPQU|ow9MxeS0m-pe#DV(c~}!| z007hEN24B!tvSVZi4bPdNsS=zJ(~FDxBW^*SnA#lz$16zbvG6LlF?Q^io8Bese2hY z_+c2HdFO%oXN8gPAG90DY>L|nwx+%=_Uwf)8crw%lkPq@?xNfavkN`$5FuX`-d-1L zS~v+FM=>BNNxi#X7v0;MZL87sRQKy)bFco1m)<{`qYAlmwX6zcv7>sRZ%)C#(v%LL&=8yM! z|9Ss;pTF+wnd^D3`F`)`zVFX>{n((&-$-xY=?WXHk(F4?3t%N5BKiAEo=`pZWg)!p z?$#Db7ZGC-YXhmZHrKil5F|JyMdY-Cf!yAX-pu)C3(k+GVq@D3v$c2tGRvZh#2rh1 z?Sy$6@l^l)#I@>Ah7W_LbExvM3*OUYc}L5(f{5lYTJO2Ht|E@V6gT<9wbsGw443u3 zK)H_^fz`xO<4#-Xpk0n{uL3F7?8%I5;_DB#!CC^UR%A75F}m&09F!~9Z9_L}v zI)~bSs@{Ujp!1i3#<)XA#Bvj!nMV)Yg-hs#Qu;^R-^KNu&1t5 zF}|i^gMF*Aq(Xb}`{N6DDy8GpFiwS`xt9If{SG6;raGk!mzDieBD_Ho*rL}*5k4m$ z{&(q9tfaZ6Wbz|&D)%sxSBZBw13*Yj034l@oz^m|Vv|a(%IWldkP6a+5l%C+kCr

      c7 zMGQ{%$s(Vo7%ynC4m{x*B)OPgoIt-JpBvt_xeVBCF6D~c#QP>z$d)%2%3kL*w_=M#4E9+k%fO^X&f$S#YKz&KDH2ZO*DRgm5y|@^b~c|ht6xR={857eHSIh&`umbH&T5mwoTJ1NBD_uX;@- zl7ak7QR&s@_QL&w=h_wc@2T8Tty@sR#;}UZSE#fsYkz0u7+7p(Ogo+!M_IXv9+Sr<2o@dEo;@vQf2 zI+3JtJEVV^jku0kc>#k|OSBdreM~jXZF3*9QF57HD2RYMz6dL&F8CYk`?@N;U zWxg{>cCci+?TEwE$|A5=VipXWZMZ`>;pmmOs4LW1yeFaA?SRUBa*b%@g|5jDa+k6C z4{>3Upy$fPKijRnIKXR85qk@yH?B%NY6e_7kv?tEP zn$cZGWisc7C%w!Me{n?Z0Z2Am!5Z8(rFNPk&q`pHV7C8r%_QT`Mnivc1UG!qc=@b;;R`~w&`UfqK?KaW?Q;?G0!#M3I0e4!q*VMeG;xUoB#t^i3ThX>m}9Lsf3a^≶|7{Gy4ZI#Rm# zAFAtrnfeHI{VC@9x}*(M7z2BnI_o$53akjJ9&$QZNO5I@7EoLXzQ1g;jfE@PZbA!SZaPOM}evY7uCWPpzuurHmf%` zGEB!xCaVK-s5!X)ZzfXS_RzxTJKTB-IL{yaTK{SV?K-qjQKF8PztZ7oLbud(@`N7i zzm`Al{?X&*Uaa84;;G#M8V@>6yev~x!GOaw^T@H#@C@KJMV%%& z=^9QKczGTHoeqZZX!WbFhcC%EiW|ZozA)X6-VO+W9Wp?(bABa!Pn5949&{l>qpD*j9#}8m~xdE2OB32Rme|fdsTNB}#v_46p4Vc!QP+T16)pA`u%HJ-PS{1y^*4{n zV1Et|nxW`&xG{`uJ<>^;POoR?dSJh3GtMjs8xZ&=9U^GNG41KxmOZ5rr<$v1(FyE{ z+TO5D4dbpB*_Nw#m7|y%=dsD1|qtu>K9&hdQ3(#e`>vu!KXV?PQY(= z)-VWQTE6odv6JD^3Qy~G^K~o44ZeA5?2xr^)r@Bf5eG-Y@eRQH2VPqpe9>dsb8_Cp0(xY@=R49$**9W2p$tR*yo z&3A$1R}Ne+6rH=?y$fgft~rlAJ6ffq`rLQJGC*0X=}(;yCM{G9iiS4DXAg_Li*2$aJ_MYeFC2>X=!vb0+9s6?A8GMQtD4JDx`l;Ljl$f5?|P zVx!p((6}bh&~442al^SR&owKviD%RDKk%Nn>YS&67q%NJn1`i3iOsUYt86!AR)7MH zkq8cPvhrjH4Y@LSg#4UdYiiM%tgdSKuH{xj=H-VbKL*>lzawn6T*KHTCQ40|vOeS` zFkX@__%c+oh@6hIJkCmIY~#9;!VLtmxK!PigEPS;jFJdH#f+eH7&G#fCvby)DeTSy z+Jx=x6iPv!?%K*GrsDkl7>eqU!KZunTQ)B67;EoW1uI&bQZ1^5b@}m4bmq-SOwLr) zTtPtj&`Ona0o+!^pW#@AAU;v@4fNhGqPx}+3+8>e71{`ygHqcY(c|y2iXB^rOb1TB z$(d8^S~8IdWk3My-duWk)uZw^G5x*2tJFVcxp?^f{+9ocPa#}N31wMMYmp#;On!o9c90R!^5716CU%Eds(c_-E zBZ=%yg)gxn+|Q-ZPUu5&yH_ud*N?V>?$3mvISi0{i?}gAbGpfttMgz&*Og0p<43&@aTyH&^c;v zU`=7zY;jH!%_kSD<7I~0)O3EL61kQkQxjmV&3d{;$7Hn&%nW(jr~x{;93C;tghlSI z92g-sFWi$MMAqmT{Xu&%w{hmyZM?fP5ZB_X?MgWCLv)$4t!+x@=Dh0)3LatjK2r+1Qrl6{i zG_ogjOp$@D_pJCamR%=E0dYowonU1Bhkn3@`cOELuG-`tgTP!XPuU3)t$vT&M&Pu- zs&_RSnRMLnbr^i7>8Qc$THf{Gdvoc}aq>2?7{e~r>NaI0ajBSPXCBS;zh^l;**c)Z z*ClzHhwSmWK@WZe+Wh4w!Co%jd`!C%fuUE~;c@cD9Bz*)^MoM4kZ)HP_VoM%K|g2h zdMmMxUN=T3sHb$_M4vsC5LJRrlPGT`wcASBNU@91Qr|&L? z%ApwU!))UgAL>yUcB!9}V#MR9+4uS6FsY?XC+2o4audC?em7+ZmjL!S{{b*e4rb)NNAmRr{A=qvq{6+m(u0?(Ud zbHh?MV8{lRY&GDXCIg*!$njG09iXT&_NTf$hN@*UwY3}9h9HOR zEdBE~j>2~?us=bwl7~3widy}v1ZSZ^JH7_c%A|hL?z#MiCwHKZ$^-cJa%j!z4a?t3 zoRbRxNH~}#A@n?qZhJ#&H_A4B~x7;I0w3(0}l@0@FGtL$Z5&}~n z*MAEz&hbu9eeb&ZYwZ0A0KYQ3^Mf{OUIn-a(g$X+)-=&czc{`wE#67dF$f8S)6}=oR?*-=m;GJgi3WhnG~maGW!7& zzk;doZoG9K$V~drE>qu5VjyAQYv0#_uhi6p5y%U6sw&vh31e>RdHv%)MiB^}z5uQ; z@1bryvr|LFw76d>e?LnG>*k_o*4j9?fVL(@QFFB>o!z*}4NTx%8C>sSB5 zXpgYCyo~rM{%Q8wyElSYr&b-jO39?nXx@Szaelv0O?|y@)v>(j#t#c$`E<Nb^p2d-^*IFczDj<`<%1S?tO+3MR{p#Oj1lFBqZ!NG7=w`Iz_d}B8w7s3R_5G#vuQc^&O$N3{_-v;=uht z$gHV253!rY9`CfvtUWJtB`9go! zeRs~7BR{k>b$dw$K}rp+#bA__M-mwK_YFo3iBdSj=*yPsTv9K|J@XdjbF0Iap^Y4y z2S0)qNqI)Lmq9+0zlXGlq^|uLduwd(Euxf`uG)9hwd;xg48|p#rAU1~=;n!7*XYRz zE)GWjVXzUh7jmK(c1zIvp5b`Lp}?J8Fn>ntqbea&-tUX@djG6NAgByo2wGMDGOShgCg zSucIKxAQGx2VDDCsRqu;J~T!k`(mZN$nX@7D|5p<7813sj{2MOtZbRZHs@EafaNTebEGm3nWcK@H}d(yXnqDta# z-T=Rq4D5}JtQ|~k9Aj_(;%0rx`jSQnlZuK;z~0c9 z?}LQY|ImTI1Zhki9c}s8*j!v(SY0?+ZR}0hUh(qsvb|(yV`papAXpsStR3}SS*#ss z|4ZZ_Iub?>2KHvQj%GI2RJU~X^lh9R1!-t*C;IdA-*p3@2?G71vI}7-qxmTDYq|aDAV%Qc!yF9vK#q^E1=~Un`Zr(rY3%iD z4B0Z~d_)fMlZZ7$6169!^Smyu3|ugBb}@Ed8C1H8laIa3P1*HnxK?$_loNf4j6#KU z4?_eA`OzmN6sjl=6voYWa=Fj{^>zt^Y6bl0UvLH}j}CYtAC*b|i}zm=Ds32$LM@ZN zIF@oQ3&`W5Lp}Ba@&8f2g`m1XVN}U81jZ`dPqsBqwkHf&)1oP-81LO24GC+dONDDo z^+w%U4)B-}@#*0vdm5W|&2ZJUapLy?Pl@)M!#+0Ed{vfNcW9xKnyh^wu=2lHRa7PJ zC+1p}=CNGXbFQHVBhC_@D--vlUZnnoxdqRN|KwP6>{>ugvtrcS6~Lfs+3tUbkjNp1 zOi&jG0~gIoj2f!F-<=^Oln-x^@KWE_u3`UW9g;8MBa!xOBt{LfyIYP07?qnLyYL_O z{%-*?_wGq}Vx~pY-(B|qPxAjIFMab~bjL7LBl!x@zx29vblF(+JPNP0nuRzoxNQp1 zx0wA$1Vvc!hL50y)eFulLL1-H9ab)43F6Kbgf7~>5f_^Ub;OM~2oEw|dqc>$#p%wp z^Kt-5r*i81bgAhiUI+azvKi42>aPy_PRErLVxLlnNC#q*8KYCUrm5s9_gtHKHGjJI zhMD@HV8I>A!hbgB088=Mwey*Q)Sj9p1E;=5?7Pe9fW5Bu@ie^72NP~;r1aq_D~*HtXDo44XEoq5fyg$QJ-!$5`XUw zg-~JaUz4Q$u@Dba6Bmjy99=(mbI!-mxND2xbjfW#@<0p9i{;m zmL+HzDgUx6dY1sXDyNa_Pzly&-^f_lX!W*#d(Y>lj@)I=&bga9=-cCGlQ<7aIlC8JSFzzs z{M9Yxh!CebLb0ac>_fQAWAUe1_~!TT@=sPF zZq6a{LS-Q=>c@$N>Xi$s#nVd>PBWE7*Qc`#WIRS-7Tv0a+C7aS%x&v9;+Q5KqjGOe z1CNS)tAVOqyo~vepfHg65lK<`a(Z0Nxs!!jkY)(jDq4~tysy*G4Ld_Cp3JAKY zeaHKYdFNu26Y=%WtAoi+kWM)rJ$}yl-cw6Z7y0GEW~JnIgZY4zTJd`jTQ%BA{#v^c11V5Mbc#gFdOAs)UR z8@iUugSzVPFq-IR?J@4)sm$|gKFiK2UfYSdk9cr0~LVHdQrY$obJA|B&;nRW-gvx%c(ulC-XU6ITm6`9&3uErBtx*Yz_M0F7r+uG2bj^LmiFm+pu}QOEfU=>y_@Vnm$Ao8 zG$`M~MToD9jsK%vAemitRBLLpv&FhFwN(CL9@!ztkQhK9-^g@C5xF?u5GF8JYVYsD<~f!u3~tW5gx`@9PW%SmxWE^EZoIlAoF4V@@7B<|mJoW;GcB{F`hdsI6;*2w zR=X0l4aOnv1R31esi7o;`qK>_E(AxmVDo(VUQ?3{DaiSk?1NL=edGmrN51^>>lp~B zR?O}zg)BHEBZMBb0>qp>)ce=u+ z)p9ALx{GLT=MykXa)(#{WB@DUKvm)LXQ)O)Rb{9LPdSh9RR<}W>fTr+lV^T`(JI;m zY*}Q7(nHYz>Z9sV2hU5kbp8ki)p~k;Jv6!y_G}7tUhOD^CB;VgC_Agdsl586k_>B`q2-T{^Ppv=V&vb}qHk*1MWm@fAQKbqs zRFx0;4f?hQ(Gu6mw^VLc+dG+!S(XPhx>CfWZ925%@}ndt===Dn@H3bB>7zQf!^uM`jz_JAif@{H+q5v z0jF5C*9e~h#C=oyx#RmJhe2LzsxsNrt2$Vfbt>OO z{P9#D1UOVpH6`LH@`z2uY4}n<1=uYDTF*->3j-JMe|W_J%Oo_K@$?7QHs4RNoN zxeCijq;4`9D!8)(SKFdfPFHWt&p{52Mo zB*ubYg@q2&mZb{z0$M@EkiUFqOUo>}lTaD&7J{v1Hfb6coW$NOiJunb|!@wm)P zrECdgjT1FD*i5CvYbtvJz zo#$TwJyh|5`J-}kJ!)T8M#Fkn_3tV z>#NX&dw2vQt~ttCphv9?(N6jmAb#JXo=@a3sA*$}C0(*-OUR7iiDHAuWPuUl$ty4S zWe15_Zx90$8O18mF%yl`pkZ<^W8GG9M#0n*0oJnzlvmAa;MQ)&d@7Phl!oiEC#P{* z)(3TSA0Vn4Khz4IdrqjWeJsTe;RGw0Hvhr}G6Xxn2^)(SYRM&$joZY|@$a8z9*MGw z9!|$k#VoPeD3Wt=$N_Z+T8*BBf@;+5&!5o)-daMf#O(0cCJB)17masIeLjk44EN00wp9ht6)D}4A<@fR*v=ex77kOQg;bXH30hWcftKt_A?sD2-Qg19glJ8aS!v!8) zWFeinvz1L+lALHUv#xJa}x zJ~bqxy;R--;e9-uFPgRJ)l?fCE08o< zU9P5e)SM|}^UPc0DzUs-wtx*2<$7{Vtrm7Bq+?iFU1ZL3pCoh7zZz{sEHX`yQ+zli zh^MqxRQhA{HHTGvvibDQb!qOK_Ok}#JdZxA_-so|M0mkxIBlfB3iF&^)7iray#a=I z>v?@~P2KiJ@#G=vc{F%r#HV>b+J9Oj>wz|-%?n&H^}cF+m~%bL_uP{wu$fsoP*J$Z z@DTPSk4I9AqO7~>@rrshG3&P?dO*Jtilq{wdGYBP{)eyGpQ}8#CfyH}`NaP%?DeKU@A)Co%UgKjzulHZvI1iFS zuP1vL@b=Jx(S&~E|K3chXwdPlLu*jQlCU8+S)Qbl1Lr=iwP_RalrPmnZaNaaS;TRk z6))e(kY%Wi@VlzTozC7$+{Cdm2MZ8Nvp$z2v#RX4I&rt`qtqpa^Z1GWEK5yxlo(rb zdsMky6y}4r%NrVhV~mbU9<1J~=LsrwC_hZSn&$qlB5m_- zJhwY*Y2NTSEG{@c^vj%SMd8xCSp}<}PX|!|yfQhrd57}{4X8l-0qX^ug|*v7CH$r1 zmf|KOwH;;EhR0b@NlBbfh2NZReP~F#3umf*st>a)_=n6+rS<%bWIQU5Kn=Hckouw0 z(}@YK{z?{?N2uUc)2E^uKfU0mMFdpP%GAU10!|}x1<91M?>;9JYP@PAHrWX6J<=L*lBdxWZS%Hno||h{H;?AW^;1j5pT5 zi^;0r58fEsMKY|kk$+o}{Ji*-NI0$^cBnqu7sLHyQ$!cA!H&pfHH)K7a{C3C%$vSq zu~cWG;!iW)$%1UR( zAC*4t6raM6qrYT7aX;SV=M;k7o5y_TgSqTv1+tgo;E_>9937UPQBOH4nNZX#=bAXP`~u-J8=G(2HVl{fd6)3nO< z_3~$eBp>jgLiuTMd`RpMHa)<7E-a2HZ!}>>M-eNfZ=T*JV1&&Kn85lM20Gn z#u}67sMnHMWuw_l=VZ1Ahij}#(LhAaL${REpPsT{W30m~RHytx%o?a}CP)Mo)n zjP`RN$9!;ZDHpSE2h9X zTps--d7&%&xOY+)v6HDG*7p0lHj8DPs^5CPmUlUO+We~i1V8HVH$;;4>wKA!Y-&Ht zx(91Z4xcH`wW%m5JeGbMZp##brnl|sezptCk{f>~nDC2zrYn-uRH+;Yzn(8a374=@8pj)@Qqx!?T6;@iWqn>Y+LKp|lTrY6 z_fH@6=d(g+j(jq73gm=_FU;gPq}y|@XW`->kZQA3XBeVstcId%Y@VnINQ7z$$g@|U zB#wmFxbCaQt-!D4>dw8ZsBa6anD@MppobrIPYd(Z$~gQq^bu}yvpFhzB=tX@=V-BP zmm0#p(StaUbiAG4x2cpJDsWQTsNiZq8dyh+`MdK*RMpX#QU~UFXNl4v@P@>l-c$f<(#|v zsgK4HX4kv*UKU>Vjl%qAl!;vze;(3kau7=zD1YE+rdct_gs=gXQI5n`UF>tq*HxIX<)Kg0Kn zTr-(FH@vF}CO@UM&Ea&qU;|o*{)o-JG%br(KXDBpIamijHf)Ha*Cy7uNa|yeDqk=) zZ&8=9R&RecHb67O+f&O{jXR(L`XEiM$#)u?`U&N<#%?uA!-DRG1*!6jYC>>%JHc8UjT%-6 z|K2H{!llGY-s&=X8$)yo=KPAKmK4&&X$-9G%!eHs5*&Mh<*1u&7TDc-+2h8iM39}P8Jgy;`J zfcF0s|Yk7JE-)M>#q5~tGA)IV>+-!_yHoYsX z5>lusRn--K)62ID4jM?}>SO)w13uw}KlRw0zmbgC?8T%JQq%n6e&_ICBl`#(V$7PP zHoyw27fXR;y6b^^WL6i#r-WKu$T5+_xF6`|Bxq#5jlI$%F02t zO+(+SUQYEi?u%oKZ0`vmlJI1$z<2um_aLYIK4rMgyE_5Z!=~V;v-t+nWP;9NRx=en zLXX+Uadt|5*1oF(^vTsXv^4sNS$0}P{jJXT6?U@jO=>mNI)%3PordTA&RNP+$+ukZ zjs5-n3;UfjwNDZOvf)D2LQVRVj_;+b1y-}kJ3v3D4CwHTM+jWL9z#+M=v86Vs!N`o zRwH1T3I0l32x=;~+HZ@jF|aQ;TP#Y}+Vi?JgJj8BkntUhlfB-e_E-jI%dKy`l?e@2 zqx}YX1V#?M%Q3^0=JV!aufZK&=hg>;Rz@G0tw%GyS~Z+NzJ4Jmtm%8`X&Y5urIuf- zv5P~-J%=NZHq7n=72YjjV%xKU@0LfJd5vcDKpH%%WhkZ_qg8e}L#2oMopr}?|LPP~ z8%o6MD@Bhz|1;RoCg|})`G)>MzSS@quI`5bNGr*mFB zP>;b>Sl~UL83o~i%E4rQqL4%wSuC>8Pr1A`pxIa;xkUD@=cQp}e&LwGrlvn@A%_Nb z@0}iZ_dVnU?lMt-G)c6gP9@k0baFO2Q?}49UFv{NR%6cf*cGC_V)Bq;`%}MXjxDEM z?^e{Szv?LxKs8*V*Y+8;9!VKn+p0{mxoeh6gLcJ4h;tm1uey}p!1%E>*8w-}!57Cd zP2?LE>Ixq04#*m~2%ypR*IZZi=MpL|5|<~w!%-aMCnN{5o&90z%U%r)Tnjekcf=Gh zJ=_G{HNlggAS*3es}04T2dkEM<`AJuePF2E?(ub3$E-1lopF@7tW%oc{3}Mmzf6(B zEs$&m{9fUT&n1gVBFUlp>e2C+wTe4i?xl-*g$J6$ylH0g*ET|L?de`L9I+JCSH7amjpt-<8R;8b@co_G*=gb4%r@+d|0gUmk$! zV;mkH$WtU=+pJZT_+l4U_uVD`>nt{<4<>;=`5oS-KFH8%C#h}?(lKkrUY<91d}`tO z{UP*@Ke&*mg3;(Ux82vQgz1=7?D_+QVs*#l^~28$)$f?2HGbcU(DVIIsx9#{dow>B z#+~=Axe}c5Ej*PacJ3Tlw>YvH(E&Ypw0m3k14i8hp`*3Gjpv)RDcn2GiU(zsaU~tw zG<@SNK^CQR{)JwuX1_8)U>n~ZvDL+wNepJQ4v0^!^V2r&#at-ds052YwD8oZc#ZRy znBvHT47qkJLst=K&I^M%&gfEP@5{?)^U`U$Il@F1o(03f=y#y0Z=w5Qv(g-(7!o{b zT;H*TQuh0u6s8i7gh~!#NtVbcabMqc!phH)&F~KFGry9(^Jfj9Y0LmG_t7xjxO$;% zhi5H*3}?e4?z$sIjLg0j!RKN?UxMaQAL+0roxQ2JHJ8=t`~wTm;0O74IWI}YLZd6g zHB^UdpZP*zxFSY^&KDy?vOTD9KRqj= zO5*m60sVUC@ATJ;`+Cwu5#v#!@v%QT(BXH)*?VhBZs935C4+K@CqUs^rQJh~^ufL% zO2I)-nheF=HAqnv(&*mTj9GrWN^KB+M)xnNAn>gSIcLibv)5+hJMham>>y25PBKLM zOFZg6$|%E%wA_;zWu|%)CX1^V*YZXSuDGA?+=USdcoxNt2U-9x%UV3pIQzlV$SL;U zQ1-uca%50OX;!X3K@96gKa(K?B8+xdgo%ELXiC&0F*t}=RG_txQifyxHw^u+GBVQK z9*A-+fQkBlV8-5JE@h6!cJf5@s@p1hX|WpzvQP+)?HND#y9Xp+%-gdPnkeF2Cg#oE zOZ}e9@bL3pE@MjHj+3jpF0v=s<6G2cx8VQP!ZaFuz+)-!)HtfI0#@2w{Iq7K^Vn;= zLjGTYRV{|H3|=a$g(qN9OZ^55EjPpi{B-pxL{p1ol{U^`C1$ z!|_se&F>QnS&S9gSmPgKVz>wS-uwXuE-uNWwSGBQ)>=(1sUmT17LZM~;II8~vrkESkzWV>kA_&&j-J#GQwAJ+wBvhTimo-I(eN z_LlF}A@VD{nkiAuin)~ZGx9I^OwD&sA&K@; zWN$vwBrC;-H6oRxgOE0@^` zh5QWty|+TZTJpBVBjUZ1^eoIEwM0^Id-^*bK>_5NK>O0rk@0OI2KpUEwht5$aIZHX zOj7Y9inY1Q>BFgxv?kYsOyXDh_0=xy{@kOzuWnE%Zx^2k&&FX-v?6xBsQKg}S3nnX z$KEVGPo2H_<^6iU-gpWmhmDfP8{9DrL@^=(XCZ70mW4z7 zQ*pS$p>IKJeKX>m)$y~+ zUzxp!VaX8XLZh3#Td?4(C0EX-I$b>^H`5v*`ODjI337ORqEHK-k6FiRdg9Sz(OABJ z@$D78^a{pZPeahn??SpbW1l}_$&~=t(bs9!@jjfNb+JQ6{D7v|b`dUOS)_7L$X>vz zL8Vj^9Z|w}up{uhwea@l%`uyP&G4}|q+hME?K+9>mu3}luYJpp;ii?VpX~ZI!m%PJ z=J|Koz5>)y@w=RjRt;JM{Xs&Gb)3yA#I7txKhQ?;jj&?X`!OH?J`QjRKp;S(n-GYdLCiiYTj<_E8}eVHM+JaU`YktZke+v$$ZnNnQTuC^ zXTHio|HHX%8X!MZH0TbOL>&A$4N!|6|4DVnX0*$H+?{jd_VFIEffeXB%oz748If^Y z1{-qPEYx?q7Zp*hgjQ;EU!+?ECcO6qSE^%`{$sL58VS_v4MTLCPAZ*`3U5l#(QwK( zQ`nLo8aB)E%u2?eviWUZkmyz{n4FeMh7yCX$#|?QX04mbWs1a)ObEPM)>kW5`f|>aU1D zlOQ4EA$L9X`bMxn^U8V7b=Er3ssPBuc3mT{Z$63#5_)$#bcX$a6liedE9ZWogOx0| z!9Lx0pW+x2GBHURnMD(HZNk1;?Cqj*Sk6hN=r?N3v zw_DYub3G}ebGkh>A{9x#%6v@u@QFG4>6EbjJEn5u0UYL|!UCgE;?t$H7H-e;XU=1v zpoO~gwUR4Fa4F3>vyGwDRM^w9GVh65t}2{W(ix9buaD#|Gqg-lnan?qNQ4oC6G$${ zE%)l-6k2u8g@OlDw5oN^Fqygkrz~()kB1DrQeieW6Nqy?jE3Xht?Rn`o904-y?-9E>XZ+VA7^W)OCCI5aig`|jF zf7WGe z*i|<8bo150Q#{J^^~cfvDAn3)-#fXVh8@GfmL2nf#)u)knwY;<(|3WqHH@dee{+19 zV3CRD2?i|`&-!OT$tYeOZG}>}$qMalrp#ZB_O+x3>Q<1riw0fPWG!$dTlUBBp~I|S z?tkQw$_gm?n29H=ivj?NXw?x3>E`@QP=xJ*6sih1po#IT{HTLxqaH#9GXAoL53EnK9TChXFnfMYy<50 zBRZBpV>z#Z74Z(oc?Tvu&0$ozlYYqDzr~`8uf{OWdBG++`X^QN{UQlep2T6iChab~ z%367te*=psM!*m$f0fg31n99b9JOTA>C-d);ZRih?PQ~l2|u%7hk?8wwDSJ3GXSTf z-rnMl%GocFLpTis6X-bT`Tt^u{s+?>Ap2gab4?~Zm*LMAK8fT3JX4b7?zxaM&iTQ| z*TWk_erL1@Po#ZQJ(NpFGs=J~tM(4VdTu$&JwmwN^^ofYvTW6#Au99-z$xU6Ts`Ch z$?N6yJT>ROv(0FEVE>L8xdVzE<9Dz2A0`6)51MH@aYL`0VHcOf;UZP}o9Mp)ikb@d z#+8ky?FE#v1q1*v|;7>dDf$bZhF><6%zM6K-4wXekG9( zk={~>32-bffm1*G`QKG?0mvF-ga;}az_}mLQvp4pe`t#Y05#a5mMrG2xn5530U+_T zFF93y@<&re(&_=^b^7nV$N%UIAK(2P~$07Rm0|H?Z;RSwc9GGCgO#A8&rEW7|^G~fCS2wE%mS>dirX>;pH3IthWk7bN z^p^7dIkXAB{HKuqUxv%@0LoGu@+?1b0#DPpJu?rC3p3cVcBSVDBR47|axc99&Ogn1I#!GT-tjI#4Eq6i^+TBU%_@ z5^Y*Od-uzC1r!=H002|^W9QOO?y5!FEc)BJwTR@cDLROoHxMU)pMu{w+ZAIaeeYsS zUB!NTLLFs-^1*2+`m4c)xH#USo3o%@?W68z|Jl2huz0xq@-G6m5|bj2)?{qei>KDg zc{siTwEI^EL(}m}QuMF_Bd>VZ+j17fZ^J&ONeT%2Q^ z`3VPYbeUFAk$itp;bD*GYbi{9)Z4Y;V{~Kstkz%etapKA#H#rR*f+Qg4SW&CdePs{ zRNb^K?Q4GK@}V$N-P2fB(3)77Z{xGJ)c1$aYwnhhHz=M-V4qGl6kEVqzxv+bpAJD1 zzNC7BU&P$|&t~%;V)!wZTTZj*gPOdn#zWoPWttVZc-QyX(2K*gv<6^UUov)plu5_4 z6;;(N1|0MwE?Om#VN)b2B>+{ zMrhlB|RC-=-heR$g>Vp?rMQeJ`BoKA-AfO@d-$=p5mqYp~<+! zAw-p*Ej_xQ9*RJC!> zugZD5D?T=CRYOFCqnQ4%lL0GFo`_135!gzuE90qX$Qk&C zd=v4O;PBU{@5NJ-$R_B%mULULc)>1&c$-!;Jo6QwAd;B>;339v+Nx!3UgH)K*{Wz^D*A6;vMoW zMz;&#mVUA2vc-AS&5GQCUO`-(jJDOan!j7PFE}KWsmznGvi`Z6eoe(XU9G7aMm$m7 zg0(0@6?o5Rx|Yu`){n2@@^>Pn(94ARPy-^Qob8FSf_L!lm4*;RBeC|@4dIIRyd|<2 z{`SpV^KGV{X0Lh=9#Dc0j{TI1R+e!d;8z>&#b2`H5#QA0wO}vRio7r>zP{b2_$&~ z=dt`TIdJH!hGkOT!cK`LczaKo9&f*h&0Di4dgVg2$ViU&Nmc7JC}M3dHX+dd_cr!v zafm{PQEdEH)845^L5u9-?Aj6{|J7TI4b~7Sg*HCPX(HTJZQ_QdDrKy?w8*z_%%zfe z8k$=FquSufark55l+Vy~{YSxF_&`15u@%J$b1&^$>`#g>-VsLap33bHV1^>9VpE4g z?yPQbwwF+Rg&M~@A(}wamI}NJnN{czB8Zjl0tE3mZ$B0-mp{geUOrK}Erh{vQTFS4`%8 zY&UldiX%aM$6FBl_+}RY_M!NbU6||OXLS$7ZZ6=%w^_DFeAtiWxZLMV%9b;#6MCcZUMHC#uuw>@>oy#z zy4X`_FGcned&^ykf6}W_ZE%5X)cM}Jj3^Li0}hYV+%5mNjdQOwk0W@7&K&ztUl*4n zz?0-iuL?)z;f)!ez{SDnlP~v(-|rka*i>$S(8j1J$_9b{;zAPLiqA8AP$?SQFMdI1 z>#y+GA&Z%lfie4=_a|;}7TnewN&#PM(KTd4hPMlZ4IRwgDn0=vX_JDy3sm`y zb54ZyA1Og@zOyMG9KL!fxqQ{KsjL4<&^QydL=M^Ehmt1w#nK|A&sJ;@!uIB=MYzOlcJ`l@HyGAk!>K|EeQZETSNXs zmq_v7_Z+8}&G2YmG?a7;7@^do&T7pe99BF%fXaG|YDz)k??*Oeq}$#E#p}EYBJXwa ztwxxS-`rPXre#Y|-lZT*sPQm;qk5+QMs}XrTY;6!zSLj0q-o@s%w;VeO;!h8oDBcB zZW7!uJdiEY@u}n9w5{QN7a5z!p54Z){TlG6B&DtJOY`3W^{xUl-rocr<_XNeS#3@- zo3%~EYP$7xCp#YIYgYspUYX?!--yTnLui?FC=_Rng1JDNf&M{2U)hWvNeY#f_F~(3 z)h6peouosqC3}KTUzZSL1mkl+C<;t^RZ2b5WJval%zJcda%(<>rox)b3Tgt$^Ilny zy5LQzMNBId=~49Q#gR;)You!6ZZ>9Z_T@`Px{@}1t*#R(k!7}LNqcdn&Yf3o&%bVN zvtuBRI-gjXj3tUK{$SAsfoxc=_tJl1U6_*V-|!!_3iZ=3IO$Mg9yr&mvIL!v0I^xR zgPzR`6vz9+`5J6nL@GS80qQ>8)9yzu&%XY;`8EG;l_l@|zRLXzeXqw|TEqtqq0P`p zVUx+52+=}$kegRa8=@oEdF;J3-3~II4b%Bz@RfIb>Y?Um>G}&g+Qu2G_iF8i z-%|Sp1k~1Z;a*b%AI=<5pP2R|0QQ74l)yLT$Fe=n>zL zI3J3~LERor_D?l@B}83)h>g){|DzN=-JQEK&H% zg9BZ0gs$eoqKtskJ85kn-As+y>)Bul?;)WZZ68E`GXGGXa;`q`#laBHzS~Oy3qwbR zMxh(K-*NgAd4{Uyoo8i|EHy;iO=M-2qugaU&+jRBc>R_o3v-&qx<4?~jSuSt%{*j1 zgeeb0^@WjS5y{i~KTmn;fUEMqbZCTZLShfq+0xes8t15&B%L`)2z6-7Iw$I@9K=>% z6xCd@^Lv_}1fQsstHm-r>=hpX4UG)arg(a7(odpF32Y{X)T|UN1-LD{kF=Dtw(i}y zHF;W@zby2lRLpl0&Z!bO>y;>?PR*jxVJoe_IBAM;?Fw6?VUsI*ZFTAB0WEYXkWc1w zM!0QmN0&Pc)iuYlY8kT$-W1G2d?JMpG6WH8O|Wn^P#J?i`2A8K$tMbxUAXj&;emp8jy1zP^uO#iV!jv2r3bWiKcrZ^p< ze(~>x`j(uwzYlX$zN)6FQys3H4)C?436*!(iz0qmAeSzV7Y3CdfUL*mU6=1_`6~ zLvA_^C+gJxM^WqV$F_cxh7vn^Jtcgoc(Vn;J2eVAN6YG!?80)jS$3J6B+2i;lwvCM zgkRZYdV&ePONs=8{a*mJg7~v@2ZJoj2$oHAHQln#E;w1ZYVp314~zjivv!F&HW2P4Cpzb0PFX%F zrQWhMmA!myqLyOO4RWcq*=AD%BY zLF<8f@e~TDhy2YDTX{s&VtQ|FRU2ooN0W-iPQ_;r zaRgsoCW1|k>UdZEc`WYldGWhyRcA8g@$Il9ZNUkM4jLQOx+1ujV2N_9JpZeXHU$T# za#=Wk{9!|M-d$aSj*eXwBfn+DgI%*=KAwYzZ35Yq4Gm5OzC;K?PSXb~q&XMQ7yRyx z-Ic0lROc2n*!z||esvR8l9lbk6Z>NXTI%N0_VT;1 zzPq6y^YQhdrpNiXaiz=b0?hh4I|5NMvEX%d&^`E^Nq?0hs>NS3?87w9`!B_h>QY?y z79kfuJ0d{Llkqc+s>6+CH4W7cw9uL{iub(F7a7mw%CDWFh6C38+}o8}4LSx|$8#7S zd*!%q`=6O_(C<9GfuTHmw3Bi9zK@CeNgWfFZydJP)T`{L7DGZ;;N(+9TR(nCsb|F6Yb$Z&I9*=jExHv`jXTc}K&=H$+rn_4c!n=#_ zMQtgCVPBq{&WS5%^;u}u98LQ0-H;$I)1du^9Y@R`%30JBxUFi}2Opm*7k!wXfz1sz zCcGSx3VUjHeVJ_KPwCyB`LJ&KSm>J5Vl1l?oBZ-8_S562hW)uQ)w(K|)9{~Z52fD# z4c7`h_Wr;XhhUL0rpW3J^qpO8WOG#EW@U9H!=4{E{W53ZJxj^42d0# zbYJh{?Ypy)kfufP=tAlUQ}^%Wr6AkzR1FLbn=+Gy!qwEQ)aAsl6|6Ugy--*LCYn7|D?(W%_aBM zHsIb5toi<_OJX%*6{qg-5K*-!1pf|sCYNA2U*~f4zT8`b7iuAcO8toT-n=c>zFZuk zTzTxy63C2!fALFPDxYG)kQOe+tFbUlgo?8y|D<6`d0M|j&kSzrcD)=Z$>#P{T~c$mkag>89e+Q4 zsGV$uPBE2^(fB4NTRjMee6CT@WpUs#<;bK?uQ~j5Ou?bhVN+gm#TYJ?*$gs`VSF<5 zjeFYr18{>2ac}twIp&i%E@4RYp?pmI=e#v#;xb#asH`*5fgp5g8f-eKx|^{6TJgyK zLssu$O;?cFNsO&br(;Yb>y_@?^yJhoM}=YZ3^X~R+HoGI?JJY*L?ryWx(3zU{%W;c zOHm(0&n0{5;h1CE_OJsb>x?~gvbdUFHePd8D*OvY_pqvfyXO%+;Gs&KK6Ml(GX5U^ zVXa*UTWag#RA<#nIXl_b!UepB^N zGC>H%rhcv{I6Bb2@&1(Wxl{O5shT~FYeco+{I{dRV=kwP`I2*vx`Q*g=mAuBvPdku zPd_o7etJBm5%C6U6~M7=Ix}%d>hE~tfhcXQHiYktPMl(+B`^}+n8pd?3QvC;6kd2h z?McxQUyG~0?|knbDCylYGtd9k-dV;)wf$>c29#1H1f>xWkeVYM5^^MjLrFK%2t!IY zh=2$PNSBI$bkEQoN(z!Qz|cr{4sjQr`}cU>-_QNr7nheX?7e2i-fRD!=lSk&#%V-> z!LnCLTS8w*rm#|z9?l#IPrLHCgvx6PtJj^TwJ#u3kR@-Wpz*t1-&wK}-s#E4I!3(e z*QV8~7azU;XV}A%dVcYNgYBgta`6fFf)M;4z2BD+sRlp%=oSp83%`NKJPRQm-BFFs za}eSb>_b66preaz+x%5+WoZ2BKAE#mN&P^iSxa2R^d+37;J2BEU77*pYdDW;G=LdE zaH)j~hy4ncUu*KtVZ!A2(Ym&BFJVky6t3oVscO6ztKZtW#b+Rx4^yOGers|zdn<qEDbP`s=~|oFM74d!86U4p*8vs$GZm!fW8T4Wrk+@A~Py zlx*z7_JQVohxyCG7F=&t2|oPeYP5~w=(ncaO(BlD4ay{fyt&IF6gIlS)Qd=OGmm7G zzwiP3?&fh1k>2PaC6{acG4ml@u~FdyQ6g)Q^6gn1*P3eo6l^C#Oy4@Oo{ZBg-ACRy zSb5`k2~#hiHu|(~p(C@?F()j6d)qnE|27Y)0D-S)JlVDhTYObpScJb}js)>zu1zf( zQx2|L(m?xz)BZ6B1zFYF!gWnYxlKowFgx z-7<>INA}$kV*g67bjH2TzE#k%k5eSE?aigv7=pa1SP>Ms$@6Y8lF2Q}th&c6hTPop z)s+ptq#31LN``6aS5RGUT=AEd16%ZuV~jp>#$-+U`*_B0lYO(X@bUif(pgzCFEpZ& zCF#jbapz-w!$H+K_iG`RnCeeKx6PSljC0i7D(B#0Jw)`nlhpw?QcvJ8p|XIlP%*v0 zYc-n3XcIw6iFhKO#t>WU>WKo`hZ}s_v(Duaxh@bKGuokyf^Y)gEtQ!QPK5G>M|l0n zy%N0Ky9^woCWpulqpRXWNTvJfbScuqJ;c zK#Nz~F)x_s616LV))1*t>o#v!$4#DZ)q1tm(7sK&OQPx~L>RbU{`zUn*7mM>1pu++;p#rl#pg zOUBy;XDbk{^)@LR?p9jrZ3Q{@Cn5P)hGNeV@3C@|H9Bf%%}^dH8EdS>QTNh;8tv!x z^iMo>D@d)O=jW80B5Cwm1K3&%)=6P+IZV0cE#OvrZ$5|CSv_i!ZEx)p!gC)@I^Ith zYp*B0C{d-6?x5AM&s0aqqN1F5C4x6;SJqYpBmASNIqWt6;lBlPoa8Z{6-6lQ z(9V>}h~CgF{`Jm+uG4dkLRTKjh<2)tf zbhn+oNt=u@rUP}AK7h)$-z*yHy`5|WztfvXmR>5;;ZGu=j0k~K+J_j7kr8@3TG@6g zGNg}N6K^}t-McJF+d@cp=BUohDvc7?%ar0iAIz!+eXqOOK#v`r#}i(@qb8r7UpKqZ zS!>{3^Q4~`iyr!9HbF-6(BE!dG$nqVhIu}u_hg8r_qZ8Uqh|^>?brCeo}{@7K*Uq1 zxkFZ7T?!c=^T3z9izPV1n4=;5t984ImpAw6`>D5EJ@^L7I$>wq=h-BeHuu2kWj~!9F4usnEE}4 z(D-pSPh(4@61Cu~rcrvV4Efplv~`?$`eGdyU+a=vSQ+aP8qKRAp~cfTdX^tCjN)0b zypy_n@$6~dMsW2a2Ku$8_{=x^4!y+v;1Hf@a$oO-t0aQy-#!!=4ZnybsQPB+qo&%- zN3tbFN9djGulAQZDCK_E9s>%dS&aY7xYvAjpNwd0F-3lA*&XlwvONw}Oz_)_Y6E!$Mv_M>NZnc2#}puELz+oCiPDaNp|Ad>|9Y!(BPq%vSNAN*x;%ixY^9a-QIe6ysTyth;(`S^*L0 zd5)O6`%1fI_x3Cuhqs{tRVwFYBP&RoNHa7iZUHlyIu7LIOSf7;2Bu;)MKr%I`{Q1V z!P1iDmu3PZ65SN%OR3~-8cs=@rDd%Okir>o@#H>95D)E+d!gMlJ;DwH`(UZmia8y< z>x`nzxP5`_Kdr)&+@!{?$@;eZE7!s7pbY@d=u-9zJ=>hZ=k%l31$5jP)$x96q&za0 z7XXKxuiCt-AX=jmjrY*!ES81-h#G#@GX*lzEPrtbu{3T_;l;sUflls?b zgg|>{*W`Y)BU}l1bD1r|5e@#$x-AK2Rp?RMc=T_c)GH9mOLZ2{mEa$(A6y`3L3_f= znSS%;{(nb;A-P-*m-Fp^ts^Ce-@KFm_pEr^^qQPwt*>^o5>SYACPiE>J!QJUr$pMr zB^Y}b?1v^l^Z(t<%ekrtv9CXiTp!DhNb~4e2Dlep(UYQx=1LoefXlZW@|f9u$5kndCHeMO#_$~z|60$sF-2^AA87(^@in9w(jrub=A*=|JC+Lxe9 zJ^Q2r&AVovNU{5pHG3k&+^6YKKp0Z`e}3LW9;xVJBI z(p)Rtrp_|a$-<`Yjq~{^GpYZ_1RAg0lqB0d?cpHFk*b7{xb}Q_22G?~n&c z#_kd@n z?YBQT0q45+tRH4w5aV&~Z3Ay5B2fxY;4GDIZrrCHLG3JXFk;TS4p8=%{DDx1`%@LU z-wQwr7~aiIyR+h4lYX`PBMx7vd3%?Jo#p;>jY6Gr3%j#XBfrXw#lG^zURSmqtWbKt zldnEs3!lt&Nb{O-z;NWB^J*A_blnKZ!!hn+xIs1!~JV=5|}6-x2}x$tsiqeQrVkM$sHY&QS(0LYE8-XW31Pj8v&IA^My-Ig(i|*`p~)Ee}@gswja#Mwyw>Sc~uL^o^8uv9QN*bE%|Ngbc!8b zw|lIIB7`=W$P3pBvr0UV*H9opGkq)%mMwAVzO@$BF0RN`7u+H}x|od*RlS;h^%ov$6 z@&E-}f?t%N8!pppcswqz&s{EpMwp4GNCQvS_RldFcRvWA02-;_Fv({vN|sLblaF29 zGKHFOasAHIPx_h!_>XPMV|--?5)E8>UEr9a+A-^#bQ84iVVc<9Rwn$JPF_N`w=sFn2I_95Z2hZw`A6Mv?puR7qR{u+l z@TRk(z#`Qh*ab#V>w_rNW|@|eZ}VQ*kv4Lzauj{NN;$m8&*#*!vkEhv=7y%`w5g6P zPLDc2Jl?JIaGjA(7|T!+{~C7=(6K0FWh3&2Ut_ueVWn(c{s0o1DqDe84S&yznSLJv z=$66Fn;)#VM#q$X4D-gU1vcuNbkaGiJ8%t^@Y`sjE(3>?Zc`iSnGZV|6SeOLff$Il zwab{-=0&R!#&_?avLK_)*@ALNC>%o>8t|pu935Eh<@3`xx zAJ(2ASJ%6`xhtVEZz>Gj){W~yE^vrMX!f&dA%OCXV`JlvrxZ8Na{ASfH0&RUx^`up zLt{1ra1Zca9%I?*(FR{MoE6c!{%D3T(bb~9-IRz7Jp#hlgh++`?I)e`st`{HZ~7Ny zqgQWw(i{4mxrpZ2XH55wavw~-p>$n^xd}yY&8uCwrrCud(gPC5K*SXGoZ+tx%%~*m zESWWLoGSE3-IK(w*VCA9lZ1{~03Mh05PATF!LdJxx4NXq^(sIc zXJ2L@S`?ZNg?HaDtaA%69&r)T@QvZ)8<}BT*BPycWfBZsgUPlSe!|I!4yCG&9Jnb?JnMK&j}_dlH5lYC!X znHjZiO<#AYH1GcCQN3JBE|n~;b6qXeCQX1W_~_4y12yh*VTyIP7fOxg>nZk)Hx{`Y zN6E)>HmkuJt<^zIV@r#l-gnO)Cak-D%k0PLX*fyW+ji$3G7}vNkW@U1VHff`pg}zg zS!aS6Zo>g8Pjvo$5L|As(&o0;Y;MjPSo-)4NdESdbe|Cl$FXQ#h-YsRQ6kR%O$j^S zG|a4Xe^uZpE@VAcS)%+qgripKu*jfLrZ?HOXF#?Zb|xr5OoO_5GvCP109+0cvFT82cAm&P~TADx}RZI7Mw=&B$?H>U8eQ=X_AKba7$ zO7hiW7NZWvX)~I&oqdT^7$p<-39*}{6osNsS6H~a~xReqHC$yj9)XYEU_%jIv?n3>$XX_GQtn2;grA@%JU zuxY1NJ|E}#QlTQh0t?}MMNorY3A#VCGhVX3cK?@ix9SAMZ9QOBT#^Wv$o-x+X;dynrl$vJ-ZYCQ(j&uWBG zmmI|z5NUsgS|2@{8D-ZhHk?D+&Sggkh`TH3APi|)r^qw@J;LCH zgN${lEfvJRmBZu9MHyxCUFUT z(N+mZLn$d)0aX*()@m7V%-`T>UR|@RUUxlwjW}G+sZsYd9jVdCmm`V`tgP8zL0Sfo zS3QJfU%x5x;|j+3i%KD~vo%EY!KT0w)B@7(Co4ae?ZRkakj@BkdItM=!i6U z?&p0uibhth;?OR!BWdx=u|Wg-UfsB8t6#3!iwS4!{Ex}sxAA!`w$~cHaEH7tj=T-; z!h7h`DX&9=11aO{NjxrjKF14eZbtdw_WtQgHRDMs0kRsn=tnK+z5i&dU+^^BdM~Gs zyHw-HI2Oy;k?_ zEUmSd>7BKsb)NV9*0x!NB#x;9qFC>i<5*m!EwCx7o#JlTY6 z-YqF_oYxnTExBm-IYrEak^_Um3#V8YAGNJw)y!9L(^Xn*tOsaCXaPdE-OqBe)}@kz zdy~G8z09y$-rNS^EIK+6JM+nCTKtcr8Ck|oo%<=0*0Ag$r>|Qx-~J>jRn6?wPMJpV z_%%a)-HfL{>(3t!0&r)oW7Cc3xfyaK=U+@7s{_8WFi6mBLZ*x?feu~WUAujZJ6Yqd zy_Nx`W6n0_G-fH0wYiKBBAwBf)y~=ZKqpVV3?Om8??-Qp znDYKk0Wsvg_a*AQwaG;)KC{ut4Q4i-cj`jT$L4_M9s@f@z*W5+K;Ed>! zOni4G@XqVRwPzsb!>fPq`++(FMbhe5p>zGM$(z{>)64GGMZoW?WmlDoCofwMzWW-` zjU;-oNb|;MT9|@2-#apqLZR-;#aWHbS`GEJF?c}#?nJe7p_Z}mVL8LQp(|Qf>xUjS zzwbE2ZZ5H#)U_FVA28|Et*G}{f+sO-{$n`QMG$U;%`r>a%6*dB51j~2qUiCyjzS$aA%DtAHl5=|N}@0o(fuMgWyqjb^YJ`)TzRQxtL z*lH*gU9HA5)3+w=>U4rB^ z{d2)dC!0SqdsUWXoJRmL*L?w2j(u|so$hfUT)WzbGA-tOKpOzIj2V90(#H2QjH#BU zE!_qsMHJWbb8)@)*Q{?FAkk9*rfB@?jnR5^h(N7jTOeE^q0hB=+*qNgqXdbnjSxTa3#i@o*0aM&vaZTC4P=3>Olikff2ma? z6D27i$l;0BqJ{gue0J1(hRD(G0v%77bs8Ca0EPDvW;X#5UbXEeFU6@-<~Xso*%{kL z%%GKKcwpd(-jQ+wKrN0dO2)HQ)=za2mqq#O&r;#gt~!Jy33NBOvggY<8KuyBdj*s- znMZf^IG$zi$vnl5Ykh<^{8nFj~dX5_o<`%ma-7Ts@mpqmt_4OFU6i%3J4!V9f$ zZ&wqJy`t|sEB_I`4p8Ahj=Klu|s8adR@;&}C{;z_^WhFrqa!ElM$&1T3Tp-;h zCzr1djk)vBqx%)eeHMRF#n)J|V!yW}MiG>|Y(LKbmRPzj|jJ)SSMu z$o~_v11FVx07_f`*AGb^*aY--`G--1k#UJ^VYtNG_$>wMxiN8jqX3G6<9v_upAL}} z09y%(ttM<3cyWN9x-8}eXRH(eTUMHcFl`NM8A;zOeb##I5}7+>ic90*Z__hgZYe9) zr}W!^PcSeTxB&?HScKrz5B9b8x_D%c>Fd~B|4?W^-AnfjaUZx)4Jh>J)qCz!V1~i- z`)R2Qpe@O-T=^?Spx4lS=Bmx$r!WJD=Bv>U!OAqC=70Wf9f9_}1m{0KDF06kNAjFh zyUGEWDuljaRn5mJ6uf`W0GV6P$$(SxT4;|J8LUV7otm2Yxi3gu97m;jEGD^5IChRx z+t-nZw|Tz7sdG4#GW=TSf7of#!@fK~yq(1W8!yX>pqCIQodc}l^7Au{EPNPS?T)y2 z71@?&79*Ascom3T>Q>kM*I1_dY5^E>Y$k+$g9w<5Py+5-#F|smjy^fnq_?p9cfIfB5_uEzSTHkPdshQho3ikqAoOj=k3XXU97WMXC5? zEMs6(0s}m6IEYNRX)Clx#V^gahr=eS9O1rao*Co{{~4pQRR0%2oXZ5ojpv@9;D9>o z0%-sQd)K`I+1j%*b{iXLY0vge-8x(4F*1MK?uMe^FtzB;ETGI zn3$4`m>8LovxB*ntr-*)O~QxxZdn)+oB<&l3FIW0JlTM$H?~nqOdr_L-{a70E3M2q zEu+p1IDKuoM7$tXoy>a*I+-jn3EROEC+7PArWL12U#(BkWomY~NLA1loj?imxH8q>g{jGgQB6;EHILjnCG~*uLPg`2wzl2wXBc_}D zi8zk9)D5%$=vbpC#Fxs!d$J1usoEQrw-vqnj;iPt<6GaKhC`} zn@?lzqnl)7H-zsyVqpSMa^HUO7D}o0K)w*1w7tc?S~h;InMk(*G_<;zri{6~JQOXk z4T6FWvx0&HwxEFz0q}u>f=vvCLInO}0Uz;PnE!he8k`IJKih9#P83lQlaT@btC%>O znc2HoI=D8zc1;H^HD{%&>8dF&$7|wX$7p2gU~IG=|iGu9!DXumG6q@o%WMU4^W@Ma<%#6$wg2-fKWc<#i=DhF4CI3?# z_$EMM>FVmp%f$5Y<44AiY>W=h7ECNWJUmRytW2z|48R!-E*|!-M(zyuE|mW+@_(-* zZsua*Y~|=`@Akumf)CD)_%|=0Ao1``~{H@-w~E{J&b_Uo-#R3$(K!GC$M*yfZ;$m2Qq}C@5hl z8F3L+cj&_`q;)%$%fY6z*R0~N``D=7*0Bmc#u1Jvpb*nB zV2WKedkQ!~*YJjK9mEYIPq_|LU{%atK)DVY#mZFAGE32NPPRpuI}VwQgAjTAPPo${ zb?U=>%0yrA%6VQ-NGgO2MT(Tx%Ko_ty5Wr`d4|@p6(ns}_gD^l%@BC7NE{#{Nk+@8 z{Yf|oTzI{Lx$_3@y*nQ*vnLXiLm6-rSKNT&t~9&GMc!o?rWN2x2H*p5R%4N)HS|5q zba^*F@xsIkTVP6W92R%&*e12R0yO}xVIISACG>o&NFP;%4g~`lBJq1TWp?=^rx#Sa zS6UYOHC4aGMBurf$rlq>(=ed=r}M~cMG&W+_i2B|Q_Fliqugh1mbUwqV>Fj>o}$&( z!^Y(v!qt<|>L2guW*K_1p01iYt=6)A?{=#n&PKvMY3TFFr7?FW-k+uQEGwfHtcmQ8 zaOJ^f@gBFG_wa0g#>Ulk>}I|{s0&L{;C=hkbH74++ALKM_r`{L#c7b3HrLop-}iQF zL*BtN2{{#!)SLOB^|)i>*zdVjZ~)StevM~z>i2x@x3OrEgDOj3Bi_6bO4w6Ul)ZFD z5TdEm%FvE)J8Tzg4RIM|k-C}hd{XlruR;U|WZJj9C32k zr9=l-C zgYop;MLPFfCIp)PFg%^_8$9!l3OzprR6R#AbwJd0>^NqL=qHwJsw#GMg|2_P1T@+= zE}AbjQm8yUU3bpk;py12yn5~6y-Cx3@lvTF8J3fb(V%yk94I-zH z6}+>2%on!+u2@~$ZU6kjse9% zpX%ywbDke8(gb}x-Jfo^Ibyu#bzNeziynfiZx$zfaW#i0JZRO7i0SA`jrx#JF1m#c z2Z=v|Q}sQqxYzw*FHoKX(7nm%>qc3PDjMdHEm1HCU<0ALdrm^gyCn(ja0=x!aXD99 zRmRTm<~s#3?&7Kl<$6!dhm+rCgpzHHvm_b4+MekR6gkMyxgLcb z=77#jRA<5aeC(Xn&^~+v1;$~~EVaTTxMRh6gn{FF7|)WP`DnK&r_x8yG(kdDIDG{^ zuVsvLdBz^t}2$zH`{KDLj?uVi*H|?7}jZT6mpmzn+RW|n|o>6YG>aw!-(j0ahS}!GhrIfCY zVW(J%%e@0mDfekUYT56p^1Up3NZ{(c9lPb}!GXRw8(~7MFQLs#v#4}2?lJ!HMTV9P z?1xS8fuzifFQln}0AZ8As9O?B?e0&e!}uW#>GRFt2mZJvJ@tsStH!1hWczS(`kTPs zWV2r+2CH%2zbQSiz&~lb>mQ7Sgt0}q4SQ*@+=6Kl@&vQR>bkRy`Z(-@Xb5#A(k+Fy zj%o%;Jg_{Od_$T&#I?pLvXAMyM)U-#^hz!^J#mLCW!!i{qQJo_OXO-lZPFprK z?tGL@aW~cgjR2@unStuOU5YO!r#|`iR#lts2osgKttE zz@+1~@@JY6PWw~L*CVlBq?*NeO;5Gkw{3@~%%$-}lU`$ey@oaD2JfFD`%=?4q@1J! zpsXYhw|Iu_s2pm>)yJpkE!hsg#M~cu`rSNi3kivyi;M_+i3%H<_BY$~l`A-9%pJLi z5A}uOL-YarppaF|Ajqe`eHp49vxGg%oJRW584e81X1>%wjIVy8@x>e-D*C7Gs_|{b zXOrlM(>TBK0m(w?DYVj18B3mH&-qqYf_Zj~vZCy2G65=j--`m$5SyxY^^oLGT&sgP ztCf!-be>8L@r4wbYtC>j1kkuLMiz#|1A~HBH9NI#q0)!1>lx|^bQWgEYgI>yrB~?j4(F;zQ=# z?DIP!UrRXTLCoVDHCNcV);D#1`sbMq=aJj}h5}l@0ymufrYnuDsy)(wJJi0#O2xI4 z-X+#Q=K$`FHKY$7f{hhpC=DAwk!p=4_O2)1+=wAwnCgtwnQR_cn&9adF?j~O{b28z z7#4S(bJ+0oXzx~%pEPzJ1}J8$fkw9Ui(_ll3^kG`rp&DIa0pkBt+&7Z`}q25W|#}Un~M?Mxr(>_FIq+G_G*>DJEhnAjZ zYEKC{Z`;jva1Ag`peVs!`TDC-17o;#U}y1duL8CI6)SN-ji^6TfWoO{)VxM zUoQ0pcY#=cO6E||mlHQtL)Rp}BZ%>?8V`(5oG|cG;6v?1r>}QB5Qzhz#mImeK~izq zC10od^X*O6#;DCW8XIY=N@o4wfSA*p>z}4O3k-)BZrm^{v40CoHo3OY)0FF3=(C9_ z9((_)tR5U_4V_KVo=tq%0lZZ^NiyEoC3u|eV~3WSKk+ptK&@Ggsn)N`7}~AVYOZ%T zj&_^o`GW>n&TCq&k8VSNo;f4+@)QIEShDoWodYcISQr8~q}ubO3hG&`3pF-D7dZcV z?t!%Y`phXsJxXzcFil~CU&B`o{OdfVyz@nd(QwP<2jNEL>J^8<8cn=})y^p&jWWT@0&{fpG->S46OWyL33=_t37#SW4eJyXDwi+wG z7}8?T(wg09(fA{IwvLY^lZOKRTCsDl{6Wy8d5~>Wz7_>gm=V*s*n?0)drR6*v&5a{rOp>m$&Jnv2UM1=Kp==40{MNz!l$Y#zLzm~8jx$gSXJaez9w{{C~0j+}#jj2C0!kJd)Fd#%>mM%6AG4t%?2Gx(v zDek>Ej30`77!UIk;Lkk_Csd=d&VDB;S}z`?B4_zM-L}kw_)=r=YV^qcF2lzX3BS&J zRjOR8yo5RK-v*M>UInD7$0TgkSQy_9@@9Q)j=pDLCtwmVuzkGmn7Hg z3ajbDkYn#{KH9EA#hgly#-GlzuOPOZa$XpWtUo8+s>Qz1v}MHk8BWS`E{cC-w7@%m zand~q2$}tkTymysJFXh4tX8oVxXBfnva1OpJ#b($xXs_+lRn_N-Fe&}Htiz_el#CwYou{I8B?_eS=3@WP=$8Obu26rfBc#Eb@<9Zo!LhpmODHpUf^!0 zARWw53xpvWIeFz(-dl9MS9s=g4_qs*N%|O%1JHP63K%G}F_ zUbi3nEF#mD=X%_3&!{MlRev9vq|Ib83C%WvO%zb9+e*=4Ov;+&I5v)ANFjC}BENOn z{6N81Zj;j`<)$_-)H+zBX5B|vk)k%9^EZ+#rd5|IQCRC`3>fHQ?lqSy-FrP&Xt0l} zR+_zkAuwZYL`AHB@(O1FwRlbv2>!yd5R6P$lGE}KTi@59@b-rOvW+q`cY|g{?KxV; zZ2bvzpzI$*WYwP`)m1m?li4keQ<4Zwxx5>LGzX z=MZE5aZoo=IlZvJ9Bs&Hiz-n|x(p}95_1kX?V=$uqbZg@eD4p7=JU;V3cuTO>&7YF zFSb7<$}%G5-1JfReC)hmG1Pe^@2WQFS~n-nBML?c6lV_>CI-tu_H;eAKG71n{dAHt zb3S8j@a}>}+_67ic!wr%q4bJmffG_m?j4OmhN0r0FR^nPP8kcz&nm8vV&@{Zufeaq zoXsY#q73YYgSXp}G)jn+SM8;ES#?{s zo2BZ;!;gV7p#a+$xD9WP#N?rG;!7&PC1M&Zk1)2<+cvGZZK&kjB`-nTRy{1_2izkz zlJrI=c(#ce!rA~?A!DGK)C-XS%`6_z^LK(&Pu9oCT)}g>b}AvhTsCaV-Z!soA>%j8 z)@2nlppnAf1sUd@vjf_)@_bRKm=ZYh5aUp~?*<7?l7YK?&Se&JZw3 zfk6Q=CO=?gvlv$`gp0E$2imG4GeiQ*A$5>r(`jXx8=5$A@Mxo9q z`4G9Ee}VZ5h(Uf;L^}!DVE^u)2>#A$L?q8e7&=;5Obo~84@P7k+xvxo`=gB5|CE29 zF4}Y_@w;vK5|p@((lP=kTtN`Jpyna9@z9J8MH(yS>vVd~5z{~7%VNu%Yu+vu2Ekzh zrGqcQcZO6wdP7M#)YMt^_qON9r11rz`P$4xc~*hy0;k!%v&Tz2#eiR zOHL4=++tN}2}noGl8atTsi2#&iAxhx8Deitl(@WjIWwVC-~M=huORbJc%P!QKRm3S zecl!;%!e6n4S{_BJU|MqwU2S*;taPG^5VsCfHMajG$`ANUX&}ef9jVe_vUZFe|Zf1 z>*gT1{8RvZs7b$nKGX%N+$tq-va|#sob@TwHrV=&1O9glU!aP2?W3DH@WVXC zS}~nfB2@#AFo+bvm#Dc+1HP`C8g4DxbEeof+C55I*wE4?s?W#?m z^Cb?|PJ?e_Sv%n6`p~uX=N0LZHM9~vFoX*PtRb3+p7cavi-xa{1o**=LO&xv@Q^{$ zq1b*6S9B12)?FwT)Sgb0ouK&=-NY$fp)j>6slZH|yT@v`MP>9cc0!2-qN;i9sdMP* zu5u_TQX3bl&(5P?|JD5Lzip-Xb`_fVG_MDuz$mCTr(?^={xS$2Vd~V~@BO5g;u=;r zA*j>N8G#@nOb~9^%pLwJYmhzfp}Pg$cZMU04ccXIz?K11X&(eO=%9wwZzBo*Z9}*LlKHdr_fWFXGPI6oy@?SbRxg zkaLJL$D&Y#oeJl&q^exIef)(KYs`8vwSc3IpmNUOZCN;{mL>c z*2~l>rq836l;ch_-8rC`Ha@r+%S}xW_tAvz>SuTNIz$QxJm+Wx!Oi?G#Nii;=CbJHF>{iC$)`9-+;qq^{mi zFhSdxuZ`Cxw|}El=okNFLI&qMZZy#L5|4%TV)YEch4!^i-sh`X@||dhob-GW7WUMy z0%Jprx2L?Zq^bQ7j@9qX^h@fo^Ca3|!b7S&6plnZ^mhwlhba23K+|XBg+1-Qb^r#N zt8`B`w!$Um%{rnGMDX^zBEl_&^(4;e_)JZc0J(u^47rHh7`7t6vWb?a_Q682t+7IG zsR?18uBzuRDPg#?74u9x_N5f&v{km{2>l@;h@2H23yT#W<`_P5@~kRmur-vK7Vj-6 zR?Lxv5-FjmXn~Myc|A1``cyVg6Mw;(EW3mioFwVzXiC|I2K_M{+xUZQaaZ-Cd1i>m zpY*VD)N^ZuXQ{v1G!s%n8N+bHCO;N*n-G3oZ`WDFeAl??0{;uFkq%8IVs_Erw0AK;bsD4 zVnB^5albbOC`$D4^D(1`bdC|mKj!mTVVK^*hc3j-X_}zEN`V7&S|`t+-LP0L45@nn zqNpb1BVw~4Oy3}mnr?x@2E{l#ZJ+9%0^i{mCfz9NNi!3 zL*E76bvWX1>dMokEe5}CW_7L|L~b!c-jdh+to(>9mK=Mduqp-&Q@XlP3)o7J}Org{k0-+f5*Q&C`gwiH$M2>*Z zkTEnMUeo#JR&LQtre|(Fv)bbjodrXPf>{k%j>;MpXUc@=wdmuf22B--vk3 z{vq2qzaS`}4^=AG$2afo`U3-aoKDtWQz*9<`xQ-dyb+>LfE%}KZ@&-~E*;tqn{Q`c zK1+C}*xr`1Dm$ofCJ9X7UjvHii3gjKa$Va_-;2=Gs?jMInd*g%>tbl3M1C4ef$Cp` z@hR^2W5ylW%R^)HZq)S77WfKY3hiPsjtVf=F|*c&&-scMx7F5Pug`P*w`&77wVcGMNWdw?9&nn*1w>uER);~2{m)0_6_Ww}+$wMH&v$Ezj0RjtyeL-`r$Rl4-3 z9@~7ldAo%CeWsLA$3mj$z&9lfshV|RAmP5`vC$b}_a&VYFg0}eEw$UMfGkC1nKbjh zwy*Wm+ew8B(t!S>yvdZUoS^K znW_Eaed{Mz(|)Tm?mG4?0VA_ksSgc8Cta}TE$M1KAtL;GA7(AG*QrterIIW)P0*>a zn$4pa8i$@WeMdukwmke=hBQ*Sad9C<3w9~;*kOS;xC1}wzcqYa`Tb@gMj%QZsu%O7 zF!JQuu*mh7aB_s_La~;*u(fMd2(ET?>Q1zM zP|&Q^lW6m|osknJP3Z}GTf-Qx)-5?9SzBa#lq;2~xjGhmNH1UYBf~MVS4gtW^n;Z^ zth23~Ep!m*^=ti#mYJ2kV%{GlMB9$lY;r?`#|Vsg2HrJ{5=?@0l@E8z^HG-SqkN!k zRy@RV;DK%GR_vg*d0v%iT?XUUHE-_D6c7Z-DWJG^>218 zOhZ5)t~fGg8ksof+(y5;k--A=IIKklMnAhkzcXXg7b(|F9hzeq2pi7XJ*L<&p*+Wa zXg^4hrTAe>ONJt+dsncl!^`H$q$*jqv}qap{%MWI=7S}T@a#*pbW^}HQcn-bT8=u z3SxCIXR=!*nh%DFi^A;i*wwn&S1${+_RUk?iTF+-NSY=%LZw@Nq&#vFtFO!OR1mmS zeXIb@iM1}e6JRl|%ZbwMow=NPjxA3d4_97a`l9o6@$WXtZjoYT21i_xI1p$4H+qiOTje*@BMy$$pFWsL~gB822m za0cS4CfPFNp#`7%PSVq_MK3mHzhl7H-g{2{78)dSiT3qy%8EGlI0y*=3;&H9{(4cI40vw{XD6B}4V0X{IKr z2}s8`PKbpcSHIRI0o0gMqi8e4fdWQbRJf4}sc5enHEB(5j#!oyPKrrz9X$wz9W~lz zL5vJsI%@4GFN_ynMS-ArRsYlP`SDWL`j$iNH|fAlUlW;Sl}|TiTr{soJ{*TREuIJ> z8WtrxebY~8UKN(T9vvE}jdz6AF+jdJFN8X*mckUB=Ql>oV}PDS4@|Kh>>=hdyf&&E z5j!Zl8iYjgJ`YpWP*L+q-M1<)<9>gwI5gmHxvFmz!QiuGYraW@|Q z_%+-^(O=FxK4&U|qLFJv^0lAVWKnj1`6ary+Ri}Y_U+jYRE;KS3gKDa4!^F0q!%9& z9JYLxd~IKJD&oI}*=zM`HtJgW*jEkzuw&k?4U;&LJwtNecw_OZkb___#pHeNk9-+T zNSm{XQdrgiBDC+I=Td_Fbc&Apu`iE7m0rM4yy!Kb-W=5@N>$E1m!9sALh)4797E+3UZ{ zeek(jlrqu5*oI*;FxC_1XUc<`DM3@zBWgRTa@tdPsc8#Z5T1V*gTi9~ly*Sh52aL} z2NTejUQBdVcI5N1_(#K_zI`P>6qxO5j2M4`8|`WvUIq7Z$9}@p_&^K;`oNWHB#u67 zjX(rU`^l2~znPrS0NHAlXEI?Zlh38$&a;+Wx1KeEj#Y!?6Dhr!y||!__NdA3x6wkC z5eyqMS$`|kyE9(tS@s^&kQS8`#ugVy%w`Nwn1#PD1WJYzD>Dg#+;EJ2GX_M``;J}r zNa6k_gYUKnL#H$VjZj> z6MXzv&(5bT;XfFz7)YU-(r#SFOglwfY zez)+-Q)ekkSkPHg!odEN7y!3B;8`d_xq5+`uY z=%v4}O1xw>U<#(m54`Z~9llXcF8@bM;C#L631|tH)GzKHqt1U1fSKhl&Cl!SfHxTQ zSEwjbJGEy2xTVlzC7aBr6_$DkC=^vMIl#JV&dUZ0$?OekjPI;x-XYX_Y753!wwx5uAEHLVn6RyM$>0St?%tYPbLh!OPRUOkWWKWRj?6 zb>~|(Z7Ldi0#O(>nZ9iwPjlTZfH|VH*vve#G5%@{pzS|1zHlkJ&fhVXJt7Uc2<_^X z@9B%@qz`9O0FnMZt%pcNB8Zc1b}ie&_6E)sds`TWEL&Zy3PzDV!DL$cM? z#KFX0JKu@_ovfmOzlMuuc&I3N`9*S#q^mNlpPEsB`jF7i*JP~0pdS67lml4pDj$Z6 zL|RNh8vhl3A2du8#iUQijFGL7CGz);ngFy$mQ_Wnb`Vte$0)c0@ULtV&_qO(BE4u{ zK;+_aU&%}7Xe1O^z9)<>tw&*MW>)o4#-uU}U@eVoc`$-A`&9jUUG@{_+}6tO(x0yo z0X;+4sfq6z^~#2GB3ML>{Cv;Gg`nv9k=QjALXMb=NyYt~(eh#vC3|h()v)Ug~fpn}RuCQjxPl%j>Yg#^$WdNy0x5K~ls7suI7YFPy)h zT!9H)%aZTWVke}Ad+ynH_u~PV{TqRoPg#XoWIY9Lx_#CkdH@Z!>!@=;;sr` zf9~l8GF(2)W_2$biWI4jYkz@n*)pbOO2D0r<$}Wxxiv%|a?O{#r{pmApf9_S8#)XN z!k$!~(N2Ng>~u4NmL>4EP{3Yg2s@f_0e4!KC+F&yUh!enZ(%5{FFrFHiE28uUm}EO z-F23kAJ7vbf8(Sy7Rpzebx=>da>*flj0HryS!M#Y9jo>@VGp4+k(*yBkE#fH6NjEE z1xJUN8cA*qy(9S4aE2)&1($OlW7607U#5o5A<8(S`fID)J00_Eb&bR;Sk0gX3*RPS zLPC!Abti8_zr?I|@_8hBOcl);$xv(8N`BgZ-0{R&(czG=!`DlCu#(C>2d!`D%+N=| z1TE%^O8W;U@^Dpyq^;`nvy)c#^KnFmK3CpR6ueA-ynB?dATHX3~=vM#8(p z?eTn%0ga6yaoE#hb$wvzasSr#uRI!^+)s?qLh1z@ATrAw*{MKjSCBR_UK60XPH~hj zTyvq%T?k~Bdv`)&y$Fc|fd0^XK3~4x^`vYD`xYh9_lI;BH!ZU_H|vtdRL=1mI623( z<1rNm1O{4^MB}wq`Lw@6())wzoNHp!aY zrg7S7zwjJYG1q=X0U{D)P0HYj<_rWFl0ys%&bAs8{A~qLGSFVukhh@;&K8d?t(cCe z1p=Ll%Hc}Jd~h;!3}pmoC*A@KZ~oJXuG(U|$S9#q=9n#v9yOHH$fR>Um&9p9ljn;^ zYg6*=Agh73g)6887_T6LL{|RU)I2cq6iSqG$brFwm{jp0ggak^^GL91_si=PwiLY7 z0{d-gr^u0BabI!)HQ7B{Yob@Ov?~gJ2aYHXb@~-S9SRU?gTtqP8A%KnPTlGPb?mA4 zL)h4~fOjf8vYac)5slL;9p-p$%opr$10jZK9EoJGp-rNaN5=c?7tr`4^lc556i2is zboLQ2`?m-9ThX|EnN?e6NQOtbmD7np29w4@r2^=PA82ON4x7epCL0|+ARpRQfqNtP zNo?5b#or?T+u>{s?lZ*pfv-N{e^u^*r~BizFeQ&v02lc?xAhu*04$kQ7e+X8G%yoR zoZY>UsCS@$;~xNczQ&QARdU0``o9<<`bp`C5py zolO?{aHmT?O3}p1GsKd#IRk4@{@nymDw{dA5bvCn=N6@*bWw4<$=zwK|I_ ze$vfKt4dGwa#^-^Bpo~ta%#`+m1%3k)DAViXHGuiNa*cvk*c;xt?_Ql?xb!T+?ANW zg`MdMVUqSX)wN>;%dKbjl&VHKHAXb!x+N!^002 znLy9OM~L-@?I-ycOlaZ5ydK}YF@u&M-dRbOXU|VD1GGnPuo};W1nE1COMjJH#q-1v zrZHx#)s2h1HqTRS{WGhlyfFIC)!!DVr9Gd_QK~?8~72RI*3fKmEt}@psX~(t}t`C4kMu!C4gKm3x}qMs1;Q7^QDn zUgn*mZW=v$p~i{kIcv`fjb%QsHf(8ZXv_*_|KOZ1raCFxBaQoGrjpC4_l*BE7UqDK zD)i?VarR^UU(bMdVRa*4%GXMIsMFA!ieUE=g{abpmo~A5(+R=YY;Etn<&aX+G&eS^mMz)c2Cw zLHXC`6@B(pR%T@#VgOL3{x1hz*=s2~vZORLI(Q7W(06f5yIsiAULqkpE*uo#!sl(w zLuTp$E(>~OR=+K7Sk*5Eg`QY`x^Kp802qme{uYoSgd73B(?bUP8UN$O^lkgE z?`j%)ugu3kYqibJ+ycCD@zxVSGF#L7(WL{hCL=?p+cSW8B3E_XS&wa^^PdRc0Dy{; z3zOR^0Er4d+zbk#`2p0ViI#v57*qP!;lePUl^?yVvu8p7+y4h%lka_$<2$Q$>vQH9 zXE6|230!av(>&F{*F%2`AX`(v7dG5pD*B#bkGTLa%b)8=>K7=vHqPh3#8Vbdo{G@G zqL$o&MRx19qG`2sE0Rgr%=5Gm#K3S3FgP@;m0leHL2k8p5!3mCmM#Km;?zgr5$7zf z7EIy}7#uugsRF;)Dtu)jVEs8L09v6^4T3!K>qb9DV6=8LZUzv*tzX^z48$eQ$4`V8{l!#HFTXqj%Sxa^(=C-FEhjqz>xEV>RDEn zdUB#*vyOdYZP5j$-H-0qt!uUT5UQEnH#T4Z$ zXIp5sGh9RSLf$7qq+l=ugrxgn@^p|iXWj`w&Vx}o7lfBX)>{Fw$|lkWfWArp04-7t zj5b6SogBX>k1-bjiGrUZID0s-t64CPvRVZ?BscxH--!WvxVN1E;Ss{n^z;3G2CyjM z!h7YzJSt=vviB3v7&Ax%U|38yZeQS(DhX_5Z<-vT+r2+xU$#;W1P_2*+8yD#cmqH| z(?|kg`}r1=9}j)|#g7-Lzd?tXPa%t!<3uB_t3}zfeVf-u=&ucGU3;`ITGrDNtfNq3 zb3+<2nNY=D2*`Xf`HalIchfnvl-=u8%#&nu`d)Swx9JwyNUYIF%+S%Ok4GOnQ5MKK zmn;iDr<-Z!i~a)qEM(H#At_Z5W)oB^e9za&NqgU$UQ*g=3P^Ek0XVMbVt4uy6M~l) ztOv8f3fu!B?eT^2T>uz^dOh{4HDKL?BLHO~8VmlIYXTrEBLLu^{oVPn#ngSZDn4!K z5Cw!uv=|5Q&;f35#4ulJ=2M1UcXo1C_TKGRRr%arR{2$EBi@doiR(1`0uXn7B><%G z0l3i`=mo~VxhN8l+czQo=39CKfwRW%ns&D-v%HV;*P%3cpCx$y{Qehex8!oTR zL7}Wp6RTm)OYR2Lr*5|Elv$BxG71jt3O4bwF=Og{aJ~xVY9mh~Pq-olTcW#>?8G>Dn^1|0PuzxC znBAlj1K>|O2%AAFRMEq;NKt(M2XFU#L6SR*YwpH(G42JkwR>BN8;3hR3~ya^L9K8E zu)Sb&;XPC_>AKx{PY>Q%=xGv2rVua%mA)N{B2r3t!RKrmQo>MvWZ>T?rPe&ZOrF)F ztRs%;*u7!5+c`p=jY-De96YBpFZS)M`Zr&eHduH1c;K^+u?!NGK871kmeZA2i&paF zA>}@?sdf##|Dt36?ad2L9PUWFI_E;@_Nsv%YX?ufv#+acTkSd5C}I)>gps9BS|p+G zB>Ru@(R>(N4_JwSPXc<1_>J*On_Pyv1Ay(X~wpoeemR>6!cEy}C7k69I&0iX*+IxdD)i+8S? zP6r5!a8iX}O&)=TCbLGcT$rw4B+~(^2^IcsG=KjVfcuDSy7a!B)zIkO5qcsQ33~TB z;~hqfWiao!D+4EfsBFH-jtXdib;>WCc%b5#h>(R;k{@m+8lTOT#?&l%dx_G#Aax|fnsnY8gCRqfF*Pqp;$bpToqk<@14LhaAr0r2&G~q)qUkJu$~)) z_P#qfZw55b=NO>}XNS}ZFFDT&%v9#uxNL7B?*ZG`#gEc+^g=kX9uk3QN4ax4egbA_ zqj@uS+Pkd0HnI`5>O0?YTb*Q(NVPJxRq|;cwj$hK!iKEA)GZt;(k5-sm@VTLe^!U2 z<>awF9`1U4-}2=Emzx3Z?t;~FTt)PBwaUU0SjR=VL9*a5CK@MvMYMY2cmKFcWVh0B zL`uS1Z)kvCuCxRp!ahMCS@8stm{nulI4@mNRqLwBVGxA*vSSh1mKcE(Y>O^w{Bu-X zX~8UX1;%TLD$_RqT6}E6^pE7or#um=K=KJ4-LL>u5q5$GwQ$^A4F$SjmU)^odKa>3bunvx~2BwJr1H;3EHYhe8`6=8r85iC0pnKCQ~yB<+`Obw_YG&-bGPl)s`Q}-;2AMwJPc*!*Fb$$u{&1%26=1N zm|N)1rZo4{_EdCEt(bg@L`+zD-cqLH0Y9o2gNBu@0L;`Ih}mxp`Bdc)Y8byEeFa!i zj)`7TMr`TZu^yzYk?JDHqNftekBhiEw#q_cabKe0U*Lv$0)ek=urjQ$Ect8ZxkL$* zJu&$;@q^5X>a(+=2>~VY(S_yIMj0vO;9iimk7}uJ&2kQhUMRY%l z@z94EWvP;}zCwMI33tzZkhmRWbd8S(Mh#>*+0PlUfyajg6C|NoZeJk<3DmltvNz6K z0A~C($M0`acb1~vQrF_PzMi<5Ofk|l>{-slb$RCOQFh58`EAO}nc)_(4YmrjEBPTi z%s=CRuOUog)GrW|&_7oXlEeFwNPhkoolW^Md&Ip2Uax><0yigNLZ7u8g|>n4PcQm! z_$OKL>4RQ$2-nzOcNJ&=Rm1hfL)924XqQ5TEP_J0^h7>2Ed|}M>t%;de;eMTZ8M5S zi2`@Hm|?1M_@m#o6z?BJBh3c)82&KiY#yyzzN`P-sZ$w|W8@^$@>RZlGWJEv00(ve ztg}Bz>n2iDcf4UvTbB6SPy`0~QgU#EN}hjHQNK4?M{j9aCBX^kjibUV4%*BRMYyBx zD4Yqo;el&dYgGp|ffaXj3vQz6VG*CbhF61Jzk)HEhe|m9Ra)DoXZ<&oH4fuUIc*u_fw;mHy!y4dBRjlTeNEY`TW4X@+{RvKQ%hoiaq9(hRiL>fuH zS`RY*yhYXnpO_&J zVk-Nr{MT&y#-I{a`o8)zazu-2lDHCqmZhr81S7#GhCrGmwNL5HNq6Q$6WK`9%LTVJ z`t7FidGWli6^%>Q9`OW6o6=HUMA8)AW50C}mhmzQ@gf7lAHLb+m*ZtHvrfDMTk1ik9BzGnq>2 zFvZG&1&J9RP(~RDt})V$c%`l4TCSg+1BBXjlzmY+6hH7V>zfU5AmTecn`)ta8FAtI z29J9hE&-?CCVUi*H^0jAjc&d|f6s52=V_MKh2a(TXBjZdXBMv=V^cl?-kue~;cHLG zxE0nP=A?}2;)}j~=E&Ko?&QW=`m{!D`qVsO z4;qo5$lzSkyU8$i(i*9V8$hEb9=em8Bn!_AkPwXf)UDL>k3x#aGuFi*-JBFdN9}{F zX^B{tkwXilA3OsPq#^_$e9Uer430#B=?yoml*T{M2@W$EMT6bEmC4mbIYNPD0#GDt zoPOqEZi9}}VfV4zxnkSTWobfEnj#DAY53$=8f=$K^dRWaC}>{{b~q3x1?g%za|tGz z`DO7Qaft>(tnnDuYBk&hW{lW6~Yhj z?{if`mv3WoBDElRWlZ~N^0{rNeBNT4tz(8ov^veHMt|{P2!b?)6P`GBQ6_P3U8oN( z1lGL^h6j@k9BPlQ^s!1gM!@s9;1X zg}xuiQ1xQT4#%l+f^R9d5k@`D*zAi0kuEZ2Mu(%bhUjnP*e-uSrI#psMYR!F7j+GY z3TNe`2H70oh+_=u2RbE!HfQ=>EV4ZculgzcWF3v#2WX3K#r~7m4FAy>D;p=Wllw{M+&EFNpdHRb!*o7$YlSGT&%vd~n;%4y7v;^OREHR?x))Oe;PF!x&2#Hx?04CmT`h$t!SUoF$BOw-=bx4YACGZbD9 z-)e%Qap?2Kux9j&tl>sUl(Goez5Uwj&wO6EqC|%_Y`V9ZeeAS2NF-t`gFZ<4KIS)%#O?uM}l|*$F15jrXd&W(`O=J z5zveHMlhhsvAXXBh;c_!0H>NZ;QU{dQP5ld|F#svbWr!`oW}{2%xV8F8nNuwi7%jq z^QP5h7t7qK3+aMIlD@QOTGfp{&P-lm&?_mI%3ZgX|MP6B1lBf_Ud;Ctf3?23Aou=V8id&PE+iBVXM~ZF6l_GZjVq;k(00T70<_hnkCf)tTfRuE`}c%h zp%P;Xx7x%9C*!`c#T;f4`tKT6=2JQ!bo3qa_K(=Y)ePL*A^|HIJ5?c zPdPQ-$Ne@g(C^UhEAyYsKIz8LxWp8QuYlt~faOa_jWC<`-f#>y7J#xrNSilb14Z*IALG;NSD*6jRj6F+#&8+)MWyzYd13vfyLcw)EtYOGbGcLEF6xn3;aOv z#y;!o=I#rgNN&t*792}`SPo}(qgihS%Q`ii-HWZ`$rhO!-TA6LI5Y|JLdaBiKc;IM zGj71oaD|%vDHJ$e9zj-%*(WFI5^1wNm%7cZ&Lp^Hn3K$2pr-ySZ`|1}^XPZ|Lh+9i z{d5|Z3Z`aCL#UGe+!kHg`Q(Wj7!(&yUo`F`ZF)ROCR@mmxw<_9rra=-F@(4=4~o^x ziN_?qS-Q0gr3R6Pr>duzZ(GGEU@$X7i%~DjXRUmPAUIC&%S4$u#)T-k&*36{rimis z2Jt`Wo(R3g_DOuROaKo?4S^D2=JsU-LdJTlI}lhJA{@?n3MOfy*G_82Exoid-XINx z$$P}e0{h{@Abg7O0^^s3@~+206_DcnpDNBhp6UIME5b7+EJ0mAuh@quOAv%(uTbz?yb1NzNeZQ;oIKSVYpU3y{ z{(LW=?eqA&->>KUb&Qt1E%a=_>vyIvFdgcg0}XoBx)c5#{1nV~E1`ek!rfmibeC?s z4s&$smDF!$>YgU%wkkox01mo4Ueb1ISohT>^WLK&| zKa3?MnU2?!_~tRv!q!mxIei5$(TvGYyHwQ8Vv|KdyzME5*2U61C1Y>FVw>}RBF=qB zxpG>@RmhJ+iR@7cf5^njVd2!OjD3sCba*Oc_@;_a8_XoNUhv zK!2a2;50b8T-a1ajpX>uI{8;(MP^tnKldyAX;C;^-FCge^6CA@crG`;D&hv0(Eq1a z1fKGYpULI*DeatfqZyw^aFv?*!+3Wrd5>d2yVQFea8UN9tl~V&+ne>v@0FLL)w(;o zd*r;G${*n+XH6i*5mS_&A9*`p_@{Z6X5w7Y4#Bnpa+{nl!a4X1{^%voBvP}P>B+Lu zcW<%@#r?mk^5~vB+|#Ovq&QSum_geRtEKpR3ayfh@Zq0LHLD^sH&ZZdSp#9cU?+mkRc@RSNV%Ltv;K zONz%rUU&Pzb!QAXk^`xB#?5yC&Pn!3L%pf7tnb=gU7lPgJwNj3A?8m5;{fDT<8GoN z94{`I^Lf1(zmZD4Wb@odE+`$3I|)-$4UFADN?}A7{d~$soIgT)>nrAgu5B{6HbUO{ z&vaD7TExcc+led(g+n89aVk(`ocsoJe4?q1%pzkt^i`DLxRG)+QyO_WsV;GCt$$Oj zrET6BlQNR*k@XTi{9S*bDinL!c1{hYajpfWKrk5*BTqU7gzl&-lfh(`k&!cr?RHw1 zxg&1*&Et(hp?xOx64fr-vubAs@0cjOv&T`dl|~4*J3pkGpHjDX_G8|I)lv%L$?59& z<1QVNMzc4Mp)t{f3upF6C~%XN_h0CZ$r_VA$YxmG+z-M}$BU+$N?SQrA1h>}_O)6R zlu(0Pf3Vfaq5Dfjx-OeW=i3x?*5l{XZ=NUm6Nswa##59f&Hd_*r7z(_4?h)kqsq=z z>D9gs->>?RZt^NUr{c_tRW7%bOm3lTtu2nkUL|ITvp??+%DH`Qt1P>Nvko`vtOBdC z*ZYI28}amqCcV)b0T0F&UQYj#;d1u7<(K7^(W_QNAW!wBT&>>;3BXZ6pY5Todk=Q^ z=<^!R9{sg;tD>2i2FmLk%d#0%n{U|Ar28scQaCzM#rKs*TX7`-52U_VSiwZfO!gQ4Ab zPWfb?$SUdXe_B6h){t%7EjT^;VEKu`ii@vlTBh7vC#>bpdP@;bAb;(gvtfOhg~jyt zJddeDmgl66$1#u5G;tmYGs^Xf$HV{)UTw_+gE5dBJOxXVn{>8QRfgK4iP3!_G0m*h1?CvSBaVJcb@R07DyfUVn*ZG} zd{-Gj0|7@H0^&0kmQWi0VYA0!Z_urv4{v-S>&K5TbQH<=5uf_n+4eIQR3N`Gh>laiTLaec#F(vJ33Nkf`9)8sx z-vz0>g6!Ljz@QxRisLISBC~}WHpz^xBdAw!2nVfQO&g`KD%N_(mbXJs`Gxs+v1G~=A>{$? zsN`p;A%%l1+t&g`CHnc|>#j6D<58XC?txf?fy`0X*Trx)E?cx`LlcR9!DWUL{MGxcdmyNloB1AIMcwYaWgE16CmhRX- z>4q|MXeOE3?D5y}l!3^}OBbejefU~eICQ%pKCnrq&Z%>X;%b);H$h;zc_}z{56;{u zIr)rQ4+BqR8eTMmFwK$B1(H9-oTROF%L%{0c`jHa`H~>g*GYEna|hJzp+B3*-9Fx? zo)^V2*Qm(5vOS&WF$#8IJ;MfOa30JxYiAoic10CQ8$9q*gE$I~3m4R#!OYUdJ2?iw z<7B?p9@vYC0wT1Q6l}*wo#dl)wvR~LmuXNq%VkrBh{ zxrPtz=CfUAS9-19QIh*oT08)`g+t0jz^lS3Y9s8m`LsDRFj7hC>DKBhgaNw;4C#SC$mUskg(;ESHTF$$G!4X@2(|;7p#XWo}I%|7YJ^@$pbhx9b*t|;Qxuj_Y zkz#;<6mY&S3ChAT8Z=Q8e4FEF>&%Pspf%Sk4uJfDuyFKm2_$fOYcM&Xx%OL+uH*uG zaT^_n8pI3lGXq+IcLzS*AtsaB`51`Y_(qX^VLcXO7Zq^FrteEAIgYnOi;v~Ze9(|e zLfY}n##q;L&9>^)tqUJLiM51CWipU%urDex5*o}I&;g7{zBZtUrUMpWh|dP&66TPn zpa)n&HL(nb7s2neq1S!dBmoOC7r;Vn#W%sjg)$C=?HAQCq=2=^WXPq@)Grrhw6y_Y z!Q+9FG5~F;DY*mvGqPH0Pbv(`P*MK mC=3jf+S+C=kzuz2|7?3msO_?co8tw#;kh|^IM&)zlK%@Ionch~ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/5-ingestion-scale.png b/website/www/site/static/images/blog/scaling-streaming-workload/5-ingestion-scale.png new file mode 100644 index 0000000000000000000000000000000000000000..04a6eaf4c3a14b21e49bfa87e0b09ac2ea614b97 GIT binary patch literal 38156 zcmeFZby!qg7e1__A}F9DB2tPH(jg54h=MeTIv-S(Wl4zd5}!MFjzsSH(--H? z5v-g$cb@OcW#EVs;S+k`ho+^pw5pu6^gUGvTXRbrvvcQIK0AMERJb5@t5w4K8F9pg zuL^Jb8Et}8xt#gPUfg2WRh=5Khm#Do+NW0?5bo2z?E8xK-R{dW30S`QlvdRF8tYS* z*y)llx{tRPF-J$!U2Oy&k0bWpT~5(!xsuX3R^a=UbLRsr&s_w*od^Ez0)N1qg!`W(1pdDX{5?&% z@Sn2;D=7s3`F@@cf1#ARw45C9zq*NonHkjawXKt10|*8*HEQ`%+euqVQN+X+!f9k` zYi!2p2C>86a!$-m1o#Rub27T;2C;!Uinxh0{B?y0@Ew1ci{aj1mpECAGiWQR-jlX< zFuNzf$<4{l03yD3@1B^0skz9Dr!xP(9rz{A@Y>1APK1lg)zy{Lm5zQ!_^s2TMCAOIzqYe7i=*w$4uC3=H@O{rBI$ z`f287`S+8cj{hDOFhDN+87>}9Zm$2f4csb*KPsYX>1Jl5^VAXohzEEEh>u%P?62$3 zoca5S|F~28?>l+8xq1G1>pxEYeyfI~nS-<~1bC(s=&hbaCf z=f92uf(8+bas7AFK*W*x_i4_ZlRPK)RO+SM`Srw0H7}o02N+J=d-(L4h}m&ql!>P)hLNmPS|0BFnMoR z4c}ImXfX+IkIv5>=hk2MwQuk8iF7p833tZrHaRgvglt^V^K|;rL(o z-4h7(@2{4UN)~{`@k1xIi9K$G<~@_V`1H3=ivLCJbF#fM2y-Jbi&`;iD^0mJ0-u_` zH$FtVUWZEhXP*TexutkcJlO6c? z7cw1!rCln7Q>r2<@3FzBdF|{e2G#55AFW(63THkeh4VKuy)L{Muud*C{*#Q}U3o6YbtOXV z!x%2zH<41oF=*-oohbn&> z1o~oH+9=4R<;>85i!a>(I&+=L{pKGgdV(2{+94I&$_n)HOVnF`QSf@4h+r&`qAK<=d|1mKz+! zdHK^P?JzU{z8Dx(t1rq!88L@&+j8d-oi^UsVDaPmh~XZLZ9T@k*`FlH%N8(V0$wNk zV-~aKbxVWam983CoG8L>i*zK5dO>I# z*zrPiEko#T4?XH#&S%eop0=sURDWy5BF zxqlo^20BXV7DRYrVWWBa8{dLBi?-^}J!shJP8-HcvC9<>5wu3s)t0{QZ;Fo(qY>t% zZhcQgz79y}@igm%ODgh9dxvb(mkdb|}o z@7N;@8P?D$w7Qtgrh90O4J((5taX7nDOBGVmBKI?ry(d&- z%HdGad=`@;Y%`gJDNzc0@r$01Zg%S56?afap$zWJUjsb+4r@`<#oYq%E^V)A7dT{N zuUDd6bgO1iHB%>7=Kg~IWL8F!6$;@~Z#ZktFf(fCQ_4{`*3{G?WCYsl;&W-dL^)=i zU%QBen?@+voF46fYI9Dxs@IDS4@M2)N`h1KsAPmI!V9!MiF+>UeuUnt$D~T&P6xN@ z4Lx^SAiqA{_r5SwGN`;qPU3d7oEl~`s_$CLKcd|X$dVRGg+6EtrH!X{YD5r{+_+gB``$Cj=x>st5K#@VMhX&h} zIP7qr{E`BD;rp-v%#kAT%yZYh#qN!_gmeWOnmR23UVxgRE%-595DgkW*TZC4t$`2B z8;Nn>rMG?`*_$gb8C7}ZMVQtB{m;>QFMX3oRMuVvohLwnhy>jrCpzLMI3xzQY22P zFbP-o&@ym0LnClli=0oCt1*#R@{F4OH55lHXFf9AUZjb!s*Zn6?$>sMopZcwe^*=se* z>#-79T+azD{#cKihM_GQAA(jQ-4A2(3z~=xc-n~Hc-oW>YhtI2Pi60>%6sxOnWl^N zd`&1erpriSt|g}o-8?I&ppw{>#>8Oo?w-lv;?^k{1xv$zDO^FA`lw<-SE{X=v!J=$P5 z`7~fIOjyjQz&6Qu<%o8l)@7_QzLW2fD*I?ISkPzJV<_(4dQ`q8O@9PD)hKO4DGVzH z-lx?MI28kJ=JN#CWI8zmLg+1~|TK0Ftqg*%ABGL7N z=2IgFnv>R~6CL}Uk8QcZdwe#hEy9}P3um`jt7!ZPaw54(hhCHo$g_1v!V_I4+SwXu z68xJldT-A(dK(^%fNddbdDYM;R{?~lWBo~pLpRZyQoO)H_HGWyq!k2=4GzrCItA2w z_gdR|hg@*^q4^rkgnzpr>Zn~{GPitsHOgboL_&nH-{@U$qZ5r#Ko6keMk z`L&0;i+ges&5Hya+Y&K$#(0Bl62x84}Py5;_x1^%EqZM%TtfhWIkmIbrQv7Y`Kn`!gkx@0JvPVLWu!{7Bdk_ITe=yL}F zUcgV*WibL92+Q(%(2}W#;(;7J;h;S+2#v59{+h;j{!?~fY9!&H>!amjd7Z7_HBWN( zVuuT3?KbaLT+jWtvaAKc;R4cRe2N&}(64+jg$>I=Vx1Cke6lO-qQt0MP# zS$(=z#zm8@N!4U@RA2lq%9LsH=jsF0qc9P(i}#>ho>Yv6aujF$!C%rTHmJ0v_Gr4_ z`v)u($#J|N>GJIR#j_`0vI6d4lx|T+^zTm1|GDk|CXu{tjf&%0P&nhIrqmFff?RU+ z9oiR0YMwd9XK1il!^=W8%GNkXaL4BlcRPJ^)H6Z_P?L^uQaGwek;x?5z}t`C7F1qp z)8C}2XLM<8+UuZ*ps&H~q4baA7$8DbN)YPTt?cSNYvBVM)*o&U4fc-F`rd~lNN)CH*SXkUf?I@jqS;sB7RlI>f3XX3i zMap+P)^Dt$Eri;_v70|y#Rqy)5>6MSDEPZxOI|b=e4AUo`dO(7wnbKYNyMq3JtAXF zY_DXOn3IfB&rS-xo6VK`7F|B^Bijwv9mik%T`7J-J7cEYa!Q0+MpzFB;sRd0xchs< zDM>dQR?%+>>5JNu-3EFjP&DyEmf!g8*_Ini?Y?h^+~qGKLNaSdFex=bY#DmxVU?94 zN;`%=7&CGvMHWFB`TVmhf#dq?>6%J}deZIv^k~%1;#&EXLj>G*dNRH=ENnR{rkQKN z-6*3*rKsP;yZ!QTrkZrUpg1g5k#qH>oQ1T+pZrONPhkxT1cqfhP`}etN0>!zQ9!)- z@&lRRf>2t~iM9NCOZ0kehEkl^c#`{}Fg$Ox3?Go_uDN$~c8Qx7C`avI&jS5EcEHHo zqkoH<0_`{YB1$11(C7mXIENq{i{7+uqj7!L&9g*PHWaQfvD?YDME0Z}_m*&DMmnru zOE$D5JAXC8D9hfzj->_Q6S_yXHtk`3uC-R{#loVLACCpP-Ei85CmS98~%aE zr1c24uFjon`ap49Ti<{Eu z8g&aDi(zbocL3GL`Q7EV{t(3iun9gGpX1XAJnP0Y zs6N0W1Mii%{;&ijcL11z3#9JH^1A~DG{ldGDP+d>!_TP4|9Rm55$XSrm8P#k#x|;7 zLE698LOLL%I7>-p{F4z$2g`!|4+WW`0vzWP z<@m}6FF}-x!1T_lR7&E*VC;eT)+Occ)kq8hB6%=Dz-Qx6n;_ z)Eh?35O>q{;u9xziAK$2d9eHMzU>3v7(F*X4H~{M&FlE6xDh2!=e>+oLeFolWGg-n^Y) zu-h}jSg|dCB-WDmct(-I`>*R&H!nH~NL5Cs3~>E*|0kJOGtsu+#YgyLC1C>^B)_RG z{%6WlJc#`Hoht9|CsPK9PLI$kQAsU2J%+x)j_;e=N-PI$=6-${`Js`mL#loL?^*qC z6f61l;_Bd5S{Sup&tjZ4r!@c&0=b10Vusu_t;VlFcIx*Hsn}O?sf^Y_wuF%~15iY^ zn~cCL0{!~uC#4RrGlmKV$G6V}3@pNcrYPh+kG%J{_WT|8PwkW&y)Uo>M9!$N*t8VN72LbWzhn*Apa zb>c(8#V1jfFrr)U!p{hnft&9wz4slnL{aB`vN(Qgb0D}bE%Z6%DQ+XKdcauwj#ypi z#&6<$E6>1fKljvnAWcp{p&R_TyfuUkey>K^(RG(<$#i!0Pks-lFmCFFcS}_d{jl>B}cy z<>k1(HmSZis{q&s4cNUpt;0gV8PtZCL2?0T3n!k_$M$H2Lbssf%inhIUn}4x@!P`< zxR$);?xJ)?_0qG4WM@@?K~?bL-2%D9%p-jGV*k@CTQhf+$)$zi%%tDDa1h)8z%Mp= zm5aL)%%eak!df{h??FLx@~4Z7!z2?oGU)3MqXXlb~(7r#%Qn1dMTe-CCMh0 zCSRGBvWQ73&eRZXG{j;(QIb`f>^i+(d*bj}v)tcHbaFAVTt5mCd%97MSkJG=7AYr+ zXxw68_uFj`u1{p_M;+OgEYK8@3hQBF(Ea|e{`_dF%ENtAhV0ChTxTUofw(;ug`mV+Gu7q z)g)v+akYu|i{oYb`AOq;TXjny^kTei&euU@1Ptc|^I2A{&O?xlcy3l+Gn%MWmxV1v zNx2}E%7y$&3PYTI7Xs)!e#Uk8cd-?ZznQ$`2Vd&QUI(!DA}L>k(hm_#x}%)>-we!; z@A_@kT;+7J_k3+zdr*d8=&RiGA#i1=*e?PdPVNz`I|^n=PK;`&R%>0Kv^{ZWeD&)S zYJDl$r>un%+?BCZi6vMcfkXF30i3d&kluS}a%*vXK!L*y2po&VvGWmSNK!a@>C|;} z3N;pMk!%8o)@o>e_yuAw41#Y}4AQ|)<AV*s7heSQzfr>y>bGdQ2wdq#ty${GfVa1P;L*jFk3* zaFe7W^LA_VOmC~4*A_79|u}1=T0k&}< zxIU;NQ5puq{?yd9zl5GBO%CF$8uIr-=c<*f%1~;*^2X`Dn9_Clz-Y4uDRmOYe+(EiDI+uc4sJ0KoX^#L&w@v_|834;FA9JYTDCL(Az(|I`1 z@HP*cy9A#cUrMk5Ln_nNf8o?7CG2=--uRN1K1-ORTLtT#rrU8DP7zz0Hq@_e!timX z#m3}3C4};L4+vk6t4cq`KZf*w^Fw*Jr$@tte~AKtcj=S^oYwoDje#JQW{-3Bbt?L{ zODs! z+XQUJxWO*tW;ykPzR%C7f*4 zGq8|=WVb-FB~Ru|!SUCrv8UY~B&@IUA?0O2-mDqqO1r4xm0#*Ss^2#4hE&R6Cn*jQ zobJnYz}aXWR(8oc4m`?$i{{O_fJ@>F?|T1gB!f6`xa{3 zx`n3KCTv0rb1l&ZWk6!F3+9Ch5lnc8BP-vAW_h<33N1Haik+=#1=}0m_kmzrW-M7O zPhcCB)n4cgJNPS%Go+KFoTUQVzhZZI(K)0b?lIJ+9(xjBc}|`}CCaIAj<(G>Cw_7j zj=EbtbLv=&3!8e{nwjj=;qcWUeK>a{7iU2bVVXg+Re~htJB}Tu215%g>HA)cbJmuGX^jyTHvu~z@W-3<2ycv~hV74>VA7#@_%CFt@upa;M%q9NkYF z$;2AShAE{EdqQn-+~4BEb$|m*7$B+5RV~6~Pp0-y{M~T^o#qxl13$YAWM$~|Zn$i- zdCrTj-uc-_ibnZN>%0=Pu@+$@(CTq!lWUb-)1-;^SsjsJ!gWvH=D6IUc=uAYW;v@) zD|35>kpEkX@g%x2aT)a14ZM%tt;_Pb?j1 z^6akS_6|fgrSS!c1AA-nM#E1WaUvAzeaV`0! zqd!FGKAPO|s=sL_b_{5e8dd!&Yqp!gGIiqubz4UJkwYHX-sDlt`>N6dP4S{H)LXR@ zL$~cBP5{Dy{Rk+x==K#w>{3T=T9rl*@<>(d7MzP?ow0y-M8U zcFxLm2VB7M)|GNAGzu{|$?8bCHc%Q=OHn*QN3BW%Et(!H(old+<)$E2?^e0uEZ%gu zc7aGmj<`in`=l}6!@^Uy-(e1?+LF9??ctrV_x0_Ehuu1>wi3P#LXOI%U@LXxnEnem?}zXsO)f1x?^tm= zgt)gTk*1Uy3fo5}mXDG+YETx#9IDgClFplSgMrlDNb+-DD7`3+zapVZF9TGaR_7&|ADTrk+PhXsR~u z)eQV&rapEYvH030+Wuw#N}-G)>920)f{&^JXno2B(V6JHF6Uj%b2|=5K46SeX=u5# zpivkhkZIMGWZVt*C$Em*#UOG8a3}CBjT(DZ4b&zs$5X?7dyA4Z6nz@RRog^iUUI!c zmew&r-)D-y?D>*kt;*d|U+C7#A+%l;T^_46^?m8kfQNmN4t*c$GiV&dVFj*?O;wtr z8|5gQD3hOfevPRxZQ)}lS36yU9}$a>5bNP3sL!||RF(49@Fw(N>pe2;pw|&+J(l@3 zzEHObWLMmwC`Mz~;9ZAoKhUy5v|)}j3?Zi6F7vYil|8BHa3uO8$HL(Rq2Ts{mQ6Fs z2#W7Zm!84&60zt?8u@x+bt+=2P!gCPgQ!r=EofDgclL;Cs&zGcfjUc}6bwjYg_ASHazDnWOy(0#Mj0v# zSJ$2TVM>+!FnR9IfZ%1V54@nPL>@K&wbk2{bMxxuoKutU_g~ky1Zd=PZxZFqnR>cu z<~gr@U`}%Sq9QTY-$u99vrV<37|OSq5L^bmVG{9)KM46GUUa#EpoYHePF_h9G1PX4 zu+^n-W?)sbvg1f}s|s#=)pNeEt@aYxGPqVc$H(~wG<8s85?O+-c<+e-RTkTiTyxjbCJKw>%h*B*J3a7c+ zEiB>GtX(89s>!CiiNXy>D+DRq(lqee0>{r8Ajj?0JEokYjFv~yh%n$7b zCTTUIC98OFYE@)$6(PJpj_TcO!5H}@dmY9mzoyzc)b`pP&lo3&sc93#mD(bvuLcE7 zp7(sb?_h7w{v4JGrrB+s)w?1wIVRCM@eSG(=t!o@ZkO!2naan5`Dq*%pMY+E)TdD8cm-O0EF-!R8L`oTL6tt6a=+|!h326!n{y}FvgWiX zUg)%@!*tMwFr4|kob%P#XBd%kl+NasO|SI_#@nu%5-=DaP9HTk7-Q3HRM{xyH*CM4 zT^?@gnF(GO3`0JlrCwzxwkv5K=~?|Ik_SXN37+!C+AR4FQ9Je<*A#s3AtUC2W$s*z zntIO;?aR)0H>*?CJYs9BYH?}F&Ra&B-3;(9YOx0vSnPWeWPlK70;1XUL8t)wpupC~ z-8Jd?*E~Hlu@U8c*Ha@-&yV2shD%h*kvJiLId9}4aSR|CItt#sA zmv^$6V6MvCaP1oj&aErshE+xi(0GAdjEmN7KLa0(l4}E0%Yf}yRCX{Ii{tS2%M7-a zw6z>Q@{G5^x~$ItY~LbeZos<;%)hZjO6xZLs7*aX8BnZ=3iZ6+#P+q@ zfK~gfEX1c8x2~AP&Bk>&z0oZQ|JmcrY(r+3y^ugx-)jIncB)xcW?Cv)avm{&v&D<9 zmnaB$wb443fZY)Tp+dW_EqG7-_-ao&NwKcq1=k++w*1J>GtQ`_7!Y?s@Q3vdxOa7| zo5$POz&2}>t#cqXmD{rc^sD4>(G^k z(yw1<=NA3YN>c^p+tpif+g7`WXhfujiPh-3&-Mu)${E}i4$Ra&-*vf{=K7sB<211Q z*F^E-qttQoO&*`0asEUSlQ}~i-r*sGox-qhV@2r#JWRv6*;90_!`((UJDS$LtMCwr zn(@57jqlW0xPL?#d2eSS^uEtgv;A{+i9u@bn|#6aa}}@b+f@$eoc9a8cYO`#19eM| z&Z%SBw5Y<;t?aaS-f!9Sf{u`Yq1tz^TXTrqk#lAf@eb@n<%A-E)(Q_EpX#b^YO@Y; z?8i0soH)lP;dH#GT{^qVaCPVt>4GIY@TFrvCU5@lB6&1sp3b{kfr!gcjUKMfoksSX zux~;Km1VTq`Wq9aotwq14?$g%8P!y=c0XJPL=#?2RUL@&ZTLLjHH<*VtfabWx~B9P z#1!~cG3|UoiUPFLawP|eu;?MR7pS8TCClghJ&WG8C4DLA{z|iFF!h+ZFwno5C~rOF ze@B@$RZwqZ&RZFM1TJ!drwJj^p*+8!{aPEe(!=a zp2$x7R`c;6@YYZGtdQsXeZ4c-Kai`(^U6goAQ7>5W5DrD(g0xbKe!OG#*!rR!F*mN0x}RuHPX-`AY1G|VviMGeQ5CP zPq=Q&Jyjn0lhod@LMh+2iF^~>q25F()Oam#WWr@E$BRek(X|_yuP=ymF2}~zTPXW5 z98}W%R_bvpK=K*ON|?|k%7^Tvd8nb{89w!T!B&i!Yo1lG-ZDD$&Qo0g%MnTiCa zq!a=l0cf&DfR@HXRnv#Tm6(e9Rn5LsVvmMiam~>kby_?j`!8~~No#Q7@nU>$sOY9p zHTGmJoBV12Y2CpH*riX3kbAob0cpCxmz;mH{*z-KCyH3}Sn>c>{o?d%SI@Yus~-)WB-stYPBs4gbwB zK1~U|5;-J);sLAC+OR=5V8eCDQ5DfVJ{HLy&XWLEY!a>K@WJpHS*xL?Kg{+KU}7c% z{G5)~FnU{nU0XvIPJy~ia(C>e0=p3a)FqW#4z}X)Qm}cf*?O1+Hgh+T!s=G{r~#}y z?y;pYwQWry?sT(y%QFVgR8j#Q{~XbG)t3-{6YguM8J-M4PBtEaF&22?3ed8!q z8xa5!H@Io@c*x@55dgBn%(A@DhoE(T5?pLN75A$*9F_Cohpp7MV*vAV1LKdolI#R2 zXuLem9Y}B~2Zq|j8bH0<5zGClA#$X<^+$6-FduvcLE)3S4%Y1VGbR!`Z_Qp@BvhX? z2@^L5xXUGYPN$F0tRH2tQ%>#S1hyKRE&r+osg>=K`o`7^wWaa9?4FzHKgt}%_K`<;;i#xz0Rwn03DyopCP3Y z>f5YCZ>|?JC%H8xdG5;LFkb*{X37QL%{Ih>tV9sSN6p-q_8$X;Y)eS*S>R*amm=i? zaOEeWvKSLS_vkOZ(XO|3AmWk3S=nYO$;h$@1_BMiT8jngi!I^>2@>O zNTEp^tM`7o*p*k03V~I5vT<$5mg#e5bj4)VW2a?R31Mk)C9tL-fcv@AA?}S_eL!w3 z2T2F2KU{{?WRK-IrZWr-0O(imq|AH2UzXZC6Or}-!f8|a%DM$8eWfvxHj4g zfQ}^6a>kcVfQpEs*Xm(6;lxKD(T(WT1)O;Hy*+po1ax+MA_wAK5$@ zgZ2Z6{9j|oWT&&WxTIGapJD*t>u{^sxWh2+K@Yi#SpQ7F5r&%Azo@ z!I{+12l7SI&sW0Qku6{)m5;=JuGJf5#riJeE%~M<{W}0GJ~3{QykYoZBz><(wAXeo zg>yDjrIB%1mT7cuqJze<%P8liiH8b?p2zsTg!Q|V$8OuNY^mObhaOyuQ2UDcHa7UZ zV7`H(IJZxr%4ub8jnBx?b;@~q_ndV!4zuMmDeq*(_TlyZKho>0?HmLuPbLM1J|SM_p23)J3-Nf^;O1j@u_Unbkk-*L)Xl&{fu?Ny%1Ie z4__9~xvUqqO#qc5^Awjf)2!!I!vSWi3fSvlxH!K!AmOd65>3aY?;9-iIoyDSMSIVL zz39Yl%SAA`M>vDj-r2L@EH%&POiKWn9q)w%>wzx%?*~RcoU?WV9(iDB<-)1AZX|8j zh7KcRj^`$jm_iY@e0_eYEFM~wC6Pn+?D9gg_@J15011VPTN?=WQ`1v3?dzBq}d}(nqVs#{~gM!bDb&YHCl`; zRPO$qfuWrPYa|ipL&%>y%h8K2iK57>Yn>WUGs4y zs!v5Mr8v>?CFR-%wjwX@o$?nQ4iiIG;)~fUIW>h^cNcD`uTyAhH-hkGDd%p(8g*#7 zhV1XrEEb=>zq7azGjUTUF5Wt?vZjc3DIf}T_|x&+2!O?{40>&pj@~6E3WXg>QQgC= ziX3(BCfXKVw)EjAJFXWT4+upD z6RwIcB)!CCl`HnWf;35m(DVf=WQSQo{6rLK=kAG=eatBx(Vf2F=Tg{k0eAY7<_pc~ z#ryVvyOQrskth&9Z=RmJzcSdy4PWxSR2Ig4-v1I=C zK&?e_0OV^4AC=^tpMC?hI*AxB7U>!VdDh{VW_T?%YR`J4hLfB`X za%ROymPn>Z_R8G{l9)77HQYFHqZKchcaFw_+U`2dMzHjitkEGtZ1}2)F8Ds_#HD&g zN&VEM+>!x#@0)Btd(Q{4Z2XkdH{lKR-5k(Uf)f+Y8da(2DH0deUbMck+r>|Q$)2+$ z;uGO%X710=1LA+X)hvb=ND|(y@GF9}hmGxGV6SzL&22|s&_(vA2+2qze>#)K|M=j` zW7O&we~Zq%Vtpky%Y;8Xxgd^0_U@Gj%2HweL6u&c^8_6#aVvKFmSj%lKJ<@qZaK=21W#qQjVp_fb|8 z&IMHMh_28)56r4b_#cYNOJX4Ghwxov3}gQ`VgIpZER1;SRMz9j+keaUKh!A?B_Oh@<@7c)I1@glMB=Ga=>w{|XQQW8 zemr%m>L|+nY?CkWyyHlDY0KYPM?icuiZ32so6}nS+gbb1SmZwdGJ3&ElK$*$PyXLM zI4SN4+<)E$0EqJcy951y(crk@k^4Z=T?ddhQI%)6;^#oNg_Q$zq)~v@^fG&A=-CV& zTB!vJ%mjeqk*dHv7JE}(DzFdc00NMrKsA8>(kl~2ryp5>yl@O)Okxj&ypCoAWKQ{= zWohi+i(~c&`MqihxBst();dDV@)$eJa;SkyR1QWuGC;BoD#He7tSlhq^h; ztyGmHA^UG6cOU->5C`Icwc%_nfuv{ykM{4%iE$WEHf4W>Ie*TCrw>q;Th>e}dh1q; zS+u@65V%65yscNW?kv~=VF5fs1+)^u?J)lw4a7gzaS5^0*&%ghWo_1cW(>U{bzW#) z)aKL%(5rHMr7vgQkuexjI-DaLMprkr=sqREDI>ht>Ig*9aN(td2$<*&41(e)?WZ&F z+<68ECctFxR_9s699mN zi}VYQXdfIp87La`z$1T}Z2f>!99?_5+d1r!p`CO?<{Ajtq`Bzl0k_D4MWuA1M+kWh z4itxn^aAD2c>04Bz6R3jap4XYy#)t=kC$M%#7C1rY$KrdXmzUE)#aN>N93a z3Si4YS2HV%ZkXb$SBdK8!xe%*RAidG3j`el zkVD{f4-5maDJNDkl9LYF-tPjD)LeZSt>_5Ar6KVNIW>JH#^%7tk!N#3etQn6?=A;q z$0lG_S+oWC*9nX=g`b~7Z>Q2+2_?G2^uZ2~73tfzP++U>;D%)k!Ej{h>O2f6^_1~Y z?Q)%Vw~*$+2aK4eR&Vr%#Rj~*ms+o|gQpyO|G|$``e7Hu3xHDH6~prZ<5gm?#PObY z2fq^Ic7geDwMv^@YSqfIb7;;bL)cbt2!Jiqqk?-^g)UOD5v0!~xy=ztr;L1u14JN9 z&MFX+&xpD+Ju=sb&FbF2ee)2A%5|*S{QZ6!LcKG$l;R%mu+=IKz5E6P+*y3Jbxe8= z4*>{p(Y1(>{?yO{C4upu2EYp#B(@vg5mGTQUszDQr#Mn%9O`$=SD9%f$zwIkrtAr# z@t)^cfsuD#}@Wh%_M^Rns{qch)38+#kM8kPJ>DiK>N!Xzi2bqs!J~)uHwgpbDQg;!>AG9^g-px~<(1Dt1AmKBCtnj zE8PSgl}&~aA6;)AiYcq?+*4BpfJ2Nq7w+=3E{j_6VLvD})QJl~B(Y&tcl301<><|Z z3r%FJfjy;}ZCvH@05zoKGybg_qqVQ{vN?EL?!hkcm#*i7-lz#{uC>0ezL3I} z+l9LUF26>uXimD(ho1JEXO41N z_DX=mwePK!vrp?Y{N%*1xy$-<*6t6~vAZX0j6kQVs5Zn$d7c{r@Mq1bNz3D$qTJ#h zk@0OeLch%zlhB=f=AHY7+f7ens(cmL2qvyNKmH}EnSp=p2ae8NDLi7@7frTsTZf(o zEdC9wuq;ibGBYryhFbCYTzjp|k=8sdrucFY(L5dIE}I2F!OhI~t%(?+qvGTC9s`z0 zuI1o&&`06LnC1di`J&5~0E^Q$K(5zuFczJN<<`xQxh{S984K(c6Da2r{f-C~GiaT7J zQb}Kcg2@4`3|-$V*?y8}qW1(Q71~xQG#(&fYV$%QVpZ|6=Y&dclDJpwgWlK81capf z2HzGWvCB(bHQP^Yvt;XAg_u@}KZLp7X`UOX*cQv`-eVuVA?Wl8HHBa z7h0`sooc>tg#d;&Bf9$+UCs*tvJ{r+r)VNcmMfBg(7<-vnc)C7ZPqJ}(ksJ8(9cOh z)eh&W$lde?wE1;Ik2*)W`0|X|Y0P3@;;w{kTpc3GF?wfmfisx7sdlnAO(2l-8Tbm= zCx+%#0##1%)SbBSlTFpQyD)|7tB*h_sNykoNYu76`-vxHM+$&2ESDb~;(Q z@=swW?vvZCqQT&HSv$Xxh#8+Uud2&4iP?{-m|yYbJs>N$Iv6Cm$eZVw^oVqz0C>fU z`K06nB=J4u!+aHNv9OPJo26j`6PP+aoi@b`7UBaTY4m1Kgzep zmRsl%t|Nl3w$;yI9f{S<%copAh0J~;90*o|sWZ(d@Adh(67&RHM`9<4z6M(-P-(gI zLbqe=iJvV3fYyHRB&%v`?vO`4R5j9`;2_xgCrHnNGs!-JIQEmW)-0bgQEx?iU@)I` zPX%$by{`j(kD4rPl>K57m0)$EJqZv@C*msGLxcII4&>|zQiJ3BelfM>-n2-#5Rcis zH4vOVmLzO8b6JQgL(=^Vv@gk?IFE~Kblg0FH_ARl2zkpL2Ra=AELO1fOR5a+BMW8s zgWiN=x7OSOccubIg2`ZVc`8A-U94Vn*Y#v|b1#~jc@cx;+pAZ4NI-nh3vGs?vrB^{ zxG>NO3-Yd0tJ2dWiy?`O3be7kbJ+cL})uc9@IflIm>={9Ox?B5qCnhF1z}%(r6B5Xx7N?gmol}(<=4LywG<;D2{zOyO#T%S*|4Y|< z2~>sIRuA#D>MIK$c&Co`iv+Siz?}AP4H){@MedwmXy+^61=cu>D|xRa%2rGgo7fa* zA5l=3{B*kFLx@e%9OvZicfq3BO=0nOaQ6)@EhEbcZDBKvX~;~Qb@jwXJWMX+@>uJ~wU(BqiX;TB*k6}cuWY>AanDl-$5pp+ib$-46fcwFZN4(h1a__a*vDUF> zy+5Dh?O;W$FMqU|?sY^m{W2d)g2Kdd)2|d8S3#476ma&f%Wyd=Wcl-_Pj;eioU-qCX$aUkaBDH6`3jJ{Sw;1%BuACy|NO@_~e`4gE z1RfCXB)cutjF`=KX}bBSXL#WaE7|JhxI^;I(=)bUd55KHi7p7S)lSZjCJO;hP>v_)@aMB$1&QEc)8=6p%0`BB@#K*`iaFQ}$7 z_w~N0peQkLu)O^-9xG_!EI*~d^5r@(f?05XaWoM23D z^hEHNcG_8F6BB;o6N(I?Pxt-%8&!~O%nl?|>vBFbJ`6feP(%-~(-@!R+)UN^a~#eWKptp7;(%9^cK;;&@uZb3%VX@> zPbT*yDdi(@S`(Sj5*DNx@zou%>4oKKf=2jUq|c>j#<35(NuLVdWpXMA(r4(sFT`E4 zWXWJx4n#QG^uhy#`XHObyjQ$XS`m@d+yQkT5`vt2%a(&;M#sF*Z)stqk+i=pZ&wH< zVGVMg%DZ(wjw|^w*ct8j1260~oR;(R5d3&7xip0ik z$Mx6k0b&L&`tsLqoE~4O7L+}#e%E8C5-?<|6!!@Z5Jt-gCOX2NM=)U^QhFG94Xn)C za%JsAU#Di}u3M*!`DC@dTgQxPXF4RN52+MKy0g4)!I3UX1A%iYskG(m-?3Yzj2)V_ zVGJsZZj^O)ncI0s=Hr{iaVa{3k#^jO|&c)Kh+LR+c~nzcb${p-Vomf zeP5T6a#w~-t392(*U|=#6=QvL<>S5mNKXD`iviw}c}F6`7JKYP+SV~ju#y%XwYE?m zC>4yw_FhSgAhKuJ7{k>c>*MA`!ZSBp-+U%yl-?J)f%>FJoh}_r-D`=2&Kcj7;4BlV z2qb##4Gl-_fYz$EN3Uz$+`~Uuy7s<7GP)eCMTc)jhym55cC+g4@Nlg6IH~-wuEp9} znqlS|I%017X+r4&2YWk3*Hh!Gmxu9N9Ino2tIU_|(_5>7p}cfBH-8Mly2(R~47H_S z$dRrAVEegsbi0;fK4hf;wll@)u3-Za#TsAxi{W8x?{WUq+>TG(@Ff-sReOY2|_c%f~7zpPdZ*WXhFbOsB=JLkQk^6(O+4 z0Bd4uN}E1oD2MG;mM~6iy;?J!LDN~K+ zqy&SKwsM=X^p5|`S|3Ig8YE(Q=02?(sScktX~fLWrC35HvfNUcZ|PR;bTO^^lKZR1 z`;Z#Hi$HRPoaXYcaGzvWb{`8XFuYjB)vzny7=DuDNP8I@5LBJw!-~l}KwiI$&UV8N zdn2{HCSU-A;X5r*UE_N*PS>=F!AFdH&)6LcZW{e##xxPVv=F8O8pxk}6Dg0@YJ_@C zW2^HHXfwuq95d9=Vd2J9WIGrh)S8q)?&K(&$4j0E?=}6S;BvvUP^{_$mz$5%;0=R( zwrg4DNMeM-s4%2%ID1_avKEuq30k^>Iw1wymxr#8TlrT7J>bZMBNl1i>qozhNpuoF zcbd*A?5p!<%rPi)i~CD_3r&mg-~1{;nXp-|=ti92Xtf zWr%2Mk!Yl?oHxc4@ZXwwLBo(9>!3dKD~5jGOu|muqJ*ArD$rmr|B=|V$z0$1*ajsO z@{)gcgz2L`(eqw}ZVOLnq^oSb{Za1y=5R+eD3@2emz|)6NH38fieY5h(EIJX)_~d2 zO@c^dY(RTGmaV9rnK$tc-AEiYZ+|Lug+U2=`?g-KAk!2Lra(+zklU!u*E*ky{SiB= zXH)*E$w9eh#9Jk^Fy=uI?5K~j-EASl32IuW^DC+oq_NMXr{L}SdJkjMnmH5iWiFB! z;`1WSMxRBC-IFB>7mHucH9ie`CNe;6L=!nx)jk*g&TX6XO)N(nW_0_rGb?4#Ts*;- z1yP@O=pYJQW3l=#XqXw#Wbvcm*OQ>7={<4JDmYOH9zGdoROaDZ+>h)7%unO*ldf{J zL@SOC8%}~T(a{rGqJ`zbLU!MK_hPbV`BP}V4K2h71U?hf7JIfPKoTO8HEXf`$N$M? zwIiseu(Mn-9LEoo-)GX|=0E?2()%M~iIqFO`ih9`ZMiHO~_*slUL0SeoRiR!LM}@?3|21+%m?XO9SE@>`{yf+#Bmjgq-bV|UKW31PnKAPnu?8oY-o&MU2}_o`BkLqWXwML(Cdx) z(lqG+u_9>vy2mgM@{9z;RTy}+PFniRjdpi~F#S%WJf*iHFKr*gu zCm*}Om-c|37B1eC7lm=yOnsJq#0}=%WRGqx4C0${SGc^Y-nV{l0h18763bk5P9GV) zpsoAdlR|*8NZ;R;>+ZBy(Es4-oksfb8+fOME9Kz}_c1oMNbYHa;?qw}?Q!cg?wwjm zk5U&F-dna(%rUJbRB)pnuk&-3f}A*ze|&6XE0`-kAB0sQ_Sc7FaZnFYDDGy)F03b} z6CH{L&(m{#L#pda3Kn0nwm``-y{LwMAW1AZafxeV-;#9oc5y{>V&l6yIBHNm`k7$! zzb??5V@BavY4t3!wmS~_25bn8Q_D&}|^ z-SiHJ2`6!XBb%q34m$MikS*jXSFCJAjDt#xVbDEsk02i6rAXhmgB)I>X95Q4^*0GE z1*^QsjOt6!r_>+W&OnZ}{5h$b>nJ9EWYJGm2w-`O$y&DGz+?Jd=hT z@+&apuExQKax$_L7jdRRETv44fpoVHcLuH{U01EHW@`7cSPmKL!$7AtM@+I}rp_O? zlAZbNxe7E&v4!9r4jk+VbW_8x; z(Dgd@v{Qok@Tn%I;2!6?D8pjvis4d-%MxE~WO?W7_y%Hv?exhaA2ZetBiM-UZyd5h z(UGBvvx2^@Hxw7yWm&>M7&$rHU#DLN!nb0xBULTQkB7%(2xi#Mo~Y!C*Ezh+Z+qVN zbw9E}fG+Sm zc$`h&_=tg)7ov;4uA4*dBgP@e?|3E%>w+sRFxh>AZ$67e^(SQIYBIJw)0nYvo$-0c zXI1&pokW!GGEVFju!$f?WPNZU&Y-AE0d5IMN8Z*}Ni2s?XjUDP-kvhC3AhWA6P)y6}8XIX=Ou<0Me=piM zs=?pnL7Fgp@t4isXQyaS%mJPB z0f1q~q&f*ulR17Rfc-H1H1xsD!EUc0hCJaZIxLX#RT<`ulPInlt*wixn!`hms>78K ze@B;O;j`=M14{fCWayAWm5`5O${dD<4?Z~=x@LH*tj4?Z#U(Y3y*6CHcG-`Lbg)Vs zY8bCrP$nZKP@#socG@I)eG>l&Ik$KO!y+WmHy^HdyL=cWhIO!vk@v9Ym22_gSa~5% zE`AWh1Qz;%ODG8QkRLISi^d3Bnr&e$lg^2SEK5nrQ8|M7uHrCe?)h%0IpkuD$iG(} z%?aQ-=hUKao=`;$EKsBnQ734Fj-}}mv;)bPj@YGcaz-GQ76@9nTm)pkanbX=Zvgy_ zsMRFMcOg8L!1k3^`viAMwT{5Ej_0}p5;G6XH147^RlFdiSX^b@Jwb-uPGjjc59dB` zhye8|Aeo^!2{_E(MNcJBV9x-fsjo1cB&cd|-p_lvCV&Y4`FO1j>@^e%k&$4Ook?Ag zsL`C{YE5z1B8DXyF5AsOsqwI1;kh#N*Y&X$ZtBL_>X1F~H3&1#8{nUs zF_fn8Fm@vQSKd(UgpRiwDP|(^4P%S;8GqDBb zut{HwoOc;#J?rfvW%tMooq((Y70Vz@{^b?Q#1kZT-!yKaI`qI_t@3)yu;YKWgESl49J zq=9u%7K-6#-ZQ6GXz9cJb>|4*$WGr$#^=epwtY z-9lG9KOYmN1~=Z5dOc*<3wn+WOS?9@_oh2S#3Y#D9OUFoi6-^XwKU=hIlf;etsLK8=jj0}(@t}p#e~v%&(E08`L7H-yYq4y&caDC zY_a*NHT|5%C53aL`TU5turX+3d9H0w|4Mba6Py-;SrF^iVX{b_G~f=gX`1xU zHG{L@wQ4XKZLeFob%<&`bEiB;+fIz{uWEVdSI#rCD6#TWvOvcS)`LsGH}iV_G!di zp(3KCN`u4(cBn&WnPQsKV-^{r7Z+tC&=ss}GRN}p@OntHvLgtKSLU=J%A_cP>Wb)n z3V86%w)fpPM!+>r zu0-{J@&A;~1ne|Lba0|sJe_2-{^H&M&od;X8-?T<=foo)x!bOA+#DhT79xz?b?z-~MnVS_1%SQvlA7in0eVgj&EhU-(q}*qq)Wbd3>- z0=z-00j0#NQ~QoUEC9Dpnf^YhpLXmOwX2lIGdzi;_C3#jyEnR<0#r^=x(&spGk1UY zjq(;?3@vh`VoFcF>)8`cNU0pVa8CgwX5Bn*yEVnB1+ikQ)`LwcFyVEwne;M(n>6!8 zZIQU%DW45Ski$k8KVmtp>A}hD4LqPE0{|MJ=wgJ7!8SpB=zjoJgB7j*YJmji6~Moo z0uqwpM6yJMM050R*F}-5eR08+SnIC2uO8~a_;<2rAYVYa<&RO5@&;SlVK7z}UkSpp0F#v-wJr$Ha2EeFPGwq!= ze0>^ zm=m3|k&(&JY@=os~q4NguIH1 zOVj=b^da&GID`9I@m^O?lbAwltmzmbSCm$*bqkzpmI?!@_E2FK{f8DQP4HL51_fF*Cw%?9%+0F%7ba^I4>6&<_@Y>+$ zV8&z#x}!%Z0d>UOizGt!S)gQrDWDe)Dfi^hBgifbSY+EkYAafC) zJ4^%`0kipO0Oc?d+^J63X*wEl1H@Lr8&&XLYQHAHewu!I@2*hA4saen%?E&BR)7tB ziqarLjvZLlv4>DkiIs#;5^aHM7mG!Dfa`M&&=sbwcXF#1;qs2jJUQZ4)RPoq1D~-w zpkkD>3lkhJ!&!;DBmQP+*$J!Ur-eD|7kE;Ow ze5N3zGF$%{FLF3Tnnrh5HRP)xEFj*l13+LhEO0e4V~LAHk22i9>eS^P;#jd9C$#(O zQm#P}&5Ob0@3(%H*aECYZ5*9c4|+tTwrQNCkFfEo$YvkQiGR)OjHdNW@2llI3*#Ex z8ZQpqitWll^AcI$xZdzi9dIW1M?$10iRBk~qP>ePQ+r56LRDn(U z>VsM+Vz=o?ky;a|8_~4d3DAueKpmGVWqZ2@Bm>;Q!-;a1t#NzlqUUF8wcEg@m6)NnP{?{CPhMt9^n%JxT1N<#m8N%e_)|m%K920FVQ5n4DiH5O1tGvBuj1Bp5T`P%;#qW4Pd{@e zJ?HXtEl@Gyw_RgNnSEEtcM}Jz9ij6uu0{RNO*u;ZcZ8B&p8XelT>yA|qpU76zWMGw zrTl=Boj}6O2fOJ4M}GPcEb~az0)BnI(>pys zH$PVp{x)j{IIODYw(iwybW=~!bJHe_(j?q+{l=kJVw_y^96LKpGdou8VCn0Z`y-Cw z*!qO*t;_e5sTc~R(ssRcXb3)m}+_N{@g?Kr+1 z1xx}5>R_@9s#^dIjST1RX%lw^N*rm(rLdKV?EB+H?xrMcNNaL>sQH@T8o#EyWr!q! zmKpO)MJs{0SI#3T{MKIxG^!6BYZwuwmNgwFl@)6gYpF;wGT%nGf1(K4&@$EMTu9nd z0a>mbDp0zRw~t zLX)R5AlXkYnYHi^Pr`CX4RBaR-VbC>+zh|U-U$`!#2P&csEnBBERQGczNcdTma5I) zsjDh6ha;~<+5%K+>i5OjAjm0Y)M~k>r|0e27rOV|SyC;NDP@WCRih$;Aqr{1w`2Un zqwQx`N26sOUb*37ADPi}oCL|kCEO-^8ufejU!YffQ{MQtQMXnZcsp^HANasP4?CqXvFE=!n4(#sDam$}t-R(?M@*z%A2}A?nZXmpMS@mQ zZ@ZB0leGh^1!>M{U4oAQZrGW5z*6U8tEd3#*(b<^e)*+0gWnr@9g{U#BSvcCw>~GQGu;u-CYmH z5U#k++-nKcOqmEJ@X@(~>v^ef#FQa}K7DUP#Ssxzd?SG9%^_vngaP8bMSGIG*zB_mS@q^&wv{M|qfTzEIFkTzs+Q z#0)f+5{Td;fe8NuavI*lZ|t{H(PWHzvR;~`Er9iLB|>WEf=<{4JMHiw#?Oa4t4(+S z$FYDlhlFnCC&2WN_9GuiTGvEvjcplvbDg~>6J3wkg zVNu=RM!U1#Q}2065x9dzKyHNCoH;`8&?8zYR1io$V%ivGu86>HDZ;As3r0s4G&wL) z`P=%}B!(rEasu`H#Vp9Y*@a9oUV<07U}QS)paDF7E+quu-!;EGtqu$kq6ZG&d$E_7 z4=U`0ZWk_BlyE+!F3nE~&ndw<#)T?i0P7u}yR#NPJ#yQT`P(z#K$^ zr!>=7dRi1CSLM6x-e1td^-8-zmV%$>D8*e`xU?=CYOI{cRK0r4PogDe2!iBJ=bu*C zv{-il>%_3+rmf9&lYVPu=Q##GdfitY$>18tYTgOlO@h6osUbF;;$@|jc*WZ%Vlp^e z0X`zyIzPK3Yd+U}2W2gh4)NMd#V${%k8V+Ht_#gi&IOtM2=v9+(MgM+(@aMFb96V( zxREg7#kS4T|0u|1562$8pQ6l#x3&fo?%kH^@WDM=m#E%|k|4CH2PtbaAA zGDMrk@v{t;T%gSoIi8R6nJ z&!g0iY-IHO{)-!!%3fKb1AoswX`WF~w@4b-lDkXz`v92iO|6%5qt;Seo);-Yr1Hbc zFt1_HIeZ%P=Ta%)Exom5SW@L_vwpNznbrWEXEf3g?8CpUocRv?c{(tGUKGeAcfskU z9>)6A8M_3KwdtXhRZ5#!;*1r7aQ*O9L>zUXrYWX!6ipvabJ zl=~8n8Xy>yMIn-ech05so=7@%=neS(?y+Q*>DY!hkOmk{>r#v(c~T$CU4^%A^4NQo zRhWkHWZ7kkR!2pM)i)Jmm9okhjS|igt0wi&d}A|bePx=8z4}!eG_pFb{6lcd)A6^f z4U1r19@KDp*%YBEg&LrNj+7ox%(sefZJO%JNNK)roH^}B>_V}m4R`VQz_J1v6a=hp z-q`q6U2Cl15!)?t69N%?yzYpI?;T)LPSm_ix}vbGWZI7llb4J1DZ23n`o)vEbR#Gk zGp7f|dFIVz)^xbPEOBrJsyV-jc3HWJ_a-{b;caPX|L&KI7!iDoVV2JXJ^O4oZI$#r zSQVE*Wq?*zvH56hO={@*%E*V`&1@^Ff6z??FO!2|Bo+fi@v+h0N&CHEtu z3O0NYSS7p*Hq2dXEVDUBo$@a;-UNCSK#&~13MK(P*=)=+12g6ndP2Uu(=cm~X zzz68&%QH9ddFG6IDClCQE-7j7gq5 zW<Q`lz2#7f3a$7pyh+GbyBUFej%k9>5I*YkBq z%+>SEgabCC1q|}AibY0QQlyDtlFN3CGKQtWSN5)_dCk=cU+Q&n^EGD0+o@s>2eG%?FR4@7xQZaLty_mNprFptDKs z$b9|DO9v82f1}JWH5JRQtehPD*p~Nzf!bAw;_6U^VhrX>+?;fo><*vNFhN3-c$zA+Y_s84i?Fa zFLV)E``vHfUCjDj;Jje#rhxU=tMDg=&UU@|GHwMlC5=gvOZ?5a=I6KfN0m zOv#v5@#477>kL4$6xkk-jya6bcTq_U#vIT&1uX!Mqy+6!DeZ};knIL8akP@dMd1_8M zbHn@j6{Hh=57X^OAUcv6%lg{Uug}5cE7Ve~^r*m7|BN{m*$qCfv|wY20{r%E;wKax zVijA1e&(h~j@>ej%OMc&ucuna97Wgx=5vlQt7O&MFlxw?_vmCQea2{CbxR#XoJQn{ zDfgdqeA!MU<(uWu;;gPxA~kb}a64Mz=q!jv97rkq8p%+#!796l7EGJuX8C#=eidUP zka=u5EBHM(m_UJE|5F&>GCNw`gbqSX*dI-3kXUqPQ{RB%^J-aH=IJUeS`s}@VZ?dF z;al$?iL|bDF~V}Xlh|X=maNqt}|d5X1w7@&m6GJu_=4 zSj6Y*p|*o&WuY?*Io9d*!S4qElYA393 zW;(KVXwp$Mx;i(V#tfwRY*twbzZuPVitO;T-IZpM(m0FCes&xDkJ(ng4`Kd-Z+#1a zHf@bMD^lj#sKPB|Lm@Y>jv^YQw7Oza`9+=t(Tk7&iS^(fG|&-#c?0SfpUL*3WFJ*L znd;5tl1h{FMB>jHh5b-J%l~bc!Vywqkhp_%Kr*W)9yY-ZEuddV^9)};+H=XYihEAX~j6Wbw#on%4X#H^T^4MS?VywYoIZ- zCLru#vLWX_f*~O(q?9YvKo40M5^yWN$vk=tytHW{d+Xm8DB(%a_&ktXZ{ubSXTjzC zN~RfSt+#}|jm}W=rVWW<=Ib%o+L{H$3dpU)6wL3<>E>T@ex^-%x>PBMQN=eQnfqK@ znY#2A&;E4ATw}qfbJ8uc^a?2^|}hI+uG`;i9!#oPfIpO zYk!8cTy4u)K-gvZY)_hI(>CMsY_!-o6Cw#8R1NESe_A**UCk~69olGm)oDIlP9Y(_ zcMq39MnX*O_7X(8sv-{%Uo!ZxTKdo1aNAJ2cx4T@ARS=zkduzheRbIiQX zsKJ=`jHM?9M4oy)wcR#2_&0@*^OMCViIjfsp)F=cd-`y(>eNw$?t{c1z~WTTv?4}W zsRzZiCJQM%9ym8&3+#7+9BZ)3O)*%cdD2Yx6PrJv&_^uMdNsqvJ4O%J6q|0WavA0Gp*)XM-@ z?AFpMlK*+-|NqSY8~tWDHXG^YU3_~$KALO$8?I20UdK&Y;be@U@^;en+Rzb1Y|;_s zo%FI(dX`%#CEYA;%_}}Ju3I`@FO@bidGhv*%GX6|=D4f$%G-uDZ@t7IGntK)dU6VN zyj#84zC(AmYWj0A0@2SH8Pe{$6g)aA>gnm2jCpxGceLX&6eowMuuhTC5*+W0*_AJV zHweAd7NQ!VOLNP8iesZxP->9;XQ@3Hqc*QZX^-?mUDR`AQ_|{B#q#2EvP;uSb&}OI z!qIQ*z~EV&j{7X4n&F&A$!0o<$^)P;1zCkHelEqG#fY>(ge$EqSeE1Hl>W!4{@1UD z?;kRJs2o2uzNKo|DpJZzf%^hYFgRaQ+fTt-S`Jc=gQ-Or^iIba759%mi`I$$Iq>5t zaK1%;KHj=rtG}2dzHyWDnsg;c)Yk5(yJ@mg%V}P3t>^FgB5^Nn;6A5g$6=y=E-N_U zM}tZRslGRiN_~$nHCs+yP-m^qdmFy|{FjXy1@5#dCmw_vure`CEig)%9v4Ty_uD#*6p(?CBnv zW{uwt@tX?2Y5nd@i7e1)pzd|{rAdvCdjry4KORQedNCf8Pj2T6&cF>IYsE!->YW)S z9db2P=KG40vg%;(Vzn6!Euw?W@G{zBTyXC4B=qdl@z_}lc>Z>o#9_e`{ocs!+t*r8;5>4c^TUK4XJ{Y z%BG|gZQNA&uI4cKbPFSN*FKX%c(DI4&&0}XsM2qbMWc7lMaKdS3M!M1(UZ6 zW5`?%lV17T_({sN@dFFLi>W|Z;`wdDdWm$|g@shUUgNhW?yS1o4Mytz-P?U7YMEY& z?=SR@(X1JT_Hh3Bh%nLz7_JP>Lzy|bMtVxL6B+RfK4Z8hgI*rJa#Tb&e8g_Bg{XMG!UlRVoB zoh6tFEw|41$40Qj$HG8?M1eLRrfzvy(zy380s>NC`l%J)ro>{SE5P1dc8wXRq*Ip;2Yly zonLvaPn~z3&wJTe@LaOB@_*@~sy!_~o=OnhN*UMjJE5+8H%PW;5+lHRZ4C3CTy-|^ zc@ABaTYihRo?hReE+Z9p+Xs)-0T~+4GZw4&a$n=aZ#47;Z*O9RW9LxIy#>@4H~kis zt%PWgF-Ip?1J8%w`xIk_10z4mYTC#InHICtmyas8U=IMnn&$DWj~tx`!UhpInNU;D zp`8D@h~Ev&ArAk5(%ghh9?6U?GdE}-NMJXPGTSHf@{cuQ@1`9ME8)~Wm}>-eNG$4S zrm=9=AbT+bSc8^Wxo?+`klC58ms)g6_~>}K3K^tKLg!U3)}O*X_?kB()jh}SP}lCd zUQS*=2hT=kCd+t7Q!`5KDMI)!x8_b)^Q*90TSZy#oi|>#*caf;A4<y^3B+BSlv<+I4iMf2K0*O{iT8x(tYq?Ovhex6WdH;z3h%nSF|=cD?1 z;&iUkYM?69hTl?1^hnjQL93q*)hu|_!4ykOqzC08`Oj3#Jf@A7GeJLI6Rxs8qeghV zgws12Y&?RpZ}IkRpI(M^b`J-Yi7>MqY-jme`P_gmnze3+&$?B5eg{Uaid>3PEAjS3 zq6qCp%~n>ZO|IRcQtmw2-tA2IlO-_}I6{k!NS`Ch1V8gKC2Upaj<+0waCyZTg@qBU z)wDm;7pE7FMr$_(wf3?$TxV<4!ZjxcM=k68NRwLOc#-2?ob{7EpHqdu`Wj7YA@&_Y z#!Y1+ZCzEeLveL`&DR+G$L85&;?L~5n_sO^{f1KaN0M$o8*K;yFG=o zD-i|}=hFr)9N(>-n0($}JPwx*CKjkYuLE2pbkD~(jaaq8ZHJIux|Pzpz3h4e&GQ$W z^ag$%wmfca^3jM(tl}0YB_qBxOz9ivdHJhf>ySoYbsx6Up-SHCg;e;(0X~AT)!V*d z$IoT}ZfMAcI&b_o zUUzlNgh$7A{&w5$hufr$9Es5VwHQhn0%zA&7l*5W*My|ej*tqD@QOLYf^E7JDxj8h z6#u}yuAnGu8N+krY@G0)a}HlM76li1FXap@GUK1ODQ$R)^)+`?#6`h7&Bf`+ zrTgcN1A9VzWf?fMC-}ej4FCQ#i3D&9&lo8G-c^{xKgy!EAv_QNZjJ`Hg#Zqu^gpBH z1;^z>HG~Dcg7zQNHSoyjxXLorfihBmZ%?Kff&RISd@A;LbD_X3(8vq?bKe5Ylp;X? zOk3Ds{@t7ya0?hlh`7J|^#6%hB(IDhX4Q(N_5vJIF@%f%epw$M2IGTKZey)!N{Y&e z`moM|8j+ta>whhQK%br_+(Wsge2SjLPjsPRdtVI}uD|H7wiwO42F>)+|NXMT4}VmR z)s8hb5}ZI2k1HxFs%@r$Tq(mnA8?XYHf#QI4E}q(q2Et*KDIu#Ikr9CWuu3R5x(nK zAhcC9cK|N`U?G&ID10(e z4WN(Jrv35N4vQTcl2E`l7cQwX55%~}FO@0v5gki!nZHNF$9n?;;rxEE26v_nkMDpO zM>6pcWl-Ov^3NF$_#X{P{JGLbDsCoEj=$89yDH7>dMo_ZB8`9HT~|*gl*kH1GkTW~ zikHd#XP$8vs~o})@ZH?unZzRWfAvmmoBT5%w6MZaw0~WL-3P)1a!-Z*-?8@Jfe#VO za#aw3>;0}YlLPp?G`K+;*1+FQ+#28;&esAq+j;6m8mmebuI*Fn_%`5;G#=XlYAu5X z7b`My@-44lTfo;o1Xwn+b(|#zt-j$(NZtQz>K( z2nYqZWFf){S=4^4dogj`{4#({9{K!ZuN|e4a H{PceSokDWs literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/5-input-size.png b/website/www/site/static/images/blog/scaling-streaming-workload/5-input-size.png new file mode 100644 index 0000000000000000000000000000000000000000..769cec6257783731f1f4171c2c96c96dec14aed4 GIT binary patch literal 31216 zcmeFZWmFtb)INy2lR)qZ5C~3i87z3P;BLX)-66r<9YWBc!QB(w-62?TmqGtc_%W1ONpq+o=e{SdGaMTvv?Eaf-$$};RdlM^eZ!ZUh} z_e;}u3+R(S?D85f5YNe!Mn69U9FGnmGqZN6bvRs|0!FIfLe|^0?*aPd% z7I)^4kgaAfO2tQ}uDKEnmx(2U1bb{SZumTtE&NsGYLiRW4T-Dgk=fkFYUv6-D zFk*_qR>)Dvg;tQYi|x1fMx|)9UZb?7#zjGeV$pA5s(L{(oUg8X(oEgzI=&(LG0fVo zlKWHA5W}l=qrUdjC*-k1V^y-NSzl?gu(%y)U&QvRnVqgQ5R`!ucGo0wd3}g#tG=Gk z?8C8JY>_bJI=BuRx+YR+jfC+-&8Erq6iBReLpkRkwS?eaE_|R=i=|rw23o~PP10CK z28tFqMuLJ4GJ}Exj-Y`LA@G5Mf{hJ?LInQe03VTTn15HHSF&OMJ63)wD5NMXDGB^l zG_W@^vUV`FarCzov;>-(Hd9h_RFjeBF|e^>)Hk&GV8rNZW&2bG3hc@Q99kJU>VsUZ zEUg`QT=~dcBTXa#Ke~TOKAR7Z(>s7gk0adlRNN+}zwu%q&bSEDS&i1_w85M}1cYYX^!yjr?mz z#K^(G-ptm~%*Gn@)UN&q8z)CTa`LAe{rB%rKaE_?{^w5C4u2mDctED7D@<<~nVJ4; z8>k9?I?MCk%+<(JUBt`^7#`pr{H$zmz`x7?Kd$`G9sgHNwg0Kf#>xG^YyPh*|M!~8 z4o3FEHdeqb9r^!fX8u}x%=54r8gyjbo^W8}6@vY+eSAkNOANJ|WQ~A>u(6Kh8 zu){xpHwB!H_JpAQYrEF~PNLNfjVp}ecS}I|(@oaDcNGCeVS^xBZ&v9e|Fje=i2XL2 z4~D7ju99B8Qe>EWo0$8!9oA^A^V8{WarwLNq0cu1iL87111R%n8=Z>d>D5_6xLN4_ z^s3ha?t^)28i!RZ9)rd#;zm^yO7B%!(_yUHL_Pr=NH<*ovJvNo(w8k1kSm?SGG1r1 z0%l11^yirc?XiO;TX|g$<0a#$=1h7bN%|PL5gRU!R#wREGnM zP15}dbgI#5BelP8i2{Z}j$;b8S#9@IXs|chnJkKHzWy1;;{8i%i^1$eLwWCt61BQFm7dBTaiLr7<$3wmGIh{(c!w4U>$6-UkqVs|k z|EfVxWPYjtc!QI5X99 z9R_qFj`%X|CJnxT9S+M`kgnH7LO3a(M(8sd@=}eOSX8Fw_bAUp&7L6{W7~thZX5~T zeE2w+$}UC7Zcdq`<8GG7Q>N9RuZ5PJuiIvr%SP+Q}vo5czCF+Gbn+T#AyIb6_4~m7#>6At>fKE&p?VvCV_OP!QaV8hnZDiPA$=_!&pF{H(73WOOy_o z9-f-(ReXE49#iJHGrl~YC&jR|KUc*dyZySQvEPs=Qm@LlTy9p^ao>3rvGmGz z-hZ0SWSE#hz`J?B?eQU}n+_HMZ8qU`8ilF1&)rFonUBQ$7jaBpfnV7H`oP@Ap}+OI`6;Un<-DCE35A;llMTKls&~z4E#NW}D>(2NTq$jaaF{>N{XU7ubb^FcRbh?v;+w z59W~R(QR5G`Ml_mVM(G|%Q^D(9}!hlff`c$>Yby@idS91mV+Eg zKfN^H00T)G>NJDanM^La?3wbjdKmg?;cWUE5lNfOuEzaila5THitM1u&{h?y$h4^0m4GLn7l!NGd4 zX}dR;^nQSB(+_JT&?>XUHCxz}xPPam1~aBu*}55U;}d_^ab1HSgP7f(@6RcgwOndw z&JYvZHdtKC{Mng8UPiBTuXKmbCAdRQH(`E$;t1zf-&Q8p8fm0+`Y8@l+(7Lhc6gud z^%XKRJKz3g1-*gSxIUvR z7v+qFxYTiZuJagkMlhAb%ABwwqx%GN%N!{^Nzd3B2FD>6{)}0IWNRBl#goKx@%e+Gh>R+$jh%(7p-|nlb4V8cI1@6A^p)r)!iTEQ_yPiCxbVvEVn?g2*_w zN$3;&xPs_Pa|(=A3lBAMLk6c@{x@7w;|?xy!yQP*A@McW!Al>8A$Rb;8~r81xmOW1CWU-V|iMio#V-Jeg$c{jQe zmpMMn>%LX0wN%HH3YtutS7a>>dL{X;H!;gyP%{Nd+%1UIy)Z+yRAZ{E@+I}IlhxY0 z*=4}&T6C;os3L{ep%1Uo)=&0xH|ftM$0agoYBUU`adMW}uDn;J=N+H1Xf(Y0JN#N8 zC}cb*7GgruYM4avG>ZAIUm@xRnEr5(W%~gJXHrUCtmIUi`({U3mSmaU zH^DI7`KkGvU-C_lV#G<2C1IH?wH4OmsCEkpOAwEiv%7^`CCblQGkIP^E&^1~%nCBL zii(y-Fm#4@wKkyn>lQ4cZ3FS z#<4Li(rEs`p=e^bOF@ozFp~HsrJvR#W5e{k=%yn-wBqmiErD%ZKih&j(;0~4KOj%< zb*7`8nL7K|e07D5XSETRimFVoeJkBMHg1f6IKP`dkdqhGM`VK%0O3n*k3rSx6N=h5 zHH(`=i1)ksY{{c8DxALdbWgwZm(eC0X;r%?w0{f<2nHV0FP>NHIkHiaq!GccB2rxZP{s+oPL4v14u+i#p|KK19 zhH?c&`1j?2CXYr0Am163A)e5mcUA-x4AsXOj1U#_ge`$~H=uzEV5k()f2a@%fPv^a z9e?+30Dt-d6)**jvHx|#AF2D|rs&y03swo@f6e4Fz|8J}Q(_q){;mOJ&tzF-w;1$v z{#1#!M&hdc?sCkfei06Fmy9ks#J9+L>+vkMQxt`|SZho8U)K(>uT8cEPRp;#tv-bv z*pgsx*Z!2>_9QtuTx?LR;D!m|tb2dv{%3r8+2Go}Rt0Te#*Z1QdsD?1ZrwwW~ z`oCLWuwW&Y1Rkf@s})1#c*wu9AehgNSvIGJQfmEx{{YBFG&V+crYzD2Wu2&8C?zl^#MD2~+?LnOpVEe@y0chVp z7526w6P54+0P%(=Kfm|2r~Nld(c!OFF7{>y0JKgE;Le0d65jqN42rMo`OR{vajF;p z3J=a}!KzNO#refh^aAyyyR7Ca^RxqwJ_`r+ z4{@&0ulGbH@gCL-x85G|YRUdFy}-h!gJVhbERat5umG+xpL!RH$6)Tk;4h);bMGYd z0&dQvldle{69+(3o(1ql>2K42*Qd zXt8QJ<94P;8JqPYm#W^`hkwgVFk2$O6AFhxtG+_1@cm13TtYTe5b!9mq0j01kJq}4 zIzJ*Owu(oQa;SzwRv~BGX+=E0N-j=%$WW2+hqYu<*|4MSzKBH_mbYAnYT!})o5Joe z!TH#uNwc=zk6BcG<3$$~`FAUzwCS+^)0AN#ub za(q8KKSkjnywpS^_UM>NtglnFtpP-Q%1sdEK5FSQM8?-FSHfO)r>0m(&! z2PdL$zJ3Y_|Gh)``G*Pu-K!pw4jjnKLP`;^x_6PU1Kbbv+$1e>PCNL~q z#~jjtGvKh|lV*&cE!<*8^WI)gBZy7AQar~X|Bkrlx&FLh)4#h4H}^XthE4-KxT*4m z+>=G=wSm*~Z*^#g#lE*XZL)@_^Oxkl?`6JE)?^IMT01TL&g=3Jhjo&p0TT7ZiE)*f z*$k=<$#t3egXp6}kOUz@y3&#)FlOSdMHaLUe=&fxh#p{}SuAHwR4!_*Mxuey6TnTw zRxpXRK_UkK1w)R)P~E&X%UZ?L75WOGUd~(O5yvLf63l<55KY`S)h2L7P$lbWWxgN9 z_AMeHWPiuz}&%>UE34f%F;6Z;{-0W#d~9sl|Y91sp<; zyv|zmLXN9fELN}D5WdIt128}G`t@spM3xY2(zneiUjd^jKd~h)GLp%cXFm0{)=J+y zo{Zk1I~W_LSuBc_1TE4v9G$_J#(Q6!%*(NxErk^6ZFf^!miul|JO_M}ZHD*FPVJ8P z+18Mv1|BO@GSaF`@XW?f-q&0`h<3A63=<*A%flT_X(K(%LV{JFR-eZR+#Ls;A1*DF zKd;5gvj)(gr9m;=#I&`i_~m-7~zz*`qy|59`kbpo8DSRgx{2@Q)tsWoAe z<;AgOt&VK+ViTrW`CwYprluvYLDlhD^b6wW&{(v+Q^?E5_{3OET%HsdS<(sqzYfyB zr7|t=?Qi%Yl49+TgrOe=uz25HubL%BV8125fTNl^t}|SFp5~gNCi_YubIr{U>I5gs z)JEFQhFGeav;vJ@tpe0sAAJT8BMTgPTsH{1-gFBt^!7c{t+7qMn z-^grj9Re9NYmwO4EH8a458EEST^bImKg#dRa4u1f{v3#>p?KN_1ZF~q2_gEZqSPSt zt3$J-m7m|k7)YFLLMaOY$F=lrPQ}oNAPkayo=J2kI2uvhWV3Id;g|g)PRYEl^}Dck z05*k5Rm<*Ey4U3#UQ|m0;FCEoh!YqFsMge&j**8!_*61yKcUR<10zsd^?dm#!VGDW zh{y^Gf3ci%*|jV3$?H_^N#~dkb^Lv>oBfFO~plRk>=EXB|>MGfUNE^cq;3 zh$oG1y2OYRtHfE-&t17VO)J{GP(tfidHU#cFvAN7(+V=w6YG@_7w5N!KZd;@Kdn|A zu?PIm_2Xr8Imyup#1X}b>Vk}HmpP*(h?(-lMK$dCtmRVE8gJpJd@aTjOpe&)=X*1dZA0H!&MpX6hwbIW?@$N1Yr=mdQsK5cSeuRhXIG$K5ddcM;+Jndal1e&x6_^r}OzXB^+73hP ziN|jBVtQ$6&2bHg+l+e&vKU7bzCF^HdxIo51D{xO9r_fDeFDEoUPFEAwT9%>^PB zu5|K*Kk9qTt&Zb2N~N#l>I%-E+;ErF11BEc!yTw0#nbA`9nc`1PILX5zqF3W0lP(7 zY8&682bi~BT;SpQi*6tWc_moRn-X&z2YOM8WOSh_e8ub<(eAJ=GZbOaBIHFG``C+je`{6v1UjEIG-RmX`@q-Y6!__2~mx8gt4Bp1`>B$TF+!&s3@cUcij= zRb#>c4bMkSXbXX-@>`^sS|%|S&+4()0ZNpfz6tCXak#k;KcI9lra0i6wQbi!vaRh6 z*?i!h_qCsrTIf86?G%KDg`ws`2q|XrLp`l*!XTM<+s@c75N~~floJ^(jLA1Skx_f+ zN?drk2e|aTm|eb5BQKBh3^8EFkhrG0Q9t_U5_*R}Ap0L3bA<(;k@a!Xz3-!33C}p2 z3Zk|4>zhU%%sHO@#NBI^2sQCF?unaTTneGi{N-=Scv(Fm8|I-N{P23H8!3p8dcL7i z_>otQY4X)#IB}~w+%8;Po%y;qX#=;fBq2;(6Yii>rxHe0Qyc)27wk&`fYAtgB-)X8 zN$v)SA9cyzjKIMZSj6OzfaPswQ3HMtfE;y}0lVcyc0d#tRco-WjRMIE6D@B!zmz7s zzwdlDSckAZ2C$t*7UIXP-HjL{qriH#)RgIT+=sj9Jcrrxby}rX)p5%UtF+BuSfXLyFWu zNYR!>4GInm%;=byLokXB!gzyKLFc-?L)kiWlNNjKqT#wO>Ue-&nG(AsFT$ArY(;_H zd%?1=&r!CD0{qX_Se{Eq8RQNX<&w+QXeBFVnqT_6X-9ueov`x83n1lEY`NJj&e=MR zHt%V~aOi3O0&ITc0H5~AQ%&a3=4u>Wp8EN|UINj2)UUzoo$<@nUOcHEAkca zcsqc}PuGD4K#5m?LsH|oKRbr0;!wAS9{$o6s(TgDP__>YM-2UC%T1{HH6kJ7O@M7k z9tbY!qvu1IXoYZ)S>bw{4K+jOjS2UKG1>h+6tyKJWI zV@J0L?vBDhGt%*|OP_N6SOZcu8OmWX!K1|ClZ(F!{0b^!lfQSxsU|JVe#6Pg^AlDW z-9hrZ`H58yP_cMN>7lD7ut#e5W3I<rAg% ziGXtf7jGGe3EiypJwA376D@c0gw|G$@w+}3?j$M0LWnZ0e!jD|Q#_wnkQ?;cp&lkt zNdCjQukL$WvenXkN|nrZ1f4~p9U#P1kh_i8(Sc67-Hv`m7Wsa-nDAnBY z$*fuxYv^{HW*k-7I`04S{q_t#!#nqtKnx2K4qBO=As$U>Y9VXB2FSXMPeQdr0^zUr9VbJLyP(@ip(G)J@H}K#rg~{^%qt z88hBqtek{s6yl#mr?cgo#MJJFeD>XS3}%aHEjXcYl~8+~=z?XN{y78mR^bw{ohxDT zoW5*NIwkQ(e}C>?+ny>t&+9kM$pKrF$JgoU+f;I{VL=A4#OXzIQ{pwo9>oc~QRgH( zje0yJYANdktrW;*JzhwmIF&6rIOU?>pFw>>q6~$S?t6#Lfk@0IysjoW!{2c%a%OTf z`4S*?pld-^T<5@(@`?ydJxE~W=HLV-I>(rCU%`fKr z1s2+5v-FfKZhhZd7l*<_3PYUC;tt^hb2fB-kKcC12-P&X`xikc;MVcyisjcV1jOArxvKu-Yx!X)b;(dj|Eopx6kQKs zmHNj>tpZ%%#phM%KMmwkz^Hju*-w29R@SWZZn0eXM)5(52hm=t!?BgOgy^Xd2o|e; zd->L={9QsxzM8*-a$VWW8qzoR@!@1^3(nHZNpiUt_aV)nPTcB zZTFD<_d^%qtP6dG*j*8>yt>XsY-aYp;ba0rnFL%C)~!WNSy#W_tn_w=2n>|+*(8$q z+`sbKsR8r?aStxbui6;ZUiXkXRBPEw231oVq-NY}_QU=ecfba;O;I-6j**-BEW_JB zh>{pyZ|ib(i0ex+OnrR$UD*QOuWY44D3M|2Sc9FR z+5O^id_LqMjm<5z`DWL`{(xN-nawiV!D1@0?ZLfzvQS=Fhv+8s?zWsdj#|b!DffVr zeWkjEz2wF38Mw!ut+h%^_cGROXm+56tVcPGZ}wZ;pUYcKrfbd9wmB`R(0nBzvG|5=-CB7th=snrPPBzD&C< z)var757!0)svSgm(Rh{W++PMZGYor2Rr99w84jNJ+SYzmXsGle^}5yvB;TFd>(@z(narE{f*|cg}hS0PN^|R zbt_&>7rRq6$M-~R(Na8G^aegJsC6Sq8SItOb zQb7(I#lhr%f^9H?f?opB^DO5o?YkoQ48yE{IQ%dxNI|$=z9mBnhD5g3NhJZK!mUi5 zL8%5}VN9Qjs%pzOGts4+b) zVnKwL6?PL;Q({t*pWi-0PC5`yXphi35Q?I4LMoS&UM$4dDTOQ-`sfv<3y5}h-_`>$dt_T{!2{tbc z7a7OI8QUZZ^GJiR#REe>!$bM;ffJ6ML8br^SLw8;LZe!y<;phld;gUb&0g@G!>-Xb z?E?MGGI=!XmbT(5s$i{x!@dTXj6B1>ScO~(>^XxdKAdJAHty|zAuqRZC?vcM!Vop= zOV1UtRxvcI7dtc}pb6~kIqr!g(4HI|%L{1Ek-fp8u~ZyNI;*8XhXV0$9FOs6S=mMY<$;$M(*?l+v@FJ+*drr0-_|&C_KK zYdlV=wt8r}OZd@kIUS419Ii6sT#T-mxLObG?zIRg4r?{thH9ylt(1<3-_ci%+}~!n zFK;K}-52vUf)I|8GzoU0PLS>2vLJ}@_|m(H#+-FwM5LbN?OW3070+~Y_Gjc+$bPnQ zcDZaJAN_pa90yRzyo5>d2VO44LnWNu6qE_A`W-BX)?wmcNV8IW3f${tQGSYOMF)p! za~CV!B{nk+?VOk0$Bx1PrVw+XOc41bOEc$CS@tDtWB+Gj5M|RKTcdW#I}9va4VzpO zD#STRlHTZc31iV62?YKWh~nY^NR1I%dU`XGSaHk_G-5c?*C_jpl1OIJlbGZ~&dciC zPwVQw(`hKs&<+or&wE)$lz_;w4E$H*t({=BGZiM!3i^)XDml{hxSEDJEgsQ^o3|6a zvVaSU5x+zP4d#qCv}h4his9V$2?##s(#Bqu;y8s2b5Uc98}?BZ9doPCoCv}C)t^Z) zb1`_HmY`pT3gr3)(ZW%Z9;NEU`>Dd^Fn(h*8c2Rbp@%7&bPveZf0&VR>2_}om;s{M z8|*;bvsvu!ano;hRA}}z-FF9XdlV~n%uHm0O=~#@X%?H2%~rg+5EqVjDBB>QTdFxx z?eUwiqr`sZ%2?)irR=pWpC!R*zN4boVQx5n=|B*Bh{nF59)l!i0;ZD}0gJf0doh%* z%Z)T>d%NEW&eL}N)c6!(&EHZeQ9IAQlafd$k{8oC_%)sUsWar5mlfM~#6xF_bR;S5 zgPl>634i%+yM3@AJ>0c>pW3fn&~OYnPoQZAbmFgev?DvvbQ_M zur$`s=VT+wB7^mLb-&uoekx+*p(OEZ{*6)nR0;7o>F~F-xSN!m2Vt?V0w!MgrMI?m zr^wj1p>Bk%AlxL|hFqkf$)(Y3uZ+z8Qmv7Rt9*EaX&)%)2hw%CPL8t^faAIlTwKoJ z5+kNe4K$6Bf8eeRAYjKP4~@+j>f*t*%4M@Y#wsd13?j4PBjz|UE$e8k-}}ix?P{e% zx?P=F<NQT0l^`FL7a~?+XhPlGf#*NxqNFoVMxg!~z=vN2X&P4}&O3f=xLGP50u$fYvdPub|sp~~N zNO8Stb2f2x3P9=tcf&Ytw-e9TBn=-Cj5dcJBC0;Jg81Q3;2{1bIoRr0ffAI&``H0z z0l7qM2jo%dI29BjB0SL`Y-)S$3Wl;p*KIMOY37BV0JDnpJO0S2xz7`R+1iJhh`h;q zO9q>`()dSq;u}HS8DrrWq2i98=&9(sCnWm)y+59c&q1I77e=lBI$utPUe2^t|!sLfJy4v6Vclh6()a1~Q%4 zy)RDD%UxxMad*(h8U#-j(?q=9WFbr_QaTjxK`kwOv-d|Xmt=qeS}7`(w09WUVUxAD z=vX9w2^Gh(!2oW|fE)Z`gsi)iN}_H>d~4Bdnr#BmL>u3b z7~Y4pKVtXNVDr93LuARML2@ajP~2EUfaY=@XtdoQ2a(btgHGc0ks(n?&zJ$s1-~Cy z$t?nfBtrf{{A0}4t-E| zqoFz0DH(R}8{jIja1ZJq%?>C9$u*OD1jADn5aE_86;;-G(@Nbs%)hD%uI{5t*=Sl| zV1Jj^)EkQRgz`|JV9vHi@I5o#f4;Gh*=AII2au3?9o`qHf_n-xAKw-Yq;mh2NDGPU z%~ftH6mOL+#hNRXKRH%3zlMgLZY5e90;36jH|MMiS8KgQ3wUp5(66-{tDi9{%?fM_ z8N!_o?JpJ|X~$S}zyIsK(ZP`tj|!N30diG3fd{s+sFixVFOx;u13>v_aE_JWu)Eu% zwq=0-+_ajlcqv5;drIo3Y177fggVrN<5G7Cv%6F})THTwddj!l>eVz;cJBt1emtN# zaW~1BSiaSaLe25T?TA?IU3}MS)++x42JLS^8trX1phvaer77s$RCP?a*yEI#VN8;W zAlbr&lysJ<&SAsHV^;S=xQCGLVoWJ~0` zoTW%EX(xLGoT$*Tw&~$YqXCJ{mdi>5CrHvJoyxl*!l^t!fy|FFkjPXgdaO*VdfCOY zqX$fI<>)jDft$^NLvt*`f zMacZ}?V-llUMccN`aYq5NTRe8UF){LPfmOEZt4QZ(Psl0W)%Rfe)A0eOg_9i7 zu#Jo>ek^SpYD6|`dd{o;S7ctNA?YsG9&M84t(T4Gdp=3&Q(tn+NH=Zut6-x%aCUbPFKhAltZt^d%}0%S`l`+Sj?i})KJ`m~vcpn($gstXtO5g#HYp2tXSk86_9c8mkL&$G|QS6r+U zT0JK`h0~t72M(iVeVQKwbPO^)E{%gYg6kybcc`Zi9yVzR;ew{z>r~Q6LxLketFu$60=LaG$#^v$V-= zC1(<0Grj>uQEY;Mfm45Aa+PxmWF-iuyR4DCV-qkf&?!<7HFF;IJDr*_*oaq@t<8_P zUqgI#7+7R!EU`RM@JN0zY)%(Mwya^i=L=|uvSPkOno|heG0Sc}HVX1+Ps``;f(-g9 zjZSAG4<2e_Y+W}#Bi>z$p%cy5AVImgwC&(!*?u1TLZ9m=ediVnFoQ%vH27!=_)I$V zSBn=bM858eixUW*3>e&Mi+m5OU!oij3`}aqi9rAZ1-e?k%2p~=``o^~&cV~$!IX%3}uwUt%E|$KmQ<(N(v%V8r zB?UC3d`{8F1K|XKX~d&jvz)?^GMfcxH_x|>+Ku54?@`k@$cXvG-Q~Hf7qBS+%Q|JT zk&vVt9r~PBtu^xjaV{q=NLeqy8A=&QIe{+?&3I+JHjqe1XP~_PDO=bYU{QG^&zW#A zIvm5C=AjSn#R{-wc}`q+yw@+h&pU?F2D&8$PU*PqDS9-Z%LU;+ty#sPR(o!0VjW&7jD&=EUtS{xrq zni|h|AxQh~EU{TGgTztEh+ zXzrJ0MT*6>uMpK+zBqIoIPnuW_%TB=-JCu4UMR8$k(3nh>g2*gMPo@vUuS1Ber!etlaSXgUj4F*?7efF1ihGQ952=c&bY=ENJQ$7b#wBkm1v6Ne~{cj0U3zl+aCD z3jBg}cze=f>KP@MtH3X=?3K5O@vc;!)?j!N?s-<@qM>;drVyWg-PzQ zyo*hy_$bKgi<~cez$332=9VB_6L;%4YvP^Cx$G&0oL{qgpX1s>z@KdMRk5J9#pD_C zqk^JQ%6J|<`taF1VAU`&wSv3ZSmGPJyWHpGLXcsYmVA^H%}29arB&VrdfqNwi8b4z zrLkKiq7$y}@ES}@vrDShv7Ou%E98Zyn5-!GP-5BX^aFhC=77AedQ6(~K)HeulgASA z;St&H`EwcaH7o|J+~f7?j1e&Jd%kpwPo_jM<5a9@il8X)Wj*?S^bwF$X7qS3eDr4P zUX~kT*l?YL8sQpK23lD+9q|%s?XhWtG-8yv&mk%bwAi$KC$^{EUF1&}T_a+4-sf_t z@4Sx^KRt6Hmb*`;a``z>LaAU5oAoMl_UZP+S5nR~ zmUe~poBXOeP7%$=cY;<9ybY}$n2@`(5kQ7~z;fN-$05;s5Z3<&^QOUF%HY9aEa&^& zB%9k#eSW@emJTxVVFI<-M984TA=s%CZtI!kYdc7r&)xF}1;bE~F#*zr* z)G7sXJc#HK z9TJMC{3J&|GuYu)1lrqrTDGq%qnA#A%W%L?7AZZhN~l_t@fy z_`na4ysE;M*di($_xEJIb*{Ml6DF(R1w~7r4%Kn5EUk%si}-U=5RM#(Cq{fk>G}gm zeUWmp0I~{9;IokBw4LjIv5U-6i;)BH6ZbpXf;AwV0xd_g2t@1++I0;F{M%<+<4uA@)CoIU}R40(XA z?sKlJtmy%=qO(Z*OwjbR)9YC!RCx^>2AHWZaEf^htW`KUx#=c+R6bWq#4q$u;Hp;YoE=X{EJNf zWA&e>JgcSHbSH8%1C)d%Komff1(pmf@9zz}`nQq331EX8_mkT35$1c-Dow1ey z)oJ|krX1fze8DXiply4Zwbm6xovv!l^n%B+kkkGASAC_BM-42hB(LkS-jOdXMn#2Q zQ@k{IiEymx@!?i@d=CLFk$?6QNG2qw1U7h!W+2^jCR56V3=>eIVwD^M`srwk)1MoF z%XX#XdGftJv>&5buSuiUaz`0RBWeQDV_Oau>TV%f4#V7m`dDC3t;=gQ&sGj7 zDKaton}OWiK};7!p3w?>Cs_A#K(WurwLx^j2jmM2^A~Qz<7;8XVzfOa`^^KH4#{K> zKZtBxU8xKtj~l`gBJG+GVOB+sVKWOB-)UI1GP4V>YeQ}$=>yO|zAU@&+r(uN+KZAC zizGHnReyg=!fG6DW~PqP-{mXTxH${BElEMI#29x2F!`|#?lu9XO+-4VV^psxkzSp= zAk+P;RazlQdDaA=C8I3Ja8C6h^*)}rC=YQie3`lmq>Y++zy}o!4LF!q&0PePL+cbpAA9%!IesBHh2X12hsICay3SFVJ`ZjUz#LaG z=pcU13Ba9@{>**}=-xBkwg}9Y8f&x{)aw>_H{%sWB;QEsj5@{~0@)i$BmB36D>+Qy zwY)~7SNyWoifns5%BtF%{`Cb}ydarb0;FPs&8fUa6L071Qdtr}FJ=E%!Dz&V+2X!wYV&lq1d%rnq$UT}-R`@!d^zr33W@{%>9+v(HSU~` z7b(yHp>EVia1!2&YedBw^E~6X;>-x>Kwd{>#HJmFLzxDh<%n52;wB(W$E%6$U2<8m zvO@ato#)}W3-fp6a_~*hbssalJ&=w4!*k2_@g%Jt(8k{F*;lAq#3OE*d9YPAsbuoS$rSrn@`~q~i~tFb zpHKjTI@cD%m^#;7;q8_G_2M>=!c$q8$>)Jmjl>R*D)zINj1U^mj_Zk}u(I_XCd@$R z!8MHqMFQeO(NX9xXTpe;l2`aYh^ufver5o@b}SinwW(i2a084ygJdlFee29YY)){8gUFhb|MQv+KpF)Bk}F6p})dl;i!+uj^$ z0`YHohf*KpevA*#iC=)m0DHz{o?nA`NIegA@I1D+rRNAXw>00l3GcCjngRHw0FcTx zmq^^{=D@=0_wNR__1^dHDXb=usG&FL5d>{~cV*0vnBzS)K&~{5P#DVix(n)fx6R8~ zC^9u=Uy9yk$!mwJe!6lb@3{770YrM?E2N*0HsJigth|qRV#Oc#5UW_zixW#DOGwBu zGDkv?yiI$^e43xpD2TLP|3tl&7yiO`Sidf^J^QxOzXJS5!Oz6N%W^I{B6S@T(-PpDxDCNgcWQXYS54JT=+e-++lUe)vRzjkE?> zs2U6;xNr<)!w@jQ;yCRUj;enao^<=*zyWRM>nW)98a6{U+ELppU}dyckX$qjj~ypW z`A34P4nZ^9p#hg^!fW|X9%Eu*LGSi?WHza7F#G{Utojw;qSQ4uTG+}(GmUpyMtV4{ zM{uShyeKB=B_e!@AV2&PEfE`wm~G9}{|1J76+ZlBJ=wQIyXxi-pT5YBwr6Bp1N!$` z-WossQCeRffRck4UkH+$7FG-bWY9z#)ws}gOAszG(u)HJm;%PTK+qb73}eJCg8om4 z2O!qoEc_CQgZ7SDP+DM2Rvke3_$d+-w*z~iv1{Y)Vc8ldb}+slm}141?wNGZ zrpU3m&P7io{aNd+5~H;px4QkuVc2fJ20GIqX6`Q(LA5WQ4Z?PTQLLcn5!nn2Vm=3a zclveCj3c&9BDwl4Y#8zBS0pChzwvj@krg={?L{CFq=a_s>_&d;zlAOog!UGlnku<( zOh3l~%^3r9B=SmyAY^NET=Y1nsVzG$r`=4b59F}Slw_04m2pQ}W zNQzL=Y6pViQTBSW&Oh^?>ov-WR5ta5Y<#M&cqh1rj^POW;znbzYkjl+PP^7OTvy~S zTHHRm*}RbR2x!_~B$fZu+F3?L8FhW1?iji|l@4iSkQ9+H5J5VmBnKQqWN4%j=@bM( z8blg~5(H@}=|;Li5S}x}{jU4@@~-u+*L8hxExmfKbN2r4|Nb3(o|bFIr6!tYrdgr0 z)Y~e6wDHW3!e_;(X@1tQ+M!$#Q?no9-qMrcvlr3AzJv#xu5-6WB2BG;apU+Ld%_&; z6;d1vlRW8Ag!S`W+wI&{5@P;2o!B8l7(oknlT6mj(%!cfoU0!M zTfO@^!?hV;5y(|H()AHt0UE=t`9`?dA0|S|C^(=wPw77>S@p!GLd&!>sDLa_tyf6; zb!GVT*)_aDe8m`?6(O6N?EueGIM#|S3{QBXw$>2Fs_zj{S9`G=DT90xVnv1Ik`q37 z%MsKWti{P3_7JB;E!9s+XCPap4RhV}R)`)cg*BKn%=&cYtt#*Q_#9xFMfW!1GqZPZ ztHsAQfn|bf%oV-fGudZyKBg-))E(}4SOsv3yW+B^b;-)cP9vI=FsUC_3nq0t^-7J<=7!W{7 zWGQk*C?@WkQ&uu};%8BX>F@2oN_D5k-xn`7LzW0;me2_$X%;y(GSyLJ!;roO4dV@e zUvTYN0)6cl0ps$osf{+3zSVH4zOb5ij53AQJ>`swYV{eHS({J7AQWAr9^ahh(>eBW zNEmG#m-Y8^xbKqo*rb`#1$p#kT%sApge-IZShBpM0P7n}(0Ct`-?7n9z8uCy4dGwa z-TEF?&Z(9zwJCLxnbPhc9c<^9VfR(96L^yZl1bzT7~MSWto&dQ@qGBM zq^FEd`LZF&j{cU^dhTnvkJj)>q*tMc=a1~If{DS3PWgw#eNpa(t@p`yt1Ldb@w@A8 z1T!s~6nUpH=^;JVp*{D-3co4c{p5S8Z9SUK;Nwt+R`hs*48}x-bd_P)_xs@? z`3K9vci)~EuU^V?@p(7-@%U2Mh$+7ylw+Eka4C9elV}%t{KV8&uV&7;Adi{6$d@d` zKv<4Tm9*!@F5}!Myg`IcR~%R1I&|WGI2uXLD-yJ~f;VoZ*&X^eftP#H2ODr~=3#1> zbTZ8Md7JP#)IS6&nn-UpBOIQ~ab{fr`Vsuo83fhxqoOF&@i^6+q#lJ_+i-vm=-&zZ z7$uV#&P!3Wip>_VmX?*HCstM*sXX(e!IDX4VJ|=h7oEJFUJDi~WB%!m;yu>O-Do?< zj1(RDqj7f?c%eMpZo9(mMz$v_$q42K*EiTkX}{`fF@0b@w1OFj>q5eWs9WvSeCFj0Ma^bwEBQ(Gj7B#$$|HHkVOl{n+XE_$k)G5YCA;RXZ z@|`V2%Zeity*Md?v+A=yNaqWvI@?;*n20#I$`M0WW=n*ddxl6S?R>3Guag;cqFrdb zBP7y0WR@}`QJ*~A$Au~5;ndR+isr@LRg!>*AEDz;nD%`l`fmN|MXq#@oFVGKX@;&} zML|f0wXS3ZT&=w?*pex;D7eN#t+nBrz|d>_b>6?$EQxS_k4~_K+RdQ5o%#$($1UZw z{H0@xChbuxc9f;CfL%U4vz1$gjgqskkRIKnZt_!ynqh;42nsyEHo}$Q`QjAKux}(u z%ZTh$N!aq+7d4r&+jPD{!nbimqTHkJBdE3?4u}vO>5)rZ%%#x?hRt|aF>%9_UIzGA z=l=L=z>B-$%jNAROq9S-(1<*pVtjfswa(h(D^}h+S9>jUEKdky4&!uiZYF1^H!{Y@ z&5zj;E$xNZ$@a5@MEwJ`zm1yykxJEhh;dF#Zoc zfCjjl+|ZeYxc*5I6rhavzb-?!du(gm&*%>CzY}{r#bTt9aqJbK*6pUD%Za(mAh}#k zWc<5ZmK9iVbI8zWxWnP-%Aob1ZM@R?SMZU4quB3uUDRcGsA07r=2|T`7dLjpZ#Rvn zF))N{rvSgPfRQS>$NPUI9Ke%eHF9Q=f&k2bX2LOp%N<45AD$U4ohU|CcQ7%7CDI>z zAB3VkKzn%R$C2^dq72TTmBluPR}E@>D5<01QE5iJkclzqDdz?_Bn<*|H7*msbv!um zXsNccn#VaRdiRbqqn-{LS!f`dI&EM$sb*g{yKy|#YoAy&ONNUYVW0c|WF$&uthD3s zojIPz&jjL(RFMRhi3xlEo(lmxMik-eZ`BUnG}v7A1C|dn{zEb7QpPfm+W{A`1TOv! z$c^sddxsqk-#n66$h@?cmQY# z#Le%JUg_o0OkyoI{6wv}B#Vlv(=G1Jnnl&1cv_&P+eq}v`{H!3X)%?M|9)PCxYNv3 zI>E}i%S#*Y&F!xf9@?w7Xhk(mp=icEoLVeXzV>6&_ zD>%0irG(g*HjcLX)sn!9?&31Tvp(*D&P>wVpb=38rM=+Uo=`n{LKe3Il%?Z7nT;ZT zPn<$pAk5F4jQ*Qh7oU` zgQ~#y7jjn@2Nb=r-M}3%&IAc9I5UsYt+2d3NTLm7A2QJu_+AIiv+4Wlrp%d7V1p|A z14^XJE(MRny}{ng?G)PafU0B_*{i4y^3?_pYd3ZCN< zAO>{Z6=_I7^=}75M-m-sJeWO~Vo~MSOd?2B&}S4?uF7e4hO2TX)Awh$OmgG#+{M;ea zaswqWyhb%^x`GDHi8Ek9Q29uv$ONQ&Zd4;urX%Rwb{)}G>%Muu$??d@{Y{Gxo+l6* za>0(iGJ${)_6T$n(wqZUS!HtnAc=aB=!NEj9Q1p=`vx~q#GKO}g|>*xT78L6TG|v1 z4|>RdK_mEtxA1YQE#r6%@|Jj!DbDA^cEHN8Iyvme!cCR$jnZfX;6c1NUCF9 zcSbpYv#A3{*NaNus$=E4lNMBAO#-6Sq|XvMOPWXjdyGP`Vobyd8CMMgOqVBcB>&nF zRtx|1BFn-BP&+(09^^heDC{r-d=6?rmr`SRTwldHi-HiOze-8pB4$S^9KOGPP7umN zKy$4&2N`53hD|BgX|_pXc2)4f{ZjXpwSm24g_lh#^L0R0Q6Fiz1`re}%ZR$H*8tQ} z{NRJ$UE!xh{vMq2ZY!F#yRC%9#cTS5H6$0M+wkZKTTDLqg>)x_6yZF~e zblX3bn{}Vwct|+8VF`d?6@Z~}6;Lhhl0q~3PFm3`grZ4A$6vYaO9!I8ZEv>RB804- z!OwJtIfdCdEci`IY(!wz-i&h_F^Vhrb6M?r8@t=CkZzHV^2zoz_Q4uZY_c2nIvG|1 zoB=b>aD8x+VtvM90JF8DrSJATjW?SBY4(Tu=hsp)BTc}}`i-x&;a-ZkEjtRSmL-Ou zqo1UeG2{^r_h*F~2!C3idYe2q$B6kAFNKrQ23B%UHIjbW{%sAjfyfN=P{DnkLFt=x z0IKoi#nCs_n$qUWI>7iq-UK31#hTo^Z@6O!=5Lj>#8V*bjwGy%;JgPSU)y)_at&>Mch~*Rd zok^+l3Np^JV%ZE0=m@hAB5i|w_{sw`5TWblZ zA(TY^!p<)KiW<4nCNaV|>hs1!E$9Im>g4!*5P7^C|p5NBs$1VZaA`G*MQ1J`v5 zB*EDG#jKX=khIr9AyP#IAbWM0=sb!WqU#1<$FQ(P3ZdMP{# z3*#%*B$|w+O@-nx{eqeA)^L6_peXjt1cZ@*+K_9g==ykhj&0@Ii0Ru1Zs9X#w_cG1 z1(`BXoKVQMi109iOKFC#ot8!59>mhhnL_*`&bDr*LhpM5O4a=Z%dICITMS&tfE+!;wb~^XAfZF z@PSW4{%vRWdD4_MU6Qy9c$U_SY2uCL?({c{Bx*f8%LR9m}=wc?Ug z490F-PcSh=MhsjtkD|{7?NvYgNpUEhUyZR=gBy*!BO0X&EMM5Br(48pKUn zEXK>K5|0nEa$y-DbP*}VP*pi6xexAHl!!i=Tf=!R%}*47Lrf6e=Nfg6X%6etd`zy4 z7a9ykTq6dvb-&wG0n|2=wEA5B4~s-~9?H4dDCE&NUzw31n$e1Q=i~77;5Vn;0=@k& zGT;mNZt(%!5vzm5!Q{52U5^q0gSFL-^bV#3CxR;Ecg%&S2$lW#-rqeQ1xRCg34ok% zJHg}rH=#*Bgob!>oQY z6X6H_)0YQdB{TX;>A4HTyktV1J3PzF?%k@ehY4G+TfE7>^Bmk?#d-8ojmCHKT&mjm zvn9mUd}Kwi_2_9KrkKY!f241(L;G8uqlIdh$or=aIYYl??0T2pr_InB9&hXr+uN+{ zH=slP#?Z*{*kiUj2_|pzaO!V_YCGQ?u*ynL@sySovZ0A`vdK0m4;@pq55l2$yiQ`A zxhB%PDjd!+o0Uc4yx{-(QB78qu&%0C0Cn%?`%JO6g-IJLRuTiD72LM3C!swBPo7V^ zV?+eFjY)sygB$T*WV^IZF$$KK`Rm~kVu+jYlz;Jih&39PRWM_eIP4V9vdtJIGk1y2 zH1XuJ95rjSz4d#_dvWG^h6>aX(+r5**G`;LX67AJ#dg=N%|Wp0kZVcw506`B$OG zp*lT5f`DS;TtVIg0@J;$xqCUeEnS-b{-He@q+xp&9r6DzCl1w+a7|d0;!lRaim?Ye z5+hmo@;LuxOq3vFx{ZiU{+GCgvVgN+=!tQ8K_THZc?Rkxv-+i-kb(h z!2PwH=r3#`jO3y7X5v_|O7F*wD3hcvhXFa^XZchRm5h`aeLptr+^+@WwRLTt>++Y! z$9l$~MzFOL{vy-jb`flR<2f*+4goAg3_mE#nV%akexkfv_DTFl)Qjj34?nE%gnz@~ z{0WdmzpA#;JW;5o^nKLZ_EBZ!KD!BkiTfZ1wk%jcflN_Q7vM_#WUJILzIg$-G{C4P ztr7){nEdWqs%SDwRcD~U6Ol9xTz~w`>*TR0T!Z^*Ub`w<`&lcYP#0jy@}eNRporAj z06bZ@8)(b4 zEyvU}sC)BpKiXmbfbz3BwOAn z6ByvNonaO}uC(q3fj0*z_0?86Ox%N7$8u)Ca@3-!mssLK4yg8Hw3rV?jl*s;@Q2YD z!E}X_eZXMlI)}j37%MRnXaE2+i8-l-P!3K#hT|zWZRsEHb1oW$(Ez3{M4@b#1F(NW zp-t?Nu{5V-Q7g{VYX(&|2^!#G&Pr4S+Y)A?m}8f}PE79L62CM2gQ7f=(zV57%^}5{7a&L3tR-A=0Xg0G zsvyEPFUn7jT^>$9g~DQ828@m=4gJf0l_~Cvu6C_=;1l5$cknx zP3RHGViRLI)Ka+pr67l>A}K(+l=@kUJbLBSuuXXIE~c{o=Jzg|K;nwvggrn5HpqXm z^>NG~Wlo_t8u=iywhcr|CKAskEitiSGpBx4BB6e^5Kb1#IsZBn_kVO*L^C*9-TZZKlr;3LoD&6+l+ z$L#m2L8X=}vVMqp6rzxjnt%7>f$8L~1lFQ+guT&u>mpnW{01-C%6gb-a!6QfZUtQ~ z+De7C3|{AT_EqG;rkDU6K_^{SIhzbeU&_F#=9e$SF~gLG1Eq;D$OD4$hbMc4njH*; z7-LwIzAN2ybR^BI0YJXb3#KKQ(n@?G!T5;aSJE;NMm#vsA|Fm&W2_WGb= z4o4t2u`6KvEIxw@1G=|cC}IL~6r@VTjx!;Ha$GHgPS$-~+_f4JQ$N#~3X9Jqh+x7k z9BUVwD^NzupOYEc0CX&gVp^*^I?aX27!VEyK z!p^$~+Z>2;VUl^0l+#vPiFO$tik2ag6bhsrA-FXVVfcY1-5}v9S^yVOr*mmK1%tTq zNd%kQ683_>$B%aTJ4v@%rmo4wD|*qQ;c)56$mO%@5$Pxn(egv@tIE%L5+mpsheh8% z1(Oj91drU@qV0~lFXjErx-sUMiOSw**AQ~ z0%#dhk+jO#rb~D^U~U)5iWeZ&iWw0zmQ43?Qo#?L$s9l;r+PFfT}3+6(F)6k!6 z5SlY{M?0XlT;gs3nOcj8L`AsUyeje(1(NCQK*IJIEiNhDYH7~`YO)$zml!uqIhHYp z#WcGHqz1bc+X9aXi_ufat7vWL=FeIqPrlBR9lUX7i=B~P4^(IapWB2ga(U{2=bK5JlgI_?w za$xQra8OjGJ!}yXJVh*0mU*~+5I%3cy$*&?(rc8QV<0C@6;k!3twCn@e`TD z0TGVFv3JAVmxiA3OTSi@N{cQb$z%OagtS47L_%zfB4I8Odx!)*W$*1p8Ji|ys<#1w z&;vicBQX0J;6@;^HPetZf?35j9E=4z#z5h_D9UzkOAAF1?)p{IjK;P;A-KTv?#Gj;uDdWWGJMc zZ#z-ULhTMsFW-+y2Rugng3)_5>hcy?2h*I!Jt?R7V*TiGBW5hX@u?48c z0{ihf1b+BO2^%mAhvz6o!URe}@EH%9=KX0@=Tdgj72z@&A?$wM1kWIDms-Ls1jV!o z!IpR#7LXi6jtC^}Xsm^i;{Xv;hTu%ao@O?a;L9f==sY;q`q*<=DMDYuj~!}ZD5N!O zB_y=xyL2qKSrZ?fKM9ZSf=aEh0xmXpgH0}-CQUe+Zlb=Y)aL?IU*w{CdpO(%8Q)IU zB2P>%iLkDIkv(ime@_;+=nDb9K{{Qgr5XrLz z;#={jkQU>TZLXhtM#u-?YCyPOCj0wSO_gh4kmQB8L1x^;6UXyY=j6Y@6=1&j#@E88 zU8l4(&mNK{-1(E4tcRgA6XKQfVNwYiJc8r74sWM6Nd~;U?otaPN@n;RB9|sQadGii z(**tuS>&|)uCXz}zdXVj%e-lxptPZ_-ntxtW{kx`xoIH_1V-Nwkt~nQqHv%meQgi;$6GJD#uIixoLX+PxgTH@04iSsy<#4HFaNv=D3%Y_gZjwpAeE@e7rHuu;U>h0%X=N}#uLd; zM~~cR-;bXH)H?f>84z*o$p2i+$#A?rw|?wJeT7;bBP}j3|9sO+BQT>WX=XTa)Voe4 zrQ;f#SJ%sh_OOiY;*E8}1Nmw8f#EGQ`Dyxr`xw;?utYlT8-^%rB?uFP+4TO5wE{qc za`3zB?03@yt|@T&@_QtHuC69opwwc{YwaX~YwGn|bahDq-_!dQhY zov+`91@N|>e4;vpDY-Ea@js~<7#5f*@~v|!vMQb|&9CnpFSq;dk-lC3V?6w$D}$nJvRN>;o-Em zz4+>)Jf!eD3)QNyVcV04t?w{lw!*E+QOCZ+$e$sE$Im<5h50OcpgyJv%)qmnSp^)- zdXgOzq>F#)LJ&mKl3~5avt&spV>}I|U?P!Q8DN!{_mA~KvC-PC9xs~HS*msBT_Mss zEVzxj9kf$n^#1Z+O7n|g>qFolAjdybc^4Ljedp@glfqwFU}60X~`N`KP-Gu z&U+)45a-%91*eH=exBV2iz**l5Fn!@iFy_ck!~6i-CfHW(C^6>uq?*<9wbp3Am!b7 zSyqI*+UJ7b{*K%1fS_}TQOXEu_9ZN(@~aDePvTsx&y>sO+V?R#s5NK+5!SL@#%_cc zed4W_voA(Da7(^e$@&J2eqc=sHxj~bRwRX#rDBYvxI2*`>wq&Gc<8S|b*~%rUtnQk z!o`#pL)wy43RLb6NYIwI|0Ko4l`{H|Y=VVaQ|Je;6}Lftb`xlMvp#MFd`^E)ywQ)X zmf;_pAxVacExR%^AW59#Vp0VBi?lTN(Z|*|aa>u+3? zc#3jdfV?^$wJOp*exHD(Z%6{&q&uHB=FR{?;?MU1QD$9{G2v{FW8~jKtk6aKaf14s z`G%n+$fMc`hJ}~1LzpXSA2(1D28Mf%!ABR&jcMF^L1`*1pP*JNph8DTQi!n^a?m^w z7~qaf6I+S+wzu(eE8X_}T_f7hkA!aDOj>p366@x-JT=v_7S&F+N{}ug(pBf|po(QX zfFz5W-1{O&v5$-|&>?6%`BbI7%6PtA+1Ge}+*Es&Kjqyw0}O@zI}Yc~nX8XCZpP-p z*F#T%;C=PTK8u6rJqIxro`0wo zZscQP*1S4(5t!%#Sw3}RTV({x%KU;ap$PW7!M~Q@E096Z$xaFJI4MoIiQeVsr$)R} z)?!J#5SX`C=+WP|xEfdK9Aa*se)(0R^^opP9{bC~#nLm=pR9i_AseDj>p@~g>CTS2 zAk+q7iy1hGtQsLi9T>GCbnw)Dl#uVWl%ZItttH*BeJuTr!7D$!P_RLHvG#F}_5tQM zYkHFB?@?-BK{YfqcrzbSl-&BSJzM*>i$?DKJ`NP?78~mxF?Nm&*`E`kKgS?9d9G)7 z?e9UdAu*@H{<`U?v!FIv^srJ{zK@juxhyQf8*fzws3I34Q|d;l-EVx^*!vXq{iFTU zF9`x0Uiwkh=X;AGp#Bp{CvFpy{UGZDlY;LQDBw;dki+Cvw_16i?gaqCo&=1nJez4= zBO~uhj2pc)zXR7fcm$0B=}AztA|NZFLsv$!rL^19{Lp4EvK=KO?Tv6Fb%L+W_(TXddh4e zfU6lWyS)?6!n00N;oS1kek|<{n{~D~Q=A`~6@3AAqr-&g{p?#oSjCajZYvR<=f}@9 z>p{1J;XET4ey5uNAjfTUTtv*QH89BotfmmzU{VJwI7;&d1^5ZoX*&nd{!tC%ywz;| zm0k&Exi*mPk?H&UJ+q+arwJX|=UnNU(8#(jh1$+GHL_tznzo?1Ep`&St@bh92kYU~ zXPN_0*e*X2o?j#By`$+ScBD8!oVPFnd2!CG9+# zsC8TI;{IaYNeHuiN-l&ae~@(szH82?d-uU7HKpFteK6hN>6XhYH{$>b-e2>zGkd{u z70X%`f)%~cYKLj{bl8w;|@x5w#lc*=5r zjB!#?U;JF_Jnw(h@O*p9*3A|E{rqI-Q>*b?W&Dk4Vd~bedw~qUHn*qij}P`IYHDA9 zS~-Ouw8n*se~wXGI>1TCeaqiNk9fW_L&vQRB;6jbBtTyltAD;zeKvW@T>7NeoMjX| zPD>RL-{zA`u#&~GS(wM;8||rAEbxBiAUO0*7;0REAZB0DkFZIj){hE+XVI?aJ1Fho2b%!nlu0B2A6`b1?uu`ETQ@6~0aS>A_Cqh{Y} z$L+DwOk$Ho`KzBtJ+Fi$Er=95A)~zxU~bO?Mg|E+<2+5eO3>`f(0Od%kBJ0YC6R@f zbbHpp?uOF$DLkL6%jqh+Z(kmiJehEggmBe*9$0h!oI3);9?g0BUW_^cIsz zXXW7`%x$3^|J?5lk2bTD+fZ{i6$KNUJd#uCt)M7s)WBn(TRQvc4&|kZh9GNw&MJ=X z?J;*%2H8%HC%ySnL4L~{wCI1S^0rPf2iZksPiR9hXhf?#vzg&Y2urwK-hfp~@E!OY zj=H+bqe%Z? zxj-Q)rLLm*;XLzX< zNTeTg8^lsr`Y0#*h~7!->Hwk0NN!%90hxm5zGa@$30P%un= zGLC7CVoX5VJJG$>SRu@pqYj|)E2M4ko!AW~g)nMU{a&AX0FQn!xxXqib*b6$=7fh4 zN3E74STaCNDS20btr^4(NsGT`Ak>GIAkLczdxE0evq!@{XGk-&52g6`=2j#D5`xb! zko){_-6HAhtB^pvXeQZFP?i`Hwx@F%?N4&J;XDUD|4d`kri7Ed3_p!wLO!+argad73_xcgA z(m2R6Hzc9=D=R0X=4~vpE)RJVNB^m=i@=Co0b_h-8;?jXwa|jhvq+`O#uqv0= z+&pW~n+CVwcbg-U(|t^>SIHC-X6h;CQqF%KZLqw&`|841Z-ucYx+vjyj6#BN2zQdo z&o=%WldzsZoKv>ayq(ux{|ig{C}K`m8ZRN2wa%);9A$&=SHF43KGFe0ydg{&l=$H9D&^B<$K4pU}!VMkYe z#h|0SH0`i}I@#-x+jNF-N}~S#%YDG%_d?^~?U&+2d@d+-;;&PdYZ7(d9B$rSUM>&y z!Fs-lJNXEot>q{|EkL5JyAlGI`9d5K_Rx6T1S|7RI91h;X1A8R)1kmgHRk9i+kj=?YwB<(gwp^EV@rFCpl!7KpQ6k9c_BZ)gvI zL&`%CQj9l44?@rMf^FUGE0s-4QRsZW)0LVP1{X+1i^5d*lPYjgUG}BD_o(lzMDS;v zwf{l>k&>41nbtSt*ZxNMybdVr>JGJQE6r9`H-jCE7(TVL(^ba83Qxp5bqU;F|3bFc zUdv8>N*Hz@S|c64Bv5UOgz-ntrUiQoC02PnKNb9LeT8|x@QzX|mVOoJXbn>> z88ZciCv?Cu;*+Pr7Ej=Sqo=?JANV|Z0vj9j1OfPd34Fw}Vg6lxx|9w3-|^_du%&a^-Jj^U?%xr9oz#WWE9=6U#?u@oh z6n`rDSC6=rJeF~>9iuB_fI%*@*=h)cqNL`JM?i5h=I>a$xo0~D4bt^l+ z!Bw)xysQ+Ce|dH{dhx@j4bRNX+|13@ZPw|M;beLLrukyYNvXwcD=hxo0ERsr;Zvy& z;7c}AxHWd>2NpL>u@nKjddHagcVTnb* zOYruvo4aHZhs68PpP{Ose7x1`LRE8PgYQqJ>%yeWd0_!3f;W2T)e=W4FAWiQA40&ulc)=1h5}E%|UzLi_+R@o#P25 zA(>c=z8K%t-(63kY=fx&e!}#q?XU~ER7ARQmgbAuaOwcZjGi$vPdM(>NhnR`dxwII zhuae+V@McdU2*6GNyU0+=9 zmJbdxw=)RcU*^ap0mGEIIh>Xc9=tH?`h;}6S3Q~v@6YSD=pd!-G|aIDxt~;1O36r8 z82LeaUSlK}|^j)ONK=u-XPH)(EnwQog<( zmt!8t&+?@s;&Y7$Zqes;-hM&zlnk znJLqMox#RWzx4qYEu?$e(9Z{XLP3@=FPw3FSXwrtMudyj0pIj}4BNN(Hk6c;MF*y#SI<3LkWd zsBvC$nbD3d({F|_x8ITnllZ#JCI;dd%z`=8^Q3*YE)Tv06TcjS2bMwsKd^?TxlAj9 zSWP2{1$e1&Vp&w#*IWVXG3b3VOvC-2Y+npr@G^TFUcots@)XI1Q|s+RK8fOATR@J* zwghy*xP8|R2WFssvIN=C3y}Flob-GlgV~2pP$-+syWb+5k@~L zC02H((%8WT4;D_Gq6#RLjkE8<7{jlszI5IX!oU5VErLyQzW}`Guy@6x;~`uJ|U9E>r(WLvGUu?qmNmc zd9z-WBf>14gpJwGv<%?FJ?3*G#Jn_0>(Uj!hgR#xjYMjrpW^UNot%V-x=gXm>iv17 zvuT|%GnHK=u|nd8ASw!@lQ?UFw9tdLn-R)EQJaA;%cxw7-*=E6IzN0l6?|ho%R*VX zLY1U9?mH$Z%(q_?_twpCx3u1x;=^ggd5&he0k(PQtmoX?cua^XY)r)4Hf3H+aug8bH@@L3)mh}_TF)vZJCUgx2|g2N0_$qAA=^9GuJZm@41Z$It$uWQ7{QRQ)N z^z$^m0E5cf%{t&5@Hwuhm5!Fh7i>^av@7EBoUD`ziI2rnJ(-PD4zZakoJ~fgG<{%~ zJB7)Eh(9m}(dZM@Z#Ld2+&dah){a0vtq$w>zLJ+G<&CU|H!HnNr=+eliKy=+q@V>a z->5ri0NAX|xmP>)*8l7s(FhX~$E%n@vsMljNaLhGjLO$wab~4GrYK)nO1>T-jh!LO z@T{9RJr-z#`g{`8p(Z!9%R z$c)8X`<9Vzj!)I9q}RbVi4jjCjjm!nZ*|526CosFqo|)`=1B(kc(nl{t)0-}g{T}XNE0ROWzHDE6rQLpx|wofl*0wi zSd_J%&qp&rwi3vehQ5{0AErjz+LIf!t9sdhU)sJhXdXwEg3COW1J%W}nG?a9b!NOH*lcz3)znnvbDj(XDd12b9#HQ~f zL2X926CSTBc`TVb^?pX*jUz}3k40?<6#T`c3Q;oeW%(5BggNF*TxIEG^a-lcHVsUu zl{Ch|lq$`TW@`uq=q%{#9u9oT&b>Qq4&9dY)F`dRsg-(+Y?}s}sYpsP(<;kx1lKaR zFEifaX)UAIR-*fB!#p{iW+3a zV7Oe^14U>yW9pLpyoBw-Mz>}yP9lR$oIn-yvF$jpP>%-Y=okGN{F!H-XJ^1nP*p2& z!cf3-qA7QYwfLd9-Bea{`68G&y)*ap-_Gwwkg$q;{ba^EM{b8RaxG(GlZ`>!e{L}Y z%$!8SJGoQu|NLf(Xbk_w>(y{=S(1Jlk|VIonF+M=9RKg0T7>SeQ)p1u_RCA*UkTwe zvL8DeJKghEe~n1N+M}<%hS|}4@Yfymc|0Ll(n|fu`S$0H{J_4%EBLhK=5Hla!N7_C zw^ax)8-nS6>V4YT4V?LFOnxB7<32@OFJ##0aW-lg5TbLT*3;e=X#ey6!ZwnmEZV+t zGYJ+>fm!~a8h89R&o%AZJXQVfuM-MKPyYJ)+%Tqr9V>o!j1S@uv#jNWXZX1GtQq|l z6R0RjC+~RpX7FrTUw9G!eK|+TC(8)vpK!=OX-H0_B|ueQ`>B8Z=#tCr#S^e^QJ!+3ELiSN}#UTZGm0UsV7|$rO2FfT;9|3tJA0O+U`LxWl-3 z7T#0vABf53B>hlBK)Lj8AubN8`ree=vohlNzp)gy5znsTXQi1awO*~oV6lt)Z`kqR z=(-7<*p?t~Api4{k-~H^BhG3^hkXB4ZvqS)BVLU`#lMo#pMil==f55KhrM3o0cGUi zOL6x8jP(yhFgc47i~e#NJ^-S;FIR()0r=1>;CMH={3{ph#lTU8^ zH^{HTgtgh1gdT3yt}hOlDWoFfyV3aMnc6PjX}|SegA) znx{Jip%C+H**0!qUjcOtP4m6`h6r7#FPG^9wyj3ioh-lmIZEBV&Vmxa2^sQE5>Jj6cQzhKvH z1C0_m*Q$PId~ml@TrN+Q;oRkUd(`(#GAwQyLSN3nvZ|QDJ3t=B5L?R>^&bd#BvUR< z`PGjulvuw)#I)8MmE?qgmRwB2qCVMY=zEnb1Ev&K&3I!G%Q|$55<$SliNzLvezldA z1hlB1%4tKVmM0%msFt640_v~+@ew&_FunawktmscnS$sM=D)^)+nnj1Unr^=E8R^U zH?0Lha_3B^Msx<9vpmk9UAdm(&Y7(J8m|-!4$mg{U~1a_Y8IvMcX#$oGV~gk??3`a zKb{rcU+DHQfdA+lX4PKRAf&?|cB8V7IF7!of!x?L(lOQV{%Rkk2S9z@l!m`x_1%Br zt&aQsY2FIXZ1i5PP2DevOw2F|X++C+|v?}9fA_(M8->a~cILTa?S zF)Z10)#h?Jk`Yx@v@(}#N&@r*XIh4dI*y^~8{ID)tSy(CYGtyeb02;5y|$ZQ<7N7M zY6A5(3mVhf=0wJ+1z(TeE!%gaD%{;(toSYLit9Hyvo`PJTa_5qtpKP9QM+qcofRcsXnA5d>7zRGNm+>WAW5Wd&l7!> zE8^dqhp>(46VOS@mR|H&Rvu?X;B{QK{oUHm3z%8^!KuhJ!aNUy)}bvTLywIF*^I(8xDvW@ zq3dmmN4f}vtzB4kdD%CmX{c|6*K&Z8*D%5dLi2wL3y+B4#?1qhAo0Q65g>xGTWu$AUfk4C`-T!qq3Tb61;xz6%jJfmjq7KHcKh7up6yMSbxy9%`yY@17o zdMg2tom9ib?7b}hVJmuc?l_I|*A+#i+B78NY4e4nqqEgEJ_t6J;afi8iuZ{E&((U| z+O$2GB+IpA@DXGq(NMPnG?2E@j~sF-LWfU!z`Nt2f9_AMOq4qPwX|^U1Q2Gg7RRm1 zTRFF&pl3MARIr93d{A)Qp0&A9F%PmdaXQvhTv~aK7l!L{wx7fZ+rJ%Ymc1!=XHKdG z54t76>f3id>$++ne3M0qmZkL4a88z3D>{4)f6C%ov-I*6b=DnSB3nX=0d{y)l%2^< zm&~Q8)|Y)@%tL^)0Ba^W0vqqef)U1o4OhLBsKmZ3t+X9o*Yn$n_8rhzvbs98O}~0? zgTq>tFAJ60^>-Y!I`<#`2#;(Is>qRDnKxxoL21hI0kDOQxBK}$QgtQ)sGK3Y$H-FI z;Mcc;B+^Sp?;P zN6Ua1W}ZSCwX_Lgm#{v7^?dtnX-F!q1b#KF!4>OW2{! zw(_#uijO&zTY(mHtju<)DOcQKaTOp#O|RY)*nGk>rI@TFeACHQRQO27aH7bp6l*f` zY-b0U0E9)2`v|At*`(S&p>Z-wmb$K0Y3-(J^%h;6+LwN>0ji`~88vS|0w>u0%_k`K zMlAp+C2qby>LV$I&Xz#w_ZNZ9a(y+JB(qo!z%|h&RS-H?U&oM=*CYf<082R0n@T}b z8AXE>(GA>acJ`~H8|{&E>mBnp!?MOj$UheTpwvZfx_N>l$fY|akn@Yu0lgd5Rlh9G zqG79C4=HLRl1Q_n-SOPO%xFlOSA{&`JGuThw%#PRL-4&hwq6Tcu{6#}<1s1$XSki5 z6y%mX!#<`cn?nEg>)rVRvtn_q4Hy~lCMCjXME6T$*rK*mt;#|HHU~4q5keayn=%IU zvF(+!CLZ)Beo|+kEJF-&{97Y-v!Rgq6saP?aCmfL>kYcKy1MPF_C(1Pb#SbW^Rma5 zcH^mkT_mx<{7QTY4_%W$co!ZX}g3jB@VGn?>!YJ^tm%M`-zmm&BvZ%iBam2<~==~P?GXQdFKqcTe_)7Ykj({ zKaf;LU&=9H5P5EYnT{f`Yhi_%Kp^q{{*w!EXPWbEcDS*jX1&L|lH^Ti)4DlIf>)HP zXRhFI7MOZ(2+Hj3f!{-zWF#RTG>F`C94rHN)v^-0)3<9q`?}=1FnnG1(hH0-eW(F% zfy>@oq#qLBsGG4WBtXs9bhVk*Z4-DkNlXmwWl+ttxSJT(oUAYs>kbtC)wIjBn^x3F z(VN-WjoW%M!skHL#sqv#v1c_XkaBX}lyo%4#DEf(GuSaK+?RGW{6+>5a2 zC#0@%3*=I$LV4zaHMPb93Lp^k-YLvyDEZ3PDLJYd8S7G1gIh>V%H$NMz!{3h{o})B zrl4}Ur@k&OIvN2SCyakXkVkR3Fyd!ANS|okZdr3Llr{;K)aKJPup*_E+CHx}DqC$s3kd;&Ju>KPv^Cl-)kanK#G^c)sya#5{ zoN$5AQHR7nCK{`7ApZNy!&S4iN6t(vdQPW)&Md9P@Zfv!n0hET1a4e{L$9}ez;!Iq z*~Q_S46O!fs!n#r0WYCZ_zdNX4*DlX%mnV8GvPQIrMj6iYQ=Qik3S8^CtTH6{3|Hv zPmt!UJwV2Lk1Lw4r}3R;1mlpfK9eDGqome z1>k?Cqt+YNA$^3NfF<7Vms0=c{~pH-&bp;(H7)>MkXw9?dBjF$d^lk>m!QA{lEOIG z=W&D|H^n~6Z%n*x=yR6f!ush*7ZV(=;XGmK*O{q{xnHouyX4hagi$K7Q3$pwm0^1a zqxH6$hc}li%X#%O|Fi+JxZ5^ZRV=Qa^6qB~3m>9xfo!BJYCzx@hdle(w?%Z%p1W>S z#$;+{#X=jYZ7S72vw_r%MqKFzkX?0?3yX$_?;&OgH2IIwA9cpe@jbn(3e*6pe8mhC zVAG2@8S}Gl9>RlwT$~7#pybw^6-#^xI57SL$2&N1m}O`|#rW7pO!=57Nr&<%N&FkK zO2ZT@24(EAwq%$Q&tBz)Ru4WB%H}=*@k{A`X%?Fp7araH%VAl?%XBZj=clgrPb1MH zaUEn;-bi&8_}6nm-aDf6;Bx5-7Nn5-?oST=TEITLMMs?hjm->{v}x0B4cwvwfBc{t zf3=+xHE#wJ5m?kB0XhhxSe5>2>BVJU`)jE=zozV~_uCNltaSPTnE84;W`G*50_?7! zLq$<7svS#^{hsjzET$Ws5FE@}?#5K7eIY_<;u&d%BNF_;Oz8N@mCqTevUy7qvOgw> zB(ph5-!(->z*2}~RMnuJ=dc5|^mQ`x_^blgk_pMSFt4nvG~oKTdK~>6y9Jo_xupC{ zT*)tBMcK`s@GX`Nbo@8clRW)kX$Xglb*^nQOj6cb0Bt4WP(=c!s9^*#kZP!JGkf~+cWYP`Mf&bKo&eLQY7gjxGQOgbHjL8|X%RzDJ&JEdLt^D;qjS$h# ztjn6~ttCUGwkrUNH6MA_Y4HpJTi<@oo#`|OKcxXrBnJyF&g&jQjSiaa2Uzvga-j*Ta2O66bPzlT}?4 zl*FJ*xrqYH$envab9}9_c-#a$6#3c-3^PbcD;D0tEE5j9+~@C@fW+x;N&}4+Lko?c z>k;8DTNOdG4HrUF?37D&-N~a;#Y)e80JifX6?rZchkN@NdJ>CdsUe|SSlx#sNUs(D z5c8D1_;q8S^e`&6Qks3btm@WC2F2Yqh^@bz$p1Np@}}iU8V~Wkd)X!O#VXgGM{$L{ zg7T0~x>DmtdR|qe`bqeOVl_U@Q-SWD;mG7bhy8 z2Cw%9);*%1b5E5rS93^`{HS}e2H9>g+`2j8xD3(75GFyCGmTWXG8`N0Y{&a6kOlWg z9vra|=Pk+7D*2hIL0)<%jLg!`xz}pEE$U|+S&d&0bv7tayx%;{r`+rGCVlEfj;0a# z(&Ln9sbuaWrg|ORY<2)Dw*cyn^W^3%p~J1rOU_>B;3^aEYYu|}nOX}^Z`KRo=?)H@ ztvkpH;>EnwW$Vh718?NpQ;w-7pg|Bq$mOs&@OZJ{@L56Ffh{?eGd>=eK=Ka|E$x6g z_qZLivV^&LL$;Rh#{xc6w9-{cRFMwYl+qLbAF zn;l?e)5(F`K!conb@KwzYL9DK(jO@GFMoJwNM)m7*q=G#p`~)o{oyOJ`7!MCbTijB zOMe4kfCCVN10dRJwnyA=h-ME5!SvS|aNf#lZ}WZKcAO>8FiLi$@jINZe1cd3U;PjJ z)!9r)w%uOu7VQ@%ZT_KqqPbv7I!k&KL1DtBCQVCCWwc^JZA=sSqdfchmd$&)Jb#1o zqEBd(IFf5;3oS7U66EU#EsM37?TDP~y|1rcYjSArCwp1^#(yT_Ps|2Wy}F0%2df5a zk}9yx<@fw0$XG}W6NlpC-bQ|1`;>vrsAD3ldR!4h;>%#y&ZpNKK~ip0n0jI?xBR~R zQOQ8)&nEGNfawuNnm42%sI*IvWF=(o$CalzjUbgy$#ZFT3;ajE71+Yj4LV!pdNwwR zoL>K>XbJsygga7fU9$IF{*yxhtet)q{C~KW0}w~lr+zrM{8!|ud6OW*-yfC&Ff9t~ zN5Ld)btf|eai*Ik+IVZ0y4fWAnhTQO765V=CQ=xW-2S|AYmgk7*_v?+lq!Y}jT;Z! z%JdTnLh}L5*ILmlq`ryK_uQv!N%b6~N|FI0ieL`_g?J`0jh8O)9w+#s?gkOqC34y< zNUupom<%%3MF&OTL#}Z2C^hvRv}GF?l_ekIlwu`O2}ZCq7huw%({&g}=fB@bIS5fc z^^J@o-&uSKWBYaL?s{k+@$>=DDD#YgTLAyrDSNLoA47fmUj4x`n;g~De%*?U0l?%K zH{2Z5mzT60%RX{(gB&yZ%zY#?Tufg5-vJwav78jA$ZueFca?uz=z*j4_9@TWkpJ-+ z*9xSU;4B)rUaY}pq{-aZz5P!~(L>6j3u<49BxY=NUD9{GSur>s`eCm2IfdWjbB*;} z7cLWq9Aj;KI1!g(Z&ViPKIk|QzbO9sS~|b!=kToT)%^u+_cTtY@%e~*F7Iy$j$0XS z0~o<1bkdV78!jMT->VI>@j4sb#@)z;jVwQMJXUYhxj9qkPC3`m07OkO{YNF^W|r^qrqy?5%9;Ic?}Og}j1!!k z8miRGOhF$V!sQx^>dTrOBuKatcV$+(-!hNG9Iz**5|*J{=9Bov6WBJ%v|~?+yUtr6 zShV(&J2cipe+s>_%mx9&8uzG>fVVuS`@0~IC7x`C5^)k3MF}*FieR?_0b2;ZW!l}3 zfSB>t8+h!x^_Oz}2cfmTpJl5DYYcVr=&LhN*OVM8&vs|n7i@z~`4D$@&i5CP4L^`G zo(3O>!=i692K;JJYbu?X5(I@~PJJ!aU0BJ*F=QAOxMbhB0XeGiinJekI9bH(7Y+7Z&2)?&BFCp z1Y{z^X;27peuXut%IeT)D*n_dtVef8z9BdOY3fLGk-aWoAyH@4 zf1la%{GLn26rCS1+KBU+(|?ROsT~}5A{KnIb0UP#iBUldfF?PdOFEwmxd#8-kp29{ z4{L*bb+}uB|Hs(E4|y&u^meDmssCH}6$ET8u_C+A{wk=rVG4neg+fpF!QUGQ{bL`E z-Txx#{yz(I{Qr;ge;54u|E5e&vchSKf!_5^(ur zfgqFIZyx%H<@fw~F8L$cNpSS_68T-%hn0t0F5`ZtJ8YCv^8$D{-*OM5LjXZ zxJ@j*9QLJ;WG5)VKB~+BBT~wLn`x^ZhG-%E;mzI<7ri>M3M`^}BD9GDgo=`?2}PWg zfP-O2wcN1A?JzNdfW=T11Zkw+8i;hH{Ayu~omz~#5PE1$wL0B_5c$9D8`gBB(tq1B z>z#3xc!P3y9`LwlBRr$%`03WqlXXq5#p39~1?#%_PQ8ceY>{na!nke zzIf9SpRDMF^o8vRD`HvSf^li2NHJkp7PB=*UKvVjDRZ$SEVZa0I%d{!J}x272uZBE z+*e_$O`l&_P`H12=R_}&0YVJL0f`~L%F!5^p6qb9^EL6Vqa^-?IB5y^Y17{i<6}h< zx;AA~l$B4YjKpmEyLMccDd~KPz>D%ud2HLCj`g}#@m>koDLl3SAB`?w`6@h#Esk@h zgMofxxWaw2YTy~trzW|va!Ad}$utTKC|6@-@|p#w_?@pg-AXs1zv^R2^?oy2E*6$H z9^L0y+T)v+bu+SUTe$OKcXi~?!;?69AU#Erk`8TJ?IQqg#Yxy zpv+R-DIRCi$lo8t;Urt%81xK16!!T=N-8dR0wYVIQt2oLK1vO7Go!rzwaX!`(ELj< z2K~2(Vl~8gRS2pu8;wU|ms7lzZ!rJ;F5=`Hn6puJ$aDx0NRk2slZ+Ag9FW9s-6z6N zf~Knv5UNaeC3`~>t(TO`2_T55S(g*?+SN`Q?KZ=@OeOL-#7l)~ ziRQK$eptI(4O>s=bpE?}rBv6*W8;wl=6P}q*)Zaf=(leXxMj0f}#6hFGQg$kD( z*6m`u6d&vgt`gYffpE6d>(*_#!LgCCSWPone9!iuEY>V9uw!lpry)(zp%_1X-h6+W z!5xlTSW)$A2Ql*-bR*C+;M<^AWI9plY`HL@Y)@lCo9{eRO7zI#P^s_C`O$fnim+r$gjoO<0lBO-O;f;g?qWXD90I;3& zlY(fGzA%+Or+&GYmJ&pnbv!_l4vr>w+PE4W(Y>7U{H*=9=y{I(tjY6Y`3JUyEu&h3 zr<^ey!-ri?PgQFIIjA!n1>&F zYP9#*IoQwTBdh3cI%r5{t#iV_+b$PQNQj` zv`=7Rk*+4)dS52U@U}y%@nPT~PQ&duwKDlX?2bV2h5hI!dZ& zd4q|@`Lxn>J&&p$EFHHzN6VW{8nv`AB;*sO0h8`(VULYZuqSNRuVQ^(!zWpvRDN$v zC`~`*1bqXm&|U~URG(TO-C^ZBV{3Bs*}rmTRbpBCz9v7!)yNX;d`q2TqZ&xyTHnKe z%hGQB(z@!`x=ZX+95ou#XL<4E`_;*=)C+oQGmY3h*aV2T-O1*b2KDTY zeiSZ!*YqXA2m`sDjDKY%npwWfK8sNZ`ME`my4C0CLpo2N*EQ^hp$&Q0syMr-P7EtN znu@rA9-48ZQar05eVft{8RKRJUX?)CRr!sgjSrG%4M7)X2jxt0;IU^_7dVx9%On2}!AQ(ZT)l=RiETXa1d&y1x1=OERb4@tJTop10o> zUF#KVas1cQa}rkD&g;0{1}!H}Vw!V_(~zDTpzqX;`<2T#eo8~i-9arLyKpY9E|&h> zDTc8L4;!S6FpcfP$e9XGV%{I@s|;~gtVQUG5O4rC{+Z>)zGY3x4zM1fMuwi74^=gA?2iBA_wZ?HBgq+F}Ga zm>(Ta*ym21&e~p3%krcMBCdvW;&c&jA>KZUh#o2q>7}w6Fr?Yu@;-t6bX>VbF~Ez- z(E!LQvvnT51x6Mxs=q*Z?nTH~>9W>e@K+m2S*BwWI1}yD<5pW4RaDHohY+?Uohyg% zDq;+~mfwzu3yME;`?c(4^UmkNlC2vO_ln1`)wqL)9}Tw;-J~4)q1T_-G62e6!2QO1 z3wLfNZx=Y_24)?Mk!inT? zAffkwAp1Qeo)&LSpyCZ;y7Nv-E^R|%qR!0qAj@Q};7slPbPsDLkCR!ATDg9Xkojn2 z{Hi41-XeW!>C6dvC+2FzVgI`VNvb8dgA2L8ZL>*a12WC3GY#E=rwC^;19AD%S+MBQ zS6go9u@>A!3GxN*$$#WQ!Xk*LJljv^Z0hyF{%i|?!VM?Bo{7pePd7<&;pWdvBVoKB zYw?r(ImnCyy z6sV+?HN_92$5G2Ha5_a|!2UF(QA)fzT`}}~pi;{p;aOlux>J>7q>Y$Vi*(KrB%vCf z3aS{F%`gR|_j1%}=4E~1csW|P#&;IWEh#kyCK%0m4{CmCbaFJyfoc~nfReij-APyJipv3n{&>Ceh(Ir&;QRyHsSk52OqT2XJB99|um&!4>ivme06dq?&GbY#J&?-Q4PaUT|ceV#O%Rppa)4*>{;R*;Z zZsbG>#c*gG#H!wF&X=Ezv?~h|aL5Xdb23&Pl8=9tK;B)c9u=SgCShPWNS@bi=hfDN zRn~ITj}c}}YHwx|pVOE{$KCxZl_=B=K;$0J4xh>R2zuFOsS|zIgGnGF2qb3Axo+Fv z$(VA~aqQ3tr~x5DY54U|WRKO{B-1(5q!?$Z&kk$719Hr$G4$na8lJZG5gb&QXn+*G zrolYROT@NypR<`*z_W(`@H@rVgC3cDEY0{#zC9oZgIAk#uT5kzH%^HMMC^TuG3QgS zzgD5Dzs}y9<$YY3?$#-BTp9ULVPpr@^w=2yCJ$&5*AGx9IUatIlIB~A4?0Tcu}a)) zy2qsQIP!Kzk@?vn03o)HK}3?2a-Q@Mg+DP8buab+k49a+N|z_zSGj# z$21=alIJVNQA@^Vu1!pWc02FTN|#MJJdSdcOb%MlYi@81yk(~Kj9r?`A|IB0>>AP? zk3*{f5y;@u#UeeBkJCc9xFrl^u^1gs&8$ct+?~h7G&SP#`L^8hSeY$?<+=47$DTRx zqa?PxkZ(S4pqUM&vQ}|pTwE#O-49eS5u0^jM}A-;y+Jr%V}u0 zG}MSKZ@-cp5jYUFs8ufh+Uvd=h!e|HTuR5>%CBBGZz;FCOkmemm{x@)w=~so3kYVk zT<5>!m((ve9XHxymp2^a&)IkXI>L3EL46_5?X!%--s?X`-l>Lj^2GDtZnbKUP0mT% zQS|#IkbJbh=kL_k=Dc@AGoR;E$}tntmwY7%;$XAehUfFXjy7*d-fG|scNh>kd#?JT zpjMtMHz73bppAgS=G1t;AQ|>Zan>x&`lapKN}2vl;sQKstiO2{u+1N*5V{K=aF-P^ zC#um#wO=gz83H+Ev~zFo&aAm6-8-L|X1FxzxJaY!gCyX`f-51yRX(rzJ$MmJIkhV) z!__u9~ASn>wn9Rb>+g?DyX>fL^Q#7^-RnD9#uNhrSO+KoSM zol2*BlC;P*V9ytAb&cam3c;^lW1^8o0>n;2h0O0+8g|rvMr*P5aETPP8O% zQ_*?NX40<=Eo;A6dJtyw8YrudV8wEssC*AL_UW~N{dpaX;?ct?(i3^a1*y?3g#W>H z2PO{Vw>-lESqW#gPga(p&719(mZauq`brY}oH|Us-g|dp6kctlx`(Ugj&8ccT=WTP z;%+m|F1IulSCSTpor-$mxo_f~d!zcE^IQR%64NP=MqN=iF2$6A$JCVL(3sRfoRWaC z!v=rvlz;Ke{LBx8RMuY zg1cgzGrjITQr}TOuuN9O4A`T%}VRkS)0nE_AZ*%PBFCTJgCXAf4;Bujlezf z2MvoZ0z|lWS`1RK$bNG@mn&aT{4#lkY@d zvZtH6Ma4in30!hsYLe4Pa|ig==X|+Z)tJy?jzt@n8$W~gx;Co^N4KP@34bO)@QX*- z=5dfEYxXItVjq~zxTSZec+jhRg)4#4xV17>ttjnno8R4Cy$v0!n}|`kijbS3-`%T~ zkKTfpS^GR?jK+FPxHD7NRZjJm!X`)wA-?w;Gx4eR;m~QDNR#B3M+c~p>RD0OUD(-& zQ;$5bSmZCSbUnZQsVe!KTt=#)-KVu@=c-<;mh|ee3kS1<#9vis3#8}+%#B1*dr0<0 z7~9%pTl{$$7DVv2DAoE_T@`gtu9Pon?-awQ2EUXXHV^a)4E;L2`xaVt61h-p<02hG zcXbf`rM#b-z|N%Tp>{?`j;ZmKZ7z3a1Zz4C=*c|WB0TQZOXE0^kCf;xTDClR0-JoG zr;!+f0!hwAvIEK=)4Y3=BUUPl(E5HY16gfDztrYfm)&!Si%NMk@tM90PaE1rP8xpv zWy)}Tw@Lvv?%Ocso6@qb?Afe3bL9i`d(E-XnuUwO?o5@_(3e$wwb7YB)L0J zK4jj1**#J-#3r}kCML6NUop62a;vABq*Hx5JRUw zxf%I<1xR#F1F?Vd+kGu_i&j3DQ=#npK=C7E4?@lNN5AMTy_x-caiK+wQYY5o29w$ z!Zq&(R~fg2o}Z*zm8(Tlh=NC)Z?;xz098ySKeuYU*Jo3rJtYqadh|H z;Z|0)$!Z|B>ga0qrfgF!Zx*48v*Fd-b?KDCm;G%!rdr|kb9&Fr5j$2lzDN-#_*@7H z3zYB5%+yX}6N*-J6MIau9)B~AdcWctn{!P}AAo?K2$&Lx0+*h7+6RdDYeKt`X&ogX zUc?AmVRN4Oc&^$zUjFc;$Ir<0Is1kQ5zaQ%nMrRPufU znH0GVLGk%?KLHzRO5(k}>Ua_{1I@Dp&&?q$IRt}3Ik$Ru%S%uDepEW4X`s99rR7NZ znVV|%`RUp;Hm2tqTrn4P6E8os0?CF&sK)%+543n&KBj%{`?WK%_t$3K%U*ZoyiIsB zoq67a!(4Gb=X}E0<1WuIM=+!a&5n%1@g{`NKy>IA2_BKO<$1l_r#*hf2>v=J5_I5B z32bf3I*-i>pGk;c@jUAtN5W}~dt;sD6r1ZWnW%Dz7oh?6q-II%0f>6A56L)|*BrdC zp=_0~7Tg6zFE7vbfBmUyRaqL3Bcbjq>&SGOBa!iSz#IZ~yp3@ZQk)~Pu|Y!mmvTaE z%eL2BtRtLu0{mG15Z{=hX_lo)+p9lR@FVYf^P&JC1ys^(_X8$$|4RUfM*||5x!u!p zf5XJ~aH;^-OM?ks`7Z>TD+UBbp%rJbe}gcD{;vRvGOuHm>%Xul^bsHOTDO|~!*Qg~ zfNE*&FC+fo8=!e!piCl~F7w~`bO(ksz~is}(BVX zAmWibW?0tSaDYFmFzViGS>`w;bss@4rNtNYZYU{0h(F<}wb!6sIi(8kCXf{Y{Po6L zz`Z};|FLoE-T~w$R=t`6$IJMv^+sSGjwEqAe2%lp2BOyeMa5;QXI_0*i7E6;Tr1u= zBxi=-6~lgtm&l9X@X%HBZl{=M#sDH-4JCFRbs&d9y^1c5J72S^B=a_57eC+c;@|%Q z05hYRA)O>{*|`Wj)T`$w<<^(d{gR3zF71nqtPw2Bm2xz@$#exxZkU<~3k%6+#+=@c zOXKYPfwjf#hizNbk_$xawKoxSxVTKb`_W zqh_H%Of|nD<1yLtl~%so5F&(&w-fVGIaV6Ga6ub;YpVTpDmO1isRC zo(JT-5|8<}wg8-ce|vc(krb!YtER|9^^3&in4gvFviV}!%L&_IBSEwGq)xZq7GrcL zlJ6I81Q8#nd0T3Td{%w^imw+qxKcdLwyE&U2*?jF4mC?~$#~ROwt#Ve-p+)p>n8tl zTNw>OH;O0WH)ROS5ak3*KmYzue+++Dl-!E*tx@uQ#Lz$g~Hh zd*;ieUHH5-Ynn8X-h&_l+x~>o0`4hpak|{$grtZrAT>Zx)4dQjIj0Ev+J(a@p}hxN zSKJ)f{Pb4mVBR(jc<{OGOum2A>jD2vAj(?Xl7XMc&ghkOLwDXsR2HY2Xb*j3qCyEQ z%|xh1ail|O>vE##gW1qEp9Zb@i%CY?$P6yEy+tD+a*Z>RZ8KexBJov+XSbvpY(ARl zxL{L1$85=|E?;0wmJ$DzPfUQgBB7w>b%E3u65q?tacJ3GWSv@Q=VTvEy8Xu*tCI6Kks(P$Em<)LdpFLwJ9DfK~14LN(6O zK^&7|ED-3FuEluNlZRMRGt7Zy;>5XRTv5QX7OZCAsff4&>dwe)HAa6Re~a>!z|7;g zyx+N6zbtq~3Lwg=!c27jLt&ND;Orn1Ci`&8WIfk8Q;lgL+sVqwf9>_-#8lbLP(|1m zi6~7&+;w7m2@*=iEDx*SCeI94Y}7!K_dMMud(3sQAHsZ0ogZc#LZiv@cC-$98~M)l zLjf#Ga@>n#SAa;+(8o-)VhBNxx5&ioZODfIKdAevuqxN6TSXeATS8j8q*IWRMYn`> zgM=V0NO#Ku1Ze@K6i^V5Mvx8(1*IjW5m7mFZMXYB=ki>fi*vD`XR}-^zV*G|`_4JX z9Aiv#I5Tzh+6Qds9HyJ)5{79BD>hw%40#e!Z0DEOOzmgjOHs3d=Xd<$Xp_CdELmM- zrYjGl2WcPaDPY2%$^UU#;V@~BjbV5m*#Ei0SDbN%J*k*UD(c+qFjlcvu7$n)8Ieew z<(Y7*KqzhFZL;v=vx7ce*ymn`2U1hbz%0L8Ne2&fBkR+hbZut(DHEs+2m~3^O5;BvoHI&wAH2fAJJoD?9ZHVnH+Y?pI_^W_6 zE5tej#8qZ6)~GYRKj+lLyXn*6j{DWcH&lFr6&FiH(@y6|suV#F5t*y;@ zaNha5Yn74v9lz^P*X3f#`|WNRqe`>hfIVHY_51O^Ec1L7CTrUcYf9mq>~epit@ z5}VIS-6DQpcl$I=WkQSexmbk$z{Sbpumg{8x==$bj&P~%y8s0@k!8x&Z7=_KXCN%L zz$1v~yy8WuFIZ)1-QCgoFer*%hOh|ZQbxaPyKP{N<2=})cur_5&y_$>XqGJSQHq2d z1{WGeWLU+(HchjInB{Xl&w80Q^s59Re@{poc1hEUBB<_VcVq5uIV2M{+f(& z+F)@BC^JmY%1X?8eeY-=R6n$qCTgj-s8VjJr10{(pQ%tkQVgSR(4FW&QMhIGrpmku ziv5i#cgeg3oGd10r|YlImXL(cFx7hLxM|^W9Mt-UsZ*X6^Hx1dI@gds($mfIO7K4b zRp6>%M2$1ocn!ZTLF5gU5YDtF#)+NjxAG6PaGot&cyL};;?({qd>;Bta8iZ(RxzG- zF@Xe!k9|h0?nH5V37%Z>!02zSQA=mSZS`K2T-gDFh?b<|kT-%~RI=1=&Mu3oKarUm zz~&4xIvbmOk6tlBTCR{aV|ULoUJFN|?4CC%@j8?1$o!r zeoG=OI*%@5kz=2ta{-xnwVI4nz5&LG_ndRtgO|zKywH9+GAwCG){J5`f#U|g!cNxx zi^u)KeWE zz`VJ-At~Y$$c;w6zm*boN{gWUBoz5w5nVnvQ95l;O{dBK6ewda{R#+NJzl{7N$T{? z?`ZQ05lkz3d)XfZLU>4!_peG4Uqy;lI~BVGUnAvqh`*L!DDV~~$N*=O9&drq#%bxN zw~ajKH!wfsRg2Y{@8Yw6%#J_U9KhByXfgHAoLfA(^eAoW;jYf_g-m0?y{E<;B|M`| z1>NL7Hi&faNXmFGu+bI#9#i~O+?9@dh$(zo$1+uGA(2{$jR1H4feLGSn3JGt*%v|W zcgm)4sJbW*R8b|VL?72KMaqCO?M7~FKO)zeJN~?kFh+@?S&Z8cB^Xt`Wz0yxLAhSY z)bXAehX#jLYwSRD2FqQs8x>5l*>2>%E%sU)dZeC}E(L-xPw1^gSZz@{qNx(gJC!h~ z4sY0;)?o~y(`TyUZv3{PqHipVO|g0D7xxy-UD$B6v2=(z{c7ZC64=)&1)T zZ*a-`Un^F=jmf;oXtu1rp&&dcCUlallHA5`oc~DkR=)$5*2|^t_+yN6l1WJtd=g(z znGL1xu~GAhCX_~?oCxfAT*icuMFz#yOBMNGY_N2}vLw>D;KTcy zW!pyX+6rOPaaLa`z2Yw8USS!-cz7{?V-A$iu0`RusOq`BcR)&7+K(^R*2QlvR~BcP zSeMbAa;n(KJx}CCj5+)%q-O38X!AbsA}=&3&4%@74l)&|HkZ5A@=>U{U?L6UMT=7d z%oPPrJddP}RTziO1@G8amuHG&Dp-BG>TX9$t-|`wH~uT(FxHZa|GtpYv?t@F&2(w_ z&w-DDJ=Z35N0A3xm_oYm(zYpONk!~k2BzZ`4xd#_Rk$1le5Xb4Z!492TEIVl5sQb5 zlD4J!w|CuUTRa{$g2IggQ7r1Sy3y_|GQy8zT!!7dhYlQe0W+6p;nrA@s-1FUF%g70 z0&(=mq!bpk-uvG3DtvsO#(SV6(!0@noTD+JD%~y;XU0Vo$3)16Cq>APhl7VcrZ+8C zgWX)1O5~$W-*16cjQ`lSW}L}UK}Bpj;U%#x)1<}lr8jb?65HX2S-~mM#&tE8x>ygy zNHEyRkvQ>>JQmF~iyh_*5bRV$J`N(7mPxfMzFyGr_^Qn z>Q*tocEA2nrD~^2*H4tlt5$)0$iBj}!DlhN%Z|fA^m%^QgG*Qh*In^Nj5ie1LVi6K zk0$FfYBRc2KyAsJOdyXICN9XU@}7v~3PPw{yCz!d{1Q8o-~fX*xuRV<#;~CfcRc{s zq~S*_PA@~q-fEU5!Y|mbG*yiK{R*}9!4;8L(|uXzQbHD8{vjVQas>58sRW*OWNVKi z#|?yqvM9p+G+R_}4VfT$zXdHlx$$;4vyH8K!~Ok3*YD6XL)e{c>RinyR9EWQDG0m6 z1rp%twp-{{SffemNVCY3tNKBd>q8>E#5wU${78fvAuI3(dn(4_TRu~FgjK5D|K?$C zuGpbe%qi?OJcY&G!#`JgW$UiZhrYWFRAz^j+%8QL^T$uGV7xS3f9U4%!`;}oW_`@( zS6kF%xsGNr=U1`-IuX%17NZB=3-!&@_t8d!GxB)y?o+Hks96`p^8HM9rDW&Jpi+q-$!Zly}|GXoNl zx#BtFES*nJO@lOFt?L-U5;v(Ein64*@h;8p(S}^6V&Rx8RipJW(--(CR4`^&Y?mUY z_Jd2|<}N3bSLmKLk6Q6kJm%`Grb{c;AGCz01xw6s;4t^Zb#orgPV=k)-Pf z6mJ^*ML+&fqHB(&QW54%$$s2Xd2V+j|AmN|PX)E3xm{C1XGrD{wmj{(8n&g^1Vg-x z>Q#4W5qc#38O8mJ^3nx^oRc>>%?9;l)-7r_^c@LvZ(Rd-7Cdb)9u>xmcsRZ*3Qgll z7G04Mue7Kl4_=qyF!%C^qG%BKY#9u|z?8|L@fw$Aw z$Zm-h+0K%Y-PTZf^6fG9dgoonAil#HJa4^jz60(?0x})80|j-$2%DI3yX>&Wp2zkk z!m=K}mlc0Bvue}}a13jzJP9n7N1m55X&IMFGIXyGIq)(WVa@9~G=*hz&0lgNM4K+s zt}Mh|oy0d+xDipIJ69)j)SfKM6HKUeg2Ol-s_a99-NzBz1k4#v&6rq{^El``cG=y83?u2%SvFR>;VQs+NoKH8>X@ z-%1ovV^3UMB74G;lBugFwnM;yZ1%SZ6pS(JtPvN=)MQ~&mj!vfkB+*EFKBT;dRVIr z&`$N>+~5N*lIW`4%$KU-y5s*DSH6poOAwm7KkwI@$f!`U>?nYBK!2yS~eqYGIAJ)tv69I1M!v(<7zr5uOy+ zu3RX#*Bj`2X~7ZYDSntEB;09S623WV?PDa#@HSFQ=q+GvJwL^LYp=1$zWO+=~or#0d{&YA+=nE6PeJ)u;mNYNx`RQ zUjQKpTEhqwKA&&xbYpJBa;qrb-c>+D4IO$B`+x9i7-`jk){FRROrhwi>N zHgd(iO3j#!nNCS4uwW$8bh6zKG z-=wIoJ7~RI{JXi}ak$-nPn-N8mVkt>`qDxdocL;^weozRG4zrzFf!bVc0(i07NW@rs@8-c~5{Ra(V^7VY1_t61-e9yW`R*=3gRd%Q}8m|L;RN zcf%r2tc!k0|we^-~TUf34M&n6ml*Art{M_E9u6+tquOWj6}YcUlO_xdD+KH-W{4YOK@o@lPxbe|TVT>l%*+L|NfzC>iK8=50BaVrX6^k1L&9}ljT zM7encB~}H%peA75nw17-Tn+#e z$N|ele*ozPjcwze^Pf-MdElBONlUR`GNLhtvI# z13p}`)}|b2C}L0}SYFzI;bt26ZapW{H_C!NM7nClb{lM!i@_K8b$YhK9&$Bsvf!tn0w(MXBkuRv>?9(!gYJm451{ zD|B`bdF75y~~IOmKg&4iKTSGbVhy!{wF_o2Ka3LO|=l}IHqa+7H zoWK_pPP@Ygx544%pvZe;rP`DT!8uqiWBoeslezmzi2wmTd`1re13iseHn@3ZrTrtc z_?Reiv-80QSV>cAFI@lfpN5ScB3D9q7KCv{l?}6;1kWn)kUM}hlTzLXfcqw$#Jy+> zDe}>1VrHg{WUgC}NX~&iMYB^e87=euOIZ4AQ&Ijo8%YKzWU)*0$MmaQ6uZs-Z58FD zW?s&{^@_uC(%Y+t+J3t9SJUOq})U$7jesWi6Rn${!I`lj1 zV}CLoc?LbpSiPkc#H-s%mh;stYrO(Q8QOf{3%6J+Q3j;e!g$qm#(u^qTBfzfUWw(F z`b_b%k3C@T;Bh{MhWAZ95>m4=^$mk^_4FI>c;9SU*^-Dkj*pv_7|$_P_837%Z4SMACmhzjiRzkmCuK-&yv;?|ld`L*R+ktLr~GWMq~Sy*{%sWe z9QPL117)T|X4~wO6yOU@fMw_ul4r1S;~!bR$nB-|ivRl1#iLmzr~`*x<&DoaV$yF= zc*;oZyu7DvQA=~9w0okH4{U*GKfk(r2W2w0c!p{%{m7?@1fVXoA0qZig8rt(J?JF9 z2L~IDiiDctG7Qlf8&d66o9U!0qC*|jct2Ew_4dk^qwuLyxn)*vXC<^e`vvVVW3lSQ zb$L!TzLq*m^|`=IJz-K2r81^0(h0)y^sAhGvA1G9!3Zh#7C8m|Z|2W=0XvRM{obic zN%XGUHuM{-%J52~&AtA5Hw0p`{5Am|mcr`QZ!-e+a8|hC*=uaD$p@bY4Y6xzFBB;e za_aTif)Th1M~)Ajd)q}0%Pcp+L9aPIS{ z*jL84=Y=*n3=d9_>K#f6H8s9ki?Ck;i|fR<7;;%C(+l$M$JTj~YuR_P_#altFSl3hfd5Al)2Yvi*p|L(-!$FgkO%Q8o+kCv9Po^zk&a!BZw2jx?c+Z_tn< z(Kb)co-}a$6$CU9CbMfyW^K%7)6Z#ZsAgPKQnpPNt8?!0>D5!u#-50)(f43 zDuy zAO+ag1r0`^S40h2#9)uP=u11^eEhHApdX#CJzMX_?-DOhJ1nuW!J>bC6!$`J=58pV`Ns`phzaiZ@~Cdfr>*xu@~`=@3M!_b{Dvoy7A^7n%tfx7W?->cq+ zLgJnPInmtxY?cCR+51F2^v zvO35_$eOhS`NG{eK?EOB93CQ5F}EQYyc`)he=D{W>(tEITQ_kBhT@Dc{MJiNbZg#I zdXF_81ex=ap#F=x?ih^p!SGw24a7J&WkQa8MRz$9x(Axzcq=`A7^A@dg+&Q@p++dO zp>3o_RX$$Pv)w8s3g_9O!7M+d9x|75hIvZ|SsTu#e zT!pQQBi_I+u)(oM8@~V3WFtVgdIq+>Jy*^oecQx#-kt})f2PxKYK=;|uZ*)_mKQSGnMX%^kPsaXH?jfq<1hG?>&*=V=Pgm$0a17D>gr~cCHTdVr( zdiNGCE6#j_Z^_jZAaaz(n^#(aP|X+hwJ1tdP|VdRbJn)R)ci`6gqx~qQP|(FSoS8P zJ?jUVFyn)MA`Ao?>)D7xOZnapJ)GP3_Js6B=c|b}BxcJ7uG-Y!nV8?;w&Pra2?Mju z4jHvQj;SkQ)x0bp2xcE<-=+?^wiGuWh1bdv`@@i09y@!}5Sm~L;ez7lN^&)bntUFg!+%?$rPyU*q1x zSZcKU`z*aQe7pRV8LC+uI--6CdAy5-snrRLc@y!3#rV2<6(kv)+cHniQUd&K>x?ghEJZXzLfYj%ef=YsAD3OdI-uOcuS%ZaCs=t$5=}T% zE&}#|j=tLm5qkLyy8*WM51+(X+~%C@(dHg%b}J_Fq^01s=t=Lp6uy1!r)6~|clpVS z-I{$lcASsAKI#->h?{|5??xMPOd!!kWTS^eZEwfdOx5~U^o#7^1?k+K)za(dJGROo zO2BEqm-Z`i-R1?u7^chJX)7Vi?B^E(joZ+w7APfMB-*L`SXE1653jvh%fNAIO~-0? zSm!$yo~?4ag*SZoUBFDyKcqsyKZeyf8M(jLZ#_VL8edq zF{t2tlwAblFm#YC_|mDCOpxXy-FGiD2MlL65B)j#VuVgO*Jr6}Z&>A7r$tV?>Gz&tfl0*iia?*I^nTfb-B+98XYB7$lgnqfWU?v%@ zh|gABA@ydB5t~2>z3t|7?ou+)S{_VyK19z@y>-)SsX(W`M72_-N=zHQm+M)6^tn;8 zpNb!YmcAhe>+U?CtM*;ffbYqeGjv#m)%=)`z4an2@T6@Qz7o&t+Y(z$4HlL)@R_Le z4KsxnQLTHdUX@MM6Z5oXMq>Nku)KItw7<9gnP7NV)b)@OU*hQ>C>gOzW-1W*3XD<2L7a77^v2uR#j1zl$nTQ;7>tSeByjSk zLP*aim0MafB65!ylg^W`u44^68O-+Yyv;kC9*4Uwxk9$TwNmkFMC|$nhveyKp&chl z(OzB=&McK6-mOy|LWhf2YV3=`gs5`(Y5OdmhmMTw(z1e$(rpnRl|h${&L4WHCfGh= zH(r2Y@)K!AZDZ==S03Sa#L)DkQBQi`5ZXJNo{y9iQI8qNjxvzXXe$ zlz7T_F^@}DUJtV~#V;;B6u?Zf>`4*$Ldj>ra%KPJg7NJV7xx=Zimrr0wo54*{CAlal6g&*M|7RUwS0 z!ymQYf+4MbY{*XO0UNESAaNqoI^Wu+f=!}o)|fqRXUfg{Sy_b@QV2@@gX7{svE?uC zQr25Ee#A}j<$V;pQJ(%}>{0Q7+O6>UuNo3Qd8$cj^I>U%L!cb7t& zHK;3ec<-ZOX z&G#oN(`qr59xR)`ttD9`(%l-hAvC4T=s?YBIPq4$&z*xZfMla)xt9pLnn3<2v(Cgo zIoCYu570vwD#5Y-;poQ6?|(tH34Anfov}*lvKE7Xb&5?WU1^idrRMX0k(No6H;7Z% zEZ5&22ux@guiO-BJNxTP?}~asq^yPMzmJfM;LF`VqWuq(Y$Ha7UAUvp_2_3tfmO=E z|5&SsV!!rplxvgv5A^~^h@pFNibDtgp+FcY->v`qw}8>Rw1NTDi}j7afxc>?Jm$eK z`gCQ2e9TL7o8OaQXz1pvPo8nzn=@@$g)bvL1{jP87-XcJY5qEKU%QLWY10HVQ0ieE zm37tID0+XD!!}2d_!-&)TeKbxLps?U#c%_5CXaD|m`SD#Kt~eRbm63$P4X&)9IpcZ z!?Ff)u=T!s#E)!44Rs5E-Hys!<@XEU>SOGN9&=Ed-Ae(RBOQ@>-=*1!>v!Y+B7j$6 zbjigcFhN({uZF2SkZa}w07nZWbx_XJXIG$r0q&G6#pgyZd-3IDh2nsd9h8P}2H*o* z$Z%iuvE{r2dvPWJU?`{p@*&eTZ@RZ`<#B%Zn^arLwHK{-B86*^7{rRi7X_|j5|Mjh zl6S`=wG7)YC?PB;1oETB?ajO_1g`Q@@#B>t<+`~yDxw$6%<5YYB)^#)91Y|Ku)o82 zbS672G(MNn^yBlS_eS{6IQSy)2mQD=JPV$x@^DK!AZK0nGkfE6d8s!E)TZjV0Dd%N zHi9>~JQVAJ54xg>gR5*G8MBQLtZ^%WhpNrjPmNxHXR*pdI@>v2Ywgty^)TJn2RS0H z9l*!-T;IBChRSr$#=tss7XYi3m2rZpck&is3yK41r5*1%Fyru1?()|AfNHId)ia}q zhY%5=d9kC}3eBO6TutH8_^nWH8jl*n;STs+B%tPf)T7`FPY*S>INuMj;){f0hR3CT z(xBkZ+h9wW;$sQJv+yK$URzTEobwu(Pd}9{`$sA8k6_5yWmMIbLQu!0^?xS9Jv@t)YT>$xV_ zCzO3NOL7M=w(F5wy`?mXY%8m?0_7q!VaHPaS|Xz`V~rH%Wi3X*RaTzZgS}F)i%to{ z72UYi@8nO_b12j~6IYkhE6zSljHNgWlnZAs8A(CQr8!Tvdk=`o$3fPI=8rbflYo`o zodjU+nFEPWm9&YwZ>#FBQI=n?+tBd30HOX~{|Sr-N3WxgQd}CP!1c)UTaB^nJZfsF z+4To-?Pe9v_QcH#fr(Q4?`ULAUQcpACu6*d9^OV9DK{fx#`5yH`YU(gP`xO!hc~l} zb^7yDd6mF#1h*&I)0noXA_9(&FB>)!Lox9;A^^P@C7jWm04M7Zo>x=_B&|R9-**Du zM2DiVkCdoox&wL=>9;mWxvQPQ%p(hSo$V9^jXyyWCJc}%-uz+HX4U4$twvQ}n9bG+ zaM+^XPuqh=f>}A9T-IXCjmqZz{wg2d=b1dueg&eBr0Q{9Jg-g7K&_scroWKZR35#QhZz|;FlH#db@IdY*h zN^(iKd?M>!|K>z_+Bc0nDRI^n*S>0=q)j7-f(W#f>=&G)t_?G6TEoSgw<+=c*hfQQ zyU5F750GQ058hL+qy&%8+D}O*_g~h*Bi>X*bUX;pr&#jciq#Ni-IDMiG83L^uJ)Q_ z11thsB)cMoGif=^83zel_0CmorfsH`2Ddp0WRoN}JY2%F?S8lq{r2;=0Q%eRRoR)& zEe{L&0on%-)fu7lZ-gfNWBiDmpU^1e${h5*8gW2FIzapL_>-4rW^lRkS!pXFw1Nxo zI9C<={Ijj)Ep~3^r#DH?+}qxmTSYdMzVMd3);oQVG3{EBAFHzN=N>Rr=5qsZYY!~K zkDKLlc2L3vAnh6sFWp6f-KD>ethY+1BWsDrTgxq%WY14EhdWE-T0~R)IFzj$*7;G> z2i=rQ7}#c%tUruW)qd2hZOO(JHxzkeooF&o@T*XLC;(tZUt%wUxYC~E>s~)&SA%k2 zu3IVhV5<8aXeGii6ZXw|Ji@`cyyVgTb=^#5ujBoDzhC!&GVJq%p8S@0mnL!Fx_=HT zl@Wx# zaBZ3tuD%Vc4W|R_3^{o@geYS{<)7cd4)s@(L%hnnXmIfN9VBR0O*lBN5vU^-UYiht zpO5fsvMEtN!Dl2B5sj;>IB=o=S)KemVxsX%`;f`zzn>T`r--<(fp_}}ua(B9+02R# z2o@sjKI%av|9nE|Ot{<(BJ*`aAiJ0v%zjA2BDqSx{a7h*4f#!Un0L`+@>unqsZH_B zBw?=MpNn1vs+v_3x(~Y)J;SeIl8c{2=kATu+_><<)^17$FPu>!a^P^B5L(}+Fw27V z;~cQ~J0ZH0LkI_X4eq~Ryi+cJ+5YOOTP+3^HcYg7AhwFM-q&L6d5#f#a+e!=bZfs` zmYUIC?uC3~IiOoP{I!xbnL$7&5e{J*15e`43stLCHwe6z;P!2qp>M!pY>x5w302B3 zE_NObLJ2D9n|!vjqtG28BHqWLI%8eSWfrI>MmxHip$c3Fk=RnJ-?{dO)8TV#5Nf;`+WKKAe*pixoBsyfmzmr1Edf+8 zZpOZ{aW#7ZV$~i|Fh%XVaqjHoC-%?=!~QuWwuDc3ELWC=e!qyvoaEv{3PS&*l+=b*tbQ zTT%=}53hma-wi`5{gHCkJt-p(4y7Y#q@a0H)V-8oXzHOT1w~oQIfdcW$Hif!53h|S zueRi$hkE2HF?LB+w%ALyqZRV7(GDs?Xt8U)g*@i*0*tt^55`Y32|Ake@{VqCcA9tK z{7gG8yv8;B0RAagn^;*yz>Lb0Db0Kd27;sz6saZ*k-h!Ch^KkwxVjI}b6QLZ{zkTkCA#HI?^I3wCI!k9=zgSoK zJUn>rJvj%%P)aO(s^lhOR3ehJmZLV3`0wf%q2H>=gxLyb7O4(-G8T{=UQDuG95%l< zs%QzFdHTD@K!xw)$|TV0rYRx-c-`q%ziTa}LRkQ5mQAjYS$#r&V* z(G{&dlFpNGQVo^vjZb5(7)it3tt(VV*9Ah(_cPz3o7_ZCa|;ugdK{bcn=(~()|~K2 zpw{!WAedgr4pCKxQFGEdgZ;k5QTfl_rC~#3&2ob*q;t71?O>!4G!u`B-$Jx(pLlBt z>7m;j))uM-mFr+1hmmIFY_k-N!MbW58B9!cANvRB4aAB@0YQyTBe^|-Xd_kw0MLK$@@wGBS$ynjxo%a=k^5mOU&7l)TY zgDS?ke_+Unh!6CiLVy;|(g<_5*{qXlsPosghhe>0nuI1A?ZEG?R1fn@iC^* zpI&>2NWOXi-L(kVe7SK)p*@OOT2>AP6IgE2RI5|6|D16dSU{vP*z_`u1<5+bnCD6|tJUZg#@iB3E7svfV zMcnNBxUCo^iP$eLmaR7`tuw6_x;<*~-X&!Ap9^?X+TLQ@!(DnR81&$7;q) zoo2%~;E3a-HnDNF;~wLmvq4K3&9*%AnVT8;-P3WoGe?MGjtOce$bOKA@_R@W@nVpECFWAu#TL|a38=&Gl0bPxlHA*43 z`P|UaZo!YUd|3WV`P008s@EA4l2DvTVY;(=>lysLS*og{H_*oiBCD7W&CG5}-L_go z^`I@G(R_dX(U)&(+_g8&COL<=q%^xP>uE25X%B}aizWHsv)Dghf1m_AJ^kLeH>s@1 z8FMSob*rxE8mPA-QKcu0Oyfd478?zlg1KU|d74BgGC@DWp@|;>ZNMr00h{Mwda?jv zi(ZUo$hGxKqrWF5zfwS4LOiwHi$ zcI^N=oRu{{X!ff3(15+pQy^Ngp}2UgQ`2CNbW#Abqzi1{KdMrq7qn5S!nefh3zGnH z#_f8ijFUI3+0IhcRl&X&fBnhZ@gj_X z6DjNEy}~%&*dF{MjMk){ug%W|UOS}Lzw(2|rvD6Tn$cH3#2oFEk#)i}1ZNd$>()`0 zLzN#;84<%VL1Soao8jAeS+omedBYziR?>ByEacwCJNAO{f5#a*CZq@Zo_52Dkh1Ll zg6%H^ut&o0mqzWQi|A)ZA10G%*mdP+u#tAm8c$?edXfa#|49i4*0IBN=n#nekcdRH z9i85V;{MQyqHHe901}!ajMqtQgFP^_r5DD`N&|W4Ri3Ett_7b-{5n09*Nx*j#kBC% z^Zrz;?Ci3?L436c-AL*OKy0#ifNpW%Cr}$pSMN&M6&)RP;H&4-LBY8#1_x9Rs%?rY z^zu&N3GhP=iRw;@6PQC0@kD^hkZnUdxX?fBoce4E;)`aD>=2f9D@bsp%d3KmC1wp0 zFT~};ti2aKC|+oWXekY9w}l$VL-jcaI4{?5X`IXja#Y1jKLj z$~*JR%}|iPZplI*ZOW%|;kHMia20BwF-OY&Sy@CYlkOC|53sjQ7g8v{FFuKB5DsANfpDCX3zUy5@pu8a(4x*Gnk}-!zsn>whSHGj@xs z8%26Dy#G&)S#=BqNM_fZ^eZPo4T)}Nak{vfd$opM8njLB6#p=XG^DhY-@pER;ew!w>dhN^A4=m63_Dm{J-EX* zHlip;{^sP@(CuzXnJI81Ps*O%N!VK^cDh}BlVwO&tyv2xVi*AQIp39XyqWCBJLcp>|=e#de zwMJ0x_a}k@#Kr|N#cMNn()-Uz)kYf?g506KuQtK>=PgJRh(ft0-=0^g{rkCM5VzJ| zn-miK`x~EP<@%DW1EGKJwOPN}^e4K>Ve7^J69+wV^=?$-v$E9v$%`&BY7JvUARNB% z?7wTkxhOCf9^UXbs%=oi&;FQ3gMZ_!3dCKu|KkIq*FOlgIhp^BHUIAmV2I4}@+N<8 z+EG0GEPd(u>fiK%1#Sr`$t>>0KktD1+;wl0+CrEQ{mMRdz9ae`0Ld*lYz{yi+?ONUkRc}1)Km2}yxs{yY83Kn!tiug z+KjlB0MgWOfDF8rTN6bT?Rfk9i0|_q$RRp_O}2j0_yO0;sOS6MQYYlcUw^?2xg**S zlHO}?2||q?x}+w7*4?izh7ENDqo#3sz-ZEh^8?xQCE+K2t69E~^kz7eG;P6AC5#%t z!pa3T_vUCw(iOMYmY*Ppw}SF#2~I;wcFnvkZ`Y3K?b&*+84XT%;KOy5Ji5&ZWyqsi z7-I@ZVxk9>4;%rH+!z9P+g}4g@>Kjjmxu=xias zIgqRcEZ((&q{WZuD4?QI829V!j%%I++ zQey@RIeBAeLZxW^xY2sIEDlQglq;{k)Wfl_WODVU$;70->!UUDE3d$VL8~sJ^0T4I z*B-E0aCvm1#{T#X5q1gh8qzv(4{HmmQW}eD$oRunozTLw_wDm%~%#)9WW4kpg;6Z6Ih(Tzb|>s z!7vG>Pe7_&g8wqra*QSU=KzLX&3*%qT~XyebNo}XEE}S%e9nSQbN@aM(MqMq{Fd@F z`j$;I!SkREB`&oX)1ProH*trGb`Q#dA+Di^L&YDP>x{kK!RzF-(AglDFV`Np*3sMz z?^%&O=U+A{~qx`Nxg4M@SS`CwRz;NLKtQaNn`z|Y?xLn}%f$i+^t zTQBT!0dAcJT^(Q;nAV7PDl4(W5d{>Sz3pU_h8-qwE%OkS4~O(S0vC^fEe0c}OnR=k z!2+LYtKp;DJX;K;Lbedfq?@-;6yQAn4Gun&@Apw17eF`^VOL$+;5V{TYgiD(4h;0zG7334C>joGsuttTL?+$oeeer;=cf$NcWAx4(F# zAa`gl#?e#=LT=li|Na%G{w@y-6KYUXBg+VK(@94zYt6Oc>zH{_{$mt8RrK^Q2$c7{ ztSAu|xas|LnmHna)$P~u8|$_RI6tE%6F$9-RfI}{o&&(#UPo+BR{sSGdE$6+@next zUx|~4o5eB}zIyxTKi6bm4z6uqd5Oce9fd{j+cpm1;gkibqr=RL66?W6*IH79(839} zK8y9k0|k`H2*0WbvX~0tkIkm<>;I!@9?p^NlZ;>7>bVk=XMJLI-I~5zwY@K9{LUKq zJ@lVCic8VqQ-_t3?+OF}Pt8xlnnZtrQ1<7SUL=E75j#2OiEWN+L!Wa_FbW^^bL=LF zP4F_&dkuWfLTLB>9?;7C9XbxSBTG)+d0fY&lRn#}33~!vuZv}diS_zkq9)Ttpm;6c z@T~KJ<fF z8NEm;$-m;I|JcU*0m2O?EhnB7gAx_f=ZC3XYAx{8u+CoaIr2&sOu4SbP(YC@XnU(J zW=_8~9VOd0145@SWFSTA?xRS{>vl^dWED_a%xX<=aIF`v-uNM(5lI%?|B(%!FLUDk z<1=WKC8B0Kt1W@C&~9LWgBkhR+e#%v%dtFxZ3hGkFAQbggYUKLY|WapQL=B!&RL48 zM$)wEWx+*=Lvp8n%3R!=Gc{R2-0&1|jSq%Bs!k#%C3n8Tt)p`-j1~!nqpz9nA2w%9 zBv5?W8Od%nr7uae@ozJqdZ?S`s87$A3S>K^dDf{4&!Tm__vCZANRBGGdP%*Xs7L1Y z-YFOeiEvJt5Y=0>xwS# zppZ*Bi>_;L94%u(yW_6-;w^OPn8w@%@4}x)<=k}j zIHW6ke`I&siqo5M7X5Ljc&3A7H_KdyhsGW8YJo0?-x$9KScD~X(xqPp--&PUhC2RZ zFU?B+Y*ns+DnlQZect<9dD62=bZtPUb0Iy7A^;{tCH!QqQ%Z&lFU_ zNjmf>IwyBV(;CW0s;)br3KvD^v|4&}igBR+sEs=z%cbc!Laq@zU5{^^NTd_4Vt|1g zSy+kdwM|)(iRuJ4_r+qRs@ej~K6I}ZgH=Arnt)1XU5FYr>AA`(5bjNj3gh@-MI7*u z_Z_%A`7)8b4VqMGsslsKS<4076jy`lTVOaGVR0)*E>EmvdOqt>0-FA@=z#O}Ybu1xoRXY-=x#+Ou&|;7#1XcmSO8 z0P}+EM8TMtj{J2>ghYxX_5Keup=Ka9>ojQo#-eo(;iFaQGI(O!^y%)_w=Wl$zMp+Q z-|<%Wuk@%snXzZE|3N0Vl10TO@_(BrKGyIRSP zpDQ9$1hC>GMVHXyZD$e9Gx>a46b9|;>0X}%y9}prQ5v?M*M`ylOB;Ny;Q&%i zJ-vX2rX#DeU{dPw%OZYQUV5IB40)qv*-pCZwwu}aXj~T6)lZMXf(yIB-{&YzJ+G=_ zfXf1_EMBX(ppchkqP~njjs0G~GR7$+VQ)COO2|JJ?ucNnZRk#f|BPqEwgHIY47&?Q3TI9 z3Yp(R>M_sGbWD8L1U1^w$Qe!^o;|@n2v<+OC=@C_ilH|^^cvK6)q7}~)}zSuXS}W{ zYuhs{^oZEKfM|B%6D+|nJD(a~er87S#qV7r5NmstUfb<9|F~UR z-qwYrImZTJy~@Tp&g*xjAq0;N8@-45gdeql4KC$DRZm)NV`?pqzq+KtR#e1Jw`oG8 zmUX1557+Ig z--w(s2DKeW@-$GD&X9)iks5UfyViD6ki@4LrXnh?4eVQ+7j&$u1rWtQ+U#ys`mCVSpTLS+u;Il?wxUYM<&VtP?RxQUpL_DK$9v`zf6<*|M4vehJ#5$O3N~G5nCdDF zakdbM_$()`H&k!`v3z;v{fD!cXAx^9E!;!2ImMeN(HJC4v^m$itm5q1B<-~*Kgtdy zoL>C+GV6ArX-$aLf@jeww);1W9b-=KVLu<^whV>uwk&4?E4AbY_@kmJL}e7vjmX4$ zh!uW+`zo6NGP!*JCo<})I9198QcLedg-E~CQ{a9^bSf}1*6wL#&5o5AuoynRmb#>G zdFq*z^$+=TM9DD9P&gDF7*zs3_KcBi(lYq4CStp?jr$@d7ubY0Y$uSzK5s$1FO^xi$Rw2cUQp zWMA(<_gH02YVFXQQAVpX^rW??{fQfMo%Rd_uH3SOshmOW9+>OVXBHc`p464A`OPlP zdT@rh%UQeFu$IkI$e9?vNs_&H?oQBGBdKW_uirs zire0MRc1!^D5H#wvNOsSiL6kPk$TQg>U%uD=XV^>?~mu{KONlb^SQ6&%6FEZAr=oVM9{&_sFSkZJ#_yKj)gNM!s2ckK zqk8H*Md4ty61AGp7@J*&$Doi9I@E*hfN!VU(w}7mw#5O_I@XM`!1s0hYyCAA_DbQ4 z{qR%ZFXr%%tABL%yYfLY1Oyk56}-3eQg)4OK>Lskz?bCC{jod~)}& ziMO}0uhX{4&XvoWq|G|1R@H2{ZSQex#j?9m?KLKa#9Y*8B5#?s`qwkBpI0O`iEo;^ z;1FmHETwlA^wiQ=gcne6|6~*A3a${GWNxL)D~y>YRv|Mpx!Aepea)i6nRkJ9$|kek zmWHO`%Jk>=i7WY~eEQA2kte>g?!1_nJ~319K()hNXgg8bcR$W`-q@`CqwzHmriQuG&VD8~Hw{>yq{x7&@cji5(~_h+gWJ^F zM(>PmhfBA|FVrljfA`7jliL-2e?~(oSjgzh)=kr1SAX9J)jO`83JnZJsLWg+XU5DQ zKI>6SQI+9Y>AvUVP}_$O#<1TdJUiF)wu=(8DX8?5j`p3`Kr7V}Unfrg?MN-!A~LHk zl--SUR#ZJrEH3Hc+S-Op8sykdT6zlB+G9)GWi@u)FOd)>*w+WKgq8C5png8?PHUe_ z;PAu5ZEBF;E>Bpsl4^RW)ck~;jc0TXk$wxQWSy0>*S??p9H@@#B zsQtQDcYjT1_@TL1%hNKO9K9?@>H$K68vDYr-O|+u8TEN@_X=F6C~n^^Ud~;?!#bTB zB4s!(%pv3T`VT<5qKSn=%Ehi-QpjpUF;@|6K>m#~TC6(W$AnyvYsKp?0YZKJ>V&6_ zs<8FCzDmdH&gHe+B3>ikuHoIn7|}G2_F6uxiIBw4xV~e7mp5>Vf7JZJwEOi;t)<_f z$3@TM)OmT6gDwqaCPz}4H3>G6=1`WT#_E$JwQX<^)IFz9EYk1(NLAH@AinA2)~fw! z4ieG9&zMWto`*+&fbIcUB+E);Rqme$;Mg0o+sXY%M8|E^{HJw=^@7b9wVae~AE))Z zQ-?IX$e9_xxd#UnJ^S0#bQ1EC`B36ZlhjJX5e5;@+{3 z+HMPK=+?inyopFj*YTlw;^qTAAcwqj$`yL;^!KXJF{?JcuF(^PWiLjrD=xzu7$L!C zn;rh$Q3D3jKJ#!sf_G=P3LI?OV^%3f_6q~i$Js}rV3Bl${5+&BR-VFh+Azgo1~j1X zIw`Le?Sb-(DwQWVzu$6{_hQ)ub~FT3bD<`nHq3x=B!Hl!C)L>5v>Q+l{KnYeK#Q?? z;9G!OeHRYYvgO9l+dxBKcWQkl($)OyaKjbi1A}Yd8kBqzy_#e}yZJ)}xU$nPDa~82 zPQdYz@+ZOq5Y*ADUD{iy=;iMj5$OW=G`$b5KF4Mz=`hvR^%qd?8OOLIHKvJ)eV+EL zrYJ%crFtRfjiL)CcednrLDfhnmZV3u&MvW<-;;pe@k@B?#x4L5HOycpQV4jP(2$HM zf<$FLsc1Xp)fT`i;=MN3r1uK3iy3DjcEL=~p)2*lXCpx-;vs-a-c%1_EHgzHb#|NY zyLXe8r0rCDg0P0rHNhYR-r}Di?IxWRMc^YXg!LQ)CFdpn`ONnp#6C7dSCgk`SUnX! zlvz4@Ff`H*qaM2yq5i&fZQjRa+*PlbnGP(1jg@}w4fxA|Y~jaj$uIFCQlJg4_FPA- z60zj=Uc{xJs*$wD^Yc0%-nE^=K^qu@z6VRFn0u^Hx`%zRUpT&8hpsjgSm<^ zu7j6P`aXwjI}9|uLarpR2Sfcj*@N(!+3F83y^u0*$CEmzd3@cq-+UYVm>5VQ31Lgt zWA@ybnLfXw?F>XS50djxfEkJLoNm17vO$o*o?w)mx>>J4ft}%4bp7P;U0~ z`65F6@p*#Fa9g$i!R0P}= z(xu<7CduA=4jChNrmSejp*`~w(#|rJ&3m)$d010F#VhkrL1-+A|4!8&QY?eoCF^{- zVh%%t0r&hgn`w8_ieVjU9loCE)zcgq%Sf}Z%vtthr<~?L;BLdnF&3H810ZmH-+**7 zIO>2zcp&W@NN_D*;6$tR}&#OX$92eP(U`avh_ zcG}u3yrgxmWr%SOf;S>;DrFJalGjIK-8YX#&e$ud{9dLR04-jd&yf1uV`U)_6o@E_ zf%3=Kt2hEp`@ywy2jHvc$`bgGZ&S445*mAC zYsl)Zt|1H%XJ3zN^R*MX9ikw(`tiL=<;ipJHv2vc7JK``s0=}eMFSQUQp7`^K~A|E z+`fCcu^l^Jei@pX=VI?N4K+iX@J3RBYQ`=4;#K$k)~~uLcja=6V)21S=oijI-3)zY@)(i%P}^vlpO6D!Lq)T)AV7I9lG)+Z0#=%{k|T<)|vS z0dVvtZ{Ai-t*F686-8H1kzkN38r@OdFV?e~rcH}TinGEB8&>sdnrL!u_(3`bcI3c{tfUX2*ssU{y4Ea#pgb-@gD zTr#VNtybSe7m8p<@@co08a^eCOw00~;A{dbtYGIcYO0a_snvx{jCMvRW$7>l*Nwb4 za(3KiT+-8y1d75Q?`x9s8=yPw0t-L|C_0Tx0uMTNIQB`$($ImeMZlM{PyYDqma{R);C=)QhA zbAkkSYx$-d)Z`6e07I|CN$WJPh2oOC-5+%D*^5D>A2IQjk&mS26W`K&n-_wH!oa49 z{c@6W06y4vv**j!PzUYCl=w?xjLp;Ia;907JJKl7bRSi70rTkxp9WPLp-uSE8W2)^-Itc40?z^$5rRSqTI8zqGov7v#5FGA`T^M)GecowrPueWX?%&hv9 z)Dd*XuV4Rk4aHuy6t}{{;3TD;{GxWs!Vm=TX?m60++(gI%HC!(`sbgQxr+!|4fuhq zE#|>SRC>~44K}Lpm96FhTh-72_m$EMZLMLVt=AJ7Jeg-jL0H6K-15`4rOjjxv>eyh z>Dvwa$t!lq$KojSD3KZZ>!r7yJyZ-6aHom8`GkyKv4_}}3(IMF$R`lys8YXSKQRDR z!UXG?BYb;+J;C-&F$0Z&`y^V(d_q{zu|}6mf+T}yXGU((k}PVx;HB*bR|8$r-Ldu+ zAwG+CXQ_@I`whp%ZtI)9ZatIPCN-~pD;Z=KmCmR}t=oY$q3=M#IHf0c_D4L2 z?)rphWj(#`LY&~*!HA$!(BwBe2E!+!JH82#=OV1{t-DyMkEa_a~bPtm!HGU0CsW8$1Q?BnhwoPdI#Z^uNL3G>y(oTiGX99$_ewVvc=3t8VPL zALgTq%SqYS*2~z*r^&))ir0K%250H{#FK{liW~Jy)+%mvMSd%3XC6+*Zmp;X$;F^3 zlf3e^oo2ZtW4d9B)}Y7!C7zL}Teqe;szTgBZDU@NI5$LWzsQ}fRu-ji-80&;`$^fz zX_MaBNIP;)lir5$+DANB^=iTt3Z)A_HswE}e*K~j{y>S_=^&?%QC@Cy|J5Aa@cq;n1v^Af!@Z6QB+jcrKf6wzTWgj^z7Gq9EmFfNYua;go z^WR*YFM9hIBNz#XUVG^vhwT69H~s+ZCaXGqSXAwrstivb;xy#){J|1w1OD*qjY7(I zNJCxTd~w?ojoR(!n{|0$)c&+M%s7FWrLXMnr>3LQ1ii$lK{3o$As%;g_mh7z-g~vw z+BU{xS~g5Tpbx3l&S>vt z`vO9GLK|1f6V2jhUyc|AQy#@5U|nKF?^IawCgTrErh(XiN)xhvUeeZyYo!KfnSL4b z&27P%jA(#ePk646n_7d>m?LmXOGM&c+*A9}Zi|2C)HgM3EcN85NEp=zkF_n3#z(o{ zI?WIX!zG$Gh8gw;Zu;&edqAj&q$TxlK%G-hlKjD)FRur|+ZDrSdo z&(_UKa)2^B#F51_t)uQ+#hoDO>Sqm=zF!yLP?kTnx@IB~IcxVgi()`@iI(KNEdka5 z3#}=aqW_x3?wi!IkO}4NpLh}0ft?mGIXj2XxZH-zjHFg^tQ9)5T@ca44aL^mm&d6E zqJw_sZg7q~@;fF)%@Xq;k#xoyy)3y*{e=B5-7U6ekubv}^f8hb+`kEZcmXNQ>ryzV z*Ne5!2nxDfOmc}{PXC&;Bn5?0e_<#~2INbi>%ytwpUc~&1*N`Ik0NcPE(y;2-E_MFzxM=It_FEgZ6U)p?D%+1KImifs_bs~!5 zMCbRg17*qic`}kD6Uf?`55U5ac-Rbm=Syu1@<&&L5orXVD7~Wwt>Cca`R^b)XR)v} zxkIE_6O`0~UjRd=(3VI^A-WktO{3$Iqn0JMS0YmQJR921LXYOz)(q<1wCx^J7I5yM zx(Xj(BRC_|M?u<}XGl_+tUa~e92;7GRdS>&=<|n=eP4(hsn4NknK`hQMWqxVCt+9> zY*UeH6J~)BLY#ZQ>=FR0?g~G^L3mv27Y1?zpRFfvn+LIwR?4cHm&Qf0P$Z_hm|;0K zqMyp4&3?n@Y|&h9@QtEN57&8ab(!z?5d@Oc+%mkJq6D2LMygdt>Gd-6Op=%HWfkQ; zkEyNA_5AbPl*n_7h|Q%*SacF92E8*iRq@;3#z*3PK6kB((w+}trA#imJKd9_>*!k0 zM#>deR65NR_AYi4J!2k$(!4%Xr(13e!yS#?F=F;3g~t-|)kOqb2a!W}v@SBz0ew&{ z?)OnpV-73&`e`SYwCciuSleuWqHA##>Uh3I+VP|?SRaa+_}M`4EaNP zWAH`5wl}BrhEk|{y#5fZm7Lq3r^Q>e8@*N+%g-Z}#9bbbGIxbbuJ5IF1wLGM?7t?u zF?WMS-rHX5J7UxdJb}e!U52mjz;R{LMwiK`C10ElF4@bjtF zx!0&8*AVPLUnGqf=SS1wRCtlmE?E}c1;E4Dtok9eh%>WuurWyITgIU&+IxWX{&Hu``n@#% zxIxg$q#T^fLo|QEbDb$sN-gte!Aa)~Y_qEbNT(&Y|-_alG?s#Q9y%nQ9RwC3< zwUmBwWNQZ+O%`+>eZk8zN1GWC-`$$Ly3-9@e?hU+*rLa~<2^m{|2kgb@wKC1!y6qe z%K594b%WwoO30f;fqnFi1bY2zO7R{H?@8jyxpQ=5p2jA^4J6mSQMuxf#G>=Be;s?P zp1(=k;P0k8iil`LS^vC-LPYWF-qf`F`$#$noF?d5%<%_ad&Cbk+l80P`G09Q?I>x8 zW)3A)4$78cEk4}ruGKr9@0(RA$$ zX9*>$x|g4R;hxQL=87&L&P*IqyUm?ARN<;%g28Zc8^{+7q@0aP2M25^(s^k|f#O{# zjy-}*sR&GMQgKELwM~zR!;?ydXArt`p{Srr06!4o z`nsvCgls!O<#0gvL2-6z3J~LdJbbn*gf`69t+ZwZp!irZTC%>$hL=qN=i_HB=KRVC z9_j-)jKnewEWwvcFmFBK1V-d+`3Q=+NP&Jlh+h3xs131z^#VSfKGoKJO_@J`-yrzV zOcEv!va^}ioyvPNpka17Hc_o-=v_^*Kq>)k)phTvk zhraa*K3d`sI`33va;3A#Tp(o*#CIONlr5pz>)g0bk!)bkDK&sYA|YKh49VY;r7$^K zo^4l~ILQKpI$3Zc8AVB0zKTu!_=mz@NRT3AJS?kUq#h_KVKA2VCua$0NWJ~?d2s8f z!UEH%_wo>%ZqNx6^Z3XNI`1*26vf?cM0UJ+W1U#@B0(?cQWX>Ii0&L%PZ?On_S=Gf zE|P8;NeCoCmQD~k9(F}a!kmYfh`$P%kz(>O_w)U&bvWn%ZW5+p{ssVWoS29uM1)pOl6>S;Qpk+ z=K}F@#)Px?;iL8QG*fws z#ZxU|&p~`-(+VMyo|zUytvMevp5(ip zS{OeyptI0H>lVLHx~M|87IeeK=rDjPqFB}xC_C40JY41!{*m{AOccxn>{6b_GLk>p zwCVi=3gz)OyH+ZcktCra^6!iI+-N`IC=5L~PYGezr9*ikBHH6oY=}(gqo-1~sLS zkLh!3;DMx(YE(_+RWeTME^0gl!d4B3Tu1=mcmfJqnV1O#55H6uo?hVUGykZMN9+EI zDbRzR#?0uD<1J>WTQ=atG&s5EW;#zHOUrF*Gpu_#Mfi) zReZ%l=4JGEP^mDm-td|Fmqhsp9&dnCO%w6AT_$`p|6tSj{LpuDv5zV)u-FNNAu)%7 z#criahhtmus1zBmZ@CIAS2<{~9IMa+a;b!R3uZ4~k#YIc<__rv>9f$;Y)D$;T%zQUHKD>zr5r44 z8PQwdVJMA}j?j}kpyH-p?ns>$P4;>QKp?9GnAUkv3n3wD~ z2VaGk=rp_zw+7BSK=c!dQYZlDpD1X?WxUb<-33aou)?qS)N&8XrE?bl$^T4F5{M4TAMa zb6um}{>|sB|kSpclFMM1Spmh><_YKy_VOckh?7 z*nq_*cxlMZ3eT0KzFA*pAGOUXeW7i)qzCiRN%r9P5o%w?v;aN5zmhre*A{>lX40rr zz3JRwYZUSKnomEol;4ITmJmRHx^s{X&{4Dfp zvMl20tQ0sNDE+hXqWmps4eUP+RjBwkoAI7Jn-ABK&DO;f|EvFQq~~Alb^p--2uG*l tY%Tetr$3FI1f0L&|F_5Q-(JKIDUs7oy8Sy@y~p6+6?rweG8xl*{{>MWq2B-i literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/6-format-transformation.png b/website/www/site/static/images/blog/scaling-streaming-workload/6-format-transformation.png new file mode 100644 index 0000000000000000000000000000000000000000..335cc4be00ae772432e4ada9f9e2d8e2f27b622c GIT binary patch literal 21764 zcmZU51ymeC(=E;-K|*ks;O_1kEChFVSv)ww-EDyoG`MSmO9H{&-3czi?G4HI{{K7g zan7C{>*<}IuC80RZV~ZbN#+$Y0WuU6)GIkz$q!Ia(CEO21B3t^VMzNya|cpBL78&bi@e>C+nt==4s#{r$a_yD$ed1kZGbw2`Mih%-(dzEJ;4hg$YM`|_+-z7+a#3u&0ks(@aOG80jYO16DcwWMeiT#zCLAtG~{kE|dg?VGa*v%y`LeyQvO!!Of5sW%xO;p#)L&tRQu z=f3Ld>or_!(*!A0rDiq2T+*t;Xcw?W2|Bx~2Ku-Dj{S&BB!e**5Q3~{^((&HIA5U^ zgorvR`}zQq`>vn(c||{mxC+Xp=+E?r9BkSBXwn7>87P)A>f**U9t@F;l>oPj9x*#O+Ljh+MhKieEH7mBG$$jJfUs;187<_<1a zVAt_dJy_sYGuCQau3Cx;{H9=gW@9t3i8-@}z2i$0C_xW?VAI~*)tJn~-p;{=-$RJv z?-l&O_RC=w3bMbKxY`O)XequYlK?xLlW{Y%GP6<$Ba@Ml2|Alu@PCk${!epYPl&?G z)zy)oh2`VNkIWxAn8D7LENpyyd@QW&EbQz|z!gj`o(`_Y9!w4{l>ct>fA1q{?qcd} z?dWO^b|8DXudxZ(%~gnk;-#bi`TO@f%{{FDwcV_`ieLE zSC7!(81Ob@XPAd$@bvA-O4+b(hahQ(l<9#CS=*yURuV9!3 zrGvh#9)SL^0h@O>eb>1HF!^GgA$?Y37v97H!qS}~GNjKp!mct#oS<)~c(1(tswl_$ zfX%&4($%nIP&5KGE=j)F=!qNL9Uidh`>=UGMjS9s#w!2LZjiBV=!5N~xaor)s4cvU zJd7x>oIO#}Q3G0&jbPn2VR{~Fc%gH{45*FdiPFkso=YyM9c*~yC!Uq~dNb{BfkMTiO& zGYehxG<2-LR^MzyHQ6V3+)Y@tic*L z&Ra~nG_Cr1drW4oyh%0myB**^>|TAk;vrsCRTjcdHSqdf-Hpr;tZkV!&3wJZ9om${ z8E1f4O_LqTjk^EYWdzAG%S{9FamBMIbM;}zu^Wjdc+sInQMY+3Q69#>qCnuJRW(FC zpP!}e&%|kCQ=20EFxdZo@ie8%=dIxL!-W?iw=c&iOWzO8Yj3MFs!}Ys^ zDs-0-K1+|i+^{#;khCUlf-cH9!Pg|-hivA(Qn>nRRTo5$#~(XXq5tgV$Avb`8VDGD zT=jo`TnA=KSo_f~#1xGw^Lh8W@VvSgO;>3;Q@ zWbc4F(=N|_rdx4ROgcki`6F1z(~c8U%hc`2iUX{!&^F=n!&$39y&bk) zG`tw>I0A?J?+cOV+fm2En!eE8FWyR3PRFHXl@pK2{uo3&OO6s`pFcgS(f2f$R#FY; zm$Hkj8iIV*Q4B%2hCWVnE~7%1!_NtEK2(e~KVBQhaLs_MFflUV#G%Q9B7)|tyATM2 z^JDqx?OM-zb#{Pp_cap2Tk&GWpdL%6Q~g6A9>Q}D^j6w;>u+T_$ka5Y3Zbr#h=3-4 zW8_hf#tHwxCM6y<=Q^oq>(;r4L5Al)8F|+wMc2YhPk>E zhQ54AIq1`}O4i)JMCw3Tal)I)?56^- z)yl+@_x19}v_SEDX$S@gnG&vxuITgYSiT*|=@7?g@E}Xa67zK9RFs^#nUeZlP~CJr zbxZ8~uCxzvdm`cozbNCT*z{b6xz>3)ZWWJGKelcXH~7LabnmS`KYbJ(q{oVH=*AYH z=AF?63t{7gb|A$^F48Oaow_IP1nBQgt%7nKR-Ff#xJ0^vD=Mx;N&8f~bI;bESvMfIvZ3Ay{-@|A{ zz>61Ee^@(My1M=f>=ATFcNzgP%+YF0H$DE@2c3^&{e3$X#pmg3HVAWwA^#Q6(tAaw z;3fa(dqdwc(?lZ}F`vX<7MOKSv6cWl1|_|EvIjI43zpfw^?pNecOQ0? z*HDlxV4q$O;X>x?;U3s|ea^DhmbSpaeT#kdbF+cbY3c#iyxc%I<4BC{x=Ua=Ej8UH zx;(0%R7zPcG6XfTXDe(gO5hUss$09Zl|SLFT=|fggYLc)zzlC)7@1&#wVuv@5E5d& z1-p|eWT?b^kMqi;%LH>+kj4ZFPb%MiyFYFTtMWavrR1cjN3W+3hDXyO>XG `L_w zxUj?IjZwh-+&EdJJUiNX>)mJDF+jv{pq#kVb^Vb>rW1ozrkHpUXELZmkc$D;Xy0OR z=OAvrFX1-I@Ijn7#m8OBmX_n|nBU6t)Ad1-@uWg|7JXOOrF7utuE?|Jx4Zo^bt7%x z&9<2JqxBBEBq5f}`pK;dz~%1Y!9n6f3E`AhrlbSlQ_W97Lfn-#grAgerdq& zpVxr1?ok9hBAgFnY^N>g2k?$htr|A>8Y=I<02f5lz_7JuQ3*mE4!1zOFl}L?FP+5| zW&1T>zrVobgNYRlUq6b?GS>M#zt>R)v}g^5qC;H}v(Gku_v??=7&{gV{>)HkO`SY% z{$~n^la)aAEK1M7Tvw;l2X4*Sb68dSfz&Bo_4aaug)_gKlkPemoYB5i7oRzji&U_w zhk$jWmaxM2?Y2?PV)D_T4<>8ha@AZxaG=mE3QxMaKu*lcSyIR9sT`<;$AEqcUgyj7 zKXhejx`P90(+7`uFbn=^6T6l6qkR~M6)P`KH=1wvAI^wegBwDI`zI7PCUtz$vc^z<@1QO@QiQ~;vhN_D<1(P%aL_k z(Cnp!d&od<(JoECdX1*P&y;_)jZtz4LSNsCt%qzb|7QMk7^=1V?U{-UzMto-pI5~HRw@_v7?@_@Y>wD_;m2%xb9lPK$P*V^p?4F*~rX>WO3Qt~`>EexfTps;+E4`0o-6@pT$uW zL)&v(qWRfCKpB6;Ng2OCs86rg*uOSWi`q6U6`OSWyNT|(gn8m!$8Y{Pk*6DlR2sWx zLHLe~tdqStGvoZ?dnMu9Erl8Oen}%U3N;SQ(s{n!&sgd*cE+dij61Ku3>&macD17b z#!fFsn>}4|$ByT+cw?zeRjRKzwO`Rk*dQJu0KA>83p{X%jiLGgQLbl9kY8#F-Iad` zw#Hb>xOT=VjmWK>2LNvZ1*3nOF!41uYOw_K!r)+<^!HlGLUNy zy!p6r9@h|sEzmlzEfw=K=D6iVSl=TfW%z2v8?0~IxY4MR{e{~;+NoRGauX z5QTXxKcHT8wb~aU@1@WM>t46O2vznuMg-{l~*8diu!2W-}H%8J{qy@a*W_{?*7Sp z03h0R@qAb0S$UDy@2_bKZ727u2a z=VF^CgSK+Az7GlY5w`=0X)GNNwKLi?tG1QjrO$HxX+ArOR~$W{J?>ORYK1qiT$bK1 zxCuu#VKrnnu4*WAm032v5txadr{H!!;+|60MiRmjOYGkmo|h51Q&E=*>1Q|2QNY>n zPt~@-XQBkRCzB*{aoOut*pNIn!I59 zt-IK8`t5GENG7jS-ir58!>k2GJo}lD*{uG@tS>#Suh&!b-KN#k%Mmk?B+P3^$_zHM z`;%xa0f?sA3z1y2(sftCF&WtbqAa^KefNIJi*YY0sNd$=f_zwo^?Ch^Cjk25C(%yh zPU5oX7>WY>U1aJQVrnW4=rpB+z-@kW#$o2Q3wph6MOp4T)KQppPkM|S9@{CJHUh6t za<6A>fsq5iTFle&>Kzb6Vcw-P&SXWf&hke+ZqH-nW*FOK`CiS>H!eDq`O;=E8PP?5 zsbNagPvY{AO4}!y-Hd7GX}mE>ybw=av325+*3d-Hk%l2RBz%g?Bc<}sLQFtZ)sPzf zvHlf3jxigO$BYzv=4op>}>A; z$gh0zN^krrmeEf!=*74YgPaD0IUnM-nJSKyGC1irKNn;V{?O|;i_yZ42GLQn8P;CT z8X7*E_%NSqs7oW#JV_=EXz07CS2Tt*&|Gv&5Dm-&nBGGF*9iYXIcjfMC#L^UqC9gt zxe-232eG~AxGj2wWY{?x&dy5RNPV1Op?SiBt9p* z)~oGl{yTeaVn>r*2MIViC5{jGFW@-e&xK;&2__BtP7I;&kt<DjY4 z3sN)o#44kY!(9#MQazII(f_#owMgWCessZnc|an!P~mB-yp}@H_H!-7348e^9L^2F zLC%ARYj>#5dtho>%9F=PRQ?bwI$g1z*75L_xt&o#RkwC*!0;usOSk-@(s2K6Gj?(* znUTcz>2~)3`4XbL@#YnTv*?SLjnVYSJg(afDh2#+t2keufCa0ZI*1jF5n6eudBAFVw-1VrYmyZ*tSsCIHGL#ywy8VtGH1dhI(r*YENG0#cK+skn#LNp zHu>ECH6*c4SIYp>hW3Z2+IPml)_Z)z*gbStq7J7@u{Q&R$`vx4*~U`9TBT<`0`L}0 z;R{ty*gpK9!!URSf8Br`ntxMY4%*fBQ`56zV9ed!Xp#{nn!x<_8xk@4X{p&0&h+R| z55;MJiZ5_t2rYAfK6^%U^nLe>SUYt|brMfIyU6T*gRXj&FLx0!rH^#^vQDZ#N*4@y zYX*5RTz7Xd7JS%c#k_suX(z16O7NAjI5c#(-6Zyvj{0*Xhw_0nan0r$3LKbsrJGe) zA@+ik6wC1krp(^{u1VE_E>2h(CTei+rzB+`hEg{o&#eS6Z6DHoTFJZ3=$>3UYK7tT zNUS(_kGk4BK_vq?j98t8V8^@Mt{grxJ;zV;q({}Gx>!dDN3az^d^GY*4Qd$A;Wk<( z9S7f+-4_+_MGdbO!6FseEIb&7(!c1)8#~&CFnz{6ce8MR6ihyj`s{G%pdsKIM%(c7 ziXF6ly_CL1?2;4tsqkSv6oj6o+^n`G@sS7%Li>k+ zTuAI!7ug>VA6gyVDW(PtNE(#Tj>trRbmWQ`Omdk( z?j^rj^WBHt{Y)NkUviSUTnAB>b;v-piA#CyV$3pX7jE-Hto3hj94R*B103l$7!s>^ z_aQ&KBCDkFSh0*t%uiw0Byu{U3A01_yayeJ?*+F8BV*wi9pN@7JDXs!{W(6T@y7k$ zlxJy6o_<+{c{8Ob$G$T?RO;F{q|C@gzYyxmbJB_<#zouU$6LmqSza zD|S~cf>Hy2=H)ZS%Wv&TM$ACu3faPP+O}uyy%-fSY*7;MfqwV)xbL@)RCLvTv(Hue zvc)~GUI~Y?p2V+$c!+X8@86~W!C`F)bzB--cz7xMLt$1cCSY%9s#9h~IZ7k+MDK?W zZk0-pH(OgT=~)KTayqjykJe5Vf&1cqkb=ml&$8i1+mVLZCVUH(OBIL6SH~C9ehb!Z+M|k^in29sEAo8?;EyFyzr)6mNM*fvW{S?vJ(BeEHAezfd0%7kM|*+ zQuJt3X>+eoXk(f@Me-+AHi`T;RMu}7YMbBAD)hzrbXh@tREweaWvR?SgqBwd0_8Yi zJjxX^b?=vjgB006VLBU91y+NklY;q5^)c$N#w5_K$B6H(DB(V1s!8SGEl7ST-KX}( z^QDj?T|o4Q`;J+Fpx%&KmFeqA8>G-S`=tFY&;&m=V(RVkI=bI!kSGo}$UfXO{kXy& zYeT7;G{7z~pYo0BrHa#WCkx|386Ae z!3_FD_IsH}U0?M$y#W`=GC;F!Hx_#@(CO#%zgB@LXvf?sif`oYh>fLefu?WJBSzF- zN#ub!VAeZz&Yi7TWO;wozz<-zXlOL0kV}V@>%loE+#rfvF~x&ocVRsquW5C0K{f@Oqf zJ3yG*V36D-^2OXyZEz*+X;uW82A)xMYaNi)5k@4N^;S`nO$=<42JV1%u)4~Ntg1@O zmnS0@rAOzi?od;QKU}ib z6=N*Z>8mBa)KTlN60~KRfd!Jma^Ey3pr?*2Y??Gg-OCk`$Y&>8PTIYxJyqRASOkCl zkd~yE{KGI_{bw>b%se8kK$sp+l5)HjgfYYwOz8XO=-sG1g$=g!XBm%p_Hl?oYQmT| zE$#PrZh97tMup_OWO>F`4{2vTZ|g7Sb!a=ua`me`*xIH^?6XsRxZ{tzU=x`KBgmt?TNX+WRR?Wi6y5FGSiYnKkl!(P!y6 zC_@b+_Sf_ANLl43hdrW++2VwqjWo+y^LMG2r78zzN8O=sf0CiR?)p}Mw?%ZM)*zqi zLYZeKMfrhxNG1*^-;ys?kT8Q_>^O@o%5q(mHn9~ys>lZEBln^m$59owXmR43679P` zMIywKC=&{2Hy)>$&q0vWXCWi7f*FM%KHa8b~KMtpDHD({_E_E(ZK zY4dli2S3+-ZLRNAFwwh_>0+azV67Q#4a*%uGjt>{p%&#>zcsgycewyKt~_RAQ4*!& zBH5x8R|EYo? z+iV6cjMUGgXMZcH#4jseIXnZLY)WE>N5m9-sbR!CExwm*DNW(i-2ObvgT z@btd6d-gn9ts(P-^@w6jc35VL^}7Kkh17;%e-I{jYa1lnBwB}2*FDlFI($dWP1O35S!$XOujH3&w9->xvW_NiD zX3R*Hri5uGWuFT64O?u()L$eT(zmtdvjRtY0-lsWlFEl<=Z!5{+T9)dh~i3n>utu@ zQdE=Hw^8ZesIIMzI~d#WK^IWQ7Bv4OH)~Qcc@C; z)~cJ|Z39k>CY`B#b6t%TnkXDXILyy;Fw1>TKR@6U$j=8;AxQ@G7mVnHr}UDo^Zv&R z?g{g#4&7Pw2J zk<8QOio&hqLlREv*Zyjb6li*8GEh@1$mx6MdPJi-oIcz?G(#chTGk8MYCu^zEI;Vt zbVcE2Sdl45%Rq8Vj0(OXT96K+#0B}jt$t;aAp`>lBhrhm!76;lTi&Z79UZnBXe6-H zi6`{UUq#ZxPs6N%;nY@dwQIFn-GjPOjJzN|*EfVO?CI$ChAU zZS}W}2n(478OniiYHGR;OSNvIvI6-=mDXizRd9vK(dR&0`yY1Z+?LeB!I({~=9!=0 zG_uyRx={x>=)@KXy|z?ok-=k0B9*z2ubyZTjLr~_^Bosv_pGz7CCpr5Le1!3Yw~Ut5@RG`3TG_d^QlVw8$ndXNpr_}wziXt`oG2e9sAP7dHz* zV#W99901jcaY7A0$^{Z9p5T$=Il698ieiUi@cW1*3~596VUOr|4u-TM64fE*_5{T^ zl1-?y$NlahO}Sm*Ph8ya?lRLc3V=^OGcpk0?2ijj4)h`RxrK1F%F+aHvM#e3R`Hyq52e$FMkQefN zcbr8ukrGB1mo(t*4qa@P9#Lz}p?!v-QbBhnO>IB24~pPKf9Y4Aq@# z$ka>55}nJu$j>g2!Uss|1Ol zDjv``_KIQ;Lyo?}FRH0EXVQ%>iFInL3%t(132z{L6AH`uOs0?3UY+qSe=si)^f1Uk zJi&CoiK2oVc|6a}hF!Ybc)wx(T8g?wyz)^Li`u--MZGb3a{MCFkF8tYXc?RD;obUK zIA-DKP0{;&zXdm<;B$n!9%EgXPiO*ki05k-CTQ5oW%==~crnH4>@oYY-&gFO;*y(n zos_XxZkwfa2T5nH@H6?mZE4IAX-XlGTh-0RRl3#$v&J_f>zC1U#6Lwnys?Q>HT0d> zu}j%AU@zVdRBzeODi}mt3}}(+e78?Yog5I}KA_*o>ae)?lqt_L9h@7Q&lHJEouqwh z((coAoF){fZYuYhwsIY_cpO;N98`do7o%-Z z-KFdtlD7z84{5<&HhZxE(?)4l0T*~{7s=P zek8G~XJ5dTV1BcdRKl8gu0h>F6F9{9H45z{f?|pFNxBsz_K7`e(&55B>@Og=5cI&4 zP2QX$1m!SQ-w}x!QFXq}Mc1XU8RwKehD~r6JvX#MU>2W==pi3SnNJJ;hE*g{j?xv7 zsu0G5{~Jo11DB#QIDD-(hl^?LIhx@G;uLs#n9Msp`s%Lmk{?Mz0PIsopCQIoWKM@lDzu4#k@0r{S zT_*x6dpF#=vrnv+^Z_5jCPk5f=Z{eG3KK&TSweD7W1F+`)kTA5nQqLII-juYKkllKwtw6?7o0D%r%bu?Zo*RJnI z0*jSCdaeU%mKs;M4PIh*gTy0Lft^q6sgsh_1m+te_!`o+&gSZu=nQ*^uQ07>z%L2e zdGTYJq7{i8n*^#EmHs3?=_8^+ayu6MR;}>JLkq^N_dY=h`30qZWqnPXaW6@}bf!%a z$rCXe@=r!a+5-nw9R|KsR94K$pOyBjuD^1DsF2VT=Xm*GJmDNGHGDUQ(=HxIq+jO# z5KjAhhZqDhX%+f98mjH+ks($BUVh<5?E9J}l{hg`fAIEKJDOj(UTyT}Ag{WyO zH0j*!YUVEK9ZZEPfvgw~^`I6k+se@}ATJ3>*~<88;5XSD{c5Olq=f4qR-ZFZ)7P0) ztun`hIjCq_sopFN;+oN-90J5r>S--cf~Ka?!|&@LT&ihlNaTO9KHb4*ow!aQmL?#m zn3LN0@``&d=Q8G_R!nq4%?xEV3`fz-%dj1@+7ll$l|`uezewzuIB>C98s|(hK3f6_ zF$Zmd<@cBo!N<)9l{yF??^-t^8Ek4Tcr0s^@?9{S99|QuE@8 zm|7fq!VuyTc}n(LiuBtV52F>;NUha~2o`Gp>T8@Hv9e!BLP0XsyX=S*X?n>ICBW<;Nkf}rOrpMGDH2S#t-B{}n7D5lgN@x(ULQe=Tj9FIIBEgla zU2jy}AReMfv1D)$uNiGN+GA3RkO%=9jUen^t^6MAL(IY_XaLWCM~u7mqSO&PtF`?Am)yi#5E9 zLQEBEFoIj1w!|vF$PRw&j`Ze9iWARWjA`XxNfqWKK8wH;Xl`+?F~)039=v7YD|r*r zxIBW(Vf)8!s|t{g0J%>xBCoyNTQ8puc?_%6bT7G2bIOxLbN9Xz+JJQ<#o@L5^5b1~ zCxMB9l;FaLKO$Qht7mA5pv2>?xUU;4t7e1CASG$=AQ8{`Z};=T5tt-$Xp?}JldHUL zgx`0>p^eMOB^f6xnkW;twn1{xBd8&*Pw(VoG(f%%=NoMLRor9JR;Ep&gEvf; z6hxvu>y>Rj$oh%%4q4s*zK$b@VO8oF=Rats9tM-VK*`N`3AEo$mSt7Fx@HA)_~mU- z8X~djt4uG3Bf}y{Bphl3H-!FPKkvGdbtkV^{gSKjS~{CMub{!kR4udK;Z{GkbB`}GJ}JkxP&uQ4&iw^I|oP07B^4Jr_*nu@HGS^LBtCm`7w z@_4=4VaRsjbj=Dwjd!jS_G)%yerDk6L+QV{V?v#XciMi$40oq~@$32BWqrI4-(sJo zi5!I2vioP{$DV)kRtKoA`#n%J4m2dY!y*Lql*pbfr&9bUvM`ZKaLEk+$n^L1M=Tg> zNk*JK^aLbGr1)ErDjoYc#Oo+?@`YJr?vBbs$i zCnR^lyI-8&J==c8MI8RT3HrkT!GflxEdP-c_D2o=&feR);)4AVyv?qZ6A@=7UK9p<*%e<%)BGaDIKDs#pCs7K;P{ z6pLl9A>HicNoI{>5LGbcnuWaObGy{KIC$}`QE9yFgkUTnr(pFw-zw#QYy~KddVFXO zI=ixAt}z4JN!iXNAe*Pi@;0-;^9Qf`LUgX?Dm9$mg%wscv4*`$kT$sOvfFd_OHmuA zYGLW1T!xIgx!(%lGn!GLC-v;)PX<$SXp!*;-Xfv%Xnz{%e_{AbJr;wb?D0K6u;*;X z3RM1=kcw*kEbGLl&-lfEko>dfK0q|F5>$N~4}iA6UMfuD5Kv;p0u;=gebX`#-%Hj& z>>uUgbHBKHEH>vO`$E$Eowrk}%5KRoSP=0)u+Dm-fFyW0?8TD-@0LoC$OLHb2$yI9 zIj$XKwwI{HQyp&%ZQY=Sd-xa;SY{ux*ygA7*DOm^iyB`M(HmH7yzSX1<#oTput-o% zw=>J^cQ>lS4Na*Dl8DXJui70X75v)Dc&XzXGk`AJc6(Lfqb@QqfIjs*Y&z{kS^Kv3 zjF05};JWLgW}Aq#v9HyB5m*D+bzkz)jcx`Ep@gANYL1-^=OP8s?Ax~sZf(g7gOw9w zS8M$&zl4^+{y{0IGXfquPt(uesVRm z?*v_)_rBDgIT^OMPAkv*+$V3|tih0YePC|Y@T)dVW+5cKMDGiVq!ZFwSs*RfybEO| zZ5E!(?z(%0Zu{JDPI<;aNV;7?`t$B$yW*+#Y&^TmZ>*@xPy5-Bfdw{6{(X}WX;7QP zYRBK9q*nSo*S=yqd`^&{dD`jZg4(X~r-6NLYc137w7K-v;efwRDn)BRsO%t`J}F}v zVpYo5 !X{R0PbYEXBMt*r0bC%W>2GU8I#_l@Rf2hiP&ik|}LSZwrX3@fTkeAK=Y z^u52=X?RfO7A`Fsy}hDSe(V`UcW<`yNfv?o9D@+GpO7*-D-s@tom$ZlM$H=>7IVTS z=$ta(Fn?2TcXUl+;E(TTzPZVK#F1qK(*cZ&ZG{8qgDU&F86R`dwBkwH`IZ!AhDD>HVI^B3-=(o<`2;p(@qCPjlQf!`0$24$H0y1FL zXarg#OgCOTt5|8-7U{c%9xM5qNzZxqL>O=X)TotjcLR)&`fpEl+pQNPwt0j{bk6%J zmchiz=>-eD15s6PM&{gXz8%a7gmQ&av)k`P8}B~6SQEj2tPjxC`)wN3mjqkmLTD`& zGIUDy9fuXuxF4Ym?3Mp>jZ60Z(l@B^8R%&&v|HQcz90AucYrkd4UnZTJe`ndna+4P z0{asJt3dk#+brz`VA0bf+aD{xZA7x#FVsJj6kZv6tixAa1IjKJ)ooMcVF;VVLE&o0 z?J&3-keYTFH~p;z0Q6A5t6gHyzIbdzoC0(`#ltp>zqQ&_72xeuTC>;(rF#>dbycsY zGC>??$@;b1MdgXhd@c*&&AcL)k(0K&bUl2>_zcbG0}SaM5Pd-Z)IgmE6k*t{hu~Hc zzeq21b^tYo`H`OAZrh13_^L$N_nmUYSihOdnF5Z$2rk`n$5G>Ay+MQ`ZQBwj=ne6Ix(zC61Y{(iQ~3bCG0+OD z;k^8@W=tH#dcF#HL&oD$xUV3l$=z( z2gskQ6yWPJPetpr0MS{)HlS7J4==$o0zruGHc6lHPIc{oS_-aSoJlG8URZQNjF(Iq zw(ORE=V(9<9cBHO?8igPfld(8f;j`V8t>Jd=_L*X4?2VtR_r2_rY0Xa@6sk%wtsJ+A?|I$w zfA&t-(a}fcYk=9mujTiAzgosE?s-8tjvMoOBYbzzc)~P#O~>Ban3zr0o48|tn6uz?~-@Z0KEP7AK|Cl zhh35EmtfS3F;MhC=;5^IQmY>JQYai_%+!bJp||_SD|&Qu80_alO`sZzjwk}}mky{y zPrR~>=pV?&Ua~INk-a&6Gc5bBduMEUR(ikjb%BtG=u?i>IU#e;@Yk5;IwNk}~ zo*qy9Ex}%sne1$tt`A0{{--EnV9!Bac1+slQIRDEY7_StksyUuJqcTl&Fo?uM|a^!OfL9pNy zG}j%X$V~vZtvTPctw{O{E!^71V z?_-t8&Xr#fb+GS&t~$eO@?5DjZHx2_>5$M*5uf-1l&O`QR=jK>tj6LPVksgO_ZAu3 z4J<&JXt+~^i`Mnvli0;;=+{d8zo{PU*deOYe-`(aPyCuPWYYAUBQi`O29QstQdVCI z&9}HPFo^eq|ho>*KSL?x8N+1-*YEchB z8BjC?m+yqufWPxD7f?O%gd^mjh7te{#^IasJGTaJquLj7gCdF&#UEC}twH4q1wX_N z=M|BPhMJgRGGOJ$X^w9_KRBU2G5U?^G#4~)Z^!xt2GB%$29r)togZt3^ZzOt5hMKHMso!9b4RPxmSE*lC z@m{!=H=x174#TWUtu*$XbbW7~G2Sj~1d+;4`M42nn69Dx1d8IYez>a8YymFKdRn@4 zpH^TrlR$J(814xd6J)by3uZ9b64q|XnVIjIv>m(WU?4AJ_LHqjRIs8Ji&qg@op}}DxZarqo z2~aRh*s=JXx{qr?O;xrxl^GCyKA$$$y4+bJa%Vpm-LWsmnsAWN8lSO};|z?Buma)U zcIOYV6!wJMIF%0Q@=3^s_a_PgNxBAh1p!Yz{6CvEMM9C{C=lxQWWm?WNT4Z}cL2Qf}#e?tTt+O;vw? zd)w?-nm1tjh1J-;^7~Gy{?3Y5aQ=5p<5b^^%vCkgDBlr#*EmBkNdQ$g;sxwyq5yU{ zM}*D6PU~nx`1P(}7K>>OOeM3Ol==)v@)b};#-rytF2#SHKYe*^;>*^mZ`i7tgGC6Z zt`d`dtO9ql(Y@+{l`V6TE!)>3vOMO83fAmE+mBARlKpQzm(eJYRsSF2G5?zEU+oOz zORX&6lsxPN4E_dON&2l(84QHKx|h0kDEj^-U4(`@j-#g390^h$hh5yZ#P`$ z%)5ubg9no72rQIpi78DaeuKCkwsEy%Ye?S!bTuDP3eMza)Reejhw0|AF*X7_M~HU&R+k zg}#ZYYcsd;GwJOwnL}dQ}oqWe6Yvf*ASwL+hff!T>8PQpwDB zOL8SwLiR~-!q)^Q*c0Ez z546P`juC`PZDMN+e!0<)oWMkuvAz-)pJDZWVNDX90v@2M+xQ+DWBoG~PBsx}ZB ze|b1+c&u$cqF|$d>&i98KoMs*-*1E)O3jh&-Z)BKNd3_Z*Im(iK2XYsl)vB5xe~#F90Ha z=)<(AXg_J*WrAo1nZUjgQDQG!Qn|Gb;zIIJ{FK=c+yOo&qJ$6yte1f z3f*ngjI=BfzU)ZKqf{}*)H>k?qcUwDdKai1UnM#|DhQr6XXBDc&b{-A{6aIPX!4;x zn_AFpBEcZZr{b?F%oIZE@@ie0RH|j-!c3Wx0e=^hBu{J_V&I&=C-f$nGkO8cq02gn zL4=V$5PsOu$)En^i(I#LTd&CFzznk6no`l3bmIKPFFsoIGS&Gq$yz8VO#GMs1%L&L z7fB8l*eWE$XZPg%Wosj5yz}daWT0#TmIReMc@&Nl<&?4#n(PF+jj+mV4&5h&*JoQ1 zM1+7u?DZQ{B|V@7g3>wcJz8BLd%eVJf2f0*_GodookePNSb>~b2n{)h1>$602`o)b z@CYJ%qS%kVcpuOPiE~}*c!69}AV_vtntCB(=a#iKl3Kp4*4*uS@;d-?BkjriV?aB8 z-Syk1+k?mo(i)EK6;(C81kLC{uW1pC7pv+g-f`(>oAC}~KpV4?Muf>0pl&h)0t&B@ z+J;GL1$s|*wRo*a(Xwn&?mRoWMv20Y`6izT`^-Q={3$ zyKmrazevNhfSQjKV`CVi=vKbB=`}m>=5#Qh0}E6BPcvu!4^{vEam!?8tTV_idktBq zGH%K;j3pzwRkujC$jFv0YQz{Zn6Z>)CW+8kLrl5rmYpm^ijcBJNQx#DmHTrJ-^aIp z`1}E%-_Bzm=e*B(pKH#0xn9rfmAQ5;I2@f)r^>T`@S+J%Ec=rsEC*Z}_4oo?B{#%i zi}w3djs3ncJFJ+^{CVe$-CIC?4s&|-+=o^hTbCt_Z81T-$+|!uU0Wponed3NUEq{| zZMkk0lIm0#*^N$fcVF`4l#v+}=T7XH1&sUD$%mr}3L{nkIUDvGv$cs}ZE1Kc(D`WA z#xKiz`q8_H+|#4jSI?~@Q(Di=c1``F#6(cG2i*U3P+0r{)F2o7kk{bM9RGJW|Jof| zX3a`5ow239bHh=@lNu^D!@0=U>&Y9couwVM-cRWgEYg5tH~K@BQ`K-SCn9xyK?Iwn za{{K-xTpv@?Hp$!=EJcW3v>)9FZaLaps)-q6-qX)8TP4X#)*`D1*F4zX3*~3CYENX zO)}$2a4=^Z6#IwHs5~3pVVs%uE+yfQ$lfYUXsVj3=O!+%V<_(bFfrDwSbd(%!@eB( zh&$#YhBOSo@kqS5sA0u5zcdQYrXq+b4=7shHj$8-!K$K zy5`LMAoJ(_w?vK4?^%mq2t|fArSU5T_K5qYwKBK(FBz`^yP)iw!Ue~VT#a4@L2%JK z2AkPNfAgLG>y(#aBqFK}O}k}~{+0-3Qf>srJ>t`M9ZSGPNtDhBr6e?G|4?{S2b6OA z>G`Ch$8Wt~U`ss#^1`cH&&1biH>6o4E~R>$%H=5%^xy*3K8R^ngO&}~eR=EEEw=w5 zWz$q#NkE<$OMA)9Q{LSHo+JXhr=ffbN=r2C#$k!V0Y!c30_T7Yb_GGBQv7@??++;x zCnY*1s7w?OdH!4{?jxO-_c#Xo%qQbdKC{2}u5{~Rji^UVT(T#VH-y*`Y7$*Mi{I=b zqZCVpZKh)JzPh4H}m<))aAwX`@J@aL-FjMJHI%5~i0M8RIF-Ib(ZhTKlSEC5iXegJrc)DM| zo(<93)wkaOl7*W%T9u*5V2kf(B_B54*nBUhXOOt(vaj7rusmK@Qb|-k!@7*^Z{~~m zimj2uUSDvfcpa2{8xcuJUw=*sVI|r0EKARb)EGVvkOeylHEnZvauog3Ma58B1>gR< za=|TjBeIS2Pz@o7?}GCqGY{$N76b*P!fE@4q?LH;n(+;HlGfjNr5#!2w%o5Y$0Kwt zQV3#)r@}r;AyYh!`K6K{+V#eg*58EM?{|KTE7glwBn?2CNslS!N_>fpl7}F{ik68O zOvVFt{4N!iAWPrVf50-?!e8hn94nc>E3VK9Gs7)eNbQAD;n&TrZLqmW9Bg9}sFT_q zgib~hp?dFyNy(?KnIvuLo!Z1Mi8Sd%Pt&N?d*}KN7mB$CD{rg4encH3eAMd|{WhW^ zbk_;N#Cf~UHh*NX1?i>+%{dw{Akny?4A~RCdi>B8cMUq3D5fhAWWCjV9V+`$>H%X7 z3lFtdWAmuQu^mQen)wOxlJ)T|ZK@mIP!H`bDlGbGy9Qbm+j`aBLK097f zIpezHAs(Qr>#Y81K@Ar7t?1FE(RQ30ujoxo_7l6~U#SL(&mc?RczC_Higg6_quaS-`(LMO`H0f5qmoQS zW#s{PlSOvthHaBx__sDT>Lz^^o-evra!gqUHYU|5%NFl^+8aA5NK2mN_lPSJkAb%L za!ul^9k=tM25Xn|A}gkIW6Lf=B$A2^eECbi%r-h*?AU#QDaXTQ!aK>N`1M4+!53pU8)QvI6FZ@z4vr zf`23qH}CV7+T)n~MfXVPiR52U78NQcAjG#%YxK*ZU31d_l!b*WAm=x{Xt`%gP!ri0 z|AR?+QqlHxm@w!5ILn`VyQ>;nj;5S^SDy~Whz+{8AK`TNw$JUk*ywod1g%7D^J$R7 zn$hg{eN3ONnuqoiVz%@%M(RehCy-(XV*K8$KaInHP@TUt?LhU2DkiAf)6ZB|#qAg; zzMN`54OD(QLJ3eU8mA%(@-P zP=Ki&5jX z6~Y*$YzBtv7bDIMF?KkTDW%XwO$e;6eoEUDeeMAZnbEvTxAB9FDztgrz$7nF>vt$7{%>s_iS!ku72n)$*?S#;orJK4Rh? z3!uP!e9NMqY-OjI-!Y5m$`H;7P3Ply+yVYr8= zHb(#Vz$ch0Lo43GC|rW)o4jg`t=;aB>bC}3w(3^xI%B+yde{N0B9uy@!-k)^^q*QX zO4M==-6>G;AY#kf$TU?bUf^T;If`b$?0%2QvM)P|mB$~#hCvHIUl*llKSlvL`ak%& ze0cT>s&jK7gGB)D5XYr7zQ@8{O1eQ&E)OqJizj@;u{UfyRYUyJ)1iZh6k1)gWQ)W& zSqsfS@)LFOc!=z!%#NSNEtTov@QQ$K^EY5wIDHV z8m1nF>#n)hZ8m%W9i^1{Ei;j;)gD`J3HPpMXc0B$D)=Ai4H$HlT8dvVc7uQ9;Cw2- zWGzhX+-zM_b)T=3*BvfD-`0R@&OIVcpJ_o0Be9%{s}GSpn#n`>c5S=~SbSY7Y}O(Q zL&R|@pESLVGXG{Y#*a-9kI8Pi6++A~i;gvQ)W&>+g1D^XJQ5JIX_XH04S@>_??}|Gd_iLT|-cc)maqLiA zWKk5u2g;)xhIS*tP0L= zcc-uDd^(2&f)8k*HLu~*=j{Rqg2DiewjZ|x&GpMa{tCX2PJZACy($goK?4)hd1XNB z4K4?G!2g$Q79u|c^+yhwdCH!sHBwF z_!X0X-*KMgd@y>QfHj{lH6te& zI2MJRiOh_BTLQ)i^!JnTnpFdjA%IWc|5EecOd5_*+iu@`ngAMK+Te~^RAJrY{sX|x Ba8CdL literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/6-ingestion-scale.png b/website/www/site/static/images/blog/scaling-streaming-workload/6-ingestion-scale.png new file mode 100644 index 0000000000000000000000000000000000000000..3a53c0db3403d3dc7e65caaee91f00ee65577b2f GIT binary patch literal 37699 zcmeEubzGEN_b;Jh5H`{%QX<_sC<4-;(j_s349(CmA_7WxNP`GUceivmLkyiWF!Yea zeK_xX-g90(_uu=u_j7-Ih7ZH@Jp0+T_S);a)_3jC%8JrN*U7Kr;NTF+ynOKr2M2#0 z2M3qq>J{J$?bm=J;2$+}2?=Ex2?<(dJ8Khj2p9*4DGVC=Ob$=wu1^T>!}iD5Hh5w` z(KbDj;xFZX$b9_@YgS6jVF~Ah%rX14iGBRzYnNG%ug#~acHJLf_%u;Mq{Z-3nQMAX zBVCeYvzo@wenwa)MTt|e{2q>K+r6WynzFjW?>@d+et8^-Uhd@xu*JB)6nV;Ycl)z( z*NT0IWHM4eX~aGH@Qd>&)$ZHd__jsk8QQu!r5H^{Z~6S_#8SL{CRIGmubcrQb`GyU zz-rclok%IAZ;pHUT-Pyc4{b6?m#?`>KsfaBHZnF z4X$XylEUjP9+&gTr|vHINlCC~lqC*&sF|D0_UwM46F&&}v`}850EsoF~Bdt2nrR<~WytZ@9oOIq(b2Nw_Z#0r2-W@cSYK@6WsV z>nZque#hmwc<|Y42^ksS?`tDFFxbl8%-X^EKu{le)tLDk4F?Scc_AZfOEv>zYeO)b zv!%^N5gZX`A>gYe*uj9-*%D%9FXSxx=#M9afbSQV*&osV@rZ+k=pzjUWm*YqJ1{LD z8wVT5BeCnWw6r32#wJ3qUP%4BIPjn7BQpmF8zFXfCnqO1CvG-tJ5zQ}K|w)w4lZ^s zE>_?PR(lsK2LoqTD|@HF>`smR`MgRHvS3AMZ z=6}^>W&iJP0Ucz&xWmrL#=-udcLPO5F0KkGn>&LcnlH>Pf#Ct_5aZ_H5&7f!-|qZX z(;WDVE!GIWEp=Inl+m zOr=#&&SuIVyTIDw_MEBeDc&R6Uq3|g;^|UBA~<#F=eBRi1zUvGR91EZ$+%Bp3sOP3 zGD8fueaJ6YKfV9$BkpzSn^L?LHy3w~?LsRS#I!ZgMj@@#WUD-wsSt0X5YHDCPTjDo zed&exub+3~?`3TEAbXY!Z41l2rYghRKS3mw*IYpK`rZPoDz|Rp{`T>@ifetmxo5ez znc^*NHTl$Eh<{taGtm}#l7{3 z?q(*e?a8_=$^T~LrtgHN>0tu*ulz#|OtJ6p?tf6}NdJds4vXS4uS5GZtp90%K3sdL z!u}vi#Qz_r>VGQy|6)i-rWOcj+}}3)k>tCGL+>_GSl1X(SXHF*z8h{ox1Y*Z&lhry z|8Va&nf*kkJm`PC-9}^0k8n29sp;C8&~w`WJ557lLQaxlSaWU5(kKt?(aL^ObV^Ek ztUV=V+e$L5PJJSyfZ)mT{a;?_RYl)lV&$FhxvKBJ?V}64X0*=3O?WX>B1badzU1Z6 zOeGq(_3xQE`t_cYqWsk&{FW2L6J?ebJ@Nd>y7_?~B-aHxooy%}1zx%J*yA}KHxkg) zg1+LfYUD-A=G914;Blf8!W@}bod@P3b30zk3}D!5qk)YV7&IFu$8#F_DkO`E9_UrX zqOm3BLm$R8>xMQj_u2P}mQTaZ&&qz@mis6Nw?-&bMNCC9zYe#$=OOyzYqAr<<=_~Z z?*^v4oCM3eOuA2zxAwa#OL4pw`86##%?(|PnRrG(Wn8^c7N1%oJw0h%Z%HNOP@s}7 z1!f1GZZ-4GxbGhAT1{8MGh+1K0av^+n8gUCV`OYRx^g9Hw9rV}V!Eo7U-Wz}Gr7nh zkOqWM6kNKy;Ln(To}4-xA`FzU5Y+6O{6r{K$pPI|(7{SrZO=d)tiLnCdA+)_wh zJlsJyD^l*9dxfnhXd z>Ji?oNLP>M)IUF3NiM#t92>`JY9dn65y5z3k&rG=ZTs_<-ghb*A z%=MH$MrZH}G36=&Zm-C$JZSvApAzV7_+Z?KMBBcHcjF8*!@pVR3G0sKtb9i5j;M#7 zKkfR})JiZ4t=~wv?w=c!1>o6c$Yo}{wCk$OT-tFcNDI~6u zi(%tiNw9ZqAzwO)*0w6lEk{hmC+NFvlp;i^mnhqUQ333A75l3L&C!}BWYznl+8ZBk zuoZm1!{{n`8Z5k7@LEZ9goK(S!edB*AO8FXtFeS1@qXv&9VHQiF&+D$WZN0%g`iy1 zz9h*OH$N)JE$_?I)lL|+frl7AzQFwS4^c8Z>F+bTJim zb&yKLd=-5??Y#OAlNxtPR#Bxb;mn6rZ>!Vh69dC46WSbE<~{QjjX}8ev7nhM8#&#A z;!XgWTxK^oz;-YuA9{HWDS@UvPd504Q3cJX?{OKnx<6PKbui4?#>22j8_gu3J(E>J z!ju}LBt>_1P4GRp532MZ zjS;%uFDL%hfOuPWlSxqWEXi$iyLc!^LHcOve5Z>Y!#!hNcVtqU%E8hb=1`35m<~r#o`z)ZG)B zE4?{l!nNG55!FuC$jS12{(!bONxW->lm(N*-j@k^0v~WT!HQcwP~(NF^QzN#Jmr#l zHiAWVUe-5KQZyE})-eRbw=2l(Jkx^C1==9h3BQ>;;?uX7m?aa*c7f5IP}6f+(tK3| zXn9x3*q&jV*k;&PBY|h)VcDdW1v0l>FUquR-0%kIM(~H8UL>%OV2XFUpBoLM_G9i6SaM_vz#p3XFx#~eUJFZ4h8wAHp*u<@3|q`jve?in<;5BArm>S^F~^60wbm(yrPi{+i}X)ROgOR4&b5TunyfT2@w&Qo)pK ztk>V|+ATK`(|w@a1aTS+P%%rF6~c~3 zvdYM;h*7lCGp|iq!;8Q<;k}}>qdwuiLN1OuZII-{LQSgbg<|TuZ^V*`&)(~RDO*Hh z8|7s3nZcs$zxLJV7SjpYep9$6SzTVuj%@X274nqaSxPrp?AWY&F z-|lMU8To#~-s+oQ%LCe^^9x{O7xjus>iu>zW$@}tmHZM)Ro7p&&i@qqKQ$ygxh0pF zMSs`2QIfO~9pnA^bw$ATgU~9svXrLu%u;$g6LSnl=3~lx6NTYA_B~)>^zhJ@TZ^i? zcD9;5e5w~M%aUg%>YAgXCjK=mM%9tI=+}4mZ(Ws77Mn)6p*<+hCaw9}66}p}r)-R9 z+>Sfia?FRmIlfNTM!{E-F#gAefDPw)vS}H?s8nWIhBQAr7PAf_=j{|zUnk)5VBe>F zb^5F5qLL^CJJWtLKZ;M$spQ zSb+hlAS^4zn+S%WGp&7;50^j|sbr!g_O$EsKLAYV)q>?3-Rd$v_d^@x@b`$?vjdz- zSiJ{i{;SNv=?(UYVA1mOKG1p7MDI|UP~#Yqdp|8}%))=)?)RO`pVp=ymtcIGF$u!D zVrDMKUK*KO(6qy6Il<$$mFl-qKo@*qF`8dr<^<0u4V$bIyA#Bl`IxX83q0|{dQIcE zH7$Ng)*nZ>y11muE}MW~{dcQ(4&e3TEm{&`d|^aj$25G4hZ3eIwGt zL!o+MdC2iXqs@#W!Ld9vI6~Dw>-PsGXlW&DA1)RRGX1I)A1U))0%T7?jXCXC*`viH z@_Ij-#s5)9jT{-+&|u)b{uFXq=jhTGdCfsayNnDiRZqjH=YJa~uMgLT1Jd-yIu%w= zpFg9T{m}J6s=({hJj&MiEP0dLdJD*C#vMrnY zw?2#O;Q@1)$;{_slraC$!BeGIC3F~9(JMn6swfUF=)Hx!{9j#F4*R3#m*KLJS0CRD zRh7njp!*AQ{6wFkJ7ZM1n)bJ9{uH)h62SMd->hr>JAA$ArGfWq`2X8-Uf=e2wQ5!W z5c{{Y1tbvJ;#+(+8gz|j;(y4Hn${0heA&YACw~g1@>5_aTW&|dE@Uc;9?wG$)SQzD zf9Ua!S^V{DKmeW7~<2n0he33`@O0^oAUsFS)so}Bl*?w z8X+ZLJ5d9rInzoBZkxVHiKBi>|Ha;v(sY#)HWN1r+sZEdK4G*_yD^*_{JC~anN3#3 znn^I%p_dPDJ*9_l*fs2L!zCVfTccna4@|xbzq8^0W|(Y*S_G1#V9xg(iDRlH_%76% z>~KR`Y?^6Vh16sayz?L-gX*95 z+S&{6F^D5TgS&Y16i?4=SXq&g(%asn{eU-$7)_yhIC47F@?bj8N<&4)ShN)p4&5hqE zM3rnp{*yDla3qQ8QHB5JBzZH&Ww_s9WHiN zeK+_{RM$|I^40c}gcyx`z1NfHRR!1!Z)Z1MySwrloNc3Ls4BqX?yo*qLw-BkX~qp? ze^7b8&YJ9T;IiDF9>Plb=Gw^%wA8s2Nl2Az9DoNKi8G{svnv7jb52!f=fSK5$)tfa zrR9YkuM4w8SGj3k`}kC{_7c}KmA|&9KUL66h6dL?WZs%3fc?u3HYT+r7!<-q)f=do z*YAaXmi_SSpz&P7qY`$6T)XKje%?O!wfeQn%nS}Ggp9L#L@`rk-CT0o4#N)*N1Q}(bLg+nwOk=u zXgy!RH%H`rhprEI&?tT3>5vLMZ4q+P8vnv;5$(~@M<=DRyWEqzP_t{2uiEtNVNGig zg=sqMWSTJsx1cxp9)<3#l#m+4j&sJHdu-tC_nof%bVwLmB8w+JA<4rPqr49qrOV9lHn9edZtnGAbIw}l|A zBhvER{<`98iUlG45RskOQZslHi!}WG6mB9GF{WdY?7s7TAASA}Ky8gxtW{>iaxIjg z?Zu&k#UZ6hAW&nDPe5GcGyPr)Iht!KLqpU=?Y3dWl2cFvwdqv3K+orAXyTJ&q&2FG z-EZ750J_`BN22H1q8dES*vbt=E`rX`!PPZ(VmmSVsQ%`VqF$@_;0uQ&I+n+!XrpK{ z=>Vu@!WEgv-Q^~?@!_OaawgSp1n|4PlVdA<0lBLrybGyMU)&>UY5*&r&YmkDVEtBt1DOeJD|9mjf%wh|ziyUsQVlpCQ`eDZND}HIbR@ z`R=^>OmXt<**>c4p+sjDq@_^9rtxyo+VX5u0Q){PsEFUJU%G7lN4Cap6{Rw5i;?Zz40;$?t#)lH{_y-NGr;s{Z`^VR9g_wAgEO z?CZTP@vqcaD3G7CjLlDUC_6=TjsjtTR|8y09R)Yo>$)*rGaeYxdXY*XR#JB!`ljrO zYO7z7^5>a|L@`NFW?}Y?0htj%^D5y7zFjyM&fIf9-DY%bz@E6E17oM%k>_xu5V1wm z(aLtz;(H7tp1uOm;xpgmg&lnOgwI_e-VtxHb$+P zCK_cXIM#Lo^nyg8P)*fW}>ljBL4A) zWYGRz@j>C1qUMzTRtL7CDJHQD|FG5`llP3yx~MIVQe9fJuEl zcz>hnsvkXiCn^^`m9yzfkz)!3U%}BM>Og=*zq)sOe@t&<$f@C(--Ml2fzjudbE@&V z`OUn9Qrr8S75399Ey8(WRx@?FnlZTo_rIcht2Yn4p8?^)Z$WlyFIEzuS7M6U)+%Qo z=0Q?*3Sd{kLGU`0-VY0ynRAYqYjcI+*l%Lf=MCP9SdqwSO@~PvB)}5YQUxtOJD8rdpZd4C5&Z@x@nymIl&kCM>rrq( zBUz4YukZ>axf_UM4#K3tr%qOH!*Ji7`VhIAozFk;<)h7oL=^{0Ggr-3Wl!TE@uX_I zD`d!ZRzdQB_e$vt5JT<#))%{p-VV!$sHO$>sh9D=jpw0*RlnuvhROFUV+EZxAh2Kt{99 zx0-GEzf>!$$uN3rIMjDYQS^kXXXn>_LH2uCuIM|5B`$wl%Bx0+1SHE=a=0HVIj)nL zP7n)kqDi&PJfIeWZIsxY4KMSTPs_O#}RNqcUy>1zEo!2`=c%vwOqpD5s!ewYT8IxPFa;ks{dsAjop}6~m zB_D#WekMc7-M;jS1V(lgK%*hD8rodL+ne&t+me_ud)eXKLcoVdMr?^i8;G&&Ev+Zw z;-gyQt;#AVOM2PbG@Cn``Da`wi=Rt*?)H?InZ;>n20aiQ6tm@C8ZOo&)iJg|HC?;k zIa;rS)lC|1VkI#m8+>3o67dEmdI=Nuc%iEj!ug4FglG_5g52yioVPJm;2&V*f3l#i z4OIoUEra)IEJSV!u4!_Er^IXl+d zyHUBak=sX|qfL1<_aYq_qJ0U^28mv=TXg$6+R9Da>>1dfXvuIgEHUppOkf7hXp zG?{H*MLV{;0fvL2_0YJE=aMS+9M}U2=B#{8)`!ilmhDuuhvV#ODY%i;6v||JI?$-7 zjrHuj5nL3KS#LXW(~$WaQ$G5T2jJ64;8ci(GP%mS`>-Y(a*kc2q! zOpIPph!}QUQE=@m3MIsSlj+h`wWs7*J6T|B=pp^^WX)Qrf`Gc9UKFzzQHDmi3Et^I zdbn4!PK<;s4ptyGx{WbYM&(b|E`-id>V{e@u_4sv)VA%TcG-?B3*6893VGrqlWjf+ z6U5T|P86$^aZs~;BhK+94 z_ZkRZsKv+kktveZj{8*eRArN+)Sg?O)pHXT3DAB^K12^KqbTY7jHuiozKMny=F>Vv zeSPk027HrrPYO%@in!k5UXkg%O%ET@ox_DztIWu#jj8a^$48O*kRS?vL`=5lnchKh zQ~C510S}^wQ(BH+06wbyuzx_?*5_VRZuq%LZt5MHgK;_%Ee!qra=(N)Mz%=SiWe9{ z^QA?>6XVAtGoiDO$@(+)Ad<1F>e^|B`_ttTJ(3qs;1CLro=cIQ`c@ntGyGa<5{N@U z+7fNTlRLTvn%jx3bP>Rw{^eMQg#U)~jtt{ZhHBf0Rx7pAYU@uAIE2QRNM*GIE8M{r za{QeXQ+(Ri=BAY%j2xw|=iVgE;BwBznO>n(xq}36Zbe~wi(|bdUt#94qI+%gS#%_Q z3?UhJ{iy8isovtuB8dp2yx$m$sk?KdINk1{IBluwH?(pmvfj9}xc6Hd0u{%7PmH(v|q4}gE zHAtBgI`O6+*YW`&VW3@o#Dgm^!1&Vvi&zZ6Q%K2ve#-=S2jBDck)#~434iRNrqAj0 z;f5|mXMNfERU*~jR}2H#mU}8SUw6MZw$f)&Z1uBKF9WlL1TuQa7bMO`Un`RCiOWo0 zA6v2$%}dmAnvOf60oImm)OJaTT*b2q%QE-`-z4Pqq+R?*EgP_gfPFeMkcUT|oqIhH zJM_~1c+E74zw>r6r7r1F`NI8}Ya5g0PE=J<8*!$|T?muJg$b*kD~ZDZ$qar@M_BEs z`!il?-qv4xXl%lg_G7ZFeGYQU2<-vMdF^YH?18O}Z`2*2DK=t?PuA&2>R@0=cRprn zVm9Tf$d=Adm)(F+JM>#vbCBZsq&3M_e8&;NjlPC@nPLkX!u2QR(-v#lhFX7PjT*&2LSQKBiNb zL8I}__s86x&Zbj#sX6tnvm^_xC$h`+8Zx#Yel>36l;5Fns-*;rL@V78vbU5EvX$`j z+~AL264;{==!NpKYA-!m*o|Tx1p^!G#^_2bvRGCK6|G^&X&QIx+nt4vnJVrBlmHTZqe7uys*2Ycg+p_8lu$t^(9`pC-Rh%L-%{9m zr!A-Je0rkI`u3fs*z_gJ(Se?2PGvhF9(kzHr&YBw9u+@58(^E0=-Rk}>QO59I6ot4 z5z(^tEQg!02p`8T=wv!&aSLPHh)=A2ccawxpxlV$L*CcjdUNBbz~t+Tj7i~1EIbdh zbIX1tu9?JR5EdcNX}&vM$c2S$M3R_f*KBGI*dYu8(D}@;=MVhgZ4f}hTNcolnSmQL6y|L z(HM|tct{T^iKE4E9KFeFejzkS(Bl3&V|+J#N9LDE)^)DxFU8R%2@~EP82BVx^}K9- zYm#%a^Qo;E@;Q9MeYXeI=to`F94s&q?*X6?YkiW43E$_twUk}=i|uIy0;oqbxvx5G zd7Wz|NP^vUjyacl!4Fp0*R%vXod@kYWf&(iHc2mQRn8rt2RYrYDj@{dg<2TrAW=+Bv;g;pyI^lOtGdyC9 z;7?w(7UzU}NC4=~D%t(1rFHEQ!}(BG<%ThN@@g%hT(lu|-*s&UT6(Xe?$z@^&OI>u z9eL*PC(>jzJ!oIMsdiqII5R^*u!%RXHtMO)STI>6ao)mxU85LuVr-3)6L+J*%If2D z)W^VUlqR}Ss0VW*=9vE|rRM(0BwvU6DXMcvbI zDIj^S@$D7cTCR*pgvN_z<2$WtiUx}-{2y2rD`(5Y>oOG+*K<9?qaOo<@2gSP?lhP< zx?C7drAcjx-Wi#ZeWEl_(w{eA&h9AA5NyCRB`Ye$cCXq&3Y@Ld*n&$2>Ky5_gZE|5 zeq7p)Rcd7vr5;mcILiHsJTexM`#4#7tusr-E0AJ~jAkX*&b+KO$|NHCPa(+S0Pz;)^^kg=iDMW?t0xWj z2sf&~EH1a)ReRE9H@G<9JAw`mEcX@};Pf)+WY<(Q%2l8L7sorm1x>rV-p1Xr~L& zD7TtkXZ;L#LlQ*+PO9Ue45KFC5Ox; zo%@jTx2XM!0DvH!Jq^%sZOb8DtmP$3(b{YWdX>6wbeYR3U%UU+-V~9mZ@S5$-P1D; z+-oZIqQrV2hTvt;|@9QxW z@EElc3hxiU-ngK&et&&l8t4p=tv6zgf(zMntHv)Js{jwz@<5L{Gb6P%*JXEEEWbS; zU4M=MXf?Gc7l3NAem7v5TAQ$ThF+55m_p9NJMQl5jju$ zM5%>ovgh%4)@oGI0l+I&6*T?k3ebBGMzyUqi%ohU5ej_UGZz$Wl45|`a{!Q*Cjr2| zLU2{tVe4dnOn(vpMzKL8?8^wiKbaUrAbZ``Y_cp@A(o>NRDQG^e8+^i-qx7iklZnVMZkm( zq1X#j^2y0=A1EkpM^#;SzrKH=KK3(F@e)T1XESWVJQKXI-AZ9S4zQce0T#clJ1x=a zx-4U^9*-B2cwKjoTZTj#OSae)hK`)J=`97iadet)1u1B}Qs-$Sn)lQ|W_!+$=Gd+W zM;!3^dK@)_z(QKoL+hP@@89E)!m{L62>INry8GG?P8fDQJO5Zn3^Qeaax8PyqH@qY z`&@VU-a7MAoKW@gdbZn*@U@}L6N{tH@^$~dM;?F03B+4&WyO;1%j0V9=69;UmtMJi z`F-%2mPKxq2Tn0Y7+o{8`l9||!om@t$D5!<0pV%QVc&$EpCoSpoOiwNQV;t?8Bu@$ zA^?K6*#-NX%5mHPTZuh8nY<-U`3KqA1E7AFU68&3K7UOb$ZqjvyogH?2m=gp!_UJ6 z``&F+CXmSa1()9bF8Zx!;D-2_NM@qr4!ZH(VihuP*V_{NyxVVGmhi6bUtw&o>kpN~ zx!2Fo+8Jgwv8Bbfloo>vl$decI_0)*^rL{MhCmqm`%e3XgKbGmWYL!C!N6{M zuuvyUjUSC?1&~4kst%g}jGU@029$y&_7F(_*v7~6qqO^^{x}vM{AAqOFq+2a!R5&NkKJe{3^HeMz@bi@AiINcjk zavaq(J$m*YI;K0OFSZcKUGu1PP?p6-GT_kfnXlQghB~6yR7qmH5pQllnr6R`+rAU_ zWg#&$!Op*jyHDgZBCj6n3Sy~WO$%T+Hecnxg2AR}U1ft69hi(nJbFJ^y$~!sv007Y zG%YDaIUmmXZPe`c?uNYR8Uz^at1<4%(KeBpv*Dd|_z+uY2DmHQvz46Nz4Lr)uK5gc z2L;-0B%pD=aqBKL$LvAq^?lN%B%zk~xOsfY>rijIZXBG=Z}C<;-pU%a1vC<0UhB%F zr@Q@U$9trVxFq{ZC7<|}=LbuV@@ip{JpxTR9xB;)J8~mKcD$}}c;l$+*!qMBkHtnwJdEQ4 z%Z@Hmwp^9;(RA!Sv~3&~sks5B$VgV0v?08bf^*B!rn`bWJtW|pb^Y1FE_V$66~cnt zvWd=vX{VX3m+Kfnx6j4kaCmpnN+!4+z)f?lXX-Swy#zp2Y7>_@SfvFeIWLIxz z9F>waW}<5Vzj2>Kqfc2qR>c%9E|ysp9um6w*G@0)awqG7WFq9Px|94Y{0-ajE`({T zW%Dpt6RM$2F{;76>)3RSaU39huSZ=r`Ifz)9EpLO(r6n6@fL*?i$(Q1K`F0aG1+9I z?d}Gaw{+cZUpGCng;6nKwP)H{m%Dm~|CC9r$UpeIA!@?&=%ctjWZd=@+rLIF$E zQ!25j6PHpA*Rq+aDK(S zKQN)mM+$`+6unPfQiB!dIWm?^TREU@sA!HsN`Gd5#{{F+5Z)b#KQE9)bm)AMLgM@C z!veXd^3_iuq?hIig)R&&DkT;>M@Q(W*T(LTZ`*SH?VJzkfqU#{LgW}-h-<_$upduZ z+xWpbyzx+o-W<;_J~(p*mm9w6HKIuA{ew0+kZtbCyxlk0G!FYdN-kt9oL&Ob7)s7h zA1-1Vs~jlZ#FRHXFkg3+D#5K}*>~=`^I87Ph-xdTdQSFDtXPn$OKMcl&C>ew^V+LR z*8_<8*cTJ)&#~3X3YonOm0yPMT}k4YLj(NM!pM)$IG(!LFrj4>q4b{v^Zq7a;G0M_7mb|x$jvdvvrf$46j@bt<8XQ1O>pBt!CdIx5aBaVRsIyG#k;gDc7G?rE+@l>gaH}5%qZx|E_&)02fKpj zES;}cd)ilIphm)vmGRR#x|QPo%C}u2T9PBXdnZt!O^*OVEG=qe^t_XI$?N>AK8Ie9t%P^Do!_mc$#$=q06CnUkh#*NM7M^>Zp{xRd)cZwOTRrT5MQqOinq z`1>D1#H}C6NUdHu$1WDNmRAL9%6KAvFa!(Slu8JvCL$d=nZmt#H-(3LesxzU4}#}2 zoLB(C4+wlq_CfV>@pdD9RNI%)d+Ur)3UEag8?#mATuZGSls+cB_irZY&eZ~v2lrDA z2I;>l`@7PnS_ApD;dBiQR!qy?d`2!!Vz*A;B>#6t!y7yRAVh@*$a-Jp5yyk<@!8CZ z8$~?tUS9ey(sjzK3mSi9!W*Igy8JJ4*$46PG`+C+4%HXI%(wp1-+uxWqHuuncO`B^ zw&ZsV^8Er0{SZud{s%Pl@&XMRm$3}|{>qFCH1vHraQ+|A5bXsT3Wevr_)CQUBw|E9 z0Xm!Uh5MgK$=(-e=$T3N?SDW+KQ6e&?|PZ^es9VDbngG+nJWr+0*>tX*I59Ml>Z-@ zMK4$NVE}|$T;v&i2Y4vXKp3Q#tzs50UclBc*H5$>KS2okqZYtT06)J3ezyLuo*hWz zxIzKBNL0oc5CV`5fJLesTRWnMOELPcQZ~ncH6MskAG@&M52ozi8Gut@{wW&?%S*i4&4x-C@u;$DVnNdlQh9C5gJ3 z%y+IA8$A;DAcq*EX#*sAQh3{{w9($O+ySVD_i7U`&Wr6~7ib22 z#+l^~AdQn1CRX?bzZ}}C*kca z6YT^ajBfgRJ({8m5itaEKTAP*K_OLU7Rai4HTf#g;VeF#7}Lrh`Q(%!z~A6$w{YS2 z2&NKVCdw<@;VeIWNasHgvR-BRJ>ZxF5R9J(Vp@5-CeO*7vz#Prs{#=K?tI65Sc%Tg zwXtiqyt?8vW$Tw)v(wq?#7S(>O8AVqA~VUg^X3oe+C?%nsOsVXfD_*&FaIx3c%Wo6 z&ztI&zgVn8mJ32LG4V8=Zh-6LG?3k!H)>VvbW7qQ!}wh!k5tpj7mGZ|gP}|%j6bfd z5Z!8b7C&&@z$UPOa%iYNi3{GoieE`Q?>tx=p5l1YO5r6=^tU8>JC@?Ou3&F9Ff40{^rz zkYF9F>$;YKxDwWD0#`5LnIS4;j_GCzha=&_Qp(_X}c5!+u>HtnkYET51BWfHH%8@#CNm|g`z$P$|y?n|}N=L=6 ziO_iBgkyUOP};e`p)iPq8(kqfZM-|Txt$QCR9>bHzHWMOScs3F>l{RUEU|O!6 z6w@a0mZ$IKP7XFa7{gs(R54Q!GN^#kgPqxHwwkcXMGthebUxF#jC_C1SPkf>W;(dF z&=BA%D=f8~9F3`c+j;!+j?x1vD-+|{JMu)$-rdHMrtrdshDbJn?j`RL=|SPG`1_z} z*9Na(;^5;>6y?4*+4|dMS>4CM?JOExTnc9ZYWA=M4!yWIiGWKDt4yNF;N>FJ*Bl00 zCtM;o0ceF)mtB_W!gjqxOz+cnFWORckuE-rzSGPEUjnlJ|Ytpnoh32`xc##xb2XThXY8q{_Q5f_qg@~q#b(GPfHUKXLQIv{gF#s5bLZPMgO}O6 zeVBzSN<>oXLd%~aIOXnLF@9+gp0I~oO>$Sq*u&=O$tO1MRH6)!mzT#X;-wWmCjR-0q8zwqI(7PUVr z15$upU7Cu0YFKhX;s{!jPwYF_DvtqQ$g)&%r{J8*l@6NYYrvTy>Ia@$&Z*r4?5TCM zZ3{XHsAd{2KISADFax>RKEaJg#_T(RLZ<-1(uIA%oz|md_rl{>KAx_dPICw9Qxjsk z+B`p4VFUW!YtKHmZPoeGs-9NnzH*<8oK2+WPLSq|8H`n*@&d-Q$A^h(WInxc3>2_@ zq^OH9hG~HgN`@4RffETv_1w2xAV-X%H3XHChzOJJIa?Wz5i_xQcKvGkdjC3`Nbsv8 zI+18DWxfLcYr)Sd`jfAY0kQOT+N!(Ihfnbh-@o@XDd1Zl%TK1sY9|V@Kc+27-(wF8 zGB{$!gilayrlQ|MOMo=>!#dofDCDdhg>2j>`_;E4tb0zY<>A4z*I5}GVYTknP5LdB zA0lj@^DU6?J{d!<|Ab!Q%JWL1cb>Otz$Gx&F{NAJdbl7|PNptc1L{qzlxbCcOry>w zG)^{l(^ThKUj^WAI0yk@x3Gy^XtXexenEbkJMY#(UyIKIjSY<=b0-RL%93S<9?Io$ zC)(n1FOVC+HiZ(&)H%-z@4O&Wf5x4M15S7pVJj=S3^>%+Dnv@DhY96`co59kY%DJG{@RZm#1 z7-;IfbSpH;QYJw|0(EfZzwQ(n2i6y99jjNu(R2f%FW(g zaYX>|j==ijkS0^JfuP?M@a>jj`=iX(dWAc23sjkGD-!gRyUT zOnH~aAN2_gwFO$3aplo1Cf}fGC&&~5FDJQKB})&#o|nenR`1M3zBgHpH6rdyS{X!; zH7yBWXv*&=dBw0y7StQ3?zo)XPy>1>#_C|!GkBvVBg$_#@I>KcQI!ntoH=8u7rVq0*rgziLUZnpJI@}yWMAjBgd_>|%q%gP63?+_4CA-5p^N`kfnT0LyKs=Vb;fWEYgb<< zV`CMWa5p_$M9P`%hKW|PDIR3_#H*R48N7x~1zNN!EFYFsJ)_k>*t*l>M_c@T=4*gR z3N>xDjX65t00s?gV_+W+PpX<4hoO^_-x8GSL zJbq8H)mM@69&4ply})4XF+-Hs&U5_KYHJwemjLx<*Ym8r_AQX&ZtuGHfZg+Re^)tC zn-f>#saf?hn5w+s=Wm&L!2{K6uZQ}Ag`~d!sCcC^{F&hLO=-s9UXtrp%J0lMrmtq! zhv-Kd2z}(~JgpwcgUs2cuhum(Qc@;5y`4>)JG>D_Le3d!OYX@(7;B4cVVLaz(qU-?Nd(B#CO4(I7_wBvTd@=r+N+HWsa$# z{4oqJkxW^ph$1pWlMM+KMu(oGN{i@4T+65B>dGR6q|)KLi84j~qJ^RJAF7hJ`(i;` zYb|SY8*^q`8Mlwhspi4_{s5Vbx6YnjnzwY55t%@Mpi6IRt3l|S6IJ#&YU6=BH2mf34001oh<0%_we%@ zSqy!s`0Qe4K3BbJ;5{SNk#5)`&MD#q^|X)zGr^U`Si)jjf_c?G08$DuU)LlOeg2Sw zv2>WuG8AL@3;!T;T`FcOzID+^asfZ-tjY48@%wCkr`V6dkn>~m<%BVIxi8)Eq*u7{ z_`O65L~BUIQFb55tYmY!5;(rKJA}J`*&UjUmX?3o%iyr$WgNryy{Pb1Gxt*4rk?Cd zyFsY>M*8@RDC-`>zA&Ez5vu7;(7kBRAxsO6L>Rlbh8uD29O$P?wnu$RHyNw7M6y4^ zSa`2=SMXbCu%fMcfLeH!uMYVV!+Mn#zi5>@fWB3_4+B47#FcwVLdWP?RZVJ|N@L2V+lL}2QK8BrTN-aTrz%X;Wcu4hNHWlmZ=i#1duGntwzl)a-|&Jm4O z;HF6d6L1I`O#LJDWMNjWmM^z;x1Yz-`xc|%Qz`eJ@*)QU6GsDINlH()%aG<(Elwrh zq)urCMci*gl^D0UBVhY8w9UB3)6{zhv;Mi^Wb9Facj@u^aX2|gghTTB&qOrtszr_g zjCHHe@qbcBYEzu-+zC=r${( zKDa%#ay`QOwfBfUsJpa0)ZoYrfBllIz$hNV2U{Ijo#VZVynPyX=O-@ScJHknLP;60|HT^l@|Ix3wxAi^-0d@G$2BeU&m ztyT;ZNrW1`@9oM^X+0V;7VYO$RP#60!9V5vL`1UR=|x+}biVq^{J@Hfnknk1Be{&- zrx#vm6}ST0a$Y^%{`R>Iu|lQC%l82~;FCGB4J`aoNtY*uUM8>T6<_?h8P(apIHTX6 ze5AM0ZOLM_5|gs35tCAizkQ4}s=IG0DmM$ics68N$FI_h+%LmP7*&NV9L*vVtfrM_ zvZX9%wZUoV8lc~V!_Pvvunu?0OHiICiQ?Pf?_Uq2G>5gudMi22iCfmZ%kS?B3Xm;4 zxV~{#Xcl-Pd~LJ;_*&{Ue-Kg>I<}J<Yx({uu}~U(;!FTtqEt zm~}!HXo40bjO3jI0V&VGwwx%qT32IVlSqdZGxN*uWFhAKWW{Ob`vJ7(XRWNyk7rYn z3v^RMjnnq9?YyQ>?=rLXvba|Zc)sbg_&sp3a zeNVHl&Z0b-3er9z7lj0T(HxtzOf$S!q6HjhgPhGn%0sp5rGxTY-VmA#hlGo_<{Y=b zRKR?-<8Z{Z16Fv>P%>$>$0iAjT7D7!=-eJ$0pU9|d-@y-eH2dAEAhpt^3*#WeJS@5 z9x7584_V1?kXiRv+CYr-$i>ZP4(STgXl(LKn0M`b5b=%Y5<~>7Fa3s}W%E-%(vDV^ z%{UcX<=e4#0C8p0XdL#~#Xr8t`T3SHq~OgJWr9Dg9Il~ z@*69M?cs1;%NO!@&sGP=_<6*>WIUZT?#eP5Q-_imF?W<#;RkI$twtc9!cz89QlB3> z207_D&o7zRc->~-j3ie~;L=CRM7bx;)Qq%3>xQ$X_tob&$A;BLIW4G|dVnR*b(>Lv z7KZ{fKQgdfqoOZ<7IdVLE=v)^*brj>M}IQv$MwQPvEcDZQ+km z&z4}$F!71%HK)#>Fyi+$hH}XVxj&>(bBy4nS!(e2Pf|$QR#G!f-*kPk04J*NrYA^E zL{y#}RD?rwQ#Fq#M=!EA-;ggctUW$TaFd?#JFwy83Uq%{7>PscK{HJK(Fo&mNJNi% zx2-;RL5b{V^^+y;P?=h~TWkthq398dFVOlrFVamgSJ5-gu_dW%!m|sbTZ^~NA+Fpo zLT9kvH%f;+ea=0{*ZsdtQClrDnfQM7wT={0FT9{@E!TPyE-4xkskno?>LI!>YTeMo z*UJw5{XK|(+tqpI=GQ^7uK5c&DF=y+HK*O_A#a+R?I!ju^1viI{J4ys#ta6m14V;! zEi7A{g-*d$`TY)ky%$y$%~ji2N2s!217@FkX)>7k2UutCIl2|Z*j>RlS#~$1o4H`E zc^1dx5?9}0pRO1OKeQvnqy#PTEq6vwG7eO7>x+XO~8SnXXwJoT-5{VQ^^+ zLC|&sshe(DkS+>LT*EJd-Ju)6Tf!$4bU6r!Aqj#1t?x_k-k(iA>S6ZkabW zPzkh%*U#9}tq81^`|Zlwe4%9bH#ay|M!yIPT4Ue$QeXPfP|R(jx^zU3C}fx+OA2aO zA!zjoIJBudo*88i2D!>K=R#gIW_Y9_(rfSH21ntkCrZ=Kf^FE_!{S+L!*1hrrTvex ziYlpJnk5YEqz>5ejI}4h*jIh@Kso~iL0_xs77XL zjWR!YP^vUniAERUNXkLYXZ36;kS1v-$IM?)jux8ci_Ar^o^2qTDeXd4+%AT!(-G?w zPfZ|)iV2EVH=}wf!(U(;g-wt9CnlPoHgojhJ>OIZ)IHcFSnMNr2|vRS%xSq(z#$Td zvQS{GexNqS6I;c&^ffhotsNJVeH#({RCgQY2qgvceA6gP7j?zV2qYz(+Qs;ix7!NR zQi6#r1{0bOYcPHER7--o8|n;xUMAYz z;gr1HC-g!ahs=d1(toH%Q$AlqY1cyhK+_tl&eYJ8;rBnwU)kaX;DftrDfp6uu=6t| z=dx^}IC*NWwFaSF_-xs#h>SDpRMGZUI}+ z6^=XJ5FV;=J`jsZO^z>OK5Fn!^Yo%ZC7z+?d`mrXhz+TQ9tGg|N6O#~FLc47X9dHK zUr5(?Meq53S6=@RK$xQGpm<+cRHZ(DiTU_|q`aU#+&@?{!1Nv3n$T`yze+3UB>4u_ ziSnh7AYDHuhjq`XL8JTS;1gO-hoy^%d#_>nyvI2-A^v^>PuF4nA5b?Y{I6V1nQRD$*lqqQ$N|q*kt=ug zBAAnf+f{z#MJXU5SN8;zSq3V0emqIG~t+@Iu~#Fp7f znrZdt;mza&@0+aTmYO=~lqfUbnx9+FSRj{#Y=%s%iiD`F9*LWgg+`h#+FF8DyrayW zrHwo-4tT$bb@0%isdv>Ch#YEl1yx!cn0A^ zWDaC6Nlj#z9M|`D5NpM`Ql)O*>>?8NhdF`;WiP3knd&7NWL6ZV6f6j9QUxm^AuZar9Equ3;05#NfH23dHVp3YKIio2{gS-P^=4O-oJl(s_|17(PBaMe_!7 z#aB_-X)!oQJTMWsWOACRj9kVDLl!9T@%pUWI8lUU&1GAjRDi!9&kJZ37MotQ6U?la z`jlc~m9GzXb9OO?C!TuS3lwMN>9`P^h<4HGjsqD|D$yO?>@$ri51LS1mKJL9c@ZgI zo|Ps$1NYo@T%j0Lzu6+7Ie(GY8jhRsbB%lq*qg#~7^8K05+Yz#pzJGA3Qr{9fl32yx=qh?cYuI*+`r?Lwba`yAP%RA%HI$H}S z6k#vuSDTk3=88XRb13o{;SeyF$E*lz$oDe0u-D^69OJHDf`DLrlAua-KvubZVtk-m zlG}{Veiawr`m+xb!${rmdMdMwlC%x9D)g=FrB~wPe94)$O((Jq79Z`Lx9qWpmwMFB z&vYa_#OGRCQ^Lygs-B2xqfzI^2yK2|#bx_3CoyD%hdUSQO}Hlz>A3Ugc5d@yAnrvo zE~+Rb0|VJ!_da$T1VO}`=wVkbk#u;OHR@D?)C5hN zA~Fg+j|A&#p@Xz-f|H#=aXW=EPV>qezP*O{vR^SGFf|6DDS}5dgWyz|Xt2z2fR01p zr_c0(kJF{w!b~F^eQ78(-$Q~u*P40x-?GE-32^77SE3;#3fRzJfza$`wN*5;IAFZy zLeIJSB+!HgV_lR$PNY;fcv^SuANDqnb_nm_IFZm5C{Qz96o2>UY;LZBezy@Tmr(W7NG#kY`91mU z3Ng0X8tbukI)=-6Y2mmlWY*7gWF z{K>7uRjq?Fo`ylZf*W;!-vlJM3A0 zsF#|CU$t5xu+Bkfo8(IG-sqRF(OuctQqtxY)o%)n+`0C!Z2x{LxfCYujBZpqxJyFx zte=LXV&LkX4A;N;uTL_*Q$%;=>kdNO*Yx-f1(7@+5gDa#khJBD_0=|I;?`QIfR_)Q zeBX;UNad8V3cu`^dqz)bmtc|Hm>w&Y*d#mKTH?exndg_Y`iIOaA?3eU$txP_CnXOP z|BY_M){d$eiCVGFC^)L|kyu81x0*wL~(K^hi&G9;JR9*+B0T zii1a)`Cnf$H8Z`RRdRk#3k=a-zEgq)N&1g~N4{&?GOld8Z|(TIiqVy{0)?aikCmQ&nYV1 z{cj5IUrgM;T;Xs460J;q(A*rfqWcm)$Rbm^sSpCW&&!zY zF17|dc&bk|$i*$`s1ka|uvnO*#Z7}0gF3jso)cz7DP#a{bg%G0R?@cs@y{J!B}qn( zU~js}c~*9$L4$I4bFAudKIdu?_7syboSS@QYDx5R6<-K&|F`Ha-d-htaEuMW5rj3} zp6OEF0Y*utfJ2V^nLePA@s>EXbg_RS;Fy;Vm`YF8jIa8(h5)Qf<0chg&rKr)5J9(M z$dUlLt8_*teFa!X^j5G69t5HRoX%3;Nk1->e*FWK^MDxp z6f&%pksTlW8^NIuR)82PC}=4k@kPm%Dk&eQGT7yExjYDD289q8S&n9IA1<7|%CyQ1 z-Wwu2sta(}C(|Y5O9~Lb9I~7zepz+%Feb(2S+>EY4>}ls$FQ!$ow8;&SebB`czg?h zgIEFX{$U2p#VbUwKK#KIyDjA_khO;U^3ti zIy0E=1ek4Vq15>j9qxKy53gbau(en9cz50#(*juHo1sy-Wko}56Pvl-s2{4A7T^3R zRT_AbL1*e@=C>GHQfZ^2xK5z{E0ni)nai;*P1GsPeSept? zwMcUa9*PDahl%NGJ3Eiv@({ptD*Ep1WNUN}VBM;KsF-}v;&?2FF}9h0TX-n0VcU2* zL1a#gs9C|$H4Y+-Jf{S<<(tw1AAQ-ktyv0|;eXH@{kEPQ#1Xw%+;!^TE~}dUB#JJ0 znoOHv--hvkGm#8Q4=i)$>I1NuE#T8r4ydHZ0Z|8E{r%l_`%dKoV>n@ z0LpfS9)v410Qumh_fVVws97*s;oaT}i0-XtY^(M8e?OSl163sMcP{~SWN`t zt|;?Ka32tiCXF8FXAW2y5BQRW_3Bs5D}dr? z$_RG-(RenknM@9_Nur{wPuKii@fT)WQQFo7%|a^ztU!kh4LIb2ZbZhYn2^?f{taQN zsW$pGpHSWG1@Og6FUJr*0`O^o0cJGT009;R?BlAIx6+u_x$JHCJ?Fkqm;JzmJNBe^ z6@Xqg?4l_|rYEC*?gq4mKFH*(_2FfPVh_MjgJCbj5ht5rxk5I!%g)3=(jFBtB!-0Q z;-S{Yq+#-tqz<83+=Xm5d{ir?56r%uNZjB)w)ztG9W*j~UoU{$1I%r9E`L1~7|=BL zL^K?$Ad(!z6JVjwY1>%#0Q~C=@fC>6#wQ&BcWBoh{IpW-GvFu1n`z+YNH+T*Sh?&< zX#VBidu#=fBauK=;{|jdoy=yU4fVrytAH(o7rKZ_5+WYnciIRfqXW_W*PoL)t9h4QVzuJ_2QA-CDXzx%TTYP%gDUcoK;7@h@ zpz^CVAZFAkR2d+19#bb_Y~lKRtJ0NHZa!#(2e%PK5 z?Bj!4q7h`OY*0exz zexKpNzcG8a2{wZS%8U-oZ`Q2My?-kCTsO2 zzA6?y@WWkwEiK!GX-2!S0sw4e8nIn3Jd4->)#azt?eXykUU%fy5ifm<88xPNXx9+? z)9OOt^V|bX);-u^SZgp@X6y0b$}zP==!w8mAU<=irS9cUaiJFkCd7n#{v8n3adSic!kGyI$qa56 zh6QK4Gb$*MX$15eD%~u)9vO%vj|i+`;*8O6Bkzp@vwjTX{P#?Jr`ooo0F`PqQ2a-4 z-;FUuGPXVM_RZCFnJ8Pnx9ggQVam3m7WCbXb>G5^*)kLHAK{OG=6<|>CVRKm9&S9*HZKy z?jFw?G0AjSSrCE&1Ujo8#(6qB8%z9QnJ()_}|_D3tRuGh%Wefw~? zxcKxY18Qyf;d=Uv3F9;bh3;n&dY0?xeh%^^{kV_)ojvn*1B&74cP&j$$8=JA8f~pF z>SK_wc()OB8u0tggDxBM{T7F(PsGeE1qR@SMIp5|r|W#p0?p_SN{l~2V)YqkK9rA- zqLT3YH$kHk(hd@8j%_bZPWZ_qlSTTPhX8U^y*m{v^J=W{sLy%zLWmK}w&~VgU{G5b zC}}`v?)W{~VSpS$@CVQvmxb$Ode$Q-*hL9;Xp9}yM|#A!+0_KIIW@G>G(pXv#Q4Ua zXGMwZh6bHY7`fMAn0Zc7H|qclSI{4{pI;LFxdWs3(ILZb@@bKYIStE2Ub>1~f5Uo% zFR_F4W=FS;ljsJeRoP@zGWj<-AB|u~5cxirudmoDCpmhaqWT6!FOBd_-bI9e;4l%P zS%e=|ybxT6pe_0En~LC8UaXearI_Ow|JK{F`rZQ|$4Vs>gAY<}We5C;kM`O%&6Rkv zukIZxWJa=9qXV@>Q&);j#vo-{F%Hk%QR;VD_W*N!JoiKh%5kFLGND-rwQWf zH!_R;V^eZvgI#iZSn4X{5OUP+tF`ouyN@>L=SAOh>GVOT=SS$$m2%qN?>J{teDJEd zY62Y}TN7XhN=&#n=H+4y)#VuUKHVpAlaiMV&l!YN-kdURdqLa)P40U@Gu@|WEdB>u z^2p#*Z_GnP2&!DSWZ|CromI~>Bpz7uk&ID`O1I5cVea_fRF=)jsLRSTA$g!ihI80= zWl!{?pkxy_<6nqS-zYs&2d2^m`nC!J z2u^=W!Zt>KYwGt~7F+4|Q88z1b0dtt*2@Gh&lYaJb%i8Q7Xf>LfYR{p_^C8Ei^7y zk#Ra(2)s#sy+WCuu4e%9NJhsitd0LE2U~9%A%U5PJ2!J)Mgujve!t{)>;E8iC|~je zN;0bF9Wv%;{#7Akto>64UnUdT*Zqs9s>{*xr#AV4eVnKg72nK3=Tjxmn{!RaUUu+b zE^JHenS>!HZJQ-?NRq(Tw|O=o(Z0#CWcoQJt|)Wa z%uS^6^tEI6YeHnTbUjeJoUZ6;0#jo0Nj{6m^kH-zQSWMeef(|J09-qx1MktcId-Vo zs6&6z%tx}s3qs@x+xZg%(@Q*C@wsBY8>}ZUtM#FN`fLln z?}Z^}!;^@uL)2fJ5M{*)l6w$D@Ma^! zrrUsuVj|5g+RgG7X{YhMCn(yw16Mo~3N$5C6g1#ZreL4Y(cR!drlkgkiSmkxIAUUQ zw|JahC5@|np1BHS7Hiz@=ke;j)Zr?P$nbU9euh;X>zT%N9>ErH-t=<&^c#4zKhb+C z$SYQB-1ag4G9BBiQ@Pt<#vQ@uaD(`=sg4jY#uu>MQDScF+9;|yeJ+(d*GtYh=QiQ_g&nN7i6Dz$@( z1Wy`l9?7tp zlRv6mT1`%ZogJAyoO9956|##hjbc?$$$tOs-WdKhGj%|$!~v#-3JoRQdx8u~n|!i{ z{p7LCY_T0bERx?rHjLp_KWe$L`P5=q6}q&)jshLp~p;t-urNU()0}xj39$S`@ zse@SM`^~Iw3N#3};0*)r=1Ht!hm+tuJ@$=}6DRLuX---%?9wmLDRgJwv!E-V+r2c=g3wH8+Nm4IXkc$On+ zaEj|XxB?ocDE9DAgLnf5KEzJ_nmOg@g=k&#;3O>*cQU)m zVS&PaA!CEbg1CaXqBOB=zWH=~JHgtv9e9>_A(diODb=JTq4SdY*IY-cXWox8MMF|8 zS6Tg>{ez4~z>TD`&0rgE4|mPKii81MVJtsmqN(b8viE+_V=lkiw`e~Qh!Euk$S9)dg;z-fherwT6 zKkFqsBa>4o`Y-=93Qa5*Xw&j?T^gf(5ZvgWHF_^!n4oTM{?6)P%MGsnCV9%(b-%iC z&-Rj!{e!`qa&y5U_~bw#M!<;IOJOA?^_ILMQ-qEE%ml7^5#;qLOa(Vi27Jye`!zC* zfnxoLXYtbT`Pb#IhGN(R!_8a|9KA}TlR?}yk4FDcefo$q$hVd9J7>_Lj8&P+lK|%^ znwALPOY*t}=K#b2rbyA063AHzGeU}_$63mnJzW#yt^~idVCk3S9*s&sP88`cZO=+^ zoi*J;wIwQm^#>=m%EVXvAz)6jp3TnUUXf;OK#v3onK9t-VQXXW9cajqEauD!3o9Lc z%LU&(Cot0fg2qu`!zsm2j=xL>*QaL&suYOxZ{<9T{fHnjbI=aUy> z-4(GnF!oSR{{aT(z{x><@$JhH#nA;|TqcwjP!8PWf9_fjt!-)xWY;vRMN6v7is26R z1T-%jj@?=qAiH>}y!iqgGQ*VhR8#8$OZqqnF|y~n@Sv`LQ!_a3H9b(}x?8J@ZOTpW^jZwIfpAz|$` zvfG@7iAF{N?tQ*Gy0Q&!Z^v!(q?zi_zR&y%Rkl|gs z6UYD(@ve!~gI{!h8h?;hN?)r!edGwi)*N;Tkw zZ(DH9$ z@)A2l-7Zs68%U{VDL3Tq-l{@~M_f2kS{zwX4W~cW9Sv)u9u~8RoZ27w$my@80F#SY zDsZlAz2=Rt##g#Xc*$Z&zerA;w#vq@1M7DiH`r$-MDerPf^w!F=V?rgb6RWM)ZNgV z*s>SZZrVImIKi_YE1<)bpHNCzty>^iEj_s2U+T7Lvmi>EBH|T|)XzR{?W zR+%xFl2HwqVP1`&la*Xr$FWI2uC^tw<`t`vY5ZI~Gn>0-c{QGRuWyU$-TwyNRgj+C zha~VDE*Wq8w&@sHC~iR?8RcSJ6ZguEZ)JG0iFXd?c8~=dsYA$*mqLuQMP2bt)>!r|OrGDhh z^q6R3`~X|?wnKUkM3;1OUDfdBt5@;WvF&L#yVx~+sy_lxMOTFVyi25_wg=wVuj?#Mxpy5!0*Yj@)OnM(5P|y||Eaa%-(4$LxArxKP86B=eTi(ivTniD zkCl4Ad6XzyR7L00lpMdK@J5Pgymh|_BU$#uH@d``h_MmxdBp57dGa)C$kp@25D71) zMen)xvjT)u!i(EjUlcm3#~5ojyBP$V-egOh^?um7I9?VVQe}D1s-$|r;On6G4 z{Eb2W|2gE}-w?3_-*l$BcIo}+HyGo+6*;omB?*zl1>(k7c+G8@7LPNPJ>sFY0uX_mWeRHmqL_+ zTb-AIO(PLcTZcsS>*lgM7eS*;&8?Q(_tM%k@PX>jin5foygxn&kPN;`tu9dJ6OMt- z)-cp}1}O7=mOr1Zc3RcCT~*&YPWPKv;7gy`1eKeN#!evog=Urjs2+ ztvR?kjqHAIXLQE<5O%~(88#C>9DBc59ZjLnILp^`YQ3c?=4N!Yl|=DH^!QQ;{)BiT z{%&;Sm6_eIuj1BuV9uqZ0a;(2^`nPT%1w0N{! zb=pbc!~BuQWS3Ff#7QB-`>lxF5dD8n6LLn0C`5rdcT0+PDn4#gueQ3S9B#r!arTwr z{GY{+gM{Ea2Hr4Bmz#YL*F2v~_`Cc>o%IFJh2L8>g}VvfTfal{zbkB|6YD3! z@?UWG6fEK0@t(TQNVBME_8f2A`hmEYk=$xJ_z{PBZr4N|e_goyc3&O^^H2{aK-nN_ zJ^-YN3&(jKte5ne4Ltg8z;!3T_;a!M!sgGV(Hp^4MxVW6tNY{xpKJELmt@K4#lC1& zqRI0ogJrS}N%_$Y#~7D)nVnh4L)_^0N9Q7+2s-iZ?E;-!>hBM(PK1bU6p}K|SHaWA zXR4If9el^)t+{+9@>(}TjZ;^PPsOT?d;4!-w5tD@KIM$MP#E?yEp|I+MM5i}H1qV* z@`yUa=E2AV{d1e>sdW#Tq<%- zLF`1Q&Gos6tyez>BDTq4^qtfBoQrP8+#sx{9Z8JQ&O)MFD?XM~iC+BX&-FuwQRYQX0g&3|2n=e|@pv||1GX))a9ryJV zl^*U(=ZLr}yRsuLzvVkxABdvMeYcN;qM*@&=GNH8F5eo&=YOmY|N7Xni@_R7a>^9} z%^9S8Tbe*Ok#IuIjkqo`Br%`y$Wt}&*1ei9+)HS@TwSX!q~!mM5GK{TF&p5wF59v0 ziH_viOm6nso4$^yWRjh*xYtUquMBQryFW|tfbW%~HlbIJ+xA}Rb5`Dm4XH5q=$d3@ zsj5|^JocH>EmbJ&ysXS9cXhvvz}fWp2u4_^y5uiesV3{aZ+UYc2fVju(V9zog)Rea zcJgPsYu1CfzCVc;woz7T-zp?aUa{ZBw^L*lxfstB5Kdj<%RH_A=la(VZShwdlb2NQ zpNT(pNC4k>7|jH11u^c&4gBGRERbQIjAzGZUVeSLCDsnEhgE-`NmOp>vx9;v(Dtfz z>-Gk1Py4ohH=i}@l$#Bl`(5uR&c7&FaK79nM#vdn!1H3;FuF_aG{@A}#cnp@5|NW< z&M~YfIW_Jt7@HF&;mwF#zP!mhSnC#ArpRsNS+$AUO|1a|L>);XOz>sAPO zd7YoH(Cmj$Rxs#aebXJ?w@xhF5O#9B6~w&mGjJ1kUy!v^D#ISV6!_pR79P#`-+9Y_ z57M6zEvqL8Bx?T(s3dgEVzOHu4oQ4Ep&cFd?Ruf(!>+2&-C45nLcZm(z%QB-GmG~m zPikOVWwl~uL!#f$FnIQF4|=W(wE{6Nj#d9K-kyqm9;`1)vRL1_7`Q;0Cb971@mhAO zS6T~0l--PVk=IOx`+dP}GmzZyuiyRh}9)dn-~b zf`;rr3P^}{xkIG?xfZR5wD^BjXO}M-Dsi&aqi?)uKa|c+tE5OnP_6b((ILZqW2`=Q@< z{>VbnG7j{en=qm;#VY@VgiHw-(thefnThcmq8PO3zF3>id6ho*(}4=TGS(JN>pvT?JV!;EaYI$J8u=$e49`+@M_E3l)_E0` zo8G#LISifb^NCdrM3k4NyH2dY>rHGgAe6P#LwDSx_ZnjE_U`J~TuO|f^X9`VT6pbY zEe|Tye%R?%e?=yF<`PQGP9vwD*FL+E&8 z*8Hep=s(W{q|>v{Yp(a}M*$zz(TlL^=l|niY=*2H(H99g|6G^Fk&acx zrM>P*Nd5ml9S4x1w-$l_njY|yeUO;9kB9y|`{!~djuZei0aqqp>K{D_T=xyWntG;B zSMWbF1K^S2g%Z#N4rcPdA0O5;eZ#MAxOq|jKAjQJgaGv|wSNW&9mhS&yRnVTa_awa z`Tzci3N(Q-QX%A@vjCOfB_q&3_mNK}{ytqY&;+A*g7JSn`}B(e{c~wygZ}sFT7f2@ z<-)W7?$iGxZqYC1Z2)EHqY%DW3(D7s>kSA7P)r0uGLIC>ETxZ zDgX!SAi@c?^kP3ZEXfeM^u_(%XGAD!!y{!Ql_S+7^)m#gW}0%`WNgOl%|KZp@T9(L zKH!MOU_0L+V7t&HB4i86+X`y_dz|B;BT17O);gL^11_=fSXqjci88~?_dh-)06WB@ zb%sX2nYU+Id#?a`t)dMIFZDP03Hke;N6J15B40}r>z|*A2Yc?m8>dfVRv!WXKlu8=;|1XT3 BQ&s=~ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/6-latencies.png b/website/www/site/static/images/blog/scaling-streaming-workload/6-latencies.png new file mode 100644 index 0000000000000000000000000000000000000000..71707af29021561698d9a738d0a39846c31065ab GIT binary patch literal 38562 zcmeFZWmH{B7dD7{a1HJxxI0`Nf(H-o!QJ&jaCZnAf+e`SCb)*+uEB!4&!PM64n5zW zS!?F!u+Ci^&Z*k9>#1i;$t6rlLFxt4OC$&gh!--_;%^}!ptd0(ATJQ$fF~5T9>0J; z)Gfrslw`!j$dnxI%q*-;At31DTw+CJp_H+MVmQhBs1Wx#QbWnQC?vUS*~woa!7&!) zbYIo5;ulUk<<6Z$-yp&=-XN~!stw})pb4F;dGU@$T8V9OTJy66+CejEh|@B^PL3ig zZ#@o#S})G^LQ7p+<(HuBPZZ(!n4zg9J?z1ia=6}?OFx^Wq%WFZT~A%9wDR`Z>fx>( z&;L$ldbsgY$Wtw2-6d5`h(yLikVhwkXG-yw7GR*mJ`OV(+;Zxd$llY>n)FV-`sg02 zHiUT$?NBw7udS<7d#ObmC|91GSqpVerv{~!#~LBv=&Tap*Rm7shDRiYJ_8Oy(s|z( z+hzD!t_1-e<)<{}Uh1be{lt$;x;ez<5Ke_>lW%iyB*Ad|Y{@7!l}Oz_uw^e7uz!t8 ziZSNbWsdu(TbQj3pMInSU17dQ9`zDFZMlFQdz7yB%D(9|*n59UY6t+WZmKC`CNB^1 z8n{P*fDEyKfC26xfzM0e0|5aY9}EEx{Kf=6;yFDnu4DegU z*wNI~*6F>S^Bet4RzTF0g{r2rro0@Vv7HT*p^2T5DU-X6{ZkbP0e3#&*2dJ?kj&l2 z+SZBBU6A6h5`4h@(_>}|vcHNrTM1HV$}5qH**Th$aWSzlu}}yhk&%%JIGUL8y%m@I zUv=OwL5laz&h~uF%x-ROOm6H`AVnk)4aPAO*!!L;wE#t*5EG#h;dJo&MJ?po7d$ZWAYPO}m$S+PEVVWj<^U1ZoWUqflT-p7?_R*Lf6<|d#_%lt>bN0_f_#Wd%;lL<8PAEo!_YHc7Vok^TS zPt(}V-wY-*+E9s`J#QVjrz3*I?=~dL=65Trv6-I=Kl=d6W;c)Kcs~Z(oi5jJc*$b^ zABD-luVIpz^_c)^3F@WqWV+yq2Z(>U!2RC}86bi}#9}C^VEBgioUIV+C^cU`Lkt4Z zAHumI8d4s?IhXty8c+HN0f4MsLRvjT13(553VRz7TjxK8!E0DKAZB0E3b^OcQN@yh zz*(S9od0O_|BL%y8q*tg|()gV(415J&`9X&M_2@6SZpAm#~e8_35l#A{g) zP8MC~habN1NNHD~cIuVB<9wAY9WuEwrEMqsdPG?f!R@>99$y`6Ea(d+zL0BWhS$D$ zTD?J3jkUULS@WUW^{5s5>6A|B17%25<(jZvnpxZr)B4BPl)a3VuZF3mcc)7Y%Otc+ zRQ;*krHeEQw=qj07-h5a^~@7`>B)FN*M|z-1!{VUUxw-BxXgp2krc0cjJjht zLFQ(+o7vzl=bq{AkQ&>=v?*IxsQVpxi56CZog{PoqYs}VE`y%-muumN7P+M>_cFP( z#&KKyD(g(|yM3rMPPbZbqvVK`o^|8vhm_O(Hcl(!q?0jHtLg5I!MKQx%+alWQ4QXg z@Pj+m>a%Q8Q85vA)=ty6KM!#S@+{~#=2;_5PtKCwF4CPU5bwSL{G-Y2iOa=)r_QN= z0N1Kb0Q;SgO@7P8Dv`*%?~PBR9Q$VDH*-qdfdC0BYhB?xPq1m8gHeG8majHVxgU=P zjl#RKbb7|UsWK+wxd|ETz@SI^Q^|Fwgz}T?kqMt)8Ixf=#=h??=i_J4KP5GI_3!mq zF1Eg}4c3;uJV>7xF2Um$AL%v){PWvtvlGj5t!hcj$*)O?R!#TPp*(cZ>VoAfNg>l* zZgV=jZMKVir*oyj{Da@^%3Bn9RzH0Ay$sjzj>2NsKub&sukJa2M{$31zE?cq)*X&B z1%;@tM^5}r)kV@*?>2!Ibo*7KJgX+A_2lGhwXK-r_Y!jK?`zM%qJLNwY zyAdPys3joQT#v_H*m9;N*HIU=Z5!=XoOJQ7FG3_vX6V^X5-wa!+aIkAC2l>q?-+(Z zP9(yeTWWI0_1f)Pl>R^y;vR0WU`Qsu;ZCAIxKcx1EUvvI3Y{~Yexw>spNNm%-|h2JR7{#@+F@Gc`tsWo+R*zF zOQl46YZD#X84}tb)@f%w_6mMMnubMwtPoe=t7)~+rZ;!l z{1rkY8-y|A!(~U8?Oor2be_%>TEA09TQ~#nPh4+Ff!4}0oM%THcv1FyAi#0uV7%R& zQmf7=Gs9)qw;`Wa#~VB!=omq+cx!wgXFm#H>9l$ zt!FAe)mZ$Lv~AqP1+$1WoR09@d;}b|ZpGtHVfK_s*d~S)5&2B}- zMxd;i?;11hut@CwmJIe``FI_5!Vf+^TneXiKRB^# zHrRjaDE(?Wn0&DhwcFslGcJCoDu5M7M1Q(9S|6#*6`eN7+`d4Pf<+@&VcPLR`dfp; z`tLgCwkuI?>f|D59kV8vJ+pQH+skD?yGvt29Sh&bd$;DpBn`dFd6i>H7~%W0-ImIr zjVsdc4eUicAri!B?C85AJXcZeIRV=Kx39R>fh6S>;29+O*}t|6Z-ZlINQrx z>iQ7gDNo|(JI94i8yl5M>&ccn0fP^BBF**j?O8Qp3!cYjC(V`C&5Rp^QPPX%g}9FA zjk>0Nam10Dwx*>Uwr2GL0$OE!g~b`Z&Orxmiu#@s#GF!?G{V`=&%!|wcw(wn&vQ(K zY=Fk2NT2-f_w?%sR0`xOyIzW#oo)Yc+^?M;z|qhlx6c;wHwcamuIT0pg?7_yaj(3) zU4Bd!351(%=p}NByjv%IjO``$TaY{-Fdom7IVlvr?-%0epx&yT)~jAar_lLEV4gv4 z*LGc?<9C040@wf|!zZu(+7t#Y%y$3#`$NAJ+HCK)w(a+>*8qX@6f*f7w?Bo@kO+~* z|M1<2X#y!xDt>U*A-e%gC{L}@^jkn!2m(U&NG88FWHCkuR@)=`271W6b#;`$`GiUm zObOt<*T~IxC%&9qZ)I!8ou7^hYuCemZ#|#V-dqhKt?OU%++%D6@}=LNO&5V0U-=%l zgdgJ~nA>KA@FNM>C2uYd?Xo)ngVIiKPk=1>zO8TXb-Q1`38-G{6L5yG<31-{_Bd8w zakX~!lFcO4hvmD&2k>Vg^i-R^$2=omIW~6yP0O5RoT*sw&n#0thR$C z)BLf8gM(9-;d3$DTOiv_h%tU{-gHW%+(@J@biI-A<_q~i>xi&9Rc!%(J&uuJk}$6= z{s>`c^PQ^2qQiAMOs+Tt1(B#@pwD3SF8b`t9s)PKhM2o<&H7{b!9(tzf#{0-N|N@o z1ES{pCG!I61uc$JrZ{*wX^MLmtTGozeZZG+OlfiZ98l&>-+UN;KhhC(>{mfg&f1+Qatb3Q{CK*-rK?|P_n zv=RW}mPC@y6?^EXLxClyZKx*ofZ&c*IfC$8pT?`XiEKnMAz4e7ZwfgbF?3%LS)b zi~8u7i7E<}$K%k%Vt2lWKZd5~ zhAP<%?z|x_$ZfxJk~kUh^jTER;AOa5_x-Ydn|A+@7m8)xb^&MqIyiV;m&NSdViC) zw;6jazUab+A`m_KAU21zpvBa-X)dpBfvAj#7w4o|P2#xte5V zW8x}utMV5D&&O%Z3~K&3wugQ^9SnB82!A_a!=jnx_i&AP>@#muul3@?8~F_UcDQxV z;^_q=b=+hh0Rgg9^I-AJ2(`iib5NyaT!BugGyP-LvVt|3Qy#v1nTVr@(H1I1Nmdh+aKnTP;j4r^Ozp(!CSZxmS%wgKk%B5u^i}u>>puxc)-KckHoAdk-yv#Kw6d@ke^7I~ zr)lM~B9prR5(Uc1cARy;b%n5%Hbk;q@uv& z=KucfDsm?hJ?D3P9aVdkUMzwIy>hsc=I(@p)8>gzm;SZSORpU{QDJzY zn5fXR5$v^(k&3vxQ2Bow_(X7`?xODTbv15fS(WZFx8GH{fc+;4g{JKbrF$h#Ppid! zFfxp06#2|Ed@dKKnqyvny!?2-{D@ex?S2|)OuTlx{W(UVL6bL^ zMqT9?n8yV1*v(7PKPSB0(<^|&%SL~|IT?Z9}??2X>UfnG!5WsUcvItu*GAT6Pn zv|FF{zWH&?{rSMz3!n5Lo3=c5o%u$YY;4~^b{!fOQfMO-)$t(;AM|bzVNP<2Ee0~i zo_N-(7gtgkZON#dykxlRzH?f70_TevWcg)lu6Q)+C|O^|4%xdozt) znQ``nfsecl`{f@YlQ|4QNL|TQ13ppMv)4bnewhN_vBYe zQ4Z3k?R$N2umCR$>bQ?wIR*K!h!A{JSP*AwKh^l6TikSh-s?Yu4d;B@0}&_@!|PXq zamopE@6eZZ91`tXx;w*7>~4UdqBM>3o)f`2)Je1Om{3GlrvmkdZzt^W{*`VC1gX2` z2!`k?nw@yj3p0q1WpGU#>MN-jtVshLzTa3WQ;m8gx@@@*NLWt?gfV(Ub5f5vqX{65 z>odJi`dKLxjU`K{*LXA{nLUP3#+_}yR@pL~va+vBsAgX1{!Ha|zF~i6d4N{y7UPv}X0!7S zCcC;{diI04&Ksj1OB+8vf~T2Ao%ecinKC`&&*Kv8*}D!_rwL^#wK?nWenSm()OFx` zhpn*vw%Y?%V|hhux3U#?uH^Gn4+wc{E;~qT-C>ClTXrTDR%2bo-sR%&C%e>|T-#EXi$XEhFT za{4iC*SI-_|A2n>?rZz7yP%g%H>TfAG__cU|L*bbuL7+J|LH5qhFNf?J9P{=eg^Im zySAu0tCdrfX>>~K(jbCB0d_?DDE|mJIQ6b>mu&(I>Fl*6^{z2#h1&S68PR{K9QuT-m`0#M~N6GdADt0Lf`CRgz9=qvU*; zoCtOE*vfP(DTalU>I#An)Los9zA!?kY+LxUuh2_v(Vg(yzZgNsLP@`&)j+8b<50#( zV+jRt|DRVW(PUZ1JL98spw(8}zz~CcyIE!TT#g?f$1Wd%biR}x^9BlrsWk9_BBRrfH#-$}{$)dX$ zRJ#k&JU$3T(mgP5oq+}7r7aKsOc*!2uY2L%sCOo}p3S&TMX;CFcUD{oKRw8)^BhsO zcBzI6s#9gfWSSzrYG8gC4k~@;pn)yCImu#6e>E1vZ@~Z1A_{H%V(I8)>L7R!iQ(`I zRLO`h6vOaHOF}+4UAF4RBfoQ6ZM4-kx5biSJ!R=4qJ|syx2nEY?SVk>XN@tDBE8#| zI`9j)49kuo#_6|G@gnG@}DurQl! z^<8xc1tpT;GDH00S0}j{9A;1COgI5vc&dYLSeoK@+KX>ZksNern;#*;tn9_IM(rL3 zVQJtLF2<$Tsk7EbX6PnHey^Dn_E<6VswLAya#KCft6Vrkq=)2s-!H4CS|DO=@@iz@ zI$Lv#@&u1`&X774ZJ*3Ch zjBMe4LDHH5ma4=4i>z)lS^MT!TA%Z`@ONVl3pttFVlC&a)!nM$mUnZ;JOwJSEyEr2 zNE=|`mR66|AUsT&BV38uUF}%smisgFufS}{J(^xm-I<8c)7wAu!BBe)aw^><+40i# zyXn)Kyd9MAV;jT8cyqOp@Iw{Ht5A*QY-0B6A*#jf7-C9a9U`yI*4ZpOrjt0oI**m6 zv;m5PrE*+Xh6n;Bp|ayEdNW9MNPa20+tsPCBdnylv@5cFL6yM%>+Li z2s4c8;i`_p5ONtoyuO=ZO(OqQ*W<5rcm1Mb#rVPg5~L!JmhGweHlg-~3PcxFWSV{p zRLR&;w!$yF%;*S>tp2*r_{f_sQsUFtZxoT>Y+qi#jG*~)3O@Yc$N$$5Ya(Qg-s8h(-`>lJ)(drRQ}=OUa`=Ts6WZFdkV9(6Z;@G>ADP!0Vi!}f=L&eLxr%4 zLz%}OrppoLyQXEUu#jMrHKJqkGL_n4P5Fdyr@RmYC!}}QPQlJP84d}h#tIJJGYC#; z%-VHBP17JbW4#MG(Xa4y8_>rGwz5qZh4iEWljS^8dhQ@;nvqIjl`MM+2U2$Q`P2dQ z>Ux8*l57a3{_niwpQ98UqHX%!&BwE^dF+%@Odz3%>mce8LRd81@JjF~SI+5~zJJlt zI6G%Bw|ymvRw;)`27XK7zxN7;Wx?VY-VOh5r-V6wVoGQ3Eqls$Iqi2aWZzf$+ko`} z?RE$6_q`}3a&mi`?@AfHtys!=m%KYhi5UV>4S$c4-4C*FMy10&7$}E4HjYe)Drdct z>7fO7+rSer2H48kPCCYuU!zqUPysDoSfrXcASOk~vhs-!e$a+wIJF%tJ76xPX2`Uk z^&dnPyMSRaAYc-8_j)_j@xVailPv$&X5~tV|GTjW>aqI?)=-|^HBm4Lj7_r0pM6YV zW6lDh2!xh|=J}D&iTgiWfQbUYPG9_?5m)T9+YfMkr36YlrT|OODuYX#A=#xvcSd4Qg_v$_6zJ|Lng4K5lQSGy#~BN}7$PQCdxx z%6Bx}W_fMi?3E=~j|k`{GXfhc?Z*WrI<;@gT6QgYFRrjQSEWd5+p!Qun*dggM;q2Y z-w$s=oG8&!fy;Kz{av#j9fp96%aB+gpN?=r?z}xF@{r0VA#}AOwlh=tS>XDr-Sqw@ z)mwP&R{Uc{*;qGqiQqTSxM!TXqN(uicKlK+l0m8bS@K1`WYo*?#-B8fkAY}!?r&af zjESKot21ifj8to4Kx1sXcUJ(5$wswtn|~9}1hi6wTre{;x;V!gs2cg^iT&4<5bX|E zvF4a_^;Wd9M4nEoT|wjdD&{+r#VHG}gN!E^^M`#aD#3k8e$6^{Bs?q$CD1X1p&Q0mtf!4>*?;yehxX?Z_b}6*4Hm< z`t@?Q#)>{NchA18FQx06ubzkVnC7=F~QrEO|v1GaLBz-7#kk4>vKlbMfgCr}j}(b9SCLhSea zJOmG7WCous$?XOFR)JFa^cR)Fa2~0kZ%iYZO0UK;IT{rH<47cEKw!NMRDE}O$B5Q}Cn>Zb0 zSF^?5{<3XP`pox0Ai~H2e#T&hA^Mq{@`rhgkpohc2)X?u1UQL7W&j*hfP^-8_@g2^ zHPq+A(W=Qw)5Kkp$)>kCQng!}cK>Mg?n9!dB^E#lEgQv9nD_=GaYcPp@d&zP{h zeK1D6g0TREmdq_n`EQA*4kiLFU1e=m;XlFy2gAO zw`;DpP{u+KGle(q-;w99;~|E$IrsSBO0P2BY0Klbk5;P^AX$3w3#zs%bXE^y; z$0cDT6&88uYV;-cBFEVrFM#%|&3hC743yFdOh(tjs~Oq0RS9b{SWk@A>i5rtLSotj zVbcw3^*P%B5>Y(aMwj`IKY{=(R{-NWpy^tE{xJ99*FF83KIUY;Ep2WWz$|7W?O$^L zu}K1;g;9J0tmjr83t9!bh!@T1^qjk7;7CNJ9P`LX#An<*LyY{$*L+4uFpm2+C#1$$ zlkt!KbX<_c_+QUE?2z2@l^dvGJy&Pg{(BUuFBKTcp1#!QN1*&+kfi*fTJ5dV*q?z8 zfoSq%#KtCa&rTKPD;k`gX^q$W+)+x5Od0Wu83GF8B{Mrca~kpFE7UWx1rFqv2(X@^ ze1U5JB^Z)33G{QssL&~ZAt&EOQ4ji$v!EOEKzR<4DmjbeF2jz-kEU_gyB^ozE=&Mi z2K`_Eb$A2-XcpDiHx~cU78>{LvC!3(9Bylv1L5R{-hED9*g`*{iotj`xjLwezhC z&99LfK*gUFMljDMQiFg@hi)PM{~2EZ(Lt1enex;PrTrOQ3K@fqi3y?YIBV4pujRoc zpGJ(-{`7^64ho2iUbsQ~*hg9KPwL{J1nub3F#bc$4F!OjZoxzUJwSLO2|WFOQua-- z|4qte`AUj-#N-!Xj44JPpS2$VquNj$8)JYas&9&PHs*#wFa6Abl@tI}2EWwLs{etd zFQ6lkSw3!cJ;fw0UZx6gOj$2D3;ysPj7iO6x%>MKV<3y)%BDq9-LcgFNl9=YhLZ59 z39!&4ti@-037F&}DQ(ZQz)?kYbl(t^wePAuCrIgFtp4QOx_ZR_44;x7Og!TGNiMpa z)vK|<)uFe4+y+394m?OCXZ=+Cus;Ja5CU4hAQXPfVRZL)-d;BuEoiMa5vtYww-(zU zxIaX9Z0}TiPxjjh=%=y1r3Pr0WPb$zr@&J-b95YGVh}?!BEMEsp)uD<>hXjDUgXe+ zKaIUf>F_6cF&q+(n`8?uHpPZ?7VCmgEcORlL&^%Z(E`Kbw=&ZqP_22*DN%o{2t0)F zl=s}I)zpO@;V1{M0DcF4G=ut?@6SU(`Te7BN&}W3M&N^hnBkK{4W@UZ{x`aS`>;BC z!xo^$DVK{7H~^|ZWqyDCM*sw3A{t;UCp%VkiGQ?aqwy!uL4PkZimy9`2mmhG`{5bj z?_eg%Tl^&MWOEP7L%KG*2{_uE2&bKBfzI=P8{5o*<`LCkRS*3~Tn-gNkw0@-&xZ;O zz~%i;)G6jTYa(?T4%qoaznt;ju$A4lx*{y)zq?@d!zmx9bz8Rs>7N)XgT5ja9&p(6 zu@kQ!6Io`ZGo=8V&Fj~{L0+i~Mp8bLIBhD^r z3&uh;Vv5PHYO;xU$GzEo35tS@F9^93fq4a$w(Tr!CG#74XNR!2cGpFF04J;&aCU{_ zS)Sz$CK5TCxR?D2w1Y*;pEodPeyrBBr!qqi$fhjU3< z(;`2ew74S&j_9~L-oN1Pui6~kEFgR&2hIYv37jsRo_zPqs+u}8@%Z53MFu5^v%>N% z?A>CPr+Ivb=TeP_%JpgrK@T_mHJG@zuCfhamk4Gk8rK#lmGm!$2$(LBl@#WlzNt+9 zcQ1ZabXv~(4#F`Bvi5IB{xZ|1{G!`j2b?<|s})Xl6}TUAsxsM{sLUV|1PE^^;HR#1s& z;%54uodiA-fmokqx%8+StM?%!5G2;a>_q!6u;MS=Kf84&(G!(TCY!>P@JUP|si|i_ z-)#6L+gW6Fo<+4;)$o+I)Z2_eZdlK%&rAXHJ6g$2-f*=mGF-_`NMc) zjHRu5OEA5qD&(1d4N8^L%J@u-y*-zXp|>lLuW*UW5u(B<*`T8J&s>KN3LK|GulRfQ z97lOKqRR~h$dL%}T42g8V&7gz;0y6BddoilyCcXO)lTS=u{xF82=Mg!10tTn?l7=8-$+J|4ZfIlAWq{ za99(Nq5rV}AO|PUIe2|ffAvXBxCMy-aGSBt(0%iQt+~z80+8%~@+VLMOa`>5O1_$5 z_G{Ej)2BemuJsbrny-D+*!cKCh>5EM;P%#WKdSaG zXe)J+hp2rh$SOhjqe85|Fc1i z*qNjeeHXNA3`RC`sT%%8L|W2dVq$j9SzCrPpQ5*c`g3abJ^H`g5ZxU$Q0y32dT7uV9Yz-E>0{$2F$WCC+hP_^k=~s#5#o2jF zzXaxT@zH&GFFn`tKDKtJkt$Nqz9{=QQ``{kI6RAJOjJIwcr`}p4wHA&l+w&A5VjzG zmHrRo$Oyn~`}0S`UTi#&wuK-xbl0}<0Evb2qaKJDRm#d!HQHbQ1~hb=zmh2qvPn6{ zgU-(*&S5AG@mN5F$vDEHWreA~-JnyD*L?jCkRV=6rFBw<1C;Bh=;2oYJFHx&A;)R| zO$#!r(06ad=g;tvL2`rTmL{5ip1An}QEn^5IHxf$U5|Bapn~OJjH;+W>-wurnNNut za(wkNFw8+tUnQRR*c}6(m-8U7?0{a+p+bIngm!*(zwv2yl4)qbTjUk-R0n5WHR3R+8RxIS?^l^`xy}b8g;HPiyyKB(sDCI`Y3EH$jAW#?HTSK9@oLIIE5a{xLRe2<9$W!D;?TG#FWFA| zUa04}%>NQS^_xie&9=2 z`?olqHb6U!ZO=?S)|9VnGAZS`j^uqc|H}%LZeZ?wB@(feurD;9CMe4-n_0tW;Uiu5 zBj&qu-~RDKWZ)CBrSPk;aZ>$Tw(4niFzY=AT`p+(>TJLK>m~tmAZIXkw_2frFgGmq02*T|_R zfmVdJ@yWsq?YHPFzfP{JTUdO%01Y>%M=86!18^avo^S!Or5W1yfsU-9k5nN9MA7e{(C_LMxtZKJWD1G znv}pO%H9$@O2xP?83D08gjd!Mg|q#-?;Nq&e)x4%#_7IaS)fpGjQ-n~{xe8_88A2i zqa*#GA;#mEL)AFT*}0FH zbDz|0YdyD$Ey6ISuLokU|Bf7qz!#k+GOFuLeu7#Udj(0^o0am+#aQkvoUOIn!>6FY zqp`wJU?k^4vO~`s5+b^fxuol-N!&6Rl()KuA1&j`g{u#x3ZC3NSQMk<{#|dx=G)+K z=b_(S=r5qWok*j7ku-Dc?oz>A*@}~Ia!=TnAGSVv&3F$U-6Z@ywIkDkIpO{JF`9pU zh-r1#c#38L2EY#V30k4*+zOc+IGm39uoZ-G!}k^4!#tgqbi9hX*ZGMPv3^jV^6G|} zF_=8nvO;ul2~sp=wJ+Vi^LT?3$+Kagu(mzB_=9-i!#&;dZ?dypHKh&_JZ8R{Zjcw3 zROfifPHm_DwSJ2v*}?lsObB^(i0L{nZMkK_&wDZ6^LjY7(Nvy{1)OEw5%!Z=a8Er^ zvrF_Qd+(=hYAnvK8F~R%71NU%z*6Q2BJqiu-N^5F=J?gYLjB&ly0@>-zG-~ho}^;| zqjDTT(&Ez%tjOoYwyhanU&RS!?2Mq$02t&w(a_sn{agV%hYxP%Z3Vj`tGQIT(` zcT~p{&ciRvKm=xv#lCTJ_dh0uTQm4Y=gZnwgU*DR^IY`iSKYqYDZ8UwQTMq2Wm-H? zo{UqsPULSK+vS#(XDkM})Zor%l>{E|#=$OV+ESoKIfgS5Z; zl?GBu-Z%M}<7jh5{~l_##c-5Her!M7$O8~kh!0Iw&w{cWfS!cJ z9AGk?DU{_8RLcS!RBXn~&#zGIHf2N1TBsor&-AetNW3k=QCbgrKS*yCm%+S*B!sE8 zw_8YftYwjl*l0uC9wvV*HI0619|X07kmn>NpCXiS@9mX^SlZP~Ea;P5!cF=LNz@O! zjWb_alnjcvsAtlG96I{`9m_c0+8FOJsZT67iJZt)wv|QqOP5k;@mi9b*n;-PzcVR1 zRIsRb4xN^4hTYqQ?$wKJ^s9s3Zm3UxVS{*9ETl<|o{gsSLk#esh78*gnCi|PK|*~- zqKa!JbEQ_`lWn_ zD7m5T=j!`k1!AlxZzN`g(PJTq=g@kY7N|R`zS>3>Gezls41!D-xebE91_6g`-UKeE zp6cpmo{`J1JIMSM-q4943-=7|^Lu1-4Qbx51-&ZQm$2%e#MDpyD8W7rg7IWK&R2JK zIi%a-RSrB8b|d+!r*6IVAX>6+IHXW-0rz50>?OnXnG^$z&9A>C?jT}+i_erJ%qwR! z{t(v@>Jafwv$SvO6*@R892Mpz!WA&#w}-PK{VY`l!&JTqAtdIf4*XW$!;=;_BuUgPPYH zUzzSjz|p(1F35*Fh<3hlI#u~)tk&aMR(-QV{^E4aDd+u%GNKY7bp`dKBb-1>*uG~O zV!=-FZ^|#xe%s!FW4M!qfUTSJnG5~B-EfgHeSzXhlBL#~aQLbpYF_t8O9%Ht5;ae3 zCL#SuEtC3`0oEKl$*&yAgij+(W&#s%2a2oS;TYnvz;+W%slO7D$I#Ug?Mx;?0+*>f z52ffuf6_dSquo8N+yIWZ(B0^w49i0wLCakn!n|kvp zCET!QvCZ+O)mBDuUZ=J%MP(lO$CMI+Gl$$FGl3e8i?-*>DgG>T3gkx*ML^ZwO0Nk)^X;2xl@PYF`XCw}-@G>HP_w zQalg|!gjiRt;rVoU`Z7n_qvXHEK*{T;++bdoj{r3ZoIYn)P zbUm!W2}oT$2(rv@p?c2NQln;HIorGIblEjF+Aqxh$x#jnNg^u;W#smtUm^-q*X@h&A8spn>*`lNU)Kcz1z* zzSV5jw zGQA16f1ybLiLmAzoM2*c+M1~!KfVR^u#`@3 zm4Wvacns0mXL8D@Q|I>jGLJLn2vt)hM2~x^4#Zg6ZL9F<0~Lwr8KcNo$lN*K3FG;! zsNae{_g`;wE1a9_-)9bPM#a{p$O1-xghlrYK0N+M*qV#5miB;6^N2LJ=S_*Dm8ZxY zq5n2m@}K_&zz2?2UrU3RXgLJ`YnFG)@ePYO4OA zmAbc@XUlZ$P8XmV)$t@_N$$4#6PVWR8hx%S*4nHWRp)BzNh%fhTr}g z?2A)cY`c-1i;0jA(xAX+$nBzet?pdULV7&m8use4M%Pg9*EiyC7J}1tHw0u~9Wn=t zQ1IXUTFSNQhu0X6_zUp9BoMw3;M)QfFfo$$+L|9W9|$iST_W}d`o0iReT*?CPlVD{ ze9OV|M%_7-V+&%eVrVRW2ON*)hEkhVoNZj-Vow5{R=_BEr+UEnF5BCFvgL0utPzVK zP}f~>ZMp9Zlzl0+Xw>`=2*J-dM(TNA~qK6I;{w%tXNCkB6X0cq4X_@P=p^Bvk zqa&X&h1E%PhTG~A(Zr-MP2b4|XP?BHSn(=wG=R8}*GJX_aVxo2KMVhruM1^BU3zD~wuf6=B;1C5WE9IfGRx(RX{ca*$@S z?+d$4)dSA;b^Nw}R#f#DxS#Im2$i51H0(*f;&ax>H8__57D4ZR@g2{bte%_YZq3K# zl*=1EqaaW0ZcnZQ{MV|PWrDs;H8FntlFg0dn5U;^C$Qsh!l3U~ds`)n6JBV@4^ks0 z@1vFNGChlo0!RL1Lk_>6@+Iaba%nfz!=eseL#-_W=xWh%Q;N5!C zOWe{5$H~83$alnTPMCF#6`D|ZpJrPa{vgkXXiQFg58V_eMru;H8kU>xtpVzi9pvMq zW?u)N`32R>vx*Z0?N&gQR)x1X+}8~V8^x5rAs;TLr#yv@;>eS#Q+e@0FtQ*rs7h}|t(WRR7f z=90{RFW8soxJE=R%92FJ+peo-9Oar^iCuUWn+7i<4}U0ER9@n9ttKI|EPhu+Ok^@L zcSBv?*O+(o6(8(_?6`CcIC4r~X0(BC2+T}#90D}gz`DA zVQF<=zRHRDb*T)^NalHOEv(;5jov1AU8jtl2E8;08a;m=H~oayy#v4Tq9qWVbf(Wl zka#!7xf|debKaZXuh7;_yNDh3`i`?JRKSQbjTL9*p+_~~uYgpnOe*@gl7rK`gMJE) zUIlEm@O(|0-W_Z(+}3bt-x;o3)H6~%?K%#PwQ28Q$)8pjP8eK3*lrBo>w(rd`So(umvm<=4_?k!Xabi-GW|eD`RsQmR9RK9F&mG;ZM5q&tpt9 z>4m@vc)!yNpJ;O;jp);fY1a{NT0%4YJ$Bxr+JeF^b55p4M`VRJNv}F(nq*f5+jIfL z#H6776}Qg_i752+s)X^nam7MAfwA`9Qf+Lxe{IXEx$0tIG-h%{SCM(I@mHz3GYXSK zxbttjN`o><B;V)geqRTOFn+b?kW7m}HyP?TW?ae5*< z>NyC16nnr{s)AFj1l3S-N0i^Yp<6GS7V-2F&ggWPkwon16xZlGV(jQ0#2wTfyIDec zW_buo=PRV$E&-{JMq>={6(NsJbW3n_1R~RGwQhMzt44<84v+gEabd4;oLW~*9UDY` zcjlrXGX{jW%boOb*s@tMD|ly|d6K(sMWhGtcX*<%?$>2m3H%07FT~Nk&T?`O8AE(m z?-fxudyk}b+wVbC=y{|}qLa)G*={8Ls(aa;vr1SEU!-P_x8yjyRrN<~G0(IsPr1nZ zdeU#;0>%*PGX87(mhl0-mqQcw{=L2^=( zNEmXMAtM3;DmmvQIWy!OBn~+b8HPCIoWACHebo1>{bRRke^t9%OHotQT}C;!Ab38|tIXAFWoD$h+5=T+lV?==A@$rpt-IHv^svLfaZfSIUOEzsw)&9!6uaoI+ zbTw!wzsqbQ?OfU1gQ%5Tm*eSjoTvC;PJ-sB(5?7QiYA*bh}d|7K~@*ORhQOoZLVqz z_qVEtNOHU0_&sOeT{5{`n!_8YE{S4fsgxM$%n3pJZoMjkerQuFSL=A8_jvS90lU(- zmLbt7-N~t|!dV{cD>z3_j_>qU{}`)QomjUR4doMwF-8zl-zlx>3@_}B<;edy-~?wg zj>moZW6guN*I2sTSl|KD^cK&{afnqmZ5wQ-Z?oVnoZO+Edr06ZTh;QWym~iTM%esg zGd6+=+1m1wtf;X3$3y($yGcSvx5#xr=_%>!PiaW3zKgZdaNILj+!%i%vB%k2}4m_DZ) zPTGLro*bAb%E{E&Anl9Ie$N9?ogZy$IwQO42TUOB^<} z=ebf`f{P+Y8%ZYDqx*JBoNq0An|>(#{Bk+V$X%^G_ZBlBNhZI6{K}Y!sP=WjjL!Kt zuYj780CplNw#L*1<|f|jjQZ<%AMWRe=XN#X5;>#NFOB}#XjWjk=7@t#bsl(3H3ji^ zwI@zmj>=B#2i2gX6pPm_m+F6>U1SufU#U*_c}HTZ=)-Dks;HYc%eIM@G+_CWfhyfTmDfd z>D=dw&M}zyD9=kMe?p>b_-ZB1J!enU&HD=YroVghXY2AmK%RbY?c|}M!@qKpt}=kv z9*^TyLb%&#v49#4$dPs4uNcc&a{pKns_d7tT|W0#Fdvr=fwx~aoVDhL$Dgg z&2N_n7&{Gdq{GF}8*N_Y9EKPl0d=qL1F&;11M*vM*I_1-Vq)1W+RUy&6Pj+CnQP`8 zD!C#jd1-9ekyPCC<9%^P&qwZdT zFKI<;uP<$+^t|E@9l_%^JXSO(olX2yU@=|&E>56COY1CVG_LjN&GBlJR8;$SuWNEJ zY(F{Vfm;)rbh67$QNTrn_Cqu#mX@Zm{1A~EBsPyZou}0y=}n3F7ZyXOc74rnccPWG zPUZ2hjlzElh71qy9!!@Ue2b$Fx8SL4l&t=w)f{4RcVo;}I`CP}jM{Yc^A}B}WB0!{ zPnSyKXSfs*nXkTc{B$Xee$ezW$5)#3h?iNDZ$8^?L4KN-v^r?^ITmUrqVew!zP?8L z-jWKH;4}+=)g{s~#B(R=*sA;}_H-Earf~BG?EDo)*m33!d12b;ZeJI~k|lp+owUEU z9Kg$b2u)ESlRjdDLu`Dhg_2#!Sns~cp!C4^qL*W8RtQOTV0iV=0u`c#8yBXs^CfPH zSL2yZ+1{PeB6i}%QQDe8E5j=DX))Jl`%syw%3(aCp4x^lRh%`uB~(P+DZN|?MC6Yz zZOFUx;o)mJrAMby$s9c=x>~TgBRa0fkoAsCS^OH;#O{u;328CA(TTMTYvkw350e-S zIm!`TLd0}@vg3XyJ7wH#LqQ~Yr#jnvR%^SaM|ozOCNW zYGElW0xtxIpB5vJ#$5xl1(g(o=0@Er0p}9$NAJu`tjBMcDHq#(v10c;!O%DthNAth!*CJlhRTo_wVMn@?p?dpr zn~~dQRUP=kB)vXYg|GQCv>{vE<3uD~rpJ#=`rjWGzrGTIEV7a4$mnjHuWOa5T{=t? z8n_W9;{cTvi&Z+1qvep~)n+BK8DZhfeB9RcD1O2j&3MFHyQCEP{5(+$dfHL+HKNfj zPBKynapfduYa^c_knWoIvPc)--F*!`OTs$goBJxFpP%;E$xRf+!)$%m{TUg$d(_At zJ8R;e+^Bc+W{D-%sk8o~MW#C(mf1w@kW4^qhyT&%ZTo`~2Orycw^RJ3PabhWY_UP- z=bN_Jd^K{%Eo?tWR1n3Fis;T3QIu8jiQ4;1Hz;5=gHDDwQ7aNAElvFn*LTa!XC$($ zrdF_{2sRXnK0Y?Eu5?rQF?MV9QhUiBS=QVr2NYW=^VV{DZc=_u{pA(&ik>O*O|AM? zhpeds+b_>MrNi#Czg78oYV9HtioW$d?v{l}n{24p9^T$JK(|T3pf5}ji<;z!=O&Ym zht2%Q$KjasBbUemp2ZObOfRgw)0UB`rv3Rc{U(=pDpW7az)(mGpduc!&I3W z>8rPis0#(MIh|3XHKkof&ulgB-fA!8(&W?R*&R7x5V3Ric}A7Bm>-dI&AD*Litzcp z?p^3~zc{lpRqDf4p%XOXq3x1hdFjC#vvQ>vZuyX>_{yx%>zF$}3?kQ~k9quI=@NI4 z#xOep6olIEF~9lyyH+gSwmM6ss$Gr*WjXI@^nd6KZOPLS>hSIQYOl#@6!CPN->W99 zIews-6GbAE>q}>A0G)n)-V!u4=S9L!FPLk_EGT;Twhgl!YNdER;zPKqh1P)xO<6a3 z>xP`}$3{=e+qL`GrxSGr+e!sC*Seyn=4=TA)jCZjzMU&}B-QL-USa3XrsxVah9H|# z&A(Wh$hSugVS7klr-IZUdQ6eCI&=5VHX}OQMu;M7Vj)MjWIDO4(QHL0B6nR4Q^mEl z@5G3#{@{6l&xg5261v z(u}~8jnmcSe139wT}e-p(TpIn4;2=o0~W7`(-@GaPiEhxFInHfG3h(Wk2Nx%n5}w* zag_SL#Xs?J-^q1R&GWBDDxv|V+orSn#lyBq2d`Eiu|lJD)XQNR+?wUPjgGN@9||(8m~@T1ou!2 zVQac!K@=PzuX2pifq>3!zDa1sBa>mVn|vSdo*wT80fC9Z2-&+s*kp^#GJZjFp71w) zK{PdeF&pWx=6c{Y95f=7Nh8pSseA>ZhyLDalBl{Lc!}E!-+bjB?&Ef*smN5}Cpm7k zFG#3D-i&XJT5J_O;f8jMe(>*HFyPsB)8&pVV|<<0LNIE*WSZM{NN46ayx|>ctx)c} z{#A2!P(m*ri>bOgUr1OgkA{NLNid#I2$k>ic*$Mi$G%Z7KUX$YZ+h`~`?CM($PQPp z{>`27m$q7|0%nz8sSHJ5ejs)^IIJ>qOt(=V7BVZQp33XKHvB%Rc(6>fPlvx=g>=nQ z!f|Qbz)tUQ(#T_`P~|QwU%QEuFEz?yU=gnWaJ&6LlDA8aPnT&@?{-6O_u)#4MLU{? z&$>wVrQHK&yzJbt?%v1c9gpOa+YPJLh(u*fm8)02KbVBnXvREFEe0tir_^51TnT}> z+0=0gUJ;Wt^EstZQxH*FT#mX!4wkx}K-0i8dZEQ5$ zhBR+9B^CXSX;8_ZU-u)~`g`ErP7&3@HEWr4b0=$XQtxfi*yEq?N9KMYac-YR(7A*p z!QMK{!;T^h^*bYrEoUybv;wu!IaiNsA$F6_3gGWh!rt!W%Zl-+(dCz~<`fe#cg-jI zcRrVll)>7@Uu`PGTd&zt&w1J!wuUH_EOAf!KCG4C;B7T9Xl>db&!jo}o;FC=xCwLH zYU3E;MK!$~^DgbWk}-jCdbGyKD)UreFE@5nV~{O}6-vVXWGrx?!MK6~j|toGW;Scd z(qU3dFgNYcERLnVuLeOUeALcja(j4?gte-v-|F^|Zi4*PlerbYAauB5msX2?fwTES z9AU2BVQMDpp7BTuGB5 z&j#l^;p6wlOoZiakvwO64(HfJ0ZefR@JA4(U0t`xdj##zF7Lc_HL!Yrz&{*g@i1L3 zz3DCAEqpG0e_r=*TFgbAt&J6EgkCyRWva!VSLg~kVb|kG_K`PZGbpKO$o{=FdFj*W z-7e{|z~YFp=O9b#rGdrsP`rz?uOM|MO+Kzfn$Cf@L@`#xfI}POBTo*iKQCmRrHcof zBfxoJV5m(ChGqcr3m*Qq{ah`rl0@T%NX^Yip z8(#AH*42s+;vhrYyOT3&HYuBtA%|`3iig3LV-mQLA)8O9qFRr~?8j`OeaepiIsHSt zC}81_+q0WiTIoI4R}-44~qD;wai zn#hNtH+14ye9VD1;X6jzZwTMyQ5@N#Zp#^SXQb3vaT9U@Y@1fd zpZz&$aFH_27|&f&`m{U@cBS_!bzX?j_NvuIV(0uf7KOPSK}cWLi}%mnv~D1OO8u2( zXnlEDRfG3s=8xN&O=Uw5*m;ZxgIy*pqB>%Q)zykVFz|fr_cogq{`krbjp-Tn&vu^) zj&p3K#@T+W5EYkg{>!*3wZSY+?b`dzX)I&ArPpy;lb|*MylloY=d}L%xinES=Y(|P z_f_0F!ogl)KMrTpZFjZam=!^56Wx7>)1h*klL~&V;cF*_B&w$ z3FMCwZ-EA5bve0yPW?EtyF}Jlkd9?AO)l;oo^cV^*rNy9)fxsJ(Tp*Irf)2<`HTGD zX{Ro8M}PCBU<(BrYmAYAnyQgK(i|-^V+5JpFHKM{IC1XDX|o_IuV<;tWJeApCz>Lk zU>hQ|+1f*M3v0%TGw!Smw&?9u4)~nzt_ z=Z6{o)a8(qGr7>hYeuGcnvB6M-mCX3aegpu?bRH0*+p<#Bz8oJYKt6f=<;uEi1T)x zxs1w*9GI0>9@fB9Txy)9P6omZ<6v5QV-UI#3$Fv^q(|HuJ7*62!o7Lol5WJSLp8ap z#VxeuQ8JQK9fgPxkt&!vIT;zvTrKtt-R;+pIcm1F`-n(xY=6`6+=tU^dpZaXTkR!YxmXprO{?HTkgf2_UpVsuqKK$f# zOGY24@%FqgdnvB7vq(3`-%wy0#S2o%3HYZ!%_eJ(mH3dBD#=D41dYSaJsuu3&aNGP z!ehu4fuRZ=e5|!tfr?pB;n(c%kp^Y`-iuFV2HUJ4Z^?H$Bg2hN!j1;v$HUt#Z5abA zDNa!WmO%ONp`nNf^NpZs-LRh&cgRO?bAH&~li>@Zn2d-y&}=D<8aF>*w;WlD>9mPY zjkfRk=Ce$luWihK_TAQUN4q8RgqL9d$tOZqCc=-9rZSkIvt-e?n>CGROJ{J=m|Nju zazlpp_Ek%%>M8G%KKB+qC*f9qV#U9(DC#?Om_}5vf3hMn7M!D5(FQu++;KqCykA0E zTN{YS94=Myz5Jf0OxHx-+H%LS*31;hkYa|h;EPdDZXzl=Pczc_-E|9sdT^i+iHAZP zC)+tL5qPnogFn0wb?cSe=dFmEliT~E8{ZYf@sf{hR@y{>1~BJ~yDNoSRnXQ=rfZ?1 zYj8-)S=mwKiU~hQ&4J^WNK1;#dUEt2jUxj(2C0deT8BRm((C`x^Y9><#MD}Ybe)^2 z*Ox=cm~CkofAe&HNL#rTotaw`FpxR)QPV3~AAgUx-`jc$l z+qm8JA>|3!xf4HnSl49SZW5+_Dg1=;grsy(1BO0Bt@Y+;3X0P;bKdCvB*T6lrC~q8 zscQD*c>j)cTW^YtACG`pgp`ZJ?#NGz^qNNa3U6n!bJb1DS?rpzhfr%{oiMhjfxyDA zE6mE1r%~4FKUkw|G9WW=`PNxy6A%W3^SeZ-HohbZ)lKJxd>Rrg8ZPpi3FrOB+9Z?(=Dv93Wg>u~>#(g4B_$Lm>e65yZ}6=i`9t2+ z(7`Q`Qun7^T8~yIzX_WWKI5d_MI#lyilSBO8v3h`>F#>bTISiuxJaKL{&*s5H)cw4 zr`)c%M*DPqWoz^nq;&$e#&CLM#j=ghY& z5BrWE4MVAn9z5}ndi0_)5!s}|3UsrvS-XLVDBtlHJNN6WSI_V}ICJYZ^Q_yxCk*0V zh1|c!O}=FV0*#1zo=(Ce)6vE+R&&EZcQN9i@5!-kV$f`ev9#5-Q?B`MpTy(_49Cn0 z{T+`+P)hN+eDzzXI2SH6{i>6<;rUti9s?(5c(lRm(*A)++mW|j{Kqe43c`Gp=C}QJ*VlBwK2cf^;~#aSPV?* zM96BN%8wjK(`8WG%uVz1sSFBI@yXyUUU3|6rV890fo^jO>ttrn-O{}}-nuj3Mdz${ ze3h>C-9@JF@IiH@JY%?{@x2J4YAOFY1m3ekXz@O)f`&Y#d|v5zA7~c0>FH`}6dGjb zciV>KAj{j7X_fY{csX`dhq!aMUeP)RR&Guz&*A9emR}s_irEy{i^nF)TRFHrv<9EJ zjSxAdqvCU}1lpbOJ%7(Wxf{9m@3IH-^jhC+NAA-&>?8NHJg4i9(F2+VZ_m%Rklxcu zMYT67#@Gd!_AD5M-Bhg%+P}>sA374FC5NDyGoU|jN z8SB;pn*yiv)R06T6ZwV?nF)zKP6=v>!*OG#dwbtv^AvS>yf%p?PW#qpa=3i(X<1)w z((Fq@AV}4zy_pGAKxKT?n_a4jRok`Rrcfd!VqLzdW!BrwN z@7C(K5ppVh)9FOw7qXFK+cpUf!=GhHJzTIhDEKsQbBLV#ne;euubh49?-FT%RPuZr zPtJEdYYICsvgeRFIO8)J{yC1|)w-20sv7rh5@@sDFAZaG+-YS}`Y}d#@(qXJiB8h! zeR<5EkyzC4+HwgU+uNSvoNPn**$bfku6)AsS8tA;WJO75OaS!;V*|^c^iX zYppX{&qZ`q^){sAUt`Z+(!nVp_>_%z_51Xl>M&v3$xegriD(6LyPmx6%8^?Bo=%~7 z=j}DS@)4Cu3K@!)T7;t0=O-0WLtE>Sd~|)Clpf+<)OXi!uF!gjV)=Iz<3mpNxT`bl zRYoC`RVLc2feX`*RO5FI8t=2fQhJ46g{Wew?vnD(X zHSNbtAVyJAW}w-%c8Rxu)Cr%-Tdj+(?Udqb22HDtAo2IH zJ5(~O_9tZa1gCT|8F|;r(>orLC|NJ}x84*|l>Shz=yb^r@jS?dq7vD1Yrfh4e$8%= zQ0ZMa$Lsiw?$=l!uzwOg;cOnfHA75Z&K^^7*A!0+7h;-Ko?CS$b^Ds5rQIVD{m1-Y zALV9kc#&kwgmt+qpW$cv=UU(4{&C%cas~&$7k-Di9+xpX?hc(-S1sLnAlaK~QSUxc zp0s}xLE9gXk_9yT(HK9NI@RNzg$wnnxDYpgJ>}wZ?Kj)loJ23&vqsdQdAQ-$8G38G z0Y$4x)?-CcRo81#*WY7!L>*DqJYVDXBl$2<8FA{2IsRJY{6uD?KY3}eIMa_i)Gk^5 zy;j0En*Ah)kX!JqBe#!iFghna+OI7#KnM8r?z?D0oQ;LVDhr-^`GZe;~9&{})&6Wzgr=4|s7Z&ejihDELqMW&hlkk{k-d&tCq)e_Bd*=)B>*|-WF z8;R(OS?FnV^s3~!27^nus=;6uk7w#-G-kZHu^WOvG^x3T?%@EPW}$nCi1+sl#1!l+ zYyhu01``r{=G-*zGvXH*zYgLPEIq;~#bzZGOHvzHE!^BM?BPW-#W@=g-rVy-oR^FlsF&>>x%|wjSe@a< zO$rd(!@tkJHCf>>OE?({&KkmM_>^urCgfx~J zO&IxK*C@U1r_PSsIk_t^#;KKvSp|%+4^kzI7rQ25r8)a#@ZyUwMJhacdD!7g{JEk#o_2EhYvB&XRMQ*g? zGVyUYL5*^n`Wt}~Ajrho=TjU=V0Cqa`h|4n^4S%HsGM2pra_uhdE-s>6eeCx2J z&rO84(bnrO{BrF4*@ZUm(v{PoDVPc9QmvZlKIi!no6HW%Su$m~xGMMn$KNE8IYwk5(B&&;+?NcEVaSug$`FB{e$zc0U~E4x*p{0G-7~sN9a3Z3 zf_mI?H(93T^fnWiAqez$?`if*7cWd&*LSD zLaaWYzP%7l=ibvv5wbs*g1@}Q(CJdKR$h35N>7_7Z>jBl^t?0*r{JD^rwS_T@{<~k zV4GJWB=zsddcBF#AXeqc=BBEKS)XJ(G#1mbq?Gso@3U>fBRhD%PPqPr4N zXVQh?B3ju~YYh9Wb2&@3uvtwjsLVJ4VeTC75{>E6%{k^;?!~`vQMG|%jiGao(P5WL zL^0TmedqJ8KkN&YyiujxLxRpuQok$Yx%r{{em53~M}(2LvsQ@L!CBvf>hra%If$|q zvxt7L$|H+bjK~sXzp}tOdxQ2t0A^;7^9WDeY|gRKMroDBjs=}9x7z+7K^gw z6ud}e1xIt0CwA4JJInU7v=w8+AcJCNRYev3JXX2?9HzZEr0Ld`sl&e~G6vKCJKUOoTax5)Awj=1SyOYDWFJ_2d^e19QUb;#Y=*H82DiV#C!jLb zz%5|}NR#RTHFQ7`{?$GwE-*VRy{9&3PABgs_n-1ok+ag@n7|;xXS_^#s9T?N&;7xr z`wyN6Kl8@*xzR@4NE-M8Kk+j2ii}Eql-Tg?C*7_RI#{>_m^e1>pPeT4Tu^o|_iNT4 z)OJ*H+eMzmW(dL|>nN)QC1IzdC96V`$F{5CRVQ2RW<9{0#q@24;-+#G@L^kNLl zp~+Z@>DkSi!%oPmwCGX5xvKK3`;eN`k}F*{QId5IOT8(lHtr;XcIchldceGz5DD*m zSFdzDw1n*rD4PN=4V*=B&eZE-7??Q3V)W`4f^%iV(?0|5ga!uA*xlEvb{vUy+EhFSg9DY@VZ$97Q&t z1=HV{s?(F8QKDxDC5XXH6!nk$}QAR4*~f?@N=bXo3zMjjU}q z`jC{^tSD7LcQfFu6aQAM5$g*}KX%R*@WL3$ZJj*Z~@jGU=7~)wIDb0gSP2^V5tl?gH-h==LZKc&EjAOSJR)%udtIf?AT!??QXFt~I@xC4Ey!jU7fW}%Y z_fYqn@PYtY&89!3ba;wY`|!)#Y_$RBqrNzU^lkrJd>WPUl#(=`?mHdt*=+Bvj_5X$ z#09Y|^kvc$U>?Z^?-EtKYq)ft!rIbN)lJarQFkb7YdET%_cB&!WWweAH2e^@kGAp8 zjdxn?P8cjQQUGlmAsfzF=YKP4&4w{6zDl#g)^@&)tk1a`^qBInY88agwx6J1b?HQt z$JP?C^!XqswE0Q1##z|@=F2;-7+Co9u44GOVE5x)XCx1IN-;1ZtZyRGf*gS%Y%-Xo znmSblOaa-~QY1VH_S&`2dX;juVlqc>$jyCWWniU{`>BPljbZkFd%sn==4c>%l#C6! zVjTIt=}VJI(4l0eQueqhi*P@e`IOiYNsbL^aHgRKJWb?eL(+~cQ%8d+3_YCRP;R%5 zwzUCccNn~8X^L@|O^%XYiiP0( zVoy>d2^>3Q$_BsN3d0GAn7@aXaOZ}W@d<5jsXNgE`!v}&5#-q+!lgV6UY%oh;ZZZD zeYQmb35D%j*6)bdmAn`!&_`C-iBDCnj9Ux7cq)7rBWP#aHxUO|Z*L4_!xYWMn8N_a z2!zfKW^?YmqOoZcRNj~K&c;_izxRjGM}!eULot?87eXp#`VQA#$AB2I6bXebUA3C8^2#x zMgKz~?duq@T6>6IRwXJ}PJtRtl3r8+)C0dp3~b71WMp3o3>UD;$ox(bFBko|2N!j7 z^zdMFmfc2)RgXf!p6kT|YH^1;ieU3o8R?vRn|K-?@^no9Tv|DCxpVK!XZfAas{N0b z2<$!PZLNB$On0Uhh+GaNkJlhBE1g?bQQ&_4fzb>9(8 zdT8FT&z{Nud1tP*F$~?g+M?FUPf7#uA>SiOVu8glEaePZqo=I%~Owh`n=u9FQYiEknw{5s{?mkTm0K;~tWvYa_a_k~-DQp>#xTvNFy)4_5xSMz zzed9}>A@bnX6=pYPTmi`0^cshrJ^rfU`Gd~H8H}Px`8f&Z9lR-0yAhK4$zc8zZDIu zahb(kKBP(}iHDs>T(;jgBbqN_6*kY%vYU^9r8pjLp)-^k*gsM>0t&2bvomBN@wGB^ zcX>YN!>;)| zLyHY*2DhB>`m*fMS<+28W_)F_ScjAClWV=cdUFhA`R6O{d9-We>l>@so7SL|U?eroN zTqKlJ;%QAU)heTMbtp93<7wH_znu^a2J(_9)7*jVG#9U+XpYHv2TQPSW{B9DQR8GS+I0uHv5nl8fM4MJ}Tnk~TC^1k2i_63FE zcVDquF*o~Dg8&kGv2Pkd3h#b5kz4%(F;}f{buTN)f7<`z!#f(3KNlHI@GJO$B1sKE z+t!Z@MBn<|LSl0`Ge?X)f=^l5*gBQ6RNkQwaR`MF;5B0iB3yBQf4`6^Lh8@G@$Ip8 z;tiiQl0;v5<5smN{`X7Q!i08n<5b0N`;ag#v|9E5czq_aRfNqe@w-l9Q#IfTaZhsd zEM}!DEazWzyk=GWp~I$L;um3ea$q57vmDF=j5XuV!6v{6$T2=eL>`O|Rg-T=5dEqH zF`FBJ3L`+Vcy)E8aInNwJrgJof5qT48_%L)PjC&oqK$@fGz%|cv@2|NI6rkL zlE?V{lcFxWnq$qVhx%WH!7OVQ0sV>Uj53t`e06ik>HKtG_V@t^qN3k?+<{H{P^SB~ zL*}*75~Sg3P(xf1hb#zr<8O@tGp?JF%n_RNwHSbp-_wfg8}(5D-`%Z@ zgibaZ*o7Gj{bJ#L+*aR&w!CY80CU7Wz(F6y0nWqs$jbMLP6ZJk@ zt=Od26|)3j`0#nhXIWy*Sk@`UKTs!AJu<2va~;%DZR+1x3oZ}m>z>Kf!P|f#K{mjh zIVcT&?VII2?H4GT<7>4_IpYBE?KG)zeaAH=^e-HCc9ZnF29hp1PiR=+Cl@p0Jfb7~ zhf01TaIvsbs&;hXJH7zY9|!JJbi-C;0Y3aVm}OK2+ut)Nvzt$5?Ns7O=~m28Y{fn1 zV%=m&z`~842^@teuzO-+hvBG0gew6R@+k*4sPFD^0fO>XBj#fyN2ymthqf=0_lp-dM z&uq`Glojs6yk1rW4AYLbAIkk`kgJ-4E@Xr5Z?-87Xvr6K$VvyRg#p#cJj^{l%H8r@t*X&bcFm7A%l-Lo*my@{Tmhn3 zxDy_+yz8uGIX=D0J7Pa95rwR3sn<2@wQfH8FqNZG0T67)n+!!3R;`omd!zZfUZ=pP zkk=n8DKS7}hq1T&#lIoYJGB;RH)#r1b=#~pU>(Hw>HRArhqD=q8gm*)psBvS-7x^m zsW^bLX_R)a$zWDu9fEakGcJ1hWxq!n!RWxd;dBKic(sepy&Lz5ztCC}ru^tp`#8iL zlCl+q{?Jno zRyx7RoN%vS^ZIpS0E@jYoC?{WDQlHU>2C@g1xF@MYHzj~=Tj&U!!;;qySfGgHVB_&;Zv z!|CP9C|=XZOb4*Wyf7R&{rm+B4IG5p2p=a1?$sqaZcVKh;)EHgH3)vA6P+N$%|2Ze zeJUbvT@WwgqwgXYzxqN_>FiT*-;G0Jb!l;|J&z=%V z&!se0I4$VjH+`yHRhKz7B~UI=Jwm}!%HH>Dsy9bnVK$Rj`y!=snH*_DB=Y2@M(CK4 zryodaDgglsWveh)Ba6vFOC(EFuAD2#5Ym(>^J=|Kc!)Jk!aO$U>h^yIxp%6oWmjRl zpq!#P`&(hh7v@@==h_KB#W^SmzPkSow60=XxDrDiv2=PjuQwfgW>=!MkK=#C2(a|Y z#SmVkazAS7vd1zp&2*Cg3wU@dyVCAl759%!r1%#*Uf?CYE8QNktK?G$e}zvR09&vr zU;cu?7kB?|0)nz09HT}0Pne{q7jvbp)w%H-8C(fpH`A`4e$P*SfoZ z-Kv!wfIQnP=KuDX3s{DQ)r8$w7W-084zbH?@&^F9ioGL#_WU`^)wFY0BaQbB(p z)nC>8^u(1k?E53qG@suUASMJ4^Pz~l_)k6YeE@=acJoW$pG)4K!KFu|5o&)L5R`=> ze3!7v=+C7mlHk(KwU=yvT0!DAXwksWGmrjUN(88q*55WY^iRbYuG5PVJ(skI{Bx;? z7=SG)K?RvVVCy?*P_N{K$FhGeeZVLECdC8MYA-5!q@MrYGe*?qY=LzluSlcJad2^BtkVA1Fp|4!ZzSkDExMeO3R*fYf7yY7vb}`<}h^NeJ*NMilt{ehXsqI$#pq z1B9eHup3y7g`qFAx7n(#OaSY6rPP-ZCSly%=K#pIy3B^(v0fSMbl&iw8xH5vJsG-K zb>k2W2r&0j2XkbQAO6&inHLr~QVG@nm(?DcY{4 z5Vl@5l=> zypoem@xCnJNDHC^p&etz#)jZO;q&cb>R{EYJ9uFYaY6rBGiGJ#5d7O~zX#Rst&N!i zY9(jC669a)Ekr>^f$hdez;QexG{62}nN$O?I)s#*wS(E}6dH{n*1>NOxsqumg~yl?@q6GW4OpDh{trazlmIJ z2RN&h^H{@0tf+WR!dIwI2I18q4gtrNwrgrbWh$9UY!r>_BZW+3yWtU(Zt}Q`KIxWQ z`<4qGRA62YKL-ouNI@3dX~1e)2%}!;_H_F*$;Ix3ErluoAHQs{%w<1!Z23ylntRpo z-wJrXnF5P&@T`U5R(&xCAaFN#R1A0l3_KPu?lEo*-i%WE8^%5is^Ak6<_*%YYeA^_ za$b(T#qbqVg6&J?FW3OgbNOb$6ivjbrnE$ z?}F~h7J&92|HvKM!XQQjrM7oKTA!U9vJWfOW{0_3krD16tQI25)n5ag{785>-+uGG zON3=JAa!SRG{EgmFYI3d{nN#UBKr4D6O(o`KPWe*djO?;WOe3X?qJ(jZdC)_{it4U zoj#DR&@!KA2&dh?!6ZK=ya*9Ra>yBq9%dH_nNp0(JmtM894Bcd*xz{Lrqc#+Ck}(B|8#PO67W{ek^D#>aRt%Yw!i zs+0q01ay~+88G33bBGZgt*Or+M*{8=4lF!}LDJzkg$^+BG*fv>a{$JkUEMK=WNRR$ zjn`pIukX8mSrtLo{tI)AUw30B#e$m2)?^l#PfLUeWG0aB9^co}DyJo{ikT)Yj&*mc z8v60NLBMwPLyAmT31CerzCb5@m|lbD+@v4OP!WmkymDFszS?@Km8#XZJh^PYY5xP-P4f1zzM5`6b|pl!XW)LFgYCnPU&~*& zRh)j^SpLUD(6Y;2#npR~=IlNv9OF?xU39-JF|XeH6Bdb|m4Fyqpzb>RSF|D~W~L4A+SP+Xp*iq(*#C|NFyIM1#9q93@$Rwz z0~E2ctCk9DNJ?nO@96;SgxtkSF8%bsM@!;(>7^axx^ z>gnqLgMvU=IExHM%&$oPjV)sIH^HTz4~8HALJ~IslK7tl7xDIwG90$mQd3e!zL2r5 zo*ZuTX>)m{%(sOUf&mUj*~e?foVR{g5yNeaBpy$=fhBfIqk~|znCN$e0$p*F0Q5kH z#ocJY5p&7K0&ab4KOl`J@A~g{z`!~u!`F9pUaQx)J8EhRy{ai`HeTA3qfvIlYJ%*( zVjggSYVHtpJ}He6uxSH3Q`FqReMuw_VE>T|x9F$7me_YMdHv!~p2$yu$)?1p@7~2m zRI%yU;K%+S7BZd}th+5xDnDAeuxj5=^B~ba>J7^Rn*w#JFDYEb7O|{xQGzXJJ!X1S zkLdPIuc&{Y$X4ge1iIrFkry%ng2piLoJXPnQLkR(d}?yMx5jJGK?wr}w8#_Sqljq1BS zw%>^F1W4n;NtuFOBk!B&*>JYi>5)wo=R1jxBVdIJHoFERupF>kea| zx36tP?o)Tge$mfrwkpv-~$T;pK!CX}$M^v@r(M-A@C09KF zCm(Bo!)4W5`(KdU&HHjkG=E>ePukZC(0Uhf4`~c$JJq}Wo65=0uET&qxn?{%J6>?> zq8+Qgp`8E~lv-N_QqQ=y59K+r5?TW`+_&Y+UdrbDW>8rEVNj5af&mxQI!JosF$Pn(t?#MB1p#)!ok)G9 zUP4@u{VN(-7w5_VyaD@PvmCm&H?b1%y9=*$34h_dv{X)$qDZ!$!m1qtXfE}T`)@S} ze7AaetY-o1hAe2Ig>@heZi+frE`3z3tAvjJJ3|v|VhV==n88gK4|c8r3C6#_Ww>DZ z9#{dSR@$b_XD}o60ZWEX5j+HCJ|z4u-YV*bC1;_!?cyAmIa&nO|L$~zdL)|7Sm95W ztj%x5^@x9|;WJ@jAmjNy<`4+_;alB(eSMl{N1$Jidn7v@?c|RX8dg|`UKmXR$$?d{ z{S|U$hG(zbviGlA;UX?z2WBPa-5jl|Wkmbhr#BWMFILuMTI3_MwHRLn=RFU)Hv(t| z%L?^Dgy#kOu`9Ni{USg;Ti!bq5hptNl~#DUhBP|!Lum`x_#R9t11B!KFA>q2_)>7# zw(scoyv?ND7_M=t*>A83I{~|~y$c?* z)(!h~tu2fLa$#%_;1Z}=-Fly2ifcNX74s&+%K^nyw49r*uk!6Q-JO!sAt)Wf(B0kKgZlCJeeV4S zu0He3%$aj`tiAVI>wVW+geb^KpuE6)0R;txA|)xR1O)}X3%s-t;D9RvU}t;ai@Ldp zh=P=e2#JD&t%O-3-@rg$zyCWu?F$E2rB9#2l=OW^I=VX0x~q}L zLI!%CM?gah2vH)l;9+`w?d?GL!o2%o*yC&Fo5p(jA1%MmB}eN6dnIMgY5eFZ>$q?w zAE6D));@fB2?r;RLc*KXk_**c4plAo*)Ia7+EZxH3BeQkQ#!1HEp!PYq61vDJk%`A z0B#7a&;<&KHZ`IYA7My(s&Wdl^RxbQz9RSRPN5XEPhgxhTp1h&+z&Xv{3(7s=bA_n zn%sEvnni`}HLEdO?i*v4I2IZfE*P;FI1V^U{#Noqd_l%={BVRM^Pf7QK83$C2qSVq zsDjgjgDc?{DrrGC?EH+3JpUO72i6(ga1d8$)5lPstFnxwiUxz|OBh7s^LSV|i4{>6 z(G>CHqi+IOUmw!ban!K&_y}=)e0=;G=`6WqN496zLRc14vp8tRZD0D+@&j zd`5tR4mO8^1wKIoFFfD{1@$Z@2nrr}e+j%qvta)F6?!-8*}u*FLc-@@Y{H`?D*pFz z;2S@gnUj+p4>Pl?t1FZ18zx%^Q)X6fZf<54HfA<9M&J%cM|T@112;w+NAf?7{MU{s z*wM(r+|J3|)`sM%T?0c~XD5C#vZs#z{rfXcu$%e+^i`@5#5^tpEGu|NQd*ep1yD z>>y%m4Rq-w@P8}w_u>Ej@$Z9t%uh4_A4&WX^Y5#GoCRL+G5=dLffpYRQ0}0hgrKBE z->JAk@1;LaQ<=o>i~R8U`Ii|S5;7_(!w>s^2|0C<&$(&`y0PteuJ!s4U zd(B%n3|t+r=2=dcSNGyszOJaWa7YpGZXV`6Y?1F^9;Cerc)dBh7|?6Oui>HnDV~8i0>oz)ZP;m{o&iqV-~Cj%~M=U17&59Y&O$355$m9}ly z;y0Iv%%|TuGt*q=%pI0LVh_&Swp(9TE@wDTX;$yo&kqLS+t4KHIE38Y>{dA39rq3K zU;Z*e3d~E?)dJdP*7QD~WJGx^9Hq|mA_G=2-Dz)z6;<%^m*aCd?+HZ-#NsWASf1Ox z8gRU-TC~kVbI61F%}OKg}S!741yG46D5^7hI{n6WMh@ zkJXxw$GKfUB(WHL-~$@TugV%ucU{z+(lkw{8vv#xRTEIdWiefzezx^f1PkP(Xu0}k zl#d*ceokJ~=?L#SaCbCtMY-*21!>v!bmKe9xN&~%IWYW~XmSbZ`Fd;cDF0;~hxJ^y zVn>A6w6-mOSNMazechNe{QzqpLF|}r?j`gyxc*ezHuE{`6#}oL&g$-$@4K(BKmOj5 z5keT~OL;U>czkXw%u*q7tP}yOyO!lmy$6R?eMI&%cc-Ja1{R6oU2&}1Npo_HR)CP} zA!~#JiF!Uw3(9b|2K42PhF+0eJ0zI2BB7@$A&UokfKcsmu)E10fIUo5M6NDP6zIA= zMoH;&(x<8p!eqDk#YHKbBOd+*7!`L!K9WC89O2H$W%UpfJ_@k_UqnIJ*a`PmR=`SS zILBQ7Hz)g7!L%Y_P8hLDRL(zoWCv{b= zsIyt%8Ho%L&7Yt@;SG@aJ-Z57-8ksTnD%qHe> zv6mcU94&^$x{W@`6hlSyy^qMV6cCvp?XknX(3iWdoX8cLnTo@&hizA`CouE3Qq<`b zgK5tZ2|~_**}m~S{3$_9SaK7|d%)#>vt0nbKcB`1q4%rnd4n&!-ooDQH)Kkk4|q9( z$Q5^2-uXP-mMaw-N5<58rlj2dMNV~l_CS$%;^6uvn4$KEu)WhD z;(D+Dqi$AyA9R`H=EzuqJh1_(!%7Hsibuvx@q`tJ?ZhFvjN_!LI%a&K7^QYLAp^2q z+fbqcYu60NqMOn7WJ%)n^ChGXTN&uW)$Gd3c>^{}5SeB$#PX^v{x zhWS||vk1-{-CvA%%NjWHzY8PWav%i*db{f&*jL-SEKz#!&oN8Uq8iKXSlk~FwP-vz&x;9%F>s4n4 zYJ^oN67gFUbZga~kU%lzY8wXOw!4$}vBpSs@xcz93r@=P)OP{)BHPwUrWx+SuWxlN zit_gGt>J5jn7$(TJhyEzW_*Z9tAyN3KxvBx1z4Bd@E8<<} zF@tiN;m7Bgwi+a_=^;__{fQqXI6jz*Y!8RBs0^Zej-BRJ9tpiU{^dB@Lrl=m@{ELa zbr|bpShc}xsQdZ=FhZ4E+xvx5`27eQC;@#sexL8Av~4k65Gy;rg?u0QYgFA3jKNbg z)Nib-JkEYfl$}x3pcZQ+3A55N;6znt8zu&P#unz0dFf&^(aPdzRgLbViDMP>;f6OB z+m%iCgvU*n-2a=Sv1Toh9N9b1*t zv+*cW5#wM&7nZam1XqVYcASO?Mm32&2CHJQn<#u$F-m~;r6&b5M)tK)mT_qx&Nt3J#T z4`B87)KlS*&lfia*{!?~y{+W0@#`>P(o;gS2=#cpzbq)cq}?5)uL!F_bnXfYKgA97 zPJ0eFzW}2eZ2qZXJwbbpGL&&*58b*~3V8+0`4K%7g~FnK?O4`BH|5#$uP&t5?mB;0 zjsU^ffk9oOKoU2>y2oHB{vbHw%vwKMZZMjyK=Se-pJM#J0ZW~*_sfFIJmZ|#>37Bq zEc~J}&WIAv57w!#uL~PobD4FkJPmfS=6xry@QOrKbCF<%PLI0KBF6K3$cz*WxlQPK}sbJh9~HYfxauS%=oBRkZc~|5$c}PB#aa*&OtBN+zrK^$J1AOua@MAjhRXd0lMlhk`?ce87|3tZC%6Z3c#f|MortP} zQ+6q8sM0-z=5-AYicG7)m`QaW7}{P3os9;cExyXqMsV!~P@xVZQ+4{7`cNm=*{FU_ zj^;7XTZ_KS9<#5#J@|^BLxGmWvR+KC%BBVTOa*a%(4eYgg_5NlofYuKbQTt0Z@W}} z^8OizIIdQ)C#gC{N`D(_d04j1D_M!XU`$q@cyS(5&gOQC8)2{0#UGS&Fbma$y)Mf zeO~XC&8~c@@6G<~#xpRFUvU+_5X%|s4cLl}miV?h`nBNGcqS?f zG<)?;2>2{6<`avB^)(pQ6ijXt34R&%C*svQ)W%h=$qoUyB+5-ooflDE(t?0gc*+(1_O( z0e!jhpBwHk{N#7j7!pDLD3uP3LL=BT&1u#p-L!vC(ZGz9B8d&J{_EB`%~v5sWt>`(4qf6f))@<%*1+~?pqza7VhZ&*<%we-AW-Pv>hZ9tx2z>Z!+{N8b0=; z>A4ANlPP22pu+ae6wC6twQY04zxc)AAtk} zKhRvymCItuM&J7)t3LkZ?7!wa_WXpZ+$>(yFsbF&JnLA5`zKQzgx3KL`7FDJDubuL zfz@v)<-bC_sp#ms02*_z5;&>MC{G=1sp4_)|rp@c9MIXzw83MfbZuEg1T-0 zO@#!rr?fyWo$d)IIBk!KHM<@%Y1=gA?bVD5oB-$vzb0SMV5=b0r@6H9+Y5(&(%_^x z8LqJt!0D#npKnMk2Ym&sNeh|p-Epwg4*;1}zV(6tDDlZf8q~IJr8G@&ee@-$*uHdvY29^tWg= z+ZBWoyBfhel6wsFH->YECU~bG^Y+3wbM|Y4sr8`4qQq<$qIz zf(fTbgiEpk9Qj=L(+$m%g?ejk?(+#{Fn|#ffT?K(p%PjgM+!c;)i@iCJ^)jPopc}L z-UsIR3g_*JWl33mN=J*jt~=QL=NGZ5>bs!-npVI@h{LQQdRRuly&1VZIp>7*UweD1 znhS>EuwMc#TfAh!hzg^Y&#(&P;O_)XplPuU&e;>{Y=HQDWvJ|cIRSjr0RVmt(=z5J zyDqwSfkzINNB2@s+aONA;eR&Ck4AfPP4SOu5rk9G?DG|i3MnAweFl7!`?DJ6{uz6{ zfsIsaGVA%;0R*_11H({8lZy=ij!5cP7B*EG={XEg;*O(V?KeOMS0e?P%Cs92+PrTE zb0P)k)IrXwy;lV0Q;qgUya8tbWMG#{h4?cGw97Z1+5X=CcyD2R@6Q{&iXVgmblpL^ zzK(?i3pEk}y*Y8=%t)#1FWJLxmY!Y2tE$@+Tx>li8C4%k4$h=WL=e!ndR`_tj`B~} zTU)oDPmKW^#dsoaH(xe&&ildyV&6yj4XXk`$(lz5rX^}Il>y8Eb~JetX=18I}4{vEsJRB8V8B+Wvy_egAwXL>%$AcLam;OpKY z(}SkX5hq?Fji@isWjtg|!~}B|&+MGnue^T?-2@(1Atcj#Q4IG!+$8v7vKGb69s`QC zU@9naZIsYbYn*TF{{C|K0}i|9H__|Y!q+U@QN;F0A;e?X4Zn9L3I=o_NSb zaPwQ)Q=Px66^{-EK=UDd*K5jCcT6CcsCYH4qA%Hi!uz?f8SnbXGwlzA3VVV~Roe0b zw3wr%C7g>U!ku0JX)nM&jiB2NG>>FmeE559=LNJ1L~;UsI$mz1l)e?d_2I9$RM9)c zLEA1n#yKXu>|7NX0N_@)bk28vnPsT!3pn0vCc{aFCYmKJZrelb(>lu5M0@J+%Jn2% zlc@f>tsaQ*V*rpl4~H5(GfA~JnGcy!>ULhK!#@Y{GYbMrB&L+R%VRK()?D`La1&q= zjOLqgoLbyVRZG+!7Z?IcV&w%i6EqAFd4p=&N=^a9far#T_qK4Z*v{`o(utkx&P37N zRXh$f+~C*M+w)!Zu^mV0Ur!WM@Nj9}lolo9&>L4+Hqs8qVavp8xtHzNc9F_R^y&6r zIo+yVcdzi`;bPu&(~Cg7PGz|PH*ZDD?k!aXx^=%3us@NgKeZlxL(()@0<%dFx##Kt zbH_y!8c3-ewj}^Km#GYZH>p@k(RE#L=9YJ)=jviWbhBKfEl4voExFa)))YEfiXnx1 z7K97Gf)X|G+^w;X^YTiC?z_$65+wHpnU3PT(9MX)r(R-Ig^hN?(81o{fRG z&;2z)TtSz3qBs}bL;%#g+Q`su2P+*eq~9C&D%Ui3n?f1Y9;S=QIURuW z=t=ulQ`xhU2>>DUrFFwP5#$n$K`tH$IP^;IlSKhoIW*7vtqI!E7-k)`!os#l9C1dp z^~ATiMg&8=a>T{Ua1m197RxS{jeH%9*5O9ub;s$qZNI0I^dn#r*2LCzGmx~Q_OBY` zUk-c)%h>q=iHQa<4BA%oCJxks5wIw*v51FEXxH9=EyRi(?_t9@-*`F#) z&C9Wwz@SP9SHO4ldvT~xaQ9?WStuR=L_rr!;P7d$<#fa)v!-X1?^K3Yf?%V&8Zx*c7`6jG38@6rjTjSY=MRFUE|?nhsACZZh=bIKAcri*kM3%h{$K5~^yk44KR zMHdU~3vf&w+dz*Uy#H!x7%YFcpV7h)1zTZkWK>N8wy#-m#DG2$*$#j5a5Iic1b8 z*>&~SCkKHfM1oPnGdy;RBqLvi12mVpecSa0K0P)c>7XWH`Et6@1b@xhkIQn3>ECob zLz}s0Z1^?iqs-}C80v&!yzLDAa=~M_?B{R(4^Kb|4$og(z#Y!&V$t)^?CNMano0D_ z^=77^5^v39=7mk;a?npA&piWcb(D9xVY5eSy%sp*eZ}8DIVtLcgM2-HSvUbCZ*b;h zzOtupE>g2(a#@``w)IcG(_y2=sDTAnh_$t{vzI93jmuUJ-~sS5iC`0|_9}agGn>4J z#X+JInyKk<%y?MVO`?~Eh8lJCNFIO;Gb^x#z5S>lyKy3XQqkRbq~b>JXlrd&51t|h z2(z$IX>hc(;jt|al7YWI4vUFuSun{ntf1FPUe3c>lsfop1BjZG{76V>B*R*Hca6w6 zpEIt&NU$&llEB7x^2!Va*mia$cAs4MQc*q`KFQf6_Ty3RO#-7@9aAz@O!XpanItb& z;qX`;pX>GBi6SGOnyJYKMj4go1bv5NdGAGUNs{WMw~zVxovD<}`R9O>xWb}3jZf}< zGC<8(+RWD}KFSsu11@&LE7~9(aRek3iORE;b;W6&rE_lBz(RzBg~dPuV~;YfuJ;N3 zwQ6Dnq7^_*N3O;)BT4Ka0D8?MIs+CPvMHRmy9My%pSm=JSWr8siHzfYji-KRB9zFbfS?cRyEa?vf0qDv-hp%Z5t4*~zoxf-hba0^G^AS)vAj zXhk=!UjxW@rbBCv8}yXHSMXnicyNPcVfz?DfR-Ei8*&ye{mm>U#uy_5`X- zz9OTsM?W7`6j2?Mn~&1V4#63Ea@wpPX#z0!dZYyX0DbU%E_yFfXB$TZAapYb(?8Sk zqMv{J<&UC`{n=qFJ2*)DB79sg#!1n{ZTp*Ly~D}-`kxggWPauc+Xb2RgwUDZ7s+A> z242(fv;kNkhI#6llHDAl9;Wfu&d_i(+_i$3x=FHu4n%#!kqxK>UF(&PMnA?5CpQ|( z#~GTW_3H+xqAj z&jn5BefHCSEK%2k_9AJ7BApu7ES19=wt%XxuUF1eX0DEsh%8qlw<`;?qZ2s@hH^Jb zUQ@E67Bd8Q1Lmhs5@!Sz;Ja(ZTa6{NA1CWidUvMQatM~C1d8B$HuP@o5l3&&*7M1#Z_Y5g!SxuFsZ;r-ZOm8 zvvAHwo$%UfR*u3qyO8$w)}-g=47AbvjjItGnBl=!00D1lOSei%^D&0UHd&o3V`eQ9 zPdz$Nb(re3MDngJ%KZE;C;f3ps?C`4Jcbnf2!0;LzRRW`iAAE#pn)_Wl@IJ_4)H|FD@$J4J9{QLGSG4`N6{c0X#KbTI>D{sm8lrE19QP3u-yixurk<}t9^G13Wy?ye=uB{n;J<;Nwnk$IUZ5j8EZDDdbVkUPd3f=Xdh~cjn%xJ= z)C7YuwS#meqBr%m#!&#;p_C9xuI4A^BVX=v9H72Kg=~*n3dENT=w!&Z#S;{0pqgOr zqUaWn7<~y^29vi$s}>KtY468lp_1!=GqFUIAsfdkmcgaKeQ6GJI`mpybvgOND$5Rf zk_5rk;qzf^w5?oq_fK74DBa|PD|H~nH<3pdOjMIp@^N9U|oGzH$H*ZJi;jrFy{b zmdwkHZ=HzrxJ2d0Hly><$rT;?*m`-k zmh#=rb+9awd+TjEMFud5F-tcfuJs2fBUFT@1TRu4p2AZ@TRA}AElG{o# zSqx|O!o)QMVC~2_izo8NhEVSv=$E_1>A9RcOY<=CWbj!uJQvQX8fLiWq)UZ%iR>r~6~glMXL8PQGBRB;&$3 z9#JWmU#&<=$fa3Km1?$jyt(VbU^`2{T;H{A``P?@^KQe#{1f({Hl7JV;jaTQ5vALhP(u!3& zZiiN_0<`I)3#x3&TUyGIy|b?5^Sz1GMXcYlrX%prkRT-I{51m+kQX?wt=rgh9Es0~ zTsEKrSei3xXXRVbcL8-@8Q@*x%+WecQVZkS5 zv5lp2_~8%VuN)~zce_6?Ti#f#_p7;XTI>Cy_andO{00(p8%3(~+}pOko<@7U;@k!4 z1U(z>rh^-NrD1wAV9BrC4_JFLvf`2$naX0a3G5t|2yrzngw;2;GN_m9*+tt_Zmsla{hx0a}m*D zxs1<8Osyfn=Vh4I_09R^3t13uikEX9EZxr4AynPg_qhMN{uU7(-p}9$&$io#I^hXv zV%zT3ur!Sgl6~8*SCBc?*)Qv+LAGP3IydF}LKSU18XUWP@5h2VGrd-htA7UJ(c)86E z;)ep&whgVy-vo)crPIey>nho&H9)8b_DQ8bn&Z75yCSO(nrr&1(tV;Nq!Vw^e(yH| z{~Ph8z9N9l-1u&(gCZA|29P(^W}#T@BRLv~_au>qq}$hjH+vi=Or62R+rU+tA%8{e zS1Olq%v32E|96OB0A4#0N@aBHAFoXX#4kca@;BH2sF)>$I4IymQ9@1D{H7ja~~|3^Uq+A%BXiDv)* z-*D7>NRJk`^ytb*bPWK3o4EnQv~lx3V0^&#e?aqP3{K}%N;9~P0)h(u4idn`n=YfK zD~u6-o%O>G-EIW8(`w>oRQ=XdJx_9BRXF3LLxs+@*vi)}DK@@$?3skn?*ZaPK>{R; zQqta9_Zcb-ujr<-{QXcfJ#%5E3D(Ba?*&vSe-hDj#<52{vVTV*+JZAKy-|8RcGRWB zqgM3^pV99PVG#5A!Ft0RO0gn>Z>>>LCw%;eti z5+bTBADAUXe13F!8vnbe@t*}w=d2;*mbAw{znH1Zdp|8DbTn@>ZM&clp<<~)PkugiWPEPr!;ffxCu%AEM(=c$Ig!9x%2Sh_j zXHCA5TXlYThrRPQfwtZ2+qc_qp>mo7>8WHj%U{6%5d!v;5Z=qHDDcBP6FZSfbeh%$lRBQP^bgE&xl<0&#MkN) zu9d&2KqYN|vXRao{X7GvF~^)0P?5_gUh|dH!b@>_d%k|TeS7IrYf(CGZ5AGpF2kIo zq2!T0^CKba1@qgozTcAuj_WW6Z8WK#@8f;L-NWtSSm`W7U%DaInmZ<%e@EQ=b(Hk7 z{7##+id=0oufZ~@6svPa@he|u^`dLFO2dh+rzo8Npy(11giN^JNR`Lqv|(@@W|auX zU$!{H6H#wK4A4qas&N--;9eu5g%QM<41 z3Bd5%+}#VD+db2GC#DrxDT41`Ufc%)*+!F`mi3wa8Qaglctfj}_ zPFbM{tPYB9u>PjDioT8kF)MPjg1F)px?WEREs0*3KGIoxWujmRya+-uMFp(0Fmyhag15H&l;av;#7+7JFXf z>e!Kd&sOPO#bvcWs6ETy|DE0=q{Dj3IgB5fb;dk;>a#&AQ!e6cTHa28 zBDT_@!zUz{p1le`_wxGZ)FQ*==qaC4qTW2FWKFS*?~S|0d7W(g4trD(CdR~G>eL&2 zPz+SV9x~Y6?n z!tgIfL>P{;D*G0FCYJPYd@0IazY;}B*_N{S)RFhBZ|GiLrZmPQuU>Y&O`t>t@zQVS zDTd204#pst89XNy+k_#}tY}u~*(`pa_bb5NP#PgLEoF*VUvD}z$7v(e_%4~^$}89Y zBrxXiSr$m4{<*pAm_`uh5pyl0vg&E)8|7wn@x_lz1Y_&B;XAjDND|<1pULf#PbJ+o z-(t_jtCG&GENf8FWNG1Drxvd-;ha0FO3JC%cXoR2S)vmx?@!D{Kz#<)C^BXF;A!lf z{I;4M!Cgw`ftxpS~*g&M_-*70sz0n)j zhV(=5=Mu`wCm9W^^WJZuOfg{9EbBm$z?K2Q(?tHW(kLTYLEl7PQDJ3PMNdBr_IdcL>82$8t?3_aGGNN8e%Av;=WL?u4OvjO(=gkl zx!UmK!&l>bVyvDPx&&i>ZI)e`0KCc?lpkvWA}vaiZ8AIO)~Wf{ay?>xRah-M+4$k^(x7wkasaE;q}9w8}mk6qWE+cGHr=6mksy?5NpzC}#p4 z@~u5xlimdRu5zwoAPdgr1Od$`|DBH8^qqReT-v5itOLL70%?@B@UUut62l9}_{~ct zjQ%OpVL}Ctpwy7gAoqfN@08)#Y7xRQY@&}W<|hMU+oxa}dIpsnqA4wx87EdLS2oAZ zYMnwg--=E)OQJ@WMu)Q3CBaASIBOxbqCR!tO8@I%PN#eI&w7A=Bt%IbWn4LoIh*83 zYHfT#OC>VWu;faiLQcQ&V{wB;N_kZCzZ9)OVL;I|?={#=vcE!h z*=5o)Tp+9Li24o5x!#o4IrtaVhEoR-A0fVSepb0s$_36JI zbH`!ojE2DLZws?Xg-njwNiPX z^Y8FCDe2J=b9+B!B8U`;fu;Q|$nKq{<#+ZxqrP+em#~`iS_Q*Adq(*j1{H6aA1fjg zVT{EnhQcYjn4GlL6Bzhs{y4HE{vEOM{EXIW8t*P3wtCkcFr1_!T2BD+!9(id;;pF!B3?`8MZQGJb2&c5CUycGv&TK%U!q7yfPt3>0q)u!A=L)hIWsqT|v8 znxK7AaIvFpvwwN>*E63Wg~868rnWSvFMQ#3;tJ`z#Ai<#x0<#Wde&B`Y?59Qi)TSx z>1W>G=yu!s=zV8us*7T+t7G58adP%%RC*fZ5)&y%MEf1v;oKGH8Gvd;vLF6Rc?ltx zyz+p@a>(x%e$YUIfO56{LF)$O?CucJxJuwO-5-kFbxEf z7#l}8fe`o}{Q>4btT0IEZ~cJ1;k+Z6L_aNIWtcNFx|5E-7c5Z}&N1o^%K!esuPGm!RRr znhd1G?q7C?#10=D+M^%^iW`qL#DcHa6Ve&e*c|FIt626o&8vYJ4T2?>B zhMH&1j!)dZDdfc4?`~4-zz0ouLqLYX8|_*_$jHEy&zLW(OBzG7S!f)8zjD4zG(Zi? zCV*UKy+V)NFaw2hYT|{xAFnrw{j)jD$0%Si&u>eB!VL98f#zVFO)kjS!zOuw$2-Sl zWyt-_L!|-RUI5x!^R8{&^NmdTq|yX!7hL!A$xa95N9{O7_JkrHMIZ<(!X_WLI>m`MgE4@mjIf5QKlU{FLs@AvKo2fKUK9E-Pd0Za;<9E?cf_Z@XSE= zz9qcg%E374!Pe^sVv-3;y7^8mwGS;wOfjwyPkHe$9MU(B8MKU#pnjQ&kjo?T$ogRW zmK6@OAM__DS?|W0l&QIo*Ob?Ra9ZMm>pVNtYWdg0=tb|FK{=4ir(_SWnorpJd=6_g z+vAryV4~3O^VYSoq4n8Ky{MnfSzp>YX0nV11Mg;5K z^85CpFg$)ORQ@|jO|?F0vt4BG$dd!l19HKgt6xsB;UZOHk3m1jf`AeZWdXe7s?vr7 z9yM|2eKbsi;lqWpueG-ALA5`6`W$&oY}(d2uFu9tfx4ETa~YUco$u~KnVV!X*U>({ zeG1w9a+oKq7fR+bS8A=x^zbgz{;ca^9;M^`-FXZgypK@L--)f`6e7?^JQPK0E0x2& z8J39>n)T>G0r5Qaz6NL#a9+Lj`kVAyrp6;fTy~Q%uiR6$P%*83VI)Rlsp>jk zMNpXYY*ER@`9xfs!2a2rb^l>~2pfjDOX$kDI z%z;;L1U_|FurCcfQDwVWFvg%>w4G)Cy~ooPO1fGDRFbtXM?`-D%X!(8%7J(G^wj}XpU-;NG{hsRded40mG zj)0T?oX8;{oWqp4L-Fgdn4V$5AD*~4G;K1ktPKZxaPpyU%8Hq}?JQ<*-kLZRz+lV+ zJeC#U>AbtdqWieSB5#QcJ{>vUnuav5Z}ul6Uk``23(~FT6%U#&1>7%YH}jMWWF6`= zy)Nf0<<0ZwD7F|tVXZ}B`anWa0(<;y>)8qYQ>o8PE9at@GB!_5?}z7cTae-8H{e16 z)37f0%~YViLHpLWeO1m#8sDbTpC8}#mwN+4>)F_GuJiZ0>7ms~0pn0zEGE8Jo|dW4 zfOK8z*-}T-+kp>DAq5%HWesy*7U$%}g6NeAAHQ>+Y!bgI)xGn)`YDmu*9l)dWvhCv zgT1?{XIGVug*n%5IFI>Uc4j&F{L{Q#GrqG8uifLl(}RZUqK9Rnj^{m!!z?AX{Sb}v zE$y>*GAtCctc|gKIl(&?vA3&H!|kRbu!D;x_v?vz+b^8AlN86{pURu8nyxogn{z~h z&z(+gRcGZ^AqDf#!c<)LojE1PO^~D&>|N%ZUIBTPiHtkgdSUFC z2G(lU*~2?+tOk`)(iRL_J8Sqj?4cz;jl`2pNM|4XFBgnrU(1D6v_1q8c~9$_)IAEu zFR;yxuC@~qtCOnVP9Og0JRxPn%LeKxbn+&r?B|p~#ukELum$llVO@r-69Iwm-p~AF zKTUo>wbd&WmEY$|?0%zo=m|P2j2+1;L_L{Q$|y zs-vIkwCz7iU=Qw=w$`_lAyFfI*9M)nV{Sa|Jog9ulkgfVJBilxaLH2uZ=1C6d*3)U zU5yRxy>VERyxtjNRt=0F$ca~=k{ur6?F+Mw-26FWhAOL`8z*vsD`0}iKdhz_*M{< z?F$-+rIwB|B$9Wq7+>sfs-z2b+rBl;IZ~d?oShduY>{>PVrw(@9NGH&#r~?(xpVZQ zz)sbv9LV^gvQl|~`tt*8{0s|dLUr0_jGX@s9F_cooK5@w#?={VEsx9y3*5=I6fWdt zD6?FQM~=*XL8f+t1vpe=qTaH!%0owRSVi)qKh}f@{+Qke*1dq`sRx6u;GqHP44n6~ z>oX6KT4EMUQg>H~YeFw|t*c!&E8DqdGIf-DU2?yW{svO}Te0>wR%CcAvLq>=VwC z9abU4TuI?-ywtChqD*{M&rewU(n&aVT*GU%Y}`W!L21?i^m za`y%%{_T~X{cH~|p-%6%@J^U+>v|G-!${&r`&^xWLQatS*O*e zIu+x-{>$U0O+zKv|De^@PfiYh25JFNMS*Amt*RyKlhFIrp%(>s=h!;>KB2G1;&*Pr zs_#gjXH4!6zZMT4bd6z)vFd)mE2t0xT|Dr9mvegHW^T7!9qnjci6YKwJ)Vw$_dxdE zHh#BaKk-{(?Eazm)U3lp(`I*AN^_PpF}p<^W2qh4joU;&og8=5j#bP$eZ`icP3=>9 za?_WDJKg*XMVgG;8Fxh^-y$TeE%+WwN>#M!`umo3#LakkBCi>o2C%ODV|5*IpLE8l z{Uua4OLVRy#BoBQ$(Y54f;ZuTv>K#{W zJ+1adkKIAs9Z>sI#N(uFmv{!EFZ6Z4N?qp+v5(La)ui^ZoYzJBPl z%FXwAV_i*|^fHYWVy`8ChcxM2*@@ClIjELyq-+N>!E-U$NakKb*G2P5iBPw3l<93$ zKh1r}yVhQJEb(od5Itw!IRk#wDmReptXPD=f!k9qyYbB#?$~4RqYMZuJH~VL_AI(jm?yQu}(dpJesZYXQVNhsn(_Xp2>|V@tTRE!V=T zdwR1g)2?nJzUKF%#SgtMxWfeJ3&eSJnG)e;s%D`v6AE*gR*oll`%t4UJWK;zHAC#E zNl2XX{1?&oteTf`(+#7YdmrqZ>H>Ng9Pvkz>oDkc!Zj1|vvmK~i2z*w%u_IRy6AiJ z42hf1Ul<=DB=SUCi)?{ybAF@62qCejIO&(Kqx65{tv}F#7{CI$$L##pUx5D;2w0SF z9h?sSP3L|K3owzKfw~tQJby}aRNeyQEq(KC%gNtvP@X~sDA|dv;eRh+J_T$)9|%;m z{OwglC~E|QqKiVB6QmzI8E6TL%CXaRY-`@h*u@6g=Q<3YoFF0(wEq4O) z!)PXN0Qe6sBziSCpiRK5718~Uy5*UGJI@S|jdKQ-C^W zF-QxfZo4~_pNr#L#}VoLY;Ok&s-OI$bApSHYyAAyQB}>gM8dIhPv{|is$qaCbz%oJ zwtbCtds%7FwMWs!KBdt$QpoJ`kfDbML$C(Djp6(gNVCV?POh$j**Sjn+oTfXCE#Vy zGRN=Ipvqa>6+mxyF=J$h;B0JQYqySrvzu1Y>A$4^l9eGa-VP+=frW?j3slCI|mo7YQBE~3KMyouU0}=tqy?6thEW+LyiEdT(^KSl3FWV`gyAw*@H_{0M`>A*!nyt$aTYL3xtZx8Af&c@kW$q=eS9q|pXukkH&z`; zKq}&URTCjIt3{1q-T0{PHMUhC%j*w6hk4(1_kfLvMX3z3`MS@+ zuin^NH0m6K1^$4AJ?t`X&55Kg`|%f0e3FgxQ~?B-qzE>G^Uio#&IJ{L>$O+2o`y#@c;-m66IzP4OHF5njPri#j&~_4697K&qzhS6RKCFZ61d z4gznT$~J5LCLP8`O7?=Ny06_5b#sZ*pdydFGkD*V^k|_kFK>N*;@l z5YZB~={|TD=I8bY3-t|v)eT+tndFat9PmQ`bNg}aNzYH#d|w&C1q;M|6EmV=L|iet zl@C6od3CLvfN7EfDB#O_#Uus48xo>SeR<2c-hT^Aj>Mu7GumOz6T-qg1kCe>7f~uQ zZtds1$qu=_DHjNVo!`K!dsnaNKSTkJBp1y6W?`j#T&fCrH=~Q|=h$UMfe&;gtpjL0 zzRN(X)Cp!gjGl*yTM-t;14jp_o|DBEZ$9Z7%8>pQ5QLb)wEA5Ug%s2x2X#Giq}``m zInO9?;y+1=>5BD)?%e$b>^gU#UVlM%XM2!lm|7v*vW;_aclpEV{gYMQA7=OMEN9;! zh9*2B6H%dw;+IA_tQ9^lxy+FGX~)^DsMFFM`$oPf2AcUAH}G2Z4FHphCuMywZ9vWa z2P3oyWGObL{Svv{ql+E4Px7MFzoiWZUj-ERZ+;3>eOco*k!}6)cCXrGx<7%EbXTM; zMT2f~V=%24JV&0r$>J!$2qf4QDM?v+SyZW8u5jBa2a#SB7k{~6&<`M5^^%qWC0a5Q z?0qJSEA8-yh&&D$F|oKdvX0fU=}>vdcSXnHcp6|!p+sJzY*2yXb;^?=?xwBSfB~d$ z{cmz%Omlk4TTg9g2@y`8uHx@Yqt~{ z9!05|F)t3vt){5kG;XiRWQAn8{v=x3-EL-`J`yt!^Sp8OAL>MFu;5qUc?zU)(bUi= zp+NAn9i&M0GYwb|cG7h<>YaYA_^v0|vVS)wwM@Acl54AoQAnh6Fsghr*N5!_kIQ7k zGcFK3fH@VZLe#`jteDhtS!6t-(Ap=<)egw^Z2RNM&&Kn`8S^}Uu&x`Bmdpvg?YON< z$flbuO$t?8XJUh*26o2(nvc|+0TQepT5OV3fuEs!xauT91J?z(UX~6v9+Jx0O#50I zJ5i?NR$Qh#tY#$(3scB#37-qX%CMNeMA7&Ul$b8HesLd$&Q?{6uIrFq3vxM7Zd$kl zmOipN4SAu6bXtEdJUz4+5`4L6_XCQ#g|L``IS7F6V+G>;O zxBfhdnlenYk!ua;u6(mXyE~M7syP+aTpWyMfuG%4wF;U20b|_-i^q;vQFs(|PEx(X zwZr+d3;ZLLc}`H8dDwLita+>?)DF~o@kRuA1jrEG0Mux8bqFGs(HnU*9x+s>6tv1B^)v0>A4_UMOdL-5as;E9|!YhI&DMRUx#Al);XLK}^)rAjMM# z)NvGPm7}7Fy+|?Pot-$`=~&k5lD`WXSvKNE zP`K8%ZZ$-S5H{ivlAr7{9_$gaYQzIK2@R8ph(JAr=xhN?4lD~5wO2EW%`4Wg4#QpZE)uj$3rkQDGq962VkuHCh3Pw=vISlY9Ux)OOu z-dn&|BnEn8OQ>yG=2mAi0Oedx4aKfiNawj*>R3(MbzMQBx)`FC4z|Y4ZqR#h{#ojt zXZU@`P1&fCnYQmY5*eow(3cD~YQ(doirOZ!P53B=cV)RMRy{jQZ5uSg+aFPKnWv0B zW(t3XN#s1oW0ge}$FY;D%2nS=msN|dH;QAvJ$hDtPoLKDTlQw^dDBMG*}}(H&fg(78)@n*6=~I# zAG9x|hwkN2t!g0HrEDI88QBu?W%)D8TAmGpUJPzCEUZAsiZ;m~Tx=|Vkr{KWh8r zO&L7&$VCWlT`9XM+bz|G*$18JOto*&J1<`IrYS~K>qnGl6LN+_1MV*B2=R?p1C?34 zQp+~)Mn*iXt0h$j(6+e)7Ol$B>PLBI{SP<}sbAxDN6g!N>%Q&n<#TXnYo1c*3)vQV zB5=>_wAnZKArQZZ(8qH5>lBfkez7zkig0ncqleP-tSzdfa=FVjn-0=_;^ISKZidb` z)jE&wU4CM=dE>2Ky1Ax)&?8_O`E&)tOKWiHppxe;II=&Uc{u&Fizr)64>{-5vPfTf z#5v9Rtsl@y%g>h4i{>EJPZEXK#qBvl##Ho;`_LMB=Y^jvMOK}Rt|F(s7%(*4w|%I# z^nEI|^d1TKpjBC{drs)UpIv&n<)J<$MX{Q)=?1Nu7&|XIp)|{rBHfI_l_v;Y=20%1 z%*0l)Y9-31(DeFa3!Tu)63y7`Av!EtIOklRI4O>;tUC7uS`rquTnm!u&&VUP&6xB9 z9s0)gF*&r&>mBZ1El++tv z9J!RoVdm5A}RI!grewD;7<^ZUMRTt)dIJ+Htoxb*50Ut9D0O?RPsJr>t5w5P9>R^lDd zUSNb9OvgAlbl>4Q>Reeb59WNP7|E0vl(QF_*`RZ@#ke(-S5ZC z6Zq)y_r;1@PK$1SY!t^;-cid6L>Go!5xgmv7lKh(RKgP^b$cHymO77d;F|~rBV7ji zKRLyOcsHcr#5OG2QSa%+A9iA_oS#bz1~jXz8Z_~td0gr!4;*w0 z=d*pm7v$LiJXs+V7D32C+d^9^jxGK3xZLZPPrpuvx{RqpLXE`hhS%y4N|$Tuo3&97 zj6%;tldck`1utllD)?O(xT!)BDj^fyJrZ;0Z5FIOym2i%j$%3)1tJM4=A{D5lr=gv zi#M~XqB^)%!;lr*H}Sm2;PfF|Z@XT$x{%7hSW{n$(j81v7XX692z>IT7R?3rkHT8# zo*tZp6?5?fMze^|!6?F5YhNM>DU2*0GGoj6eUtlIS>C#cu^5M2y3C8N+xHa5(=r#y zMEt5Dfp)0)_muMN>lFVolSN^MszC;uXHP+za&BBaF5+m}ifoB!sQ=ygGKaU%*YYu> z<b#dKS@@=l@j=u5`JIZa2K61e=sn)&3QOK^7422mMd44=$ZjGKSDcc$5 z;-XI-6t(?$ymN~-7w6>Y8x-A2VBSOVEAN=ABO`7uTF=6J5Gq}qBfCACBLjrfPQ@$~ z4GA9)t?nI5z=3+&JKkySG!f2BuixZ?Om_rVcoQ-z1E;p5>g((hO`JV>09yHXRd;1S zeQoU+viJ#|S-Rmld5+YsFwu);{W{~MXM{}5dHi#G-SM#))wXs|8Y0X4ANDi5(2z7@2*}R= zup#qs+GuDd-_%7J!c}INSpWJ!vX*}|J9|iNyqf+e7fEA2;)Ex$og)$QqO7{Qlx)p=dMvl_u?JFR!2;XtIK9 z;&VV7+46akKXv`R|BOW+(=EGnK$HOY*u*=tMd_Vg{EII}0h&DAOB058WnZ=?Fjnsl zSO-cv*ChY2F2JqgheE#EjqZx({_4Rz6j2I&Ilh1A!7tNk5_wKNcS#V8qdswUxpb0n zE3wJ0sJT~-{R0oxe1JBwdEH0`M^#C(C5d#)BMi5HE9E(zc_4119e!6R=6~J>iBc|j z6$?Rgw_N6;P=CC;Ro!D?X5>|_sKmhf#x{*RODL-Vnmva#Z2hA z*(ZGs-Ki^#W&G@HCcC|s?fd{pF%I1AKV}4pQcyarZNdE>9MO?6%B=@|d(n&!dRiNm z`rAp8>pa7DVdoim&BiZp`?Tf{3VxR-i7s_y8wp40Tj&2BCg`ub1*_P>{q4=)nU+T z8F|$+X1TDChgkS)%eu~Us?kw?T%gvyQ#8>DhE;Sr(~CFipMNpI3~yycuQVPnPY92W z3Zh08I#^&-OdN4ENh18u#Sknm1f-!%s7EfDhPgkB#~0@;yS6vvyFR_q_+g`vJS`hW z`Sp=LpvOG{h%rrCuWaCH5U^5#)hrm&Q`LstlV{PR@YrwN#Nj$(w|%J=|K}=0@aI-N zB!OCTmuU73z=774rI{+Bh$aG9^P@pydI*d`H&lfjEasX%q0*DS& zOs@yxxH}%0wRy(S-+-Jm3jG04mDnCWv(6EirwYu35VRdyRMJbVpzqOjVa{I2UiU!A zYmX%vglQDL{|_k^#+YKmfAG-uCom!z1nMeA*%w&#f>%J~cQE0y)WrLN*Q609%qB|$ zj4)GxgyAkv5u@6^-vOfi#SY7*U8Ej6Zf~Gj(R~V>7Bzvo99WsPWGz;?d8$?o>N5PN zmjKHq0&V6-RlCXwGu3&jGXDA?L80pNoT;-+l|t&#VxwJO4M=5%4pSvD7P9}dt-+8k zFyfYH6SXx%L!+7hhr0xL=XOZ3pKFFyu+7Ip0#^P$uv-|*K8XHk^Di9j2P6;>%>uDk zf5XgU7&y)i{2Kp@A#|Dm9p6SzSN!V@wFlD#Ttv3N-2Vqq|Aw{!Q(K6Ip4$HnGePtK zh3`nda{s$%#Xu;^W7VP={qH6#xM2if(o^%=|2{k!Mw&iD8Y=!P+PH`W7rvEQ9sU~` zPvz($q2s_;Bl7?t$b5kiXYTlp4hkc}Wtk zts72ct6rA7EatVR_ubS1M%4e;eCYudyzc|x@??zl!YmRhj zX68%lhw<+wM2Uq{qgX}9jeOay8`o8{9i5I)W?E$R8yM+q0+*dsU>zgV@%vZ64G;`* zs4zGxt^IFFg)7qWYyk62RhVrl)H0ITx-gO?6<~xWpimw&=06~^#a;@v1fTo0ugUVu zSPA95uqWKVzvC%P{1?6zj?qpQ|LRFKj}wCB*q|ob`trX%g0OE>#ei4h)RFeOiWd{t zA(u_Lp;PTj99Z)HLzQzya%aTzbvQJ-Ss5LuQ8pftIjxTE|JMT$7GTuS^epE<-JU)! z`TNUSk;h}FH_g`hep`d-Ee4h_F}SJcr2OfGEG|?6m?#cn zn;J4R^cYl$HlXz3*4Gvox&jYDFzVB+;p2}AyVHb0z^-$yzsalqGl}XaGm=kkPkYnc zL0pL4KLdJLlaKR8R@e?JZC+I7iKM<^EzXNNKz4Rgr2h=~M{S*Fv-QZ6K;;a~yk^zj zu?qq#?q(-e!1!srxP!S0V{+6tGJv=G?ScPNqW7xbw*CF}3EPR|-8%4ENb+PzY%HY8 zbFMKR+VTK?Q@vNJUT&bpsQ{@o_5IhgUn zOrb35zEHkAP}SlgxiYEJXI%g%3eqU`96`<{V1--+JU1TVJnWNa0Y*>Lnxu^T?Pn9% z`zeQ^alj|_yYMJb-IG?P6YQFcV21mB0w&6^c7hSG+bROyPLQWzDpi8$IKZVA=V26P zMg_y{j*-3|Lp`zrDA?Dl5-0@Nbt*+~0ex(}?zTgTmNk-R|Gep@X%qN#M8kRxfkl~K zxzs*icp>2HR?VD%_Cr!42Xq+=eNs!Sfj?hmti0l;qWJN+kkdKwj4Ny2Mr*}PUSR;+U7 zK_1~~+j|KNiIjoMi0AUVQFX!C*FaNS_6sl-62w5(a0_nb@{|$GVD&59({&g=g@E2k z39PFy!Z$qNP@YFpJ)tRnp(nCeQ7p&I<1kl~?F^u*KN$+74Huhdzy&63h+5|7X#RVc z9cpbi=+AM(49W_OWoN;U^l1n{d0e+}DlKgjwKgx!=Jq^(G|%_>eFRoL4U-C#%5fx| za;(gLfafwwh#8(A{&oSE&;4>%NY!rY_=D}oOQe}&&Vud~X1 z`z4J^Z7ta)m)!{SQU2aCo8VSX5Fg3g@~3l9d8uFytUQFj0+NaxXuS#FiTj;e_at@b zvPnJ-xFws2G|zA&umiI3>xzwsAg|Z+ju`2wXEZ+yicgUvjVg*adL{Ji>3WpJXw=jl z{rp)LCh3q)LfTHz$`VQo%+gywlyBz_*o7~iefCf=LD;MkwaT!Iv@;;D8>Q#|c%GWT z*yoU%Y87$}+KooR_a};e;4!b+pNpP!gHCmWxX<|t=wv-;o50U-A64sW%t&x=W?nvX zU9gOb;gUFjmOraxWD(r^TB1z&F!m;$2kap_q$zU0HtRPdLo?r{8K$GerlL4?`Uoq3 zOzWi$VR7VOa<@+~F*Lu)-ZRQ}KLNl@Q4gS z=&JXVA6HYBoYZf1w}|Q8eu8z9!e>=I4bD^yHgsikKY*+6W~th^UKyi8d~{zTwbMy* zJ}f+^^#Ygift}~S?;e|S`El0w5=?@aMqB8QSrRWcg+4U11}T)QSm&FKoRzGC4*7#f z!DvQjbY~>*5d>D1NuIV4szVB&XCeNNx*i;Squq@sn-8p_wUXGZ;yn4H^=E2&Gh&Qq z4*@}2DepJK%+<_QFQI09`bm_d3Hi{%h%>BfrT~$agZV$NC?>>46vcDNK-5z^uKT9K zXjzg#=WKXSbD943KHDxoCau6h1lXk?2yL=i#XLxAQaWfdW7XMLwFybls2~vns4&9& zNm>Z6$vuc=^i$|_wT(ZA1!&yAm&P*|9ftDYPoVTsFEQ_7_qD;r?I%m1kTLGAN8G2T z3fS;;JoeiprlicG^DHE>p|INHeqzK<7~@$_!dt8Db7soKmS&x|iVLeP-}4Ao9PLaN zy95?NG1={hq>z(hWJq-cFMS0I_x5VJV|Yw6zAH%HABT&Rf{uMq1i2LHeJ1Mf^psNj ztK^y#IWKyeA7CrGiS-e9sQv!+(`pbh??HNtqqb|UZbYR5UkyS-BuSxO<7xe%;?pCIr*lh07dgHrh}zU#StP6`=|mi zQcl|(?nZd>Bn15|q(LjxV$PDj=b5VwJwBR%g_cJdzmr!T#$*xjR99#_kw6I8OZdrk zPMl(K8wx$k`K&%)Yc_fq_gJbfuac!uWhv-tMQF+8-HMU{XQKqrIdIVYwqQ!9j8;b1 zXSHGkzwR&aikDB^y9D!8ZH!iYL)dJ3_CX`-Q5xz+81LlEZ_!Y=PuRDy!8Ql=6MER{ z87E{*S+zqNc5}*sS?~4N2~*4B>p?P`(JMPq{}{qsXQ8AcXq(5u`X?p?3ffD?VIJr4 z*{Fy;wVY#z_Um`8Qjx5{K>s%s&BC0d*)!iph&dDxdHXA@DJrK6M-?ptpS@f0#mEKI^)4eV{ZNpzhxUMx-nBN1dnX=cUW25Rd#^wZ1H^PUqaoG)*5@IQupFiNw zrk$tH_1+7AT~xFwu?4^OAT1r0HgIBQq#~1k=1KMgU2$zI+afKAE3G#tEg}*Xb6OuG zJBHFp8sTX5!YVE^@Qq(rv2=?)OLOSbC*{j-s3gd&A^cf87UM=G<_qgl-Q_vm<4Ft9 zg|RW>t?G1*KZug~;Cp3LGQxOlHH7x3WFA8xwdBTPB@^;Kpw zzHVb39!W3cGuHl*W09&vFpglAQy;ghK$=n%hyKBUQ;RC&zERw#U70wCaIMo8)mDvEQ;PrT2yh3!VbJnAcT~wB7}<(jQDdgGYQW+_D=V%t43P3ux9Q6YhzT(j#86OKWe$eWGD0CLB@hHBCns7hpiLg z>%Hg}9nzN9$Q;)l!1}xUL%2FNSxOk)*dtvlUf3^0Uhbh%##hfw_Vh%;rrvgptzKL4#DA7PUM-3pl?O z>Ihi$)x<>gP~3gC3{!is5s657u^|*s$Py)sO_Y1WXKOSpb$PMYUT*Rc%ZJ5;3F(1S zg)o}7p+4!79G1tk<(4-hSd#|L@1WEU;Z;{wrjvpdqf2C5Iz*Vs&T@~x-4j!FW*JRp z(Jg{Us&oR-fY+Zf{TNrOMfyk*l#1h7n;`MrOdU&!sLi2EMiJ8Ak&sTX8x3;aDtD&G z5f;DVmuapF+@@gFXDCNzny%swaZrMFmwR3zqpwq;7(Cd7J#XX=i_UzDOOeKCbv_cn zC_i{^@EQTe@;R_Bd^ID!=)ISIXq0KLi3FD`;r2Abtz1`Pd<7%}r$JW(*(w8ClDh;R z_&F^vm%$6Ml6UCaQvgDmB@Q!`duQ!}V&&Gz;BR8ToQtH9!z zR?nyX_>>P*EX6MbT91OB0Uvk8YguD#X>8eSsuUSheMxH-!ee5-)Bu{)%3N>9oNu__ zn(vF%scG==XRUxt<>DyJ0;vGLOWhHx@1A{)Uo2>eB!8zf?z!|?t{_pRtRbBDRO{@x zp&|sn(^^GY?Fk1~vhYlYeX6t8Z(d6aE-B-!4#;KJ5FWP~i-=eRR4x=5itSzi zv|PwGB=womh_xdoR^~}JLPASp9mBIGyv07;I8+i1-_PZQI^0-XCi z3ph%NEc!6=a8EJHwxh?s!4}?obvBE$C5QG8jY?qrtjjfEB_YPY7#)sJHvbJ@jbQM7 z#ZYy?A&;MyVIS{8;~Bink0cE0s#?z0{3w@}#lD$Uv4?2zqCc9_9GZQ(vZ^f>_9h%R zuO`L%zJ}P@3A8HzqL+^1qYviommNsE{V~w!89ka`|R#7zaUou^=8D=!` zt=;A|q39-mXOaJidp9n21T_Mg%*)m?O(SwHFH-bp>{h~lzIZ^H9rXA}ZJZhto+RL$ zEtsf@^Tt#MwMB@fvaJQsY?S)N$-L=?)^A1xg-ut@~A>pWD1N0e3=FM_SuU}med;hR^wyww|T)T zHbVQBoqx#uxTak-4_D`rM#Sfd-i)@|{7;O@i6N?UyV2MGny3KqYXs)rIs)mf{^KqH zoA^@#c$^Sw{NH~CgX!b|9U88YZHN3tnIPaV)A;`i*kM>*HVk(9KbO(kq)Q6K#Znp{ zc>WhVPn6O}^7Sxt83Za+5^2rp#G zzgKI9nf18fpZ|6ImWqMl?oUF*1OCI73R=(}K}^`P2Z5=l4Zu~qNoqHuWWNGe7TiVP zv;6~WdE;ha@Vs$ZV4Dlf=GpRIJp@>WS*af=(c^N#u#zcYUEDqgds3B z9G5C*vmV;z5f$15Da=DRPd;~7?EGZCe;D|9Z-Sm~v?OSN+UKr=lQ$=T%UVk{^G(Lc zdm>n;-77a|83gsW62NI+-A1i7fG)0QF$Dn4kN5Gm@`Lc(7?ouE~LwSzgdb zGY{#NKEpUl1>Xl1ls=Z{y2mzw={wA@F-)918%}RO&XL`~_;vt<3mmDjjD}f$m#Qmg zUGusuJ^C}@2q4?WOU-AhIq(2lF6a4SMz_61O{y4gmkkRjn1|0N9c{ z0NwiCS}jiS#}b3V?_v$WGAt2y=qncJ|z5tyS=bG*C!?ri}^gaOos7*Ozjzw=?~G*9?V z`}_sVu=T@IJb7k5b-h?3*-RN_c63gq6uIRlMjHpq2KB0#lA;=QKH}0y6wXXub_M)~ zck28NxnIw2?|+ZK^BDgbeDrfw{c7)Lw#Ze$m|x~u?v==upXBTDGrzHI!TEMSk7>RM zskyh0pRw3H$^zr1vNQ4M+1o)p~li?$JwDbW*_Dz zp+(|!>wO6xAN?;He%_pIo1n&`$cvlu=e2{@%iN?Tmoor&@TCXdSq~C8J%_y?Nqjqn<_s0fe2IE$iMqxvU~bXsrC#d9dS z&NaxJ?4$Y6j))zk2hzTZN3`Ehv7nIKUn_y3IV%;~#X7Gb7tdDOW`BJplarkQeT8@J zFgNFvbD%&zFnS&lg7@GK&6;U{ce?xon{L^HpwHo#CZ8cCfX2QSN`OP?VH%C5U2OMj z%*HZlaoz>pw0$reZ3EVB@c~1Fp#6eYeOUc3ER0LIqOSoq`RENQCz^B*=s(Q6n3L+* z*1mi!@s&g^7uNVn5kSIiEru3~7eZkaRs+p@Toy&O?b&KW*u*W558gi-LXoN2DVa34 zB|UevD2aN9!!Unzfc5IERp&TcrN_5K*{f+c_$}N&oQ;6+pfeQFB@r0@ud?OpL^h}4 z!=ngV_=7!cJA$ZvcHql;OiKp3JebyzkGZ4g+(z(=D;e+@gDJ#b(un&DH(58$ zzSlyHc<*y$E;<_MI$NbbN;M`Cu8;B-3(L3Mx1Wp%X>f~1#yl8ikg58niD%rt`Ym}1D@~_9Ks3zCvRXByUbe>g(B*EY_U%@S+#3{4`JLR2PA9iV%~Tf z@uf>Jjewfb1H$;o2njpu6pw+LD35iUgu}?A zHCG~}iF7UUeNRDHgJ+SwgC$SB+qwd!MRH#O3ai*7`$x1R=SHM-1VfXxA|6{t{lm>Y zfvh&nyZGOruwiz#!5XjEe~)EyKVA)F^A*ISj2OHdycZ-P+lnRU0nowxlrL-6lC&$8 z^>Nf7d04+g^3Q_G)N{h<(D)voRjpu@E9f?&t>NW=FO&5gWgb5r^VtE0TV~Ja}l!l@#^B+UK|*)ZI`{ zHZ|1xlqes{X^0CVE}B@oVSAKtt1{S%*U#vZC$aLs{;*V$n@FFe>UqJaCLP%}VJ$;_ zUqrtxV5H=Oo!R!yzRz^^baQw#9kf06XA0=jrPuQxIi%tXz9QP7DFNe4TH@x?HI&BF z`+^GI)NgPGbB~DX;Gh<(AtrdvLng)xQ=VwpIwC&@j3fJ{CI^K!x4@_f6ibc_ z89Ku<2N)vAlr-;|^;;p%1`aPGl!*jPgxc%74c|mkhQYduyYM~moUJ>sO0I8v5A z4BekJSKES(m`EJ9Dzn23i=l25JDbIFCl>}~7$;NCX0(WR4A>yQv0*65Y!^p#cIvrA&~4TYB%eDcXzTf;r;#AaWFfT`0J7gM6Dai8f7T7u7nPHRd?|AY#*0gNE0wMB18JGc*>W;2U?7Zjw z!H)*_0RXz>U0R8^z$xF(nDUB!kP;q6(!Tx#dmoGJiR}vuXhVdB+xL&5bv$lozhr7H zwUlYls7HN)&!b40m+v-Dj+LNoXOsc;RzLaq=a^h+rtypKQM4^0R!sp*y$uhQGM)If zdg}3~{erJL;SzUW(mgb+eY8)sf3)6x4++LI!k8%VyYl^=-T$Jd)$lM|8s$QItqY#t zJn1qLMB!qz>*x1WYoanFSldhOm*h8zpU3Vt84dcJ>QaQxtgm+f6r8_s}X7&G-pSd7ogTC6mmq_Ex&ILR7(5@J;de!4VupKGzPU97v=KJT+3 zS~*kyn49uuY-;Jd=+Hn~;0C(bDH8R~82f42q2B6sqEL#lcvBpt)$WtVY{PZy*nrP3 z?=jI6xz$`UY)ZkAsX#k&zaJ}8?aHF36v;pCG^VC1dwdGK$9#%Xe0){RG6T+4lbp|b zpZWy?C*FCDUueFl-2jvF*yj`e+RzrDD z^H(ZCz}{{gc&^yFig;9vcN1!~A@C*oDrVo0@7=w5T4Fn`!Xx*Mfq^u~M2LMFtLujs zJ{9q{c&BKB{X4O8A}Y_cl^R7{I;m%Hb7 zo{vIsMr89Mx4LSngCZJQXhV`zeQ#smeUyT>R->1DV7P6#1Gjz0;JI4Ay+oFHPwxBY z90r=f9^Up{>F6GFC$|omwqt5u1(t=gmoS#Ftmpk0+UN{#y&9AKCyO`Ip2J;&cdhwgMb*8ZtUvImI zrr}rOdB>kRKiUQS^z_=v1)HT5H#8o{7TLk^%Vas-W~$#V9;V97Op2#taq*B`!->3QG3UAFrOt(3z438J~#}yJQHSFb^Icl$LB9>cjTWHEqa3#Rgw#lBhJuhKx(SIMT8+lxh9v z)1=U>qofoXSZW`AwMfLAjJP)7>mpXkIsRzd!QJsG;)sthF!Ze*EO`F87a~>!hrw3` zC~`#%S@NdQv#L#no^IajH<$eqh?u`WZan$)NSZ17H2pQ+Q4X5wuMZP9Hu`P4$xT-|t~m`=-D&kwDt^_gtD4 zJyrVE5Lz<&uk0~UbtXPqsqQ>5BK2m=5aplmiifaBm#o@&Ia$lmOGA&gF0&xMhFt8% zFR5xYLMQHj&O=z2RAuvZnq|Z~`s04FR_ju;lPE3quzs!S;!U)1WnC)m^ZWYGh%79E z^g=;R538pXFB;$x3)w*<+wP=N`4~Tw3K1pD5^tmcv>6lY($PZk0_7N%%f*{sA1R!@ zaqGq=Ak02%3U$rEbHZCA7jl^DJ#BHA?gJBXM^g9Z--3ymYnYgdWZ-pS$@_Sz$q8h` zNW9W$)k^kbK+e;hcmehaJnYLz)zc+oQ7KeN(K<7nYL~k+`N<@R`6&Qj@#cxNd@SAF zgS$RCYEkbd*XI#0nBtja7Nn-c1{Ea6JU3$a83N02Q}J%zcV%v&KPn3DCez3lTkJL= zSexi)O$2?}>jA=+S$QR?w<3yQ&QUCw-D_nj@)NFm3ay&_bdrX4gZ|Xo#gbcgKuv~8 z0%#kxFZN~`H5UxVa%wrPnZJv^bFWrDlO0FFYPH5g( zB6EPcr2k@JKDkOry?kIvv=j<)3!xW&f{WVlLMq(XMFes-!rx=L(X7Wf4aORDjU|xr zcB2rk5(t&ZL zEzH_k+bO?(-ifMBMsTb3@#5H|ewuk)Yuc)J;E>2$Y}LWyT~8Y*{8rthqT3?2{HHQ~ z2k$122O&ZlHvbKnouCVSs9$C6ayHKFr6Kyy?x#bF4U^((sMRihu?*%iB|?DI4$628 z4*8WFbj&L8=0~WO` zyawXim!Sh-PEvfGkjs*_$7wcuwN7a)GjIP`0gJO+1B&P~$U8yXD#dM(LI2tRNTH_I z35aC*tD-)KP5wrSNQGdb47)7U(Kpwt#jby=K%Vp?Z1z)coIz*hr|J#!J!=zjtf#_C z`T@|72TYsSKc-6K?ts`bKAIHNIdpbQ5sjpB!eA6EDijBBRc>y)JYEcIOT**3O37AG zs=|fiBVkjBfKL7w0GxpND<*atuD$_Q+MQivIiq<(G0WdR^W8~eJ#yts`)3=0T+fB% zp3>}t#h>Oj5n?x%>tMSxmM6OXo@GClm%j0mqU#>hay{c?(4BX`qW+pY7?0cvAPvaK zB3@7=2JzZpOK2Vasd%Cie8B^~Z>*QC?PYH)bEAeIIyRTJ1d40N&cU9^m z(d>~aAocCAq6=lq4e0iuMqd)A*$EC_68k73=`_bNU!l!tved^dw^{?`PtE%M&OMMy zp5eb0_8xDbdF)jzbgjv2-yq61Cqb$T6haAaxL2E=U^ z^i8C#`74oA#zbL;92O!G+GA6^Yy@-I*h>Q~JbL|yJ0 z{+lCM5L&^o5Y#Q_IVsINNOPwTav^e&28je1GtL8U6fi)3^b3GkaTRfQ-zy|wVX6~8QOdZXPZ|`A`eWRqLl`t$uX&%B6Ml*qlF(Rt zsZy%?YdqI#sgqk;HK=4$(!;6^ANIBoyoDpQ5`pleD}ze-|CVxu6h>&%D=&8azl%Gf zGgwi|`r}vHP>Mfq0;`Vh8OVQr>&1HiTm}!}jylwRki>bhf!R-`|9_XF#}@_?9!#kF zV_uQ0A3ZVw|NSo&W(2KR3bPxm_4qrnocgyb0hicYtZt zMD~re#GevNB8W*Ml!2+{J@`t@d9ZolBvp?}V04W{8$$XG2VvUaG_R9GLJ_V!DnP3w zX^;Gk9}yw|633H1*SEWS6C$vOB4vp;AU_A-!}kFA>UW>L08Re3;Gegf5Lyj9zEgm% zdBK#gj4YS1p?DEW3j?4Uh;WwwrEXqMN6Z)E3t})eZw)5Hmn+cq-`yI`F-(3kWHN%* z4)l_TfI#I9((|WkFetrt3Eg8zl#G@t-+hSf5VWgZ0rAv0@g7gI%aBDQH!J+wu-0^! zE^q??XU&gL9Dy;P=~|WH+e3g)N_>#eUV_o^3`H86hlBYx;=d9c@1FyXnKxMsQTDlWC_no9E5<*+J-^ zmqx^F0MM7HAwX}MNWzn!0g&SmjGE#jbOL$&i!811oY@1wQEolXKpUvuGBvW*WGIeKhz(6s*i`4Xc_U@7bA+CawZ-#c zz;*{lEI1D4O9Zv$@Y%$=fRWbC{-8B=__S@8g|P}yhX+Bd?^25&LD$IQ={ki?EmN4w zA_o|x6Dm4`r!BwTSd+*)D4N>j<49ToTHGJ)3NZGk;K0$6;+2O#H zd&O&_$)wqFQ}F2*dkUPB#_Y5$u{=J+SnwSB_pG&2l;6(*Ue32NjXX!2iCz+JbLYj$ zJs2No`3THLp%ig1aT$qvg0d-b>Zn-ev=``&#$oz?g4%B*yVV>Rh z%vZx7d9y55^G=n~ZKnWow2-?ERMjq5IfkNSYDySUaN|dS5akMneh$7v*h({b_tYhv zwT8c4H>Hr0C_C-*fhFE3 zKxq^70MPa8r>`Da!$?mi6iA61vm*TCF=df4p8zm(2=;|QV;iyBy>uL4N|&#{%DPW! zUrKpU+k-&YS&#@%aYXitKkSlfHN#h*HP18EX}&h^jj_eVLW_$D0JipgNv=S=fo$b* z1NA!9Hm1KTC@R~4NT|O|)wRlc)%Z{He87gp=Qy{^72QKd!HRSN0$esp{EJf%bV`TK zcRV~Jve)XMAN>-#C>BPX{e#YLgLY9NOlSy9-|8ZDSP#F{FT7)qu{N%llufr zrqGN~zabt&b$|08(J$wNFVPB`VNa~(q=O!731GRl)2zyI$cQXd6i(NrW@v~~Wmaop z2BQJ6iKo4qq@V~Tl%Qy*%+5>)Bg@!1U}-W5Y$V$b!9)1O;(5SA-pfi;_|PXyq6yt9 znly4#urg1VX=(=R5xQ#QPeG{UXOZFOpKm0_dv5}B<=nAcr< z-D#9w`<<$}1MJs?z+)XWN*c$+I2L1GcFFw$^hn{6uT9vNgBs>Pq?TV2QNg1~4cbJQ zeeiB(jxygg72Q4bTwsaBkUHj0n^1;gH&jQ#~U zNF~eB6_ZnWFuYP-$Re~|^#Q}qaJC;lQjI2Gdoe>VEB1c(z5mr=Uc1K9x*E*~AOY79 zA(F03HoLBnKZW42&s!%FbeDXzp9Cl4N7HO2>ZmJ4@|cWL5mbVcO33S&+ zQmdbNKaCWGwz?W%^Cb6Hs{gkpFo?$~Qde3%$~n7K1hG8$So*F5J^3_S`tf+mMm$pw zAtSHX(@_fhBQ=76Y`|fea=c&S!hN(o6ODP8DG7XZwr0P)BAn*$l5wQ%bHxL3wCEbu@kTAx{|e?3c`55W z5|X?>;RRer%OBL{FZ1mRGyglSD#e8iKsY%d|JLd<1?P0h9tmmB=0QVDMRR}V&mM-Q zBpWLDFfC&JCeP{$w5s<1ifRG{DcfTL6r6`yoyrMJG lHE>ym=zsk0#)=q@B^jmX=T%#=72&{t%JLd=wK5iA{}0d|P!s?F literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/7-current-shuffle-total.png b/website/www/site/static/images/blog/scaling-streaming-workload/7-current-shuffle-total.png new file mode 100644 index 0000000000000000000000000000000000000000..8161f6f6ff32fcb60a17898d63b3865e4c7362b7 GIT binary patch literal 5101 zcmZWt2|Scr*dI%_OCe!o8QYL;m=J?$>?Ve3j4iT^#=d18dzM6@#g^=)43T}yPIe(% zq)27UPGls$*S+_<_x`?be!ny4dC#+*^E~JL&-1=u2zPpu$9PxpoW}_bv2U1jURXQhH9d_ z^bB^zZ%K+)D}nWnsR$+CBA>F;(dn>2RB{{hL2q7xDzxw3N~Ep`My>iX1XIOk(b{=Z zy<}wcrmHXlO;PuP<1V1~SRj^SjA&K9xU3B0C(M2{J-e!ffq8AHCud@b+?iklZYgji z_s0m4;ZsUuPf+6v7dd3{a1OZ}@cfH6WRqpZWtFJ4nYg{VaS?7tF{&{)=+x-=Ak(pJ zpx6X;yLf(ohH^R^I=Yw2sF#hbgtoiP%+q(dxoQ1a34LJHVi>_zsjLK2F3!gPG=A3n z?r3}h#En0fe?s8+I6ReTYdhQ7x>>t)d_3!Oe0xjvIC|_kHhs0FgE& znxh>s7?1=&Gk~b>ID=>biVA2vKm&nj9>sv@fu0>`TDjE!a1V27{-IS)1l5f-(P*GI zw)Z9yJ$#%zeVsJyp97}Go$=V3ljfz#6YTOD{( z6LRwP^+Ld40RaIr0T*RFy&Yk4%F4_ zxAK3Pe=DlOP8R;3K>QWuKSzNuBbiiT|D+7blyxdt8w6rKjn-1flc?6R=pS%sbHoV5 zDoZ}55#k1QMk@=(UZHY0RX#vPv%sr<3O@`_jpUY%qQV>Tf)QQtWF=quPrC|((+Piwt{Eq zBZY@!S-oOBwX)VQezci$q0CCu2@F(@`#NJ&GkD!x3ncC3-#?NX55B*or1>mnxYwjh z)qRev-+96xvO8{8@pC4)|7JIzSnz7OF6Lfg?Jxe|RXOLG#(KAkUZ|hx9X4T|p0(Op z86&mb#wys35I{FrKdi)fk^ zj?>&W;>xl>_+fb~y>r#!cE4r4K*(l<=q2Y7EmI5PISaS)&XJ0=kJVS0FWYx9PkA(~ z!6gv^>sA(i<0i(Vvq3A;^}7?6^N;NlAC8+^N-G#O?sp4>_xKE{nYlZJZ?@4Bx4-qm zy*{^^~8*IY}km>eAL47s~iIla>eJKWN>ciL8ToiO0`iRbJ3H0{|UZFb!+ z~A2hc7T0eFZsT68+M(nY=(iE83YP z&6^+nCh#HA$A>#c*xXg04wg%0Is*LRhg%=3!;gPoN^ff4`u%g(lVTIH)yedJ)R053 zKX^XH?1iYKwQ&MusW>woS;~&5rsYHV9n@70BKl>dew2(%<86g9c3Xy@|$F5(iKJ3ixXe&!8~%vaOsL3 zp?cTwBT9kzQU$Q3GG6a^ z3Z&F~DeumcCoTOl4Lw!Pqf!Z_FH4(0$Y-aRUPBKmePs?l4OK@97M?-w+e$tQ*qDLd zeiZ2IU9|_%CQSgd5%+ASyl|_Vi%E6wc_3JiM5!#zG&R_j&8J%4HJs^v%JWgK?#y{b zgFPTpkqa!LImsXNW{i-OHy@^->vJjp{xRyVIspZdcYjF#zPgQf;>7)WC(<3wc-DJr z4@g*UjZxjVNBWU)l(|#hZED=L^XCUuHdOnE+`6TDn2U<=lfd5YNHRi#Yi7fK<+_;L z>PU&qnW(Y!c}0LJ+x@Uwc=vq)f#kS$;H+>iMClPK7-10FETusijHnuHw9UI__fjE`SYEB-#n6!Lmd;r`(o?!2iH;-#2lSM zoWZrF*6Kz*QdZTf`S_@;$H%%{8Yy1?NxK(;S2>B_Z?KIW6FU8th6R^yaJ43EkI~zP zvgeEshY5u3+nx6rvlN>&U_9=VETuImac@-HY^9kaKTBll+dTK=7qH1E?nXOGZK`hS z?mR~Lx!hh%{(=@g-#pB`+Yf8$Azyu_rOsiJA|*MB~ovb)0vUpf27C}UR@4H4UL=0YN^uF6%DtG-{Zq_6_B zY_=lJ-8LNS}>aUezz~G#srrZ1jC@ zi&}g9+TKX$lS$S&*{*QVUcS2fhpOJ)2t%e9DS0e&OC!B+^rlPoOX^u=7-0*vEl z=^FSQKTVd~YlEjZTi=;-RTa|@NV z&t2+FSR{XP=EyPP;kO*%bsEV>kKok$&atxIjSY1nlRd~Xcw$AfcbR_`8_X9)|Iq?r@q@Eal{EUdP0`G|Wm zuf=CfH`4|i*J`E>B$WU&CGq0ROf%%TNjKK4co-yXb8MmtX==;T@|d+;5WuNL34I6* zrAFJ?&PRU>CbEjAdNzWq*Q*JT;^w0F?>Rp&UU> z^4^w4?RDIUYo#(iOs=#Te5T-bT4~O3q-uYyu(C?K(*_nG ze{d}>kHIiYbr7Y6-*GQ0H6coGtd!I$-bb=$AAY14RiJHn@YAoGFJO|nY43xpCB;DeyL_b;xp8uJvcwa( z2WGMeN7Sv|Vb45B17KR_TTjO53%9~!Q=*Iwm9^I!cMO%6a--a^SZmz%`|}qcq<=1} z+yWN^=p@&B2qmT0hXHAuj_!YR$#p`?(%xY{+W)IfTRu0!mrT~|J&012WqsMM_Op!6 z_f}kC|3l@sv2t;Ww>K8YjExk0boj`ABSHuFD&)=&hb=GS3ZNu*R~#CpE%5@avnn-1 zxT~nLCr0@aSU#j#-R*A}+;8wc`)xA}&siiLm6Im(2tRrt(oY_g!*MyD-$jkuLfYe% zI7VZGgv&=a2}1Xdq7Q@H{;)x>o_)<(_)78u#iIW^ue6weBvi8tbF^FGLSo@DwvQ=o ztK^xNE`GZ$16tXWRmE}ZbjK}4m0vv_7v%0+ZE_G);!YEy#2;im|K>PiEN0H4#>c_) zR+a^wi9e$aUDguDi}tQ|t7u5MULaC8ml-JVU{QuYpi0!8LZ6v!q;T0oY~nOPwb!3} zwuvzjRnDN)!F4Zr(TBDV1eAX8B`wcLVXPGO;AW@LJcTy`j%o!r(`EIvRPEnKZglL- z)d2;Of_SgzP1*%%>Z30hh0U0ElwfF!#Tx<3OiJbO*2-=I>KT!A)Y#sVA=9v0F>twD zM1>w2GbPN%CI?tN79y^Sul+gI~SpFH3Z;aS~TQmRQ|e z?pv*23H!{D%uemmA(>{XO_4w>l6T)FE-Izp@j)6Uq6!5c^le6mLEp&PCGXp)JR}mz z;VU|iLm6za>zB-RbAFE@m9)}dC`D@NIcjCF55fB!{Z-w)wwX^qN|lU6^woR z>K#Z63xtLFlP;Iqw=@ixl7}|Z7*PiovJ3mwbOpGN_XB&MxZh185r%l8;MsTWOG_HuCua*f?Qd#SZTR4#7(dm&GB zif%k9!NU^Y-JciY(p;oF>E%zi_dUu1J0XJFeKh`SYXC z{c2)g@`coxs>CP&3PW$ioW8S^?bM5ao^UL&i;wUfv2Als}r*D0)d%cc^uK;j7 z9`jz^GJ;`8KScIR=%TSn-b=LDFd;tf2&iHw_b|~rqCyue(Vf_lLqbR8LzF4mh47j! z#Wdh|84I|qJMR4;j(eW11ryLzh=pdDvf3sk@DdvL5B)h1X6WoERz0akGE>iIR=M~h z4p0zxI*oQ$h}JX$O|zQ8@riIxTFMo_{49F$DWW?P(c#w5H|8XAX_`u%$8bu-w(6(C zBR7<&F6O?au^rqgDdoh%n78q0*FzXhc zQ03yh+CVp|p)@70*pSjoOMUjGZ;iZvAu%#fbuY}mrCM*5_wLSp`8`s{GB@L8nj>0d zvTim3j7%smpQB(6w>uQXbaOe_dT#7H1yuUUXI$pgoSc!p$y3gvFwIlIHiDE)SWL>+ zvH1DGEI<<7(3#)#w_7IHItl+BCw3K+mQ+&W)rhA_4tK)v_yTY#%_Td2f(D@kU(JfsE@9y(~O= zUB-qn)TK2a9*kLjW4IyoSt1lFpADdq004~h?+a#xbRlN}a|TYTZEWI~PvErFs7X5! z1qJx&!z2Jg!%pnly?IWw84L{K8Vn3P9~K&PN2Jrg7xa&Y zg@}lvl!yqiqJyoOg%tn{j5^-L(9l?lo*HOmWM~K+rK5p&a8(Wqi&8f99qa1pLg}qV zoC+W8gNgwM=NF_vWX8kv`qAG7o5yq%FyiqehqbwpZlZ1ZMsmC{s9#d{hT4ygqJa}v z@(tX$Vml!3Jv6j95;0G1TLD;aC0MOkgkKCqt*79z6RanASQey_EqECmoC9>N0@wn? zAZ|E~;4Kod4i%gfFF|-#hDti3^E==TZ>jsYZozbvFaS;_t}G5cZUD}*Klua{=S;fb z>@F(?vnmV5M^lypR#WD$%+$=B5MuB+4misGRth1!A*RrL&;-OwVclS1(LzR1gf6f( z&<4=ZWn6+~ZK%fG5r~LO5jZ%I&Zx#ixPp5=#)h2L6~r~vXoPuDYnl<$QPIR!gt>%s zL~n1t$pAycwZ5+Nrv10KHOIHNw;w_N{tuLjwj^NYXGV;%PO&y1POAenrOf2yz-U2b zSTOKV3ouAf2^{pn1AV~2-X(;9!GONsgFd3U5dV1!ah?0_KV|Uz-xY;aM5LrZ-zp{! z0Dz6-XIrOVCvUr;uI4OMHJvo&WVuajtr?9>ZH)npZq|0cn}G4Uaf6E104F12H)|^! zM{YMh(!XkOgUY{eGm#SiRmI7Yk5p4mkyymm0YJ>o$jr!0$`4OWOw8+GYR0WBD*mTA z=s!Nv&rVKu+)PZauC9!(tcCxVV^@S(sQ@7(g``9Nlf4jNBM(9LfIftY;A~t_iJQq>+HlwO8R@C|Ni|=C&10(e;a9z|LG$a?_c%*A5Z=_;y+qy{;wqm8~Z<7{^QC2y``EX zz(K^;8Z@L6|NmCzPvd_+{L_$^>G#b4!HK`w{MTI&pZVc=nf|+G{P0Xl;-O$*f?!gj zLaJ`y$5~Lz7_*N9MBE14g(4Wtf=V?A_+levdBC-c;ugvM`8G0NYegO)m%%ld(Uo;worFRK1e z-M^Xl|27|T%f}x+^88*@SYH2w>Eb2!=i4RQsUkW21?#32h&4*~za^NM9envI6#3;B zCHKR2rGM+4%EZ8_&S|uW;bfi|i}Rr>*YkPLR>8>l3Obei*UO#Z;nN>y**QqEB4r)V zmohC!l|LY5g2-Yxe`X7~uDTbmjP3f0gd($XTz#ul<9ygogc5*$PmKY=YF3qU=BrPG z&taXI&prPYE1GV{YMRK$qH(5NU7nG*!f48Qx!OLX?R<4@>giZqxk$FO;7V$zi^*4$ zk;tnIbeg5F|Aehxz=i6&kBUMST zP#2N=cKmIzF`;!gH>6Lgav`}+9-(@g|2r5l%&QWrApWV68$x?T^0+EU5E&Hn%dC3I zjBE7HUy>7j6kaH*eCoTJc8%lg)N>3@47;0T6M_44p|5c{MdghW={j+$m74v_kqJO* zq) z0_}z`+0(^58_o7xnr z0|c%?zU))ad-cnfx|GTCWwPa$hAorxH8oTAL~xKAab2 z8P8%g$urn(6I&42hh_xkhTO;=FGPN|aghr=*DCYBtSM`Ce31#?Re1HT@Os{PpH^oW z7;Vt*Qs(ur<)GDUn&<^cE}O$B9=N=+Od4|wHm&@3lotX+M>{~>HPro$xg+A3}K(k>*8Pq9|6Bu1mx`n6oIX|^Lv(-`{G zXY^Hd&K`p+cA$g-Jy!imQ|H_uuZq6uy~i`WR)<4f#S!)zQ&ZF@Ip8PT#T-k(KDLbM z%wd)-$$4aP45y-Tt0;-LXuU-q$DEu z1f!QP3-gB597a7a-%6PH-6k>WaepEDv9qIY9xdW=tx)eMKJBj6dUG>XD6c6#>?J{+&$+uy)H6C1lrxYbRpqXn_4-zbWm05cn|VloH|a^IJm&u=DhFt?b7+Zq1=om$+98}J%8c?qPOu6 zN0isMR0umkJzs{o^ze^W&oIlHfRtB7h8MMRwL|3^Rod#OUe5|#mww)WjVU>Yd34r7 z>$0I*%d4Tw%_rYss1-cvd9owmS6s`T#V4P{p4&1b=uc%kdlqFoKTm(8HqJV$Ydi|@ z$l&1%b?n)w?UdCFS5*6+KW?(~-41==G30d1IlMgPxf;1Z+Nn2@X`Rdc(7fhkr9o~v zZ<|zhnr*X|{Ha_+h>-Qf^R>f8eciVm5Ue|2R?2|y{CbzfAU#8d%|X0MwxVT7@1Hv7 zY0G{X%`G-ZhkqkkG?H08;aPDXhDKIkIbW%)s!E>4?VJEZ5*k9oY z__Q%#C!a4iSPrXc=ux0n;t^k0c07KuSgbFdsnF42Y`<1c<8`k+ZG6bj+#St8C?7nR zaP6q+3STSJU@Fz9%9e9HX_$xinrG|X45%;8_V`Yjr023AM(BC`k?Xj+r$D(-+F{)v zX2|#LY3^#G^L+i_2xdm>feLt?l>%*SR*=l=_g>eNv$Uqt6KB6yG$`7eH3&z&R z!TQGMd0YPyx^=heY1{Jpcx0`mam02hy5@2;C&&MCt&+j<8fjW>+z0QU zp#fse@e;y}{b)Md#&wNb2s)QKXw#YD=38a5Hm$g1@G%w3Wta{7=_G{Es1*Hbw(WQj z&;Kbi0}eVmv>R$Iy1ah|!MO-*t}kAJIJe+UJ9ovTw*LH1Qe$HkNBl zh~U9yY_v1Y07jMqM7RO_Vib5J#9TIxAV9h zq_S4o9Z53@<_sEV>U<`>TXu}SK3URO9}5KYN7_IliQ)_k2UdTXrFFmP!``W%manrI z=eyHwIqP`Z-bpcNlv=q=<#H^_blVJfi37*}7R&z#jF1X*K}s?3X+gwimpyI2(|>q8 zs)91A7J1pA6wpDpN1`~+SP^6zi2m6K#Mu< zjU|0F>DN@ceLCyZ;ean7;Izx-1x@O9-}(n0+%EX&SE#0qp3q^?ma=$)93bo6vOpJm4($jQ!`+*wNYOQh zN#KlSR%$fr328*+gY*t?xr7!7yFr)d?Cx;(>k`Z1uvz|< z?6_<{u&`x(As*oNf_v5qr@}Nz@CQ`5e}G?V~LF1I^eSlvtRkD}Qt?pK=qj zX@?BR08y|!f}f5*_(mEIQFri)aPIJ%>z=1I$1ef_4=^Tu62FEui`Vd5e)XgHnCSm% z7muB#xa& zqTSgQ`yU|&)UA5R)&ronE(17_VN)flO~vd9?IsTRU00(q7)|&w||MJ?R^3kcYh*E4E4ycBP^0J5(2rT`QXKy7oM#i{#YT zKUvPno(BfwCL0D}>$`PpKN0#q+aFOve7w0BAkqPcn+yayy4SI7$DAgBv0!T5C(R%< zh9x~&Fweo%v_~Vo@&n)s6Fq$Q9E#>x{oIF5c}>c97}92FFV@8^uZt*asMenW8RiZv z7X(A-CV0EzYVfcTN>qNH!ROhS>GODSD>^WUjP|_}`=a?5BH3%8W&kDu_}o1*0rxKs zXdHSYh=WN9BI`9DL*E}RC%RPhCOJBO&aKuk zd(xx%qi22EgapIX%-ig{bTGe0gIS)93%q?GjQa&Vc{5V{?M3hm|vSV$Ut{`g&Sn=?Xc3@OVYH*laYN;>T&LrRFgTE=w;mvb_(R z#Z+(h5$Ay2sbyH-597WF`={w)=X?Ogk0^lkfFM#J$#oiQFL!?$%|XdD`E@}0hkH;dCZiktIw6Md>jY6f>6gxA)4XQ7R>2F z8e;fhQhtqt*zNl|w>r9#V9JIJJ?8szWMEXo|9OJA%wMteU^?=wyLs)9k0T?gFXJ5| zJL(7yneLv|+@R$GMHN<+1b~D3{Prx=)m<$;4*{$LpuJ)F?<;t&Quk0B&D0dK1OF;U!B>JZgq2_Am{U*YKNh}{^ zV!%f*djz|tZmyMDfXvc3=fGAAci(2TPvm@#ekFcW>myom$2ICzS}jmw4u4Of9@Ua4 zrrVHsX|CY2vpFwZfiT$I=ljr3bI7ZkxA_tN@%8r*k=|@u=7hMnYh~%YSGT-JCi+z_ z?rDa)G7(X11j$=#(X%ZIz7SHpR zT6YApqQSl`tRpOyf^I$Yc}f%uUnob7g-cq&ELtD?835Q@Ca*6{W!t)2%5{KY_J^7K ztsG;@+g2#ei3@h(cssw+BjCs3f@#9p5nWajt(^ACpqu5i`7^VhRw`RAK}}eZDb|h2 z?yz@-;>&N`-q9jD%G}+ z(T@-yU&NZ^8*GWzv65tjkVsK4n2)7?`&cX)7uC8iDz0oo^7l!S} zK7mfEHs_0OFi4*_)SJ{4R0cN--+B`?l)2P=)M(r78fuh)RrhRSBxos4jpy}nZX;zn z2whWVs!fo~+W4*>WN(HtoU{9Hb&8GBQ|I=Du2_+Q7k#>|y?0DznZQ>^iCC!vLg0bL|I5J8Lr0 zVo8Cm;?<|)euuSb)}0Yw`T9WUdw*B167}%B#fI<3SAuxWr88A3-9BY2X|S;$@5g%* z-V9j3)kj^Va$Vg&Dy=9g_V+N2Cj-jZhn38jHz+<>KHp!=rnA0T=<+UMY4J~59c@KN zBJCn|74Bd9erkM&`*pI|T&3kCUZB9MWy+-zf)Z?id18s6 z%;={-kJ5BMs_(|3@liy4@);)~s}}AD^z`H3CC1qc!4`kaqcQa=6cIoF)%^C<`G$aN zM8LAinAzWA?6o0cru?1tOR=-b14gC-k2U|4EgO;_k0zH98c&)!g*`;~BiF`wx(uxGNroy^p$ z?!<+c5!=Aa9vGH?Ueo^tpT!1i!`dXN@dlsm%hzbuN@-(=qs2L_CWd-0+v8jd~?dY#pbt{GhdgbX0!$C9rTJ9NsOv72JL#-v?h3VftT^- z=AbCGWgBkX=RJY9njL%tPdRO4+*tGQUbA<5BcOv>k8}|c3&axI4=}6!?bJ>)N)o|Q z4g4R@bh{s{=bDkTdc1 zefvFfgu}xLzCibrH09WbBW&z7qJRQGim;F)8ivBVDoNStU&=SWH5|G6K$8&)G#TnD z2=CrvBOh^CMeD5g%kgA4ZM@S{SC`m6N$3?%8*EfM+H)CsjP7&pwJ&+n@}>-V&8)L2 zF3|IQ`;dNf_c&oT?PBMIU^#a(GR%}0#%?g9M7 z4;Kxn_dHuN6tLCCJ?-!Km+aI&3jfls{#eQu52k_C9A!vK^Zaci!BA9ThW;E1(~Ax2 zCa8*s(tk6OTYP(=s%?Fk-7ddtA0v?&6pFU$Gz|zZu0A#qDDO#ShA%5edtx!}g*|%o zAI!C*`MIrszmwmdS|8py^8F-cfZ|JE{;}T?ysoFBLGy;;`!cjACFPW;5H`HmmGg_x zLfnCF<=9+w#|`-auwlMKkR{D-pZPX%N91$=%{f}wz)+Z$GOo1+-iQ(6&@CwqBPzT& z751eUT|twB1_NZnw0-nAC`_Sex*sMp*y9RhnXEGNGh#Tk8TDr)ur*wFjdjFFFY1Q1 zEpf_*rS$4yfJN83*d0JPb9*_Mcix(#j1FvA0coNZ#QPQ~%`;Zaya(%_g_P;BAftsv z%{*BIB8^nBQn$u3D&=wrhXe1~s0iri_Pi{NX=1(l>hS9{1ObWusRp8aQ6GYSVXQ5m z;9XP3=Nm^JXkWg&bf4*+&i+fE;auF>$eOr;N4nOzb|7e-xew;7b96rZG7PAI_F5>) zamV-Pf|85M@vel_Tr-LmSkP=kTh9)AD$?HA;Hj$Q2wSsmVxKccDdQJOfXvMQW-1Oj zwxy!Qkti%8UJ%0I*1yJCpl&ice_yLZ*M=rQvZaq6;5^xkxPKaFuLg@-gZ0yAD*%nf zAO|i@0e+GY@Q0Dl+VOjMLA-N5pt)Vm-RzTt6B}Z%K(=L%6@7X*sKVC zSX2PYJ|dpQ_Vrn0^@CrOub81?vb>|x(QuRw9@%3!$pD!K19t9Xn8m?_7vh?s=lxIv z>El)*h8BI?I2QYp`JvX49Z2ExuWru$I=HO+aB7vh3zw`pEr@AC!C2T4riRd>BwL%N z)9GC3-`Z~%hgrw%ik38Hgk!IuPeh3}D-s^k<--g5nFN9yxz)Rt-nStM5#Dc{Oz@nw zq4Pu9T70eO0E7=L$@uoS+WZVIZr(I29BBEr2lMLy#Xb$W8dF@pb`50 zOI5?SJ8?!tx$GKB+ORjD4rV1zx+gD1d$Jaut>r zeI2eM|LMg&ppJ2H|3VqeoUnks>pT1rfc+gm9b6fChu?dKt|26Z2DHvfscNERYYv2p zBk(1%j~iG%_xGR3I~Zc%s_)&A+(i7&nk3}QXSY65fh8|~w(gv0sKLUp{|} z*<~V(jnv7W+%zl|_uf#5J?rY(j0&&Rf2X;g@jUqmYfE&#;$p6|*&v|5aIOav2;@SV zjOocB{D6MKbU-;f$hvSO zeuQcZSN0>LvyBjjt^@6=a?VRGu@X>IwHH%`FG^P|hp(B|>cC?4Nid4)H}Hl6IpgaS zD)2IJOWm;U2_$8Zni(fb+w#e&cqGpsaHS5Y&z0g;dA=z49H8MCm>N!r5fA-@j(L^d z#Ha+rVN~?Al8j?2ihd=D@{zF2x}O$bZDh+0mbVy8Jd?F|Yf~2AnqbXofGF>0)AoF& zp5=-+amR(;v#4-6IQFIguUIRYw%Y+ZB8d62R0(|?MNmh-a;O!W`$e-^QTKjx*CtTv zr#?h8=m968wNr$9uh>?Wr0=8sN-pCh1SH=vOne@Ju>Oi|b#}Dhe0Q_`fJ5QreBX1e z9J@ytAOQ?+8y$REfh2WfHIiuA@qD^!b6Z7+bvC&+jW%otG7v0?$7}6w$yVU#E=C~A zlrkYiAQ3l8eOsR4LeITT%Rh%-(~NuD*w<`IfM03V4Y&>y$@jc>Bg2ky5uWfjC>JIN zCXR8FHo&J|amxvRw_yd@=8tr!VdF5iLy1pN0{A%!p8O^`Tyx4)KtW$>N8*owCUObM zEQHS}76Y17m~Ri?5Dk7N3q{N^AX)5!M^2N?zrf9HR8XZzjHxiT%jCC z{%gX;LGpjXpP=#*2r0c%{U@GE><@;Xn~P$%S9AF9xkH>Spsvd z{Eo){#Xt;d|CX!W!TeZ0nw!J9cu>@K@b$?#p$C~qH6#{?CSSsr0t8OS)2e+3;ugE6 zC{mNZ!0?l+m8r*n5fhuL5rH{ae(-A}kr2w^&>IZ-9|SG9ftt#Bp>eQ!V#>9j`wnne zZ*d9CSI`uCyGIY)=|Nddss#)WB?jgseecE#oLO69$K zj&xZonal^IKqD0nPf83qGPO}7;h&Peds<^E9R3>rbml8)_Iy$N!M%55Du2LTxkflR zGf*}qBlGCJde}c;WMV>pbOp}U2zS`q*B2Nq`%bS1cu>5#74wR-Ln&XJ#dGXK8joxC zCsr)aAb9aOkO!-uT%}18nL?i|l}I)0D3YCE{muGv@Eq;(kd8^FBAsnJl*sq1QhrAq zrCd6Yj;WJ+sWI5}=RF*ItnO1R)}!tb+Fa;A2_g zl$6wHOv;Zf)vMwic7`nKRoO18*U3Ii?|i0VvEi)HX{uF!q)noeDN|ldyCNk)vFgwKRc57{3JhYlW+jEw&0Hk#cJVa;8 zH0JwL5|q{VE%V}nqH=AS0-+J(HbB16QJ&0p%#v;53%BzWIal3CYVJS*r$fIR)rVV9#<;RdMkDEiy3av7AximJ*WalJY)?J;J>b(b$?+uN6J6P+$h#a7*}7!JH2)aHs`Yb2oEsf7we z29O>0R$OI{k}FJK10Ec%J_8R(%CGFGIT-e`X3@LkFMG3R66rr zZP=$(nJ$89p=%ZtVq3%mq(ovd)Y_kyDUj1^HY?)IROJ9kq>AOlW6ei)C=b*x1plQl z!-R>I>&b9COal!}OEjwVO}`eO$cDQsB-u;#j;pr{8e#lnP%t>iU@$0eMURLZ@_*?) zYzT~NMnxR4eUp;*KA|He+q(OSgfh=FXHS#J)A)V zEB#A3qhf#t6FQ)oOZ?C{(0{f~oF&NC zaIBEfBzjNmUyruGVq435wVGMKGamJ4XAzns5!o4U5OZvozMiqNO3-8^Uh5n)hl^C@B!4K)xlR$)ULD)61+u9MQJ)L6%D`Oj@mI!ue||?VIWT>Mnl-eR6 z?bR!87BHq;u8WBA==%S*5VxU0hT!r(%p9o%tf@-Z4s1YMnK-X2eiL+$cC1wWcn8P4 zwX66hGpZ2KjDTfI5li^5*~PJdnterM3G^amHX4fX`Xb-m)K~T+8oxwF-qxzatB%Fe zpi^ie-eA|hjB4vYr(Hny9vB=atf4Y_#WgZ9xPjcvO833zV2V{ z42Wt;5c}dqoAs(8RYfuB{dle1SWemH5ot^6J^}-YIzz0!RskHP{wC(>x8QWfYW5(a zO?2W(=j!~EZC?63TMHGhLf36V2D0Yt3`g7jy{|% zbUD#>IOTea+aMDk+MhI#0s(Uc(n+)q{Rb-)81PbW{wPu)Bd9=IDc&S=sE=nYXFhJ3KpAu7g&L+4Lshu?gAj&1OQSUq*-xq1gNd2(%gSBJR010rPg znXo-!J>j>4b3wh|&9DW;@+}TN@Qu|4yxb?s$DbCvBo5@@&=dKbSAJ#CQP2MHT#xwT zmb$Xmsz!IJk6jC=RooDTyD+W|i0-%c2^M>LdIXfVRCeoRYFbrw-hthy+Y+ z4_+?E6`SgO&hTRK3Tk^C8dfedLweX4AOg&kFcnB8QeF-XGYvIblcX{G1dNi+zvkyE zsKgd*^`{hl$8u^!Rh*Nbj-H^`wY^Di$}6mkycGvJ9~`a%~Nd`yE;?8U4kt;yZC^WM`z_R&0RJO zT-QvN{-F+=tZ*g``TU9uWcqv90Vgi)h$wt{P=f1r30X|sKdtIEEtLnmeMR@H3fBV9 zI~#`yA()UWU||vggS=w}2n|c!eeXc`^cJmWLsxISz@pV`r)Q%U(@+DckFT^Wv4Tuv z`}ld_VWA)ys9!;c@Whx(1;1>4+EJh$&VN%!VNR%}WrN47V7GdYYd7BIzQ>Er_J%pk z{!6OG=IIks3VED7gh!{>qx*Z&iR}BMvHlrub0{pf1Sr@I&%hQKm1yrmLu8q{P0h) z$es(p(AUO}>oAaQSsBelQCL1PFG>O@^>ljznD7Hy-97P%!iSa zwH`Nk2d;Nn8!n)vM(HoJ-)ZX1Mr$iUNsnH1*tfg}bGE;Td-xwz4#S9i zt)-6FtyMI=(k}(R2eXVq{NWiD=};Z@#7lGmmFf90p$@w_n+CLWy`qlIKi60H2j;Vh zCuV0U)pf30CkpkM#uP+&avFdd@A%!;2iU$p*r(}0QsQ(z5cz(fE6()LZZ>A(y(`z9 zDNDI-IrS*jEX}L8TvESX^<-M81w=ZJ{?9`Rjty^^`ep*ta&`Kn(0N#$#kbq5<)J9U zm|}dfBT8$(ZcZ3W7D5TO*E8;%PE4t5$Igkn!)9wv7ZwuMsxwHgMzQKtT{WDu_%lsS zAU=7VIE4I0MW9yLp=yxBr=zwMv(GK9Q7(wkM2HV^>mf%~=+C4gS=yB^rzT}8Mtm*EO~T~^t)+`rMOk9~_+)%iN)YwB-5YK=PZLggih zrOrviE;m6!pIP;8S{1QQF2Qd)U@JM8pL*u~c5;Y)ix?OLN#gv6Af!r1fiW~DTP#IF z5f|9|$$^DZ_rt>P&iWNw%^Zh@>`BX8D}gG5OYZb*_1qR1bKD)K`NpdYmmoiKL8^9) zlj*31epksC-h-(<)fDgkdU7a(yzLVq)`MS!ItHQkm`3a5)JJ{}SU0pjpSc+(qd@Kr z{%<~=HHW|_DSGaM1#^Z$i~Cj?k1rY7x@2o+Mck=39w_n+Y}(_hR~r}PsF0x3+G3xg zVb{%Fu=tys8E&ZG)s1~-fiQNcTabgfKyRfs8mk%|CZfkEl}cp{wYrSMH{{dFSD?Ap_rL|w)9zEgEr#UzC=PkqB%w!(aoV406Vj@I5GKSoA-|Yn3XwvWkSA6uGNsd%at*}7P_R4Fm z-GYj{0a2LL@a4d)9Z;r4sHb8r;EMA-kQ`*oe3~2!X_iz`xujhZ?M-CoorFmy902Ts zy3fj~7}gymrL|HmA=hryaK#^OLJcl#6(jT>B@9D)Ex{K)yfF=qwAl>eaT7zL!!x zKOpWJg*a4Kh(PC{eI2mqTxX4j^s%8F~v$;N0wt+QJ@9$E8{?3X5STiM|NApx1_L=TEB)kL(;e#~mj{){g2TNGGdO2i7E1?5dNX z<>ubCG~K8*Yr7I3)4+O%WQw{B^2^o}H5epPr-v5A``t!nfXSHDe5_NCEQG(+3~!I1 zG$&+!e_oN3D0z6q8GB8LcBD&-A;Y9e{{A|1{^;YSW;>Q~O*C)+xHW;(eHzsjslV=T zBi%7}pNh_VEP0_&ka#u}jvcKJs+h-Vi;<-2Ckw()@2TP5l2fSy zyVZER!#hS@Wva64E;Kn0$_!)J*Cb(N#!PB!2rGORN)U$0!8<`Q-zT}5Z5VJI+&nEt z(rJ4hKFEZcpf7Tt(bV3zw8$3PPS29UewaZJ?mt;n5dS1LH_NaF{FJsj*2mpvl6$J8 z4@G}^_CSxAh`Bade(jTabn!Sb{QCLk%7K2`G_-3zQ9b;%Rwd9dNt<1B7D?PX%je3i zn$0OW!ivg>2VRjlfJLX07l`;%ZR-^B!$`QyVvN1O) zy%{!Lh@Y2lt7Y}J9w<|2yhU-dY;1d&T%(A18=(0CL^9`KKpTio0PPo_Xc(Dra$mm#5Sy{fcAwzbV%M;^2f7n;7X`#mfv!DNRao)Gk{^6`m>g5>FT zm!Zcox=w=SAICh?R`;0K+eKZxHqiozPNvy+zr1~3dfzUv&qqvj2E(bLg0{x}ONo7m zGafuK+wwqLTtD)FR$?hz@^xr++*16BG~UCMm4W^;-k9)ak~89QzgU&L>t}n0n&ZxFUnm^2%>z_SaTRd4B)9q^ThzDS8vfCIxa(~zy9M4Eaeh(jSS=q>y(ko0a-Y4u+ zh}qH?_>{G`(96y5U_eRPFC0hahDja|LeHy5@?wb(@A%H1QWJy7V(%<}+-Eb-CRg1- z!}1{BCR*digendd`&~`N9!V|y0xv(fGsP_qtl&qnlP_RhU+jK}mt0b6Qmw+>YpEQk zBQPwiYaij^fSnLNGZEOA6Sl=mcP=Wo(ANLPI7=E9s!YnE%+z)sH-%r2axI(a1M%fS z&R6R7D;Rib0E^m2!5z-Bqti-`Hr8H%*UM(4C7M7xKbrxf^pu)wJqUY>unEKKT%9Aa zt#EC<&qp*@bioU~{S3;zQa4GMQvLHKYPWi&NX=SX+JXW5m-xl-KZ=lwMZdQlKC>6S~zdb|#Bc^sy?V!l%5;mvhZ zxfZ5-t5btjHo7vIZzQS7Y~y zxR=yULk*!;S%Orj{griUHfD}6#(-2p)Z3Bbhy);ZlPY@{(AK8&?o8Tv?3d0s@IGK4 z^D8viYq&}mT00a3-}A>yJ=YaBkl>kj_6xo~_|SVl8%y9?MLX@Qvy;BE(_h_>ng!C{ zPV8`J;WxB;3}SHRL+^~=`&Ccl>(t|Gk0A1%%FYvan_S3Z2@%gVV8wqbW`B7vyIA7H zTRc<3kjAMNZJ4#(ae#(2KgDUtB%FpXrKv!3h&uAMKlU z*peQ@E-}Y8Yn^rdJ<*U&z-woBg75nQWeOum0qEs2)8atdIZom0d=Kc7iDU)DUn!>u zE4)M7cVdHmANx6vc9OkdIBDG09=k$MJ|5!j(X=9+HqXx6?9!5!_WLwNLcQJ~ya|=T zHTdJ~LX!t!-UrB)MtuI~VlOY?&^*0jmD9>_l5Oi^r8gvkHz}Z?ZnVQ?>~? z0_NOcHUTOPHM+56U*UY4>htcP~9$#1Hy(A87{8}cvo5eGjtlcrCGHdsC{5785hXVZ?H3dRj|~UyCz4ti;st7#gp#_8T#$O15_Y1aPRL@z*g@+;oI^sMs ztmV@Da!~xg(y8R)iaJ`;DSqhCTy~I(6LiwV#>2+`pjqX-9#4nGyNs^f;DHtSKoIy< zm@kEO&BH>Aa;u-KVjA4Pmk;kQl~_MG`rhK1i0NFs{!OcE!jv^OBq zMFbz5hOmOfiV4-Mq`F-F0>J^R}}FDw9>?qc56cbp@7z%bhQpsCJbAO$6a}bg!+*mlOwXa zw5Q5$(2#0Do9V(!&VmWJi<2^w9^0VtJWd?VQJ~smy@lx7Vwp`$3yrn=WTOf?&W6O( zIygw>Lp&TCP|8jWmctw+`(4bZXK78MWvhvKQ@7eloQ9R50H~|nJCDxAr}!SW4qFxW zlJD+FGKYn5NDH`HAZ+ z*R)h0@mbb^`yv@eTB1KlBx@IA-(spo`Q^Lk0) zNBkc_t@CkFRb}&S2owZ19Sbg=-6Gs;(B}F!)gD)~ z92l)x;f^RTF`+0uV~}}SvHL2^j$JW79n5Wf2G%OnnRG_qAG{n}3E(GRT*bh;> zjFYUr7ZT(;3YjY%HxaJ2)2zhVu?E^H z(*;_`kIu)q2_I`}w}j<3!5 zf>tCdkJye&Kki+XGAV?#%6dB{1 z?Q?6Y`0B3)-YUdT-mQmw4TQ3P!$NU5;9g_zh56^#%BkkQFU!dZ4rz3%X;n-dH;Ji=FM(R2Kt1-skO+jcj zmB^^QY#y0HGcvhHRDN@SK2obEI{yz?p)y-Mc!$xb?3&WVQ&-w+JHOy{su=019AVvvCd=}SazjaeE$S6&YqbZlVXmK0 z9=sF&!A7SQVjS_I+OK>h{UE+7CX}rIq)!T5@d8R=Ljcb)sw>?ZwS?pHXR=grc{-~8A*AZ6yL~-e`A{V2afzJrX{pb zF*o@hh^;~K^6tYR;vunv(M~9_3$zv4YS~eA#17Ca)7{LNFS}X;J4<#Z81N_YK~NFo z&lgWP4z|(@x_u;JKy#A3d7v*nXx?@u` zer9cNftTnowhNutSd2vcNmcNo0);wwrtWc}A7mIVbW%Dn9(KQPi+KeWwdS~CuIrb- z384<_FJ*MF#&G?~G!tY7i9+#<W);l!7bEK!M_D8`mcy7-8Cu{1zawzMgfPiz)As zZ)NL}n>Eq&f72?yf`Cec1d0B}%biGuG>GIXMrlwBZ-@83P;-%Ga;5^Jb+0hpE0nbi zf3Q-J+=TsJXZD;D&)Gb9q5e1@hzDLUm@9>=MZ|MO~* z2N6)4!46qKLJ=~AlVY4HCj4o&my242Q@d3EZ2w76*z+#mvh`(JKK{G@Khz0!{Pz96 zg!KJ-|Mn~x_ z)BgyN{2lpw|NNX<5Z*2LUHz5(J0Va6NE5dLa%BFK31b@eyG||wAi?bY=P4K1wU7cN z2G;++0|u7UzZSg8ckX-6A-SBhuU~_~eBE5?2gYBqO(a~)yz0?^giMlH3gVlhWRYAB zBj{ZgiFtXXEzh+w*77|PQC}zXI}?iJG9*Fh#dg&64}_^(qWd`23T?6PY4rp=PH|&d zD3Z0?|5tl&8CB&My^8{Zv~-CAf|PWFw4k&oN_R^mAt8-Q30p)uRYD~;CC#Qwy1S8X zHeF|J3x5Ce<&JUh9pijBWAN$Syz^Z#*Idt3P{YmF0zyM;^7 zL;E}A$G3tu*Lg+l@a&k{quwWgcHW-m8<)=;gh*tTEZ zU9U@uALM}PDlf6q1>i+2BzbqQQ}VeNuJ)vSbP_s}bz6T;mS$L|YhrMIoaPkE?QnjI z*Zfxe>;oM{P{$oEsC-B53qCuGy4s>cgtYPjpB(UadlRbS&E1+KZniZ}`y0cMjY>20 zQg`UdHKBtkd!g27=)3ED-?X_P%@0Dax2y^|nyinx^|WcM65Jt|)Ps4rX~K+Nv+Ct* zFeoLlcH+9p7ricOVXTvB&V*L5ezzQWN9}d;_(AB=y*OSAT4=?|C+ktt_n5PcLn01q z;#!Lliv)^2@YZ{H*u1*7MlOE(S;o<|m48$eggh|j+MeEJHh%?s^Xbz!l}3%>!7M#G zkg?nQ?5pJr7mb8qdTaN9X$Ba`0eSs22UCmwZ)vQ0h0-I%b}Y)`J8z%n=-e-J*BC$IS6~s=N|@JT(<$L>r#In0j>0XH`(`sz=vLq%@VwMXIMu_9g*v6D+PqWk=M%G= zDGUYQ{xxM3vk$5-ev7o6*P1DFFbk8qpjKmAo7akcWdIueI)pS}w)F$Ejpq`j_Hbd_Dy6-ZKDmgwHg01-vF>Xv6!!>7y5-_@ zui293F>bYA*&n%WKiNXYA4C-BU0+YZ^Rv3r^8I6vr;5J=HzZKdlz%Y^{XdGn3+f-W zt75SkU1uNE@k}eUD9C(!Q#dCVt&-6aEea#q2;NF9p4`0^5qUUPZcX{eMjt3lebTE; z?Q|YgXpkpK(;Y3!2U+|rAQgWuRABP`M6|xC=Q8z6lXn^Sgq>DtTqZn=3vGwx=Q(0H z1SXv#GG@6GzkE1+t1OG%@GM=kppUIHPG5{Y+08DDWLoN|NU78q@6+M=A;j{2ie|=9{L*M5&?cSvvE7Kkd_cDBLI*g=b ztYOX0q*RUyR(_e+QUpoPQhYg>i}c5&G%KJSyjMz%k&#;I&opOjVHjCd3_6rvNn2uG zjC|EZuVZp@v@8$v9#0>t73+t+mR0N9iHM|9R%D?283!J3_L!kU*q2tv4GfM;l?hW1%PcraYkG-`+_~ z4n73Ad2Pj`Yus3bpU;dt&@I-tJlJioW$ zRL#AOWpn4{`H#5Y@h-pAp;-Xv!gKe+uIwHB9gLu()~4ETYDXFtC6K>U2_oD9`ouaS zPvAC89Gr1%Z$K40eaHpfg&TiWbN@*ho6*52yK5t10=s|;G`=1HU{GhMpPosn*f#te znd>y-5+C;JM~^ymLM;>;p<##~hxF9=xm|>uHT2$CIaGzyh9Y8{fT z?AQuBE``TRlsn^7pgtyU2n>fF7x*B@bNXELjWe{M%bW zcWyo6*S|hvE}WHix1F7_FcAN z@*~YSCm9jO$)0vOP7VjlpY^`r{YT3^z{s<|?NeFHXR&#l8$26OQx9A4+=w3y(W<%? zmxw~e&W7JUeD)q^8A*V;k7v|bPJ4md_#d($l%)f+f>E-(_nx^pB5Ecq#n#wdylD@BCUp%x<-lR!9gW8jj>&3=(4*V5^>nX2 zXqz4k&^;NoOp|f5NAE$FP5)so^l!7cjc)?l95&HfqQ^AMMKN?c@7b{2#q`ufwFJ zxW|pLeka#NYV+GJ_n(i&ks>(^1rLq;r+kJ-t@vm5QCYA2M8! zy^p#lwV?$WrI#`~wm`EfAJ`jyjF&lypeJJX+l;zRbVK9n#X5DRf3)dGwa%Z5wUF6A zx)N0FbTPydC^feczeX4yszw?lRb=s9y!XyWwg%f>@lAvDldBiA37|O!t*;*`eA$+$ zKFm1rKLOQ9s90KEz2k4lkS~dm=)xo}Ovxx$>QdGpNWL^6DF$ew<2J6_|3D`&%5ztI z{Yc z7{xbV(rT{cYo@4b>^grECl|rx(W=Hoeb6PPUB6F~Q+U4pnd%Dvd$446a@&I?8v+XwB~1 zX!mQ(E%8UZ%)qsh{^bn%_3dGLPM39&VehJkMT8`Gp1woy1a6bCq^h;Y4QFeR%6G=3 zKm=k{PiHMcJ8LzoX)l%^jRERFwl*Wc_1J@DmLQ)-o8P#HU&od`%OtD6@>nTscp{D@ zx)p|NCL4;|I+r*oJrNWeCww8m2X#BK+W&MtaHL+XaolYBQ^PcHU?L*srUOy5T4GXw z0bW3<{gH0wSFPl^@m_<>ong6}n~<(rOyR9Md#YLT&wCuV)zJ4ejVc^&@g1|4>J2#$ z4sfqVDE};6A2;bqZBZ$BFkL#rRQeXvo$cKvOA2&DHc}$Pb03iMcqq8*dqpCGE~#Fc zJo=`2b78zLqmOUkBo8_;w{ahv`iX2roe@>_G~Pq@&0Kjjp? zO7|K&>0mad#F;ITOFtnN9!&_U{A-=#NOH{$y_pAl|iSBXsE(8q%BF5v7$io)E%Uok(2`llbto`*dvCASI)>A zPSCK^+zQgsM1vlnk1DD#PM6`%-~hFB?@$&?KZkQue{ff!Z4_vTIRE|S^C); z83_`%ay#?;$n-)&2h0V}AmZi-)>C;8JI|$_w0sayjNn2Wq>W+k#Q0F!i)zygNK@0m zLAGpZ#bID|xMzP)xUKk}GVa3hk4$LmTMC|3r>`p9UbU#Po4v~0b5j{w4?-3hwFRAGU511o z^n7*>ke5YfYZuASu3f3G3Vl|x`Xh4<*S1J#5O!cx`1WW9TWDsp?x()Mn90uFowvPp zKTj4S)qm~@ufMltbiu_w-+Y);Lbxby@Xq6|QK}-Jzm-ljI@M5V{givj@cybJN3&hk zDf^Rj#YnK~Gz|l+j(Vr-+MoBfr&tw%@Wp|7>dekAahEY?h?D4+G~Gl9x|#U-c8HG0 z($AjSz!UIN`9OF$J?6=N-N)_K9ggNgi(XUsB1f8YW)xt!BYybmcz&+s3F}Ikv*qXt z=Ae0@Se@@eqW8z8!F+DJiAnC~GtzGBhh*)}FJcs9@2av_uF6|j=E$uj<5CLzh&4}U zX^mowsJkY5TZdi!6H(xxIr-{@o4WYnSpYTexjupf@tx@YnABKK6|ga*fSyRaON7!p z92mYoUzJ&LVPTcKDuj9lI%({0^tnV?9_f0r% zTq(kX9k0I}Ho|#iZ+AAnn0&nM{U}{Ao`ERz#n&h04>kPnuN0J-8F)Ap_qOqUuc+s9 z-eD_l8Qp*}nkPG_FSzp(yE&z-(PEp7gGI@?z5S+~0bnP&21unmb;(>g7hXahRBUhSS?-F<+1{_+vl4ij+-{ zHS&%+mO92LVB9Gg8{M#_JIIiOCXvbdU&XEcw6!ZR!RwbA39m>Au;#t@afNx@bZ*md z`jpQ~EgJh}5_QK$YJ~E4QyP?kny>Te!!P#7*Us|H-gV{QcP#HXpCPKgVsYk&w!N8x z%)bRr^$gPHgMb|$Cs{Knub({IC zT4ghq+p6%`n27kXHL+Ug{T1coAEm^0K2ZVD=hp=+iAi_JpD8EuH)x@K?~LNmH)K`l z&yJDt3RQGJ35U#xjc#F;1eGS4cgEgi=T=L;nR|Y=T?K6#DqK9D^iICTb~wP+*3tSy z8(L-AAj0Zr5F|->&%CbaQ|ZJMN|06eAwQ2^^0aFOy+4xMaiYsl?SlGKvXAL*b>)V$ zS2aypYcP^M&tBhfmu*=YTc7fNj48cRFzzy{be-3lhS5c|CD_e`)2miFKZMKOdMLMh z_sEgdPr!V!V7g9*Twz_P*OqofiFQh|>`IAA!GRE3=nMW$U@<_+f- zC%xa1?8{<6>{06DFT zWYDVSis!d+7_Xerh@!f|*>%RCoWxpicBH^>H(`{O!v@V`#3hfuQKzaFeOIfdtu(h2`A zzIV$rw#SrLvrR1X>DNyL%zWAOSSa~*6AebZ9-QhT1U~c|`7Am4ww$g+!S>(sNY%I+ z35xNdo%T3cnn!}H8q&67-Bd|{`DvG8@1}DffuA`= z?IINRUP{ouh!yLw>ZbA(?~`XkxgS|I%d(`IW9qw09rNxN)rYN7#t;klA9&5wAhn9r zf>hqQII+Q&C(2^%Y;kY!6YmryeAwQsO+T& z=d~na=?Z}Ec2&^Abj*g7UF${|$CGrYAUVn1mBGmJ^$NO*$x@Y()N#+^@wIkR|H=@h zGi6-2#@JUY&oQb`(FTX5u27Fz%=EL*AL-fTDB-fS#S`fyXSYoq1yDCx7RH{yejHU> zNoR>k#eIGoUoE?1h6aOWB)SK`x`AqTU8*VD(=OU7r=q8f&s~UHFZmWk+qUf1 zssmB%Lb%Kg+ldboEF?doBmD91P)SIqPrq}P;ZeNOsrf^^ zLyU)YCUHx{Jo!GMoHgauMk{FfLM2$e+c-@#d9a*u938?qoLls-e;sJpR>=xahN@IDo zSH1FD4%EqFgPa6gDEy?5Y>{_D?ftz^ghV@{BOjdHI=O4{eziBxf7S9N$D4bA9#!$) zH;b&|pjV&YcDc^9)XcZ?x65lp5Vd@M)XT1yU$6MkTkop)mIfvl)br|o8eqX5X<4<0 zAo9+InMreQ04$T|Z>`uI&ui)s)MB*ds%5pY*llJYt_s+3$vA}eW$;T*CtZG2?vGVQ z#56@Gb4E9?rH+^%goKv7neD<7rQ+tHwMibIXXqiJQl-0dR(QTr`~Y`N z8>RHYvbVnW6kS$e}XvO zP8*wOOl{4~Im?LEI7~|L=#lGL7+FEHCEILzV<(zL<=Q@Wyq%c^oi>Kx$bXSMUK^CN z<^Orihn9tz{GL@XM!ywQG4liKBl=J^rm5>GuAtHWl?{cxamgk z{T$df){bce%iQoSe>A&?bXRb0eT7qO*Cj+Mw?&6;y4A;~U&EQe06?ON>ZQ)!q@DPz zpP-T#uU+Lsq1{6JbVPQIbop6r&`eft=L}MC9;K~f%;)W%R*uo9B{^?Ic2l1v-}D~i za&W17*-qKmnf_>7G8NF>W6cTbHwVm|?Sa-m_A^57ZI94RuKB1aVt;hD0*_VDO;?kI((66Ux0gh5$=?kndS`&$&J*gW#65^7$;3<{nnWT=jT)0hcX1*I zr!|u}32r|zV=;K_o0{#*4j0)Jd~l&nc8{YV^s?{AO#wKz?aHHYFZ(W-5a}|7#Hrb(Q&LFBOe14dF8Y_wX}yKXou;A9?sZ^1~3g{9RcZ3DiXgqLdCJ)yt=P zS(^8tJ+}#LFFR5|u@2C6^e+V(s+ZaWuH6xybh%f5FSvjELoVkz|2-Om57PqIqs)Ii z4Wh>K&NyCJIde=mR#o%oyTd)dn(fr1S3xT*iYZ*Fd4HQ+U-h`gWW`4~^W?i$t+ zaY__Ok^6M21@6J_b%=bp@DLypENO-yBHOa1I~~F<->_zKImo0vhZ8NE0y%w?z0ZUy z)^&!4aVy{kt!|kUW|QNkZ;fFfsKS$6XE4iGlFv^v`m!{65>_MC^}@x^4y5)b zy}gCD>WS*MIwH>juU}7rx$@FLcKpeH)w%QfOqst=uq(;#P@0k6zR-Jf(@1cKvj>PK zTb0F6i68C)-t^}0gL;$`jmPBWjQbNWSt_9rj__i$z$=1~X{%gS`QZRw0)D3vo2ND?iCu#lc1QhYU1ltF*km8XPaC%VzoH(UF zqqJXGIx-RGyCYw}a`pZz$4=3~Mx|si-7k#6;gUPk(N?~GVhWlMI&sLKpa7!{jSDb4UJ6<)`k1!-`1#J1x*w$pP5a6{6 z>qVae(#;niaJIr)3K6wdB9z63&H4d&c3VhvSv4>K)aZo+9)VsJq?y2ZmWaV)-f=@1 zzh?0U>=1ZQ3r5K~t9u}`7yLbbN=SES8Y%4xiL_VxzS41u@muyadp-LMXvMiL`N{h2Lb>ck(@V`_)?u zI>lIKn;^yFqE|mJ1W5hYJM-0HHMtRP8 zdxB7epPu1Ao&#hC`wpQ6jk=pSFE`_5x}fk#K-k1yb>iU5)k%>MS4*)`l;e2#g>h_g z?tzZS6Gz)V9+vObj0v0QpJ=_0G`}>vTF*ufASk)N!yf9pWzDL_@JIwoHP52ET-R9A z_da!wzuk>o59%lwLBKib{mn$<8FsH{WZP!{b|wrlU*LMv9@O^`oNZxDF2q%loUHQp zW|`mmx|*gCt);Ga+rF3h+hf`VWFsmnl-*Yv(Th6pj~TBgl0J2x*U6F%IN%=KPV*`v zoctOf@!~2J26x9{_@?LB&bK{e46%4RBtuQcz`Ep*j(2TC!352Z%F7vnRK@l-6TJc3 z{hF14hgrnGdo~(zj08|UXS;z+f2eWl3I&^XQQ=CAd4LbcwCCyWpm61Bv2eRL{*K_) z$(*XW_X??i)N5Mc)V}>=5)(zGGlE2h0q$8?DXyzHcKE=Am(8rB%W&iq{!$V#S z1`f>uuNznF&iNz+B+w=L>P!zn>7VFH@AGjl7-5u)VoZay7OE?agn-9RyTCv$#w<1K zaqa-!`MrK3{5AZ|@X6#(`p5b|4j#R(`Wz_t;XuI{rvY`!Dy+ldSRt?a>ejVra==dc z98{c0Pw?CuuD^_r@C{IomNsO$APir-0&ev$wTD|eMs&{>b~SyxCO{tOq=+h6t#9b8 zH9c*2HM4}Lx;W|IoFhE>L#Q-;pD|xxxUV*Gi~?Kt0MCG%8Mbzv0wAWH^R;o!FE>=L zBTYp#D9$8F@Dgl&!^nAFBPfkeA`|q5V?Dg)vX4}CO_Vn!%iD!4(Pgmgj89UG- zJ+QzNRo#TVUMt;_@IgXGLB+s-29qfEsni~Sg&*x*?8i^; z|M_R|lSDfjDx6PTvP|aRvy^xrA6R_YU4*#bFY}S%cMYDPE>ZpZ1mc;mEegt!?cL?S z|AC6prz0VuzeAMz^50viWtbR?KW8tS~GM8Eq5n))Tg=RyY_w3F)v1>3J0F7E z38N#x-0i+RX%=r26$VT?@AH}Zq!BCWi&-Q?rh{kp{XpHx3n{@5^klmD0Dr0v!IA1n zL-R>?@x(~PIRJFSJBwb*=zW3z+Fp$XT8eg#JK?!M4oB3%xak(cRwR48R;)i-26*N( z0E*+i>C`3Zb_%R|znVTP0=HVUvK6zEivAg1KB9x=OlVKhsBJvB#pxd?0KRc1&%>Y5 z7CotQ7OO*fHqPE>hvvamr^0K6Pez_M-5gyGetmX!MhxlTc84$W8rA}$6VT6xt|c-O}0dh8LpnCPUUSJld1o?7a-kRHopo_+;Sgwy2& z>`RL3I?wNq09f=y2gkagoqq|KUInRuB6Ec*fV`JL#V@Ou1q5XEUS@4E;a*^;aN4Xv z&j{psgG2GYcodXn=>_*QBO@BmiX>)HjwsF#Zk;*~`0;vA_ppk;fCs{htHR`-FhH#O zVgvhqHzfuEAYKZHkL_Nq=jx?=l*dC%hXDpil;| z-h6n~xU1bfogxuNb0QB0jg3M<*%bb9FxS~W2)Lx|GWJ~N?kT+2vEV@a986+*O9Ug{ zn9|*~!rtxA8Wdz$1Qp3I*Gt{{T_!!F`6oSgJZc_U&$`VLJ*#TVJUk9D@H%SS=uIdM z>8V;^>wvi4T=`^c%na^xY!IbpdO$t9qJd&!x`W5w!=};M&Xgp-wNt_w1;IJ*KK+`k z2a_=&SBr51FbLjAkm02M`Qgb!IVgbjR9`|;L>|l<>pljGRSvH~rzQ@u@&si$GIomh z0;t@dO)#J!lt!$ID&_?dOHSGHH>VmjvQ$8@2|Tk8-_FT=v%4YC;~gfl-JHPujuLvn zctq?U4sOs}$(Zx9Rq%WpQQ_fcM07ioo%;E4I$0rQL+Q+>ptjx;C;(+CN~@g1^jy^o zsBuQ3%S5Th_Yp@AHnOjR;jqttR&_h!ko!t@FSr*RUv`>n;Q#@wkkqE)ZO|2$Ghu4q z^W8!H~eZ8Tw=mHlA6?Y8IPO* zdg0-%Z|zb~Ad}j>26>u7X;;Cu!P4UaicPhh9IO^}Zlg&kVtne8N_qbwKFIrg@NU(y z#S_ND9_xWVK^1$o`F`lvBCnO1Z=L|&XNOgSGw2~pesgRH!I1)c>FxGdo($E=Lt3%i zJSB5>-GvuXn7=yXdxTkUH>j9+mv0=Xw3~?tEEZBzDGOnipX=9moqjDe1Ur}sdGfZ$ zg(WHGXJIpF-GwsLyPD4BFkg6^v)@@h+Gf9Ma&RU`skrhu~+8pRvDf+Ou~O zZQMjcd(USrQ9A-|ILKb-x7N_^@~TEgPd$h~2WQ}Uof9xa#Vf*ZYhWcvfj;F-Ts_|H ze1oZ=B%vt!0#FHo<(#k9WlQNcWW-#bs0+65vl5bB4=03=Id=XO3zi(XtubCE|6tKv zIk~W7L$JqU=IT&$qW+VhvY4&(7J4xzu2YjO)!~_jkR5d9?J4kUiF_Dg>CW>41&>YQ z-_|}Q7KO`_3c`dZeqvAcm0HZhH7Rx;99Cq_lRaMqTV&ITW;B~a688Bb$rnT7tuvI9 zC)7@X`us!^NiPAywy&ULj`FYP<5vw5`9q!?+u{gL*w^7M0eq!D`yp2M#b--VOkT5B zV(IB`hpihSr>&O6dF;z^{GO%@ltIukT8?bj`cSs;D5BQ;oaf+pvp{J*u;;rzmSOs* zfiM(;z$P34E(a<}hY#cwkPk6{n42eKy1sbt`G(HcOFn`}-X~j4Hg0X;a$5qRLk z3_8^h)VXf{y^j|VWau5rgGF@<l2C0iTx^9CvlDP12Wi(fQW^-(hXtjJtTE2~d-!>c(kjVIS z=P947TFSCVtQr@AfCDRsiVwq>pSEu?~iz-0&M`Btv#3i*FgjOy~IG>}lioZ|OZq!E{0twU^zd(2$i{6+H!yMTr zSQx}{N%Z~XEa^VN=Ox`t!XIEx`4^MQnONd}uMSxg5v$7KyS$-GUoT*84U3B~1_S&@2R@>i5dTWSFESzj^Q`q!QCLMpN(%U|V(e&YYU^ZS z=d80|tpap4WvQy^tSKkUV{B)`XlP<*WXkAfWB<|wjL(e+c(gHfHY9Pgv9@*MapNcZ zU4sXBetFGAM)JFgvlTy?rko;)h@GP;$y-KdMrJYrcoGs4K1UNX9%WJSzs-Sf{A3o+ z&h|V^OdmgfWcMTBQw)~`UaZvy}ad7 zv~)AI))KX}0muW4A;8AY!S}oV|EJ`?BmPfI&HuLKU}pZ`mj6@o|8A+~Wa=nlX9EoB zEb!l*`P=w^3;#CcV|rQn|4`x&nSZ|p=qv!w$Ml~)6M&CT>hS{u69SVG6;^cvKhTBw ziqVAwf(^kGjxhqagx5qZBTr^=%C5n)K%I|`W+dBs4pxkIGGSs#cRLA ziiWFVJ~WcQ?+jpLvYbeM7lweLQ9}S>Fn}+=s{o~l|5F-I3t`hcLm7e3DKBt; zz{b7e3z@6i=&C-Pp8 z$Gg6)U?N(lt-(H(%VqCt-8kh~J08o)LN#?{iZx)GylG-s9!vRO9mA?!#&V@Vx;3_o zv^Ni@J;Wefrav^uv>=T^6*-=-{n2zaXQfhq_CPQjwOk^wmb9x zvnydx5cP8-?YKuR_qJB_xlS_y!n{XyIf!_yKa-)7`6f=M^*l=!18AQBs<8*@=ZGUA zXxCV$AmYE3?N6Yin%4JeUU1(|JDb$dpXHy4w`;jI<39a~v02uM948#`s_#eZ6Rn1x zdtTGcT6j&!jQ928W^v`O&)t#u+OHo(yj$*%c)4#k;_}48u|YRBwTqMnYf1W`dZ9x? z6O-Bn$MA#7VS%xFCv!p2Mb|`u%I0D1a!sIk$@}W!1xmRuqM{=A0?!Xlw>Rt2o4_$( zEx36&A5IFxq7(lqbJ26QIj|XkBEUQZ({H`p(oEpC;>Z2D_52sFSyB%~p;C_cSdIir zfqKc8>eI*T<)q8~Y3(oD>7Mc}Pj^=Jr``CQ!NefchfNJW4T8N;1a1sEkske^H9LclHpA#jLoAr8z{tfTVwhSV#;UkNoK^#I z`WYJ!*0y^P09F7tR}urlO+||2;ozQEf zPsLi5d@cgvy^yOm^EHtn#1E${0Vp1M(g|UbDitPD$Dvf|Cpo)D?M|kR4eOrK{8uO6 z6f%x_RLnly>e#nKgJxEPiERxVz|K13Ej=1WLa8xaMxbMeB~tS82j7w*_p4GK?)vxR z^SRkac{c1+To`vplzU$8O`e~3b`gGDh$Tnb%aWashT&8E1-d`n1}0BK2C6h0;+sIo z-Sbj-qAGr(eop(iG@C2?J(je*L#6TK@zNP^K1F|qUqgQ0tz8D~OZf3W z<|XN!`M@x(AlMIaOoy=A`%$3uT`hSYbKE=q&};Upe3v2Mz0rKV6eohrE$Tc(_eBM#uR4Xt^tFi&t*;aJdDxxWuNv3CVb) z+F~?2L^WoG^e|^R-F*v>Mk)JK+br^X_>taNaxomA>~Ln`)XX&)?2}p{UdcZ)G)51~ z(RAm0d${&>n+#8c4i~NxYXtV*GCUToW+H85nDjwKCo-=nro%A*b#ft2p>{&YA<-Tr zmQVI4R@XU8DnXuphs`^`XT3HGu>AU*Xo2^4*T+nn#X8=XlVPK5Sb9M-B)@pO`fONf0=ET5ukh+sX%9hI$B^1uyi8(SBIbtF>K}-Hc?l znl7~`BVn0KAG*;h^$RovEun&gH>NLr5;T*BqWmMN<>b~pt%dd~a$xNbd)El&FJ z?h^W2;5WI{xe{mu?3rsoE*#>ogPUO|Y(YQ5VDT|ipuOB3AwnK#y$M;o_xY)1IYpJ0 z=RgpbF1b1SZ04 zAnqD4G>GeVrIJu=_V*v8aywirfT_}bbuHWIc@s1=hgOr(E@g9nSc_8|NMB4K84zO; z@TtTx$|xc5zDzry3~ML_mxMH+-cSZ7nW`p!^)*xvn}h|i^{S>xTOX5>^00mQ-Pr(> zL^8VK6eJ4YiuwB9t5hC>Eupe$V>JH~P?TfmL>KSXjPW5l0_*n+VGA~bc_BALMb=Ny za1>6~5ZMiknSQ82f}0<5Zmr9=(>-@yyNdU7 zZ{Xx1!(Ys>c9`rN+Mg(hn{IHi3eQ(NpSU{cndBd@5ktKFj?g4pQRwnBgr z8WZhd>AB}@)>{5%AeN`%)<#2o%TW8v{McnLd32qH9@+eqbJm94b%#2KE}#1h#P=%) zg*Q&aHjm?75ioqCjB3W74be+XD%U6XriBvq#1hRb41Ct}bHL4Ag18Y};+&-CQIH=F zDHDF^@ExbuMmv0IZ@cEuu#o`P7|a-ae!@4E)k%Ty)#Jneo^D^~L{vZU?>JS55-=EG zoxGP5D*Vio7Rtel#&;pX(bBg>SIO%5+}jZvrD`A8McF%?!HPnIxbJ};^%D? zYRa{aWtnXdJuI9!ENDSdbq&=6<0bY)SCKig%&wPjuT>upD=O@mKVDj%;7Ak(U8}w! zMw3`Z-!AxRdrQKkh$|w&g$)NN4rkYkkerJ=7M}eM#PQT35;{%fUWn9CQ`MT<7ygz8>b~_PwIKL!p&ug!Br z55Yp|xOYB%<4%D9lVnj@4ag}(Ce!^XMD-gKC#l&fwH`l>MWtkU-Fg`y%pp`Lgo< zdlBddhUa0J`HS_BChQBFiwn+rfeG1v@NZDeR}dyUfqpy#6n}e-F@owHQ9-1xB>(R) zfWrT3k3Q~?4iF*~hDIWyz>+x__3wotNz6&=+L?WjjA2!>a>~A!AXh?&Q z&4?Z{-PVsbw})%~V7bmzS@}6b4tPB9veVLJgpr2n z|12q1JU{2Ni8@67G-z|ym16vVt(-|?7?)lrW#t#K3?}_0A)sk802@)z6<7afzX3xF z6W9m@Xjg3Mzce7J1}v|9Y=tfHe?(D4B@to5oYA8G*YyOMA_Xqb)#W3gXtNr3BWksR znxjXOS&U6)$_y(^2e1VraQ;m%j0{3+ctoREla$}%A|{HES9-csck@LrcUbS9^BjRFl4O{gRgdbo ziFie~Hk0%dHwTjN$)ACzvr9*U1DsDA&pSEM{cBy}Y~?6gRR2!t=Sf6wKqEi9nzxgg zudxA?PDaOr@}7R~?XTp(=BBbZUoa&m4_fhb%@i{d7Jqx+v#KW4RwUUz-4SCH8Ya-4FJ8-h+JvY#WD%rlJ;SG!|->;Bs)memA*}CMi&o~V$ zwvn)C{LouGSO%=II$t=EJmki^OyPic<0STgG0pm6g1SKCM>--dv(S>)xt#k+JB-Ns zdR-$xgITb`jc{a`z_9b!Xd?ny{WxYH7BVz$jZ);2s0ZvE~k9-ZC zBd!CS_0c2a<2%4srJGr*aab4kli;fSHhjW<9z_|C-A0}lT zgV?<3BAVD%2L^bi)V#Y?^lbwA;vpObwEt{gyQ& z2kq6?+NQ58^~O1)8Yhv|e-$-|!61QF4LtJs=-|2X*d`jAr?=7)j`hByQ;6o_3F0-DLDn931TglMwBiJy15>2*M8InQY zPA;%kVsw9eyq?NU=B|Lu>7v4u{uJBzfXbP2!${cr` z`Fm5Z`L=yr_NF>+Ixpp@wwZrBR%X~qHf_y%K{lOdjrvE1iIRhv3HUsH#D4h2t4x9C zan!wTp?Z`0A2Hd>1x1}@u9H;u?hkjxe1OU{3pPRSa8aUQy~v&^vc~w^5eZnTyHZsu z48GQe+L2q{K7c!X`N1|C;G1`LH*eRP;C{Z9$2#D(8HlsZ_i)l#zvcJ%(1tnBBzY^c z@-5XCoq$!83?eWREPYqKTyplJOC`YSR z{nqA37mg@bwy*R=ydmHSK26txZa0%0fm3lqZfImi`r!qjtKztKjqUw?h0_)4Xs;QOlCoG4LchbaboNH&ckdPfgytZ@Ge6Ds%ive;al1C##zV8u-f2882Wdxj z;IG1@u#Z77m7VuXS**NY$LdDj;0yhKcqunNg%db7m48c$Ab-6h+&6!_iRUSLJ& z&lJcJ!df4f-1B8@soD&yy#Oo%t_tY|uXEm;=&RKY04EV7z}bh5=xw%I+N^auH3|<_ z+3bM5{4km=_LG5tW*$J&WP%&Aq70nNVQD`Qgav$D%6R~=&>m*dhnJiB`e-rF`~J$6 z*nE81Z9UQznI9KdFTia?7-o~1|K|Wk{yOO)a9vnY9j$slF}G(faTUYVWwr*P5Zg`* zK3>GB5eh(9(CxXeg;Mnc1W3%0dO<(fZ5K4{zhvYrj2S9916SO8{dGC&+$7hjqugfD@fKaE7>IG^pr$JIKoAZ`7opeZ#}(#(P;ZRJ|FC=&&*Cl;gm{`rueG*l--(ni-Lte%@5GQ+ zYQ~>d_sD0T^*J#=xIuFUU@SgeYPUk=%O#&@?*R=QXmuR^*2Ohy$O~1n>!Z4L$_wfMF57Ve+_^}KaT)ZqGN4+*=d-6y8z0%^^%=i0S7Xj zk&@4JFP)L-!R-j*u^0ha0WS)n0yG^wH?G#eX*ou5y!dhX^5VPJam4^*CD%YAL#3l1 z73}6g!+z-!U{e|$x&gp3zX6ZqhGe4;pySNhO~|Ez@#|7+dFbZ4Dezy$b6QQ&E2urC zvudH#z4@Hqxdi|*S1-Wdu2C1AeoIrnt6h=coac;Tu)FzJhoEX*4y1HeyH4$tABGVE z>Lnrh2rzWxkK;LR%~a`b9gl+?X?MhK4Lj%Yq}un+hT91MNMbKc$<%b>UoEHhOms(N z4C8$Pz{0ntI~^1~ZfAC9*^*?w<(cn-_z~QGGG!Xzj!}rX zrwqjP3F5E&=ljLxL7!VzD>&cm{S0IU<#H;Ap04_-JXD*=}EfkL*3 zx^GN9QT^j;q$9Amh?JR~C$Vo#eRp@H>}o=UNRp=j(DZ1zrwhP~whQ{&#zma?`0U+g zj98q1wq8zYN8+vXM5r>P<)j--^lH4(73>4RFY^dumk6_+hOpYM@WRx`8vsYW91%dp zZIHfy;^(*NRVi5BpQ|n+{`L*rxCXd(q5?M)smcoUnj3KFRq}KcIB-GffX#8mShoW4 z%sJ=lILW)*aiVB=>L(UNu48PN*@DN|JHS9lSwF5IxELMnRbD4bC=5gFRhCZ$c6aTN zrQC20$k;nWO%nqG2z}*KVG?W=1x_}Y4qr}xnr2n`AKhYocO*X}-W@J5z_ZM`E%{`#Z%DaWf3Vi>j7dReAo>sP) z_e@xfvxi>tE?U6;m5YCae)TI5(QUmi4-u480QpRmA*F9$`)CjnzV-ex02=7VJV)Mq z=N$x(GN{$)-U5l=kbuE_?+s`~&I}6nX`8FIVvr+q&yEuIfxRgD!1}+y~_t z#W=5;x$UkMOL9}bn(y8J3fqQw1HiYl^1%R9s0oD=blgE=x|WUb;hyW>-Ee|@(sXQq ze=@{O3PhWwEf*k51fJYv%v~bJiFCSv=019TcCJ0`nW*!o{=sBDADSGx{i+!Khi-s- zfYzGq4T+y&(FJYUJq+Aw`6DAP@pMS4hO|U_p&?6;)T9vX!G^)a2We(cPHbg+P0y#> zZ814$w8^L>Xkm~ zpbm$Ozd7lbwm~$a?!j0}?cFq^6UJ*J@+>MgSQhR8(oP3Jk<|l@l^gfmrLPz&gJSvs zlP4WQg1#E%KFT$D=3K^Ygam0?DRj;fX9W}H8zlS`X>0|_o1BuErgmQ;3qf4Iuf~89 z)=Drd%I5sDha&gwRkGeDH*4P;5<>SDz@t&4==V)*2rPRGwi9hNCUbJSod6{oaB79j zZ6+e)0>A_TT|453=~4z_pNDei-LWndb+y)2qzGaAv7i3%2vEwWB5f=#RZS$x-2HwO zQMVLd*5DZ08Ei&(ZyHa~sNd=x^$8(eQt>qbjy6!-)P$Z5QSS_dE8zZkuRgN|P2&id zi&u{Lk}kr10Icb{b%->44fu?@mtcc&`Y2@f?9G;rPM9U06!D63lB8MbPg;R+@pf)?Has$aT(d>~A5N2=LAX$h9MtBh! z%ZnW#L?h~igW|-LtuaJ67d`xfNoY;PYYyp&(O=loZiYyFX91MBx$R^zs$OVdlku-spL1G&~mS>-+0s)IiVolK+Cy1jgqE?OEqtj&O&ooVa_fz8?XYbVooY%wdy;E71Gj((WA})K>wJnW>pA> z%rDP0#)Y^v<<@HjJsz|;J@|`YHDzB4pRIV8Uimae`xNqAIfo}Ks&t#Uxau9%p_sF6 z>k==DFTm)!Zze2BQKeNK_Ow2)4xunP=$2^DS`at&36IJAvIkGIG=Xmc=k&^l5mk4<%8RcJa=jhxsT_hH;|mE zmROu>l)N{U_i4aA9d8VG+eL+0hh0}VJemTz9UFi2!`ClNIlaL8k^aTtKwuAws*dCk zFqhI>5*TrRT9L)LLd84GpZl5r^ghv9TL(2B?`(6#_HUXpDo`w2N5#qK*O9Gnrk5gj zn;?I~4PP-W8MEB6sm^%Ml?+YkVr>OM4hJA^iNZ0Kq>;ukh7M|nMA!$xS;$fCH-Nvx zSGcp1cZcD4&A@QIsY6$@$&Q1~?Zimo%_fz?3Q;7IJuG9G;&cy@=U`1;SJ6k!1_Q5) z+7BW|wi3b~7b5Hh7~k9vjJF{|VKLbu3Tl@A_zHVeh17jCcbaJNdL6oC|OWaG&2*;_qFb9)a*n6t(y)Vd94C-Q|5lp z8M*|3;W-q44hszg`&tTUqi>CWmeqG|l`ttw(PsFEA^L!GO&N{~({e$b6dEn&Gk&M~c1$)%j_7GKhGQ?h^qR3@IEG5K#>Ge#c(pU%S#Aa>)Fv;g%Nf@D@qh}pYh zS`n_uSi7PRSCAMsCS@Cvj}idlNg%)r7syvuQ2qfrv!MX&=z4{3_1BPoslj>y%XJg* z4>5Eli-|>SRxSAK@m=gmKkiGKZpZuIMt=n{_~7b)jK}vJR{amhtxQ~)n&^Mbx?Ifv zEyo^0iGRTcS|4i;Co25|0^a;ZOF?M9&QaV1PTU16sJ)UwOI1?FA}a;v`th?)+e(jm z_UnFsYMw=a;c{;s^rX9q?@g94*fI_~1<593LcBToHd}@X+wG9SWmybB6oyJ;L~p&j zWm7ILF|``3Q>NZq%_No@{08yRYc2YVlO*8#423&eR#9gOLg$j z^1nX=XyG3J;MH#$Rp9y!Ft)xU^0rR)UUI+Yx9Vf4U7~Eh)4>}pL}O|t)BUwi3BVJT zCjgEraa3oeDEc+~DPTJb=aR_m{`XR}!rH!6TjhyI?m6~#@jD*Omg<#!8BF6;xU!w- zeSbZ)L378N$^(+gmrrFJ`tmk5FdVy2Bp|r=TV!gkN?SzElRB+;MDLOg#=4AP_q0G2R=yNoDeCJEw2 zWj%tY&tIpj!x6nr$|9Z*5N7Kv<1!?a8iB=r;P1csO{*|i2-A-OuB)erFSO5UDr8h0-NI0rQGNLFx7%=~ea z_S>R!e2d1<^HMoMktTN$cBX5&Qe(~c&}TdAodCd{c-{~_U^$q9=_zw~QHU3K9^ zFdyqg?#bAlt+PZ1h)3@|VY8f35829G(U15}6y8B>D8Rp;h|PfPpcFoF?E{NhU{=oZ z7f3mTt+I5g&{-;~9UR4206n@?>MZ1VipC)e+>q?f|7iU>e|r#eqE)&5oqSOYd&>E_ z+tDyOJP*{|P+@lH!M&52?U_r5Vjhb7>OuGqK%ElQc87$YE4|t8UWzS`ym)dxtX%S8 zl5ab<@DtzHH)@is#DC8*57b%b;HV~n+84M%zQwlj0sci;HlII*BoNTPfH(p79VGk5 zLAm&VIB_N){2Soo3yTQ`Fak6fYdCO*esg4uEg-+fBbm?t<17Jt30nf|rT-gH6C%{` z;>F4A|{!u}g~_r;y^e<$+)gCaVr;kZ|KW7%`;HQ_JhjBx>|ZLXVS zDjwKANbF-=FPnZZW2PEX{d<2foq){=zXj{HI&UBRi^<1uzqF9oH<6QLB;wLwJSy{? znn=R>?dHXVd8seLlwTI9SgnxWsKmIZDT6%M_*pjX+zH;vT>bB!YrNc9VLB|b+Xq|m zzrqc-EBk;F6F%T<;4O=#D7BNY1v?*E^e^ZyzWD=Hie>_^qX?7eNoC%S=HNqHGo6VH zhNi;H0T7A>F5>3ij}8Z~j0R@?(x~9RF0@s3Qe(5Bgby(PIU>}2z=#Ma_wShSkZsC> zMaF{m2nP9kXPk_;U=AwdxO=fYL^S`(>KNspnQkOtTLe|!Ig>Lo$U#RPgpd+kd^4*< zL`@mUx!L-%P9saa-*Rxbq+ZyE`t0Sv-ZCyt2{BtrPDB~+m`P18l;Ej!wCFRxm3-X=X?|bgGFDHFS_h^A zV~oUl-)8g7szrH3HG!ak6VGPqhHx)L2S%yz7pC1n%lwqiXQq_`F^CQY!?m486m7U2 zthN0rM%#c7G`@tU+@T<0vZdt$C{?8i!8GPv+PXQDj1B52QBpvyKn?L z;&9jrrn%DeFb~$>+JTVL3st{gm(>h6KR_d5(AqM~&5R?2XC6h1sW$e!3tD~IOTQan z?L9sDk_$+z_B|6S1+|rh}cs;$c zZvi8-_~tu)uCvCm<|(fHA4wDo^e3^Vv;i_*v$Qb0I=5KaJtT^R_4%!GtdR=^5fNv+ zYeazKxCzyWUC3LqAv!ue>)mfQS;od01|99KOdQ3J5t7yW7$a_6y;gwx|#tudU_t!e!a_5pm_L_ z1F-NA%K-;t<_@V49KXmtW}Filla9j}cXC4`@>@7$2Wa6?$#KKWa5c^YC5gp2DGa(F z*#yX5BiFc0n$)Y12PgB*P5IQd88$#CD}uloeC{M8FI|8lESBZNV$WqsZE}H#?wS~{ z7h}+Z;ho@6dZ&}`Ey*q?+#_Q`8uyC_I>o(=fZ3G`XHD!K1+b7X7zovRa|6MESm?>I zQ`M0eqxKU*<1VP--Zzdjjh%Xt(W?JG$Rm1R%U$U4qyHVAY4@ei_8rl8Jl8+#lj* zR`Pdt5@LWF;7qOQ=+1K4r_&8?%aawDY0PE24SX=^LI{GvdQoC zPChz&(7z>U(JW3re82zM8b834!YOSY0`4-9a8dR6eJisxX6sH(@MZ#al%b_@U>7Z? zrn=_Wl}CdmLx+Itb={tRHoY$8WiwJZ9hAF{9a0)cV z?_mL7XX_>qq82BeFgA?Yv8TAuK@bqth5Gewq8XB3-Rs;H`FoA=W)iK};{9sCW*yv{ z_wzA@thxf!FcIuZ`hGib$l)2HzI8kk%X_V&MZIkXpr7s*KV)VST*XPbVyzZ!T2mqnnx0;|<-j2Hv z5FXYT%&AUJaPC~Vllci(Hoz^04`Hp{IS!H?#&(P*#4+HIA;fC>?FCM+r8GLje4d2Y z3S=MLrxxeOqqU77eOm>GlWX)OV*nZFXS;ng9ZV`na{=3kb~yOAx-g}nMb{?GLrOe{4ciV zY4~rTdIs2YZGMRiz*f{7M1T`MY@qY6&MW3`1;f9;i_*~GdobAAjmKR} z&{sXK4@`|4qe;V=O#e0LDr^DZ!8D$znJPM8TRNa#54owd*PZ3qMyo8oYZfz0C%Q?) zz{F0#4WIDOg)!gyf&qTBGs4YC#h60yE{JH1H<>R@6?36s6}Kr>6T_VsuR=nv%+siA zp9o!kkhG1B8U{jMy5}B-8B}NA4~_;o_6cmO*20fcPK;$9le7(I-~oXmnX8v&+3j zdPrUl`^shz<;D3d7z9#pT-=yryADY`^5L1_0Rbs~YoEZ9-#&O}_I>K{yx@DSVBpAQGrNa(?%Eo# z95a&@6rBg^vCII%G_K-QXUhH4l(%3`$RPXxYh`+oog|yJ97LeaIhBUH+8W-(^U&(a zyK7p^2TG=NOauK5f<(plD|2^VExcoO9R1b5)A`1X`N3&^`;nvTQf;y2E}sLGC!M(P z%X$|C#zclntr`TnZhJyYSdIRP)@BrP^Cn6|&j@|JDKpXS^;#;Ix_f8}&CM&q)z3CP zdY*?g$IZ`v&RmD8DI8Wcs!KFpt1;FsVjKd?m>Q~{iHE&wc_{wkTYP%AgDdbqecxFO<( zhXw!)L-}k3^14q!@R5S|xT!LpUTf^l_D5KNsYo-KL;e^S%Hd$iAH%&3Lzl%*DV%V6 zyk8DQKrSVih67UVj0oJ@s6-6OHuKEKL$6U(lHFv}(NSo4JcUsy~#oZ4wfLb0t2W?VH^Y#%*t-4^m4#q*J|x8vg!KHIgV&>wp<&*@D7ar)dTZ04n3oyXuD zZw!NPZ&x5N+C@-MV&OOGU~x6a&Ez)fjH5zBR5S${8z0~;o0dW9UPE6RW#-@J->8AJ zxtZ)3SJ*DT2RuhXphBr0P7P0P=?OJknq(6CwgVUpJS`t*40c?;w?`X;Gw z7@xDh7wx&(X1gFiBQ`(XU4F^KJ9q1uIOm!!b`z&|5tOg0_Cr%K6#dP{=WtKas+|<` z47Q-g`O?OnwG;E(cmU9=oW0e}j^|#&l z8$U4=LZ}<5ACg(twcR%-r%jSxbC>|kKc==^-Mr7(cq6BM@_2nVv#ua1XWFspIwLI4 z_2i~~67h6G4y1IP_EUVA^{-Uv8GnEtrm>lqX{^bYOkw4{Ym{y57wa^G)7 zR`#H~ydQZ}5@k!ndD*HX0H<8NU}I>55+a(b5z_f%chfE?oFMp zV^@by7T)gW53)0J*^P1N!N zd$Abz^w*~im)8+T;?cnnjSQBCuL;s86M=R>vw*V`dVTiZJk`YeG4gR^uILVz?buaM zW7e+K+jQ-x6syTjPWM-0B34!L=G!mLEg0cc1T79%TIP4CX?fELs{5}XAnzPIL>Eg{ z#lF0q)$ay9gtl_stqm}lW;EAEnhH9gHTW7h?T`)uQ6=)&^m>U*%QcG#2uiTLnwc4{ zAGEm-%0v$9K_^+H-6Q28@STgo0m)Cx_))h)86_tnd* zt)>m%9DF|Il|k+0!M|JUl7T{Jm^g|8+>TT|HMWPKyXB!&;$n;oFFgBsq57jc{LP4Q z-rH;VbBIDQdgZyAW0It+gEwsEzbf_)vg3?%f{>tK&?k_;*MG3|^wGInZ(W_AFli5f z*U<*u+l0%d>&qvTh!DA}(yGrXX`IYY#4@;C3YZP{U|$9kV}EI2s7tnY#|*TSmDYrH z<$ILnUO)JC^^K;gNN2XfF+~3JyiIKyVqIQqCNYusfy4ZIgU|Y-(b^GnXz(%VP%04S z;$x`vth7~Q(0)w`S6?Nzi~pKJaxpYOTZ{q+lex#`aT+(L{FK$J2{Ofz@cS7H)%@9!%%Y%2Glj45oO&La7toF@ zo*tlsj(0|h`+KML5;|4gz#h+s)pdP|Nc0O(puw{a6A!)>62mN`vGDW z?1M_k6+Tq^K>q{bU>P)jFyBb0I-`_Q7p1S3BXZ{i6)AZU{*ENj81csxUxxb~+_Nf! z0*0Sow8}aD-LAPVfzYD9D%_Q=x9VnfEP1X{K=a0?MomF;R&}w)b!#UBiHJtAs#1&M zArUp=$Hvp+)`IbG_$?}v6wZf}8uw#7wzan2RYOR1PbLp;+}W^9$s^s_lF$6Qfg?QM z_oqs=iIrHbB^chbN5djK`Kl}|xG&F&AoUS^>bnaQ(CEvF=A(D9nl5K%Zr2cRp(IqW zV#@!xM;$XaSAA>|ca8j9>dIh5`VcS#PmD@5@*6AOrw9y1Rn^BK)(2#Q?mDrax26VR zX(B_okD_qo>q*(O3yfEcHRB>kuQ{wuT&3u(8$PzS;p&n|$JQ4Ex~Zd5^wAX2Ocjd7 zoa(n-QP+)F3C$VtE(~;bJ-XSVZPPti)zJs`DDH1+)JMq4ZZ~A|R%+&;-1jk9bQ;pZ zSBD$q31s19hu?>>*-fYl;$efl#zK4WY#AWKfAa4i7Rb6+OYc@gzZKz2YTP}43KuAf z0!IG(-!68Bf3BgeR%=|!Be%_b(+*{M z7GE$Xgy)Ta*~|^7zZ>S0LO_puOFSWx?+3_PE^6gVy@^tMRF1l__!r=0Oqvllj+C zsgrh5ARUuX<(zaa<3sRV`8)UUfaO3brbsujmH1ch3w~LZq4}reJlaDP|B$Bxl0eJF zU

      uhEhsC41DnqfM2nO?jI0_E^&EBaMr7MY5jwcltwE^Y=_tjnz1!Ix_ZY>x0m5& z)zdQjt*y=vRWk%Lz)V1ySwoJUDnzO?>})n#OQ&Y16b7AzGIDMA0|~Kl78S+eFo&bI zg^a;kkU+Ma*GpSeS5<4Z5{ZCUp8cvEbbj4FSM?0ahHC}Nc@-%<28lpcCS1jK`Xz%r z+4D-%ZLmNOxxTkQyXA>KOn})#@-swF-g&wE4NBnt?Eto+J&CIA_s;I}|5$=6JpE zgz;+2r6G*C5SfE`&-dwkSkPpfR?TFkP1{M7U#dCWbN44QNgIaSMoa_2q8~MZm z&&w$o*YpoKZwI=brQ?iyK*i`6h`Ykjf}3RpPxz-kYeYZAF!= zLOy|Z_qnh*2iK~yaLDefVAPvt-0s&6IAUomlsd=lIXbf{Fd&rd-XIF|3Z? z{Fl=~0^$A+aEd6Dv7o*9>t3+h!Iu}6cLCM^NvoiG!Qr9sG5%#T^wI+_*op)-@&DL4 z|HnGX99iB>mn}&ue$cIXIE|(}&9zSXtn4{$V-7_X2ebZf($rzFCP*Sax1DNdr_=kU zLqRs?SsF!6%>?63VP9Q37^N-;%=l9*`r~O5UmXi}vwU;IaV&&8ZWjb{(Kfqq^sC~= zDCNI?H?MrgQW}i9Lq0Y0kargzdAL1?HU|;snrp646v~o)#W@+`T$UWbOgpJE8+x6R zEAeyfXXTv-g$X6Pyjv&lZ9skn0}$!*lC?Z*9L>Mc28lBG^TWMMhh|5?2PFKr7oBIG zXkKq~&O2C2fEao=XM6SMCm_f7!XUGvq7nW}^S|E$0J0PMO!!UFja@$S&rp5Arq?vc zhaz&{h($ZLj*ilpY0Z~UE0p-bm~ zNgyR6^meOv`6qc#zJ{U^e_GXJWe39e2*lS3U1toJ&?E>}cXu_@1*n&Lh<(1wzOC<` zgrqJeFwMo*z(E3{AdBhagB5^SRHq@TRx}`nbA@KStU|l`!{+X2(h?8}d1!vW;56(G z=-KB!INi{DGvAeRyl_e)7DVuYaQMwMud}}N^tvsVgPGC!3R9Vub9t9n-gg(LL_oCG z0JibwCk*+My~(2OG{=6LbjFLOmjqRnpp-qM^RN0BT=f@YGR;7E5@ozTkgLI;w&N68 z6fxi<@P*ydXgJzaJqn1p)fqQ8`UK=0FL<8yQHc~yRavUo0jZEn0J3XkV*@0-M7bpg zU(Z=FHAdfZVMyYn>n*o7k=5&#)Ar(Y%K~*riiT8>m^UPJULT} zgo`G8tRyt z0g~@ca8XuC=!Nzs3f7x}kjN`uj3=?^I1^kRY;}FFQ!(YCHZq00v)nNg zqYVJPURIIQjVdfi98o#^8W@meoTd9s7S9g&wFNW>Qj4APCcLf@HFVBzhqPh!P~O$f zyUGZ!fyDWIDrXOm?DX)CW3U(Yhu*{dsJJcl`M`XDSlgS!GZ`70_QuBbvaGQ_c=v5Y3G6PTv1hcRyBq(zUPN?P>Q135fEPcWS5MqI#+Q%sRobkeX{{z9C2*J zeB*1MCO3gB#VMzv)Z{`vDn})8O9wwKfI;YWIEc=G*xI|zmrNW-AcP)2iVJiN7_6U< zTkhq53EW%rvJupEAcA*Xw|jK6!MYdb(gImZY*-sh9%gS#3EOr2-@cP`@5VFNYz_tj zt7}J-8ko-~Kl2E`Pm6(pr8P3Wo06dI8UV6Ij#Gt8Tkbw64Fz^}wgM4mu=X%C#?=nK zHDo>`K*q`Dgo;9SO~yBKAD^pPbL=Re({7X_fNt}SJ6XZ+m-jlf#G?pkbQ_$}(k2|V zhjEvMZRw~9pZJ8Igo>>S55*`g zwsIz8hMdZmZt~J_vpuQR%zaL^qYjU)!jhxdoBKjODV6d7Zy0+;?5*xOrER?|cjFcV z|NBDoN8BwJuSPqoy}=UwX4qL(S4Z5wc2Pn7Xm4KG3&I9GtcCN0Bx-5E=#Dsc-YqB!+HQ|wq!<$~f zs%QPSb=25u53Zb>!?%DqptYTTg^!xZQ=?k#x95KD;6jnETI7SA)rEP*hziI@X6IU$ zwBUt$2NSE?T*XKU$%2`2<*pZBJ!_}RUIq^$?rX9_`SIzvwX^gFWA6+0^);*|Jh66(vXj(W(8_;PV#2!jpr%qxK6ea@O#dRU z&PwY@j``keG7>}bj3t%6N#)+4KQcClnS2_N*n+$R=xIVW)s6D&p>;w2lzA#Xagt`f zMv-J(rI}*J9%AwCl3r)Ool$D)%9Jj(y@(e*-ihHCER^rqn&a8DwQx z=&^0L{Hwh?P138*_*0z;rLZOTq5pT>fz@c*HMFtLsLjuUPMbruXL8XLkt@+*?(Sc& z_M3iWaWD3BHw}zO%=o^V-_D)9+yJEMTHe*XjxAmSA+oLd)%HEJM@3Y8Wx zjvE=J)!>(743+wRAw#>pGTzs9Ll>gFRZca9f0WtnUZ0)iOmAZHg#1W|ETfUqOh(gp zuXal1`9pDphrzS{vVG`#cF1(S!kh-)z>7Xp$pS)!ha;^rjNziF5G?99+raRnHB->y z4e~qaRV(^%uvSC2*Zzo`d-=$Fs7YZHw`Xbk(zp3pl$uEpFArEncS>ti24kGJC@08m zh~B34U@pPrRXZx~5}{sCVThTrzdl$v|S@VO~Wp> z7^nXL0o*cJPh9J^9R3tCE%Q4E8y_Kcg+!2^K8_+2}cJp2unf&UhXFyE5(uH2GMJH+u1WOqxV{TT+_Tgtg&xQ@TwzT*827yajD z7fx8JP)6Eu>cC=tI^7G!yZXvFj}g>e^WnwMwvmeZm3!|eS~|4%Q#e}bs6{Eoph8f3 zvR6Q0h(2SmG+-K&^Uteo<3%Wul# zR#nLY{L&&hmt2yJ5mlE6_cx45ATD7fuGNR$Cx6od3+(>NSXeM5ejsr^u070V2k>y; zlqQYn07IQZjkcC}eC@%@!zQ-;{m?#2cA-MaWuI7A_KuTP2v4AktVH30bMk>wKm#)5 z*{$UHyvZBEWQYPesT7Uxnjepbx~>wWXw7)rmU?CM>7H99o`&oWlXXyY+d-_+2ioTq|jN)g^jK5c{V-T!h~lZ8S%XGo<#?l5!ziB zcD1V@^|eCs2?z%G`%)Ar-U?Wbw6^Eoc2eT^Epd8Q`r`4J73UI{6$aZ~fYr(n531DH zGiXY}$x(krpYjr>R=NZCPKp&;ULC;Xrk+JPwXVb;#V^)(e7fR`(Lah7`Q4MokfrSc zk36a)R_gK*g35}}Wfp$7n&MVHq+BIo5i*49+1t9{zas~36S%cvDw?B^=IrXJkmNyP zjV~pud0Xvl3~%^`sa5oir#HUP29lyYbgv7{!=`^~qYn$8Li3jim>oHprLk7s@g6H5 zD}Ab1q;G<8wRGRO*Y<^ziw4%u-6ps_aQCS${8IE2W%WDcxV^j)305g?OS}><~Mm)1+Apz-Ih~DH}^wj#Grp^c$&7yXGtFQnCyty zy|snvdxyjRJcaqF@xoo@2Jg2>~|*bq|x>qa0pU zK6r@IHZ<`|isd=7C|P1E5tI7`C$zIDw?S~)T-pTcxk8iXORhG#N^^Hf<*aB=>U1G5k94I|j2byS;-P*2(_1)io=1BL)9kq<3j8WW$hBW-+nNZpV6%8-{KzH;9L95xnI*%6FFveg^xJvNp zIBz=HpjHu$m!C<;1ky9ILE5Gd#*!e;=eb(_OFTl#GFfXJ&6qwP9bP% z8yG#0Tj0%?g*U=i-sJPz#Cp9{FxsZE7C-I^#KWQRA$mBHQ0O5)kw?GXv})yOFt3m) zj!C(t73C*l(5Q$kV7WijQ^wT_85V$PXeGR+Go&722{yrl{33gjCU4}iPWoc4m?z{l z=}&R8?4Ow2zJ^lQYN#l)M>%<@tR`nUVLGtb03apkd8nJXp|Vbu41W<4{gl?VCaVNu z_vryKFNR=p;eZ;hh3Fm;zL4k$@bR*(dp3<@Y!SN0wMDXd{G}(rZ}g0X`g+6uQ)1L4 zs_QF}8UI|Ht;(Amyvh>MG3B;J-PpP!-R3Z{kE%V`Lr-F~Pc4^iv+SG=;a`IIDQ0s^ zJk=*-H?~55v!Bi^yiN`sJdpW1EP0w6ZiX*PS$kz~@%mS8IS|>%peiSUBFKa_@(b>Sq1lDks#Rz;ZHY(m zi>%P&i%2zAIc#RQ%}zUpfYf=v4Vi2hb@%d(TGX`H{=)n0NQ{2G)%y-dcxZ<1S92u1 z`7r6$_>|AM$nd}xNUW-$F*EsN#pIkYAt5G$y|8W%Xh47o5gwz+peQ=6t8^;NS&St> zn_!!ggNGA+X__pyReL^|mYPS$-MLx&Iy8~!@?q`0y6`q8l(1$?$3#765#NtxaCBox zg#{-c5%UCNRBktZUl4JA+GmqbvVW@Bl2_Z3kvaWeGbHthL?q+Su0sTX$9AZbVpP@f8Vyb_+hVBvS%|pvkT8|12%V8dzYr3*Get(mNkW;Xw(h- zc3I4N2#14TtYYQ2I3Qj?{3j+~;>lII2G2BB#-=Dh_>hrB&X-XwV9Dt!J~Zmj(y?63 zsjD%NcO$}-F*eeznNplI{8HXON{#U6{yYG|ESOYCXG?soSWnNIN(Ft7NkWI>UWK=t zQQ5S@EEpknP6{X0*Ct~`VpV*`YGH$YSk2fh5TPvr7?MU3R~cn2|3YALGJpX|??1%w zKQcctf>5VrS5>rs?O zOZTW4Bjl0oS1A575CA8BgcBgA+pwB$lRkd}@S%R)mp*2jx{KZ?gAtRNRd?VY0kVAd z2!&V-Ri{Iedj>0Jw?0GEf7Vgpj+kQpIZgNO(4bGEysohS=hINcr;?gaAHU=L zHRam0SN9By^=th94ju5FfXzA7sLonHz$BefGHoUXI8&6TW3LGR?>^$NQ9_1B93~dQ z#{c?_T87x<#6E(;LfiG9FCV}j>7)?fnX9#8&GNl0;yncf`u@Y^j!!@YdSo@&bob!H zWgsXaE%-NJkDZJ#HfcElS#dvretE!VKC+PMUk>X(7k5O+w)`MOQ5Hru4j(TE#16fr zq|i1|TTo0#)6{v)F*I;@4@}{$zyb|T>J@;8`24CMiOeKrBh1daPo@W63%gk!-uV^C zIAW4C;z~Ab^9lX$-UvbgAri`tbLTJxWe}-d%D#7zz0DA;xzJLe8e z=guug1`2p1#)Yt`JL>v7gjB?gx@-Fm^2C$}pezn)=>H&CKE!YEBtu>cK}5AuYQWp zDY{A~kXLf}RsLxE!}s*NkwqenFc0L=DUAct%!RlB+-&_Cb4?%+PGS`FI5utADQ{Y< zoieIvW>rhy5?^d+cb#`o;#j;x)Qx=!mnBCFZr{l}R!xxgD(p1x6g3kae^ zT9;c22v0((hc8NWD`72L5O!lQ*YGlKlkzD&O+f1>ay zquJz(7Viep;!v}CggFTS)+k*$tUCOSf%~c{fOdZzU{KlnYZq$JD|xm2*`Tb^6#_!| z?yo9XzIIa$tj*Qb1ghJOz;4idUU)7l66Z z^$IXZJm4~Fp1w7Nk^{80!VO4&H=DMwh*{jPc57mn=Zl|lsFPbGFzJi~@9}u#%U1ve zekXWXp;Zu5fK5;VKuMj07j|C-sXI4BhqoN6!BM;brOR3Fk77Y|vgKM&{NEE~iHV>e zw>RHzZ+LX7qwxHI>bDt$l=Z?@@J-W`AhC7NAPoGjA*urqUWN$~JHSjaYL75#SG^do z5Yj_1ZM+bMlrNjPA4wQSxi=lOG)*;oIPC~aL^3e|$=%W}+56TJgiWdR5HMS8mdBTz z5C@>db)~jJFz;`0O+2Lftm$F4m*>v0+2@x#6<4nT?oX)xiCX=f=s6U64IGS?7l$_@ zfxq6!&Sc5^MM$!{O>VlBPb2L+EOPc#k7A1I;zCoCNQCk2pJ!?O7BK~5O zC%0Rf4@XkWhQX!*h|!x(VnkWu3n{1zfCPAdtnJ%_Sdc2A*)41kCR~RVpbf$Zrln`9c-! zoHrMBdgZeh0vpf9p710iQ15-?$T={!IoKs1f!MqrN@-i^vsWK6cs6C=6kYV;Y~D%T z16cB1U#}Q$<=JdMTSKeOeQfh?>Y1(~GXhlx&O5ybw6$cXqG?~O;rh5=ky*3|hzzM* zCY~OHci%f>k!53+<-QVuaw>&c?7B-a4;VN%?CHR)!x7*H*c0v=JHV5-!twG~L8Me3 zQ3WDTwmrHfH7iLp{pG5e_>hl zy(*g|_kuo+P3F*jevLunE|Ww!cM8Ur*-cJjMv>FAv3EzS-B*9)RIw(R&ZILPx+ok0 zSl$QLuRSRvdje|??a%D=Ge7Pr$Xt!+Bvs%j8v%e{`sHUvJ+*^{uZpaOO|dNq3BCe3 z4$z@1VN*(aey&AZH+1Y}uC^Rykgm{M?N)5P!RD@S^+h2zTS62x8s~I`qr{VgIdM!l zez=E+y?+z124?8l3AKP_-0xrG-M+wYb@otxv8b-q&g(aHq!S+Fuwrp>fOhy4>-lV* zjg<-1#(N!asHW5{@>PVp>7?^6txELn359aw)JWV zo6=QidyE0diJiY8)zcvvY#TU5TEQ!=(i{V?kh@O+P0*TL7M?Jy929TQ zMZ)K0(IRNc@;TE_BYp8A5vvm=KHQ+{@B^<&XsWIT4~*wGvwbiYiZ9a^$8U}<8nTbYzj}FT3@JZO zZj9nc0(i}BlByw(&}28$YoGVD6mXDRB`OOsH97y?8AI{OFXFOq^!$+)LDc@UN(P)^ zoWHSG`Za}k*t%fI>kg)8k&%OkI6$gbDb`N)k*K^NCP9d{;m8LRalP1xr$s2mGJExg zGrW993t~p_+o}!R!Mp4niI;l59@B`{k69AN@qST(jsoI%>KJwp>5XP`OY61jDONJs z#80}6U6q@Q0si4>yKAS+9(O-3?KX`z!0}1UEBZiyvCUzoQXi$vo&(}P;dwl3kIl#N z$+AMjMeTU*+{XmD_ze|g&dV1%5d!V}WSQe)0Wap_Sfw=VR#8v6XK`IbRB*}%;J)>) z%a#Rdc!b;V`LG8l4oUHSB^51i*lh}{WHs7Cp6W^F@ZmD*O6|)=hkdzRMir!)@Gh@t zA8UsM;7`QD7};t*OFfGY^&r6*N6|Ih4K_<;?TsZ!6G<&fS8Zki43qb zk-+!$+|TESsTL!m!r(sx-P>N!nCPY^z^Bj#aHnSi#Mt>sc2O{iv==qTg91I!SHCyn zbtzG6C*VZC_4OAMe$+~|rxF~v4(49GF3gsIJPIocH-+|a+akplfVOobD>?7NUR%b0 z&?YMKsvM}qNv4Q=a|GFlrCZi-o$i_rNqt2AAr~93NyjFEmh?}z=l)?=xux8V%To@G zWQ+&#b&%EWrmplL(SE|&aL*NBpBZpX7w4o9ww6B@I+JbOt%cPKJW7}lpWnCP$Vmj*A<`h!%V*!(cZxZ+xy zF$KTCoaHYca2du>bf>9gf!h`tl#Kom-vjLm16zK&ttur#*molBXPuexylA_QB>b(S zTpX;*gkku=eQWGfQ5ON`Uuf|wazV3H9<0nZI^%f&$maYt zp8M%&^@MLY=p({-|@();& z!RSRNIt)a^hNw_~lP#`$4;=kF^SI`9y~;yaukcHrKbw17Jcg!nF9EwR?4xpZ@cYf6 z(ZGn3ul?y)t^QRo`{x)E?!}HcgZGb?Ov7$|!;?~+E*|eM3u?aR0~UrS!}AiP!JWFjWuA`eKWirE>!rSt(){Yc5M!& zR;WOz;$u#^m#xNwqGiU%Asc8!n$1V3HZp*P!Ijxh`5Z;?X1bk7ldFsPvpn==9My?9 z%>;{;384y3%o95ApqvuiS*F)X*xgSAGwR>0U&IKjIV!~5!_0rS2pRv}3`&rpf@ECK zO}JyPCISgJY9)ulhXu|8OmZkqrMA50JO0|AI>aGdmei8^p8O25QOd~8SkB$SqDPJ? zw(o&B#pWwyHX&+;32kk7V-1%R({`9tt@o2&yjYRMAGHaT-vu6mZ&avPah!0dn3NUh zzF{&Aj__icyy{Sh#lmHGQ5kB?8q+>`Q$t)!sQ)#ay%Sq@$v8TUisL2guU8r!CHIZ> zZUUE(b=fB>f*^BP&k`9MX67y&gd7C(kgN7rl47#_vV>)YodWB^CUdY9 zCwtjLIUA+7ih+&8S-!MKs~VTtrfuBk|2ZfGZGc~cIOv$4W@!&+5d9K*iYhC=Ug`e_ zYm9XpqgRq_T}bWKCwX@mR~J8-3=A1r+}-r!4fN%IuVPx(-xG<_8-8rhLY&u8lXIX* zAR!G8-!?U$S^u;@;t8jB1HNc;{U&ghR(|40`)RZ;!9)9vShzvqx|7#F+(_1tHzq*R z)!85Z%D<83-#5aVZXZ}QeN-x=Ap*RuRmDR$QJhFq8pW2Mi(V^1j)P?6!fKc_k*w>W zrK82Xd7XWdoK)xVCnzQzu2t7WeM(ZYzvwGG0*(ETLcWsn#EX68eiU1WhBb@n-3r4F z9KnN$O`knwN#VENu^p46&b|!YXytmIyA$$RFG&=gygAluWSU7%uW}i>PF9t%3Wr8^bCSCv|)7cvlCv4oy}2s$^`{7#-Cr zSqg@#{$(}miExYCgakBF*UCTpzAK)LUXp#E+pmGy05{(mcwBT?@W-f4; zdE37%L$(hTI5RQwIwWB(N^JPU7NrfNovXrxk4OTNcz-Kkn>Gh3wu*k0RV}7j&6jh& z+Q+VA;VffgAON7^j}|)l?!aUPNHujp-n0Fgg{2i8zn?=EL}I&(8W!!#(FzdF5G0BH zk6&-KlHD3l0tE27yR|KUj~VK2s6&Wk0ouvH#(Rfo1mGY9;Gu2=B})IQtrh490op_? z8{u$=YM6=uz=~o)w)8)5Y=QsCw&?Vy!*fp&I zJw+9NT+pUfcR$bE!Seh*XnHZ{F$k~_nV1a;Agpl&wvWmHe<2d%pEH92>;~8wJLLhq zpc=qT4WLZ}JagT*!55C9M&NX+T8HbK(+nO59S32L3+*LXB{>mrLrj-`Uk(`q zyF+QNq3B!S+L;G12|nv-m?%K+Y^v6k!W*YQzfSzUGo`-$ql^RiUDh_Sd0iYXBYJco zoUOUoY`>oBAg^|bbe;k@7GSaL)sE|r8$d-d53E9jq9*b7U|0U)>r-^7;FvKl?tR2A znZ91H9wfJB6<^O1Oy>xm!Vm~>Tr%cb--uxe>{k-c%Y_?(2y z1i(?1YNf`LO#5~$KA%fxt(U+i?Aeg?t#_g}RT8M*7Yf+SYTG0BImdtuK34{amwn)= zm}`>mXE={3^SNvz^p@`gtbPov0Me%9u(Y5$?nO?*D6D&D%J>KFEcQ_g;k_3 zmVlD%00QjXly2L1iB*9WSgG>Yt#Qj@fK|>rbW`W|M(DW6zg7AsQ;jYg5*-4`TZgqz z2hCEgiw3$PnG^!gU+$i~b5Jno^_34T-)Q?GOQ+Z1(}g+~^RQp5_>(266OtF!Pe=E$;z$YBM4V zsIc|$MOVphMDL);@fII1lPx_699g5nNHQ@HUIQXPK_}MVplH%Jmk68-xVEw&+C_0m zwZabD0#S%aCyMk$AXxR;3kbtJE~=L6<0rRgw{zm>@U%YRm~ST~sUIE|C5_-}Y4Q>! z{1Q`r@L^OjL3<3UtzC|bwa%CHKZ31Qd05exKO0e8SL>s<=sNeix_KSDnh=Q;*x5Hs+>(+U9U50~gL@6ki#4rO1+ugafnhD0Z$EI!yvALL z(TB!HmSeJM`Dzky)&&-t=zfowQFpq}6i{9jZUP={0cy6tj<~a`h2*!L1H=8!APR4c ze?Nbu@zgyaw6PwaK+k+Q)23xaBPOoedWNs$N1ihARdnoN1gNu+Vcm;*??nAO07l4a z)bd##7MLo3nICnuogx0rj*Ml8`)&Xgx_r`^B$gPmC zFzo^e)I8AbyCDqqToOC!tGA(7Spv7jakJU7sKg9}C}0k)1E;W5!j#n(r!$9xB@|4l zGlMCDjgK$s=^*SFD7RV-{mah-hY6!8aRscg$+-ub+%DgMB0xU=WnkNTCJj@{;B^Uw zrqxI$_j@&>_YJSj9*YaOst)*gd(+0obeC{B}_OS4g7w%ErEIEqY z(qgR4o6sTbNHad;)pgPWzn?UQuAHQJ%Fe&qT3>_};qQb|65BG=5scKB20 zE8JG^81sQAM5<7(jBt~I3{WHeVkXz?8~CvE2_^q}o^qn~2TQeP4gKc|5=immmKP4~ z=P_ilGBSSvIWhjhh8TR>(jD_2dvL63q8F1us1V`OFSkTbx7qAe{`KwL%*4Ru}Iuc{$y?7Q&x+tFgrG?0-E{#G+C? z+fcsi)q8HCzLXj!QTb4NEWwINPSDOZtQG#X^_7et>;}BPtjMzve}X~@%sin6;x%H{ z&(ok5+^Ie@tNY`(V}@rpWv! zx0J?g%>>pm3N2H+ET+!Ao8{F4>ic58kGy9f3_0{qyN(%9rouH@!+HK55tNf6qf2va zIBgR*LyW2E3f~?bC9lfCX;t4gz*vK$A86%IfA|Zdy?GafL`Irfwp5N;tN&10!`xfd zgYC&x|2OkLgYpg+uq7xaZpP7DuR8b8N|e^M#H8j>(}tM;Ud@k@lZLgt2;Qu^qsHd% z$7UO!)!(Q$Z@o47esr|Lb8z8tmik25hpD#H^vZf5UF_(+M|E_}iM;=nH(;%=O%(>u z_HH7})`G4d>+J&+WRaRF#^32R_ne zgarE&5P)ehqD1GSF`wi2;6o1LXBiOwJIIlFe|MZO?DT;v1i#BBqNfWrim=V+R>TBP zkq(0Q67qcnf6=Rg157N$Upx->hk7zMrZ$p*Pr-pjya@^9rM(ZnmK*&07wm*69=y&i z-+&WPu?dmLFDM`o0hZ;6+NN~r^+)zsYk;43kt`$Et?sU zJ>)!mma8{he$0MS4k8cvH7xP2g3fGWZ1Rf zE`r?aiFmKahRl7n6gxV;GC{kA`RzK;GA@xe4M#7pi0P9VC@zZCyH64z2`#Ct=$%X^{zQ>{<(KsilG>W}651I)(}JPcU}Gkp??(Ob8m zHE6iE?RF3A&v5U{v7Um=DDZShJ@RdR{_N_;Pqlsoxtjv$92xG7lH7w$p|gP&q9VB= zPn(&eWRxB5ts@|g*bn|jv^$gPc3w+utUWP5?qMAWZ-x>3ADHrY&l3JKTtOhfg!CL4 zF#M5Nj-S$roCt{MaQ4Bo5I;Bxr{uo@lCfPVx+f^^Sj@U}EkG-TgOmbtu4KeGUVgTW z=ar>ZcxSu?n~^cvi2*G#3Gf&z_U?E5tTGA9I1&A%x^!snM~ouZ$s`{;TmZ75_~Dp$ zgqqEv)E;gD)K|`lqjffODya|8m9B0;m`fVm!Oa835$a;*sPxo>H&_N|g0tXH&5;4-Kb{Cpyi<2+Ojm_GFvAQOJkmp4tAzOJ3m@F>{ zrTidSm7Nq~(;oa^2dnCRosI5hw?5+BSkD;)eTFk7g~*;%!99OAxw-93RSa8m60V9s z-nZd=6`jv*)89~pQ_0!{G(T7j?AN=e#B>P?;G}QgUb_Mv9Z(p~r49@;1_mB@&P-#4^F(qDALsDT@FmtDF3ttdGW@`$DmbIl$#FB%*_q3siP#+U_W z%h=_*EnM58#5_CmBY21zMT~`*X&`*DF`M7{KnT|4q^htc-}YF>^y~Ti7ZCAH zi-DBo4?;F?`Y$g+X0GT&-iE7Jx++{p{<|i~V3H68r0@_reH^(j4p#@Lb|CM907*MM_5JzaQ@?*>nLQF>Wso%l zn@Cq*{aIvcQ}F<4DlIRe+4C!liGjRUl~lQ1kCUp+JrBPiA4XL1erdL7Uk8}|92FyCil+eD&JjO4}L&; z|Bs#{)%;qf_{8-2rPmSKx!!ZlV6R>dTJpS3BmKMpXd`r?b+5Sqc+EhonyMUm_pA5Xrw?bI&wl^-UP<@J|LP66{K#zBShYx z0)A0idE_a)5%ypoL!TBz9bKLD*-0Zmwg6F#gPzN2D2zmz{Bh(djYO7K-00h&?{vNpUfY0%v33xFHl##M#QF8!7LK%I8K zhq30C1JQ&lwUJRN#u;oq+=Lgsoi^sd!f0~~+749-Gz=g@Bp$~Qf|pfP@XrDPvYbbI z@~pRR-i*?s0wCzR0ZES7KrHC$>WWXm#EcZI!N30p;%84xIt_4e= zySEd_N1Q;6WuDF zfWK1@V%3a@B|N}AhhJ!8%KBZHh8zsXye+X4O9k^p5?ZarjQS?40UL zC`Q0ti5LFQOd*W`$>U$Ze`-LmIO@JEc=m7919^I5{DZ1H87yYdx&+^o(8yGe=2E#Vp`bmueaN%BDp@zDF{Ckz|3q=Xd4d% zeqJe@dcWJh`G5G0>KsO<5dv+nzQI2b{h2z7cgiFek?RJWNfRFicM_$jQ@|l<1Rb7V zPzoSqJ3Yl8ClTEQ>lO>tX@%Vn;d3!Shs@vLNXv7XmON*Y3dM7Rr|P+l=QDeYX9 z0C9fB-5m8R7|Wp1PK*=R^Us(Ck8k=7bV~J%^SuQA3fasiHS!3O{4=%C19n}zI3;sl zuQCQ>0*lA;3|=0iI7?>BW^jKsPAMCspXrBQf^OIPs5@eA`_@vb+w`1pn!U*q^02Pf z(frXaUW@h#5BiQmEQxkMTqm`dAS|+nH?vk1sj}Ac`^}YPXN*PJc zy+=q@P3Ws%{@zM{i4M>IO!*$m40v-ty&Diftf8IqiOdo~GBxt&69l3!`S>h!1YHY% zm~(x;|K{ra#k0&HQKyZRSjm|>fo?>P#AwBDdKLD95&wIkxHo`z!;4GFMfnihxDm9( z%0**uGv#&aSX$*7{Y)O@N*OD&k>W3mIsO`SI~-h#N&O4X?*izL{j;aQM`_O+2>u3~ z5%j_j5Aoz*HaZt2x2@3|9d;7R>R9^L6aU7b@kByGJyceZ)dqRcuY9$PH!RoIr-Xc< zC-yd!_#8!hOrrWU$LSBf%$nHW9e_nLSjlI~4s0z`oA1gz zi-wP?zkhz4O8Jt_{P#*Yrb)Oz+q*n8IlWfSdu{%!`O_M#fBXA^AEi;17QgubtRK0> zm*K-I1hw^fz@os#&;y>=G)HH%Jk%(boAZmU*@<82N;3dCZ)3JC7eXIJloUY*9FL^u zx}31vb_+m{9PX^@=Ni!16vE7C>nFdgWp4h-hm-hGiYUbXuDY9wx4i~K$mODEbS&bU z83np1!lQ+-;HIn#cGugdRb2`$76qP`=ke@38`msy`V@`%udJ!oJ&4 z6s#|6;PJl+ME1Q}*#d;EFAUzN?+qLycw;IJ9xR|cT?62SD&KVjy~>~{&jT(wrbSuBVxrSDBf*1oMYlS%3wszY?Cg#{(Qc~t6EWSUYnKJ{i8Qq_G(HS^8HMYsO*P&D**)ecEh zMU*ONeK^6;Jy}f1cdbeRJmVU>#g-80L!W(_C7PJa@-=Ox~I4YdM@eiutr!%_svi;0r|^ z7ao?;FI6zhO7;9F2O%K~a5UZu$;kD*OK<2Pj`ZLOWtK1|VXI2ia~n;#VR^R zIfaHT$THqjCT-vCm|NTAWpHkmj4R^r5|JcIkan!2e-gXPL+lR;*cvlA`Xvzg&U&fZ zUGhPI@=4bJ`>PN-t`{7T;%%+FEFF!JF3}4OwS4(5ry@ZCS0R^OEOmDZZ5TKODNcRp zKgk0EwE@p>1UsPk@}H20_(dDIPF>j2eYLxz(j(Hc|L-)RMs5C9B6l|%BD3_(V#asJ TKPMT21pX;2swtGq!2|yfFxqO^ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/7-previous-shuffle-total.png b/website/www/site/static/images/blog/scaling-streaming-workload/7-previous-shuffle-total.png new file mode 100644 index 0000000000000000000000000000000000000000..cfe9d61c306a2a67fcc75dbceff20c1a5838fc87 GIT binary patch literal 5388 zcma)AbzD>J+aC<(B&0(G88uL1q&gTe87V0!5|RQ#8b(P8C|v?l5~HyQ9bFY zgF=9B%whLEVB@xfva+6rvNBB1!`05g$rc0>dg7IczCo_f{5U~|r<4EergVBF>^-mQ z)ha2TOH>fiQhYm!Ai-WT=9xda2R=MYDSCKzA^%o4>i|4*l0a($SJ#uA8M{-ULbuhx z9pO1AXNK36K-RE;ZgsF6OgC0HS3ZB7o5vf?&KQ~g>Z4S6r50L?K*=AEc}-7B75D!J@=7fQsSJa7h~!Hui&K5U$7 z^y49qsm!pQiSMlnv>KsjXol1o*VFSfySPqf@8Y@2L7pW)MsDI+R6`(}lBqO=N~-oi z$})~==H-4>WznMQ?7_g>4tDdsyP14SBt|TCzrW&c<4;QBiTYdr+{0GOb?h#;6;SBg zws$n_w6#H(0U8P-i*Nu@02CRpIDiELfs?{PXMigsurPS?|8U9l@!Rpd}RUB*%oI7 z^L2J|@s#zI=lvrg3(%+A2wvD95uBqu?;UMDn6j&fE$oVzxR^Mv0u>AfLwVe@lf8*i z{aYRQCC`h+;oM{q2p=CGF&`;0R}XuH1QLlvh)W_QB}D-VQBOY?oRzPriznY-M*d?5 zW9w<-;oydIaCL#5+O@KF^}@;X@}4^S@8_>LZG9d7@#Nz9cUnM#h*J(iLQEX-U)z8x z>U39D&%xK$$r$6{43r1(p&)Yw^+){wF#d7)FU33mC`w6P{kP`7IR9JomZz;s{CgcD6;|;3i01tra*-R?nL8B;>ER`Q>OJUDFl~`gd&huomx6i@5bo-|KVId{II%`8=Pun5m*J|cv;?D-y^&JiZUV1e}28ZB{kc7QsW$Q zk$!ci!Kbd>M5|fC=3{5TZm;0WS8p_X%T`~!EK9oJQj7i=&m}C?D{au*XMkV{-f3t1 z{v|$-Dfy0?tkJGSA&J8-$&_V5Bj6J#pmPlU`T-|6w3&AQt z4yFg2HFF-{zFuWwU7pA1bA8n_? zcip9F@Aia!Zs1~G#Kk0;F|(mTEc@d}5LNJ`OT(sS5;jj(v#8+dUGE9o6WOTdAA>g} zyG7on)KA+X<8QNAOp7Fajk`LF#Z;aAKDv;D8>T~TKjaM9ZmmA~wLfPX;oT$A!Q6fn zC76@ga`M|$to|!+hTn3=QuUNevVMVT*IB_VB7VAR%-obXX<`#6^N6x;y{zHhF=B&w3Z9k6ei4>yhS=uzvJGwfRfD+$SJ28(SWEGtlN#O`3D{getbPh$4BDs7&`% zCtEh7E@0Hy-k1TJ{h%(qDAh!hsPuE%Q+MdvgS|C@Zn1_}OUj;~ImC-nErR^&cKhT! zOGI$;%RbtA|aT~gqHZ?Qe$!D41{B%>@{l7#A z#OvN-T6fGBNN;k0j;phO$l9=D6-4KkQ zxq5&?U04)tkFOaSz0jM<`!6#bayza2H-~f;pK4#3;Lk7{x$^C-31V}!NP(d=5_4mQ zx6o@y`)WqlY)f#k)Z4P!1u93}6U+eg$|RPr^;=X_)Iq{TOp6zgjqoH9saSk|@{9ud z0j`r;GOY1=S{lfi`Gl#MD^s#rOr7R64WqJfGif2F$h5{~u*TKm2EtpumJN2XFXnl3 zE93%n@r=8t;_%swmhJb{5C(+FkGP%C>A?LNa*gZvu+LvTp;I8e>tr9=Elk#tv6El52aM)r%= zO!ErACM0pXy(0@1F1|=Plw}{Ml%y-am%?D#$fo|I_v#$Mzc4VK6Dhfp9P4-IWV?;7 z?!kD+i~Obv7|uFn1^|m9?Pt+6*MxJA{eWhUOB}eqN?OI4MlU5b>dcAJg{+wdR^wrk^#s*b0+VeZt2nrPLYd9pq0 zd`Uf}1f8bBU27GzaNu_98PUM$TKD4Y86degtHsYg0nH-fGwmLM0y@>Pb6QHNg(ij6yt-De8a=#YEE2G0AH9T%mn?N@#ueXEhyQrjYX zeK|AC8@3Y`RsWt!xKkj_%=$Tw!am2Z*m=lLX&RfyB&Z|Jm}f)oAI}4vYpt#G(?H^O z^7voJdt(+Jt3_#z$`8mQ%A$PfR z0Rg#g3iTQhi8fK+Ym4gTTR1mHB%8M z{NiRSW26hK2nbSIWbxUt9oqTDxGXZdZv zf>d0gq1E^Nz~U|dhn*=x%l800Oy=?)=MjBYWL0fFTHdf#AI`(!(nOuf1=g(fuHa7! z&3$%->G3%_@*cM1V$zcW%csQ8R`tqw+{gKVPnSy4kG{6iDH1lVgGTQdRmzwMVR{eD z)|f)zeEQOO06}!wn;sQze@1JO7KoAsUEhBU=|20S_;a4V4fqQcNwWy5-jmw&WRz;!P*bs|BV{5Qa+I7v8k8ai=-g z%#1?v#&u07o?3(JzOXJJC1A@pbD`4bqdbnVk)&CY-bL$&1olNVv5@qUIr!4KL$6We z4oDag)1`~#f8J$seU>gvPTEZM=5y|(j}r-&TMOWJx=8ONM51UID%~ccaejpBo+4xX3MG<-UXC@l=4fS_f3s-3mmP7pRm%P=m~2xd&28+Ddk> z!}NOso5=(am`^ZsQHr?KdY+(W!6ko#t4od{xZPpnB%yHf?0IT z!50tmb)_?E#;-J8D>fcHVcFef@O-Ws1;5Z+btm4-rd-fiO<)QeQ7Z?NYt}!-jJWxH z))Wi2u{gVoaJFpaAN|W|U_{SHk`_#t&_7}KOjOYkzF%h~9+NBXo{W8Xa4mn}3%A%K z8+hx^vlmn$nTgn2MnAYO$F6qPWCtveN9BuN&#C46Zn1SjFB0@JUg<6mv2Z!y?u;LG z?YPci1ljM|XvOoJjEqZ6LV`rrZMS1a&JalkH*JCz9`%fG9vE63ptbPK@aPic>G-M# zC)dfhC108V=FxWy>dNmlq-j}nJ^u{4TEMK@P4VgNAHl@%hsjuu%Q?%ZW3N&M^jcKv z*%bRff-qV#!2xMQRPUeZ7jVWX%2ev{RzgnEGwb__4{LSASLXX-r_--C#j(v1{6Sfw z;802!HDn+H3I~nRxN|$l58(z5`R~zbN&-}m*lclwico!sQ(KPp(q>Ke$?#)r|-R< zKY55>*=*%9A_7_2fD}6+ms>* zdM+KqhUa9#0mRN6bek+iwlDJQ#%DBmm6A4wL7JdU%vcOl<)-DZ8vZ1dVZz1ns;^Ah z{#s4o8bZKQAFDq!8~wJg-{6J0;uaA4UI^$j&t*n+<~K3$1}RL5ySvjs?GC!iT)5;j zh8**H=TT@M=Av|vUOLLpgP8Ft>^>e{)rB{y9C_I~6?(q{U8UHdGlF}qK&#J)OZAj1 z=FtqjpNI~`-)>QjdzFcaVGw$>wg3b-p@d(kLFQs)*s~epGpmq{Tr&4l@(Nn#H`q;V znWb;4V+zb5(k=Wv9jI3$i`4-dwCD1N#M#g`-loP-4#`_X3fNjx;dls@hC$hlG%<8o zx^ex-$&5UO?{2@a?>Tg*zwdWrY?I>tS9j`F>Kba~hs1I# z&Mt6Xa+guJFEM1djc3vl{fNqyw2gjTAn^O@=eQl8g_Brng#qBP!~|+~{2KF0D_k4? zk$wOb6^(RX!0fs*CS>}7ywNC}jtBuJOKZxk30t$5No2IvpWJ;y7o zR0AHhbD?}Ieo>d|jo<4`Q;)|u@??%?$@9pnk@Hw{6=75jEM)#x7w&>*PKT%xmf8H z4Z~i9WwUj(BSrf|MW$T)6sf*ze5KwFcIDydbG0CP0h!&R`;Ro`M(aFR@H-J3>|nIK z|4)Lf#L-j0SRvQY0102Pal6>2Bi`yOF)y`GmlHqOVFR|NiB)NJjR{VDeAsCakJR-O zxY4bfQ9I9M8+GKaof!jw-Mup*(I=IK1uK#cgZ!y^?aK>`4&tN)zC>e+%fF`txnHjPq-9VRPG0Z~wy%Od@KeJ#)0f_1 zdR;!X`_k(VvQ|S0WzKu%4S(=>LKOhrss6EVeLx-W#{w`LNA`1#i%bnR1)!|m(ZS&i zfWS&|45hxfoEGi+Kd&9;p~u{0+}Zo7r-&Y}!~(5FT2T6$51&FXnV`Na7=YV(s9^|T rjb+|X9{@f#0L`0d|0wHGlAnNc_8Qz@&2C7az5vru(ZQ5mw+#IsQ-HUl literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/scaling-streaming-workload/7-resources.png b/website/www/site/static/images/blog/scaling-streaming-workload/7-resources.png new file mode 100644 index 0000000000000000000000000000000000000000..4950f219334fedd97931bbb5d005c32493b30c7d GIT binary patch literal 34148 zcmeFZcT`l%_AV+xk|YHMM1mrcv*avD7Eq8NIW`D1IVZ_T&N(R{A~`ipP6Coaa?Uw4 z85(H%F7`QRyU%vNG2UPAzH#r5(G82WR;{X((tiSsBSU zw{D@qZrwt8j)ei7(RqZr0Q{qFE+L^TDGyM+(1AYA; zeM}5EcFu2tgTvnFd-pfDH4}U<$Nl-StNnHa3W|srEiU`R`yO>2%~+q=wtaiu>N2^Y z5T?QUnG=}-h<}HS!U_EcCfX`KG8rU_VbO~3=X)3!Qh3yYS@k)$z8Bvre;M*20=3*- zY{vo19VIv&-M|{902|v5qg?6MBx)DgM+UJ|JZddEY*`_SkLjr@DY%YkKTd@5T|c#m zr4R(0kfxC-kTR3`lFs-%9lXsqk|H*`#zoAo%1O*&%$dVw%pS*1&(4SX5{J}|^o@_D zQlLN5Jfl%K=qHA05N!l^AOvna=?kVx+Y6Mg;p_U8TS4J2~j9*IQO`TO|5pD9~E zz6F9AutqvWf&odZnP|wGzIt_w5je)Wg%V_b3mrH@0lp7`FEA)EfwwS$zxRNzWESdw zuA;%R(Ef9b^8DsTaTN(!S>UgVk)4SN*xt^sG+rrFOIX{P zQ1h^|v$8)C!J($67P2!o6?h{l_3z@qf5K1992{%}*w~z%omri^Sgq|qY#jXj{A}!; zY@D1dz#T01u3!fP7Z$KR&A%S0||aj>$p{rPO5sL;(>0cCR+6H84=b1OhRKpi4n?0iCh-2dB^ zUp4-vq{gq39PI48e=Yi#OMfr=*51TU!rBU`(?R6d(EPjbUoZZ?Y95fM}RKk_9Terk+$x4c=x}fZ&->z3}C+Q>zmyt>N;6ED@{4qW2!|X?j ziCK!Tq@><;Wm@0GcACZRGE1te625#Pr;j|MNIPj<1vf10?p;!dL^ik^?e#hzrtx!W zF)e<7`h*(wzdp)c=)T|Xim3RNye&u&y)U9tvR#K2-0Y41Ue*;!5EvmThW^*jw+J!i zrQAYw3Znq+TJ)_1@Xt8TyM!oz`S`qeLY>MUh!bd&Uf4O@|eaGj%w7!mJl52SjXnpJ^H zwW5f@mhN9F|Fa=J#HdGW{T1+!aJ4!oDsX-rX=ocOH9e*N8Stz#}U5A~<> zKNP0NM0wURR54VY%s$Xr*ARBt?{{@KDz#ThaXpbn(M|HV76Of=4*1}l?tH=H<~-P< z0{)12C*``1Pnp4MoiSRV$99U6PW)%n{&GMsigIjq3_gZw)2g|ialyck8~*X>S$5d$ zmz1V+_IR$3SJN?%hOX^BqxXpZ@}xh9KtqgLufhHCs_^_Rblrb!sW&QoWBv{U4Bn)u z6#N0}_m3(q3Ipt|thzzy?^nfW38M~mu?g?{|4+f+B=A4N`G27~_Sj2p1frF;%N?tB z7F9Dx9pu&%540=|s@{(tdOy;$X}LXmkEB8MH`UEz_VIl?cX_f6;+=I+nVfiVQ0;j( zJHA)5PMPT|io6>rQd#pxp>}shwV6|KJHxXW#-@QkFqfOmfC;? zSJO!kTT>+p8iqzs0!xwgeB)S_xL+?pjy46KO_ng?c$mldM^qeBz(Eywwsx%E)xJQl z$*W2VEBDcD;U8bY^xi($y)maU{;;oW=VlW_d zq%fy?eZEXU&Sk2wV}G0MM{4}62-rDynm8S}m%41MX+qT_? z5#CiN>nxDW1yZb)cg>7s+*{A|Zp5Py7-8C=Ke@7Hl{ucXQT(l!)Tp6>Vl50X`lhpa z?|-{M~-u|IZo8_W`iti;G)z|sXA%K?GRc%3g|de5sU z39k+;e`_2NxztGIu_}~}eKu!>$jkJS3*_*;a1cBhQ3+qi_C_Mks+C0GPhHbpD4b_( z$BK0XEoN)0D1e6q28l%hrF(g%%xJVc4%$6Q^U~}>`ck-=v{{ef+a*mG&h>zBuCMm$ zVBk(F?|KS{LGiIdO)jL6c`%N-Hn?WhvaJ0<#mTtde82FSwwC9~Xm<6AW0)P~{l;ZR zs<8(9M-4%+|0mI_%w>9CyGlgw4d+FwXW4op;p>^P#>$_QH9;7{KXxW75d4kV`jn0% zuXTqI=gW~+b-NHQ(*oy76B2S^x5`;}SVQ3Fz8#(r#(^iG4A(EhSNm;VQjgGDa%5vI zgD9Lz*<%%x*y{Ivb=)>{Psg;Z;b&GAeLOr?Q_rdNgl>;%7~x%>PFwE+-?Qy$Dz}Zt zR8Cvo-_xI89Sw+H{~XCzL0nwNtG(+AB%s>ELj$EdCh&^cv;kua2CA;Mf-VN0wqR0` zj!zKiAFg!ei?3gKPLvv$?3rW>xESB-YXp7Hw)jo89noa}n>No&udllRWDL{Uem!y5$yd)H;C`W2 zH5fAWzF^X-de)Rg$052ejenSF-FenAG2Lm(d@idw?5vM>-m-_at`Ha%b=?;4!!Ekr z+U?>at|Kk0s`!(=S-}#Qv#rq{azZs-R}&=V*rgG(Sg`YQNBC~EG-Vm9PVIRqpg6|f zMfFINl$V`487I3#$|6FkW_PNphZ*l2nCh0x;k*z5 z$1Q)YZc(1@KKwQkc z^ri#JH4%(W=z%XH>iB%81uJ0CcCOU^LZlBSQoNVbx z%InGQ^GH6a_{(mr*yIyFadS|hu%fNfnSwT-0NsCkbbxcE?23*f=L!l;e>JV^x_aLo zzM6=b2PPX>r_PCcW-PjtL&u@5*0Qc`8DpRp&fcTQH}8#X2&WJn3+gMp4(FZQXsUpu zWhT4tly%nM>o4T3IR0T#(y%59C-A!kKJjm<_2Buw>^$Q=^PUo7;V1mehgaqJg}khyET80O&oR@VIfG~U6y@`L2jwM28)uwF9S@# zj5U6BH)DazxD$`1gwJ}7JZnu3F73=>G9HN6)wB=LKXup>r3uUP4k_#PH2Vm>%l<$o z`ocm@P|)S;Y$^EJ!o7i7S5xGqK?d2$HR5u5p~bI3UKo5@Xxo|8%z8vh$Sx8_B|7I_ zZtB$?$6&Jbc8=|8OqhIw!t;2@i%5_s)u_+wrN8DeG}*z6HKK{nKu!0_Zz3+;xamND z@5Q%*;`iIdb$c&}PbZ8BN*+}hkoz(rf~dSU#?#MsCWm>ZEh{(&ESnIgUXVw*sTA|u z<0UZAL2^+Ch1X_oB4pm{vX5!$>N$C9hhtwV@7W&A@Z|4P3g?6_M5X4D`^ zzOdLm5a?p*5LAcY9@k!Wu4!5fB;WafhQAL!-}GYb$4puFW;ejR#H4g-7I4^Dn1%CB z8pr1Jct^A_J7~@92yU*iyJ^)gx8S=!4E9(*n^^Wwb|c=)8p)e$<36u68+vjQGp|Z;<{F3W zesCDbebL*@H9erTi8$)&S0_%owP*fbz?|I1mzCbbt-O89!z*s(@I5q4cKJH6-d?3>6Z_^HQBJA7kqcFiE6*VLn~&@ha+iK9`7DIC0h-Zxu~lHz`~k)5 zmGD@yQKaw~Yu!$HfYTsbu_^=PH>1jhy&`zTy4vug8ygHS>>Fq=`urp2%SOoGY?N5| z3C(kGtPx{+M zpEpmbR~v}!-Tm~n3=Vjpf%=pI?;vE zB=kKHpTZPaMylFL^k$rJn1vD7Rua4tn+uB`YMaw5l#-u^yF!EJKZvprm#=*JZP6m z6@s&A8Gba?wGC=7?_sqNy*`dbTphVsz)Y8-G4LL0ATCa<4)wg@u=eZBxHH26@piqx zn|ZmUWBwNEEe< zoHLT)Z=zmigftf-FgVf?+tgg!X;%x0ivdKDS0Nuz37ediK9Go`6IZt(aU z*D>|M5EIg_Vw+-$4HkEi%ww&n_1$=@Asq*Ee3XnCSOH!1T$^y6>w+XpOTn*G5x zKg%zv=Ay?@e!JuIUKI%}S{g=qDiE`w8Jm3TX-H86;U#fJCocq`|o? znx?68P5$}71CJqXy_IhFp{UoA|6pg+#Ud#Mr^-6h^|PhCSL>lj2+alM+Ooq&3#L|6 ztYTSUD0k&sb>EwlKl{4LgydPC%NRcg+MA*Ekb{ga>nbojw0O z7ksv!0E=YpkL`v(DiKG`JVjEM5QboP|C9=Ym-sdL@}{%mx7r)#bM;2UzB9)E8cO9hKwC--$ck40CpAvQ z$AG|r0U^m4@LzW@+pT(@Cq-J28q%uY$dWO!c^@WnfWkRUdB3Jd+WfDvn_Or!uy$VJ z3)NO)F>>ELM)?)`$`id;3nx=C&j_E9iBNF0uZ@sfw{?y%ZXPq*-oYA4u80x6nb9x) z2%&DvXQ{Qy22%t9;bIjTm#IdDX`)oHu@QfK00GrHyc-bN zMT+Cetg5es55z{k}1T{>?9O{A*hR_`OdBCSLXXpy>v7Or}#ZhMZK^~H|-a^3DGKJkaTnosXX z{CA(lh6tn9Uk(|^Ke=E7p2vvuyDV?Z>jR#H(I^{EYqvTA>i4c*1qbc z*!lY3bL!Ozvnc-_sG;2bDu6m5jP!N<-J$gz{r0xvlwM<{`(|nsu&Un&%)(!W=nvy6 ze)sFfK+BECx?SrwCGmydm9W0p@%U?!%{i1 z_Q{#-2WKlM0*>!5t6*J3+^f`gqv1#EbZfoIvF89Z^91Cm!HljbcOpsUqQ=L!d+3LCDke)KK~!YM(vKVq5$D57PHiU$ zCQ7rrfxu#d#r5j-ga6E@f3Gw?DK9>p?4lKKTNBYKRsyc^I{+|#FZZ)2N2S}H?9s*V zIEivFIxzk!!9(wiZQrfb^|6)5ZNB_dyuS{VpP;`lV;9K@XalzA9%>!%T8uHLjph9& z)0=gZ_FfXFNeK7*ksrt%ouYSD<$;CCXNAEIe*bW{Dx1&C#x(>01DDM|Qz(V>up}zL zGh^wrSkA2X>T7r$w<=fQHl;h|JsNEocwv`~4-xU)R#PKtT2^m++6C^me(lOc@>z#a zd0#Pn)X%4#o&b==y{0yYt+;&1)!}aJ_(Aiz*A4;1ICX3cYP-AwPqAy3A|3vM+>O<0 z($%p5A_mb6Zi|F)w1&SXVuNbgMZ zt69?fUM*>qCI5&iuD{(sU{@aS5yKI{{K=c!Mk%-U_Esc6smMa3~Ax4okxly zE0uKKMT)GsOjjwva(>#6jv{t*6N4tqKwqh6$-xsv3YdZuI+ybl&Xc_J9v0bB;gJ4x zLD$tG#M~$KpFsr339rjtUM=(X`=0N-ZM#>aJ0GoM)`|*Tmo$lt=(-~ur|aFF>tNTI zJ6CtuAe4f88^kBKaS$LHsR@H1Dieyd7xRE3`YycegpZEvP0u>dTV#53{2YUoXGD$< z4+w#fbw{k1TLmRn8-u)s_IVLwz?Oh<+OTk9#A5s)N-;A~p~0Dqiu*HdfaXEOkG?ySh!HYY6k+|KxcMC*d*e zCDk$tAMKGmp*tHUFKN7}#e*K0ZciFXd+%zg^RhKvG4g&z%+YJ%o+fCTntSi&wn*&s z*X?42=MRlQZOid7Px&2U>IoA#{MiG^j=r@LPVK;#D~WzpUo8gf$% zntH%j>GcTSRlhgyL0D+?szIb8u70xCEiu|RaleN(US^Eu>r5x;IBCv1H9U7IEojoK zyExA-s_ka%ircM2iudoUjCVvL+gDrGojTK<>)fEp;SYst&AQdJ$vzt0Fmn-5~!tzM;+ybmC=G|eQO~)mk zm!}z2y8}O>vpwpn(o)f}JSA#*8mqXPAIMKKT9Z>T-ZuJLr_DbNp0@JPQ_m=W%WUOQ zYbyrf5h?xbIX6%>dkL#95p_JsO}s24K`a8Ow4u&5MWjuc>h!sG2Z60Nsi?LSJe~&_ z{~dc?R_oF7c$Tv7_cDwaBB{!&4mLa9+Tscq;+j|tW=V${KBD^hX&?Z**Qpjg*0*Yw zk5k}*#(Dp3&;xj{du^}{C2f2YEdAd8B#35b!4Lnqdf05nZW_6B&`Ei~wV!5;0h@h9p^p%DH8+PkCA@})J7F*rI zA;T_uVIi|9;ll-lNS`02XD`7-1pkGM;PQ_B?nvwfyn%cvBh(0brtAK!ZPqY!57NGt zdI@vb9}q!)wyNt>gPgvzm?|j2tJ!n3xjn&QH2IFhPN`7PZJH*flohYPpcf3O- zF*MgQzB)Vz1*7wz36ZkWO8U9pq=0nl{4j#*oPlxbYxTTWr&5&vAy)=o-6u?#0u`+)?6C&Lz)Hg6xbJ!N(#REI!y zahK1{1RS(EP+yj<0ec`FHDrM%u_1!8rUu9{EpUzUR-Kd1JYGgq*&VjV?~2YLFv>a@ zWKcN3&Je+>Q}5$jiLTk-H*>q4BD;N~L<&ARBo)eHN^Utg>Pm67TA0umQ*qpTjM%^^ zgT6P|-tMPIjQn05}nL@H4eDhC`PKEvdM(bO?Xq! zlB_BhF|O_GK^z&Y#99IlEBATj8#bSrEU&Ps9(miE6o4YrW4JJ>1Oq(Dc)#lOa=d%L znQOHg|G>rEW&rGYeH8MTEUw6%%4Lx0&bpq>r%zFmtGw$N1WfgkNoDh;TGcD7dtgFD zNGiW%M;LsLP5ejuw)O(Vlr&uj^1A#1!BW{ygSV%O zsJt)A6d05E1I7Wb4-BhSnqtn;_?N*IPf^n~r&VuED0kR>=h`#o0W#LKo^{+s0}V>- zKV$svMyd|^S%zgLlyG*3jc6b7SpL|#9pXZ|`)$u01tG|a<=3nX- zA!rG^?@T;cP`m}Z@0t`weiobVw5B;$FaU@;u#tsvbL!lXL2K1&d6IN9c|D_=0?94b zmhF0-`qWgNU$m9xEyDhj7V*m~*-1isbj%7R)ODB@#=cZ$i}NfdGd8r=4&{pICXa0j3WKci91KdQ5nkp30dt_0?{Qi!L_g1 zSGD$I8L%Dga7~P61(m30uO41jwT0H0;0HbOh!RHjsyKbH->p5pAa7;4wx6op#NSOktL9RsTp6?7-5MLN=zqX8*5BZC;+OgDT^m(6$b@3K zRZ*b0VXZrWNAF$7LF$7|dJR2=3){100)u9cd@ZBzAL!^yQXc`817}W~Qaj^7dxgFuoIHy23XXTfCM(GD<~Jgu{d7 zNV87Siuba%Tpi4jL*X>>*sD}!?F%X*rpR(=C}Sf5tJD<148jth+*-j6r1Cg_&G}L# zPm%jHBDgy_FEhmNcmM1{vTEN%RLIke_RHzv1@{ioS{R{gz^!fm?^ z9CPlpsyUYNcI4hL&$CMtB_QyKJG0|o{2pu7?1lxa4GSEScT9q$twFR7ac5p|DW_H}&+qP-K#wBK%M-L>v@MKixZC@^tuDyeVI6k84 z_7$Ur>(2gcPq|xfu$w_Iv4qnyPF(; za^GgMuXB5w+r&Ri`<&aQ=p=UpNM=v7EL7~etUs=_-h z(A2KV3OB=s6%Q^MAGv#o>Vk6k#|=(3s=XaI|eNoa%B60{I%2)g{vvMCwZ z08ikxS#M>SQTgNPN>fW0jSjeIp_ZBLQyn1r#9}rD{pf*uqJX1*{+@n*B&wEMer{UK zN$9+*$c2i@9qHj62VO1iL971L^6i}U-efBlt;$mQnJ)zS<$hY7j%gMeWGlUPTt?6+ zlBtK&BYe_L6>8}%Q(*UO+;6Lj-8awzSCd#umz77{19LC@zU!A1vN`nd0uZX zXlK@lbnhyQ&LVmz?;_s?#`+lZFXSyxm?u^Vw71@2+B-AyF7@hvNO|MxomRXJqH?W7 zz#u+>dD3a$Il}7LX8Fh94I56x>!6j6T0UaGQjXamu$s%du4l|p!*&F+25)%Ik!zVI ztSZ;Gbz$$_*&X~MttuW^#cwMu z%O+>h2tm##f;6SCep$G$@r8%b<5 z*_|6s&_wJjjYj!VA)-`P?`5|d;I|ANpk=UntVd=tiCp^=H5kO`ZGCiWUU}0IQXW2F za7Eaes|1^K(rzN!1vcj^0pDSAxB9x;n2SWMT?1lgzPx-P;O( z3`3Cta9e5CayU-vOaOP~(2;;{l9)i?o%wvON<8QXwh*UJ#AaN}opZzn$w~}4`QLtK zmETu7y@9Qg!42-;44bXvhWEy0E4&pv%yPXGR-H)&KX-tpT!cm@l2l^SKXj`(NiE{@ z6j|*&6V_m*g-41#BAGhJE}mulg7CSPe?3Z(dt=?ybeM_l9uw-f1|K}RoRGXMb+9c^ zTNT#Harh^)xLmSWV|SfGV6OJVIG(>i+W$fYePXBqnDA-&>yN+oq5gvx(wp2+Soode z|9?N*iLAP46DwmC1-bw}Zi0vJ*Gl)-+61Ck z`*L?mqR>)Ay&G>Bp-w>4r$mv=`&rb88!E5*1V_#eKnDfbu$3IdIpc$o09A#n9&f z08v|q0FvlaJd0!n!mq^|$#X*3p zi(1zg^;p6({;gV>^PFp^+v(0G3=er%Dq+ zbK}(x&kz7Z8K9mfvg*9pFU#zmFk}|j4_Uze@c8!W54uc`CD7sx=@O}fjHfN_@6zeX z_^~+sLE>`VMjc$OeBH(7?~%6?2ur&mk2Jv7QX9dRjqtTbFB_3!X28>^J{^*#g4>0^ z&H||1jaRm!CIas$xklp|RKA?=SI^kVQ*yWFdN}UQOnQ3*y5&J(Jyn@sk{PGxUrvRK z|Ms=|*F5X`gTRssfJl0h@$6`=?`R$12s2RyIUw_kb7-=U{xX6o=nu?nUS>cf@elN} zFxG6c>u%?MXt@U9ia3v3iSRmD<20#8)Ahx+6F?xRE4d-tpZ0PMoWlSPdC+dffQasp zl&lg$aRQ*_h2l|(%usrr8wf51(_GoN1>#j?M+kHSyi0*gfM#d~EQe(rMjbA<*Aw-; z$|fs7m$0tHTC|zi0f02_jS~>Op6Ry5fTS=Hg0f~GAN2!RYBxal;}ir)o3SlvwjE@; z4A%>O1XqHs0RY&@|*Ov4l3mU~roO9GKH~@?>kGBZ7XN!@KLx$JIrV}t<&o9|c zC6nQ?3`hgw8epdE6jt;z+OGoqc*f=T4FGkTZ2f7w^sRxAhO@b8w$LmOBkj-HPx4JR zhm}ODbQ|2@wRY?2&Qs!;6f5ywd*Muxq}@M9M0173|c$xctE3Q+v>iM&J^8Pxly@brT;@(CSh8oA}m5N8?H9 z1!#Y5YSAiYEtt*)376{5{jmKAvKJVGc4CiDxQeJ71iAne;cC;qlz3P4Tn-Y0gy>{< z^jw+?M3Pa&f!sbo);9kr**f6MCHS#4;)c5gLu-6RJDfk9>$8hLYd&ekQki~0K#Mv?IKWP??< z*!_MWJ7F|I!|)?#^(P??nuq?|`_bQt&MT2Y2PEZb4Tt&Nsh&z_jECh)nIc~EsHTkv zx5%EGMWuUR!>^;wG;7b7LJ@X%i7osOiJ$7YO5TkuU;?%|o|{5!7xS(n$$9%>CM}#{ zfR!6`kIAZEw(^T+9MMa#P|jn2LLINE{*|0U(CK*?>JPv=>H@KJrm&moNk3N$4!M;j1c>~YUmFu3tkzrN%uPiZv~*A8Q`L+>o~?o3a%$LNXXux9zS&`Ot4o2CeIfF zDQ8C7>Xe%~&~F&W*Tfj8aZg(DTYH7LkTTCb0djCw=Ay^htg{sWkXEAM^HB@M92|Ag zWghonCT%Lb?Ue;-AQh=esWt4V8AMP?4U{!rk4c9=Gy&*%bq0hu_gLhiW7*A^ElE{c|6~(2-7)oUImS356 zR-0q|EI-%Aghv(sN-h&KzekU~nqFwY>pHVO31z)dv86wO?>(RYBoVaoZno4km8_OX zNoa_*>B_D|EK4jwS~&w)7QLtS{qSNUE2PpfJmu`Ci)StGg zGY`kbVUzMv&_$T*(m={oBu5k=E0@h{v4qLrGkVj!zI}Shk#zc0o@fZG49J9}H_Rl8@oK5RqTL!zkt;>Q_ zxA!8KJ?=Zl^zA&{)_u*KZ6e&2?*)r)hib|{(iwnRhh23l zP#csk3!Z&eSoj%P-E7yxvJPiAA{%RyraajjR1!MypO+*^G{U{zo5FRO*%kX?fB%90 za_ItRm|?rAn`K0&2wLxtiZ980FT<&Z&bFaf=;st?3f`*(@FqIV^@!3S^wkUxCG~5D zYSPd9isG9fBblcvuDz=2^lc*`o@S%2E+9K>M_BDENe^?yLf-!VmA;?k9KZR`r=r`G zVN%zt49K@R#mBJuc^l*nHM0!&b{i?BsLU5(3RU$toZ@;f|70*_1rpLTtfmXK33~;s zl23;0u~DhRN3Ki*)2lXm_sh4rDjE!|7{QJ7J%yN{wQw@4IA3l5Oz}3-4@=F3{H%3D zlRHp$UoDPqg>URkeU^{q(0vV1s*zds_V~0ZcqcZAqQ`?04NQkPOoh$4mA;?t0gve^ zFZ@Mvr;HeB{T&k_uq;Y9mW8t|_S*WcM(Ul+7C%h)>T3tKm9v>Py?w%FzGgvub$7EjwKDf=m7lMC@bA0~1$KcCTC)*mQ0L<$`&FYD8!f zZU@<`-F916a}gyM{M>t_NbDLW8CR|FX0Ems!-E-RK_|{4Az!&zmaCJLaOvqtY!t@c zYn_iYx%Piquiv4+=g;NJceMM#OzPJ$(l;f?G+C4Pj6PrdQ{yu&D>RnPIQ*G#G zYg;;Rt*ut=<C0KgDKp6+uj~kXmK2^4QmD`X8(D4z}`mY&(2B+^|RZ??C7YduKgu}aZ#7<$QLzt)Qr z^!7ckAs)-?Z_MF9-w;+BO8#{mptyLJ9bSL zz_f+{Az1t++=D6&2z#u6faerQuLOA>1O77va4H$LJp*hc9NSNTK)RG`KxlXjSo3Mv z3f?_?m9laad0LQU+@HqZ*qGCC8LK2x283BZ&Fp`6HD2yn0Q-hcwLNdWt|J8Z02sd} zr+bdbs_xqkEso1$R3M(p1kwn#?xdH55dpw?6^Wk5P#_+;DZpx!!{9gS2s54+r#b}o zw4asO5^i+?S;{JFSHMo{TL#a=n|!)?@ddgx8(-n0Uao6lfMHYy?8SWQ6=b@|Wo#`p zOIyki1;UeEAR>&cQ(c>3DycUFa!xgp?c))-oJpP+n|Wh^UrYrcwI-#Ux#iQ#--3-^ z{uH!Zkr7;plCT7FGOGcKnY&lzeIP-tm;#7U`ETox!ILgNn>BvXl|K`H*Eu|!eRQ8r zxv2Z{t-6j0{XQIrVh|8=GR)}Cte)l~GqFty5j_XMt~re7>SN(PeU66&GIcZ5FUYj# zUDwlHg;kPyBPj%VH1VlCk6+iFPFXNXa8BMr6CZ>7ZjCuGddmh@dnA8h=`sq zptYI3)&Up+^V1tc*#$c%Ly&@Q-OnFLp6mmGzM5=}z}bwwRlt3pxBBv;Eu@@tU0x7~V1+ApdYkg`eUkQJn`o!{^N=wI)Fz z{Cx6DHttPUp&rc@ptX?*r%0h`gzHST)k?5!GK z{^FLq7rMyB22FN!*5dh2#ekhx(mJKbiU#@J>6)I;9n!1f820&*mEs} zGHqkQF@aXPul$0cJchqp6HB{3y2WIRin~x(13k7WFK7E#s@H{rlzS%N&R7Wjy%}|<0Z%Q#h^>F)rF400^v;>kEJ*7?Z$PSCq*uI zmdP0+vda%T$P;Z_T50pVk=G)kwoL6)(`81TPwg={pbr5|t9xK27Ac-&1pxIY2n1ca zL(EIe-Iald7KMu-wti8hM_UmN956Zcp{_v6zGZee&!HcZQ1_S%H;_-7CWH=!Yvsm{ zSpiw~CO^gCPa@=*%lvoCIw}9a`uqq)HvFa?$8}iZ#gBy-p#V&DO#$V>SgffUd)E`Z z{aTtL-XT3lD8sIsVXW5CVtb~h!kG+zKnO5Yvd+By95k2%>^AkLTjy7~Z0JwmA;BzU? zL`4wnqBwd0Mw#nS@a+$_Zn~=}h}HHy*%YejAgPUtl&iHWDHIs`{w@{XIhX94w+(c` zDrqY7WxhBYhf>ME+(*UEAl^Lmw^OR20cM$vpSSfFfggDOFXvRQe}uBLcS;x)6hRJ6 z!g+iI;uOtdlXtW6N&rgdArG_&7-A7UepwfpN_;pG@;a!FjVp=eUFE)oKF{}zg)cg8azC-j=Bzw^1h_Z^jUiMg}%C@*Jb|z_8D8OTVVbiFS zqm!#>vK0F+fwQXTCUj?$@S%m@brgQ3&K4VOhZ%==NcC8W0kfC>F`z|>*3y0;sag@b z70gvX2(u6j%@)~k{8PTM0l$a$XM5;r*lB#OqC88Ls*4c(BZ@n)2JO#jq!j`$45E@A zd=TIy{h;@{r%;y#zgp$>F5U&}>~ zVMTYUiz@)AWIhgYu%faOHNw6)87V{eq&33Ynw+vI(J@WCt1HI;oypIw8W6+9DcjcT zyCzbaCSfl{B@mw?Kb|&icAL47WGb+-_rG{SAv6BW2sG++Nt~91q-X+wz?+_z@K!DezSn_-oO`^8tt`!qdaqZ`+^T{}O8V zvnKM{O!M=_hvokEd1nIxB<$Wz&71MI=9L6Je~CC;(7W)XcpJFm0=wllq19E{LSA&k_are+GdLu>0i){4eC>XI$B+8 z=U=U3e8(RDL@R>4I`ZFMxChQkTTys8+bp_h`Y^W35ibrTeqDni+3IMDo#%hK9rqDH z0R$=_O1{k(nU$DCyw~2|A zgilmI8?*&}^g+YC=*fwP=HaYPI0knO$GzhHp~ROREh?R2u!_-zy&fO4@iSMxl??rg zdnU!mnO!hJ9_>5%XQ!Mjj5`FrbuMk6F<|fZLHN?p*ab3EsUMEl7+iF6=5jm) zKf6x;DL)tMG$-x3s@td%IJ3YdPT`1_;(s)8P(oAOPaR`JQMkT_F^IihzExtf)Mnqm zuy~Zg8N~<|&Kc0!^0QZ0;%jfI=9rQ8{Pc*=FGn;&ME-I;Zq)zC@E~t?N6YY_okR&@ z)UUrX-_@=FlL@CfxsAV4ka|~%k4CB$FYHQ?bM>Q&zNdCR?|!c;rlC?-6&h6wy2-e!9c;Rr%kR1R8P64JldPrY_Y&-!Mafi2EGuMmFtLgWd!IoZa~ zfNG80x_+_bHB}pThJA)KHb`F)>Z8@#2R*kKLed9qYDCek5S>Jcd?!otbbmRY$T$RH zQc0@J2S=?_<>}=h?pQIdh8;@HNd<7q3ETQX&|zQF14)*`_OK@~J`LD+`yx|5D!te2 z9$_Vl;KFjb_Q_ST0lGcfLr5pGnQC5rH_w<3$PkV#^sCHPv^^f@JmG&ViZ8S;Cfe^4 zpz`@1xaqcD2}9d4zswhxfYV;GBE)%zC?R}+EI8K6bH(8>sxjW~?u9I?GWcetvH@>5 zvf8I}65a0*vyq7h98l54XJj#X3`hH>S3d+(fT+htt_mjcwaLE^iaQCQZ=TcfZVJd3 zo=XE{9UkcbIg?R--$ClTKbP^dPTZk=uGLriDaDbe(XZeG3?3h)!XMJ6k=I!prE|28 zn}d5EP_AaW$tGv@TFCu}3_ySEHO_FE<3g zR>$!_?GA&?lhJz4ewtwif4X`ylUY0fDpb@9mW*tx8?KT~-l`|qnMZy_hN+NB5VC;^ zWd;MEBu^Q2h_g%5&xgtD6NVc->kvvbukSdaPM*4USk~CW6}aX2eK&b!JCZ99&J$BB zi7>eOi2uR(VS&^UhgQ?0PR`=1m;5!da(X;3+|tYVC!Yn{dq^!u=i?{jS$}NI8;KaI z8}eizs=;~y%OMUqR1_kepn1Nasu4GrwZZuz2Tc-6ayuF29uA&2I6W991l==DsW+75 z!@5r@CiDD}_me}aZcZElG-Bc$i{hGRi>6?YbKJpARhR12Njb(tp?F@BM{}; zZv;dL;iB`mWW`AhxxPOae*?V7hDaj)jf|g+CJq`1B&?vHh5dr&;bFMPn%Cj&RlgPz z9Hx5|x1=LCacA6e3Dc;cSz*dw(sM@%Qa+>U#`$N}(^1Lsi@Ac7S4{oy0WXp|^E3NN z-rAz<(-L&pO=Uudd(B|s5FcRhZnP0>s~W^?`zk!@9T4Yck$Mvu&4Ox1yk|oSgnO{59HyF`c+ADt z&od%os3Z=xu1b!T+{_9g2(yVdauyJo-}wc6u*dGrszrQ~jnRf6e91$fUicgE?Q>4Ju zX^rxh9by#AsDX|1VISy36CP0dfK+@;jx75!$QH0iu#zh{+?IXbJ;{MTj502aeOcO} zV5NBVf}XZd@w&Jx3Wp}1euFYn^of>tBZwjh(9tOu1kB;D5?ISa!y3o&zUG`S#B`$3$sN+Z8^RuXy(3rUzMF?5kgh3morv!g#S>zRbz|2wl`wr zxjrO;JY%1Tzz?IgB9l+#N!UOqelOF^{h5?)cky7n!?o&ORf)by(s1~fNQld2TE`9n z%TNbc&$FoPL`2m}j`k(6!=GVPn4#ZPz^yJ-DJogZsxepF`EDX=tR&nFJVs(y3WQ1M z>0$@DsIaZljEdYE@YXPWAyo_SGOZTkUBxeVMU>vA?F%KyW;XIS>C7@ z5^Jd4~)J4etPHP)5GG{o`x=xW^AQbEcC5RQ-LJp>95^tsKxwr z41(6XZ&z`ZuI9cnWr(EU#j&_AJ$TE6^JLHi%-Lx2YyN9?pR7bRY{`$+9tXT;LYqvSqsnK=$I?VgDf4N$ zmt%A!tsU)@UhadBiju4qaFZ**W?xyKP`Pxica6Ozxa?%eS{o3rFbm(lctRr>ZCsxC z3c8x14iI3}cV}bj{*QD#pv;{VJ4lNkX>N;WjY-q_s)3=%Yzs=7EIG^zrC!ecBJ@ z)#B2SHc*91JRbWq{KL^{x~UnVl7*IARaOC;+E~0c7U52v8tc%weNQ@YQo zp7^9Wi+u~&)s_7!FK$h<5sw-;%zaJUrn+SO@W(dqPwDi>n~N>4zE#U z^etG6P_g;8DGn`|;65G^&K7^g#DcuyzzXxC(wJ$ehcITNs%`h)3x6(JDN}4=W5gS) zyrWAocjUaLB>jqlg3q=nMVXn!^mo1DMWq`vJ|<5km${Lf3NfZ5KR3QBCH7;MC1=lR z!EcqqrHm?N93^ibtKkgkJ-B9~dWSj4&x?;Q1YK&mqSEnvDMM^E!j0A1X2zYN(J(Cuddw@>q=@yl28Y6!6W6OlYWN8s_?nicm2Z&qFeR*7L^%-E1{RX^Gcz`}fkZyKHilNCD|PEmfx z{gicBYuW)*Ln8zS_w8$2KRc}pcZlfCg8;0voTeF;gGiSk!ikT4E^8x zFQ*#kD|uwHWadCUdf%^kMi}3vJn#KYD1BX=)TfKhbuO&%{M8YITT!C*e#Tp?RmPO{ zp!vSyrTm`4Msvp!PS*y{F4}d!PwJk#>lC(=k>=b5R~QAHRtI9SSRiaMzGED4Bp4Xv z0bI~~u@}@-Pnr9WY?m<~ z5-v0Z3q{jx*p;#%*{m*el|m+~+td|P{Wy%IgEUXk>J_+Ek52+vBGZ^sDMwf_OZ@Ni z{J!p&z?lz6mTNVtWsR`7eba$eo$U63r|B$ctfmA987Yi%M19&!&B5Dz#!9p2MeVO= zS{Z?%{qY`*ZP}aK*Q9Hxart-IhM0u=f;^WN?Zz_VoD(tFk_zSec7I>p-)?M|LA`B$ z4nsOdiUOY}CDfHGj15y0Ki@ksK{a#uS7vNN=(k`rC7}f72E;$Tw}zsQun>l9njgD7 zCnNWsvOqhsBeN2dDSdJIb6LnFQiRB znDY3a5Ekn47NgI@QAqvkT=|5O2rMa1fB6uH13$YhBq-f&M-WM6JyGY&x6`LDpO*|u z#h`i}j)I%*a+D%MfjdX>9z|%7p457q7draj*fhQBF&a7kgZ?4Tl%rMCM>0IveQx|J ztNwe;2`ss6Bcy%SviQXcqA&Dtv)}NDM)JPQDkO;b{_qR<%P-_fZ3NF%XiuCp3gU(M z6URA?@%f90w%`YH;zdi5->I8ri;#i+V|sjiID&4)!_q&Nr=B-goO#Y5+5#16gGU2aqcqEHBVHq6?}c=?G_@4P*!UzoTp%PXC?2CfC1FmT!On)e zqxKQ8(*s$wJq+rZI=2TeAhi|rgR=271>m^Sq;*5RVu_4hzh-WQ7I)4wT=oN$C5=E^ zEEa=!9e;MHV12nJq~L^HJH*+3aXThE)~I#$18bf0 z0ds*$@4eniKatH`o`>sCF8K$o3p#X);|B+rK_c{Uf1!*D@h!SRMG6wu(bO{kCyU;k zRN8>=<;R;~QDkN*sw$|h67UQZD17PI43pE#&*4!mUitQPe$TWkK&cu2vv~h!b;n7H z6lv(uP^HieuZS`!OQjtC1B!b*JJPk|h3}(`T3S22 z_v|RlzETr-S)FowGsJNZjRYSJpPy)n$}OPpk}N#S5Tau9Tj1x(%y9X>XDioPitLmA z1l71+YFQL(RRuz{A;)U93bTh(UYK6RA6AN0$T*G+*XeY*M+bl36TK&AGrUwP6Q~{K z;8D)9?AlKhT9WxTxKG8{oHLv&%?_A}J{uNsmdg#!T>3D6Sgtfi?lao^XfBwS zELdd3m%w`X zBRNAK#rTH#J-_eep&qh|3Rfgm*n1M@+3}d$k>?~F_EOL0X@;KnMc+qgtFBg51=rX( zR#;Au-A4DcSiGVB4!0H+d7JR}80B4+6ZyfZF<&#sLc__Y5t}$!7nb-FM&ayXVx&r$ z=KRG*V=}*zV0|X)+!P)fPYVwefGNdu&nlC4TciB19p|IK-ZQH#HEX0lCQu8(S>pcX2?) zxNW)m7^2`GYWlNVxibkO(c;Ivm5wCv<4#Qq5@px5^*CTSDa~Op+Em?3^99sBXByE3 zEYgwhxye{I`06azr|!}e>Qj)nF7K87`^5dN9Wv(c=U#;{ouznkG}a)dl|~pasl=c} z3)=XlIfQ^Wg##}}_UFs$71Rtf8&|CMX1z_wzzTRoXPK&WnQR#0dyf%+^i=N3o}k;2 zWZO>mqOs_-D0S*r8X083Z`s#pMK;`hUSfBYTST2}#{prXXV6?94o9_!Xf>)C%@$0h z9Yn9_YcIzu&T3OJ)cY7l%$Pk91(ORfRaT-V(&=Vx_|eRz)tc&Hoott6x8n0~{RPEJ zsl@0Q-vQP-3oP2CD4HbIEMBpgn2FxY704RoKNQayZ5l5FJOkjhLv_?8wcyN#ufQF(T2%Nw4<597qp-=^U1D5Nh zRh#whbM$-bzLD7%x*xnAer;%d!sDt)Ho`GW(TM&?O+$S*Tr`_QlXwAAm}vX8aH8inOr%Ckz>JpU&( zYYPpoYBG}S`?QD+g;Eg;6~fI}5qi>By1eUTvjnd|qntMB z-D}Uz5Tts6*t(x0>z)2eT%ho{*jcL!c2IUwVN&orvYG4hrqyEb!70Mcrxzw2&$;w6 zkY`Xqhna`Nl`s+$MuaB)>2+g#Y2*;xf6ssEBqz=RZ*n;sj8I;0YIWspy55kML2^N| z>?)}ev?Q+Io_E(OQA1txNREjEPGGZgqEJ%$a)PC9Rm3$2{sv z*gtjHd`8pQ?3|m_dvpBL+X+Q);dMvmF}JCLN{(6CCs_W|@a!5`R}X#1)At%wGWk;2 zt)Nm^!&}{Rw*hy_Ax#DzE&e;YD8Ch_7Lwh`Ij4?4J6_vg(5h83$Hj)q5x_<^Kx;j= z#bfX>4vW0{nB!3@pd_^3gu$_-=6k7fJGcHK78$fz)RxG9Vr;Z=_TN_?git9-0YWbV z`)RKKou{G6#L$A!{H|B>zg(_hc}G8GRla`lztB%<05|miihFMwINy(Au$ zf%u=O+@a&{U*5n;iBL!@SHDK-*vG4jL;(N?%hw8C&);*QoXmhcJN!6L*xlN`@c_`R zAN%r*y1PtJ~OChCI6e4g8WKCN9_B%$h3aw$c18D*Y^Ng7FcR3q?GW#gSTqGsG zH7ZyXDz?AaQnPwFf=~5!!LDrn+Xu!w7^MYJ%fE16611OY2^tH!i~e_y^YGQ*Da;2A z>DlC3adn_%hryn(h+$}Ww3IHH1-S`&9Mx2{|YTxPa-_k8rg}HVBxZ}bD zB@Qz{qB`mTfbY)%ct3J+c0KlUrAl5utW(M#*Eug?lR)l|>L;rC&iE8cP za=QTl_gtL-0y{-PiLGA0=YB-pF1mCh)`5>@Kpfoib7g(38_fn_4M0cS?~)scu4Usn5pk=SD4K?zdw`vCBm3Kk%?frmOL4QDY`G<(tVU8g| zh-Md%w(~@s;|{I>!d_L~0RZ1yf2Z->!k%IHHik>E!v)geb0ZS_?AUp}zT^PNGg#Xg(SC}JZ%K#s-%fa^aeO$LbjA;Lqw(fH(~?XT~lPAs^tM0ovEVnYCzuk`8G*R_-#Cr5Dmj^ccr^q$C9fZ}K z#c;=AgIkWSH#1CBeesT-!S(=Z`qa!!H`onO=elaxAXx&)Gk5xxiS&!O54UeOgL~1T zAOJ)ANqzqUzu~$zQiEts=E~OQMVCcCj+N-r5c6vC_8p&o0OaU&Ih)!R{x&Dd?_|6J z5(au&EPw>5GqYrfGJ-xJD)D~^Ybo^DD5)kks8*>m&ke-))!?iM55i9qw`vSE9we-w~efMiaIBz7o;0W)?KAN9RRs_ zGjm4@g7Hblf#&Yo)J;WvI1n4*r%{&&h?AEuDb>jE1y}~LcfY>G3PTAU!k;ZdO__+~ z2NqnPnEUs5(9KyapCWxCBo<|u@$F{rn+6%Lz{MC=3EAdLXXG);Kp zrjkg4*@myFL8X2Q?9F>Fc!$Go;9}9)J2+x-C|*1kyWwEbsl}*m@m(PC11TIm;2wx; z4gtm8o)EMYzS!QL(0#rUaYY;T!00ueA)0jyDpGG!O;Wn0%=K`}cZRT{zIdkWD2;Jf z$t{FqA1~q3np@`}MGuFBF)9#_5VFe4vc6GYYxg{$avr^mF%ZV3lMQ8HIi=v$?3Cel zv}${pM+gG6^ST6exu=%80+6mEh^&v8*26sY+>0PJs$>*vVFn3rUenWPTv^->H*YSE z)kO0+xqA(84zzDFYm-`8VB%Z9CC%KC_qK*S>c3P%nb8pam43Ro?=P#B5lKHZ|6R~d z-5>pvijU1XwV;Wo{9qcXpf2fNE6A-cfK>3cwWfx#I-P5oDl9P9{@QLKmOkQ=m;;m!AG`!CZ~4u{qi_jp4~7^at{&8z|Y9UMZ)yor|J= zUc7ws!a*X6tTi(5zBs#A1V zkFswD95}jhAtGtw#f?+Aib~!**yi+&PqsO~b(9fd-|-*PM+4%s%gf^Bq4_~oy|;W@ zs8#um9-6i>D~)!**^sTnpw-oE35Z~mrN7Q=vK&sXoQyq0v1CxMlZwEsnzkfQ&Cnkm znPnOc_HbmQkJQ%%?brJwV3h-3W;`2%Gra7|n=lfL(G)2 znXj{EH1!qj6v|4{q&T1_O1ONxlca&Ed?5N>^i2eVJGu+$45!3wb^!6O30|<24myQT zfy*;AvAX!K6Lf!cB&ohy9CEnjxXB0AkXoMI)GOH4Tn(L-T< z9=nQ~*R62fD8COkNc7{EI5YA!TNg{C{^N>at7J(-+mBRoVlmn%YwznlkyNx0QWx&^ zJ2aVu!kgo-Ct=`D)fTd)ipFoqaPlMrhmXfB<&LAsciXJU&BckwVxA`)Di}pgOG0!G2G)LOPqbgAHsrc_#(~R{AB_5s4=%0{ zGdzfb(o>bdSe}RsRzairgSSOQi@qSO4{*9(9&NRiBtQV%;j`Q`73%3OE_sCcbb@bovt|+my z!SBJbpg4ip%JMeMrp!`Lqs{WAXEq@Wze>4;7D@2{J!#E&bYsX!;G_cc0Ko4wr2Cw! z>5I(M$KR+v8Vy4xX}>#U)aTt*GBldL6m zW=oQZIX6y+Y?kg-;%toKg{`?A#@~QVLpF_2U}qJl`7SC44<< ze=H&~!Oujm-4B<*OMI5{>J;H0z+mAMqZFIijqP3=CcJ3BM(y6TR3ElfK=3g`yd+F< zeDzjZvK{$jbOTo~JZg!YqW;G@Q?eRx@dm=!-Wa zPQ61v49nMTg?~fMcM%+VDpZP^udhCgfR3&xL+n9G z0w!Wn1*#{Mt_Xy+^60yOtCebO<>cz%jyfww=OUib0RjG`g!hbrIIY|cKeL9g<- zp66OA*$i^{nYiu$&#A926|#kW)SkG~Wtq9vy<$&mH+R{w%{-u>jaQ41HsH9Lt2_^D zm?T`WM_Pdf?md>2Euo)5@S0?^t8*T0wgE3>5%c_1@7-?GunvD`OP$8^jeo<`+|V#; ze5;MOq}RQGXsFaKsISmneeJMpKYD?mhUzs_(-gmnq-t&8q$G)d-V=Qwrg#W&tzceS zUG0k0Tk(vVuNm5qh!TH2MktT%sRZTT@n;|HY;d3h=Vy;4eYieiyVCReb3l@|NZ!f+ z^rZevQ?S^t86ht4ix4xid}=H*vD5XZ7{296B-Ar0bZfl|71o9!vUoV zv6vn~Vs!@GHUG6%|5ZdjgXxiRkE`aN1#6RXd%FyLUK3{hd!_kjh1H3;y}3+hFc}6u#Nhaox9Z=P`1==0(eWX86sl~06g&}& ziV3aqej~>Gr!{ba2N?LwWdAhU-%YWVK?#Fvcd+H9{Ampg-~no6%AW-{76R60HzxbP z=!X0vGPO@1+%JEUEY~?9t24dkGBddYkA@a6Ubu)||4!WbI6>@AK&vGJhyk^)XC!r# zI#sY)^scTKc6L@ZIv!lo)~zBSzXSaI`ib2K2QxL$-q9Ht|Bid3=EPM6Qd1TyXWSa6 z5yCtX7CiUgSgJ$V_txER*JYSrZfNjW?3H^|=IOR~F+Uumr!JBups`twq^-06ex4~= z#%GX0Tj^_W;OOIKV`nlD6V@(bJ`*3Z-tY{x-l`LyG|q#D;;<>-x{Rt{o6WsNV$)w}t37sxsBb5}E) z@xlx}dJ^>_ozBh8>hY&_X3Np;AoykL0sU;o`AeyQwb0|^RruH#Dk6$LfahjolwEQ3 zBxcHZo(zP%xVmyWANkT$){58%8L58+yGeTfEIKxD$tI>339?=~S>Im2go@FwbZyNE zc`WRZ&UI6mv>gxYez7))|7{xw=~_NM7gSCd<;o#5L7q`%{YOW;dxc_%{3|7(^)jzC&_^+{K%>sTw}u)#YzE@$qK4eP(D9 zDnZL!GLi@`#f{EwpHwJ=g{X-?n9|(81_cy$_vJHM5jw8a0k5Vmk3AqxZlgbkfPTqf zdpwsH8`5Cn?z&5QauW2|irDa^ih2EHwkCG5MqJ`yr=9-QSuWTz4J!(~7mpQK_k2|I zV#ftcH63cQ@PXuy$%ZsCWeD=^hPHLlKIkFI(P8)bW$fhRCc!zB4ZLpG-sLQrzs(8t z(f=N|3maqXDVYo=_0wFo`)~;9;yb(^#w&5kq7W?=A1u>G%;qem=@$~lGUw6dumK4} z(T=n;>nl4i?631_e zI5^IB$$8kIB_g(a0~K|VJr}-NT@fGdVuYVq=#8DKEEP470!s0unE zoEbP}Em}LM0g2f9t^CU&`>UP&YNm4rkdqlt!2i>L`5=a#VIID;D*!D$=PmK4C-vf7 zq^ObbF>lUpPZuYz5x3HuvCn~KNbH<$K08+dfi4?UL~eIR4*i>?Yfj-eK8k5&YQA+> z8>^hhhI?8da{VTDjH7H#%${600s8o&=B9Mji(e{Ah!Umrp+f$E85iYPc%EqI%+nI9 zr0Ts}d}eyS38s+n;B_sLtFNObW4!kj*>6|S?XKXrj_wbyz@X)bZ*V)z>T`dUAt>!`SO#dEenZ{=#=*@%R^dw;@vasM0w zyLS9!n_nAHH>M2T-B+8DHtzENy0=5KOb2#c`cQH;Wg=1I&+zW%d>RgVw?#Wc=_2Oz zwkxaVkCBKamCqFFC1F!?FAw44aHX!lqz9^J#3-@pve)!KD}63)f5pctUv_(0Czl9D zMcy<$&W4*vt~>~^>-V~h*7EiBDtUe4~t@^E3Oi?zQAE zZhJA;RaXOu&8bFfa;!>5&YKP;dNvcM3FK@^JjxBH2|{zp8`ceZX!a@{KqbAkB84sHfszG=3okDmM~(xH@C;kEmmoC2!W1CvPXU@*HF$syGA?TN*{8PWaJoPW84|y*F|p(RzVTH z9`$iXW~)-$DvZzi?CWXO|H>m?=2A$qEAkgV~w zn>rOiK5pSYnBAiGkY9nvC23JYz3wYM*5XE3wg-9Hnlr)z5vOOJX&81#U#7Iw$8C?r z+OOZIKD!O`6F?7JG`dKnBohq!`k64%AgJ}3^95v~GDtkBtl^KDk6$A-iUD=qK=DU- zbiR(y8Q+0Lm6+y)?2ahu;M#T{MalIy3D(2_mt$EGmGWjT0#jvN7Q|%&f=^AY&y6cx ze}}f(1Q@DQ2ptaAe}I|z{0?v3N=mG3jBDF6YQHGI(A;wfJm7Sw=Rn~_mAE*3xxV!J zxVGg*&utlf7P*N+Qecs$xzgUPn!{DvZk?ykXMcpm+!h-2BV}B7+vC6sF6w%@fwY=+ zzyI{2LeSVJnnlyD-YcJ^=}XUSIH(oRS|=9(tjEw z5^%(+*WCZV>V_V=UOxi^gXXVyx2QH+{Fi;DLc}p(-l9p}a6=2Qupc_KvX7wIdA@CX zVf>F#3Y@MY(W6^{bx?SYu2)oaXJ;pxT%eOk)a%f3C;Y#L1_}lydNjtM+u^-?C=Lp* zq+d70w8XUUP zI(e}T6v5K3ka>A|bHL>smAZA01svyyIy*b(?X5nO3%=9;qd!nEa8Z4NCEjj)%LMNG z7z0U4wzajHPnT=(nD<9N!a4#POT$R`3we22&yXht$?wdE{?|G19X3%vN$p8eD#91l zi$IIV+XkiA_t-=tN$1~BtAC9Ud=&IlkWqbA{lYZ#{{Yp2K&b!# literal 0 HcmV?d00001 From e6e180b68965e872822f5f32281b30c65da16af3 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 4 Jan 2024 00:13:02 -0500 Subject: [PATCH 207/224] Switch to use self hosted runners for linux unit tests (#29907) --- .github/workflows/go_tests.yml | 2 +- .github/workflows/java_tests.yml | 8 ++++---- .github/workflows/python_tests.yml | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 2d180afc79e3..ae5960666363 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -36,7 +36,7 @@ concurrency: cancel-in-progress: true jobs: build: - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] name: Go Build steps: - name: Check out code diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index dc0f58d04f79..bcb073f5a977 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -47,7 +47,7 @@ jobs: check_gcp_variables: timeout-minutes: 5 name: "Check GCP variables set" - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] outputs: gcp-variables-set: ${{ steps.check_gcp_variables.outputs.gcp-variables-set }} steps: @@ -70,7 +70,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [[self-hosted, ubuntu-20.04, main], macos-latest, windows-latest] steps: - name: Check out code uses: actions/checkout@v4 @@ -127,7 +127,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [[self-hosted, ubuntu-20.04, main], macos-latest, windows-latest] steps: - name: Check out code uses: actions/checkout@v4 @@ -161,7 +161,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest,windows-latest] + os: [[self-hosted, ubuntu-20.04, main], windows-latest] if: | needs.check_gcp_variables.outputs.gcp-variables-set == 'true' && ( (github.event_name == 'push' || github.event_name == 'schedule') || diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 5596e06945b0..a179939e4df2 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -45,7 +45,7 @@ jobs: check_gcp_variables: timeout-minutes: 5 name: "Check GCP variables" - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] outputs: gcp-variables-set: ${{ steps.check_gcp_variables.outputs.gcp-variables-set }} steps: @@ -70,7 +70,7 @@ jobs: ) needs: - check_gcp_variables - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] steps: - name: Checkout code uses: actions/checkout@v4 @@ -133,7 +133,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [[self-hosted, ubuntu-20.04, main], macos-latest, windows-latest] python: ["3.8", "3.9", "3.10", "3.11"] steps: - name: Checkout code @@ -158,7 +158,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [[self-hosted, ubuntu-20.04, main], macos-latest, windows-latest] python: ["3.8", "3.9", "3.10", "3.11"] steps: - name: Checkout code From 02e26c1828f248b4128bea6d068c7011a5c78661 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Thu, 4 Jan 2024 12:00:39 -0500 Subject: [PATCH 208/224] raise error if transforms are provided while read mode in MLTransform (#29909) * raise error if transforms are provided in read artifact location * add test --- sdks/python/apache_beam/ml/transforms/base.py | 5 +++++ sdks/python/apache_beam/ml/transforms/base_test.py | 7 +++++++ 2 files changed, 12 insertions(+) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index d5f4d1b60e14..e436ef8cba69 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -262,6 +262,11 @@ def __init__( if read_artifact_location: artifact_location = read_artifact_location artifact_mode = ArtifactMode.CONSUME + if transforms: + raise ValueError( + 'Transforms should not be passed in read mode. In read mode, ' + 'the transforms are read from the artifact location.') + else: artifact_location = write_artifact_location # type: ignore[assignment] artifact_mode = ArtifactMode.PRODUCE diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index e07959436198..d8121e29b43a 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -304,6 +304,13 @@ def __call__(self, x): write_artifact_location=self.artifact_location).with_transform( Add())) + def test_read_mode_with_transforms(self): + with self.assertRaises(ValueError): + _ = base.MLTransform( + # fake callable + transforms=[lambda x: x], + read_artifact_location=self.artifact_location) + class FakeModel: def __call__(self, example: List[str]) -> List[str]: From a5f0cda4f3bd33d3098dd01eab55f9ddaeef542c Mon Sep 17 00:00:00 2001 From: Cheskel Twersky Date: Thu, 4 Jan 2024 20:09:11 +0200 Subject: [PATCH 209/224] Fix: creating templates with the Typescript SDK (#29887) * fix #29884 * add try back * fix formatting * fix sort * add test without dryrun * update args --- .../runners/dataflow/dataflow_job_service.py | 6 +++++ .../dataflow/dataflow_job_service_test.py | 23 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py index 5ba6dbed4e64..2bd8605c9928 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service.py @@ -19,6 +19,7 @@ import logging import sys +from apache_beam.portability.api import beam_job_api_pb2 from apache_beam.runners.dataflow import dataflow_runner from apache_beam.runners.portability import local_job_service from apache_beam.runners.portability import local_job_service_main @@ -34,6 +35,11 @@ def _invoke_runner(self): runner = dataflow_runner.DataflowRunner() self.result = runner.run_pipeline( None, self.pipeline_options(), self._pipeline_proto) + # The result can be None if there is no need to send a request + # to the service (e.g. template creation). + if not getattr(self.result, 'has_job', None): + self.set_state(beam_job_api_pb2.JobState.DONE) + return self.result # Prefer this to result.wait_until_finish() to get state updates # and avoid creating an extra thread (which also messes with logging). dataflow_runner.DataflowRunner.poll_for_job_completion( diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service_test.py index 896496d43775..e2f880085cb8 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_job_service_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_job_service_test.py @@ -48,6 +48,29 @@ def test_dry_run(self): temp_location='gs://bucket/dir', region='us-central1', dry_run=True, + template_location='gs://bucket/dir', + ) + with beam.Pipeline(options=options) as p: + _ = p | beam.Create([1, 2, 3]) | beam.Map(lambda x: x * x) + finally: + job_servicer.stop() + + +@unittest.skipIf(apiclient is None, 'GCP dependencies are not installed') +class DirectPipelineTemplateTest(unittest.TestCase): + def test_template(self): + job_servicer = local_job_service.LocalJobServicer( + None, beam_job_type=dataflow_job_service.DataflowBeamJob) + port = job_servicer.start_grpc_server(0) + try: + options = PipelineOptions( + runner='PortableRunner', + job_endpoint=f'localhost:{port}', + project='apache-beam-testing', + region='us-central1', + staging_location='gs://apache-beam-testing-stg/stg/', + temp_location='gs://apache-beam-testing-temp/tmp', + template_location='gs://apache-beam-testing-temp/test/template', ) with beam.Pipeline(options=options) as p: _ = p | beam.Create([1, 2, 3]) | beam.Map(lambda x: x * x) From 308223f447f4291fb3fd64dd97a566599e782a86 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 4 Jan 2024 14:12:34 -0500 Subject: [PATCH 210/224] [release] 2.53.0 Website Updates (#29856) * First set of announcement changes * Add release date * remove template example line --- CHANGES.md | 11 +- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.53.0.md | 185 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 14 +- 4 files changed, 198 insertions(+), 14 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.53.0.md diff --git a/CHANGES.md b/CHANGES.md index 3f48ebbf8795..7a40f9af4df5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -89,12 +89,11 @@ * ([#X](https://github.com/apache/beam/issues/X)). -# [2.53.0] - Unreleased +# [2.53.0] - 2024-01-04 ## Highlights * Python streaming users that use 2.47.0 and newer versions of Beam should update to version 2.53.0, which fixes a known issue: ([#27330](https://github.com/apache/beam/issues/27330)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). ## I/Os @@ -127,14 +126,6 @@ * (Python) Fixed sporadic crashes in streaming pipelines that affected some users of 2.47.0 and newer SDKs ([#27330](https://github.com/apache/beam/issues/27330)). * (Python) Fixed a bug that caused MLTransform to drop identical elements in the output PCollection ([#29600](https://github.com/apache/beam/issues/29600)). - -## Security Fixes -* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). - -## Known Issues - -* ([#X](https://github.com/apache/beam/issues/X)). - # [2.52.0] - 2023-11-17 ## Highlights diff --git a/website/www/site/config.toml b/website/www/site/config.toml index e1b9fd612ed7..df90aa617636 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.52.0" +release_latest = "2.53.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.53.0.md b/website/www/site/content/en/blog/beam-2.53.0.md new file mode 100644 index 000000000000..864f1870504f --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.53.0.md @@ -0,0 +1,185 @@ +--- +title: "Apache Beam 2.53.0" +date: 2024-01-04 09:00:00 -0400 +categories: + - blog + - release +authors: + - jrmccluskey +--- + + +We are happy to present the new 2.53.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/) for this release. + + + +For more information on changes in 2.53.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/17). + +## Highlights + +* Python streaming users that use 2.47.0 and newer versions of Beam should update to version 2.53.0, which fixes a known issue: ([#27330](https://github.com/apache/beam/issues/27330)). + +## I/Os + +* TextIO now supports skipping multiple header lines (Java) ([#17990](https://github.com/apache/beam/issues/17990)). +* Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676)) +* Adding support for LowCardinality DataType in ClickHouse (Java) ([#29533](https://github.com/apache/beam/pull/29533)). +* Added support for handling bad records to KafkaIO (Java) ([#29546](https://github.com/apache/beam/pull/29546)) +* Add support for generating text embeddings in MLTransform for Vertex AI and Hugging Face Hub models.([#29564](https://github.com/apache/beam/pull/29564)) +* NATS IO connector added (Go) ([#29000](https://github.com/apache/beam/issues/29000)). + +## New Features / Improvements + +* The Python SDK now type checks `collections.abc.Collections` types properly. Some type hints that were erroneously allowed by the SDK may now fail. ([#29272](https://github.com/apache/beam/pull/29272)) +* Running multi-language pipelines locally no longer requires Docker. + Instead, the same (generally auto-started) subprocess used to perform the + expansion can also be used as the cross-language worker. +* Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)). +* Python 3.11 images now include google-cloud-profiler ([#29561](https://github.com/apache/beam/pull/29651)). + +## Breaking Changes + +* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) + +## Deprecations + +* Euphoria DSL is deprecated and will be removed in a future release (not before 2.56.0) ([#29451](https://github.com/apache/beam/issues/29451)) + +## Bugfixes + +* (Python) Fixed sporadic crashes in streaming pipelines that affected some users of 2.47.0 and newer SDKs ([#27330](https://github.com/apache/beam/issues/27330)). +* (Python) Fixed a bug that caused MLTransform to drop identical elements in the output PCollection ([#29600](https://github.com/apache/beam/issues/29600)). + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.53.0 release. Thank you to all contributors! + +Ahmed Abualsaud + +Ahmet Altay + +Alexey Romanenko + +Anand Inguva + +Arun Pandian + +Balázs Németh + +Bruno Volpato + +Byron Ellis + +Calvin Swenson Jr + +Chamikara Jayalath + +Clay Johnson + +Damon + +Danny McCormick + +Ferran Fernández Garrido + +Georgii Zemlianyi + +Israel Herraiz + +Jack McCluskey + +Jacob Tomlinson + +Jan Lukavský + +JayajP + +Jeffrey Kinard + +Johanna Öjeling + +Julian Braha + +Julien Tournay + +Kenneth Knowles + +Lawrence Qiu + +Mark Zitnik + +Mattie Fu + +Michel Davit + +Mike Williamson + +Naireen + +Naireen Hussain + +Niel Markwick + +Pablo Estrada + +Radosław Stankiewicz + +Rebecca Szper + +Reuven Lax + +Ritesh Ghorse + +Robert Bradshaw + +Robert Burke + +Sam Rohde + +Sam Whittle + +Shunping Huang + +Svetak Sundhar + +Talat UYARER + +Tom Stepp + +Tony Tang + +Vlado Djerek + +Yi Hu + +Zechen Jiang + +clmccart + +damccorm + +darshan-sj + +gabry.wu + +johnjcasey + +liferoad + +lrakla + +martin trieu + +tvalentyn \ No newline at end of file diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index e426fa48f0d8..cc65d6d1f99f 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,18 @@ versions denoted `0.x.y`. ## Releases +### 2.53.0 (2024-01-04) +Official [source code download](https://downloads.apache.org/beam/2.53.0/apache-beam-2.53.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.53.0/apache-beam-2.53.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.53.0/apache-beam-2.53.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.53.0) +[Blog post](/blog/beam-2.53.0). + ### 2.52.0 (2023-11-17) -Official [source code download](https://downloads.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip). +[SHA-512](https://archive.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip.sha512). +[signature](https://archive.apache.org/beam/2.52.0/apache-beam-2.52.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.52.0) [Blog post](/blog/beam-2.52.0). From 24e5f3a6c01cc2eb1dfcd3c6743e527ad05fc299 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 4 Jan 2024 14:48:04 -0500 Subject: [PATCH 211/224] [release] Add extra context for release branch cut action input (#29920) --- .github/workflows/cut_release_branch.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/cut_release_branch.yml b/.github/workflows/cut_release_branch.yml index cff45e2f229e..4c03e45913b4 100644 --- a/.github/workflows/cut_release_branch.yml +++ b/.github/workflows/cut_release_branch.yml @@ -29,7 +29,7 @@ on: workflow_dispatch: inputs: RELEASE_VERSION: - description: Beam version of current release + description: Beam version of current release (branch being cut) required: true default: '2.XX.0' NEXT_VERSION: From f88c78b79ca134cb111b82639663c2cefd3a03ac Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 4 Jan 2024 15:51:30 -0500 Subject: [PATCH 212/224] [release] Update Release Guide Instructions (#29919) --- contributor-docs/release-guide.md | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index fb566bc2a752..e347eb988860 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -271,12 +271,6 @@ fixes. This is because bugs are always being introduced as part of normal development. We cut the branch to prevent new bugs being introduced and _then_ we fix and cherrypick any truly release-blocking problems. -In order to run this workflow, you will need to provide a Apache ID and Jenkins -API token. Your Jenkins API token can be generated by visiting -https://ci-beam.apache.org, signing in with your Apache credentials, then going -to `https://ci-beam.apache.org/user//configure` and clicking -`Add new token` in the API token section. - - [ ] Run [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) (click `run workflow`) @@ -353,13 +347,9 @@ issues that would block the creation of the release candidate. ``` (cd release/src/main/scripts && ./verify_release_build.sh) ``` - 4. Trigger all Github Action and Jenkins PostCommit jobs from the PR created by the previous step. + 4. Trigger all Github Action jobs from the PR created by the previous step. For GitHub Action jobs, they should be triggered by the pull_request_target event of a specific placeholder file added to the PR (`release/trigger_all_tests.json`), so no additional action should be needed. - You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. - Or manually add one trigger phrase per PR comment for Jenkins tests, or rerun the workflow for GitHub Action tests. - See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) - for a full list of phrases. - **Tasks included in the script** - Installs `hub` with your agreement and setup local git repo; From 8bd4a1e525ba5f3256124ae331acf0cc48ad65f9 Mon Sep 17 00:00:00 2001 From: Cheskel Twersky Date: Fri, 5 Jan 2024 16:15:32 +0200 Subject: [PATCH 213/224] [Typescript] stop service when done (#29929) --- .../src/apache_beam/runners/dataflow.ts | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/sdks/typescript/src/apache_beam/runners/dataflow.ts b/sdks/typescript/src/apache_beam/runners/dataflow.ts index e7da1f7ada51..0a88b17f695d 100644 --- a/sdks/typescript/src/apache_beam/runners/dataflow.ts +++ b/sdks/typescript/src/apache_beam/runners/dataflow.ts @@ -36,13 +36,20 @@ export function dataflowRunner(runnerOptions: { augmentedOptions.experiments.push("use_runner_v2"); augmentedOptions.experiments.push("use_portable_job_submission"); augmentedOptions.experiments.push("use_sibling_sdk_workers"); - return new PortableRunner( + const service = PythonService.forModule( + "apache_beam.runners.dataflow.dataflow_job_service", + ["--port", "{{PORT}}"] + ); + const result = new PortableRunner( runnerOptions as any, - PythonService.forModule( - "apache_beam.runners.dataflow.dataflow_job_service", - ["--port", "{{PORT}}"] - ) + service ).runPipeline(pipeline, augmentedOptions); + result.then((res) => { + res.waitUntilFinish().then((_state) => { + service.stop(); + }); + }); + return result; } })(); } From c6f0d9c462a1d9968b4ac7bc622fc0d62b15287a Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 5 Jan 2024 13:49:26 -0500 Subject: [PATCH 214/224] Add notebooks for text embeddings (#29893) * Add notebooks for embeddings * Truncate embeddings in the output * Update huggingface_text_embeddings.ipynb * Update huggingface_text_embeddings.ipynb * Add files via upload * Update vertex_ai_text_embeddings.ipynb * Add files via upload * Update huggingface_text_embeddings.ipynb * Update scale_data.ipynb * Update compute_and_apply_vocab.ipynb * Apply suggestions from code review Co-authored-by: Danny McCormick * Update examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb * Address comments * Address comments for the Vertex AI notebook * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Remove header * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --------- Co-authored-by: Danny McCormick Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../compute_and_apply_vocab.ipynb | 4 +- .../huggingface_text_embeddings.ipynb | 404 ++++++++++++++++++ .../scale_data.ipynb | 4 +- .../vertex_ai_text_embeddings.ipynb | 316 ++++++++++++++ 4 files changed, 724 insertions(+), 4 deletions(-) rename examples/notebooks/beam-ml/{mltransform => data_preprocessing}/compute_and_apply_vocab.ipynb (97%) create mode 100644 examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb rename examples/notebooks/beam-ml/{mltransform => data_preprocessing}/scale_data.ipynb (98%) create mode 100644 examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb diff --git a/examples/notebooks/beam-ml/mltransform/compute_and_apply_vocab.ipynb b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb similarity index 97% rename from examples/notebooks/beam-ml/mltransform/compute_and_apply_vocab.ipynb rename to examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb index 97b54eba20dc..3e25a1e23d72 100644 --- a/examples/notebooks/beam-ml/mltransform/compute_and_apply_vocab.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb @@ -49,10 +49,10 @@ "\n", "\n", " \n", " \n", "
      \n", - " Run in Google Colab\n", + " Run in Google Colab\n", " \n", - " View source on GitHub\n", + " View source on GitHub\n", "
      \n" ], diff --git a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb new file mode 100644 index 000000000000..cb996da659f6 --- /dev/null +++ b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb @@ -0,0 +1,404 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "code", + "execution_count": 27, + "metadata": { + "id": "UmEFwsNs1OES" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Generate text embeddings by using Hugging Face Hub models\n", + "\n", + "\n", + " \n", + " \n", + "
      \n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
      \n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "\n", + "## Text embeddings\n", + "\n", + "Use text embeddings to represent text as numerical vectors. This process lets computers understand and process text data, which is essential for many natural language processing (NLP) tasks.\n", + "\n", + "### Uses of text embeddings\n", + "The following NLP tasks use embeddings:\n", + "\n", + "* **Semantic search:** Find documents or passages that are relevant to a query when the query doesn't use the exact same words as the documents.\n", + "* **Text classification:** Categorize text data into different classes, such as spam and not spam, or positive sentiment and negative sentiment.\n", + "* **Machine translation:** Translate text from one language to another and preserve the meaning.\n", + "* **Text summarization:** Create shorter summaries of text.\n", + "\n", + "This notebook uses Apache Beam's `MLTransform` to generate embeddings from text data.\n", + "\n", + "Hugging Face's [`SentenceTransformers`](https://huggingface.co/sentence-transformers) framework uses Python to generate sentence, text, and image embeddings.\n", + "\n", + "To generate text embeddings that use Hugging Face models and `MLTransform`, use the `SentenceTransformerEmbeddings` module to specify the model configuration.\n" + ], + "metadata": { + "id": "yvVIEhF01ZWq" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Install dependencies\n", + "\n", + "Install Apache Beam and the dependencies needed to work with Hugging Face embeddings. The dependencies includes the `sentence-transformers` package, which is required to use the `SentenceTransformerEmbeddings` module.\n" + ], + "metadata": { + "id": "jqYXaBJ821Zs" + } + }, + { + "cell_type": "code", + "source": [ + "! git clone https://github.com/apache/beam.git\n", + "! cd beam/sdks/python\n", + "! pip install beam/sdks/python\n", + "! pip install sentence-transformers" + ], + "metadata": { + "id": "shzCUrZI1XhF" + }, + "execution_count": 28, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "import tempfile\n", + "import apache_beam as beam\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "from apache_beam.ml.transforms.embeddings.huggingface import SentenceTransformerEmbeddings" + ], + "metadata": { + "id": "jVxSi2jS3M3b" + }, + "execution_count": 29, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Process the data\n", + "\n", + "`MLTransform` is a `PTransform` that you can use for data preparation, including generating text embeddings.\n", + "\n", + "### Use MLTransform in write mode\n", + "\n", + "In `write` mode, `MLTransform` saves the transforms and their attributes to an artifact location. Then, when you run `MLTransform` in `read` mode, these transforms are used. This process ensures that you're applying the same preprocessing steps when you train your model and when you serve the model in production or test its accuracy.\n", + "\n", + "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation." + ], + "metadata": { + "id": "kXDM8C7d3nPV" + } + }, + { + "cell_type": "markdown", + "source": [ + "### Get the data\n", + "\n", + "The following text inputs come from the Hugging Face blog [Getting Started With Embeddings](https://huggingface.co/blog/getting-started-with-embeddings).\n", + "\n", + "\n", + "`MLTransform` operates on dictionaries of data. To generate embeddings for specific columns, provide the column names as input to the `columns` argument in the `SentenceTransformerEmbeddings` package.\"" + ], + "metadata": { + "id": "Dbkmu3HP6Kql" + } + }, + { + "cell_type": "code", + "source": [ + "content = [\n", + " {'x': 'How do I get a replacement Medicare card?'},\n", + " {'x': 'What is the monthly premium for Medicare Part B?'},\n", + " {'x': 'How do I terminate my Medicare Part B (medical insurance)?'},\n", + " {'x': 'How do I sign up for Medicare?'},\n", + " {'x': 'Can I sign up for Medicare Part B if I am working and have health insurance through an employer?'},\n", + " {'x': 'How do I sign up for Medicare Part B if I already have Part A?'},\n", + " {'x': 'What are Medicare late enrollment penalties?'},\n", + " {'x': 'What is Medicare and who can get it?'},\n", + " {'x': 'How can I get help with my Medicare Part A and Part B premiums?'},\n", + " {'x': 'What are the different parts of Medicare?'},\n", + " {'x': 'Will my Medicare premiums be higher because of my higher income?'},\n", + " {'x': 'What is TRICARE ?'},\n", + " {'x': \"Should I sign up for Medicare Part B if I have Veterans' Benefits?\"}\n", + "]\n", + "\n", + "text_embedding_model_name = 'sentence-transformers/sentence-t5-large'\n", + "\n", + "\n", + "# helper function that returns a dict containing only first\n", + "# ten elements of generated embeddings\n", + "def truncate_embeddings(d):\n", + " for key in d.keys():\n", + " d[key] = d[key][:10]\n", + " return d" + ], + "metadata": { + "id": "LCTUs8F73iDg" + }, + "execution_count": 30, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "\n", + "### Generate text embeddings\n", + "This example uses the model `sentence-transformers/sentence-t5-large` to generate text embeddings. The model uses only the encoder from a `T5-large model`. The weights are stored in FP16. For more information about the model, see [Sentence-T5: Scalable Sentence Encoders from Pre-trained Text-to-Text Models](https://arxiv.org/abs/2108.08877)." + ], + "metadata": { + "id": "SApMmlRLRv_e" + } + }, + { + "cell_type": "code", + "source": [ + "artifact_location_t5 = tempfile.mkdtemp(prefix='huggingface_')\n", + "embedding_transform = SentenceTransformerEmbeddings(\n", + " model_name=text_embedding_model_name, columns=['x'])\n", + "\n", + "with beam.Pipeline() as pipeline:\n", + " data_pcoll = (\n", + " pipeline\n", + " | \"CreateData\" >> beam.Create(content))\n", + " transformed_pcoll = (\n", + " data_pcoll\n", + " | \"MLTransform\" >> MLTransform(write_artifact_location=artifact_location_t5).with_transform(embedding_transform))\n", + "\n", + " transformed_pcoll | beam.Map(truncate_embeddings) | 'LogOutput' >> beam.Map(print)\n", + "\n", + " transformed_pcoll | \"PrintEmbeddingShape\" >> beam.Map(lambda x: print(f\"Embedding shape: {len(x['x'])}\"))" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "SF6izkN134sf", + "outputId": "524d3506-d31f-4dee-9079-1ed6d7cadf1a" + }, + "execution_count": 31, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "{'x': [-0.0317193828523159, -0.005265399813652039, -0.012499183416366577, 0.00018130357784684747, -0.005592408124357462, 0.06207558885216713, -0.01656288281083107, 0.0167048592120409, -0.01239298190921545, 0.03041897714138031]}\n", + "Embedding shape: 10\n", + "{'x': [-0.015295305289328098, 0.005405726842582226, -0.015631258487701416, 0.022797023877501488, -0.027843449264764786, 0.03968179598450661, -0.004387892782688141, 0.022909151390194893, 0.01015392318367958, 0.04723235219717026]}\n", + "Embedding shape: 10\n", + "{'x': [-0.03450256213545799, -0.002632762538269162, -0.022460950538516045, -0.011689935810863972, -0.027329981327056885, 0.07293087989091873, -0.03069353476166725, 0.05429817736148834, -0.01308195199817419, 0.017668722197413445]}\n", + "Embedding shape: 10\n", + "{'x': [-0.02869587577879429, -0.0002648509689606726, -0.007186499424278736, -0.0003750955802388489, 0.012458174489438534, 0.06721009314060211, -0.013404129073023796, 0.03204648941755295, -0.021021844819188118, 0.04968355968594551]}\n", + "Embedding shape: 10\n", + "{'x': [-0.03241290897130966, 0.006845517549663782, 0.02001815102994442, -0.0057969288900494576, 0.008191823959350586, 0.08160955458879471, -0.009215254336595535, 0.023534387350082397, -0.02034241147339344, 0.0357462577521801]}\n", + "Embedding shape: 10\n", + "{'x': [-0.04592451825737953, -0.0025395643897354603, -0.01178023498505354, 0.011568977497518063, -0.0029014083556830883, 0.06971456110477448, -0.021167151629924774, 0.015902182087302208, -0.015007994137704372, 0.026213033124804497]}\n", + "Embedding shape: 10\n", + "{'x': [0.005221465136855841, -0.002127869985997677, -0.002369001042097807, -0.019337018951773643, 0.023243796080350876, 0.05599674955010414, -0.022721167653799057, 0.024813007563352585, -0.010685156099498272, 0.03624529018998146]}\n", + "Embedding shape: 10\n", + "{'x': [-0.035339221358299255, 0.010706206783652306, -0.001701260800473392, -0.00862252525985241, 0.006445988081395626, 0.08198338001966476, -0.022678885608911514, 0.01434261817485094, -0.008092232048511505, 0.03345781937241554]}\n", + "Embedding shape: 10\n", + "{'x': [-0.030748076736927032, 0.009340512566268444, -0.013637945055961609, 0.011183148249983788, -0.013879665173590183, 0.046350326389074326, -0.024090109393000603, 0.02885228954255581, -0.01699884608387947, 0.01672385260462761]}\n", + "Embedding shape: 10\n", + "{'x': [-0.040792081505060196, -0.00872269831597805, -0.015838179737329483, -0.03141209855675697, -7.104632823029533e-05, 0.08301416039466858, -0.034691162407398224, 0.0026397297624498606, 0.009255227632820606, 0.05415954813361168]}\n", + "Embedding shape: 10\n", + "{'x': [-0.02156883291900158, 0.003969342447817326, -0.030446071177721024, 0.008231461979448795, -0.01271845493465662, 0.03793857619166374, -0.013524272479116917, -0.0385628417134285, -0.0058258213102817535, 0.03505263477563858]}\n", + "Embedding shape: 10\n", + "{'x': [-0.027544165030121803, -0.01773364469408989, -0.013286487199366093, -0.008328652940690517, -0.011047529056668282, 0.05237515643239021, -0.016948163509368896, 0.02806701697409153, -0.0018120920285582542, 0.027241172268986702]}\n", + "Embedding shape: 10\n", + "{'x': [-0.03464886546134949, -0.003521248232573271, -0.010239562019705772, -0.018618224188685417, 0.004094886127859354, 0.062059685587882996, -0.013881963677704334, -0.0008639032603241503, -0.029874088242650032, 0.033531222492456436]}\n", + "Embedding shape: 10\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "You can pass additional arguments that are supported by `sentence-transformer` models, such as `convert_to_numpy=False`. These arguments are passed as a `dict` to the `SentenceTransformerEmbeddings` transform by using the `inference_args` parameter.\n", + "\n", + "When you pass `convert_to_numpy=False`, the output contains `torch.Tensor` matrices." + ], + "metadata": { + "id": "1MFom0PW_vRv" + } + }, + { + "cell_type": "code", + "source": [ + "artifact_location_t5_with_inference_args = tempfile.mkdtemp(prefix='huggingface_')\n", + "\n", + "embedding_transform = SentenceTransformerEmbeddings(\n", + " model_name=text_embedding_model_name, columns=['x'],\n", + " inference_args={'convert_to_numpy': False}\n", + " )\n", + "\n", + "with beam.Pipeline() as pipeline:\n", + " data_pcoll = (\n", + " pipeline\n", + " | \"CreateData\" >> beam.Create(content))\n", + " transformed_pcoll = (\n", + " data_pcoll\n", + " | \"MLTransform\" >> MLTransform(write_artifact_location=artifact_location_t5_with_inference_args).with_transform(embedding_transform))\n", + "\n", + " # The outputs are in the PyTorch tensor type.\n", + " transformed_pcoll | 'LogOutput' >> beam.Map(lambda x: print(type(x['x'])))\n", + "\n", + " transformed_pcoll | \"PrintEmbeddingShape\" >> beam.Map(lambda x: print(f\"Embedding shape: {len(x['x'])}\"))\n" + ], + "metadata": { + "id": "xyezKuzY_uLD", + "colab": { + "base_uri": "https://localhost:8080/" + }, + "outputId": "96babb3b-c61b-40a2-96f7-572a2a46bd83" + }, + "execution_count": 24, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n", + "\n", + "Embedding shape: 768\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "### Use MLTransform in read mode\n", + "\n", + "In `read` mode, `MLTransform` uses the artifacts generated during `write` mode. In this case, the `SentenceTransformEmbedding` transform and its attributes are loaded from the saved artifacts. You don't need to specify the artifacts again during `read` mode.\n", + "\n", + "In this way, `MLTransform` provides consistent preprocessing steps for training and inference workloads." + ], + "metadata": { + "id": "aPIQzCoF_EBj" + } + }, + { + "cell_type": "code", + "source": [ + "test_content = [\n", + " {\n", + " 'x': 'This is a test sentence'\n", + " },\n", + " {\n", + " 'x': 'The park is full of dogs'\n", + " },\n", + " {\n", + " 'x': \"Should I sign up for Medicare Part B if I have Veterans' Benefits?\"\n", + " }\n", + "]\n", + "\n", + "# Uses the T5 model to generate text embeddings\n", + "with beam.Pipeline() as pipeline:\n", + " data_pcoll = (\n", + " pipeline\n", + " | \"CreateData\" >> beam.Create(test_content))\n", + " transformed_pcoll = (\n", + " data_pcoll\n", + " | \"MLTransform\" >> MLTransform(read_artifact_location=artifact_location_t5))\n", + "\n", + " transformed_pcoll | beam.Map(truncate_embeddings) | 'LogOutput' >> beam.Map(print)\n" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "RCqYeUd3_F3C", + "outputId": "4c5b61d2-df39-4af3-8520-f529f243e3b1" + }, + "execution_count": 26, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "{'x': [0.00036313451710157096, -0.03929319977760315, -0.03574873134493828, 0.05015222355723381, 0.04295048117637634, 0.04800170287489891, 0.006883862894028425, -0.02567591704428196, -0.048067063093185425, 0.036534328013658524]}\n", + "{'x': [-0.053793832659721375, 0.006730600260198116, -0.025130020454525948, 0.04363932088017464, 0.03323192894458771, 0.008803879842162132, -0.015412433072924614, 0.008926985785365105, -0.061175212264060974, 0.04573329910635948]}\n", + "{'x': [-0.03464885801076889, -0.003521254053339362, -0.010239563882350922, -0.018618224188685417, 0.004094892647117376, 0.062059689313173294, -0.013881963677704334, -0.000863900815602392, -0.029874078929424286, 0.03353121876716614]}\n" + ] + } + ] + } + ] +} \ No newline at end of file diff --git a/examples/notebooks/beam-ml/mltransform/scale_data.ipynb b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb similarity index 98% rename from examples/notebooks/beam-ml/mltransform/scale_data.ipynb rename to examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb index c69ea0f8f4cd..4f99a7a1dcbb 100644 --- a/examples/notebooks/beam-ml/mltransform/scale_data.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb @@ -49,10 +49,10 @@ "\n", "\n", " \n", " \n", "
      \n", - " Run in Google Colab\n", + " Run in Google Colab\n", " \n", - " View source on GitHub\n", + " View source on GitHub\n", "
      \n" ], diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb new file mode 100644 index 000000000000..bf38c276987d --- /dev/null +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -0,0 +1,316 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "muiqKarukWj0" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Generate text embeddings by using the Vertex AI API\n", + "\n", + "\n", + " \n", + " \n", + "
      \n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
      \n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "Text embeddings are a way to represent text as numerical vectors. This process lets computers understand and process text data, which is essential for many natural language processing (NLP) tasks.\n", + "\n", + "## Uses of text embeddings\n", + "The following NLP tasks use embeddings:\n", + "\n", + "* **Semantic search:** Find documents or passages that are relevant to a query when the query doesn't use the exact same words as the documents.\n", + "* **Text classification:** Categorize text data into different classes, such as spam and not spam, or positive sentiment and negative sentiment.\n", + "* **Machine translation:** Translate text from one language to another and preserve the meaning.\n", + "* **Text summarization:** Create shorter summaries of text.\n", + "\n", + "This notebook generates embeddings from text data by using Apache Beam's `MLTransform` with the Vertex AI Python SDK.\n", + "\n", + "Use the Vertex AI text-embeddings API to generate text embeddings that use Google’s large generative artificial intelligence (AI) models. To generate text embeddings by using the Vertex AI text-embeddings API, use `MLTransform` with the `VertexAITextEmbeddings` class to specify the model configuration. For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). \n", + "\n", + "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation.\n", + "\n", + "## Requirements\n", + "\n", + "To use the Vertex AI text-embeddings API, complete the following prerequisites:\n", + "\n", + "* Install the `google-cloud-aiplatform` Python package.\n", + "* Do one of the following tasks:\n", + " * Configure credentials for your Google Cloud project. For more information, see [Google Auth Library for Python](https://googleapis.dev/python/google-auth/latest/reference/google.auth.html#module-google.auth).\n", + " * Store the path to a service account JSON file by using the [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/application-default-credentials#GAC) environment variable." + ], + "metadata": { + "id": "bkpSCGCWlqAf" + } + }, + { + "cell_type": "markdown", + "source": [ + "To use your Google Cloud account, authenticate this notebook." + ], + "metadata": { + "id": "W29FgO5Qv2ew" + } + }, + { + "cell_type": "code", + "source": [ + "from google.colab import auth\n", + "auth.authenticate_user()\n", + "\n", + "# TODO: Remove the project name before merging.\n", + "project = 'google.com:clouddfe' # Replace with a valid project id." + ], + "metadata": { + "id": "nYyyGYt3licq" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Install dependencies\n", + " Install Apache Beam and the dependencies required for the Vertex AI text-embeddings API." + ], + "metadata": { + "id": "UQROd16ZDN5y" + } + }, + { + "cell_type": "code", + "source": [ + "! git clone https://github.com/apache/beam.git\n", + "! cd beam/sdks/python\n", + "! pip install beam/sdks/python[gcp]" + ], + "metadata": { + "id": "BTxob7d5DLBM" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "import tempfile\n", + "import apache_beam as beam\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "from apache_beam.ml.transforms.embeddings.vertex_ai import VertexAITextEmbeddings" + ], + "metadata": { + "id": "SkMhR7H6n1P0" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Transform the data\n", + "\n", + "`MLTransform` is a `PTransform` that you can use for data preparation, including generating text embeddings.\n", + "\n", + "### Use MLTransform in write mode\n", + "\n", + "In `write` mode, `MLTransform` saves the transforms and their attributes to an artifact location. Then, when you run `MLTransform` in `read` mode, these transforms are used. This process ensures that you're applying the same preprocessing steps when you train your model and when you serve the model in production or test its accuracy.\n", + "\n", + "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation." + ], + "metadata": { + "id": "cokOaX2kzyke" + } + }, + { + "cell_type": "markdown", + "source": [ + "### Get the data\n", + "\n", + "`MLTransform` processes dictionaries that include column names and their associated text data. To generate embeddings for specific columns, specify these column names in the `columns` argument of `VertexAITextEmbeddings`. This transform uses the the Vertex AI text-embeddings API for online predictions to generate an embeddings vector for each sentence." + ], + "metadata": { + "id": "-x7fVvuy-aDs" + } + }, + { + "cell_type": "code", + "source": [ + "artifact_location = tempfile.mkdtemp(prefix='vertex_ai')\n", + "\n", + "# Use the latest text embedding model from the Vertex AI text-embeddings API documentation.\n", + "# https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text-embeddings\n", + "text_embedding_model_name = 'textembedding-gecko@latest'\n", + "\n", + "# Generate text embeddings on the sentences.\n", + "content = [\n", + " {\n", + " 'x' : 'I would like embeddings for this text'\n", + " },\n", + " {\n", + " 'x' : 'Hello world'\n", + " },\n", + " {\n", + " 'x': 'The Dog is running in the park.'\n", + " }\n", + " ]\n", + "\n", + "# helper function that returns a dict containing only first\n", + "# ten elements of generated embeddings\n", + "def truncate_embeddings(d):\n", + " for key in d.keys():\n", + " d[key] = d[key][:10]\n", + " return d" + ], + "metadata": { + "id": "be-vR159pylF" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "embedding_transform = VertexAITextEmbeddings(\n", + " model_name=text_embedding_model_name, columns=['x'], project=project)\n", + "\n", + "with beam.Pipeline() as pipeline:\n", + " data_pcoll = (\n", + " pipeline\n", + " | \"CreateData\" >> beam.Create(content))\n", + " transformed_pcoll = (\n", + " data_pcoll\n", + " | \"MLTransform\" >> MLTransform(write_artifact_location=artifact_location).with_transform(embedding_transform))\n", + "\n", + " # Show only the first ten elements of the embeddings to prevent clutter in the output.\n", + " transformed_pcoll | beam.Map(truncate_embeddings) | 'LogOutput' >> beam.Map(print)\n", + "\n", + " transformed_pcoll | \"PrintEmbeddingShape\" >> beam.Map(lambda x: print(f\"Embedding shape: {len(x['x'])}\"))" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "UQGm1be3p7lM", + "outputId": "b41172ca-1c73-4952-ca87-bfe45ca88a6c" + }, + "execution_count": null, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "{'x': [0.041293490678071976, -0.010302993468940258, -0.048611514270305634, -0.01360565796494484, 0.06441926211118698, 0.022573700174689293, 0.016446372494101524, -0.033894773572683334, 0.004581860266625881, 0.060710687190294266]}\n", + "Embedding shape: 10\n", + "{'x': [0.05889148637652397, -0.0046180677600204945, -0.06738516688346863, -0.012708292342722416, 0.06461101770401001, 0.025648491457104683, 0.023468563333153725, -0.039828114211559296, -0.009968819096684456, 0.050098177045583725]}\n", + "Embedding shape: 10\n", + "{'x': [0.04683901369571686, -0.013076924718916416, -0.082594133913517, -0.01227626483887434, 0.00417641457170248, -0.024504298344254494, 0.04282262548804283, -0.0009824123699218035, -0.02860993705689907, 0.01609829254448414]}\n", + "Embedding shape: 10\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "### Use MLTransform in read mode\n", + "\n", + "In `read` mode, `MLTransform` uses the artifacts saved during `write` mode. In this example, the transform and its attributes are loaded from the saved artifacts. You don't need to specify artifacts again during `read` mode.\n", + "\n", + "In this way, `MLTransform` provides consistent preprocessing steps for training and inference workloads." + ], + "metadata": { + "id": "JLkmQkiLx_6h" + } + }, + { + "cell_type": "code", + "source": [ + "test_content = [\n", + " {\n", + " 'x': 'This is a test sentence'\n", + " },\n", + " {\n", + " 'x': 'The park is full of dogs'\n", + " },\n", + "]\n", + "\n", + "with beam.Pipeline() as pipeline:\n", + " data_pcoll = (\n", + " pipeline\n", + " | \"CreateData\" >> beam.Create(test_content))\n", + " transformed_pcoll = (\n", + " data_pcoll\n", + " | \"MLTransform\" >> MLTransform(read_artifact_location=artifact_location))\n", + "\n", + " transformed_pcoll | beam.Map(truncate_embeddings) | 'LogOutput' >> beam.Map(print)\n" + ], + "metadata": { + "id": "r8Y5vgfLx_Xu", + "colab": { + "base_uri": "https://localhost:8080/" + }, + "outputId": "e7cbf6b7-5c31-4efa-90cf-7a8a108ecc77" + }, + "execution_count": null, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "{'x': [0.04782044142484665, -0.010078949853777885, -0.05793016776442528, -0.026060665026307106, 0.05756739526987076, 0.02292264811694622, 0.014818413183093071, -0.03718176111578941, -0.005486017093062401, 0.04709304869174957]}\n", + "{'x': [0.042911216616630554, -0.007554919924587011, -0.08996245265007019, -0.02607591263949871, 0.0008614308317191899, -0.023671219125390053, 0.03999944031238556, -0.02983051724731922, -0.015057179145514965, 0.022963201627135277]}\n" + ] + } + ] + } + ] +} From a6948ad71e341aa966fd5ad2b31ceaa987c0d096 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 5 Jan 2024 14:07:11 -0500 Subject: [PATCH 215/224] Install beam from pypi instead of github in mltransform notebooks (#29934) * Update compute_and_apply_vocab.ipynb * Update compute_and_apply_vocab.ipynb * Update huggingface_text_embeddings.ipynb * Update huggingface_text_embeddings.ipynb * Update scale_data.ipynb * Update vertex_ai_text_embeddings.ipynb --- .../data_preprocessing/compute_and_apply_vocab.ipynb | 4 +--- .../data_preprocessing/huggingface_text_embeddings.ipynb | 8 +++----- .../notebooks/beam-ml/data_preprocessing/scale_data.ipynb | 4 +--- .../data_preprocessing/vertex_ai_text_embeddings.ipynb | 4 +--- 4 files changed, 6 insertions(+), 14 deletions(-) diff --git a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb index 3e25a1e23d72..2094a2af11ae 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb @@ -98,9 +98,7 @@ }, "outputs": [], "source": [ - "! git clone https://github.com/apache/beam.git\n", - "! cd beam/sdks/python\n", - "! pip install beam/sdks/python\n", + "! pip install apache_beam>=2.53.0 --quiet\n", "! pip install tensorflow-transform --quiet" ] }, diff --git a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb index cb996da659f6..2eb76497151a 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb @@ -100,10 +100,8 @@ { "cell_type": "code", "source": [ - "! git clone https://github.com/apache/beam.git\n", - "! cd beam/sdks/python\n", - "! pip install beam/sdks/python\n", - "! pip install sentence-transformers" + "! pip install apache_beam>=2.53.0 --quiet\n", + "! pip install sentence-transformers --quiet" ], "metadata": { "id": "shzCUrZI1XhF" @@ -401,4 +399,4 @@ ] } ] -} \ No newline at end of file +} diff --git a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb index 4f99a7a1dcbb..029c3a278669 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb @@ -106,9 +106,7 @@ { "cell_type": "code", "source": [ - "! git clone https://github.com/apache/beam.git\n", - "! cd beam/sdks/python\n", - "! pip install beam/sdks/python\n", + "! pip install apache_beam>=2.53.0 --quiet\n", "! pip install tensorflow-transform --quiet" ], "metadata": { diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb index bf38c276987d..352e1124d82d 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -129,9 +129,7 @@ { "cell_type": "code", "source": [ - "! git clone https://github.com/apache/beam.git\n", - "! cd beam/sdks/python\n", - "! pip install beam/sdks/python[gcp]" + "! pip install apache_beam[gcp]>=2.53.0 --quiet" ], "metadata": { "id": "BTxob7d5DLBM" From b9ae59a99789b76948b771a9c9240f5de9a1c9de Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 5 Jan 2024 14:50:55 -0500 Subject: [PATCH 216/224] Remove TODO in the notebook (#29935) * Update vertex_ai_text_embeddings.ipynb * Update vertex_ai_text_embeddings.ipynb * Update vertex_ai_text_embeddings.ipynb * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb index 352e1124d82d..1207793f5a18 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -107,8 +107,7 @@ "from google.colab import auth\n", "auth.authenticate_user()\n", "\n", - "# TODO: Remove the project name before merging.\n", - "project = 'google.com:clouddfe' # Replace with a valid project id." + "project = '' # Replace with a valid Google Cloud project ID." ], "metadata": { "id": "nYyyGYt3licq" From 0b0d97325c10292d662ee88ad64aa3264c4f9c7f Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 5 Jan 2024 12:08:10 -0800 Subject: [PATCH 217/224] Update pmc finalization in release-guide.md (#29936) Make the copy instructions precise. --- contributor-docs/release-guide.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index e347eb988860..b3b3213d9c97 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -1217,13 +1217,16 @@ Ping [dev@](mailto:dev@beam.apache.org) mailing list for assistance if you need Copy the source release from the `dev` repository to the `release` repository at `dist.apache.org` using Subversion. ``` +export RELEASE_VERSION= +export OLD_RELEASE_VERSION= + svn co https://dist.apache.org/repos/dist/dev/beam dev # Checkout the `dev` artifact repo. svn co https://dist.apache.org/repos/dist/release/beam release # Checkout the `release` artifact repo. mkdir release/$RELEASE_VERSION -# Copy files from the `dev` artifact repo to the `release` artifact repo. +cp -R dev/$RELEASE_VERSION/* release/$RELEASE_VERSION/ cd release From f79eaddf5749af2602ae17273faac10760fac52c Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Fri, 5 Jan 2024 13:16:34 -0800 Subject: [PATCH 218/224] Preparing the data processing notebooks for import to devsite (#29937) * Preparing the data processing notebooks for import to devsite * Update the title casing for the notebook link in the readme --- examples/notebooks/beam-ml/README.md | 19 ++++--- .../compute_and_apply_vocab.ipynb | 38 +++++++------ .../huggingface_text_embeddings.ipynb | 5 +- .../data_preprocessing/scale_data.ipynb | 54 +++++++++---------- .../vertex_ai_text_embeddings.ipynb | 5 +- 5 files changed, 57 insertions(+), 64 deletions(-) diff --git a/examples/notebooks/beam-ml/README.md b/examples/notebooks/beam-ml/README.md index 5a90f1c68b49..f1c19747fc71 100644 --- a/examples/notebooks/beam-ml/README.md +++ b/examples/notebooks/beam-ml/README.md @@ -16,7 +16,7 @@ specific language governing permissions and limitations under the License. --> -# ML Sample Notebooks +# ML sample notebooks Starting with the Apache Beam SDK version 2.40, users have access to a [RunInference](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.RunInference) @@ -27,13 +27,13 @@ The model handler abstracts the user from the configuration needed for specific frameworks, such as Tensorflow, PyTorch, and others. For a full list of supported frameworks, see the [About Beam ML](https://beam.apache.org/documentation/ml/about-ml/) page. -## Using The Notebooks +## Use the notebooks These notebooks illustrate ways to use Apache Beam's RunInference transforms, as well as different use cases for [`ModelHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.ModelHandler) implementations. Beam comes with [multiple `ModelHandler` implementations](https://beam.apache.org/documentation/ml/about-ml/#modify-a-python-pipeline-to-use-an-ml-model). -### Loading the Notebooks +### Load the notebooks 1. To get started quickly with notebooks, use [Colab](https://colab.sandbox.google.com/). 2. In Colab, open the notebook from GitHub using the notebook URL, for example: @@ -48,6 +48,14 @@ to your project and bucket. This section contains the following example notebooks. +### Data processing + +* [Generate text embeddings by using the Vertex AI API](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb) +* [Generate text embeddings by using Hugging Face Hub models](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb) +* [Compute and apply vocabulary on a dataset](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb) +* [Use MLTransform to scale data](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb) +* [Preprocessing with the Apache Beam DataFrames API](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/dataframe_api_preprocessing.ipynb) + ### Prediction and inference with pretrained models * [Apache Beam RunInference for PyTorch](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch.ipynb) @@ -85,8 +93,3 @@ This section contains the following example notebooks. ### Model Evaluation * [Use TFMA to evaluate and compare model performance](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/tfma_beam.ipynb) - -### Data processing - -* [Preprocess data with MLTransform](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/mltransform_basic.ipynb) -* [Preprocessing with the Apache Beam DataFrames API](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/dataframe_api_preprocessing.ipynb) diff --git a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb index 2094a2af11ae..76f26f2aabe0 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb @@ -45,7 +45,7 @@ { "cell_type": "markdown", "source": [ - "# Compute and Apply Vocabulary on a dataset using `MLTransform`\n", + "# Compute and apply vocabulary on a dataset\n", "\n", "\n", "
      \n", @@ -63,12 +63,12 @@ { "cell_type": "markdown", "source": [ - "[ComputeAndApplyVocabulary](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ComputeAndApplyVocabulary) is a data processing transform that computes a unique vocabulary from a dataset and then maps each word or token to a distinct integer index. Use this transform to change textual data into numerical representations for machine learning tasks.\n", + "[`ComputeAndApplyVocabulary`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ComputeAndApplyVocabulary) is a data processing transform that computes a unique vocabulary from a dataset and then maps each word or token to a distinct integer index. Use this transform to change textual data into numerical representations for machine learning (ML) tasks.\n", "\n", - "Generating a vocabulary on the incoming dataset is a crucial preprocessing step while training machine learning models that deal with text data. By mapping words to numerical indices, the vocabulary reduces the complexity and dimensionality of dataset, allowing ML models to process the same words in a consistent way.\n", + "When you train ML models that use text data, generating a vocabulary on the incoming dataset is a crucial preprocessing step. By mapping words to numerical indices, the vocabulary reduces the complexity and dimensionality of dataset. This step allows ML models to process the same words in a consistent way.\n", "\n", "This notebook shows how to use `MLTransform` to complete the following tasks:\n", - "* Use `write` mode in `MLTransform` to generate a vocabulary on the input text and assign an index value to each token.\n", + "* Use `write` mode to generate a vocabulary on the input text and assign an index value to each token.\n", "* Use `read` mode to use the generated vocabulary and assign an index to a different dataset.\n", "\n", "`MLTransform` uses the `ComputeAndApplyVocabulary` transform, which is implemented by using `tensorflow_transform` to generate the vocabulary.\n", @@ -120,15 +120,13 @@ { "cell_type": "markdown", "source": [ - "### Artifact location\n", + "## Artifact location\n", "\n", - "Artifact location are used to store the artifacts, such as vocabulary file generated by `ComputeAndApplyVocabulary`, in `MLTransform` write mode.\n", + "In `write` mode, the artifact location is used to store artifacts, such as the vocabulary file generated by `ComputeAndApplyVocabulary`.\n", "\n", - "**NOTE**: Artifact location must be empty otherwise a `RuntimeError` will be raised.\n", + "**NOTE**: The artifact location must be empty. If it isn't empty, a `RuntimeError` occurs.\n", "\n", - "During the `MLTransform` read mode, `MLTransform` will fetch artifacts from the specified artifact location.\n", - "\n", - "**NOTE**: In read mode, make sure to pass the same artifact location that was used in write mode. Otherwise, it could result in `RuntimeError` or `MLTransform` will produce unexpected results in read mode.\n" + "In `read` mode, `MLTransform` fetches artifacts from the specified artifact location. Pass the same artifact location that you used in `write` mode. Otherwise, a `RuntimeError` occurs or `MLTransform` produces unexpected results in `read` mode.\n" ], "metadata": { "id": "vfarBxAMFvRA" @@ -165,9 +163,9 @@ { "cell_type": "markdown", "source": [ - "In this example, `MLTransform` in `write` mode uses `ComputeAndApplyVocabulary` to generate vocabulary on the incoming dataset. The incoming text data is split into tokens and each token is assigned an unique index.\n", + "In this example, in `write` mode, `MLTransform` uses `ComputeAndApplyVocabulary` to generate vocabulary on the incoming dataset. The incoming text data is split into tokens and each token is assigned an unique index.\n", "\n", - " The generated vocabulary is stored in an artifact location that you can use on a different dataset in `read` mode with `MLTransform`." + " The generated vocabulary is stored in an artifact location that you can use on a different dataset in `read` mode." ], "metadata": { "id": "oETBJNVfRws_" @@ -210,15 +208,15 @@ { "cell_type": "markdown", "source": [ - "### Understanding and Visualizing Vocabulary in Data Processing\n", + "## Understand and visualize vocabulary\n", "\n", - "When working with text data in machine learning, one common step is the generation of a vocabulary index. This process is effectively demonstrated through the `MLTransform` using `ComputeAndApplyVocabulary` transformation. Here, each unique word in your text data is assigned a specific index. This index is then used to represent the text in a numerical format, which is essential for machine learning algorithms.\n", + "When working with text data in machine learning, one common step is the generation of a vocabulary index. `MLTransform` completes this step by using the `ComputeAndApplyVocabulary` transformation. Each unique word in your text data is assigned a specific index. This index is then used to represent the text in a numerical format, which is needed for machine learning algorithms.\n", "\n", - "In the provided example, the `ComputeAndApplyVocabulary` transformation is applied to the `feature` column, creating a vocabulary index for each unique word found in this column.\n", + "In this example, the `ComputeAndApplyVocabulary` transformation is applied to the `feature` column. A vocabulary index is created for each unique word found in this column.\n", "\n", - "To visualize and understand this generated vocabulary, you can use the `ArtifactsFetcher` class. This class allows you to retrieve the vocabulary list from your specified location. Once you have this list, you can easily see the index associated with each word in your vocabulary. This index corresponds to the numerical representation used in the transformation output of `ComputeAndApplyVocabulary`.\n", + "To visualize and understand this generated vocabulary, use the `ArtifactsFetcher` class. This class allows you to retrieve the vocabulary list from your specified location. When you have this list, you can see the index associated with each word in your vocabulary. This index corresponds to the numerical representation used in the transformation output of `ComputeAndApplyVocabulary`.\n", "\n", - "By examining this vocabulary index, you gain insight into how your text data is being processed and represented numerically. This understanding is crucial for debugging and improving your machine learning models that rely on text data." + "Examine this vocabulary index to understand how your text data is being processed and represented numerically. This understanding is useful for debugging and improving machine learning models that rely on text data." ], "metadata": { "id": "hvTvzOw8iBi9" @@ -272,7 +270,7 @@ { "cell_type": "markdown", "source": [ - "### Frequency Threshold\n", + "## Frequency Threshold\n", "\n", "The `frequency_threshold` parameter identifies the elements that appear frequently in the dataset. This parameter limits the generated vocabulary to elements with an absolute frequency greater than or equal to the specified threshold. If you don't specify the parameter, the entire vocabulary is generated.\n", "\n", @@ -319,7 +317,7 @@ { "cell_type": "markdown", "source": [ - "In the above output, if the frequency of the token is less than the specified frequency, it is assigned to a `default_value` of `-1`. For the other tokens, a vocabulary file is generated." + "In the output, if the frequency of the token is less than the specified frequency, it is assigned to a `default_value` of `-1`. For the other tokens, a vocabulary file is generated." ], "metadata": { "id": "h1s4a6hzxKrb" @@ -361,7 +359,7 @@ "source": [ "## `MLTransform` for inference workloads\n", "\n", - "When `MLTransform` is in `write` mode, it produces artifacts, such as vocabulary files for `ComputeAndApplyVocabulary`. This allows you to ensure that you are applying the same vocabulary (and any other preprocessing transforms you apply) when you are training your model and serving it in production or testing its accuracy.\n", + "When `MLTransform` is in `write` mode, it produces artifacts, such as vocabulary files for `ComputeAndApplyVocabulary`. These artifacts allow you to apply the same vocabulary, and any other preprocessing transforms, when you train your model and serve it in production, or when you test its accuracy.\n", "\n", "When `MLTransform` is used `read` mode, it uses the previously generated vocabulary files to map the incoming text data. If the incoming vocabulary isn't found in the generated vocabulary, then the incoming vocabulary is mapped to a `default_value` provided during `write` mode. In this case, the `default_value` is `-1`.\n", "\n", diff --git a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb index 2eb76497151a..30bbadf5b9b0 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb @@ -63,12 +63,9 @@ { "cell_type": "markdown", "source": [ - "\n", - "## Text embeddings\n", "\n", "Use text embeddings to represent text as numerical vectors. This process lets computers understand and process text data, which is essential for many natural language processing (NLP) tasks.\n", "\n", - "### Uses of text embeddings\n", "The following NLP tasks use embeddings:\n", "\n", "* **Semantic search:** Find documents or passages that are relevant to a query when the query doesn't use the exact same words as the documents.\n", @@ -148,7 +145,7 @@ "The following text inputs come from the Hugging Face blog [Getting Started With Embeddings](https://huggingface.co/blog/getting-started-with-embeddings).\n", "\n", "\n", - "`MLTransform` operates on dictionaries of data. To generate embeddings for specific columns, provide the column names as input to the `columns` argument in the `SentenceTransformerEmbeddings` package.\"" + "`MLTransform` operates on dictionaries of data. To generate embeddings for specific columns, provide the column names as input to the `columns` argument in the `SentenceTransformerEmbeddings` package." ], "metadata": { "id": "Dbkmu3HP6Kql" diff --git a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb index 029c3a278669..abeeba2264ee 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb @@ -63,30 +63,28 @@ { "cell_type": "markdown", "source": [ - "Scaling data is an important preprocessing step for training machine learning models because it helps to ensure that all features have a similar weight or influence on the model. This can be beneficial for several reasons:\n", + "Scaling data is an important preprocessing step for training machine learning (ML) models, because it helps to ensure that all features have a similar weight or influence on the model. The following are benefits of scaling data:\n", "\n", - "1. Improves the convergence of gradient descent algorithms: Many machine learning algorithms, such as linear regression and neural networks, use gradient descent to optimize their parameters. Gradient descent works by iteratively moving the parameters of the model in the direction that reduces the loss function. If the features are not scaled, features with larger ranges can have a much larger impact on the gradient, making it difficult for the model to converge. Scaling the features helps to ensure that all features contribute equally to the gradient, which can lead to faster and more stable convergence.\n", + "* **Improved convergence of gradient descent algorithms:** Many machine learning algorithms, such as linear regression and neural networks, use gradient descent to optimize their parameters. Gradient descent iteratively moves the parameters of the model in the direction that reduces the loss function. If the features aren't scaled, features with larger ranges can have a much larger impact on the gradient, making it difficult for the model to converge. Scaling the features helps to ensure that all features contribute equally to the gradient, which can lead to faster and more stable convergence.\n", "\n", - "2. Uniformity in Features: If one feature has a much larger range than the other features, it can dominate the model and make it difficult for the model to learn from the other features. This can lead to poor performance and biased predictions. Scaling the features helps to prevent this by bringing all of the features into a similar range.\n", + "* **Uniformity in features:** If one feature has a much larger range than the other features, it can dominate the model and make it difficult for the model to learn from the other features. This lack of uniformity can cause poor performance and biased predictions. Scaling the features brings all of the features into a similar range.\n", "\n", "\n", - "To scale your dataset using Apache Beam, use `MLTransform` with the following scaling data\n", + "To scale your dataset using Apache Beam, use `MLTransform` with one of the following transforms:\n", "\n", "* `ScaleTo01`: Calculates the minimum and maximum of an entire dataset, and then scales the dataset between 0 and 1 based on minimum and maximum values.\n", "* `ScaleToZScore`: Calculates the mean and variance of an entire dataset, and then scales the dataset based on those values.\n", "* `ScaleByMinMax`: Scales the data in a dataset, taking minimum and maximum values as input parameters.\n", "\n", - "For each data processing transform, `MLTransform` runs in both `write` mode and `read` mode.\n", + "For each data processing transform, `MLTransform` runs in both `write` mode and `read` mode. For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation.\n", "\n", - "## MLTransform in write mode\n", + "### MLTransform in write mode\n", "\n", - "When `MLTransform` is in `write` mode, it produces artifacts, such as minimum, maximum, variance etc for different data processing transforms. This allows you to ensure that you are applying the same artifacts (and any other preprocessing transforms you apply) when you are training your model and serving it in production or testing its accuracy.\n", + "When `MLTransform` is in `write` mode, it produces artifacts, such as minimum, maximum, and variance, for different data processing transforms. These artifacts allow you to ensure that you're applying the same artifacts, and any other preprocessing transforms, when you train your model and serve it in production, or when you test its accuracy.\n", "\n", - "## MLTransform in read mode\n", + "### MLTransform in read mode\n", "\n", - "In read mode, `MLTransform` uses the artifacts generated in `write` mode to scale the entire dataset.\n", - "\n", - "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation." + "In read mode, `MLTransform` uses the artifacts generated in `write` mode to scale the entire dataset." ], "metadata": { "id": "F8-yTcjPUHLv" @@ -179,7 +177,7 @@ "source": [ "## Scale the data between 0 and 1\n", "\n", - "Scale the data so that it's in the range of 0 and 1. To scale the data, the transform calculates minimum and maximum values on the whole dataset, and then performs the following calculation:\n", + "Scale the data so that it's in the range of 0 to 1. To scale the data, the transform calculates minimum and maximum values on the whole dataset, and then performs the following calculation:\n", "\n", "`x = (x - x_min) / (x_max - x_min)`\n", "\n", @@ -262,11 +260,11 @@ { "cell_type": "markdown", "source": [ - "In the above dataset, the minimum and maximum values for columns:\n", - "* int_feature_1 : 5 and 63.\n", - "* int_feature_2 : -65 and -10\n", + "In the this dataset, the following are the minimum and maximum values for the columns:\n", + "* `int_feature_1`: 5 and 63.\n", + "* `int_feature_2`: -65 and -10\n", "\n", - "In the output for the column - `int_feature_1`, the data is scaled between 0 and 1 using the values `5` and `63`. `5` is scaled to `0` and `63` is scaled to `1` and rest are scaled between 0 and 1 using `x = (x - x_min) / (x_max - x_min)`" + "In the output for the column `int_feature_1`, the data is scaled between 0 and 1 by using the values `5` and `63`. `5` is scaled to `0`, and `63` is scaled to `1`. The remaining values are scaled between 0 and 1 by using the formula `x = (x - x_min) / (x_max - x_min)`." ], "metadata": { "id": "Hwvh4UyI23pz" @@ -275,7 +273,7 @@ { "cell_type": "code", "source": [ - "# MLTransform in read mode.\n", + "# MLTransform in read mode\n", "with beam.Pipeline() as pipeline:\n", " data_pcoll = pipeline | \"CreateData\" >> beam.Create(test_data)\n", "\n", @@ -311,19 +309,19 @@ { "cell_type": "markdown", "source": [ - "`MLTransform` learned in `write` mode that int_feature_1 ranges from 5 to 63.\n", + "`MLTransform` learned in `write` mode that `int_feature_1` ranges from 5 to 63.\n", "\n", - "In read mode, when it encounters 29 in test_data for `int_feature_1`, it scales it using\n", + "In `read` mode, when it encounters 29 in `test_data` for `int_feature_1`, it scales it by using the following formula:\n", "```\n", "(value - min) / (max - min)\n", "```\n", "\n", - "Now plugging the values\n", + "The following calculation shows the formula with the values:\n", "```\n", "(29 - 5) / (63 - 5) = 0.41379312\n", "```\n", "\n", - "So 29 is scaled based on the min and max values generated during `write` mode.\n" + "Twenty-nine is scaled based on the minimum and maximum values generated in `write` mode.\n" ], "metadata": { "id": "jwSiMoAH4vq5" @@ -332,9 +330,9 @@ { "cell_type": "markdown", "source": [ - "## Scale the data by using the z-score\n", + "## Scale by using the z-score\n", "\n", - "Similar to `ScaleTo01`, use [ScaleToZScore](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ScaleToZScore) to scale the values by using the [z-score]([z-score](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_z_score#:~:text=Scaling%20to%20z%2Dscore%20subtracts%20out%20the%20mean%20and%20divides%20by%20standard%20deviation.%20Note%20that%20the%20standard%20deviation%20computed%20here%20is%20based%20on%20the%20biased%20variance%20(0%20delta%20degrees%20of%20freedom)%2C%20as%20computed%20by%20analyzers.var.).\n" + "Similar to `ScaleTo01`, use [ScaleToZScore](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ScaleToZScore) to scale the values by using the [z-score](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_z_score#:~:text=Scaling%20to%20z%2Dscore%20subtracts%20out%20the%20mean%20and%20divides%20by%20standard%20deviation.%20Note%20that%20the%20standard%20deviation%20computed%20here%20is%20based%20on%20the%20biased%20variance%20(0%20delta%20degrees%20of%20freedom)%2C%20as%20computed%20by%20analyzers.var).\n" ], "metadata": { "id": "VH1CmCGm_PtS" @@ -343,7 +341,7 @@ { "cell_type": "code", "source": [ - "# MLTransform in write mode.\n", + "# MLTransform in write mode\n", "with beam.Pipeline() as pipeline:\n", " data_pcoll = pipeline | \"CreateData\" >> beam.Create(data)\n", "\n", @@ -381,7 +379,7 @@ { "cell_type": "code", "source": [ - "# MLTransform in read mode.\n", + "# MLTransform in read mode\n", "with beam.Pipeline() as pipeline:\n", " data_pcoll = pipeline | \"CreateData\" >> beam.Create(test_data)\n", "\n", @@ -417,7 +415,7 @@ { "cell_type": "markdown", "source": [ - "## Scale the data by using ScaleByMinMax\n", + "## Scale by using ScaleByMinMax\n", "\n", "Use [ScaleByMinMax](https://github.com/apache/beam/blob/9e8a310f0c0faddfba28176df5893d8ad8fd10a0/sdks/python/apache_beam/ml/transforms/tft.py#L450) to scale your data into the range of `[min_value, max_value]`." ], @@ -431,7 +429,7 @@ "min_value = 1\n", "max_value = 10\n", "\n", - "# MLTransform in write mode.\n", + "# MLTransform in write mode\n", "with beam.Pipeline() as pipeline:\n", " data_pcoll = pipeline | \"CreateData\" >> beam.Create(data)\n", "\n", @@ -469,7 +467,7 @@ { "cell_type": "code", "source": [ - "# MLTransform in read mode.\n", + "# MLTransform in read mode\n", "with beam.Pipeline() as pipeline:\n", " data_pcoll = pipeline | \"CreateData\" >> beam.Create(test_data)\n", "\n", diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb index 1207793f5a18..db9e550dc913 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -65,7 +65,6 @@ "source": [ "Text embeddings are a way to represent text as numerical vectors. This process lets computers understand and process text data, which is essential for many natural language processing (NLP) tasks.\n", "\n", - "## Uses of text embeddings\n", "The following NLP tasks use embeddings:\n", "\n", "* **Semantic search:** Find documents or passages that are relevant to a query when the query doesn't use the exact same words as the documents.\n", @@ -73,9 +72,7 @@ "* **Machine translation:** Translate text from one language to another and preserve the meaning.\n", "* **Text summarization:** Create shorter summaries of text.\n", "\n", - "This notebook generates embeddings from text data by using Apache Beam's `MLTransform` with the Vertex AI Python SDK.\n", - "\n", - "Use the Vertex AI text-embeddings API to generate text embeddings that use Google’s large generative artificial intelligence (AI) models. To generate text embeddings by using the Vertex AI text-embeddings API, use `MLTransform` with the `VertexAITextEmbeddings` class to specify the model configuration. For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). \n", + "This notebook uses the Vertex AI text-embeddings API to generate text embeddings that use Google’s large generative artificial intelligence (AI) models. To generate text embeddings by using the Vertex AI text-embeddings API, use `MLTransform` with the `VertexAITextEmbeddings` class to specify the model configuration. For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). \n", "\n", "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation.\n", "\n", From 3260a7bf25aab25186717823d81cb31af0fae346 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Fri, 5 Jan 2024 22:24:54 -0800 Subject: [PATCH 219/224] Final edit on data preprocessing notebooks (#29940) --- .../compute_and_apply_vocab.ipynb | 14 +++++++------- .../beam-ml/data_preprocessing/scale_data.ipynb | 8 ++++---- .../vertex_ai_text_embeddings.ipynb | 6 ++---- 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb index 76f26f2aabe0..ee47cb7711fa 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb @@ -63,9 +63,9 @@ { "cell_type": "markdown", "source": [ - "[`ComputeAndApplyVocabulary`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ComputeAndApplyVocabulary) is a data processing transform that computes a unique vocabulary from a dataset and then maps each word or token to a distinct integer index. Use this transform to change textual data into numerical representations for machine learning (ML) tasks.\n", + "The [`ComputeAndApplyVocabulary`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ComputeAndApplyVocabulary) data processing transform computes a unique vocabulary from a dataset and then maps each word or token to a distinct integer index. Use this transform to change textual data into numerical representations for machine learning (ML) tasks.\n", "\n", - "When you train ML models that use text data, generating a vocabulary on the incoming dataset is a crucial preprocessing step. By mapping words to numerical indices, the vocabulary reduces the complexity and dimensionality of dataset. This step allows ML models to process the same words in a consistent way.\n", + "When you train ML models that use text data, generating a vocabulary on the incoming dataset is an important preprocessing step. By mapping words to numerical indices, the vocabulary reduces the complexity and dimensionality of dataset. This step allows ML models to process the same words in a consistent way.\n", "\n", "This notebook shows how to use `MLTransform` to complete the following tasks:\n", "* Use `write` mode to generate a vocabulary on the input text and assign an index value to each token.\n", @@ -120,7 +120,7 @@ { "cell_type": "markdown", "source": [ - "## Artifact location\n", + "## Use the artifact location\n", "\n", "In `write` mode, the artifact location is used to store artifacts, such as the vocabulary file generated by `ComputeAndApplyVocabulary`.\n", "\n", @@ -163,7 +163,7 @@ { "cell_type": "markdown", "source": [ - "In this example, in `write` mode, `MLTransform` uses `ComputeAndApplyVocabulary` to generate vocabulary on the incoming dataset. The incoming text data is split into tokens and each token is assigned an unique index.\n", + "In this example, in `write` mode, `MLTransform` uses `ComputeAndApplyVocabulary` to generate vocabulary on the incoming dataset. The incoming text data is split into tokens. Each token is assigned an unique index.\n", "\n", " The generated vocabulary is stored in an artifact location that you can use on a different dataset in `read` mode." ], @@ -270,7 +270,7 @@ { "cell_type": "markdown", "source": [ - "## Frequency Threshold\n", + "## Set the frequency threshold\n", "\n", "The `frequency_threshold` parameter identifies the elements that appear frequently in the dataset. This parameter limits the generated vocabulary to elements with an absolute frequency greater than or equal to the specified threshold. If you don't specify the parameter, the entire vocabulary is generated.\n", "\n", @@ -317,7 +317,7 @@ { "cell_type": "markdown", "source": [ - "In the output, if the frequency of the token is less than the specified frequency, it is assigned to a `default_value` of `-1`. For the other tokens, a vocabulary file is generated." + "In the output, if the frequency of the token is less than the specified frequency, it's assigned to a `default_value` of `-1`. For the other tokens, a vocabulary file is generated." ], "metadata": { "id": "h1s4a6hzxKrb" @@ -357,7 +357,7 @@ { "cell_type": "markdown", "source": [ - "## `MLTransform` for inference workloads\n", + "## Use MLTransform for inference workloads\n", "\n", "When `MLTransform` is in `write` mode, it produces artifacts, such as vocabulary files for `ComputeAndApplyVocabulary`. These artifacts allow you to apply the same vocabulary, and any other preprocessing transforms, when you train your model and serve it in production, or when you test its accuracy.\n", "\n", diff --git a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb index abeeba2264ee..3c8946362a31 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb @@ -78,11 +78,11 @@ "\n", "For each data processing transform, `MLTransform` runs in both `write` mode and `read` mode. For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation.\n", "\n", - "### MLTransform in write mode\n", + "## MLTransform in write mode\n", "\n", "When `MLTransform` is in `write` mode, it produces artifacts, such as minimum, maximum, and variance, for different data processing transforms. These artifacts allow you to ensure that you're applying the same artifacts, and any other preprocessing transforms, when you train your model and serve it in production, or when you test its accuracy.\n", "\n", - "### MLTransform in read mode\n", + "## MLTransform in read mode\n", "\n", "In read mode, `MLTransform` uses the artifacts generated in `write` mode to scale the entire dataset." ], @@ -146,7 +146,7 @@ { "cell_type": "code", "source": [ - "# data used in MLTransform's write mode.\n", + "# data used in MLTransform's write mode\n", "data = [\n", " {'int_feature_1' : 11, 'int_feature_2': -10},\n", " {'int_feature_1': 34, 'int_feature_2': -33},\n", @@ -156,7 +156,7 @@ " {'int_feature_1': 63, 'int_feature_2': -21},\n", "]\n", "\n", - "# data used in MLTransform's read mode.\n", + "# data used in MLTransform's read mode\n", "test_data = [\n", " {'int_feature_1': 29, 'int_feature_2': -20},\n", " {'int_feature_1': -5, 'int_feature_2': -11},\n", diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb index db9e550dc913..ed4da57e297b 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -72,7 +72,7 @@ "* **Machine translation:** Translate text from one language to another and preserve the meaning.\n", "* **Text summarization:** Create shorter summaries of text.\n", "\n", - "This notebook uses the Vertex AI text-embeddings API to generate text embeddings that use Google’s large generative artificial intelligence (AI) models. To generate text embeddings by using the Vertex AI text-embeddings API, use `MLTransform` with the `VertexAITextEmbeddings` class to specify the model configuration. For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). \n", + "This notebook uses the Vertex AI text-embeddings API to generate text embeddings that use Google’s large generative artificial intelligence (AI) models. To generate text embeddings by using the Vertex AI text-embeddings API, use `MLTransform` with the `VertexAITextEmbeddings` class to specify the model configuration. For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings) in the Vertex AI documentation. \n", "\n", "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation.\n", "\n", @@ -156,9 +156,7 @@ "\n", "### Use MLTransform in write mode\n", "\n", - "In `write` mode, `MLTransform` saves the transforms and their attributes to an artifact location. Then, when you run `MLTransform` in `read` mode, these transforms are used. This process ensures that you're applying the same preprocessing steps when you train your model and when you serve the model in production or test its accuracy.\n", - "\n", - "For more information about using `MLTransform`, see [Preprocess data with MLTransform](https://beam.apache.org/documentation/ml/preprocess-data/) in the Apache Beam documentation." + "In `write` mode, `MLTransform` saves the transforms and their attributes to an artifact location. Then, when you run `MLTransform` in `read` mode, these transforms are used. This process ensures that you're applying the same preprocessing steps when you train your model and when you serve the model in production or test its accuracy." ], "metadata": { "id": "cokOaX2kzyke" From 992e4fd5019f38e4dcb68ebc7ae273df593120fd Mon Sep 17 00:00:00 2001 From: Cheskel Twersky Date: Sat, 6 Jan 2024 21:17:48 +0200 Subject: [PATCH 220/224] upgrade prettier --- sdks/typescript/package-lock.json | 25 ++-- sdks/typescript/package.json | 2 +- .../src/apache_beam/coders/coders.ts | 6 +- .../src/apache_beam/coders/required_coders.ts | 29 +++-- .../src/apache_beam/coders/row_coder.ts | 24 ++-- .../src/apache_beam/coders/standard_coders.ts | 2 +- .../src/apache_beam/examples/wordcount.ts | 2 +- .../src/apache_beam/examples/wordcount_sql.ts | 4 +- .../apache_beam/examples/wordcount_textio.ts | 2 +- .../src/apache_beam/internal/environments.ts | 12 +- .../src/apache_beam/internal/pipeline.ts | 53 ++++---- .../src/apache_beam/internal/serialize.ts | 10 +- sdks/typescript/src/apache_beam/io/avroio.ts | 12 +- .../src/apache_beam/io/bigqueryio.ts | 8 +- sdks/typescript/src/apache_beam/io/kafka.ts | 20 +-- .../src/apache_beam/io/parquetio.ts | 12 +- sdks/typescript/src/apache_beam/io/pubsub.ts | 28 ++-- .../src/apache_beam/io/pubsublite.ts | 12 +- .../typescript/src/apache_beam/io/schemaio.ts | 10 +- sdks/typescript/src/apache_beam/io/textio.ts | 32 ++--- sdks/typescript/src/apache_beam/pvalue.ts | 50 ++++---- .../src/apache_beam/runners/artifacts.ts | 18 +-- .../src/apache_beam/runners/dataflow.ts | 6 +- .../src/apache_beam/runners/direct_runner.ts | 48 +++---- .../src/apache_beam/runners/flink.ts | 8 +- .../runners/portable_runner/runner.ts | 34 ++--- .../src/apache_beam/runners/runner.ts | 16 +-- .../src/apache_beam/runners/universal.ts | 6 +- .../src/apache_beam/testing/assert.ts | 16 +-- .../testing/multi_pipeline_runner.ts | 22 ++-- .../src/apache_beam/transforms/create.ts | 2 +- .../src/apache_beam/transforms/external.ts | 62 ++++----- .../src/apache_beam/transforms/flatten.ts | 4 +- .../transforms/group_and_combine.ts | 50 ++++---- .../src/apache_beam/transforms/internal.ts | 38 +++--- .../src/apache_beam/transforms/pardo.ts | 45 ++++--- .../src/apache_beam/transforms/python.ts | 8 +- .../src/apache_beam/transforms/sql.ts | 16 +-- .../src/apache_beam/transforms/transform.ts | 18 +-- .../src/apache_beam/transforms/utils.ts | 4 +- .../src/apache_beam/transforms/window.ts | 22 ++-- .../src/apache_beam/transforms/windowings.ts | 10 +- .../src/apache_beam/utils/service.ts | 48 +++---- .../typescript/src/apache_beam/utils/utils.ts | 2 +- sdks/typescript/src/apache_beam/values.ts | 5 +- .../typescript/src/apache_beam/worker/data.ts | 6 +- .../worker/external_worker_service.ts | 10 +- .../src/apache_beam/worker/logging.ts | 6 +- .../src/apache_beam/worker/metrics.ts | 18 +-- .../src/apache_beam/worker/operators.ts | 121 +++++++++--------- .../src/apache_beam/worker/pardo_context.ts | 26 ++-- .../src/apache_beam/worker/state.ts | 18 +-- .../src/apache_beam/worker/worker.ts | 50 ++++---- .../src/apache_beam/worker/worker_main.ts | 4 +- 54 files changed, 570 insertions(+), 552 deletions(-) diff --git a/sdks/typescript/package-lock.json b/sdks/typescript/package-lock.json index 55d510ccce8c..1eb77b748005 100644 --- a/sdks/typescript/package-lock.json +++ b/sdks/typescript/package-lock.json @@ -1,12 +1,12 @@ { "name": "apache-beam", - "version": "2.52.0-SNAPSHOT", + "version": "2.54.0-SNAPSHOT", "lockfileVersion": 2, "requires": true, "packages": { "": { "name": "apache-beam", - "version": "2.52.0-SNAPSHOT", + "version": "2.54.0-SNAPSHOT", "dependencies": { "@google-cloud/pubsub": "^2.19.4", "@grpc/grpc-js": "~1.4.6", @@ -39,7 +39,7 @@ "istanbul": "^0.4.5", "js-yaml": "^4.1.0", "mocha": "^9.1.3", - "prettier": "^2.5.1", + "prettier": "^3.1.1", "typedoc": "^0.23.23", "typescript": "4.7" } @@ -3193,15 +3193,18 @@ } }, "node_modules/prettier": { - "version": "2.5.1", - "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.5.1.tgz", - "integrity": "sha512-vBZcPRUR5MZJwoyi3ZoyQlc1rXeEck8KgeC9AwwOn+exuxLxq5toTRDTSaVrXHxelDMHy9zlicw8u66yxoSUFg==", + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.1.1.tgz", + "integrity": "sha512-22UbSzg8luF4UuZtzgiUOfcGM8s4tjBv6dJRT7j275NXsy2jb4aJa4NNveul5x4eqlF1wuhuR2RElK71RvmVaw==", "dev": true, "bin": { - "prettier": "bin-prettier.js" + "prettier": "bin/prettier.cjs" }, "engines": { - "node": ">=10.13.0" + "node": ">=14" + }, + "funding": { + "url": "https://github.com/prettier/prettier?sponsor=1" } }, "node_modules/proto3-json-serializer": { @@ -6398,9 +6401,9 @@ "dev": true }, "prettier": { - "version": "2.5.1", - "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.5.1.tgz", - "integrity": "sha512-vBZcPRUR5MZJwoyi3ZoyQlc1rXeEck8KgeC9AwwOn+exuxLxq5toTRDTSaVrXHxelDMHy9zlicw8u66yxoSUFg==", + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.1.1.tgz", + "integrity": "sha512-22UbSzg8luF4UuZtzgiUOfcGM8s4tjBv6dJRT7j275NXsy2jb4aJa4NNveul5x4eqlF1wuhuR2RElK71RvmVaw==", "dev": true }, "proto3-json-serializer": { diff --git a/sdks/typescript/package.json b/sdks/typescript/package.json index 556ea1f6322a..8fc999bc9c9d 100644 --- a/sdks/typescript/package.json +++ b/sdks/typescript/package.json @@ -11,7 +11,7 @@ "istanbul": "^0.4.5", "js-yaml": "^4.1.0", "mocha": "^9.1.3", - "prettier": "^2.5.1", + "prettier": "^3.1.1", "typedoc": "^0.23.23", "typescript": "4.7" }, diff --git a/sdks/typescript/src/apache_beam/coders/coders.ts b/sdks/typescript/src/apache_beam/coders/coders.ts index 780987138fbb..1cf19b01ff2e 100644 --- a/sdks/typescript/src/apache_beam/coders/coders.ts +++ b/sdks/typescript/src/apache_beam/coders/coders.ts @@ -70,7 +70,7 @@ class CoderRegistry { registerConstructor( urn: string, - constructor: (...args: unknown[]) => Coder + constructor: (...args: unknown[]) => Coder, ) { this.internal_registry[urn] = constructor; } @@ -142,7 +142,7 @@ export interface Coder { function writeByteCallback( val: number, buf: { [x: string]: number }, - pos: number + pos: number, ) { buf[pos] = val & 0xff; } @@ -162,7 +162,7 @@ export function writeRawByte(b: unknown, writer: HackedWriter) { function writeBytesCallback( val: number[], buf: { [x: string]: number }, - pos: number + pos: number, ) { for (let i = 0; i < val.length; ++i) { buf[pos + i] = val[i]; diff --git a/sdks/typescript/src/apache_beam/coders/required_coders.ts b/sdks/typescript/src/apache_beam/coders/required_coders.ts index fe852b349251..74b7154ba114 100644 --- a/sdks/typescript/src/apache_beam/coders/required_coders.ts +++ b/sdks/typescript/src/apache_beam/coders/required_coders.ts @@ -321,7 +321,7 @@ export class IterableCoder implements Coder> { } for (var i = 0; i < count; i++) { result.push( - this.elementCoder.decode(reader, Context.needsDelimiters) + this.elementCoder.decode(reader, Context.needsDelimiters), ); } } @@ -369,7 +369,7 @@ export class LengthPrefixedCoder implements Coder { decode(reader: Reader, context: Context): T { return this.elementCoder.decode( new Reader(reader.bytes()), - Context.wholeStream + Context.wholeStream, ); } } @@ -384,7 +384,10 @@ export class FullWindowedValueCoder static URN: string = "beam:coder:windowed_value:v1"; windowIterableCoder: IterableCoder; // really W - constructor(public elementCoder: Coder, public windowCoder: Coder) { + constructor( + public elementCoder: Coder, + public windowCoder: Coder, + ) { this.windowIterableCoder = new IterableCoder(windowCoder); } @@ -405,17 +408,17 @@ export class FullWindowedValueCoder InstantCoder.INSTANCE.encode( windowedValue.timestamp, writer, - Context.needsDelimiters + Context.needsDelimiters, ); this.windowIterableCoder.encode( >windowedValue.windows, writer, - Context.needsDelimiters + Context.needsDelimiters, ); // Windows. PaneInfoCoder.INSTANCE.encode( windowedValue.pane, writer, - Context.needsDelimiters + Context.needsDelimiters, ); this.elementCoder.encode(windowedValue.value, writer, context); } @@ -423,11 +426,11 @@ export class FullWindowedValueCoder decode(reader: Reader, context: Context): WindowedValue { const timestamp = InstantCoder.INSTANCE.decode( reader, - Context.needsDelimiters + Context.needsDelimiters, ); const windows = this.windowIterableCoder.decode( reader, - Context.needsDelimiters + Context.needsDelimiters, ); const pane = PaneInfoCoder.INSTANCE.decode(reader, Context.needsDelimiters); const value = this.elementCoder.decode(reader, context); @@ -472,8 +475,8 @@ export class InstantCoder implements Coder { decode(reader: Reader, context: Context): Instant { const shiftedMillis = Long.fromBytesBE( Array.from( - reader.buf.slice(reader.pos, reader.pos + InstantCoder.INSTANT_BYTES) - ) + reader.buf.slice(reader.pos, reader.pos + InstantCoder.INSTANT_BYTES), + ), ); reader.pos += InstantCoder.INSTANT_BYTES; return shiftedMillis.add(Long.MIN_VALUE); @@ -503,7 +506,7 @@ export class PaneInfoCoder implements Coder { static INSTANCE = new PaneInfoCoder(); static ONE_AND_ONLY_FIRING = PaneInfoCoder.INSTANCE.decode( new Reader(new Uint8Array([0x09])), - null! + null!, ); private static decodeTiming(timingNumber): Timing { @@ -520,7 +523,7 @@ export class PaneInfoCoder implements Coder { throw new Error( "Timing number 0b" + timingNumber.toString(2) + - " has more than two bits of info" + " has more than two bits of info", ); } } @@ -635,7 +638,7 @@ export class PaneInfoCoder implements Coder { toProto(pipelineContext: ProtoContext): runnerApi.Coder { throw new Error( - "No proto encoding for PaneInfoCoder, always part of WindowedValue codec" + "No proto encoding for PaneInfoCoder, always part of WindowedValue codec", ); } } diff --git a/sdks/typescript/src/apache_beam/coders/row_coder.ts b/sdks/typescript/src/apache_beam/coders/row_coder.ts index f93abe1f5ece..6c2b1f1a75d9 100644 --- a/sdks/typescript/src/apache_beam/coders/row_coder.ts +++ b/sdks/typescript/src/apache_beam/coders/row_coder.ts @@ -129,8 +129,8 @@ export class RowCoder implements Coder { default: throw new Error( `Encountered a type that is not currently supported by RowCoder: ${JSON.stringify( - f.type - )}` + f.type, + )}`, ); } return obj; @@ -265,14 +265,14 @@ export class RowCoder implements Coder { return new BoolCoder(); default: throw new Error( - `Encountered an Atomic type that is not currently supported by RowCoder: ${atomicType}` + `Encountered an Atomic type that is not currently supported by RowCoder: ${atomicType}`, ); } break; case "arrayType": if (typeInfo.arrayType.elementType !== undefined) { return new IterableCoder( - this.getCoderFromType(typeInfo.arrayType.elementType) + this.getCoderFromType(typeInfo.arrayType.elementType), ); } else { throw new Error("ElementType missing on ArrayType"); @@ -290,14 +290,14 @@ export class RowCoder implements Coder { const logicalTypeInfo = logicalTypes.get(typeInfo.logicalType.urn); if (logicalTypeInfo !== undefined) { const reprCoder = this.getCoderFromType( - typeInfo.logicalType.representation! + typeInfo.logicalType.representation!, ); return { encode: (element: any, writer: Writer, context: Context) => reprCoder.encode( logicalTypeInfo.toRepr(element), writer, - context + context, ), decode: (reader: Reader, context: Context) => logicalTypeInfo.fromRepr(reprCoder.decode(reader, context)), @@ -312,8 +312,8 @@ export class RowCoder implements Coder { default: throw new Error( `Encountered a type that is not currently supported by RowCoder: ${JSON.stringify( - t - )}` + t, + )}`, ); } } @@ -342,7 +342,7 @@ export class RowCoder implements Coder { let encPosx = schema.fields.map((f: Field) => f.encodingPosition); if (encPosx.length !== this.encodingPositions.length) { throw new Error( - `Schema with id ${this.schema.id} has encoding_positions_set=True, but not all fields have encoding_position set` + `Schema with id ${this.schema.id} has encoding_positions_set=True, but not all fields have encoding_position set`, ); } // Checking if positions are in {0, ..., length-1} @@ -352,7 +352,7 @@ export class RowCoder implements Coder { } this.hasNullableFields = this.schema.fields.some( - (f: Field) => f.type?.nullable + (f: Field) => f.type?.nullable, ); this.components = this.encodingPositions .map((i) => this.schema.fields[i]) @@ -409,7 +409,7 @@ export class RowCoder implements Coder { if (attr === null || attr === undefined) { if (!this.fieldNullable[i]) { throw new Error( - `Attempted to encode null for non-nullable field \"${this.schema.fields[i].name}\".` + `Attempted to encode null for non-nullable field \"${this.schema.fields[i].name}\".`, ); } } else { @@ -472,7 +472,7 @@ export class RowCoder implements Coder { obj = this.addFieldOfType( obj, this.schema.fields[i], - sortedComponents[i] + sortedComponents[i], ); }); diff --git a/sdks/typescript/src/apache_beam/coders/standard_coders.ts b/sdks/typescript/src/apache_beam/coders/standard_coders.ts index a9714247f8e5..8f78bc67b828 100644 --- a/sdks/typescript/src/apache_beam/coders/standard_coders.ts +++ b/sdks/typescript/src/apache_beam/coders/standard_coders.ts @@ -225,5 +225,5 @@ globalRegistry().register(IntervalWindowCoder.URN, IntervalWindowCoder); import { requireForSerialization } from "../serialization"; requireForSerialization( "apache-beam/coders/standard_coders", - exports as Record + exports as Record, ); diff --git a/sdks/typescript/src/apache_beam/examples/wordcount.ts b/sdks/typescript/src/apache_beam/examples/wordcount.ts index e34b8cab3c5c..3ba11878b22d 100644 --- a/sdks/typescript/src/apache_beam/examples/wordcount.ts +++ b/sdks/typescript/src/apache_beam/examples/wordcount.ts @@ -53,7 +53,7 @@ async function main() { "And the earth was without form, and void; and darkness was upon the face of the deep.", "And the Spirit of God moved upon the face of the waters.", "And God said, Let there be light: and there was light.", - ]) + ]), ); lines.apply(wordCount).map(console.log); diff --git a/sdks/typescript/src/apache_beam/examples/wordcount_sql.ts b/sdks/typescript/src/apache_beam/examples/wordcount_sql.ts index 2ea4204886c2..5c5cc575d74d 100644 --- a/sdks/typescript/src/apache_beam/examples/wordcount_sql.ts +++ b/sdks/typescript/src/apache_beam/examples/wordcount_sql.ts @@ -39,8 +39,8 @@ async function main() { .apply(beam.withRowCoder({ word: "str" })) .applyAsync( sqlTransform( - "SELECT word, count(*) as c from PCOLLECTION group by word" - ) + "SELECT word, count(*) as c from PCOLLECTION group by word", + ), ); filtered.map(console.log); diff --git a/sdks/typescript/src/apache_beam/examples/wordcount_textio.ts b/sdks/typescript/src/apache_beam/examples/wordcount_textio.ts index f618effea1c3..0f8144b95723 100644 --- a/sdks/typescript/src/apache_beam/examples/wordcount_textio.ts +++ b/sdks/typescript/src/apache_beam/examples/wordcount_textio.ts @@ -38,7 +38,7 @@ async function main() { // python apache_beam/runners/portability/local_job_service_main.py --port 3333 await new PortableRunner("localhost:3333").run(async (root) => { const lines = await root.applyAsync( - textio.readFromText("gs://dataflow-samples/shakespeare/kinglear.txt") + textio.readFromText("gs://dataflow-samples/shakespeare/kinglear.txt"), ); lines.apply(wordCount).map(console.log); diff --git a/sdks/typescript/src/apache_beam/internal/environments.ts b/sdks/typescript/src/apache_beam/internal/environments.ts index 4e45800839ea..fa70e80a3e98 100644 --- a/sdks/typescript/src/apache_beam/internal/environments.ts +++ b/sdks/typescript/src/apache_beam/internal/environments.ts @@ -40,7 +40,7 @@ export function jsEnvironment( urn: string, payload: Uint8Array, resourceHints: { [key: string]: Uint8Array } = {}, - artifacts: runnerApi.ArtifactInformation[] = [] + artifacts: runnerApi.ArtifactInformation[] = [], ): runnerApi.Environment { return { urn: urn, @@ -55,7 +55,7 @@ export function jsEnvironment( function asNewEnvironment( env: runnerApi.Environment, urn: string, - payload: Uint8Array + payload: Uint8Array, ) { return { urn: urn, @@ -69,7 +69,7 @@ function asNewEnvironment( export function asExternalEnvironment( env: runnerApi.Environment, - address: string + address: string, ) { return asNewEnvironment( env, @@ -80,17 +80,17 @@ export function asExternalEnvironment( authentication: null!, }, params: {}, - }) + }), ); } export function asDockerEnvironment( env: runnerApi.Environment, - containerImage: string + containerImage: string, ) { return asNewEnvironment( env, "beam:env:docker:v1", - runnerApi.DockerPayload.toBinary({ containerImage: containerImage }) + runnerApi.DockerPayload.toBinary({ containerImage: containerImage }), ); } diff --git a/sdks/typescript/src/apache_beam/internal/pipeline.ts b/sdks/typescript/src/apache_beam/internal/pipeline.ts index fc33fdb25072..6d5878fdaecb 100644 --- a/sdks/typescript/src/apache_beam/internal/pipeline.ts +++ b/sdks/typescript/src/apache_beam/internal/pipeline.ts @@ -39,7 +39,10 @@ export class PipelineContext { private coders: { [key: string]: Coder } = {}; - constructor(public components: Components, private componentPrefix: string) {} + constructor( + public components: Components, + private componentPrefix: string, + ) {} getCoder(coderId: string): Coder { const this_ = this; @@ -51,7 +54,7 @@ export class PipelineContext { this.coders[coderId] = globalCoderRegistry().getCoder( coderProto.spec!.urn, coderProto.spec!.payload, - ...components + ...components, ); } return this.coders[coderId]; @@ -61,7 +64,7 @@ export class PipelineContext { return this.getOrAssign( this.components.coders, coder.toProto(this), - "coder" + "coder", ); } @@ -82,14 +85,14 @@ export class PipelineContext { return this.getOrAssign( this.components.windowingStrategies, windowing, - "windowing" + "windowing", ); } private getOrAssign( existing: { [key: string]: T }, obj: T, - prefix: string + prefix: string, ) { for (const [id, other] of Object.entries(existing)) { if (equal(other, obj)) { @@ -134,7 +137,7 @@ export class Pipeline { preApplyTransform< InputT extends pvalue.PValue, - OutputT extends pvalue.PValue + OutputT extends pvalue.PValue, >(transform: AsyncPTransformClass, input: InputT) { const this_ = this; const transformId = this.context.createUniqueName("transform"); @@ -153,7 +156,7 @@ export class Pipeline { if (this.usedStageNames.has(uniqueName)) { throw new Error( `Duplicate stage name: "${uniqueName}". ` + - "Use beam.withName(...) to give your transform a unique name." + "Use beam.withName(...) to give your transform a unique name.", ); } this.usedStageNames.add(uniqueName); @@ -172,11 +175,11 @@ export class Pipeline { applyTransform< InputT extends pvalue.PValue, - OutputT extends pvalue.PValue + OutputT extends pvalue.PValue, >(transform: PTransformClass, input: InputT) { const { id: transformId, proto: transformProto } = this.preApplyTransform( transform, - input + input, ); let result: OutputT; try { @@ -190,11 +193,11 @@ export class Pipeline { async applyAsyncTransform< InputT extends pvalue.PValue, - OutputT extends pvalue.PValue + OutputT extends pvalue.PValue, >(transform: AsyncPTransformClass, input: InputT) { const { id: transformId, proto: transformProto } = this.preApplyTransform( transform, - input + input, ); let result: OutputT; try { @@ -208,26 +211,26 @@ export class Pipeline { postApplyTransform< InputT extends pvalue.PValue, - OutputT extends pvalue.PValue + OutputT extends pvalue.PValue, >( transform: AsyncPTransformClass, transformProto: runnerApi.PTransform, - result: OutputT + result: OutputT, ) { transformProto.outputs = objectMap(pvalue.flattenPValue(result), (pc) => - pc.getId() + pc.getId(), ); // Propagate any unset pvalue.PCollection properties. const this_ = this; const inputProtos = Object.values(transformProto.inputs).map( - (id) => this_.proto.components!.pcollections[id] + (id) => this_.proto.components!.pcollections[id], ); const inputBoundedness = new Set( - inputProtos.map((proto) => proto.isBounded) + inputProtos.map((proto) => proto.isBounded), ); const inputWindowings = new Set( - inputProtos.map((proto) => proto.windowingStrategyId) + inputProtos.map((proto) => proto.windowingStrategyId), ); for (const pcId of Object.values(transformProto.outputs)) { @@ -235,7 +238,7 @@ export class Pipeline { if (!pcProto.isBounded) { pcProto.isBounded = onlyValueOr( inputBoundedness, - runnerApi.IsBounded_Enum.BOUNDED + runnerApi.IsBounded_Enum.BOUNDED, ); } // TODO: (Cleanup) Handle the case of equivalent strategies. @@ -246,9 +249,9 @@ export class Pipeline { (a, b) => { return equal( this_.proto.components!.windowingStrategies[a], - this_.proto.components!.windowingStrategies[b] + this_.proto.components!.windowingStrategies[b], ); - } + }, ); } } @@ -262,11 +265,11 @@ export class Pipeline { | runnerApi.WindowingStrategy | string | undefined = undefined, - isBounded: runnerApi.IsBounded_Enum | undefined = undefined + isBounded: runnerApi.IsBounded_Enum | undefined = undefined, ): pvalue.PCollection { return new pvalue.PCollection( this, - this.createPCollectionIdInternal(coder, windowingStrategy, isBounded) + this.createPCollectionIdInternal(coder, windowingStrategy, isBounded), ); } @@ -276,7 +279,7 @@ export class Pipeline { | runnerApi.WindowingStrategy | string | undefined = undefined, - isBounded: runnerApi.IsBounded_Enum | undefined = undefined + isBounded: runnerApi.IsBounded_Enum | undefined = undefined, ): string { const pcollId = this.context.createUniqueName("pc"); let coderId: string; @@ -292,7 +295,7 @@ export class Pipeline { windowingStrategyId = windowingStrategy; } else { windowingStrategyId = this.context.getWindowingStrategyId( - windowingStrategy! + windowingStrategy!, ); } this.proto!.components!.pcollections[pcollId] = { @@ -325,7 +328,7 @@ function objectMap(obj, func) { function onlyValueOr( valueSet: Set, defaultValue: T, - comparator: (a: T, b: T) => boolean = (a, b) => false + comparator: (a: T, b: T) => boolean = (a, b) => false, ) { if (valueSet.size === 0) { return defaultValue; diff --git a/sdks/typescript/src/apache_beam/internal/serialize.ts b/sdks/typescript/src/apache_beam/internal/serialize.ts index 3a4bf81140e0..ccc3532d29b9 100644 --- a/sdks/typescript/src/apache_beam/internal/serialize.ts +++ b/sdks/typescript/src/apache_beam/internal/serialize.ts @@ -38,11 +38,11 @@ export function serializeFn(obj: unknown): Uint8Array { JSON.stringify( serialize_closures.serialize( obj, - serialize_closures.defaultBuiltins.concat(registeredObjects) + serialize_closures.defaultBuiltins.concat(registeredObjects), ), (key, value) => - typeof value === "bigint" ? `${BIGINT_PREFIX}${value}` : value - ) + typeof value === "bigint" ? `${BIGINT_PREFIX}${value}` : value, + ), ); } @@ -51,8 +51,8 @@ export function deserializeFn(s: Uint8Array): any { JSON.parse(new TextDecoder().decode(s), (key, value) => typeof value === "string" && value.startsWith(BIGINT_PREFIX) ? BigInt(value.substr(BIGINT_PREFIX.length)) - : value + : value, ), - serialize_closures.defaultBuiltins.concat(registeredObjects) + serialize_closures.defaultBuiltins.concat(registeredObjects), ); } diff --git a/sdks/typescript/src/apache_beam/io/avroio.ts b/sdks/typescript/src/apache_beam/io/avroio.ts index b6ebf45cf285..a638fde18184 100644 --- a/sdks/typescript/src/apache_beam/io/avroio.ts +++ b/sdks/typescript/src/apache_beam/io/avroio.ts @@ -25,31 +25,31 @@ import { withCoderInternal } from "../transforms/internal"; export function readFromAvro( filePattern: string, // TODO: Allow schema to be inferred. - options: { schema: Schema } + options: { schema: Schema }, ): beam.AsyncPTransform> { return schemaio>( "readFromTable", "beam:transform:org.apache.beam:schemaio_avro_read:v1", - { location: filePattern, schema: options.schema } + { location: filePattern, schema: options.schema }, ); } export function writeToAvro(filePath: string, options: { schema: Schema }) { return async function writeToAvro( - pcoll: beam.PCollection + pcoll: beam.PCollection, ): Promise<{}> { // TODO: Allow schema to be inferred. if (options.schema) { pcoll = pcoll.apply( - withCoderInternal(RowCoder.fromSchema(options.schema)) + withCoderInternal(RowCoder.fromSchema(options.schema)), ); } return pcoll.applyAsync( schemaio, {}>( "writeToAvro", "beam:transform:org.apache.beam:schemaio_avro_write:v1", - { location: filePath, schema: options.schema } - ) + { location: filePath, schema: options.schema }, + ), ); }; } diff --git a/sdks/typescript/src/apache_beam/io/bigqueryio.ts b/sdks/typescript/src/apache_beam/io/bigqueryio.ts index ff4800cb0bc9..886728801aef 100644 --- a/sdks/typescript/src/apache_beam/io/bigqueryio.ts +++ b/sdks/typescript/src/apache_beam/io/bigqueryio.ts @@ -32,19 +32,19 @@ const bigqueryIOConfigSchema = RowCoder.inferSchemaOfJSON({ export function readFromBigQuery( options: | { table: string; schema?: Schema } - | { query: string; schema?: Schema } + | { query: string; schema?: Schema }, ): beam.AsyncPTransform> { return schemaio>( "readFromBigQuery", "beam:transform:org.apache.beam:schemaio_bigquery_read:v1", options, - bigqueryIOConfigSchema + bigqueryIOConfigSchema, ); } export function writeToBigQuery( table: string, - options: { createDisposition?: "Never" | "IfNeeded" } = {} + options: { createDisposition?: "Never" | "IfNeeded" } = {}, ): beam.AsyncPTransform> { if (options.createDisposition == undefined) { options.createDisposition = "IfNeeded"; @@ -53,6 +53,6 @@ export function writeToBigQuery( "writeToBigquery", "beam:transform:org.apache.beam:schemaio_bigquery_write:v1", { table, createDisposition: options.createDisposition }, - bigqueryIOConfigSchema + bigqueryIOConfigSchema, ); } diff --git a/sdks/typescript/src/apache_beam/io/kafka.ts b/sdks/typescript/src/apache_beam/io/kafka.ts index 8fd717ec214d..99600d75e6c6 100644 --- a/sdks/typescript/src/apache_beam/io/kafka.ts +++ b/sdks/typescript/src/apache_beam/io/kafka.ts @@ -48,28 +48,28 @@ const defaultReadFromKafkaOptions = { export function readFromKafka( consumerConfig: { [key: string]: string }, // TODO: Or a map? topics: string[], - options: ReadFromKafkaOptions = {} + options: ReadFromKafkaOptions = {}, ): beam.AsyncPTransform> { return readFromKafkaMaybeWithMetadata( "readFromKafkaWithMetadata", "beam:transform:org.apache.beam:kafka_read_without_metadata:v1", consumerConfig, topics, - options + options, ); } export function readFromKafkaWithMetadata( consumerConfig: { [key: string]: string }, // TODO: Or a map? topics: string[], - options: ReadFromKafkaOptions = {} + options: ReadFromKafkaOptions = {}, ): beam.AsyncPTransform> { return readFromKafkaMaybeWithMetadata( "readFromKafkaWithMetadata", "beam:transform:org.apache.beam:kafka_read_with_metadata:v1", consumerConfig, topics, - options + options, ); } @@ -78,7 +78,7 @@ function readFromKafkaMaybeWithMetadata( urn: string, consumerConfig: { [key: string]: string }, // TODO: Or a map? topics: string[], - options: ReadFromKafkaOptions = {} + options: ReadFromKafkaOptions = {}, ): beam.AsyncPTransform> { return beam.withName( name, @@ -89,8 +89,8 @@ function readFromKafkaMaybeWithMetadata( consumerConfig, ...camelToSnakeOptions({ ...defaultReadFromKafkaOptions, ...options }), }, - serviceProviderFromJavaGradleTarget(KAFKA_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(KAFKA_EXPANSION_GRADLE_TARGET), + ), ); } @@ -107,7 +107,7 @@ const defaultWriteToKafkaOptions = { export function writeToKafka( producerConfig: { [key: string]: string }, // TODO: Or a map? topics: string[], - options: WriteToKafkaOptions = {} + options: WriteToKafkaOptions = {}, ): beam.AsyncPTransform, {}> { return beam.withName( "writeToKafka", @@ -118,7 +118,7 @@ export function writeToKafka( producerConfig, ...camelToSnakeOptions({ ...defaultWriteToKafkaOptions, ...options }), }, - serviceProviderFromJavaGradleTarget(KAFKA_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(KAFKA_EXPANSION_GRADLE_TARGET), + ), ); } diff --git a/sdks/typescript/src/apache_beam/io/parquetio.ts b/sdks/typescript/src/apache_beam/io/parquetio.ts index e7c7f7dc66e0..ade4d289ecdd 100644 --- a/sdks/typescript/src/apache_beam/io/parquetio.ts +++ b/sdks/typescript/src/apache_beam/io/parquetio.ts @@ -30,7 +30,7 @@ export function readFromParquet( filePattern: string, options: { columns?: string[]; - } = {} + } = {}, ): (root: beam.Root) => Promise> { return async function readFromParquet(root: beam.Root) { return root.applyAsync( @@ -38,21 +38,21 @@ export function readFromParquet( path: filePattern, format: "parquet", ...camelToSnakeOptions(options), - }) + }), ); }; } export function writeToParquet( filePathPrefix: string, - options: { schema?: Schema } = {} + options: { schema?: Schema } = {}, ): ( - toWrite: beam.PCollection + toWrite: beam.PCollection, ) => Promise<{ filesWritten: beam.PCollection }> { return async function writeToJson(toWrite: beam.PCollection) { if (options.schema) { toWrite = toWrite.apply( - withCoderInternal(RowCoder.fromSchema(options.schema)) + withCoderInternal(RowCoder.fromSchema(options.schema)), ); delete options.schema; } @@ -62,7 +62,7 @@ export function writeToParquet( path: filePathPrefix, format: "parquet", ...camelToSnakeOptions(options), - }) + }), ), }; }; diff --git a/sdks/typescript/src/apache_beam/io/pubsub.ts b/sdks/typescript/src/apache_beam/io/pubsub.ts index c5513fe41718..c7ae7ee5462d 100644 --- a/sdks/typescript/src/apache_beam/io/pubsub.ts +++ b/sdks/typescript/src/apache_beam/io/pubsub.ts @@ -53,11 +53,11 @@ type ReadOptions = // TODO: Schema-producing variants. export function readFromPubSub( - options: ReadOptions + options: ReadOptions, ): AsyncPTransform> { if (options.topic && options.subscription) { throw new TypeError( - "Exactly one of topic or subscription must be provided." + "Exactly one of topic or subscription must be provided.", ); } return withName( @@ -65,17 +65,17 @@ export function readFromPubSub( external.rawExternalTransform>( "beam:transform:org.apache.beam:pubsub_read:v1", camelToSnakeOptions(options), - serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET), + ), ); } function readFromPubSubWithAttributesRaw( - options: ReadOptions + options: ReadOptions, ): AsyncPTransform> { if (options.topic && options.subscription) { throw new TypeError( - "Exactly one of topic or subscription must be provided." + "Exactly one of topic or subscription must be provided.", ); } return withName( @@ -83,13 +83,13 @@ function readFromPubSubWithAttributesRaw( external.rawExternalTransform>( "beam:transform:org.apache.beam:pubsub_read:v1", { needsAttributes: true, ...camelToSnakeOptions(options) }, - serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET), + ), ); } export function readFromPubSubWithAttributes( - options: ReadOptions + options: ReadOptions, ): AsyncPTransform< beam.Root, beam.PCollection @@ -98,7 +98,7 @@ export function readFromPubSubWithAttributes( return ( await root.applyAsync(readFromPubSubWithAttributesRaw(options)) ).map((encoded) => - PubSub.protos.google.pubsub.v1.PubsubMessage.decode(encoded) + PubSub.protos.google.pubsub.v1.PubsubMessage.decode(encoded), ); }; } @@ -107,15 +107,15 @@ type WriteOptions = { idAttribute?: string; timestampAttribute?: string }; function writeToPubSubRaw( topic: string, - options: WriteOptions = {} + options: WriteOptions = {}, ): AsyncPTransform, {}> { return withName( "writeToPubSubRaw", external.rawExternalTransform, {}>( "beam:transform:org.apache.beam:pubsub_write:v1", { topic, ...camelToSnakeOptions(options) }, - serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET), + ), ); } @@ -123,7 +123,7 @@ export function writeToPubSub(topic: string, options: WriteOptions = {}) { return async function writeToPubSub(dataPColl: beam.PCollection) { return dataPColl // .map((data) => - PubSub.protos.google.pubsub.v1.PubsubMessage.encode({ data }).finish() + PubSub.protos.google.pubsub.v1.PubsubMessage.encode({ data }).finish(), ) .apply(internal.withCoderInternal(new BytesCoder())) .applyAsync(writeToPubSubRaw(topic, options)); diff --git a/sdks/typescript/src/apache_beam/io/pubsublite.ts b/sdks/typescript/src/apache_beam/io/pubsublite.ts index 2798d44079bf..510b39b9bf9a 100644 --- a/sdks/typescript/src/apache_beam/io/pubsublite.ts +++ b/sdks/typescript/src/apache_beam/io/pubsublite.ts @@ -28,28 +28,28 @@ const PUBSUBLITE_EXPANSION_GRADLE_TARGET = // TODO: Schema-producing variants. export function readFromPubSubLiteRaw( subscriptionPath: string, - options: { minBundleTimeout?: number; deduplicate?: boolean } = {} + options: { minBundleTimeout?: number; deduplicate?: boolean } = {}, ): beam.AsyncPTransform> { return beam.withName( "readFromPubSubLiteRaw", external.rawExternalTransform>( "beam:transform:org.apache.beam:pubsublite_read:v1", { subscription_path: subscriptionPath, ...camelToSnakeOptions(options) }, - serviceProviderFromJavaGradleTarget(PUBSUBLITE_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(PUBSUBLITE_EXPANSION_GRADLE_TARGET), + ), ); } export function writeToPubSubLiteRaw( topicPath: string, - options: { addUuids?: boolean } = {} + options: { addUuids?: boolean } = {}, ): beam.AsyncPTransform, {}> { return beam.withName( "writeToPubSubLiteRaw", external.rawExternalTransform, {}>( "beam:transform:org.apache.beam:pubsublite_write:v1", { topic_path: topicPath, ...camelToSnakeOptions(options) }, - serviceProviderFromJavaGradleTarget(PUBSUBLITE_EXPANSION_GRADLE_TARGET) - ) + serviceProviderFromJavaGradleTarget(PUBSUBLITE_EXPANSION_GRADLE_TARGET), + ), ); } diff --git a/sdks/typescript/src/apache_beam/io/schemaio.ts b/sdks/typescript/src/apache_beam/io/schemaio.ts index c9f666b3a772..20cb7f8abca4 100644 --- a/sdks/typescript/src/apache_beam/io/schemaio.ts +++ b/sdks/typescript/src/apache_beam/io/schemaio.ts @@ -25,12 +25,12 @@ import * as protobufjs from "protobufjs"; export function schemaio< InputT extends beam.PValue, - OutputT extends beam.PValue + OutputT extends beam.PValue, >( name, urn, config, - configSchema: Schema | undefined = undefined + configSchema: Schema | undefined = undefined, ): beam.AsyncPTransform { // Location is separate for historical reasons. let maybeLocation: { location?: string } = {}; @@ -58,8 +58,8 @@ export function schemaio< urn, { config: encodedConfig, ...maybeLocation, ...maybeSchema }, serviceProviderFromJavaGradleTarget( - "sdks:java:io:google-cloud-platform:expansion-service:shadowJar" - ) - ) + "sdks:java:io:google-cloud-platform:expansion-service:shadowJar", + ), + ), ); } diff --git a/sdks/typescript/src/apache_beam/io/textio.ts b/sdks/typescript/src/apache_beam/io/textio.ts index 7dedbcf26989..fb2cb41ec679 100644 --- a/sdks/typescript/src/apache_beam/io/textio.ts +++ b/sdks/typescript/src/apache_beam/io/textio.ts @@ -27,7 +27,7 @@ import { Schema } from "../proto/schema"; import { RowCoder } from "../coders/row_coder"; export function readFromText( - filePattern: string + filePattern: string, ): beam.AsyncPTransform> { return async function readFromText(root: beam.Root) { return root.applyAsync( @@ -35,8 +35,8 @@ export function readFromText( "apache_beam.io.ReadFromText", { file_pattern: filePattern, - } - ) + }, + ), ); }; } @@ -50,9 +50,9 @@ export function writeToText( shardNameTemplate?: string; header?: string; footer?: string; - } = {} + } = {}, ): ( - toWrite: beam.PCollection + toWrite: beam.PCollection, ) => Promise<{ filesWritten: beam.PCollection }> { return async function writeToText(pcoll: beam.PCollection) { return { @@ -63,7 +63,7 @@ export function writeToText( pythonTransform("apache_beam.io.WriteToText", { file_path_prefix: filePathPrefix, ...camelToSnakeOptions(options), - }) + }), ), }; }; @@ -71,7 +71,7 @@ export function writeToText( export function readFromCsv( filePattern: string, - options: {} = {} + options: {} = {}, ): (root: beam.Root) => Promise> { return async function readFromCsv(root: beam.Root) { return root.applyAsync( @@ -79,7 +79,7 @@ export function readFromCsv( path: filePattern, format: "csv", ...camelToSnakeOptions(options), - }) + }), ); }; } @@ -87,9 +87,9 @@ export function readFromCsv( export function writeToCsv( filePathPrefix: string, schema: Schema | undefined = undefined, - options: {} = {} + options: {} = {}, ): ( - toWrite: beam.PCollection + toWrite: beam.PCollection, ) => Promise<{ filesWritten: beam.PCollection }> { return async function writeToCsv(toWrite: beam.PCollection) { if (schema != undefined) { @@ -102,7 +102,7 @@ export function writeToCsv( format: "csv", index: false, ...camelToSnakeOptions(options), - }) + }), ), }; }; @@ -110,7 +110,7 @@ export function writeToCsv( export function readFromJson( filePattern: string, - options: {} = {} + options: {} = {}, ): (root: beam.Root) => Promise> { return async function readFromJson(root: beam.Root) { return root.applyAsync( @@ -120,7 +120,7 @@ export function readFromJson( orient: "records", lines: true, ...camelToSnakeOptions(options), - }) + }), ); }; } @@ -128,9 +128,9 @@ export function readFromJson( export function writeToJson( filePathPrefix: string, schema: Schema | undefined = undefined, - options: {} = {} + options: {} = {}, ): ( - toWrite: beam.PCollection + toWrite: beam.PCollection, ) => Promise<{ filesWritten: beam.PCollection }> { return async function writeToJson(toWrite: beam.PCollection) { if (schema != undefined) { @@ -144,7 +144,7 @@ export function writeToJson( orient: "records", lines: true, ...camelToSnakeOptions(options), - }) + }), ), }; }; diff --git a/sdks/typescript/src/apache_beam/pvalue.ts b/sdks/typescript/src/apache_beam/pvalue.ts index e3b7972f6654..6a6c34cdb3b6 100644 --- a/sdks/typescript/src/apache_beam/pvalue.ts +++ b/sdks/typescript/src/apache_beam/pvalue.ts @@ -50,7 +50,7 @@ export class Root { } async applyAsync>( - transform: AsyncPTransform + transform: AsyncPTransform, ) { if (!(transform instanceof AsyncPTransformClass)) { transform = new AsyncPTransformClassFromCallable(transform); @@ -88,7 +88,7 @@ export class PCollection { } apply>( - transform: PTransform, OutputT> + transform: PTransform, OutputT>, ) { if (!(transform instanceof PTransformClass)) { transform = new PTransformClassFromCallable(transform); @@ -97,7 +97,7 @@ export class PCollection { } applyAsync>( - transform: AsyncPTransform, OutputT> + transform: AsyncPTransform, OutputT>, ) { if (!(transform instanceof AsyncPTransformClass)) { transform = new AsyncPTransformClassFromCallable(transform); @@ -109,7 +109,7 @@ export class PCollection { fn: | (ContextT extends undefined ? (element: T) => OutputT : never) | ((element: T, context: ContextT) => OutputT), - context: ContextT = undefined! + context: ContextT = undefined!, ): PCollection { if (extractContext(fn)) { context = { ...extractContext(fn), ...context }; @@ -127,9 +127,9 @@ export class PCollection { : fn(element, context); }, }, - context - ) - ) + context, + ), + ), ); } @@ -137,7 +137,7 @@ export class PCollection { fn: | (ContextT extends undefined ? (element: T) => Iterable : never) | ((element: T, context: ContextT) => Iterable), - context: ContextT = undefined! + context: ContextT = undefined!, ): PCollection { if (extractContext(fn)) { context = { ...extractContext(fn), ...context }; @@ -155,9 +155,9 @@ export class PCollection { : fn(element, context); }, }, - context - ) - ) + context, + ), + ), ); } @@ -184,7 +184,7 @@ export type PValue = */ export function flattenPValue( pValue: PValue, - prefix: string = "" + prefix: string = "", ): { [key: string]: PCollection } { const result: { [key: string]: PCollection } = {}; if (pValue === null || pValue === undefined) { @@ -231,7 +231,7 @@ class PValueWrapper> { apply>( transform: PTransform, - root: Root | null = null + root: Root | null = null, ) { if (!(transform instanceof PTransformClass)) { transform = new PTransformClassFromCallable(transform); @@ -241,14 +241,14 @@ class PValueWrapper> { async applyAsync>( transform: AsyncPTransform, - root: Root | null = null + root: Root | null = null, ) { if (!(transform instanceof AsyncPTransformClass)) { transform = new AsyncPTransformClassFromCallable(transform); } return await this.pipeline(root).applyAsyncTransform( transform, - this.pvalue + this.pvalue, ); } @@ -264,20 +264,20 @@ class PValueWrapper> { class PTransformClassFromCallable< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > extends PTransformClass { expander: ( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => OutputT; constructor( expander: ( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform - ) => OutputT + transformProto: runnerApi.PTransform, + ) => OutputT, ) { super(extractName(expander)); this.expander = expander; @@ -286,7 +286,7 @@ class PTransformClassFromCallable< expandInternal( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { return this.expander(input, pipeline, transformProto); } @@ -294,20 +294,20 @@ class PTransformClassFromCallable< class AsyncPTransformClassFromCallable< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > extends AsyncPTransformClass { expander: ( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => Promise; constructor( expander: ( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform - ) => Promise + transformProto: runnerApi.PTransform, + ) => Promise, ) { super(extractName(expander)); this.expander = expander; @@ -316,7 +316,7 @@ class AsyncPTransformClassFromCallable< async expandInternalAsync( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { return this.expander(input, pipeline, transformProto); } diff --git a/sdks/typescript/src/apache_beam/runners/artifacts.ts b/sdks/typescript/src/apache_beam/runners/artifacts.ts index f5678621ba56..3777dacada7b 100644 --- a/sdks/typescript/src/apache_beam/runners/artifacts.ts +++ b/sdks/typescript/src/apache_beam/runners/artifacts.ts @@ -33,7 +33,7 @@ const defaultArtifactDir = path.join( os.homedir(), ".apache_beam", "cache", - "artifacts" + "artifacts", ); /** @@ -43,7 +43,7 @@ const defaultArtifactDir = path.join( export async function* resolveArtifacts( client: IArtifactRetrievalServiceClient, artifacts: Iterable, - localDir: string = defaultArtifactDir + localDir: string = defaultArtifactDir, ): AsyncGenerator { const resolved = await client.resolveArtifacts({ artifacts: Array.from(artifacts), @@ -51,11 +51,11 @@ export async function* resolveArtifacts( }).response; async function storeArtifact( - artifact: runnerApi.ArtifactInformation + artifact: runnerApi.ArtifactInformation, ): Promise { if (artifact.typeUrn === "beam:artifact:type:file:v1") { const payload = runnerApi.ArtifactFilePayload.fromBinary( - artifact.typePayload + artifact.typePayload, ); // As we're storing artifacts by hash, we can safely re-use if we've // ever seen this before. @@ -116,7 +116,7 @@ export async function* resolveArtifacts( export async function offerArtifacts( client: IArtifactStagingServiceClient, stagingToken: string, - rootDir: string = defaultArtifactDir + rootDir: string = defaultArtifactDir, ) { const call = client.reverseArtifactRetrievalService(); call.responses.onMessage(async (msg) => { @@ -138,7 +138,7 @@ export async function offerArtifacts( switch (msg.request.getArtifact.artifact!.typeUrn) { case "beam:artifact:type:file:v1": const payload = runnerApi.ArtifactFilePayload.fromBinary( - msg.request.getArtifact.artifact!.typePayload + msg.request.getArtifact.artifact!.typePayload, ); const filePath = path.normalize(payload.path); if (!filePath.startsWith(rootDir)) { @@ -146,7 +146,7 @@ export async function offerArtifacts( "Refusing to serve " + filePath + " as it is not under " + - rootDir + rootDir, ); } const handle = fs.createReadStream(filePath); @@ -173,7 +173,7 @@ export async function offerArtifacts( default: throw new Error( "Unknown artifact type " + - msg.request.getArtifact.artifact!.typeUrn + msg.request.getArtifact.artifact!.typeUrn, ); } break; @@ -200,6 +200,6 @@ function tempFile(dir) { crypto .randomBytes(16) .toString("base64") - .replace(/[^a-zA-Z0-9]/g, "_") + .replace(/[^a-zA-Z0-9]/g, "_"), ); } diff --git a/sdks/typescript/src/apache_beam/runners/dataflow.ts b/sdks/typescript/src/apache_beam/runners/dataflow.ts index 0a88b17f695d..5e1c052148fa 100644 --- a/sdks/typescript/src/apache_beam/runners/dataflow.ts +++ b/sdks/typescript/src/apache_beam/runners/dataflow.ts @@ -30,7 +30,7 @@ export function dataflowRunner(runnerOptions: { return new (class extends Runner { async runPipeline( pipeline: Pipeline, - options: Object = {} + options: Object = {}, ): Promise { var augmentedOptions = { experiments: [] as string[], ...options }; augmentedOptions.experiments.push("use_runner_v2"); @@ -38,11 +38,11 @@ export function dataflowRunner(runnerOptions: { augmentedOptions.experiments.push("use_sibling_sdk_workers"); const service = PythonService.forModule( "apache_beam.runners.dataflow.dataflow_job_service", - ["--port", "{{PORT}}"] + ["--port", "{{PORT}}"], ); const result = new PortableRunner( runnerOptions as any, - service + service, ).runPipeline(pipeline, augmentedOptions); result.then((res) => { res.waitUntilFinish().then((_state) => { diff --git a/sdks/typescript/src/apache_beam/runners/direct_runner.ts b/sdks/typescript/src/apache_beam/runners/direct_runner.ts index 999026a884ee..d616e2d4f129 100644 --- a/sdks/typescript/src/apache_beam/runners/direct_runner.ts +++ b/sdks/typescript/src/apache_beam/runners/direct_runner.ts @@ -85,7 +85,7 @@ class DirectRunner extends Runner { } for (const windowing of Object.values( - proto.components!.windowingStrategies + proto.components!.windowingStrategies, )) { if ( ![ @@ -122,7 +122,7 @@ class DirectRunner extends Runner { descriptor, null!, new state.CachingStateProvider(stateProvider), - [impulse.urn] + [impulse.urn], ); await processor.process("bundle_id"); @@ -134,7 +134,7 @@ class DirectRunner extends Runner { const shortIdCache = new metrics.MetricsShortIdCache(); const monitoringData = processor.monitoringData(shortIdCache); return Array.from(monitoringData.entries()).map(([id, payload]) => - shortIdCache.asMonitoringInfo(id, payload) + shortIdCache.asMonitoringInfo(id, payload), ); } })(); @@ -151,10 +151,10 @@ class DirectImpulseOperator implements operators.IOperator { constructor( public transformId: string, transform: PTransform, - context: operators.OperatorContext + context: operators.OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); } @@ -189,17 +189,17 @@ class DirectGbkOperator implements operators.IOperator { constructor( public transformId: string, transform: PTransform, - context: operators.OperatorContext + context: operators.OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); const inputPc = context.descriptor.pcollections[ onlyElement(Object.values(transform.inputs)) ]; this.keyCoder = context.pipelineContext.getCoder( - context.descriptor.coders[inputPc.coderId].componentCoderIds[0] + context.descriptor.coders[inputPc.coderId].componentCoderIds[0], ); const windowingStrategy = context.descriptor.windowingStrategies[inputPc.windowingStrategyId]; @@ -212,11 +212,11 @@ class DirectGbkOperator implements operators.IOperator { windowingStrategy.outputTime !== runnerApi.OutputTime_Enum.END_OF_WINDOW ) { throw new Error( - "Unsupported windowing output time: " + windowingStrategy + "Unsupported windowing output time: " + windowingStrategy, ); } this.windowCoder = context.pipelineContext.getCoder( - windowingStrategy.windowCoderId + windowingStrategy.windowCoderId, ); } @@ -245,7 +245,7 @@ class DirectGbkOperator implements operators.IOperator { const encodedKey = parts[1]; const window = operators.decodeFromBase64( encodedWindow, - this.windowCoder + this.windowCoder, ); const maybePromise = this.receiver.receive({ value: { @@ -347,13 +347,13 @@ function rewriteSideInputs(p: runnerApi.Pipeline, pipelineStateRef: string) { transform.inputs[side] = sideCopyId; const controlPCollId = uniqueName( pcolls, - sidePCollId + "-" + side + "-control" + sidePCollId + "-" + side + "-control", ); pcolls[controlPCollId] = pcolls[transform.inputs[mainPCollTag]]; bufferInputs[side] = controlPCollId; const collectTransformId = uniqueName( transforms, - transformId + "-" + side + "-collect" + transformId + "-" + side + "-collect", ); transforms[collectTransformId] = runnerApi.PTransform.create({ spec: { @@ -402,17 +402,17 @@ class CollectSideOperator implements operators.IOperator { constructor( public transformId: string, transform: PTransform, - context: operators.OperatorContext + context: operators.OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); const payload = deserializeFn(transform.spec!.payload!); this.parDoTransformId = payload.transformId; this.accessPattern = payload.accessPattern; this.sideInputId = payload.sideInputId; this.stateProvider = DirectRunner.inMemoryStatesRefs.get( - payload.pipelineStateRef + payload.pipelineStateRef, )!; const inputPc = @@ -423,7 +423,7 @@ class CollectSideOperator implements operators.IOperator { const windowingStrategy = context.descriptor.windowingStrategies[inputPc.windowingStrategyId]; this.windowCoder = context.pipelineContext.getCoder( - windowingStrategy.windowCoderId + windowingStrategy.windowCoderId, ); } @@ -435,7 +435,7 @@ class CollectSideOperator implements operators.IOperator { this.elementCoder.encode( wvalue.value, writer, - CoderContext.needsDelimiters + CoderContext.needsDelimiters, ); const encodedElement = writer.finish(); this.stateProvider.appendState( @@ -444,9 +444,9 @@ class CollectSideOperator implements operators.IOperator { this.accessPattern, this.sideInputId, window, - this.windowCoder + this.windowCoder, ), - encodedElement + encodedElement, ); } return operators.NonPromise; @@ -465,10 +465,10 @@ class BufferOperator implements operators.IOperator { constructor( public transformId: string, transform: PTransform, - context: operators.OperatorContext + context: operators.OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); } @@ -498,7 +498,7 @@ class InMemoryStateProvider implements state.StateProvider { getState( stateKey: fnApi.StateKey, - decode: (data: Uint8Array) => T + decode: (data: Uint8Array) => T, ): state.MaybePromise { return { type: "value", @@ -512,7 +512,7 @@ class InMemoryStateProvider implements state.StateProvider { getStateEntry(stateKey: fnApi.StateKey) { const cacheKey = Buffer.from(fnApi.StateKey.toBinary(stateKey)).toString( - "base64" + "base64", ); if (!this.chunks.has(cacheKey)) { this.chunks.set(cacheKey, []); diff --git a/sdks/typescript/src/apache_beam/runners/flink.ts b/sdks/typescript/src/apache_beam/runners/flink.ts index db6c7e8b0a1f..c34a50c1939c 100644 --- a/sdks/typescript/src/apache_beam/runners/flink.ts +++ b/sdks/typescript/src/apache_beam/runners/flink.ts @@ -39,7 +39,7 @@ export function flinkRunner(runnerOptions: Object = {}): Runner { return new (class extends Runner { async runPipeline( pipeline: Pipeline, - options: Object = {} + options: Object = {}, ): Promise { const allOptions = { ...defaultOptions, @@ -54,7 +54,7 @@ export function flinkRunner(runnerOptions: Object = {}): Runner { } if (!allOptions.artifactsDir) { allOptions.artifactsDir = fs.mkdtempSync( - path.join(os.tmpdir(), "flinkArtifactsDir") + path.join(os.tmpdir(), "flinkArtifactsDir"), ); } @@ -62,8 +62,8 @@ export function flinkRunner(runnerOptions: Object = {}): Runner { allOptions.flinkJobServerJar || (await JavaJarService.cachedJar( await JavaJarService.gradleToJar( - `runners:flink:${allOptions.flinkVersion}:job-server:shadowJar` - ) + `runners:flink:${allOptions.flinkVersion}:job-server:shadowJar`, + ), )); const jobServer = new JavaJarService(jobServerJar, [ "--flink-master", diff --git a/sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts b/sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts index 9b36cc6f0ab0..ad73d4106518 100644 --- a/sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts +++ b/sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts @@ -67,7 +67,7 @@ class PortableRunnerPipelineResult extends PipelineResult { constructor( runner: PortableRunner, jobId: string, - completionCallbacks: completionCallback[] + completionCallbacks: completionCallback[], ) { super(); this.runner = runner; @@ -142,7 +142,7 @@ export class PortableRunner extends Runner { constructor( options: string | { jobEndpoint: string; [others: string]: any }, - private jobService: JobService | undefined = undefined + private jobService: JobService | undefined = undefined, ) { super(); if (typeof options === "string") { @@ -161,7 +161,7 @@ export class PortableRunner extends Runner { new GrpcTransport({ host: this.defaultOptions?.jobEndpoint, channelCredentials: ChannelCredentials.createInsecure(), - }) + }), ); } return this.client; @@ -184,19 +184,19 @@ export class PortableRunner extends Runner { async runPipeline( pipeline: runnerApiProto.Pipeline, - options?: PipelineOptions + options?: PipelineOptions, ): Promise { return this.runPipelineWithProto(pipeline, options); } async runPipelineWithProto( pipeline: runnerApiProto.Pipeline, - options?: PipelineOptions + options?: PipelineOptions, ) { options = { ...this.defaultOptions, ...(options || {}) }; for (const [_, pcoll] of Object.entries( - pipeline.components!.pcollections + pipeline.components!.pcollections, )) { if (pcoll.isBounded == runnerApiProto.IsBounded_Enum.UNBOUNDED) { (options as any).streaming = true; @@ -221,7 +221,7 @@ export class PortableRunner extends Runner { // Replace the default environment according to the pipeline options. pipeline = runnerApiProto.Pipeline.clone(pipeline); for (const [envId, env] of Object.entries( - pipeline.components!.environments + pipeline.components!.environments, )) { if (env.urn === environments.TYPESCRIPT_DEFAULT_ENVIRONMENT_URN) { if (loopbackAddress) { @@ -233,7 +233,7 @@ export class PortableRunner extends Runner { environments.asDockerEnvironment( env, (options as any)?.sdkContainerImage || - DOCKER_BASE + ":" + version.replace("-SNAPSHOT", ".dev") + DOCKER_BASE + ":" + version.replace("-SNAPSHOT", ".dev"), ); const deps = pipeline.components!.environments[envId].dependencies; @@ -244,7 +244,7 @@ export class PortableRunner extends Runner { ["pack", "--pack-destination", tmpDir], { encoding: "latin1", - } + }, ); if (result.status === 0) { console.debug(result.stdout); @@ -252,7 +252,7 @@ export class PortableRunner extends Runner { throw new Error(result.output); } const packFile = path.resolve( - path.join(tmpDir, result.stdout.trim()) + path.join(tmpDir, result.stdout.trim()), ); deps.push(fileArtifact(packFile, "beam:artifact:type:npm:v1")); @@ -267,8 +267,8 @@ export class PortableRunner extends Runner { fileArtifact( path, "beam:artifact:type:npm_dep:v1", - new TextEncoder().encode(dep) - ) + new TextEncoder().encode(dep), + ), ); } } @@ -296,8 +296,8 @@ export class PortableRunner extends Runner { Object.entries(options).map(([k, v]) => [ `beam:option:${camel_to_snake(k)}:v1`, v, - ]) - ) + ]), + ), ); } const client = await this.getClient(); @@ -314,10 +314,10 @@ export class PortableRunner extends Runner { new GrpcTransport({ host: prepareResponse.artifactStagingEndpoint.url, channelCredentials: ChannelCredentials.createInsecure(), - }) + }), ), prepareResponse.stagingSessionToken, - "/" + "/", ); } @@ -340,7 +340,7 @@ export class PortableRunner extends Runner { function fileArtifact( filePath: string, roleUrn: string, - rolePayload: Uint8Array | undefined = undefined + rolePayload: Uint8Array | undefined = undefined, ) { const hasher = crypto.createHash("sha256"); hasher.update(fs.readFileSync(filePath)); diff --git a/sdks/typescript/src/apache_beam/runners/runner.ts b/sdks/typescript/src/apache_beam/runners/runner.ts index 3679cdad9ac2..b5e69cdc7256 100644 --- a/sdks/typescript/src/apache_beam/runners/runner.ts +++ b/sdks/typescript/src/apache_beam/runners/runner.ts @@ -38,8 +38,8 @@ export class PipelineResult { return Object.fromEntries( metrics.aggregateMetrics( await this.rawMetrics(), - "beam:metric:user:sum_int64:v1" - ) + "beam:metric:user:sum_int64:v1", + ), ); } @@ -47,8 +47,8 @@ export class PipelineResult { return Object.fromEntries( metrics.aggregateMetrics( await this.rawMetrics(), - "beam:metric:user:distribution_int64:v1" - ) + "beam:metric:user:distribution_int64:v1", + ), ); } } @@ -101,7 +101,7 @@ export abstract class Runner { */ async run( pipeline: (root: Root) => PValue | Promise>, - options?: PipelineOptions + options?: PipelineOptions, ): Promise { const pipelineResult = await this.runAsync(pipeline, options); const finalState = await pipelineResult.waitUntilFinish(); @@ -119,7 +119,7 @@ export abstract class Runner { */ async runAsync( pipeline: (root: Root) => PValue | Promise>, - options?: PipelineOptions + options?: PipelineOptions, ): Promise { const p = new Pipeline(); await pipeline(new Root(p)); @@ -128,7 +128,7 @@ export abstract class Runner { abstract runPipeline( pipeline: runnerApi.Pipeline, - options?: PipelineOptions + options?: PipelineOptions, ): Promise; } @@ -136,7 +136,7 @@ export function defaultRunner(defaultOptions: Object): Runner { return new (class extends Runner { async runPipeline( pipeline: runnerApi.Pipeline, - options: Object = {} + options: Object = {}, ): Promise { const directRunner = require("./direct_runner").directRunner(defaultOptions); diff --git a/sdks/typescript/src/apache_beam/runners/universal.ts b/sdks/typescript/src/apache_beam/runners/universal.ts index 2a30570adf64..01fd0c88c919 100644 --- a/sdks/typescript/src/apache_beam/runners/universal.ts +++ b/sdks/typescript/src/apache_beam/runners/universal.ts @@ -27,14 +27,14 @@ export function universalRunner(runnerOptions: { return new (class extends Runner { async runPipeline( pipeline: Pipeline, - options: Object = {} + options: Object = {}, ): Promise { return new PortableRunner( runnerOptions as any, PythonService.forModule( "apache_beam.runners.portability.local_job_service_main", - ["--port", "{{PORT}}"] - ) + ["--port", "{{PORT}}"], + ), ).runPipeline(pipeline, { directEmbedDockerPython: true, ...options }); } })(); diff --git a/sdks/typescript/src/apache_beam/testing/assert.ts b/sdks/typescript/src/apache_beam/testing/assert.ts index 3e7fca2fb5a8..062ef229cfac 100644 --- a/sdks/typescript/src/apache_beam/testing/assert.ts +++ b/sdks/typescript/src/apache_beam/testing/assert.ts @@ -45,7 +45,7 @@ function callAssertDeepEqual(a, b) { *``` */ export function assertDeepEqual( - expected: T[] + expected: T[], ): beam.PTransform, void> { return beam.withName( `assertDeepEqual(${JSON.stringify(expected).substring(0, 100)})`, @@ -54,15 +54,15 @@ export function assertDeepEqual( assertContentsSatisfies((actual: T[]) => { const actualArray: T[] = [...actual]; expected.sort((a, b) => - JSON.stringify(a) < JSON.stringify(b) ? -1 : 1 + JSON.stringify(a) < JSON.stringify(b) ? -1 : 1, ); actualArray.sort((a, b) => - JSON.stringify(a) < JSON.stringify(b) ? -1 : 1 + JSON.stringify(a) < JSON.stringify(b) ? -1 : 1, ); callAssertDeepEqual(actualArray, expected); - }) + }), ); - } + }, ); } @@ -74,7 +74,7 @@ export function assertDeepEqual( * of the provided elements is not well determined. */ export function assertContentsSatisfies( - check: (actual: T[]) => void + check: (actual: T[]) => void, ): beam.PTransform, void> { function expand(pcoll: beam.PCollection) { // We provide some value here to ensure there is at least one element @@ -97,13 +97,13 @@ export function assertContentsSatisfies( kv.value?.filter((o) => o.tag === "actual").map((o) => o.value) || []; check(actual); - }) + }), ); } return beam.withName( `assertContentsSatisfies(${beam.extractName(check)})`, - expand + expand, ); } diff --git a/sdks/typescript/src/apache_beam/testing/multi_pipeline_runner.ts b/sdks/typescript/src/apache_beam/testing/multi_pipeline_runner.ts index 4389828ef20c..e7f4fb7acca1 100644 --- a/sdks/typescript/src/apache_beam/testing/multi_pipeline_runner.ts +++ b/sdks/typescript/src/apache_beam/testing/multi_pipeline_runner.ts @@ -57,7 +57,7 @@ export class MultiPipelineRunner extends Runner { constructor( private underlying: Runner, - private options: PipelineOptions = {} + private options: PipelineOptions = {}, ) { super(); } @@ -74,7 +74,7 @@ export class MultiPipelineRunner extends Runner { async runAsync( pipeline: (root: Root) => PValue | Promise>, - options?: PipelineOptions + options?: PipelineOptions, ): Promise { if (this.nextTestName === undefined) { this.setNextTestName("pipeline"); @@ -84,7 +84,7 @@ export class MultiPipelineRunner extends Runner { await new Root(p).applyAsync( withName(this.nextTestName!, async (root) => { await pipeline(root); - }) + }), ); this.nextTestName = undefined; return this.runPipeline(p.getProto()); @@ -92,7 +92,7 @@ export class MultiPipelineRunner extends Runner { async runPipeline( pipeline: runnerApi.Pipeline, - options?: PipelineOptions + options?: PipelineOptions, ): Promise { if (options) { throw new Error("Per-pipeline options not supported."); @@ -108,13 +108,13 @@ export class MultiPipelineRunner extends Runner { console.log(this.allPipelines); const pipelineResult = await this.underlying.runPipeline( this.allPipelines, - this.options + this.options, ); const finalState = await pipelineResult.waitUntilFinish(); if (finalState != jobApi.JobState_Enum.DONE) { // TODO: Grab the last/most severe error message? throw new Error( - "Job finished in state " + jobApi.JobState_Enum[finalState] + "Job finished in state " + jobApi.JobState_Enum[finalState], ); } this.allPipelines = undefined; @@ -147,23 +147,23 @@ export class MultiPipelineRunner extends Runner { } mergeComponents( pipeline.components?.transforms, - this.allPipelines.components?.transforms + this.allPipelines.components?.transforms, ); mergeComponents( pipeline.components?.pcollections, - this.allPipelines.components?.pcollections + this.allPipelines.components?.pcollections, ); mergeComponents( pipeline.components?.coders, - this.allPipelines.components?.coders + this.allPipelines.components?.coders, ); mergeComponents( pipeline.components?.windowingStrategies, - this.allPipelines.components?.windowingStrategies + this.allPipelines.components?.windowingStrategies, ); mergeComponents( pipeline.components?.environments, - this.allPipelines.components?.environments + this.allPipelines.components?.environments, ); this.allPipelines.requirements = [ ...new Set([...this.allPipelines.requirements, ...pipeline.requirements]), diff --git a/sdks/typescript/src/apache_beam/transforms/create.ts b/sdks/typescript/src/apache_beam/transforms/create.ts index bd654340386a..d08704086007 100644 --- a/sdks/typescript/src/apache_beam/transforms/create.ts +++ b/sdks/typescript/src/apache_beam/transforms/create.ts @@ -27,7 +27,7 @@ import { Root, PCollection } from "../pvalue"; */ export function create( elements: T[], - reshuffle: boolean = true + reshuffle: boolean = true, ): PTransform> { function create(root: Root): PCollection { const pcoll = root diff --git a/sdks/typescript/src/apache_beam/transforms/external.ts b/sdks/typescript/src/apache_beam/transforms/external.ts index 155e260f6a52..700e9cb86863 100644 --- a/sdks/typescript/src/apache_beam/transforms/external.ts +++ b/sdks/typescript/src/apache_beam/transforms/external.ts @@ -71,24 +71,24 @@ const defaultRawExternalTransformOptions: RawExternalTransformOptions = { // a cleaner way to specify them than using internal.WithCoderInternal. export function rawExternalTransform< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, >( urn: string, payload: Uint8Array | { [key: string]: any }, serviceProviderOrAddress: string | (() => Promise), - options: RawExternalTransformOptions = {} + options: RawExternalTransformOptions = {}, ): transform.AsyncPTransform { return new RawExternalTransform( urn, payload, serviceProviderOrAddress, - options + options, ); } class RawExternalTransform< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > extends transform.AsyncPTransformClass { static namespaceCounter = 0; static freshNamespace() { @@ -103,7 +103,7 @@ class RawExternalTransform< private urn: string, payload: Uint8Array | { [key: string]: any }, serviceProviderOrAddress: string | (() => Promise), - options: RawExternalTransformOptions + options: RawExternalTransformOptions, ) { super("External(" + urn + ")"); this.options = { ...defaultRawExternalTransformOptions, ...options }; @@ -126,7 +126,7 @@ class RawExternalTransform< async expandInternalAsync( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ): Promise { const pipelineComponents = pipeline.getProto().components!; const namespace = RawExternalTransform.freshNamespace(); @@ -155,7 +155,7 @@ class RawExternalTransform< } for (const [output, coder] of Object.entries( - this.options.requestedOutputCoders! + this.options.requestedOutputCoders!, )) { request.outputCoderRequests[output] = pipeline.getCoderId(coder); } @@ -164,11 +164,11 @@ class RawExternalTransform< Object.assign(request.components!.coders, pipelineComponents.coders); Object.assign( request.components!.windowingStrategies, - pipelineComponents.windowingStrategies + pipelineComponents.windowingStrategies, ); Object.assign( request.components!.environments, - pipelineComponents.environments + pipelineComponents.environments, ); const service = await this.serviceProvider(); @@ -178,7 +178,7 @@ class RawExternalTransform< new GrpcTransport({ host: address, channelCredentials: ChannelCredentials.createInsecure(), - }) + }), ); try { @@ -190,7 +190,7 @@ class RawExternalTransform< response.components = await this.resolveArtifacts( response.components!, - address + address, ); return this.splice(pipeline, transformProto, response, namespace); @@ -208,12 +208,12 @@ class RawExternalTransform< */ async resolveArtifacts( components: runnerApi.Components, - address: string + address: string, ): Promise { // Don't even bother creating a connection if there are no dependencies. if ( Object.values(components.environments).every( - (env) => env.dependencies.length === 0 + (env) => env.dependencies.length === 0, ) ) { return components; @@ -225,7 +225,7 @@ class RawExternalTransform< new GrpcTransport({ host: address, channelCredentials: ChannelCredentials.createInsecure(), - }) + }), ); // For each new environment, convert (if needed) all dependencies into @@ -237,12 +237,12 @@ class RawExternalTransform< let result: runnerApi.ArtifactInformation[] = []; for await (const dep of artifacts.resolveArtifacts( artifactClient, - env.dependencies + env.dependencies, )) { result.push(dep); } return result; - })() + })(), ); } } @@ -254,11 +254,11 @@ class RawExternalTransform< pipeline: Pipeline, transformProto: runnerApi.PTransform, response: ExpansionResponse, - namespace: string + namespace: string, ): OutputT { function copyNamespaceComponents( src: { [key: string]: T }, - dest: { [key: string]: T } + dest: { [key: string]: T }, ) { for (const [id, proto] of Object.entries(src)) { if (id.startsWith(namespace)) { @@ -274,14 +274,14 @@ class RawExternalTransform< // Some SDKs enforce input naming conventions. const newTags = difference( new Set(Object.keys(response.transform!.inputs)), - new Set(Object.keys(transformProto.inputs)) + new Set(Object.keys(transformProto.inputs)), ); if (newTags.length > 1) { throw new Error("Ambiguous renaming of tags."); } else if (newTags.length === 1) { const missingTags = difference( new Set(Object.keys(transformProto.inputs)), - new Set(Object.keys(response.transform!.inputs)) + new Set(Object.keys(response.transform!.inputs)), ); transformProto.inputs[newTags[0]] = transformProto.inputs[missingTags[0]]; delete transformProto.inputs[missingTags[0]]; @@ -292,13 +292,13 @@ class RawExternalTransform< Object.keys(response.transform!.inputs).map((k) => [ response.transform!.inputs[k], transformProto.inputs[k], - ]) + ]), ); response.transform!.inputs = Object.fromEntries( Object.entries(response.transform!.inputs).map(([k, v]) => [ k, renamedInputs[v], - ]) + ]), ); for (const t of Object.values(response.components!.transforms)) { t.inputs = Object.fromEntries( @@ -307,7 +307,7 @@ class RawExternalTransform< renamedInputs[v] !== null && renamedInputs[v] !== undefined ? renamedInputs[v] : v, - ]) + ]), ); } @@ -320,23 +320,23 @@ class RawExternalTransform< pipeline.getProto().requirements.push(...response.requirements); copyNamespaceComponents( response.components!.transforms, - pipelineComponents!.transforms + pipelineComponents!.transforms, ); copyNamespaceComponents( response.components!.pcollections, - pipelineComponents!.pcollections + pipelineComponents!.pcollections, ); copyNamespaceComponents( response.components!.coders, - pipelineComponents!.coders + pipelineComponents!.coders, ); copyNamespaceComponents( response.components!.environments, - pipelineComponents!.environments + pipelineComponents!.environments, ); copyNamespaceComponents( response.components!.windowingStrategies, - pipelineComponents!.windowingStrategies + pipelineComponents!.windowingStrategies, ); // Ensure we understand the resulting coders. @@ -360,7 +360,7 @@ class RawExternalTransform< } else if (outputKeys.length === 1) { return new PCollection( pipeline, - response.transform!.outputs[outputKeys[0]] + response.transform!.outputs[outputKeys[0]], ) as OutputT; } } @@ -368,14 +368,14 @@ class RawExternalTransform< Object.entries(response.transform!.outputs).map(([k, v]) => [ k, new PCollection(pipeline, v), - ]) + ]), ) as OutputT; } } function encodeSchemaPayload( payload: any, - schema: Schema | undefined = undefined + schema: Schema | undefined = undefined, ): Uint8Array { const encoded = new Writer(); if (!schema) { diff --git a/sdks/typescript/src/apache_beam/transforms/flatten.ts b/sdks/typescript/src/apache_beam/transforms/flatten.ts index 99fcd74e8a05..712cf85c2578 100644 --- a/sdks/typescript/src/apache_beam/transforms/flatten.ts +++ b/sdks/typescript/src/apache_beam/transforms/flatten.ts @@ -32,7 +32,7 @@ export function flatten(): PTransform[], PCollection> { function expandInternal( inputs: PCollection[], pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { transformProto.spec = runnerApi.FunctionSpec.create({ urn: flatten.urn, @@ -40,7 +40,7 @@ export function flatten(): PTransform[], PCollection> { // TODO: UnionCoder if they're not the same? const coders = new Set( - inputs.map((pc) => pipeline.context.getPCollectionCoderId(pc)) + inputs.map((pc) => pipeline.context.getPCollectionCoderId(pc)), ); const coder = coders.size === 1 ? [...coders][0] : new GeneralObjectCoder(); diff --git a/sdks/typescript/src/apache_beam/transforms/group_and_combine.ts b/sdks/typescript/src/apache_beam/transforms/group_and_combine.ts index 777d89636645..2cf2d44281af 100644 --- a/sdks/typescript/src/apache_beam/transforms/group_and_combine.ts +++ b/sdks/typescript/src/apache_beam/transforms/group_and_combine.ts @@ -80,7 +80,7 @@ export class GroupBy extends PTransformClass< */ constructor( key: string | string[] | ((element: T) => K), - keyName: string | undefined = undefined + keyName: string | undefined = undefined, ) { super(); [this.keyFn, this.keyNames] = extractFnAndName(key, keyName || "key"); @@ -99,15 +99,15 @@ export class GroupBy extends PTransformClass< combining( expr: string | ((element: T) => I), combiner: Combiner, - resultName: string + resultName: string, ) { return withName( extractName(this), new GroupByAndCombine(this.keyFn, this.keyNames, []).combining( expr, combiner, - resultName - ) + resultName, + ), ); } } @@ -122,11 +122,11 @@ export class GroupBy extends PTransformClass< */ export function groupBy( key: string | string[] | ((element: T) => K), - keyName: string | undefined = undefined + keyName: string | undefined = undefined, ): GroupBy { return withName( `groupBy(${extractName(key)}`, - new GroupBy(key, keyName) + new GroupBy(key, keyName), ); } @@ -153,15 +153,15 @@ export class GroupGlobally extends PTransformClass< combining( expr: string | ((element: T) => I), combiner: Combiner, - resultName: string + resultName: string, ) { return withName( extractName(this), new GroupByAndCombine((_) => null, undefined, []).combining( expr, combiner, - resultName - ) + resultName, + ), ); } } @@ -190,7 +190,7 @@ class GroupByAndCombine extends PTransformClass< constructor( keyFn: (element: T) => any, keyNames: string | string[] | undefined, - combiners: CombineSpec[] + combiners: CombineSpec[], ) { super(); this.keyFn = keyFn; @@ -202,7 +202,7 @@ class GroupByAndCombine extends PTransformClass< combining( expr: string | ((element: T) => I), combiner: Combiner, - resultName: string // TODO: (Unique names) Optionally derive from expr and combineFn? + resultName: string, // TODO: (Unique names) Optionally derive from expr and combineFn? ) { return withName( extractName(this), @@ -215,8 +215,8 @@ class GroupByAndCombine extends PTransformClass< combineFn: toCombineFn(combiner), resultName: resultName, }, - ]) - ) + ]), + ), ); } @@ -231,8 +231,8 @@ class GroupByAndCombine extends PTransformClass< }) .apply( internal.combinePerKey( - multiCombineFn(this_.combiners.map((c) => c.combineFn)) - ) + multiCombineFn(this_.combiners.map((c) => c.combineFn)), + ), ) .map(function constructResult(kv) { const result = {}; @@ -259,7 +259,7 @@ export function countPerElement(): PTransform< > { return withName( "countPerElement", - groupBy((e) => e, "element").combining((e) => e, count, "count") + groupBy((e) => e, "element").combining((e) => e, count, "count"), ); } @@ -270,7 +270,7 @@ export function countGlobally(): PTransform< return withName("countGlobally", (input) => input .apply(new GroupGlobally().combining((e) => e, count, "count")) - .map((o) => o.count) + .map((o) => o.count), ); } @@ -293,7 +293,7 @@ interface CombineSpec { * commutative and associative). */ export function binaryCombineFn( - combiner: (a: I, b: I) => I + combiner: (a: I, b: I) => I, ): CombineFn { return { createAccumulator: () => undefined, @@ -307,7 +307,7 @@ export function binaryCombineFn( // TODO: (Typescript) Is there a way to indicate type parameters match the above? function multiCombineFn( combineFns: CombineFn[], - batchSize: number = 100 + batchSize: number = 100, ): CombineFn { return { createAccumulator: () => combineFns.map((fn) => fn.createAccumulator()), @@ -353,7 +353,7 @@ function multiCombineFn( // TODO: Consider adding valueFn(s) rather than using the full value. export function coGroupBy( key: string | string[] | ((element: T) => K), - keyName: string | undefined = undefined + keyName: string | undefined = undefined, ): PTransform< { [key: string]: PCollection }, PCollection<{ key: K; values: { [key: string]: Iterable } }> @@ -370,22 +370,22 @@ export function coGroupBy( key: keyFn(element), tag, element, - })) - ) + })), + ), ); return P(tagged) .apply(flatten()) .apply(groupBy("key")) .map(function groupValues({ key, value }) { const groupedValues: { [key: string]: any[] } = Object.fromEntries( - tags.map((tag) => [tag, []]) + tags.map((tag) => [tag, []]), ); for (const { tag, element } of value) { groupedValues[tag].push(element); } return { key, values: groupedValues }; }); - } + }, ); } @@ -398,7 +398,7 @@ export function coGroupBy( // ): [(element: T) => K, P | P[]] { function extractFnAndName( extractor: string | string[] | ((T) => K), - defaultName: string + defaultName: string, ): [(T) => K, string | string[]] { if ( typeof extractor === "string" || diff --git a/sdks/typescript/src/apache_beam/transforms/internal.ts b/sdks/typescript/src/apache_beam/transforms/internal.ts index 9097a4222ba7..e39785d8d598 100644 --- a/sdks/typescript/src/apache_beam/transforms/internal.ts +++ b/sdks/typescript/src/apache_beam/transforms/internal.ts @@ -49,7 +49,7 @@ export function impulse(): PTransform> { function expandInternal( input: Root, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { transformProto.spec = runnerApi.FunctionSpec.create({ urn: impulse.urn, @@ -66,14 +66,14 @@ impulse.urn = "beam:transform:impulse:v1"; // TODO: (API) Should we offer a method on PCollection to do this? export function withCoderInternal( - coder: Coder + coder: Coder, ): PTransform, PCollection> { return withName( `withCoderInternal(${extractName(coder)})`, ( input: PCollection, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => { // IDENTITY rather than Flatten for better fusion. transformProto.spec = { @@ -84,12 +84,12 @@ export function withCoderInternal( urn: urns.IDENTITY_DOFN_URN, payload: undefined!, }), - }) + }), ), }; return pipeline.createPCollectionInternal(coder); - } + }, ); } @@ -108,7 +108,7 @@ export function withCoderInternal( * invoke cross-language transforms. */ export function withRowCoder( - exemplar: T + exemplar: T, ): PTransform, PCollection> { return withCoderInternal(RowCoder.fromJSON(exemplar)); } @@ -131,7 +131,7 @@ export function groupByKey(): PTransform< function expandInternal( input: PCollection>, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { const pipelineComponents: runnerApi.Components = pipeline.getProto().components!; @@ -144,8 +144,8 @@ export function groupByKey(): PTransform< return input .apply( withCoderInternal( - new KVCoder(new GeneralObjectCoder(), new GeneralObjectCoder()) - ) + new KVCoder(new GeneralObjectCoder(), new GeneralObjectCoder()), + ), ) .apply(groupByKey()); } @@ -184,12 +184,12 @@ groupByKey.urn = "beam:transform:group_by_key:v1"; * help reduce the original data into a single aggregator per key per worker. */ export function combinePerKey( - combineFn: CombineFn + combineFn: CombineFn, ): PTransform>, PCollection>> { function expandInternal( input: PCollection>, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { const pipelineComponents: runnerApi.Components = pipeline.getProto().components!; @@ -198,7 +198,7 @@ export function combinePerKey( try { // If this fails, we cannot lift, so we skip setting the liftable URN. CombinePerKeyPrecombineOperator.checkSupportsWindowing( - pipelineComponents.windowingStrategies[inputProto.windowingStrategyId] + pipelineComponents.windowingStrategies[inputProto.windowingStrategyId], ); // Ensure the input is using the KV coder. @@ -207,14 +207,14 @@ export function combinePerKey( return input .apply( withCoderInternal( - new KVCoder(new GeneralObjectCoder(), new GeneralObjectCoder()) - ) + new KVCoder(new GeneralObjectCoder(), new GeneralObjectCoder()), + ), ) .apply(combinePerKey(combineFn)); } const inputValueCoder = pipeline.context.getCoder( - inputCoderProto.componentCoderIds[1] + inputCoderProto.componentCoderIds[1], ); transformProto.spec = runnerApi.FunctionSpec.create({ @@ -227,7 +227,7 @@ export function combinePerKey( accumulatorCoderId: pipeline.context.getCoderId( combineFn.accumulatorCoder ? combineFn.accumulatorCoder(inputValueCoder) - : new GeneralObjectCoder() + : new GeneralObjectCoder(), ), }), }); @@ -250,16 +250,16 @@ export function combinePerKey( for (const value of kv.value) { accumulators[ix % 3] = combineFn.addInput( accumulators[ix % 3], - value + value, ); } return { key: kv.key, value: combineFn.extractOutput( - combineFn.mergeAccumulators(accumulators) + combineFn.mergeAccumulators(accumulators), ), }; - }) + }), ); } diff --git a/sdks/typescript/src/apache_beam/transforms/pardo.ts b/sdks/typescript/src/apache_beam/transforms/pardo.ts index 8abe24b9de9a..60a3d11aa18d 100644 --- a/sdks/typescript/src/apache_beam/transforms/pardo.ts +++ b/sdks/typescript/src/apache_beam/transforms/pardo.ts @@ -88,10 +88,10 @@ export interface DoFn { export function parDo< InputT, OutputT, - ContextT extends Object | undefined = undefined + ContextT extends Object | undefined = undefined, >( doFn: DoFn, - context: ContextT = undefined! + context: ContextT = undefined!, ): PTransform, PCollection> { if (extractContext(doFn)) { context = { ...extractContext(doFn), ...context }; @@ -99,7 +99,7 @@ export function parDo< function expandInternal( input: PCollection, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { // Extract and populate side inputs from the context. const sideInputs = {}; @@ -135,8 +135,8 @@ export function parDo< urn: isGlobalSide ? urns.GLOBAL_WINDOW_MAPPING_FN_URN : mainWindowingStrategyId === sideWindowingStrategyId - ? urns.IDENTITY_WINDOW_MAPPING_FN_URN - : urns.ASSIGN_MAX_TIMESTAMP_WINDOW_MAPPING_FN_URN, + ? urns.IDENTITY_WINDOW_MAPPING_FN_URN + : urns.ASSIGN_MAX_TIMESTAMP_WINDOW_MAPPING_FN_URN, value: new Uint8Array(), }, }; @@ -161,7 +161,7 @@ export function parDo< }), }), sideInputs: sideInputs, - }) + }), ), }); @@ -169,7 +169,7 @@ export function parDo< // coder to encode the various types that exist in JS. // TODO: (Types) Should there be a way to specify, or better yet infer, the coder to use? return pipeline.createPCollectionInternal( - new GeneralObjectCoder() + new GeneralObjectCoder(), ); } @@ -198,12 +198,12 @@ export type SplitOptions = { // TODO: Naming. export function split( tags: string[], - options: SplitOptions = {} + options: SplitOptions = {}, ): PTransform, { [P in keyof X]: PCollection }> { function expandInternal( input: PCollection, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) { if (options.exclusive === undefined) { options.exclusive = true; @@ -229,7 +229,7 @@ export function split( urn: urns.SPLITTING_JS_DOFN_URN, payload: serializeFn(options), }), - }) + }), ), }); @@ -237,9 +237,9 @@ export function split( tags.map((tag) => [ tag, pipeline.createPCollectionInternal( - pipeline.context.getPCollectionCoderId(input) + pipeline.context.getPCollectionCoderId(input), ), - ]) + ]), ) as { [P in keyof X]: PCollection }; } @@ -248,11 +248,11 @@ export function split( export function partition( partitionFn: (element: T, numPartitions: number) => number, - numPartitions: number + numPartitions: number, ): PTransform, PCollection[]> { return function partition(input: PCollection) { const indices = Array.from({ length: numPartitions }, (v, i) => - i.toString() + i.toString(), ); const splits = input .map((x) => { @@ -275,7 +275,7 @@ export function withContext< ContextT, T extends | DoFn - | ((input: unknown, context: ContextT) => unknown) + | ((input: unknown, context: ContextT) => unknown), >(fn: T, contextSpec: ContextT): T { const untypedFn = fn as any; untypedFn.beamPardoContextSpec = { @@ -375,7 +375,7 @@ interface SideInputAccessor { export class SideInputParam< PCollT, AccessorT, - ValueT + ValueT, > extends ParDoLookupParam { // Populated by user. pcoll: PCollection; @@ -384,7 +384,7 @@ export class SideInputParam< constructor( pcoll: PCollection, - accessor: SideInputAccessor + accessor: SideInputAccessor, ) { super("sideInput"); this.pcoll = pcoll; @@ -398,7 +398,7 @@ export class SideInputParam< function copySideInputWithId( sideInput: SideInputParam, - id: string + id: string, ): SideInputParam { const copy = Object.create(sideInput); copy.sideInputId = id; @@ -407,7 +407,7 @@ function copySideInputWithId( } export function iterableSideInput( - pcoll: PCollection + pcoll: PCollection, ): SideInputParam, Iterable> { return new SideInputParam, Iterable>(pcoll, { accessPattern: "beam:side_input:iterable:v1", @@ -417,7 +417,7 @@ export function iterableSideInput( export function singletonSideInput( pcoll: PCollection, - defaultValue: T | undefined = undefined + defaultValue: T | undefined = undefined, ): SideInputParam, T> { return new SideInputParam, T>(pcoll, { accessPattern: "beam:side_input:iterable:v1", @@ -444,7 +444,10 @@ export function singletonSideInput( * The superclass of all metric accessors, such as counters and distributions. */ export class Metric extends ParDoUpdateParam { - constructor(readonly metricType: string, readonly name: string) { + constructor( + readonly metricType: string, + readonly name: string, + ) { super("metric"); } } diff --git a/sdks/typescript/src/apache_beam/transforms/python.ts b/sdks/typescript/src/apache_beam/transforms/python.ts index 2292f9ce0e66..d1ff02ef4c1a 100644 --- a/sdks/typescript/src/apache_beam/transforms/python.ts +++ b/sdks/typescript/src/apache_beam/transforms/python.ts @@ -43,12 +43,12 @@ import * as row_coder from "../coders/row_coder"; */ export function pythonTransform< InputT extends beam.PValue, - OutputT extends beam.PValue + OutputT extends beam.PValue, >( constructor: string, args_or_kwargs: any[] | { [key: string]: any } | undefined = undefined, kwargs: { [key: string]: any } | undefined = undefined, - options: external.RawExternalTransformOptions = {} + options: external.RawExternalTransformOptions = {}, ): beam.AsyncPTransform { let args; if (args_or_kwargs === undefined) { @@ -78,9 +78,9 @@ export function pythonTransform< async () => PythonService.forModule( "apache_beam.runners.portability.expansion_service_main", - ["--fully_qualified_name_glob=*", "--port", "{{PORT}}"] + ["--fully_qualified_name_glob=*", "--port", "{{PORT}}"], ), - options + options, ); } diff --git a/sdks/typescript/src/apache_beam/transforms/sql.ts b/sdks/typescript/src/apache_beam/transforms/sql.ts index 9f0e117f2f81..f219d40b0c02 100644 --- a/sdks/typescript/src/apache_beam/transforms/sql.ts +++ b/sdks/typescript/src/apache_beam/transforms/sql.ts @@ -41,10 +41,10 @@ import { serviceProviderFromJavaGradleTarget } from "../utils/service"; * )); */ export function sqlTransform< - InputT extends PCollection | { [key: string]: PCollection } + InputT extends PCollection | { [key: string]: PCollection }, >( query: string, - inputTypes = null + inputTypes = null, ): transform.AsyncPTransform> { // TOOD: (API) (Typescript): How to infer input_types, or at least make it optional. async function expandInternal(input: InputT): Promise> { @@ -60,7 +60,7 @@ export function sqlTransform< "SqlTransform can only be applied to schema'd transforms. " + "Please ensure the input PCollection(s) have a RowCoder, " + "or pass a prototypical element in as the second argument " + - "of SqlTransform so that one can be inferred." + "of SqlTransform so that one can be inferred.", ); } return pcoll; @@ -78,9 +78,9 @@ export function sqlTransform< input[tag], inputTypes === null || inputTypes === undefined ? null - : inputTypes[tag] + : inputTypes[tag], ), - ]) + ]), ) as InputT; } @@ -89,9 +89,9 @@ export function sqlTransform< "beam:external:java:sql:v1", { query: query }, serviceProviderFromJavaGradleTarget( - "sdks:java:extensions:sql:expansion-service:shadowJar" - ) - ) + "sdks:java:extensions:sql:expansion-service:shadowJar", + ), + ), ); } diff --git a/sdks/typescript/src/apache_beam/transforms/transform.ts b/sdks/typescript/src/apache_beam/transforms/transform.ts index 799413e02d12..0d628818bac0 100644 --- a/sdks/typescript/src/apache_beam/transforms/transform.ts +++ b/sdks/typescript/src/apache_beam/transforms/transform.ts @@ -84,7 +84,7 @@ export function extractName(withName: T): string { /** @internal */ export class AsyncPTransformClass< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > { beamName: string | (() => string); @@ -99,7 +99,7 @@ export class AsyncPTransformClass< async expandInternalAsync( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ): Promise { return this.expandAsync(input); } @@ -108,7 +108,7 @@ export class AsyncPTransformClass< /** @internal */ export class PTransformClass< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > extends AsyncPTransformClass { expand(input: InputT): OutputT { throw new Error("Method expand has not been implemented."); @@ -121,7 +121,7 @@ export class PTransformClass< expandInternal( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ): OutputT { return this.expand(input); } @@ -129,7 +129,7 @@ export class PTransformClass< async expandInternalAsync( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ): Promise { return this.expandInternal(input, pipeline, transformProto); } @@ -143,14 +143,14 @@ export class PTransformClass< */ export type AsyncPTransform< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > = | AsyncPTransformClass | ((input: InputT) => Promise) | (( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => Promise); /** @@ -179,12 +179,12 @@ export type AsyncPTransform< */ export type PTransform< InputT extends PValue, - OutputT extends PValue + OutputT extends PValue, > = | PTransformClass | ((input: InputT) => OutputT) | (( input: InputT, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => OutputT); diff --git a/sdks/typescript/src/apache_beam/transforms/utils.ts b/sdks/typescript/src/apache_beam/transforms/utils.ts index 8edcbb40e923..7523ade3c8ee 100644 --- a/sdks/typescript/src/apache_beam/transforms/utils.ts +++ b/sdks/typescript/src/apache_beam/transforms/utils.ts @@ -30,8 +30,8 @@ export function reshuffle(): PTransform, PCollection> { .apply( withName( "groupByRandomKey", - groupBy((x) => Math.random()) - ) + groupBy((x) => Math.random()), + ), ) .flatMap(withName("dropKeys", (kvs) => kvs.value)); } diff --git a/sdks/typescript/src/apache_beam/transforms/window.ts b/sdks/typescript/src/apache_beam/transforms/window.ts index f53cc9ce97f8..ba52466d136c 100644 --- a/sdks/typescript/src/apache_beam/transforms/window.ts +++ b/sdks/typescript/src/apache_beam/transforms/window.ts @@ -40,7 +40,7 @@ export interface WindowFn { export function createWindowingStrategyProto( pipeline: Pipeline, windowFn: WindowFn, - windowingStrategyBase: runnerApi.WindowingStrategy | undefined = undefined + windowingStrategyBase: runnerApi.WindowingStrategy | undefined = undefined, ): runnerApi.WindowingStrategy { let result: runnerApi.WindowingStrategy; if (windowingStrategyBase === null || windowingStrategyBase === undefined) { @@ -78,14 +78,14 @@ export function createWindowingStrategyProto( */ export function windowInto( windowFn: WindowFn, - windowingStrategyBase: runnerApi.WindowingStrategy | undefined = undefined + windowingStrategyBase: runnerApi.WindowingStrategy | undefined = undefined, ): PTransform, PCollection> { return withName( `WindowInto(${extractName(windowFn)}, ${windowingStrategyBase})`, ( input: PCollection, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => { transformProto.spec = runnerApi.FunctionSpec.create({ urn: parDo.urn, @@ -95,29 +95,29 @@ export function windowInto( urn: urns.JS_WINDOW_INTO_DOFN_URN, payload: serializeFn({ windowFn: windowFn }), }), - }) + }), ), }); const inputCoder = pipeline.context.getPCollectionCoderId(input); return pipeline.createPCollectionInternal( inputCoder, - createWindowingStrategyProto(pipeline, windowFn, windowingStrategyBase) + createWindowingStrategyProto(pipeline, windowFn, windowingStrategyBase), ); - } + }, ); } // TODO: (Cleanup) Add restrictions on moving backwards? export function assignTimestamps( - timestampFn: (T, Instant) => typeof Instant + timestampFn: (T, Instant) => typeof Instant, ): PTransform, PCollection> { return withName( `assignTimestamp(${extractName(timestampFn)})`, ( input: PCollection, pipeline: Pipeline, - transformProto: runnerApi.PTransform + transformProto: runnerApi.PTransform, ) => { transformProto.spec = runnerApi.FunctionSpec.create({ urn: parDo.urn, @@ -127,13 +127,13 @@ export function assignTimestamps( urn: urns.JS_ASSIGN_TIMESTAMPS_DOFN_URN, payload: serializeFn({ func: timestampFn }), }), - }) + }), ), }); return pipeline.createPCollectionInternal( - pipeline.context.getPCollectionCoderId(input) + pipeline.context.getPCollectionCoderId(input), ); - } + }, ); } diff --git a/sdks/typescript/src/apache_beam/transforms/windowings.ts b/sdks/typescript/src/apache_beam/transforms/windowings.ts index 782ae820d855..158d4fe02f88 100644 --- a/sdks/typescript/src/apache_beam/transforms/windowings.ts +++ b/sdks/typescript/src/apache_beam/transforms/windowings.ts @@ -47,7 +47,7 @@ export function globalWindows(): WindowFn { export function fixedWindows( sizeSeconds: number | Long, - offsetSeconds: Instant = Long.fromValue(0) + offsetSeconds: Instant = Long.fromValue(0), ): WindowFn { // TODO: (Cleanup) Use a time library? const sizeMillis = secsToMillisLong(sizeSeconds); @@ -79,7 +79,7 @@ export function fixedWindows( export function slidingWindows( sizeSeconds: number | Long, periodSeconds: number | Long, - offsetSeconds: Instant = Long.fromValue(0) + offsetSeconds: Instant = Long.fromValue(0), ): WindowFn { const sizeMillis = secsToMillisLong(sizeSeconds); const periodMillis = secsToMillisLong(periodSeconds); @@ -150,13 +150,13 @@ requireForSerialization("apache-beam/transforms/windowings", exports); requireForSerialization("apache-beam/transforms/windowings", { millisToProto }); requireForSerialization( "apache-beam/transforms/windowings", - FixedWindowsPayload + FixedWindowsPayload, ); requireForSerialization( "apache-beam/transforms/windowings", - SlidingWindowsPayload + SlidingWindowsPayload, ); requireForSerialization( "apache-beam/transforms/windowings", - SessionWindowsPayload + SessionWindowsPayload, ); diff --git a/sdks/typescript/src/apache_beam/utils/service.ts b/sdks/typescript/src/apache_beam/utils/service.ts index b53dbd626fb8..4b73e7ae53fc 100644 --- a/sdks/typescript/src/apache_beam/utils/service.ts +++ b/sdks/typescript/src/apache_beam/utils/service.ts @@ -71,7 +71,7 @@ class SubprocessServiceCache { [...this.services.values()].map((service) => { service.cached = false; return service.stop(); - }) + }), ); } } @@ -89,7 +89,7 @@ export class SubprocessService { constructor( cmd: string, args: string[], - name: string | undefined = undefined + name: string | undefined = undefined, ) { this.cmd = cmd; this.args = args; @@ -108,7 +108,7 @@ export class SubprocessService { static createCache(): SubprocessServiceCache { SubprocessService.cache = new SubprocessServiceCache( - SubprocessService.cache + SubprocessService.cache, ); return this.cache!; } @@ -129,19 +129,19 @@ export class SubprocessService { const port = (await SubprocessService.freePort()).toString(); console.debug( this.cmd, - this.args.map((arg) => arg.replace("{{PORT}}", port)) + this.args.map((arg) => arg.replace("{{PORT}}", port)), ); this.process = childProcess.spawn( this.cmd, this.args.map((arg) => arg.replace("{{PORT}}", port)), { stdio: "inherit", - } + }, ); try { console.debug( - `Waiting for ${this.name} to be available on port ${port}.` + `Waiting for ${this.name} to be available on port ${port}.`, ); await this.portReady(port, host, 10000); console.debug(`Service ${this.name} available.`); @@ -195,7 +195,7 @@ export class SubprocessService { } if (!connected) { throw new Error( - "Timed out waiting for service after " + timeoutMs + "ms." + "Timed out waiting for service after " + timeoutMs + "ms.", ); } } @@ -203,7 +203,7 @@ export class SubprocessService { export function serviceProviderFromJavaGradleTarget( gradleTarget: string, - args: string[] | undefined = undefined + args: string[] | undefined = undefined, ): () => Promise { return async () => { let jar: string; @@ -216,7 +216,7 @@ export function serviceProviderFromJavaGradleTarget( } } else { jar = await JavaJarService.cachedJar( - await JavaJarService.gradleToJar(gradleTarget) + await JavaJarService.gradleToJar(gradleTarget), ); } @@ -234,7 +234,7 @@ export class JavaJarService extends SubprocessService { constructor( jar: string, args: string[] | undefined = undefined, - name: string | undefined = undefined + name: string | undefined = undefined, ) { if (!args) { // TODO: (Extension) Should filesToStage be set at some higher level? @@ -245,13 +245,13 @@ export class JavaJarService extends SubprocessService { static async cachedJar( urlOrPath: string, - cacheDir: string = JavaJarService.JAR_CACHE + cacheDir: string = JavaJarService.JAR_CACHE, ): Promise { if (urlOrPath.match(/^https?:\/\//)) { fs.mkdirSync(cacheDir, { recursive: true }); const dest = path.join( JavaJarService.JAR_CACHE, - path.basename(urlOrPath) + path.basename(urlOrPath), ); if (fs.existsSync(dest)) { return dest; @@ -264,7 +264,7 @@ export class JavaJarService extends SubprocessService { const request = https.get(urlOrPath, function (response) { if (response.statusCode !== 200) { reject( - `Error code ${response.statusCode} when downloading ${urlOrPath}` + `Error code ${response.statusCode} when downloading ${urlOrPath}`, ); } response.pipe(fout); @@ -284,7 +284,7 @@ export class JavaJarService extends SubprocessService { static async gradleToJar( gradleTarget: string, appendix: string | undefined = undefined, - version: string = beamVersion + version: string = beamVersion, ): Promise { if (version.startsWith("0.")) { // node-ts 0.x corresponds to Beam 2.x. @@ -304,8 +304,8 @@ export class JavaJarService extends SubprocessService { artifactId, version.replace("-SNAPSHOT", ""), "SNAPSHOT", - appendix - ) + appendix, + ), ); if (version.includes("SNAPSHOT") && !projectRoot) { @@ -318,14 +318,14 @@ export class JavaJarService extends SubprocessService { } else if (version.includes("SNAPSHOT")) { throw new Error( `${localPath} not found. Please build the server with - cd ${projectRoot}; ./gradlew ${gradleTarget})` + cd ${projectRoot}; ./gradlew ${gradleTarget})`, ); } else { return JavaJarService.mavenJarUrl( artifactId, version, undefined, - appendix + appendix, ); } } @@ -336,7 +336,7 @@ export class JavaJarService extends SubprocessService { classifier: string | undefined = undefined, appendix: string | undefined = undefined, repo: string = JavaJarService.APACHE_REPOSITORY, - groupId: string = JavaJarService.BEAM_GROUP_ID + groupId: string = JavaJarService.BEAM_GROUP_ID, ): Promise { if (version == "latest") { const medatadataUrl = [ @@ -374,7 +374,7 @@ export class JavaJarService extends SubprocessService { artifactId: string, version: string, classifier: string | undefined, - appendix: string | undefined + appendix: string | undefined, ): string { return ( [artifactId, appendix, version, classifier] @@ -408,13 +408,13 @@ export class PythonService extends SubprocessService { "..", "..", "resources", - "bootstrap_beam_venv.py" + "bootstrap_beam_venv.py", ); console.debug("Invoking Python bootstrap script."); const result = childProcess.spawnSync( PythonService.whichPython(), [bootstrapScript], - { encoding: "latin1" } + { encoding: "latin1" }, ); if (result.status === 0) { console.debug(result.stdout); @@ -446,7 +446,7 @@ export class PythonService extends SubprocessService { private constructor( pythonExecutablePath: string, module: string, - args: string[] = [] + args: string[] = [], ) { super(pythonExecutablePath, ["-u", "-m", module].concat(args), module); } @@ -482,7 +482,7 @@ function getBeamProjectRoot(): string | undefined { const projectRoot = path.dirname(findGitRoot(__dirname)); if ( fs.existsSync( - path.join(projectRoot, "sdks", "typescript", "src", "apache_beam") + path.join(projectRoot, "sdks", "typescript", "src", "apache_beam"), ) ) { return projectRoot; diff --git a/sdks/typescript/src/apache_beam/utils/utils.ts b/sdks/typescript/src/apache_beam/utils/utils.ts index d89b7c365846..ba5b28d94f4b 100644 --- a/sdks/typescript/src/apache_beam/utils/utils.ts +++ b/sdks/typescript/src/apache_beam/utils/utils.ts @@ -28,6 +28,6 @@ export function camelToSnakeOptions(options: { [key: string]: any }): { return Object.fromEntries( Object.entries(options) .filter(([k, v]) => v != undefined) - .map(([k, v]) => [camelToSnake(k), v]) + .map(([k, v]) => [camelToSnake(k), v]), ); } diff --git a/sdks/typescript/src/apache_beam/values.ts b/sdks/typescript/src/apache_beam/values.ts index b484eb3eec85..a97185002c8c 100644 --- a/sdks/typescript/src/apache_beam/values.ts +++ b/sdks/typescript/src/apache_beam/values.ts @@ -36,7 +36,10 @@ export class GlobalWindow implements Window { } export class IntervalWindow implements Window { - constructor(public start: Instant, public end: Instant) {} + constructor( + public start: Instant, + public end: Instant, + ) {} maxTimestamp() { return this.end.sub(1); diff --git a/sdks/typescript/src/apache_beam/worker/data.ts b/sdks/typescript/src/apache_beam/worker/data.ts index 53a98ac4cc16..1c973d0914c8 100644 --- a/sdks/typescript/src/apache_beam/worker/data.ts +++ b/sdks/typescript/src/apache_beam/worker/data.ts @@ -41,7 +41,7 @@ export class MultiplexingDataChannel { endpoint, grpc.ChannelCredentials.createInsecure(), {}, - {} + {}, ); this.dataChannel = this.dataClient.data(metadata); this.dataChannel.on("data", async (elements) => { @@ -59,7 +59,7 @@ export class MultiplexingDataChannel { for (const timers of elements.timers) { const consumer = this.getConsumer( timers.instructionId, - timers.transformId + timers.transformId, ); try { await consumer.sendTimers(timers.timerFamilyId, timers.timers); @@ -83,7 +83,7 @@ export class MultiplexingDataChannel { async registerConsumer( bundleId: string, transformId: string, - consumer: IDataChannel + consumer: IDataChannel, ) { consumer = truncateOnErrorDataChannel(consumer); if (!this.consumers.has(bundleId)) { diff --git a/sdks/typescript/src/apache_beam/worker/external_worker_service.ts b/sdks/typescript/src/apache_beam/worker/external_worker_service.ts index 176c63f1ece4..cd29a238ec67 100644 --- a/sdks/typescript/src/apache_beam/worker/external_worker_service.ts +++ b/sdks/typescript/src/apache_beam/worker/external_worker_service.ts @@ -49,7 +49,7 @@ export class ExternalWorkerPool { const workerService: IBeamFnExternalWorkerPool = { startWorker( call: grpc.ServerUnaryCall, - callback: grpc.sendUnaryData + callback: grpc.sendUnaryData, ): void { call.on("error", (args) => { console.error("unary() got error:", args); @@ -62,8 +62,8 @@ export class ExternalWorkerPool { { controlUrl: call.request?.controlEndpoint?.url!, }, - {} - ) + {}, + ), ); callback(null, { error: "", @@ -72,7 +72,7 @@ export class ExternalWorkerPool { stopWorker( call: grpc.ServerUnaryCall, - callback: grpc.sendUnaryData + callback: grpc.sendUnaryData, ): void { this_.workers.get(call.request.workerId)?.stop(); this_.workers.delete(call.request.workerId); @@ -98,7 +98,7 @@ export class ExternalWorkerPool { this_.server.start(); resolve(this_.address); } - } + }, ); }); } diff --git a/sdks/typescript/src/apache_beam/worker/logging.ts b/sdks/typescript/src/apache_beam/worker/logging.ts index 0a7441ae88ca..14a1b542c3d2 100644 --- a/sdks/typescript/src/apache_beam/worker/logging.ts +++ b/sdks/typescript/src/apache_beam/worker/logging.ts @@ -124,10 +124,10 @@ export function createLoggingChannel(workerId: string, endpoint: string) { } startCapture(process.stdout, (out) => - logQueue.enqueue(toEntry(out, guessLogLevel(out))) + logQueue.enqueue(toEntry(out, guessLogLevel(out))), ); startCapture(process.stderr, (out) => - logQueue.enqueue(toEntry(out, guessLogLevel(out))) + logQueue.enqueue(toEntry(out, guessLogLevel(out))), ); const metadata = new grpc.Metadata(); metadata.add("worker_id", workerId); @@ -135,7 +135,7 @@ export function createLoggingChannel(workerId: string, endpoint: string) { endpoint, grpc.ChannelCredentials.createInsecure(), {}, - {} + {}, ); const channel = client.logging(metadata); diff --git a/sdks/typescript/src/apache_beam/worker/metrics.ts b/sdks/typescript/src/apache_beam/worker/metrics.ts index 1a12479df685..3c0d991ac48c 100644 --- a/sdks/typescript/src/apache_beam/worker/metrics.ts +++ b/sdks/typescript/src/apache_beam/worker/metrics.ts @@ -174,7 +174,7 @@ class Distribution implements MetricCell { metricTypes.set( "beam:metric:user:distribution_int64:v1", - (spec) => new Distribution(spec) + (spec) => new Distribution(spec), ); class ElementCount extends Counter { @@ -188,7 +188,7 @@ class ElementCount extends Counter { metricTypes.set( "beam:metric:user:element_count:v1", - (spec) => new ElementCount(spec) + (spec) => new ElementCount(spec), ); /** @@ -202,7 +202,7 @@ class ScopedMetricCell { public value: MetricCell, public name?: string, public transformId?: string, - public pcollectionId?: string + public pcollectionId?: string, ) { this.is_set = false; } @@ -243,7 +243,7 @@ export class MetricsContainer { getMetric( transformId: string | undefined, pcollectionId: string | undefined, - spec: MetricSpec + spec: MetricSpec, ): ScopedMetricCell { const key = spec.metricType + @@ -257,7 +257,7 @@ export class MetricsContainer { createMetric(spec), spec.name, transformId, - pcollectionId + pcollectionId, ); this.metrics.set(key, cell); } @@ -278,7 +278,7 @@ export class MetricsContainer { .map((metric) => [ shortIdCache.getShortId(metric), metric.value.payload(), - ]) + ]), ); } @@ -313,7 +313,7 @@ export class MetricsShortIdCache { asMonitoringInfo( id: string, - payload: Uint8Array | undefined = undefined + payload: Uint8Array | undefined = undefined, ): MonitoringInfo { const result = this.idToInfo.get(id); if (payload !== undefined) { @@ -326,7 +326,7 @@ export class MetricsShortIdCache { export function aggregateMetrics( infos: MonitoringInfo[], - urn: string + urn: string, ): Map { const cells = new Map>(); for (const info of infos) { @@ -342,7 +342,7 @@ export function aggregateMetrics( } } return new Map( - Array.from(cells.entries()).map(([name, cell]) => [name, cell.extract()]) + Array.from(cells.entries()).map(([name, cell]) => [name, cell.extract()]), ); } diff --git a/sdks/typescript/src/apache_beam/worker/operators.ts b/sdks/typescript/src/apache_beam/worker/operators.ts index 6338fc0dab58..941ad62ef2d5 100644 --- a/sdks/typescript/src/apache_beam/worker/operators.ts +++ b/sdks/typescript/src/apache_beam/worker/operators.ts @@ -80,7 +80,7 @@ export class Receiver { constructor( private operators: IOperator[], private loggingStageInfo: LoggingStageInfo, - private elementCounter: { update: (number) => void } + private elementCounter: { update: (number) => void }, ) {} receive(wvalue: WindowedValue): ProcessResult { @@ -113,7 +113,7 @@ export class OperatorContext { public getStateProvider: () => StateProvider, public getBundleId: () => string, public loggingStageInfo: LoggingStageInfo, - public metricsContainer: MetricsContainer + public metricsContainer: MetricsContainer, ) { this.pipelineContext = new PipelineContext(descriptor, ""); } @@ -121,7 +121,7 @@ export class OperatorContext { export function createOperator( transformId: string, - context: OperatorContext + context: OperatorContext, ): IOperator { const transform = context.descriptor.transforms[transformId]; // Ensure receivers are eagerly created. @@ -136,13 +136,13 @@ export function createOperator( type OperatorConstructor = ( transformId: string, transformProto: PTransform, - context: OperatorContext + context: OperatorContext, ) => IOperator; interface OperatorClass { new ( transformId: string, transformProto: PTransform, - context: OperatorContext + context: OperatorContext, ): IOperator; } @@ -156,7 +156,7 @@ export function registerOperator(urn: string, cls: OperatorClass) { export function registerOperatorConstructor( urn: string, - constructor: OperatorConstructor + constructor: OperatorConstructor, ) { operatorsByUrn.set(urn, constructor); } @@ -182,16 +182,16 @@ export class DataSourceOperator implements IOperator { constructor( transformId: string, transform: PTransform, - context: OperatorContext + context: OperatorContext, ) { const readPort = RemoteGrpcPort.fromBinary(transform.spec!.payload); this.multiplexingDataChannel = context.getDataChannel( - readPort.apiServiceDescriptor!.url + readPort.apiServiceDescriptor!.url, ); this.transformId = transformId; this.getBundleId = context.getBundleId; this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); this.coder = context.pipelineContext.getCoder(readPort.coderId); this.loggingStageInfo = context.loggingStageInfo; @@ -225,7 +225,7 @@ export class DataSourceOperator implements IOperator { } this_.lastProcessedElement += 1; const maybePromise = this_.receiver.receive( - this_.coder.decode(reader, CoderContext.needsDelimiters) + this_.coder.decode(reader, CoderContext.needsDelimiters), ); if (maybePromise !== NonPromise) { await maybePromise; @@ -249,7 +249,7 @@ export class DataSourceOperator implements IOperator { onError: function (error: Error) { endOfDataReject(error); }, - } + }, ); } @@ -258,7 +258,7 @@ export class DataSourceOperator implements IOperator { } split( - desiredSplit: fnApi.ProcessBundleSplitRequest_DesiredSplit + desiredSplit: fnApi.ProcessBundleSplitRequest_DesiredSplit, ): fnApi.ProcessBundleSplitResponse_ChannelSplit | undefined { if (!this.started) { return undefined; @@ -277,7 +277,7 @@ export class DataSourceOperator implements IOperator { // the end. var targetLastToProcessElement = Math.floor( this.lastProcessedElement + - (end - this.lastProcessedElement) * desiredSplit.fractionOfRemainder + (end - this.lastProcessedElement) * desiredSplit.fractionOfRemainder, ); // If desiredSplit.allowedSplitPoints is populated, try to find the closest // split point that's in this list. @@ -287,9 +287,9 @@ export class DataSourceOperator implements IOperator { ...Array.from(desiredSplit.allowedSplitPoints) .filter( (allowedSplitPoint) => - allowedSplitPoint >= targetLastToProcessElement + 1 + allowedSplitPoint >= targetLastToProcessElement + 1, ) - .map(Number) + .map(Number), ) - 1; } // If we were able to find a valid, meaningful split point, record it @@ -315,7 +315,7 @@ export class DataSourceOperator implements IOperator { } finally { this.multiplexingDataChannel.unregisterConsumer( this.getBundleId(), - this.transformId + this.transformId, ); this.started = false; } @@ -335,11 +335,11 @@ class DataSinkOperator implements IOperator { constructor( transformId: string, transform: PTransform, - context: OperatorContext + context: OperatorContext, ) { const writePort = RemoteGrpcPort.fromBinary(transform.spec!.payload); this.multiplexingDataChannel = context.getDataChannel( - writePort.apiServiceDescriptor!.url + writePort.apiServiceDescriptor!.url, ); this.transformId = transformId; this.getBundleId = context.getBundleId; @@ -349,7 +349,7 @@ class DataSinkOperator implements IOperator { async startBundle() { this.channel = this.multiplexingDataChannel.getSendChannel( this.getBundleId(), - this.transformId + this.transformId, ); this.buffer = new protobufjs.Writer(); } @@ -383,10 +383,10 @@ class FlattenOperator implements IOperator { constructor( public transformId: string, transform: PTransform, - context: OperatorContext + context: OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); } @@ -410,10 +410,10 @@ abstract class CombineOperator { constructor( public transformId: string, transform: PTransform, - context: OperatorContext + context: OperatorContext, ) { this.receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); const spec = runnerApi.CombinePayload.fromBinary(transform.spec!.payload); this.combineFn = deserializeFn(spec.combineFn!.payload).combineFn; @@ -431,7 +431,7 @@ export class CombinePerKeyPrecombineOperator maxKeys: number = 10000; static checkSupportsWindowing( - windowingStrategy: runnerApi.WindowingStrategy + windowingStrategy: runnerApi.WindowingStrategy, ) { if ( windowingStrategy.mergeStatus !== runnerApi.MergeStatus_Enum.NON_MERGING @@ -442,7 +442,7 @@ export class CombinePerKeyPrecombineOperator windowingStrategy.outputTime !== runnerApi.OutputTime_Enum.END_OF_WINDOW ) { throw new Error( - "Unsupported windowing output time: " + windowingStrategy + "Unsupported windowing output time: " + windowingStrategy, ); } } @@ -450,7 +450,7 @@ export class CombinePerKeyPrecombineOperator constructor( transformId: string, transform: PTransform, - context: OperatorContext + context: OperatorContext, ) { super(transformId, transform, context); const inputPc = @@ -458,13 +458,13 @@ export class CombinePerKeyPrecombineOperator onlyElement(Object.values(transform.inputs)) ]; this.keyCoder = context.pipelineContext.getCoder( - context.descriptor.coders[inputPc.coderId].componentCoderIds[0] + context.descriptor.coders[inputPc.coderId].componentCoderIds[0], ); const windowingStrategy = context.descriptor.windowingStrategies[inputPc.windowingStrategyId]; CombinePerKeyPrecombineOperator.checkSupportsWindowing(windowingStrategy); this.windowCoder = context.pipelineContext.getCoder( - windowingStrategy.windowCoderId + windowingStrategy.windowCoderId, ); } @@ -479,7 +479,7 @@ export class CombinePerKeyPrecombineOperator } this.groups.set( wkey, - this.combineFn.addInput(this.groups.get(wkey), wvalue.value.value) + this.combineFn.addInput(this.groups.get(wkey), wvalue.value.value), ); } if (this.groups.size > this.maxKeys) { @@ -512,7 +512,7 @@ export class CombinePerKeyPrecombineOperator windows: [window], timestamp: window.maxTimestamp(), pane: PaneInfoCoder.ONE_AND_ONLY_FIRING, - }) + }), ); toDelete.push(wkey); if (this.groups.size - toDelete.length <= target) { @@ -536,7 +536,7 @@ export class CombinePerKeyPrecombineOperator registerOperator( "beam:transform:combine_per_key_precombine:v1", - CombinePerKeyPrecombineOperator + CombinePerKeyPrecombineOperator, ); class CombinePerKeyMergeAccumulatorsOperator @@ -560,7 +560,7 @@ class CombinePerKeyMergeAccumulatorsOperator registerOperator( "beam:transform:combine_per_key_merge_accumulators:v1", - CombinePerKeyMergeAccumulatorsOperator + CombinePerKeyMergeAccumulatorsOperator, ); class CombinePerKeyExtractOutputsOperator @@ -584,7 +584,7 @@ class CombinePerKeyExtractOutputsOperator registerOperator( "beam:transform:combine_per_key_extract_outputs:v1", - CombinePerKeyExtractOutputsOperator + CombinePerKeyExtractOutputsOperator, ); class CombinePerKeyConvertToAccumulatorsOperator @@ -600,7 +600,7 @@ class CombinePerKeyConvertToAccumulatorsOperator key, value: this.combineFn.addInput( this.combineFn.createAccumulator(), - value + value, ), }, windows: wvalue.windows, @@ -614,7 +614,7 @@ class CombinePerKeyConvertToAccumulatorsOperator registerOperator( "beam:transform:combine_per_key_convert_to_accumulators:v1", - CombinePerKeyConvertToAccumulatorsOperator + CombinePerKeyConvertToAccumulatorsOperator, ); class CombinePerKeyCombineGroupedValuesOperator @@ -645,7 +645,7 @@ class CombinePerKeyCombineGroupedValuesOperator registerOperator( "beam:transform:combine_grouped_values:v1", - CombinePerKeyCombineGroupedValuesOperator + CombinePerKeyCombineGroupedValuesOperator, ); // ParDo operators. @@ -668,7 +668,7 @@ class GenericParDoOperator implements IOperator { context: any; }, transformProto: runnerApi.PTransform, - operatorContext: OperatorContext + operatorContext: OperatorContext, ) { this.doFn = payload.doFn; this.originalContext = payload.context; @@ -676,7 +676,7 @@ class GenericParDoOperator implements IOperator { this.sideInputInfo = createSideInputInfo( transformProto, spec, - operatorContext + operatorContext, ); this.metricsContainer = operatorContext.metricsContainer; } @@ -686,10 +686,10 @@ class GenericParDoOperator implements IOperator { this.transformId, this.sideInputInfo, this.getStateProvider, - this.metricsContainer + this.metricsContainer, ); this.augmentedContext = this.paramProvider.augmentContext( - this.originalContext + this.originalContext, ); if (this.doFn.startBundle) { this.doFn.startBundle(this.augmentedContext); @@ -709,7 +709,7 @@ class GenericParDoOperator implements IOperator { windows: [window], pane: wvalue.pane, timestamp: wvalue.timestamp, - }) + }), ); } return result.build(); @@ -719,7 +719,7 @@ class GenericParDoOperator implements IOperator { function reallyProcess(): ProcessResult { const doFnOutput = this_.doFn.process( wvalue.value, - this_.augmentedContext + this_.augmentedContext, ); if (!doFnOutput) { return NonPromise; @@ -732,7 +732,7 @@ class GenericParDoOperator implements IOperator { windows: wvalue.windows, pane: wvalue.pane, timestamp: wvalue.timestamp, - }) + }), ); } this_.paramProvider.setCurrentValue(undefined); @@ -780,7 +780,10 @@ class GenericParDoOperator implements IOperator { } class IdentityParDoOperator implements IOperator { - constructor(public transformId: string, private receiver: Receiver) {} + constructor( + public transformId: string, + private receiver: Receiver, + ) {} async startBundle() {} @@ -795,7 +798,7 @@ class SplittingDoFnOperator implements IOperator { constructor( public transformId: string, private receivers: { [key: string]: Receiver }, - private options: SplitOptions + private options: SplitOptions, ) {} async startBundle() {} @@ -805,7 +808,7 @@ class SplittingDoFnOperator implements IOperator { const keys = Object.keys(wvalue.value as object); if (this.options.exclusive && keys.length !== 1) { throw new Error( - "Multiple keys for exclusively split element: " + wvalue.value + "Multiple keys for exclusively split element: " + wvalue.value, ); } for (let tag of keys) { @@ -821,7 +824,7 @@ class SplittingDoFnOperator implements IOperator { "' for " + wvalue.value + " not in " + - this.options.knownTags + this.options.knownTags, ); } } @@ -833,7 +836,7 @@ class SplittingDoFnOperator implements IOperator { windows: wvalue.windows, timestamp: wvalue.timestamp, pane: wvalue.pane, - }) + }), ); } } @@ -847,7 +850,7 @@ class AssignWindowsParDoOperator implements IOperator { constructor( public transformId: string, private receiver: Receiver, - private windowFn: WindowFn + private windowFn: WindowFn, ) {} async startBundle() {} @@ -878,7 +881,7 @@ class AssignTimestampsParDoOperator implements IOperator { constructor( public transformId: string, private receiver: Receiver, - private func: (any, Instant) => typeof Instant + private func: (any, Instant) => typeof Instant, ) {} async startBundle() {} @@ -900,7 +903,7 @@ registerOperatorConstructor( parDo.urn, (transformId: string, transform: PTransform, context: OperatorContext) => { const receiver = context.getReceiver( - onlyElement(Object.values(transform.outputs)) + onlyElement(Object.values(transform.outputs)), ); const spec = runnerApi.ParDoPayload.fromBinary(transform.spec!.payload); // TODO: (Cleanup) Ideally we could branch on the urn itself, but some runners have a closed set of known URNs. @@ -911,24 +914,24 @@ registerOperatorConstructor( spec, deserializeFn(spec.doFn.payload!), transform, - context + context, ); } else if (spec.doFn?.urn === urns.IDENTITY_DOFN_URN) { return new IdentityParDoOperator( transformId, - context.getReceiver(onlyElement(Object.values(transform.outputs))) + context.getReceiver(onlyElement(Object.values(transform.outputs))), ); } else if (spec.doFn?.urn === urns.JS_WINDOW_INTO_DOFN_URN) { return new AssignWindowsParDoOperator( transformId, context.getReceiver(onlyElement(Object.values(transform.outputs))), - deserializeFn(spec.doFn.payload!).windowFn + deserializeFn(spec.doFn.payload!).windowFn, ); } else if (spec.doFn?.urn === urns.JS_ASSIGN_TIMESTAMPS_DOFN_URN) { return new AssignTimestampsParDoOperator( transformId, context.getReceiver(onlyElement(Object.values(transform.outputs))), - deserializeFn(spec.doFn.payload!).func + deserializeFn(spec.doFn.payload!).func, ); } else if (spec.doFn?.urn === urns.SPLITTING_JS_DOFN_URN) { return new SplittingDoFnOperator( @@ -937,14 +940,14 @@ registerOperatorConstructor( Object.entries(transform.outputs).map(([tag, pcId]) => [ tag, context.getReceiver(pcId), - ]) + ]), ), - deserializeFn(spec.doFn.payload!) + deserializeFn(spec.doFn.payload!), ); } else { throw new Error("Unknown DoFn type: " + spec); } - } + }, ); /// @@ -958,7 +961,7 @@ export function encodeToBase64(element: T, coder: Coder): string { export function decodeFromBase64(s: string, coder: Coder): T { return coder.decode( new protobufjs.Reader(Buffer.from(s, "base64")), - CoderContext.wholeStream + CoderContext.wholeStream, ); } diff --git a/sdks/typescript/src/apache_beam/worker/pardo_context.ts b/sdks/typescript/src/apache_beam/worker/pardo_context.ts index f60c103d2650..62e04751075e 100644 --- a/sdks/typescript/src/apache_beam/worker/pardo_context.ts +++ b/sdks/typescript/src/apache_beam/worker/pardo_context.ts @@ -58,7 +58,7 @@ export class ParamProviderImpl implements ParamProvider { private transformId: string, private sideInputInfo: Map, private getStateProvider: () => StateProvider, - private metricsContainer: MetricsContainer + private metricsContainer: MetricsContainer, ) {} // Avoid modifying the original object, as that could have surprising results @@ -82,8 +82,8 @@ export class ParamProviderImpl implements ParamProvider { if ((value as ParDoParam).parDoParamName === "sideInput") { this.prefetchCallbacks.push( this.prefetchSideInput( - value as SideInputParam - ) + value as SideInputParam, + ), ); } } @@ -92,7 +92,7 @@ export class ParamProviderImpl implements ParamProvider { } prefetchSideInput( - param: SideInputParam + param: SideInputParam, ): (window: Window) => operators.ProcessResult { const this_ = this; const stateProvider = this.getStateProvider(); @@ -106,7 +106,7 @@ export class ParamProviderImpl implements ParamProvider { while (reader.pos < reader.len) { yield elementCoder.decode(reader, CoderContext.needsDelimiters); } - })() + })(), ); }; return (window: Window) => { @@ -118,7 +118,7 @@ export class ParamProviderImpl implements ParamProvider { param.accessor.accessPattern, param.sideInputId, window, - windowCoder + windowCoder, ); const lookupResult = stateProvider.getState(stateKey, decode); if (lookupResult.type === "value") { @@ -133,7 +133,7 @@ export class ParamProviderImpl implements ParamProvider { } setCurrentValue( - wvalue: WindowedValue | undefined + wvalue: WindowedValue | undefined, ): operators.ProcessResult { this.wvalue = wvalue; if (wvalue === null || wvalue === undefined) { @@ -155,7 +155,7 @@ export class ParamProviderImpl implements ParamProvider { lookup(param) { if (this.wvalue === null || this.wvalue === undefined) { throw new Error( - param.parDoParamName + " not defined outside of a process() call." + param.parDoParamName + " not defined outside of a process() call.", ); } @@ -201,7 +201,7 @@ export interface SideInputInfo { export function createSideInputInfo( transformProto: runnerApi.PTransform, spec: runnerApi.ParDoPayload, - operatorContext: operators.OperatorContext + operatorContext: operators.OperatorContext, ): Map { const globalWindow = new GlobalWindow(); const sideInputInfo: Map = new Map(); @@ -216,7 +216,7 @@ export function createSideInputInfo( break; default: throw new Error( - "Unsupported window mapping fn: " + sideInput.windowMappingFn!.urn + "Unsupported window mapping fn: " + sideInput.windowMappingFn!.urn, ); } const sidePColl = @@ -225,12 +225,12 @@ export function createSideInputInfo( ]; const windowingStrategy = operatorContext.pipelineContext.getWindowingStrategy( - sidePColl.windowingStrategyId + sidePColl.windowingStrategyId, ); sideInputInfo.set(sideInputId, { elementCoder: operatorContext.pipelineContext.getCoder(sidePColl.coderId), windowCoder: operatorContext.pipelineContext.getCoder( - windowingStrategy.windowCoderId + windowingStrategy.windowCoderId, ), windowMappingFn: windowMappingFn, }); @@ -243,7 +243,7 @@ export function createStateKey( accessPattern: string, sideInputId: string, window: Window, - windowCoder: Coder + windowCoder: Coder, ): fnApi.StateKey { const writer = new protobufjs.Writer(); windowCoder.encode(window, writer, CoderContext.needsDelimiters); diff --git a/sdks/typescript/src/apache_beam/worker/state.ts b/sdks/typescript/src/apache_beam/worker/state.ts index ad8493ade208..5a340cbb64f0 100644 --- a/sdks/typescript/src/apache_beam/worker/state.ts +++ b/sdks/typescript/src/apache_beam/worker/state.ts @@ -41,7 +41,7 @@ export type MaybePromise = PromiseWrapper | ValueWrapper; export interface StateProvider { getState: ( stateKey: fnApi.StateKey, - decode: (data: Uint8Array) => T + decode: (data: Uint8Array) => T, ) => MaybePromise; } @@ -59,7 +59,7 @@ export class CachingStateProvider implements StateProvider { // serialized key, only constructing this proto when interacting with // the runner. const cacheKey = Buffer.from(fnApi.StateKey.toBinary(stateKey)).toString( - "base64" + "base64", ); if (this.cache.has(cacheKey)) { return this.cache.get(cacheKey)!; @@ -84,14 +84,14 @@ export class CachingStateProvider implements StateProvider { export class GrpcStateProvider implements StateProvider { constructor( private multiplexingChannel: MultiplexingStateChannel, - private instructionId + private instructionId, ) {} getState(stateKey: fnApi.StateKey, decode: (data: Uint8Array) => T) { return this.multiplexingChannel.getState( this.instructionId, stateKey, - decode + decode, ); } } @@ -121,7 +121,7 @@ export class MultiplexingStateChannel { endpoint, grpc.ChannelCredentials.createInsecure(), {}, - {} + {}, ); const metadata = new grpc.Metadata(); metadata.add("worker_id", workerId); @@ -145,7 +145,7 @@ export class MultiplexingStateChannel { getState( instructionId: string, stateKey: fnApi.StateKey, - decode: (data: Uint8Array) => T + decode: (data: Uint8Array) => T, ): MaybePromise { if (this.closed) { throw new Error("State stream is closed."); @@ -160,7 +160,7 @@ export class MultiplexingStateChannel { function responseCallback( resolve, reject, - prevChunks: Uint8Array[] = [] + prevChunks: Uint8Array[] = [], ): (response: fnApi.StateResponse) => void { return (response) => { if (this_.error) { @@ -189,7 +189,7 @@ export class MultiplexingStateChannel { const continueId = "continueStateRequest" + this_.idCounter++; this_.callbacks.set( continueId, - responseCallback(resolve, reject, allChunks) + responseCallback(resolve, reject, allChunks), ); this_.stateChannel.write({ id: continueId, @@ -235,7 +235,7 @@ export function Uint8ArrayConcat(chunks: Uint8Array[]) { return new Uint8Array(); } else { const fullData = new Uint8Array( - chunks.map((chunk) => chunk.length).reduce((a, b) => a + b) + chunks.map((chunk) => chunk.length).reduce((a, b) => a + b), ); let start = 0; for (const chunk of chunks) { diff --git a/sdks/typescript/src/apache_beam/worker/worker.ts b/sdks/typescript/src/apache_beam/worker/worker.ts index 0d7f2609f295..a68ae7100da2 100644 --- a/sdks/typescript/src/apache_beam/worker/worker.ts +++ b/sdks/typescript/src/apache_beam/worker/worker.ts @@ -80,7 +80,7 @@ export class Worker { constructor( private id: string, private endpoints: WorkerEndpoints, - options: Object = {} + options: Object = {}, ) { const metadata = new grpc.Metadata(); metadata.add("worker_id", this.id); @@ -88,7 +88,7 @@ export class Worker { endpoints.controlUrl, grpc.ChannelCredentials.createInsecure(), {}, - {} + {}, ); this.controlChannel = this.controlClient.control(metadata); this.controlChannel.on("data", this.handleRequest.bind(this)); @@ -152,7 +152,7 @@ export class Worker { async progress(request): Promise { const processor = this.activeBundleProcessors.get( - request.request.processBundleProgress.instructionId + request.request.processBundleProgress.instructionId, ); if (processor) { const monitoringData = processor.monitoringData(this.metricsShortIdCache); @@ -164,7 +164,7 @@ export class Worker { processBundleProgress: { monitoringInfos: Array.from(monitoringData.entries()).map( ([id, payload]) => - this.metricsShortIdCache.asMonitoringInfo(id, payload) + this.metricsShortIdCache.asMonitoringInfo(id, payload), ), monitoringData: Object.fromEntries(monitoringData.entries()), }, @@ -183,7 +183,7 @@ export class Worker { async split(request): Promise { const processor = this.activeBundleProcessors.get( - request.request.processBundleSplit.instructionId + request.request.processBundleSplit.instructionId, ); console.log(request.request.processBundleSplit, processor === undefined); if (processor) { @@ -193,7 +193,7 @@ export class Worker { response: { oneofKind: "processBundleSplit", processBundleSplit: processor.split( - request.request.processBundleSplit + request.request.processBundleSplit, ), }, }; @@ -237,11 +237,11 @@ export class Worker { } else { this.processBundleDescriptors.set( descriptorId, - maybeStripDataflowWindowedWrappings(value) + maybeStripDataflowWindowedWrappings(value), ); this.process(request); } - } + }, ); return; } @@ -260,7 +260,7 @@ export class Worker { requiresFinalization: false, monitoringInfos: Array.from(monitoringData.entries()).map( ([id, payload]) => - this.metricsShortIdCache.asMonitoringInfo(id, payload) + this.metricsShortIdCache.asMonitoringInfo(id, payload), ), monitoringData: Object.fromEntries(monitoringData.entries()), }, @@ -290,7 +290,7 @@ export class Worker { return new BundleProcessor( this.processBundleDescriptors.get(descriptorId)!, this.getDataChannel.bind(this), - this.getStateChannel.bind(this) + this.getStateChannel.bind(this), ); } } @@ -309,7 +309,7 @@ export class Worker { if (!this.dataChannels.has(endpoint)) { this.dataChannels.set( endpoint, - new MultiplexingDataChannel(endpoint, this.id) + new MultiplexingDataChannel(endpoint, this.id), ); } return this.dataChannels.get(endpoint)!; @@ -319,7 +319,7 @@ export class Worker { if (!this.stateChannels.has(endpoint)) { this.stateChannels.set( endpoint, - new MultiplexingStateChannel(endpoint, this.id) + new MultiplexingStateChannel(endpoint, this.id), ); } return this.stateChannels.get(endpoint)!; @@ -348,7 +348,7 @@ export class BundleProcessor { descriptor: ProcessBundleDescriptor, getDataChannel: (string) => MultiplexingDataChannel, getStateChannel: ((string) => MultiplexingStateChannel) | StateProvider, - root_urns = ["beam:runner:source:v1"] + root_urns = ["beam:runner:source:v1"], ) { this.descriptor = descriptor; this.getDataChannel = getDataChannel; @@ -370,7 +370,7 @@ export class BundleProcessor { consumers.get(pcollectionId)!.push(transformId); }); } - } + }, ); function getReceiver(pcollectionId: string): Receiver { @@ -380,8 +380,8 @@ export class BundleProcessor { new Receiver( (consumers.get(pcollectionId) || []).map(getOperator), this_.loggingStageInfo, - this_.metricsContainer.elementCountMetric(pcollectionId) - ) + this_.metricsContainer.elementCountMetric(pcollectionId), + ), ); } return this_.receivers.get(pcollectionId)!; @@ -400,9 +400,9 @@ export class BundleProcessor { this_.getStateProvider.bind(this_), this_.getBundleId.bind(this_), this_.loggingStageInfo, - this_.metricsContainer - ) - ) + this_.metricsContainer, + ), + ), ); creationOrderedOperators.push(this_.operators.get(transformId)!); } @@ -414,7 +414,7 @@ export class BundleProcessor { if (root_urns.includes(transform?.spec?.urn!)) { getOperator(transformId); } - } + }, ); this.topologicallyOrderedOperators = creationOrderedOperators.reverse(); } @@ -425,10 +425,10 @@ export class BundleProcessor { this.stateProvider = new CachingStateProvider( new GrpcStateProvider( this.getStateChannel( - this.descriptor.stateApiServiceDescriptor!.url + this.descriptor.stateApiServiceDescriptor!.url, ), - this.getBundleId() - ) + this.getBundleId(), + ), ); } else { this.stateProvider = this.getStateChannel; @@ -474,7 +474,7 @@ export class BundleProcessor { split(splitRequest: ProcessBundleSplitRequest): ProcessBundleSplitResponse { const root = this.topologicallyOrderedOperators[0] as DataSourceOperator; for (const [target, desiredSplit] of Object.entries( - splitRequest.desiredSplits + splitRequest.desiredSplits, )) { if (target == root.transformId) { const channelSplit = root.split(desiredSplit); @@ -510,7 +510,7 @@ function isPrimitive(transform: PTransform): boolean { } function maybeStripDataflowWindowedWrappings( - descriptor: ProcessBundleDescriptor + descriptor: ProcessBundleDescriptor, ): ProcessBundleDescriptor { for (const pcoll of Object.values(descriptor.pcollections)) { const coder = descriptor.coders[pcoll.coderId]; diff --git a/sdks/typescript/src/apache_beam/worker/worker_main.ts b/sdks/typescript/src/apache_beam/worker/worker_main.ts index e70e056abb63..bc6c60c2efa6 100644 --- a/sdks/typescript/src/apache_beam/worker/worker_main.ts +++ b/sdks/typescript/src/apache_beam/worker/worker_main.ts @@ -48,7 +48,7 @@ async function main() { console.error( `**ERROR** Unable to require module '${m}' used in requireForSerialization: - please ensure that it is available in the package exports.` + please ensure that it is available in the package exports.`, ); // Explicitly exit the process to avoid the error getting swallowed // by a long traceback. @@ -62,7 +62,7 @@ async function main() { { controlUrl: argv.control_endpoint, }, - options + options, ); if (pushLogs) { await pushLogs(); From 4cf1b15024d19258c0793f56e5cf65139d8e33d9 Mon Sep 17 00:00:00 2001 From: liferoad Date: Mon, 8 Jan 2024 08:36:24 -0500 Subject: [PATCH 221/224] Updated the doc with Security Fixes for 2.53.0 (#29941) --- CHANGES.md | 8 ++++---- website/www/site/content/en/blog/beam-2.53.0.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7a40f9af4df5..f59ee274343c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -113,10 +113,6 @@ * Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)). * Python 3.11 images now include google-cloud-profiler ([#29561](https://github.com/apache/beam/pull/29651)). -## Breaking Changes - -* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) - ## Deprecations * Euphoria DSL is deprecated and will be removed in a future release (not before 2.56.0) ([#29451](https://github.com/apache/beam/issues/29451)) @@ -126,6 +122,10 @@ * (Python) Fixed sporadic crashes in streaming pipelines that affected some users of 2.47.0 and newer SDKs ([#27330](https://github.com/apache/beam/issues/27330)). * (Python) Fixed a bug that caused MLTransform to drop identical elements in the output PCollection ([#29600](https://github.com/apache/beam/issues/29600)). +## Security Fixes + +* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) + # [2.52.0] - 2023-11-17 ## Highlights diff --git a/website/www/site/content/en/blog/beam-2.53.0.md b/website/www/site/content/en/blog/beam-2.53.0.md index 864f1870504f..14c55956bc13 100644 --- a/website/www/site/content/en/blog/beam-2.53.0.md +++ b/website/www/site/content/en/blog/beam-2.53.0.md @@ -49,10 +49,6 @@ For more information on changes in 2.53.0, check out the [detailed release notes * Framework for adding Error Handlers to composite transforms added in Java ([#29164](https://github.com/apache/beam/pull/29164)). * Python 3.11 images now include google-cloud-profiler ([#29561](https://github.com/apache/beam/pull/29651)). -## Breaking Changes - -* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) - ## Deprecations * Euphoria DSL is deprecated and will be removed in a future release (not before 2.56.0) ([#29451](https://github.com/apache/beam/issues/29451)) @@ -62,6 +58,10 @@ For more information on changes in 2.53.0, check out the [detailed release notes * (Python) Fixed sporadic crashes in streaming pipelines that affected some users of 2.47.0 and newer SDKs ([#27330](https://github.com/apache/beam/issues/27330)). * (Python) Fixed a bug that caused MLTransform to drop identical elements in the output PCollection ([#29600](https://github.com/apache/beam/issues/29600)). +## Security Fixes + +* Upgraded to go 1.21.5 to build, fixing [CVE-2023-45285](https://security-tracker.debian.org/tracker/CVE-2023-45285) and [CVE-2023-39326](https://security-tracker.debian.org/tracker/CVE-2023-39326) + ## List of Contributors According to git shortlog, the following people contributed to the 2.53.0 release. Thank you to all contributors! From 66f98a0670c5da77e5aca3316fe87c754cbf02c4 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Mon, 8 Jan 2024 11:52:28 -0500 Subject: [PATCH 222/224] Update finalize_release.yml (#29918) --- .github/workflows/finalize_release.yml | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/.github/workflows/finalize_release.yml b/.github/workflows/finalize_release.yml index d96b59ee7cfe..f3eb544139c7 100644 --- a/.github/workflows/finalize_release.yml +++ b/.github/workflows/finalize_release.yml @@ -13,11 +13,8 @@ on: RC: description: Integer RC version for the release that we'd like to finalize (e.g. 3 for RC3) required: true - PYPI_USER: - description: PyPi username to perform the PyPi upload with - required: false - PYPI_PASSWORD: - description: PyPi password to perform the PyPi upload with + PYPI_API_TOKEN: + description: PyPi API token to perform the PyPi upload with required: false PUSH_DOCKER_ARTIFACTS: description: Whether to push SDK docker images to docker hub Apache organization. Should be yes unless you've already completed this step. @@ -98,12 +95,8 @@ jobs: echo "::add-mask::$PYPI_PASSWORD" - name: Validate PyPi id/password run: | - if [ "${{ github.event.inputs.PYPI_USER }}" == "" ] - then - echo "Must provide a PyPi username to publish artifacts to PyPi" - exit 1 - fi - if [ "${{ github.event.inputs.PYPI_PASSWORD }}" == "" ] + echo "::add-mask::${{ github.event.inputs.PYPI_API_TOKEN }}" + if [ "${{ github.event.inputs.PYPI_API_TOKEN }}" == "" ] then echo "Must provide a PyPi password to publish artifacts to PyPi" exit 1 @@ -125,7 +118,7 @@ jobs: cd "dist.apache.org/repos/dist/dev/beam/${RELEASE}/python/" echo "Will upload the following files to PyPI:" ls - twine upload * -u "${{ github.event.inputs.PYPI_USER }}" -p "${{ github.event.inputs.PYPI_PASSWORD }}" + twine upload * -u __token__ -p "${{ github.event.inputs.PYPI_API_TOKEN }}" push_git_tags: if: ${{github.event.inputs.TAG_RELEASE == 'yes'}} From 75cfbee1591b99ff02d0a6a19631199e719b44fa Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 8 Jan 2024 17:10:00 +0000 Subject: [PATCH 223/224] Update MLTransform docs (#29910) * Update MLTransform docs * Update MLTransform docs * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update preprocess-data.md * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Move embeddings content higher up * Fix website checks * Revert "Fix website checks" This reverts commit 229f7eddf4ce8b2847ab248c0fadb20fa4f898d1. * Fix website checks --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- .../en/documentation/ml/preprocess-data.md | 75 ++++++------------- 1 file changed, 24 insertions(+), 51 deletions(-) diff --git a/website/www/site/content/en/documentation/ml/preprocess-data.md b/website/www/site/content/en/documentation/ml/preprocess-data.md index 2b291b9c75a5..1365926d3cc1 100644 --- a/website/www/site/content/en/documentation/ml/preprocess-data.md +++ b/website/www/site/content/en/documentation/ml/preprocess-data.md @@ -23,16 +23,11 @@ preprocessing data for training and inference. The `MLTransform` class wraps the various transforms in one class, simplifying your workflow. For a full list of available transforms, see the [Transforms](#transforms) section on this page. -The set of transforms currently available in the `MLTransform` class come from -the TensorFlow Transforms (TFT) library. TFT offers specialized processing -modules for machine learning tasks. - ## Why use MLTransform {#use-mltransform} - With `MLTransform`, you can use the same preprocessing steps for both training and inference, which ensures consistent results. -- Use `MLTransform` to transform a single example or a batch of - examples. +- Generate [embeddings](https://en.wikipedia.org/wiki/Embedding) on text data using large language models (LLMs). - `MLTransform` can do a full pass on the dataset, which is useful when you need to transform a single element only after analyzing the entire dataset. For example, with `MLTransform`, you can complete the following tasks: @@ -45,18 +40,33 @@ modules for machine learning tasks. - Count the occurrences of words in all the documents to calculate [TF-IDF](https://en.wikipedia.org/wiki/Tf%E2%80%93idf) weights. - ## Support and limitations {#support} -- Available in the Apache Beam Python SDK versions 2.50.0 and later. -- Supports Python 3.8 and 3.9. +- Available in the Apache Beam Python SDK versions 2.53.0 and later. +- Supports Python 3.8, 3.9, and 3.10. - Only available for pipelines that use [default windows](/documentation/programming-guide/#single-global-window). -- Only supports one-to-one transform mapping on a single element. ## Transforms {#transforms} -You can use `MLTransform` to perform the following data processing transforms. -For information about the transforms, see +You can use `MLTransform` to generate text embeddings and to perform various data processing transforms. + +### Text embedding transforms + +You can use `MLTranform` to generate embeddings that you can use to push data into vector databases or to run inference. + +{{< table >}} +| Transform name | Description | +| ------- | ---------------| +| SentenceTransformerEmbeddings | Uses the Hugging Face [`sentence-transformers`](https://huggingface.co/sentence-transformers) models to generate text embeddings. +| VertexAITextEmbeddings | Uses models from the [the Vertex AI text-embeddings API](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings) to generate text embeddings. +{{< /table >}} + + +### Data processing transforms that use TFT + +The following set of transforms available in the `MLTransform` class come from +the TensorFlow Transforms (TFT) library. TFT offers specialized processing +modules for machine learning tasks. For information about these transforms, see [Module:tft](https://www.tensorflow.org/tfx/transform/api_docs/python/tft) in the TensorFlow documentation. @@ -73,18 +83,10 @@ TensorFlow documentation. | TFIDF | See [`tft.tfidf`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/tfidf) in the TensorFlow documentation. |: {{< /table >}} -Apply the transforms on either single or multiple columns passed as a -`dict` on structured data. Keys are column names and values are lists containing -each column's data. - ## I/O requirements {#io} -- Input to the `MLTransform` class must be in one of the following formats: - - A `dict` of `str` - - Primitive types - - List of primitive types - - NumPy arrays -- `MLTransform` outputs a Beam `Row` object with NumPy arrays. +- Input to the `MLTransform` class must be a dictionary. +- `MLTransform` outputs a Beam `Row` object with transformed elements. - The output `PCollection` is a schema `PCollection`. The output schema contains the transformed columns. @@ -197,32 +199,3 @@ Replace the following values: For more examples, see [MLTransform for data processing](/documentation/transforms/python/elementwise/mltransform) in the [transform catalog](/documentation/transforms/python/overview/). - -### ScaleTo01 example {#scaleto01} - -This example demonstrates how to use `MLTransform` to normalize your data -between 0 and 1 by using the minimum and maximum values from your entire -dataset. `MLTransform` uses the `ScaleTo01` transformation. - -Use the following snippet to apply `ScaleTo01` on column `x` of the input -data. - -``` -data_pcoll | MLTransform(write_artifact_location=).with_transform(ScaleTo01(columns=['x'])) -``` - -The `ScaleTo01` transformation produces two artifacts: the `min` and the `max` -of the entire dataset. For more information, see the -[Artifacts](#artifacts) section on this page. - -## Metrics {#metrics} - -When you use MLTransform, the following metrics are available. - -{{< table >}} -| Metric | Description | -| ------- | ---------------| -| Data throughput | The number of records processed per second. This metric indicates the processing capacity of the pipeline for `beam.MLTransform.` | -| Memory usage | The number of records processed per second. This metric indicates the processing capacity of the pipeline for `beam.MLTransform`. | -| Counters | Tracks the number of elements processed. Each `MLTransform` has a counter. |: -{{< /table >}} \ No newline at end of file From c9e036e40e4f1d21f33ce6829fdf919c934da7de Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Mon, 8 Jan 2024 15:52:31 -0500 Subject: [PATCH 224/224] [Python BQ] Retry get_table for quota errors (#28820) * retry get_table on quota errors * add tests * only retry on transient reasons --- .../apache_beam/io/gcp/bigquery_test.py | 201 +++++++++++++++++- .../apache_beam/io/gcp/bigquery_tools.py | 2 +- sdks/python/apache_beam/utils/retry.py | 33 ++- 3 files changed, 229 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 035edffc03f7..e53204a5ebc6 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -43,6 +43,7 @@ from apache_beam.internal import pickler from apache_beam.internal.gcp.json_value import to_json_value from apache_beam.io.filebasedsink_test import _TestCaseWithTempDirCleanUp +from apache_beam.io.filesystems import FileSystems from apache_beam.io.gcp import bigquery as beam_bq from apache_beam.io.gcp import bigquery_tools from apache_beam.io.gcp.bigquery import ReadFromBigQuery @@ -82,11 +83,13 @@ try: from apache_beam.io.gcp.internal.clients.bigquery import bigquery_v2_client from apitools.base.py.exceptions import HttpError + from apitools.base.py.exceptions import HttpForbiddenError from google.cloud import bigquery as gcp_bigquery from google.api_core import exceptions except ImportError: gcp_bigquery = None HttpError = None + HttpForbiddenError = None exceptions = None # pylint: enable=wrong-import-order, wrong-import-position @@ -323,7 +326,9 @@ def test_repeatable_field_is_properly_converted(self): self.assertEqual(expected_row, actual) -@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') +@unittest.skipIf( + HttpError is None or HttpForbiddenError is None, + 'GCP dependencies are not installed') class TestReadFromBigQuery(unittest.TestCase): @classmethod def setUpClass(cls): @@ -454,6 +459,200 @@ def test_create_temp_dataset_exception(self, exception_type, error_message): mock_insert.assert_called() self.assertIn(error_message, exc.exception.args[0]) + @parameterized.expand([ + # first attempt returns a Http 500 blank error and retries + # second attempt returns a Http 408 blank error and retries, + # third attempt passes + param( + responses=[ + HttpForbiddenError( + response={'status': 500}, content="something", url="") + if HttpForbiddenError else None, + HttpForbiddenError( + response={'status': 408}, content="blank", url="") + if HttpForbiddenError else None + ], + expected_retries=2), + # first attempts returns a 403 rateLimitExceeded error + # second attempt returns a 429 blank error + # third attempt returns a Http 403 rateLimitExceeded error + # fourth attempt passes + param( + responses=[ + exceptions.Forbidden( + "some message", + errors=({ + "message": "transient", "reason": "rateLimitExceeded" + }, )) if exceptions else None, + exceptions.ResourceExhausted("some message") + if exceptions else None, + HttpForbiddenError( + response={'status': 403}, + content={ + "error": { + "errors": [{ + "message": "transient", + "reason": "rateLimitExceeded" + }] + } + }, + url="") if HttpForbiddenError else None, + ], + expected_retries=3), + ]) + def test_get_table_transient_exception(self, responses, expected_retries): + class DummyTable: + class DummySchema: + fields = [] + + numBytes = 5 + schema = DummySchema() + + with mock.patch('time.sleep'), \ + mock.patch.object(bigquery_v2_client.BigqueryV2.TablesService, + 'Get') as mock_get_table, \ + mock.patch.object(BigQueryWrapper, + 'wait_for_bq_job'), \ + mock.patch.object(BigQueryWrapper, + 'perform_extract_job'), \ + mock.patch.object(FileSystems, + 'match'), \ + mock.patch.object(FileSystems, + 'delete'), \ + beam.Pipeline() as p: + call_counter = 0 + + def store_callback(unused_request): + nonlocal call_counter + if call_counter < len(responses): + exception = responses[call_counter] + call_counter += 1 + raise exception + else: + call_counter += 1 + return DummyTable() + + mock_get_table.side_effect = store_callback + _ = p | beam.io.ReadFromBigQuery( + table="project.dataset.table", gcs_location="gs://some_bucket") + + # ReadFromBigQuery export mode calls get_table() twice. Once to get + # metadata (numBytes), and once to retrieve the table's schema + # Any additional calls are retries + self.assertEqual(expected_retries, mock_get_table.call_count - 2) + + @parameterized.expand([ + # first attempt returns a Http 429 with transient reason and retries + # second attempt returns a Http 403 with non-transient reason and fails + param( + responses=[ + HttpForbiddenError( + response={'status': 429}, + content={ + "error": { + "errors": [{ + "message": "transient", + "reason": "rateLimitExceeded" + }] + } + }, + url="") if HttpForbiddenError else None, + HttpForbiddenError( + response={'status': 403}, + content={ + "error": { + "errors": [{ + "message": "transient", "reason": "accessDenied" + }] + } + }, + url="") if HttpForbiddenError else None + ], + expected_retries=1), + # first attempt returns a transient 403 error and retries + # second attempt returns a 403 error with bad contents and fails + param( + responses=[ + HttpForbiddenError( + response={'status': 403}, + content={ + "error": { + "errors": [{ + "message": "transient", + "reason": "rateLimitExceeded" + }] + } + }, + url="") if HttpForbiddenError else None, + HttpError( + response={'status': 403}, content="bad contents", url="") + if HttpError else None + ], + expected_retries=1), + # first attempt returns a transient 403 error and retries + # second attempt returns a 429 error and retries + # third attempt returns a 403 with non-transient reason and fails + param( + responses=[ + exceptions.Forbidden( + "some error", + errors=({ + "message": "transient", "reason": "rateLimitExceeded" + }, )) if exceptions else None, + exceptions.ResourceExhausted("some transient error") + if exceptions else None, + exceptions.Forbidden( + "some error", + errors=({ + "message": "transient", "reason": "accessDenied" + }, )) if exceptions else None, + ], + expected_retries=2), + ]) + def test_get_table_non_transient_exception(self, responses, expected_retries): + class DummyTable: + class DummySchema: + fields = [] + + numBytes = 5 + schema = DummySchema() + + with mock.patch('time.sleep'), \ + mock.patch.object(bigquery_v2_client.BigqueryV2.TablesService, + 'Get') as mock_get_table, \ + mock.patch.object(BigQueryWrapper, + 'wait_for_bq_job'), \ + mock.patch.object(BigQueryWrapper, + 'perform_extract_job'), \ + mock.patch.object(FileSystems, + 'match'), \ + mock.patch.object(FileSystems, + 'delete'), \ + self.assertRaises(Exception), \ + beam.Pipeline() as p: + call_counter = 0 + + def store_callback(unused_request): + nonlocal call_counter + if call_counter < len(responses): + exception = responses[call_counter] + call_counter += 1 + raise exception + else: + call_counter += 1 + return DummyTable() + + mock_get_table.side_effect = store_callback + _ = p | beam.io.ReadFromBigQuery( + table="project.dataset.table", gcs_location="gs://some_bucket") + + # ReadFromBigQuery export mode calls get_table() twice. Once to get + # metadata (numBytes), and once to retrieve the table's schema + # However, the second call is never reached because this test will always + # fail before it does so + # After the first call, any additional calls are retries + self.assertEqual(expected_retries, mock_get_table.call_count - 1) + @parameterized.expand([ param( exception_type=exceptions.BadRequest if exceptions else None, diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index 41ef57b5b27c..fab5a8611591 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -751,7 +751,7 @@ def _insert_all_rows( @retry.with_exponential_backoff( num_retries=MAX_RETRIES, - retry_filter=retry.retry_on_server_errors_and_timeout_filter) + retry_filter=retry.retry_on_server_errors_timeout_or_quota_issues_filter) def get_table(self, project_id, dataset_id, table_id): """Lookup a table's metadata object. diff --git a/sdks/python/apache_beam/utils/retry.py b/sdks/python/apache_beam/utils/retry.py index 6eed2900b9a4..485fc9d627e9 100644 --- a/sdks/python/apache_beam/utils/retry.py +++ b/sdks/python/apache_beam/utils/retry.py @@ -28,6 +28,7 @@ # pytype: skip-file import functools +import json import logging import random import sys @@ -57,6 +58,7 @@ # pylint: enable=wrong-import-order, wrong-import-position _LOGGER = logging.getLogger(__name__) +_RETRYABLE_REASONS = ["rateLimitExceeded", "internalError", "backendError"] class PermanentException(Exception): @@ -166,17 +168,38 @@ def retry_on_server_errors_and_timeout_filter(exception): def retry_on_server_errors_timeout_or_quota_issues_filter(exception): - """Retry on server, timeout and 403 errors. + """Retry on server, timeout, 429, and some 403 errors. - 403 errors can be accessDenied, billingNotEnabled, and also quotaExceeded, - rateLimitExceeded.""" + 403 errors from BigQuery include both non-transient (accessDenied, + billingNotEnabled) and transient errors (rateLimitExceeded). + Only retry transient errors.""" if HttpError is not None and isinstance(exception, HttpError): - if exception.status_code == 403: + if exception.status_code == 429: return True + if exception.status_code == 403: + try: + # attempt to extract the reason and check if it's retryable + content = exception.content + if not isinstance(content, dict): + content = json.loads(exception.content) + return content["error"]["errors"][0]["reason"] in _RETRYABLE_REASONS + except (KeyError, IndexError, TypeError) as e: + _LOGGER.warning( + "Could not determine if HttpError is transient. " + "Will not retry: %s", + e) + return False if GoogleAPICallError is not None and isinstance(exception, GoogleAPICallError): - if exception.code == 403: + if exception.code == 429: return True + if exception.code == 403: + if not hasattr(exception, "errors") or len(exception.errors) == 0: + # default to not retrying + return False + + reason = exception.errors[0]["reason"] + return reason in _RETRYABLE_REASONS if S3ClientError is not None and isinstance(exception, S3ClientError): if exception.code == 403: return True

    2. hiTvm27V zQ1Bi1J^fEls0oh}?vsj3oj%yr)pTc_6VP0~yv`sqgm5r+NEAMq^pi%hZH|2+cK=x* zj(EsP?Zcg@^M#jtglI(!eXUKXEBmt`30Icg4gBV2$LZ|^;W_I!S4-~q-1zYIb0ZuK8b{pgD{+mPO$LHQrcDI20+9$-6= zPwuS7q+q3x2GAeqUMG>6783-(%^sXwY-L`GpTNIHTueSnjoNPSGomRwoOoD_$oh3r zlX5JSB!01F^jNd39;NXI!$RkKzHh##L~i8=FXZ&B^y>A%wCCt|C9zqbQ)C>n1{Q@C5g{Dxh8-LqKU3VF6*bBY=oOrZs z`>yeux4+B8az^MQS?Hval*39iK-*&y4dCy^PC}A5L|t7(7w~66yYrMBsnt&+8ollN zTE#qHqMxiGVG^dCD6b}x`r9f)A$;;r|~q=+ACybC#{d@A$!QB*F)x(ptLm`Mkx36@GEqxcv_YBZqO zNSgFij7|q~E|GhLml)eCH+*Jr0`rAFzR4^AG^EI6h4 z^S9s(O?DC9;JE6V~p+jOf{dp2ycl*GAT+y1kuH^pe=3jFAbXl8z;#3v8vvjg^zN zmf@Fem(`Hb%Y9(AVP$4jV)fC=(u$($Q5AX>iBPJ{oPwI5hFeUTiU;jfg(pN*puy(@TuPumY|T)sgd&2PsYp<_eL|3>-h1 z<(W0D;PxhRviGL6Wk=-W4%1tlTP9ivYMU?-F;(MfSO==-jXnK>^+m|G{se9N$+p+_ z`zij)erw_3r|T1I@7DI5S2k?Mc=HZ}BL|t*ZTO;ug23;6Qs(mHN2*4b*H+dNHg|?> zhqp(E2OY=d$5M+m3SD27s*(-Y46d%NuCt6m^L@pyhq2?H7C(La6eDm1vyOeKsYN+J z1leCSxy3GE6D9LLLEzKCxzrsOYekf&n5kJi zPp8=QR`ylvn|hC(HAQ?;2<;U}MM*WUMyV#AHk#I+Zg6?kfY1=)GU_tE6>{?8T=(4h zB#}fJ^+AwLkX(>_WI;}T&VYTo@j2@^*4@|HuSdW0rE#WdDoQF=DLzO$PR~h`Pa{e* zz^Wn4;PzVItN0G7bE{pdv$3z^5Hpu(4UT=NK&jurS9_q#pjTS=a#n2Cte&gB$-&hL zzqZsBvMvgHvdq?>Gr3rlpP1iT(N{6u-Ni6l)KwHv*d1=lY?5*K%njAO|43-zaf)$S#PZw_nFB>QJaXI&rPl2R`x`nL;C`o!^ z{Bk}|HxFxqIn@R5wpk4Mr&`skWSS=rKRG? z&g5?6pm1>h;pYcay7Zi&s0^Vqqt7otvsST1d`L*E=K2D9(UBByT;Jvq&P3vAoVH!t zx!4!QLzu~PwRUIf&d$5b^>;@fj!4!GYT0J2>+{z1 zZGN&(ayNUXDcqE)f1?kpqoiZ!VflOO_oWWs9J!Yo`y)DM6Q(m0V{7;gn1ltdqh+K& zFj&#CG9~E_FuOcX4CSXed!fr)rtyewl3p@*F*o9U{hl;f&5Y(;@+7MRQc%A3J$Fg= zoW@!SPr)vIfz`C-L;jDCKOhP}@((_7D{*9Udrk$c7q6$9k#n${arRfH@w%Fg7k|~7 zHdi%U?YnAf%`?&JoaUu-Oj$|kN#}wMvb=_jBv>WL$?>ni`=(jRGFj4~)y=joDR-e= zr>6K_dR+`%5XL2@#=Xs81C8q`2(9DGX`}uhn~B(o!3i^(gkPP`*|$x9bdxfXRW1NsC8&+dor~dFB;!a6kde)s>z_;z}3|I z=UF}GGomgYuPNFYH%HZpPiI!C8=<|ey^^V!HnMXLCp;@4E}e+#1kh!@Lgg1n&x?}; z!qSu!k~V(4@$9PdshxCVSN%EM#-g&mLc3wR%h_emX41)wRnKAA+ys7Isk^YF?=0db zvKs0-YWGIW8QpOc<|Jl%L}A2s5Cs+&51gAsV9U#G`r#pEy!3G-vyc=oH%#+-e__#^ zrj^D+(Jb9-MBpR5uW&HGq^h;5GCfkjd*{VkvZ}e3c~s^hzbmk_aB=)3d_raq{bJCu zh2UvJ`!4FV!1M--yPj3!mUmsDx5;@K1EPkaJ|@^>g>ZH1$L8SpG>B@pc9N|+=kVF} zN`;Pljs7fi{UyAwxZz!cDXelwZH5PIa2a$-*1#{wA0>q5GjPT_Kc6n$D9u7DPv*Ox za{lftccE}{IK{lAKhi14#lS~s#eE!OKdfIc&)4c&>v(8q*&l**ZM97@JAEv7@cH`q z8&4ppy2;shdIo>m^;++$p?|kq=!wt7RSbN8EB}L%lCaX5*jf2m(dGCjPz-eEWEBPr zt9w+kpn(v8+uL*J=P@@zK_SAxJUpCT%Kb|9v4p#s8u?!4*Msl^+6;t|mTOoo>8a~5 zVsi%5-s#bT(NtJ6Z^uNfRG}=R8!VWK>>jtI7UObi}(lu88|`&z7Y@!5$^tWjDR4ANc68` z6-3&9-h+gI5M+sP=b!gz1AlM6qJbZv^*{e2#|9#x0-qiMKku@T{_Adxr7YzCIz}}D zt|5r3ip$CZe^regOik?^EkI6^51?bfiFXu-@ThNo5M@=U_7D(|CM{oSI%z7r z;4=o*}l2y2SLD%4>+_nb$U(iW@}^T$mb?V@!K7I!12v#77Fs;ZgH{} zq|j7QCKm@en38idvof<%2%(dclM6VQnDMDdNd2=p@JW!u!pZ3k9}A1Ct1GiB2Q$dQ zoP~{-mzRZ=orRs93AlsF(cRAJwHuS2BjrCj`LBK?OdX9KEZ;a;g6zm|`h9H#a&{7= zptu?6Utj-tPE$9_-$$}@{O7cQ39{T=VPRutW%;vaPL^i>OS79R|7iBxy#6tqz)fd- z%9d`XHkuNaw!l&Yk0!*y%E>A4+c^K@((h0DkEUvlrVipDTcD+r(C?e|&&Ibe{;}b2 zV`}|ACfjpf*4ty=x^%1Q%_R6#OdUZs&NqvwW@qUn#4f<{XQ%(zO6&J#LToql`>oOK zvwv)%`KK*zpZ#MCB?n7jAzt4+nh@JRN4S0NpZ5!}+-&|WVfcrn{dN`*H6e5XmVb$@ z5W48I3PuD35d>KY(N}JWTeGNXkJYN1=ZE7ApEwXYL{pK=;USjA4AZnhpa zrZKvw-Y?7m%l4t420~F-NEe%sV&$VUeB!$a9?Q=qKBc0Ob#N%BB7n*wF2+5M z79lqb&w`$#PC>}NXK{)mAl|_i`TM`J-l9{f4Y;#0{ZeaSJo-umL^$r*2`lff_23^$ zFCyY2*~HO<@ryGM!@!O<_y~FsC;s=Rf77C%0O`(>#t8rS#$Q>$|DP;DZYy5{3BQYs zm0W1tx0!A6c_ob?A68Ytq-GjF|(m$Z%aR(pbE7~(ON*OzR3I_IAGu)aFqTvTP7kxMQ8R7 z`6Vm`Q~0s7(x>wzxJB@<&0o3d&O%4Df!91?YN@HOjZgDwwRexl^p(G{l(kNvR}pFj z(XTIU9!{{JxKCaK=db-IMVxP$*^0{+hg{GSQ=tu6Tz5J3OA8FLs_ZKRN`l%9&!?x%*@ zg4O5myLq2yis<*7?H~Son;$O+_%=D`h9t&YrZMDUM}L1}JcC9T*!w)!tT!b#IhoFR zb4;Pf@F-M-7tLn2UTy4)4hJ&#aTXW#k2r2u&`wD*C$?euSCMe|sOf2#oO|1c!!i18Xdj87oQ9 zvzp+}m#{nBHWTC#gj}&F+tc!?+_t$OC|ISzWjlZ?yaCH=}u!(=WX__C;PWvACEqncFs{SgS&c+jfe| z-QE2W+MQg>;ZH+No~{I}h9V)m&a6*9>@T)=hZAu0SWi~lrW%7;8u#83dv29Q@>!do z9j&ul4pC`+HT37@Dbk0OHXg{;olIG8tJ(ZVP*Q~J1~(8wy_kOwg0s%h8Fi{(FxRh9 z*rU=V<_Dwr`<~7RvD?i&pRBbvo+vaiG3nqM*QNA@cN@@4d~(ne$V$j#%8`nQO$CAL zDrY}zcwvHlcj{J`0v-qjpV>IA5B+FFrIZQ}rW| zS#Q@q(yHICp6S}GDmNg3(SqAnTNVUy{QR3;RnkOSMb&D)BVLfof^A`=L7E_^Qp^*f^p|}8g{wy?Iiexb@#wR z(Cw+JxVLYSENAPh@>7+B;H|3pxHsLE=rzd6$%P)R4Suh|jQ{s4|Ie$+f;Xrv1D0O& zJFYHhL-O(8Z6|R}6Sz8EjLpdK88B@NC!lm(>DedI2>8VBv_@_{UV@8x;eyXVdku&X zZDQS~7yVEw%8XFiR(}#3r4cI_%(jenj;~&UMaXrB;X)lA@y06_4o;i`tHkI zLoZAx&{%Quq;Ku&{K+&Zu`iupzEr=-t9_tLA(b26_p)7Rqaf#7Mn39A+vPd*nps2N zJ?{>ZfAFdU_J0JSKj#C0ar6-EIQsiijA|MJ8<0^Xyqf0NMV+M=&;q$Sv+noeWIgvs zC5=WzMtK*>1n8C>7;dKY?2^lRtfMsq$8@CN<;QANN8|I8?a28f1>^_97hiLj6Vlbc zX#W^0eldL~K-7IHM%wF~SitS8h17Yy^Cn*;i*}_C*rvT~2K`~g@# z!Q+&9A#GM4b@ha`iX~=1S;?P3BfbgUve>^*^N&aGfavwX%dfY%36W!L&Du8~SH+@u zSDq;8Wiy0^hI$3GaImv4eW{s?{iKk}_i3O|m4@TzjR{x)qxA0hj%k_8rt>)yuM&r+ z_onjj2U<>7e<21;L1cYseTC5v^r*bP@Sy)U7U!0wL6j%&occ+0TWUV%`=K-k zb#rIPJnW6!#VQ`(C-vN7Q(^^kMGu_|S#1Y`x%eB*I$5H!C!yQa92~=7osTsd zrTW|a5QfsmAsi|Ys6cchQP&})scUBJ{vqHVB0YM)U2hbX@@+V+o-Ri5K&gJ>AC?!rT zx{CbF!nn9N?9X81O1gxvX1b2J?#}5&@@>#cr&m?D?&`yV2U*U~^ohkJ;tob}UR_#} z3BzNOK+fbUI*ooRPNmIy$OH8J?KGV4b*4a{R6pBw3Q z12%NkqKH4!=d5sRrjFxPY0K4WhG$7nrm(@0`#^T&PImhYB%&n%RUQ~&R$=l#qP+aA z=&^avKBi>9nY^_JFCTl)9`-o1PK2?lPy)=_98J zUtc)7|MW*SSsVE1Fl}2CVx|gX@Ii#Hrn_bss4C6!uBI*C>&WCVkKCGT(%Bd(T(m63 z1@H>{YEH|f&kuCM@gFGKoVC}Vi%LSzP7pQ)iu&J-Ta&Q#!e;6m6?B@^UCS=2u0m)s zqHz?Tw>0Wwjy%w~U~k`i8>ZQnA?WFqT2~6)&Y`KYo7LuX-P!U4H>lu0xA=Jppc&$z zk!JJ`-;Kh2ljW}X$iHzE`Wi|xp`^dPGZi5;eEaL{lN|J7|c4> z3^S7&hbtwqwh(vk3P9f9+wpUP9LlNin07tpDDo;^RZQnoJF341e1}|rhM>0+|5CKL zz;SmX3x1Pu3=;g93i9G8B^{c6Jcpi8^n6I*wRT@3?KG+TP(eSTYGee zah8yjT46{>{*qAPeg*MlOoLX!RuVNvwfCk~ubt)Hnf|m<1IGTM(XlfZVQF>mN@(n8 z4wa0{;u*Ccd6i%!>>Gt*L5TA}jEl`ELW& zRf7dDDF?R}%MyGp&pO8qe77x12+ja$uQ2aVz+h<{7-SwxrD#xt?`}azP3Ara_SRZ)bF>ZRqqGT;c)yCwsWkE4X^D=C#*?%I>nI2@!v| zIpW(I{P18--DHnmW{GZ!ITEZnnP9&$1}sE(Tt(S)Oa+ zVZSg`j~ejLM7zA)QRC z(Ds==4)ZGCkfghh=hnyj=|{3d+fxnxjcX3yV#Zl`TuvX@aoo=}-0xz+H=@i6M&a(Y zqxSKcG?%bm--er>g+Y7MX3%_WI)vfX>fDpjW9Id#HzEbvt3I1BU-WI7?EVkAuj)LT z8$Pl*8J~PoT8_wr1#LWHvnx16CPGe0e@0V7LhU?ESoa|LT$(7u>J+}A5#C<}wk-%( z62X5xW4|=rX7qSBRo8_ujC?1Vi(&;U%1)?fzj0d0%YN_+TMEY@5_$?c>*$QveA?9&L=u4-E~O00{P= zkAK%~%1ZeZ*gz&Ak`XKDPv>t+{>ruKEg-qNhSdQDdpG2S8M4v>-Z?*~Z))%`O5i}| zQ2X%k&=kN{2VD%MnnB{wly!rOHtQ2t*L{qR@%MKB}nhU#_lWYIj6B3x83JG->O*7@Z5>{j>Z4NYIG_SS8p{NggM|ilP90ZoZTPLU}9!gU-SZ> zYt%6t7jp*2V5Ot1Y-qLw*(&drSp8(W(v!T>?xP4yXnDdo3R8`mnmS}IKTeTPw)pO2 z0O7_1{*Qk|il?&oF23&10@bJuZ6qo6a|=o2NUn)mz1f|We2rn8<@ro0{_;L+vXiYn z^-O;e`!=_tzN>Aa_NG!T*tTQ2wZ&XFlzZVD~ElCIPo?2Ez__vTEu5Ol6rVyB;aUqn!Ym-Yx54moZnv(K+ z+_C=j#c`=`d^?%1F0iz`?R11cF9BUkZ8`U7W+8`YRhUIOW^bRG4!fGGjQ1Q7o^CU< zfKRi3LxCH@m+MN|1atiYhcA@KFM&Q7$8{Yea+-Ed?=C?u4!RHKJa^rv&eSX*6=uD$ z<{9o%Si%=?AkPkd20ZXs$-z7(#%!zU8u%#Q%Xro=bY2`z#MAeg8?xG$&X8hy9nR7B zW@Iyxne&OTPou66d^yryy+oG)U2449iOJ5OV(2soWo)z7hhX;->EKwv`qa^TBDvH#$l)VP^{uA=E!Gq0GhzHVvbBlN_ zG|>Igbuj%vmXvB;)isfn@`Xib*EC+0w-!%F@u<3@t!trI?GW5t#j;ISBD1U~zqqX_ zyPa~V%%@Z{@O~9fi-Ro4-m(h`6IYA8wCtzyAd1HiD%uZvH^0>a}_T_-wi9YFoi}PSq<+o*6?kA6K%G zt!;Hc@k;d?s-2&I0zpo85Zp04ToZr8t>K`mLF*xSLpQ+F%&4t^`6(glZG_<<=K|54 z8J0i%v~~G^u;50ME=@^d==h0vhuT^ekANH(n_Qjo>*g`PL9o55ek?! z&%vKoMYYJQRdX!Ob5^^s?eV$UQZp%oN{g7qqukt%&Df~t3uw}wvy)$62jnX@-C0NlwjR9Ey2(Vg|v76nK$t{sw0iz zJ5hr<QKScsn5gYTW)eMU%nh!;foY(7TmZe6%dF?Jm(6@9v^9N1(mUMeVx*C9#zW@!>{c9 z7(_}BI3=$0N|t9|r|Kcrqo0>cj37v87753csp<5`Lxe$Qal|q)xgxQT@8(?FiPOZ- zOZ6^kAB^w7x;YM;K+Ux4tRZV033VTwW+IP0Wwjj?+OZ{D#M~DWiv$3YYb}gxF+yZl zG*ZOj%Wk53s8^Fqww;O*_Q2}ZR-`cr6GT%`pWWIW%D!l^RN<(Sijk|Ry$^lqE?LL^ zdj0xQ^scca?q8?~sw%+F1{C&c-qN2}6#jtHpY>HshT(yace(&y+@ps(RfGCjC*Jd9 ztBl+D%muY_Q#OJK9~D7*gNlJceymuF5(sl1k!T1vxbAX8xZ=Evf3T1nK*`@6k7@*M ziv{#?DeG6vG?dN*RD{EwrBa=mzTtkFOfQOgJhTa-2B>mYd$rA^5y;5ScmD_e!+?Dt zq+{@48I{4uWHbQJ3&cK1_ZyIZ}jw~N*>t~DlWOWSa8ajnbyL}Ref2ShNi#hF}ZFPDs& z7%XI*tSj3l2wvvRC~@9zt@geGAq=)D^+WXgb{&fLQNk8@!nS&5x7LF~_mIxikY($ToZ(~%|PC7oO-bSV&&0VsR1XhtIK&S?9Vzw z1KcWXYf?2fHWZmd{_{(Sk9U9JKFZR06|CD1gqb@z2|3$ffCEgb>D2{X-MJ79fH=VO zig^QkmN;|M(a?ZTZ3Ivf*eM8`F*fU|sY&K@{=79^qhvKUllfzc#ZP}#AMg{q>p*-) zk@9h_RN$b4lHF-dxt|Ed!NH;Dr-$o_cdte1IDpMc%mp(Hwjh{q!` zsAvref#`Z}SFykTc1H;ZVlNO;K#T7$wLbTrf0-a0^fYaBpHHj)PV+l~)w zAeX&3%{#6cV}Kv+heZpbLDL`3!QMTmKW0KWap-2MnQ-m3)icaW-cb@feg(BIR4YoD zahbMtfX%vk_UV!`)>T#<5J=TqazdNWgHS$CQRjuzlJU1gh3g^P+Oj>Fby=7!jFa)_sXb_`g0?CNWKd0J#*{*}3= zHSk{;(O%|%Aa$Em@kf6aosqGV9Agj-&VP9y7hpbwFOT8?s`TES&;Wc&=jB9d@XH*j zNL9{g9gb!o${2rX@X#fN+m`-@;06*;e4i>Tho9Zh+*G16u=u%#yz60mzi<93#XCGL#R9~=e7kq94?B`8z**TOaFNtrw z)?_vODZ*)cO6|rEd!HRW-1MoBsDy}P@wUz}O)ryY`L~_?JgHoAvV2vMoSQ$g-?MTa zp(@~vnzOK3g_9?}=s#4gxX+;n?a&s=K2%`0Uk{*85u-Q@H>JoU;bC&WPUR^sb2z9d zR%nNLOZzg)`tQMsZa_}{s=@B_Ad*s%)idv0g>-RoRLrI0{;udmo#d2J?Gl|-(pZKq#| zb49#zV#e2fBbxi{qcP#>E;BO9{WltA23SZzBm4pz1v;>jcqR2K9;*fkuG9XDGKbZe z-C#lTCf^l0cGqqVYEt%x-fis3L*YdC@j{ri8Xl689+3>~ERAQ`VeUdPeYQV)&erLr zfym)@-_Ofwe9dPVn^Nv4w5@*;Tg64eSc;&A1)WU(8Me56&i?@aW)XK0Ic}ZJ5<{N| zsQGfwZl;#)lc9ByckM0!GLl4>ce3_M!Dn(neBAgZ|NVyV#lAGOXTQng7`>{qvoi+3 z2SF!aFfoxJ>z`7on3#G$J9q&^^>i7ToisAcz-|6+`#g~2=ms#_Mgc}mjz7$UWCzHp z5_}#)aIcBcPc(~Vv7H3iv3v{?mW+iO3-SJO8`vPy!6L!x_Xom9oHXQ6ah0bGJ`<-g zjdfce9ny8|=JnlQ4>CDOMoAMN*49opm0P)X;t^11E443u+xlRpi-|H`gRDs{m{(mz zFkS;D@PDLbXc#6epI#aGQ3`PV2Rotd;^+G|H&$5skVQQTe>>_g%XM+30T>YK$dYvL zme<0S%d(xW1|Mnx=IiLj?j@!@ae^&8XivQL4Qqx}S2-7f4n`x_Hc4=BKX|Fmy&5G+&gP zcISTm%9?bMpxDAv5#ah6_y7Ar3s&&{_uU4NY(9G zP+?WR-+q$FTjI2nP&o8SQA1j9wo)q6y9pmTW&66qduOcF#6YzAzNO>G34Fyhkkw>g zW-h=#L^{s$&k~U^RjF=vpLi3Q6b@$m|F=+yVm#_$FnY;kO55 z3izxsswya1vztIfU73++Vcl8 zAf)Uj!8v^-2qmao2m&gs;A%LK={aVwvYaf>vq10xvhM|7?5w$AKnTSDYPII0=&sM? z`E-@_D%y(hek;!Mjrn$Bb3ZT{G<5O&33TjF;+)1|uDGTQBsFzC*7LGqYGKf~80pum znZBfh!XqEc_NS~%f7?xzJD$lx-u0`t2=#_ynRznQJUkktZa)!~;s83Qv+VTSuwQ5Z z1@?!mnFGCjqNjE{=bt$hLAZoE$lxI7TJ1Tlt6k;vs*f#gJ41D%23lWx>kxie#c36N zp`_K_JVb|S1sR`>{Xh)jUZVlR_^3i}#t6WnX1C2IRdiJ#%!gz7Ch{cA?LIlkldK6k z;gY#9koJ8tH6G#YnZS24X^+0P@V+vaFtY0^IX~Ek0#|Rq!6l!l=h@`l29v9?b*Z11 zuOn5Dc$|+PDm~x0a4Ft*S+jU;jsY&gl|JVbe)2u3@v$3B(p6dWOTWi;|t6Rnch@FUrshM%VeaQ9YZiewq z9fj~Uyg%{`P_pMI&gMYjMY-duqEeIk+FSp2QqLDgz0sws_eljbA;imLI-q#KHg>%< zhtkp0$KJ%O0M#StSyd-!H3H3^djc4k6&qKa-A!# z(X7=q5r{(P^i0jt$hd8iZfX=FjU89}xB_;#!oUr>uCvY=;pAe3#foIl4NV=X}EX-%~ zTZm)7pvIinp#qGK-Rtzt`bc41en#j4zGm4bK0w4zu>pZ@`F-K&<7pp5@e`1g&*dd- z<})Z|PFup!Hi&=#`Pm?l!G-(_Doqzpukc*(Q4eXlQJB<-9y zS3up3F^WA8kj!@3o!h>5P@}ju*VI764<;?rsVT7_KLtoJ8!j%$YELqycn}tDFLtTu zI^o5a#rRwc!&;O`pdMi}V>r`egKjViWy+dgFBhBn5RxGP6!#e9#8677jPC)30eo!( zn*%jSoYrG6oc3FQteNRzTSO%H2?^u}GQNRVnHNwG1U~dwNj8@d)-JX=OK}%g8H68g zO%4y|0@)k~AP1S!?|+^JrV|m~82PNm5W*QKWbFfzc?0Y8f!_o0#nZa8U>ahawMmoc zM%Z**IX_6cK)+&yg$=3of$bZU0!5xP^9+icjMe4|V_e$XSpXDkSSnW}-wD62NSygb zoD6ZUZ=zq>o=g>r(N|E6X2v}G;iXs^2#^?4Nge`o_6GW1LE+omUf*Rg7co(^YpBg5 ziYE_?Migzynb`Sc0F07B*E7Q^S$QE9odaq52dS5{9HKd5rAo;2`HJZqt`lhxGg#EC zlBN@F0#1u)5KeDwDTu0eEj!E$dDVGK#{YC`9ViKs{4yhnPt^#dS5_`VMx>uJc>30# z&I4AqN`trBdHy|faV>zgNRG!M=T>DFEm*H^a?t?&F-GU5A>t3Mz)KQs?&ZL`*Q*$? zF6{!A%+Pyd3~b^v#yW$m$!knm`OotOrRmzjR$niOE_;x z?Vp%dDM@6iLu0^;1RBVdtG1U?SfC43=|qpHlrbw~A7WG+{z-U)$~lPB;v5%>9)G; z1P~ft0o7u%0VtR=L1O0sRE_V9qd7;(2_l^2=3Kjsm1iy+m4J@?K)6X}jcgtPjMYs+ zNFA`=msmj?+CTzxxA9PMMhb{cIgoeGk1;au=~iq)E>ET%04)n_3v*p0yFLvPxY+xi z9fnE~e`Dn5JT{p)ihOfjc8t&P5HQ<-g2bkC>~maYk&2}T`BfC}Tze0B>En!x^aEdr zF2h|(_sBCD{uJ+vocMks(dA(Zt}u@m97n=HdU8;Q)oNOkh$_2Zvn|qfCJ`S>ms#RL zS7HrKZqUj|akbT5ZDn_z`hC6rw<9%C0KI>HgI|Yo%U>;E50LN0egF)M{teGux8FvT zgYwGy4J=nSZVAYZMOYR9w$S~iVi2e>N!*!jkQR^*sNgcbINyb^D5P+POsGGwk~#*8 zmpb<5zkjdZo^7akk@o!M4YZI|P&lp}hD*|W=yEJWfUX_B-J_APvLYS!d;`#T-9Q%O zrbOc9v~>VbH`T?_FJR2<@I=7vO{*g(wn2o@*%~D;z&lY^j|ObG0cJK8W#e%-N4D{r z*js=w?^qkmp`Ot}p9vuIIh|i=A_Cm#d9*nM!1>uAGk{28F~VwtHTc%AQEKn=mr7xp zubfzTzdBx&Z26YNY=69CV5)cGAg$Be?4#b!7RfiORbd`7BXoEl?%e+1(U)r`&+)Dt zN9H(|@y3;PX$By>@4fk5tz@?16GQ=hxtQ4l+kHB*Uf4Cd^C?#*pXp)H+&E0UN*!|2| zWW*ee%Q-0g>XH@t<$uc?1sn+0#ETFn8yX6$95u)qGUa8~d<^2>lXYD+631o>=y*Hf!_J+%9 z>{3N38_6XL!oYOaNL4<=>a9%C5Y+&}oD$CN+#jlU!M9ZAP<`75`O-l*K}{Ge>^YoY zUSsGw`GjY$h?YstwYL6B zUoL;e*Cl&5{<>g<;@=lCE5mQUZDP=jlUIBbaJxRm(t8IX>w9TAC`TKLA*cQ^FKa5dMXllAyF_y^H=u%&+V5; z{0(E%1vO&+&uTz#FT`&XB7b=E8kJbQ(67Em)d&=os$JnZ`rmd$|6VBm+tmTZfiEB#SImE=-18H8sqqD%7J?{Fg zt0CKslGu)2i_&z_22Jta)mdQ2_+}m2&>`I*~%8S{-t%#~_<=rdG@n8hg1~t>E3~;dw zUI`0$w^LSy8OVMRg&|#CN`eFIjt1!*c4C(CKR3CF$H0rHX{v0ao*=?c2&=orvMzQU zw0bgx{=O`I&TA&6?6@M^UFwci_<=aNFVdr|Atje>OsIJ) z)^p(XjZt;-yeWNN{9BafpPSVGVMdh;@0={5EGKljt7t9>Z(nW4E@kJa<~n}jF?Wu6 zXbti3RMx))i+o~YICA>x?y7vXuLu_K)=yb|f(!H|@*HtDlY3rUCF(yM?v^7E;YAlX zG4Omm@k*yiVTG5NlTH_4=(T22jH8h$@#=)D^} za^n}z?|NO)`lI}U(uGp*4S%M9dxY?4g6(?!1UyAC*C#knzD|hh#V(J^TgiOCPbmjg zBh5@<;dL*Oa{;`hKPyt%btldz)Fgz!l+jEEpJQUQ60F0$v+ z^W-jde>EPtAQSm|H5amH1lK*`FZ3TG8s3%zB7o%YR|z7H77tgNmXcM;`4H1m*JkuC zFYpIS&NU`~K)EAGc_R-sH{|uqH_f9rfJqpX#LlDR+lAJkm^KYwpQTI2es~i&u({+}ZH>aIVFZu0JNq9pJux@?<_VDoP8J+2$j6o|$%t*6ZB z#_b&d#gj6{EyXwvQFWt@_^zZPC0jkWX!fmTukI>u3Xf|H6uil#ls)(9WG@H*_tG?nroe16*}XRA(B|F zHG!E`J-+vADA2 z9d6URrro5Vl|5l6IpzumfgE!iZ%fuI+|uO`pLQM2?n_M2a;GEyNHpHDaTq()N*Csg zWBg!ZjMp;38S@~+%@VVa^gMeKK()6Lw0Ci7!jGk1;VG_~K2Ap}{1M74AVmOtC?M`Laz% z6B*Ro#@l@qaA?#4CD^n@Fac%7K=7&3&b4F8k&S{fcv>3va$of$*59_DSz>55tZ&O_ z^JKUEIE(N-$;;0#Vt1g|vOW`r7l8zb>th0)7bB(1!aSIEV0Q=op6{SJk|Pr(kTwlD zP>&>g-i9cPfrC-HD8N5^D*f&tFs4dam({2I;p>^tq9RzJcoSyQ0dL5`RwWYLMe@|| zieISX0pJv*Hu6CCI|}B<8#?-n!j>j2sWG%+yJbcr%u~vb*`7e_+f(9&W(2PQEV9Gy zwMYkhv5&G?wb2B z3|RK(20@8a)sG$Xj zf`HO{=twUCLJK93@D}&o`;OB(&VNF4>Dq z{={J%Mtx};sByFWUORflzH*;T6Wok*Fr2?Aw{~E_i^jwm81(%Ea z_5o8*ewawx0sc-v$;*&Kf!WI`@hfhbc~}031wH4#v|sRn%!J;1M`Mv8S=GGh_5A{*RRF9H8_iyG+zJUYkSC$ z8$Gw$kMWh`MM&?o*o@oP=`GWZ1E$IC<|}dA#`oA9m6*> zPNrI&+~<}v^~gUFB7)2-nizw-9Vd~QMEINRV(A{%9`~;8V;-c$Qi`dktqwgT;w8qC zW(`YG@XW6ZlxAqh{jJ^9_LrD*D`s7(Klv%A=9QB6;Hh7Ho-A*c&1rnB!|F~sbyv|X z#&CTs>Br1VWq6i|??LO*J;sM`-s78^5i0f0Nc(o!?DVeZ$H2F4xskF@Ud;p>PZw3E zNAjj)Q=YaDGhw!WelX-lJiPQz80)|Oh5!0lYo2@EB>8%di9t_S0d+yU&dTuTT!QuY zq6*u~@L#3nJa85S|vf!2L{eym%{4ti-$I zx#3(}F!ozzxVdq3gxy}Kg0#ULl-1F-_*5b$;#z@sdrx89FKt`y zNVMEtk9MXyHr(#=#UZm3kp#}uV*DYouMAe0+1~FgHSB2io*oF-DsfEYhV{`u3vGfeG`RATz#Q8CqqGMob{o9?iGyGIB0JS%$e@oS1o7=$l`B z{wD0h!3E?rOFn$l54fBGHQ@D!aLqyq;JAe&&;5@b-(D-+>8cEOt;UD`vAe;64=YZl z+%+^Uv9yNq<|m5eKvaJC_T9L8g|vbkJGQ-TTx@MNA3Y%vVd7I< z)?IOGFr@C-4djXLQPSNLuHAkI=8Xt@#WKf(q=?`m*1N+s^mFTfj@G|7ep?^xaX5Kp%+nn|q@>PRf-|N;J)AD$Hn1yk^Lo?h;l~`&r+!wOlH{qW-2; z{)W%|%)>Wxiq*f;wBkireIxMgr+gI9JqMPzaG9}o1wU%Fkm;aAA@_gw|5klUJ~_N3WiN%=F6iuLP?eT`U=3ck4)Ui$1Gj85?5oS)w= zAvoeb*pQ^(*h8xAphtB}_2HG!!LPKD= z`mbYri!i3P?cVuutNRa=Ts zZ+6oX5yf9Z@CE}biMln8BlM7JHGHCMn9d1K7$F5~2P_of8)(EvZM>}OuC zo$frIfBhHy<>|Yv_iHem^5{FguU7RlSGrHWF7Xu+#t-j=NYuFRnIgUt!Ej_u)5Oa+ z6}NJ1?M!npzjXe|ZO)O2-Jl8D^e?mG1qe>J{cNhu&6suaO7PSsnSVwF&=5w4G#ZC` zWiJ7&48O}YvP>tpldwK~ZG1kxScP$~JgjEBH>S}n9bvWWo$n~FfAvqr4+pO0`!lZB zR>izMzpJ1`G|UrXGWAYOU(YTfMBTuaOgx(VCr}8g#VRFbxV=1di;hV!rv&#S;6LYe z>DddXnGOfMf!|CIShltZELuuxkPq1^tcU(oeKe~B@%Vl}+hUX3=7LPHg^@#lV z@k0D1=70P4eZQG9G!k*&e*^1G?P8T>*KX%)Nq^`^HuqNftk3FMyS~ex6H3^BeJ z9_Ky6l{!bZ>1&9~xu8Dp!&nsW@mI1j?PPT|`M9;-eozCoo^R$4#I(N^aI)>(SlZ?fsKz7 z2Ur^3mjc3IuKBoYHIpI1!NBQ20cP2kW0X|wL-wd54!~uVjwFgl`ccJUlV~ZGHyE%w zKBZe>n`k{!u5`5PKN2eg`MA7g-um^QqAg8f%tsQ6Ie{+;37kk6IucNz6)J4UrWRz| z{7OsWHGl#csFP$Ec@4P0va+Yj(QI>gX#mNL`=T%+%FCzU;Rzy$7$8|~Az%-AfU$xQ;VF>wwQey1!;G7)G28;lvcw}2{8g7h zqq)s%5mDR4c9SyQc-#cw4_(UuvZ9_E_f5Cz_ZTEaHwSwI3oKh!wO4X`C(6YZN(<)~ zO2PWo({ZG?@+_yqegQuDF=>AS)Dp1PP4Ep|NS61Yk6I?6xf>qzLWgMHdb&6>kR_Iq zefx_SCXN9KEN0IvnR1;_IUR0kmAZrFI`mPVnczGUfp@1reRzEckjtZY2?b*q*H&Om zU;+pdCEoikn}lc`N+^c3ls^j?PiNbwB^kev9J^rVPP1on{S$>B1Od^gq%jcrjM z)e<}W^l9)gN==t?k|X5G=D#eoUI*XcoBPfXS6iaG@nyYjgTT9Cf-q1G+<8x)uwQ`G zTAdKaQr_eXq#GO$-!F5P6TxjQdYCDz#`lFE5W*_VPR+(yjNfn3*rHF2nt7kwamBoe zcr<|C3z9t~xFqv%Ih~lcQek30JOr-WpBZH9o7RW*br>$Doo3HTdHMbA&AXZmkhU|W zX?|xM&&|Jo5JHpn(6M9h`Q`gw?}E=XTsi-Z>tvtC`+Xu@kvP!j)Y-Ax!T)d>n`HJm zse_C^T?1WFw7w8PAxqaaJvjV%Wp4WsGL&i`4OxkW`~M&YOwMUs$j1G{5#az-&@swgxBklp~DKGkgI*g#q2jR3lid9bAS|$>5ZJUTQ zMA%j@-h~gbe@X6mb1Oo;XBk+T$aTEU(qwgFHY%0y-y>K2?!p!{Ut?!-uto?W(ryD(dGO z(rL~coq0UWT&QZ)yb0IVo_!m|^2N4esJN$Xg8$3oc7kg<`Om<0A*Wb0v&eRhLUWtK zWmr~tw7~TQ+dv!bIOpe;lb0kCSGrBs`Ac;aOv0|E1yhR|kT*0nf2NY_O-z#w7Dsk9 z7@DLuP&qfyZDXd~p3d?UW)z1$q}E2z_=EP)!So(3QA4|w&MUgpS3MDP@bjZyB)MpA z_Bg7cr^<77KL3oGKcF`7*)wq@EoNtf*Huil&z9ttW^-TE!jy)19rzq=Px~9lPq!m| za0Z?>HIkOI?zW8R>!`FCIeoRFoT~CB{MmX0`bRFS4~nlY>&OwRt!{tS3NA=LKk1_q z$KMzp#n^;%14cGNnB@}1rTYc;G*!ye%@HCGS1XMz+2O4kA@i2HT9Y>1nES!?%QShp z{4zRAq`U6`DSz0ymN@W!WF~yy?g|}QKgatapvsqmS2FYq2D@yCCjqKcJcYc^W@{`t zlExSWnN!Svo9aFan5i!MWrw}NGNob$#PXoBjUlt-IAG=0G3vcDW|7{9IgB;*iM!m? zX%IKso=}zK3QQeD_YczNQ!+|`??O{8F_5A5 zV^&qP%vkdLfOn?q{?kXU(7Q|z9A3S=Y}nc@q}*v@xo3d;{_+!vtK!4F?Tz_<9iM9K zYRzVcu8|ivZ!&g=Vd<)Q!HRL+zw|&|UMGYj>c7%Nu|d#lbtBHbZFf4`_@>i;Ro3@$ zl%^NHmx9Krv2^fj*YY(idpDHz@lne2wLeE+^dGWv=`fi4e5SO0D58^3iegZEwF1!mkpzRw6UgZ^YF5jTp@dg~5@HQ{Oncsk@Ht`fwrez6@$sn5b~nK_Nxj&>15{|qT0T!) z*oJR%r!d>7)L0E^kL%Vc2wJmnTB$j5Vz?IWbGW+r%>&A{Fe?1;bfLm&HVM8luLFHk zWh2<9F<-mU=Z;_7d)p$q9U2@QunN6t;aA+Dj6;W6i!RJgK9y-d>K(g_^S!r*=;QFI z_T`HPA1haFNpqycx{8GS(;l587q6c7d+)s|msqp8cV#OR_Cq>yc zei#mml>~a}QkVwx$5U?Zyqw0Sm$ATg>7GGy^+_uRtr1T`ZK?31W0x-NYPD7ai-1<9 zrxJRc7@8T*%3SGnU}h9AYU{X~tCd3lMtn7=8OEXPhws^v_#lwDLz+~w!;c$5mjj|U z?*VajvJJ!l*$|Yeir)hIzF|IR(ZjL{%l9>)NuowL!zVX1R3!Qryv(1eiLQQe>?n@D z#l)Xntqcd(1LJF82(|MhsF4xQm5-%*PqVPJtLNA*v}s_Ye;obAE7<=Fjjt4GAzin* zEZh~X{3Ud35*HNSb6;KbN7JIWJD zP#C{>q?mCTi~8a`x=2aKuiQ@H;VwY)ataCrOR@!RUi0Wz zf2I>2WHpf2*9P_#m(m%_CcwK@i5(A3YYD1O{$xolCP7%Br1t4=x5I_K6P)FGMw<0v=F*FD%NWegYUlpuHd%-Zdoc2>3quNPI zc8+0uoc~6%Eadc@+=DyAG30bUk4m1rXCJoZ0_RY9JZ&-PiiC~BpcS~qZj{;#q=>#)Ztp0me8?Sa((wC+^Xq7ZVw#&X&+C+DY zfN46oPNc0EYga@FiRSE#4$<4jWo{%=upcu^s0;pyk;b4Tjf}vgvF5^Dw%?co(>s1V z3WK6c!^DIdy3a-zNY^Da*Q|{J&3|dT@W?BhU>QE156QyHq90*NHsQ$5z;lxX`KY@l z0qanUN0aNbv4-P;FZ-#8z)WyjzMl*n;^F~=vYFhs4%|CT(KqWl#PEZR1(yumj4+)j zpaBTA&@6s}?Ag!!fX`rs>x4^D+)BY&zKniW5X|Q>{npU7X1&S(PnvxXgX z{_*3-WLqESJ)XWgXwE~uL*M}xugF{5$5s6OPq}Xj(BR6YynKEwmHipen6)j$;+y-m z4HJcQkQ*#^Uysd!=|dPie`+_e7ZQMF0pclv7hnItwfb<*Tnk)>Fisc+U2F5Fh_BuwhfD3Qtg|dim<-u^Rljb^~8&Et*4VHK?RRsyisZrnI)_(2z zAfHNR;gjb2BknCk)ojdqjiwq$u1vTb`}blB@bm$1mFutt8-}21F%3?zNS^qwSfI4* zEq_w?<7UWi?V{~VDw-L#v0ik%Qzx9Ycr+e0b-{%bQ!1r4LGrPnqAYYLpgfR zFx@PEeE`kB4}(;9Z->1?jCIumdwezZOy_W2HGiw&_}o1|0*sj>K+vpLPd*v*>L(S$a-jLmG{^7de9krcTl<=9wreVcYmy zX9Rn9Q9q=SuKE3H)E*F-0oUSSf1;0?xi+^?^^1IesK}_@n;|f#kh|M_fgxOhw2%4= z*z5hqONOJ$o`8=28K`%Uq#EBhJ(i1`+b%r4YDA~sDT9iVY4%Z*J&tx;4fgLBFzYCT zoY#}0ZX#yAmsyD|$|Qgpq`UmH9xesl(aYB+Y4=MyeISDw!(chFRCy*uW|Ee>?8A!0 z;lXCYQG=gz$Nf-r7dY58{8abFA2>i+!*Ip5-RuB5o zHY;dcIN`=HzkVT6FV*n@t~A*5ZjXKXJd1YZ>YjWjpRLypa(1ao8uvnCdSQG8#b$fe z8#B-6jY%Uqcd~k%wCvA|W@eDW1dyjM%sH8RI@xG6a@kbWjMfTW)yATgJjPYmlMe-1 z5*r*Afh*LvRY#Xc&IjqLJM&GIh2~QE4(~WlFMVDdM&~c~=f`?mL9dAjSUHVZ9i+|k zK%Z#X>B}OTV8&rWp%k896PuvwFbV*T`a@-u)Ru5g#kfbSq}5`T!7&o3gS7_u>nArV zMr%fM85G%r&ZrN3%21M~3?%ykI#U&CSfOO@~1 zz&Kw`@Do~&OT^q0!4YsRS6wHz#8|LmsFOCMH_=q`!!Eo`$mv;3MF~W)hvUQB-fHj4 zR|#cH9Fq*~1d-PVa}}LKFMnel(A@kLGyo~tXn;*pF??5l23*Dt%Z&IgscR~B(45~9A1q%>jc*}JMe=@x6s1z_4^7$E zpAucihfd@m!Eu+}FK@s@UyZXC zSb5~o3EaEt43t8z-nIxOH9Y_l9vjt8rFmaYz5Knq)ZswgDv_>L0a;Vzpy|)852O;Z z^<@jwPWGXhAsq?E?JrLdweQdcxE*{@yWFELt#CsMYwjELh%JC{g!gE3PUrRgQb-+; z|8_gzCdAM6POh^lbwzy-5Kk0oKk{jP(ws)M>P$DY&zYjtLNq;ACFH`2{jfg({b4@> zpu~C6Py%TYzTZYavkxxSa}m5@ypB@h`M=V)16V@)s*ov39Rd(+ieRJD13bgzjhpBK&adV7~Id(#(Df-E%T9#^qu{wcGbS4_!ZW?Pv;ri zJLzXQFEcSta9@qVyLv|ufk4Wv%_SdM-cr39z|CjnzZU=L{zacpp?*mk3{Kv_A$(a3 z@3oz*zgoLvu-}&Y+9Rq7xc|mi+R*%guX=)Wg*MsWshMY!Xjlft$<_&OAsgKn$w7Uu zE#Jx?dk(0Q%e|+cioeNu+b z!^uEkUgs^j6VyL|q+#`0k8sBNKE^;N8xJ3b%w?}(eOWZ5Bz*UGxS;9-6RA$2L9(5? zU=T;rE#tYx^R|9nO2lz^2y5qH>tuJ7$llKo!o3r|2WGQ+n_M#wQf71b{ZO7&F@^Xr zlxzS6^8~f>>R;t=pHN>-Ikhf(kCPnAs~ne~Z;opVg?*SA&g)Ot8rw&y8E4UF%z5Ay;T&8iOJnbLdK#8 zKx76aD@Vl{(Nn&M@KT5J7!=J}xa@1YEIot_)v)~2qH^#8us(GXSvCQYV*yfSap%~0ln%M$^k6-FP_CatK! zDZV0RSkG5ipRpHHl-BSvgxgej=v7p~PiHUEYp*ciNkTh=AKeU@yO=a$TRHKvkdzw$ z!?Hii{Z%kATob-omWF@La=otvtO_jai#&$6HfZ4EN=&Z66z589lz**5HSijrT6a2> zXX*^=>N3`cD2>8(1Utd7TB+MSVR=uS%ko>iI*BUpk2KYM@LXQo$M^QeI}@Lvf_W{# z9S_qFVj!t-dYE=Gb8gK+(3S=ZX`9kJVQU^!G3hN<9<*Ue7aS6!s$&-8co%(AgHHl$ z*XJ*1Pu4;oBlcChM`$(oa#S-v?99uIYinNMsNvlmDd=*;`j8I{oR?g3gG=Cowh zAQH`ZgEVrUXCMkq!aS;zMK(uVzJJqs3mAzujyC}R{X{WkQP#6F_y?M@Mh@geSx73d z5i!G|Z**o=iv;e7jBQ$S#>+(;MFn#V zJrWNC#7`x8EamPJXe{t~l)z(u~PYM z%L~{ym&~E2j1l!Sid>qe1MliL_5y!K#T1?=x<7=*W77gRntdfk4t71GyAMSqQkLqH5gThpi-@33V+Y4E$vtkiFF1cS#cj)KgO(&^$nR)T1w0hKj)b2Ih1Pl(O@|4%)ogD ztLEBE_mrauZXvQGUbbb)hhJ>lz9*X|yw}KUyQE>75S()|h~r9GNcf#veQAXDJ4$QH zxa=cO?c$mFL#QtEHLO6|v&OQxCMUg&Hm7%z&S*6|!{m%K1@=6rl!)+&&h+3v#I(z5 z?xdUCTf=IWVVR3`h;8QXLb#1^ zNs_sM&9LAmXk_DoqP;sjk1@J`DKD7iHk7uQC$vyY1>&NypoaAK1yK}pz$T&?Imh`0 z^&yt3Prfl=U7xm_<~-V>w<)j%!lS*1!%`rKSeX^n9S!v0XkT!-HZqb!lv5-?k^qt; z_K~b3>&+|(ZQaR25Oy*=um(x02FhNWE~V&uz#}xVn<}^djYqm4?R2Z|YFlr-hi9=f_Cub-FCxoHD2(a;cRW)scY8irzetUV&JyxzQe0UXd9^) z1Ucn|!f-ZUFIO&7Z>AEy>8DxZXr%=pgC7l1NbAXnC{Fg3h*(5Ly^G~B*KgF9+FT>Q zuQR;ub=g_!=d^b-CT{IS!{vr4xi%Ad=*=P_cW9HU>?QOoWH1ZOlR)m%M?i$k$>4Jpu83(nokf%nndTz#j8i&=+&YC; zmX{+3s_{zKKe<1ahSWLn4tQ&K%H}c5OMlel{}3p0dScOvT_we}pl3$XbleZeHyq$@sW0#-*mjju@Jq`mS3C+-+sJ(^7G zZ1_&K z5ns^;_BjI|O&A$8uY*I%FetsVZ) ztBqCm#U#5zF0l)c3WP6BpUKlnq0gjHo!G!5nm_%9mkWBnzUDgG z9XQ#~U(y2FW(z5rT#i?ne)07eE!FPlR{CM6woy6G?HP8{07H89RPC1%H-^;9E}Hd= z02JOUqF^9>f96fDErlexm=R_DwUv-u3hbflK|HWl3meNC;2$k3c*_jfTk? z>O;Nmje|%P>7Ojv)G?Z?d9pfK>xwZK(*TIpnx{#vkhD=snK9b<={_&#`j(O6%(~yF za4+0F3x|LhC+nbNmLjNeyA(Ua8?8XiZ1`blQ7?W3>@O;*vVAOm)In*hXqi>X(Q?>!;`%6kON#Bnl0#}>k4F9uk;OrrWG%1+Q|COBlLqZ! z=B#2#*{Rq=d(B(a{+)sy$RX1360!7n?3;g_ZyJ0 z&4Y@{93e02X-FkbAfbMprp~g~d7_ut;%oI0HRfCuXc6bZXoq0y7Kn?Pl~O^10!0s%)-ha;b_(!K0pZCY>gRE*Z9p2x*Z3@!T-1Pa7 z=je`H#!vcc&qmN6@_cBLtjoIPg0d@oO98X! zpn(xSmj-(8kIX#|)r?=uqa3g-8F8QP4akeLA+Tj)$n&0H!(&gT*;xzLUP>M-t}f8% z+35e%N}W}AI{fN`o~^}RHVghuV8TDd7jI|FEkjx+eD-Y;ar)e^gOGc9k*eN>L`st-mhIYpb*y=u1=4DN zlig(pU{jx+4`D}3l{_xhWoNTJ=Zjs;u6h;#T|)uhgM{a>!bjU{^@y>;E$bBNI{$sK ziH7+gyZGt@EfY!YK2jY^lPeiSk2o?93^+f-1JTRCJ+A8cyR~E)oX(C{`l7Sv=u){gURPnv3cX>w|gv9Y#*y_?J$&`Kzj|Qd}PA z9A10r=WE(OMUd_8Ag98aJ4F71K-LIM{D09V|Dz!1Qqj2OML$J9tLs&M zc;_UduVmVNKG6&=fXd5LNYs9Oa(7Zi>g$CLjYl2sxs_`Ql}bMO85WZ-(8A1&0c~J| zBryWNkyrMj$o~n?k8H4XSK|bR-58|JkigXe9W3_vHO@O;vo1 z%Pl9WaaIBfR0N$);B=nz6yQbi&d-CWg>!6H4%j0zjV~i9Hkz%iRSJ)HR*nL3;G~@f zJNZ_Pl;_``QumMtvZ9*W`&GELaTS#V>EuO7%2E2!ka2{>6l*H&anf{d|6FwphEVq~ z-T$lKvExh%R-QAaS*33=h91;3)+*Jm0(n*Ml!Ko21h??m!uyG8R}5brv(Xzm zVY|e2*UKdlr*1p#M)!@rfVT88Uj^F(_x-@^HOdab_i)5(Z#LF;>&t`({<)4Gb$^j> z52DujuAa}%>X?9B|2hGD`=xhl3R~=yYBLg#PnMfM+zL%oI3<>Kv@A(a!G0OdF5&mM z^p`?u^C>Fl!{mXxb&m?J|9*TJo=Q(HkGY6x=~w@k7r;pS5fU;N#Nxwu@$PfJqZQ?q zhb{#+)KIHg{|832b9<=bdSgo^BKXMoX-GN!X6SIHlO+!Ayk{$Cm?{_oXD!|PkiM{Y zvdG{d*(}r#;)loQ?4?TU5;5Bq{sKmdB;AXGFX^ds8ANH}z1Q!*d^_Ye9dy%lA2ka_~PQDG5H9rKeeARR|T3guAmyCAb_{kP#cz+gLR6=3|hPKXZ z@WKx-Yqa)Tm;hqrl{fV^P})8kU~aNWJ_DLmL1y!v{eB`z6j;j(08)=H~q2oT`W+kRVeJlN~j2wY8Dh|=_w=$i$fXTA%V}>T}cly!1ntdL% z9Iyq#F-q++xFiJu;NUp0g?idkoJNyKch84fT;H*IXUJ63YVjnZp}U# z7Q4lbsHJY^_26s(ERWTlmZ25x&(603qn}2iSUQH4dNu3tj^`HkbfYgjo!yh|EG1Q@ zRpbK>hf5X+R?|CCwS)=)t-_CqKN0WAGRH*vdlF==HW|$9a#eKk)Ozc zT3l+vRlwE?tcso-OFlA>du!C^?pGjh<1gdsW1s4YHB_&Vac-pfd2bh`B}7g^4Xgt{ zCPBDul}XRiun*|#%$xWE_8@i%U@<2^!au$IL77H#?_$4e;(>dmZSBvmwe8YIR@BJ; z7Ca!BEy)#+fErxEOx?iV#NaS-j#-%Njpi}8fIPB@%Gs+xC}wjUL{GdFM{NheGE^+j z2w%5Q4hm(Aa&?^1VOC6 zs1Ww6U$_>yHDv+%nJM5tIHeCG0u~Px`0_qvTGZ{$CAiht!38P6#Lp%OU)ce9wFRfs zS|9+f0N+vcR~2}JruXvwgX>8&??|#c!C-yRM*7iESTGfVyF5lSa_*X#pz#1r;F7@t zi^!E*Yv~!(PPR*7?5MX$9}FdI580x41Yk>aFgZSu-`q~VW{Hs^;kXV-lbtwU{KpAw zt}||jqW5lJLEYr!#6SfNr|lkqfn7RTC~kS|5XojEf{d!WcAC?~s>ohcp zhHi|DQHukQ>%K~7ZM1FfTBgh)O#}OB)U5ecg@zOLs$@X>_$>AqEi;@ZOElzk46xKN zQ}sJWqk8W$W8gYi2Y8iEhd-Rtlfj|%O9S&wCNlsz{XTdYQ=l;T+}zSdcyriK_yU-ESm9*()2$#(7G@&eOw(Py>P`^TTx^I241y6y1Bi_i0%xD2@KH>TR$maI zy*2PfPZoDuJ6O&B8#ku7&I%)SX{j#F>{n3^e+mpjACiQ@!y~EoDDZO!dx6~Jc*>#_ zCuy1y1K-h?`cPPboZ+aRkN}2TSJJP7q3CX|(0=I0_{oUjj$kuaB(2F;8f5I(X;3-{= zM_Y*?5i(y4GaIfVW)r;CBWhYOi#sFLj)VTDFwXp{xz-}Iw_;ej<74D9Actnz zM#}9CpkIQl(<_@=s(WBvAq-r)TNgSb+@DHw_Z|g@uAf|t`@u`u42DJ0Py9-?2O*n` zQuDjL1DX5ajtPCLO)V?o8X)@`*TnAA+K&#vLf6})Z_|ntp*VLtE~i_FIaW1C2ATy>x7UNV0kn}9+-BJ6Bc=)!*?+= z_VmoYozw3O_(HvZJSOi->y_V7T3GgsS68Y8x!-)GQc z4^+%enrSOQQS_R!Xt7qwUW55Eu)()LMz}ji|rEDgL`jGB} z$g*@JA9>5h0haME)wF0tRQ(Q3OFofQBo5a6M!XvEuboOCG++M$AqYQo!eNA3FsF>& z-iCwYd@@Cqkn-pazodL~A!=-LJ)O~p%nvNC_oQqp42{^VUL5z#*?4c**WkJ86TJjU#f{$XVOZhSP$aHkM{7)DE<-Cz0`^sP1sXv%ZXyBfc=c6hZF z7LgYMs--S%)cqMaMHIg-DX_rw$;sDYKcyT2y-UMswiI@6HSD~t2fCxsGsJLoGh?Fg zKE|21t16?YR#I#na+5w}93-8MR)&ttBc3|kS|DAo)1CTk9A1n_w3`+TA^;9r^~q7E zUZwTsYo&QL8D>n-yspwvLne=~t$EqMs2r?aI#gvJCUi>LF8xCk_1axe8p*?1l=~~% zpMmBj5=`jt;bh=7od61k)OE&_yyRcmGI|moTko2oK`f@U@ME|&5Et29E?uxH!{F=9 zUQS6t(a_lgG!elg zmM%E;Mgx@4rv+v~+Z8ZBIye%yT&Cw8+483%*T-&$yru_J?{0;{=C)fw|6&RMsok1K za~PP4)OQ{0=R)_XQ*Lix=K3y%1l&sDU$ffJAAo#aoBa8_6Dctp&?;F7eSMnNz&#Nf{MJ`nJ? z&JF!k#T~^UqJ}BJ2JA9qMA#{k;D(>Fb&@h&!adTmto(506^NHL>7)-J zlhahK0r!S%AjfUM;?(tM&hU_=J!-hQ-FhSF>Pwe#cQdTERt1VoA(@|Wt+hL5K-38E zE9$3!ugvgK^p|iKN{ietV>?Ky&7w<0BWXgbH=h})kCigz=MXKR=M^|$5oKu=%+{Ed za*H>M6W^&d&)~sJ76GQwmNi*oX7ySV&WU!jfDtNC-LWqVu{cVC#cC&FI5|025`vv1 zdh83$1P*oO1T^z|kI;#w8nQ@n4u29A&O<{PtXWxi3S7Z8G;Hc8WRg^b_6%dL z``DyZXa;L<;Pv1c1l{+6L35HTuD;zj>MN#sGZ>JMGidgL4p$muIOj5Z&Uff9 z%HSVVdo<^NbZQ5$|K5GEK9>HbOwFF`Hg#si@EZ-929N+?Hd^d*uX(5o4+d0^NI=Y6 zTV?}bi?N#x+tFN8|3*u_Bh*H3l#{7QEi|BGOvlVa2FJ$+e14f7Zh^!F)vu$`;9!5t z=~tm%D!WeO-j0`j?F9{0eWy5J_QgI>%}#TYhdPSky9Zdb7zYB>`2MO^Gr0E58j6+x zUKFav$D1pwLszKMz5a=z30K1q{+v@Rsc$LEuzc_(E%O{PLJcZ62<&JNy7{XiTjvzP zIuz-4WGHVsSY&i1MAd4NlmVg6s~9p(DEro4;(KEoiE?C1bc%ccY~cmYIw$6>w6oGw z27Ma|jXNR5V2fMFpJHNhpSh5(qTWqpw@Bt3WI6c@(21|7tH6#@z*mGGUPXPy)`#Vt zNVEESGlVPhzQX0VotH)DgJdcwB(9>bRPS#%QO`kA==4Ff%|^yh6M1{g)zH&B-1Et` zYH*FEM{qEjOR;MX>&QNuFN?&-XM7cWeIQts)or-*5Y4yRa+LiIe)T28fmY@f>_U9$ z&$|h`wbrh{(s#=%li2v|pJ-9z$Fw`Z)lVI1NHM8ZE9l>8oXmql+j2McEAJI9$4@jQzddg06sAfgTbki_feD*)!IY9v>`Lg#x+8@d= zyM7klWZQt2zUrJ{855N3+z}N!aAYIy>Ai?_)p5{Uw-;h5mJLLj*=*`Jv#73Q!>s3V zQ-@{y=nt*d=H=1qUOQ=`FSq)S2gSk;@Pas7%juD zHs_w9*LnbX7oY#Nm_s?RQ-vaGd1th?|66hpN(l|ocXXD_%fy3HFoFv5lsuaay^IB6 zmPUk=WUYWlE4*OIp4ycOICL^&+D?hRLV8BCdOLJbE=7-41${4`>V=#8+pN@mR-CY8 zfJe61^=Fsc(@N+?_cMIOIa>2vtkA2yFNZLe=g9NBSRP*Jsi|{8$YR(XZFr;O7*wG< zA!%zaVC})31Ma6`*^Ys~$u$4bk`5o;N)IS#R2-2p?*`mme)?5m9 zS?p;nCm~g9so+Lxe2=+vw+Rzc=tlW~J$kVP){%{3sah&inz6Hh+Oc|#C%vX-u29m) zRs2y`D&xx|Q|MvF)Hjc)_u?4R=**T2vnP=22NOOOQuJ8gg~_H?^mR+L)@wWJ7m=`I zP^-P{B#*oypoke;oFPa4fUSXyi)TwS35dEXWJaop z^zCSw!l-&98vZy0fO4kP7?N0rXq4#xaR!6 z)@F~78&DY81l$bRj0)b4f`wuMHFLR2BiW|Mo!Ir5ekf)tDkjzE_^ES(uA}xr$lD); z!(pXbnK#ig30h@Jc`r)(wcwdy>K`Z+bWZt|rNJ)plViQWlQ}#5h(><7w)P2*Qig{w zeRn^5I-+|xoM?9-&S>;BH$M6&-r2Qd$(j{#awG^bN<0B+x}|+bB#PkcsP<&1)*Xl@%lAEO(%P(9J`WCys95 zrMjCMV!NQbx4Er6qGWm;qr>m4NY$FibPw>^>@BDnf!5qV&6UUr7UyYg=y==Ju1UU;33xZrUTi>v^ z{H`QG0VoeZ{%i8*)fEq@nB=I4hX@KLGGUsZ!Ya6)Fe*XpIp+>Lm zgOQB6_Sn=BaTE9IniQ(%Im#kL;yGRtguRXQRG_|rVO0Q#CvJY$UcMGph|3mg~~o;pKijuL{xtmz;SAB=mNC#|Zs@dg_N{fiI+oOV?h07cvRYgjtZx zwX(nb!Zvi4apMRIYXN|+$&phzhU}4`Vk}Vs_|{c3;LR#;j!SAL))caOd51?g4T{(% z!L-K;o#{WR0H^C^a^d-ChEe|bP1n?pV-o3$K81UWlAW~~4_Khq4}UL2|A*<5$qB$N zr=4q7|8sB2|Jd=UK6$-ylP5v;-_N@LR1yErbN+s7s+NG&y`l@Qce*gbIe}8}f{;og&eVP5sNB^$Bf8RfU-@kwNAAk2> z|F*yW%ewiW5B%MK{@s86-GBZs_TDq9$@Fde6%-39>PV4}og#u%=^z8BNE7K@q=pWm zlK=`R2&f21?;R2;p+iIjl-@%RiqrrB0)&>ZugtUG{m%1j|9fW5+H39oVdl%QkR*5R z`zq&kp2zV!|BXNYjX(ctrTpJ#_W$Z zO(*lmNq3}-Q-cnANkCj__BzT(k4)Zo7WUn=B>=g8Qi8Be?UX;}iXdoTO89b7;{(NS z6L*LC^D(|WSt}Y%e4$tqquNOyhJ3S2QCvMe^ zux<8_Wd|cypnm`#3h6>`Nt-3q9QNw=U3idyp~+QC(#=yFM5$tgKvoTO6nqH^n|{m3 zD19M;X6#!s1V$#916Q^=AVOLLY1Y=%$GE<7B_E#}t!g|dHk4x%M=3%S;5=^d?WwwS zaSxNx-rg>m!kKK=GGf_cLZNjyH6CoJ3R|_)6F}?9M8Jwk0&2HJ3X$MeMejWcy;Y#) z6_Yzyd(LhH+Ur?UtW^>)ZB`)W6d!9hF+|D+?0sl@enK3jQw$BbYXiA%A9q#cXT8%y zbI|v>tjC`-U{lNAY<09euMed%3u=bqlqaw<<<`O91$uZpio>%{=a+r|Xwm%Rf^rm_ z9dg(8oaCP~o&O6x_CJ32P&x3Y-M8h!Wd_n++V9wdu}6gAcr5~+C&ztUsNAcE(V7s; z_qf%Tqd^7!c))kY8LpS`O-?T%=PGM^J>MFDgCt#4znh`XI#TE5><(~2gLa8?vKt^N z&wSAz2nNN@jDhlJH79$QwtUG<)TKL_Q><`cAWsB6TSqFxk(!NszO0HoG#kyhUXWWg zk2CePtCB#$++jlxMS5PR$Grsw_d5jeh4Y*SfYcTOIv_t|1PzqGxE?ST=KC9P(dd&!tcblfigcQE3`!IO4k`*JR?zX2$IL>&My@7RW7_6w{pV!Z>9 z8_a?K81^5FzJHqK??Zuap?^ff0J6JKSKj$@9^xFJ-$Xc_nrPALe`h?o0;)QYDGkx> z$_(SZHX&MB8*(64(s6P|{CR_En&49%9aDibcE^2(lDxt6N&-@~5%K~CnKlYAE@hPs zdPq~{%lxr9afa>ZTYIk9$H^?Yldmfj=&S+7qX69z1SqgQ$PyieorO*x*q7B+rt^ux z4nT)xtN4~SpRrD?_sm(p1{okm=ql^Dog~IWis0@YnJ?z$Wdmh|bw&=?FBhzbbo3(O z1AQLH=1>GsP3dFCi^i%mE5(MRn-ATh$(eR&V3}!YNs`oa9Q~YF8 zL#z&)H0%!E@wiN>fEUPjv^u zzl_HPktZp4R%W*gsbgSq(4|)>7)G?SV06%7bnaTt&%n1~I~4b zh{hO8UK=RlQ#hC_L&sFQgZjQTkWi_!{Ej|;@_lS<0&wc*P`R7;`)JAYkAg82X^^xH z=o<+aHKgyX!dsbbK3K&W+#sh6mI9hg5)e4vJexSb?+WW0uhX|!0vZA_%c~3?li*Rj zHdAn*Y&?j%RUr-O_1~pe0jIJPNV?PjN_Ph4L2nl5Ub(ZMJ^~R$reZ+FROwM2=q%ng z2@B&Lp!lO;0=0zmy{x&L8Gn%jrwq3Dm0&XdHV%)M6TfO1skfx*q&`-g^Gc|cIY)Z0 zfyBZvSCy}GU_?gdijw%!(k#TxYCczdh*3Ly`Y`qt%Xo!VDNUroGqSm|``n(*;E{PR z?fKW2ws2+M2E#&mDiJ=1xY0RWe*Y)4fuhdev>+tDN&#`z$_J@=)+++&flRAhA;~L; z;bQBP-^#o6(4Ar(0$NVP%f_j9Oaq1qRp_G5_t+wF>&Cbn^O9=hoOb{1W(NYvS*P-y zVw+NXCre7ZTEq+AGZ*27LghAkJynV#rnqUEW5QV$F`2H$24Zdljq4qp1BIREh9ci2 z#q|kER-IK|Q_1y1obm_aSmn$2W5dH~2f(uW#R?9V*glCT4be=-E(%#|F8ZL*{`7`M zssjaS##;8|hyg*{1z9O%*?LA8)PV>fAOOgKp{HerwBRYUXy4qz6ac(#u2n~zGNDh> z_JiSV9XQH?6Y;!KX(qp#F=41gS9a)edAOmd^LS;+if+d)OK#?HCzFxGo$-{-FhkE& z5xaNLHwAN-BPe!5JCAWtTinur_Ezb~o5JmA!;3?Ee_-pr{xsClNJ^;eS~*?f4n&H} z9*F`ui?loQnNfalxjc4U8k}4sV+u`Oyn9ZugJMF^nhgXRyN+7`&bXc2f5*V6&nJJ; z!?;`S@L*fo7@($>MKdeyaKh4<-7WMXsi!gF9$~BJ0V=h`eFkXHs@kd%Ot5MxN3fQb zJV;_95iWXv4`J#&CwFi@y4Vs5@QKWA)yJ1cCIHuTS(E^fid8eX+8wadBmuRMSgLG+ zB1Xof4ywLjIF5qo#o$HpVBHqHcvHOsqhr2H=UuoJyXr2DlCT;$YuHM=!V?WY`?30ooW1Pa_XFra>F? zdTnaadO&}E8MJ#}_I<(s1Clh^Fp5^G-Ri5#jSWJW)H;2WC!#}_Pr%6@HnbvhJ|=0F zWtWMxDGH(jOc{7H?gU$W00i1LnN@+eW>VXN;EV^nRd1F4K8#97^z;GJ z(rf%tGK$Z*5e6Bb6xG2?miCnHxZ!d|& zAEOV~<0B#{v6_T)-ZC)%An*acC_zq`)Oh`!QeVlCIV_-U7$8+^&i>$pVMMZEysWp) zEJ7XQ+1}L#bUt#&8i_WMRYJStFUX=H%a7dmal$h`Iz6R(8agb!{!B)K!v4}%+a)naa-@wJYiTG}kKyIiQ>P7&2OJBa*4tsx+J@!?sc_G21`M z+qI-sjTIm^=#)D-8pbE=({-DQ!~JXLqdtKFWeu>^OajXj?mZ}lFV!l88Q($h`CtM9 zrTdbKTt#w4TmS>h;GJZVl#Yih9Yqhr%KJnP&HZySPa~~(I-NC77Q-UWkKJPO1~v4VSI}g8 z&OHY8A3^Std)q4z3MY0abqy5c=K+f%u?Ui1H28Ba50cuQQxR@L>pSJz#!g|{xW+v5 zcLq-J& z`r+C2!2B%Do<6el;(tl7;1>4v+)IeNynmUZF~XjcBZKRGX>nE5ruTs zgd0?Ecn%iA^-T}2xF2B9UpeO4k?%ryA8ICs1_txPxRirDQND}eh67J*TNy$u04z?M zI1svXyLvi!P_R+i|HrGF4xrg~e5g`}Y?s`b)Vg(gu3F%haqOr%X5E{ zp~sKbR~pYJ81o<8_(Nh^4+PZ@oX<=egH^)Ouy^Utj2@~XVD^a!TBn(yNLpUZM<{c zj^hDo@N*sF5Rs1x&^e=hdhCHx>?n}Fvk)7IQ|0r1BV+YR(ZM}J_c!|P)-o-UrNDVc z{6Pwu)vbH3T+qu*_4EvcUbx7f-7SHX@ycg|%ftf| z*QaEkxrA3ed*m|QKHEyqbEiM~(f>3i|J-Uf$Rmi6R;o@D1|6=r;bJIbuo8sd92L%f zLv7KqGC4M1j{g z@rc{zk1z{HIT<)Rn|B*wpSRnv;*$q^HeF(f(VB_k@<%nht5v+hdoiPPAPx;P5wj09 zkhWihPg)gWAkSxcJ3pKn!jYQ9hp^b!RwfuME-OS{JHJsWN~&R^7+K6WJgvoCF9bs1R1l291a@k)Iu z>&_3}hA0<*VE7$n{4rT=e+uIq@PyxVdugbna;|9jhY7BT77KC!pDQ^GwY1z#Yj+p+ zOWQcgHG7!-FFPo45dQFwt!Td&K3@f$ky@u5Q;= z%001DAZ#mcBt{!D=$_x8xNE|AUgj_^Gl=mEXvm&3F3PumWyW7%Uk@Bd{LKa>d3*dZ z>Ip*EuR5UGd5bQgR*P4j6y@1V46mq!h@+w>V8xF6m!>L?RS4*a0pu{NgRcH?gNXuZ za-7v8*%_b+Kpsg#9F^BZb*Yrs>awIe z40razD*uHVWse|l6T)U+X)z?fZpv=AHhPhtAExzY(gZSO7Te{~rOW+iLtCCv+69 ztgLV)#=S$!TP9;dq;KGSCl-BF+fu^KzN--pyt?|`O=OkY68y5>v6^9SG>qP7aN2Yz zwA_x>Gj)B=N*Xu8d89&2Hd~WGo?>*iP$(5)gIFNfHXUaj(OsC$$0Za=r`v>qAuzG9 zay7(1YPd|NV6Y?)hD$6Y=->+ds#mvo4Vg@CGIVOxiPi27&Pt4RC(HeZS^j6K;l^+I z500y3)w#gj1m%#scOGA*J=js1>#Vz#%4XQbRi_X3EvQBs_@FY5G zdUpojB_xGNU}GD8U1^H?%r;&%n{0U}6=nC;cewL<4K5dBo%l*_=mT4om1-S_)3O4q zPJ%+!BXPMyq;E2KdnXPFf7yMfSE=s7yDWN5WF)LWD21^E!F|~VedT#;pWDy5DZ%uB zAG{B3y`!!A9YvBK!-IefyzN%m*DDe>>E=J~o#v+p;8RfmB7u#S?unphCKl($wxP3GMjE@ODFqYn}V0R>~?kkEm4kd1W&uS9!){G3j!R6F}s zu}O<*ZPVz4uJ?lyp}J3p!184~_<7SVFU5NPX4tEXHxogQ=nEk)tK!(;J4etjX89pb zf52ys+IFeHQohOGtI?iK$KLg_5xWX`J!{zCe9={~Hq7y)*N3mB` zqACdPZP?_#x%PMua*x>mVgcNHD;YP5x`$!Y(R6KkW@6;=x^B(I(ik$=9@>5Nw5x)8 zl?1IDV=ec6oLLpLeu-tL`JCK{QjH732oZG%$d_M9B1Vp=*ijJb&tMBXv)iBg0w;5n zK*~AVmxcLHhbZ>*Dzauv0VO}vIJXLVc5K7)#2@B!*B8efOiDofj>EaraaD=@H`(Xy zg0$fqF8ewUmVr0=WslyZ-~Q&}MFBISX9G_&gNcBLoM#UmFyWLBEJgAni~&a5GjT7O z0F|(j@>nn@=J4tQSMH>M&i79OzN$f~r;W8jn$*N62T2tSXXux`FftP7zefA83~DV9 zYVSmSt7Y?|ovfQ~02mwBc|FsP6KXxZuys6UNkGaZ?I8Rbc_(tH1os2rZZ61)*UnZG z6wy%r-aSh549_*tPQ%(mhH{FSSg52!x)7o(O za@}M}Kt<8~dytPb8N!lkF`WyuJ@%i8PQ^@QqXFjuCx+aXAhSZhak`otb zWWAlP*>LYgjV!Hfh!x0KRL**D=>_U2FriyIfySJb#@$%!1LoWjNOkZktZz%FWT+6O zhH5X!)XWU0j)X~2+-89#I3%Mh=}05**=z>>s}4}p4!!(IRAG^wNl}X(p#XB|&nBV~ z5^DTG8%;W&0ozv^F9XFR;GbFhhFne$vZoqhKpQ!q(EQLRK#S#jHYfXHJs_ojX%&lx zvlR~|7X45HZZ%=CdsO~RtzI)b3OXq>VBqp0`d`ENzRZQK4{6LJ@9iPD@81O(D55(8 zb`F^EafTO6{RaE1`9d0pt9Lh2(&R5qn7DnrPDm2$DW!KKHt!@p;uq)p1Y&I6RGGRt z10bu$A5L)j!A~NqSn?s2h!#Ztk2pSZ!~OYIuo5wbtxw*q_2Q97eBa}CNz{Fo&6s8) zuV;Z7(2$eK`e8g)VLxQbaPlCtofjYvUt>Ec5Rr9fl-LSW$~b@n?co{=9(p74-JbKh z4do4PC3kr-`yP!%SRr=y^+}s&(jDE2GBP%rv{yC%Ys9xouDYFC+8xt5*6;0m{4C!O z?#)#_&u^mWduO$;8|l1M+w(dnGmZu9=-SAwIyZ=ZimiYY z4!eU&kVn?&LqwrV%#$SoTZ#zh-k;63|DnEi?eyTa97jK_Xw%b3Q&QSM0_mNNBDUtT z=IToLsOacHJ>yE%XAbpVI4V)T@$+8V@BE{6nYvRq5A@4FI_e^si7O98X9{Nh0|vwW zcNV4dTD-g5KSzrWR9jU8oN|40-_~X$t}b=zR+%IUvAS}ErP`Q2{2ay%&6FVL-hT+w zD`&?E=#@|5A0Zay<{}bTEh|3|#sY1{kQ(`mhuc3BT^jue@^kNYbR4Kc^aeBQ`6%^B z$Ji+n#k8UP#%5`-z|~uGhWk?z{ic@cfxX&3@Pk)yq6QH^41M*gK8rIjFwB88Ue8{I zGuAX~WTQ`ov!=Tk4_Y>SqFBZZNJGJSoYOoWZ|-M^$oc2EWJ40B%$Oas3|qWfoFgZ; z+_CmKa&{b`%IA?af|n!b`=orOyD}AD|SD;Nmg*?Q+aBI-+$NQ36naMkS@n}CybWI%J}UxaMDRhNfEm1umyGd^*tj( z-nZ1{Q$DmCc$|2X{Tp_Q6lDu@sSn0(86F%EKt04}MGq#p2dvRqO1zpw*zIMSlye#q zv+oxsszCfZs%j8DrW>0wwXJ|MYQtwsmRlQdJOh3&HEE)~oK~D2A56Xx&M_#P*W(cRB192hkVp0wBmew*M523Kq$FBl9&^LzJ zPxVK^Yg;QbD@R=iT1PGKVEV)aqW@HL`%jgClG7e1Gj)KL}(Jx*~P1&kL}ZLZv5rLoNrw-@7dwavRfA0epVQGUq>=t18s0P zfybK3Hiw+%R;{N#;~wk1t(v6E91y#!2RbEKvDkR@; zNt6_WY>>tIZP38*^8?x4pD}`hS{n2tBG0~_&r?6WbpGwncwK>+@l^T8>4x>fuO&0T zsE+yS&9k7n^L5}Q42Os_C2B?p%svsHW*94Z_=TvVX1r>tW`ETCvKCik!_tU|ZEyM) zy9nk4&&CKZu610hC`5Pj$>;{@^={wkQYyIAMpeLJHAm8QWFEy)JnYd<&EBJ%CaSRf zU~KnZObJLh^-bCy)HUy{^q@2f*l8VxcOPDd6`xe8+%QAz4Q!9<7Iymje0g~RIM`kL z?m~YbNDG&|!4v2OYI4r5XD@brlGmc1okDgrsOkFFsVWkr8f~#!p1T#9Vy#Da26FkK z{`JovKMq*h+ym7-DX49q>)7^wwq1uzx7lG0Vt%HrF5A6vj4`cyED0w$mizs}Uq=$@ z(4e`K=1P+|fud)rRo*-h$<$l?K{|0gru|`Q*1@yoX5V1Bg8O!9HbO{G{5yjxv021( z;kTq{4F|ew7)Ufao|*i8Z?vh=9sO<4ZtEvuXDj&Uu3=@Nm19dqxfOrH9rq;50->uL zY>7n2Ihix3f>%nolNwifxtu2*<*g#P%*|%^A)<2jEKqArnSA z$WwXKZhYH{XZ%nUx(q77{A1<7@LG9^>D(aCwLr~S9`42u@8S~5fy1$2sKtzMluz5H zoj_xywXF)`;DbDa=5^06^Qc!dO+#X`7)wwKm;fZjNdW2@cq&A53qWJEEg0u4qh&k` zXaZ5TC$ElW%a3`E7+TJ>1rFy5wBcHoS!ZR3-_HcxQiXo$IZL~F%ZA4z>0@zENM+M@ z%+A63jBmzPYaSY0)o1gOot=t#K-`wX>aPnw*!Md9l|iuBhg9eMA!sO}$1($VGfk3Y z=i!yAm8D#h9cT&FjF}gJt)KP+S?9Xk+yhySoh{+yTbiq++KKUJIaPJf80v7QgrcFl zD!C{=jJFl~u)~b}0|r`4)KP$E^%qCR#L2}ABma7jU3tKDF{H#R z=j%S>=!S4m+sa}j=7%3M{c#5y>Yv!8(vh=ElY3GLl76My*1mjQTO;LjZS0s5uXvd$ z>0)zNn)k@xp8bTcm?bcKF(9whh)`pD;Wufc`%VN9Ggm+)Qm`KfD^uK`7s6Q_q7Jtt2gc%ZH58L~0V;g}0pU zq}Md^KX1Dk=i%bLn02EnxecX-^^GsMBP^iks3<==?+lJEjFC0#t*nzkz!ED8ih|{S z-9QAd>l{R~6 zW05N1ZvE6ey)dHcs;qNr#G{l)338}ZI>5BgEAD7_E0lxR&~|eI13D|fpAnvmYV*y8 zcnX}y(h-4`J3em9sL44+N<2;T9ku!R72TR>QO9+e%2}}%VTSpn3x>=wC4~2@J8$?P zZkO?9L5Z+oBH#h_kk-QF9u?nHD3B!Da*Rj)AnXUNzKW&4vjG6bE}*X9xwPAjd_13A zQFjcu_+%12B1MN^hLV=&YQ~WA2JnEEXa=ni9gY!D+D;+Mf=&X0%|x(APJ&c->`)=- zG7{#M%?x^-4ST23-=vJE28S35n1Fg{OK%LdzK=(zi8_wOR771Hlk3o;P!Na!H>0ex zs9*X$lr_{VwilYSy?;-NRw;J&&Uv}qr4MnsKaiuY5{|yDf5T+)6)ayUPldM(W#{uVc2f@`cPo+|0y+De5u^e_k^`7{G|eED%ELVlwh;dVBN5aLNm z_8qGdjxD8zb2Q^d@?=)+#0j)4WGaXySpQ&4&zbHkw!-X2@8NDVV@)izJ6{)WICXtw z_!tLhKm;;7c$DohD0D1}0ZgR0$3pJ{%G(n}NqM1E(tXFK0;x&->G`FlHDRHgXAnnr zvu_=kGL+EmDLh|)H6+GId!=$3#;VtDDQz?ls+^D64?25kWjmmG!H>tXj}%%ohlaol z=K&|r4Rkb%26>73&-M(@_P`mx4kU4F-?s=lOI>)0@1U)?1%*4Pihu>DB8RAZ+OhAB z8#o#{eSUGu!kyE1;j*~@&e|ZCn_9NKHcbv3bkF=Q4JkqCUP|YXj6Y* zzN#Yd02IgWCYprm9u)7jUlDMoX=#mszWlgO_7_9&%Z#N24cJpjD1m#!Pz(Sgjea(26FW>4lnp}5bteelB)d^+&~h4V zW(!|I67No;61x#d_5WmSJK7L!yb{UF?e|452NaE`fx_9pj2G)FA2Grn;`j?Z+~L`({E-l9F6N{uTRFcsJ= z?e&~1G)rOj^;OtChpyuuV7e|g!^bJEgrA9UF6$G=r>*;3uP*w;s-1+L4-Hr{A-$Wl z3*kC~m6XUwVB#NTwK!DjZb-fo4a(CwulXiI=(~}`_ZJr@Ku6R7za>QV4_a~OsrrQ) zA!|BdHTNZdy&C<3%BAn+^`q_9KD`PjY82@tt{wP+b*EYHee6iM(92w!Y3}KTG|(4o z$A;>SSlVH}`qB1&P66641Y!7|W0XeG`~&DUHJh2A(1WExVh14g^@6JQfwL*Q zrvrR5`ox2Q%2-#*D2*lRsIH6vrCxbrk+~nuqr6y9{o$c)#fY>I8+Qvm`}!uo7PPr* zUhF>iK#3~9vSB|V;lw!(HZ?@u+iUC^Ot*~})a6K<3lm=RDPGEioulIplfIZ&J z3%fr>%#2U_CtxCtN7vzQzMYS&w4NnfH$`KM-6o!`+yE$<;rdI z9loWXHT>a5@4NAyQxV9clQiG}?dbw2B2{V}$LHF5qWdDw(BIPRCP_9r4t*9&IF`H8 zjpxduyDX`^qcpw&1X1{C`#Z}vLl_hiYQJ@pVxK1EDV^;}k;3dMtS)sfdL7ryS$eVw zlECvO@C-~};fUM~^hP_MdWzAE60otqS{fOv^Wv!LjnMnz{x?c5?`W3Ff2@&i8Zciz z)hl37`-tJ-X`6ChF;4deDeqf~PY<=YpQ&fy&Ou1`I$s2ak$=&P z>neK0y@V6c!UMf?VQ`MnqMPYl@qEa7A^X5!PVsREVK2anQl6pT@dUMgr5@2~8~v)+ zjWF}+gIz-TlFnKus25xuE(NLWXEZOJ+t!Lg5rB7ZgLAeM*5m$MJ*BvFGA4c4JLpfb3Mi{yS@ z8SLH$u!SqpOZT-;=S;LE8hZCQmPAsZA-Zo0yP1H{peDk`d${I)5RM7x=Z1|&Kykf1 zHGc1RFaOfa6>kvBt5a>lwB}<#lZfkMQ>1i5IIJsc7ur~wIc>0_2zyiAT~HGqY#uGS z#68cM1if8acPzWB2qT(;Es&qjsLm*BGkS%;2%4NFBuV>}HL%I@au4Hw{|JO+QgZk2 zLVnOnQyNq0mMmpwKxomAlJDM;#7A<-j1>rL)qgqaH}op%F-MTwDhBmYZLVk=l)9xM z{iT*|wII$6VK;)(>4ag15yB)oky^%WsklP7NhNu&VQ!QyT=OP1Uv1F~GY^vQx9joG zOyu^E<1Pi5J_=Ayhe3A9mkcC+;jN1k>D^~xH?O8J{YW{H$$aV0#l!z_!pP5;KRiX`!{S45*{dQ?@l|f=-?*XO_KL{ zUv~J&ogCEbtumwk+SnyIO7(+`i)>OIKaP}>w9vXc7Tk_8YnbAR@vrxwtxP2l7Z=nU zb0p5+?~qzZbT&cjEKQ_OZ8<5wVLn808P%K7vBt6A|A~&nyJW05luVJtann-=**x|* zIa9Nr9NvC!Sy?6?q`O69k&aMAaAY4 z=?k@V1-rg)Kh4cug32r|1ynR|kx`&Jn5Or)RT^EU&^*3x1I#t|`9qM`fztl|WcqzO z*y%Z~i=M2dg$pGvX&As5-t2(mTiDDB8mwSLq9P*rq{zrfEv^((J(qqkw&sV;-~oA` z6ll0wgtJ#c9Z^^3M^-SWP8$oP_`>a*saj?Rky}$UN~4cR=BhMGyhW8lm3yPm8Nw)m z^I-afh0A-ZPkNOwn(d;fYhhq96&I?Id>AkR(1jB^dL|6IS;UggPFCmoQX?SfXf8f9 zY=eygWEv^&jKB;vAZx_<0X1G$z$&OjK*zAqn_-|l6{6rmCWB%jYYmLFms2?F&i1%B zGwO2e?by4+PinhXN?q;qZCpOo8%xBtO!GdVr}tkGl1yAYxU zeP5Aj`GQyg>{2CTXt<|aTQS7AbLfEWiMW1*B)~91_^@)Y8GsNDWwhk z^!U8~Z>NlzT4%)t1TIB|%dtg1OK+!^8~fOlveDeiSfW{Zz;si*rYaA0rWGTN-q-5z z`4PK3XBAO}@jN%D#`hepjfYkFc{aq@@O#cY58U<^B%f07Jj@8_xaeX2NT;FQ4 zh7EJDr<6???Mg1-*CtM=SksXc5l zWWi4DP|pz)yom+k=VEnQ^5(h)4IS?X?Vuc8`X`BVnyPqjVilo-STW*?Kt7XEm_8Kz`2!#a3K?fP;RpL6% zvI}V$E%{}~V0LB;QP$4;U1F3IS!c1JZ`ZW5^wTh^42ZhI*o}&Bub5HJz}2@*PAu$6 zq&7|EqS~@&JT8NvNCleTTxj+#s$gtxP=Oo9bQUffdf~5k4PL(K96r5imm8#gshSI&qA7TD(X2^!~K$76OvT)j_W6LUDWhk9?Y-nrRO1 z+y^yANv{>*)5*c>Ggi&%-C0#D=PNOAeyJyusFP2Kd6DOs#?)yhjZXdGGkPD@l7PJlhRcBf%Xb zgAtPa?_KGim}xk9vK$|4#i`QJ6sprqeTfj-;In-^D(=o;wXMVWow8mTG`&w$VRmPD z8P9l-O-z^X1JO!z1Gw>herqZ8A01jaoo9Y}fR>`wKx58##ym=jn_D&Ix51{%1v|Jt zvp6|N|N28?gH8>}#caXAhg3P980sW5gBJ`AC0zZWz&38n!FO#M$;y9)T#stm8&~xA zQ!V6^y)CQpnDqxC|9NEaM8(;gD;<)u`OV#6A6OA7F+14an{hw8-$4wUaU|ll^CB>l zV;-}T!(ZY%2nRIpn*u6{mb24cZb z_RqH%e}MW6X$6P$3hk2AR7x(fN;FAMR@N~Q9OEUR$NHdz<I7O$>yWFfWev;Wstg zdAdd^h|LcMZ8x3ZbZ}J@cl&uiPwFa<%TzcvYDabvZ9On<30xHu67$J32*48k*c=gw z$(m}`fNjJqIB~JWKhdL{CT>nCM{3Vsmb%e~w1nOoxqNaAbLTR&h%B`vSl(rOt#@8+hv|10To`t5 zy0r23&gm;}lHmHomhM;I!EDu?I4lVlKJ0frB{p@otYY06>mlzRNQas$Hy3X>?+hLWHt_A$?wxQJWRvL zZ%BWfqYGrq&d3{s$m61Lf8Nk0@MY8(lv{^T-KXW!JMxj!e5NOFd5qG0R^%bG|B16K z^s9o3t<>$cQ?DvuS3fe7fZL{gqd=mK_AoX*JL0j)0VS^<{ z4CDf@o>6~tMBmWnMLn-&rj`09Y&^rqT*rLX_}eS38E-7r>U;(qmm|4dd_l?X`yDf! zqQWc$QDbd?9(Fwm?Y;h#2#$kcZI0}DLvgXXhCF!7%DFUQ8+C8;fJBiOeoCv?v1Yua zcwksT9m;ne+Z|&D+qD`wo>J#A^~&qm*CED|qcn85gMt7P!cCfnxx%b>$i)xOyole1 zZcl2-V=jOK-WkmE(I;h&gDDrd=V6n>Ro^=6x;gq&c$cFX0ay0EOfHXVW7$+uet9lj zEy={PH%NL!4#TMWn+`n^5X|LI z(mKx_7D)Yqo;w@;6cpn(_3X8dXhE2_-1!xQ{B!6fxZga05AoVRm;7DxJhyWCU_i|W ztJ_^Z?a*R_L`Ns*j>noPAqWmR))KrH(ct%TPA`v-q*I<7cbqlCZy>MJ^2jBhm70{l$+>i^uk=Z`c~T;(UHx)ny6pSY;AS;~=a zxZ?6CBan0ZTqg}0qqDApIP_H_YRtZ<21rOme7pZcQTx9a{2WcVPJWH1`L3do6;w&) zu>Egv!ax4BLLn$aF{~EYtR^Qx{BD5*4FXt+;1-0!*XQiVK)qrZx^TXO{e0S+`M*(D zJXvv>3O4Jc$oD*#^7c8jgF{0UZTIcMQ1jfE!GE0Kf;@S9!I81pJ*_*0{@8h2DeAGC zDp73A)(vQou%7(+Eha^1*2n-fcbNoDRwe)na}tEYCIJBr4PUPQb!2*kTJ*A}9u_tK zbUqlawVrmH8SPz4S(gx{rO$CABbUgco(`}<_ei~7n;ncMo`r?QIABfRjOkzqQ!|%- zNk&ZI1y{n!r2pG#Gma<7hcH(ImW4WmNnI^4{_$($Q^X_w`bqSXk9~cEB@gr9PyUa0 z4HDz)%~55_lcW7RUu;Ez>$~ZiYX3-8m{*(MUk5q=Gk^JCuSmZ~eTr)Ll7y2LUwtxC#TMYHFpH;Y_ zj-j_F_L&;G}g`#*;=zq~xG&a|Ok;vc@j!&B$qnDH1aDgE`A z{nhpVdjtL*0sl^c|F748-V76Tols*9r}2&&GbD@m*jVK!#hd(?O4j?hKdn~3n!vyM zo$XQTXJ4I<@&(=a#lrh{*6YDk`6=cN(3g3IMash^GIAU9!H45D{ybTKl}5yW=;Llowj;cNM~)WX)C+-~aD|6n~eAHBe(NG>Z!_~iYai~PlR zv!eehrAT&5k*uLp@H~i$RIol+DD}}#Ae0OJ6!`YeOwVk7i611nIkCUZV4j8^jGr^p zRQ4>qmk#7icDS?I0sr80pO^%m+0`+l5&35euzWfr&B?FG7L;<*-#9M)YA_Vu23yFG z9@W`j4TZn)yL=@o>J5l%NuTvU+=2>EgDq%FNB-!quI_LA&R#1p@W9X*Qo;PIE%L8_ zm%ap6FueeV-Xs6;3mDRKaiJM8ERKJ}Aor^qnqLcofPaU;zeC{PRp8%M;NOkl-;Lnk zSm57S;NNiYZ-joo`jh`3go6v35s|kmyVTUwoSK4Zocc1AT6CNI|H0vaaz+v5?(OZJ zW?^Lo8OYs@v3O*z7+su>yvQ_ZhGSMwCrbx*xVHBUP_bm9`u z?6$u~f&ysK-7Sawq9VTd%Ui#Er`~(!fUftw+O)wJFliSfzZV_&ic<<#ZPhk!(-3G z3;J`^8>&rD|6%Hl>e!>eK>wKOzu`Oh$at|M2?H3xZ>pTCmDLJF`$7 zqdgA}Ek|zM_=kx<`A^{T|M}fN&n*uQZPXt>(Es)M?3cHX>LM2x_L}$azy8O+yea8N zz}-9j>FMLY{LBCF0slVye;@uo9J>F`{Qsw!Z?9^;@$(x+_T&yyf%zq&1BU-$j;0?8 zs>&0sfuxx%+~_eh{$%_poU-v~(go4Y=D+w4;M8=-(-fp>!C41eTNif83N(@y>xY*S z2i%jj0Ic_md+c}q7Ykq}aNM(p+nac*!%l z9QP_47nAMk_`PA^__6Q{`saMuA+7sudGSJF_zkWxpw(e;`)!@JllmxjT|tH2p?4xI-qhjIHqA_R=_ zQw`1dLO-A$^5}Nf2o^~@a_bu#=aX+D$+2J z0zB?3vF8!DYrmG%E*16>hL3mOoq4?$rj@h`+AMkiv}UEu?1i92rfT4ro7h>OX(pCV zxOLPNA!>QIR1RHdT5I&o0z{KN|8S41M63HBF5REQ`%zS>Qb0R;G0*DR_=U9|3``}v za^RlfQ|iN=Hn|B3hE8J=w!bpJ4Ib`-#L=M(YsAa&YRM9d0@N)R;E1mSz}(yOM!j1O zhh)+pC2UZ>cK3YOVW1UadTYz?_03f8EA=<=g;$1XeOH@rRM}AB zYr2qwo@1ZU{gNxA#IRIBJ%CQ_Z{hu&+Y+w0r&=DEnrvkZmrO5WIEl$o zjSsIs1XPb2FqxF0d>^#s4&urkfM#KpB16Blj2Hksuwa;51G94cVoioIq z;P5t2I%T^C&;L3^&c-}U5HFBz9jcV)8CT>XREnZ*rup};fLZS&s@isxB>zw$UFNhu zu?`zay)oot1#~o8A730c#ZP>}@X)Lp98<g}5BA?w8hF~EDsllrhoJW5MFOUZL5Ck}@EU-A8) z9+&*tlV6TvNw?aD4=8*>N{>6zoH*JQsCz#4S@}Krm=KIA`6Apfy7QPL)^Qiu61*t% zYi&W8-`?d0`gpg_eX!c1PPXcqr&I6J|DJC?4m#sqHyl(vCqo|vbmpWi*CDorsfM~F zac9$r8()YaJB^vlx&~IUjD$N+M;XkG>`jS)4EmAv&ePvbnMFW@>UArx=igHVgsFiU z%xTz9M}^)hDUu2L;x&^=6{T(Adf)f0*K$IQXAV*KLZPx1kTMDPIeHlI7sAegCZbqTi_|*?Yqq(_W^4G3amwpqm?+%b62-FunRPu``T+LH+6=HP$jyoH})z>Orn zLKQZ&hvt`oHL*5EU6RLDQu&HGwS#KcZ?x8aD=5roPRXtti&{bGXxTm6e%05C=_aR& ztAB($fga!C-9dod>LIoKQ89wJ17>%{B_hzv-8}xYh>oweQcIJHhU!@R)itpEUenFK zm2~Xq&3?h(f)W{f*{J~0_)9S{9rq+9p5mxoa8d_b$b%h|2R*4W<++Xm8uTw0m}q8; zp7)!PH{!p_du?c)@C#-L;0c8oOmFKk)%4X>klE)Qk+)yHLpF&^rQ0+NaOQZ@Ab=Bo z%+a605D^6kYJQHx?_x!Mw8@cOhbQMfZUzHKjl{Bp^+gJq8OW9VMi1OM#M; zfD|s#>yMD-Pt)j+sU@EoD73^aJO;&y5nBPJ=)E9+z^VS8Ha3x` z!C3_m1bI`kChTV&B<$nUNc@HGgMw+^%V;vMl>l3Xob83zNDM{K8wGqkUSoE7Au+s! zUSPSM4W(mek*Ru=ZHTo~pewiO@ic>zpm(FG*&L-#sbnAIZ6O@(S0`$mKb6JSCZG1F=VeHMZ)@S#|H!p0o#|K$`uCEb(VX2kP1M3=WMo_}qAOA4cc*9+ z%PKl4tm;)S1L2MAOlnj$|1KA8h2C7Laf@XA&%_(##IhlEU>Bay^SISpYS#4CZvWNx zb*PZ`dNe6+_sM>8lz8FQG$~X1+5_)>R9eCFez4avtVRP2!&&%(j;?8gfK1`Fl_K5v zsR7_@xzs3tQVALZzyol2!Rfjp`3Tp!Hfg0Ply52qgvGi@I9tG_d{CrUm0!2;padwhq=@E}(Z?$=e;+ zR!zwp>XTp5ZH3n{-XDYMgq1NB1&o<_l#GH+taxs50vVv*Mwjn}E>W+Ey;gP)Y#q&zC}1$SYd8GG)ex~Q$AHqq z3z&MGEpe}kVy&(0yYt9YvRmun#oo%C*NM77yDRPmHseqYs1qyjO`DFtweA&5-zp$I zY+C?!6=8h0@Ytb8=SQ67q>ds^V$uC=+!ok~RJ{EAw@MK)G2!R{V709?Feun;%q4x9 z(x^caS7b)AwlA||Dg+=2W6>T6)!P4qxbF;WGF#Uc1XL_I78Ip9C`d=7NGJgdNEeV^ z6#^nnfe>j4Wd=u)q97fD^p4UAC7?p+p+`DO2_zID)Ii9$*n7{}XYcXMoPGX$zu?N1 zcV)e+KJC75&4jM!02A~0^-tCWHm_r4{v)JKo^tUvL$WgN(qa^;>UBjCt0K#ANtM0q zi%d!mJlDx|5djJhU$W;d->w+*8kwtx-fQt9;vGIwyKWMm4RgbndWn*YJ1dk#Z8Sc8 zIAgKzB}rqLZ2?Z#Bl3=I>mW3Co5YbnC6l?mr(YKY>^{T?|7r!@a_NH?ZaiH?4R3me zrNJ}(%ggm!>d$QUZQfNUnpxK5a0x+*sAyqNg-WvU6^ZU^H}4Q8N#9e`$iF;eapn?c zt(#>Np27w(aiiQ`%Rk6gB~;H4!u1rF>DjG;)<8B3?XQ&Q?XMX=~hr7u-b|0fAxlN&rq7>HEHu_%M9xS0pOhD=dfdTNeW2 zHdmTe#KuaqyhYTuB#^GJ?ep-A56GZB?0;!c?qT%iyct@Q$noP~R%$cT+Eji?kTqcM z)@ui15Fb7V`1v~mQr5iz#jVNUp0d4wAX6l@4{gua zS5IGG+JT~PhvMo0rQH(XJwApLo?A={6=QuqMYjZm#1Cd&!i??tTnrrVjJ%Y*n7j0B z2o#)-?iZ?m=INX2o2Ar#S=5J<4@tnR0Q<50>2sM2yMS2%e{|`#gk$gJ&5?;cnLLBW zm#kUu8(@X!3P*M_K90&#m$$yGoj=vQ zZ&d(GcJoRdCrgxYX)bfsm)orFa-i&XA-86aN=^=^lPuW9p}Sw%J~a?CJZ^nxuY}D# z2mG?z_@#aRRlTq2F_}|e*v*|^pIv|hsC`IaqUq^Fx}J!ixWv!Ax+u;%-SgpR?q6~z z#h{T|xL^0JDOL$XMQY$Mt!Xv)VD|{9J8d?Lk(T#p}w}nEk4szF3#E{;;Eg0!=qJB^b$zTF{|LyGy&E}{YW(=?_KI(F$f=Qq!!-(oez z>-x3hBrb6lIvuq zam4dn-o6SwIWBd6Z=(`>hwF&fu%iG1a9Zc6Ie=Gjf7S@k`_zB;!Ufe9ZZzI`S*eLly# z+CJi_*cfpIUpZ|2Ei-OW%<6ejD3Dp$`0S*>oWKJRPY;NtTYl9k9I0KJgk>R&ec#IVLMc8Ra@BDc9N;`&SG(V9TqEpudaR<} zF@`22`OYQZS;E~ql{OuF-wWrYQs&KYr*Yk3tg@jF%9o>HbY>K}^Vfw-h7rkFYOXAycR**tib^XV=7T{%7o2G9(!PCOTNO_x3sInkpX zH$JUX533LJLU8s8E^wXslPUw*G~5^ zFV$DRAzHeX`^e7X!is!jV<=P?n`iT^XMD%aE)YWb^%kmcPK<3OJHf5KLv>AQd+A6- zcX@Ru0NoP22XIv%nS%E2gR<}+73wJS@};I;8Os}ZDVe*aVsoKbeD&tSJB@{-SX!&%ALN90-Wq&?7`~SWWcX3PPEUpXg*}tQBkUf!&Uf zLTQWmsI|SqtGr*0u~fq|uZjyA(?Q(ecr(F|wdx+PihlQtfp%MO=EqOXUmo3x$G{bI zgdwpe2NID?uMYL@Pz2BKE0*Yj@0YQ54@enhL$S~T9?2-+!GhKw z)-fA=&>Si#kUxO1+Mon5Mu2*}|>yEeNP?M5f9K>K}8BZi0cO#wT# zH+b~~oxaO(5+@Yt7kBm=GA!S_;41IUWS_P1zL$`I@Qboy#PWN?wYpH6r;$W`5tf$v zxDJtA13#fFBzf=N2gXvtrlXG(O?uYzB)Qx`wZBZ8BJjj|p7aK6xNuSW2BX5i0z2Hh zDh0p|&mB!M0^7Z;VX7zyQ7P5GdbRe5{DYwNM5kNT@9hK$elC)%% z>eE{!U*>1&VWL)Gu|5M~aqmpgbe*rw5?MfU+<1~om*NzI*Ys^(9X`=pv{J^kOC?}+ zXwD;PDvgJe^ZOgyCk6-N)siJb1xZrN=f(#3bIZKAMb@b*$p<}>9bC{o@DJ9`er67y z+BH&2bfKtv$fe=2o4Yc@PpXTy3zfK|nCmYx#hX_z4#aS|EJ(wWQJyzv{R%rM)h|mr z(BDy%)4?m!6T3Egv5NH#V|2V$ay6-)C|z!y%=%K06!|@WgqY1Tm9GOA*%~UJ2x+A4 zN!IUrJ^gZxR}{=bJ!P36F*z`SX)Wz7M$;f1RuIO_l*)p)Z!uOPGZyc$^z#f5VvK65GX zNUx95so85W6fd3;Ke_fP?SIPo4KY%ChS+TGzb~c-7u54sh)5ofgX zJj`p5%o4`7XnDZSarK5WLpJ7gV2O%Q>b=U|LRGz!HD_HSXgiw-R{gbu57*N}7zdu8 z;CdXzzHZXyl^YNMmZpGNk)%X!Uy!7VV9s}-i^FQ&7s1EMt^`b$bjLoHGgZZ}eA$1? z*v#qJYIA;cha+Z}_Wm}r zyKXuBhAH!7VUO&R0CO4HL>%{`pXhUdbP?|;0lpym`-}!>lyh}E)fUX!CrTJ|FgYnh zp3g*2)kPQ+Lh61O>8BRl-I`W*t=$TP_dAm}KhFIf#Luxk}*%1>bK zYX?%7swbiz|dB4luOrZ+}nWwR5J3zaSFhh9kPC2F)<%U1FCHy<#P%v z*7S92JpncBV=g@H#A_dI!Y9b!V?{@F1^IfRfF*iZQmU}x7HXSX3nLzNdCtmc+#`2~ zA(ypiWwZTzlO)afg@2)zwq~I!;ozRbf#i0$!hp9ij#WM@FKFMYK8{ac?JF+{df$I! z9l(oUkOck4-$}$xR#0% z@u8;{HfA^o+tv60?-)t~)J*W$Wme~MuewC4ca#CU53CF&ei&knJK9?r&+UzPZP1Nr zqvjK8Tvf6*)#EQ3N@L$oqnJnNtIL!zB1N*1cxz{{MtiUT(T6ozNbxHcYzsm)-a2X# zF+(2Bq9p_)*}fDs`pe*}9r#7v4IE_g6DMoS@Xej?lGas%ykR(9p<=($QDtqc7v6`U zxOJfj^*v5kPM~F)J}96$ZX;3oR z?WbiC#|ik>qRs^?$IQ1$0E$=0h11;LyISwOlX4fW2|u3ZkyLbaf%1z|zK~<^(wi8g zg7o69cxoTwK4zngl3C;gcO7uh)PR9pc=0RY@}K`bkn^{ER3GC$*3xK)*b#qPk-&yV z&O(_bxj_KDhfEE;uks7~_Z*&(v(M5FoMwJKugkE5K=gb8JEw6w2K;un?XLg{4Sby1GLSQele3|#cgoSBt9=gFS zBzV5-WyK{}X<3@Q{9Hv{Q;dG4CRE~Xqv8G_rErVmXC|2BkQ$}{KOK{Vbw@EhYBk+_>X{0!T@h0eL)7*oHE^J2!3)NZ;w z_ZbDjtx|qN)oV~?02%KC0rC&+c<$oVue6O3@w{a4Y{19?lm$U@{=zb_Ub4_#J7x#7+h27i(CP)`C~gntXFuCR|GeXxgs|n4KWV+||?R+Uc7a zseT#{7>5Z3bAAs-;)9IWMvLGBfj1k6SKL8j3;4a#aD798)54gpKA#f2Fe%BNVY|&e zH{98%%#BP5cR4hC0fP~UqDGvNO zW6w(XpnQj{kJDmybhPgMpu?a6NMj+|f?I-Co*f@tREnDjr7)B7bw41)^PK1?U-X&W zN6E0|B`m1xzvaj#^NduMF)`m2-Wp-U^sN(s$1?J5IQPqEWyL@EoHqheqfS}KsuMKt6>s1%;t9>ea^LK-YP-WE5 z!lk=-K6jZ@=8H}6_tN65o;^kH)oF>qTjnFN{a#$M-w3DPR?W(abDOhXZJ^<9L8`EI z=WmA|n)yeB_SnPCQsQ#y(_r7NF|F@DwSyoz9QXDJvj4kkrN_xs*O~VZLdPBuIuU69 z0nX2RJHE3u@)71@$DT}(eYrli)|Dres;x3Z8$x~W0mbfus$^Ya898gU+wwqHOZ>@ePF!D*Srzlo_qKfd7GKeL)3)5p#gm_WJhCnF#4-x-*0?*>`M$ z7CkvjQ-$fcCH+`B5b#j#yr)XeFh`U9a$x2$JC8!H9AxcXYX`*qX*PZri*55ua?E2a zIkK6jo%}OkP+{V=*5PJbRb__9MrW>OIggSgq&;Rku11=veD^x{GuJt(iPMy4oy;@qd)T|CaC5kYt{1?PKw#v-^8<-GXjC7hHG}J5wY_Xz zCS`7!zH%imJ!$E_y=e!HpgFX(+x0BZU?hxrYDCi0)!5f1|-o2@Lm#H~IAz#76yM9#2_WVTf4%=`i@cfoGQZ5u#tBPK1k#^#w&A z&rln`^3qYlZeG+IbDPae$Mux=rc`9MPJ2D}Oul|?W+^ycTd%?=Y(lby6D`b3Z>+pd za2P>VUJ?c)Vp?yIEbmqvzVzG0pt4;Ry}@l1%h3EGRCd|bb(K1DauEHfDBH6|Lob?# zn&&HEh!rWyK7PW+9JtBD!Y+hIzQ4-af|jeSwhe5Sg|VzliR`!yHqJ-rV*(&MvqGef zL^oW%ewX=*%>8dZACicy$>$b)#@%X^tarE`&cMIQty#5Io?M9;N| zLzd@vTxq7;Afj&oJv~<%#M52u?WvgVbOe6Eu*dBx)%C`n+zr3ibPe{&t0t0{&i>Gw z0$A_puTA26=gO!mUtS_bjP9xH>ph8QW)(Ity>?|@({mtwL89$pc8816TDmac0KaNG zxu39$w!GJP44L|}{3gQ_)`S6c$@bMy%PXNlu5VrjjqUP~0HN^k&yoC&^<_^ponuM7 zHjU=2&4w|znU4zv2y@>>SNMe)-^(r$G~?G{_7w6hDv@almE6dqyc%mZMI7l?%3Z*i zMIi6p^0P`YHcB0=$doz`1KZVsWA$vtqrl<>wZNN3tJP8J386txJS|?J)7`n~lq_Shm>2|xL{RU5}%JwH4MEqY{OrZ^)mt##nU| z?AAhm3v}L3I)&Zy(N!p*7Uh{L-+rMu* zmJ^)kMU8Ma{^N$`o@%eR%9W)aNOZ?+WM(+YIyqGrZx>o|_c8m3SrcUz43@`}hNsz? zS7zDjO0TvgOq}dYE!Z~lVk?<9pM7?b2o)mNNw%1CoU}`<)Y_8Ki4bmbO_h4;GtaO+ zQea@AOqcJITzvbf&9YR!B)9H@$IxCY;`905N+qd2&tm3yO2i)7uael!Qt61D#z=8x zQW-OoD!&yLe_nr>OJt>&nhg6IIuL;;B)aESl0x<{(pN?nO;riq-LWw0*fb{baxUBC z!RBgsl6MODVi{NaG=>q&D|j5WgFIm=U8vN2ZXf;l)xIO}hcyi~=k7I>SnrBw(3~-C z52qYTR){OlMI|P9Vwc|sU9j?*zmnY`<^Q!HKv!erWcRsE)}jG4veuZzl$m30aMfoi zc3<$q3$^u$hqX&%p77eq1j>4a_UN5A2C-%mBHEDDzkmCjXi&JTOgL*gl&a~v%oMuj0pX=dE2no#O zyXQ2b|K=q!;QsdrT0`z+D##GXC*5br8RWRlpr{U=c>o>BAoQp;>yw*>dL-hjk$!w6;@ly)YDfIbQpe9hqB;M&%*3kS=!)Qt7N+y ze5sT-L7t+CHH=TrMiwhWg^@N{A)@bfpcPKo-s*vuv)X33CT_KY@j>o(|3xB#-_d2r zQKfCqk4wJd*aZ#uRbtP3i0T|?0nI2rU~!ZaEk>YhC40MT?=&1_v#Trc@Gz5yhh^G~ z?=w47a$G9MeVn$ke;Q`*|DYBa>BxmDVQ-FNXI#NcfMO{J+i*10Q%B$Ep72V|`On#v z^S#_O*x7_~#lkNAW@C42;Y`cI&U_Cv=0KA zSc3$rNs|eRMTYT2h^(-D-C4^|j>+@a3zP(gxJ1fa=DV~>3Nx_}E9Ftx&EQTN<|s#n z6H6j)Y40K4+_y|e&9QHO*plnxn{YN6-6D!jE`bdr9I2F_cPss6*T7(-y2}Nim$5bE z9eiBo(NiTC;hi^-FPZ+jUwH)Do_4yCf5DA1`l5<&os67F>Yl1FKBvmCJ;5|Y=82Cy z(lWt!SPE~bM3X!{S~OqeHn%TBeU;l7gyndQyA&u| z2$0ONW43$zm6aC*D6Kz1VVa!MkE7c+51k%kut^V-VZDByzwRfPzoCDE7WntWUv(dtF%%q4<`|S4s17% zq9Kp@=w)fH8roE4$iOvI6ej;yCHZln1^bn;e3`zM4Gk-qjVT%1o(R4F5Q|;HkVk(~ z1s8gDnhAN7!5_AlAfb#poV2Z zz<%v}(GwYM^P$*Yzy7X=5vQ9hD2YcS8k5_CQdf$S8R?TOcl(sA7slCw+71<&@vuyp zr*2O!_tzL2=d`y<%X%D)N$P8|PTF*#I(*(LbHsa8ig=6pOOyZL}f9vC4-<0nYcJ$JF@4Xcve2aPwsHT$8HmE*|ZAHQ%8wqwu}i($W-MWlX!EN;wDPgy`yAE!KY z4Wt6496RPIvYHVsC+}5Ip00HfxuRm`a8|GHyJ%x&j$e@L`PS9@*`zjS#eK=8bRJqx zQ_z_waYoTif@n#gIp5#9b})i@$|=oaD9u3FboHiOg(h4hG7Q3Uby8m}kHEIjTLm&p zxMWbqRwym#W698wptPuV8P+p^mW&aO)QwM|iFD@OjemCbk(B#%OO6$b5Z}FaaE<&1 zX}TJkSRu;XGN(s84C5$ItAUmoorya|f45&{>&41xOneV~y5|g?`})UDhG*A(1ltMQ zIt&99)-!Ep?8{rTvNFgnh53xA-PrU8%wA-fIS3&yV{xbFnBc*;5dKUN*Z7NdU}dQ; zn-YxMp1I+CxuR&3I?QX2p(P!)3F9S`xu+Hlkb=0UpNqvEpE+9XToJ{dnu^x@f<^X^ ziS!St1jb?N?>pRJUNHXoQyLv9(zgs)Xcr^GLtt1Ox9=|m9Z1m}#+c|j7SLW!#d%NVQf)}fPr=VSDcg9|KwPS7t zLTF^Mreckja;V`N8J24pYYchwYS|DL>V0wHTVQ{~Ae5!vabgV~~h06w2KdtR_ zIrH(pX?ndS_FYNw36EgZwfP>A z4e!er8kG}IfwEEU!V?#Qe`sT^GXny{>VgGZ>eqgJ_cw?wvE*6`A)CV^PW6S!%s+Sd^83%lwUI*EnNuuskwx`?UQ(VMhu*++11Kis0?HcHf zm-Z6q7J0sBJypD@cIwh{#F&s%xyUct?+2S^n z<=YHl#?bTZr=67VpE-Hv+Pl?vKieDKH-SHecXFJGC-{c_XV`|IKrw+vx z)1{N-vzJGg=AX!GLy4Q@BFtGGL&dW-%f90pwl8)wE<(tD6$mmbVWvzM>ED>Bi7kS#&uTU~FUbljrPyN~_MJzd>QiYHD z{XuOzzS?bLI8NJ_vg!-;*oVRia&kY7yQ#HKQib$BWx`Wof<~@fa?w#;= zQllCnUGFr47Bwdw!hZ#-RuxhF0lCrpB+qLnNP80?8Iy{SqP?@iAr+PhWVB1s#q8;@ zQ=quiSW(?6g(B97u@zmPXw2m!tS%AX=p*WQ$j;la-A6SkLu$f<*m-iu`CMtjSo8`v zV*R>R_=Lqd5+%d3rJ?v3-W%=OwQYr(vL8NI|NYla5L4~H9OW+8wVwd08*(n~OaSC~ zW%p(6Dv_G_$kSoAx1ngBQ*r16?WgSG$wW}^w7!8!GVGWr-%m(KHJ=%SNfGro_gd&l ztFJ~aF?3wFBJwhtJax$^YP|#c25vr-J%rdXgaC_>I$+tU#Acqk{YjC5l#h2yP=h{c zV%WY|P`;qVS==cbH=?lePB_I)$Yr^%IcXM;vcB*eY5;!g(Da0&1gR@<^nCooy$1eH zuG-v1{!zCvxz0d&{KYNP3)A#5;pN9l)H?Y@va6YyTTE0cU3+QFb~g8R9ITyt+22XB zs@b>r?rI3eZza^)dDR)(UstMSzULN&_pUJSS%cMMt?nY*){O9h|C7;b{C=Z`mhmJzxMjW zQ~?#OTybdYcS#FD>nDW1e){=+g5%hu&G<6tcB?MQbWJx%c>@;IK#5xT@ClZs`(2Mq zB(Cye^c&}4xca0wLVXNATeGw>h6bW_3ri}CL_a<2Si4|uCl376_|zghJ5eY+tJ`lS zN*QwwyVVqIN5Bd6vkEVOQS+4en5c8txIupw2QcQ`9``(Dbhf>`AU>i z*GjhHqT}|L7_H(Dnmuti(9O5@AA`bPzeZDll4 zIAXiC!(5{)6KH>asHSZMM*p@M&196xpmjm^xFJbouA0RQT?wmaGK<-9K0*?iQte$N zN;_U|PwedL5A20E?*K6XSGgE~0I9fMkW#^9i%>nW)=#_%O?i9j(Xepe#%+b2fst$R zG}$pvgY2ibV-g$cmnLhNRQBf7proD0ibOfQ<3Zu5pRgxB?tqVmk93F~?ex&AMj)VS z0^PEki?woj(G#>KM5Zx(){x>46gt{+BH}6tYH~ulIA@fv*!lQ4OlswT+@giZN!?W8x( zEtRA0nu?D!sJNZoNRO(~qcwD;&WXd~zC-OSb@3S%YB&$ExBiw5q!1aHNy@wDzA@r` zg*o7;+Z8s87`lP{6lgAq$+Liut*#rV zuj>RgJXxESXyg>`OQy zmTpbt?z3EFAJ~nSe4~nhnT@rcGx5qy-i4X}{D)l$_GzCoZ?UoVNwuK-{b-VkQ?&Ag z-P>GE#(IrbXmwO|{Rxq-Dr-8G(3x*dBURM*57B=0+B3C^Wy6g7ZvjN)Z6DS13V|SmlePSuX12cdraQsHqf=4?5yWKw?L*nm zFQr7hKb}tMjxVtLq#aVc)X>y*+fs3z+p`e>{`w+*kp>@DXp67PY~4z?5uRLBWB{fCx9j%W8GVP zTUXhiDR07ydsc^vvh=CakI;>DI1a=#V zVlPW-KEOlWn0%l+Y@gc~Rw?FawY@D5fi7<~Pr7LPW4GH&Fst6_<*_d2O2Sd%65~6A zb_FY7-yz&Kub*2p z?E*@FtDemk%^nu~Et1DA#4vMdUfV~KUK%o99|U2MiU+qC{WR21dT<@P7lJ_8#uxl{ zWK?*@a@of?3e7~j!h4nYpesGvZzVdQ=()o5rA49%f#W`U>$89uV!ZM+b-*|<{+UP^ z6EER8*p)nD5cHD;lVKSv-XO{@Y_BC8>R{&C=Bk&j7`%Zx0)WwNM z=aL=bEezbDct2CTouN*8roQ{4p(_79*^ez0y(48@kC*Jqx=LZ*S+$cvM-zILr3tH; zz$c84+}S$nF5|1l0xRGZv`(3kg3fw0Hc-C=YQ6d=~xi!pXdY=siFC}psMTNR(4Et@DX9- zk5>U;wO%mn8ju*A+$gk}=`=i&wzuE_m7hj-m1_Um!Cx%-hJ zqen1eC~>&zu{DJ$4*a8i;VZB1XAw0DQ!~%$>zcU2Vo5&8cXM#C;^LJtz$4co_bIM`4`!yl!JSzt!AM-#ujzbuQzQ*4UFv zebK7=Q;MrWls#B)j78N{Z5&+E2cuhg4WVa`#)n}Gx(Wf(J9x?RLwRBebVs&C{ zT}Q!?>r{1TNc!QQQ5)&A(sad0={*5sw%fgI(ZyC?#vU6p2x+znJJ(X`u3RUcB?w$1 zn=loUxmePOvj?`ZI!e|Q^`x6h-x>ZkJMJ_-G)lDcNH~qIPUTSW^79pFM=r$VOHI_c zJ9eW=)_J~v9ZBqD8GI1N?2%yCshb|4FEe4IQ`!`wN#Jck;ab6j%h0oY-}a10 z^Ibpjt2MlJb59ce^#^ACt5aV zdHQ}8K%pVBv~`0j_JI&8pH_8enM7hbB98>*1mjCy?HWKD)o0p!{8aL_>Xk3pEC81> zm_13nWxxDE#vvI#=*}QqMeB$0|(yNHc2kAkUZUd{5%BOZ*-- zt=vhrSHfgSyjfbl)34QJ#^NDlh-xo%d`gmm>qxZF74r74;(qUrOXUQwP=3{8WQuy~ zclnA+g!+{H?_2K?Qh(2l+weJs=58goOz++c41bvCb}0;_aA*K8HMAsU$C($s zW6FURpQ0@ETQ>W0sr$EfnN7q>ir`J*kxxUyHiyLFTCr~&;+R%r-q_Rtp_WqraCI9$ z$ptU$r97<>yz2$wf%aC9CtWMi3sFhv%<1|R;xA16{3maaE!s*`)(dndpKxT2nDY+t zYudijdQD7`>OEgHEtl7IFPV0)L@N8M#yG1{>_Wa&{lbZ@){9}7VEXB9bVCnj+6+x8 zMuvC#?VH)8)&!t!jdeL=$<`rBQLqYHsaeimFN+$Oz!Pac$*0dJ%69i(adVUhQGR*k zk%Hy=`hmV|6hNpgarf_u>Z5Is6rsk*R;uJ0UxuZ4G1F3FRJ}FCk1?E9CN`a}y zc#uiaA=no?1dg&B>EwB&1n)zv-LKvruWy_LvN&W5f%Z~_w|U1ieCDpAg|)o9q7Gm9 z?(yWQ%kB9StTNnv`v(LbQdB@Wkt7K`^NUHiEysxSLv-txt)qdEam+*ze;_&Y^2pvD zx(C8zPki~pnOEq54TKUL8DG%n-*M2`@l>rnahk0oMI>U-G2N1v)aneaeEB*i3u+c&@4d{w`Rx1KohlY7&S!joHPa;syV^9h z1F%doReVFR5Mf<^lEKJ<$z4iy&GgYW)E*J#di|-uMcra|pqW>}u@!YB+gRPcg-PkM zN$!b+wz4By?a-l`gM%a=Ot91HmD_@2{9}DF^Pav(KZ`e7A5qT3<}9T^aE9Sg08E5& zh-ulstJGB+mUx@`qf{^Dq8fu>)(5uQk9UTJ)f?4ACZBP$UQ6E!?8&rqILZUZ@-1R;hEI-HQ(%sB*S?v-$F{y3DZhR#i zQMOsYhWsAv(p+H?3gl_SB^^f=t)3s#;_tF!T`Lw$v)_3sN*158rK!V#(TN& z(z*Tb$y9`6X|C-A$==qC(<+vde{iLCZXr$%;q*`>Sf8hU>*bpxE2gDUMA+u8;K@Ow{-m zIs}Q49Ojuu=%BdNiKz3w383=`6gfehTG4S1Gz&`|Y02sw`cO&biCTttVvhZiZ+aCi zFWirL<4^!X-eZavAHbk9%WoI5t-{JAmGH&hJ0|{|(U|KZJF+U(-ii8%Rz(?=`MMMw zrC_haLCK?AWPN4WzoUPuO>68EhS=&k*5`>@B2tiuG4CmZy_gp1(wdAFAcD!GG-o!f zh?-STZ%hw|JgyG;X{>LyTCtr}&SRi^K|3R*zyOh3%)vou|IjfNUw3y6J5tBvBYKHw zv2@T;B#Kx`b<3CZhlbaT>h^svA<(b8i|g=G0ibVVCVX+pWll+lViD=2VW|Uv<6RyO ziYQu=o-Xvw9wuc8J||5#LG!FabEWJ$ufvPU79ZLss>krAQ*yDZm71;16N}Lviufbz z7d0{X`5x-4^>}P-`Ga6)Qowf%R;zsCQ!EYZxgKb;ZCPQmQLS~o_+>$o`FiZiQIGK0M=7;qLW&2w4VFXqo?1~p5Q~Qk%7;tnnu8S7r0LOw>|+I*B$6S9NGcbF)k~eq zX&)t_-f`47H(4D5ykU3ylq&(= zp6(l_OB)-x*6#_k^u&otI4mVx9PdlXdu&vmu_jJVSs4(QrOe`;*pKZR*dw!R%20c? z-r8OZwcZD1V*|w>4x85v6f$nZmwlxQVitRb~DwD+L>;xu`Vr0 zM?8q)i==IWdi>n&DIWTlB^=W;{x6~o41A)w>2_%y-lb_Qmp%<77JQepdebUq-5as- zjI(HI>FJcBSZR2*)wCbSvKInqOQNHOmVx%6+eHOC)PS|jaar`!(*hC(5t%Peo0?Jf zxe)j{@&RSQ;zprpdUfTR?U*VEGaRJ|twsBNjpuE&BJcL0_sVE#RRy-=k9vf^l)S_=T)S6piOtIi>wc9=y0q|gkdVDR-W6*t(t!G-cqkN5uhN`oOaTn&1ZAx8xox5_I zt9~*GQ`g#S0gcWGHmWz(E{B4URK&y%fQx$xV~Z9^bL_ad`N0`gbkGzR{WYknq+oQJ z@+xv>CM8LPdRkARRQH$1bkYLuX{?7V(f@+e>}aRQc$-=Ceke`~B~SF04x_7ijEAjT zv_Xg{u4tJJEgpVGJRO4h2)E&QfC&W6H=$Fc-|;sEiFNV;yFRLea(!;vt&E#j_dE zUW}Hk93!pDO?z$M4Av{QUMk+N&l$$1uR=4WvfAiY)X=J`*e+RA%RAk9Dh_sI6&jpFm^%G+w=&WBrg z_+AhFE%e=>A@Bsa7-eo%S1)e7a!|0XHhxLe{A#BuR?MTT+yjJw>wh}r5`%&N(Y3cv zX?Vi0ecjE;e(6H&@(T5AB=mE;j=flHsA+z;Og0t0t7uYmM}`-zdx>QGsAB3 z-bSSq@Up;V+`E$Gl97tnjcvL{rDZKjJlB`!r$4>v#TNKB8)HX15~0PW5-8+#Y@g*Y z6g6IJZ?#jZQQ9W7x!aLE(h)D89WVbRRYcLG*AkPFd*hiaz#MvgJqk3QF{hBH|F{g| zlmh{0S9$pks-NlS{Di;tt>Zx8J}XN#KzbjlN1FE*VE3qth{xA$$_sxGkqvxnlvRaZ2#kX!`}f4t!S;**YN3_yHiSFDWTKOg6_V4SmW zj!FLU`1!-={A=Sj%|{QnpjT?TS$;STe>V0F_YZ?dS>f%Mf3Q^l>_KpHh=ax0ynOTG zfBGZ?qroNcs8GLX)PKB=?Q0uqIk3z!f7-QwoufGSK!Muj3PI~b!pfkt!$*@(E_`NGk=_~zxbrvAXQDder_j4pXJp5W;*`mJAoSDVWhN?_oZMMVrXXwA0&>#FlV3zQ$dR{*q(|`WX z=U*rrDz{Lwvtl<<;Z(qg(d$Fv(UjG7#Oe%IgfSfYnnm?ZvCi^^8jBPE8lzj2eJm*zbv&+sn``+wJ5Fw?9Uk012p zE9g~jRQGe0>pMRfw7~kf7%lX7Vx00^7W(tyYc<+EA(XK^7+5$){h#s1*RgDMXK~q zyY=HyXV?z^QH%e*Q$Og?|BqLUlMV6ixcL_gt?%w6C^d3&1uUwwKoZ-9UEFF^#I(fj zLSeGu?+%4lT|<(bN5VD`h6c=BKBa)BODZ2o&G#2aOaJ3tjac_B{wCM5pZ;_pI*k72 zA2n{3!gpL`|1pj>ml+-3KLd3QWl+bKxYS+1d>n3P)c5<`gzPU!k%-rS+VIJ^dPe#V zmOe7-N9kN-EdpSY&S9tGu>wHYt1a0e^j{CRGo6Y=0W7!de&v)0Ko^u9zW|cqUG{+Q z&r0UC(mz&j!+9pA=?I2Sk^g$|hyTPq4zz35mDt=lS!cnst{q&{lAtPPt>tpo~rA zABT3_5aW?HeCd4+?}~&WbGyaA-zR~&r@%xQuEuEo>5>Y(#ySd^xE7NDW6Q>QCy9%{ zcR|@ZMhI_o`}afF_9xI9yENtZY5x85e;sEVR_6T`Dds<5Xq^19{QaZf=cj}HDqeqP z=&`+9SouGqB3T;B%6s=)&BcEI#UEc5%~#*swZ8G{0tB$HKCqPjm$@44kh|)9=a+vx zB*DHvf_m54{=Bgt^t!_bJXyMT^1zSJk3Tye|94~ZgW+!|I0r_P%*JrwUx)ACzWVC+ zB&Zs8-H&3t)_Qz_ktyp%!{YyA@2lgY>ejX;L{Sh0lU7MdNojac5J>?ENu{MEhGq~@ z5T(08krE`OTND_&YZ$tRhGA&FwLOo%=X;Mn?|Yu}{rUYjzk%6%ueH~@*B#e&-6dnA z(1wrAZd=j{5LzisDFdwEQ8Xd(S5Efdcga#Pnzbp;9a`E_~l}NFY%hxNvo4 zz$Mef4VUmuTQwB|Lpu;i`XOPnI;l~;coIrGwTm4e%)VjCF@+Rejf?u!%xyo(?#^&{ z3_*-&%j37YTknm5<+9jM-_PBP73xMCwdVq|XBw34K{nh4TT8dFj)3CxoYu-cvPVY% zCBdh++pHc7z;x>JIn2zBG#+k+Kq#yTr{CJ8CvF(ss-|VxSGn6}orsw`9H`zdn)T;y zH%%A?3U>TMim8>LAa0cdjG8gvK0ONruZznmbvdmI3C3KuJ%|vwF>{i`Nn-OQqwteX zKA{Ezo1LG_kdqkLG)3~6+W81PG7+mg*xvq4Bn!6HM?x{2s!44;x<3z}U3z6}I% zjR8JCsXu_bd;keh>$z?-EJ(C=U=)elIG$ty-TebtuB#3OQ@@E>~QH1?g?`E z3c(6z$BvfA2-?(Wtj$iV=EPyvB)H}3Nie+FPDv6=>ce?ylCnsMDMB&s4ph`h1lmAu zzHgclXYY{p`ynzdI$)_qu1|MgEC8+yKSb;CLqLRBp#5E$l*Ku<_~5*Y07H*cE<0p) zg{(#^Bg85h8(Rq(<2(7pGl(Bd)o``mR-?23>4>L=)ixi06;1g3=`Vbo+HA56ZhVs| z4v?{}J~^$(*h&KwdfR})hT!C*2hSJoMgf*kYMW2OOeL5;D)nG2P{6)1@+Z>RfRB#V+o4y$b^xMibLGu8idNrR! zo~^}4q+5wglZevl_qG7J#S3ACW1;0L&((*`iU*2X35WXt!k6bfGDSTw0!kp(e4|cN z4i`Xg)d%Dg;S)yDE7OD@p4>I7EP%4PfEO}1%~shMYD?0LJfYm3XHtT4FfSf;LC;o$ z!6s)?8F=4u`{`IkGeFrZL|`8IU)VdAbaC7@bm-C|BmiHy_%W;B+r;gdzj{63!l`M- zXr3NLduW@+0r{#NomM5B?3+5i>B2M2ZIWPd19FRiJFza+!Um&lIfvWC*#!LqR{&0E zlQ>1y`=P7uQ_1LXJm6a;jE{yXik1x-hRH*W0DIM}FxrWd${9BK0`hda(^!C`aVh+E zS?jZBx5sXa-mu&dA)SJZk}FKLky6w^EO{A8V0;aL zSmm|!aH}>QLWfiV?pcv%F5XVTBrMEXRCCR4G$+4-e3^m>;MS6ghBgf%l7OOVu>dG6 ze@FHu>^AhM;C@t*Zl8dGH>3p942mB^1SXYyV+HB)PA3`$MOy2(OxUJMHj|IC&USzF zPdRtE>V^^UR=qLEj(YRt=gg|Gf&AG=?2DPd2nn)3;}SZ^o`5^37-M}^Uv9nUC& zzH<-I2cTRhyhuBTKNzKd5WqK_0Z_mOvBZ3X$%TOwR;5qkPCBP2I6NC&^Wpz2@kKzo zzzWhKGZ6&I{^ZJ6@9;-v1=k^a6vLlebNwuR=^f=Dr>{4S*$1!o0~jC&4S;CylPr~t z!M&d~J4adUD7sCOAsa3nZ;BoF1S%TD0Uh^uF=8G^$qS7e?|NgFc2IRYsnt9OL!d6+ zBCUh_N&HZob0p4`(tvyaMcGn8%P?`%VHMnm#^#!aX%74v?eWeWKLOo>}&BF?_r=bbtPN1pr0|z$x%jW zwQALU)nN_jvvwmgMnq6bNp>TV1%Q!He5Qe;z6bg%a#|igRoiX-TWX0($(^}qP;NWw zTq#K;8|BVhLSIUdG!lz)*IR5|54RDBe1PB~Jl7^j1d}s%!tLUg$8A7Q<@R!`I`T9g zD#7G1YZj}%Kor+uzmQ#^L3cBDz<-|Y`p{AQsz6k?xIu>Ub{kvCNOuff%s@-Ec9-aC ztG%M4do>BKbuqfb<435;BrLdj! zyud%^W@r8!v`FCKA4;LdWbk>r7UV>Qj?Y0y`iw!k>ss9@$GsrnjHDTEh)oAe4#PYK z>#6$MV?C1BLU3(8YcFCe4uW zW=zM@wU_ZHYC+?c0_R#h>dtJSyN~Kqn4iPH9=TE4D3SxXUpb(Gou$+=gxOtQ+v%P4 zW>{H&CkhALvqNeXR*LXe)QmhnB7Q9E!@HKK>aA41rYhHim5SBTgO-{D*)XCJ)J&T- z+>YXTOzklQvqz;MI|z3~HkxiCk9d@v7FCjlW7r%22=opIVpdaaK-a8GBEXzWZ`+R9 znWYvw78wGrE{0CQ0bj)|5nO?I=+2YRjqWnH25eszFY%jYpXu>Lzw4ZQaihtQE+ z+oOO~ZJ@y|-yT%{L81pbw629qU7HzQ`>0;^aTl$3JnT|6!)Co87bb<1`lJrh_u{kI zgFta+gIgaMMKX5>JPrriw+sXnn+tr)&uP0Xq_#mJuERBfV>_yOcFg*ANRZnHJ7L8} zgA`ZProh04Fh%bSz|)y;goXM~Nz=I7Ba;X`&BVh(H}yB$_^oq2W;1xTF(EtD;;WG>S$qYE6Nt*@*@_3>bF&XMORGGX@@)lv zft!ia_RNY)0X1P$hazHBM=?x_gpgLDVOMtsv{tdkO6G2+$%$zbOfLadBCOq^K;0xJ zUwf7rRdX=IA&(nS;l0i+ijGVk!>_!2%%xXuAh3eoO;VxMO z#K6&qbl1jhMK2>L7;V{Sv}L!KLLw)&opq#fD-}UkllJupb5-*0@X%T;{;Bl!hq<8* zcL?c@0r<7oeCs7~%ev|V@>fvGWo9Z51p>=zaN|VtK8*+NRz$qV_tP@x|I z3<6!qV(N39G;B(evuwHXGbH+#57E51uv&L6!E`zn2i!oi;5fNjNF(0Wh0Yu-J6W0S zPSVOP=wOlQw8<48<#I1S%t*`fJj=QGJ#lFwdB3o-hSe%`{ymAceSWhM7J`yS5`2(%|cf`W&f(&^) zbxDLb6n(}F0tM2zEqx+jt+hN{_tOp#OlCaF%^;pT?nV}BnH1Lm%#W)AcW*5?OAEMf z?QdFIyK2S3KnAD8D%9QM!whIf@eD7&WlMmAR4E?TvK!30lrN{hF2k76Te%UDOZn(9 zbu`4DEF;FwIY?yNbgY=udbDzTu(i^Rb^}C=Rb=<$Sz83sNQI+evcYglTdF3T7z-qn zN30mkr^epYoI@lsSbw=+|0-OAp8M#_tFyj30r4cT^A=pTjA*w}!^<}MbXW!uJ~ z*(kR4j-<$1iD@9d_zU5|BRxqzlzqr=0FyFmI*(Ij7%7zQzBme}HiCXOE_kq#=+8$N$~^IsPDe}zy~xP+yOGPKWd zJ5)#yqLWJujpTiX@GrwoU82`ujTx4825|+dH)gs70BQJ6Z}H|j%Xcj8#^ln#!e@d$ z#3lFOPcTNa<7^PxvmmuQlB96y@$AxQa_FL=UI`9DGG5bR_uv7%{ib4!Q+o!~o6&ry zMLo+18v-41ZOJG#3d%r!a5&)}T`l}x-PrLO$#Ewod}re3nNhY&=^+wS{>K0`l`;fS z-YxSC`z&=qxj-lbX{qS2Ygor2fhBy&5x~RMBT2mx++InB3Og~a3{?AcPA!ZAB|@X( z3$U@VVcAHqcO8lc) z4Gzyd1Cf=AfhQlu=Cq+8XHYW$fDLCX$6ZWn&x;Hlg~yY{FnW%rPVY3qMVIpGhb79| zzjDz_RPAQsjTc%s7bPc{M(Y9h6d<$23&-I5#IZMtHJXvvyPIwhi(Y2c=!{|5=NlLW zx%aFpzByqU`R8uw?{9DQ2ADS&8GY!R9@YTV|K#2Kb8J2IFDj6N17j;sjP^nt?2@ty zccOM1>=JwH`T%P#m*C{|T=j@$u)A&zDvf8fxqSLSA*Q`is=`aB##W?5kvue6|z3qZ1Uc$*XMO>Jy%+6o`+c6h9052;dO6JB96 z83U|B&8peQ4rck0L<5*S-9rHqQJ{)jV%-R6EiIp&DgN*>R_VT)PBQ~lbSV0!iHVp& z4rJW6Zs%GxNU0Q|BQUZ)D}qCqp=e(gL)*8ngwZdBkVDo73pP@Z&(C3KU|P%wio3SWI_5S zmTsMCNRlC-Z}I5geSI@<{@RH|0}jniJNI^XuA$uszYF*E-3h;R6mBY|`jh#Kg$Gak z8SB`e*Vs;~S;*t-|5se!&y4en+6S~sr_QbbISKD#(-yW83+^5UQ(lRW9d5JSr%o>Q z*9{Zir@Cd}glHAY*l-oyN(t1(oa1Me2t*tuEfpdT5|*r9huzR)&I~$R^h~)(tZ~H6!xPSF)PEZFxT7} z3_0I59ACocZcxTk+hNG4%@MRpE7r=DCFhF=y+$mJ_d!a=K}iZTEP%5sDZnV?6j(v& z@g1C*eB2$JtlRIjA0|j{eooD36eU~qix-&>^mjeAeNxW&}&gfA$Y>O8X7eJ`;#Pcs{DJ+ zF8*n~b8vTE`le&h6-WEYqq3F07QG3goALS}bXgEE5^E3RiEk3^sJdx=z%*wznjI2q z*<3XfD$8!>vEdU`VBJ#1JW#MP`!!peLCWI*8*D$Zf|S;fJ!eyX82{pj>1xhi znDCqgc`&BpLx@7+D<2kTzD11M>xtQzg(*_QB-gtLucQtb!ku?YxxBHjXW<8oRjvqt zcrr)vA{GDZ<^K0F!gtRd4Lsdd3l-d#WLuh1Acv7J05W02FS_xueG*on-xtiJ8z2AN zBhnKwtqqthqhXgp4Kx!hAT3UOAAdTXZrlZ=lCAe!<~E+(!^tHKW~G-Hu0Pojv9v{K zTQL)<_;(_e=Ho9Fxi2p&H3!%9PsPt?Sc=!oL!AVhU1~kV!kl zFPLS-K*VrsJi4COH*vO@uO+@XmnG==AqYp<-gg8O*d~n`5GckJw))V2CcLD%p78VjtS38kUWE0Vm&ce9JeALJea&v zV_IRReId>^heT+TNgBRnKtsXP2+9$B^LDu;0++)Q#D;oh#Q}Cdo~CLxX8s9+QwP_* zP=VfV|9+$v|BcKbw4s+6Ad{+VeDf9EdZ*xnzN8@hhVViE(u(GuP2%*`R^%P!?s?h( zH!IMxMnN(U|Lz!6KxvFZtLCtneF&bqovPlsWiQM(9I@P!8=}3RUMS&Cj1W-f7hfRj z%Ik{m|^g zquc|4>P}%`bFHs%lh4IpNwueuHQKogLpJ~wTaSu?cdk0=n>J%rE2yL}Gf~Y3mG3nQ zbG`k?$YT&gmpB9TvdH}MwdL2Y@NcU(X?E{dtfD8v`i>)V-BO*0dlLDkCrLT?zzXx| zoL!?qd-V2{u#CQSj#ybcoq-GP2D&7BTC_;N&a0h5 z0}i?Zmk_%*$6b8djmk4Dr=|!iJ5H;RIJ8h3V`S)QDsVYVcS3B=$ZOS(l3rY_eo@~I zXSZsZi2ffAV_Fc{mD!VfCre=%pS@H@fq+tI@ z1KY}W!pDQy*~OX>WO4^#Ne3Sg$P@6b6dD+=#_>YZk=d)FY>>qEP7?qZZ{ z7t6~lL=>Q5aCvpt_83|hyVx2^)CVBW59c=RS2GK#7_+vY)5pA%Ed!|zb(j5N6wvmI z1x=|czsG_7aYPHLy#Xf**y|Inz`VmjMk`D8_Is;>!l3l<%{#DO^zBeVlWv=u_sf}} zp)1Gn28U&yomULm`M4L`P_Se-+7ELFq*BvdiffTC7C>#ON~U3c>3Ci6!%& z!aRl`39iJ6m+Eb)S|~1Wgoz)7(%0alJI>%!$o#|Dwwi)EHMa{^)A{!$Y0m|^HEhjs z71_lYjNs(!o3Ws1zNP&Q%Dc6J4uU;_6+4q}iIJtXaO|I6 zC?D0##Xq7yeFSp!)>w7iCIGe{w2L6K&}>#}G`yzgxC#WLW+}#Yq{76phW=wlQ96rV zEf~hyvT#BDf^&OqHZ`E*NfaUy-H(3EAI(H|F3K5R={X9!=^Hv!!s>Vr6=Od_Qo;r+ z-j<6Rp94K-Kg-m|f(D|TVgoghi{||iDE1(QfUgu%d#Fo%%a+yOQ7qyl`k_Ioa0!P)fT9mCJV8M%;pRV=CREm^lA|AxYy~n=w1t_F<;SjdJ-me zzHBrn1H5x)66YpKL7cp6#h@H@&RK+MQ|?}g-W;NydOHtvTrdj`K8#G>5BJ#d^_aKl z2OUqxzWWihaIVb(iTwmW0++PS{D--7cegw6fh0yKx)ZEVRBM8e;|)!MzJ&dUxvt&G ze#X1L_%n=>aKcs+X&>f)*xVDIF|?e{ijmwB^@4we=)CGHj*FI+5jXVeAzK?l&p}~d z%;(*9$LgL1MSW9ds7oF4MgB(FO220~W^D_b61XBLy(b_MujurJY#8AIqDI~!pU_86 zAiR6#U8}CDj_X9mlo&d`@Q$3?@Bp;09HcYYdxL9FCMR$q^_)Tt$ic-LF0DQaT^5NM zgObRz!1-9GZg-CXM<-V*JRS)eHU`*NDb;~doNjTKE zX`SjBu%^hsTk{p-;PN0W_2~4b?j^VsHA9~Td*JoOSCD#fCTdrzN8TWs^mKxIVu|+F zEkZuwiI-%<29sy?cK#4T+@-_<&9h7Oq4Ebn0}0K#2TIizi)bUM-fia|DJo$tV}GrL zt|}rLbN+$Za7aE%K2q}m?nZ?-+5;HdsIE33Za1`$>uTE;_oe`(Yxv_V3S=tgDUT6b zHScXft2|z)O@vU`;gr~sLAr+^b_*@cjE%NHgAI15fL8`2pK=O!%Y4E^#}6w{(x+Bi z&29pRx&VHY*-Yt9A{QBvt=LBj?Qp59C}e{8VdcUN?2`7`hW**_l|6Ur$H_tzaIbZ= zPmpAX8PT6sTYvE+sQ4d><`Mk42!3VFS^UJQI-#D2Bo&Vc`Pl2=h+Y@ap0~J%@p+M< zlDzmT6v)mj%FM3e^OHQ*fr^whaCnq`-AOtgs)}C z-5q75_9V{ykuAW}lnAq6hpTA&HeB=T?JQN z(~t1KfHXmM!lO6x&a(Fj`WqjS57|c`0@4}#p)^t)q*i9hsl|n0tX;hj z9O0xq3TO{Zg8&2m8u+;5Az?_ORh&(7iT7ncvgps#ZY62$f_GbmDWy3W^{Z#;) zL_$b6%NI6WBT?Ng&i9bPv`2I5H(HHShT7c-2v!~i zDfD8^46>{7K7_k6oNoT3z)j}~a_LPOHa8=GGcA7ucKHx)t>={+1R5sJ0x2&`S4w(U zhrmJ;AkDJ;-ZnejhjP_zP6#>0JrG)H31ly-0q6?r8X@&akDP3}@&TvKgB29hZ z#u96J8H12`8p6sIt2@s@Wsn~sET;0H2E>`92m#IZeGtCc2zbebq19q$C%_U&=C?NV zS%6!xmOR7~Wl-SF$nyxyZ5Stwq&)DhdpEmGEviCfD~~|B0N81^L(*(wXo=)>Vw8^u zp>78S*#pF9?lK}r4+VpKVUKVFp@A3LlFHrDjKsHEDBg6_-U^6h$fBdw`;mWY<$;_Y z%OFj?zi@iYn3YaFr!O_9qp$OTFls^zkOb#a&GC^Xh~-!yD^#NSnS0}Vkh~~JHkhEl z_Lif%lzL=gRhlZHxuG*me^HRO=3EZ73~Z^rNO7-rX$lLC0VFDk;51>AM^D^nZ(uoC-LVJuQA=T0D#uTH6PWBThAzi1 zh0^Zh3+sV$oF&*G-q>4&Y5-Uh5Jge+-ts=fB2Y=X=~B$GKgpf6+iHsm$I)y6y$kvf zul0Py#&JkYDZ~Ia#oFu?Mv#N+hD5i3hb>VbJ=j(XqMNPU$hxY)bQd>~Rnt8(?LmuhmX2fs1>x7$b=8{GWY+y)$#a_QEFM*)nDxgbt9?n9-@!S+8Feq~x)w!r z1S#Wn_*|b7S&7mbrz2_FBTG53(nRR zA5AT*G9&EMSRSTfaPSJ%SKw`N-#`i{8l;T&my5x!q&$s$a>#m$%3di9{WO8CCZ|R> zsk|nU;c$xgK2?2R(t4`@+*MgBO@$mRloCZ^*fD8w<#rwj7!x)nM%Sl8Y)&w8|4B0h zjDe&ymn^ITOExSb79PwVwBx({JVqnQqPaclPjP5knFiscdn|?`^&O((;IaT7RxJp{JE<$J=^==JW#lr_zJEPJ`QId`!81vYL0+ zIz)fq!2hZI18^j)<&1?3N}hMgs@yPtV!2p?YD4K|hxI!RZRGioK)%Q0tAzQzcTh4t z?PHw9C&ZQu9=+x>Zq)r6 z*+YQGMXAMP?;o(nG+MF~@y?RQ1g156TH|3t;hWs4uFJs}j3Gx4O!?*O&D?rDwYIKl3WC#*A{tliBa zB%!}izvk_ZEM3e;hYj$2)aDRI z9H|*b<(fUyF0@JsdYM11201imZDdSq{~6C4&|~yY#Iup!M9*4MO)(n?_@+ONnHrMX z{26@mv=TZwLJi!0Y zgxBfw8X>juyA+~McjZxv$GyyLmAjjb8sLI3Y#x$y9mj=PB}YpP%;E((e-2mu&_$?na%c zGynAm`?HW7mDMRrfG|g;xc%gNU48Xp?&CxKnDFmO@oYNWx0l(Tl^BAe>U_)Pwq;qi zTR;4opAPu{{Gm_|&?H!q6MPUY`L{2?GloNH^g*>D;};^P0~k}vLM{7FoRJ_+`mNu5Hm z-*3kZ{Wx127c6Fl3JDN#N!o#0?HQXdiMkD*)pL4?U4rEIs zo~IAf{eC{dwbiUVK^FQG*&PQE@tkhfD|0OQ^k)Kmk2P5TKv*)B?y6V^pi=$gGWyxL zn+<{~yFYIJo9EVF#J>zuWKim~4U`>5zEd4QTr!Xi>g#O)zA@Yy!)>yt)*AX_N%kn| zBSPuhAm#_uw=bb}2WndyW^(JtvB60VyyU4a?jMog4>R>o#|gA{(vNT^q}P@@{@Ctp976`X_-( zsjRrB*M@Z!rr&sd^4ou7acvO1`@u)sU-}Fn3TzaaL4Pi2PDp-Ys+ODO&x!oq<9mza zHseRg`-(jI?~q*2AqGyzV#~bcpXT8wZsLFZa0s}CV$b z|1<#q$S2z2y_x1b1PD_GSj1c?V@dw6FOf_Gw(TLr-{3$3D%`v3Ox&iewZ0%!W@{o_^rc$)vV3;*@5#tGn;nk6lkod5Ro zZb<_Az1YRRzkpKy=JtgO;8@8<&hSP5+t0fP1UhPH-YouaSNqRVd1^9(lc#^#mO}1- zJvl$7>;G@lpVRoe5C7RV-Q0WV_e*;eodEeUu?ZAbS-;=@ESa^yO;Yb>i;Hdxz3%Z; zyYDX~qkgj_S;~OY3Cr%eV&`A%cm_%Fv~Yw{G0|@~X6;u(C+>|w3!^5QxVW+(>*g1i z`t_gmBcY>i=G2R}A8?0Xo6%ov7tefPl>h%t|I!}(znb3h<3h*F`VN=oj+0OpBcA6; zPrq}r!M!N&sekWH&mSsG{Wq?cB00Vh;+I@@e}jLYS?#`O`wO@iZp2|g z^=6M6v?X%I4mYC?D?9A4gYh99tuc1n@vD`1IH$!=;GTbR0-yH8zwxmXK}g%QI!*jy zQu1&7`p2Eb;rQpL=NbbF|E+uf`1PNUoE%0b9z2GiIPuN*-yFn0U&!;|G;LEy5QT}v ze=;6M#uv0TVf-iFko+5S`OlB>{b#!u#DkM)_}jhylkq4hoREMsTA#Q`^Y6di|9Ya0 zn=fd^#H`{^{wL!}h(FJ5WxaO*HtMzjjhF>% zUAC7N_OPp~E?e^WTo3Qyeqk@y%INR^8l}MpY850PPm#z1Z{PB`S3AacodVz+ISsX=tj0? zVdd4DdpE38?rVkVQC!NGWOz$kBdNyn>hJ8sWbX5WtC6>`?nY*=C#^GGLYc;1?y^M? ztn)F@N;p4y6rq);?-?iLm~*9dpufNTlDc-jO)YLo_0gVriD}mjnwDBmJRQ&}P6xY! zN)T?1Dk!>Kqiozuy%|VbBhr7y`R_c_3EZ~x!6t5BciO}!pJ$N<(^g9dQ6Dje9Qa@3 zu6(XLy7YF2Ss;qOZ%i6f3pCD$QAUmkr%ZszP&3oq>}fi@$Df!m^91Np#+=bZGDKw-#Qt@ zvo1Kz;(guuL=g9KgMs$pE||>2U4{=GLIu5kmwhKyN2pM5+HzAipJi=aZ?b0D_;B`b z*|!ZkPE5<5aK|*y>Cfz*)$aRy_$(|eXB7q~^X{At%)`HtSMYd9mbN8<{Vmw)IuR=R ze`l9INT~JSB~I1F-x!lQZNEB-iu#281YQ9jCu)pruvg~gombP42QX-(>Y#~V!ecP4{YX&9e;5v_cZY2>VnnF=lT{0niDn@U|nXXKo(|C3$(Uw;&5 zCNniKVw(zK_=lcxzg zkr8f}lu!Jf=NBaNEMUF2F~pgv8BW@*&e3}HbG9tMx^5Q=wlLfi;kewzWAUJsG zN!QX26ZNv;#RuwLDb_vDmpp+x(1`>2T`wXYg#5jY!H4hSz+xxjs#;ntQeGkE`^yRtp0(qH(qP*hc?YFZ^`%XF9VVHgQ8RfL6)7qYqK0>T^vf_@p zZ3gU5zU3GJCUa!G&KtEhQIATN@K!E7Yseoa?&`e;v~epK6P>=8pn*yW4DZxA{{p9( z2=#JB(cQ0XH3u6jUT4`~Z##aC0O(oTTm7JLPWEZ8K-vF}txQnfvM)GYw^go8!<9#-rmwTZ{A4Juz*6i>p;39jk<0kr-+N$rJ#|t>%pq4 zPdfMZT+-2>WTV)-R?5o-q4NeQ^*QX)FW%#uKgNTk&)L>TB3K!9lu_ep9iJS+(&4Ks z9?m%|vQ-re-Jcx8;WgAoy8gLTGt4@Bo^jacQ`A|0fl+&JM&;$=?{tamFBMvWJf{4c zo)l@45>Sw+0D27R)1&5Wx|M?q3Gs5PVv=!U9@S!Lj!sl4(StQfNdIH^=?cK|D%5Dp z&H2pDu`G7b%YP}R!KFGrVPKqUleQaVnIaWnD^2bEA!r)_-yQYbF%h6aNc~7t z$j!6zALr*huqa2}uOk(Xt5S)dLP__wdK6Kh&E$f8;MdN{lI{mAWqs;;BFs*s`>JA~ zBwOMN0IMT_lQ_cF!a8MDIzd1)1}Fu~perx?stp?G+sp&J5I`TvJA!=fUU>}XlRv)l zppKZt33CiTI_^J=B?4UtpGi6djOT0p16o?0r-L$fLYfPdPV<> z+52=iS-Sb{(MRu!irW=LwiM@97usWZazkj3_vwkPo;>nK_qb<9?OYMfu}+OIa^oV+ z(gQ+SM~C|5y>#7CS3x!Js07p-r=H&mKBp=)%7IgTcXz{<<_x+JVXN(*loc%&$8)-; z8f;tZF^uHO@3yVyRUAJj`cYoM$I$pbjuufX(=oKp<}o0B?zl^m68U%XE40+yEL#q`uG=LnR9dK>wO zNzH;BfEG|Gvr1ZN($QA>%6v+YLhB?@XqWgoK zF=&a}Dsy1nU*+nk1tWdm!n0gL&}b?yC(TEUtGwSO9@^QxOj18xp2IV}EQc5M#jMdT zfH7&Ut*>HIRR|>b%_;6#7t+lS_m2|~fs*|z+L{aWeE&KL-{{xYvt6)#BR)*sE6foK z8;8v!L6IzumyY8)pT`yLmxjUB=B(V9&&;Zyt4?WNu6l9>JG$ka5?&#YP z1N_}@$@eLL=aGtMoezGE1?U^U?qLA_KG>?Uu3Y!IQuLtJd2bC;?JKsyh96pmt|{3RQQCX2}5s8(^{! zWg#7oyXpa8uNys1*o7X(H;X_hpsa^ya4%WFZD-9?jO7(sXEr|ghgjs(w`?QpjWl{r zF9p7KYFDw`a&C;uBu)gPQdXb_xVfyf|2gMK*D}A8FJp)E7xuH8taM`N7IvaF?GGoy z&0Mhz*9XxxTZ8@P7zM%&mG^tKvuOmqW7{RK;EF*w^|7ZH>A&WFil1Bj80q7uz{cXJ z<9hD7PC3>_b~gRxYUcBqOCs7O)h*Hz!_V&Fx{^(CfHUN%D#bXGhl0m6cprevmH-kh zsTArmULzYW>FK;VTZV&2purhQe^(!%PB30>Mw(Au`*L4y%UpKp8L&xs3sZpNFOw>j zXTNxUXoZGT;y7zWmRJ`Ok};HCm-N|b1Kr`GRWJo9^kfC z)ij>GdOy9i*#S7WromFxg&Y^@Z?c_7d^YPL_gm*Z`;vh2zCT9~FF$NiRm|=s0sZ|> z2#a@xN9Cn(Lfmro{)&9siY_Qax#>K<8cF3XZimTH9u^OP6o5W~>SMr}98Kn4euH1E z*3VR)AH%ngUiw9>bWOQ9H)2{aS%_x*EJg1sCm zWXdcs7)&AX{`4!oFCN6F^Bf(YlzxC!u_TDPmuF>UJZJgzS-Fnek0G_|A$CE?)45sw z+RSY|ZhwZDz!(HKTyUcL)srC`$YCml$s+#M=zO1%&UdHXz*an(dU=K(r`m75h)r=m zMLNg<7~n$eJey+6Mp}FTqCR9O`nEx=(<2%_i)hH)SB~qwp9YrH@S-s~1*(|q)W8o} zhcD~h649JD6P8nLY*C@fcfl-PT4^uURZ*D&GJ#7V>P@<>twqx-odr(bFKyAc>rL)) z(>2w*+NG8Ep87OeSzs!qtdgRf-ZgxUJt7O6gsZl&* z`S$ro4jsJ3Q=y{UAJ6sH@C+y#s;oMEdgbmaYNYmfugI7tUQtxw)W0}hO3(1g$(2{1 z&bG-AuDO<5T{2Q&r#hM&xPGgd&6foDNjkS=55KWRSgmxcn;$u~O_o{hio&eET+b?v zh%#SVqe~rg3`=)gW%sa8b6v{jb*~&#@2@zt$_$*e%DPlFbz2M26D@gdm~oB(#AYT) zMx18&b|d9f)tq%+9f88?I}E#4JLez!eYx-%lT*DFBVdCKq3k+DMhOF6g>aL$ah3s@ zuE88B*{Uq(4FqEqUM%O>nGcTaKo^gKyO!@pBwX@~3*pktbv=fOs=IsdzJ1#{69m(R zOn@~?#rAh5;3AImnb3uldWccN;5-K8Ge^J{pA{Kmu_*R(+Z@Su>XE-L?6&jVaWOMp zkkD=By0Fz==Q_~$=>Z%ZR|Q-T_od*H8}8aeppbpYB`l4r_KgvoxSJ!utGP%bL>oqDY_|u#^5}9)%HWA*i$ra zodaE9Sx0FMP@xs2u(y$35TdLR-geh z-tjPzSKf|&8$g^rXYyOtpP3QjwDlz?_A}#(hmO$aOEWY zX5Dlp06$X>k%vWF0b*w%h$RtnRvr#oAoGShfP~$6E~B&F0ERr)csiJ|kki9Kd!TV9 zN^gZf7bwP=K9^v(Y@K|zFaz2(&H&lGqSsaZyb;B6G|mADMb4Pq*b9^3Az-ohB+@jc z2}r1y2EJKtOd}Lvm$sA21Gfhaf~1y(<6KHaBD9HvmvDWi zthzd_n+;j=$&cPaPX?i`lxn^1-%}dmNCZ zdd&v#{jkqI+`9`pNyc4rw&&jcfu<@X+z@&BQmn79?$wx8)xHe*gFRB#F|S|yJswc zrX5_xOT(`me3Qljy29RPHGdM$c&OSAng(){mJi!F>y!iPie(SSrK5%Pq-`<65Au{q zCM61s`IInGTC)b+82moc!1O|ZC~@&Du_<{zX@)xkuvtRn&#zBd3ujx#F_-S(4k^ z9*t)^SRAn4MyhG?7e3!}x)vMFQkgasc|sJ$ns++>8A(1 zSt8h@Ef?&U8nkv$j*->$;*1rhZpP|sP(ji}VHwX`m^R)~+t%pa_l^P&yY}taZ9&KJ%o&)3Qx}{;t}h$$ZMzXn{vPzno5#q1!!(pUwGn2o5c6qR8Ba) zXKwXg^BlKmpLMkJ=HAheNz$td?i^q0w>eeYk2!|J`JD(m3U`y=2hjlUEWBp#^$o>t zp!HZ&$H8b+81yBfuWn zPY{c2$BPt;-XnXn9-(%p$)G=5Gvun?K5y0M;kUv$=HuQ3r%ccL4u^i7WHVwg?cm|Q z8|?s+#Z`?qHla+S>PUcOKPM*BPByKH=puq+RU z?6a1=yljVe)socX7aP(HJTQIgH*Z%jihpq>zTYV=<7^VvcmNnkLZs6sDyu+fGyO_m z5{Mg)1bZ$u^YJA(43K>;f5BQw)) zOzMNU7cUbFLIlt^7#Bf=58qnIon4=+O)v zK|}zhK=&aqG=YJ#2h1dQ0iQfF?3Agy93(^Ze(F`u{LN;dN6_=Ze=m9iY{tEL5}_qd z!1j%K`H}o{GKmx^UrSsOK7z~7%xG_i+hIv#HAV)xmysMKwvoY+!%@BS9(y9yz|ts| zN_-N=P3&db&QmI$n8kBKQa81g?w~K(8WzS68+wFGgJN(7QRy{F~^DXURTy(FL7Z z@El9lObfPrMau#B8?m@zRz>TLBO3m`xZ&K=46)lPLIc^jdl^~qKkyWJI_ctE_`9lf z%*#F#W9rB+()XN*l2myL-(UNrDbs5V@|1=iVopF(5CsS$rkcit|C&-4C&z8U=04v` zExaPCJ)Z-gq)@NQYloqQ1|!sY#FF7XG>5beNe8+*V0WZ$u7wMhtmm;?we&Sxs#j=2 zdd)`(^*YRr)U#hDWCfnxHen{eGB^?V`bT3dkq!!(0uKm9IjA#Gd&)Mhf_v`uQ zU^v!T>z?&!yPq0s990{snjJtwtRoqI-yXT|y9a$MHyW^A8?`TLl>1t#&KpxhpT+p7AW zp@51$mqev-lK(qT%u;ME4vb*x{lO`!`%^Kn3eF+E>|k)@b?KfDKSZ@3x=_r!i^GAD zq%+FJCTfxftCWJ8WN6me8D{@z5azrl`h%tYHQ(|_iFjLLQejV5N0NZ4jTm|r&v68V zwGum3d&A)W@fqt={I!|9!EzL^FQZUx3>z2T4^nw zA%cplgKo%YeGqUm!R6b>?>1+n_n40d$QBn#YgVV1LDso*63$pkivRg&qeL@tO!OlM z7F^O=r{JCF1$*e-Pg?x6L?1rB&7Bc(V0UJxL~sM5w4)Wp88ktLF^P8uq}?@voSXz`+SM7zPrK-R! z8zEd;LlLn7$=$wsxdWoY|ICW~Qvi=v68+PYYIH0cMPg-r21dqI|J{Ba%Y+iz!7prH z7e__Wm1UDRPFZM=kV)%Lmyz8q2Pted!z57A=W(qkxFnP}Mf61{&em*!7XFz0HAqv> zC&+_=yEZ-6p!Qp*uh^w)a5!S@i}k>$*`{%m&!2fykrxk2YqDz!BcPc0O#1^%DBMfy z)txm6auqTwl-F0`KO)d*15~QegFa|m@6@EJtlj^eW22++U8YrUz&4HJ4k`D6an$Y( zbDS0qGR|xJ#b5XeGg0@ZH@~g%FxlMhef@Kp*zdySppk0#*3P|}kx?P8q~+Y;1#|;n zq#16ttrz(We}gs%E8XX&-Y4&2f{rKuN6H5%Jj zsKk#ckp($2Ic%fW)1CP>rP@_7tfTC(uJr;>{+$u-GuqTVY-vaZt((2{gGI z6$XOa?j^+~IJoA(tPYE)pmW!Tr4OK+kaWTq(I439i>7UiqLggVZ+7PnR)4n!_=e8D zWN)1$Eb@_zJFEHickv$>k*@6Mt{PnGj5RoFT^#VFnLM}Fr0@hEeMi#7&yK2_fPk4> zuTI!RI?}dhIc-?GGOP<~Ft8(QErueA^eDgPM8o6U(Os@^hmE7J6PY&b7sDsVR{gGj zL(6=j<179z9I%K^LG9)&nH0bplc90z;;0zuvnl zj!S$L2Y4VMyk^{aP!K#ftro#xftA!V$1)0)6hWj+356){Z$(?h3U2|sqaVo%8`!i0 z=<$u>m-S4?b*@^2rDbGTLDU_!g#g(|+bkE1|Oe>u+BWW@EFz^FgvWoB7N7g}~_AY`}&*uWFej=kSX_-q68M4uCdS&fc+gec1Rr2JW0Fv(K0ZQtXy?_F7 z@kF<0yY-nQS!|bB4Tv+X!=ZeA#xo44LJ^(s$h?$-`Y|uDxO{9D^}unT+MpW!BE)0t z-x@V?gL}2jWmMF?dpimSEv+UTRyRUxMSI)u5EQ7&-cgELL*^~R!6iuOUq=StzqX$Z?97GO$$Tq}AT|LRFChl8 ze`<`~oNQ0Q9!ba?YO+4B3l3yt@OI^H1atK0tm+V51P1=eTBT5#Dok?Fz=sdnucyWn*{b|Qb?)OngodmJHrLDzo@7`oZ_L_asj z@+5l>^`3&8_U%TN*uB$5pVnj!bWmQ zHirb7!i7$ztMn1EtO4v}^(I7iZ+B)-s=F#8d5X5xrv~%Vy|?lZHT)Huxl@`(SO#Cg z`@8quck+ucRYe zxdvK9p8n{*c8l8{E!L2ZX+82ezDS}jDRGaQ^QesFkR0!qJYF~h>E_Al)XIA#$Ppo* zUqtqJR=#U;sH#iSu`rQw$jv+rN2UixyLh?NeqMb|j<>w6XGqr|aysvD{R0zsg>h}) z)rKCYf$}lk-u-}#hrb(D0#$rguO*pzXO}MUxiOcRMYeURrxuHWu2w-6egy5c!P`_j3G-z zJ-5)woVgP=3~eHbC)Y(0(C!GtBF4vRas`t!;46L6V>E^Q8ia`@9X~sQ^04OA(H~fM z@DNIpX{w=Le5ZjKa&%ccSy(eZi^*Us9_fyeXa5CdZ5?1F!dMvmGW!wheHmMExvBt9 zqcgnKEgctL@Dk!=O`WouG z1;7LUiZtjKh-~_1+5ZC&H?NHWJ3N_kypzuXjW6?|%p2rSJvg}qpR)>pVj`*a*4uoG@~&_uD~A zHTo$Ki=5y6gUB@l``(nEOKOAyY0E2la8pWNWW}BaLFZ(t{>zZU?ItAfrRROVs=px& z+>$AXF>PO+ z<%x%%M>WnC&nC?#&K`o>|GPGA<9y$zCSV&ykf}n3rZb$XVf#tv@I7zSysAdy=hm*k z=ykXnMc|ou%J$>pGzbb40WdNT{ldJa3NAL!4}lC~m`y5QfQ z<9aK#}%U_&6UOMU3U@A3U3MtO-b)6r@ z_adpQq~?Quwt?QnJq-hu%}L_P4mm#8uKOK9oJ3j2vWyC?2J6D9*M4-jMa>#^r=Rt@ zQk!}ZyW{ED&e&;H1p|VE%K>>=IPW z4^6_{U;c?z-a@>;Tje!VG={(a-V+L`&!yKNws1%rK@`&CRZLmhYO*zn-LH4(cz!E7 z|64q*$d3x#{A{Nn!o^kW7>FBx20nbcw374Yo=Re8mQnX-^QF6Q))VMdvgCqm2dSjv zBQ-uL2aGwka9~XCJj3(=kePd1tF@&Y$meLRcUXU|f3j*WVgoVj`J)NM`Q;H9T9$nW z8POi;F3IY32CYCZU?#ABl+xSgX1QIWXutoBqo}r z%|6O%18G7=r60g|-s!APehfwO5$lpD*vLL6PRHg2E$b{qR^|?Cg!)B(F`=y&AARmA zMHh4FJjm{=QCJ4Rnj=`m@@;ryEx6t6q_{z*?nJVYsqbah9>@6?ZcDl~0T#nkc-2r> zWc3w(jmYXDU}Kyi#rJU;t&4)?YbLG84r64 zYV~@$PYZ!Tk1ujMMAWIH7ZZjY#h3v=-t~J)rr8V-dtY54>G>TX{0I{) z-_C{tZN^%6ILS6L&~hKIwL~x#Ad*Tgzl_5Cm!+XiM17blUvlP3%fPkM9to-;RS`)k z1reMGXIs+Njyni0=F(03y&nvr!VG4)cY^!p+PU`InRIKtv>%ZQPQ&~&oDQmH?sQ7G zI|Tn!!xtzyM$%flU@AqxV*F9hsC3U98n-52s8PTZbr7y@?R5v7b#qtGvF<2cC-F^h z>7nq9*QlXsH5^^FJ-g8qJh;-rgCtp&eVk6%T-H7FmG8l$VUe!guV-Qw*&HhOB+|e% zy|fm9b1d={9a(kX2Y4*{#4wkrWTkEIBp5w5Rj9GNIi^cApbzVz&A~tgkr!GQgWW%no_~A3Y zVVITqUik5Xti0R`8**CLk-XT7*TO>dh8`j(UVnCjL{`RBea~a%PPZ)Q0D})=x}=E# z2DVc?iAkcb=*Fu4UQC5sDPG{%N31SiC*ga>dg#Tm6DiLl;2xoc8`?h`a+)4Gg~iEC zHL9!DJ-FRo5@Oiwa*9WM(1;@;+pIGi%JM57b78EpTWXMhf z1#xq)0XTO{1wTQf6>D;_Sr6A?r|aDkU7JgaNkn(p-hTg>g}I<+5-Z_0@-eV3fm<(a zYip}1vdI*dRsw+iu<6P1@vo^#FZ4XCtE=VR`qYNO{gUXdB2)AoEzY~WN0|(B{0!cHnHawM!o;aH(z8-Ckp|U=V5+dIIae z4K=;hDVV?N&oLz0Qr<&rO>f~Mqd)LjG*-8>e7L(6x2LftEZ@cJwJ>T|9ct2O3VRA- z@_){M0vf)>a7S8bm8>z6R@k%IyEy`k!5Hj(mpZr>nTN z;a1Z`;>k-74Fio;6IiK}!C6l|C+`N%No+ zO1pPr0#Xmci`S}{UO(G}Sml4{8RN`!*Wr5?vYta-Tb$AbS<$F^ZxT%WJ9G3uzx=-fG_4gA8$DaaNJYK+pWckrdjAx@vhXb|)P5|ssjjrfB53DYbv@s6JmT)56NQU< zJzHqr4OLD(u3%1FV{FGh)cwrYex}Jv0u&`Oc_qE{f3ra3cYK}2-i9*Q*ntoidwI9~ z!a@hINlQFVcVDDx=lf~>`gXY|K7D`^Ct4u~h zc0`W7t+(HYt0M2Rc)mb$O(9D4;BW2-pe;=e7L8}o(=E@n-;te1&|5~33L##EIq(zS zEva8prJGW*=YwGW_N&Rs?#0WO*c0r)2b~o1)f>P1pIScL-1RxAiyP;28TFg6QBu3Q zKZjhDSytbn9qn(|lbA35O;P!~cHl3|N%HCYIO{gto0rx8_M&J(%8^XF|9Oi}+1Iw_ zYqe%#FP+%SmE?KT&kvW4kRlCHOWg0Tye)BQh5r6W|K-#F=Rb%0?yFAlyPR}Pw2af! zp4$wzt(LIlbjROm6)W&WGMJ*l5Yb43tS6HHX0`t3hyBiRZ#1Dc#0Y3;WDTustK7RX z=ak_(Os(x()k6f{hSkwA%7k^&4dVqZjg=A}JV{7Zf6<(9rA^(D$`+;S^7+3sF~&fA z+q`7xYgei{alaK5Ai7Gw(alHHbs5?_qm>|CCZ==_`LV!dl0QLs9`JkN|Bq{wF@~$@ zW7apRTouDdG*^ANQDLMc*Nwa+;i}UL0?)12##?E+piYeIK5S zB>rkGYU?M)kM|)Ms#Lqiw7+ea|M&!v|3S2o3hAZmheF}(l-qD#%zk)Gsp3j4G-S&^ zarA9eK>8>+2&iRLT<1hZQ&@Znu-I!Lufv)jp zfsa=RI0`ps?EatJ7r(<_|6G+SnR|K-j`qjL$Gf-Ag)Y3Un!9NiWp?EbjZ9{D6E6Ve zHIFm|{^_fNNlk8h0vU#eQ1`~0c>&CN7vMlni$1r_1UG0@g(EjnF#27;}~?l=#Qa^2^SX9gHej^2wd}2 z>)>5R28NWTC7_M&Io=qdl&;hKjlR=|nk?Apzi0 z(G^(1`mah3)#FI=R?Df9C_10wHY|A&D|+AA*&HUvBh0`kAfBxnFc7xIh-9w0%kfHgW$cpMOhW;d3?4O?L+PyrVd?|IF3ze@6+MK<&EjH!n>sbXJCI0}k1FKND)eLK+; zS$XtdJ+80RF;9R?xX3tlcRzRPM=?c%4U z_Lat#xpLb7XFlqkz>iI4%TkSSRrIIk)c5|J)WP8Ddrzg9c0h;z-<(Q+S+6%#pi9HS z6!<^;m_(p#h7AN||8wj84jujdHvPZMfEc3T51^(n0HIK%&E zrO6;74~vMa*J*E|SqiVe5|LJX4D<)bJ9E5qKL!KAwfF;)$Q#R3a3M-zbKFp@1!;=^ zV1+$)k!t24!9!WHT>$Gk_a}D&VKek=m=Vb1vH@b!8^KtK^(Phvo(BPqw>Ot?fS0Gt zc79XJ&T>{Os+rU!C1BQGSsO~-M%vPW7&wLf49OQEm0iHWw)g_WZ9-o|+nl-<1iqRI zvZnh=mJ}s$bgn)wJf%qvPaU08xdK!QnzwQTPfqLP)-RPXz5D;Sb==8?%}iOcR>RhiTlF2y1(-A|cDm&&yX8e?3k%96>;bO9MmR_rL!2FDKcCE-KaIxNmQ=nml&^{_+82CkI8) zlJhMS`mb)CDPn!09sWQ(g1LvE47 zst@4iwWo8oB6$Yvzkqz+0Wbu91bjgUpc;2brPKprD5x>H$db)(s|;YiQQ!2Jk?BY>B`R_dPCZsdK(`8xZsSuxGME;GxS_p88bZK3|0jx4}HlU_-z{==n6`Ih08@96gv}WpA0) z1(uZOBjW2|jR#{#-WCDn!frbzYt`Tv9p{=BNtAD@1`RacV5~+%c7!m$QL5Da!pq2# zbf89`Ao1L9Yay#>Q$nhr+CE(&4I9ni8`uC-o+D61x^`bR0W^0P1a~_$EPe~ax7fl} z<-rPRCm+VS-;3EYTP5qqJ9wmn&%_mGm&$cW{Ql* zPPvKuZtqPW28`O2#=#D)gx9O45bH^$V1;#|s%EZOcp5%gzDzR@TUfY+hW_g47xj1! z5=@%MQP2Of$p8A!S!!P-fdL1Zw8FNKqvF<^ixQn$n_VD2IeM7`(7}OFxZKyL_%9a@ zj1iBSBIY1jaj~%hVhH{C+CzY!Pr|(~ceJ{29`sc5XTI@W$-G87<=)K!_#p+2x(Of_ z`gd~gVnSXSLjrvO{(o~NYnX+q%n0%Ck8Z(Ij$EGtxvR7fzf3hC46~}i6zn#jFmLzB zhcQ;*3rcamLm8~3c3#G2XbTHH`o}stv(U?o}-DPhkzO`SAOtfnVbPiBC}^wNz6j00F)O!K?U*P zOt(xKVh%)tL4R$qGAbVfQeVp=>BGyvtBfyj6GVKxsk)JqOjnSO4&Jxl?~p|_-N3>4 z!CgSII3mt2!S}64PLok0f@DL%WO_^Fsm05$-W!N2j^xhZNH;~cmfLD9<$+s+YY-Ok znow?@K*=@4QihIM2kBs_h!lZH29O5R$|n3+q1HP=2E0lYJum%&&}X{qHC3XA8y#Ki zg4A!Ox(d$I+YS+lJ|C%-B|YPvv#Ml!v#^a6EQx4XXTB)CaidlK5j>VFXGzs90(7WX z-|rIN&*D$ycPdC%6U&-G;&1uay$)XGOuJI*=Kawnf|r1^>kCDc0^6L(oM`zdQ>-gL zO0w%ynH-x3gQk+Z9#lo~HC7RQyb0k;C}(N^AzaAit2qn_pLQyh%V#YDLzn*CWW`0P;JfIkM*cGPmIC zEsyIPBdtB0X;Pd0V|5s-U&8a|;;}_0ym*e&Sq5L8bkHQGq1h$GAAfyZz*-&UDZ3** zCH;kd5Dc-RC9!Zepg$FIR*;>-HWQ6Uqv+EI1Z@?QquiXm+XSo!Q6C7VQ^8-j^StSK z4u3J5ZYz={X1=tT6leQ18es-wsgKL7h8BBnX;JF6W{Smaxr`oTZT7Ro)u%+`DezoY zc*zI%J~ZE0RyO_%oCwP*9qZ?n-7~FN!zt!)I|-wr^4|p zL%OHy4%oGbwb`kaq*^x57Fd=cAOm-!&V_C94`(EHm=;^0L(C>Zbn>yK3HE{WTK{qu zT0)Tc&?pV_-{C_Uau^#%w?M14asx~SlIM{wcEMfL%HheuoswZhPl9ne*bt=dT~9Pz z@A|rtPaS{>Y(rA>#xKd-D&u**62(VBu4%48{2fE!K-vZ6GemIy6qGu$MZi-bS7*J0 z`Z|gihGZv0#}A*l_#a3M+NN!EQT8EoVtA1z7U1s{f2W1@206+>sACu2$jd+2dB!k? zG_1Yusd`=t1=AW5RGJ<%4)V-#VNm|8k+m->YkwdY9wdM_1q5Ndt(GAB+won(5Y-!Z zaNK$&ww|vMlgVb;PlTaq`!6xM&HI30>Ff*OO$OF5d1ZdkrRX%g&$>o>TjqfpV7IpG z9Wd~Rhp+Z?^Rw3g_gy+4BNQC+>qxv=^G`CKi3K$nk$^h_x8c}-HI=e8GCKrjD(!(9Oj!#o*tsN;QP?8E-c&y>!8mZDiNccn^xLa-q(0gv zbJR4vhO=U%2sd8eJs+LZ`^t3@DPzJiptV=HO!qoTj6QYJ3q>}5B9bUUT#bI6i-5&5 zN9Fr`iNscsW;(JWN~IDj@M8}BJ^`QR>gil{h7pnAS3b_;=MPNPLCb97qO?$)uI9#- z9BSeOR2F(guHN}=at$nMA`R;2{EtOrUXr;SX0QE$E{cu0vWBj9OVP=a^8pDEW2BWok=)TbHsvHTZ@s{)HW0d` zPAowYk|_}6h}H93bSxdMR$)@PRE9H>=uvW1vmSd8*_mLeqgdTJr-)+K9{zB zi9#I@hQe<+<8$^sB=EcF&wgCm?JODwkzCcJs%&ti zn~pe|Z}|ub*;+)_>hW$dA{8c!1o@;dE&EL4buUZG6qLL>ycOIgt3fMZ=hYWkJTktaQZ>|rk!bTE zNr|Y7(J)W^7{NjI;0HAohkYeR@~YAY(D*Ne<~040{1NDL7(d{824W=M?`VpVE-SrC zNEmFF{!VTf?hTX?Qg|*>b+kY2zL1YD^%>LKSD=bHsc%6s9h%%obU-CYtTJbpYu=x{D9zmd2kx& zv!3ugS8{F2nG&ERuD;tC$EEUWI`GxPR$jEpz48|2bzq3m3cE;T+eNBTynm<==J~$` z3e%I9W*E+|R2H3(f+yJ2&tsy%@?+&44^?5TH^IplAFjt)$-o%q5krhUWIRYKv)Bem z9mzJTdAD#dQHt~o*37YT&jptho>j;0anI8s1bN0?3U@ZsO70T>%s`c(MT1}?a~6V8 z5O}Yx)ls%ys-(Qr(bL;OjzHXxG|=$CFy$DLXOFVeZbiDwkj5Gt$22riI(=MhUjBsH zVJi`5l7OBNrEWH1ohkBHdLkx}uZ0gmtJz|jD;7*-N1&|}r(^x1Pt}DN?x+&|iXq6` z0;K`1y+zAj&-G_O*lc-!Nexg`r7&L9+&dxwpnBjYyJGNQ|lCd;S1s~ zn236Fy*PvnHfhB!6HU`*R?aQVzjga?3c+|D&A2kAd(^#q7Yy*dB-iP77N zRlamnB>`S@uxnf{+GKvRW8E25xc;`wetmfw@u+sZQbyP-{+OOfv`~!v;Ij4_<_PS@ zXTDMl{>M+1e3cm|CC%(UX$ha=2*m$OD9bFTZ^%i?3k2HV0!bN~s8l6ic7S?kAv;~1 z1)?wWjsE!wQiR;a>W7Yf@)IaIu`n=K;R+NB!z&Yt3nR;<)zmGKz1Luh))9q2sMcJF zUW;}|j6JHls6J?RQ(}KdB`QrcjU1x=soQfoL`@P^0`_6-c0-ebw!Zg4r~%gty?YNZ zVvP|t4A?!8LYV=E=MjQ5Tb>wBD-vKCUTSSvFNDvGqH*^+n0!InzP(*&%IlgqlD{kI zcLT3g+2R-Wr#e@6S&+j4!P zCce@zIS~(Sti`ZhEmm*W89dAMmYFKr#YE7hNJSbHc)@^(mV&+ILKaToq_@dG_Jbh! z7nQ4Ox-n;TFNhA2n}~Mp{^NX0a^O|V78^12lxT=%*Qz6dPEEgM^{(cr85}D6tyoM3 zQrIomE^a?8ux!Fft#{^tYD$fzh$*ru$MCqpF&8 zPjWP8ZvMtIJyb5Tbe+ZxC1UZG`bcCY@>SM)_!=S#YF|x|m_`daV6rXkk=w^x3QlDF zw^xU$LZ)kDw%gvcZuVa-yTYn7~^u*z<35KUB5sE}p1h{F~x3>2<&(l}|rJrSKl zODO2z$@WanaEr{TGd$jr+46)Ci*M{!ibntlbOe{vb7EceBrnLO(T?)TDrR2)^7#RZ z>~n(2zoLx8X~={mSoN!gzqyXx@y&vqf!Shf0b2N=U?6Z=@xQDaV+2wCDFBpk*FT5n zvQW_>61`N?F_97!4}#06e#HIx`ja3h?z03_Ik!8p)3jVWbKt(*{M`^VeW8yMH>4@> zg5OhpXTSRa@7te@5LYQtn&>jc_LaKe01PH>(eDUK?7|DFD{A`yRGLikMO0CsHg=XZ zRwMCGSsP3y7&MENx60w7PU4LI&W;x_b7M!JSv$=rk8v=d`WoXSRgnkio2_dJUP;jz z9p@jjr7L}(4nbvt%=S`gSJfD|AyzHcd*UJVeH>W&f2LP<6Ir@O#m+j&07(|a+VKghPIN!xXJb1* zZ8@lG89yPhc~9h++0yn4L=f$_OIn^L!8E!RUH}PxAbjwXyD*> zx*f0d-BKW3O?ThQ-|)n+oG|@%%YS_|mTmg{_{CP^xPp`-Gmeeso?*jG$=r`PUX;K?WJd|*i@a#ufB;G^y?zn<4{+hl8f3I4iH)r zZ&vesRpYuTEvK_qqg211#^7k5rgy5^BDcosrR{_t+BUN9IuL%lzS1~O&LQNHyi+wM zpSEXR3tvNjZ#JBOw%0puur_O3h+S)`?x*&oHF4|ZDG!#bir1SATXQW!9iCG`%b)+) zp*u6e-p!lZ6NlLsk{-^o(!A+j_y#Pntj*CVGx$yG=%FN^Lp&POru#S(glM?P_s&ly zYkbr4G`)}!7xQky%UcDf55r>U=e|2ceDD1lVr?(UGz}fe-hjP1`?cNFgG@>DU#! z_88Vq#kaT(+%MRxJH*TQX$)2Aetn_UOgxbyk9|)AHU;8~7r2$0$|6S+mi>^6gS1LT zvKs1vBs?l6b4Xbt31n5zzUzr_OdZVuG{`ddJ@w=l^VdOEgZ|xs4H-CzG3n^rE_6Fl zdy6q-I*>53(PW2@f@L;h^@JlhPIkkTws)=-&E4$nXFZwTvLm6^ zbizpaI`ClHU3jW2RcFBXi>r3K749aBX4!WS?(n>PF?E{mS|iBQZLMW6#Pjqz zp&Cym(IhLFQ#o1H6WFXxhEg`sQ_jLDuKH>}AAIc)BWyq$=PmT)(4&QD1Wrz_%DYc0 zr%~0!=d&!q=!o8vXI6>#tMdd@x0rzwEW$s3mQ{uCSgGN(Fs*nYo^+2pTR@36$h|=zO9(+qkk_|CzG3Cfbx0}J`|%fu z?XW*16yc8Y6uzzz;>;I* z2`tn(w>qjkLc}w7v;<6`R%xcpR->aO$+~+#PeeWraXxn?Y(;BG`Y1e~zSmGP8NwbY1CJ~ zY4pGJoxUg1;Pv+?ev0^8do5n6Ux@zezLbH zLaHY=$J0wm8r&%n^z6f*=}+6krCcpeK&Q-=l=}Sy+L$LxA>H_TN0a5MsEMEVXxP@? zc$O@X{(W86wB*2A-cq$7A>laScO2^`7QG!7KHcw00QUNW%Ht zX}WZ--0}2*fDKf6&tqI9G;vpK3mphuy;^PB?HTi!yJYnd>E0LSdl+!k^CXGyLw%A~ z$xv=9H~L8qc%|=!3MZWP3zHrhq{jI8g+-&`Vz!p{7bdH`0^HSNLiIOKmp5g^El!=? z4!ps=z8T%c&1ULtJILeZ;m=`)7W;BLrv@cHIE*PVPVwvz?Ib3ObSbBz`qL79pOgkR?qlbVW_D>N{-Yn;0qQ=3&A@bWXU5Sl zE2hPN);&$8AW!$g8uuMzi(@WN)X)d7w6LD&Jt18bilSeeh&(a22GU-x|fu>#>W{FjDm5>R)w zPFRtD8|ppGfI5otBc{$RAcjtn!XdcKDm`50Z4#AdB%PeTDK)+~xW?mgSw` zuwsmsG{W$HKN#E{t1x&!GE_?Z5EOg`s7x4kX0l z<41q^rr=MgWllfjbk=4Q-@*J-f#bV4&$?V_x?2ifP@S33=Cc5H?61x?%5}P@`7JB; z0xh2y>#TQ2|a`dL~n%O#M_H<*`2l(6&2o|Qc<_j&1`*Il; zD2a{=F`^s<&|@TC_ZoFtH=!=iJ=8#dG?^2Zvb!T-;ky%dB%1)9N3JClF<_n6)#1%f z>H}6KhCMBA>kMSr+JVbnsgHLQpC(L{$4r`Z6I?I;S!eu#(2BbA5F$%yS7R@=#Y0yXmUkR5+!pgH5{0kEc%vJo49B zgJ)May#R^5a9HO?ra`<2NU0RGPc_vf4g>5@D>F`*#9CxwG~KMrQd*<~c=3dN?H=0T zxJ*r_FJ`JF#(MnbR4^G&q2<&Li;yQ2mh}FzS{BIm4J-7pFXwgBC89^~qi=9c8Tgyi zcB%EYygSlM+#*kJT6HJuqs1EfD=i|bgLIv$uSgO6okqk=?H+(`Kul0b+Za$^g;_eB4>LG@h}H>vfjw-L9F(0nf^M z3YZ%=_5(nq&m-esnjZumtU6(Fej$(B;g_&GS0)6JrKe}*Nt zWVR~ttOdzZs?y=C1GCWPTo?)V;R}-iLfVLGdrqa|64t}AH0)M3X1!A>6sM)799R7Z zGl6n8N8>#k{AvPNmz%CklRZ=urh-=GH8S;kBOi05qG{alws-+agS!`{bzNV=daSWo zg5nmU3Ndcl+{Z3Cc^-M&*ox$TpVS)C^ zA9?4el(){Qo!qB=U z{%HjnHfJ99W2yLLt{Yu*IfP$?94z--iqQkoOE>uef;(d^H`xY3c=Y+p z_@g5qKOtQIsrT&DT6$Dn#-~+mgWw&Vs6?}$zGaXyNBY~%*&=7FkGb(#=GL_^$qev6 zU{8HFI1WadEdUn_PdYkG!h)j3c8?933j8Vzu@4J!%DXihkEjZ$HeQUCbE~Y6c#n&1 z_}5o(SpPnk$l^PXKKj2GU63o1-wp0s;&E7$Jq2mnmvDRbb(Y3oOQ$l5C3g1FE;4TW zJ3Yhy%mSc_@56Mh!4Fvnc-g`V7cQLZh2mEd$RaWi1pYv&j0I?Zw=DH+jIHdo17=UEC`@{TE);o+RCRWTQ&xSxCW+!)gR z$GbW8+~izEjJh9+5C^w!5F8xOfGM|zHakEZ7w3YX$&gU%(OWh z5d^DZ&a9UGwBhQL>82>RU_Uhcp~+)^ekS$iiAIYVD&HZ);nxi_^{9&>jRCR(Pdh z!GcS5TI08c=9`v61JYKDo4?tPa{dG``xAwNj2+K(i7{%TNBeK4T9C~5<;_b!Ozlza z!aZq*SEIME`US}^OUD0SPh!V=5SY|HqwNkvQ0hp?u-%_7739CX-qwz)1J?s;zArAe z7#dXLqY7!vtT(s6;FwKnYN>qtUCZ|759eyV1S33tEqfw_YV^1I9{SjWkV_UzFAX}F+P*EJr&BHo_aVt4+2=T@AaUR!!L~16*U*KKvv^R(q2ocQ^yw5Q& zgK|c}#&nOKQsSd}jsa?!WPe0zG1p-P&{7FDpAXs&N%weD{9sYYxvPf56_5T!A_E52 zX=f=K6M>k!B;9fCAYO>vp()t`TyxJ7%b|m1zNI~S7?b#0wKQ*_`N_M_&x-YD=<5B8 zt3CzqF9b4VS3l50p*!`8{EXhy@wtSA5c+%VC3Q<7c)q3wFG7DzZa5sg4>WTjjeKIP zvmK8fCJa&;HX9eOZM1<@#kuJI{B;|)cO;^Ppl4bsSZ-{z)aQD;^3r%d2*4*%@LI}* z7Z)Eq)3RE{LwvDv+Ct^_%CBq2`!mLeS^H&^l7Qd_yp8YdPUMGwHK z=MZW>?g{hgb7?tDEn-&UZ7p5BOceHVuOaj!&C;E}%+^PRcH_z*_JV+XBHxN!n{@hV zx-WuFZMMo_jm2(_eTmz?QvdM(A5vSA0j# z$(!rQy6+6y0!@JyE@ejjQOaBa97mP(6n-|)MIL^?`*wD!C2mL9#0s_-6*ZCs5({V# zJ&dmw`Yy#&Os^(`^LQmfyj9Ny1m_eHJ(J$BIJ`n_s~wj!OM}ZhD)hf5lzlTW_(kQ+f+dP$`3&bAhb|wb`Mod@g%dsE_H&oX(-pm#Jgf$w_S zV&tVmZ(*dw)^{SgYLiB|H0$Ug7Y$0p~mKhn3#(W{=aQqOS7v2=Ak8`gCjV z69O?QoT?}E=?6kO=51tqZZMBg;Bcw<-g|%GGf(%!9s97ulkU&42Mj{-_qVLKKmKC- z^GEc3!t2S}me@ZYOOV4)>Ubn@S_t%A4e?zGrq)3yzuQU3$MH8g*E~a)xanu&p{CU~ zThnv>5xxXCc#c4mnv+1nnI;+q8};bKk$j&HyT24_eNZ547>91!RWo@(0%D5kKd~-j zVBy$loU8iZzX$bWXPg_>-?mDM3lj!>yAQt}pulImDADe!<=*cPZFzP&87^OKw--!4 z?zy&Z$J~tG-AtjMqkwBZz$FaD@lVN{M4hTR;e0mr;-XWd1rhtJkppd~lTmrIzSAA` zkDKk*8_UX;?JQ3j<*Hk3pR-pd1x2TEwN3vQdv6{N_4@yhM=GU~2qjdCgb-20=(NZ# zvagYStL)oMv?ygM`#Q>&eHnWgEtYI!-v(pf8OAckF!Q^o<($rWJLmm5e|@jtb-l0T zQfB7$dfoT^+@H_KBl!|5ABOoR05Z^Je%{+t!BhWq7D5_@OlaieSuGFO`paN9^%YPe z<_UqA!**dNM(#tF{Vx+$CV8n9x!f*S8hwJO^&?MNUQu}f5A3a7?l_8)+)Y>G>bT$X zxiQ9TV!ga+iG!+z(0;;1YPWhkj6~?oBmiyFo2K}{0CZet=3_*|%%Tw-k1vcATX z8U>;5U3q(!v+5}E{TvSg%9SgzJGKTJP@^`NlO{u|sTU8AfVtKdtp^;*DQOlC=IYxh zDy|TSIRk+pJ=io2yquC}fNd$CLV@b5rC9D+Xhr-!)a<%b+)$JV(VRq^)6CY6)FNKKMS7;?;3AYX-M06X%Gdlo^UtZ)mY%%*wc z{h1%1H+dvqE{`3!EuL-Dk3oyf*Li9|IUaEO6YuFGHVuzH4j;kZ%_4Ao#m~n5lsdqh z#4#k5`3R&P_p_h$*q~gI5--=B5uk2y{O;U;LT{J+hbnc2SnvV0Qo8yse3k8hKeO zUw7qkPUpC2RHN1_Kkir~Tx+0}NLpjWRZ4ai&FUQW#XT zZF{U^=1Rltb?Ed%vr^XUjv(wZ`&dtq5~;tgYN%gJcfRCAwQWIiniDL!!9D6qPM*yh zhnM-_=Yg#8?oSmw)Z-?PboxJYjbk7TkB;MK0bG#z0^&5)lHL_slxS^wGI6CJthUF4O;$x_{HuB< z_vEW{$>_ui&W#qo^aV#@);R;Lhn5RHzm#D|02QsUq|=pZ9;EpABn1X4{J@=aQksKZ zZRKs_ZKI**RU(DnuE(U}Tz}hte|<&^8QnW!_x178sH@t3p;s93urv_>*HlCaKj~(& z_A3WDzQV@RehNf6TKQk~U1B?xePg=fZ2cEwe+9vc0;Tv42Beg!%chsAiI?Bx$Mq%8 zbI=G)sMf^%t4H0mRe{nP(Ie#}B8>RROVMj3ZP)=?Lt9{}xN~;2(s;PUb%L&ewQEuR!eCvdRfO7&)m~j}*hFFt_)7mE!!}M@X#@(lF8j#nEeY zX+)4ks?1eopv+ZafDU%sdOeQ=dtgs*R}kJ#G+86Y^k(v{&vdZT*V`}w7Zg{ued33{ zKKGcnk60qxlclHLdwX%YPI!+X_EV%GV$FXvDDq zml69`SklsLhou=|B0ERF^a94wYl67>V0!Y zAwLrgvn1V4ycxDDXv_=-+H3A1LT|R{&@dEg_2ru>xSY=;=c0C6_szQLp|NWv_I5}R zTt77D@a8AkNoV&0-kW-A^zsIP*;eyEYO5E0>Q-eIaXv*UCcjrT5q{wKrQBAiT3>V5 z#*GTEn$D>ZrBy01C3L8vAA4Rb%zl(Kh=h^ZAJmCKZbA>{jRscYzmVb8uD1PbeKV3- z7mc&F_D|OOdm3DMdV1k5!;tRLdvOsqLHH7*YDTwHg=3yqkfr!p8q4+qrAgTjO z1Bhruj$PObj6B*}%d7Z$X$eg1;;l^FrMUU>&lh&j-Qe`cJZaiGKGuDzIS1z$i$&o2 zV-SXfI2OaDn2VL4KbNMFEyzLS^&s06Jnh-QN9(ti2!WMH@%8Jk#5Ia9&s(P}76@Yb z0#<8jqXXB(*FNM4#6?>hIWBW)UxKFA@Wq-MUo)L-=Roe6nVm&^tb7nBzcVt%+S3M$ zeh@bz!ZvvXAC$H0j=1yOd=m;Hj@~#JG#gxeara)0lqpx?DK6;eYU5_gvmt6`(u|F-8}$3f#Hq`|FcaM;`n24!>9)&j*|mA^FgL7cu;aj*^bWeDsr>Zfa_xNB zSIN?7dpTlw9STuOX3ov(qOHS(2hVR~Uvy6bO7$a_dTBU-ckaoj$A)1Nc?1J2U$$O} z{mKG|L_`6&Kp?(E`DH(u?ue710AN^Zi$$@peao zG3(2=#faDGjN21I`*hKM;rC~F#hI3OR9Fw8;{1#By+7bXX+%4~CdTin+ZF`oGbpH_ zsUS6#N?Q}uyS(f)mseY4y+gqeBl#=+d7qk$3{=m$kLB0@{Q0@_rxpSF9!kJJN0lp3^C|D){6z zH5=%Mn=jGUnw5c)#0kz)W}uoZNvi|#Z=^g{Zc*qW+5vV+2xz^`E&%<8D*&i8Y#s-$ z3iAMc&HCY!8=()hPyWGoZoG%iDT7Vou7YEr`?5-9h)36|V#Pl43k@wV2kq?HUXj&C zLn*b*@w1-J*!VJMY{@ZHU$y7w*Ja~$O_%Sph3O#7m5Ne_pT6!m{GU(lv&O7l zW~<}!JR5fl6}NQ-{Q=oS&Y)g$C~epWhp)R%v?j_T%Rc`8PMJjNB{VPe8qgC$VCwLF z>xBz@JKj}+`a)zlzs{Sjtpw1-VFfs_>djGD!{@TUs!U5xpJasybIHM8GKL3&Dmr%D z;o0WebkN%0P`_s#dJfy<5HVNevrhl*hn!4BQ-4;J@4hJL#d2vDFjOVEfzhN&y4WoB z(|@d5Lin!zY}w-(soROp&)wces=%*W?ow0ZBwr2s{9#G{kF@UKqBLXOxXyN$KTiGj z*ZAW`e?F8D6%~Q~CbsJT`1(KOMYRFkt_@+v>QATqpC9qZ<((oQJ=#2&=xY40Z}+_! zxZOgp)q+2K;oqI&|MG)`B9K47R!O1%Jc(EP?dn!~^!*X{{#2d+v7Udd&$nN( za|31hd5S&qzueq^E}OLuxLpAwDLbD3c8$O8oXDOD4c>0LvRPn+E za0AZv*8lnjWLQ{$4XBoU)c)_yM;FlK4skKM+nn})xMA?CmTKUJxGo(WTKV^y+!;6% zdXp*4zplYQU+ssTae4qe0)z6Mm%RVAGhrDA?0Q#e+Ku10=KuPGfj%(KEZ7~&@SkV= z|N2fET^?X!|F@<8w737;(*G`cKL3-_|EiS#C#C;gn*w9>KgIjcC+Yu(wp2<_OG#i` zd)N_J)mLcKP9c=?-9^{d^7dLg8266=#0y28jL-AY@_=mQ*0pE*t@3+7GcO#p30D?a zw!Z5qcEE_b%=L)c^%Zo0(MImso_~n*R3#~?fk|B}-t7*c08-^CVx)lp zxX05YF9nqW*0WY;k@IX#% z%GhtZ-GzTJCDsZFo^}_tY_|1BTCVs+9w7ssc!Axc6jJq=qp+5QtGx$7Dz+UU4#5Gs zlwRRyjSNkV@r&WNwY0jmT__BuFF&r|o&d=+CBXlSs?Rp8kT7s+y@>Q3lv7?KEux7UOh&!+C|Rjn#%Oi>BzX^MKNlNd#{M zMro!Ipe9%3V(}088W|pDp=(f{%1hBVw|Nkm*452WIo< z0qYBF&2U3j`WgxBjsha4&eIvekOIw8m z9ps34FlJ*k)0L&C(|d$Z^WLm`3kf&g&<^07;s7#5*c#|M1@PKhnw3uDpf9sYGgUcG zH`+j-LUcESLBi5=a#X-L6aR$XKj@(MQk0~ZYg55?StK9Leb-Nr-racF20FaL09AP` zfGvGZfxu3$7#A7C11vW0VLGE*pv(I_9~iwRCV_8~0UT0g{4hZN%4_%<$i8M#meUzu z+^(?*f9=0d;B=$_f4Q&RURPFrLzx9v%|2{J(4bx+$Ax6^jIf`@!o*u-qlqOUgkk^$ab(q2aHB!K=AVW*d8 za^nGIu3V7^?)4C`1Sx=#C;2h}^U?-rueWTuH)-oBt%5JjmZMg=OsgJIRd3cmrCk8U z@pUbGhv^Ma5Df*~uId_~0mVh3!ecePK8R}1YX^Wk}Y=&Ty;+>_`b3XB_I#=VLjhDqssiWpPxmKr^inZ?EOtT;Oki77; zmFoc4t&xi1TkOK`PQ%%)*K_YST{bY=MuFQ#RJEBkz#A6=S5yHF+7ffQpVjykwiR%S9Qp(@zaJ;#tzH;F4dD>K(yu0B z+h*jACl9+#N@8@byan{a1ttBlPa}C@LsdRHtw80ZXrJLeKMiEPsPf*|Hzv^WPx;6L zJw1S9>zugGKSby6PXejG=#(cKwoQ!bGqIOnJNHB!z_9PNS)J(&(DWZ7T*=ZaK5wLW zn*_iDQHB(BLt&)TN%J}sUJ~fQHYKaEy4SrMz-Y!5*``5droq&KI_T90WtND}!k<-+ zZS;_;9(4{WNZqbb?}s7vVW!o`eFzXZn|1cPWVSF+vv5r2OtB9yMAFdCl~a4p<;m7t zWFr9>P#XXh(s#}WV3Fr0C_Tp2;754ch@upnB030Qx(dtrB}Fuf*18n-%$?A`;eUHXUH|1|kCA^_T#2 z$yTe9Kz1Va_ZQ@v0-3^UJIHd$7*DcH0cc7IV4Bou20&dAqQWL_uSl6=rl@BB-gzug zW-9>>UR;mg0|mjzt$6m@<&CMt_9k&mESSuUs>D}dlfhr_G$2=2fW2y(V#6_H5*~pA zK9t-wzk>o9k^sHvTZK?5Ml2c7P@kIzpK1jLw3^KuD2QG!ckT+5n0YU1*^-c7rn!wQ z{5*2j3LxoVXCRn$=9}e2pUG9g&ZlQz_^|~Cscnp`{Nnn~Q2u?7n$j-PHYtrSEdZyE z$!BrNe~3t%j#AHr0Tc$%Zs6R2$j5{JO5Ua@5gsYf`@?JA6QtD`iv(KPq*NOT*!=uw z37s;4z&%9fR342e6H#&?Xn7W3M34&2^ImZ?gey2q1(& z%mZ~$oIjeaUz)Ey2cX~NDZL(Q$FH6Gd-vnL9mU>yZxzIEbIyFSXs-`#$x7p1&uIWC zV`c%}&{L`~Fk+~e)|s0IxU8l}ImvDk#5u$_Cm`N^B1#lA9~J+dL-6BNz|K#8;MV;IWBfOtaq$F5Hts(EhWqd~YW;8C z^L;`8pKYs$OC`6}_UqfH2?!z!t$+h>1-wS9sg}1^0PRx&=v|wxs{lzC&H>Cz0Z8m; zE`a|1b^tQgl37M+4=Dq}*9Kf%Fgp29LLmTF&(Vl8i))*u zGM1zfv+W5!4~E8^$Xj@=`Q$%`v2||aVOnhfRICj!HES1o14P?zer{e+N*fq6iLfsj zewN+@P$Lyt^zUEo11y0VMPL76m);BbEEIO?|4{yaIrTEjG$xvx0718y^%W*n1#WA; z2^itbB;SMZ0$>}+5{x`XKz%&Os4@=G5_w- z?#|9+N_z`vZII?G1GXS@0d#&H%QzRbjV}Bl1qsh+8iS3lu_&l-r#>tzt3Z!o6o3^j z&HO}h)08g_KQS5itP?zj1;D%uYy|t3xDa z3}A*TQ+h|+0f~G&rDr~W>*iAsZvnub3b72ydv(E10L~*By0u|83mgr304-?6Tn7X2 zjY$+VMV(5>_I5SFof^Sn-@MOI#L+ZbG~LT^2G4$yzsTKRp5a7GIQ7C6yEH4KzpVAQ zb#OZ71K8sPY#Jz`@DWjnRa??ZS$DTzJ53eLXaHqdh$0AN+e@##$ATm0w1V9O;04k0Z7$4SAC4*Jd;dl5O zf2V%esT~ksELW3Z$Dvp7i1+$I4@CxB`II;eINrk}@St14XYJ7aS}S@D;&w(O;q$Ij z>%cjgru1rA6$tziM>92i@sLK4KnqeNuq#GjV*qyYG?-pP;XtSxp_#3(Q{=tSU)s9? zs3h9}K6pb>bv?iz(`k$mcQAa=`@`A#tJpC4F3?e}FG*loX4~_ObRwERK)sot8y)q+ zglcG#91v6D8aFAiUY}ake!@R2y}F82;n$+;J4gVrE5?p8UabZ~CV#UC0Xsp}JjFEcK5#5FCk2c_i0Q>kI9m(nINb%NS|=Fh?d?q&i<#zb_|x{b?DOHb#5y3x zl{w7p>opJkX#SD7bAo!Fa|7+_?)Z42xN``&_qZuT^f_}%E~SrRnPY_d_WlruKPmXa z6Nh}(8DwCTu@?Zgn#v*N!DR6Pg*G~N8e-d%d*#Dih3F2*nk&F|F+-cyY`WdBPPeyx znczi5IC=6{iwaCU)CpC?MCIyU#%;9tqZMVYfSRzQ9ItFc>(lgjT>-{;-Eer?COLpH;Po-F^s`eY0DhWRTmMKCu2?`c~202T}sPmGG;ZnO10*0*q9IBGg)gDeFnK(u!x)S{cTF1<1s86vyF=JnV`!ADK@*F z+_#yZ>cjZjxq_~|0m>q{%oXMwo(I7ev} zLo>}=zbpU=2;1DNIBNd&PS3=ZjYNK?LsC=J^Cf>QnSF3VZX-Ff0d7FVG)2 zxbc|OastHf5>6dk!)3$|5lyd*`hK1jnR&YFhoI6sNmN|oUci!{Ev1y4=l%>k&BU5L|fC_!wpi0tb9yGfk2cEe6RWvBJ9(46z0$f>u zO0^<%*SMfTndkr*jI{zx;L{cmqE0jrtulaA>7NP^IG@|P10TK@h6~+2%R+aA88<{M z_p;@SI|89zl?^rUlW z-|vZvG5j;PZb~}h?FMAp+ema}AAZlbKYoEFoTRb0hSTDX^?Pp1whShHnpIKBQi<88 zBAcP;O}8_A_Sn(fO|`p^5IEi?0(wgyR(FE(IV&9LU{NLCLCVe7YvzP<(5?|3sTKzH z7Uig*3hU07O&j>edJkbeXh%~B5-EDi=?k{_(nE`7P6;Q=M8>|ct!hc1aNr+0ya><# z^<4clksV1go7Z6FrzoB-h^zz*exwGJf&Q;IuZI9kf&BTzn*sHMqd}1Z$h-ssc-4`e zLlgvN6GGdLO|ypZBg=d&I6IKe&hGC>Uj=kVV#nMt9mc+AG1_XCSD*gO;qZLR;ha6v z!YO~T|Jx$nk%67vmAge9K+b7c!*+9QYtLHJb`27mUCpTot#u$1A$pNH zm=iHAPWg_ROcQ+s#&=SHmqLlW1$6RH3#zTg?-`5Oy{vV_`sz)%w6|J$bPOOq>h6E1 zb9ubDx!O-9?GZ^e{dPrTYG0Bl_}9(V3vla@w&T*t$2uFl(v{>L*F&xE2rj3I@iz6o zmWGsBn4V-sO7uW$Bt4E;v<nJ$@-OhvfO4L?H$p{lZ8ku7glxB!Ks=+XzMUQ@7< z^P!`^9?!cT+k@q9yquo(B-E9ul$seB7j!mKmX@Uu}{seQ>^6j z*={X6FKj`4b<|RZSp5$fDNCvv^MAB?{>4zz-a|dZAnb9yRJwD!P@rXy8aM)S07&ST zOC%-So(58)#BhsqI!wz@jsij7J|1BdpGENwK??O=1A&t=7giZr3FgvSW@{8T$OJe! zO3bb=)J{+!LMswXbIyRgQjrJ6?g3&B3BABxoC9!qZG&a*9S1{hR+LhU80=?m&KmD<)s)r&gEJgMJH@)D)BV<&8WZup{z3IoxKnCjfjEC(rB7mn7 zuIBT_ire<6+A4p~c?k1Ct^A{AF}U#JB+9VwG)PP-l2=s7p)O$lzreaP-9(545D3C4 zVDW9%y|`a$*$wG3-)Aeeuex!AGF_NH^K-GkA5)))KFiS7^~QDIrG~i7 zANKnIb%?AMN}XK=ZN(2iO7R%v6E>RLH9d7;rX{fsqlN2fg6#53s*Vz=@XUGBl%F&DO+0E2$9U&6~OR zemunupRb8eYM6!j51q)p1{{9mwMh0S+a3MDL(i*-L0)5Ay%`imqg!zHMgnPZ_;Yb@ zY~)2UWi9|@-BbX_LLD{EG<*|~hwuP9eLrTE>nv=hBh3nA{jH|klE3BD8Y6|Rz=S_K zGviNJDl4lBTIfB?ip+4Op8P(?pvx5wbnorN_ODJv!7Ujf=*tqOA0?dVLy=d?Bt%`-J~b(Y zhJ)mr&pgXi>DGr_iZuNyL32Y!P?@opz8dW3v8z%si>QdSe|2r0aKA&V_ssn z01f*QXP0GHCH*D?wTiB&JFS(>5x)13r5@Gi=jg8X!9U&hBN;#1{Zr_Ze$J}fTn~Ll z$a8qYsi`qQgLvqb(T!6-me$forZE_%BuDV08H=z49eN2_yXyUR6s``=|3i%X#}mkK zffE25T%FA*DtFaY5gFA=tNWpFi)2CEf=*05PeqMB4E#KSlmC>_`~6w^^b_);mbh5S z-m)vU%ci|2zuS|RLK^NMW)5^=b_wE4OCI&YCH7hD)Sq|mkK6j=r1_lZxj!h_m0BNV zy=&C9)PCs6x3FC1!#+h04USW6DwCASh!zg&ao3o&OTPc7lXYl4=8>_N!7qWRPn?{L z45``aVjSJ~+x_cd{L=wrq}!Z|4=}I6e(9EN|MuWj&HSnar}#L6ejFPWwWKEau)tLp z(QP*JFHbBJ!SvjC)5hGMUlRW{u`CtO-iqKy?fY>P2?8|SN1ztBz?*s)s29W}f9n%9 z&6rJ>mf};6yyQ^Gh>!{^bH%>q-W=ey0#F|feR-QvvTdA|#HAHnda@O~TUkZ6f~+7} z;9dAISBcF~4nZQ$S~W~4>4emB5&{d*b>1(Vj4D>rMDYvoBi*UY(mSPQJXpG2!=Ku% zrpAq5iQ7Y2%L4tOr!0oI07ChFkIkjpb}+ox=Cg+*vgXxM_4Tdxtgmqb+PMi|?Fi2( zqcC7Fa4bUu`%lTX^!h)U%BtpbE!slR*bLxkt@=v}{;}}}lsZRu*7@@jWOZFFH|j3 zk|<^yRBTdbVnNi7w?!vheVkrIsr!MB8F+eqPGjT+Gx;*$0q`LZtH239KXCu^0S)x- zgU2J!mkjxWGDOb7_>6O>e<;K6ic9i^OvuM|tdIyv`QeMW-M`DT&v`+R0c~)l+VM>W z6P9VOkUL_XW?iLqk3_Nw=&$xO_Sy_eiKo==6S;z%z6PMB$zmVeCok_5cwxL*BK>-U z{%GetFq*$Q+h(Wbbwz-$Q5T+iZE$T#jJ^i;ptn5$Bpg?JFFMZ}k}S*0*$X!+p#u#! zh?o3ckJ8&*I00L(IWUb9_u`du>2_9%U#PDRVu?Sy=q5J0@sYgv+G!CI8`cz$x%X7l z-jbgcIlNIWVQPyt*b_cqVS%kQo7HrQmD!)<{;p?UCn{c@)9I^!%bwW%gh~o_rsla0a9YRVGMo{nt?vE7|I-+VBiXvXJJ zo!AmwNRzcI&0aF@O-Ne|WMe?o7XCO!L-uVmSFYLegcJTES(o`05_VS%|J8wD93zf< z`YD82zWptEGPVFyz^D!aODp$|!|`fb*>y z43eYMNg>MT>|cW9?y>kG#*Zr$iiw6IJDW2z03N?kCy$lp{;!$ddh_w1C}Ivk!C3(S z<1Tt15PupByU2U!>d5RBYQ z@&;1JQJl$qCz2SAN@5h;Fq;|N>I8wxhW>FFni@ag7~253JpP0(ocG*rdo9{c&wlLHxvXN=3ff1keX) z3bR5k&|e-476rw4Cc79B3uWDMw+DTo$nbJp;8){rY{3BhIN-cEm8yotjlYrjK6|Sg z0>U0=74w>$uL`NHG}iJmPWO}y&If$2UQ4D{i8$H|O6GOPI~huh7_cM{$RfE{=L!{O zI$dS%a&PkL$SgeRhU@baO-@>_KYjCPc4RkQoaRvLhis*w{W`af^1675KiYF8Dk7B&xI3I^=8i_>LzTRBRzG?M3hEXAmT-o8LtUXD6x_5-H-mQm zJ-Hclv4aFCEVD^XgPfunfq}^=3gpA1%noq$<5KyW;?P2iLTeDva8_@SN)le+(ymb~zgCa?FUzTVu4VigJN=ZN#|ChGK>$#(%TT|4P&Stw zcXg=ROeG?&rsT8ASv`~%#X;ca#D5gU2O;YaY8~tn6A2?8)1T@sCX)#wuqZ?A6-$rl zr`2hC_;{8fHdZT})3tJ`00zPdd97UP{$B2CERl#l7kQJ~{2<%=1It+u7MP3^>}9$730T4!a> z6edr%&(3X$VA0kfcqL{xv-ufLU`nm<6^0>ev2OR$XDe1YMfsQ^@e^CqHB5*b-7IUZ zHr+l}L%9|0h9QHYHHw$TR^QJW*mei%yG^`>Bt=1pCQHFPp9RR(Lf2!SUS;w2cMlLK zFCQsg>3ED(;e*G)HTgaG?cB_i`n)?xyMkWy$^;A(In*@m1{C)5p`tD`dM{`5TH1=jH3q5pDE7Gb_Cb&3 zD67VkwPE97*~jm%c(&s#IOHf!h%m8LGDj%~GQV8C+Qr`wTkcl7GH%lAyQd*N!Vtc- zPh<@%?O{^nO=zc2CBP~#>n#i>T69iM04N0czV-S(TcWfd=~|!NSf$l)tYo{RqeV*c zQ`436Ub_P!SRS+5zWn4x9nTe&=@*|$vkW)x>FTXFuCa@rniuo3?Zf95Cj;B4kfv9h zFM)!rs(OqJ8ddZKhDhreOnmL_l}=RcN0_KyD_$5MO)x!id%Eg5>ywT|iyW<>LNf)A zvPE|t!xChs*V@u2#^0EeDnA@hxwa;8;=sB6x;?WAi3mrJ!rzF30{mX#%q-==+u@h& z`CM#hV2LK*bN8&AsQctS|K7H=5MLEA+nurtMo=0it-9&zxK&HpF23K}yCdu4Q`FX! zoU9fAZnT1(lceIKcn-PkbY9a6Mtw7vrGW+LT>}sj8exjMl@hB{n%3^$CE#szu73OMpJB)Qt3I?f zIk8XID%w4`KdxVN6TbM}b)Tc54moFD7lh#rk&Pg8E-2rRR`%He5_A-W&ptQ5_|3O@ z1j0IDI;fQ{a-!NcWudLy>$2|@WUz(3l zZ+YTl4tlf`GpMaW_|_^3i|%BXwzm$UBA}$8M$xXQ0G-m?do~R~S)z{xF1Wz8?^P6T z!ih*P{kp4DoM!d<;7oGjXjY`5VWH^2T3pZ(^AltnVL|3opk=^K1#5V ze}Pc{n7kY}*NjWc8Shh%u&hVr7^|# zB^#OLX=cfuShmA#x#igdyWbvzBH6!X|8QhEoq6BYuM7p7wVXzG3VE|fLlDrIag``i ztJ9($>YQQis?BV}+;aVJuu*s2ONpifbwM&?ddRKUURpZ$?;kIY)a~su*<#PlU8^2{ zx?>f>62#<93@)zlQ1WM!XW2(JEn7*%wBb@YdG|xz&!KVPZuzc!91$6(8cTML&f8pv z+n*)!k7>x~pz8MevFyZgUHagpH{Q7AXivCW`I6#MwCzNa%$G#e`&ILE-AA7Rw-eR8 zyEgQm1w3q$j%u|PyXHQ>vhu85u*r93`NGmsUb|Wn%vJuFD(jqD^rQ#~b&W)ulrnhgvbnw@U9+Q5>Wyo^P;Fz$Rh=(Tp_Db-ESu!SOn2zZRE zQC?H>9)HKL>R0Zbq_1;mk`mf&*$)B+7GDy}_=m>9Q3TsglT!iGQvto|7v=YG7ZyOw zByry&r$c9HuK-dDN_w%8-;mhXtr;~YGFH2vXte%`r42M`Hq_679^7g0O&PS_k+Y48 z?G603lJ0(f7S_pY@PbnNOFs2Y`Tq#-dyZ0Se>Xp_OMZUpbnyFm{T3S)+^Q0SGe6+{ z5y%$!RpsV|G`M(n2c1{6nKe4wV6!HFQfMQPGq&4O2s$oJ!Wl)R>O8I5WPiO-wxJ%d z`^02zpw55;2_KT-yR@EmF#pVX)#3R+Tb3tcc+RG#NA}?CG+-;GQew5t0@znDRJbSA+{Sm%C zf9cL6B8;p1*)Ydf322wM1!QO5okitLGZAcCxOJ`(HBR4II4T-zF=Rst*G98=Jf(>x}88v z+1ZF9`=?t!^0EfH#&v^x!sB2xPmJ)>iESlTJW0ZPIAfkMQ?Hp$j87!F!$@Wxz@9R_ zTqr{?ik1p7Zg3D)6xh0R6kc*}VVp1l2YkeDTdp}EV|KA=?e1fCZMjM3J~T(=AR%>t zk7iY@aX&J-1P?JHW6h|DTx*_o(QY73+atY)#TPqiF}c!X&}}x(bzB$)^ z=AaFakCPN(^9ue@v%gmzLqh2g+&rqjlCj~o1VQrZ_=w_$Fqc=O8waniBb%qZOi|ug z5DT+(v^~N6?%EbI+3mNFdeWrF`B7WNsaXW5>lb1JADmCJaY(8rBFn+fY%!j(UQ8C= z7w_~bHAe-sH#ElA+{Ydu-|p9d4xBRWzRKlK5YVY3eyQ}U5U8HC1df)~{L7q!i!V~I ze;Eq9a-;NxgPWGW4>KQg6L4IiN*?J#m(H}sVvKa$j#a-eAY&>-^su+PKM3vIWb;Q9 z)*1(R(|cZY%OO3j)}R_+>b6UY@R93fZVhT)%FAoAHDAOldFI9E z$@dV8$i+r3UULj;PyH8*b8GBq*Xt7N0-W+Wyag7kZ+4l%Ux>$&uvY6O*Yden`jTNG zN?%bKX`Vw|u=U$I1Kn{Oy`m346?CL7G6h7&k#ma4mjrmrWpzm_pQo!Q;k?o(6Qqdh zI*FMSPn9x0O_@g_pkUHjpg1p?s^T@mm}Y^RfpHrEtG*NqdXK+20XEUpva+%jfGC+H zxlqF84e)tT=AVKRHYq6D+G21W$OS;GuG*NRU-y`}>~e?FR$mjqhA7hLngt~SP(bn! zvThvV1cX6^-3-#-s`9EHLEqcv@oCuFXDU9#x8Tp+QT%5Tt=F_F@iFG})EEF{O5dMIs5TCO_0QM16v9yn$&PhwQ5 zn+LTyE3I6`gih*Ca(l;&LE6ubJY#GCMLfPi z&l__a`GWHcJ;p;rhilNOOT69}xo1AlXhz)fUfI^3N#+JAkIBNcik-O8L8WW10j+Jj!UO# zJHXQ%cy|+Tnrl0#`0J1zwiasBsYCHSnX{sVXv-K z&uk*Bx3a4<=a%4kPYF8x1;a=}C1yw}f3r;58-ZWvGWS3&zl_q;d4BW*c`^fwGJg_e z0?H%ZxexExtra-)`8g~rfs}wz5o%zKRja77Fzo6Pv+l|gFPrM((k!g~=ll!a8wbww zq2J7^Yv!4ZucmGxGlTBjOJzNASmjP#`_!ko72(`78;VJ+I~d`uQ!4I@c;FS>jJ`i6 zsaCvjfc&~(k_tvi-?=QnmkrFY?s096)vzs$t&r*p@)jSnlxEpwHqj7^@_cjWVG*vxBC5;u{>R+BJr<80-+8}|=Q|FQ-8EKQ!EHF(T^Y(QNlqjOOU0~~ z_Vq~#t#xMU;YUib7pIlXcCxRI3ClTrlZBRR#`I)sd;40}C1@u?$MzXirXJ{AEW z)y3+$wb#;=@dLo=n%nBgcH!PF<{=KmqzV|t&`$X=iov&Es6TesNOyO4>W!Do$-p%> zGs8t;O~)zOS1#E={w@In?aK0DagF1t0&1Intpt0*RI9ZUXCZ#!nlyQBS+D}-T7>eX8B+k&OOT({ zlk21Zp4-Ybpk_#7ze;d5+)B%j8d;9(5;O|ss8~+z<)iL06EcCeR z&?3IpB-})7rZ^5}Fw9F0ljqup_)3o{zS}kH<*9%%-HnY5Kwgkq9l|1bIPXkEWY-$| zCZfvAMKi43=S~7q8LO?)P>2U5%8Y$Qr$&=MDNZ5@_0yEPY9O=$ViaYPLaOg*wacs`f$`F?4Q~ex1NTAg=N^{QkO=}jr(7&qTSjv7` z)HlJZctk~5yL5divmfLI>k zBmQNC#&s~N2i75C_mH4oQD$Dcj$x(U%2caTxYgHZk>fA`W}B7XrhWeblYdm;2n2F2 z1b5(+*RkW@n}6N{jSKWGrF~@hTYFV9wLuPqQUn4yUosl4p7{9jwL0wlGw2AZcU4+& z1NQl15M-6Sn-2eZH)UkX@=%X3=pxciZjcU=tU)5Y2b%(oHh6Q(D?lGZv6ku&xQB!$ znp2ln4I=tqhf6{&Oum z$j1le3+CuXH&q>Pn16lqm1M!!0nH*$d~1$X0VRW+0xV)_x?M(uXyZbZ!}Bevt)VP} zbqE%O44QUqAg#e`I3GoFYqDrtNaQwGk+NKPAjT4Z@X%GwTyBxzVxf z9IK-*7z3IP8kJ(z$y?d=nPNtmK1NneDM>OJ;mWb-9JWd(i+PI5wZG!FD-*YyPNF-j zD)S_qm-g(nB|mCkU(M_5kL$5}-ZNdjEu-2?oxe{lG(Sy7XTTFHmZ8&RNS<8om*zoT z#t)8hms-79TON15jFLvV!ylmRq%qjhKF;Qg(uNtgyCdyN)*6?Sw7L$>N8UYt99Nw4 zhGc4yc7*@f@yY1Mw~U?p7%e1xH@(>)t~7+w805oe&6g)=c-YV0HZXh#S90zg-*ETK zVi?_HjR|Xr_T0TYT0H)%@hh9clzc^p09K=7$XF!Pd%=cVk7y`gCs=*N2i>kQpVxn% z0ftx4P-X&iQ%;6)sX^uC{X;`T>zd;v|#ZWtGx88tL3hm{g^Tg=jMcQyX+RL_|))AmJ` zkRS--htqeY`CKm6wJXfbWlKnrKd~OgV2Wqn8zrBkb@pnHm_P8MN)NZUu{!ik&`Ab2 z1!h>3`&PaSr*;ab%SKT`J0uxJsL z?UlQqCdcpdp(Rc}mQ4K>tqMt6TgW=`O4M>l>MjOk48kA-@+N>2d73B&ow%OiknLSE zGqiGd|LrAHMm0sTsUpP0F{j_+{hx#?YVFkwr7iG%lF|BBajCK?d*n&zel3n>hMNfs z=~GW$x!SKWos(al&n(ABI&|*Uy8B4+hLul-Q_GWfv#k*pC!N7AGu>dAF>3eZv8Kxt z*T;JrnxD!I^c}%7il@p z7|)o2ao2arBD`slF({T#TVsYVc-0Ut5Q(thmsqF@ozeVO@9k2y8JqcY zF9MaE#^)8CY<=UxHu|i!NcYX>+-tw!ZCA5rHIkOaq7LtIEl`pm%3g4q>AmovncmOnBVH|^%qO?ru)MOaRfHp3^|T3CY(Qk z6~cNtJmlM66+}_+2F6K5Z>%F|v<#=fLPM$Du|vA;nj)>isoO=jR5e54Z~Hyxj%v*w{j#d)+>|5=^Yh;U zeU;I8Pkz}oI%)9KK93XEuFo248nJLNz`P!oA?Rp3^H3a8zu{w6n!TG<;0z8?h{rWI zZ$F;3)i)7LElE5)iX?eUiJlsUdv#Xe+5}NfE`=KdJg05G+=1}WiS{7YOm&vC1$}vO z75RSkb)$lB#u|MNXH2j&!cX7MTf(8x$~T;Q1WXs*c=>w9X&-l&<3sPI=U)wJUPNnI zpgJ9XySh&Zh6ZZ*PV-^M`F9D^n|6Hq&eM;h-ysuCYO0GB3<3l{Wn{b!vueItW8L*u z&S7QKVRNQwh=^DS`(o&=KcD|hAX?CbD^#E&P`&QuPt3`sIdZ|m9 zpJoo#95Tar!eN_pIbRG*O*&q%D8TQA>>>tfc3(N}(p5=*HRN&S&HfB(uaz-@=950d zk-b{-*x1+9lk~@36}|HLJ*JNF;x74Jii`ZQAwDN)To(FyT?gG$)jdJ|?rXNehuHEs znWT-nx?8DJpeO7#kYNJ=<9Q04fJl9SR&bb&jjyn<5I5|{Sr3R(^r+5BNf}(Y(9cH; z%HOhv9@A&ia&jU-6;WTdhx5|?1dncqP%T6E8$yCTay)UT)6y0&!p$^4Z{YWH;d5Tz zc<`kU*Qoj{%ncRR)Ej@1`eB;0b=dn~`l)_b)s8eXcbxF99E7*+H1B3Vh9DQlD)wG4 z!bWwW7|gmf#a}#Pyz;Jh#9Qz2KHSSSJC_4?bj}RDecJBBh9N8H@QL|FJ{yIrCf4B@ z=&OaOq?O}ZQC~yH?6tn$6d3cl#GXN0b^D{(b)rf$t1*kSe4pojx#_L(BrHi;+H2F? zLYh;WByyR&)?GN%!+Yn2)U0$7f|_UN^F_O535_1-c_p~}=qPnp&VaX%NCy!{Ow(_1 zNJ?5cIu?4ewb#*EDJ4o#x7o6(m$tIod!3&Bqm<41gQk_ky_>zW`9Kw?Dy`L7>7Jfv zdjBNig+X^%+n@!_Lfrev5=|8b<&)DpYxf{JwYT5(wNE?-(gO>X*^Cy4BeIu?2cw?~qKLca_Kz)KH~7ub+aL%Wj?3;Hqy ze6#Y8Xmk~Zz}!s7JGXXh5!zlv4ml6TW_Wh)7Ojvqy8m$($fTWPKLcxhcf*(d)MP}c zs5d7U3_svX3^>BcDJVYR6w(NX7^SDD-vgsyx^6}~I%&&O!QtWDn5_lZt&RNe&crX@ z%PvqoU(%X$*&|Zs4L44hscUL z(b@)_u{d}^jy2b^|l1H%vTgUIFt)Yaup6+M*$Y z>=_vP{&Ll!tbXYQB?acQO_lynIICPT-L%A@IDZn;{JZee4N@_@BJ(J>+IrUz)C<(ygKIdu8*YHT`DRGJe6pYGw!|Q*uW~v1E8;u@4vl-2CFCyCAFT~F$dFb= zm7GzI)4Fe1ViTwC_hsm-Qifcl|3=xd-n>orkonyCtE;iAhLay%G|ORnx#__T0j4`8 zjGgHP*+N+s46&KoK39I8ncvPlvRr$s8+46NE^6(?ix&?8R7xEfP@1~lTwJ{l=4XQe zm7BQb6C%h=S)GLb_M0z9{`>cJ45LA8;>tk*-YtKaEavz*7xy3isR(xL6=h%*8Wz>Te?*ieWEvNj|9^zNbzD_j)HSRK(r^G# zx{($E>F!35mhSHE?nXjs=?3Wz>6Vh_&<%$^H1GC?&%O6~zxVsYpGyvGJp1gm=9puS zIp#${iP09N3j$fkLQ=a`P|M>?=B#KuOS$9=WQ>bSvuz|YpRRDIRowfG+uGrVD>h0s zNYN1O5Ypy-`xq8NXA~+>g^=812%N3+dYFLJyj>8P5Hq}t$!O@Yo+x+x_&C0&1}|y0 z?94lwH1?QBb{ms4Ui~I9w9r&5*x(vc?ZR>Ehb0Jr3a*%SBKJj2_4X^c0{*}btkEVp zcaf_*Mp9Ru+vUmAo|3b4CS#vR^idpY9#`FpmWvhW1rO%5`LhBoXywa1Zo=;gpPk1U z4T0bfw~d$7d`+i*0uhJB=!`O=rh10OKmdT%X+j6)Icv5G zvtf{wCPibdxf)10^Nc)6B<}?`?b>~NnguGfa{*na;HZ9q(xG3(B(%5pIrhnY`zhGu zAyY9JHgG~XntMq3XF4HFqS5It@*|Z-yV_2|rW%=t zB7$AJUB+DVgQ6n6m$;0>gaitQb6u$$gvJXsHC*W;@|)_W0Oz3P!sT?k(f2Pip#+%E z^|c|q&N=&YM)m08c%$-a-(cy;ItEODI63sA$U#fEPkmInu!9FanqQfdHLqd5_VuSE zmO(RZWoLnoN#_h|6}lAqn=|w@HkrR+y9EL;f#O-2+0hIeGxG;vj12L>#f%OkxAQ@4 zI=73IvvVB^A=ebf?yXZ563JjDk6V@Ha*MjXg99ZSn+nj2Mxm~Lq+C_{M&?f-``2Xw zL?e()@r3CCNIx_k@H4SG`>)=R4gY@CQ5mMM82Zv9kvO*xYQnQ$?%u}1h`AH}D^x|B zL^-_vGrIYaM|OPExh0+}&2GYv7!fX+5W>2y@8nG+Ss> zJ;QwrmL%?)vnPqJO<{?X$Z|e277Rcod&a% ztw(lvciVB>P}*{4`yF>DypW#AI{TW_r0y_{td#=W?g75x?P-1bx^_A2k;Ssl$D8Q1 zHwxbnN4EaEeju1$?wv4!yGBb*()gBq~0tmi&<0 zeRWCuqKq*0@UK&eL=%=QA!i?G1h^=uH4qaO9Vk@DIWMU7n#=erqW(VInqd5N_3y4< z^*9FY+M)f9WPdGw?85G>P3{)>V*N(S0=YY9HX2!?XKbhIv-hWitwi4ZwnjnS49 z#IVrNn&utq5uN5Ej4VrKN-r)6yGs zpYhNYav!}QG6_u;fq&6Ks=LQ-?HqO2xy~+aWajr~2-hJSCB?O=#QQOy(J|MZytkfw zuNg0xGawPeHeDdr`ry;{Q3A{+1oIfmvs$k3zb3M8KC zu}>#e18#5p+AK;avP`9U* z>Xup@t72{(eAx6Qxj~h}mu2wvW@+H|ldYib`9dtly5>OEC1Fnb8bjOJ&-Tl*-Xll} zlTzvrZ`x1gk=8yJzwYt=l_W=f7UCodO9c?N%prJj}9`Ut%FGfJjg>9!4~bekMW@0w?_F ztvZY1p@aR zd`{w{cdR1%WDN+QHOMyOR%@RoWWH4sGOfc9e#M6i{mFZiYs2aE^>s_%k|w{%WN`ma zobEnp?G|wpjDh=&lA}+ECfYgM`t!OGjU!E`?DhvU?KZEZMRNJq1&qq&LYV7VU@i}9 zHzJ?XR<+nz=O4AU-$~%%?NMqvu&&oRA%Ar&O#10l6kSqk z|JB|xX?;YfBli^SzG-Hrz)al|V}b)8kcr!0j_Dw{R_M`lM5f&Y@DI|cjzAqWzd@%V zpRo=~&y~keD@04MXj*c)&ymyj|6cuwTSB}ZzBO4@4 zAM}WFg{blvRMNtuq*0~cldC#f&4D>xY zz`Dv&IGglF^knm1melAM?#u?Z(*FjgztdM>0a(^$>E9vWKYEp4$Ep3}8?_=j2A5E^ zEQHcffnZWk^@E(tz|eqN{N$!OMncIkp<6^;x~BRfFK4=HNX?y-mi7Z#VxgGN6!R!A znY%01%Lr*FOr}K)J}kykS*WRivbB)KJc(mK@!w*a8t-y6DkWl$FOKVb?b2XDcD+ zd(bk~sLCuXj-K0LHq-1K32r>{bsvg5K?l<=kENh43&@D#h7d69d9KK6zx9fG6U7+pW`lWY3;46 z#MNM|wkYTGP?Jg=nZL|TEAPHq>iSm2>Y!x{g|nAOj9D<~nhHmqWV{hg4e6-$SEfAX z#6yLCtSBYbI{HCQmNS2WVmkJeGS@J0S&NtD`(bY2&Rr|-I8-0xs+hq@OXUNc(o-+RNh8zYP4F_b%xhvXmJkKd8dlksy5)D~V9W#=s3rMcw)M#{jazF)p$G9i6ef{u)dV%Kw1=KW{$)S%*`%E6x9W7XSD!{y)Bm(0v6kgbfn% zk&y45|LdYx7+JSf#F#Jtb%Bmm(zn|MV-i}j%FWI%v65%Y9}Z;x{jUFcBK)x?Ii4zT zda-`?;dfN$pha*c1jU30YzYE4*O40FpSfQ!!c+a`L4bwd3T!;fbJz+);K=xYz8!+s zFran**4)urcClWURo&pfZ)X-h77my{8-FZA5xDXW8thfVE}BW01I0go&HwM(4~+yQ z5y{U-B~<>o6DNkyH&ysQ4+Mc90Co9c-qWIo{r#)|@yP$@!~EwzKav4gj^R6F3^yyH z+y76F2Ye$M@c3jV!~B1|e}QJicT+P`!mS==x6bH;Pyc!4-)pW&0ePl}pcm^dIF4xl z`Jik?FjsE$#W&101^+KGUtk-fu%svj5tif8AUHlO6G8%;lt2s7*bC! zttcWcW-d7v^qS136k0RO(ivrpxN$vBV$jN~!zF6v&7KK))e-=9amcO3yGvU+1s#CT zgWl}VHCN`x_epHWacWS#E_j{85$i=ae8Mm7S>@v7|B#RdikcBcq3}ncAbzY7yxjN4K~?YfR`$O;mJ~CKgkMD7@;|K8pIaG}CXk5fxlPC9?VzF% zU(me(Pib&`RViX(W35waK_;*0_Jgt)QkF*cowPSZg0O_7gDg9nlGUT3`!z+$d0Z_f zO!o~Ow@L~1P1`hFN`I50(KUBoev8Nid^AR>fWST%tht57Cg3F#r(ANEsbFSi*7Syz zm31W$(AgRR?Y32(*Y@ouWjE`#nRaD_J z`a2o^ePnd3lKsiUOG%*3aISd&-Wq={qUm8!=9~=E-xK|P&JbY3f}CDqxSIXGm0BSJ z<82J1hHLJZNqoXo(xXgNQtb*XlwKWb46|a-JVS_iDrd}wMEWp0JQN?I6G!DfFvCmg zo7a7^d7mxDsfx2-)GH$~5Zk{q8=D=K>aFFap`*Hvx!!d`LetuANLzpj&XH z@Rybcdm&CrBk2A&fGVjAW4)lDUKu|8fd*t|k`cs2(}nASp1a&mHj?g$!Y{ceP%n%rCw=u*3nPgXfS`F{>Q zKQ!{Uke-gX6PjMg+YN%xf1hGm5kM7bJ!wdPngs9H&IycovXt5-8o&N~8wLhFA=^G$ zIBq1EZPk`Kt&QWs(@2)a`a>-w6>-;6QO6J{Zaa8mqoK9R;c|ER9h%E*N#Ky-NEc zt1^Qfn6}hA<>q@_B_4wirhxI)?2xMRu*66Dfi2UP14u)D)vXSR6UYf(!Z@)eeo$R^ zNQ#J-jg1ssPrJVi4UN6pc(;qiE_rEnCzC83i>~MTHUObc4iQS8V}NFnd(s>HyhynZ zqlm0!9jhONt?eG_odjPDO=6YGnVM0N7S<}`OX{Lg>w}6tTq?VRKdSn0!)zPh5+wnS-|KI zmxhk+6X1lb19aNK6rdrVYc)HUy$Jw4=BHYJfCfU)*#KQ!AiZXTJYeZUZqs%R-1MWy z@-}!hg4pd054mK)J0HLoQVKBbbMgi{;5EDqjQ;il;G*RZyaPM{3Y~Fmos54+;J?El zS7?V~=FfrfCF|8W;0y}Xze9r=)%#Dti^hX_u|dpQkOD9D*YTv4+3}R}%6q@FB(M0u z*P`l+FqOF=U$?wiWXyA&`~4VZ&n@m3K-w9~m{vQ!Ob46BI-pN}w~`{~Q}*9OKqWw=U zgLBV z?5-nV*|f$VWBN5~_brg_#q-&g4g?!5Ox#DNCu~7%wQjMtwSCm!u4`$l&XSRjT%uo% z^0(ezXaD31$`DcSd^2}^B3)&0-Ta|AE!a|8*$ckJ0VxBKrmerx*~?-Aln())8&zwq zG<^E`h_jo{Lc4Qq?dGJe_}iUC)^3!7 zG1<737%!`@%&*@_vc$;B*v&gD!N?)nzF|q?(~Xplv>E)?bH_9jj!Dd~nQANY)rrp! z@ZZ^$M8Re5rtB3b6*-T1$q?Vt!gGB(#CQ*T4KxzB9O2{Q(trUoRYE2{q;|kt+5*Ez zR5YmNd_vmW{|=}c@3dWkfsuymp(k;f@66(5(l`{})&qX+W(q#--j2?2_E8+4OuFh$88{Cq``9#px+cX?v+KvF1h8vzF$NO8%0*CjFf8LKvHfd)Y_JMWfL zrrB)fEvxe88*2{uPS|VEr1#~ijS*m4s;8Yr`08-3P$hnrTKe%Kt4XCoG1KC%yVvgq zjy3=X%vazyKWo&LpGnuJb3+#v8eSLi<+Wdz*ps-SC_$N+(Cg6hBUTYibmKe#U{JWE z>Nu5!C&awZv!C5(arCFsZ?owEwaewN{0Jv|y+8la2|gi~Ap2Gz*q+174fVSx!WGnp zD@z5IL!~`u)(R%LqJV{B8wKU{a zX>MLzl!S`!IVyzXb9Z$j*C^=;@c@6?uV~jcVYiy6c(H-c{FiR!_mk)d>nEUpcfyGg z841Ry5EjCb6Rtj_16KOwU6O@Jou6qmD4&}hT{y+uW~>wh6scCY(i&n zyJC?pN@|$x`hZw5)@DU*Vaehp{EKxBSx3zA)>n&8ZE0s`K2r0(`JtWpTC3%?K)p7U zjNkL-8Z&unL78Y2g$$Qbh(iq{IEJ>{{vXr+z{i%ksR}S8!OT(jsc*N>|sM8=kDn z$L>f)E95b+(1KKj)6gHTA!VI*WMM8_U>P?zDr`OlslY_WUnBBRNevNVH&b#(f0xW2 zA-_%G132@sH5-sOjSURg)e28}8iX1X_rI1gwNIIA^{AJQZFhlaFBzR~4EQxk0p49T z{q3ctGJrV|b|otEe*(V*2?4Bs*vYoe)XAeYZuGSS%0JkWKcMQb+})(}$To>X{I8cd zkPrxF*CR;G(3}=`!46?3vYg%4+e`rfn2a@4pRg9-11Yyz0YYM z8!^JR)vBwIRB+Rys)Yhq8mM?EoS=X;^eD;*g$Pn=>MwBts_N1cWcfhmq7fV8hSLXI ziu3|w?naqWY6^+nDJhnKUgIM#`3KJDD;}g|WK!|W@Ep-_-$K&JA`#j?ev45Zqu`)l zdDaJGFurG4>!^*Fr*Li#YhL>dr~9h)HtL=54nx`*$yEp`ZEI^WOh$-W;*%B(n)a`4 zZw};)-1|ZvLLe3fuCI`KY|Ga1f;z)|f7T+F>a=UuS)G*)+b!DlHNLip*kEDt8lzBs zmiUc)O*od{dXZxZ7US&14{kq#EA*76A0F|)QCS=Vfugss?MgmHE7uOhfOv+zrQxkz-N;vaOnmZJKb&v%M&#zX>)TZLy+s(Vo!P&{Sb;saBs~gew z9~jN5bb=fxLWgT!sd=iGZwm-9SwqNX$kJKw8ql!U4Tv9amMF{nPlO9HqV(rZuiMY- z@*V|YIe-cl1FND?8`<>=gx_I4>WfzpCMcid+LkiuqrOL^+dQj}N@U31Dye=q5-2pm zW>nZO{HDd?B4om6aiA#*>XRo`V91TVK654psxGJVykOh%C_=jBX_%ed8tT#@dB1Y;pezqlA=oBB9T3~axGj^ zueh5wzbCD#V7=&_=C5tb7_q{Y*4@pw(UF`u`dR;=Ln$kn~?n7G>Hj~Y#W%3Rb<~@>uL|DzHqygTQGxW@%N|cDzW6zHzmT_K+}-L#>scdydIvT64bsse4oEs z158h3EHMgb=s)M;yr^zZ`ExgS$#gKz8DnJJ$$eMzO%_;A?SC<>;<$d%w%#$Lh7B)>#1Q-|ZxiYZlvU8E!+BHr!WzRERU2LC)^UH$+QFW2!dn zPpEcOnipWvcwBXmV%gD#CElDkW@K zyfiKy-{ZkyYNP3L{{DsE7WTT|=2~A*UXYLM2w!+Zms`J*2EB$sQRtzbsAXQ+sTb7; zJM?{ObsYgP;(mMU(q>P}zP?!nPoR4T(L~Fl_@a)%!3p(;jPXVD`v&tquR{+&#MJy0 z_7skLreDSBO;nj8bQ3Xo$*f9mX^k(_d6fZ>lUTd-_OOu0k`Tw0Vs!c>H`TAMC=_IT z;!U*WXE3Qj$#nCJ-h!ZDQh)8P;ZAt=zVlM~Lyb$$3kbMav~BlUFM`Oxv)=THYzO4> zzKe**qD%`{wt_;5MYH0^LiJ?F&jAuy!g}gJ>JZA~wSq&~NsxUg4|B&}T@vxA@3WC^YB9|$(J+@g8%@5bWxbg%Qi0f_?(rf-;^OB)-49`)R$au(;U7;{{C*CEu?eq^H)*K422i|*=DJUHIyyms-sE-%c z)YPnIc`qmT)Bo=KccB6;1%VGb6NuxpEX{gJE?qE& zr%L(kQ*mxXgT)w_+fLhtN9U5Y6Zqk074U5_CtkxYua4bjZ?^R$Q6C6r?`=A_z6fQM zZHEyp-FtVpPm*7uMZta-Bsrd>Bq0AMnnCdE0P=rlQAc*IpMc)+Q^G;qB!S>VMlCTn z#w3OlS1buX05eZ7yJk2saz+FI-U4nsM=vzj`@7L&37m6&TZtt;9^OV5mG=?mxxO_h z&_?I7uJme5XCo4y=u__bLpe<{_L0K4vqL(wNMDHj+t?oJRZ@`)?STFfW1Bsua;@l{ zBe7WYILHJ$5kQadnWbjoaOW2W739Wm?Z= zglcZL?o~O)5HuZ3mahU!tTeXw@F*nmt7-%;t*(ZP4p}U#WB{GH9G|Yxy&C_6MthP! zp@mYy@nO@zrIF1*r1XQ%k6We{jxy@Ntfji}cpmPO$p&kk#}QF9>B%QE?9mv|o=EAV zSOu!*&i1rGNtO6v)g)3TOp$7~UyEL2Qy|`c>^hKI^BwU1$b@Gf78)AlR`=#|WUYC7~ z;&iHbKNuuM_4pAsDj<%U4>O9am$I@~rBoYwqPJ*zr;j%2sDHAAH4scI)onCO2iwyd0kL&o7-a|1DjaF&O{5L{g&+e ztFR)CgT+qNq}CIvZ$SskOM*{sd+%(9*pFO=^)XY(thcPxx6D(RcGS`tmBtT|olAdy zXA_x+%yDY~?8K*GQY_6wjZeE67`yKtHyU-g1CnBF3YU4 zjfuMvs#6u{WVH!pil_t~C*W-hrLfY5%Y%Ku(ASkK8!*j%->7+=g-1;5T&J>8A(Ta= zSyuy)@Szv=T=9D^+Kme8iH|u!ZYbTsTRzXI(q2wngh|Zb_oe5iDb?+lb5y9!ky~87 zU{O-5zexIw`>d~ z3i64|LB#1jcdG$QChtIZ)l)6uRTmO7Gon(Pj1H{#JfzY|S8eORMihSG^>FS$e%T))-rcv8^H&@nc2=90jIDy#MQU@fkp2%cxizV7S3;@%Jw7l$W6 zii>~=L!4a>iTHaL5R?453n~$TEn#+i{YMm$%{r?Habx@#ek-il49K=MnQ@=OFABoD zJ~}^Kb2)}<;y)0btgc)P>`%5a{n|_`7rI!$pGnh^Gg8sVv!X}vr?*7yK3+;1U0A$L z9v)(S&~%pvejcEfjE*iIjVT7B=C7SA-mhQw_UI;*xL6xehvdfh?UUzrNZr^V*6!Cg zNs|0)q{mMH)Ci;;4mNI*VFG)GeWxxPXURU>!;tJpt&LnPMXzz1bbk^=k5XnOa$FGsr(*V&K2sV3j*hDfOr;{@193Bb95Ak=^=ETG2tmKjECljl!Jw< zFQt#ioZ_H}ZRn{FKbc~?T{X)Rz~;)~g_%;?-d)Xc8?IATGz@xufrraJnk>JrCi&R* zEyL`-yYlgL8axD~E#u0j-1~zbj`I)80!rQpR#AGa0bwRuRILtTv?t5j*s{0nr9^-67vt%dND z-%h`LZW1zamtJ1D5_)HuddQ)dOpXeVxBzIG0j?HQAbem+4>YZOLTL#4;c%i@A9k3_ z1hgo@L4~D%W*DH)+@r)DX-ElJN8{>GS3K_6#Ivp7%$zTXb7?e_?p@KxgwzrOw>C}NOht5s zJI?w$MZ|(swRKLjh!gS5JFELenNb32pcK%b#Q1WsH%4b-T{seH8{@Vy{OlSK6>lF9 zBbL5l5-B5ix{Zb4$9@MDZAA*@+%E4Oc0`8hxn+~C{b&T2l*Gcdg7hv_h#1TwtMz`% z`9S@0w%J#`J>1Was6654gTWQ< zqf*cq6Kt~U1sD&6s&)7Lc`Ys;D+r@D;{K-zdjdh;WhVW#f~GW*;=zbCPQl?Ai0y)T z3EKRebKY%KN>R_HEt>Bt_dAQ(7q4cD1Ng-0r=X*_o7*c`Pxb5;-qQu6&aGjljuNIZ z4n1hO|g|1(ydk0H?Omr{t!<~m=+>S zya%(>6BGU9*cjO^ep$W;nFGwzZ)R5o1AUocT0(qmsE_AygWE;#_YHfj>-$lD>1mFk z{*TH(l|KhW22-?U$ei-_%0 zJ1w)7jR?=m0}x*+@fG*@Df9E_Nrh|k5;}rTjj?4f%38?rPc$F3d2)Sn*t|+W6|wnY z1uGeL-Yq>`A4%^6;Z31IAM<5sE?!9>o)|rGp^nBNNw|<>=UzBwfk*o|6?La@Zo-CD zGBYJfprGSsRKkuU$9GK73@IM@u*V`On(f`x{+Zv-_&Y#h6HOJHmCe)zl(n4?QO53m z&L2$^bHQkz6WT8l2#VMm1W;wtJnPGRU9s%c7B1AL^~&aM!zkHys^;*3p?mGqSnK!~ zvED2@+Nx{GDuI!u*(AngTg20nq122Epf_g_KITydagCpC3>+T7eWtk$L$6>w{F420&^p}L zX78{}l_%TxsAl}Ebs#x z)jqj*CSH&x^{$vt!<|@$+k?=whg&w!E%)**Yjs-ZDN~Nkw3~Z=uJKL`RnNk8y-{+0>ub2GdgSdL^?YI0{ zFSQ1TbVIcCmC&AFxL0fCcA4#tt%}h8py_A+hzD%jD+}5|#{`eNXue91)_vR<@KA5| zi}wriCMt5@hPY`_LoRb2H-Fgn)1b}h)r9ZZaPxll5{}eSoZgVvkDi%wSh$AeyWYTZ z1DsnfM1D8ce%f1V-h~bT10^VcQFSL4G4=*^B*aFc@#umNfT__YFt0=znsy5g1Lk3^ zh|Hj?270o*vi=X%5-s|HU7&%llc8Wl#eCmDj{#%6r?N!TAcD@ zurN@qo(6Kd(i7SwHH&|FNl+N-amXst^KR`)`!}T*&?DU-DG~WH0wmo~G=e;lFzu3( zawVa72C2&N7lI=TuO-8-KPc5GAPmo2*k<%EG|3IR_pt;qB| z*o*sI0mNo@ubX1>a$(pDDAE15G`stp=#HP~$VREGp5ZH#X{f30b>wJ$X(dxU&k*NQ z@N-(7Ks5_9ecg>S&15(peVKLxhs$KDMB6ai1EIaEcgFaE!Mz_bH}v3O7f8C#KQ8~= zlvB{bE^5z%!mdAAb^H*&}?Z}Z-eMjSDUQrJ_TzLN#P0ofIeO-sw%d)tTk0{hdy_8o*k+I^NhvjKW zd}rKqI@r(3EF``zVJFkA=`K6CYATiDRpJEOk#j#`+gK(NpRuX9`mJgv9DrZ<>J2-; z6o7Rdrh3h%&MBcp3*VdG=~Gf~C-fOzg=b*tF0YN~_>F;hMxXh+tvM9{XHytms-mHt ze9&-{*D^?5Xqc&U>1qrJNNRk3BIjp;4`Fs){yFor(894|2$2tvE@^9hJ@hadv&B*a zq$GCg?{D4wDct;+h%|j%1|yJY2^oCT_k9n0?4g)}FYo zsd?9$_D!(}6PO$_<%Nudv;j<9PoAqbS(rZE$=SQVjn$I_;?%#@N~Y+4RVT}$Bf%M* zm5l!`PXbK}BfsRBDP!JrF|A_vCkrpv$=9qrfg%E^2%m|so2O{5tM-^o2$4FPBES@B zO#*>4g+-*aLIERs5*hu#!!xkxzG`aU{!C_&LU!{Ub9UkO9Yio6Wlg!_5UBVoPMLhZ z#j~aTC{cR5qE2~`R;&`nQ5%$WwD*&!x!Gmk9`blWH99D9zdpplZwhN^{36~VBz zg#GiSTGX@B{ofsX_=GC@-gh?eV4jF1K{7kb@6*q2zeTuNMI*ZE0 zi!vr9!iR2n<$aXC9-S34O^BDE7#$WELw&0)JmprJYs^jA*+|Z5kQF@`a|ueFIbd*T zT62kQ{rFiCLm$Q=h@2~}Uk|j2uh+&4s{&i8D%o}ajEbD2b)6lTpt|)rlgs=Qmu{*u ziJ46;SFjyPw;KR;T=}}`LP7w>!u<_LW)OF3hQX}3jAlIFSlOJ8$NU3 z2X|n=BkY-fy5odoe9HXBk9EQjxBs=5!UO;+=Pn|;w7tH*+wbB(wJnMtUw$?5$=ocN zf`<)0asgSH&krENT z()Vbd5g31rP?8DWH57QIP(%dUNYb@j)w@7X4zf2XUVrwIq|0`Qr1tQTaP1?oU}&V` zH;eK1Ef}=kpdrr_mK4r9>q5j@nUMz%LOc5l&60NM?)RO zm!=Cq&iGEd(C8Lj)Q0U4D?y;1a=*WkY^~l3^fRC4sTuVac7w|X_nS1C(Z*{yLhMCu z_6pte);{ZVHpI_;)uS&rgd>~n$8|2MVTLsft`}0g#RVz}hif?0IThW=58y4vPRqrg zl$470L8s89{X~vA`cC%Rq(+Fg>ZM?!pDXl`@P4uD@zu( zzP(Jz#gM01T1z#Z5WzdKGp_xW#%JGnqMwzh)<{NzyAFC}T6rwENQK-b3*LAFw%xfR~ z5PigA8ScDOSwa3*?vPtZhZwXMGHGd8XAm?8KdMN)DG%G1AHc!?l;CicIyNCLXSsS2 zOlXW^%^lMLcyH_=Y%@jtnrXRX2#m_&!+opks~FRNq4Rh%v&sK>9b>srCtKcjW4=3G zlKkS;D{ce1bS~v+{s-5B7|HIwz9C?gCB@nH*bK~r+u4@cs+W;b-e?XX*}f078O*BH z@E>yy5C5(f`$}C;&r>wNg`>RBwyx!Co1Cb>#wG&(vG|=~49?phpPg!z zX;XC4`NZx>`9HIM^y!a|MkN^FPy@>k;VVzgH{tbYT4CUVhN6k(0!#-U0(i6c=lbxb z>EuYX!@~A=MFOYD5+;PO3E!5DDAq#5OydCzYEr5=L)wk7>@fdjrjWCIly7X5XX<`_ zJU^2~Dhn5v@Q!nG++pPkz>$cvalOJ(`b(sJ|U1UO*Z_nB^ri!sGBi(`)L2S>f zr`ozMkYIvwv*+u3NapX4!!nUYAX?*i`1l(`a8PbVVOV1uSi6D=Da-3S)a#uAjnep^ zlzdd1x!X$sCx9gbD@8EJL{gdZEYwx;NeMw;L6 zIdg!CdAzIBoYt{m;5!9=E(06QSYrsp^jMn7gMFUfl(UVu?S|gPtySpnX%lFM@P5I( z!TSB`$B<>EPg!z{B^G{ zd0sV*{072*N#XHIu4ybb4B$Xv1EE(d>=;MDtX{HO+m<|UTcnPHwmLEB;a6{b1MEbB zY@5g+Xfj|;Adz#!KsacZdM`X1{a!lQL@jF@M+NV=?Bm-BPLs=Pq+ZY%m8U;$d5Esp zcwZX3jgRfImN@E%80DqxYyzUIjDtkpsa=1F``v=D2FlE9!mjuL%FR-yW6>-?D(-wiT9PyuLnI2zm9fTtOJ8bxoCv}8t zp`M;&K7zadtV=IIUBby<;-N?WtsMV{@K=GVADtptNa;hhB{vzH5RMi8!>qN4aj4~& z{A@P7p{1r_^`=%x;TJA{jp@#3O?Vg(PoE!`wPF-p2<62I{qqQUq~_6*$M;N}@*=$F zB1%K?p0%3I*t_WWBvr_-Bm0PNElP%*M9u|#WX?wlHy_v07l3js$5559M8dp{$dBk* z#%w>8!b11D)tKGSjAVDX!Zm92(l#1j!+(UJ(@TqXTS!(;W+mN^g6qDkmlORV8O6nn zN`uK#-)P!f9fNeY3{rWP?V;g55oi!~4p5YD zYA74Sg*5v5UUnRz>WhbWdxH=c)A(7?ttD4XDPj{ca>$d{W=~`g>jg`lM;RV1?Pgu~ zuc!6bpP-Ud5pq0CJz?Ez6%c)_9f{75F$uJ)QyY0B-V|bwCjqobi)34un7Ukeyyt>Q z;?zIRs&mnLeE6tCxJx(+7u5U{F>pFvk2|S-oq+t3|9UaiA8CB#?zmMy;^ZWq)%Y^; zrWFk$#6)}J@R!vNA>90Vb$u(Z!qiZNLjV#b`pc$tQj+|72Ah3*s#HMhlC|bo5GVCZ zk43wh4N>+v<>XJg$8WdgSXnaL{mwt|qUXECDu*-VbJ;*Zt(Vq!+|>xxt)X+9ltZW0xn2AEI}Jxd zWo88EdbDA~+2&Jzi@mloI(&LP_=3;-QuPhf1JE;_#MkwDAKGc^6tkY8cKg6m1yDCB zy;UEs?hTh?BC-6G52f4*KkDO}c=&*SoYL4GRUSP;lJyI8TefQ{y-D zaVngX!TKDRMCSbIM(KM~J3zg!LZqkHLuzSmFdk+bjIqeduzj2cEvK0Iz&MOJdcFs* zCM~ND7X8D=;V=a;!vq-o@@@DFi+G*&5-n(hT~vt9b4rB39UZR=D{_Q zfiX<3ukV2(wCPV+LJ-H%{a|bvX@Q*8dh1vDt8T;)9rIoesuIE)6Pnk%g79PF0Yv%( zp#@OLzI$);&Yea3*vhufZ#2e*KqTj#NJw&S#)HcJp!On;`Q#1MNbuj|-z5 zNYD4Sv8(fq*~5;3eA3COh-YJEla?nz&`^~F8&~+%f7r-Wrt?T9tdMJR(Kee(HKMUsu$;5-+BnPa!P{S)BbV-apRo!xHM z*O10U#bqQBxIU}(td5z*Wx1M`*}1OeVId;#olWym>rt1_4s?!GanwG_nPUsdLg#KUc zqX22(*Cn-5s8LW_o-cH&oGV)|&&D%M+BBoUn>dR+UM!pI#h_FVumUZDZ{Y=re<^|t zT{S$B73#uB&;^~%!5ANHBX%}k>l1df!lO6j6wk{&1@*q03Tb;+{{^`&G)>!1!ZYyR z2!((u6op`TP^rd={Id6(Ng;LCmjJ4U>?`3GtEBjl2E_3lQ0*kn|6%K_qoQ2YwofUb zNVnwB-Q6;DH&TP3fHcxMAT_iosf2>k-QC^YDc#M`e2@FQ=bUx+{{9#iux8eRC+_Et z>-ybejP2RA8%xxi95a<&VYrxCZWHP$%fA=0wr^p_m`=Dg3_16tEQ9m!Ka*D#>P)AU zCIuyQjxDVf=?!5N@(I58fAa_8#!{p(I7Kt|ffPkyC9QQA!%3mB@(i_$E90ojD3iEb zU*%n{g0RfF4b@DMyPUxUHo zj~4aP;bF|-sUxlaO);wRh60Jo5XzN3x#PZVwqHH2*L;43AOUmfMMSWhu0*}j6Y96t z%vi!1?|CWt4;K6z-HJRoOg-g@Ug6sX^~_AKJ45%q?4Qmz12l$ZLqJqDS?UP)Dos6X zF#mN)p4BX(A_NMZUv|#&na;y9cHS=rPgU&@m`&a2!lY{TZ0)M;*i*+WKczTcKh}Qz zV`A7f<`~u?j5c?FG1G_9apjSo=6i8pl=i2uXb}89B7eVRKu4y7-1E19@vZMx>|W(s zcK%48WJA*xlRNkk=wMVEkSXTDUpF|k_g1K6ZEhNGtTLM zs^h?YrBb>xt~cp?MgT}OUk)A76*#_`U+H~NWz*Nd$k#wA=e@Y?BtHD+N-1Uu9L~h& z3tNji?=9#0hbSW61J|g%8kK?2kJi#gHp-);BYUhs)PJb^XdSh&li+46Jy-Z^lmyx7 zs0#m;P5ikTMr6=Y^z__E(L>D*urKF+FF8hrabaKJ;^b+?=@TM3(U7uF<9PUbOX0ht zeH{g~H?UKc3T0SYet&q%PEyWy#vb+GOKFYemDr?a^^sED!XLR|3wAtfKQNqOR_&T_ zI!}?G+UWaHfID6htX|PHTg3LBZ9z+SL87mPbY$+#hVqg#2%#9qC=PhF)ojx_wHzf% zx8l4kTuHh9hQ=ztdKD*s#FJ7)2Xi_3H9n#QK*lzk>>Cv*VvU zM%U`xur_;;(c{Eq zgU7PP>+>zjn!W~-f@eJM4_~Nqks&ACo!L)^^I)4p0i>boOCM;=pDtevI95*d!1zh) zWKHk@z*y`3w6-Z&h#hwKM<4b<9=*F%M8rxCDLSFN=q}18$er@Ol*Q|2_3Pe@R&(YO z9jcHsVs$lTld4>%`!gM+>01DAnrpL0TCXqSf*~|SzDW5So)|BUo$}G7;r{$P$a7i} zk^6OfwtTaBw7WGn8)fXN-AmpR8#i4syazd63{O z$~XIU`Eqr)8L(T{gCF%f(R{x}*rpdP(byTDwi(~NwoN1QU@&?(l8d8hY2LR$9A7mf za&#p80|rn+y7CR5l$bkhZC3gRAq}W8zKh`efuy}IohM&kJKfQE#pUNN9FfBEzb)aG zSoxJ4<6?|995jpe_g8Bi9eO7Dxs1-?nsNS9HrTbaWVmBrw{2m(F;+7rS3g}CK(#v& zZYRAYnkEepIm6%_QP7yXFg2h(^~z!wh#gRA2E@p1b}u`wVXO{8Gj6F0aLtSZ)2AmT zVO4;Z82^cw{;lZ-ED};-`6c}FTk*HQZchfokInz2-Kh65asxK z_mfW6@)GV+-f}&Xp2!3_4e`ViDJoM}Ahn*2*IV%Vq3iF}9Jz zMx2w?i4hj>9JdBmO*0=b305bSKji-K3+9nZIv{qR*OLc4*OT5Rwb5Yv7@q4C%6xb6 z^zme-VXcFx)zs3$BR7jk5~5}H+HXzTN1tlz(BK^avP%Z=aHbB0?#&E7%$1W1yA}~P z7VB-{K9!Rc{FMT5P=%p2o8j*Huk{w_90ne%mN*_z<-&sD#zo|sD_3F52#h>ND$)&! z>5j9he$z{UT*IyC`T6T&P}eX(2Fo(cud1x+fM&)nJO9!}IX0PR={=`pLZ^7C&>Q+HBcEH#~zS{Jd^jg#P*xz7qjiq@j z@6hTVi-k8*bQLsvyiAlj2B$m|!kRS~i65kG>n%!NED8U6+WsFydmS0TkxVzy-18xIA6SaTTETXpRqBoYv?-CtD8Wu zjaIAL`Xh+VLe_h}6#+mm|Na(Zbwob=O%Ch>i%?$g18{9FyoZ||O$KiZQ=VJLjnt{4 zAH9rEYG`VjB0yiPPoG-vkGQP;n!Q;=<5tJ4lLa(Ip4aBHD8;wb%gVGD>rwJiZhUg8 z88G?ry{Jx;Hg`iz`llr@trFC)k?VWr{*Aer4U{d7PpB$g0jCH>d3{-9*CNpj!PhJj<3uACMW*lB88Up{ghQDLOM0!4*g=jHsyM#4b>=@(8SM>iX(m zS#;4=+hilbP6xY1ib)a|sNW+;pAhcd#W_1nEW7?8|4m4f9Yvi-_s!}_Pyx!l%?cYk zvw>W_p2NyV*N(dFqo@(F)d3rq^S!C)#i>hJRpb6(Xv!fVuT=!>kf}aOVOQ%e65W`B z{43@dYm+%g*_$2L9*M$hA#xjkL<3Hk$I<3pBAe^|Z_gE$xnEm;tKVT^Sae@^VOX3( zL3zwJ@@;N#au{z)VDpi`!T1K6rO9Fr9X=-WHr(1e5V&|Lz`_usGt`F-8(OlQp(PR9 z8*3NVE6{ZvV&mA2$FXI=6YN9zkkIPX61G-hL-`pnYAm%s%ReL2g%XQiE04zRO{Q_C z18HCZx${7DcbZ?kUp4p83&v370QVT8kDv4 zcGF!D7O0!W9BET1DhnNDYc@dh*O?KU?PRxOxCKjF6vl5Gs^$gYh8_0Zy#E`k|ATD+ z=6DJyNB~7>3(U#=5bN^Ull8xe|6eK6;#YsDAl%+p@P}GcR++*vnOE_q;nVPF2t@0^ zGe5xq08AwC30&H_pt`hh65g>5G2lyYb z9IRH?cdy^fbX+anQ$v1$=UXvwj^XxK+RxmJ@z7>*bSO1Mv9gINodes0xnRtO+_2FD zH@C-x8#k~o6_@zsc$y6_JJ)0~a|)JYgX5-uCw&$g_?i>eNW~diGM1$GX<28eJQ*Tq zADH?aueFK>K3J--;)o4{ox2-6uYedvl33|r`#22NTs^-3Arv%YGTt_-VA)gK?D6{f zr^bua*rQda8R@v&Gkb5Hr)oEK%Pv$H;%gz@ZzsZ*&+~bn0RA+9%zLM6VSN3P{LOOQ zydn6P_L z3OhpF2W^IL4;!E;_-mPqFcPBeoHKbEBws2^436g&FDz>4?QTyiZ!hwMvXmu=;xn=I zpVmkJaVm+BM~c_Jr$ZQ*Fcb<*0f>8jBxYqyAF67qoTA{`-FB?p%u}?fY$;Mrm+j3~ zewWu^bl;s81WJ@LYV?j$lMhC_ z1|mf5Zs0pA`{DU2BRxP$te`k$+ME^j;)SY<3QT{FWg!IC1Q@56)Bq_jfDF*gW{GDc z1$mpVY({M_t3w#Pr+jU{qJcc6kBJHJIItI`^zA_P*_Y&p;j)qP+W@>|#25J_^t(O4 zYE3Q0v8(z~G}S9j{Py{8_0?2oAI-Y2YIznx*Z4>Cy9Gv{eu3J&WTb{x3gy@B41Z*v zWGVr)=Uu;|Fo9__y5RaKVYz>}c)m>84VS;p$Jw4|fWi}by_%dOe z+DU9hT&ly=NB|hCZ|h)O7TkiT;eU*2UW>oTnH%orywb2JW*B6=ec_@VuINn3L#wMb zPIra1kkMMnLMjbU#y;q!W2f-cU|r#Gvigt=z=!bSLmbFl=3+oG3PW~HefO)WyI7g> z!^DlhE1}o4pBl7fiz?yElC&2Cx$z#ODPJ_;OlbVAq(+b{EY398nU!yOJka4>q7hb( z)3_j-FP$o|M0DK1p}AaXirAg~GZOK0G@R<}1L3YITcwsWtB!ML3N?4Mg1mjMTy;)2 z+#&hHVTA7RC77s=HCO+mzncdr>3_-nP@h=5K$V>Uhsdzn#XKp?H&1q}e#y->D zpGWd{MEY~LTfjBZjmX*C7`}pA;qIdVhVI;3_ucy+U!^mJ1zz8{gRL()x5yWhjlKuf zL00r(lm51HnUX5)#+#xNsa`rULa&E z-C4+Y)a%=%$))xb)vj6FECz6I_-L)%zJ9ojdKkj^5n2D%CeS`%;arO<&fZoUiPn)A z?WfF@ZXmE^YxvEK;no320GQF-$x93k(F}9S44E?1tlq5f`X3keu(Y%H1i@+zESqk)Cwj@ z#xt`jrfP1g%FDH5(Zbd=hY%CB`Jw4IXuJXTAoynZ{c0U)^CCc>AcO(y&8iH1ru*LC z@KtyEohZ_N`PsoQs}8BLDHNu!uBp#!=pkt*Jp0x*mlsGsduRtFuayYTkm!j!4zVZa zjd0MH^?^Z{0lwAgj4Wd}?)%&Km7&&Lc5;Em`MD!VS%8@b@m4Cy1gIgjh>FSNB8e%3 zYuNW~+TYKhqkB^nHp=205dQW$_AhA$00oqYD6cueZ~5UWuAZfC+mfC>WrpDS_#vK6 zOAL+s`vT;29=}^NfEw=~fzUL*f5fNvPM~Kcc~%MCXWkn72H@!Zl&~YGS@%;^e;#`s zYjHKSM^2^bXA^^mNqrS$4Y(bjqK3A!0Y2Dv*ZoaF;Sb(|ThBtnD-gT4)xO0jEOHdq zQp`1^BaqLZ8Fh#V=$=@`wC2*G!(Xu+qE0^u$EMm){39Lx&!b)v_1_$?TamoRm(RLw zNtOS0nmB9h>&m--(rS7?CJM$>} zMsb{SpDr;L3nOKPqrAl2%gq>T^LmG+qjBP^N&su~HidglAG_gSGt#}Sc;O10d3>Dp z#uDt!2nL2~BRD*H+u>=;`TgZKbLWiow@!8>{Un54@J($_Y|v z$q9zGAvQ(qe4_&@n~q3kcT0!-UkYQU-zsNzcob`}6&)oU3HgfKwiZO2M#=ZrgqViq zCSxLf@lE5R$ao3AzzFrJwlFWgTix*I9)*+GF#jLpc>ov5tCNgus6l6I_;cp(H~)b^ z{C~fRltnPRJBShFL$EqqEi8?Ulpq!T2brw}A3Ny_>4ONeT`-Cv+xZCaS|Cp(drD zZ;8T^W;{kmwb|p}Xak=*{XyW`%21HFKjqw6AJbKf5lHnP(e@s|}zyY>KLzQjuoI`#wz6{F{*E;}- z@=L7xZB&kMm6j{wsE3kkGHw@5yPL4p7bNQe25}~1S;-^Z!pe;dhnZ2Tg_FetPD(Ic zq0mDeu1$L^(%Np%Sr}8Y?E0dS4)fTl5^Xl^==3D}SFse(!o8dqZv~yPH zL2t9Epxnbey45+tH(!&N8{)S_mfX3V!muCeUqWm9MuJ5gi>rOsFdNp}6T-07cXgt_Wxp4nXW&I$}I^ zslOZN<~)ByFRDA~KfK7)<#UyIoVi!gCM{?s89tHk?%&GSziumEvznTmquTAq%CQzY zbs*{ea3RDwo?qVCcSc6{=$Y)RvLeJ0chYe4hT{K|-A9J{>%H}Xoj3?WRTV|M0zcdR zO^hpVs);sGo6QGK9X6~{l^iL2fip0EMi@R-r^Wo->}-jLMxzh!GY{FT_lxWkY~b43 z4+C>$VtMG`Yl9>Qc&r%o*BUCYnCKPP$TCY@joyR)vXO6FQPJ4d8$ruf1Gy%0mFDJZ zEey=_Uf8WuwD9Ww3aZV3h6s8EpElzvA}F<+E}My3(n0SVE*-Bfg)0UkDjMu5=sh<1 z+r$FGi)GMZCSlznVzT7}TisW~lcK=SosBa}>#z!_ofY z7AV^wX>y8<7c`bC@}@e8*Ao^uj-0@eZj)9dqb6sMeV6SVYq}KeFL|QsqnEmQyxVuA z6{(^{3osMvk6DfPYY%p;Vuzb zR~w>r?mm_t?vP&JO5c^Nxd)26qnRSG7aE?I?s$+KdXR*{ajEH z7SsMIsYb+wp#&UY!NVH~0;7us6$9-I-T9*~@ny;^JA*vf&e9=auI>5g&_TX`}i2P2JY zuD#P1o&U^pI_N~j{YKyH4k7djBLB_w6eG>1)Nu@D+ZmAq5WC35Fn0<(S-@V0NQ)g% z8=i{sx-7jVN0OlDraxc&zH1FYthAPFEo1Y`Z~%*Rh&qXb?D)Ak-XMVJ30W6&QY36; zFGW`vKhdOOw6HSwIL+I=Qk&<&SnA_?)Y!xs;tL<8R`0GXW&?br43mW2F^nN|3mZM5 zTC##$v!3!loLZT&=fV(F`7al^hy*{b;}^SuCo0IH>I^Bj%2r#T3iB2+tACw;ooY@7veCpWB4A7In1d=3bqL`%lL zx(#6Q7N=d(>jYUIsq1n02ZjJd1+Iq>o~cPxq`}bZzm?~Ie#B-DxP=^K6H9387s&nZ z&_WVv944u4wAA;hEk*2P1?%m3i9QY`5yS6+sl@1rD1y>m0$?s20@1X6NkC4Q@#}Pw zfwgPOeLa?lG)A?ee4Hr0;SHzzMBlwH_uS&jcl=g{8<};AN+&b|D%0+mM-2_&Zs!Uj zY+^?CZ4^JLzmpQ87pJ!36}+1f=N+L;BbHbvc~S&}1qcpvIiUxzsA%?%jC?2FK8RI) zR*LBPedzl-K{T~>nlvx@D+XdLoN3e>uQ)4^>?NY^k7rL_5t}LIMQ)Hroufz#4aXAW z9W5r!TxrykeuiUvB4c>iDWCgVOrWcp1?w&!=-C=`Z~3&)$DVFW8(*eU=;R6tD89`e z4KsH!EBH4t_>1)SQagF=iCc>P^amf0VKos&U!3u=FASDLCYk51?y%jp;(U-`2oUcN zb?oNWsbBe8Dy4qK0?!-lQ!u{r8$2<#i02c;Y!4$HdN8(of$1a&Ua;RX#`tKt{&Z*L zu=2x#70xNkvX2bLB|Nv*f{-1D=`#meinq>$&rTe>-b5gEpZLuO!y5DX$j!n9VWwfY zhiZ{{l@|HO566j!xN*)^cQ-V z3Qhl41$F+6Rpwj+3Wc^S28*VwzRKNo=C!gkVvp)W&a_<^GLbP(RG-i-@O;zJ`T767 z2=HI(Cet#smJ^bp9H*1sOh37_0wF1%!pd%O|>D zF@nTYFXXVu#hMM%D)f{+MS)KmN zv-{6^rv>nul0Hi=n>9FP=DXM#mcx)AC<9uqjQYKKI|CYgqPimejQXsks+5?*sY^v} zT50T^a9E^+p@C|tRdKO;?5wx2VTp-xOMDrpiGfKN@pdfBS+HqinS>p)X2TN6QWwLo z